1 Commits

Author SHA1 Message Date
Luke Parker
ce3b90541e Make transactions undroppable
coordinator/cosign/src/delay.rs literally demonstrates how we'd need to rewrite
our handling of transactions with this change. It can be cleaned up a bit but
already identifies ergonomic issues. It also doesn't model passing an &mut txn
to an async function, which would also require using the droppable wrapper
struct.

To locally see this build, run

RUSTFLAGS="-Zpanic_abort_tests -C panic=abort" cargo +nightly build -p serai-cosign --all-targets

To locally see this fail to build, run

cargo build -p serai-cosign --all-targets

While it doesn't say which line causes it fail to build, the only distinction
is panic=unwind.

For more context, please see #578.
2025-01-15 03:56:59 -05:00
65 changed files with 3368 additions and 1908 deletions

View File

@@ -1 +1 @@
nightly-2025-01-01
nightly-2024-07-01

2899
Cargo.lock generated

File diff suppressed because it is too large Load Diff

View File

@@ -30,13 +30,53 @@ pub trait Get {
/// is undefined. The transaction may block, deadlock, panic, overwrite one of the two values
/// randomly, or any other action, at time of write or at time of commit.
#[must_use]
pub trait DbTxn: Send + Get {
pub trait DbTxn: Sized + Send + Get {
/// Write a value to this key.
fn put(&mut self, key: impl AsRef<[u8]>, value: impl AsRef<[u8]>);
/// Delete the value from this key.
fn del(&mut self, key: impl AsRef<[u8]>);
/// Commit this transaction.
fn commit(self);
/// Close this transaction.
///
/// This is equivalent to `Drop` on transactions which can be dropped. This is explicit and works
/// with transactions which can't be dropped.
fn close(self) {
drop(self);
}
}
// Credit for the idea goes to https://jack.wrenn.fyi/blog/undroppable
pub struct Undroppable<T>(Option<T>);
impl<T> Drop for Undroppable<T> {
fn drop(&mut self) {
// Use an assertion at compile time to prevent this code from compiling if generated
#[allow(clippy::assertions_on_constants)]
const {
assert!(false, "Undroppable DbTxn was dropped. Ensure all code paths call commit or close");
}
}
}
impl<T: DbTxn> Get for Undroppable<T> {
fn get(&self, key: impl AsRef<[u8]>) -> Option<Vec<u8>> {
self.0.as_ref().unwrap().get(key)
}
}
impl<T: DbTxn> DbTxn for Undroppable<T> {
fn put(&mut self, key: impl AsRef<[u8]>, value: impl AsRef<[u8]>) {
self.0.as_mut().unwrap().put(key, value);
}
fn del(&mut self, key: impl AsRef<[u8]>) {
self.0.as_mut().unwrap().del(key);
}
fn commit(mut self) {
self.0.take().unwrap().commit();
let _ = core::mem::ManuallyDrop::new(self);
}
fn close(mut self) {
drop(self.0.take().unwrap());
let _ = core::mem::ManuallyDrop::new(self);
}
}
/// A database supporting atomic transaction.
@@ -51,6 +91,10 @@ pub trait Db: 'static + Send + Sync + Clone + Get {
let dst_len = u8::try_from(item_dst.len()).unwrap();
[[db_len].as_ref(), db_dst, [dst_len].as_ref(), item_dst, key.as_ref()].concat()
}
/// Open a new transaction.
fn txn(&mut self) -> Self::Transaction<'_>;
/// Open a new transaction which may be dropped.
fn unsafe_txn(&mut self) -> Self::Transaction<'_>;
/// Open a new transaction which must be committed or closed.
fn txn(&mut self) -> Undroppable<Self::Transaction<'_>> {
Undroppable(Some(self.unsafe_txn()))
}
}

View File

@@ -74,7 +74,7 @@ impl Get for MemDb {
}
impl Db for MemDb {
type Transaction<'a> = MemDbTxn<'a>;
fn txn(&mut self) -> MemDbTxn<'_> {
fn unsafe_txn(&mut self) -> MemDbTxn<'_> {
MemDbTxn(self, HashMap::new(), HashSet::new())
}
}

View File

@@ -37,7 +37,7 @@ impl Get for Arc<ParityDb> {
}
impl Db for Arc<ParityDb> {
type Transaction<'a> = Transaction<'a>;
fn txn(&mut self) -> Self::Transaction<'_> {
fn unsafe_txn(&mut self) -> Self::Transaction<'_> {
Transaction(self, vec![])
}
}

View File

@@ -39,7 +39,7 @@ impl<T: ThreadMode> Get for Arc<OptimisticTransactionDB<T>> {
}
impl<T: Send + ThreadMode + 'static> Db for Arc<OptimisticTransactionDB<T>> {
type Transaction<'a> = Transaction<'a, T>;
fn txn(&mut self) -> Self::Transaction<'_> {
fn unsafe_txn(&mut self) -> Self::Transaction<'_> {
let mut opts = WriteOptions::default();
opts.set_sync(true);
Transaction(self.transaction_opt(&opts, &Default::default()), &**self)

View File

