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,5 +1,5 @@
use core::{marker::PhantomData, ops::Deref, future::Future, time::Duration};
use std::{sync::Arc, collections::HashMap, time::Instant};
use core::{ops::Deref, time::Duration};
use std::{sync::Arc, time::Instant};
use zeroize::{Zeroize, Zeroizing};
use rand_core::{RngCore, OsRng};
@@ -13,23 +13,25 @@ use ciphersuite::{
use tokio::sync::mpsc;
use scale::Encode;
use serai_client::{
primitives::{PublicKey, SeraiAddress},
validator_sets::primitives::{Session, ValidatorSet},
Serai,
};
use message_queue::{Service, Metadata, client::MessageQueue};
use serai_client::{primitives::PublicKey, validator_sets::primitives::ValidatorSet, Serai};
use message_queue::{Service, client::MessageQueue};
use tributary_sdk::Tributary;
use serai_task::{Task, TaskHandle, ContinuallyRan};
use serai_cosign::{SignedCosign, Cosigning};
use serai_coordinator_substrate::{NewSetInformation, CanonicalEventStream, EphemeralEventStream};
use serai_coordinator_tributary::{Transaction, ScanTributaryTask};
mod db;
use db::*;
mod tributary;
use tributary::{Transaction, ScanTributaryTask, ScanTributaryMessagesTask};
use tributary::ScanTributaryMessagesTask;
mod substrate;
use substrate::SubstrateTask;
mod p2p {
pub use serai_coordinator_p2p::*;
@@ -44,8 +46,6 @@ mod p2p {
static ALLOCATOR: zalloc::ZeroizingAlloc<std::alloc::System> =
zalloc::ZeroizingAlloc(std::alloc::System);
type Tributary<P> = ::tributary::Tributary<Db, Transaction, P>;
async fn serai() -> Arc<Serai> {
const SERAI_CONNECTION_DELAY: Duration = Duration::from_secs(10);
const MAX_SERAI_CONNECTION_DELAY: Duration = Duration::from_secs(300);
@@ -111,7 +111,7 @@ async fn spawn_tributary<P: p2p::P2p>(
db: Db,
message_queue: Arc<MessageQueue>,
p2p: P,
p2p_add_tributary: &mpsc::UnboundedSender<(ValidatorSet, Tributary<P>)>,
p2p_add_tributary: &mpsc::UnboundedSender<(ValidatorSet, Tributary<Db, Transaction, P>)>,
set: NewSetInformation,
serai_key: Zeroizing<<Ristretto as Ciphersuite>::F>,
) {
@@ -131,18 +131,11 @@ async fn spawn_tributary<P: p2p::P2p>(
let start_time = set.declaration_time + TRIBUTARY_START_TIME_DELAY;
let mut tributary_validators = Vec::with_capacity(set.validators.len());
let mut validators = Vec::with_capacity(set.validators.len());
let mut total_weight = 0;
let mut validator_weights = HashMap::with_capacity(set.validators.len());
for (validator, weight) in set.validators.iter().copied() {
let validator_key = <Ristretto as Ciphersuite>::read_G(&mut validator.0.as_slice())
.expect("Serai validator had an invalid public key");
let validator = SeraiAddress::from(validator);
let weight = u64::from(weight);
tributary_validators.push((validator_key, weight));
validators.push(validator);
total_weight += weight;
validator_weights.insert(validator, weight);
}
let tributary_db = tributary_db(set.set);
@@ -165,161 +158,15 @@ async fn spawn_tributary<P: p2p::P2p>(
let (scan_tributary_task_def, scan_tributary_task) = Task::new();
tokio::spawn(
(ScanTributaryTask {
cosign_db: db.clone(),
tributary_db,
set: set.set,
validators,
total_weight,
validator_weights,
tributary: reader,
_p2p: PhantomData::<P>,
})
// This is the only handle for this ScanTributaryMessagesTask, so when this task is dropped, it
// will be too
.continually_run(scan_tributary_task_def, vec![scan_tributary_messages_task]),
ScanTributaryTask::<_, _, P>::new(db.clone(), tributary_db, &set, reader)
// This is the only handle for this ScanTributaryMessagesTask, so when this task is dropped,
// it will be too
.continually_run(scan_tributary_task_def, vec![scan_tributary_messages_task]),
);
tokio::spawn(tributary::run(db, set, tributary, scan_tributary_task));
}
struct SubstrateTask<P: p2p::P2p> {
serai_key: Zeroizing<<Ristretto as Ciphersuite>::F>,
db: Db,
message_queue: Arc<MessageQueue>,
p2p: P,
p2p_add_tributary: mpsc::UnboundedSender<(ValidatorSet, Tributary<P>)>,
p2p_retire_tributary: mpsc::UnboundedSender<ValidatorSet>,
}
impl<P: p2p::P2p> ContinuallyRan for SubstrateTask<P> {
fn run_iteration(&mut self) -> impl Send + Future<Output = Result<bool, String>> {
async move {
let mut made_progress = false;
// Handle the Canonical events
for network in serai_client::primitives::NETWORKS {
loop {
let mut txn = self.db.txn();
let Some(msg) = serai_coordinator_substrate::Canonical::try_recv(&mut txn, network)
else {
break;
};
match msg {
// TODO: Stop trying to confirm the DKG
messages::substrate::CoordinatorMessage::SetKeys { .. } => todo!("TODO"),
messages::substrate::CoordinatorMessage::SlashesReported { session } => {
let prior_retired = RetiredTributary::get(&txn, network);
let next_to_be_retired =
prior_retired.map(|session| Session(session.0 + 1)).unwrap_or(Session(0));
assert_eq!(session, next_to_be_retired);
RetiredTributary::set(&mut txn, network, &session);
self
.p2p_retire_tributary
.send(ValidatorSet { network, session })
.expect("p2p retire_tributary channel dropped?");
}
messages::substrate::CoordinatorMessage::Block { .. } => {}
}
let msg = messages::CoordinatorMessage::from(msg);
let metadata = Metadata {
from: Service::Coordinator,
to: Service::Processor(network),
intent: msg.intent(),
};
let msg = borsh::to_vec(&msg).unwrap();
// TODO: Make this fallible
self.message_queue.queue(metadata, msg).await;
txn.commit();
made_progress = true;
}
}
// Handle the NewSet events
loop {
let mut txn = self.db.txn();
let Some(new_set) = serai_coordinator_substrate::NewSet::try_recv(&mut txn) else { break };
if let Some(historic_session) = new_set.set.session.0.checked_sub(2) {
// We should have retired this session if we're here
if RetiredTributary::get(&txn, new_set.set.network).map(|session| session.0) <
Some(historic_session)
{
/*
If we haven't, it's because we're processing the NewSet event before the retiry
event from the Canonical event stream. This happens if the Canonical event, and
then the NewSet event, is fired while we're already iterating over NewSet events.
We break, dropping the txn, restoring this NewSet to the database, so we'll only
handle it once a future iteration of this loop handles the retiry event.
*/
break;
}
/*
Queue this historical Tributary for deletion.
We explicitly don't queue this upon Tributary retire, instead here, to give time to
investigate retired Tributaries if questions are raised post-retiry. This gives a
week (the duration of the following session) after the Tributary has been retired to
make a backup of the data directory for any investigations.
*/
TributaryCleanup::send(
&mut txn,
&ValidatorSet { network: new_set.set.network, session: Session(historic_session) },
);
}
// Save this Tributary as active to the database
{
let mut active_tributaries =
ActiveTributaries::get(&txn).unwrap_or(Vec::with_capacity(1));
active_tributaries.push(new_set.clone());
ActiveTributaries::set(&mut txn, &active_tributaries);
}
// Send GenerateKey to the processor
let msg = messages::key_gen::CoordinatorMessage::GenerateKey {
session: new_set.set.session,
threshold: new_set.threshold,
evrf_public_keys: new_set.evrf_public_keys.clone(),
};
let msg = messages::CoordinatorMessage::from(msg);
let metadata = Metadata {
from: Service::Coordinator,
to: Service::Processor(new_set.set.network),
intent: msg.intent(),
};
let msg = borsh::to_vec(&msg).unwrap();
// TODO: Make this fallible
self.message_queue.queue(metadata, msg).await;
// Commit the transaction for all of this
txn.commit();
// Now spawn the Tributary
// If we reboot after committing the txn, but before this is called, this will be called
// on boot
spawn_tributary(
self.db.clone(),
self.message_queue.clone(),
self.p2p.clone(),
&self.p2p_add_tributary,
new_set,
self.serai_key.clone(),
)
.await;
made_progress = true;
}
Ok(made_progress)
}
}
}
#[tokio::main]
async fn main() {
// Override the panic handler with one which will panic if any tokio task panics

View File

@@ -0,0 +1,160 @@
use core::future::Future;
use std::sync::Arc;
use zeroize::Zeroizing;
use ciphersuite::{Ciphersuite, Ristretto};
use tokio::sync::mpsc;
use serai_db::{DbTxn, Db as DbTrait};
use serai_client::validator_sets::primitives::{Session, ValidatorSet};
use message_queue::{Service, Metadata, client::MessageQueue};
use tributary_sdk::Tributary;
use serai_task::ContinuallyRan;
use serai_coordinator_tributary::Transaction;
use serai_coordinator_p2p::P2p;
use crate::Db;
pub(crate) struct SubstrateTask<P: P2p> {
pub(crate) serai_key: Zeroizing<<Ristretto as Ciphersuite>::F>,
pub(crate) db: Db,
pub(crate) message_queue: Arc<MessageQueue>,
pub(crate) p2p: P,
pub(crate) p2p_add_tributary:
mpsc::UnboundedSender<(ValidatorSet, Tributary<Db, Transaction, P>)>,
pub(crate) p2p_retire_tributary: mpsc::UnboundedSender<ValidatorSet>,
}
impl<P: P2p> ContinuallyRan for SubstrateTask<P> {
fn run_iteration(&mut self) -> impl Send + Future<Output = Result<bool, String>> {
async move {
let mut made_progress = false;
// Handle the Canonical events
for network in serai_client::primitives::NETWORKS {
loop {
let mut txn = self.db.txn();
let Some(msg) = serai_coordinator_substrate::Canonical::try_recv(&mut txn, network)
else {
break;
};
match msg {
// TODO: Stop trying to confirm the DKG
messages::substrate::CoordinatorMessage::SetKeys { .. } => todo!("TODO"),
messages::substrate::CoordinatorMessage::SlashesReported { session } => {
let prior_retired = crate::db::RetiredTributary::get(&txn, network);
let next_to_be_retired =
prior_retired.map(|session| Session(session.0 + 1)).unwrap_or(Session(0));
assert_eq!(session, next_to_be_retired);
crate::db::RetiredTributary::set(&mut txn, network, &session);
self
.p2p_retire_tributary
.send(ValidatorSet { network, session })
.expect("p2p retire_tributary channel dropped?");
}
messages::substrate::CoordinatorMessage::Block { .. } => {}
}
let msg = messages::CoordinatorMessage::from(msg);
let metadata = Metadata {
from: Service::Coordinator,
to: Service::Processor(network),
intent: msg.intent(),
};
let msg = borsh::to_vec(&msg).unwrap();
// TODO: Make this fallible
self.message_queue.queue(metadata, msg).await;
txn.commit();
made_progress = true;
}
}
// Handle the NewSet events
loop {
let mut txn = self.db.txn();
let Some(new_set) = serai_coordinator_substrate::NewSet::try_recv(&mut txn) else { break };
if let Some(historic_session) = new_set.set.session.0.checked_sub(2) {
// We should have retired this session if we're here
if crate::db::RetiredTributary::get(&txn, new_set.set.network).map(|session| session.0) <
Some(historic_session)
{
/*
If we haven't, it's because we're processing the NewSet event before the retiry
event from the Canonical event stream. This happens if the Canonical event, and
then the NewSet event, is fired while we're already iterating over NewSet events.
We break, dropping the txn, restoring this NewSet to the database, so we'll only
handle it once a future iteration of this loop handles the retiry event.
*/
break;
}
/*
Queue this historical Tributary for deletion.
We explicitly don't queue this upon Tributary retire, instead here, to give time to
investigate retired Tributaries if questions are raised post-retiry. This gives a
week (the duration of the following session) after the Tributary has been retired to
make a backup of the data directory for any investigations.
*/
crate::db::TributaryCleanup::send(
&mut txn,
&ValidatorSet { network: new_set.set.network, session: Session(historic_session) },
);
}
// Save this Tributary as active to the database
{
let mut active_tributaries =
crate::db::ActiveTributaries::get(&txn).unwrap_or(Vec::with_capacity(1));
active_tributaries.push(new_set.clone());
crate::db::ActiveTributaries::set(&mut txn, &active_tributaries);
}
// Send GenerateKey to the processor
let msg = messages::key_gen::CoordinatorMessage::GenerateKey {
session: new_set.set.session,
threshold: new_set.threshold,
evrf_public_keys: new_set.evrf_public_keys.clone(),
};
let msg = messages::CoordinatorMessage::from(msg);
let metadata = Metadata {
from: Service::Coordinator,
to: Service::Processor(new_set.set.network),
intent: msg.intent(),
};
let msg = borsh::to_vec(&msg).unwrap();
// TODO: Make this fallible
self.message_queue.queue(metadata, msg).await;
// Commit the transaction for all of this
txn.commit();
// Now spawn the Tributary
// If we reboot after committing the txn, but before this is called, this will be called
// on boot
crate::spawn_tributary(
self.db.clone(),
self.message_queue.clone(),
self.p2p.clone(),
&self.p2p_add_tributary,
new_set,
self.serai_key.clone(),
)
.await;
made_progress = true;
}
Ok(made_progress)
}
}
}

View File

@@ -5,7 +5,7 @@ use serai_db::{DbTxn, Db};
use serai_client::validator_sets::primitives::ValidatorSet;
use ::tributary::{ProvidedError, Tributary};
use tributary_sdk::{ProvidedError, Tributary};
use serai_task::{TaskHandle, ContinuallyRan};
@@ -13,16 +13,9 @@ use message_queue::{Service, Metadata, client::MessageQueue};
use serai_cosign::Cosigning;
use serai_coordinator_substrate::NewSetInformation;
use serai_coordinator_tributary::{Transaction, ProcessorMessages};
use serai_coordinator_p2p::P2p;
mod transaction;
pub use transaction::Transaction;
mod db;
mod scan;
pub(crate) use scan::ScanTributaryTask;
pub(crate) struct ScanTributaryMessagesTask<TD: Db> {
pub(crate) tributary_db: TD,
pub(crate) set: ValidatorSet,
@@ -35,7 +28,7 @@ impl<TD: Db> ContinuallyRan for ScanTributaryMessagesTask<TD> {
let mut made_progress = false;
loop {
let mut txn = self.tributary_db.txn();
let Some(msg) = db::TributaryDb::try_recv_message(&mut txn, self.set) else { break };
let Some(msg) = ProcessorMessages::try_recv(&mut txn, self.set) else { break };
let metadata = Metadata {
from: Service::Coordinator,
to: Service::Processor(self.set.network),
@@ -152,7 +145,7 @@ pub(crate) async fn run<CD: Db, TD: Db, P: P2p>(
// Have the tributary scanner run as soon as there's a new block
// This is wrapped in a timeout so we don't go too long without running the above code
match tokio::time::timeout(
Duration::from_millis(::tributary::tendermint::TARGET_BLOCK_TIME.into()),
Duration::from_millis(tributary_sdk::tendermint::TARGET_BLOCK_TIME.into()),
tributary.next_block_notification().await,
)
.await

View File

@@ -1,456 +0,0 @@
use std::collections::HashMap;
use scale::Encode;
use borsh::{BorshSerialize, BorshDeserialize};
use serai_client::{primitives::SeraiAddress, validator_sets::primitives::ValidatorSet};
use messages::sign::{VariantSignId, SignId};
use serai_db::*;
use crate::tributary::transaction::SigningProtocolRound;
/// A topic within the database which the group participates in
#[derive(Clone, Copy, PartialEq, Eq, Debug, Encode, BorshSerialize, BorshDeserialize)]
pub(crate) enum Topic {
/// Vote to remove a participant
RemoveParticipant { participant: SeraiAddress },
// DkgParticipation isn't represented here as participations are immediately sent to the
// processor, not accumulated within this databse
/// Participation in the signing protocol to confirm the DKG results on Substrate
DkgConfirmation { attempt: u32, round: SigningProtocolRound },
/// The local view of the SlashReport, to be aggregated into the final SlashReport
SlashReport,
/// Participation in a signing protocol
Sign { id: VariantSignId, attempt: u32, round: SigningProtocolRound },
}
enum Participating {
Participated,
Everyone,
}
impl Topic {
// The topic used by the next attempt of this protocol
fn next_attempt_topic(self) -> Option<Topic> {
#[allow(clippy::match_same_arms)]
match self {
Topic::RemoveParticipant { .. } => None,
Topic::DkgConfirmation { attempt, round: _ } => Some(Topic::DkgConfirmation {
attempt: attempt + 1,
round: SigningProtocolRound::Preprocess,
}),
Topic::SlashReport { .. } => None,
Topic::Sign { id, attempt, round: _ } => {
Some(Topic::Sign { id, attempt: attempt + 1, round: SigningProtocolRound::Preprocess })
}
}
}
// The topic for the re-attempt to schedule
fn reattempt_topic(self) -> Option<(u32, Topic)> {
#[allow(clippy::match_same_arms)]
match self {
Topic::RemoveParticipant { .. } => None,
Topic::DkgConfirmation { attempt, round } => match round {
SigningProtocolRound::Preprocess => {
let attempt = attempt + 1;
Some((
attempt,
Topic::DkgConfirmation { attempt, round: SigningProtocolRound::Preprocess },
))
}
SigningProtocolRound::Share => None,
},
Topic::SlashReport { .. } => None,
Topic::Sign { id, attempt, round } => match round {
SigningProtocolRound::Preprocess => {
let attempt = attempt + 1;
Some((attempt, Topic::Sign { id, attempt, round: SigningProtocolRound::Preprocess }))
}
SigningProtocolRound::Share => None,
},
}
}
// The SignId for this topic
//
// Returns None if Topic isn't Topic::Sign
pub(crate) fn sign_id(self, set: ValidatorSet) -> Option<messages::sign::SignId> {
#[allow(clippy::match_same_arms)]
match self {
Topic::RemoveParticipant { .. } => None,
Topic::DkgConfirmation { .. } => None,
Topic::SlashReport { .. } => None,
Topic::Sign { id, attempt, round: _ } => Some(SignId { session: set.session, id, attempt }),
}
}
/// The topic which precedes this topic as a prerequisite
///
/// The preceding topic must define this topic as succeeding
fn preceding_topic(self) -> Option<Topic> {
#[allow(clippy::match_same_arms)]
match self {
Topic::RemoveParticipant { .. } => None,
Topic::DkgConfirmation { attempt, round } => match round {
SigningProtocolRound::Preprocess => None,
SigningProtocolRound::Share => {
Some(Topic::DkgConfirmation { attempt, round: SigningProtocolRound::Preprocess })
}
},
Topic::SlashReport { .. } => None,
Topic::Sign { id, attempt, round } => match round {
SigningProtocolRound::Preprocess => None,
SigningProtocolRound::Share => {
Some(Topic::Sign { id, attempt, round: SigningProtocolRound::Preprocess })
}
},
}
}
/// The topic which succeeds this topic, with this topic as a prerequisite
///
/// The succeeding topic must define this topic as preceding
fn succeeding_topic(self) -> Option<Topic> {
#[allow(clippy::match_same_arms)]
match self {
Topic::RemoveParticipant { .. } => None,
Topic::DkgConfirmation { attempt, round } => match round {
SigningProtocolRound::Preprocess => {
Some(Topic::DkgConfirmation { attempt, round: SigningProtocolRound::Share })
}
SigningProtocolRound::Share => None,
},
Topic::SlashReport { .. } => None,
Topic::Sign { id, attempt, round } => match round {
SigningProtocolRound::Preprocess => {
Some(Topic::Sign { id, attempt, round: SigningProtocolRound::Share })
}
SigningProtocolRound::Share => None,
},
}
}
fn requires_whitelisting(&self) -> bool {
#[allow(clippy::match_same_arms)]
match self {
// We don't require whitelisting to remove a participant
Topic::RemoveParticipant { .. } => false,
// We don't require whitelisting for the first attempt, solely the re-attempts
Topic::DkgConfirmation { attempt, .. } => *attempt != 0,
// We don't require whitelisting for the slash report
Topic::SlashReport { .. } => false,
// We do require whitelisting for every sign protocol
Topic::Sign { .. } => true,
}
}
fn required_participation(&self, n: u64) -> u64 {
let _ = self;
// All of our topics require 2/3rds participation
((2 * n) / 3) + 1
}
fn participating(&self) -> Participating {
#[allow(clippy::match_same_arms)]
match self {
Topic::RemoveParticipant { .. } => Participating::Everyone,
Topic::DkgConfirmation { .. } => Participating::Participated,
Topic::SlashReport { .. } => Participating::Everyone,
Topic::Sign { .. } => Participating::Participated,
}
}
}
/// The resulting data set from an accumulation
pub(crate) enum DataSet<D: Borshy> {
/// Accumulating this did not produce a data set to act on
/// (non-existent, not ready, prior handled, not participating, etc.)
None,
/// The data set was ready and we are participating in this event
Participating(HashMap<SeraiAddress, D>),
}
trait Borshy: BorshSerialize + BorshDeserialize {}
impl<T: BorshSerialize + BorshDeserialize> Borshy for T {}
create_db!(
CoordinatorTributary {
// The last handled tributary block's (number, hash)
LastHandledTributaryBlock: (set: ValidatorSet) -> (u64, [u8; 32]),
// The slash points a validator has accrued, with u64::MAX representing a fatal slash.
SlashPoints: (set: ValidatorSet, validator: SeraiAddress) -> u64,
// The latest Substrate block to cosign.
LatestSubstrateBlockToCosign: (set: ValidatorSet) -> [u8; 32],
// The hash of the block we're actively cosigning.
ActivelyCosigning: (set: ValidatorSet) -> [u8; 32],
// If this block has already been cosigned.
Cosigned: (set: ValidatorSet, substrate_block_hash: [u8; 32]) -> (),
// The weight accumulated for a topic.
AccumulatedWeight: (set: ValidatorSet, topic: Topic) -> u64,
// The entries accumulated for a topic, by validator.
Accumulated: <D: Borshy>(set: ValidatorSet, topic: Topic, validator: SeraiAddress) -> D,
// Topics to be recognized as of a certain block number due to the reattempt protocol.
Reattempt: (set: ValidatorSet, block_number: u64) -> Vec<Topic>,
}
);
db_channel!(
CoordinatorTributary {
ProcessorMessages: (set: ValidatorSet) -> messages::CoordinatorMessage,
}
);
pub(crate) struct TributaryDb;
impl TributaryDb {
pub(crate) fn last_handled_tributary_block(
getter: &impl Get,
set: ValidatorSet,
) -> Option<(u64, [u8; 32])> {
LastHandledTributaryBlock::get(getter, set)
}
pub(crate) fn set_last_handled_tributary_block(
txn: &mut impl DbTxn,
set: ValidatorSet,
block_number: u64,
block_hash: [u8; 32],
) {
LastHandledTributaryBlock::set(txn, set, &(block_number, block_hash));
}
pub(crate) fn latest_substrate_block_to_cosign(
getter: &impl Get,
set: ValidatorSet,
) -> Option<[u8; 32]> {
LatestSubstrateBlockToCosign::get(getter, set)
}
pub(crate) fn set_latest_substrate_block_to_cosign(
txn: &mut impl DbTxn,
set: ValidatorSet,
substrate_block_hash: [u8; 32],
) {
LatestSubstrateBlockToCosign::set(txn, set, &substrate_block_hash);
}
pub(crate) fn actively_cosigning(txn: &mut impl DbTxn, set: ValidatorSet) -> Option<[u8; 32]> {
ActivelyCosigning::get(txn, set)
}
pub(crate) fn start_cosigning(
txn: &mut impl DbTxn,
set: ValidatorSet,
substrate_block_hash: [u8; 32],
substrate_block_number: u64,
) {
assert!(
ActivelyCosigning::get(txn, set).is_none(),
"starting cosigning while already cosigning"
);
ActivelyCosigning::set(txn, set, &substrate_block_hash);
TributaryDb::recognize_topic(
txn,
set,
Topic::Sign {
id: VariantSignId::Cosign(substrate_block_number),
attempt: 0,
round: SigningProtocolRound::Preprocess,
},
);
}
pub(crate) fn finish_cosigning(txn: &mut impl DbTxn, set: ValidatorSet) {
assert!(ActivelyCosigning::take(txn, set).is_some(), "finished cosigning but not cosigning");
}
pub(crate) fn mark_cosigned(
txn: &mut impl DbTxn,
set: ValidatorSet,
substrate_block_hash: [u8; 32],
) {
Cosigned::set(txn, set, substrate_block_hash, &());
}
pub(crate) fn cosigned(
txn: &mut impl DbTxn,
set: ValidatorSet,
substrate_block_hash: [u8; 32],
) -> bool {
Cosigned::get(txn, set, substrate_block_hash).is_some()
}
pub(crate) fn recognize_topic(txn: &mut impl DbTxn, set: ValidatorSet, topic: Topic) {
AccumulatedWeight::set(txn, set, topic, &0);
}
pub(crate) fn start_of_block(txn: &mut impl DbTxn, set: ValidatorSet, block_number: u64) {
for topic in Reattempt::take(txn, set, block_number).unwrap_or(vec![]) {
/*
TODO: Slash all people who preprocessed but didn't share, and add a delay to their
participations in future protocols. When we call accumulate, if the participant has no
delay, their accumulation occurs immediately. Else, the accumulation occurs after the
specified delay.
This means even if faulty validators are first to preprocess, they won't be selected for
the signing set unless there's a lack of less faulty validators available.
We need to decrease this delay upon successful partipations, and set it to the maximum upon
`f + 1` validators voting to fatally slash the validator in question. This won't issue the
fatal slash but should still be effective.
*/
Self::recognize_topic(txn, set, topic);
if let Some(id) = topic.sign_id(set) {
Self::send_message(txn, set, messages::sign::CoordinatorMessage::Reattempt { id });
}
}
}
pub(crate) fn fatal_slash(
txn: &mut impl DbTxn,
set: ValidatorSet,
validator: SeraiAddress,
reason: &str,
) {
log::warn!("{validator} fatally slashed: {reason}");
SlashPoints::set(txn, set, validator, &u64::MAX);
}
pub(crate) fn is_fatally_slashed(
getter: &impl Get,
set: ValidatorSet,
validator: SeraiAddress,
) -> bool {
SlashPoints::get(getter, set, validator).unwrap_or(0) == u64::MAX
}
#[allow(clippy::too_many_arguments)]
pub(crate) fn accumulate<D: Borshy>(
txn: &mut impl DbTxn,
set: ValidatorSet,
validators: &[SeraiAddress],
total_weight: u64,
block_number: u64,
topic: Topic,
validator: SeraiAddress,
validator_weight: u64,
data: &D,
) -> DataSet<D> {
// This function will only be called once for a (validator, topic) tuple due to how we handle
// nonces on transactions (deterministically to the topic)
let accumulated_weight = AccumulatedWeight::get(txn, set, topic);
if topic.requires_whitelisting() && accumulated_weight.is_none() {
Self::fatal_slash(txn, set, validator, "participated in unrecognized topic");
return DataSet::None;
}
let mut accumulated_weight = accumulated_weight.unwrap_or(0);
// Check if there's a preceding topic, this validator participated
let preceding_topic = topic.preceding_topic();
if let Some(preceding_topic) = preceding_topic {
if Accumulated::<D>::get(txn, set, preceding_topic, validator).is_none() {
Self::fatal_slash(
txn,
set,
validator,
"participated in topic without participating in prior",
);
return DataSet::None;
}
}
// The complete lack of validation on the data by these NOPs opens the potential for spam here
// If we've already accumulated past the threshold, NOP
if accumulated_weight >= topic.required_participation(total_weight) {
return DataSet::None;
}
// If this is for an old attempt, NOP
if let Some(next_attempt_topic) = topic.next_attempt_topic() {
if AccumulatedWeight::get(txn, set, next_attempt_topic).is_some() {
return DataSet::None;
}
}
// Accumulate the data
accumulated_weight += validator_weight;
AccumulatedWeight::set(txn, set, topic, &accumulated_weight);
Accumulated::set(txn, set, topic, validator, data);
// Check if we now cross the weight threshold
if accumulated_weight >= topic.required_participation(total_weight) {
// Queue this for re-attempt after enough time passes
let reattempt_topic = topic.reattempt_topic();
if let Some((attempt, reattempt_topic)) = reattempt_topic {
// 5 minutes
#[cfg(not(feature = "longer-reattempts"))]
const BASE_REATTEMPT_DELAY: u32 =
(5u32 * 60 * 1000).div_ceil(tributary::tendermint::TARGET_BLOCK_TIME);
// 10 minutes, intended for latent environments like the GitHub CI
#[cfg(feature = "longer-reattempts")]
const BASE_REATTEMPT_DELAY: u32 =
(10u32 * 60 * 1000).div_ceil(tributary::tendermint::TARGET_BLOCK_TIME);
// Linearly scale the time for the protocol with the attempt number
let blocks_till_reattempt = u64::from(attempt * BASE_REATTEMPT_DELAY);
let recognize_at = block_number + blocks_till_reattempt;
let mut queued = Reattempt::get(txn, set, recognize_at).unwrap_or(Vec::with_capacity(1));
queued.push(reattempt_topic);
Reattempt::set(txn, set, recognize_at, &queued);
}
// Register the succeeding topic
let succeeding_topic = topic.succeeding_topic();
if let Some(succeeding_topic) = succeeding_topic {
Self::recognize_topic(txn, set, succeeding_topic);
}
// Fetch and return all participations
let mut data_set = HashMap::with_capacity(validators.len());
for validator in validators {
if let Some(data) = Accumulated::<D>::get(txn, set, topic, *validator) {
// Clean this data up if there's not a re-attempt topic
// If there is a re-attempt topic, we clean it up upon re-attempt
if reattempt_topic.is_none() {
Accumulated::<D>::del(txn, set, topic, *validator);
}
data_set.insert(*validator, data);
}
}
let participated = data_set.contains_key(&validator);
match topic.participating() {
Participating::Participated => {
if participated {
DataSet::Participating(data_set)
} else {
DataSet::None
}
}
Participating::Everyone => DataSet::Participating(data_set),
}
} else {
DataSet::None
}
}
pub(crate) fn send_message(
txn: &mut impl DbTxn,
set: ValidatorSet,
message: impl Into<messages::CoordinatorMessage>,
) {
ProcessorMessages::send(txn, set, &message.into());
}
pub(crate) fn try_recv_message(
txn: &mut impl DbTxn,
set: ValidatorSet,
) -> Option<messages::CoordinatorMessage> {
ProcessorMessages::try_recv(txn, set)
}
}

View File

@@ -1,466 +0,0 @@
use core::{marker::PhantomData, future::Future};
use std::collections::HashMap;
use ciphersuite::group::GroupEncoding;
use serai_client::{
primitives::SeraiAddress,
validator_sets::primitives::{ValidatorSet, Slash},
};
use tributary::{
Signed as TributarySigned, TransactionKind, TransactionTrait,
Transaction as TributaryTransaction, Block, TributaryReader,
tendermint::{
tx::{TendermintTx, Evidence, decode_signed_message},
TendermintNetwork,
},
};
use serai_db::*;
use serai_task::ContinuallyRan;
use messages::sign::VariantSignId;
use serai_cosign::Cosigning;
use crate::{
p2p::P2p,
tributary::{
db::*,
transaction::{SigningProtocolRound, Signed, Transaction},
},
};
struct ScanBlock<'a, CD: Db, TD: Db, TDT: DbTxn, P: P2p> {
_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>,
tributary: &'a TributaryReader<TD, Transaction>,
}
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;
}
// 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;
};
// 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,
);
// 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,
},
);
}
fn handle_application_tx(&mut self, block_number: u64, tx: Transaction) {
let signer = |signed: Signed| SeraiAddress(signed.signer.to_bytes());
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;
}
}
match tx {
// Accumulate this vote and fatally slash the participant if past the threshold
Transaction::RemoveParticipant { participant, signed } => {
let signer = signer(signed);
// 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",
);
return;
}
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");
}
};
}
// 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 }
}
},
)
}
};
}
}
}
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);
}
}
}
}
}
pub(crate) struct ScanTributaryTask<CD: Db, TD: Db, P: P2p> {
pub(crate) cosign_db: CD,
pub(crate) tributary_db: TD,
pub(crate) set: ValidatorSet,
pub(crate) validators: Vec<SeraiAddress>,
pub(crate) total_weight: u64,
pub(crate) validator_weights: HashMap<SeraiAddress, u64>,
pub(crate) tributary: TributaryReader<TD, Transaction>,
pub(crate) _p2p: PhantomData<P>,
}
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 {
_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,
tributary: &self.tributary,
})
.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)
}
}
}

