Update all of serai-coordinator to compile with the new serai-client-serai

This commit is contained in:
Luke Parker
2025-11-16 11:50:24 -05:00
parent 9891ccade8
commit 7a314baa9f
37 changed files with 1014 additions and 1081 deletions

View File

@@ -107,8 +107,8 @@ jobs:
- name: Run Tests - name: Run Tests
run: | run: |
GITHUB_CI=true RUST_BACKTRACE=1 cargo test --all-features -p serai-client-serai
GITHUB_CI=true RUST_BACKTRACE=1 cargo test --all-features -p serai-client-bitcoin GITHUB_CI=true RUST_BACKTRACE=1 cargo test --all-features -p serai-client-bitcoin
GITHUB_CI=true RUST_BACKTRACE=1 cargo test --all-features -p serai-client-ethereum GITHUB_CI=true RUST_BACKTRACE=1 cargo test --all-features -p serai-client-ethereum
GITHUB_CI=true RUST_BACKTRACE=1 cargo test --all-features -p serai-client-monero GITHUB_CI=true RUST_BACKTRACE=1 cargo test --all-features -p serai-client-monero
GITHUB_CI=true RUST_BACKTRACE=1 cargo test --all-features -p serai-client-serai
GITHUB_CI=true RUST_BACKTRACE=1 cargo test --all-features -p serai-client GITHUB_CI=true RUST_BACKTRACE=1 cargo test --all-features -p serai-client

1403
Cargo.lock generated

File diff suppressed because it is too large Load Diff

View File

