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

View File

@@ -28,7 +28,7 @@ use tokio::{sync::mpsc, time::sleep};
use crate::{
Db,
processors::Processors,
tributary::{TributarySpec, SeraiDkgRemoval, SeraiDkgCompleted},
tributary::{TributarySpec, SeraiDkgCompleted},
};
mod db;
@@ -222,19 +222,6 @@ async fn handle_block<D: Db, Pro: Processors>(
// Define an indexed event ID.
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
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

View File

@@ -347,11 +347,18 @@ async fn dkg_test() {
}
#[async_trait::async_trait]
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!(removed.is_empty());
assert_eq!(self.key_pair, key_pair);
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(
frost::dkg::musig::musig_key::<Ristretto>(
&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

View File

@@ -26,17 +26,14 @@ mod sync;
#[async_trait::async_trait]
impl PublishSeraiTransaction for () {
async fn publish_set_keys(&self, _set: ValidatorSet, _key_pair: KeyPair, _signature: Signature) {
panic!("publish_set_keys was called in test")
}
async fn publish_remove_participant(
async fn publish_set_keys(
&self,
_set: ValidatorSet,
_removing: [u8; 32],
_signers: Vec<SeraiAddress>,
_removed: Vec<SeraiAddress>,
_key_pair: KeyPair,
_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),
});
{
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];
OsRng.fill_bytes(&mut block);

View File

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

View File

@@ -1,8 +1,6 @@
use core::ops::Deref;
use std::collections::HashMap;
use rand_core::OsRng;
use zeroize::{Zeroize, Zeroizing};
use ciphersuite::{group::GroupEncoding, Ciphersuite, Ristretto};
@@ -25,7 +23,7 @@ use crate::{
processors::Processors,
tributary::{
*,
signing_protocol::{DkgConfirmer, DkgRemoval},
signing_protocol::DkgConfirmer,
scanner::{
RecognizedIdType, RIDTrait, PublishSeraiTransaction, PTTTrait, TributaryBlockHandler,
},
@@ -278,21 +276,6 @@ impl<
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
// execution occurs
pub(crate) async fn handle_application_tx(&mut self, tx: Transaction) {
@@ -582,7 +565,15 @@ impl<
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) => {
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) => {
AttemptDb::recognize_topic(
self.txn,

View File

@@ -66,16 +66,6 @@ pub fn removed_as_of_set_keys(
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>(
txn: &mut D::Transaction<'_>,
tributary: &Tributary<D, Transaction, P>,

View File

@@ -1,8 +1,6 @@
use core::{marker::PhantomData, ops::Deref, future::Future, time::Duration};
use std::sync::Arc;
use rand_core::OsRng;
use zeroize::Zeroizing;
use ciphersuite::{group::GroupEncoding, Ciphersuite, Ristretto};
@@ -29,13 +27,7 @@ use tributary::{
},
};
use crate::{
Db,
processors::Processors,
substrate::BatchInstructionsHashDb,
tributary::{*, signing_protocol::*},
P2p,
};
use crate::{Db, processors::Processors, substrate::BatchInstructionsHashDb, tributary::*, P2p};
#[derive(Clone, Copy, PartialEq, Eq, Debug, Encode, Decode)]
pub enum RecognizedIdType {
@@ -72,12 +64,11 @@ impl<
#[async_trait::async_trait]
pub trait PublishSeraiTransaction {
async fn publish_set_keys(&self, set: ValidatorSet, key_pair: KeyPair, signature: Signature);
async fn publish_remove_participant(
async fn publish_set_keys(
&self,
set: ValidatorSet,
removing: [u8; 32],
signers: Vec<SeraiAddress>,
removed: Vec<SeraiAddress>,
key_pair: KeyPair,
signature: Signature,
);
}
@@ -137,8 +128,14 @@ mod impl_pst_for_serai {
#[async_trait::async_trait]
impl PublishSeraiTransaction for Serai {
async fn publish_set_keys(&self, set: ValidatorSet, key_pair: KeyPair, signature: Signature) {
let tx = SeraiValidatorSets::set_keys(set.network, key_pair, signature);
async fn publish_set_keys(
&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 {
if matches!(serai.keys(set).await, Ok(Some(_))) {
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:?}");
}
}
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,
> 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) {
// 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
// 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
@@ -404,16 +335,6 @@ impl<
Topic::DkgConfirmation => {
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) => {
let id = processor_messages::coordinator::SubstrateSignId {
session: self.spec.set().session,

View File

@@ -1,9 +1,8 @@
/*
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,
and for removing another validator before the DKG completes, an operation requiring a
supermajority of validators.
This is used for confirming the results of a DKG on-chain, an operation requiring all validators
which aren't specified as removed while still satisfying a supermajority.
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,
@@ -65,7 +64,7 @@ use rand_core::OsRng;
use blake2::{Digest, Blake2s256};
use ciphersuite::{
group::{ff::PrimeField, Group, GroupEncoding},
group::{ff::PrimeField, GroupEncoding},
Ciphersuite, Ristretto,
};
use frost::{
@@ -79,10 +78,8 @@ use frost_schnorrkel::Schnorrkel;
use scale::Encode;
use serai_client::{
Public, SeraiAddress,
validator_sets::primitives::{
KeyPair, musig_context, set_keys_message, remove_participant_message,
},
Public,
validator_sets::primitives::{KeyPair, musig_context, set_keys_message},
};
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
// 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(
spec: &TributarySpec,
removed: &[<Ristretto as Ciphersuite>::G],
our_key: &Zeroizing<<Ristretto as Ciphersuite>::F>,
mut map: HashMap<Participant, Vec<u8>>,
sort_by_keys: bool,
) -> (Vec<<Ristretto as Ciphersuite>::G>, HashMap<Participant, Vec<u8>>) {
// Insert our own index so calculations aren't offset
let our_threshold_i =
@@ -243,10 +236,6 @@ fn threshold_i_map_to_keys_and_musig_i_map(
for threshold_i in threshold_is {
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
let mut participants = vec![];
@@ -302,15 +291,13 @@ impl<T: DbTxn> DkgConfirmer<'_, T> {
key_pair: &KeyPair,
) -> Result<(AlgorithmSignatureMachine<Ristretto, Schnorrkel>, [u8; 32]), Participant> {
let participants = self.spec.validators().iter().map(|val| val.0).collect::<Vec<_>>();
let preprocesses = threshold_i_map_to_keys_and_musig_i_map(
self.spec,
&self.removed,
self.key,
preprocesses,
false,
)
.1;
let msg = set_keys_message(&self.spec.set(), key_pair);
let preprocesses =
threshold_i_map_to_keys_and_musig_i_map(self.spec, &self.removed, self.key, preprocesses).1;
let msg = set_keys_message(
&self.spec.set(),
&self.removed.iter().map(|key| Public(key.to_bytes())).collect::<Vec<_>>(),
key_pair,
);
self.signing_protocol().share_internal(&participants, preprocesses, &msg)
}
// 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>>,
) -> Result<[u8; 64], Participant> {
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
.share_internal(preprocesses, key_pair)
@@ -339,83 +326,3 @@ impl<T: DbTxn> DkgConfirmer<'_, T> {
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,
},
DkgRemoval(SignData<[u8; 32]>),
// Co-sign a Substrate block.
CosignSubstrateBlock([u8; 32]),
@@ -223,9 +221,6 @@ impl Debug for Transaction {
.field("attempt", attempt)
.field("signer", &hex::encode(signed.signer.to_bytes()))
.finish_non_exhaustive(),
Transaction::DkgRemoval(sign_data) => {
fmt.debug_struct("Transaction::DkgRemoval").field("sign_data", sign_data).finish()
}
Transaction::CosignSubstrateBlock(block) => fmt
.debug_struct("Transaction::CosignSubstrateBlock")
.field("block", &hex::encode(block))
@@ -389,15 +384,13 @@ impl ReadWrite for Transaction {
Ok(Transaction::DkgConfirmed { attempt, confirmation_share, signed })
}
5 => SignData::read(reader).map(Transaction::DkgRemoval),
6 => {
5 => {
let mut block = [0; 32];
reader.read_exact(&mut block)?;
Ok(Transaction::CosignSubstrateBlock(block))
}
7 => {
6 => {
let mut block = [0; 32];
reader.read_exact(&mut block)?;
let mut batch = [0; 4];
@@ -405,16 +398,16 @@ impl ReadWrite for Transaction {
Ok(Transaction::Batch { block, batch: u32::from_le_bytes(batch) })
}
8 => {
7 => {
let mut block = [0; 8];
reader.read_exact(&mut block)?;
Ok(Transaction::SubstrateBlock(u64::from_le_bytes(block)))
}
9 => SignData::read(reader).map(Transaction::SubstrateSign),
10 => SignData::read(reader).map(Transaction::Sign),
8 => SignData::read(reader).map(Transaction::SubstrateSign),
9 => SignData::read(reader).map(Transaction::Sign),
11 => {
10 => {
let mut plan = [0; 32];
reader.read_exact(&mut plan)?;
@@ -512,37 +505,32 @@ impl ReadWrite for Transaction {
signed.write_without_nonce(writer)
}
Transaction::DkgRemoval(data) => {
writer.write_all(&[5])?;
data.write(writer)
}
Transaction::CosignSubstrateBlock(block) => {
writer.write_all(&[6])?;
writer.write_all(&[5])?;
writer.write_all(block)
}
Transaction::Batch { block, batch } => {
writer.write_all(&[7])?;
writer.write_all(&[6])?;
writer.write_all(block)?;
writer.write_all(&batch.to_le_bytes())
}
Transaction::SubstrateBlock(block) => {
writer.write_all(&[8])?;
writer.write_all(&[7])?;
writer.write_all(&block.to_le_bytes())
}
Transaction::SubstrateSign(data) => {
writer.write_all(&[9])?;
writer.write_all(&[8])?;
data.write(writer)
}
Transaction::Sign(data) => {
writer.write_all(&[10])?;
writer.write_all(&[9])?;
data.write(writer)
}
Transaction::SignCompleted { plan, tx_hash, first_signer, signature } => {
writer.write_all(&[11])?;
writer.write_all(&[10])?;
writer.write_all(plan)?;
writer
.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)
}
Transaction::DkgRemoval(data) => {
TransactionKind::Signed((b"dkg_removal", data.plan, data.attempt).encode(), &data.signed)
}
Transaction::CosignSubstrateBlock(_) => TransactionKind::Provided("cosign"),
Transaction::Batch { .. } => TransactionKind::Provided("batch"),
@@ -601,7 +585,7 @@ impl TransactionTrait for Transaction {
}
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 !signature.verify(*first_signer, self.sign_completed_challenge()) {
@@ -644,8 +628,6 @@ impl Transaction {
Transaction::InvalidDkgShare { .. } => 2,
Transaction::DkgConfirmed { .. } => 2,
Transaction::DkgRemoval(data) => data.label.nonce(),
Transaction::CosignSubstrateBlock(_) => panic!("signing CosignSubstrateBlock"),
Transaction::Batch { .. } => panic!("signing Batch"),
@@ -666,8 +648,6 @@ impl Transaction {
Transaction::InvalidDkgShare { ref mut signed, .. } => signed,
Transaction::DkgConfirmed { ref mut signed, .. } => signed,
Transaction::DkgRemoval(ref mut data) => &mut data.signed,
Transaction::CosignSubstrateBlock(_) => panic!("signing CosignSubstrateBlock"),
Transaction::Batch { .. } => panic!("signing Batch"),