Remove the RemoveParticipant protocol for having new DKGs specify the participants which were removed

Obvious code cleanup is obvious.
This commit is contained in:
Luke Parker
2023-12-14 23:45:15 -05:00
parent b60e3c2524
commit 2532423d42
17 changed files with 144 additions and 615 deletions

1
Cargo.lock generated
View File

@@ -7735,6 +7735,7 @@ version = "0.1.0"
dependencies = [ dependencies = [
"frame-support", "frame-support",
"frame-system", "frame-system",
"hashbrown 0.14.3",
"pallet-babe", "pallet-babe",
"pallet-grandpa", "pallet-grandpa",
"parity-scale-codec", "parity-scale-codec",

View File

@@ -28,7 +28,7 @@ use tokio::{sync::mpsc, time::sleep};
use crate::{ use crate::{
Db, Db,
processors::Processors, processors::Processors,
tributary::{TributarySpec, SeraiDkgRemoval, SeraiDkgCompleted}, tributary::{TributarySpec, SeraiDkgCompleted},
}; };
mod db; mod db;
@@ -222,19 +222,6 @@ async fn handle_block<D: Db, Pro: Processors>(
// Define an indexed event ID. // Define an indexed event ID.
let mut event_id = 0; let mut event_id = 0;
if HandledEvent::is_unhandled(db, hash, event_id) {
let mut txn = db.txn();
for removal in serai.as_of(hash).validator_sets().participant_removed_events().await? {
let ValidatorSetsEvent::ParticipantRemoved { set, removed } = removal else {
panic!("ParticipantRemoved event wasn't ParticipantRemoved: {removal:?}");
};
SeraiDkgRemoval::set(&mut txn, set, removed.0, &());
}
HandledEvent::handle_event(&mut txn, hash, event_id);
txn.commit();
}
event_id += 1;
// If a new validator set was activated, create tributary/inform processor to do a DKG // If a new validator set was activated, create tributary/inform processor to do a DKG
for new_set in serai.as_of(hash).validator_sets().new_set_events().await? { for new_set in serai.as_of(hash).validator_sets().new_set_events().await? {
// Individually mark each event as handled so on reboot, we minimize duplicates // Individually mark each event as handled so on reboot, we minimize duplicates

View File

@@ -347,11 +347,18 @@ async fn dkg_test() {
} }
#[async_trait::async_trait] #[async_trait::async_trait]
impl PublishSeraiTransaction for CheckPublishSetKeys { impl PublishSeraiTransaction for CheckPublishSetKeys {
async fn publish_set_keys(&self, set: ValidatorSet, key_pair: KeyPair, signature: Signature) { async fn publish_set_keys(
&self,
set: ValidatorSet,
removed: Vec<SeraiAddress>,
key_pair: KeyPair,
signature: Signature,
) {
assert_eq!(set, self.spec.set()); assert_eq!(set, self.spec.set());
assert!(removed.is_empty());
assert_eq!(self.key_pair, key_pair); assert_eq!(self.key_pair, key_pair);
assert!(signature.verify( assert!(signature.verify(
&*serai_client::validator_sets::primitives::set_keys_message(&set, &key_pair), &*serai_client::validator_sets::primitives::set_keys_message(&set, &[], &key_pair),
&serai_client::Public( &serai_client::Public(
frost::dkg::musig::musig_key::<Ristretto>( frost::dkg::musig::musig_key::<Ristretto>(
&serai_client::validator_sets::primitives::musig_context(set), &serai_client::validator_sets::primitives::musig_context(set),
@@ -362,16 +369,6 @@ async fn dkg_test() {
), ),
)); ));
} }
async fn publish_remove_participant(
&self,
set: ValidatorSet,
removing: [u8; 32],
signers: Vec<SeraiAddress>,
signature: Signature,
) {
().publish_remove_participant(set, removing, signers, signature).await
}
} }
// The scanner should successfully try to publish a transaction with a validly signed signature // The scanner should successfully try to publish a transaction with a validly signed signature

View File

@@ -26,17 +26,14 @@ mod sync;
#[async_trait::async_trait] #[async_trait::async_trait]
impl PublishSeraiTransaction for () { impl PublishSeraiTransaction for () {
async fn publish_set_keys(&self, _set: ValidatorSet, _key_pair: KeyPair, _signature: Signature) { async fn publish_set_keys(
panic!("publish_set_keys was called in test")
}
async fn publish_remove_participant(
&self, &self,
_set: ValidatorSet, _set: ValidatorSet,
_removing: [u8; 32], _removed: Vec<SeraiAddress>,
_signers: Vec<SeraiAddress>, _key_pair: KeyPair,
_signature: Signature, _signature: Signature,
) { ) {
panic!("publish_remove_participant was called in test") panic!("publish_set_keys was called in test")
} }
} }
@@ -226,17 +223,6 @@ fn serialize_transaction() {
signed: random_signed_with_nonce(&mut OsRng, 2), signed: random_signed_with_nonce(&mut OsRng, 2),
}); });
{
let mut key = [0; 32];
OsRng.fill_bytes(&mut key);
test_read_write(Transaction::DkgRemoval(random_sign_data(&mut OsRng, key, Label::Preprocess)));
}
{
let mut key = [0; 32];
OsRng.fill_bytes(&mut key);
test_read_write(Transaction::DkgRemoval(random_sign_data(&mut OsRng, key, Label::Share)));
}
{ {
let mut block = [0; 32]; let mut block = [0; 32];
OsRng.fill_bytes(&mut block); OsRng.fill_bytes(&mut block);

View File

@@ -19,7 +19,6 @@ use crate::tributary::{Label, Transaction};
pub enum Topic { pub enum Topic {
Dkg, Dkg,
DkgConfirmation, DkgConfirmation,
DkgRemoval([u8; 32]),
SubstrateSign(SubstrateSignableId), SubstrateSign(SubstrateSignableId),
Sign([u8; 32]), Sign([u8; 32]),
} }
@@ -46,14 +45,12 @@ pub enum Accumulation {
create_db!( create_db!(
Tributary { Tributary {
SeraiBlockNumber: (hash: [u8; 32]) -> u64, SeraiBlockNumber: (hash: [u8; 32]) -> u64,
SeraiDkgRemoval: (spec: ValidatorSet, removing: [u8; 32]) -> (),
SeraiDkgCompleted: (spec: ValidatorSet) -> [u8; 32], SeraiDkgCompleted: (spec: ValidatorSet) -> [u8; 32],
TributaryBlockNumber: (block: [u8; 32]) -> u32, TributaryBlockNumber: (block: [u8; 32]) -> u32,
LastHandledBlock: (genesis: [u8; 32]) -> [u8; 32], LastHandledBlock: (genesis: [u8; 32]) -> [u8; 32],
FatalSlashes: (genesis: [u8; 32]) -> Vec<[u8; 32]>, FatalSlashes: (genesis: [u8; 32]) -> Vec<[u8; 32]>,
FatalSlashesAsOfDkgAttempt: (genesis: [u8; 32], attempt: u32) -> Vec<[u8; 32]>, FatalSlashesAsOfDkgAttempt: (genesis: [u8; 32], attempt: u32) -> Vec<[u8; 32]>,
FatalSlashesAsOfFatalSlash: (genesis: [u8; 32], fatally_slashed: [u8; 32]) -> Vec<[u8; 32]>,
FatallySlashed: (genesis: [u8; 32], account: [u8; 32]) -> (), FatallySlashed: (genesis: [u8; 32], account: [u8; 32]) -> (),
DkgShare: (genesis: [u8; 32], from: u16, to: u16) -> Vec<u8>, DkgShare: (genesis: [u8; 32], from: u16, to: u16) -> Vec<u8>,
PlanIds: (genesis: &[u8], block: u64) -> Vec<[u8; 32]>, PlanIds: (genesis: &[u8], block: u64) -> Vec<[u8; 32]>,
@@ -85,7 +82,6 @@ impl FatallySlashed {
existing.push(account); existing.push(account);
FatalSlashes::set(txn, genesis, &existing); FatalSlashes::set(txn, genesis, &existing);
FatalSlashesAsOfFatalSlash::set(txn, genesis, account, &existing);
} }
} }

View File

@@ -1,8 +1,6 @@
use core::ops::Deref; use core::ops::Deref;
use std::collections::HashMap; use std::collections::HashMap;
use rand_core::OsRng;
use zeroize::{Zeroize, Zeroizing}; use zeroize::{Zeroize, Zeroizing};
use ciphersuite::{group::GroupEncoding, Ciphersuite, Ristretto}; use ciphersuite::{group::GroupEncoding, Ciphersuite, Ristretto};
@@ -25,7 +23,7 @@ use crate::{
processors::Processors, processors::Processors,
tributary::{ tributary::{
*, *,
signing_protocol::{DkgConfirmer, DkgRemoval}, signing_protocol::DkgConfirmer,
scanner::{ scanner::{
RecognizedIdType, RIDTrait, PublishSeraiTransaction, PTTTrait, TributaryBlockHandler, RecognizedIdType, RIDTrait, PublishSeraiTransaction, PTTTrait, TributaryBlockHandler,
}, },
@@ -278,21 +276,6 @@ impl<
Ok(()) Ok(())
} }
fn dkg_removal<'a>(
&'a mut self,
removed: &'a [<Ristretto as Ciphersuite>::G],
data: &'a SignData<[u8; 32]>,
) -> DkgRemoval<'a, T> {
DkgRemoval {
key: self.our_key,
spec: self.spec,
txn: self.txn,
removed,
removing: data.plan,
attempt: data.attempt,
}
}
// TODO: Don't call fatal_slash in here, return the party to fatal_slash to ensure no further // TODO: Don't call fatal_slash in here, return the party to fatal_slash to ensure no further
// execution occurs // execution occurs
pub(crate) async fn handle_application_tx(&mut self, tx: Transaction) { pub(crate) async fn handle_application_tx(&mut self, tx: Transaction) {
@@ -582,7 +565,15 @@ impl<
DkgCompleted::set(self.txn, genesis, &()); DkgCompleted::set(self.txn, genesis, &());
self.publish_serai_tx.publish_set_keys(self.spec.set(), key_pair, Signature(sig)).await; self
.publish_serai_tx
.publish_set_keys(
self.spec.set(),
removed.into_iter().map(|key| key.to_bytes().into()).collect(),
key_pair,
Signature(sig),
)
.await;
} }
Accumulation::Ready(DataSet::NotParticipating) => { Accumulation::Ready(DataSet::NotParticipating) => {
panic!("wasn't a participant in DKG confirmination shares") panic!("wasn't a participant in DKG confirmination shares")
@@ -591,98 +582,6 @@ impl<
} }
} }
Transaction::DkgRemoval(data) => {
let signer = data.signed.signer;
let expected_len = match data.label {
Label::Preprocess => 64,
Label::Share => 32,
};
if (data.data.len() != 1) || (data.data[0].len() != expected_len) {
self.fatal_slash(signer.to_bytes(), "unexpected length data for dkg removal").await;
return;
}
let Some(removed) =
crate::tributary::removed_as_of_fatal_slash(self.txn, genesis, data.plan)
else {
self.fatal_slash(signer.to_bytes(), "removing someone who wasn't fatally slashed").await;
return;
};
let data_spec = DataSpecification {
topic: Topic::DkgRemoval(data.plan),
label: data.label,
attempt: data.attempt,
};
let Accumulation::Ready(DataSet::Participating(results)) =
self.handle_data(&removed, &data_spec, data.data.encode(), &data.signed).await
else {
return;
};
match data.label {
Label::Preprocess => {
RemovalNonces::set(self.txn, genesis, data.plan, data.attempt, &results);
let Ok(share) = self.dkg_removal(&removed, &data).share(results) else {
// TODO: Locally increase slash points to maximum (distinct from an explicitly fatal
// slash) and censor transactions (yet don't explicitly ban)
return;
};
let mut tx = Transaction::DkgRemoval(SignData {
plan: data.plan,
attempt: data.attempt,
label: Label::Preprocess,
data: vec![share.to_vec()],
signed: Transaction::empty_signed(),
});
tx.sign(&mut OsRng, genesis, self.our_key);
self.publish_tributary_tx.publish_tributary_tx(tx).await;
}
Label::Share => {
let preprocesses =
RemovalNonces::get(self.txn, genesis, data.plan, data.attempt).unwrap();
let Ok((signers, signature)) =
self.dkg_removal(&removed, &data).complete(preprocesses, results)
else {
// TODO: Locally increase slash points to maximum (distinct from an explicitly fatal
// slash) and censor transactions (yet don't explicitly ban)
return;
};
// We need to only handle this if we're not actively removing any of the signers
// At the start of this function, we only handle messages from non-fatally slashed
// participants, so this is held
//
// The created Substrate call will fail if a removed validator was one of the signers
// Since:
// 1) publish_serai_tx will block this task until the TX is published
// 2) We won't scan any more TXs/blocks until we handle this TX
// The TX *must* be successfully published *before* we start removing any more
// signers
//
// Accordingly, if the signers aren't currently being removed, they won't be removed
// by the time this transaction is successfully published *unless* a malicious 34%
// participates with the non-participating 33% to continue operation and produce a
// distinct removal (since the non-participating won't block in this block)
//
// This breaks BFT and is accordingly within bounds
// TODO: The above isn't true. It blocks until the TX is published, not included the
// finalized chain. We just need to inline remove_participant into set_keys to avoid
// all of this.
LocallyDkgRemoved::set(self.txn, genesis, data.plan, &());
self
.publish_serai_tx
.publish_remove_participant(self.spec.set(), data.plan, signers, Signature(signature))
.await;
}
}
}
Transaction::CosignSubstrateBlock(hash) => { Transaction::CosignSubstrateBlock(hash) => {
AttemptDb::recognize_topic( AttemptDb::recognize_topic(
self.txn, self.txn,

View File

@@ -66,16 +66,6 @@ pub fn removed_as_of_set_keys(
removed_as_of_dkg_attempt(getter, genesis, attempt) removed_as_of_dkg_attempt(getter, genesis, attempt)
} }
pub fn removed_as_of_fatal_slash(
getter: &impl Get,
genesis: [u8; 32],
fatally_slashed: [u8; 32],
) -> Option<Vec<<Ristretto as Ciphersuite>::G>> {
FatalSlashesAsOfFatalSlash::get(getter, genesis, fatally_slashed).map(|keys| {
keys.iter().map(|key| <Ristretto as Ciphersuite>::G::from_bytes(key).unwrap()).collect()
})
}
pub async fn publish_signed_transaction<D: Db, P: crate::P2p>( pub async fn publish_signed_transaction<D: Db, P: crate::P2p>(
txn: &mut D::Transaction<'_>, txn: &mut D::Transaction<'_>,
tributary: &Tributary<D, Transaction, P>, tributary: &Tributary<D, Transaction, P>,

View File

@@ -1,8 +1,6 @@
use core::{marker::PhantomData, ops::Deref, future::Future, time::Duration}; use core::{marker::PhantomData, ops::Deref, future::Future, time::Duration};
use std::sync::Arc; use std::sync::Arc;
use rand_core::OsRng;
use zeroize::Zeroizing; use zeroize::Zeroizing;
use ciphersuite::{group::GroupEncoding, Ciphersuite, Ristretto}; use ciphersuite::{group::GroupEncoding, Ciphersuite, Ristretto};
@@ -29,13 +27,7 @@ use tributary::{
}, },
}; };
use crate::{ use crate::{Db, processors::Processors, substrate::BatchInstructionsHashDb, tributary::*, P2p};
Db,
processors::Processors,
substrate::BatchInstructionsHashDb,
tributary::{*, signing_protocol::*},
P2p,
};
#[derive(Clone, Copy, PartialEq, Eq, Debug, Encode, Decode)] #[derive(Clone, Copy, PartialEq, Eq, Debug, Encode, Decode)]
pub enum RecognizedIdType { pub enum RecognizedIdType {
@@ -72,12 +64,11 @@ impl<
#[async_trait::async_trait] #[async_trait::async_trait]
pub trait PublishSeraiTransaction { pub trait PublishSeraiTransaction {
async fn publish_set_keys(&self, set: ValidatorSet, key_pair: KeyPair, signature: Signature); async fn publish_set_keys(
async fn publish_remove_participant(
&self, &self,
set: ValidatorSet, set: ValidatorSet,
removing: [u8; 32], removed: Vec<SeraiAddress>,
signers: Vec<SeraiAddress>, key_pair: KeyPair,
signature: Signature, signature: Signature,
); );
} }
@@ -137,8 +128,14 @@ mod impl_pst_for_serai {
#[async_trait::async_trait] #[async_trait::async_trait]
impl PublishSeraiTransaction for Serai { impl PublishSeraiTransaction for Serai {
async fn publish_set_keys(&self, set: ValidatorSet, key_pair: KeyPair, signature: Signature) { async fn publish_set_keys(
let tx = SeraiValidatorSets::set_keys(set.network, key_pair, signature); &self,
set: ValidatorSet,
removed: Vec<SeraiAddress>,
key_pair: KeyPair,
signature: Signature,
) {
let tx = SeraiValidatorSets::set_keys(set.network, removed, key_pair, signature);
async fn check(serai: SeraiValidatorSets<'_>, set: ValidatorSet, _: ()) -> bool { async fn check(serai: SeraiValidatorSets<'_>, set: ValidatorSet, _: ()) -> bool {
if matches!(serai.keys(set).await, Ok(Some(_))) { if matches!(serai.keys(set).await, Ok(Some(_))) {
log::info!("another coordinator set key pair for {:?}", set); log::info!("another coordinator set key pair for {:?}", set);
@@ -151,42 +148,6 @@ mod impl_pst_for_serai {
log::info!("published set keys for {set:?}"); log::info!("published set keys for {set:?}");
} }
} }
async fn publish_remove_participant(
&self,
set: ValidatorSet,
removing: [u8; 32],
signers: Vec<SeraiAddress>,
signature: Signature,
) {
let tx = SeraiValidatorSets::remove_participant(
set.network,
SeraiAddress(removing),
signers,
signature,
);
async fn check(serai: SeraiValidatorSets<'_>, set: ValidatorSet, removing: [u8; 32]) -> bool {
if let Ok(Some(_)) = serai.keys(set).await {
log::info!(
"keys were set before we personally could publish the removal for {}",
hex::encode(removing)
);
return true;
}
if let Ok(Some(participants)) = serai.participants(set.network).await {
if !participants.iter().any(|(participant, _)| participant.0 == removing) {
log::info!("another coordinator published removal for {:?}", hex::encode(removing),);
return true;
}
}
false
}
common_pst!([u8; 32], check);
if publish(self, set, tx, removing).await {
log::info!("published remove participant for {}", hex::encode(removing));
}
}
} }
} }
@@ -231,30 +192,6 @@ impl<
P: P2p, P: P2p,
> TributaryBlockHandler<'_, T, Pro, PST, PTT, RID, P> > TributaryBlockHandler<'_, T, Pro, PST, PTT, RID, P>
{ {
async fn attempt_dkg_removal(&mut self, removing: [u8; 32], attempt: u32) {
let genesis = self.spec.genesis();
let removed = crate::tributary::removed_as_of_fatal_slash(self.txn, genesis, removing)
.expect("attempting DKG removal to remove someone who wasn't removed");
let preprocess = (DkgRemoval {
key: self.our_key,
spec: self.spec,
txn: self.txn,
removed: &removed,
removing,
attempt,
})
.preprocess();
let mut tx = Transaction::DkgRemoval(SignData {
plan: removing,
attempt,
label: Label::Preprocess,
data: vec![preprocess.to_vec()],
signed: Transaction::empty_signed(),
});
tx.sign(&mut OsRng, genesis, self.our_key);
self.publish_tributary_tx.publish_tributary_tx(tx).await;
}
pub async fn fatal_slash(&mut self, slashing: [u8; 32], reason: &str) { pub async fn fatal_slash(&mut self, slashing: [u8; 32], reason: &str) {
// TODO: If this fatal slash puts the remaining set below the threshold, spin // TODO: If this fatal slash puts the remaining set below the threshold, spin
@@ -266,12 +203,6 @@ impl<
// TODO: If during DKG, trigger a re-attempt // TODO: If during DKG, trigger a re-attempt
// Despite triggering a re-attempt, this DKG may still complete and may become in-use // Despite triggering a re-attempt, this DKG may still complete and may become in-use
// If during a DKG, remove the participant
if DkgCompleted::get(self.txn, genesis).is_none() {
AttemptDb::recognize_topic(self.txn, genesis, Topic::DkgRemoval(slashing));
self.attempt_dkg_removal(slashing, 0).await;
}
} }
// TODO: Once Substrate confirms a key, we need to rotate our validator set OR form a second // TODO: Once Substrate confirms a key, we need to rotate our validator set OR form a second
@@ -404,16 +335,6 @@ impl<
Topic::DkgConfirmation => { Topic::DkgConfirmation => {
panic!("re-attempting DkgConfirmation when we should be re-attempting the Dkg") panic!("re-attempting DkgConfirmation when we should be re-attempting the Dkg")
} }
Topic::DkgRemoval(removing) => {
if DkgCompleted::get(self.txn, genesis).is_none() &&
LocallyDkgRemoved::get(self.txn, genesis, removing).is_none() &&
SeraiDkgCompleted::get(self.txn, self.spec.set()).is_none() &&
SeraiDkgRemoval::get(self.txn, self.spec.set(), removing).is_none()
{
// Since it wasn't completed, attempt a new DkgRemoval
self.attempt_dkg_removal(removing, attempt).await;
}
}
Topic::SubstrateSign(inner_id) => { Topic::SubstrateSign(inner_id) => {
let id = processor_messages::coordinator::SubstrateSignId { let id = processor_messages::coordinator::SubstrateSignId {
session: self.spec.set().session, session: self.spec.set().session,

View File

@@ -1,9 +1,8 @@
/* /*
A MuSig-based signing protocol executed with the validators' keys. A MuSig-based signing protocol executed with the validators' keys.
This is used for confirming the results of a DKG on-chain, an operation requiring all validators, This is used for confirming the results of a DKG on-chain, an operation requiring all validators
and for removing another validator before the DKG completes, an operation requiring a which aren't specified as removed while still satisfying a supermajority.
supermajority of validators.
Since we're using the validator's keys, as needed for their being the root of trust, the Since we're using the validator's keys, as needed for their being the root of trust, the
coordinator must perform the signing. This is distinct from all other group-signing operations, coordinator must perform the signing. This is distinct from all other group-signing operations,
@@ -65,7 +64,7 @@ use rand_core::OsRng;
use blake2::{Digest, Blake2s256}; use blake2::{Digest, Blake2s256};
use ciphersuite::{ use ciphersuite::{
group::{ff::PrimeField, Group, GroupEncoding}, group::{ff::PrimeField, GroupEncoding},
Ciphersuite, Ristretto, Ciphersuite, Ristretto,
}; };
use frost::{ use frost::{
@@ -79,10 +78,8 @@ use frost_schnorrkel::Schnorrkel;
use scale::Encode; use scale::Encode;
use serai_client::{ use serai_client::{
Public, SeraiAddress, Public,
validator_sets::primitives::{ validator_sets::primitives::{KeyPair, musig_context, set_keys_message},
KeyPair, musig_context, set_keys_message, remove_participant_message,
},
}; };
use serai_db::*; use serai_db::*;
@@ -212,15 +209,11 @@ impl<T: DbTxn, C: Encode> SigningProtocol<'_, T, C> {
// Get the keys of the participants, noted by their threshold is, and return a new map indexed by // Get the keys of the participants, noted by their threshold is, and return a new map indexed by
// the MuSig is. // the MuSig is.
//
// If sort_by_keys = true, the MuSig is will index the keys once sorted. Else, the MuSig is will
// index the validators in the order they've been defined.
fn threshold_i_map_to_keys_and_musig_i_map( fn threshold_i_map_to_keys_and_musig_i_map(
spec: &TributarySpec, spec: &TributarySpec,
removed: &[<Ristretto as Ciphersuite>::G], removed: &[<Ristretto as Ciphersuite>::G],
our_key: &Zeroizing<<Ristretto as Ciphersuite>::F>, our_key: &Zeroizing<<Ristretto as Ciphersuite>::F>,
mut map: HashMap<Participant, Vec<u8>>, mut map: HashMap<Participant, Vec<u8>>,
sort_by_keys: bool,
) -> (Vec<<Ristretto as Ciphersuite>::G>, HashMap<Participant, Vec<u8>>) { ) -> (Vec<<Ristretto as Ciphersuite>::G>, HashMap<Participant, Vec<u8>>) {
// Insert our own index so calculations aren't offset // Insert our own index so calculations aren't offset
let our_threshold_i = let our_threshold_i =
@@ -243,10 +236,6 @@ fn threshold_i_map_to_keys_and_musig_i_map(
for threshold_i in threshold_is { for threshold_i in threshold_is {
sorted.push((key_from_threshold_i(threshold_i), map.remove(&threshold_i).unwrap())); sorted.push((key_from_threshold_i(threshold_i), map.remove(&threshold_i).unwrap()));
} }
if sort_by_keys {
// Substrate expects these signers to be sorted by key
sorted.sort_by(|(key1, _), (key2, _)| key1.to_bytes().cmp(&key2.to_bytes()));
}
// Now that signers are sorted, with their shares, create a map with the is needed for MuSig // Now that signers are sorted, with their shares, create a map with the is needed for MuSig
let mut participants = vec![]; let mut participants = vec![];
@@ -302,15 +291,13 @@ impl<T: DbTxn> DkgConfirmer<'_, T> {
key_pair: &KeyPair, key_pair: &KeyPair,
) -> Result<(AlgorithmSignatureMachine<Ristretto, Schnorrkel>, [u8; 32]), Participant> { ) -> Result<(AlgorithmSignatureMachine<Ristretto, Schnorrkel>, [u8; 32]), Participant> {
let participants = self.spec.validators().iter().map(|val| val.0).collect::<Vec<_>>(); let participants = self.spec.validators().iter().map(|val| val.0).collect::<Vec<_>>();
let preprocesses = threshold_i_map_to_keys_and_musig_i_map( let preprocesses =
self.spec, threshold_i_map_to_keys_and_musig_i_map(self.spec, &self.removed, self.key, preprocesses).1;
&self.removed, let msg = set_keys_message(
self.key, &self.spec.set(),
preprocesses, &self.removed.iter().map(|key| Public(key.to_bytes())).collect::<Vec<_>>(),
false, key_pair,
) );
.1;
let msg = set_keys_message(&self.spec.set(), key_pair);
self.signing_protocol().share_internal(&participants, preprocesses, &msg) self.signing_protocol().share_internal(&participants, preprocesses, &msg)
} }
// Get the share for this confirmation, if the preprocesses are valid. // Get the share for this confirmation, if the preprocesses are valid.
@@ -329,7 +316,7 @@ impl<T: DbTxn> DkgConfirmer<'_, T> {
shares: HashMap<Participant, Vec<u8>>, shares: HashMap<Participant, Vec<u8>>,
) -> Result<[u8; 64], Participant> { ) -> Result<[u8; 64], Participant> {
let shares = let shares =
threshold_i_map_to_keys_and_musig_i_map(self.spec, &self.removed, self.key, shares, false).1; threshold_i_map_to_keys_and_musig_i_map(self.spec, &self.removed, self.key, shares).1;
let machine = self let machine = self
.share_internal(preprocesses, key_pair) .share_internal(preprocesses, key_pair)
@@ -339,83 +326,3 @@ impl<T: DbTxn> DkgConfirmer<'_, T> {
self.signing_protocol().complete_internal(machine, shares) self.signing_protocol().complete_internal(machine, shares)
} }
} }
pub(crate) struct DkgRemoval<'a, T: DbTxn> {
pub(crate) key: &'a Zeroizing<<Ristretto as Ciphersuite>::F>,
pub(crate) spec: &'a TributarySpec,
pub(crate) removed: &'a [<Ristretto as Ciphersuite>::G],
pub(crate) txn: &'a mut T,
pub(crate) removing: [u8; 32],
pub(crate) attempt: u32,
}
impl<T: DbTxn> DkgRemoval<'_, T> {
fn signing_protocol(&mut self) -> SigningProtocol<'_, T, (&'static [u8; 10], [u8; 32], u32)> {
let context = (b"DkgRemoval", self.removing, self.attempt);
SigningProtocol { key: self.key, spec: self.spec, txn: self.txn, context }
}
fn preprocess_internal(
&mut self,
participants: Option<&[<Ristretto as Ciphersuite>::G]>,
) -> (AlgorithmSignMachine<Ristretto, Schnorrkel>, [u8; 64]) {
// We won't know the participants when we first preprocess
// If we don't, we use our key alone as the participant
let just_us = [<Ristretto as Ciphersuite>::G::generator() * self.key.deref()];
let to_musig = if let Some(participants) = participants { participants } else { &just_us };
let (machine, preprocess) = self.signing_protocol().preprocess_internal(to_musig);
// If we're now specifying participants, confirm the commitments were the same
if participants.is_some() {
let (_, theoretical_preprocess) = self.signing_protocol().preprocess_internal(&just_us);
assert_eq!(theoretical_preprocess, preprocess);
}
(machine, preprocess)
}
// Get the preprocess for this confirmation.
pub(crate) fn preprocess(&mut self) -> [u8; 64] {
self.preprocess_internal(None).1
}
fn share_internal(
&mut self,
preprocesses: HashMap<Participant, Vec<u8>>,
) -> Result<(AlgorithmSignatureMachine<Ristretto, Schnorrkel>, [u8; 32]), Participant> {
let (participants, preprocesses) = threshold_i_map_to_keys_and_musig_i_map(
self.spec,
self.removed,
self.key,
preprocesses,
true,
);
let msg = remove_participant_message(&self.spec.set(), Public(self.removing));
self.signing_protocol().share_internal(&participants, preprocesses, &msg)
}
// Get the share for this confirmation, if the preprocesses are valid.
pub(crate) fn share(
&mut self,
preprocesses: HashMap<Participant, Vec<u8>>,
) -> Result<[u8; 32], Participant> {
self.share_internal(preprocesses).map(|(_, share)| share)
}
pub(crate) fn complete(
&mut self,
preprocesses: HashMap<Participant, Vec<u8>>,
shares: HashMap<Participant, Vec<u8>>,
) -> Result<(Vec<SeraiAddress>, [u8; 64]), Participant> {
let (participants, shares) =
threshold_i_map_to_keys_and_musig_i_map(self.spec, self.removed, self.key, shares, true);
let signers = participants.iter().map(|key| SeraiAddress(key.to_bytes())).collect::<Vec<_>>();
let machine = self
.share_internal(preprocesses)
.expect("trying to complete a machine which failed to preprocess")
.0;
let signature = self.signing_protocol().complete_internal(machine, shares)?;
Ok((signers, signature))
}
}

View File

@@ -160,8 +160,6 @@ pub enum Transaction {
signed: Signed, signed: Signed,
}, },
DkgRemoval(SignData<[u8; 32]>),
// Co-sign a Substrate block. // Co-sign a Substrate block.
CosignSubstrateBlock([u8; 32]), CosignSubstrateBlock([u8; 32]),
@@ -223,9 +221,6 @@ impl Debug for Transaction {
.field("attempt", attempt) .field("attempt", attempt)
.field("signer", &hex::encode(signed.signer.to_bytes())) .field("signer", &hex::encode(signed.signer.to_bytes()))
.finish_non_exhaustive(), .finish_non_exhaustive(),
Transaction::DkgRemoval(sign_data) => {
fmt.debug_struct("Transaction::DkgRemoval").field("sign_data", sign_data).finish()
}
Transaction::CosignSubstrateBlock(block) => fmt Transaction::CosignSubstrateBlock(block) => fmt
.debug_struct("Transaction::CosignSubstrateBlock") .debug_struct("Transaction::CosignSubstrateBlock")
.field("block", &hex::encode(block)) .field("block", &hex::encode(block))
@@ -389,15 +384,13 @@ impl ReadWrite for Transaction {
Ok(Transaction::DkgConfirmed { attempt, confirmation_share, signed }) Ok(Transaction::DkgConfirmed { attempt, confirmation_share, signed })
} }
5 => SignData::read(reader).map(Transaction::DkgRemoval), 5 => {
6 => {
let mut block = [0; 32]; let mut block = [0; 32];
reader.read_exact(&mut block)?; reader.read_exact(&mut block)?;
Ok(Transaction::CosignSubstrateBlock(block)) Ok(Transaction::CosignSubstrateBlock(block))
} }
7 => { 6 => {
let mut block = [0; 32]; let mut block = [0; 32];
reader.read_exact(&mut block)?; reader.read_exact(&mut block)?;
let mut batch = [0; 4]; let mut batch = [0; 4];
@@ -405,16 +398,16 @@ impl ReadWrite for Transaction {
Ok(Transaction::Batch { block, batch: u32::from_le_bytes(batch) }) Ok(Transaction::Batch { block, batch: u32::from_le_bytes(batch) })
} }
8 => { 7 => {
let mut block = [0; 8]; let mut block = [0; 8];
reader.read_exact(&mut block)?; reader.read_exact(&mut block)?;
Ok(Transaction::SubstrateBlock(u64::from_le_bytes(block))) Ok(Transaction::SubstrateBlock(u64::from_le_bytes(block)))
} }
9 => SignData::read(reader).map(Transaction::SubstrateSign), 8 => SignData::read(reader).map(Transaction::SubstrateSign),
10 => SignData::read(reader).map(Transaction::Sign), 9 => SignData::read(reader).map(Transaction::Sign),
11 => { 10 => {
let mut plan = [0; 32]; let mut plan = [0; 32];
reader.read_exact(&mut plan)?; reader.read_exact(&mut plan)?;
@@ -512,37 +505,32 @@ impl ReadWrite for Transaction {
signed.write_without_nonce(writer) signed.write_without_nonce(writer)
} }
Transaction::DkgRemoval(data) => {
writer.write_all(&[5])?;
data.write(writer)
}
Transaction::CosignSubstrateBlock(block) => { Transaction::CosignSubstrateBlock(block) => {
writer.write_all(&[6])?; writer.write_all(&[5])?;
writer.write_all(block) writer.write_all(block)
} }
Transaction::Batch { block, batch } => { Transaction::Batch { block, batch } => {
writer.write_all(&[7])?; writer.write_all(&[6])?;
writer.write_all(block)?; writer.write_all(block)?;
writer.write_all(&batch.to_le_bytes()) writer.write_all(&batch.to_le_bytes())
} }
Transaction::SubstrateBlock(block) => { Transaction::SubstrateBlock(block) => {
writer.write_all(&[8])?; writer.write_all(&[7])?;
writer.write_all(&block.to_le_bytes()) writer.write_all(&block.to_le_bytes())
} }
Transaction::SubstrateSign(data) => { Transaction::SubstrateSign(data) => {
writer.write_all(&[9])?; writer.write_all(&[8])?;
data.write(writer) data.write(writer)
} }
Transaction::Sign(data) => { Transaction::Sign(data) => {
writer.write_all(&[10])?; writer.write_all(&[9])?;
data.write(writer) data.write(writer)
} }
Transaction::SignCompleted { plan, tx_hash, first_signer, signature } => { Transaction::SignCompleted { plan, tx_hash, first_signer, signature } => {
writer.write_all(&[11])?; writer.write_all(&[10])?;
writer.write_all(plan)?; writer.write_all(plan)?;
writer writer
.write_all(&[u8::try_from(tx_hash.len()).expect("tx hash length exceed 255 bytes")])?; .write_all(&[u8::try_from(tx_hash.len()).expect("tx hash length exceed 255 bytes")])?;
@@ -572,10 +560,6 @@ impl TransactionTrait for Transaction {
TransactionKind::Signed((b"dkg", attempt).encode(), signed) TransactionKind::Signed((b"dkg", attempt).encode(), signed)
} }
Transaction::DkgRemoval(data) => {
TransactionKind::Signed((b"dkg_removal", data.plan, data.attempt).encode(), &data.signed)
}
Transaction::CosignSubstrateBlock(_) => TransactionKind::Provided("cosign"), Transaction::CosignSubstrateBlock(_) => TransactionKind::Provided("cosign"),
Transaction::Batch { .. } => TransactionKind::Provided("batch"), Transaction::Batch { .. } => TransactionKind::Provided("batch"),
@@ -601,7 +585,7 @@ impl TransactionTrait for Transaction {
} }
fn verify(&self) -> Result<(), TransactionError> { fn verify(&self) -> Result<(), TransactionError> {
// TODO: Check DkgRemoval and SubstrateSign's lengths here // TODO: Check SubstrateSign's lengths here
if let Transaction::SignCompleted { first_signer, signature, .. } = self { if let Transaction::SignCompleted { first_signer, signature, .. } = self {
if !signature.verify(*first_signer, self.sign_completed_challenge()) { if !signature.verify(*first_signer, self.sign_completed_challenge()) {
@@ -644,8 +628,6 @@ impl Transaction {
Transaction::InvalidDkgShare { .. } => 2, Transaction::InvalidDkgShare { .. } => 2,
Transaction::DkgConfirmed { .. } => 2, Transaction::DkgConfirmed { .. } => 2,
Transaction::DkgRemoval(data) => data.label.nonce(),
Transaction::CosignSubstrateBlock(_) => panic!("signing CosignSubstrateBlock"), Transaction::CosignSubstrateBlock(_) => panic!("signing CosignSubstrateBlock"),
Transaction::Batch { .. } => panic!("signing Batch"), Transaction::Batch { .. } => panic!("signing Batch"),
@@ -666,8 +648,6 @@ impl Transaction {
Transaction::InvalidDkgShare { ref mut signed, .. } => signed, Transaction::InvalidDkgShare { ref mut signed, .. } => signed,
Transaction::DkgConfirmed { ref mut signed, .. } => signed, Transaction::DkgConfirmed { ref mut signed, .. } => signed,
Transaction::DkgRemoval(ref mut data) => &mut data.signed,
Transaction::CosignSubstrateBlock(_) => panic!("signing CosignSubstrateBlock"), Transaction::CosignSubstrateBlock(_) => panic!("signing CosignSubstrateBlock"),
Transaction::Batch { .. } => panic!("signing Batch"), Transaction::Batch { .. } => panic!("signing Batch"),

View File

@@ -8,15 +8,10 @@ use serai_validator_sets_primitives::*;
pub enum Call { pub enum Call {
set_keys { set_keys {
network: NetworkId, network: NetworkId,
removed_participants: Vec<SeraiAddress>,
key_pair: KeyPair, key_pair: KeyPair,
signature: Signature, signature: Signature,
}, },
remove_participant {
network: NetworkId,
to_remove: SeraiAddress,
signers: Vec<SeraiAddress>,
signature: Signature,
},
allocate { allocate {
network: NetworkId, network: NetworkId,
amount: Amount, amount: Amount,

View File

@@ -126,36 +126,22 @@ impl<'a> SeraiValidatorSets<'a> {
.await .await
} }
pub async fn musig_key(&self, set: ValidatorSet) -> Result<Option<[u8; 32]>, SeraiError> {
self.0.storage(PALLET, "MuSigKeys", (sp_core::hashing::twox_64(&set.encode()), set)).await
}
// TODO: Store these separately since we almost never need both at once? // TODO: Store these separately since we almost never need both at once?
pub async fn keys(&self, set: ValidatorSet) -> Result<Option<KeyPair>, SeraiError> { pub async fn keys(&self, set: ValidatorSet) -> Result<Option<KeyPair>, SeraiError> {
self.0.storage(PALLET, "Keys", (sp_core::hashing::twox_64(&set.encode()), set)).await self.0.storage(PALLET, "Keys", (sp_core::hashing::twox_64(&set.encode()), set)).await
} }
pub fn set_keys(network: NetworkId, key_pair: KeyPair, signature: Signature) -> Transaction { pub fn set_keys(
network: NetworkId,
removed_participants: Vec<SeraiAddress>,
key_pair: KeyPair,
signature: Signature,
) -> Transaction {
Serai::unsigned(serai_abi::Call::ValidatorSets(serai_abi::validator_sets::Call::set_keys { Serai::unsigned(serai_abi::Call::ValidatorSets(serai_abi::validator_sets::Call::set_keys {
network, network,
removed_participants,
key_pair, key_pair,
signature, signature,
})) }))
} }
pub fn remove_participant(
network: NetworkId,
to_remove: SeraiAddress,
signers: Vec<SeraiAddress>,
signature: Signature,
) -> Transaction {
Serai::unsigned(serai_abi::Call::ValidatorSets(
serai_abi::validator_sets::Call::remove_participant {
network,
to_remove,
signers,
signature,
},
))
}
} }

View File

@@ -5,14 +5,14 @@ use rand_core::OsRng;
use sp_core::{Pair, sr25519::Signature}; use sp_core::{Pair, sr25519::Signature};
use ciphersuite::{group::GroupEncoding, Ciphersuite, Ristretto}; use ciphersuite::{Ciphersuite, Ristretto};
use frost::dkg::musig::musig; use frost::dkg::musig::musig;
use schnorrkel::Schnorrkel; use schnorrkel::Schnorrkel;
use serai_client::{ use serai_client::{
primitives::insecure_pair_from_name, primitives::insecure_pair_from_name,
validator_sets::{ validator_sets::{
primitives::{ValidatorSet, KeyPair, musig_context, musig_key, set_keys_message}, primitives::{ValidatorSet, KeyPair, musig_context, set_keys_message},
ValidatorSetsEvent, ValidatorSetsEvent,
}, },
SeraiValidatorSets, Serai, SeraiValidatorSets, Serai,
@@ -26,19 +26,6 @@ pub async fn set_keys(serai: &Serai, set: ValidatorSet, key_pair: KeyPair) -> [u
let public = pair.public(); let public = pair.public();
let public_key = <Ristretto as Ciphersuite>::read_G::<&[u8]>(&mut public.0.as_ref()).unwrap(); let public_key = <Ristretto as Ciphersuite>::read_G::<&[u8]>(&mut public.0.as_ref()).unwrap();
assert_eq!(
serai
.as_of_latest_finalized_block()
.await
.unwrap()
.validator_sets()
.musig_key(set)
.await
.unwrap()
.unwrap(),
musig_key(set, &[public]).0
);
let secret_key = <Ristretto as Ciphersuite>::read_F::<&[u8]>( let secret_key = <Ristretto as Ciphersuite>::read_F::<&[u8]>(
&mut pair.as_ref().secret.to_bytes()[.. 32].as_ref(), &mut pair.as_ref().secret.to_bytes()[.. 32].as_ref(),
) )
@@ -46,18 +33,6 @@ pub async fn set_keys(serai: &Serai, set: ValidatorSet, key_pair: KeyPair) -> [u
assert_eq!(Ristretto::generator() * secret_key, public_key); assert_eq!(Ristretto::generator() * secret_key, public_key);
let threshold_keys = let threshold_keys =
musig::<Ristretto>(&musig_context(set), &Zeroizing::new(secret_key), &[public_key]).unwrap(); musig::<Ristretto>(&musig_context(set), &Zeroizing::new(secret_key), &[public_key]).unwrap();
assert_eq!(
serai
.as_of_latest_finalized_block()
.await
.unwrap()
.validator_sets()
.musig_key(set)
.await
.unwrap()
.unwrap(),
threshold_keys.group_key().to_bytes()
);
let sig = frost::tests::sign_without_caching( let sig = frost::tests::sign_without_caching(
&mut OsRng, &mut OsRng,
@@ -66,13 +41,13 @@ pub async fn set_keys(serai: &Serai, set: ValidatorSet, key_pair: KeyPair) -> [u
Schnorrkel::new(b"substrate"), Schnorrkel::new(b"substrate"),
&HashMap::from([(threshold_keys.params().i(), threshold_keys.into())]), &HashMap::from([(threshold_keys.params().i(), threshold_keys.into())]),
), ),
&set_keys_message(&set, &key_pair), &set_keys_message(&set, &[], &key_pair),
); );
// Set the key pair // Set the key pair
let block = publish_tx( let block = publish_tx(
serai, serai,
&SeraiValidatorSets::set_keys(set.network, key_pair.clone(), Signature(sig.to_bytes())), &SeraiValidatorSets::set_keys(set.network, vec![], key_pair.clone(), Signature(sig.to_bytes())),
) )
.await; .await;

View File

@@ -5,7 +5,7 @@ use sp_core::{sr25519::Public, Pair};
use serai_client::{ use serai_client::{
primitives::{NETWORKS, NetworkId, insecure_pair_from_name}, primitives::{NETWORKS, NetworkId, insecure_pair_from_name},
validator_sets::{ validator_sets::{
primitives::{Session, ValidatorSet, KeyPair, musig_key}, primitives::{Session, ValidatorSet, KeyPair},
ValidatorSetsEvent, ValidatorSetsEvent,
}, },
Serai, Serai,
@@ -58,7 +58,6 @@ serai_test!(
.collect::<Vec<_>>(); .collect::<Vec<_>>();
let participants_ref: &[_] = participants.as_ref(); let participants_ref: &[_] = participants.as_ref();
assert_eq!(participants_ref, [public].as_ref()); assert_eq!(participants_ref, [public].as_ref());
assert_eq!(vs_serai.musig_key(set).await.unwrap().unwrap(), musig_key(set, &[public]).0);
} }
let block = set_keys(&serai, set, key_pair.clone()).await; let block = set_keys(&serai, set, key_pair.clone()).await;

View File

@@ -16,6 +16,8 @@ rustdoc-args = ["--cfg", "docsrs"]
ignored = ["scale", "scale-info"] ignored = ["scale", "scale-info"]
[dependencies] [dependencies]
hashbrown = { version = "0.14", default-features = false, features = ["ahash", "inline-more"] }
scale = { package = "parity-scale-codec", version = "3", default-features = false, features = ["derive"] } scale = { package = "parity-scale-codec", version = "3", default-features = false, features = ["derive"] }
scale-info = { version = "2", default-features = false, features = ["derive"] } scale-info = { version = "2", default-features = false, features = ["derive"] }

View File

@@ -257,10 +257,6 @@ pub mod pallet {
type PendingDeallocations<T: Config> = type PendingDeallocations<T: Config> =
StorageMap<_, Blake2_128Concat, (NetworkId, Session, Public), Amount, OptionQuery>; StorageMap<_, Blake2_128Concat, (NetworkId, Session, Public), Amount, OptionQuery>;
/// The MuSig key for a validator set.
#[pallet::storage]
pub type MuSigKeys<T: Config> = StorageMap<_, Twox64Concat, ValidatorSet, Public, OptionQuery>;
/// The generated key pair for a given validator set instance. /// The generated key pair for a given validator set instance.
#[pallet::storage] #[pallet::storage]
#[pallet::getter(fn keys)] #[pallet::getter(fn keys)]
@@ -343,13 +339,6 @@ pub mod pallet {
let set = ValidatorSet { network, session }; let set = ValidatorSet { network, session };
Pallet::<T>::deposit_event(Event::NewSet { set }); Pallet::<T>::deposit_event(Event::NewSet { set });
// Only set the MuSig key for non-Serai sets, as only non-Serai sets should publish keys
if network != NetworkId::Serai {
MuSigKeys::<T>::set(
set,
Some(musig_key(set, &participants.iter().map(|(id, _)| *id).collect::<Vec<_>>())),
);
}
Participants::<T>::set(network, Some(participants.try_into().unwrap())); Participants::<T>::set(network, Some(participants.try_into().unwrap()));
} }
} }
@@ -411,26 +400,6 @@ pub mod pallet {
} }
} }
impl<T: Config> Pallet<T> {
fn verify_signature(
set: ValidatorSet,
key_pair: &KeyPair,
signature: &Signature,
) -> Result<(), Error<T>> {
// Confirm a key hasn't been set for this set instance
if Keys::<T>::get(set).is_some() {
Err(Error::AlreadyGeneratedKeys)?
}
let Some(musig_key) = MuSigKeys::<T>::get(set) else { Err(Error::NonExistentValidatorSet)? };
if !musig_key.verify(&set_keys_message(&set, key_pair), signature) {
Err(Error::BadSignature)?;
}
Ok(())
}
}
impl<T: Config> Pallet<T> { impl<T: Config> Pallet<T> {
fn account() -> T::AccountId { fn account() -> T::AccountId {
system_address(b"ValidatorSets").into() system_address(b"ValidatorSets").into()
@@ -658,7 +627,6 @@ pub mod pallet {
} }
pub fn retire_set(set: ValidatorSet) { pub fn retire_set(set: ValidatorSet) {
MuSigKeys::<T>::remove(set);
Keys::<T>::remove(set); Keys::<T>::remove(set);
Pallet::<T>::deposit_event(Event::SetRetired { set }); Pallet::<T>::deposit_event(Event::SetRetired { set });
} }
@@ -754,6 +722,7 @@ pub mod pallet {
pub fn set_keys( pub fn set_keys(
origin: OriginFor<T>, origin: OriginFor<T>,
network: NetworkId, network: NetworkId,
removed_participants: Vec<Public>,
key_pair: KeyPair, key_pair: KeyPair,
signature: Signature, signature: Signature,
) -> DispatchResult { ) -> DispatchResult {
@@ -763,32 +732,21 @@ pub mod pallet {
// (called by pre_dispatch) checks it // (called by pre_dispatch) checks it
let _ = signature; let _ = signature;
let session = Self::session(NetworkId::Serai).unwrap(); let session = Self::session(network).unwrap();
let set = ValidatorSet { network, session };
let set = ValidatorSet { session, network };
Keys::<T>::set(set, Some(key_pair.clone())); Keys::<T>::set(set, Some(key_pair.clone()));
Self::deposit_event(Event::KeyGen { set, key_pair });
Ok(()) // This does not remove from TotalAllocatedStake or InSet in order to:
// 1) Not decrease the stake present in this set. This means removed participants are
// still liable for the economic security of the external network. This prevents
// a decided set, which is economically secure, from falling below the threshold.
// 2) Not allow parties removed to immediately deallocate, per commentary on deallocation
// scheduling (https://github.com/serai-dex/serai/issues/394).
for removed in removed_participants {
Self::deposit_event(Event::ParticipantRemoved { set, removed });
} }
Self::deposit_event(Event::KeyGen { set, key_pair });
#[pallet::call_index(1)]
#[pallet::weight(0)] // TODO
pub fn remove_participant(
origin: OriginFor<T>,
network: NetworkId,
to_remove: Public,
signers: Vec<Public>,
signature: Signature,
) -> DispatchResult {
ensure_none(origin)?;
// Nothing occurs here as validate_unsigned does everything
let _ = network;
let _ = to_remove;
let _ = signers;
let _ = signature;
Ok(()) Ok(())
} }
@@ -850,128 +808,79 @@ pub mod pallet {
fn validate_unsigned(_: TransactionSource, call: &Self::Call) -> TransactionValidity { fn validate_unsigned(_: TransactionSource, call: &Self::Call) -> TransactionValidity {
// Match to be exhaustive // Match to be exhaustive
match call { match call {
Call::set_keys { network, ref key_pair, ref signature } => { Call::set_keys { network, ref removed_participants, ref key_pair, ref signature } => {
let network = *network;
// Don't allow the Serai set to set_keys, as they have no reason to do so // Don't allow the Serai set to set_keys, as they have no reason to do so
// This should already be covered by the lack of key in MuSigKeys, yet it doesn't hurt to if network == NetworkId::Serai {
// be explicit
if network == &NetworkId::Serai {
Err(InvalidTransaction::Custom(0))?;
}
let session = Self::session(NetworkId::Serai).unwrap();
let set = ValidatorSet { session, network: *network };
match Self::verify_signature(set, key_pair, signature) {
Err(Error::AlreadyGeneratedKeys) => Err(InvalidTransaction::Stale)?,
Err(Error::NonExistentValidatorSet) |
Err(Error::InsufficientAllocation) |
Err(Error::NotEnoughAllocated) |
Err(Error::AllocationWouldRemoveFaultTolerance) |
Err(Error::AllocationWouldPreventFaultTolerance) |
Err(Error::DeallocationWouldRemoveParticipant) |
Err(Error::DeallocationWouldRemoveFaultTolerance) |
Err(Error::NonExistentDeallocation) |
Err(Error::NonExistentValidator) |
Err(Error::DeallocationWouldRemoveEconomicSecurity) |
Err(Error::BadSignature) => Err(InvalidTransaction::BadProof)?,
Err(Error::__Ignore(_, _)) => unreachable!(),
Ok(()) => (),
}
ValidTransaction::with_tag_prefix("ValidatorSets")
.and_provides((0, set))
.longevity(u64::MAX)
.propagate(true)
.build()
}
Call::remove_participant { network, to_remove, signers, signature } => {
if network == &NetworkId::Serai {
Err(InvalidTransaction::Custom(0))?; Err(InvalidTransaction::Custom(0))?;
} }
// Confirm this set has a session // Confirm this set has a session
let Some(current_session) = Self::session(*network) else { let Some(current_session) = Self::session(network) else {
Err(InvalidTransaction::Custom(1))? Err(InvalidTransaction::Custom(1))?
}; };
// This is needed as modify storage variables of the latest decided session
assert_eq!(Pallet::<T>::latest_decided_session(*network), Some(current_session)); let set = ValidatorSet { network, session: current_session };
let set = ValidatorSet { network: *network, session: current_session };
// Confirm it has yet to set keys // Confirm it has yet to set keys
if Keys::<T>::get(set).is_some() { if Keys::<T>::get(set).is_some() {
Err(InvalidTransaction::Stale)?;
}
// This is a needed precondition as this uses storage variables for the latest decided
// session on this assumption
assert_eq!(Pallet::<T>::latest_decided_session(network), Some(current_session));
// This does not slash the removed participants as that'll be done at the end of the
// set's lifetime
let mut removed = hashbrown::HashSet::new();
for participant in removed_participants {
// Confirm this wasn't duplicated
if removed.contains(&participant.0) {
Err(InvalidTransaction::Custom(2))?; Err(InvalidTransaction::Custom(2))?;
} }
removed.insert(participant.0);
}
let mut participants = let participants =
Participants::<T>::get(network).expect("session existed without participants"); Participants::<T>::get(network).expect("session existed without participants");
// Require signers be sorted to ensure no duplicates are present let mut all_key_shares = 0;
let mut last_signer = None; let mut signers = vec![];
let mut signing_key_shares = 0; let mut signing_key_shares = 0;
for signer in signers { for participant in participants {
if let Some(last_signer) = last_signer { let participant = participant.0;
if last_signer >= signer { let shares = InSet::<T>::get(network, participant)
Err(InvalidTransaction::Custom(3))?; .expect("participant from Participants wasn't InSet");
} all_key_shares += shares;
}
last_signer = Some(signer);
// Doesn't use InSet as InSet *includes* removed validators if removed.contains(&participant.0) {
// Only non-removed validators should be considered as contributing continue;
let Some(shares) = participants }
.iter()
.find(|(participant, _)| participant == to_remove) signers.push(participant);
.map(|(_, shares)| shares)
else {
Err(InvalidTransaction::Custom(4))?
};
signing_key_shares += shares; signing_key_shares += shares;
} }
// Check 67% are participating in this removal {
// This is done by iterating over InSet, which isn't mutated on removal, and reading the let f = all_key_shares - signing_key_shares;
// shares from that if signing_key_shares < ((2 * f) + 1) {
let mut all_key_shares = 0; Err(InvalidTransaction::Custom(3))?;
for shares in InSet::<T>::iter_prefix_values(network) {
all_key_shares += shares;
} }
// 2f + 1
if signing_key_shares < ((2 * (all_key_shares - signing_key_shares)) + 1) {
Err(InvalidTransaction::Custom(5))?;
} }
// Perform the removal
let Some(removal_index) =
participants.iter().position(|participant| &participant.0 == to_remove)
else {
Err(InvalidTransaction::Custom(6))?
};
participants.remove(removal_index);
// Verify the signature with the MuSig key of the signers // Verify the signature with the MuSig key of the signers
if !musig_key(set, signers) // We theoretically don't need set_keys_message to bind to removed_participants, as the
.verify(&remove_participant_message(&set, *to_remove), signature) // key we're signing with effectively already does so, yet there's no reason not to
if !musig_key(set, &signers)
.verify(&set_keys_message(&set, removed_participants, key_pair), signature)
{ {
Err(InvalidTransaction::BadProof)?; Err(InvalidTransaction::BadProof)?;
} }
// Set the new MuSig key
MuSigKeys::<T>::set(
set,
Some(musig_key(set, &participants.iter().map(|(id, _)| *id).collect::<Vec<_>>())),
);
Participants::<T>::set(network, Some(participants));
// This does not remove from TotalAllocatedStake or InSet in order to:
// 1) Not decrease the stake present in this set. This means removed participants are
// still liable for the economic security of the external network. This prevents
// a decided set, which is economically secure, from falling below the threshold.
// 2) Not allow parties removed to immediately deallocate, per commentary on deallocation
// scheduling (https://github.com/serai-dex/serai/issues/394).
Pallet::<T>::deposit_event(Event::ParticipantRemoved { set, removed: *to_remove });
ValidTransaction::with_tag_prefix("ValidatorSets") ValidTransaction::with_tag_prefix("ValidatorSets")
.and_provides((1, set, to_remove)) .and_provides(set)
.longevity(u64::MAX) .longevity(u64::MAX)
.propagate(true) .propagate(true)
.build() .build()

View File

@@ -98,14 +98,13 @@ pub fn musig_key(set: ValidatorSet, set_keys: &[Public]) -> Public {
Public(dkg::musig::musig_key::<Ristretto>(&musig_context(set), &keys).unwrap().to_bytes()) Public(dkg::musig::musig_key::<Ristretto>(&musig_context(set), &keys).unwrap().to_bytes())
} }
/// The message for the remove_participant signature.
pub fn remove_participant_message(set: &ValidatorSet, removed: Public) -> Vec<u8> {
(b"ValidatorSets-remove_participant", set, removed).encode()
}
/// The message for the set_keys signature. /// The message for the set_keys signature.
pub fn set_keys_message(set: &ValidatorSet, key_pair: &KeyPair) -> Vec<u8> { pub fn set_keys_message(
(b"ValidatorSets-set_keys", set, key_pair).encode() set: &ValidatorSet,
removed_participants: &[Public],
key_pair: &KeyPair,
) -> Vec<u8> {
(b"ValidatorSets-set_keys", set, removed_participants, key_pair).encode()
} }
/// For a set of validators whose key shares may exceed the maximum, reduce until they equal the /// For a set of validators whose key shares may exceed the maximum, reduce until they equal the