@@ -93,10 +93,10 @@ members = [
"substrate/runtime", "substrate/runtime",
"substrate/node", "substrate/node",
"substrate/client/serai",
"substrate/client/bitcoin", "substrate/client/bitcoin",
"substrate/client/ethereum", "substrate/client/ethereum",
"substrate/client/monero", "substrate/client/monero",
"substrate/client/serai",
"substrate/client", "substrate/client",
"orchestration", "orchestration",

View File

@@ -42,7 +42,7 @@ messages = { package = "serai-processor-messages", path = "../processor/messages
message-queue = { package = "serai-message-queue", path = "../message-queue" } message-queue = { package = "serai-message-queue", path = "../message-queue" }
tributary-sdk = { path = "./tributary-sdk" } tributary-sdk = { path = "./tributary-sdk" }
serai-client = { path = "../substrate/client", default-features = false, features = ["serai"] } serai-client-serai = { path = "../substrate/client/serai", default-features = false }
log = { version = "0.4", default-features = false, features = ["std"] } log = { version = "0.4", default-features = false, features = ["std"] }
env_logger = { version = "0.10", default-features = false, features = ["humantime"] } env_logger = { version = "0.10", default-features = false, features = ["humantime"] }

View File

@@ -21,7 +21,6 @@ workspace = true
blake2 = { version = "0.11.0-rc.0", default-features = false, features = ["alloc"] } blake2 = { version = "0.11.0-rc.0", default-features = false, features = ["alloc"] }
borsh = { version = "1", default-features = false, features = ["std", "derive", "de_strict_order"] } borsh = { version = "1", default-features = false, features = ["std", "derive", "de_strict_order"] }
serai-abi = { path = "../../substrate/abi", default-features = false, features = ["std"] }
serai-client-serai = { path = "../../substrate/client/serai", default-features = false } serai-client-serai = { path = "../../substrate/client/serai", default-features = false }
log = { version = "0.4", default-features = false, features = ["std"] } log = { version = "0.4", default-features = false, features = ["std"] }

View File

@@ -3,11 +3,13 @@ use std::{sync::Arc, collections::HashMap};
use blake2::{Digest, Blake2b256}; use blake2::{Digest, Blake2b256};
use serai_abi::primitives::{ use serai_client_serai::{
abi::primitives::{
balance::Amount, validator_sets::ExternalValidatorSet, address::SeraiAddress, balance::Amount, validator_sets::ExternalValidatorSet, address::SeraiAddress,
merkle::IncrementalUnbalancedMerkleTree, merkle::IncrementalUnbalancedMerkleTree,
},
Serai,
}; };
use serai_client_serai::Serai;
use serai_db::*; use serai_db::*;
use serai_task::ContinuallyRan; use serai_task::ContinuallyRan;
@@ -85,7 +87,7 @@ impl<D: Db> ContinuallyRan for CosignIntendTask<D> {
// Check we are indexing a linear chain // Check we are indexing a linear chain
if block.header.builds_upon() != if block.header.builds_upon() !=
builds_upon.clone().calculate(serai_abi::BLOCK_HEADER_BRANCH_TAG) builds_upon.clone().calculate(serai_client_serai::abi::BLOCK_HEADER_BRANCH_TAG)
{ {
Err(format!( Err(format!(
"node's block #{block_number} doesn't build upon the block #{} prior indexed", "node's block #{block_number} doesn't build upon the block #{} prior indexed",
@@ -95,8 +97,8 @@ impl<D: Db> ContinuallyRan for CosignIntendTask<D> {
let block_hash = block.header.hash(); let block_hash = block.header.hash();
SubstrateBlockHash::set(&mut txn, block_number, &block_hash); SubstrateBlockHash::set(&mut txn, block_number, &block_hash);
builds_upon.append( builds_upon.append(
serai_abi::BLOCK_HEADER_BRANCH_TAG, serai_client_serai::abi::BLOCK_HEADER_BRANCH_TAG,
Blake2b256::new_with_prefix([serai_abi::BLOCK_HEADER_LEAF_TAG]) Blake2b256::new_with_prefix([serai_client_serai::abi::BLOCK_HEADER_LEAF_TAG])
.chain_update(block_hash.0) .chain_update(block_hash.0)
.finalize() .finalize()
.into(), .into(),

View File

@@ -9,7 +9,8 @@ use blake2::{Digest, Blake2s256};
use borsh::{BorshSerialize, BorshDeserialize}; use borsh::{BorshSerialize, BorshDeserialize};
use serai_abi::{ use serai_client_serai::{
abi::{
primitives::{ primitives::{
BlockHash, BlockHash,
crypto::{Public, KeyPair}, crypto::{Public, KeyPair},
@@ -18,13 +19,14 @@ use serai_abi::{
address::SeraiAddress, address::SeraiAddress,
}, },
Block, Block,
},
Serai, TemporalSerai,
}; };
use serai_client_serai::{Serai, TemporalSerai};
use serai_db::*; use serai_db::*;
use serai_task::*; use serai_task::*;
use serai_cosign_types::*; pub use serai_cosign_types::*;
/// The cosigns which are intended to be performed. /// The cosigns which are intended to be performed.
mod intend; mod intend;

View File

@@ -29,7 +29,7 @@ schnorrkel = { version = "0.11", default-features = false, features = ["std"] }
hex = { version = "0.4", default-features = false, features = ["std"] } hex = { version = "0.4", default-features = false, features = ["std"] }
borsh = { version = "1", default-features = false, features = ["std", "derive", "de_strict_order"] } borsh = { version = "1", default-features = false, features = ["std", "derive", "de_strict_order"] }
serai-client = { path = "../../../substrate/client", default-features = false, features = ["serai"] } serai-client-serai = { path = "../../../substrate/client/serai", default-features = false }
serai-cosign = { path = "../../cosign" } serai-cosign = { path = "../../cosign" }
tributary-sdk = { path = "../../tributary-sdk" } tributary-sdk = { path = "../../tributary-sdk" }

View File

@@ -7,7 +7,7 @@ use rand_core::{RngCore, OsRng};
use blake2::{Digest, Blake2s256}; use blake2::{Digest, Blake2s256};
use schnorrkel::{Keypair, PublicKey, Signature}; use schnorrkel::{Keypair, PublicKey, Signature};
use serai_client::primitives::PublicKey as Public; use serai_client_serai::abi::primitives::crypto::Public;
use futures_util::{AsyncRead, AsyncReadExt, AsyncWrite, AsyncWriteExt}; use futures_util::{AsyncRead, AsyncReadExt, AsyncWrite, AsyncWriteExt};
use libp2p::{ use libp2p::{
@@ -104,7 +104,7 @@ impl OnlyValidators {
.verify_simple(PROTOCOL.as_bytes(), &msg, &sig) .verify_simple(PROTOCOL.as_bytes(), &msg, &sig)
.map_err(|_| io::Error::other("invalid signature"))?; .map_err(|_| io::Error::other("invalid signature"))?;
Ok(peer_id_from_public(Public::from_raw(public_key.to_bytes()))) Ok(peer_id_from_public(Public(public_key.to_bytes())))
} }
} }

View File

@@ -1,11 +1,11 @@
use core::future::Future; use core::{future::Future, str::FromStr};
use std::{sync::Arc, collections::HashSet}; use std::{sync::Arc, collections::HashSet};
use rand_core::{RngCore, OsRng}; use rand_core::{RngCore, OsRng};
use tokio::sync::mpsc; use tokio::sync::mpsc;
use serai_client::{SeraiError, Serai}; use serai_client_serai::{RpcError, Serai};
use libp2p::{ use libp2p::{
core::multiaddr::{Protocol, Multiaddr}, core::multiaddr::{Protocol, Multiaddr},
@@ -50,7 +50,7 @@ impl ContinuallyRan for DialTask {
const DELAY_BETWEEN_ITERATIONS: u64 = 5 * 60; const DELAY_BETWEEN_ITERATIONS: u64 = 5 * 60;
const MAX_DELAY_BETWEEN_ITERATIONS: u64 = 10 * 60; const MAX_DELAY_BETWEEN_ITERATIONS: u64 = 10 * 60;
type Error = SeraiError; type Error = RpcError;
fn run_iteration(&mut self) -> impl Send + Future<Output = Result<bool, Self::Error>> { fn run_iteration(&mut self) -> impl Send + Future<Output = Result<bool, Self::Error>> {
async move { async move {
@@ -94,6 +94,13 @@ impl ContinuallyRan for DialTask {
usize::try_from(OsRng.next_u64() % u64::try_from(potential_peers.len()).unwrap()) usize::try_from(OsRng.next_u64() % u64::try_from(potential_peers.len()).unwrap())
.unwrap(); .unwrap();
let randomly_selected_peer = potential_peers.swap_remove(index_to_dial); let randomly_selected_peer = potential_peers.swap_remove(index_to_dial);
let Ok(randomly_selected_peer) = libp2p::Multiaddr::from_str(&randomly_selected_peer)
else {
log::error!(
"peer from substrate wasn't a valid `Multiaddr`: {randomly_selected_peer}"
);
continue;
};
log::info!("found peer from substrate: {randomly_selected_peer}"); log::info!("found peer from substrate: {randomly_selected_peer}");

View File

@@ -13,9 +13,10 @@ use rand_core::{RngCore, OsRng};
use zeroize::Zeroizing; use zeroize::Zeroizing;
use schnorrkel::Keypair; use schnorrkel::Keypair;
use serai_client::{ use serai_client_serai::{
primitives::{ExternalNetworkId, PublicKey}, abi::primitives::{
validator_sets::primitives::ExternalValidatorSet, crypto::Public, network_id::ExternalNetworkId, validator_sets::ExternalValidatorSet,
},
Serai, Serai,
}; };
@@ -66,7 +67,7 @@ use dial::DialTask;
const PORT: u16 = 30563; // 5132 ^ (('c' << 8) | 'o') const PORT: u16 = 30563; // 5132 ^ (('c' << 8) | 'o')
fn peer_id_from_public(public: PublicKey) -> PeerId { fn peer_id_from_public(public: Public) -> PeerId {
// 0 represents the identity Multihash, that no hash was performed // 0 represents the identity Multihash, that no hash was performed
// It's an internal constant so we can't refer to the constant inside libp2p // It's an internal constant so we can't refer to the constant inside libp2p
PeerId::from_multihash(Multihash::wrap(0, &public.0).unwrap()).unwrap() PeerId::from_multihash(Multihash::wrap(0, &public.0).unwrap()).unwrap()

View File

@@ -6,7 +6,7 @@ use std::{
use borsh::BorshDeserialize; use borsh::BorshDeserialize;
use serai_client::validator_sets::primitives::ExternalValidatorSet; use serai_client_serai::abi::primitives::validator_sets::ExternalValidatorSet;
use tokio::sync::{mpsc, oneshot, RwLock}; use tokio::sync::{mpsc, oneshot, RwLock};

View File

@@ -4,9 +4,8 @@ use std::{
collections::{HashSet, HashMap}, collections::{HashSet, HashMap},
}; };
use serai_client::{ use serai_client_serai::abi::primitives::{network_id::ExternalNetworkId, validator_sets::Session};
primitives::ExternalNetworkId, validator_sets::primitives::Session, SeraiError, Serai, use serai_client_serai::{RpcError, Serai};
};
use serai_task::{Task, ContinuallyRan}; use serai_task::{Task, ContinuallyRan};
@@ -52,7 +51,7 @@ impl Validators {
async fn session_changes( async fn session_changes(
serai: impl Borrow<Serai>, serai: impl Borrow<Serai>,
sessions: impl Borrow<HashMap<ExternalNetworkId, Session>>, sessions: impl Borrow<HashMap<ExternalNetworkId, Session>>,
) -> Result<Vec<(ExternalNetworkId, Session, HashSet<PeerId>)>, SeraiError> { ) -> Result<Vec<(ExternalNetworkId, Session, HashSet<PeerId>)>, RpcError> {
/* /*
This uses the latest finalized block, not the latest cosigned block, which should be fine as 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 in the worst case, we'd connect to unexpected validators. They still shouldn't be able to
@@ -69,10 +68,11 @@ impl Validators {
// FuturesUnordered can be bad practice as it'll cause timeouts if infrequently polled, but // 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 // we poll it till it yields all futures with the most minimal processing possible
let mut futures = FuturesUnordered::new(); let mut futures = FuturesUnordered::new();
for network in serai_client::primitives::EXTERNAL_NETWORKS { for network in ExternalNetworkId::all() {
let sessions = sessions.borrow(); let sessions = sessions.borrow();
let temporal_serai = temporal_serai.borrow();
futures.push(async move { futures.push(async move {
let session = match temporal_serai.session(network.into()).await { let session = match temporal_serai.current_session(network.into()).await {
Ok(Some(session)) => session, Ok(Some(session)) => session,
Ok(None) => return Ok(None), Ok(None) => return Ok(None),
Err(e) => return Err(e), Err(e) => return Err(e),
@@ -81,12 +81,16 @@ impl Validators {
if sessions.get(&network) == Some(&session) { if sessions.get(&network) == Some(&session) {
Ok(None) Ok(None)
} else { } else {
match temporal_serai.active_network_validators(network.into()).await { match temporal_serai.current_validators(network.into()).await {
Ok(validators) => Ok(Some(( Ok(Some(validators)) => Ok(Some((
network, network,
session, session,
validators.into_iter().map(peer_id_from_public).collect(), validators
.into_iter()
.map(|validator| peer_id_from_public(validator.into()))
.collect(),
))), ))),
Ok(None) => panic!("network has session yet no validators"),
Err(e) => Err(e), Err(e) => Err(e),
} }
} }
@@ -153,7 +157,7 @@ impl Validators {
} }
/// Update the view of the validators. /// Update the view of the validators.
pub(crate) async fn update(&mut self) -> Result<(), SeraiError> { pub(crate) async fn update(&mut self) -> Result<(), RpcError> {
let session_changes = Self::session_changes(&*self.serai, &self.sessions).await?; let session_changes = Self::session_changes(&*self.serai, &self.sessions).await?;
self.incorporate_session_changes(session_changes); self.incorporate_session_changes(session_changes);
Ok(()) Ok(())
@@ -206,7 +210,7 @@ impl ContinuallyRan for UpdateValidatorsTask {
const DELAY_BETWEEN_ITERATIONS: u64 = 60; const DELAY_BETWEEN_ITERATIONS: u64 = 60;
const MAX_DELAY_BETWEEN_ITERATIONS: u64 = 5 * 60; const MAX_DELAY_BETWEEN_ITERATIONS: u64 = 5 * 60;
type Error = SeraiError; type Error = RpcError;
fn run_iteration(&mut self) -> impl Send + Future<Output = Result<bool, Self::Error>> { fn run_iteration(&mut self) -> impl Send + Future<Output = Result<bool, Self::Error>> {
async move { async move {

View File

@@ -1,7 +1,7 @@
use core::future::Future; use core::future::Future;
use std::time::{Duration, SystemTime}; use std::time::{Duration, SystemTime};
use serai_primitives::{MAX_KEY_SHARES_PER_SET, ExternalValidatorSet}; use serai_primitives::validator_sets::{ExternalValidatorSet, KeyShares};
use futures_lite::FutureExt; use futures_lite::FutureExt;
@@ -30,7 +30,7 @@ pub const MIN_BLOCKS_PER_BATCH: usize = BLOCKS_PER_MINUTE + 1;
/// commit is `8 + (validators * 32) + (32 + (validators * 32))` (for the time, list of validators, /// commit is `8 + (validators * 32) + (32 + (validators * 32))` (for the time, list of validators,
/// and aggregate signature). Accordingly, this should be a safe over-estimate. /// and aggregate signature). Accordingly, this should be a safe over-estimate.
pub const BATCH_SIZE_LIMIT: usize = MIN_BLOCKS_PER_BATCH * pub const BATCH_SIZE_LIMIT: usize = MIN_BLOCKS_PER_BATCH *
(tributary_sdk::BLOCK_SIZE_LIMIT + 32 + ((MAX_KEY_SHARES_PER_SET as usize) * 128)); (tributary_sdk::BLOCK_SIZE_LIMIT + 32 + ((KeyShares::MAX_PER_SET as usize) * 128));
/// Sends a heartbeat to other validators on regular intervals informing them of our Tributary's /// Sends a heartbeat to other validators on regular intervals informing them of our Tributary's
/// tip. /// tip.

View File

@@ -5,9 +5,10 @@ use serai_db::{create_db, db_channel};
use dkg::Participant; use dkg::Participant;
use serai_client::{ use serai_client_serai::abi::primitives::{
primitives::ExternalNetworkId, crypto::KeyPair,
validator_sets::primitives::{Session, ExternalValidatorSet, KeyPair}, network_id::ExternalNetworkId,
validator_sets::{Session, ExternalValidatorSet},
}; };
use serai_cosign::SignedCosign; use serai_cosign::SignedCosign;
@@ -49,6 +50,7 @@ fn tributary_db_folder(set: ExternalValidatorSet) -> String {
ExternalNetworkId::Bitcoin => "Bitcoin", ExternalNetworkId::Bitcoin => "Bitcoin",
ExternalNetworkId::Ethereum => "Ethereum", ExternalNetworkId::Ethereum => "Ethereum",
ExternalNetworkId::Monero => "Monero", ExternalNetworkId::Monero => "Monero",
_ => panic!("unrecognized `ExternalNetworkId`"),
}; };
format!("{root_path}/tributary-{network}-{}", set.session.0) format!("{root_path}/tributary-{network}-{}", set.session.0)
} }
@@ -103,7 +105,7 @@ mod _internal_db {
// Tributary transactions to publish from the DKG confirmation task // Tributary transactions to publish from the DKG confirmation task
TributaryTransactionsFromDkgConfirmation: (set: ExternalValidatorSet) -> Transaction, TributaryTransactionsFromDkgConfirmation: (set: ExternalValidatorSet) -> Transaction,
// Participants to remove // Participants to remove
RemoveParticipant: (set: ExternalValidatorSet) -> Participant, RemoveParticipant: (set: ExternalValidatorSet) -> u16,
} }
} }
} }
@@ -139,10 +141,11 @@ impl RemoveParticipant {
pub(crate) fn send(txn: &mut impl DbTxn, set: ExternalValidatorSet, 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 this set has yet to be retired, send this transaction
if RetiredTributary::get(txn, set.network).map(|session| session.0) < Some(set.session.0) { if RetiredTributary::get(txn, set.network).map(|session| session.0) < Some(set.session.0) {
_internal_db::RemoveParticipant::send(txn, set, &participant); _internal_db::RemoveParticipant::send(txn, set, &u16::from(participant));
} }
} }
pub(crate) fn try_recv(txn: &mut impl DbTxn, set: ExternalValidatorSet) -> Option<Participant> { pub(crate) fn try_recv(txn: &mut impl DbTxn, set: ExternalValidatorSet) -> Option<Participant> {
_internal_db::RemoveParticipant::try_recv(txn, set) _internal_db::RemoveParticipant::try_recv(txn, set)
.map(|i| Participant::new(i).expect("sent invalid participant index for removal"))
} }
} }

View File

@@ -12,10 +12,8 @@ use frost_schnorrkel::{
use serai_db::{DbTxn, Db as DbTrait}; use serai_db::{DbTxn, Db as DbTrait};
use serai_client::{ #[rustfmt::skip]
primitives::SeraiAddress, use serai_client_serai::abi::primitives::{validator_sets::ExternalValidatorSet, address::SeraiAddress};
validator_sets::primitives::{ExternalValidatorSet, musig_context, set_keys_message},
};
use serai_task::{DoesNotError, ContinuallyRan}; use serai_task::{DoesNotError, ContinuallyRan};
@@ -160,7 +158,7 @@ impl<CD: DbTrait, TD: DbTrait> ConfirmDkgTask<CD, TD> {
let (machine, preprocess) = AlgorithmMachine::new( let (machine, preprocess) = AlgorithmMachine::new(
schnorrkel(), schnorrkel(),
// We use a 1-of-1 Musig here as we don't know who will actually be in this Musig yet // 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.into()), key, &[public_key]).unwrap(), musig(ExternalValidatorSet::musig_context(&set), key, &[public_key]).unwrap(),
) )
.preprocess(&mut OsRng); .preprocess(&mut OsRng);
// We take the preprocess so we can use it in a distinct machine with the actual Musig // We take the preprocess so we can use it in a distinct machine with the actual Musig
@@ -260,8 +258,11 @@ impl<CD: DbTrait, TD: DbTrait> ContinuallyRan for ConfirmDkgTask<CD, TD> {
}) })
.collect::<Vec<_>>(); .collect::<Vec<_>>();
let keys = let keys = musig(
musig(musig_context(self.set.set.into()), self.key.clone(), &musig_public_keys) ExternalValidatorSet::musig_context(&self.set.set),
self.key.clone(),
&musig_public_keys,
)
.unwrap(); .unwrap();
// Rebuild the machine // Rebuild the machine
@@ -296,9 +297,10 @@ impl<CD: DbTrait, TD: DbTrait> ContinuallyRan for ConfirmDkgTask<CD, TD> {
}; };
// Calculate our share // Calculate our share
let (machine, share) = match handle_frost_error( let (machine, share) = match handle_frost_error(machine.sign(
machine.sign(preprocesses, &set_keys_message(&self.set.set, &key_pair)), preprocesses,
) { &ExternalValidatorSet::set_keys_message(&self.set.set, &key_pair),
)) {
Ok((machine, share)) => (machine, share), Ok((machine, share)) => (machine, share),
// This yields the *musig participant index* // This yields the *musig participant index*
Err(participant) => { Err(participant) => {

View File

@@ -14,9 +14,14 @@ use borsh::BorshDeserialize;
use tokio::sync::mpsc; use tokio::sync::mpsc;
use serai_client::{ use serai_client_serai::{
primitives::{ExternalNetworkId, PublicKey, SeraiAddress, Signature}, abi::primitives::{
validator_sets::primitives::{ExternalValidatorSet, KeyPair}, BlockHash,
crypto::{Public, Signature, ExternalKey, KeyPair},
network_id::ExternalNetworkId,
validator_sets::ExternalValidatorSet,
address::SeraiAddress,
},
Serai, Serai,
}; };
use message_queue::{Service, client::MessageQueue}; use message_queue::{Service, client::MessageQueue};
@@ -61,9 +66,7 @@ async fn serai() -> Arc<Serai> {
let Ok(serai) = Serai::new(format!( let Ok(serai) = Serai::new(format!(
"http://{}:9944", "http://{}:9944",
serai_env::var("SERAI_HOSTNAME").expect("Serai hostname wasn't provided") serai_env::var("SERAI_HOSTNAME").expect("Serai hostname wasn't provided")
)) )) else {
.await
else {
log::error!("couldn't connect to the Serai node"); log::error!("couldn't connect to the Serai node");
tokio::time::sleep(delay).await; tokio::time::sleep(delay).await;
delay = (delay + SERAI_CONNECTION_DELAY).min(MAX_SERAI_CONNECTION_DELAY); delay = (delay + SERAI_CONNECTION_DELAY).min(MAX_SERAI_CONNECTION_DELAY);
@@ -213,11 +216,13 @@ async fn handle_network(
&mut txn, &mut txn,
ExternalValidatorSet { network, session }, ExternalValidatorSet { network, session },
&KeyPair( &KeyPair(
PublicKey::from_raw(substrate_key), Public(substrate_key),
ExternalKey(
network_key network_key
.try_into() .try_into()
.expect("generated a network key which exceeds the maximum key length"), .expect("generated a network key which exceeds the maximum key length"),
), ),
),
); );
} }
messages::key_gen::ProcessorMessage::Blame { session, participant } => { messages::key_gen::ProcessorMessage::Blame { session, participant } => {
@@ -284,12 +289,13 @@ async fn handle_network(
&mut txn, &mut txn,
ExternalValidatorSet { network, session }, ExternalValidatorSet { network, session },
slash_report, slash_report,
Signature::from(signature), Signature(signature),
); );
} }
}, },
messages::ProcessorMessage::Substrate(msg) => match msg { messages::ProcessorMessage::Substrate(msg) => match msg {
messages::substrate::ProcessorMessage::SubstrateBlockAck { block, plans } => { messages::substrate::ProcessorMessage::SubstrateBlockAck { block, plans } => {
let block = BlockHash(block);
let mut by_session = HashMap::new(); let mut by_session = HashMap::new();
for plan in plans { for plan in plans {
by_session by_session
@@ -481,7 +487,7 @@ async fn main() {
); );
// Handle each of the networks // Handle each of the networks
for network in serai_client::primitives::EXTERNAL_NETWORKS { for network in ExternalNetworkId::all() {
tokio::spawn(handle_network(db.clone(), message_queue.clone(), serai.clone(), network)); tokio::spawn(handle_network(db.clone(), message_queue.clone(), serai.clone(), network));
} }

View File

@@ -10,7 +10,10 @@ use tokio::sync::mpsc;
use serai_db::{DbTxn, Db as DbTrait}; use serai_db::{DbTxn, Db as DbTrait};
use serai_client::validator_sets::primitives::{Session, ExternalValidatorSet}; use serai_client_serai::abi::primitives::{
network_id::ExternalNetworkId,
validator_sets::{Session, ExternalValidatorSet},
};
use message_queue::{Service, Metadata, client::MessageQueue}; use message_queue::{Service, Metadata, client::MessageQueue};
use tributary_sdk::Tributary; use tributary_sdk::Tributary;
@@ -39,7 +42,7 @@ impl<P: P2p> ContinuallyRan for SubstrateTask<P> {
let mut made_progress = false; let mut made_progress = false;
// Handle the Canonical events // Handle the Canonical events
for network in serai_client::primitives::EXTERNAL_NETWORKS { for network in ExternalNetworkId::all() {
loop { loop {
let mut txn = self.db.txn(); let mut txn = self.db.txn();
let Some(msg) = serai_coordinator_substrate::Canonical::try_recv(&mut txn, network) let Some(msg) = serai_coordinator_substrate::Canonical::try_recv(&mut txn, network)

View File

@@ -11,7 +11,7 @@ use tokio::sync::mpsc;
use serai_db::{Get, DbTxn, Db as DbTrait, create_db, db_channel}; use serai_db::{Get, DbTxn, Db as DbTrait, create_db, db_channel};
use serai_client::validator_sets::primitives::ExternalValidatorSet; use serai_client_serai::abi::primitives::validator_sets::ExternalValidatorSet;
use tributary_sdk::{TransactionKind, TransactionError, ProvidedError, TransactionTrait, Tributary}; use tributary_sdk::{TransactionKind, TransactionError, ProvidedError, TransactionTrait, Tributary};

View File

@@ -24,7 +24,7 @@ borsh = { version = "1", default-features = false, features = ["std", "derive",
dkg = { path = "../../crypto/dkg", default-features = false, features = ["std"] } dkg = { path = "../../crypto/dkg", default-features = false, features = ["std"] }
serai-client = { path = "../../substrate/client", version = "0.1", default-features = false, features = ["serai"] } serai-client-serai = { path = "../../substrate/client/serai", default-features = false }
log = { version = "0.4", default-features = false, features = ["std"] } log = { version = "0.4", default-features = false, features = ["std"] }

View File

@@ -3,7 +3,13 @@ use std::sync::Arc;
use futures::stream::{StreamExt, FuturesOrdered}; use futures::stream::{StreamExt, FuturesOrdered};
use serai_client::{validator_sets::primitives::ExternalValidatorSet, Serai}; use serai_client_serai::{
abi::{
self,
primitives::{network_id::ExternalNetworkId, validator_sets::ExternalValidatorSet},
},
Serai,
};
use messages::substrate::{InInstructionResult, ExecutedBatch, CoordinatorMessage}; use messages::substrate::{InInstructionResult, ExecutedBatch, CoordinatorMessage};
@@ -15,6 +21,7 @@ use serai_cosign::Cosigning;
create_db!( create_db!(
CoordinatorSubstrateCanonical { CoordinatorSubstrateCanonical {
NextBlock: () -> u64, NextBlock: () -> u64,
LastIndexedBatchId: (network: ExternalNetworkId) -> u32,
} }
); );
@@ -45,10 +52,10 @@ impl<D: Db> ContinuallyRan for CanonicalEventStream<D> {
// These are all the events which generate canonical messages // These are all the events which generate canonical messages
struct CanonicalEvents { struct CanonicalEvents {
time: u64, time: u64,
key_gen_events: Vec<serai_client::validator_sets::ValidatorSetsEvent>, set_keys_events: Vec<abi::validator_sets::Event>,
set_retired_events: Vec<serai_client::validator_sets::ValidatorSetsEvent>, slash_report_events: Vec<abi::validator_sets::Event>,
batch_events: Vec<serai_client::in_instructions::InInstructionsEvent>, batch_events: Vec<abi::in_instructions::Event>,
burn_events: Vec<serai_client::coins::CoinsEvent>, burn_events: Vec<abi::coins::Event>,
} }
// For a cosigned block, fetch all relevant events // For a cosigned block, fetch all relevant events
@@ -66,16 +73,16 @@ impl<D: Db> ContinuallyRan for CanonicalEventStream<D> {
} }
Err(serai_cosign::Faulted) => return Err("cosigning process faulted".to_string()), Err(serai_cosign::Faulted) => return Err("cosigning process faulted".to_string()),
}; };
let temporal_serai = serai.as_of(block_hash); let temporal_serai = serai.as_of(block_hash).await.map_err(|e| format!("{e}"))?;
let temporal_serai_validators = temporal_serai.validator_sets(); let temporal_serai_validators = temporal_serai.validator_sets();
let temporal_serai_instructions = temporal_serai.in_instructions(); let temporal_serai_instructions = temporal_serai.in_instructions();
let temporal_serai_coins = temporal_serai.coins(); let temporal_serai_coins = temporal_serai.coins();
let (block, key_gen_events, set_retired_events, batch_events, burn_events) = let (block, set_keys_events, slash_report_events, batch_events, burn_events) =
tokio::try_join!( tokio::try_join!(
serai.block(block_hash), serai.block(block_hash),
temporal_serai_validators.key_gen_events(), temporal_serai_validators.set_keys_events(),
temporal_serai_validators.set_retired_events(), temporal_serai_validators.slash_report_events(),
temporal_serai_instructions.batch_events(), temporal_serai_instructions.batch_events(),
temporal_serai_coins.burn_with_instruction_events(), temporal_serai_coins.burn_with_instruction_events(),
) )
@@ -84,22 +91,14 @@ impl<D: Db> ContinuallyRan for CanonicalEventStream<D> {
Err(format!("Serai node didn't have cosigned block #{block_number}"))? Err(format!("Serai node didn't have cosigned block #{block_number}"))?
}; };
let time = if block_number == 0 { // We use time in seconds, not milliseconds, here
block.time().unwrap_or(0) let time = block.header.unix_time_in_millis() / 1000;
} else {
// Serai's block time is in milliseconds
block
.time()
.ok_or_else(|| "non-genesis Serai block didn't have a time".to_string())? /
1000
};
Ok(( Ok((
block_number, block_number,
CanonicalEvents { CanonicalEvents {
time, time,
key_gen_events, set_keys_events,
set_retired_events, slash_report_events,
batch_events, batch_events,
burn_events, burn_events,
}, },
@@ -131,10 +130,9 @@ impl<D: Db> ContinuallyRan for CanonicalEventStream<D> {
let mut txn = self.db.txn(); let mut txn = self.db.txn();
for key_gen in block.key_gen_events { for set_keys in block.set_keys_events {
let serai_client::validator_sets::ValidatorSetsEvent::KeyGen { set, key_pair } = &key_gen let abi::validator_sets::Event::SetKeys { set, key_pair } = &set_keys else {
else { panic!("`SetKeys` event wasn't a `SetKeys` event: {set_keys:?}");
panic!("KeyGen event wasn't a KeyGen event: {key_gen:?}");
}; };
crate::Canonical::send( crate::Canonical::send(
&mut txn, &mut txn,
@@ -147,12 +145,10 @@ impl<D: Db> ContinuallyRan for CanonicalEventStream<D> {
); );
} }
for set_retired in block.set_retired_events { for slash_report in block.slash_report_events {
let serai_client::validator_sets::ValidatorSetsEvent::SetRetired { set } = &set_retired let abi::validator_sets::Event::SlashReport { set } = &slash_report else {
else { panic!("`SlashReport` event wasn't a `SlashReport` event: {slash_report:?}");
panic!("SetRetired event wasn't a SetRetired event: {set_retired:?}");
}; };
let Ok(set) = ExternalValidatorSet::try_from(*set) else { continue };
crate::Canonical::send( crate::Canonical::send(
&mut txn, &mut txn,
set.network, set.network,
@@ -160,10 +156,12 @@ impl<D: Db> ContinuallyRan for CanonicalEventStream<D> {
); );
} }
for network in serai_client::primitives::EXTERNAL_NETWORKS { for network in ExternalNetworkId::all() {
let mut batch = None; let mut batch = None;
for this_batch in &block.batch_events { for this_batch in &block.batch_events {
let serai_client::in_instructions::InInstructionsEvent::Batch { // Only irrefutable as this is the only member of the enum at this time
#[expect(irrefutable_let_patterns)]
let abi::in_instructions::Event::Batch {
network: batch_network, network: batch_network,
publishing_session, publishing_session,
id, id,
@@ -194,14 +192,19 @@ impl<D: Db> ContinuallyRan for CanonicalEventStream<D> {
}) })
.collect(), .collect(),
}); });
if LastIndexedBatchId::get(&txn, network) != id.checked_sub(1) {
panic!(
"next batch from Serai's ID was not an increment of the last indexed batch's ID"
);
}
LastIndexedBatchId::set(&mut txn, network, id);
} }
} }
let mut burns = vec![]; let mut burns = vec![];
for burn in &block.burn_events { for burn in &block.burn_events {
let serai_client::coins::CoinsEvent::BurnWithInstruction { from: _, instruction } = let abi::coins::Event::BurnWithInstruction { from: _, instruction } = &burn else {
&burn
else {
panic!("BurnWithInstruction event wasn't a BurnWithInstruction event: {burn:?}"); panic!("BurnWithInstruction event wasn't a BurnWithInstruction event: {burn:?}");
}; };
if instruction.balance.coin.network() == network { if instruction.balance.coin.network() == network {
@@ -223,3 +226,7 @@ impl<D: Db> ContinuallyRan for CanonicalEventStream<D> {
} }
} }
} }
pub(crate) fn last_indexed_batch_id(txn: &impl DbTxn, network: ExternalNetworkId) -> Option<u32> {
LastIndexedBatchId::get(txn, network)
}

View File

@@ -3,9 +3,14 @@ use std::sync::Arc;
use futures::stream::{StreamExt, FuturesOrdered}; use futures::stream::{StreamExt, FuturesOrdered};
use serai_client::{ use serai_client_serai::{
primitives::{SeraiAddress, EmbeddedEllipticCurve}, abi::primitives::{
validator_sets::primitives::{MAX_KEY_SHARES_PER_SET, ExternalValidatorSet}, BlockHash,
crypto::EmbeddedEllipticCurveKeys,
network_id::ExternalNetworkId,
validator_sets::{KeyShares, ExternalValidatorSet},
address::SeraiAddress,
},
Serai, Serai,
}; };
@@ -49,10 +54,10 @@ impl<D: Db> ContinuallyRan for EphemeralEventStream<D> {
// These are all the events which generate canonical messages // These are all the events which generate canonical messages
struct EphemeralEvents { struct EphemeralEvents {
block_hash: [u8; 32], block_hash: BlockHash,
time: u64, time: u64,
new_set_events: Vec<serai_client::validator_sets::ValidatorSetsEvent>, set_decided_events: Vec<serai_client_serai::abi::validator_sets::Event>,
accepted_handover_events: Vec<serai_client::validator_sets::ValidatorSetsEvent>, accepted_handover_events: Vec<serai_client_serai::abi::validator_sets::Event>,
} }
// For a cosigned block, fetch all relevant events // For a cosigned block, fetch all relevant events
@@ -71,11 +76,11 @@ impl<D: Db> ContinuallyRan for EphemeralEventStream<D> {
Err(serai_cosign::Faulted) => return Err("cosigning process faulted".to_string()), Err(serai_cosign::Faulted) => return Err("cosigning process faulted".to_string()),
}; };
let temporal_serai = serai.as_of(block_hash); let temporal_serai = serai.as_of(block_hash).await.map_err(|e| format!("{e}"))?;
let temporal_serai_validators = temporal_serai.validator_sets(); let temporal_serai_validators = temporal_serai.validator_sets();
let (block, new_set_events, accepted_handover_events) = tokio::try_join!( let (block, set_decided_events, accepted_handover_events) = tokio::try_join!(
serai.block(block_hash), serai.block(block_hash),
temporal_serai_validators.new_set_events(), temporal_serai_validators.set_decided_events(),
temporal_serai_validators.accepted_handover_events(), temporal_serai_validators.accepted_handover_events(),
) )
.map_err(|e| format!("{e:?}"))?; .map_err(|e| format!("{e:?}"))?;
@@ -83,19 +88,11 @@ impl<D: Db> ContinuallyRan for EphemeralEventStream<D> {
Err(format!("Serai node didn't have cosigned block #{block_number}"))? Err(format!("Serai node didn't have cosigned block #{block_number}"))?
}; };
let time = if block_number == 0 { // We use time in seconds, not milliseconds, here
block.time().unwrap_or(0) let time = block.header.unix_time_in_millis() / 1000;
} else {
// Serai's block time is in milliseconds
block
.time()
.ok_or_else(|| "non-genesis Serai block didn't have a time".to_string())? /
1000
};
Ok(( Ok((
block_number, block_number,
EphemeralEvents { block_hash, time, new_set_events, accepted_handover_events }, EphemeralEvents { block_hash, time, set_decided_events, accepted_handover_events },
)) ))
} }
} }
@@ -126,48 +123,40 @@ impl<D: Db> ContinuallyRan for EphemeralEventStream<D> {
let mut txn = self.db.txn(); let mut txn = self.db.txn();
for new_set in block.new_set_events { for set_decided in block.set_decided_events {
let serai_client::validator_sets::ValidatorSetsEvent::NewSet { set } = &new_set else { let serai_client_serai::abi::validator_sets::Event::SetDecided { set, validators } =
panic!("NewSet event wasn't a NewSet event: {new_set:?}"); &set_decided
else {
panic!("`SetDecided` event wasn't a `SetDecided` event: {set_decided:?}");
}; };
// We only coordinate over external networks // We only coordinate over external networks
let Ok(set) = ExternalValidatorSet::try_from(*set) else { continue }; let Ok(set) = ExternalValidatorSet::try_from(*set) else { continue };
let serai = self.serai.as_of(block.block_hash); let serai = self.serai.as_of(block.block_hash).await.map_err(|e| format!("{e}"))?;
let serai = serai.validator_sets(); let serai = serai.validator_sets();
let Some(validators) = let validators =
serai.participants(set.network.into()).await.map_err(|e| format!("{e:?}"))? validators.iter().map(|(validator, weight)| (*validator, weight)).collect::<Vec<_>>();
else {
Err(format!(
"block #{block_number} declared a new set but didn't have the participants"
))?
};
let validators = validators
.into_iter()
.map(|(validator, weight)| (SeraiAddress::from(validator), weight))
.collect::<Vec<_>>();
let in_set = validators.iter().any(|(validator, _)| *validator == self.validator); let in_set = validators.iter().any(|(validator, _)| *validator == self.validator);
if in_set { if in_set {
if u16::try_from(validators.len()).is_err() { if u16::try_from(validators.len()).is_err() {
Err("more than u16::MAX validators sent")?; Err("more than u16::MAX validators sent")?;
} }
let Ok(validators) = validators let validators = validators
.into_iter() .into_iter()
.map(|(validator, weight)| u16::try_from(weight).map(|weight| (validator, weight))) .map(|(validator, weight)| (validator, weight.0))
.collect::<Result<Vec<_>, _>>() .collect::<Vec<_>>();
else {
Err("validator's weight exceeded u16::MAX".to_string())?
};
// Do the summation in u32 so we don't risk a u16 overflow // Do the summation in u32 so we don't risk a u16 overflow
let total_weight = validators.iter().map(|(_, weight)| u32::from(*weight)).sum::<u32>(); let total_weight = validators.iter().map(|(_, weight)| u32::from(*weight)).sum::<u32>();
if total_weight > u32::from(MAX_KEY_SHARES_PER_SET) { if total_weight > u32::from(KeyShares::MAX_PER_SET) {
Err(format!( Err(format!(
"{set:?} has {total_weight} key shares when the max is {MAX_KEY_SHARES_PER_SET}" "{set:?} has {total_weight} key shares when the max is {}",
KeyShares::MAX_PER_SET
))?; ))?;
} }
let total_weight = u16::try_from(total_weight).unwrap(); let total_weight = u16::try_from(total_weight)
.expect("value smaller than `u16` constant but doesn't fit in `u16`");
// Fetch all of the validators' embedded elliptic curve keys // Fetch all of the validators' embedded elliptic curve keys
let mut embedded_elliptic_curve_keys = FuturesOrdered::new(); let mut embedded_elliptic_curve_keys = FuturesOrdered::new();
@@ -175,52 +164,51 @@ impl<D: Db> ContinuallyRan for EphemeralEventStream<D> {
let validator = *validator; let validator = *validator;
// try_join doesn't return a future so we need to wrap it in this additional async // try_join doesn't return a future so we need to wrap it in this additional async
// block // block
embedded_elliptic_curve_keys.push_back(async move { embedded_elliptic_curve_keys.push_back({
tokio::try_join!( let serai = serai.clone();
// One future to fetch the substrate embedded key async move {
serai.embedded_elliptic_curve_key( match serai.embedded_elliptic_curve_keys(validator, set.network).await {
validator.into(), Ok(Some(keys)) => Ok(Some((
EmbeddedEllipticCurve::Embedwards25519 validator,
), match keys {
// One future to fetch the external embedded key, if there is a distinct curve EmbeddedEllipticCurveKeys::Bitcoin(substrate, external) => {
async { assert_eq!(set.network, ExternalNetworkId::Bitcoin);
// `embedded_elliptic_curves` is documented to have the second entry be the (substrate, external.as_slice().to_vec())
// network-specific curve (if it exists and is distinct from Embedwards25519) }
if let Some(curve) = set.network.embedded_elliptic_curves().get(1) { EmbeddedEllipticCurveKeys::Ethereum(substrate, external) => {
serai.embedded_elliptic_curve_key(validator.into(), *curve).await.map(Some) assert_eq!(set.network, ExternalNetworkId::Ethereum);
} else { (substrate, external.as_slice().to_vec())
Ok(None) }
EmbeddedEllipticCurveKeys::Monero(substrate) => {
assert_eq!(set.network, ExternalNetworkId::Monero);
(substrate, substrate.as_slice().to_vec())
}
},
))),
Ok(None) => Ok(None),
Err(e) => Err(e),
} }
} }
)
.map(|(substrate_embedded_key, external_embedded_key)| {
(validator, substrate_embedded_key, external_embedded_key)
})
}); });
} }
let mut evrf_public_keys = Vec::with_capacity(usize::from(total_weight)); let mut evrf_public_keys = Vec::with_capacity(usize::from(total_weight));
for (validator, weight) in &validators { for (validator, weight) in &validators {
let (future_validator, substrate_embedded_key, external_embedded_key) = let Some((future_validator, (substrate_embedded_key, external_embedded_key))) =
embedded_elliptic_curve_keys.next().await.unwrap().map_err(|e| format!("{e:?}"))?; embedded_elliptic_curve_keys.next().await.unwrap().map_err(|e| format!("{e:?}"))?
else {
Err("`SetDecided` with validator missing an embedded key".to_string())?
};
assert_eq!(*validator, future_validator); assert_eq!(*validator, future_validator);
let external_embedded_key =
external_embedded_key.unwrap_or(substrate_embedded_key.clone());
match (substrate_embedded_key, external_embedded_key) {
(Some(substrate_embedded_key), Some(external_embedded_key)) => {
let substrate_embedded_key = <[u8; 32]>::try_from(substrate_embedded_key)
.map_err(|_| "Embedwards25519 key wasn't 32 bytes".to_string())?;
for _ in 0 .. *weight { for _ in 0 .. *weight {
evrf_public_keys.push((substrate_embedded_key, external_embedded_key.clone())); evrf_public_keys.push((substrate_embedded_key, external_embedded_key.clone()));
} }
} }
_ => Err("NewSet with validator missing an embedded key".to_string())?,
}
}
let mut new_set = NewSetInformation { let mut new_set = NewSetInformation {
set, set,
serai_block: block.block_hash, serai_block: block.block_hash.0,
declaration_time: block.time, declaration_time: block.time,
// TODO: This should be inlined into the Processor's key gen code // TODO: This should be inlined into the Processor's key gen code
// It's legacy from when we removed participants from the key gen // It's legacy from when we removed participants from the key gen
@@ -238,7 +226,7 @@ impl<D: Db> ContinuallyRan for EphemeralEventStream<D> {
} }
for accepted_handover in block.accepted_handover_events { for accepted_handover in block.accepted_handover_events {
let serai_client::validator_sets::ValidatorSetsEvent::AcceptedHandover { set } = let serai_client_serai::abi::validator_sets::Event::AcceptedHandover { set } =
&accepted_handover &accepted_handover
else { else {
panic!("AcceptedHandover event wasn't a AcceptedHandover event: {accepted_handover:?}"); panic!("AcceptedHandover event wasn't a AcceptedHandover event: {accepted_handover:?}");

View File

@@ -8,10 +8,14 @@ use borsh::{BorshSerialize, BorshDeserialize};
use dkg::Participant; use dkg::Participant;
use serai_client::{ use serai_client_serai::abi::{
primitives::{ExternalNetworkId, SeraiAddress, Signature}, primitives::{
validator_sets::primitives::{Session, ExternalValidatorSet, KeyPair, SlashReport}, network_id::ExternalNetworkId,
in_instructions::primitives::SignedBatch, validator_sets::{Session, ExternalValidatorSet, SlashReport},
crypto::{Signature, KeyPair},
address::SeraiAddress,
instructions::SignedBatch,
},
Transaction, Transaction,
}; };
@@ -19,6 +23,7 @@ use serai_db::*;
mod canonical; mod canonical;
pub use canonical::CanonicalEventStream; pub use canonical::CanonicalEventStream;
use canonical::last_indexed_batch_id;
mod ephemeral; mod ephemeral;
pub use ephemeral::EphemeralEventStream; pub use ephemeral::EphemeralEventStream;
@@ -37,7 +42,7 @@ pub struct NewSetInformation {
pub set: ExternalValidatorSet, pub set: ExternalValidatorSet,
/// The Serai block which declared it. /// The Serai block which declared it.
pub serai_block: [u8; 32], pub serai_block: [u8; 32],
/// The time of the block which declared it, in seconds. /// The time of the block which declared it, in seconds since the epoch.
pub declaration_time: u64, pub declaration_time: u64,
/// The threshold to use. /// The threshold to use.
pub threshold: u16, pub threshold: u16,
@@ -96,9 +101,9 @@ mod _public_db {
create_db!( create_db!(
CoordinatorSubstrate { CoordinatorSubstrate {
// Keys to set on the Serai network // Keys to set on the Serai network
Keys: (network: ExternalNetworkId) -> (Session, Vec<u8>), Keys: (network: ExternalNetworkId) -> (Session, Transaction),
// Slash reports to publish onto the Serai network // Slash reports to publish onto the Serai network
SlashReports: (network: ExternalNetworkId) -> (Session, Vec<u8>), SlashReports: (network: ExternalNetworkId) -> (Session, Transaction),
} }
); );
} }
@@ -171,7 +176,7 @@ impl Keys {
} }
} }
let tx = serai_client::validator_sets::SeraiValidatorSets::set_keys( let tx = serai_client_serai::ValidatorSets::set_keys(
set.network, set.network,
key_pair, key_pair,
signature_participants, signature_participants,
@@ -192,7 +197,7 @@ pub struct SignedBatches;
impl SignedBatches { impl SignedBatches {
/// Send a `SignedBatch` to publish onto Serai. /// Send a `SignedBatch` to publish onto Serai.
pub fn send(txn: &mut impl DbTxn, batch: &SignedBatch) { pub fn send(txn: &mut impl DbTxn, batch: &SignedBatch) {
_public_db::SignedBatches::send(txn, batch.batch.network, batch); _public_db::SignedBatches::send(txn, batch.batch.network(), batch);
} }
pub(crate) fn try_recv(txn: &mut impl DbTxn, network: ExternalNetworkId) -> Option<SignedBatch> { pub(crate) fn try_recv(txn: &mut impl DbTxn, network: ExternalNetworkId) -> Option<SignedBatch> {
_public_db::SignedBatches::try_recv(txn, network) _public_db::SignedBatches::try_recv(txn, network)
@@ -219,11 +224,8 @@ impl SlashReports {
} }
} }
let tx = serai_client::validator_sets::SeraiValidatorSets::report_slashes( let tx =
set.network, serai_client_serai::ValidatorSets::report_slashes(set.network, slash_report, signature);
slash_report,
signature,
);
_public_db::SlashReports::set(txn, set.network, &(set.session, tx)); _public_db::SlashReports::set(txn, set.network, &(set.session, tx));
} }
pub(crate) fn take( pub(crate) fn take(

View File

@@ -1,8 +1,10 @@
use core::future::Future; use core::future::Future;
use std::sync::Arc; use std::sync::Arc;
#[rustfmt::skip] use serai_client_serai::{
use serai_client::{primitives::ExternalNetworkId, in_instructions::primitives::SignedBatch, SeraiError, Serai}; abi::primitives::{network_id::ExternalNetworkId, instructions::SignedBatch},
RpcError, Serai,
};
use serai_db::{Get, DbTxn, Db, create_db}; use serai_db::{Get, DbTxn, Db, create_db};
use serai_task::ContinuallyRan; use serai_task::ContinuallyRan;
@@ -31,7 +33,7 @@ impl<D: Db> PublishBatchTask<D> {
} }
impl<D: Db> ContinuallyRan for PublishBatchTask<D> { impl<D: Db> ContinuallyRan for PublishBatchTask<D> {
type Error = SeraiError; type Error = RpcError;
fn run_iteration(&mut self) -> impl Send + Future<Output = Result<bool, Self::Error>> { fn run_iteration(&mut self) -> impl Send + Future<Output = Result<bool, Self::Error>> {
async move { async move {
@@ -43,8 +45,8 @@ impl<D: Db> ContinuallyRan for PublishBatchTask<D> {
}; };
// If this is a Batch not yet published, save it into our unordered mapping // If this is a Batch not yet published, save it into our unordered mapping
if LastPublishedBatch::get(&txn, self.network) < Some(batch.batch.id) { if LastPublishedBatch::get(&txn, self.network) < Some(batch.batch.id()) {
BatchesToPublish::set(&mut txn, self.network, batch.batch.id, &batch); BatchesToPublish::set(&mut txn, self.network, batch.batch.id(), &batch);
} }
txn.commit(); txn.commit();
@@ -52,12 +54,8 @@ impl<D: Db> ContinuallyRan for PublishBatchTask<D> {
// Synchronize our last published batch with the Serai network's // Synchronize our last published batch with the Serai network's
let next_to_publish = { let next_to_publish = {
// This uses the latest finalized block, not the latest cosigned block, which should be
// fine as in the worst case, the only impact is no longer attempting TX publication
let serai = self.serai.as_of_latest_finalized_block().await?;
let last_batch = serai.in_instructions().last_batch_for_network(self.network).await?;
let mut txn = self.db.txn(); let mut txn = self.db.txn();
let last_batch = crate::last_indexed_batch_id(&txn, self.network);
let mut our_last_batch = LastPublishedBatch::get(&txn, self.network); let mut our_last_batch = LastPublishedBatch::get(&txn, self.network);
while our_last_batch < last_batch { while our_last_batch < last_batch {
let next_batch = our_last_batch.map(|batch| batch + 1).unwrap_or(0); let next_batch = our_last_batch.map(|batch| batch + 1).unwrap_or(0);
@@ -68,6 +66,7 @@ impl<D: Db> ContinuallyRan for PublishBatchTask<D> {
if let Some(last_batch) = our_last_batch { if let Some(last_batch) = our_last_batch {
LastPublishedBatch::set(&mut txn, self.network, &last_batch); LastPublishedBatch::set(&mut txn, self.network, &last_batch);
} }
txn.commit();
last_batch.map(|batch| batch + 1).unwrap_or(0) last_batch.map(|batch| batch + 1).unwrap_or(0)
}; };
@@ -75,7 +74,7 @@ impl<D: Db> ContinuallyRan for PublishBatchTask<D> {
if let Some(batch) = BatchesToPublish::get(&self.db, self.network, next_to_publish) { if let Some(batch) = BatchesToPublish::get(&self.db, self.network, next_to_publish) {
self self
.serai .serai
.publish(&serai_client::in_instructions::SeraiInInstructions::execute_batch(batch)) .publish_transaction(&serai_client_serai::InInstructions::execute_batch(batch))
.await?; .await?;
true true
} else { } else {

View File

@@ -3,7 +3,10 @@ use std::sync::Arc;
use serai_db::{DbTxn, Db}; use serai_db::{DbTxn, Db};
use serai_client::{primitives::ExternalNetworkId, validator_sets::primitives::Session, Serai}; use serai_client_serai::{
abi::primitives::{network_id::ExternalNetworkId, validator_sets::Session},
Serai,
};
use serai_task::ContinuallyRan; use serai_task::ContinuallyRan;
@@ -36,7 +39,8 @@ impl<D: Db> PublishSlashReportTask<D> {
let serai = self.serai.as_of_latest_finalized_block().await.map_err(|e| format!("{e:?}"))?; let serai = self.serai.as_of_latest_finalized_block().await.map_err(|e| format!("{e:?}"))?;
let serai = serai.validator_sets(); let serai = serai.validator_sets();
let session_after_slash_report = Session(session.0 + 1); let session_after_slash_report = Session(session.0 + 1);
let current_session = serai.session(network.into()).await.map_err(|e| format!("{e:?}"))?; let current_session =
serai.current_session(network.into()).await.map_err(|e| format!("{e:?}"))?;
let current_session = current_session.map(|session| session.0); 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 // Only attempt to publish the slash report for session #n while session #n+1 is still
// active // active
@@ -55,14 +59,13 @@ impl<D: Db> PublishSlashReportTask<D> {
} }
// If this session which should publish a slash report already has, move on // If this session which should publish a slash report already has, move on
let key_pending_slash_report = if !serai.pending_slash_report(network).await.map_err(|e| format!("{e:?}"))? {
serai.key_pending_slash_report(network).await.map_err(|e| format!("{e:?}"))?;
if key_pending_slash_report.is_none() {
txn.commit(); txn.commit();
return Ok(false); return Ok(false);
}; };
match self.serai.publish(&slash_report).await { // Since this slash report is still pending, publish it
match self.serai.publish_transaction(&slash_report).await {
Ok(()) => { Ok(()) => {
txn.commit(); txn.commit();
Ok(true) Ok(true)
@@ -84,7 +87,7 @@ impl<D: Db> ContinuallyRan for PublishSlashReportTask<D> {
async move { async move {
let mut made_progress = false; let mut made_progress = false;
let mut error = None; let mut error = None;
for network in serai_client::primitives::EXTERNAL_NETWORKS { for network in ExternalNetworkId::all() {
let network_res = self.publish(network).await; let network_res = self.publish(network).await;
// We made progress if any network successfully published their slash report // We made progress if any network successfully published their slash report
made_progress |= network_res == Ok(true); made_progress |= network_res == Ok(true);

View File

@@ -3,7 +3,10 @@ use std::sync::Arc;
use serai_db::{DbTxn, Db}; use serai_db::{DbTxn, Db};
use serai_client::{validator_sets::primitives::ExternalValidatorSet, Serai}; use serai_client_serai::{
abi::primitives::{network_id::ExternalNetworkId, validator_sets::ExternalValidatorSet},
Serai,
};
use serai_task::ContinuallyRan; use serai_task::ContinuallyRan;
@@ -28,7 +31,7 @@ impl<D: Db> ContinuallyRan for SetKeysTask<D> {
fn run_iteration(&mut self) -> impl Send + Future<Output = Result<bool, Self::Error>> { fn run_iteration(&mut self) -> impl Send + Future<Output = Result<bool, Self::Error>> {
async move { async move {
let mut made_progress = false; let mut made_progress = false;
for network in serai_client::primitives::EXTERNAL_NETWORKS { for network in ExternalNetworkId::all() {
let mut txn = self.db.txn(); let mut txn = self.db.txn();
let Some((session, keys)) = Keys::take(&mut txn, network) else { let Some((session, keys)) = Keys::take(&mut txn, network) else {
// No keys to set // No keys to set
@@ -40,7 +43,8 @@ impl<D: Db> ContinuallyRan for SetKeysTask<D> {
let serai = let serai =
self.serai.as_of_latest_finalized_block().await.map_err(|e| format!("{e:?}"))?; self.serai.as_of_latest_finalized_block().await.map_err(|e| format!("{e:?}"))?;
let serai = serai.validator_sets(); let serai = serai.validator_sets();
let current_session = serai.session(network.into()).await.map_err(|e| format!("{e:?}"))?; let current_session =
serai.current_session(network.into()).await.map_err(|e| format!("{e:?}"))?;
let current_session = current_session.map(|session| session.0); let current_session = current_session.map(|session| session.0);
// Only attempt to set these keys if this isn't a retired session // Only attempt to set these keys if this isn't a retired session
if Some(session.0) < current_session { if Some(session.0) < current_session {
@@ -67,7 +71,7 @@ impl<D: Db> ContinuallyRan for SetKeysTask<D> {
continue; continue;
}; };
match self.serai.publish(&keys).await { match self.serai.publish_transaction(&keys).await {
Ok(()) => { Ok(()) => {
txn.commit(); txn.commit();
made_progress = true; made_progress = true;

View File

@@ -36,7 +36,7 @@ serai-task = { path = "../../common/task", version = "0.1" }
tributary-sdk = { path = "../tributary-sdk" } tributary-sdk = { path = "../tributary-sdk" }
serai-cosign = { path = "../cosign" } serai-cosign-types = { path = "../cosign/types" }
serai-coordinator-substrate = { path = "../substrate" } serai-coordinator-substrate = { path = "../substrate" }
messages = { package = "serai-processor-messages", path = "../../processor/messages" } messages = { package = "serai-processor-messages", path = "../../processor/messages" }

View File

@@ -1,16 +1,14 @@
#![expect(clippy::cast_possible_truncation)]
use std::collections::HashMap; use std::collections::HashMap;
use borsh::{BorshSerialize, BorshDeserialize}; use borsh::{BorshSerialize, BorshDeserialize};
use serai_primitives::{address::SeraiAddress, validator_sets::primitives::ExternalValidatorSet}; use serai_primitives::{BlockHash, validator_sets::ExternalValidatorSet, address::SeraiAddress};
use messages::sign::{VariantSignId, SignId}; use messages::sign::{VariantSignId, SignId};
use serai_db::*; use serai_db::*;
use serai_cosign::CosignIntent; use serai_cosign_types::CosignIntent;
use crate::transaction::SigningProtocolRound; use crate::transaction::SigningProtocolRound;
@@ -124,7 +122,7 @@ impl Topic {
Topic::DkgConfirmation { attempt, round: _ } => Some({ Topic::DkgConfirmation { attempt, round: _ } => Some({
let id = { let id = {
let mut id = [0; 32]; let mut id = [0; 32];
let encoded_set = borsh::to_vec(set).unwrap(); let encoded_set = borsh::to_vec(&set).unwrap();
id[.. encoded_set.len()].copy_from_slice(&encoded_set); id[.. encoded_set.len()].copy_from_slice(&encoded_set);
VariantSignId::Batch(id) VariantSignId::Batch(id)
}; };
@@ -234,18 +232,18 @@ create_db!(
SlashPoints: (set: ExternalValidatorSet, validator: SeraiAddress) -> u32, SlashPoints: (set: ExternalValidatorSet, validator: SeraiAddress) -> u32,
// The cosign intent for a Substrate block // The cosign intent for a Substrate block
CosignIntents: (set: ExternalValidatorSet, substrate_block_hash: [u8; 32]) -> CosignIntent, CosignIntents: (set: ExternalValidatorSet, substrate_block_hash: BlockHash) -> CosignIntent,
// The latest Substrate block to cosign. // The latest Substrate block to cosign.
LatestSubstrateBlockToCosign: (set: ExternalValidatorSet) -> [u8; 32], LatestSubstrateBlockToCosign: (set: ExternalValidatorSet) -> BlockHash,
// The hash of the block we're actively cosigning. // The hash of the block we're actively cosigning.
ActivelyCosigning: (set: ExternalValidatorSet) -> [u8; 32], ActivelyCosigning: (set: ExternalValidatorSet) -> BlockHash,
// If this block has already been cosigned. // If this block has already been cosigned.
Cosigned: (set: ExternalValidatorSet, substrate_block_hash: [u8; 32]) -> (), Cosigned: (set: ExternalValidatorSet, substrate_block_hash: BlockHash) -> (),
// The plans to recognize upon a `Transaction::SubstrateBlock` being included on-chain. // The plans to recognize upon a `Transaction::SubstrateBlock` being included on-chain.
SubstrateBlockPlans: ( SubstrateBlockPlans: (
set: ExternalValidatorSet, set: ExternalValidatorSet,
substrate_block_hash: [u8; 32] substrate_block_hash: BlockHash
) -> Vec<[u8; 32]>, ) -> Vec<[u8; 32]>,
// The weight accumulated for a topic. // The weight accumulated for a topic.
@@ -293,26 +291,26 @@ impl TributaryDb {
pub(crate) fn latest_substrate_block_to_cosign( pub(crate) fn latest_substrate_block_to_cosign(
getter: &impl Get, getter: &impl Get,
set: ExternalValidatorSet, set: ExternalValidatorSet,
) -> Option<[u8; 32]> { ) -> Option<BlockHash> {
LatestSubstrateBlockToCosign::get(getter, set) LatestSubstrateBlockToCosign::get(getter, set)
} }
pub(crate) fn set_latest_substrate_block_to_cosign( pub(crate) fn set_latest_substrate_block_to_cosign(
txn: &mut impl DbTxn, txn: &mut impl DbTxn,
set: ExternalValidatorSet, set: ExternalValidatorSet,
substrate_block_hash: [u8; 32], substrate_block_hash: BlockHash,
) { ) {
LatestSubstrateBlockToCosign::set(txn, set, &substrate_block_hash); LatestSubstrateBlockToCosign::set(txn, set, &substrate_block_hash);
} }
pub(crate) fn actively_cosigning( pub(crate) fn actively_cosigning(
txn: &mut impl DbTxn, txn: &mut impl DbTxn,
set: ExternalValidatorSet, set: ExternalValidatorSet,
) -> Option<[u8; 32]> { ) -> Option<BlockHash> {
ActivelyCosigning::get(txn, set) ActivelyCosigning::get(txn, set)
} }
pub(crate) fn start_cosigning( pub(crate) fn start_cosigning(
txn: &mut impl DbTxn, txn: &mut impl DbTxn,
set: ExternalValidatorSet, set: ExternalValidatorSet,
substrate_block_hash: [u8; 32], substrate_block_hash: BlockHash,
substrate_block_number: u64, substrate_block_number: u64,
) { ) {
assert!( assert!(
@@ -337,14 +335,14 @@ impl TributaryDb {
pub(crate) fn mark_cosigned( pub(crate) fn mark_cosigned(
txn: &mut impl DbTxn, txn: &mut impl DbTxn,
set: ExternalValidatorSet, set: ExternalValidatorSet,
substrate_block_hash: [u8; 32], substrate_block_hash: BlockHash,
) { ) {
Cosigned::set(txn, set, substrate_block_hash, &()); Cosigned::set(txn, set, substrate_block_hash, &());
} }
pub(crate) fn cosigned( pub(crate) fn cosigned(
txn: &mut impl DbTxn, txn: &mut impl DbTxn,
set: ExternalValidatorSet, set: ExternalValidatorSet,
substrate_block_hash: [u8; 32], substrate_block_hash: BlockHash,
) -> bool { ) -> bool {
Cosigned::get(txn, set, substrate_block_hash).is_some() Cosigned::get(txn, set, substrate_block_hash).is_some()
} }

View File

@@ -9,8 +9,9 @@ use ciphersuite::group::GroupEncoding;
use dkg::Participant; use dkg::Participant;
use serai_primitives::{ use serai_primitives::{
address::SeraiAddress, BlockHash,
validator_sets::{ExternalValidatorSet, Slash}, validator_sets::{ExternalValidatorSet, Slash},
address::SeraiAddress,
}; };
use serai_db::*; use serai_db::*;
@@ -25,7 +26,7 @@ use tributary_sdk::{
Transaction as TributaryTransaction, Block, TributaryReader, P2p, Transaction as TributaryTransaction, Block, TributaryReader, P2p,
}; };
use serai_cosign::CosignIntent; use serai_cosign_types::CosignIntent;
use serai_coordinator_substrate::NewSetInformation; use serai_coordinator_substrate::NewSetInformation;
use messages::sign::{VariantSignId, SignId}; use messages::sign::{VariantSignId, SignId};
@@ -79,7 +80,7 @@ impl CosignIntents {
fn take( fn take(
txn: &mut impl DbTxn, txn: &mut impl DbTxn,
set: ExternalValidatorSet, set: ExternalValidatorSet,
substrate_block_hash: [u8; 32], substrate_block_hash: BlockHash,
) -> Option<CosignIntent> { ) -> Option<CosignIntent> {
db::CosignIntents::take(txn, set, substrate_block_hash) db::CosignIntents::take(txn, set, substrate_block_hash)
} }
@@ -113,7 +114,7 @@ impl SubstrateBlockPlans {
pub fn set( pub fn set(
txn: &mut impl DbTxn, txn: &mut impl DbTxn,
set: ExternalValidatorSet, set: ExternalValidatorSet,
substrate_block_hash: [u8; 32], substrate_block_hash: BlockHash,
plans: &Vec<[u8; 32]>, plans: &Vec<[u8; 32]>,
) { ) {
db::SubstrateBlockPlans::set(txn, set, substrate_block_hash, plans); db::SubstrateBlockPlans::set(txn, set, substrate_block_hash, plans);
@@ -121,7 +122,7 @@ impl SubstrateBlockPlans {
fn take( fn take(
txn: &mut impl DbTxn, txn: &mut impl DbTxn,
set: ExternalValidatorSet, set: ExternalValidatorSet,
substrate_block_hash: [u8; 32], substrate_block_hash: BlockHash,
) -> Option<Vec<[u8; 32]>> { ) -> Option<Vec<[u8; 32]>> {
db::SubstrateBlockPlans::take(txn, set, substrate_block_hash) db::SubstrateBlockPlans::take(txn, set, substrate_block_hash)
} }

View File

@@ -14,7 +14,7 @@ use schnorr::SchnorrSignature;
use borsh::{BorshSerialize, BorshDeserialize}; use borsh::{BorshSerialize, BorshDeserialize};
use serai_primitives::{addess::SeraiAddress, validator_sets::MAX_KEY_SHARES_PER_SET}; use serai_primitives::{BlockHash, validator_sets::KeyShares, address::SeraiAddress};
use messages::sign::VariantSignId; use messages::sign::VariantSignId;
@@ -137,7 +137,7 @@ pub enum Transaction {
/// be the one selected to be cosigned. /// be the one selected to be cosigned.
Cosign { Cosign {
/// The hash of the Substrate block to cosign /// The hash of the Substrate block to cosign
substrate_block_hash: [u8; 32], substrate_block_hash: BlockHash,
}, },
/// Note an intended-to-be-cosigned Substrate block as cosigned /// Note an intended-to-be-cosigned Substrate block as cosigned
@@ -175,7 +175,7 @@ pub enum Transaction {
/// cosigning the block in question, it'd be safe to provide this and move on to the next cosign. /// cosigning the block in question, it'd be safe to provide this and move on to the next cosign.
Cosigned { Cosigned {
/// The hash of the Substrate block which was cosigned /// The hash of the Substrate block which was cosigned
substrate_block_hash: [u8; 32], substrate_block_hash: BlockHash,
}, },
/// Acknowledge a Substrate block /// Acknowledge a Substrate block
@@ -186,7 +186,7 @@ pub enum Transaction {
/// resulting from its handling. /// resulting from its handling.
SubstrateBlock { SubstrateBlock {
/// The hash of the Substrate block /// The hash of the Substrate block
hash: [u8; 32], hash: BlockHash,
}, },
/// Acknowledge a Batch /// Acknowledge a Batch
@@ -250,11 +250,11 @@ impl TransactionTrait for Transaction {
signed.to_tributary_signed(0), signed.to_tributary_signed(0),
), ),
Transaction::DkgConfirmationPreprocess { attempt, signed, .. } => TransactionKind::Signed( Transaction::DkgConfirmationPreprocess { attempt, signed, .. } => TransactionKind::Signed(
borsh::to_vec(b"DkgConfirmation".as_slice(), attempt).unwrap(), borsh::to_vec(&(b"DkgConfirmation".as_slice(), attempt)).unwrap(),
signed.to_tributary_signed(0), signed.to_tributary_signed(0),
), ),
Transaction::DkgConfirmationShare { attempt, signed, .. } => TransactionKind::Signed( Transaction::DkgConfirmationShare { attempt, signed, .. } => TransactionKind::Signed(
borsh::to_vec(b"DkgConfirmation".as_slice(), attempt).unwrap(), borsh::to_vec(&(b"DkgConfirmation".as_slice(), attempt)).unwrap(),
signed.to_tributary_signed(1), signed.to_tributary_signed(1),
), ),
@@ -264,7 +264,7 @@ impl TransactionTrait for Transaction {
Transaction::Batch { .. } => TransactionKind::Provided("Batch"), Transaction::Batch { .. } => TransactionKind::Provided("Batch"),
Transaction::Sign { id, attempt, round, signed, .. } => TransactionKind::Signed( Transaction::Sign { id, attempt, round, signed, .. } => TransactionKind::Signed(
borsh::to_vec(b"Sign".as_slice(), id, attempt).unwrap(), borsh::to_vec(&(b"Sign".as_slice(), id, attempt)).unwrap(),
signed.to_tributary_signed(round.nonce()), signed.to_tributary_signed(round.nonce()),
), ),
@@ -303,14 +303,14 @@ impl TransactionTrait for Transaction {
Transaction::Batch { .. } => {} Transaction::Batch { .. } => {}
Transaction::Sign { data, .. } => { Transaction::Sign { data, .. } => {
if data.len() > usize::from(MAX_KEY_SHARES_PER_SET) { if data.len() > usize::from(KeyShares::MAX_PER_SET) {
Err(TransactionError::InvalidContent)? Err(TransactionError::InvalidContent)?
} }
// TODO: MAX_SIGN_LEN // TODO: MAX_SIGN_LEN
} }
Transaction::SlashReport { slash_points, .. } => { Transaction::SlashReport { slash_points, .. } => {
if slash_points.len() > usize::from(MAX_KEY_SHARES_PER_SET) { if slash_points.len() > usize::from(KeyShares::MAX_PER_SET) {
Err(TransactionError::InvalidContent)? Err(TransactionError::InvalidContent)?
} }
} }

View File

@@ -83,6 +83,12 @@ impl Header {
Header::V1(HeaderV1 { builds_upon, .. }) => *builds_upon, Header::V1(HeaderV1 { builds_upon, .. }) => *builds_upon,
} }
} }
/// Get the UNIX time, in milliseconds since the epoch, for when this block was proposed.
pub fn unix_time_in_millis(&self) -> u64 {
match self {
Header::V1(HeaderV1 { unix_time_in_millis, .. }) => *unix_time_in_millis,
}
}
/// The commitment to the transactions within this block. /// The commitment to the transactions within this block.
pub fn transactions_commitment(&self) -> UnbalancedMerkleTree { pub fn transactions_commitment(&self) -> UnbalancedMerkleTree {
match self { match self {

View File

@@ -24,7 +24,9 @@ simple-request = { path = "../../../common/request", version = "0.3" }
hex = { version = "0.4", default-features = false, features = ["alloc"] } hex = { version = "0.4", default-features = false, features = ["alloc"] }
borsh = { version = "1", default-features = false, features = ["std"] } borsh = { version = "1", default-features = false, features = ["std"] }
serai-abi = { path = "../../abi", version = "0.1" }
bitvec = { version = "1", default-features = false, features = ["alloc", "std"] }
serai-abi = { path = "../../abi", version = "0.1", default-features = false, features = ["std"] }
async-lock = "3" async-lock = "3"

View File

@@ -0,0 +1,52 @@
pub use serai_abi::{
primitives::instructions::SignedBatch,
in_instructions::{Call, Event},
UnsignedCall, Transaction,
};
use crate::{RpcError, TemporalSerai};
/// A `TemporalSerai` scoped to the in instructions module.
#[derive(Clone)]
pub struct InInstructions<'serai>(pub(super) &'serai TemporalSerai<'serai>);
impl<'serai> InInstructions<'serai> {
/// The events from the in instructions module.
pub async fn events(&self) -> Result<Vec<Event>, RpcError> {
Ok(
self
.0
.events_borrowed()
.await?
.as_ref()
.expect("`TemporalSerai::events` returned None")
.iter()
.flat_map(IntoIterator::into_iter)
.filter_map(|event| match event {
serai_abi::Event::InInstructions(event) => Some(event.clone()),
_ => None,
})
.collect(),
)
}
/// The `Batch` events from the in instructions module.
pub async fn batch_events(&self) -> Result<Vec<Event>, RpcError> {
Ok(
self
.events()
.await?
.into_iter()
.filter(|event| matches!(event, Event::Batch { .. }))
.collect(),
)
}
/// Create a transaction to execute a batch.
pub fn execute_batch(batch: SignedBatch) -> Transaction {
Transaction::Unsigned {
call: UnsignedCall::try_from(serai_abi::Call::from(Call::execute_batch { batch }))
.expect("`execute_batch` wasn't an unsigned call?"),
}
}
}

View File

@@ -25,6 +25,9 @@ pub use coins::Coins;
mod validator_sets; mod validator_sets;
pub use validator_sets::ValidatorSets; pub use validator_sets::ValidatorSets;
mod in_instructions;
pub use in_instructions::InInstructions;
/// An error from the RPC. /// An error from the RPC.
#[derive(Debug, Error)] #[derive(Debug, Error)]
pub enum RpcError { pub enum RpcError {
@@ -58,8 +61,8 @@ pub struct Serai {
/// from this block will be cached within this. This allows future calls for events to be done /// from this block will be cached within this. This allows future calls for events to be done
/// cheaply. /// cheaply.
#[derive(Clone)] #[derive(Clone)]
pub struct TemporalSerai<'a> { pub struct TemporalSerai<'serai> {
serai: &'a Serai, serai: &'serai Serai,
block: BlockHash, block: BlockHash,
events: Arc<RwLock<Option<Vec<Vec<Event>>>>>, events: Arc<RwLock<Option<Vec<Vec<Event>>>>>,
} }
@@ -176,7 +179,7 @@ impl Serai {
/// ///
/// This will yield an error if the block chosen isn't finalized. This ensures, given an honest /// This will yield an error if the block chosen isn't finalized. This ensures, given an honest
/// node, that this scope will be available for the lifetime of this object. /// node, that this scope will be available for the lifetime of this object.
pub async fn as_of<'a>(&'a self, block: BlockHash) -> Result<TemporalSerai<'a>, RpcError> { pub async fn as_of(&self, block: BlockHash) -> Result<TemporalSerai<'_>, RpcError> {
if !self.finalized(block).await? { if !self.finalized(block).await? {
Err(RpcError::NotFinalized)?; Err(RpcError::NotFinalized)?;
} }
@@ -184,10 +187,7 @@ impl Serai {
} }
/// Scope this RPC client to the state as of the latest finalized block. /// Scope this RPC client to the state as of the latest finalized block.
pub async fn as_of_latest_finalized_block<'a>( pub async fn as_of_latest_finalized_block(&self) -> Result<TemporalSerai<'_>, RpcError> {
&'a self,
block: BlockHash,
) -> Result<TemporalSerai<'a>, RpcError> {
let block = self let block = self
.block_by_number(self.latest_finalized_block_number().await?) .block_by_number(self.latest_finalized_block_number().await?)
.await? .await?
@@ -215,7 +215,7 @@ impl Serai {
} }
} }
impl<'a> TemporalSerai<'a> { impl<'serai> TemporalSerai<'serai> {
async fn call<ResponseValue: Default + JsonDeserialize>( async fn call<ResponseValue: Default + JsonDeserialize>(
&self, &self,
method: &str, method: &str,
@@ -282,4 +282,9 @@ impl<'a> TemporalSerai<'a> {
pub fn validator_sets(&self) -> ValidatorSets<'_> { pub fn validator_sets(&self) -> ValidatorSets<'_> {
ValidatorSets(self) ValidatorSets(self)
} }
/// Scope to the in instructions module.
pub fn in_instructions(&self) -> InInstructions<'_> {
InInstructions(self)
}
} }

View File

@@ -2,12 +2,14 @@ use borsh::BorshDeserialize;
pub use serai_abi::{ pub use serai_abi::{
primitives::{ primitives::{
crypto::KeyPair, crypto::{Signature, KeyPair, EmbeddedEllipticCurveKeys},
network_id::{ExternalNetworkId, NetworkId}, network_id::{ExternalNetworkId, NetworkId},
validator_sets::{Session, ExternalValidatorSet, ValidatorSet}, validator_sets::{Session, ExternalValidatorSet, ValidatorSet, SlashReport},
balance::Amount, balance::Amount,
address::SeraiAddress,
}, },
validator_sets::Event, validator_sets::{Call, Event},
UnsignedCall, Transaction,
}; };
use crate::{RpcError, TemporalSerai}; use crate::{RpcError, TemporalSerai};
@@ -24,9 +26,9 @@ fn rpc_network(network: impl Into<NetworkId>) -> Result<&'static str, RpcError>
/// A `TemporalSerai` scoped to the validator sets module. /// A `TemporalSerai` scoped to the validator sets module.
#[derive(Clone)] #[derive(Clone)]
pub struct ValidatorSets<'a>(pub(super) &'a TemporalSerai<'a>); pub struct ValidatorSets<'serai>(pub(super) &'serai TemporalSerai<'serai>);
impl<'a> ValidatorSets<'a> { impl<'serai> ValidatorSets<'serai> {
/// The events from the validator sets module. /// The events from the validator sets module.
pub async fn events(&self) -> Result<Vec<Event>, RpcError> { pub async fn events(&self) -> Result<Vec<Event>, RpcError> {
Ok( Ok(
@@ -108,7 +110,7 @@ impl<'a> ValidatorSets<'a> {
) )
} }
/// The stake for the current validators for specified network. /// The stake for the current validators for the specified network.
pub async fn current_stake(&self, network: NetworkId) -> Result<Option<Amount>, RpcError> { pub async fn current_stake(&self, network: NetworkId) -> Result<Option<Amount>, RpcError> {
Ok( Ok(
self self
@@ -142,4 +144,38 @@ impl<'a> ValidatorSets<'a> {
.map(Some) .map(Some)
.map_err(|_| RpcError::InvalidNode("validator set's keys weren't a valid key pair".to_string())) .map_err(|_| RpcError::InvalidNode("validator set's keys weren't a valid key pair".to_string()))
} }
/// Create a transaction to set a validator set's keys.
pub fn set_keys(
network: ExternalNetworkId,
key_pair: KeyPair,
signature_participants: bitvec::vec::BitVec<u8, bitvec::order::Lsb0>,
signature: Signature,
) -> Transaction {
Transaction::Unsigned {
call: UnsignedCall::try_from(serai_abi::Call::from(Call::set_keys {
network,
key_pair,
signature_participants,
signature,
}))
.expect("`set_keys` wasn't an unsigned call?"),
}
}
/// Create a transaction to report the slashes for a validator set.
pub fn report_slashes(
network: ExternalNetworkId,
slashes: SlashReport,
signature: Signature,
) -> Transaction {
Transaction::Unsigned {
call: UnsignedCall::try_from(serai_abi::Call::from(Call::report_slashes {
network,
slashes,
signature,
}))
.expect("`report_slashes` wasn't an unsigned call?"),
}
}
} }

View File

@@ -48,7 +48,10 @@ pub(crate) fn module<
Err(e) => Err(e)?, Err(e) => Err(e)?,
}; };
// Always return the protocol's bootnodes // Always return the protocol's bootnodes
let mut all_p2p_addresses = crate::chain_spec::bootnode_multiaddrs(id); let mut all_p2p_addresses = crate::chain_spec::bootnode_multiaddrs(id)
.iter()
.map(ToString::to_string)
.collect::<Vec<_>>();
// Additionally returns validators found over the DHT // Additionally returns validators found over the DHT
for validator in validators { for validator in validators {
let mut returned_addresses = authority_discovery let mut returned_addresses = authority_discovery
@@ -66,9 +69,11 @@ pub(crate) fn module<
// It isn't beneficial to use multiple addresses for a single peer here // It isn't beneficial to use multiple addresses for a single peer here
if !returned_addresses.is_empty() { if !returned_addresses.is_empty() {
all_p2p_addresses.push( all_p2p_addresses.push(
libp2p::Multiaddr::from(
returned_addresses returned_addresses
.remove(usize::try_from(OsRng.next_u64() >> 32).unwrap() % returned_addresses.len()) .remove(usize::try_from(OsRng.next_u64() >> 32).unwrap() % returned_addresses.len()),
.into(), )
.to_string(),
); );
} }
} }

View File

@@ -217,6 +217,7 @@ pub struct SlashReport(
); );
/// An error when converting from a `Vec`. /// An error when converting from a `Vec`.
#[derive(Debug)]
pub enum FromVecError { pub enum FromVecError {
/// The source `Vec` was too long to be converted. /// The source `Vec` was too long to be converted.
TooLong, TooLong,