View File

@@ -1,340 +0,0 @@
use core::{ops::Deref, fmt::Debug};
use std::io;
use zeroize::Zeroizing;
use rand_core::{RngCore, CryptoRng};
use blake2::{digest::typenum::U32, Digest, Blake2b};
use ciphersuite::{
group::{ff::Field, GroupEncoding},
Ciphersuite, Ristretto,
};
use schnorr::SchnorrSignature;
use scale::Encode;
use borsh::{BorshSerialize, BorshDeserialize};
use serai_client::{primitives::SeraiAddress, validator_sets::primitives::MAX_KEY_SHARES_PER_SET};
use messages::sign::VariantSignId;
use tributary::{
ReadWrite,
transaction::{
Signed as TributarySigned, TransactionError, TransactionKind, Transaction as TransactionTrait,
},
};
/// The round this data is for, within a signing protocol.
#[derive(Clone, Copy, PartialEq, Eq, Debug, Encode, BorshSerialize, BorshDeserialize)]
pub enum SigningProtocolRound {
/// A preprocess.
Preprocess,
/// A signature share.
Share,
}
impl SigningProtocolRound {
fn nonce(&self) -> u32 {
match self {
SigningProtocolRound::Preprocess => 0,
SigningProtocolRound::Share => 1,
}
}
}
/// `tributary::Signed` but without the nonce.
///
/// All of our nonces are deterministic to the type of transaction and fields within.
#[derive(Clone, Copy, PartialEq, Eq, Debug)]
pub struct Signed {
/// The signer.
pub signer: <Ristretto as Ciphersuite>::G,
/// The signature.
pub signature: SchnorrSignature<Ristretto>,
}
impl BorshSerialize for Signed {
fn serialize<W: io::Write>(&self, writer: &mut W) -> Result<(), io::Error> {
writer.write_all(self.signer.to_bytes().as_ref())?;
self.signature.write(writer)
}
}
impl BorshDeserialize for Signed {
fn deserialize_reader<R: io::Read>(reader: &mut R) -> Result<Self, io::Error> {
let signer = Ristretto::read_G(reader)?;
let signature = SchnorrSignature::read(reader)?;
Ok(Self { signer, signature })
}
}
impl Signed {
/// Provide a nonce to convert a `Signed` into a `tributary::Signed`.
fn nonce(&self, nonce: u32) -> TributarySigned {
TributarySigned { signer: self.signer, nonce, signature: self.signature }
}
}
/// The Tributary transaction definition used by Serai
#[derive(Clone, PartialEq, Eq, Debug, BorshSerialize, BorshDeserialize)]
pub enum Transaction {
/// A vote to remove a participant for invalid behavior
RemoveParticipant {
/// The participant to remove
participant: SeraiAddress,
/// The transaction's signer and signature
signed: Signed,
},
/// A participation in the DKG
DkgParticipation {
participation: Vec<u8>,
/// The transaction's signer and signature
signed: Signed,
},
/// The preprocess to confirm the DKG results on-chain
DkgConfirmationPreprocess {
/// The attempt number of this signing protocol
attempt: u32,
// The preprocess
preprocess: [u8; 64],
/// The transaction's signer and signature
signed: Signed,
},
/// The signature share to confirm the DKG results on-chain
DkgConfirmationShare {
/// The attempt number of this signing protocol
attempt: u32,
// The signature share
share: [u8; 32],
/// The transaction's signer and signature
signed: Signed,
},
/// Intend to co-sign a finalized Substrate block
///
/// When the time comes to start a new co-signing protocol, the most recent Substrate block will
/// be the one selected to be cosigned.
Cosign {
/// The hash of the Substrate block to sign
substrate_block_hash: [u8; 32],
},
/// The cosign for a Substrate block
///
/// After producing this cosign, we need to start work on the latest intended-to-be cosigned
/// block. That requires agreement on when this cosign was produced, which we solve by embedding
/// this cosign on chain.
///
/// We ideally don't have this transaction at all. The coordinator, without access to any of the
/// key shares, could observe the FROST signing session and determine a successful completion.
/// Unfortunately, that functionality is not present in modular-frost, so we do need to support
/// *some* asynchronous flow (where the processor or P2P network informs us of the successful
/// completion).
///
/// If we use a `Provided` transaction, that requires everyone observe this cosign.
///
/// If we use an `Unsigned` transaction, we can't verify the cosign signature inside
/// `Transaction::verify` unless we embedded the full `SignedCosign` on-chain. The issue is since
/// a Tributary is stateless with regards to the on-chain logic, including `Transaction::verify`,
/// we can't verify the signature against the group's public key unless we also include that (but
/// then we open a DoS where arbitrary group keys are specified to cause inclusion of arbitrary
/// blobs on chain).
///
/// If we use a `Signed` transaction, we mitigate the DoS risk by having someone to fatally
/// slash. We have horrible performance though as for 100 validators, all 100 will publish this
/// transaction.
///
/// We could use a signed `Unsigned` transaction, where it includes a signer and signature but
/// isn't technically a Signed transaction. This lets us de-duplicate the transaction premised on
/// its contents.
///
/// The optimal choice is likely to use a `Provided` transaction. We don't actually need to
/// observe the produced cosign (which is ephemeral). As long as it's agreed the cosign in
/// question no longer needs to produced, which would mean the cosigning protocol at-large
/// cosigning the block in question, it'd be safe to provide this and move on to the next cosign.
Cosigned { substrate_block_hash: [u8; 32] },
/// Acknowledge a Substrate block
///
/// This is provided after the block has been cosigned.
///
/// With the acknowledgement of a Substrate block, we can whitelist all the `VariantSignId`s
/// resulting from its handling.
SubstrateBlock {
/// The hash of the Substrate block
hash: [u8; 32],
},
/// Acknowledge a Batch
///
/// Once everyone has acknowledged the Batch, we can begin signing it.
Batch {
/// The hash of the Batch's serialization.
///
/// Generally, we refer to a Batch by its ID/the hash of its instructions. Here, we want to
/// ensure consensus on the Batch, and achieving consensus on its hash is the most effective
/// way to do that.
hash: [u8; 32],
},
/// Data from a signing protocol.
Sign {
/// The ID of the object being signed
id: VariantSignId,
/// The attempt number of this signing protocol
attempt: u32,
/// The round this data is for, within the signing protocol
round: SigningProtocolRound,
/// The data itself
///
/// There will be `n` blobs of data where `n` is the amount of key shares the validator sending
/// this transaction has.
data: Vec<Vec<u8>>,
/// The transaction's signer and signature
signed: Signed,
},
/// The local view of slashes observed by the transaction's sender
SlashReport {
/// The slash points accrued by each validator
slash_points: Vec<u32>,
/// The transaction's signer and signature
signed: Signed,
},
}
impl ReadWrite for Transaction {
fn read<R: io::Read>(reader: &mut R) -> io::Result<Self> {
borsh::from_reader(reader)
}
fn write<W: io::Write>(&self, writer: &mut W) -> io::Result<()> {
borsh::to_writer(writer, self)
}
}
impl TransactionTrait for Transaction {
fn kind(&self) -> TransactionKind {
match self {
Transaction::RemoveParticipant { participant, signed } => {
TransactionKind::Signed((b"RemoveParticipant", participant).encode(), signed.nonce(0))
}
Transaction::DkgParticipation { signed, .. } => {
TransactionKind::Signed(b"DkgParticipation".encode(), signed.nonce(0))
}
Transaction::DkgConfirmationPreprocess { attempt, signed, .. } => {
TransactionKind::Signed((b"DkgConfirmation", attempt).encode(), signed.nonce(0))
}
Transaction::DkgConfirmationShare { attempt, signed, .. } => {
TransactionKind::Signed((b"DkgConfirmation", attempt).encode(), signed.nonce(1))
}
Transaction::Cosign { .. } => TransactionKind::Provided("Cosign"),
Transaction::Cosigned { .. } => TransactionKind::Provided("Cosigned"),
// TODO: Provide this
Transaction::SubstrateBlock { .. } => TransactionKind::Provided("SubstrateBlock"),
// TODO: Provide this
Transaction::Batch { .. } => TransactionKind::Provided("Batch"),
Transaction::Sign { id, attempt, round, signed, .. } => {
TransactionKind::Signed((b"Sign", id, attempt).encode(), signed.nonce(round.nonce()))
}
Transaction::SlashReport { signed, .. } => {
TransactionKind::Signed(b"SlashReport".encode(), signed.nonce(0))
}
}
}
fn hash(&self) -> [u8; 32] {
let mut tx = ReadWrite::serialize(self);
if let TransactionKind::Signed(_, signed) = self.kind() {
// Make sure the part we're cutting off is the signature
assert_eq!(tx.drain((tx.len() - 64) ..).collect::<Vec<_>>(), signed.signature.serialize());
}
Blake2b::<U32>::digest(&tx).into()
}
// This is a stateless verification which we use to enforce some size limits.
fn verify(&self) -> Result<(), TransactionError> {
#[allow(clippy::match_same_arms)]
match self {
// Fixed-length TX
Transaction::RemoveParticipant { .. } => {}
// TODO: MAX_DKG_PARTICIPATION_LEN
Transaction::DkgParticipation { .. } => {}
// These are fixed-length TXs
Transaction::DkgConfirmationPreprocess { .. } | Transaction::DkgConfirmationShare { .. } => {}
// Provided TXs
Transaction::Cosign { .. } |
Transaction::Cosigned { .. } |
Transaction::SubstrateBlock { .. } |
Transaction::Batch { .. } => {}
Transaction::Sign { data, .. } => {
if data.len() > usize::try_from(MAX_KEY_SHARES_PER_SET).unwrap() {
Err(TransactionError::InvalidContent)?
}
// TODO: MAX_SIGN_LEN
}
Transaction::SlashReport { slash_points, .. } => {
if slash_points.len() > usize::try_from(MAX_KEY_SHARES_PER_SET).unwrap() {
Err(TransactionError::InvalidContent)?
}
}
};
Ok(())
}
}
impl Transaction {
// Sign a transaction
//
// Panics if signing a transaction type which isn't `TransactionKind::Signed`
pub fn sign<R: RngCore + CryptoRng>(
&mut self,
rng: &mut R,
genesis: [u8; 32],
key: &Zeroizing<<Ristretto as Ciphersuite>::F>,
) {
fn signed(tx: &mut Transaction) -> &mut Signed {
#[allow(clippy::match_same_arms)] // This doesn't make semantic sense here
match tx {
Transaction::RemoveParticipant { ref mut signed, .. } |
Transaction::DkgParticipation { ref mut signed, .. } |
Transaction::DkgConfirmationPreprocess { ref mut signed, .. } => signed,
Transaction::DkgConfirmationShare { ref mut signed, .. } => signed,
Transaction::Cosign { .. } => panic!("signing CosignSubstrateBlock"),
Transaction::Cosigned { .. } => panic!("signing Cosigned"),
Transaction::SubstrateBlock { .. } => panic!("signing SubstrateBlock"),
Transaction::Batch { .. } => panic!("signing Batch"),
Transaction::Sign { ref mut signed, .. } => signed,
Transaction::SlashReport { ref mut signed, .. } => signed,
}
}
// Decide the nonce to sign with
let sig_nonce = Zeroizing::new(<Ristretto as Ciphersuite>::F::random(rng));
{
// Set the signer and the nonce
let signed = signed(self);
signed.signer = Ristretto::generator() * key.deref();
signed.signature.R = <Ristretto as Ciphersuite>::generator() * sig_nonce.deref();
}
// Get the signature hash (which now includes `R || A` making it valid as the challenge)
let sig_hash = self.sig_hash(genesis);
// Sign the signature
signed(self).signature = SchnorrSignature::<Ristretto>::sign(key, sig_nonce, sig_hash);
}
}