Finish merging in the develop branch

This commit is contained in:
Luke Parker
2025-01-30 03:14:24 -05:00
parent 258c02ff39
commit a275023cfc
62 changed files with 452 additions and 508 deletions

View File

@@ -3,7 +3,7 @@ use std::{sync::Arc, collections::HashMap};
use serai_client::{
primitives::{SeraiAddress, Amount},
validator_sets::primitives::ValidatorSet,
validator_sets::primitives::ExternalValidatorSet,
Serai,
};
@@ -28,7 +28,7 @@ db_channel! {
CosignIntendChannels {
GlobalSessionsChannel: () -> ([u8; 32], GlobalSession),
BlockEvents: () -> BlockEventData,
IntendedCosigns: (set: ValidatorSet) -> CosignIntent,
IntendedCosigns: (set: ExternalValidatorSet) -> CosignIntent,
}
}
@@ -110,7 +110,7 @@ impl<D: Db> ContinuallyRan for CosignIntendTask<D> {
keys.insert(set.network, SeraiAddress::from(*key));
let stake = serai
.validator_sets()
.total_allocated_stake(set.network)
.total_allocated_stake(set.network.into())
.await
.map_err(|e| format!("{e:?}"))?
.unwrap_or(Amount(0))

View File

@@ -11,8 +11,8 @@ use scale::{Encode, Decode};
use borsh::{BorshSerialize, BorshDeserialize};
use serai_client::{
primitives::{NetworkId, SeraiAddress},
validator_sets::primitives::{Session, ValidatorSet, KeyPair},
primitives::{ExternalNetworkId, SeraiAddress},
validator_sets::primitives::{Session, ExternalValidatorSet, KeyPair},
Public, Block, Serai, TemporalSerai,
};
@@ -52,13 +52,13 @@ pub const COSIGN_CONTEXT: &[u8] = b"/serai/coordinator/cosign";
#[derive(Debug, BorshSerialize, BorshDeserialize)]
pub(crate) struct GlobalSession {
pub(crate) start_block_number: u64,
pub(crate) sets: Vec<ValidatorSet>,
pub(crate) keys: HashMap<NetworkId, SeraiAddress>,
pub(crate) stakes: HashMap<NetworkId, u64>,
pub(crate) sets: Vec<ExternalValidatorSet>,
pub(crate) keys: HashMap<ExternalNetworkId, SeraiAddress>,
pub(crate) stakes: HashMap<ExternalNetworkId, u64>,
pub(crate) total_stake: u64,
}
impl GlobalSession {
fn id(mut cosigners: Vec<ValidatorSet>) -> [u8; 32] {
fn id(mut cosigners: Vec<ExternalValidatorSet>) -> [u8; 32] {
cosigners.sort_by_key(|a| borsh::to_vec(a).unwrap());
Blake2s256::digest(borsh::to_vec(&cosigners).unwrap()).into()
}
@@ -101,12 +101,12 @@ pub struct Cosign {
/// The hash of the block to cosign.
pub block_hash: [u8; 32],
/// The actual cosigner.
pub cosigner: NetworkId,
pub cosigner: ExternalNetworkId,
}
impl CosignIntent {
/// Convert this into a `Cosign`.
pub fn into_cosign(self, cosigner: NetworkId) -> Cosign {
pub fn into_cosign(self, cosigner: ExternalNetworkId) -> Cosign {
let CosignIntent { global_session, block_number, block_hash, notable: _ } = self;
Cosign { global_session, block_number, block_hash, cosigner }
}
@@ -166,7 +166,10 @@ create_db! {
// one notable block. All validator sets will explicitly produce a cosign for their notable
// block, causing the latest cosigned block for a global session to either be the global
// session's notable cosigns or the network's latest cosigns.
NetworksLatestCosignedBlock: (global_session: [u8; 32], network: NetworkId) -> SignedCosign,
NetworksLatestCosignedBlock: (
global_session: [u8; 32],
network: ExternalNetworkId
) -> SignedCosign,
// Cosigns received for blocks not locally recognized as finalized.
Faults: (global_session: [u8; 32]) -> Vec<SignedCosign>,
// The global session which faulted.
@@ -177,15 +180,10 @@ create_db! {
/// Fetch the keys used for cosigning by a specific network.
async fn keys_for_network(
serai: &TemporalSerai<'_>,
network: NetworkId,
network: ExternalNetworkId,
) -> Result<Option<(Session, KeyPair)>, String> {
// The Serai network never cosigns so it has no keys for cosigning
if network == NetworkId::Serai {
return Ok(None);
}
let Some(latest_session) =
serai.validator_sets().session(network).await.map_err(|e| format!("{e:?}"))?
serai.validator_sets().session(network.into()).await.map_err(|e| format!("{e:?}"))?
else {
// If this network hasn't had a session declared, move on
return Ok(None);
@@ -194,7 +192,7 @@ async fn keys_for_network(
// Get the keys for the latest session
if let Some(keys) = serai
.validator_sets()
.keys(ValidatorSet { network, session: latest_session })
.keys(ExternalValidatorSet { network, session: latest_session })
.await
.map_err(|e| format!("{e:?}"))?
{
@@ -205,7 +203,7 @@ async fn keys_for_network(
if let Some(prior_session) = latest_session.0.checked_sub(1).map(Session) {
if let Some(keys) = serai
.validator_sets()
.keys(ValidatorSet { network, session: prior_session })
.keys(ExternalValidatorSet { network, session: prior_session })
.await
.map_err(|e| format!("{e:?}"))?
{
@@ -216,16 +214,19 @@ async fn keys_for_network(
Ok(None)
}
/// Fetch the `ValidatorSet`s, and their associated keys, used for cosigning as of this block.
async fn cosigning_sets(serai: &TemporalSerai<'_>) -> Result<Vec<(ValidatorSet, Public)>, String> {
let mut sets = Vec::with_capacity(serai_client::primitives::NETWORKS.len());
for network in serai_client::primitives::NETWORKS {
/// Fetch the `ExternalValidatorSet`s, and their associated keys, used for cosigning as of this
/// block.
async fn cosigning_sets(
serai: &TemporalSerai<'_>,
) -> Result<Vec<(ExternalValidatorSet, Public)>, String> {
let mut sets = Vec::with_capacity(serai_client::primitives::EXTERNAL_NETWORKS.len());
for network in serai_client::primitives::EXTERNAL_NETWORKS {
let Some((session, keys)) = keys_for_network(serai, network).await? else {
// If this network doesn't have usable keys, move on
continue;
};
sets.push((ValidatorSet { network, session }, keys.0));
sets.push((ExternalValidatorSet { network, session }, keys.0));
}
Ok(sets)
}
@@ -345,8 +346,8 @@ impl<D: Db> Cosigning<D> {
/// If this global session hasn't produced any notable cosigns, this will return the latest
/// cosigns for this session.
pub fn notable_cosigns(getter: &impl Get, global_session: [u8; 32]) -> Vec<SignedCosign> {
let mut cosigns = Vec::with_capacity(serai_client::primitives::NETWORKS.len());
for network in serai_client::primitives::NETWORKS {
let mut cosigns = Vec::with_capacity(serai_client::primitives::EXTERNAL_NETWORKS.len());
for network in serai_client::primitives::EXTERNAL_NETWORKS {
if let Some(cosign) = NetworksLatestCosignedBlock::get(getter, global_session, network) {
cosigns.push(cosign);
}
@@ -363,7 +364,7 @@ impl<D: Db> Cosigning<D> {
let mut cosigns = Faults::get(&self.db, faulted).expect("faulted with no faults");
// Also include all of our recognized-as-honest cosigns in an attempt to induce fault
// identification in those who see the faulty cosigns as honest
for network in serai_client::primitives::NETWORKS {
for network in serai_client::primitives::EXTERNAL_NETWORKS {
if let Some(cosign) = NetworksLatestCosignedBlock::get(&self.db, faulted, network) {
if cosign.cosign.global_session == faulted {
cosigns.push(cosign);
@@ -375,8 +376,8 @@ impl<D: Db> Cosigning<D> {
let Some(global_session) = evaluator::currently_evaluated_global_session(&self.db) else {
return vec![];
};
let mut cosigns = Vec::with_capacity(serai_client::primitives::NETWORKS.len());
for network in serai_client::primitives::NETWORKS {
let mut cosigns = Vec::with_capacity(serai_client::primitives::EXTERNAL_NETWORKS.len());
for network in serai_client::primitives::EXTERNAL_NETWORKS {
if let Some(cosign) = NetworksLatestCosignedBlock::get(&self.db, global_session, network) {
cosigns.push(cosign);
}
@@ -487,12 +488,12 @@ impl<D: Db> Cosigning<D> {
Ok(())
}
/// Receive intended cosigns to produce for this ValidatorSet.
/// Receive intended cosigns to produce for this ExternalValidatorSet.
///
/// All cosigns intended, up to and including the next notable cosign, are returned.
///
/// This will drain the internal channel and not re-yield these intentions again.
pub fn intended_cosigns(txn: &mut impl DbTxn, set: ValidatorSet) -> Vec<CosignIntent> {
pub fn intended_cosigns(txn: &mut impl DbTxn, set: ExternalValidatorSet) -> Vec<CosignIntent> {
let mut res: Vec<CosignIntent> = vec![];
// While we have yet to find a notable cosign...
while !res.last().map(|cosign| cosign.notable).unwrap_or(false) {

View File

@@ -14,8 +14,8 @@ use zeroize::Zeroizing;
use schnorrkel::Keypair;
use serai_client::{
primitives::{NetworkId, PublicKey},
validator_sets::primitives::ValidatorSet,
primitives::{ExternalNetworkId, PublicKey},
validator_sets::primitives::ExternalValidatorSet,
Serai,
};
@@ -104,7 +104,7 @@ impl serai_coordinator_p2p::Peer<'_> for Peer<'_> {
#[derive(Clone)]
struct Peers {
peers: Arc<RwLock<HashMap<NetworkId, HashSet<PeerId>>>>,
peers: Arc<RwLock<HashMap<ExternalNetworkId, HashSet<PeerId>>>>,
}
// Consider adding identify/kad/autonat/rendevous/(relay + dcutr). While we currently use the Serai
@@ -135,7 +135,8 @@ struct Libp2pInner {
signed_cosigns: Mutex<mpsc::UnboundedReceiver<SignedCosign>>,
signed_cosigns_send: mpsc::UnboundedSender<SignedCosign>,
heartbeat_requests: Mutex<mpsc::UnboundedReceiver<(InboundRequestId, ValidatorSet, [u8; 32])>>,
heartbeat_requests:
Mutex<mpsc::UnboundedReceiver<(InboundRequestId, ExternalValidatorSet, [u8; 32])>>,
notable_cosign_requests: Mutex<mpsc::UnboundedReceiver<(InboundRequestId, [u8; 32])>>,
inbound_request_responses: mpsc::UnboundedSender<(InboundRequestId, Response)>,
}
@@ -312,7 +313,7 @@ impl serai_cosign::RequestNotableCosigns for Libp2p {
impl serai_coordinator_p2p::P2p for Libp2p {
type Peer<'a> = Peer<'a>;
fn peers(&self, network: NetworkId) -> impl Send + Future<Output = Vec<Self::Peer<'_>>> {
fn peers(&self, network: ExternalNetworkId) -> impl Send + Future<Output = Vec<Self::Peer<'_>>> {
async move {
let Some(peer_ids) = self.0.peers.peers.read().await.get(&network).cloned() else {
return vec![];

View File

@@ -6,7 +6,7 @@ use std::{
use borsh::BorshDeserialize;
use serai_client::validator_sets::primitives::ValidatorSet;
use serai_client::validator_sets::primitives::ExternalValidatorSet;
use tokio::sync::{mpsc, oneshot, RwLock};
@@ -68,7 +68,7 @@ pub(crate) struct SwarmTask {
outbound_request_responses: HashMap<OutboundRequestId, oneshot::Sender<Response>>,
inbound_request_response_channels: HashMap<InboundRequestId, ResponseChannel<Response>>,
heartbeat_requests: mpsc::UnboundedSender<(InboundRequestId, ValidatorSet, [u8; 32])>,
heartbeat_requests: mpsc::UnboundedSender<(InboundRequestId, ExternalValidatorSet, [u8; 32])>,
notable_cosign_requests: mpsc::UnboundedSender<(InboundRequestId, [u8; 32])>,
inbound_request_responses: mpsc::UnboundedReceiver<(InboundRequestId, Response)>,
}
@@ -324,7 +324,7 @@ impl SwarmTask {
outbound_requests: mpsc::UnboundedReceiver<(PeerId, Request, oneshot::Sender<Response>)>,
heartbeat_requests: mpsc::UnboundedSender<(InboundRequestId, ValidatorSet, [u8; 32])>,
heartbeat_requests: mpsc::UnboundedSender<(InboundRequestId, ExternalValidatorSet, [u8; 32])>,
notable_cosign_requests: mpsc::UnboundedSender<(InboundRequestId, [u8; 32])>,
inbound_request_responses: mpsc::UnboundedReceiver<(InboundRequestId, Response)>,
) {

View File

@@ -4,7 +4,9 @@ use std::{
collections::{HashSet, HashMap},
};
use serai_client::{primitives::NetworkId, validator_sets::primitives::Session, SeraiError, Serai};
use serai_client::{
primitives::ExternalNetworkId, validator_sets::primitives::Session, SeraiError, Serai,
};
use serai_task::{Task, ContinuallyRan};
@@ -24,11 +26,11 @@ pub(crate) struct Validators {
serai: Arc<Serai>,
// A cache for which session we're populated with the validators of
sessions: HashMap<NetworkId, Session>,
sessions: HashMap<ExternalNetworkId, Session>,
// The validators by network
by_network: HashMap<NetworkId, HashSet<PeerId>>,
by_network: HashMap<ExternalNetworkId, HashSet<PeerId>>,
// The validators and their networks
validators: HashMap<PeerId, HashSet<NetworkId>>,
validators: HashMap<PeerId, HashSet<ExternalNetworkId>>,
// The channel to send the changes down
changes: mpsc::UnboundedSender<Changes>,
@@ -49,8 +51,8 @@ impl Validators {
async fn session_changes(
serai: impl Borrow<Serai>,
sessions: impl Borrow<HashMap<NetworkId, Session>>,
) -> Result<Vec<(NetworkId, Session, HashSet<PeerId>)>, SeraiError> {
sessions: impl Borrow<HashMap<ExternalNetworkId, Session>>,
) -> Result<Vec<(ExternalNetworkId, Session, HashSet<PeerId>)>, SeraiError> {
/*
This uses the latest finalized block, not the latest cosigned block, which should be fine as
in the worst case, we'd connect to unexpected validators. They still shouldn't be able to
@@ -67,13 +69,10 @@ impl Validators {
// FuturesUnordered can be bad practice as it'll cause timeouts if infrequently polled, but
// we poll it till it yields all futures with the most minimal processing possible
let mut futures = FuturesUnordered::new();
for network in serai_client::primitives::NETWORKS {
if network == NetworkId::Serai {
continue;
}
for network in serai_client::primitives::EXTERNAL_NETWORKS {
let sessions = sessions.borrow();
futures.push(async move {
let session = match temporal_serai.session(network).await {
let session = match temporal_serai.session(network.into()).await {
Ok(Some(session)) => session,
Ok(None) => return Ok(None),
Err(e) => return Err(e),
@@ -82,7 +81,7 @@ impl Validators {
if sessions.get(&network) == Some(&session) {
Ok(None)
} else {
match temporal_serai.active_network_validators(network).await {
match temporal_serai.active_network_validators(network.into()).await {
Ok(validators) => Ok(Some((
network,
session,
@@ -105,7 +104,7 @@ impl Validators {
fn incorporate_session_changes(
&mut self,
session_changes: Vec<(NetworkId, Session, HashSet<PeerId>)>,
session_changes: Vec<(ExternalNetworkId, Session, HashSet<PeerId>)>,
) {
let mut removed = HashSet::new();
let mut added = HashSet::new();
@@ -160,11 +159,11 @@ impl Validators {
Ok(())
}
pub(crate) fn by_network(&self) -> &HashMap<NetworkId, HashSet<PeerId>> {
pub(crate) fn by_network(&self) -> &HashMap<ExternalNetworkId, HashSet<PeerId>> {
&self.by_network
}
pub(crate) fn networks(&self, peer_id: &PeerId) -> Option<&HashSet<NetworkId>> {
pub(crate) fn networks(&self, peer_id: &PeerId) -> Option<&HashSet<ExternalNetworkId>> {
self.validators.get(peer_id)
}
}

View File

@@ -1,7 +1,7 @@
use core::future::Future;
use std::time::{Duration, SystemTime};
use serai_client::validator_sets::primitives::{MAX_KEY_SHARES_PER_SET, ValidatorSet};
use serai_client::validator_sets::primitives::{MAX_KEY_SHARES_PER_SET, ExternalValidatorSet};
use futures_lite::FutureExt;
@@ -38,7 +38,7 @@ pub const BATCH_SIZE_LIMIT: usize = MIN_BLOCKS_PER_BATCH *
/// If the other validator has more blocks then we do, they're expected to inform us. This forms
/// the sync protocol for our Tributaries.
pub(crate) struct HeartbeatTask<TD: Db, Tx: TransactionTrait, P: P2p> {
pub(crate) set: ValidatorSet,
pub(crate) set: ExternalValidatorSet,
pub(crate) tributary: Tributary<TD, Tx, P>,
pub(crate) reader: TributaryReader<TD, Tx>,
pub(crate) p2p: P,

View File

@@ -7,7 +7,7 @@ use std::collections::HashMap;
use borsh::{BorshSerialize, BorshDeserialize};
use serai_client::{primitives::NetworkId, validator_sets::primitives::ValidatorSet};
use serai_client::{primitives::ExternalNetworkId, validator_sets::primitives::ExternalValidatorSet};
use serai_db::Db;
use tributary_sdk::{ReadWrite, TransactionTrait, Tributary, TributaryReader};
@@ -25,7 +25,7 @@ use crate::heartbeat::HeartbeatTask;
#[derive(Clone, Copy, BorshSerialize, BorshDeserialize, Debug)]
pub struct Heartbeat {
/// The Tributary this is the heartbeat of.
pub set: ValidatorSet,
pub set: ExternalValidatorSet,
/// The hash of the latest block added to the Tributary.
pub latest_block_hash: [u8; 32],
}
@@ -56,7 +56,7 @@ pub trait P2p:
type Peer<'a>: Peer<'a>;
/// Fetch the peers for this network.
fn peers(&self, network: NetworkId) -> impl Send + Future<Output = Vec<Self::Peer<'_>>>;
fn peers(&self, network: ExternalNetworkId) -> impl Send + Future<Output = Vec<Self::Peer<'_>>>;
/// Broadcast a cosign.
fn publish_cosign(&self, cosign: SignedCosign) -> impl Send + Future<Output = ()>;
@@ -131,13 +131,13 @@ fn handle_heartbeat<D: Db, T: TransactionTrait>(
pub async fn run<TD: Db, Tx: TransactionTrait, P: P2p>(
db: impl Db,
p2p: P,
mut add_tributary: mpsc::UnboundedReceiver<(ValidatorSet, Tributary<TD, Tx, P>)>,
mut retire_tributary: mpsc::UnboundedReceiver<ValidatorSet>,
mut add_tributary: mpsc::UnboundedReceiver<(ExternalValidatorSet, Tributary<TD, Tx, P>)>,
mut retire_tributary: mpsc::UnboundedReceiver<ExternalValidatorSet>,
send_cosigns: mpsc::UnboundedSender<SignedCosign>,
) {
let mut readers = HashMap::<ValidatorSet, TributaryReader<TD, Tx>>::new();
let mut readers = HashMap::<ExternalValidatorSet, TributaryReader<TD, Tx>>::new();
let mut tributaries = HashMap::<[u8; 32], mpsc::UnboundedSender<Vec<u8>>>::new();
let mut heartbeat_tasks = HashMap::<ValidatorSet, _>::new();
let mut heartbeat_tasks = HashMap::<ExternalValidatorSet, _>::new();
loop {
tokio::select! {

View File

@@ -6,8 +6,8 @@ use serai_db::{create_db, db_channel};
use dkg::Participant;
use serai_client::{
primitives::NetworkId,
validator_sets::primitives::{Session, ValidatorSet, KeyPair},
primitives::ExternalNetworkId,
validator_sets::primitives::{Session, ExternalValidatorSet, KeyPair},
};
use serai_cosign::SignedCosign;
@@ -43,22 +43,21 @@ pub(crate) fn coordinator_db() -> Db {
db(&format!("{root_path}/coordinator/db"))
}
fn tributary_db_folder(set: ValidatorSet) -> String {
fn tributary_db_folder(set: ExternalValidatorSet) -> String {
let root_path = serai_env::var("DB_PATH").expect("path to DB wasn't specified");
let network = match set.network {
NetworkId::Serai => panic!("creating Tributary for the Serai network"),
NetworkId::Bitcoin => "Bitcoin",
NetworkId::Ethereum => "Ethereum",
NetworkId::Monero => "Monero",
ExternalNetworkId::Bitcoin => "Bitcoin",
ExternalNetworkId::Ethereum => "Ethereum",
ExternalNetworkId::Monero => "Monero",
};
format!("{root_path}/tributary-{network}-{}", set.session.0)
}
pub(crate) fn tributary_db(set: ValidatorSet) -> Db {
pub(crate) fn tributary_db(set: ExternalValidatorSet) -> Db {
db(&format!("{}/db", tributary_db_folder(set)))
}
pub(crate) fn prune_tributary_db(set: ValidatorSet) {
pub(crate) fn prune_tributary_db(set: ExternalValidatorSet) {
log::info!("pruning data directory for tributary {set:?}");
let db = tributary_db_folder(set);
if fs::exists(&db).expect("couldn't check if tributary DB exists") {
@@ -73,15 +72,15 @@ create_db! {
// The latest Tributary to have been retired for a network
// Since Tributaries are retired sequentially, this is informative to if any Tributary has been
// retired
RetiredTributary: (network: NetworkId) -> Session,
RetiredTributary: (network: ExternalNetworkId) -> Session,
// The last handled message from a Processor
LastProcessorMessage: (network: NetworkId) -> u64,
LastProcessorMessage: (network: ExternalNetworkId) -> u64,
// Cosigns we produced and tried to intake yet incurred an error while doing so
ErroneousCosigns: () -> Vec<SignedCosign>,
// The keys to confirm and set on the Serai network
KeysToConfirm: (set: ValidatorSet) -> KeyPair,
KeysToConfirm: (set: ExternalValidatorSet) -> KeyPair,
// The key was set on the Serai network
KeySet: (set: ValidatorSet) -> (),
KeySet: (set: ExternalValidatorSet) -> (),
}
}
@@ -90,7 +89,7 @@ db_channel! {
// Cosigns we produced
SignedCosigns: () -> SignedCosign,
// Tributaries to clean up upon reboot
TributaryCleanup: () -> ValidatorSet,
TributaryCleanup: () -> ExternalValidatorSet,
}
}
@@ -100,50 +99,50 @@ mod _internal_db {
db_channel! {
Coordinator {
// Tributary transactions to publish from the Processor messages
TributaryTransactionsFromProcessorMessages: (set: ValidatorSet) -> Transaction,
TributaryTransactionsFromProcessorMessages: (set: ExternalValidatorSet) -> Transaction,
// Tributary transactions to publish from the DKG confirmation task
TributaryTransactionsFromDkgConfirmation: (set: ValidatorSet) -> Transaction,
TributaryTransactionsFromDkgConfirmation: (set: ExternalValidatorSet) -> Transaction,
// Participants to remove
RemoveParticipant: (set: ValidatorSet) -> Participant,
RemoveParticipant: (set: ExternalValidatorSet) -> Participant,
}
}
}
pub(crate) struct TributaryTransactionsFromProcessorMessages;
impl TributaryTransactionsFromProcessorMessages {
pub(crate) fn send(txn: &mut impl DbTxn, set: ValidatorSet, tx: &Transaction) {
pub(crate) fn send(txn: &mut impl DbTxn, set: ExternalValidatorSet, tx: &Transaction) {
// If this set has yet to be retired, send this transaction
if RetiredTributary::get(txn, set.network).map(|session| session.0) < Some(set.session.0) {
_internal_db::TributaryTransactionsFromProcessorMessages::send(txn, set, tx);
}
}
pub(crate) fn try_recv(txn: &mut impl DbTxn, set: ValidatorSet) -> Option<Transaction> {
pub(crate) fn try_recv(txn: &mut impl DbTxn, set: ExternalValidatorSet) -> Option<Transaction> {
_internal_db::TributaryTransactionsFromProcessorMessages::try_recv(txn, set)
}
}
pub(crate) struct TributaryTransactionsFromDkgConfirmation;
impl TributaryTransactionsFromDkgConfirmation {
pub(crate) fn send(txn: &mut impl DbTxn, set: ValidatorSet, tx: &Transaction) {
pub(crate) fn send(txn: &mut impl DbTxn, set: ExternalValidatorSet, tx: &Transaction) {
// If this set has yet to be retired, send this transaction
if RetiredTributary::get(txn, set.network).map(|session| session.0) < Some(set.session.0) {
_internal_db::TributaryTransactionsFromDkgConfirmation::send(txn, set, tx);
}
}
pub(crate) fn try_recv(txn: &mut impl DbTxn, set: ValidatorSet) -> Option<Transaction> {
pub(crate) fn try_recv(txn: &mut impl DbTxn, set: ExternalValidatorSet) -> Option<Transaction> {
_internal_db::TributaryTransactionsFromDkgConfirmation::try_recv(txn, set)
}
}
pub(crate) struct RemoveParticipant;
impl RemoveParticipant {
pub(crate) fn send(txn: &mut impl DbTxn, set: ValidatorSet, participant: Participant) {
pub(crate) fn send(txn: &mut impl DbTxn, set: ExternalValidatorSet, participant: Participant) {
// If this set has yet to be retired, send this transaction
if RetiredTributary::get(txn, set.network).map(|session| session.0) < Some(set.session.0) {
_internal_db::RemoveParticipant::send(txn, set, &participant);
}
}
pub(crate) fn try_recv(txn: &mut impl DbTxn, set: ValidatorSet) -> Option<Participant> {
pub(crate) fn try_recv(txn: &mut impl DbTxn, set: ExternalValidatorSet) -> Option<Participant> {
_internal_db::RemoveParticipant::try_recv(txn, set)
}
}

View File

@@ -17,7 +17,7 @@ use serai_db::{DbTxn, Db as DbTrait};
use serai_client::{
primitives::SeraiAddress,
validator_sets::primitives::{ValidatorSet, musig_context, set_keys_message},
validator_sets::primitives::{ExternalValidatorSet, musig_context, set_keys_message},
};
use serai_task::{DoesNotError, ContinuallyRan};
@@ -141,7 +141,7 @@ impl<CD: DbTrait, TD: DbTrait> ConfirmDkgTask<CD, TD> {
Self { db, set, tributary_db, key, signer: None }
}
fn slash(db: &mut CD, set: ValidatorSet, validator: SeraiAddress) {
fn slash(db: &mut CD, set: ExternalValidatorSet, validator: SeraiAddress) {
let mut txn = db.txn();
TributaryTransactionsFromDkgConfirmation::send(
&mut txn,
@@ -153,7 +153,7 @@ impl<CD: DbTrait, TD: DbTrait> ConfirmDkgTask<CD, TD> {
fn preprocess(
db: &mut CD,
set: ValidatorSet,
set: ExternalValidatorSet,
attempt: u32,
key: &Zeroizing<<Ristretto as Ciphersuite>::F>,
signer: &mut Option<Signer>,
@@ -162,7 +162,9 @@ impl<CD: DbTrait, TD: DbTrait> ConfirmDkgTask<CD, TD> {
let (machine, preprocess) = AlgorithmMachine::new(
schnorrkel(),
// We use a 1-of-1 Musig here as we don't know who will actually be in this Musig yet
musig(&musig_context(set), key, &[Ristretto::generator() * key.deref()]).unwrap().into(),
musig(&musig_context(set.into()), key, &[Ristretto::generator() * key.deref()])
.unwrap()
.into(),
)
.preprocess(&mut OsRng);
// We take the preprocess so we can use it in a distinct machine with the actual Musig
@@ -256,8 +258,9 @@ impl<CD: DbTrait, TD: DbTrait> ContinuallyRan for ConfirmDkgTask<CD, TD> {
})
.collect::<Vec<_>>();
let keys =
musig(&musig_context(self.set.set), &self.key, &musig_public_keys).unwrap().into();
let keys = musig(&musig_context(self.set.set.into()), &self.key, &musig_public_keys)
.unwrap()
.into();
// Rebuild the machine
let (machine, preprocess_from_cache) =

View File

@@ -14,8 +14,8 @@ use borsh::BorshDeserialize;
use tokio::sync::mpsc;
use serai_client::{
primitives::{NetworkId, PublicKey, SeraiAddress, Signature},
validator_sets::primitives::{ValidatorSet, KeyPair},
primitives::{ExternalNetworkId, PublicKey, SeraiAddress, Signature},
validator_sets::primitives::{ExternalValidatorSet, KeyPair},
Serai,
};
use message_queue::{Service, client::MessageQueue};
@@ -153,14 +153,13 @@ async fn handle_network(
mut db: impl serai_db::Db,
message_queue: Arc<MessageQueue>,
serai: Arc<Serai>,
network: NetworkId,
network: ExternalNetworkId,
) {
// Spawn the task to publish batches for this network
{
let (publish_batch_task_def, publish_batch_task) = Task::new();
tokio::spawn(
PublishBatchTask::new(db.clone(), serai.clone(), network)
.unwrap()
.continually_run(publish_batch_task_def, vec![]),
);
// Forget its handle so it always runs in the background
@@ -197,7 +196,7 @@ async fn handle_network(
match msg {
messages::ProcessorMessage::KeyGen(msg) => match msg {
messages::key_gen::ProcessorMessage::Participation { session, participation } => {
let set = ValidatorSet { network, session };
let set = ExternalValidatorSet { network, session };
TributaryTransactionsFromProcessorMessages::send(
&mut txn,
set,
@@ -211,7 +210,7 @@ async fn handle_network(
} => {
KeysToConfirm::set(
&mut txn,
ValidatorSet { network, session },
ExternalValidatorSet { network, session },
&KeyPair(
PublicKey::from_raw(substrate_key),
network_key
@@ -221,15 +220,15 @@ async fn handle_network(
);
}
messages::key_gen::ProcessorMessage::Blame { session, participant } => {
RemoveParticipant::send(&mut txn, ValidatorSet { network, session }, participant);
RemoveParticipant::send(&mut txn, ExternalValidatorSet { network, session }, participant);
}
},
messages::ProcessorMessage::Sign(msg) => match msg {
messages::sign::ProcessorMessage::InvalidParticipant { session, participant } => {
RemoveParticipant::send(&mut txn, ValidatorSet { network, session }, participant);
RemoveParticipant::send(&mut txn, ExternalValidatorSet { network, session }, participant);
}
messages::sign::ProcessorMessage::Preprocesses { id, preprocesses } => {
let set = ValidatorSet { network, session: id.session };
let set = ExternalValidatorSet { network, session: id.session };
if id.attempt == 0 {
// Batches are declared by their intent to be signed
if let messages::sign::VariantSignId::Batch(hash) = id.id {
@@ -254,7 +253,7 @@ async fn handle_network(
);
}
messages::sign::ProcessorMessage::Shares { id, shares } => {
let set = ValidatorSet { network, session: id.session };
let set = ExternalValidatorSet { network, session: id.session };
TributaryTransactionsFromProcessorMessages::send(
&mut txn,
set,
@@ -282,7 +281,7 @@ async fn handle_network(
} => {
SlashReports::set(
&mut txn,
ValidatorSet { network, session },
ExternalValidatorSet { network, session },
slash_report,
Signature(signature),
);
@@ -298,7 +297,7 @@ async fn handle_network(
.push(plan.transaction_plan_id);
}
for (session, plans) in by_session {
let set = ValidatorSet { network, session };
let set = ExternalValidatorSet { network, session };
SubstrateBlockPlans::set(&mut txn, set, block, &plans);
TributaryTransactionsFromProcessorMessages::send(
&mut txn,
@@ -481,10 +480,7 @@ async fn main() {
);
// Handle each of the networks
for network in serai_client::primitives::NETWORKS {
if network == NetworkId::Serai {
continue;
}
for network in serai_client::primitives::EXTERNAL_NETWORKS {
tokio::spawn(handle_network(db.clone(), message_queue.clone(), serai.clone(), network));
}

View File

@@ -9,7 +9,7 @@ use tokio::sync::mpsc;
use serai_db::{DbTxn, Db as DbTrait};
use serai_client::validator_sets::primitives::{Session, ValidatorSet};
use serai_client::validator_sets::primitives::{Session, ExternalValidatorSet};
use message_queue::{Service, Metadata, client::MessageQueue};
use tributary_sdk::Tributary;
@@ -27,8 +27,8 @@ pub(crate) struct SubstrateTask<P: P2p> {
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>,
mpsc::UnboundedSender<(ExternalValidatorSet, Tributary<Db, Transaction, P>)>,
pub(crate) p2p_retire_tributary: mpsc::UnboundedSender<ExternalValidatorSet>,
}
impl<P: P2p> ContinuallyRan for SubstrateTask<P> {
@@ -38,7 +38,7 @@ impl<P: P2p> ContinuallyRan for SubstrateTask<P> {
let mut made_progress = false;
// Handle the Canonical events
for network in serai_client::primitives::NETWORKS {
for network in serai_client::primitives::EXTERNAL_NETWORKS {
loop {
let mut txn = self.db.txn();
let Some(msg) = serai_coordinator_substrate::Canonical::try_recv(&mut txn, network)
@@ -48,7 +48,7 @@ impl<P: P2p> ContinuallyRan for SubstrateTask<P> {
match msg {
messages::substrate::CoordinatorMessage::SetKeys { session, .. } => {
KeySet::set(&mut txn, ValidatorSet { network, session }, &());
KeySet::set(&mut txn, ExternalValidatorSet { network, session }, &());
}
messages::substrate::CoordinatorMessage::SlashesReported { session } => {
let prior_retired = crate::db::RetiredTributary::get(&txn, network);
@@ -58,7 +58,7 @@ impl<P: P2p> ContinuallyRan for SubstrateTask<P> {
crate::db::RetiredTributary::set(&mut txn, network, &session);
self
.p2p_retire_tributary
.send(ValidatorSet { network, session })
.send(ExternalValidatorSet { network, session })
.expect("p2p retire_tributary channel dropped?");
}
messages::substrate::CoordinatorMessage::Block { .. } => {}
@@ -108,7 +108,10 @@ impl<P: P2p> ContinuallyRan for SubstrateTask<P> {
*/
crate::db::TributaryCleanup::send(
&mut txn,
&ValidatorSet { network: new_set.set.network, session: Session(historic_session) },
&ExternalValidatorSet {
network: new_set.set.network,
session: Session(historic_session),
},
);
}

View File

@@ -11,7 +11,7 @@ use tokio::sync::mpsc;
use serai_db::{Get, DbTxn, Db as DbTrait, create_db, db_channel};
use scale::Encode;
use serai_client::validator_sets::primitives::ValidatorSet;
use serai_client::validator_sets::primitives::ExternalValidatorSet;
use tributary_sdk::{TransactionKind, TransactionError, ProvidedError, TransactionTrait, Tributary};
@@ -33,13 +33,13 @@ use crate::{
create_db! {
Coordinator {
PublishOnRecognition: (set: ValidatorSet, topic: Topic) -> Transaction,
PublishOnRecognition: (set: ExternalValidatorSet, topic: Topic) -> Transaction,
}
}
db_channel! {
Coordinator {
PendingCosigns: (set: ValidatorSet) -> CosignIntent,
PendingCosigns: (set: ExternalValidatorSet) -> CosignIntent,
}
}
@@ -48,7 +48,7 @@ db_channel! {
/// This is not a well-designed function. This is specific to the context in which its called,
/// within this file. It should only be considered an internal helper for this domain alone.
async fn provide_transaction<TD: DbTrait, P: P2p>(
set: ValidatorSet,
set: ExternalValidatorSet,
tributary: &Tributary<TD, Transaction, P>,
tx: Transaction,
) {
@@ -211,7 +211,7 @@ async fn add_signed_unsigned_transaction<TD: DbTrait, P: P2p>(
}
async fn add_with_recognition_check<TD: DbTrait, P: P2p>(
set: ValidatorSet,
set: ExternalValidatorSet,
tributary_db: &mut TD,
tributary: &Tributary<TD, Transaction, P>,
key: &Zeroizing<<Ristretto as Ciphersuite>::F>,
@@ -350,7 +350,7 @@ impl<CD: DbTrait, TD: DbTrait, P: P2p> ContinuallyRan for AddTributaryTransactio
/// Takes the messages from ScanTributaryTask and publishes them to the message-queue.
pub(crate) struct TributaryProcessorMessagesTask<TD: DbTrait> {
tributary_db: TD,
set: ValidatorSet,
set: ExternalValidatorSet,
message_queue: Arc<MessageQueue>,
}
impl<TD: DbTrait> ContinuallyRan for TributaryProcessorMessagesTask<TD> {
@@ -430,7 +430,7 @@ impl<CD: DbTrait, TD: DbTrait, P: P2p> ContinuallyRan for SignSlashReportTask<CD
/// Run the scan task whenever the Tributary adds a new block.
async fn scan_on_new_block<CD: DbTrait, TD: DbTrait, P: P2p>(
db: CD,
set: ValidatorSet,
set: ExternalValidatorSet,
tributary: Tributary<TD, Transaction, P>,
scan_tributary_task: TaskHandle,
tasks_to_keep_alive: Vec<TaskHandle>,
@@ -469,7 +469,7 @@ pub(crate) async fn spawn_tributary<P: P2p>(
db: Db,
message_queue: Arc<MessageQueue>,
p2p: P,
p2p_add_tributary: &mpsc::UnboundedSender<(ValidatorSet, Tributary<Db, Transaction, P>)>,
p2p_add_tributary: &mpsc::UnboundedSender<(ExternalValidatorSet, Tributary<Db, Transaction, P>)>,
set: NewSetInformation,
serai_key: Zeroizing<<Ristretto as Ciphersuite>::F>,
) {

View File

@@ -3,7 +3,7 @@ use std::sync::Arc;
use futures::stream::{StreamExt, FuturesOrdered};
use serai_client::Serai;
use serai_client::{validator_sets::primitives::ExternalValidatorSet, Serai};
use messages::substrate::{InInstructionResult, ExecutedBatch, CoordinatorMessage};
@@ -152,6 +152,7 @@ impl<D: Db> ContinuallyRan for CanonicalEventStream<D> {
else {
panic!("SetRetired event wasn't a SetRetired event: {set_retired:?}");
};
let Ok(set) = ExternalValidatorSet::try_from(*set) else { continue };
crate::Canonical::send(
&mut txn,
set.network,
@@ -159,7 +160,7 @@ impl<D: Db> ContinuallyRan for CanonicalEventStream<D> {
);
}
for network in serai_client::primitives::NETWORKS {
for network in serai_client::primitives::EXTERNAL_NETWORKS {
let mut batch = None;
for this_batch in &block.batch_events {
let serai_client::in_instructions::InInstructionsEvent::Batch {
@@ -201,7 +202,7 @@ impl<D: Db> ContinuallyRan for CanonicalEventStream<D> {
let serai_client::coins::CoinsEvent::BurnWithInstruction { from: _, instruction } =
&burn
else {
panic!("Burn event wasn't a Burn.in event: {burn:?}");
panic!("BurnWithInstruction event wasn't a BurnWithInstruction event: {burn:?}");
};
if instruction.balance.coin.network() == network {
burns.push(instruction.clone());

View File

@@ -4,8 +4,8 @@ use std::sync::Arc;
use futures::stream::{StreamExt, FuturesOrdered};
use serai_client::{
primitives::{NetworkId, SeraiAddress, EmbeddedEllipticCurve},
validator_sets::primitives::MAX_KEY_SHARES_PER_SET,
primitives::{SeraiAddress, EmbeddedEllipticCurve},
validator_sets::primitives::{MAX_KEY_SHARES_PER_SET, ExternalValidatorSet},
Serai,
};
@@ -130,16 +130,13 @@ impl<D: Db> ContinuallyRan for EphemeralEventStream<D> {
let serai_client::validator_sets::ValidatorSetsEvent::NewSet { set } = &new_set else {
panic!("NewSet event wasn't a NewSet event: {new_set:?}");
};
// We only coordinate over external networks
if set.network == NetworkId::Serai {
continue;
}
let Ok(set) = ExternalValidatorSet::try_from(*set) else { continue };
let serai = self.serai.as_of(block.block_hash);
let serai = serai.validator_sets();
let Some(validators) =
serai.participants(set.network).await.map_err(|e| format!("{e:?}"))?
serai.participants(set.network.into()).await.map_err(|e| format!("{e:?}"))?
else {
Err(format!(
"block #{block_number} declared a new set but didn't have the participants"
@@ -222,11 +219,11 @@ impl<D: Db> ContinuallyRan for EphemeralEventStream<D> {
}
let mut new_set = NewSetInformation {
set: *set,
set,
serai_block: block.block_hash,
declaration_time: block.time,
// TODO: Why do we have this as an explicit field here?
// Shouldn't this be inlined into the Processor's key gen code, where it's used?
// TODO: This should be inlined into the Processor's key gen code
// It's legacy from when we removed participants from the key gen
threshold: ((total_weight * 2) / 3) + 1,
validators,
evrf_public_keys,
@@ -246,7 +243,8 @@ impl<D: Db> ContinuallyRan for EphemeralEventStream<D> {
else {
panic!("AcceptedHandover event wasn't a AcceptedHandover event: {accepted_handover:?}");
};
crate::SignSlashReport::send(&mut txn, *set);
let Ok(set) = ExternalValidatorSet::try_from(*set) else { continue };
crate::SignSlashReport::send(&mut txn, set);
}
txn.commit();

View File

@@ -10,8 +10,8 @@ use borsh::{BorshSerialize, BorshDeserialize};
use dkg::Participant;
use serai_client::{
primitives::{NetworkId, SeraiAddress, Signature},
validator_sets::primitives::{Session, ValidatorSet, KeyPair, SlashReport},
primitives::{ExternalNetworkId, SeraiAddress, Signature},
validator_sets::primitives::{Session, ExternalValidatorSet, KeyPair, SlashReport},
in_instructions::primitives::SignedBatch,
Transaction,
};
@@ -35,7 +35,7 @@ pub use publish_slash_report::PublishSlashReportTask;
#[borsh(init = init_participant_indexes)]
pub struct NewSetInformation {
/// The set.
pub set: ValidatorSet,
pub set: ExternalValidatorSet,
/// The Serai block which declared it.
pub serai_block: [u8; 32],
/// The time of the block which declared it, in seconds.
@@ -82,24 +82,24 @@ mod _public_db {
db_channel!(
CoordinatorSubstrate {
// Canonical messages to send to the processor
Canonical: (network: NetworkId) -> messages::substrate::CoordinatorMessage,
Canonical: (network: ExternalNetworkId) -> messages::substrate::CoordinatorMessage,
// Relevant new set, from an ephemeral event stream
NewSet: () -> NewSetInformation,
// Potentially relevant sign slash report, from an ephemeral event stream
SignSlashReport: (set: ValidatorSet) -> (),
SignSlashReport: (set: ExternalValidatorSet) -> (),
// Signed batches to publish onto the Serai network
SignedBatches: (network: NetworkId) -> SignedBatch,
SignedBatches: (network: ExternalNetworkId) -> SignedBatch,
}
);
create_db!(
CoordinatorSubstrate {
// Keys to set on the Serai network
Keys: (network: NetworkId) -> (Session, Vec<u8>),
Keys: (network: ExternalNetworkId) -> (Session, Vec<u8>),
// Slash reports to publish onto the Serai network
SlashReports: (network: NetworkId) -> (Session, Vec<u8>),
SlashReports: (network: ExternalNetworkId) -> (Session, Vec<u8>),
}
);
}
@@ -109,7 +109,7 @@ pub struct Canonical;
impl Canonical {
pub(crate) fn send(
txn: &mut impl DbTxn,
network: NetworkId,
network: ExternalNetworkId,
msg: &messages::substrate::CoordinatorMessage,
) {
_public_db::Canonical::send(txn, network, msg);
@@ -117,7 +117,7 @@ impl Canonical {
/// Try to receive a canonical event, returning `None` if there is none to receive.
pub fn try_recv(
txn: &mut impl DbTxn,
network: NetworkId,
network: ExternalNetworkId,
) -> Option<messages::substrate::CoordinatorMessage> {
_public_db::Canonical::try_recv(txn, network)
}
@@ -141,12 +141,12 @@ impl NewSet {
/// notifications for all relevant validator sets will be included.
pub struct SignSlashReport;
impl SignSlashReport {
pub(crate) fn send(txn: &mut impl DbTxn, set: ValidatorSet) {
pub(crate) fn send(txn: &mut impl DbTxn, set: ExternalValidatorSet) {
_public_db::SignSlashReport::send(txn, set, &());
}
/// Try to receive a notification to sign a slash report, returning `None` if there is none to
/// receive.
pub fn try_recv(txn: &mut impl DbTxn, set: ValidatorSet) -> Option<()> {
pub fn try_recv(txn: &mut impl DbTxn, set: ExternalValidatorSet) -> Option<()> {
_public_db::SignSlashReport::try_recv(txn, set)
}
}
@@ -160,7 +160,7 @@ impl Keys {
/// reported at once.
pub fn set(
txn: &mut impl DbTxn,
set: ValidatorSet,
set: ExternalValidatorSet,
key_pair: KeyPair,
signature_participants: bitvec::vec::BitVec<u8, bitvec::order::Lsb0>,
signature: Signature,
@@ -180,7 +180,10 @@ impl Keys {
);
_public_db::Keys::set(txn, set.network, &(set.session, tx.encode()));
}
pub(crate) fn take(txn: &mut impl DbTxn, network: NetworkId) -> Option<(Session, Transaction)> {
pub(crate) fn take(
txn: &mut impl DbTxn,
network: ExternalNetworkId,
) -> Option<(Session, Transaction)> {
let (session, tx) = _public_db::Keys::take(txn, network)?;
Some((session, <_>::decode(&mut tx.as_slice()).unwrap()))
}
@@ -193,7 +196,7 @@ impl SignedBatches {
pub fn send(txn: &mut impl DbTxn, batch: &SignedBatch) {
_public_db::SignedBatches::send(txn, batch.batch.network, batch);
}
pub(crate) fn try_recv(txn: &mut impl DbTxn, network: NetworkId) -> Option<SignedBatch> {
pub(crate) fn try_recv(txn: &mut impl DbTxn, network: ExternalNetworkId) -> Option<SignedBatch> {
_public_db::SignedBatches::try_recv(txn, network)
}
}
@@ -207,7 +210,7 @@ impl SlashReports {
/// slashes reported at once.
pub fn set(
txn: &mut impl DbTxn,
set: ValidatorSet,
set: ExternalValidatorSet,
slash_report: SlashReport,
signature: Signature,
) {
@@ -225,7 +228,10 @@ impl SlashReports {
);
_public_db::SlashReports::set(txn, set.network, &(set.session, tx.encode()));
}
pub(crate) fn take(txn: &mut impl DbTxn, network: NetworkId) -> Option<(Session, Transaction)> {
pub(crate) fn take(
txn: &mut impl DbTxn,
network: ExternalNetworkId,
) -> Option<(Session, Transaction)> {
let (session, tx) = _public_db::SlashReports::take(txn, network)?;
Some((session, <_>::decode(&mut tx.as_slice()).unwrap()))
}

View File

@@ -2,7 +2,7 @@ use core::future::Future;
use std::sync::Arc;
#[rustfmt::skip]
use serai_client::{primitives::NetworkId, in_instructions::primitives::SignedBatch, SeraiError, Serai};
use serai_client::{primitives::ExternalNetworkId, in_instructions::primitives::SignedBatch, SeraiError, Serai};
use serai_db::{Get, DbTxn, Db, create_db};
use serai_task::ContinuallyRan;
@@ -11,8 +11,8 @@ use crate::SignedBatches;
create_db!(
CoordinatorSubstrate {
LastPublishedBatch: (network: NetworkId) -> u32,
BatchesToPublish: (network: NetworkId, batch: u32) -> SignedBatch,
LastPublishedBatch: (network: ExternalNetworkId) -> u32,
BatchesToPublish: (network: ExternalNetworkId, batch: u32) -> SignedBatch,
}
);
@@ -20,19 +20,13 @@ create_db!(
pub struct PublishBatchTask<D: Db> {
db: D,
serai: Arc<Serai>,
network: NetworkId,
network: ExternalNetworkId,
}
impl<D: Db> PublishBatchTask<D> {
/// Create a task to publish `SignedBatch`s onto Serai.
///
/// Returns None if `network == NetworkId::Serai`.
// TODO: ExternalNetworkId
pub fn new(db: D, serai: Arc<Serai>, network: NetworkId) -> Option<Self> {
if network == NetworkId::Serai {
None?
};
Some(Self { db, serai, network })
pub fn new(db: D, serai: Arc<Serai>, network: ExternalNetworkId) -> Self {
Self { db, serai, network }
}
}

View File

@@ -3,7 +3,7 @@ use std::sync::Arc;
use serai_db::{DbTxn, Db};
use serai_client::{primitives::NetworkId, validator_sets::primitives::Session, Serai};
use serai_client::{primitives::ExternalNetworkId, validator_sets::primitives::Session, Serai};
use serai_task::ContinuallyRan;
@@ -24,7 +24,7 @@ impl<D: Db> PublishSlashReportTask<D> {
impl<D: Db> PublishSlashReportTask<D> {
// Returns if a slash report was successfully published
async fn publish(&mut self, network: NetworkId) -> Result<bool, String> {
async fn publish(&mut self, network: ExternalNetworkId) -> Result<bool, String> {
let mut txn = self.db.txn();
let Some((session, slash_report)) = SlashReports::take(&mut txn, network) else {
// No slash report to publish
@@ -36,7 +36,7 @@ impl<D: Db> PublishSlashReportTask<D> {
let serai = self.serai.as_of_latest_finalized_block().await.map_err(|e| format!("{e:?}"))?;
let serai = serai.validator_sets();
let session_after_slash_report = Session(session.0 + 1);
let current_session = serai.session(network).await.map_err(|e| format!("{e:?}"))?;
let current_session = serai.session(network.into()).await.map_err(|e| format!("{e:?}"))?;
let current_session = current_session.map(|session| session.0);
// Only attempt to publish the slash report for session #n while session #n+1 is still
// active
@@ -84,11 +84,7 @@ impl<D: Db> ContinuallyRan for PublishSlashReportTask<D> {
async move {
let mut made_progress = false;
let mut error = None;
for network in serai_client::primitives::NETWORKS {
if network == NetworkId::Serai {
continue;
};
for network in serai_client::primitives::EXTERNAL_NETWORKS {
let network_res = self.publish(network).await;
// We made progress if any network successfully published their slash report
made_progress |= network_res == Ok(true);

View File

@@ -3,7 +3,7 @@ use std::sync::Arc;
use serai_db::{DbTxn, Db};
use serai_client::{primitives::NetworkId, validator_sets::primitives::ValidatorSet, Serai};
use serai_client::{validator_sets::primitives::ExternalValidatorSet, Serai};
use serai_task::ContinuallyRan;
@@ -28,11 +28,7 @@ impl<D: Db> ContinuallyRan for SetKeysTask<D> {
fn run_iteration(&mut self) -> impl Send + Future<Output = Result<bool, Self::Error>> {
async move {
let mut made_progress = false;
for network in serai_client::primitives::NETWORKS {
if network == NetworkId::Serai {
continue;
};
for network in serai_client::primitives::EXTERNAL_NETWORKS {
let mut txn = self.db.txn();
let Some((session, keys)) = Keys::take(&mut txn, network) else {
// No keys to set
@@ -44,7 +40,7 @@ impl<D: Db> ContinuallyRan for SetKeysTask<D> {
let serai =
self.serai.as_of_latest_finalized_block().await.map_err(|e| format!("{e:?}"))?;
let serai = serai.validator_sets();
let current_session = serai.session(network).await.map_err(|e| format!("{e:?}"))?;
let current_session = serai.session(network.into()).await.map_err(|e| format!("{e:?}"))?;
let current_session = current_session.map(|session| session.0);
// Only attempt to set these keys if this isn't a retired session
if Some(session.0) < current_session {
@@ -62,7 +58,7 @@ impl<D: Db> ContinuallyRan for SetKeysTask<D> {
// If this session already has had its keys set, move on
if serai
.keys(ValidatorSet { network, session })
.keys(ExternalValidatorSet { network, session })
.await
.map_err(|e| format!("{e:?}"))?
.is_some()

View File

@@ -3,7 +3,7 @@ use std::collections::HashMap;
use scale::Encode;
use borsh::{BorshSerialize, BorshDeserialize};
use serai_client::{primitives::SeraiAddress, validator_sets::primitives::ValidatorSet};
use serai_client::{primitives::SeraiAddress, validator_sets::primitives::ExternalValidatorSet};
use messages::sign::{VariantSignId, SignId};
@@ -97,7 +97,7 @@ impl Topic {
/// 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> {
pub(crate) fn sign_id(self, set: ExternalValidatorSet) -> Option<messages::sign::SignId> {
#[allow(clippy::match_same_arms)]
match self {
Topic::RemoveParticipant { .. } => None,
@@ -115,7 +115,7 @@ impl Topic {
/// Returns None if Topic isn't Topic::DkgConfirmation.
pub(crate) fn dkg_confirmation_sign_id(
self,
set: ValidatorSet,
set: ExternalValidatorSet,
) -> Option<messages::sign::SignId> {
#[allow(clippy::match_same_arms)]
match self {
@@ -227,41 +227,48 @@ pub(crate) enum DataSet<D: Borshy> {
create_db!(
CoordinatorTributary {
// The last handled tributary block's (number, hash)
LastHandledTributaryBlock: (set: ValidatorSet) -> (u64, [u8; 32]),
LastHandledTributaryBlock: (set: ExternalValidatorSet) -> (u64, [u8; 32]),
// The slash points a validator has accrued, with u32::MAX representing a fatal slash.
SlashPoints: (set: ValidatorSet, validator: SeraiAddress) -> u32,
SlashPoints: (set: ExternalValidatorSet, validator: SeraiAddress) -> u32,
// The cosign intent for a Substrate block
CosignIntents: (set: ValidatorSet, substrate_block_hash: [u8; 32]) -> CosignIntent,
CosignIntents: (set: ExternalValidatorSet, substrate_block_hash: [u8; 32]) -> CosignIntent,
// The latest Substrate block to cosign.
LatestSubstrateBlockToCosign: (set: ValidatorSet) -> [u8; 32],
LatestSubstrateBlockToCosign: (set: ExternalValidatorSet) -> [u8; 32],
// The hash of the block we're actively cosigning.
ActivelyCosigning: (set: ValidatorSet) -> [u8; 32],
ActivelyCosigning: (set: ExternalValidatorSet) -> [u8; 32],
// If this block has already been cosigned.
Cosigned: (set: ValidatorSet, substrate_block_hash: [u8; 32]) -> (),
Cosigned: (set: ExternalValidatorSet, substrate_block_hash: [u8; 32]) -> (),
// The plans to recognize upon a `Transaction::SubstrateBlock` being included on-chain.
SubstrateBlockPlans: (set: ValidatorSet, substrate_block_hash: [u8; 32]) -> Vec<[u8; 32]>,
SubstrateBlockPlans: (
set: ExternalValidatorSet,
substrate_block_hash: [u8; 32]
) -> Vec<[u8; 32]>,
// The weight accumulated for a topic.
AccumulatedWeight: (set: ValidatorSet, topic: Topic) -> u16,
AccumulatedWeight: (set: ExternalValidatorSet, topic: Topic) -> u16,
// The entries accumulated for a topic, by validator.
Accumulated: <D: Borshy>(set: ValidatorSet, topic: Topic, validator: SeraiAddress) -> D,
Accumulated: <D: Borshy>(
set: ExternalValidatorSet,
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>,
Reattempt: (set: ExternalValidatorSet, block_number: u64) -> Vec<Topic>,
}
);
db_channel!(
CoordinatorTributary {
// Messages to send to the processor
ProcessorMessages: (set: ValidatorSet) -> messages::CoordinatorMessage,
ProcessorMessages: (set: ExternalValidatorSet) -> messages::CoordinatorMessage,
// Messages for the DKG confirmation
DkgConfirmationMessages: (set: ValidatorSet) -> messages::sign::CoordinatorMessage,
DkgConfirmationMessages: (set: ExternalValidatorSet) -> messages::sign::CoordinatorMessage,
// Topics which have been explicitly recognized
RecognizedTopics: (set: ValidatorSet) -> Topic,
RecognizedTopics: (set: ExternalValidatorSet) -> Topic,
}
);
@@ -269,13 +276,13 @@ pub(crate) struct TributaryDb;
impl TributaryDb {
pub(crate) fn last_handled_tributary_block(
getter: &impl Get,
set: ValidatorSet,
set: ExternalValidatorSet,
) -> Option<(u64, [u8; 32])> {
LastHandledTributaryBlock::get(getter, set)
}
pub(crate) fn set_last_handled_tributary_block(
txn: &mut impl DbTxn,
set: ValidatorSet,
set: ExternalValidatorSet,
block_number: u64,
block_hash: [u8; 32],
) {
@@ -284,23 +291,26 @@ impl TributaryDb {
pub(crate) fn latest_substrate_block_to_cosign(
getter: &impl Get,
set: ValidatorSet,
set: ExternalValidatorSet,
) -> Option<[u8; 32]> {
LatestSubstrateBlockToCosign::get(getter, set)
}
pub(crate) fn set_latest_substrate_block_to_cosign(
txn: &mut impl DbTxn,
set: ValidatorSet,
set: ExternalValidatorSet,
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]> {
pub(crate) fn actively_cosigning(
txn: &mut impl DbTxn,
set: ExternalValidatorSet,
) -> Option<[u8; 32]> {
ActivelyCosigning::get(txn, set)
}
pub(crate) fn start_cosigning(
txn: &mut impl DbTxn,
set: ValidatorSet,
set: ExternalValidatorSet,
substrate_block_hash: [u8; 32],
substrate_block_number: u64,
) {
@@ -320,33 +330,33 @@ impl TributaryDb {
},
);
}
pub(crate) fn finish_cosigning(txn: &mut impl DbTxn, set: ValidatorSet) {
pub(crate) fn finish_cosigning(txn: &mut impl DbTxn, set: ExternalValidatorSet) {
assert!(ActivelyCosigning::take(txn, set).is_some(), "finished cosigning but not cosigning");
}
pub(crate) fn mark_cosigned(
txn: &mut impl DbTxn,
set: ValidatorSet,
set: ExternalValidatorSet,
substrate_block_hash: [u8; 32],
) {
Cosigned::set(txn, set, substrate_block_hash, &());
}
pub(crate) fn cosigned(
txn: &mut impl DbTxn,
set: ValidatorSet,
set: ExternalValidatorSet,
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) {
pub(crate) fn recognize_topic(txn: &mut impl DbTxn, set: ExternalValidatorSet, topic: Topic) {
AccumulatedWeight::set(txn, set, topic, &0);
RecognizedTopics::send(txn, set, &topic);
}
pub(crate) fn recognized(getter: &impl Get, set: ValidatorSet, topic: Topic) -> bool {
pub(crate) fn recognized(getter: &impl Get, set: ExternalValidatorSet, topic: Topic) -> bool {
AccumulatedWeight::get(getter, set, topic).is_some()
}
pub(crate) fn start_of_block(txn: &mut impl DbTxn, set: ValidatorSet, block_number: u64) {
pub(crate) fn start_of_block(txn: &mut impl DbTxn, set: ExternalValidatorSet, 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
@@ -376,7 +386,7 @@ impl TributaryDb {
pub(crate) fn fatal_slash(
txn: &mut impl DbTxn,
set: ValidatorSet,
set: ExternalValidatorSet,
validator: SeraiAddress,
reason: &str,
) {
@@ -386,7 +396,7 @@ impl TributaryDb {
pub(crate) fn is_fatally_slashed(
getter: &impl Get,
set: ValidatorSet,
set: ExternalValidatorSet,
validator: SeraiAddress,
) -> bool {
SlashPoints::get(getter, set, validator).unwrap_or(0) == u32::MAX
@@ -395,7 +405,7 @@ impl TributaryDb {
#[allow(clippy::too_many_arguments)]
pub(crate) fn accumulate<D: Borshy>(
txn: &mut impl DbTxn,
set: ValidatorSet,
set: ExternalValidatorSet,
validators: &[SeraiAddress],
total_weight: u16,
block_number: u64,
@@ -511,7 +521,7 @@ impl TributaryDb {
pub(crate) fn send_message(
txn: &mut impl DbTxn,
set: ValidatorSet,
set: ExternalValidatorSet,
message: impl Into<messages::CoordinatorMessage>,
) {
ProcessorMessages::send(txn, set, &message.into());

View File

@@ -10,7 +10,7 @@ use dkg::Participant;
use serai_client::{
primitives::SeraiAddress,
validator_sets::primitives::{ValidatorSet, Slash},
validator_sets::primitives::{ExternalValidatorSet, Slash},
};
use serai_db::*;
@@ -41,7 +41,10 @@ pub use db::Topic;
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> {
pub fn try_recv(
txn: &mut impl DbTxn,
set: ExternalValidatorSet,
) -> Option<messages::CoordinatorMessage> {
db::ProcessorMessages::try_recv(txn, set)
}
}
@@ -58,7 +61,7 @@ impl DkgConfirmationMessages {
/// across validator sets, with no guarantees of uniqueness across contexts.
pub fn try_recv(
txn: &mut impl DbTxn,
set: ValidatorSet,
set: ExternalValidatorSet,
) -> Option<messages::sign::CoordinatorMessage> {
db::DkgConfirmationMessages::try_recv(txn, set)
}
@@ -70,12 +73,12 @@ impl CosignIntents {
/// Provide a CosignIntent for this Tributary.
///
/// This must be done before the associated `Transaction::Cosign` is provided.
pub fn provide(txn: &mut impl DbTxn, set: ValidatorSet, intent: &CosignIntent) {
pub fn provide(txn: &mut impl DbTxn, set: ExternalValidatorSet, intent: &CosignIntent) {
db::CosignIntents::set(txn, set, intent.block_hash, intent);
}
fn take(
txn: &mut impl DbTxn,
set: ValidatorSet,
set: ExternalValidatorSet,
substrate_block_hash: [u8; 32],
) -> Option<CosignIntent> {
db::CosignIntents::take(txn, set, substrate_block_hash)
@@ -88,13 +91,13 @@ impl RecognizedTopics {
/// If this topic has been recognized by this Tributary.
///
/// This will either be by explicit recognition or participation.
pub fn recognized(getter: &impl Get, set: ValidatorSet, topic: Topic) -> bool {
pub fn recognized(getter: &impl Get, set: ExternalValidatorSet, topic: Topic) -> bool {
TributaryDb::recognized(getter, set, topic)
}
/// The next topic requiring recognition which has been recognized by this Tributary.
pub fn try_recv_topic_requiring_recognition(
txn: &mut impl DbTxn,
set: ValidatorSet,
set: ExternalValidatorSet,
) -> Option<Topic> {
db::RecognizedTopics::try_recv(txn, set)
}
@@ -109,7 +112,7 @@ impl SubstrateBlockPlans {
/// This must be done before the associated `Transaction::Cosign` is provided.
pub fn set(
txn: &mut impl DbTxn,
set: ValidatorSet,
set: ExternalValidatorSet,
substrate_block_hash: [u8; 32],
plans: &Vec<[u8; 32]>,
) {
@@ -117,7 +120,7 @@ impl SubstrateBlockPlans {
}
fn take(
txn: &mut impl DbTxn,
set: ValidatorSet,
set: ExternalValidatorSet,
substrate_block_hash: [u8; 32],
) -> Option<Vec<[u8; 32]>> {
db::SubstrateBlockPlans::take(txn, set, substrate_block_hash)