@@ -11,7 +11,7 @@ use crate::{Client, Error};
#[allow(dead_code)]
#[derive(Debug)]
pub struct Response<'a>(pub(crate) hyper::Response<Incoming>, pub(crate) &'a Client);
impl Response<'_> {
impl<'a> Response<'a> {
pub fn status(&self) -> StatusCode {
self.0.status()
}

View File

@@ -25,13 +25,12 @@ rand_core = { version = "0.6", default-features = false, features = ["std"] }
blake2 = { version = "0.10", default-features = false, features = ["std"] }
schnorrkel = { version = "0.11", default-features = false, features = ["std"] }
ciphersuite = { path = "../crypto/ciphersuite", default-features = false, features = ["std", "ristretto"] }
dkg = { path = "../crypto/dkg", default-features = false, features = ["std"] }
ciphersuite = { path = "../crypto/ciphersuite", default-features = false, features = ["std"] }
schnorr = { package = "schnorr-signatures", path = "../crypto/schnorr", default-features = false, features = ["std"] }
frost = { package = "modular-frost", path = "../crypto/frost" }
frost-schnorrkel = { path = "../crypto/schnorrkel" }
hex = { version = "0.4", default-features = false, features = ["std"] }
scale = { package = "parity-scale-codec", version = "3", default-features = false, features = ["std", "derive", "bit-vec"] }
borsh = { version = "1", default-features = false, features = ["std", "derive", "de_strict_order"] }
zalloc = { path = "../common/zalloc" }
serai-db = { path = "../common/db" }
@@ -44,6 +43,9 @@ tributary-sdk = { path = "./tributary-sdk" }
serai-client = { path = "../substrate/client", default-features = false, features = ["serai", "borsh"] }
hex = { version = "0.4", default-features = false, features = ["std"] }
borsh = { version = "1", default-features = false, features = ["std", "derive", "de_strict_order"] }
log = { version = "0.4", default-features = false, features = ["std"] }
env_logger = { version = "0.10", default-features = false, features = ["humantime"] }

View File

@@ -24,6 +24,15 @@ pub(crate) struct CosignDelayTask<D: Db> {
pub(crate) db: D,
}
struct AwaitUndroppable<T: DbTxn>(Option<core::mem::ManuallyDrop<Undroppable<T>>>);
impl<T: DbTxn> Drop for AwaitUndroppable<T> {
fn drop(&mut self) {
if let Some(mut txn) = self.0.take() {
(unsafe { core::mem::ManuallyDrop::take(&mut txn) }).close();
}
}
}
impl<D: Db> ContinuallyRan for CosignDelayTask<D> {
type Error = DoesNotError;
@@ -35,14 +44,18 @@ impl<D: Db> ContinuallyRan for CosignDelayTask<D> {
// Receive the next block to mark as cosigned
let Some((block_number, time_evaluated)) = CosignedBlocks::try_recv(&mut txn) else {
txn.close();
break;
};
// Calculate when we should mark it as valid
let time_valid =
SystemTime::UNIX_EPOCH + Duration::from_secs(time_evaluated) + ACKNOWLEDGEMENT_DELAY;
// Sleep until then
let mut txn = AwaitUndroppable(Some(core::mem::ManuallyDrop::new(txn)));
tokio::time::sleep(SystemTime::now().duration_since(time_valid).unwrap_or(Duration::ZERO))
.await;
let mut txn = core::mem::ManuallyDrop::into_inner(txn.0.take().unwrap());
// Set the cosigned block
LatestCosignedBlockNumber::set(&mut txn, &block_number);

View File

@@ -1,5 +1,5 @@
use core::future::Future;
use std::time::{Duration, Instant, SystemTime};
use std::time::{Duration, SystemTime};
use serai_db::*;
use serai_task::ContinuallyRan;
@@ -77,27 +77,17 @@ pub(crate) fn currently_evaluated_global_session(getter: &impl Get) -> Option<[u
pub(crate) struct CosignEvaluatorTask<D: Db, R: RequestNotableCosigns> {
pub(crate) db: D,
pub(crate) request: R,
pub(crate) last_request_for_cosigns: Instant,
}
impl<D: Db, R: RequestNotableCosigns> ContinuallyRan for CosignEvaluatorTask<D, R> {
type Error = String;
fn run_iteration(&mut self) -> impl Send + Future<Output = Result<bool, Self::Error>> {
let should_request_cosigns = |last_request_for_cosigns: &mut Instant| {
const REQUEST_COSIGNS_SPACING: Duration = Duration::from_secs(60);
if Instant::now() < (*last_request_for_cosigns + REQUEST_COSIGNS_SPACING) {
return false;
}
*last_request_for_cosigns = Instant::now();
true
};
async move {
let mut known_cosign = None;
let mut made_progress = false;
loop {
let mut txn = self.db.txn();
let mut txn = self.db.unsafe_txn();
let Some(BlockEventData { block_number, has_events }) = BlockEvents::try_recv(&mut txn)
else {
break;
@@ -128,13 +118,12 @@ impl<D: Db, R: RequestNotableCosigns> ContinuallyRan for CosignEvaluatorTask<D,
// Check if the sum weight doesn't cross the required threshold
if weight_cosigned < (((global_session_info.total_stake * 83) / 100) + 1) {
// Request the necessary cosigns over the network
if should_request_cosigns(&mut self.last_request_for_cosigns) {
self
.request
.request_notable_cosigns(global_session)
.await
.map_err(|e| format!("{e:?}"))?;
}
// TODO: Add a timer to ensure this isn't called too often
self
.request
.request_notable_cosigns(global_session)
.await
.map_err(|e| format!("{e:?}"))?;
// We return an error so the delay before this task is run again increases
return Err(format!(
"notable block (#{block_number}) wasn't yet cosigned. this should resolve shortly",
@@ -191,13 +180,11 @@ impl<D: Db, R: RequestNotableCosigns> ContinuallyRan for CosignEvaluatorTask<D,
// If this session hasn't yet produced notable cosigns, then we presume we'll see
// the desired non-notable cosigns as part of normal operations, without needing to
// explicitly request them
if should_request_cosigns(&mut self.last_request_for_cosigns) {
self
.request
.request_notable_cosigns(global_session)
.await
.map_err(|e| format!("{e:?}"))?;
}
self
.request
.request_notable_cosigns(global_session)
.await
.map_err(|e| format!("{e:?}"))?;
// We return an error so the delay before this task is run again increases
return Err(format!(
"block (#{block_number}) wasn't yet cosigned. this should resolve shortly",

View File

@@ -70,7 +70,7 @@ impl<D: Db> ContinuallyRan for CosignIntendTask<D> {
self.serai.latest_finalized_block().await.map_err(|e| format!("{e:?}"))?.number();
for block_number in start_block_number ..= latest_block_number {
let mut txn = self.db.txn();
let mut txn = self.db.unsafe_txn();
let (block, mut has_events) =
block_has_events_justifying_a_cosign(&self.serai, block_number)

View File

@@ -3,7 +3,7 @@
#![deny(missing_docs)]
use core::{fmt::Debug, future::Future};
use std::{sync::Arc, collections::HashMap, time::Instant};
use std::{sync::Arc, collections::HashMap};
use blake2::{Digest, Blake2s256};
@@ -104,24 +104,6 @@ pub struct Cosign {
pub cosigner: NetworkId,
}
impl CosignIntent {
/// Convert this into a `Cosign`.
pub fn into_cosign(self, cosigner: NetworkId) -> Cosign {
let CosignIntent { global_session, block_number, block_hash, notable: _ } = self;
Cosign { global_session, block_number, block_hash, cosigner }
}
}
impl Cosign {
/// The message to sign to sign this cosign.
///
/// This must be signed with schnorrkel, the context set to `COSIGN_CONTEXT`.
pub fn signature_message(&self) -> Vec<u8> {
// We use a schnorrkel context to domain-separate this
self.encode()
}
}
/// A signed cosign.
#[derive(Clone, Debug, BorshSerialize, BorshDeserialize)]
pub struct SignedCosign {
@@ -136,7 +118,7 @@ impl SignedCosign {
let Ok(signer) = schnorrkel::PublicKey::from_bytes(&signer.0) else { return false };
let Ok(signature) = schnorrkel::Signature::from_bytes(&self.signature) else { return false };
signer.verify_simple(COSIGN_CONTEXT, &self.cosign.signature_message(), &signature).is_ok()
signer.verify_simple(COSIGN_CONTEXT, &self.cosign.encode(), &signature).is_ok()
}
}
@@ -306,12 +288,8 @@ impl<D: Db> Cosigning<D> {
.continually_run(intend_task, vec![evaluator_task_handle]),
);
tokio::spawn(
(evaluator::CosignEvaluatorTask {
db: db.clone(),
request,
last_request_for_cosigns: Instant::now(),
})
.continually_run(evaluator_task, vec![delay_task_handle]),
(evaluator::CosignEvaluatorTask { db: db.clone(), request })
.continually_run(evaluator_task, vec![delay_task_handle]),
);
tokio::spawn(
(delay::CosignDelayTask { db: db.clone() })
@@ -446,7 +424,7 @@ impl<D: Db> Cosigning<D> {
// Since we verified this cosign's signature, and have a chain sufficiently long, handle the
// cosign
let mut txn = self.db.txn();
let mut txn = self.db.unsafe_txn();
if !faulty {
// If this is for a future global session, we don't acknowledge this cosign at this time
@@ -502,3 +480,30 @@ impl<D: Db> Cosigning<D> {
res
}
}
mod tests {
use super::*;
struct RNC;
impl RequestNotableCosigns for RNC {
/// The error type which may be encountered when requesting notable cosigns.
type Error = ();
/// Request the notable cosigns for this global session.
fn request_notable_cosigns(
&self,
global_session: [u8; 32],
) -> impl Send + Future<Output = Result<(), Self::Error>> {
async move { Ok(()) }
}
}
#[tokio::test]
async fn test() {
let db: serai_db::MemDb = serai_db::MemDb::new();
let serai = unsafe { core::mem::transmute(0u64) };
let request = RNC;
let tasks = vec![];
let _ = Cosigning::spawn(db, serai, request, tasks);
core::future::pending().await
}
}

View File

@@ -35,7 +35,7 @@ tributary-sdk = { path = "../../tributary-sdk" }
futures-util = { version = "0.3", default-features = false, features = ["std"] }
tokio = { version = "1", default-features = false, features = ["sync"] }
libp2p = { version = "0.54", default-features = false, features = ["tokio", "tcp", "noise", "yamux", "ping", "request-response", "gossipsub", "macros"] }
libp2p = { version = "0.52", default-features = false, features = ["tokio", "tcp", "noise", "yamux", "ping", "request-response", "gossipsub", "macros"] }
log = { version = "0.4", default-features = false, features = ["std"] }
serai-task = { path = "../../../common/task", version = "0.1" }

View File

@@ -11,7 +11,8 @@ use serai_client::primitives::PublicKey as Public;
use futures_util::{AsyncRead, AsyncReadExt, AsyncWrite, AsyncWriteExt};
use libp2p::{
core::upgrade::{UpgradeInfo, InboundConnectionUpgrade, OutboundConnectionUpgrade},
core::UpgradeInfo,
InboundUpgrade, OutboundUpgrade,
identity::{self, PeerId},
noise,
};
@@ -118,18 +119,12 @@ impl UpgradeInfo for OnlyValidators {
}
}
impl<S: 'static + Send + Unpin + AsyncRead + AsyncWrite> InboundConnectionUpgrade<S>
for OnlyValidators
{
impl<S: 'static + Send + Unpin + AsyncRead + AsyncWrite> InboundUpgrade<S> for OnlyValidators {
type Output = (PeerId, noise::Output<S>);
type Error = io::Error;
type Future = Pin<Box<dyn Send + Future<Output = Result<Self::Output, Self::Error>>>>;
fn upgrade_inbound(
self,
socket: S,
info: <Self as UpgradeInfo>::Info,
) -> <Self as InboundConnectionUpgrade<S>>::Future {
fn upgrade_inbound(self, socket: S, info: Self::Info) -> Self::Future {
Box::pin(async move {
let (dialer_noise_peer_id, mut socket) = noise::Config::new(&self.noise_keypair)
.unwrap()
@@ -152,18 +147,12 @@ impl<S: 'static + Send + Unpin + AsyncRead + AsyncWrite> InboundConnectionUpgrad
}
}
impl<S: 'static + Send + Unpin + AsyncRead + AsyncWrite> OutboundConnectionUpgrade<S>
for OnlyValidators
{
impl<S: 'static + Send + Unpin + AsyncRead + AsyncWrite> OutboundUpgrade<S> for OnlyValidators {
type Output = (PeerId, noise::Output<S>);
type Error = io::Error;
type Future = Pin<Box<dyn Send + Future<Output = Result<Self::Output, Self::Error>>>>;
fn upgrade_outbound(
self,
socket: S,
info: <Self as UpgradeInfo>::Info,
) -> <Self as OutboundConnectionUpgrade<S>>::Future {
fn upgrade_outbound(self, socket: S, info: Self::Info) -> Self::Future {
Box::pin(async move {
let (listener_noise_peer_id, mut socket) = noise::Config::new(&self.noise_keypair)
.unwrap()

View File

@@ -50,7 +50,7 @@ mod ping;
/// The request-response messages and behavior
mod reqres;
use reqres::{InboundRequestId, Request, Response};
use reqres::{RequestId, Request, Response};
/// The gossip messages and behavior
mod gossip;
@@ -66,6 +66,14 @@ use dial::DialTask;
const PORT: u16 = 30563; // 5132 ^ (('c' << 8) | 'o')
// usize::max, manually implemented, as max isn't a const fn
const MAX_LIBP2P_MESSAGE_SIZE: usize =
if gossip::MAX_LIBP2P_GOSSIP_MESSAGE_SIZE > reqres::MAX_LIBP2P_REQRES_MESSAGE_SIZE {
gossip::MAX_LIBP2P_GOSSIP_MESSAGE_SIZE
} else {
reqres::MAX_LIBP2P_REQRES_MESSAGE_SIZE
};
fn peer_id_from_public(public: PublicKey) -> PeerId {
// 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
@@ -135,9 +143,9 @@ struct Libp2pInner {
signed_cosigns: Mutex<mpsc::UnboundedReceiver<SignedCosign>>,
signed_cosigns_send: mpsc::UnboundedSender<SignedCosign>,
heartbeat_requests: Mutex<mpsc::UnboundedReceiver<(InboundRequestId, ValidatorSet, [u8; 32])>>,
notable_cosign_requests: Mutex<mpsc::UnboundedReceiver<(InboundRequestId, [u8; 32])>>,
inbound_request_responses: mpsc::UnboundedSender<(InboundRequestId, Response)>,
heartbeat_requests: Mutex<mpsc::UnboundedReceiver<(RequestId, ValidatorSet, [u8; 32])>>,
notable_cosign_requests: Mutex<mpsc::UnboundedReceiver<(RequestId, [u8; 32])>>,
inbound_request_responses: mpsc::UnboundedSender<(RequestId, Response)>,
}
/// The libp2p-backed P2P implementation.
@@ -168,9 +176,19 @@ impl Libp2p {
Ok(OnlyValidators { serai_key: serai_key.clone(), noise_keypair: noise_keypair.clone() })
};
let new_yamux = || {
let mut config = yamux::Config::default();
// 1 MiB default + max message size
config.set_max_buffer_size((1024 * 1024) + MAX_LIBP2P_MESSAGE_SIZE);
// 256 KiB default + max message size
config
.set_receive_window_size(((256 * 1024) + MAX_LIBP2P_MESSAGE_SIZE).try_into().unwrap());
config
};
let mut swarm = SwarmBuilder::with_existing_identity(identity::Keypair::generate_ed25519())
.with_tokio()
.with_tcp(TcpConfig::default().nodelay(true), new_only_validators, yamux::Config::default)
.with_tcp(TcpConfig::default().nodelay(true), new_only_validators, new_yamux)
.unwrap()
.with_behaviour(|_| Behavior {
allow_list: allow_block_list::Behaviour::default(),

View File

@@ -10,7 +10,7 @@ use futures_util::{AsyncRead, AsyncReadExt, AsyncWrite, AsyncWriteExt};
use libp2p::request_response::{
self, Codec as CodecTrait, Event as GenericEvent, Config, Behaviour, ProtocolSupport,
};
pub use request_response::{InboundRequestId, Message};
pub use request_response::{RequestId, Message};
use serai_cosign::SignedCosign;
@@ -129,6 +129,7 @@ pub(crate) type Event = GenericEvent<Request, Response>;
pub(crate) type Behavior = Behaviour<Codec>;
pub(crate) fn new_behavior() -> Behavior {
let config = Config::default().with_request_timeout(Duration::from_secs(5));
let mut config = Config::default();
config.set_request_timeout(Duration::from_secs(5));
Behavior::new([(PROTOCOL, ProtocolSupport::Full)], config)
}

View File

@@ -17,7 +17,7 @@ use serai_cosign::SignedCosign;
use futures_util::StreamExt;
use libp2p::{
identity::PeerId,
request_response::{InboundRequestId, OutboundRequestId, ResponseChannel},
request_response::{RequestId, ResponseChannel},
swarm::{dial_opts::DialOpts, SwarmEvent, Swarm},
};
@@ -65,12 +65,12 @@ pub(crate) struct SwarmTask {
tributary_gossip: mpsc::UnboundedSender<([u8; 32], Vec<u8>)>,
outbound_requests: mpsc::UnboundedReceiver<(PeerId, Request, oneshot::Sender<Response>)>,
outbound_request_responses: HashMap<OutboundRequestId, oneshot::Sender<Response>>,
outbound_request_responses: HashMap<RequestId, oneshot::Sender<Response>>,
inbound_request_response_channels: HashMap<InboundRequestId, ResponseChannel<Response>>,
heartbeat_requests: mpsc::UnboundedSender<(InboundRequestId, ValidatorSet, [u8; 32])>,
notable_cosign_requests: mpsc::UnboundedSender<(InboundRequestId, [u8; 32])>,
inbound_request_responses: mpsc::UnboundedReceiver<(InboundRequestId, Response)>,
inbound_request_response_channels: HashMap<RequestId, ResponseChannel<Response>>,
heartbeat_requests: mpsc::UnboundedSender<(RequestId, ValidatorSet, [u8; 32])>,
notable_cosign_requests: mpsc::UnboundedSender<(RequestId, [u8; 32])>,
inbound_request_responses: mpsc::UnboundedReceiver<(RequestId, Response)>,
}
impl SwarmTask {
@@ -222,21 +222,25 @@ impl SwarmTask {
}
}
SwarmEvent::Behaviour(event) => {
match event {
BehaviorEvent::AllowList(event) | BehaviorEvent::ConnectionLimits(event) => {
// This *is* an exhaustive match as these events are empty enums
match event {}
}
BehaviorEvent::Ping(ping::Event { peer: _, connection, result, }) => {
if result.is_err() {
self.swarm.close_connection(connection);
}
}
BehaviorEvent::Reqres(event) => self.handle_reqres(event),
BehaviorEvent::Gossip(event) => self.handle_gossip(event),
SwarmEvent::Behaviour(
BehaviorEvent::AllowList(event) | BehaviorEvent::ConnectionLimits(event)
) => {
// This *is* an exhaustive match as these events are empty enums
match event {}
}
SwarmEvent::Behaviour(
BehaviorEvent::Ping(ping::Event { peer: _, connection, result, })
) => {
if result.is_err() {
self.swarm.close_connection(connection);
}
}
SwarmEvent::Behaviour(BehaviorEvent::Reqres(event)) => {
self.handle_reqres(event)
}
SwarmEvent::Behaviour(BehaviorEvent::Gossip(event)) => {
self.handle_gossip(event)
}
// We don't handle any of these
SwarmEvent::IncomingConnection { .. } |
@@ -246,14 +250,7 @@ impl SwarmTask {
SwarmEvent::ExpiredListenAddr { .. } |
SwarmEvent::ListenerClosed { .. } |
SwarmEvent::ListenerError { .. } |
SwarmEvent::Dialing { .. } |
SwarmEvent::NewExternalAddrCandidate { .. } |
SwarmEvent::ExternalAddrConfirmed { .. } |
SwarmEvent::ExternalAddrExpired { .. } |
SwarmEvent::NewExternalAddrOfPeer { .. } => {}
// Requires as SwarmEvent is non-exhaustive
_ => log::warn!("unhandled SwarmEvent: {event:?}"),
SwarmEvent::Dialing { .. } => {}
}
}
@@ -324,9 +321,9 @@ impl SwarmTask {
outbound_requests: mpsc::UnboundedReceiver<(PeerId, Request, oneshot::Sender<Response>)>,
heartbeat_requests: mpsc::UnboundedSender<(InboundRequestId, ValidatorSet, [u8; 32])>,
notable_cosign_requests: mpsc::UnboundedSender<(InboundRequestId, [u8; 32])>,
inbound_request_responses: mpsc::UnboundedReceiver<(InboundRequestId, Response)>,
heartbeat_requests: mpsc::UnboundedSender<(RequestId, ValidatorSet, [u8; 32])>,
notable_cosign_requests: mpsc::UnboundedSender<(RequestId, [u8; 32])>,
inbound_request_responses: mpsc::UnboundedReceiver<(RequestId, Response)>,
) {
tokio::spawn(
SwarmTask {

View File

@@ -51,14 +51,6 @@ impl Validators {
serai: impl Borrow<Serai>,
sessions: impl Borrow<HashMap<NetworkId, Session>>,
) -> Result<Vec<(NetworkId, 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
bypass the cosign protocol unless a historical global session was malicious, in which case
the cosign protocol already breaks.
Besides, we can't connect to historical validators, only the current validators.
*/
let temporal_serai = serai.borrow().as_of_latest_finalized_block().await?;
let temporal_serai = temporal_serai.validator_sets();

View File

@@ -3,11 +3,9 @@ use std::{path::Path, fs};
pub(crate) use serai_db::{Get, DbTxn, Db as DbTrait};
use serai_db::{create_db, db_channel};
use dkg::Participant;
use serai_client::{
primitives::NetworkId,
validator_sets::primitives::{Session, ValidatorSet, KeyPair},
validator_sets::primitives::{Session, ValidatorSet},
};
use serai_cosign::SignedCosign;
@@ -15,7 +13,7 @@ use serai_coordinator_substrate::NewSetInformation;
use serai_coordinator_tributary::Transaction;
#[cfg(all(feature = "parity-db", not(feature = "rocksdb")))]
pub(crate) type Db = std::sync::Arc<serai_db::ParityDb>;
pub(crate) type Db = serai_db::ParityDb;
#[cfg(feature = "rocksdb")]
pub(crate) type Db = serai_db::RocksDB;
@@ -78,10 +76,6 @@ create_db! {
LastProcessorMessage: (network: NetworkId) -> 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,
// The key was set on the Serai network
KeySet: (set: ValidatorSet) -> (),
}
}
@@ -99,51 +93,21 @@ mod _internal_db {
db_channel! {
Coordinator {
// Tributary transactions to publish from the Processor messages
TributaryTransactionsFromProcessorMessages: (set: ValidatorSet) -> Transaction,
// Tributary transactions to publish from the DKG confirmation task
TributaryTransactionsFromDkgConfirmation: (set: ValidatorSet) -> Transaction,
// Participants to remove
RemoveParticipant: (set: ValidatorSet) -> Participant,
// Tributary transactions to publish
TributaryTransactions: (set: ValidatorSet) -> Transaction,
}
}
}
pub(crate) struct TributaryTransactionsFromProcessorMessages;
impl TributaryTransactionsFromProcessorMessages {
pub(crate) struct TributaryTransactions;
impl TributaryTransactions {
pub(crate) fn send(txn: &mut impl DbTxn, set: ValidatorSet, 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);
_internal_db::TributaryTransactions::send(txn, set, tx);
}
}
pub(crate) fn try_recv(txn: &mut impl DbTxn, set: ValidatorSet) -> 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) {
// 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> {
_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) {
// 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> {
_internal_db::RemoveParticipant::try_recv(txn, set)
_internal_db::TributaryTransactions::try_recv(txn, set)
}
}

View File

@@ -1,434 +0,0 @@
use core::{ops::Deref, future::Future};
use std::{boxed::Box, collections::HashMap};
use zeroize::Zeroizing;
use rand_core::OsRng;
use ciphersuite::{group::GroupEncoding, Ciphersuite, Ristretto};
use frost_schnorrkel::{
frost::{
dkg::{Participant, musig::musig},
FrostError,
sign::*,
},
Schnorrkel,
};
use serai_db::{DbTxn, Db as DbTrait};
use serai_client::{
primitives::SeraiAddress,
validator_sets::primitives::{ValidatorSet, musig_context, set_keys_message},
};
use serai_task::{DoesNotError, ContinuallyRan};
use serai_coordinator_substrate::{NewSetInformation, Keys};
use serai_coordinator_tributary::{Transaction, DkgConfirmationMessages};
use crate::{KeysToConfirm, KeySet, TributaryTransactionsFromDkgConfirmation};
fn schnorrkel() -> Schnorrkel {
Schnorrkel::new(b"substrate") // TODO: Pull the constant for this
}
fn our_i(
set: &NewSetInformation,
key: &Zeroizing<<Ristretto as Ciphersuite>::F>,
data: &HashMap<Participant, Vec<u8>>,
) -> Participant {
let public = SeraiAddress((Ristretto::generator() * key.deref()).to_bytes());
let mut our_i = None;
for participant in data.keys() {
let validator_index = usize::from(u16::from(*participant) - 1);
let (validator, _weight) = set.validators[validator_index];
if validator == public {
our_i = Some(*participant);
}
}
our_i.unwrap()
}
// Take a HashMap of participations with non-contiguous Participants and convert them to a
// contiguous sequence.
//
// The input data is expected to not include our own data, which also won't be in the output data.
//
// Returns the mapping from the contiguous Participants to the original Participants.
fn make_contiguous<T>(
our_i: Participant,
mut data: HashMap<Participant, Vec<u8>>,
transform: impl Fn(Vec<u8>) -> std::io::Result<T>,
) -> Result<HashMap<Participant, T>, Participant> {
assert!(!data.contains_key(&our_i));
let mut ordered_participants = data.keys().copied().collect::<Vec<_>>();
ordered_participants.sort_by_key(|participant| u16::from(*participant));
let mut our_i = Some(our_i);
let mut contiguous = HashMap::new();
let mut i = 1;
for participant in ordered_participants {
// If this is the first participant after our own index, increment to account for our index
if let Some(our_i_value) = our_i {
if u16::from(participant) > u16::from(our_i_value) {
i += 1;
our_i = None;
}
}
let contiguous_index = Participant::new(i).unwrap();
let data = match transform(data.remove(&participant).unwrap()) {
Ok(data) => data,
Err(_) => Err(participant)?,
};
contiguous.insert(contiguous_index, data);
i += 1;
}
Ok(contiguous)
}
fn handle_frost_error<T>(result: Result<T, FrostError>) -> Result<T, Participant> {
match &result {
Ok(_) => Ok(result.unwrap()),
Err(FrostError::InvalidPreprocess(participant) | FrostError::InvalidShare(participant)) => {
Err(*participant)
}
// All of these should be unreachable
Err(
FrostError::InternalError(_) |
FrostError::InvalidParticipant(_, _) |
FrostError::InvalidSigningSet(_) |
FrostError::InvalidParticipantQuantity(_, _) |
FrostError::DuplicatedParticipant(_) |
FrostError::MissingParticipant(_),
) => {
result.unwrap();
unreachable!("continued execution after unwrapping Result::Err");
}
}
}
#[rustfmt::skip]
enum Signer {
Preprocess { attempt: u32, seed: CachedPreprocess, preprocess: [u8; 64] },
Share {
attempt: u32,
musig_validators: Vec<SeraiAddress>,
share: [u8; 32],
machine: Box<AlgorithmSignatureMachine<Ristretto, Schnorrkel>>,
},
}
/// Performs the DKG Confirmation protocol.
pub(crate) struct ConfirmDkgTask<CD: DbTrait, TD: DbTrait> {
db: CD,
set: NewSetInformation,
tributary_db: TD,
key: Zeroizing<<Ristretto as Ciphersuite>::F>,
signer: Option<Signer>,
}
impl<CD: DbTrait, TD: DbTrait> ConfirmDkgTask<CD, TD> {
pub(crate) fn new(
db: CD,
set: NewSetInformation,
tributary_db: TD,
key: Zeroizing<<Ristretto as Ciphersuite>::F>,
) -> Self {
Self { db, set, tributary_db, key, signer: None }
}
fn slash(db: &mut CD, set: ValidatorSet, validator: SeraiAddress) {
let mut txn = db.txn();
TributaryTransactionsFromDkgConfirmation::send(
&mut txn,
set,
&Transaction::RemoveParticipant { participant: validator, signed: Default::default() },
);
txn.commit();
}
fn preprocess(
db: &mut CD,
set: ValidatorSet,
attempt: u32,
key: &Zeroizing<<Ristretto as Ciphersuite>::F>,
signer: &mut Option<Signer>,
) {
// Perform the preprocess
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(),
)
.preprocess(&mut OsRng);
// We take the preprocess so we can use it in a distinct machine with the actual Musig
// parameters
let seed = machine.cache();
let mut preprocess_bytes = [0u8; 64];
preprocess_bytes.copy_from_slice(&preprocess.serialize());
let preprocess = preprocess_bytes;
let mut txn = db.txn();
// If this attempt has already been preprocessed for, the Tributary will de-duplicate it
// This may mean the Tributary preprocess is distinct from ours, but we check for that later
TributaryTransactionsFromDkgConfirmation::send(
&mut txn,
set,
&Transaction::DkgConfirmationPreprocess { attempt, preprocess, signed: Default::default() },
);
txn.commit();
*signer = Some(Signer::Preprocess { attempt, seed, preprocess });
}
}
impl<CD: DbTrait, TD: DbTrait> ContinuallyRan for ConfirmDkgTask<CD, TD> {
type Error = DoesNotError;
fn run_iteration(&mut self) -> impl Send + Future<Output = Result<bool, Self::Error>> {
async move {
let mut made_progress = false;
// If we were sent a key to set, create the signer for it
if self.signer.is_none() && KeysToConfirm::get(&self.db, self.set.set).is_some() {
// Create and publish the initial preprocess
Self::preprocess(&mut self.db, self.set.set, 0, &self.key, &mut self.signer);
made_progress = true;
}
// If we have keys to confirm, handle all messages from the tributary
if let Some(key_pair) = KeysToConfirm::get(&self.db, self.set.set) {
// Handle all messages from the Tributary
loop {
let mut tributary_txn = self.tributary_db.txn();
let Some(msg) = DkgConfirmationMessages::try_recv(&mut tributary_txn, self.set.set)
else {
break;
};
match msg {
messages::sign::CoordinatorMessage::Reattempt {
id: messages::sign::SignId { attempt, .. },
} => {
// Create and publish the preprocess for the specified attempt
Self::preprocess(&mut self.db, self.set.set, attempt, &self.key, &mut self.signer);
}
messages::sign::CoordinatorMessage::Preprocesses {
id: messages::sign::SignId { attempt, .. },
mut preprocesses,
} => {
// Confirm the preprocess we're expected to sign with is the one we locally have
// It may be different if we rebooted and made a second preprocess for this attempt
let Some(Signer::Preprocess { attempt: our_attempt, seed, preprocess }) =
self.signer.take()
else {
// If this message is not expected, commit the txn to drop it and move on
// At some point, we'll get a Reattempt and reset
tributary_txn.commit();
break;
};
// Determine the MuSig key signed with
let musig_validators = {
let mut ordered_participants = preprocesses.keys().copied().collect::<Vec<_>>();
ordered_participants.sort_by_key(|participant| u16::from(*participant));
let mut res = vec![];
for participant in ordered_participants {
let (validator, _weight) =
self.set.validators[usize::from(u16::from(participant) - 1)];
res.push(validator);
}
res
};
let musig_public_keys = musig_validators
.iter()
.map(|key| {
Ristretto::read_G(&mut key.0.as_slice())
.expect("Serai validator had invalid public key")
})
.collect::<Vec<_>>();
let keys =
musig(&musig_context(self.set.set), &self.key, &musig_public_keys).unwrap().into();
// Rebuild the machine
let (machine, preprocess_from_cache) =
AlgorithmSignMachine::from_cache(schnorrkel(), keys, seed);
assert_eq!(preprocess.as_slice(), preprocess_from_cache.serialize().as_slice());
// Ensure this is a consistent signing session
let our_i = our_i(&self.set, &self.key, &preprocesses);
let consistent = (attempt == our_attempt) &&
(preprocesses.remove(&our_i).unwrap().as_slice() == preprocess.as_slice());
if !consistent {
tributary_txn.commit();
break;
}
// Reformat the preprocesses into the expected format for Musig
let preprocesses = match make_contiguous(our_i, preprocesses, |preprocess| {
machine.read_preprocess(&mut preprocess.as_slice())
}) {
Ok(preprocesses) => preprocesses,
// This yields the *original participant index*
Err(participant) => {
Self::slash(
&mut self.db,
self.set.set,
self.set.validators[usize::from(u16::from(participant) - 1)].0,
);
tributary_txn.commit();
break;
}
};
// Calculate our share
let (machine, share) = match handle_frost_error(
machine.sign(preprocesses, &set_keys_message(&self.set.set, &key_pair)),
) {
Ok((machine, share)) => (machine, share),
// This yields the *musig participant index*
Err(participant) => {
Self::slash(
&mut self.db,
self.set.set,
musig_validators[usize::from(u16::from(participant) - 1)],
);
tributary_txn.commit();
break;
}
};
// Send our share
let share = <[u8; 32]>::try_from(share.serialize()).unwrap();
let mut txn = self.db.txn();
TributaryTransactionsFromDkgConfirmation::send(
&mut txn,
self.set.set,
&Transaction::DkgConfirmationShare { attempt, share, signed: Default::default() },
);
txn.commit();
self.signer = Some(Signer::Share {
attempt,
musig_validators,
share,
machine: Box::new(machine),
});
}
messages::sign::CoordinatorMessage::Shares {
id: messages::sign::SignId { attempt, .. },
mut shares,
} => {
let Some(Signer::Share { attempt: our_attempt, musig_validators, share, machine }) =
self.signer.take()
else {
tributary_txn.commit();
break;
};
// Ensure this is a consistent signing session
let our_i = our_i(&self.set, &self.key, &shares);
let consistent = (attempt == our_attempt) &&
(shares.remove(&our_i).unwrap().as_slice() == share.as_slice());
if !consistent {
tributary_txn.commit();
break;
}
// Reformat the shares into the expected format for Musig
let shares = match make_contiguous(our_i, shares, |share| {
machine.read_share(&mut share.as_slice())
}) {
Ok(shares) => shares,
// This yields the *original participant index*
Err(participant) => {
Self::slash(
&mut self.db,
self.set.set,
self.set.validators[usize::from(u16::from(participant) - 1)].0,
);
tributary_txn.commit();
break;
}
};
match handle_frost_error(machine.complete(shares)) {
Ok(signature) => {
// Create the bitvec of the participants
let mut signature_participants;
{
use bitvec::prelude::*;
signature_participants = bitvec![u8, Lsb0; 0; 0];
let mut i = 0;
for (validator, _) in &self.set.validators {
if Some(validator) == musig_validators.get(i) {
signature_participants.push(true);
i += 1;
} else {
signature_participants.push(false);
}
}
}
// This is safe to call multiple times as it'll just change which *valid*
// signature to publish
let mut txn = self.db.txn();
Keys::set(
&mut txn,
self.set.set,
key_pair.clone(),
signature_participants,
signature.into(),
);
txn.commit();
}
// This yields the *musig participant index*
Err(participant) => {
Self::slash(
&mut self.db,
self.set.set,
musig_validators[usize::from(u16::from(participant) - 1)],
);
tributary_txn.commit();
break;
}
}
}
}
// Because we successfully handled this message, note we made proress
made_progress = true;
tributary_txn.commit();
}
}
// Check if the key has been set on Serai
if KeysToConfirm::get(&self.db, self.set.set).is_some() &&
KeySet::get(&self.db, self.set.set).is_some()
{
// Take the keys to confirm so we never instantiate the signer again
let mut txn = self.db.txn();
KeysToConfirm::take(&mut txn, self.set.set);
KeySet::take(&mut txn, self.set.set);
txn.commit();
// Drop our own signer
// The task won't die until the Tributary does, but now it'll never do anything again
self.signer = None;
made_progress = true;
}
Ok(made_progress)
}
}
}

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::{NetworkId, PublicKey},
validator_sets::primitives::ValidatorSet,
Serai,
};
use message_queue::{Service, client::MessageQueue};
@@ -23,17 +23,13 @@ use message_queue::{Service, client::MessageQueue};
use serai_task::{Task, TaskHandle, ContinuallyRan};
use serai_cosign::{Faulted, SignedCosign, Cosigning};
use serai_coordinator_substrate::{
CanonicalEventStream, EphemeralEventStream, SignSlashReport, SetKeysTask, SignedBatches,
PublishBatchTask, SlashReports, PublishSlashReportTask,
};
use serai_coordinator_tributary::{SigningProtocolRound, Signed, Transaction, SubstrateBlockPlans};
use serai_coordinator_substrate::{CanonicalEventStream, EphemeralEventStream, SignSlashReport};
use serai_coordinator_tributary::{Signed, Transaction, SubstrateBlockPlans};
mod db;
use db::*;
mod tributary;
mod dkg_confirmation;
mod substrate;
use substrate::SubstrateTask;
@@ -149,25 +145,11 @@ fn spawn_cosigning<D: serai_db::Db>(
});
}
async fn handle_network(
async fn handle_processor_messages(
mut db: impl serai_db::Db,
message_queue: Arc<MessageQueue>,
serai: Arc<Serai>,
network: NetworkId,
) {
// 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
core::mem::forget(publish_batch_task);
}
// Handle Processor messages
loop {
let (msg_id, msg) = {
let msg = message_queue.next(Service::Processor(network)).await;
@@ -198,7 +180,7 @@ async fn handle_network(
messages::ProcessorMessage::KeyGen(msg) => match msg {
messages::key_gen::ProcessorMessage::Participation { session, participation } => {
let set = ValidatorSet { network, session };
TributaryTransactionsFromProcessorMessages::send(
TributaryTransactions::send(
&mut txn,
set,
&Transaction::DkgParticipation { participation, signed: Signed::default() },
@@ -208,84 +190,45 @@ async fn handle_network(
session,
substrate_key,
network_key,
} => {
KeysToConfirm::set(
&mut txn,
ValidatorSet { network, session },
&KeyPair(
PublicKey::from_raw(substrate_key),
network_key
.try_into()
.expect("generated a network key which exceeds the maximum key length"),
),
);
}
} => todo!("TODO Transaction::DkgConfirmationPreprocess"),
messages::key_gen::ProcessorMessage::Blame { session, participant } => {
RemoveParticipant::send(&mut txn, ValidatorSet { network, session }, participant);
let set = ValidatorSet { network, session };
TributaryTransactions::send(
&mut txn,
set,
&Transaction::RemoveParticipant {
participant: todo!("TODO"),
signed: Signed::default(),
},
);
}
},
messages::ProcessorMessage::Sign(msg) => match msg {
messages::sign::ProcessorMessage::InvalidParticipant { session, participant } => {
RemoveParticipant::send(&mut txn, ValidatorSet { network, session }, participant);
let set = ValidatorSet { network, session };
TributaryTransactions::send(
&mut txn,
set,
&Transaction::RemoveParticipant {
participant: todo!("TODO"),
signed: Signed::default(),
},
);
}
messages::sign::ProcessorMessage::Preprocesses { id, preprocesses } => {
let set = ValidatorSet { 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 {
TributaryTransactionsFromProcessorMessages::send(
&mut txn,
set,
&Transaction::Batch { hash },
);
}
}
TributaryTransactionsFromProcessorMessages::send(
&mut txn,
set,
&Transaction::Sign {
id: id.id,
attempt: id.attempt,
round: SigningProtocolRound::Preprocess,
data: preprocesses,
signed: Signed::default(),
},
);
}
messages::sign::ProcessorMessage::Shares { id, shares } => {
let set = ValidatorSet { network, session: id.session };
TributaryTransactionsFromProcessorMessages::send(
&mut txn,
set,
&Transaction::Sign {
id: id.id,
attempt: id.attempt,
round: SigningProtocolRound::Share,
data: shares,
signed: Signed::default(),
},
);
todo!("TODO Transaction::Batch + Transaction::Sign")
}
messages::sign::ProcessorMessage::Shares { id, shares } => todo!("TODO Transaction::Sign"),
},
messages::ProcessorMessage::Coordinator(msg) => match msg {
messages::coordinator::ProcessorMessage::CosignedBlock { cosign } => {
SignedCosigns::send(&mut txn, &cosign);
}
messages::coordinator::ProcessorMessage::SignedBatch { batch } => {
SignedBatches::send(&mut txn, &batch);
todo!("TODO PublishBatchTask")
}
messages::coordinator::ProcessorMessage::SignedSlashReport {
session,
slash_report,
signature,
} => {
SlashReports::set(
&mut txn,
ValidatorSet { network, session },
slash_report,
Signature(signature),
);
messages::coordinator::ProcessorMessage::SignedSlashReport { session, signature } => {
todo!("TODO PublishSlashReportTask")
}
},
messages::ProcessorMessage::Substrate(msg) => match msg {
@@ -300,7 +243,7 @@ async fn handle_network(
for (session, plans) in by_session {
let set = ValidatorSet { network, session };
SubstrateBlockPlans::set(&mut txn, set, block, &plans);
TributaryTransactionsFromProcessorMessages::send(
TributaryTransactions::send(
&mut txn,
set,
&Transaction::SubstrateBlock { hash: block },
@@ -366,16 +309,10 @@ async fn main() {
// Cleanup all historic Tributaries
while let Some(to_cleanup) = TributaryCleanup::try_recv(&mut txn) {
prune_tributary_db(to_cleanup);
// Remove the keys to confirm for this network
KeysToConfirm::take(&mut txn, to_cleanup);
KeySet::take(&mut txn, to_cleanup);
// Drain the cosign intents created for this set
while !Cosigning::<Db>::intended_cosigns(&mut txn, to_cleanup).is_empty() {}
// Drain the transactions to publish for this set
while TributaryTransactionsFromProcessorMessages::try_recv(&mut txn, to_cleanup).is_some() {}
while TributaryTransactionsFromDkgConfirmation::try_recv(&mut txn, to_cleanup).is_some() {}
// Drain the participants to remove for this set
while RemoveParticipant::try_recv(&mut txn, to_cleanup).is_some() {}
while TributaryTransactions::try_recv(&mut txn, to_cleanup).is_some() {}
// Remove the SignSlashReport notification
SignSlashReport::try_recv(&mut txn, to_cleanup);
}
@@ -439,7 +376,7 @@ async fn main() {
EphemeralEventStream::new(
db.clone(),
serai.clone(),
SeraiAddress((<Ristretto as Ciphersuite>::generator() * serai_key.deref()).to_bytes()),
PublicKey::from_raw((<Ristretto as Ciphersuite>::generator() * serai_key.deref()).to_bytes()),
)
.continually_run(substrate_ephemeral_task_def, vec![substrate_task]),
);
@@ -480,32 +417,12 @@ async fn main() {
.continually_run(substrate_task_def, vec![]),
);
// Handle each of the networks
// Handle all of the Processors' messages
for network in serai_client::primitives::NETWORKS {
if network == NetworkId::Serai {
continue;
}
tokio::spawn(handle_network(db.clone(), message_queue.clone(), serai.clone(), network));
}
// Spawn the task to set keys
{
let (set_keys_task_def, set_keys_task) = Task::new();
tokio::spawn(
SetKeysTask::new(db.clone(), serai.clone()).continually_run(set_keys_task_def, vec![]),
);
// Forget its handle so it always runs in the background
core::mem::forget(set_keys_task);
}
// Spawn the task to publish slash reports
{
let (publish_slash_report_task_def, publish_slash_report_task) = Task::new();
tokio::spawn(
PublishSlashReportTask::new(db, serai).continually_run(publish_slash_report_task_def, vec![]),
);
// Always have this run in the background
core::mem::forget(publish_slash_report_task);
tokio::spawn(handle_processor_messages(db.clone(), message_queue.clone(), network));
}
// Run the spawned tasks ad-infinitum

View File

@@ -19,7 +19,7 @@ use serai_task::ContinuallyRan;
use serai_coordinator_tributary::Transaction;
use serai_coordinator_p2p::P2p;
use crate::{Db, KeySet};
use crate::Db;
pub(crate) struct SubstrateTask<P: P2p> {
pub(crate) serai_key: Zeroizing<<Ristretto as Ciphersuite>::F>,
@@ -47,9 +47,8 @@ impl<P: P2p> ContinuallyRan for SubstrateTask<P> {
};
match msg {
messages::substrate::CoordinatorMessage::SetKeys { session, .. } => {
KeySet::set(&mut txn, ValidatorSet { network, session }, &());
}
// TODO: Stop trying to confirm the DKG
messages::substrate::CoordinatorMessage::SetKeys { .. } => todo!("TODO"),
messages::substrate::CoordinatorMessage::SlashesReported { session } => {
let prior_retired = crate::db::RetiredTributary::get(&txn, network);
let next_to_be_retired =

View File

@@ -21,21 +21,10 @@ use message_queue::{Service, Metadata, client::MessageQueue};
use serai_cosign::{Faulted, CosignIntent, Cosigning};
use serai_coordinator_substrate::{NewSetInformation, SignSlashReport};
use serai_coordinator_tributary::{
Topic, Transaction, ProcessorMessages, CosignIntents, RecognizedTopics, ScanTributaryTask,
};
use serai_coordinator_tributary::{Transaction, ProcessorMessages, CosignIntents, ScanTributaryTask};
use serai_coordinator_p2p::P2p;
use crate::{
Db, TributaryTransactionsFromProcessorMessages, TributaryTransactionsFromDkgConfirmation,
RemoveParticipant, dkg_confirmation::ConfirmDkgTask,
};
create_db! {
Coordinator {
PublishOnRecognition: (set: ValidatorSet, topic: Topic) -> Transaction,
}
}
use crate::{Db, TributaryTransactions};
db_channel! {
Coordinator {
@@ -158,101 +147,12 @@ impl<CD: DbTrait, TD: DbTrait, P: P2p> ContinuallyRan
}
}
#[must_use]
async fn add_signed_unsigned_transaction<TD: DbTrait, P: P2p>(
tributary: &Tributary<TD, Transaction, P>,
key: &Zeroizing<<Ristretto as Ciphersuite>::F>,
mut tx: Transaction,
) -> bool {
// If this is a signed transaction, sign it
if matches!(tx.kind(), TransactionKind::Signed(_, _)) {
tx.sign(&mut OsRng, tributary.genesis(), key);
}
let res = tributary.add_transaction(tx.clone()).await;
match &res {
// Fresh publication, already published
Ok(true | false) => {}
Err(
TransactionError::TooLargeTransaction |
TransactionError::InvalidSigner |
TransactionError::InvalidSignature |
TransactionError::InvalidContent,
) => {
panic!("created an invalid transaction, tx: {tx:?}, err: {res:?}");
}
// InvalidNonce may be out-of-order TXs, not invalid ones, but we only create nonce #n+1 after
// on-chain inclusion of the TX with nonce #n, so it is invalid within our context unless the
// issue is this transaction was already included on-chain
Err(TransactionError::InvalidNonce) => {
let TransactionKind::Signed(order, signed) = tx.kind() else {
panic!("non-Signed transaction had InvalidNonce");
};
let next_nonce = tributary
.next_nonce(&signed.signer, &order)
.await
.expect("signer who is a present validator didn't have a nonce");
assert!(next_nonce != signed.nonce);
// We're publishing an old transaction
if next_nonce > signed.nonce {
return true;
}
panic!("nonce in transaction wasn't contiguous with nonce on-chain");
}
// We've published too many transactions recently
Err(TransactionError::TooManyInMempool) => {
return false;
}
// This isn't a Provided transaction so this should never be hit
Err(TransactionError::ProvidedAddedToMempool) => unreachable!(),
}
true
}
async fn add_with_recognition_check<TD: DbTrait, P: P2p>(
set: ValidatorSet,
tributary_db: &mut TD,
tributary: &Tributary<TD, Transaction, P>,
key: &Zeroizing<<Ristretto as Ciphersuite>::F>,
tx: Transaction,
) -> bool {
let kind = tx.kind();
match kind {
TransactionKind::Provided(_) => provide_transaction(set, tributary, tx).await,
TransactionKind::Unsigned | TransactionKind::Signed(_, _) => {
// If this is a transaction with signing data, check the topic is recognized before
// publishing
let topic = tx.topic();
let still_requires_recognition = if let Some(topic) = topic {
(topic.requires_recognition() && (!RecognizedTopics::recognized(tributary_db, set, topic)))
.then_some(topic)
} else {
None
};
if let Some(topic) = still_requires_recognition {
// Queue the transaction until the topic is recognized
// We use the Tributary DB for this so it's cleaned up when the Tributary DB is
let mut tributary_txn = tributary_db.txn();
PublishOnRecognition::set(&mut tributary_txn, set, topic, &tx);
tributary_txn.commit();
} else {
// Actually add the transaction
if !add_signed_unsigned_transaction(tributary, key, tx).await {
return false;
}
}
}
}
true
}
/// Adds all of the transactions sent via `TributaryTransactionsFromProcessorMessages`.
/// Adds all of the transactions sent via `TributaryTransactions`.
pub(crate) struct AddTributaryTransactionsTask<CD: DbTrait, TD: DbTrait, P: P2p> {
db: CD,
tributary_db: TD,
tributary: Tributary<TD, Transaction, P>,
set: NewSetInformation,
set: ValidatorSet,
key: Zeroizing<<Ristretto as Ciphersuite>::F>,
}
impl<CD: DbTrait, TD: DbTrait, P: P2p> ContinuallyRan for AddTributaryTransactionsTask<CD, TD, P> {
@@ -261,87 +161,49 @@ impl<CD: DbTrait, TD: DbTrait, P: P2p> ContinuallyRan for AddTributaryTransactio
fn run_iteration(&mut self) -> impl Send + Future<Output = Result<bool, Self::Error>> {
async move {
let mut made_progress = false;
// Provide/add all transactions sent our way
loop {
let mut txn = self.db.txn();
let Some(tx) = TributaryTransactionsFromDkgConfirmation::try_recv(&mut txn, self.set.set)
else {
break;
};
let Some(mut tx) = TributaryTransactions::try_recv(&mut txn, self.set) else { break };
if !add_with_recognition_check(
self.set.set,
&mut self.tributary_db,
&self.tributary,
&self.key,
tx,
)
.await
{
break;
}
let kind = tx.kind();
match kind {
TransactionKind::Provided(_) => provide_transaction(self.set, &self.tributary, tx).await,
TransactionKind::Unsigned | TransactionKind::Signed(_, _) => {
// If this is a signed transaction, sign it
if matches!(kind, TransactionKind::Signed(_, _)) {
tx.sign(&mut OsRng, self.tributary.genesis(), &self.key);
}
made_progress = true;
txn.commit();
}
loop {
let mut txn = self.db.txn();
let Some(tx) = TributaryTransactionsFromProcessorMessages::try_recv(&mut txn, self.set.set)
else {
break;
};
if !add_with_recognition_check(
self.set.set,
&mut self.tributary_db,
&self.tributary,
&self.key,
tx,
)
.await
{
break;
}
made_progress = true;
txn.commit();
}
// Provide/add all transactions due to newly recognized topics
loop {
let mut tributary_txn = self.tributary_db.txn();
let Some(topic) =
RecognizedTopics::try_recv_topic_requiring_recognition(&mut tributary_txn, self.set.set)
else {
break;
};
if let Some(tx) = PublishOnRecognition::take(&mut tributary_txn, self.set.set, topic) {
if !add_signed_unsigned_transaction(&self.tributary, &self.key, tx).await {
break;
// Actually add the transaction
// TODO: If this is a preprocess, make sure the topic has been recognized
let res = self.tributary.add_transaction(tx.clone()).await;
match &res {
// Fresh publication, already published
Ok(true | false) => {}
Err(
TransactionError::TooLargeTransaction |
TransactionError::InvalidSigner |
TransactionError::InvalidNonce |
TransactionError::InvalidSignature |
TransactionError::InvalidContent,
) => {
panic!("created an invalid transaction, tx: {tx:?}, err: {res:?}");
}
// We've published too many transactions recently
// Drop this txn to try to publish it again later on a future iteration
Err(TransactionError::TooManyInMempool) => {
drop(txn);
break;
}
// This isn't a Provided transaction so this should never be hit
Err(TransactionError::ProvidedAddedToMempool) => unreachable!(),
}
}
}
made_progress = true;
tributary_txn.commit();
}
// Publish any participant removals
loop {
let mut txn = self.db.txn();
let Some(participant) = RemoveParticipant::try_recv(&mut txn, self.set.set) else { break };
let tx = Transaction::RemoveParticipant {
participant: self.set.participant_indexes_reverse_lookup[&participant],
signed: Default::default(),
};
if !add_signed_unsigned_transaction(&self.tributary, &self.key, tx).await {
break;
}
made_progress = true;
txn.commit();
}
Ok(made_progress)
}
}
@@ -461,8 +323,6 @@ async fn scan_on_new_block<CD: DbTrait, TD: DbTrait, P: P2p>(
/// - Spawn the ScanTributaryTask
/// - Spawn the ProvideCosignCosignedTransactionsTask
/// - Spawn the TributaryProcessorMessagesTask
/// - Spawn the AddTributaryTransactionsTask
/// - Spawn the ConfirmDkgTask
/// - Spawn the SignSlashReportTask
/// - Iterate the scan task whenever a new block occurs (not just on the standard interval)
pub(crate) async fn spawn_tributary<P: P2p>(
@@ -543,45 +403,38 @@ pub(crate) async fn spawn_tributary<P: P2p>(
// Spawn the scan task
let (scan_tributary_task_def, scan_tributary_task) = Task::new();
tokio::spawn(
ScanTributaryTask::<_, P>::new(tributary_db.clone(), set.clone(), reader)
ScanTributaryTask::<_, P>::new(tributary_db.clone(), &set, reader)
// This is the only handle for this TributaryProcessorMessagesTask, so when this task is
// dropped, it will be too
.continually_run(scan_tributary_task_def, vec![scan_tributary_messages_task]),
);
// Spawn the add transactions task
let (add_tributary_transactions_task_def, add_tributary_transactions_task) = Task::new();
tokio::spawn(
(AddTributaryTransactionsTask {
db: db.clone(),
tributary_db: tributary_db.clone(),
tributary: tributary.clone(),
set: set.clone(),
key: serai_key.clone(),
})
.continually_run(add_tributary_transactions_task_def, vec![]),
);
// Spawn the task to confirm the DKG result
let (confirm_dkg_task_def, confirm_dkg_task) = Task::new();
tokio::spawn(
ConfirmDkgTask::new(db.clone(), set.clone(), tributary_db.clone(), serai_key.clone())
.continually_run(confirm_dkg_task_def, vec![add_tributary_transactions_task]),
);
// Spawn the sign slash report task
let (sign_slash_report_task_def, sign_slash_report_task) = Task::new();
tokio::spawn(
(SignSlashReportTask {
db: db.clone(),
tributary_db,
tributary_db: tributary_db.clone(),
tributary: tributary.clone(),
set: set.clone(),
key: serai_key,
key: serai_key.clone(),
})
.continually_run(sign_slash_report_task_def, vec![]),
);
// Spawn the add transactions task
let (add_tributary_transactions_task_def, add_tributary_transactions_task) = Task::new();
tokio::spawn(
(AddTributaryTransactionsTask {
db: db.clone(),
tributary_db,
tributary: tributary.clone(),
set: set.set,
key: serai_key,
})
.continually_run(add_tributary_transactions_task_def, vec![]),
);
// Whenever a new block occurs, immediately run the scan task
// This function also preserves the ProvideCosignCosignedTransactionsTask handle until the
// Tributary is retired, ensuring it isn't dropped prematurely and that the task don't run ad
@@ -591,6 +444,10 @@ pub(crate) async fn spawn_tributary<P: P2p>(
set.set,
tributary,
scan_tributary_task,
vec![provide_cosign_cosigned_transactions_task, confirm_dkg_task, sign_slash_report_task],
vec![
provide_cosign_cosigned_transactions_task,
sign_slash_report_task,
add_tributary_transactions_task,
],
));
}

View File

@@ -22,9 +22,6 @@ bitvec = { version = "1", default-features = false, features = ["std"] }
scale = { package = "parity-scale-codec", version = "3", default-features = false, features = ["std", "derive", "bit-vec"] }
borsh = { version = "1", default-features = false, features = ["std", "derive", "de_strict_order"] }
dkg = { path = "../../crypto/dkg", default-features = false, features = ["std"] }
serai-client = { path = "../../substrate/client", version = "0.1", default-features = false, features = ["serai", "borsh"] }
log = { version = "0.4", default-features = false, features = ["std"] }

View File

@@ -4,7 +4,7 @@ use std::sync::Arc;
use futures::stream::{StreamExt, FuturesOrdered};
use serai_client::{
primitives::{NetworkId, SeraiAddress, EmbeddedEllipticCurve},
primitives::{PublicKey, NetworkId, EmbeddedEllipticCurve},
validator_sets::primitives::MAX_KEY_SHARES_PER_SET,
Serai,
};
@@ -26,14 +26,14 @@ create_db!(
pub struct EphemeralEventStream<D: Db> {
db: D,
serai: Arc<Serai>,
validator: SeraiAddress,
validator: PublicKey,
}
impl<D: Db> EphemeralEventStream<D> {
/// Create a new ephemeral event stream.
///
/// Only one of these may exist over the provided database.
pub fn new(db: D, serai: Arc<Serai>, validator: SeraiAddress) -> Self {
pub fn new(db: D, serai: Arc<Serai>, validator: PublicKey) -> Self {
Self { db, serai, validator }
}
}
@@ -145,10 +145,6 @@ impl<D: Db> ContinuallyRan for EphemeralEventStream<D> {
"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);
if in_set {
if u16::try_from(validators.len()).is_err() {
@@ -181,16 +177,14 @@ impl<D: Db> ContinuallyRan for EphemeralEventStream<D> {
embedded_elliptic_curve_keys.push_back(async move {
tokio::try_join!(
// One future to fetch the substrate embedded key
serai.embedded_elliptic_curve_key(
validator.into(),
EmbeddedEllipticCurve::Embedwards25519
),
serai
.embedded_elliptic_curve_key(validator, EmbeddedEllipticCurve::Embedwards25519),
// One future to fetch the external embedded key, if there is a distinct curve
async {
// `embedded_elliptic_curves` is documented to have the second entry be the
// network-specific curve (if it exists and is distinct from Embedwards25519)
if let Some(curve) = set.network.embedded_elliptic_curves().get(1) {
serai.embedded_elliptic_curve_key(validator.into(), *curve).await.map(Some)
serai.embedded_elliptic_curve_key(validator, *curve).await.map(Some)
} else {
Ok(None)
}
@@ -221,22 +215,19 @@ impl<D: Db> ContinuallyRan for EphemeralEventStream<D> {
}
}
let mut new_set = NewSetInformation {
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?
threshold: ((total_weight * 2) / 3) + 1,
validators,
evrf_public_keys,
participant_indexes: Default::default(),
participant_indexes_reverse_lookup: Default::default(),
};
// These aren't serialized, and we immediately serialize and drop this, so this isn't
// necessary. It's just good practice not have this be dirty
new_set.init_participant_indexes();
crate::NewSet::send(&mut txn, &new_set);
crate::NewSet::send(
&mut txn,
&NewSetInformation {
set: *set,
serai_block: block.block_hash,
declaration_time: block.time,
// TODO: Why do we have this as an explicit field here?
// Shouldn't thiis be inlined into the Processor's key gen code, where it's used?
threshold: ((total_weight * 2) / 3) + 1,
validators,
evrf_public_keys,
},
);
}
}

View File

@@ -2,16 +2,12 @@
#![doc = include_str!("../README.md")]
#![deny(missing_docs)]
use std::collections::HashMap;
use scale::{Encode, Decode};
use borsh::{BorshSerialize, BorshDeserialize};
use dkg::Participant;
use borsh::{io, BorshSerialize, BorshDeserialize};
use serai_client::{
primitives::{NetworkId, SeraiAddress, Signature},
validator_sets::primitives::{Session, ValidatorSet, KeyPair, SlashReport},
primitives::{NetworkId, PublicKey, Signature, SeraiAddress},
validator_sets::primitives::{Session, ValidatorSet, KeyPair},
in_instructions::primitives::SignedBatch,
Transaction,
};
@@ -30,9 +26,22 @@ pub use publish_batch::PublishBatchTask;
mod publish_slash_report;
pub use publish_slash_report::PublishSlashReportTask;
fn borsh_serialize_validators<W: io::Write>(
validators: &Vec<(PublicKey, u16)>,
writer: &mut W,
) -> Result<(), io::Error> {
// This doesn't use `encode_to` as `encode_to` panics if the writer returns an error
writer.write_all(&validators.encode())
}
fn borsh_deserialize_validators<R: io::Read>(
reader: &mut R,
) -> Result<Vec<(PublicKey, u16)>, io::Error> {
Decode::decode(&mut scale::IoReader(reader)).map_err(io::Error::other)
}
/// The information for a new set.
#[derive(Clone, Debug, BorshSerialize, BorshDeserialize)]
#[borsh(init = init_participant_indexes)]
pub struct NewSetInformation {
/// The set.
pub set: ValidatorSet,
@@ -43,37 +52,13 @@ pub struct NewSetInformation {
/// The threshold to use.
pub threshold: u16,
/// The validators, with the amount of key shares they have.
pub validators: Vec<(SeraiAddress, u16)>,
#[borsh(
serialize_with = "borsh_serialize_validators",
deserialize_with = "borsh_deserialize_validators"
)]
pub validators: Vec<(PublicKey, u16)>,
/// The eVRF public keys.
///
/// This will have the necessary copies of the keys proper for each validator's weight,
/// accordingly syncing up with `participant_indexes`.
pub evrf_public_keys: Vec<([u8; 32], Vec<u8>)>,
/// The participant indexes, indexed by their validator.
#[borsh(skip)]
pub participant_indexes: HashMap<SeraiAddress, Vec<Participant>>,
/// The validators, indexed by their participant indexes.
#[borsh(skip)]
pub participant_indexes_reverse_lookup: HashMap<Participant, SeraiAddress>,
}
impl NewSetInformation {
fn init_participant_indexes(&mut self) {
let mut next_i = 1;
self.participant_indexes = HashMap::with_capacity(self.validators.len());
self.participant_indexes_reverse_lookup = HashMap::with_capacity(self.validators.len());
for (validator, weight) in &self.validators {
let mut these_is = Vec::with_capacity((*weight).into());
for _ in 0 .. *weight {
let this_i = Participant::new(next_i).unwrap();
next_i += 1;
these_is.push(this_i);
self.participant_indexes_reverse_lookup.insert(this_i, *validator);
}
self.participant_indexes.insert(*validator, these_is);
}
}
}
mod _public_db {
@@ -190,6 +175,8 @@ impl Keys {
pub struct SignedBatches;
impl SignedBatches {
/// Send a `SignedBatch` to publish onto Serai.
///
/// These will be published sequentially. Out-of-order sending risks hanging the task.
pub fn send(txn: &mut impl DbTxn, batch: &SignedBatch) {
_public_db::SignedBatches::send(txn, batch.batch.network, batch);
}
@@ -198,6 +185,10 @@ impl SignedBatches {
}
}
/// The slash report was invalid.
#[derive(Debug)]
pub struct InvalidSlashReport;
/// The slash reports to publish onto Serai.
pub struct SlashReports;
impl SlashReports {
@@ -205,25 +196,30 @@ impl SlashReports {
///
/// This only saves the most recent slashes as only a single session is eligible to have its
/// slashes reported at once.
///
/// Returns Err if the slashes are invalid. Returns Ok if the slashes weren't detected as
/// invalid. Slashes may be considered invalid by the Serai blockchain later even if not detected
/// as invalid here.
pub fn set(
txn: &mut impl DbTxn,
set: ValidatorSet,
slash_report: SlashReport,
slashes: Vec<(SeraiAddress, u32)>,
signature: Signature,
) {
) -> Result<(), InvalidSlashReport> {
// If we have a more recent slash report, don't write this historic one
if let Some((existing_session, _)) = _public_db::SlashReports::get(txn, set.network) {
if existing_session.0 >= set.session.0 {
return;
return Ok(());
}
}
let tx = serai_client::validator_sets::SeraiValidatorSets::report_slashes(
set.network,
slash_report,
slashes.try_into().map_err(|_| InvalidSlashReport)?,
signature,
);
_public_db::SlashReports::set(txn, set.network, &(set.session, tx.encode()));
Ok(())
}
pub(crate) fn take(txn: &mut impl DbTxn, network: NetworkId) -> Option<(Session, Transaction)> {
let (session, tx) = _public_db::SlashReports::take(txn, network)?;

View File

@@ -1,21 +1,14 @@
use core::future::Future;
use std::sync::Arc;
#[rustfmt::skip]
use serai_client::{primitives::NetworkId, in_instructions::primitives::SignedBatch, SeraiError, Serai};
use serai_db::{DbTxn, Db};
use serai_client::{primitives::NetworkId, SeraiError, Serai};
use serai_db::{Get, DbTxn, Db, create_db};
use serai_task::ContinuallyRan;
use crate::SignedBatches;
create_db!(
CoordinatorSubstrate {
LastPublishedBatch: (network: NetworkId) -> u32,
BatchesToPublish: (network: NetworkId, batch: u32) -> SignedBatch,
}
);
/// Publish `SignedBatch`s from `SignedBatches` onto Serai.
pub struct PublishBatchTask<D: Db> {
db: D,
@@ -41,52 +34,32 @@ impl<D: Db> ContinuallyRan for PublishBatchTask<D> {
fn run_iteration(&mut self) -> impl Send + Future<Output = Result<bool, Self::Error>> {
async move {
// Read from SignedBatches, which is sequential, into our own mapping
let mut made_progress = false;
loop {
let mut txn = self.db.txn();
let Some(batch) = SignedBatches::try_recv(&mut txn, self.network) else {
// No batch to publish at this time
break;
};
// If this is a Batch not yet published, save it into our unordered mapping
if LastPublishedBatch::get(&txn, self.network) < Some(batch.batch.id) {
BatchesToPublish::set(&mut txn, self.network, batch.batch.id, &batch);
}
txn.commit();
}
// Synchronize our last published batch with the Serai network's
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
// Publish this Batch if it hasn't already been published
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 our_last_batch = LastPublishedBatch::get(&txn, self.network);
while our_last_batch < last_batch {
let next_batch = our_last_batch.map(|batch| batch + 1).unwrap_or(0);
// Clean up the Batch to publish since it's already been published
BatchesToPublish::take(&mut txn, self.network, next_batch);
our_last_batch = Some(next_batch);
}
if let Some(last_batch) = our_last_batch {
LastPublishedBatch::set(&mut txn, self.network, &last_batch);
}
last_batch.map(|batch| batch + 1).unwrap_or(0)
};
let made_progress =
if let Some(batch) = BatchesToPublish::get(&self.db, self.network, next_to_publish) {
if last_batch < Some(batch.batch.id) {
// This stream of Batches *should* be sequential within the larger context of the Serai
// coordinator. In this library, we use a more relaxed definition and don't assert
// sequence. This does risk hanging the task, if Batch #n+1 is sent before Batch #n, but
// that is a documented fault of the `SignedBatches` API.
self
.serai
.publish(&serai_client::in_instructions::SeraiInInstructions::execute_batch(batch))
.await?;
true
} else {
false
};
}
txn.commit();
made_progress = true;
}
Ok(made_progress)
}
}

View File

@@ -22,82 +22,66 @@ 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> {
let mut txn = self.db.txn();
let Some((session, slash_report)) = SlashReports::take(&mut txn, network) else {
// No slash report to publish
return Ok(false);
};
// 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.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 = current_session.map(|session| session.0);
// Only attempt to publish the slash report for session #n while session #n+1 is still
// active
let session_after_slash_report_retired = current_session > Some(session_after_slash_report.0);
if session_after_slash_report_retired {
// Commit the txn to drain this slash report from the database and not try it again later
txn.commit();
return Ok(false);
}
if Some(session_after_slash_report.0) != current_session {
// We already checked the current session wasn't greater, and they're not equal
assert!(current_session < Some(session_after_slash_report.0));
// This would mean the Serai node is resyncing and is behind where it prior was
Err("have a slash report for a session Serai has yet to retire".to_string())?;
}
// If this session which should publish a slash report already has, move on
let key_pending_slash_report =
serai.key_pending_slash_report(network).await.map_err(|e| format!("{e:?}"))?;
if key_pending_slash_report.is_none() {
txn.commit();
return Ok(false);
};
match self.serai.publish(&slash_report).await {
Ok(()) => {
txn.commit();
Ok(true)
}
// This could be specific to this TX (such as an already in mempool error) and it may be
// worthwhile to continue iteration with the other pending slash reports. We assume this
// error ephemeral and that the latency incurred for this ephemeral error to resolve is
// miniscule compared to the window available to publish the slash report. That makes
// this a non-issue.
Err(e) => Err(format!("couldn't publish slash report transaction: {e:?}")),
}
}
}
impl<D: Db> ContinuallyRan for PublishSlashReportTask<D> {
type Error = String;
fn run_iteration(&mut self) -> impl Send + Future<Output = Result<bool, Self::Error>> {
async move {
let mut made_progress = false;
let mut error = None;
for network in serai_client::primitives::NETWORKS {
if network == NetworkId::Serai {
continue;
};
let network_res = self.publish(network).await;
// We made progress if any network successfully published their slash report
made_progress |= network_res == Ok(true);
// We want to yield the first error *after* attempting for every network
error = error.or(network_res.err());
}
// Yield the error
if let Some(error) = error {
Err(error)?
let mut txn = self.db.txn();
let Some((session, slash_report)) = SlashReports::take(&mut txn, network) else {
// No slash report to publish
continue;
};
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 = current_session.map(|session| session.0);
// Only attempt to publish the slash report for session #n while session #n+1 is still
// active
let session_after_slash_report_retired =
current_session > Some(session_after_slash_report.0);
if session_after_slash_report_retired {
// Commit the txn to drain this slash report from the database and not try it again later
txn.commit();
continue;
}
if Some(session_after_slash_report.0) != current_session {
// We already checked the current session wasn't greater, and they're not equal
assert!(current_session < Some(session_after_slash_report.0));
// This would mean the Serai node is resyncing and is behind where it prior was
Err("have a slash report for a session Serai has yet to retire".to_string())?;
}
// If this session which should publish a slash report already has, move on
let key_pending_slash_report =
serai.key_pending_slash_report(network).await.map_err(|e| format!("{e:?}"))?;
if key_pending_slash_report.is_none() {
txn.commit();
continue;
};
match self.serai.publish(&slash_report).await {
Ok(()) => {
txn.commit();
made_progress = true;
}
// This could be specific to this TX (such as an already in mempool error) and it may be
// worthwhile to continue iteration with the other pending slash reports. We assume this
// error ephemeral and that the latency incurred for this ephemeral error to resolve is
// miniscule compared to the window available to publish the slash report. That makes
// this a non-issue.
Err(e) => Err(format!("couldn't publish slash report transaction: {e:?}"))?,
}
}
Ok(made_progress)
}

View File

@@ -39,8 +39,6 @@ impl<D: Db> ContinuallyRan for SetKeysTask<D> {
continue;
};
// 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.map_err(|e| format!("{e:?}"))?;
let serai = serai.validator_sets();

View File

@@ -21,14 +21,13 @@ workspace = true
zeroize = { version = "^1.5", default-features = false, features = ["std"] }
rand_core = { version = "0.6", default-features = false, features = ["std"] }
scale = { package = "parity-scale-codec", version = "3", default-features = false, features = ["std", "derive"] }
borsh = { version = "1", default-features = false, features = ["std", "derive", "de_strict_order"] }
blake2 = { version = "0.10", default-features = false, features = ["std"] }
ciphersuite = { path = "../../crypto/ciphersuite", default-features = false, features = ["std"] }
dkg = { path = "../../crypto/dkg", default-features = false, features = ["std"] }
schnorr = { package = "schnorr-signatures", path = "../../crypto/schnorr", default-features = false, features = ["std"] }
scale = { package = "parity-scale-codec", version = "3", default-features = false, features = ["std", "derive"] }
borsh = { version = "1", default-features = false, features = ["std", "derive", "de_strict_order"] }
serai-client = { path = "../../substrate/client", default-features = false, features = ["serai", "borsh"] }
serai-db = { path = "../../common/db" }

View File

@@ -15,35 +15,20 @@ use crate::transaction::SigningProtocolRound;
/// A topic within the database which the group participates in
#[derive(Clone, Copy, PartialEq, Eq, Debug, Encode, BorshSerialize, BorshDeserialize)]
pub enum Topic {
pub(crate) enum Topic {
/// Vote to remove a participant
RemoveParticipant {
/// The participant to remove
participant: SeraiAddress,
},
RemoveParticipant { participant: SeraiAddress },
// DkgParticipation isn't represented here as participations are immediately sent to the
// processor, not accumulated within this databse
/// Participation in the signing protocol to confirm the DKG results on Substrate
DkgConfirmation {
/// The attempt number this is for
attempt: u32,
/// The round of the signing protocol
round: SigningProtocolRound,
},
DkgConfirmation { attempt: u32, round: SigningProtocolRound },
/// The local view of the SlashReport, to be aggregated into the final SlashReport
SlashReport,
/// Participation in a signing protocol
Sign {
/// The ID of the signing protocol
id: VariantSignId,
/// The attempt number this is for
attempt: u32,
/// The round of the signing protocol
round: SigningProtocolRound,
},
Sign { id: VariantSignId, attempt: u32, round: SigningProtocolRound },
}
enum Participating {
@@ -94,9 +79,9 @@ impl Topic {
}
}
/// The SignId for this topic
///
/// Returns None if Topic isn't Topic::Sign
// The SignId for this topic
//
// Returns None if Topic isn't Topic::Sign
pub(crate) fn sign_id(self, set: ValidatorSet) -> Option<messages::sign::SignId> {
#[allow(clippy::match_same_arms)]
match self {
@@ -107,33 +92,6 @@ impl Topic {
}
}
/// The SignId for this DKG Confirmation.
///
/// This is undefined except for being consistent to the DKG Confirmation signing protocol and
/// unique across sets.
///
/// Returns None if Topic isn't Topic::DkgConfirmation.
pub(crate) fn dkg_confirmation_sign_id(
self,
set: ValidatorSet,
) -> Option<messages::sign::SignId> {
#[allow(clippy::match_same_arms)]
match self {
Topic::RemoveParticipant { .. } => None,
Topic::DkgConfirmation { attempt, round: _ } => Some({
let id = {
let mut id = [0; 32];
let encoded_set = set.encode();
id[.. encoded_set.len()].copy_from_slice(&encoded_set);
VariantSignId::Batch(id)
};
SignId { session: set.session, id, attempt }
}),
Topic::SlashReport { .. } => None,
Topic::Sign { .. } => None,
}
}
/// The topic which precedes this topic as a prerequisite
///
/// The preceding topic must define this topic as succeeding
@@ -180,22 +138,21 @@ impl Topic {
}
}
/// If this topic requires recognition before entries are permitted for it.
pub fn requires_recognition(&self) -> bool {
fn requires_whitelisting(&self) -> bool {
#[allow(clippy::match_same_arms)]
match self {
// We don't require recognition to remove a participant
// We don't require whitelisting to remove a participant
Topic::RemoveParticipant { .. } => false,
// We don't require recognition for the first attempt, solely the re-attempts
// We don't require whitelisting for the first attempt, solely the re-attempts
Topic::DkgConfirmation { attempt, .. } => *attempt != 0,
// We don't require recognition for the slash report
// We don't require whitelisting for the slash report
Topic::SlashReport { .. } => false,
// We do require recognition for every sign protocol
// We do require whitelisting for every sign protocol
Topic::Sign { .. } => true,
}
}
fn required_participation(&self, n: u16) -> u16 {
fn required_participation(&self, n: u64) -> u64 {
let _ = self;
// All of our topics require 2/3rds participation
((2 * n) / 3) + 1
@@ -241,11 +198,11 @@ create_db!(
// If this block has already been cosigned.
Cosigned: (set: ValidatorSet, substrate_block_hash: [u8; 32]) -> (),
// The plans to recognize upon a `Transaction::SubstrateBlock` being included on-chain.
// The plans to whitelist upon a `Transaction::SubstrateBlock` being included on-chain.
SubstrateBlockPlans: (set: ValidatorSet, substrate_block_hash: [u8; 32]) -> Vec<[u8; 32]>,
// The weight accumulated for a topic.
AccumulatedWeight: (set: ValidatorSet, topic: Topic) -> u16,
AccumulatedWeight: (set: ValidatorSet, topic: Topic) -> u64,
// The entries accumulated for a topic, by validator.
Accumulated: <D: Borshy>(set: ValidatorSet, topic: Topic, validator: SeraiAddress) -> D,
@@ -256,12 +213,7 @@ create_db!(
db_channel!(
CoordinatorTributary {
// Messages to send to the processor
ProcessorMessages: (set: ValidatorSet) -> messages::CoordinatorMessage,
// Messages for the DKG confirmation
DkgConfirmationMessages: (set: ValidatorSet) -> messages::sign::CoordinatorMessage,
// Topics which have been explicitly recognized
RecognizedTopics: (set: ValidatorSet) -> Topic,
}
);
@@ -310,7 +262,7 @@ impl TributaryDb {
);
ActivelyCosigning::set(txn, set, &substrate_block_hash);
Self::recognize_topic(
TributaryDb::recognize_topic(
txn,
set,
Topic::Sign {
@@ -340,10 +292,6 @@ impl TributaryDb {
pub(crate) fn recognize_topic(txn: &mut impl DbTxn, set: ValidatorSet, 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 {
AccumulatedWeight::get(getter, set, topic).is_some()
}
pub(crate) fn start_of_block(txn: &mut impl DbTxn, set: ValidatorSet, block_number: u64) {
@@ -364,12 +312,6 @@ impl TributaryDb {
Self::recognize_topic(txn, set, topic);
if let Some(id) = topic.sign_id(set) {
Self::send_message(txn, set, messages::sign::CoordinatorMessage::Reattempt { id });
} else if let Some(id) = topic.dkg_confirmation_sign_id(set) {
DkgConfirmationMessages::send(
txn,
set,
&messages::sign::CoordinatorMessage::Reattempt { id },
);
}
}
}
@@ -397,24 +339,19 @@ impl TributaryDb {
txn: &mut impl DbTxn,
set: ValidatorSet,
validators: &[SeraiAddress],
total_weight: u16,
total_weight: u64,
block_number: u64,
topic: Topic,
validator: SeraiAddress,
validator_weight: u16,
validator_weight: u64,
data: &D,
) -> DataSet<D> {
// This function will only be called once for a (validator, topic) tuple due to how we handle
// nonces on transactions (deterministically to the topic)
let accumulated_weight = AccumulatedWeight::get(txn, set, topic);
if topic.requires_recognition() && accumulated_weight.is_none() {
Self::fatal_slash(
txn,
set,
validator,
"participated in unrecognized topic which requires recognition",
);
if topic.requires_whitelisting() && accumulated_weight.is_none() {
Self::fatal_slash(txn, set, validator, "participated in unrecognized topic");
return DataSet::None;
}
let mut accumulated_weight = accumulated_weight.unwrap_or(0);

View File

@@ -6,7 +6,6 @@ use core::{marker::PhantomData, future::Future};
use std::collections::HashMap;
use ciphersuite::group::GroupEncoding;
use dkg::Participant;
use serai_client::{
primitives::SeraiAddress,
@@ -28,14 +27,13 @@ use tributary_sdk::{
use serai_cosign::CosignIntent;
use serai_coordinator_substrate::NewSetInformation;
use messages::sign::{VariantSignId, SignId};
use messages::sign::VariantSignId;
mod transaction;
pub use transaction::{SigningProtocolRound, Signed, Transaction};
mod db;
use db::*;
pub use db::Topic;
/// Messages to send to the Processors.
pub struct ProcessorMessages;
@@ -46,24 +44,6 @@ impl ProcessorMessages {
}
}
/// Messages for the DKG confirmation.
pub struct DkgConfirmationMessages;
impl DkgConfirmationMessages {
/// Receive a message for the DKG confirmation.
///
/// These messages use the ProcessorMessage API as that's what existing flows are designed
/// around, enabling their reuse. The ProcessorMessage includes a VariantSignId which isn't
/// applicable to the DKG confirmation (as there's no such variant of the VariantSignId). The
/// actual ID is undefined other than it will be consistent to the signing protocol and unique
/// across validator sets, with no guarantees of uniqueness across contexts.
pub fn try_recv(
txn: &mut impl DbTxn,
set: ValidatorSet,
) -> Option<messages::sign::CoordinatorMessage> {
db::DkgConfirmationMessages::try_recv(txn, set)
}
}
/// The cosign intents.
pub struct CosignIntents;
impl CosignIntents {
@@ -82,28 +62,10 @@ impl CosignIntents {
}
}
/// An interface to the topics recognized on this Tributary.
pub struct RecognizedTopics;
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 {
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,
) -> Option<Topic> {
db::RecognizedTopics::try_recv(txn, set)
}
}
/// The plans to recognize upon a `Transaction::SubstrateBlock` being included on-chain.
/// The plans to whitelist upon a `Transaction::SubstrateBlock` being included on-chain.
pub struct SubstrateBlockPlans;
impl SubstrateBlockPlans {
/// Set the plans to recognize upon the associated `Transaction::SubstrateBlock` being included
/// Set the plans to whitelist upon the associated `Transaction::SubstrateBlock` being included
/// on-chain.
///
/// This must be done before the associated `Transaction::Cosign` is provided.
@@ -113,7 +75,7 @@ impl SubstrateBlockPlans {
substrate_block_hash: [u8; 32],
plans: &Vec<[u8; 32]>,
) {
db::SubstrateBlockPlans::set(txn, set, substrate_block_hash, plans);
db::SubstrateBlockPlans::set(txn, set, substrate_block_hash, &plans);
}
fn take(
txn: &mut impl DbTxn,
@@ -128,32 +90,32 @@ struct ScanBlock<'a, TD: Db, TDT: DbTxn, P: P2p> {
_td: PhantomData<TD>,
_p2p: PhantomData<P>,
tributary_txn: &'a mut TDT,
set: &'a NewSetInformation,
set: ValidatorSet,
validators: &'a [SeraiAddress],
total_weight: u16,
validator_weights: &'a HashMap<SeraiAddress, u16>,
total_weight: u64,
validator_weights: &'a HashMap<SeraiAddress, u64>,
}
impl<TD: Db, TDT: DbTxn, P: P2p> ScanBlock<'_, TD, TDT, P> {
impl<'a, TD: Db, TDT: DbTxn, P: P2p> ScanBlock<'a, TD, TDT, P> {
fn potentially_start_cosign(&mut self) {
// Don't start a new cosigning instance if we're actively running one
if TributaryDb::actively_cosigning(self.tributary_txn, self.set.set).is_some() {
if TributaryDb::actively_cosigning(self.tributary_txn, self.set).is_some() {
return;
}
// Fetch the latest intended-to-be-cosigned block
let Some(latest_substrate_block_to_cosign) =
TributaryDb::latest_substrate_block_to_cosign(self.tributary_txn, self.set.set)
TributaryDb::latest_substrate_block_to_cosign(self.tributary_txn, self.set)
else {
return;
};
// If it was already cosigned, return
if TributaryDb::cosigned(self.tributary_txn, self.set.set, latest_substrate_block_to_cosign) {
if TributaryDb::cosigned(self.tributary_txn, self.set, latest_substrate_block_to_cosign) {
return;
}
let intent =
CosignIntents::take(self.tributary_txn, self.set.set, latest_substrate_block_to_cosign)
CosignIntents::take(self.tributary_txn, self.set, latest_substrate_block_to_cosign)
.expect("Transaction::Cosign locally provided but CosignIntents wasn't populated");
assert_eq!(
intent.block_hash, latest_substrate_block_to_cosign,
@@ -163,71 +125,20 @@ impl<TD: Db, TDT: DbTxn, P: P2p> ScanBlock<'_, TD, TDT, P> {
// Mark us as actively cosigning
TributaryDb::start_cosigning(
self.tributary_txn,
self.set.set,
self.set,
latest_substrate_block_to_cosign,
intent.block_number,
);
// Send the message for the processor to start signing
TributaryDb::send_message(
self.tributary_txn,
self.set.set,
self.set,
messages::coordinator::CoordinatorMessage::CosignSubstrateBlock {
session: self.set.set.session,
cosign: intent.into_cosign(self.set.set.network),
session: self.set.session,
intent,
},
);
}
fn accumulate_dkg_confirmation<D: AsRef<[u8]> + Borshy>(
&mut self,
block_number: u64,
topic: Topic,
data: &D,
signer: SeraiAddress,
) -> Option<(SignId, HashMap<Participant, Vec<u8>>)> {
match TributaryDb::accumulate::<D>(
self.tributary_txn,
self.set.set,
self.validators,
self.total_weight,
block_number,
topic,
signer,
self.validator_weights[&signer],
data,
) {
DataSet::None => None,
DataSet::Participating(data_set) => {
let id = topic.dkg_confirmation_sign_id(self.set.set).unwrap();
// This will be used in a MuSig protocol, so the Participant indexes are the validator's
// position in the list regardless of their weight
let flatten_data_set = |data_set: HashMap<_, D>| {
let mut entries = HashMap::with_capacity(usize::from(self.total_weight));
for (validator, participation) in data_set {
let (index, (_validator, _weight)) = &self
.set
.validators
.iter()
.enumerate()
.find(|(_i, (validator_i, _weight))| validator == *validator_i)
.unwrap();
// The index is zero-indexed yet participants are one-indexed
let index = index + 1;
entries.insert(
Participant::new(u16::try_from(index).unwrap()).unwrap(),
participation.as_ref().to_vec(),
);
}
entries
};
let data_set = flatten_data_set(data_set);
Some((id, data_set))
}
}
}
fn handle_application_tx(&mut self, block_number: u64, tx: Transaction) {
let signer = |signed: Signed| SeraiAddress(signed.signer().to_bytes());
@@ -236,14 +147,13 @@ impl<TD: Db, TDT: DbTxn, P: P2p> ScanBlock<'_, TD, TDT, P> {
// TODO: The fact they can publish these TXs makes this a notable spam vector
if TributaryDb::is_fatally_slashed(
self.tributary_txn,
self.set.set,
self.set,
SeraiAddress(signer.to_bytes()),
) {
return;
}
}
let topic = tx.topic();
match tx {
// Accumulate this vote and fatally slash the participant if past the threshold
Transaction::RemoveParticipant { participant, signed } => {
@@ -253,7 +163,7 @@ impl<TD: Db, TDT: DbTxn, P: P2p> ScanBlock<'_, TD, TDT, P> {
if !self.validators.iter().any(|validator| *validator == participant) {
TributaryDb::fatal_slash(
self.tributary_txn,
self.set.set,
self.set,
signer,
"voted to remove non-existent participant",
);
@@ -262,23 +172,18 @@ impl<TD: Db, TDT: DbTxn, P: P2p> ScanBlock<'_, TD, TDT, P> {
match TributaryDb::accumulate(
self.tributary_txn,
self.set.set,
self.set,
self.validators,
self.total_weight,
block_number,
topic.unwrap(),
Topic::RemoveParticipant { participant },
signer,
self.validator_weights[&signer],
&(),
) {
DataSet::None => {}
DataSet::Participating(_) => {
TributaryDb::fatal_slash(
self.tributary_txn,
self.set.set,
participant,
"voted to remove",
);
TributaryDb::fatal_slash(self.tributary_txn, self.set, participant, "voted to remove");
}
};
}
@@ -287,52 +192,28 @@ impl<TD: Db, TDT: DbTxn, P: P2p> ScanBlock<'_, TD, TDT, P> {
Transaction::DkgParticipation { participation, signed } => {
TributaryDb::send_message(
self.tributary_txn,
self.set.set,
self.set,
messages::key_gen::CoordinatorMessage::Participation {
session: self.set.set.session,
participant: self.set.participant_indexes[&signer(signed)][0],
session: self.set.session,
participant: todo!("TODO"),
participation,
},
);
}
Transaction::DkgConfirmationPreprocess { attempt: _, preprocess, signed } => {
let topic = topic.unwrap();
let signer = signer(signed);
let Some((id, data_set)) =
self.accumulate_dkg_confirmation(block_number, topic, &preprocess, signer)
else {
return;
};
db::DkgConfirmationMessages::send(
self.tributary_txn,
self.set.set,
&messages::sign::CoordinatorMessage::Preprocesses { id, preprocesses: data_set },
);
Transaction::DkgConfirmationPreprocess { attempt, preprocess, signed } => {
// Accumulate the preprocesses into our own FROST attempt manager
todo!("TODO")
}
Transaction::DkgConfirmationShare { attempt: _, share, signed } => {
let topic = topic.unwrap();
let signer = signer(signed);
let Some((id, data_set)) =
self.accumulate_dkg_confirmation(block_number, topic, &share, signer)
else {
return;
};
db::DkgConfirmationMessages::send(
self.tributary_txn,
self.set.set,
&messages::sign::CoordinatorMessage::Shares { id, shares: data_set },
);
Transaction::DkgConfirmationShare { attempt, share, signed } => {
// Accumulate the shares into our own FROST attempt manager
todo!("TODO: SetKeysTask")
}
Transaction::Cosign { substrate_block_hash } => {
// Update the latest intended-to-be-cosigned Substrate block
TributaryDb::set_latest_substrate_block_to_cosign(
self.tributary_txn,
self.set.set,
self.set,
substrate_block_hash,
);
// Start a new cosign if we aren't already working on one
@@ -345,32 +226,32 @@ impl<TD: Db, TDT: DbTxn, P: P2p> ScanBlock<'_, TD, TDT, P> {
not-yet-Cosigned cosigns, we flag all cosigned blocks as cosigned. Then, when we choose
the next block to work on, we won't if it's already been cosigned.
*/
TributaryDb::mark_cosigned(self.tributary_txn, self.set.set, substrate_block_hash);
TributaryDb::mark_cosigned(self.tributary_txn, self.set, substrate_block_hash);
// If we aren't actively cosigning this block, return
// This occurs when we have Cosign TXs A, B, C, we received Cosigned for A and start on C,
// and then receive Cosigned for B
if TributaryDb::actively_cosigning(self.tributary_txn, self.set.set) !=
if TributaryDb::actively_cosigning(self.tributary_txn, self.set) !=
Some(substrate_block_hash)
{
return;
}
// Since this is the block we were cosigning, mark us as having finished cosigning
TributaryDb::finish_cosigning(self.tributary_txn, self.set.set);
TributaryDb::finish_cosigning(self.tributary_txn, self.set);
// Start working on the next cosign
self.potentially_start_cosign();
}
Transaction::SubstrateBlock { hash } => {
// Recognize all of the IDs this Substrate block causes to be signed
let plans = SubstrateBlockPlans::take(self.tributary_txn, self.set.set, hash).expect(
// Whitelist all of the IDs this Substrate block causes to be signed
let plans = SubstrateBlockPlans::take(self.tributary_txn, self.set, hash).expect(
"Transaction::SubstrateBlock locally provided but SubstrateBlockPlans wasn't populated",
);
for plan in plans {
TributaryDb::recognize_topic(
self.tributary_txn,
self.set.set,
self.set,
Topic::Sign {
id: VariantSignId::Transaction(plan),
attempt: 0,
@@ -380,10 +261,10 @@ impl<TD: Db, TDT: DbTxn, P: P2p> ScanBlock<'_, TD, TDT, P> {
}
}
Transaction::Batch { hash } => {
// Recognize the signing of this batch
// Whitelist the signing of this batch
TributaryDb::recognize_topic(
self.tributary_txn,
self.set.set,
self.set,
Topic::Sign {
id: VariantSignId::Batch(hash),
attempt: 0,
@@ -398,7 +279,7 @@ impl<TD: Db, TDT: DbTxn, P: P2p> ScanBlock<'_, TD, TDT, P> {
if slash_points.len() != self.validators.len() {
TributaryDb::fatal_slash(
self.tributary_txn,
self.set.set,
self.set,
signer,
"slash report was for a distinct amount of signers",
);
@@ -408,11 +289,11 @@ impl<TD: Db, TDT: DbTxn, P: P2p> ScanBlock<'_, TD, TDT, P> {
// Accumulate, and if past the threshold, calculate *the* slash report and start signing it
match TributaryDb::accumulate(
self.tributary_txn,
self.set.set,
self.set,
self.validators,
self.total_weight,
block_number,
topic.unwrap(),
Topic::SlashReport,
signer,
self.validator_weights[&signer],
&slash_points,
@@ -426,6 +307,10 @@ impl<TD: Db, TDT: DbTxn, P: P2p> ScanBlock<'_, TD, TDT, P> {
have a supermajority agree the slash should be fatal. If there isn't a supermajority,
but the median believe the slash should be fatal, we need to fallback to a large
constant.
Also, TODO, each slash point should probably be considered as
`MAX_KEY_SHARES_PER_SET * BLOCK_TIME` seconds of downtime. As this time crosses
various thresholds (1 day, 3 days, etc), a multiplier should be attached.
*/
let mut median_slash_report = Vec::with_capacity(self.validators.len());
for i in 0 .. self.validators.len() {
@@ -466,7 +351,7 @@ impl<TD: Db, TDT: DbTxn, P: P2p> ScanBlock<'_, TD, TDT, P> {
// Create the resulting slash report
let mut slash_report = vec![];
for points in amortized_slash_report {
for (validator, points) in self.validators.iter().copied().zip(amortized_slash_report) {
// TODO: Natively store this as a `Slash`
if points == u32::MAX {
slash_report.push(Slash::Fatal);
@@ -479,7 +364,7 @@ impl<TD: Db, TDT: DbTxn, P: P2p> ScanBlock<'_, TD, TDT, P> {
// Recognize the topic for signing the slash report
TributaryDb::recognize_topic(
self.tributary_txn,
self.set.set,
self.set,
Topic::Sign {
id: VariantSignId::SlashReport,
attempt: 0,
@@ -489,24 +374,24 @@ impl<TD: Db, TDT: DbTxn, P: P2p> ScanBlock<'_, TD, TDT, P> {
// Send the message for the processor to start signing
TributaryDb::send_message(
self.tributary_txn,
self.set.set,
self.set,
messages::coordinator::CoordinatorMessage::SignSlashReport {
session: self.set.set.session,
slash_report: slash_report.try_into().unwrap(),
session: self.set.session,
report: slash_report,
},
);
}
};
}
Transaction::Sign { id: _, attempt: _, round, data, signed } => {
let topic = topic.unwrap();
Transaction::Sign { id, attempt, round, data, signed } => {
let topic = Topic::Sign { id, attempt, round };
let signer = signer(signed);
if data.len() != usize::from(self.validator_weights[&signer]) {
if u64::try_from(data.len()).unwrap() != self.validator_weights[&signer] {
TributaryDb::fatal_slash(
self.tributary_txn,
self.set.set,
self.set,
signer,
"signer signed with a distinct amount of key shares than they had key shares",
);
@@ -515,7 +400,7 @@ impl<TD: Db, TDT: DbTxn, P: P2p> ScanBlock<'_, TD, TDT, P> {
match TributaryDb::accumulate(
self.tributary_txn,
self.set.set,
self.set,
self.validators,
self.total_weight,
block_number,
@@ -526,22 +411,12 @@ impl<TD: Db, TDT: DbTxn, P: P2p> ScanBlock<'_, TD, TDT, P> {
) {
DataSet::None => {}
DataSet::Participating(data_set) => {
let id = topic.sign_id(self.set.set).expect("Topic::Sign didn't have SignId");
let flatten_data_set = |data_set: HashMap<_, Vec<_>>| {
let mut entries = HashMap::with_capacity(usize::from(self.total_weight));
for (validator, shares) in data_set {
let indexes = &self.set.participant_indexes[&validator];
assert_eq!(indexes.len(), shares.len());
for (index, share) in indexes.iter().zip(shares) {
entries.insert(*index, share);
}
}
entries
};
let id = topic.sign_id(self.set).expect("Topic::Sign didn't have SignId");
let flatten_data_set = |data_set| todo!("TODO");
let data_set = flatten_data_set(data_set);
TributaryDb::send_message(
self.tributary_txn,
self.set.set,
self.set,
match round {
SigningProtocolRound::Preprocess => {
messages::sign::CoordinatorMessage::Preprocesses { id, preprocesses: data_set }
@@ -552,13 +427,13 @@ impl<TD: Db, TDT: DbTxn, P: P2p> ScanBlock<'_, TD, TDT, P> {
},
)
}
}
};
}
}
}
fn handle_block(mut self, block_number: u64, block: Block<Transaction>) {
TributaryDb::start_of_block(self.tributary_txn, self.set.set, block_number);
TributaryDb::start_of_block(self.tributary_txn, self.set, block_number);
for tx in block.transactions {
match tx {
@@ -585,7 +460,7 @@ impl<TD: Db, TDT: DbTxn, P: P2p> ScanBlock<'_, TD, TDT, P> {
// errors, mark the node as fatally slashed
TributaryDb::fatal_slash(
self.tributary_txn,
self.set.set,
self.set,
SeraiAddress(msgs.0.msg.sender),
&format!("invalid tendermint messages: {msgs:?}"),
);
@@ -601,10 +476,10 @@ impl<TD: Db, TDT: DbTxn, P: P2p> ScanBlock<'_, TD, TDT, P> {
/// The task to scan the Tributary, populating `ProcessorMessages`.
pub struct ScanTributaryTask<TD: Db, P: P2p> {
tributary_db: TD,
set: NewSetInformation,
set: ValidatorSet,
validators: Vec<SeraiAddress>,
total_weight: u16,
validator_weights: HashMap<SeraiAddress, u16>,
total_weight: u64,
validator_weights: HashMap<SeraiAddress, u64>,
tributary: TributaryReader<TD, Transaction>,
_p2p: PhantomData<P>,
}
@@ -613,13 +488,15 @@ impl<TD: Db, P: P2p> ScanTributaryTask<TD, P> {
/// Create a new instance of this task.
pub fn new(
tributary_db: TD,
set: NewSetInformation,
new_set: &NewSetInformation,
tributary: TributaryReader<TD, Transaction>,
) -> Self {
let mut validators = Vec::with_capacity(set.validators.len());
let mut validators = Vec::with_capacity(new_set.validators.len());
let mut total_weight = 0;
let mut validator_weights = HashMap::with_capacity(set.validators.len());
for (validator, weight) in set.validators.iter().copied() {
let mut validator_weights = HashMap::with_capacity(new_set.validators.len());
for (validator, weight) in new_set.validators.iter().copied() {
let validator = SeraiAddress::from(validator);
let weight = u64::from(weight);
validators.push(validator);
total_weight += weight;
validator_weights.insert(validator, weight);
@@ -627,7 +504,7 @@ impl<TD: Db, P: P2p> ScanTributaryTask<TD, P> {
ScanTributaryTask {
tributary_db,
set,
set: new_set.set,
validators,
total_weight,
validator_weights,
@@ -643,7 +520,7 @@ impl<TD: Db, P: P2p> ContinuallyRan for ScanTributaryTask<TD, P> {
fn run_iteration(&mut self) -> impl Send + Future<Output = Result<bool, Self::Error>> {
async move {
let (mut last_block_number, mut last_block_hash) =
TributaryDb::last_handled_tributary_block(&self.tributary_db, self.set.set)
TributaryDb::last_handled_tributary_block(&self.tributary_db, self.set)
.unwrap_or((0, self.tributary.genesis()));
let mut made_progress = false;
@@ -662,7 +539,7 @@ impl<TD: Db, P: P2p> ContinuallyRan for ScanTributaryTask<TD, P> {
if !self.tributary.locally_provided_txs_in_block(&block_hash, order) {
return Err(format!(
"didn't have the provided Transactions on-chain for set (ephemeral error): {:?}",
self.set.set
self.set
));
}
}
@@ -672,7 +549,7 @@ impl<TD: Db, P: P2p> ContinuallyRan for ScanTributaryTask<TD, P> {
_td: PhantomData::<TD>,
_p2p: PhantomData::<P>,
tributary_txn: &mut tributary_txn,
set: &self.set,
set: self.set,
validators: &self.validators,
total_weight: self.total_weight,
validator_weights: &self.validator_weights,
@@ -680,7 +557,7 @@ impl<TD: Db, P: P2p> ContinuallyRan for ScanTributaryTask<TD, P> {
.handle_block(block_number, block);
TributaryDb::set_last_handled_tributary_block(
&mut tributary_txn,
self.set.set,
self.set,
block_number,
block_hash,
);
@@ -700,6 +577,7 @@ impl<TD: Db, P: P2p> ContinuallyRan for ScanTributaryTask<TD, P> {
pub fn slash_report_transaction(getter: &impl Get, set: &NewSetInformation) -> Transaction {
let mut slash_points = Vec::with_capacity(set.validators.len());
for (validator, _weight) in set.validators.iter().copied() {
let validator = SeraiAddress::from(validator);
slash_points.push(SlashPoints::get(getter, set.set, validator).unwrap_or(0));
}
Transaction::SlashReport { slash_points, signed: Signed::default() }

View File

@@ -25,8 +25,6 @@ use tributary_sdk::{
},
};
use crate::db::Topic;
/// The round this data is for, within a signing protocol.
#[derive(Clone, Copy, PartialEq, Eq, Debug, Encode, BorshSerialize, BorshDeserialize)]
pub enum SigningProtocolRound {
@@ -182,7 +180,7 @@ pub enum Transaction {
///
/// This is provided after the block has been cosigned.
///
/// With the acknowledgement of a Substrate block, we can recognize all the `VariantSignId`s
/// With the acknowledgement of a Substrate block, we can whitelist all the `VariantSignId`s
/// resulting from its handling.
SubstrateBlock {
/// The hash of the Substrate block
@@ -259,7 +257,9 @@ impl TransactionTrait for Transaction {
Transaction::Cosign { .. } => TransactionKind::Provided("Cosign"),
Transaction::Cosigned { .. } => TransactionKind::Provided("Cosigned"),
// TODO: Provide this
Transaction::SubstrateBlock { .. } => TransactionKind::Provided("SubstrateBlock"),
// TODO: Provide this
Transaction::Batch { .. } => TransactionKind::Provided("Batch"),
Transaction::Sign { id, attempt, round, signed, .. } => TransactionKind::Signed(
@@ -318,36 +318,6 @@ impl TransactionTrait for Transaction {
}
impl Transaction {
/// The topic in the database for this transaction.
pub fn topic(&self) -> Option<Topic> {
#[allow(clippy::match_same_arms)] // This doesn't make semantic sense here
match self {
Transaction::RemoveParticipant { participant, .. } => {
Some(Topic::RemoveParticipant { participant: *participant })
}
Transaction::DkgParticipation { .. } => None,
Transaction::DkgConfirmationPreprocess { attempt, .. } => {
Some(Topic::DkgConfirmation { attempt: *attempt, round: SigningProtocolRound::Preprocess })
}
Transaction::DkgConfirmationShare { attempt, .. } => {
Some(Topic::DkgConfirmation { attempt: *attempt, round: SigningProtocolRound::Share })
}
// Provided TXs
Transaction::Cosign { .. } |
Transaction::Cosigned { .. } |
Transaction::SubstrateBlock { .. } |
Transaction::Batch { .. } => None,
Transaction::Sign { id, attempt, round, .. } => {
Some(Topic::Sign { id: *id, attempt: *attempt, round: *round })
}
Transaction::SlashReport { .. } => Some(Topic::SlashReport),
}
}
/// Sign a transaction.
///
/// Panics if signing a transaction whose type isn't `TransactionKind::Signed`.
@@ -365,12 +335,10 @@ impl Transaction {
Transaction::DkgConfirmationPreprocess { ref mut signed, .. } => signed,
Transaction::DkgConfirmationShare { ref mut signed, .. } => signed,
Transaction::Cosign { .. } => panic!("signing Cosign transaction (provided)"),
Transaction::Cosigned { .. } => panic!("signing Cosigned transaction (provided)"),
Transaction::SubstrateBlock { .. } => {
panic!("signing SubstrateBlock transaction (provided)")
}
Transaction::Batch { .. } => panic!("signing Batch transaction (provided)"),
Transaction::Cosign { .. } => panic!("signing CosignSubstrateBlock"),
Transaction::Cosigned { .. } => panic!("signing Cosigned"),
Transaction::SubstrateBlock { .. } => panic!("signing SubstrateBlock"),
Transaction::Batch { .. } => panic!("signing Batch"),
Transaction::Sign { ref mut signed, .. } => signed,

View File

@@ -92,7 +92,7 @@ impl Neg for FieldElement {
}
}
impl Neg for &FieldElement {
impl<'a> Neg for &'a FieldElement {
type Output = FieldElement;
fn neg(self) -> Self::Output {
(*self).neg()

View File

@@ -37,11 +37,11 @@ pub(crate) fn challenge<T: Transcript, F: PrimeField>(transcript: &mut T) -> F {
// Get a wide amount of bytes to safely reduce without bias
// In most cases, <=1.5x bytes is enough. 2x is still standard and there's some theoretical
// groups which may technically require more than 1.5x bytes for this to work as intended
let target_bytes = usize::try_from(F::NUM_BITS).unwrap().div_ceil(8) * 2;
let target_bytes = ((usize::try_from(F::NUM_BITS).unwrap() + 7) / 8) * 2;
let mut challenge_bytes = transcript.challenge(b"challenge");
let challenge_bytes_len = challenge_bytes.as_ref().len();
// If the challenge is 32 bytes, and we need 64, we need two challenges
let needed_challenges = target_bytes.div_ceil(challenge_bytes_len);
let needed_challenges = (target_bytes + (challenge_bytes_len - 1)) / challenge_bytes_len;
// The following algorithm should be equivalent to a wide reduction of the challenges,
// interpreted as concatenated, big-endian byte string

View File

@@ -33,7 +33,7 @@ pub struct ArithmeticCircuitStatement<'a, C: Ciphersuite> {
V: PointVector<C>,
}
impl<C: Ciphersuite> Zeroize for ArithmeticCircuitStatement<'_, C> {
impl<'a, C: Ciphersuite> Zeroize for ArithmeticCircuitStatement<'a, C> {
fn zeroize(&mut self) {
self.constraints.zeroize();
self.C.zeroize();

View File

@@ -247,7 +247,7 @@ impl<C: Ciphersuite> Generators<C> {
}
}
impl<C: Ciphersuite> ProofGenerators<'_, C> {
impl<'a, C: Ciphersuite> ProofGenerators<'a, C> {
pub(crate) fn len(&self) -> usize {
self.g_bold.len()
}

View File

@@ -203,15 +203,14 @@ pub trait SignMachine<S>: Send + Sync + Sized {
/// SignatureMachine this SignMachine turns into.
type SignatureMachine: SignatureMachine<S, SignatureShare = Self::SignatureShare>;
/// Cache this preprocess for usage later.
///
/// This cached preprocess MUST only be used once. Reuse of it enables recovery of your private
/// key share. Third-party recovery of a cached preprocess also enables recovery of your private
/// key share, so this MUST be treated with the same security as your private key share.
/// Cache this preprocess for usage later. This cached preprocess MUST only be used once. Reuse
/// of it enables recovery of your private key share. Third-party recovery of a cached preprocess
/// also enables recovery of your private key share, so this MUST be treated with the same
/// security as your private key share.
fn cache(self) -> CachedPreprocess;
/// Create a sign machine from a cached preprocess.
///
/// After this, the preprocess must be deleted so it's never reused. Any reuse will presumably
/// cause the signer to leak their secret share.
fn from_cache(
@@ -220,14 +219,11 @@ pub trait SignMachine<S>: Send + Sync + Sized {
cache: CachedPreprocess,
) -> (Self, Self::Preprocess);
/// Read a Preprocess message.
///
/// Despite taking self, this does not save the preprocess. It must be externally cached and
/// passed into sign.
/// Read a Preprocess message. Despite taking self, this does not save the preprocess.
/// It must be externally cached and passed into sign.
fn read_preprocess<R: Read>(&self, reader: &mut R) -> io::Result<Self::Preprocess>;
/// Sign a message.
///
/// Takes in the participants' preprocess messages. Returns the signature share to be broadcast
/// to all participants, over an authenticated channel. The parties who participate here will
/// become the signing set for this session.

View File

@@ -59,7 +59,7 @@ pub(crate) fn prep_bits<G: Group<Scalar: PrimeFieldBits>>(
for pair in pairs {
let p = groupings.len();
let mut bits = pair.0.to_le_bits();
groupings.push(vec![0; bits.len().div_ceil(w_usize)]);
groupings.push(vec![0; (bits.len() + (w_usize - 1)) / w_usize]);
for (i, mut bit) in bits.iter_mut().enumerate() {
let mut bit = u8_from_bool(&mut bit);

View File

@@ -28,7 +28,6 @@ ciphersuite = { path = "../../crypto/ciphersuite", default-features = false, fea
dkg = { path = "../../crypto/dkg", default-features = false, features = ["std", "evrf-ristretto"] }
serai-client = { path = "../../substrate/client", default-features = false }
serai-cosign = { path = "../../coordinator/cosign" }
log = { version = "0.4", default-features = false, features = ["std"] }
env_logger = { version = "0.10", default-features = false, features = ["humantime"] }

View File

@@ -3,14 +3,12 @@ use std::sync::{LazyLock, Arc, Mutex};
use tokio::sync::mpsc;
use scale::Encode;
use serai_client::{
primitives::Signature,
validator_sets::primitives::{Session, SlashReport},
primitives::Signature, validator_sets::primitives::Session,
in_instructions::primitives::SignedBatch,
};
use serai_cosign::SignedCosign;
use serai_db::{Get, DbTxn, Db, create_db, db_channel};
use scanner::ScannerFeed;
@@ -183,11 +181,17 @@ impl signers::Coordinator for CoordinatorSend {
fn publish_cosign(
&mut self,
cosign: SignedCosign,
block_number: u64,
block: [u8; 32],
signature: Signature,
) -> impl Send + Future<Output = Result<(), Self::EphemeralError>> {
async move {
self.send(&messages::ProcessorMessage::Coordinator(
messages::coordinator::ProcessorMessage::CosignedBlock { cosign },
messages::coordinator::ProcessorMessage::CosignedBlock {
block_number,
block,
signature: signature.encode(),
},
));
Ok(())
}
@@ -208,15 +212,13 @@ impl signers::Coordinator for CoordinatorSend {
fn publish_slash_report_signature(
&mut self,
session: Session,
slash_report: SlashReport,
signature: Signature,
) -> impl Send + Future<Output = Result<(), Self::EphemeralError>> {
async move {
self.send(&messages::ProcessorMessage::Coordinator(
messages::coordinator::ProcessorMessage::SignedSlashReport {
session,
slash_report,
signature: signature.0,
signature: signature.encode(),
},
));
Ok(())

View File

@@ -221,16 +221,20 @@ pub async fn main_loop<
signers.queue_message(txn, &msg)
}
messages::CoordinatorMessage::Coordinator(
messages::coordinator::CoordinatorMessage::CosignSubstrateBlock { session, cosign },
messages::coordinator::CoordinatorMessage::CosignSubstrateBlock {
session,
block_number,
block,
},
) => {
let txn = txn.take().unwrap();
signers.cosign_block(txn, session, &cosign)
signers.cosign_block(txn, session, block_number, block)
}
messages::CoordinatorMessage::Coordinator(
messages::coordinator::CoordinatorMessage::SignSlashReport { session, slash_report },
messages::coordinator::CoordinatorMessage::SignSlashReport { session, report },
) => {
let txn = txn.take().unwrap();
signers.sign_slash_report(txn, session, &slash_report)
signers.sign_slash_report(txn, session, &report)
}
messages::CoordinatorMessage::Substrate(msg) => match msg {

View File

@@ -29,8 +29,8 @@ pub(crate) fn generators<C: EvrfCurve>() -> &'static EvrfGenerators<C> {
.or_insert_with(|| {
// If we haven't prior needed generators for this Ciphersuite, generate new ones
Box::leak(Box::new(EvrfGenerators::<C>::new(
(MAX_KEY_SHARES_PER_SET * 2 / 3) + 1,
MAX_KEY_SHARES_PER_SET,
((MAX_KEY_SHARES_PER_SET * 2 / 3) + 1).try_into().unwrap(),
MAX_KEY_SHARES_PER_SET.try_into().unwrap(),
)))
})
.downcast_ref()

View File

@@ -7,11 +7,11 @@ use borsh::{BorshSerialize, BorshDeserialize};
use dkg::Participant;
use serai_primitives::BlockHash;
use validator_sets_primitives::{Session, KeyPair, SlashReport};
use validator_sets_primitives::{Session, KeyPair, Slash};
use coins_primitives::OutInstructionWithBalance;
use in_instructions_primitives::SignedBatch;
use serai_cosign::{Cosign, SignedCosign};
use serai_cosign::{CosignIntent, SignedCosign};
#[derive(Clone, Copy, PartialEq, Eq, Debug, BorshSerialize, BorshDeserialize)]
pub struct SubstrateContext {
@@ -100,9 +100,7 @@ pub mod sign {
Self::Cosign(cosign) => {
f.debug_struct("VariantSignId::Cosign").field("0", &cosign).finish()
}
Self::Batch(batch) => {
f.debug_struct("VariantSignId::Batch").field("0", &hex::encode(batch)).finish()
}
Self::Batch(batch) => f.debug_struct("VariantSignId::Batch").field("0", &batch).finish(),
Self::SlashReport => f.debug_struct("VariantSignId::SlashReport").finish(),
Self::Transaction(tx) => {
f.debug_struct("VariantSignId::Transaction").field("0", &hex::encode(tx)).finish()
@@ -166,11 +164,11 @@ pub mod coordinator {
/// Cosign the specified Substrate block.
///
/// This is sent by the Coordinator's Tributary scanner.
CosignSubstrateBlock { session: Session, cosign: Cosign },
CosignSubstrateBlock { session: Session, intent: CosignIntent },
/// Sign the slash report for this session.
///
/// This is sent by the Coordinator's Tributary scanner.
SignSlashReport { session: Session, slash_report: SlashReport },
SignSlashReport { session: Session, report: Vec<Slash> },
}
// This set of messages is sent entirely and solely by serai-processor-bin's implementation of
@@ -180,7 +178,7 @@ pub mod coordinator {
pub enum ProcessorMessage {
CosignedBlock { cosign: SignedCosign },
SignedBatch { batch: SignedBatch },
SignedSlashReport { session: Session, slash_report: SlashReport, signature: [u8; 64] },
SignedSlashReport { session: Session, signature: Vec<u8> },
}
}
@@ -322,8 +320,8 @@ impl CoordinatorMessage {
CoordinatorMessage::Coordinator(msg) => {
let (sub, id) = match msg {
// We only cosign a block once, and Reattempt is a separate message
coordinator::CoordinatorMessage::CosignSubstrateBlock { cosign, .. } => {
(0, cosign.block_number.encode())
coordinator::CoordinatorMessage::CosignSubstrateBlock { intent, .. } => {
(0, intent.block_number.encode())
}
// We only sign one slash report, and Reattempt is a separate message
coordinator::CoordinatorMessage::SignSlashReport { session, .. } => (1, session.encode()),

View File

@@ -102,7 +102,6 @@ pub trait TransactionPlanner<S: ScannerFeed, A>: 'static + Send + Sync {
///
/// Returns `None` if the fee exceeded the inputs, or `Some` otherwise.
// TODO: Enum for Change of None, Some, Mandatory
#[allow(clippy::type_complexity)]
fn plan_transaction_with_fee_amortization(
&self,
operating_costs: &mut u64,

View File

@@ -40,7 +40,6 @@ serai-db = { path = "../../common/db" }
log = { version = "0.4", default-features = false, features = ["std"] }
tokio = { version = "1", default-features = false, features = ["rt-multi-thread", "sync", "time", "macros"] }
serai-cosign = { path = "../../coordinator/cosign" }
messages = { package = "serai-processor-messages", path = "../messages" }
primitives = { package = "serai-processor-primitives", path = "../primitives" }
scanner = { package = "serai-processor-scanner", path = "../scanner" }

View File

@@ -69,12 +69,7 @@ impl<D: Db, E: GroupEncoding> BatchSignerTask<D, E> {
let mut machines = Vec::with_capacity(keys.len());
for keys in &keys {
// TODO: Fetch the context for this from a constant instead of re-defining it
machines.push(WrappedSchnorrkelMachine::new(
keys.clone(),
b"substrate",
batch_message(&batch),
));
machines.push(WrappedSchnorrkelMachine::new(keys.clone(), batch_message(&batch)));
}
attempt_manager.register(VariantSignId::Batch(id), machines);
}
@@ -111,12 +106,7 @@ impl<D: Db, E: Send + GroupEncoding> ContinuallyRan for BatchSignerTask<D, E> {
let mut machines = Vec::with_capacity(self.keys.len());
for keys in &self.keys {
// TODO: Also fetch the constant here
machines.push(WrappedSchnorrkelMachine::new(
keys.clone(),
b"substrate",
batch_message(&batch),
));
machines.push(WrappedSchnorrkelMachine::new(keys.clone(), batch_message(&batch)));
}
for msg in self.attempt_manager.register(VariantSignId::Batch(batch_hash), machines) {
BatchSignerToCoordinatorMessages::send(&mut txn, self.session, &msg);

View File

@@ -1,7 +1,6 @@
use core::future::Future;
use serai_primitives::Signature;
use scale::Decode;
use serai_db::{DbTxn, Db};
use primitives::task::ContinuallyRan;
@@ -100,11 +99,17 @@ impl<D: Db, C: Coordinator> ContinuallyRan for CoordinatorTask<D, C> {
// Publish the cosigns from this session
{
let mut txn = self.db.txn();
while let Some(signed_cosign) = Cosign::try_recv(&mut txn, session) {
while let Some(((block_number, block_id), signature)) =
Cosign::try_recv(&mut txn, session)
{
iterated = true;
self
.coordinator
.publish_cosign(signed_cosign)
.publish_cosign(
block_number,
block_id,
<_>::decode(&mut signature.as_slice()).unwrap(),
)
.await
.map_err(|e| format!("couldn't publish Cosign: {e:?}"))?;
}
@@ -114,12 +119,15 @@ impl<D: Db, C: Coordinator> ContinuallyRan for CoordinatorTask<D, C> {
// If this session signed its slash report, publish its signature
{
let mut txn = self.db.txn();
if let Some((slash_report, signature)) = SignedSlashReport::try_recv(&mut txn, session) {
if let Some(slash_report_signature) = SlashReportSignature::try_recv(&mut txn, session) {
iterated = true;
self
.coordinator
.publish_slash_report_signature(session, slash_report, Signature(signature))
.publish_slash_report_signature(
session,
<_>::decode(&mut slash_report_signature.as_slice()).unwrap(),
)
.await
.map_err(|e| {
format!("couldn't send slash report signature to the coordinator: {e:?}")

View File

@@ -9,8 +9,7 @@ use serai_validator_sets_primitives::Session;
use serai_db::{DbTxn, Db};
use serai_cosign::{COSIGN_CONTEXT, Cosign as CosignStruct, SignedCosign};
use messages::sign::VariantSignId;
use messages::{sign::VariantSignId, coordinator::cosign_block_msg};
use primitives::task::{DoesNotError, ContinuallyRan};
@@ -35,7 +34,7 @@ pub(crate) struct CosignerTask<D: Db> {
session: Session,
keys: Vec<ThresholdKeys<Ristretto>>,
current_cosign: Option<CosignStruct>,
current_cosign: Option<(u64, [u8; 32])>,
attempt_manager: AttemptManager<D, WrappedSchnorrkelMachine>,
}
@@ -63,34 +62,26 @@ impl<D: Db> ContinuallyRan for CosignerTask<D> {
let mut txn = self.db.txn();
if let Some(cosign) = ToCosign::get(&txn, self.session) {
// If this wasn't already signed for...
if LatestCosigned::get(&txn, self.session) < Some(cosign.block_number) {
if LatestCosigned::get(&txn, self.session) < Some(cosign.0) {
// If this isn't the cosign we're currently working on, meaning it's fresh
if self.current_cosign.as_ref() != Some(&cosign) {
if self.current_cosign != Some(cosign) {
// Retire the current cosign
if let Some(current_cosign) = &self.current_cosign {
assert!(current_cosign.block_number < cosign.block_number);
self
.attempt_manager
.retire(&mut txn, VariantSignId::Cosign(current_cosign.block_number));
if let Some(current_cosign) = self.current_cosign {
assert!(current_cosign.0 < cosign.0);
self.attempt_manager.retire(&mut txn, VariantSignId::Cosign(current_cosign.0));
}
// Set the cosign being worked on
self.current_cosign = Some(cosign.clone());
self.current_cosign = Some(cosign);
let mut machines = Vec::with_capacity(self.keys.len());
{
let message = cosign.signature_message();
let message = cosign_block_msg(cosign.0, cosign.1);
for keys in &self.keys {
machines.push(WrappedSchnorrkelMachine::new(
keys.clone(),
COSIGN_CONTEXT,
message.clone(),
));
machines.push(WrappedSchnorrkelMachine::new(keys.clone(), message.clone()));
}
}
for msg in
self.attempt_manager.register(VariantSignId::Cosign(cosign.block_number), machines)
{
for msg in self.attempt_manager.register(VariantSignId::Cosign(cosign.0), machines) {
CosignerToCoordinatorMessages::send(&mut txn, self.session, &msg);
}
@@ -118,19 +109,12 @@ impl<D: Db> ContinuallyRan for CosignerTask<D> {
let VariantSignId::Cosign(block_number) = id else {
panic!("CosignerTask signed a non-Cosign")
};
assert_eq!(
Some(block_number),
self.current_cosign.as_ref().map(|cosign| cosign.block_number)
);
assert_eq!(Some(block_number), self.current_cosign.map(|cosign| cosign.0));
let cosign = self.current_cosign.take().unwrap();
LatestCosigned::set(&mut txn, self.session, &cosign.block_number);
let cosign = SignedCosign {
cosign,
signature: Signature::from(signature).encode().try_into().unwrap(),
};
LatestCosigned::set(&mut txn, self.session, &cosign.0);
// Send the cosign
Cosign::send(&mut txn, self.session, &cosign);
Cosign::send(&mut txn, self.session, &(cosign, Signature::from(signature).encode()));
}
}

View File

@@ -1,9 +1,7 @@
use serai_validator_sets_primitives::{Session, SlashReport as SlashReportStruct};
use serai_validator_sets_primitives::{Session, Slash};
use serai_db::{Get, DbTxn, create_db, db_channel};
use serai_cosign::{Cosign as CosignStruct, SignedCosign};
use messages::sign::{ProcessorMessage, CoordinatorMessage};
create_db! {
@@ -13,16 +11,16 @@ create_db! {
LatestRetiredSession: () -> Session,
ToCleanup: () -> Vec<(Session, Vec<u8>)>,
ToCosign: (session: Session) -> CosignStruct,
ToCosign: (session: Session) -> (u64, [u8; 32]),
}
}
db_channel! {
SignersGlobal {
Cosign: (session: Session) -> SignedCosign,
Cosign: (session: Session) -> ((u64, [u8; 32]), Vec<u8>),
SlashReport: (session: Session) -> SlashReportStruct,
SignedSlashReport: (session: Session) -> (SlashReportStruct, [u8; 64]),
SlashReport: (session: Session) -> Vec<Slash>,
SlashReportSignature: (session: Session) -> Vec<u8>,
/*
TODO: Most of these are pointless? We drop all active signing sessions on reboot. It's

View File

@@ -11,13 +11,11 @@ use ciphersuite::{group::GroupEncoding, Ciphersuite, Ristretto};
use frost::dkg::{ThresholdCore, ThresholdKeys};
use serai_primitives::Signature;
use serai_validator_sets_primitives::{Session, SlashReport};
use serai_validator_sets_primitives::{Session, Slash};
use serai_in_instructions_primitives::SignedBatch;
use serai_db::{DbTxn, Db};
use serai_cosign::{Cosign, SignedCosign};
use messages::sign::{VariantSignId, ProcessorMessage, CoordinatorMessage};
use primitives::task::{Task, TaskHandle, ContinuallyRan};
@@ -61,7 +59,9 @@ pub trait Coordinator: 'static + Send + Sync {
/// Publish a cosign.
fn publish_cosign(
&mut self,
signed_cosign: SignedCosign,
block_number: u64,
block_id: [u8; 32],
signature: Signature,
) -> impl Send + Future<Output = Result<(), Self::EphemeralError>>;
/// Publish a `SignedBatch`.
@@ -74,7 +74,6 @@ pub trait Coordinator: 'static + Send + Sync {
fn publish_slash_report_signature(
&mut self,
session: Session,
slash_report: SlashReport,
signature: Signature,
) -> impl Send + Future<Output = Result<(), Self::EphemeralError>>;
}
@@ -409,13 +408,19 @@ impl<
/// Cosign a block.
///
/// This is a cheap call and able to be done inline from a higher-level loop.
pub fn cosign_block(&mut self, mut txn: impl DbTxn, session: Session, cosign: &Cosign) {
pub fn cosign_block(
&mut self,
mut txn: impl DbTxn,
session: Session,
block_number: u64,
block: [u8; 32],
) {
// Don't cosign blocks with already retired keys
if Some(session.0) <= db::LatestRetiredSession::get(&txn).map(|session| session.0) {
return;
}
db::ToCosign::set(&mut txn, session, cosign);
db::ToCosign::set(&mut txn, session, &(block_number, block));
txn.commit();
if let Some(tasks) = self.tasks.get(&session) {
@@ -430,7 +435,7 @@ impl<
&mut self,
mut txn: impl DbTxn,
session: Session,
slash_report: &SlashReport,
slash_report: &Vec<Slash>,
) {
// Don't sign slash reports with already retired keys
if Some(session.0) <= db::LatestRetiredSession::get(&txn).map(|session| session.0) {

View File

@@ -3,8 +3,11 @@ use core::{marker::PhantomData, future::Future};
use ciphersuite::Ristretto;
use frost::dkg::ThresholdKeys;
use scale::Encode;
use serai_primitives::Signature;
use serai_validator_sets_primitives::Session;
use serai_validator_sets_primitives::{
Session, ValidatorSet, SlashReport as SlashReportStruct, report_slashes_message,
};
use serai_db::{DbTxn, Db};
@@ -17,7 +20,7 @@ use frost_attempt_manager::*;
use crate::{
db::{
SlashReport, SignedSlashReport, CoordinatorToSlashReportSignerMessages,
SlashReport, SlashReportSignature, CoordinatorToSlashReportSignerMessages,
SlashReportSignerToCoordinatorMessages,
},
WrappedSchnorrkelMachine,
@@ -69,14 +72,12 @@ impl<D: Db, S: ScannerFeed> ContinuallyRan for SlashReportSignerTask<D, S> {
let mut machines = Vec::with_capacity(self.keys.len());
{
let message = slash_report.report_slashes_message();
let message = report_slashes_message(
&ValidatorSet { network: S::NETWORK, session: self.session },
&SlashReportStruct(slash_report.try_into().unwrap()),
);
for keys in &self.keys {
// TODO: Fetch this constant from somewhere instead of inlining it
machines.push(WrappedSchnorrkelMachine::new(
keys.clone(),
b"substrate",
message.clone(),
));
machines.push(WrappedSchnorrkelMachine::new(keys.clone(), message.clone()));
}
}
let mut txn = self.db.txn();
@@ -104,12 +105,12 @@ impl<D: Db, S: ScannerFeed> ContinuallyRan for SlashReportSignerTask<D, S> {
Response::Signature { id, signature } => {
assert_eq!(id, VariantSignId::SlashReport);
// Drain the channel
let slash_report = SlashReport::try_recv(&mut txn, self.session).unwrap();
SlashReport::try_recv(&mut txn, self.session).unwrap();
// Send the signature
SignedSlashReport::send(
SlashReportSignature::send(
&mut txn,
self.session,
&(slash_report, Signature::from(signature).0),
&Signature::from(signature).encode(),
);
}
}

View File

@@ -16,10 +16,10 @@ use frost_schnorrkel::Schnorrkel;
// This wraps a Schnorrkel sign machine into one with a preset message.
#[derive(Clone)]
pub(crate) struct WrappedSchnorrkelMachine(ThresholdKeys<Ristretto>, &'static [u8], Vec<u8>);
pub(crate) struct WrappedSchnorrkelMachine(ThresholdKeys<Ristretto>, Vec<u8>);
impl WrappedSchnorrkelMachine {
pub(crate) fn new(keys: ThresholdKeys<Ristretto>, context: &'static [u8], msg: Vec<u8>) -> Self {
Self(keys, context, msg)
pub(crate) fn new(keys: ThresholdKeys<Ristretto>, msg: Vec<u8>) -> Self {
Self(keys, msg)
}
}
@@ -39,10 +39,10 @@ impl PreprocessMachine for WrappedSchnorrkelMachine {
rng: &mut R,
) -> (Self::SignMachine, Preprocess<Ristretto, <Schnorrkel as Algorithm<Ristretto>>::Addendum>)
{
let WrappedSchnorrkelMachine(keys, context, msg) = self;
let WrappedSchnorrkelMachine(keys, batch) = self;
let (machine, preprocess) =
AlgorithmMachine::new(Schnorrkel::new(context), keys).preprocess(rng);
(WrappedSchnorrkelSignMachine(machine, msg), preprocess)
AlgorithmMachine::new(Schnorrkel::new(b"substrate"), keys).preprocess(rng);
(WrappedSchnorrkelSignMachine(machine, batch), preprocess)
}
}

View File

@@ -21,7 +21,7 @@ pub enum Call {
},
report_slashes {
network: NetworkId,
slashes: SlashReport,
slashes: BoundedVec<(SeraiAddress, u32), ConstU32<{ MAX_KEY_SHARES_PER_SET_U32 / 3 }>>,
signature: Signature,
},
allocate {

View File

@@ -12,7 +12,7 @@ pub type CoinsEvent = serai_abi::coins::Event;
#[derive(Clone, Copy)]
pub struct SeraiCoins<'a>(pub(crate) &'a TemporalSerai<'a>);
impl SeraiCoins<'_> {
impl<'a> SeraiCoins<'a> {
pub async fn mint_events(&self) -> Result<Vec<CoinsEvent>, SeraiError> {
self
.0

View File

@@ -9,7 +9,7 @@ const PALLET: &str = "Dex";
#[derive(Clone, Copy)]
pub struct SeraiDex<'a>(pub(crate) &'a TemporalSerai<'a>);
impl SeraiDex<'_> {
impl<'a> SeraiDex<'a> {
pub async fn events(&self) -> Result<Vec<DexEvent>, SeraiError> {
self
.0

View File

@@ -15,7 +15,7 @@ const PALLET: &str = "GenesisLiquidity";
#[derive(Clone, Copy)]
pub struct SeraiGenesisLiquidity<'a>(pub(crate) &'a TemporalSerai<'a>);
impl SeraiGenesisLiquidity<'_> {
impl<'a> SeraiGenesisLiquidity<'a> {
pub async fn events(&self) -> Result<Vec<GenesisLiquidityEvent>, SeraiError> {
self
.0

View File

@@ -9,7 +9,7 @@ const PALLET: &str = "InInstructions";
#[derive(Clone, Copy)]
pub struct SeraiInInstructions<'a>(pub(crate) &'a TemporalSerai<'a>);
impl SeraiInInstructions<'_> {
impl<'a> SeraiInInstructions<'a> {
pub async fn last_batch_for_network(
&self,
network: NetworkId,

View File

@@ -8,7 +8,7 @@ const PALLET: &str = "LiquidityTokens";
#[derive(Clone, Copy)]
pub struct SeraiLiquidityTokens<'a>(pub(crate) &'a TemporalSerai<'a>);
impl SeraiLiquidityTokens<'_> {
impl<'a> SeraiLiquidityTokens<'a> {
pub async fn token_supply(&self, coin: Coin) -> Result<Amount, SeraiError> {
Ok(self.0.storage(PALLET, "Supply", coin).await?.unwrap_or(Amount(0)))
}

View File

@@ -80,7 +80,7 @@ pub struct TemporalSerai<'a> {
block: [u8; 32],
events: RwLock<Option<EventsInBlock>>,
}
impl Clone for TemporalSerai<'_> {
impl<'a> Clone for TemporalSerai<'a> {
fn clone(&self) -> Self {
Self { serai: self.serai, block: self.block, events: RwLock::new(None) }
}
@@ -319,7 +319,7 @@ impl Serai {
}
}
impl TemporalSerai<'_> {
impl<'a> TemporalSerai<'a> {
async fn events<E>(
&self,
filter_map: impl Fn(&Event) -> Option<E>,
@@ -389,27 +389,27 @@ impl TemporalSerai<'_> {
})
}
pub fn coins(&self) -> SeraiCoins<'_> {
pub fn coins(&'a self) -> SeraiCoins<'a> {
SeraiCoins(self)
}
pub fn dex(&self) -> SeraiDex<'_> {
pub fn dex(&'a self) -> SeraiDex<'a> {
SeraiDex(self)
}
pub fn in_instructions(&self) -> SeraiInInstructions<'_> {
pub fn in_instructions(&'a self) -> SeraiInInstructions<'a> {
SeraiInInstructions(self)
}
pub fn validator_sets(&self) -> SeraiValidatorSets<'_> {
pub fn validator_sets(&'a self) -> SeraiValidatorSets<'a> {
SeraiValidatorSets(self)
}
pub fn genesis_liquidity(&self) -> SeraiGenesisLiquidity {
pub fn genesis_liquidity(&'a self) -> SeraiGenesisLiquidity {
SeraiGenesisLiquidity(self)
}
pub fn liquidity_tokens(&self) -> SeraiLiquidityTokens {
pub fn liquidity_tokens(&'a self) -> SeraiLiquidityTokens {
SeraiLiquidityTokens(self)
}
}

View File

@@ -5,10 +5,10 @@ use sp_runtime::BoundedVec;
use serai_abi::primitives::Amount;
pub use serai_abi::validator_sets::primitives;
use primitives::{MAX_KEY_LEN, Session, ValidatorSet, KeyPair, SlashReport};
use primitives::{MAX_KEY_LEN, Session, ValidatorSet, KeyPair};
use crate::{
primitives::{EmbeddedEllipticCurve, NetworkId},
primitives::{EmbeddedEllipticCurve, NetworkId, SeraiAddress},
Transaction, Serai, TemporalSerai, SeraiError,
};
@@ -18,7 +18,7 @@ pub type ValidatorSetsEvent = serai_abi::validator_sets::Event;
#[derive(Clone, Copy)]
pub struct SeraiValidatorSets<'a>(pub(crate) &'a TemporalSerai<'a>);
impl SeraiValidatorSets<'_> {
impl<'a> SeraiValidatorSets<'a> {
pub async fn new_set_events(&self) -> Result<Vec<ValidatorSetsEvent>, SeraiError> {
self
.0
@@ -238,7 +238,12 @@ impl SeraiValidatorSets<'_> {
pub fn report_slashes(
network: NetworkId,
slashes: SlashReport,
// TODO: This bounds a maximum length but takes more space than just publishing all the u32s
// (50 * (32 + 4)) > (150 * 4)
slashes: sp_runtime::BoundedVec<
(SeraiAddress, u32),
sp_core::ConstU32<{ primitives::MAX_KEY_SHARES_PER_SET_U32 / 3 }>,
>,
signature: Signature,
) -> Transaction {
Serai::unsigned(serai_abi::Call::ValidatorSets(

View File

@@ -20,71 +20,71 @@ workspace = true
name = "serai-node"
[dependencies]
#rand_core = "0.6"
#zeroize = "1"
#hex = "0.4"
#log = "0.4"
rand_core = "0.6"
zeroize = "1"
hex = "0.4"
log = "0.4"
#schnorrkel = "0.11"
schnorrkel = "0.11"
#ciphersuite = { path = "../../crypto/ciphersuite" }
#embedwards25519 = { path = "../../crypto/evrf/embedwards25519" }
#secq256k1 = { path = "../../crypto/evrf/secq256k1" }
ciphersuite = { path = "../../crypto/ciphersuite" }
embedwards25519 = { path = "../../crypto/evrf/embedwards25519" }
secq256k1 = { path = "../../crypto/evrf/secq256k1" }
#libp2p = "0.52"
libp2p = "0.52"
#sp-core = { git = "https://github.com/serai-dex/substrate" }
#sp-keystore = { git = "https://github.com/serai-dex/substrate" }
#sp-timestamp = { git = "https://github.com/serai-dex/substrate" }
#sp-io = { git = "https://github.com/serai-dex/substrate" }
#sp-blockchain = { git = "https://github.com/serai-dex/substrate" }
#sp-api = { git = "https://github.com/serai-dex/substrate" }
#sp-block-builder = { git = "https://github.com/serai-dex/substrate" }
#sp-consensus-babe = { git = "https://github.com/serai-dex/substrate" }
sp-core = { git = "https://github.com/serai-dex/substrate" }
sp-keystore = { git = "https://github.com/serai-dex/substrate" }
sp-timestamp = { git = "https://github.com/serai-dex/substrate" }
sp-io = { git = "https://github.com/serai-dex/substrate" }
sp-blockchain = { git = "https://github.com/serai-dex/substrate" }
sp-api = { git = "https://github.com/serai-dex/substrate" }
sp-block-builder = { git = "https://github.com/serai-dex/substrate" }
sp-consensus-babe = { git = "https://github.com/serai-dex/substrate" }
#frame-benchmarking = { git = "https://github.com/serai-dex/substrate" }
frame-benchmarking = { git = "https://github.com/serai-dex/substrate" }
#serai-runtime = { path = "../runtime", features = ["std"] }
serai-runtime = { path = "../runtime", features = ["std"] }
#clap = { version = "4", features = ["derive"] }
clap = { version = "4", features = ["derive"] }
#futures-util = "0.3"
#tokio = { version = "1", features = ["sync", "rt-multi-thread"] }
#jsonrpsee = { version = "0.16", features = ["server"] }
futures-util = "0.3"
tokio = { version = "1", features = ["sync", "rt-multi-thread"] }
jsonrpsee = { version = "0.16", features = ["server"] }
#sc-offchain = { git = "https://github.com/serai-dex/substrate" }
#sc-transaction-pool = { git = "https://github.com/serai-dex/substrate" }
#sc-transaction-pool-api = { git = "https://github.com/serai-dex/substrate" }
#sc-basic-authorship = { git = "https://github.com/serai-dex/substrate" }
#sc-executor = { git = "https://github.com/serai-dex/substrate" }
#sc-service = { git = "https://github.com/serai-dex/substrate" }
#sc-client-api = { git = "https://github.com/serai-dex/substrate" }
#sc-network-common = { git = "https://github.com/serai-dex/substrate" }
#sc-network = { git = "https://github.com/serai-dex/substrate" }
sc-offchain = { git = "https://github.com/serai-dex/substrate" }
sc-transaction-pool = { git = "https://github.com/serai-dex/substrate" }
sc-transaction-pool-api = { git = "https://github.com/serai-dex/substrate" }
sc-basic-authorship = { git = "https://github.com/serai-dex/substrate" }
sc-executor = { git = "https://github.com/serai-dex/substrate" }
sc-service = { git = "https://github.com/serai-dex/substrate" }
sc-client-api = { git = "https://github.com/serai-dex/substrate" }
sc-network-common = { git = "https://github.com/serai-dex/substrate" }
sc-network = { git = "https://github.com/serai-dex/substrate" }
#sc-consensus = { git = "https://github.com/serai-dex/substrate" }
#sc-consensus-babe = { git = "https://github.com/serai-dex/substrate" }
#sc-consensus-grandpa = { git = "https://github.com/serai-dex/substrate" }
#sc-authority-discovery = { git = "https://github.com/serai-dex/substrate" }
sc-consensus = { git = "https://github.com/serai-dex/substrate" }
sc-consensus-babe = { git = "https://github.com/serai-dex/substrate" }
sc-consensus-grandpa = { git = "https://github.com/serai-dex/substrate" }
sc-authority-discovery = { git = "https://github.com/serai-dex/substrate" }
#sc-telemetry = { git = "https://github.com/serai-dex/substrate" }
#sc-cli = { git = "https://github.com/serai-dex/substrate" }
sc-telemetry = { git = "https://github.com/serai-dex/substrate" }
sc-cli = { git = "https://github.com/serai-dex/substrate" }
#sc-rpc-api = { git = "https://github.com/serai-dex/substrate" }
sc-rpc-api = { git = "https://github.com/serai-dex/substrate" }
#substrate-frame-rpc-system = { git = "https://github.com/serai-dex/substrate" }
#pallet-transaction-payment-rpc = { git = "https://github.com/serai-dex/substrate" }
substrate-frame-rpc-system = { git = "https://github.com/serai-dex/substrate" }
pallet-transaction-payment-rpc = { git = "https://github.com/serai-dex/substrate" }
#serai-env = { path = "../../common/env" }
serai-env = { path = "../../common/env" }
[build-dependencies]
#substrate-build-script-utils = { git = "https://github.com/serai-dex/substrate" }
substrate-build-script-utils = { git = "https://github.com/serai-dex/substrate" }
[features]
#default = []
#fast-epoch = ["serai-runtime/fast-epoch"]
#runtime-benchmarks = [
# "frame-benchmarking/runtime-benchmarks",
default = []
fast-epoch = ["serai-runtime/fast-epoch"]
runtime-benchmarks = [
"frame-benchmarking/runtime-benchmarks",
# "serai-runtime/runtime-benchmarks",
#]
"serai-runtime/runtime-benchmarks",
]

View File

@@ -111,7 +111,13 @@ impl From<Call> for RuntimeCall {
serai_abi::validator_sets::Call::report_slashes { network, slashes, signature } => {
RuntimeCall::ValidatorSets(validator_sets::Call::report_slashes {
network,
slashes,
slashes: <_>::try_from(
slashes
.into_iter()
.map(|(addr, slash)| (PublicKey::from(addr), slash))
.collect::<Vec<_>>(),
)
.unwrap(),
signature,
})
}
@@ -295,7 +301,17 @@ impl TryInto<Call> for RuntimeCall {
}
}
validator_sets::Call::report_slashes { network, slashes, signature } => {
serai_abi::validator_sets::Call::report_slashes { network, slashes, signature }
serai_abi::validator_sets::Call::report_slashes {
network,
slashes: <_>::try_from(
slashes
.into_iter()
.map(|(addr, slash)| (SeraiAddress::from(addr), slash))
.collect::<Vec<_>>(),
)
.unwrap(),
signature,
}
}
validator_sets::Call::allocate { network, amount } => {
serai_abi::validator_sets::Call::allocate { network, amount }

View File

@@ -1010,7 +1010,7 @@ pub mod pallet {
pub fn report_slashes(
origin: OriginFor<T>,
network: NetworkId,
slashes: SlashReport,
slashes: BoundedVec<(Public, u32), ConstU32<{ MAX_KEY_SHARES_PER_SET_U32 / 3 }>>,
signature: Signature,
) -> DispatchResult {
ensure_none(origin)?;

View File

@@ -210,36 +210,9 @@ impl Slash {
#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
pub struct SlashReport(pub BoundedVec<Slash, ConstU32<{ MAX_KEY_SHARES_PER_SET_U32 }>>);
#[cfg(feature = "borsh")]
impl BorshSerialize for SlashReport {
fn serialize<W: borsh::io::Write>(&self, writer: &mut W) -> borsh::io::Result<()> {
BorshSerialize::serialize(self.0.as_slice(), writer)
}
}
#[cfg(feature = "borsh")]
impl BorshDeserialize for SlashReport {
fn deserialize_reader<R: borsh::io::Read>(reader: &mut R) -> borsh::io::Result<Self> {
let slashes = Vec::<Slash>::deserialize_reader(reader)?;
slashes
.try_into()
.map(Self)
.map_err(|_| borsh::io::Error::other("length of slash report exceeds max validators"))
}
}
impl TryFrom<Vec<Slash>> for SlashReport {
type Error = &'static str;
fn try_from(slashes: Vec<Slash>) -> Result<SlashReport, &'static str> {
slashes.try_into().map(Self).map_err(|_| "length of slash report exceeds max validators")
}
}
impl SlashReport {
/// The message to sign when publishing this SlashReport.
// This is assumed binding to the ValidatorSet via the key signed with
pub fn report_slashes_message(&self) -> Vec<u8> {
(b"ValidatorSets-report_slashes", &self.0).encode()
}
// This is assumed binding to the ValidatorSet via the key signed with
pub fn report_slashes_message(slashes: &SlashReport) -> Vec<u8> {
(b"ValidatorSets-report_slashes", slashes).encode()
}
#[test]