Add a cosigning protocol to ensure finalizations are unique (#433)

* Add a function to deterministically decide which Serai blocks should be co-signed

Has a 5 minute latency between co-signs, also used as the maximal latency
before a co-sign is started.

* Get all active tributaries we're in at a specific block

* Add and route CosignSubstrateBlock, a new provided TX

* Split queued cosigns per network

* Rename BatchSignId to SubstrateSignId

* Add SubstrateSignableId, a meta-type for either Batch or Block, and modularize around it

* Handle the CosignSubstrateBlock provided TX

* Revert substrate_signer.rs to develop (and patch to still work)

Due to SubstrateSigner moving when the prior multisig closes, yet cosigning
occurring with the most recent key, a single SubstrateSigner can be reused.
We could manage multiple SubstrateSigners, yet considering the much lower
specifications for cosigning, I'd rather treat it distinctly.

* Route cosigning through the processor

* Add note to rename SubstrateSigner post-PR

I don't want to do so now in order to preserve the diff's clarity.

* Implement cosign evaluation into the coordinator

* Get tests to compile

* Bug fixes, mark blocks without cosigners available as cosigned

* Correct the ID Batch preprocesses are saved under, add log statements

* Create a dedicated function to handle cosigns

* Correct the flow around Batch verification/queueing

Verifying `Batch`s could stall when a `Batch` was signed before its
predecessors/before the block it's contained in was cosigned (the latter being
inevitable as we can't sign a block containing a signed batch before signing
the batch).

Now, Batch verification happens on a distinct async task in order to not block
the handling of processor messages. This task is the sole caller of verify in
order to ensure last_verified_batch isn't unexpectedly mutated.

When the processor message handler needs to access it, or needs to queue a
Batch, it associates the DB TXN with a lock preventing the other task from
doing so.

This lock, as currently implemented, is a poor and inefficient design. It
should be modified to the pattern used for cosign management. Additionally, a
new primitive of a DB-backed channel may be immensely valuable.

Fixes a standing potential deadlock and a deadlock introduced with the
cosigning protocol.

* Working full-stack tests

After the last commit, this only required extending a timeout.

* Replace "co-sign" with "cosign" to make finding text easier

* Update the coordinator tests to support cosigning

* Inline prior_batch calculation to prevent panic on rotation

Noticed when doing a final review of the branch.
This commit is contained in:
Luke Parker
2023-11-15 16:57:21 -05:00
committed by GitHub
parent 79e4cce2f6
commit 96f1d26f7a
29 changed files with 1900 additions and 348 deletions

View File

@@ -156,20 +156,37 @@ pub mod sign {
pub mod coordinator {
use super::*;
pub fn cosign_block_msg(block: [u8; 32]) -> Vec<u8> {
const DST: &[u8] = b"Cosign";
let mut res = vec![u8::try_from(DST.len()).unwrap()];
res.extend(DST);
res.extend(block);
res
}
#[derive(
Clone, Copy, PartialEq, Eq, Hash, Debug, Zeroize, Encode, Decode, Serialize, Deserialize,
)]
pub enum SubstrateSignableId {
CosigningSubstrateBlock([u8; 32]),
Batch([u8; 5]),
}
#[derive(Clone, PartialEq, Eq, Hash, Debug, Zeroize, Encode, Decode, Serialize, Deserialize)]
pub struct BatchSignId {
pub struct SubstrateSignId {
pub key: [u8; 32],
pub id: [u8; 5],
pub id: SubstrateSignableId,
pub attempt: u32,
}
#[derive(Clone, PartialEq, Eq, Debug, Serialize, Deserialize)]
pub enum CoordinatorMessage {
CosignSubstrateBlock { id: SubstrateSignId },
// Uses Vec<u8> instead of [u8; 64] since serde Deserialize isn't implemented for [u8; 64]
BatchPreprocesses { id: BatchSignId, preprocesses: HashMap<Participant, Vec<u8>> },
BatchShares { id: BatchSignId, shares: HashMap<Participant, [u8; 32]> },
SubstratePreprocesses { id: SubstrateSignId, preprocesses: HashMap<Participant, Vec<u8>> },
SubstrateShares { id: SubstrateSignId, shares: HashMap<Participant, [u8; 32]> },
// Re-attempt a batch signing protocol.
BatchReattempt { id: BatchSignId },
BatchReattempt { id: SubstrateSignId },
}
impl CoordinatorMessage {
@@ -179,16 +196,18 @@ pub mod coordinator {
// This synchrony obtained lets us ignore the synchrony requirement offered here
pub fn required_block(&self) -> Option<BlockHash> {
match self {
CoordinatorMessage::BatchPreprocesses { .. } => None,
CoordinatorMessage::BatchShares { .. } => None,
CoordinatorMessage::CosignSubstrateBlock { .. } => None,
CoordinatorMessage::SubstratePreprocesses { .. } => None,
CoordinatorMessage::SubstrateShares { .. } => None,
CoordinatorMessage::BatchReattempt { .. } => None,
}
}
pub fn key(&self) -> &[u8] {
match self {
CoordinatorMessage::BatchPreprocesses { id, .. } => &id.key,
CoordinatorMessage::BatchShares { id, .. } => &id.key,
CoordinatorMessage::CosignSubstrateBlock { id } => &id.key,
CoordinatorMessage::SubstratePreprocesses { id, .. } => &id.key,
CoordinatorMessage::SubstrateShares { id, .. } => &id.key,
CoordinatorMessage::BatchReattempt { id } => &id.key,
}
}
@@ -203,9 +222,11 @@ pub mod coordinator {
#[derive(Clone, PartialEq, Eq, Debug, Zeroize, Serialize, Deserialize)]
pub enum ProcessorMessage {
SubstrateBlockAck { network: NetworkId, block: u64, plans: Vec<PlanMeta> },
InvalidParticipant { id: BatchSignId, participant: Participant },
BatchPreprocess { id: BatchSignId, block: BlockHash, preprocesses: Vec<Vec<u8>> },
BatchShare { id: BatchSignId, shares: Vec<[u8; 32]> },
InvalidParticipant { id: SubstrateSignId, participant: Participant },
CosignPreprocess { id: SubstrateSignId, preprocesses: Vec<Vec<u8>> },
BatchPreprocess { id: SubstrateSignId, block: BlockHash, preprocesses: Vec<Vec<u8>> },
SubstrateShare { id: SubstrateSignId, shares: Vec<[u8; 32]> },
CosignedBlock { block: [u8; 32], signature: Vec<u8> },
}
}
@@ -350,10 +371,12 @@ impl CoordinatorMessage {
}
CoordinatorMessage::Coordinator(msg) => {
let (sub, id) = match msg {
// Unique since this embeds the batch ID (hash of it, including its network) and attempt
coordinator::CoordinatorMessage::BatchPreprocesses { id, .. } => (0, id.encode()),
coordinator::CoordinatorMessage::BatchShares { id, .. } => (1, id.encode()),
coordinator::CoordinatorMessage::BatchReattempt { id, .. } => (2, id.encode()),
// Unique since this is the entire message
coordinator::CoordinatorMessage::CosignSubstrateBlock { id } => (0, id.encode()),
// Unique since this embeds the batch ID (including its network) and attempt
coordinator::CoordinatorMessage::SubstratePreprocesses { id, .. } => (1, id.encode()),
coordinator::CoordinatorMessage::SubstrateShares { id, .. } => (2, id.encode()),
coordinator::CoordinatorMessage::BatchReattempt { id, .. } => (3, id.encode()),
};
let mut res = vec![COORDINATOR_UID, TYPE_COORDINATOR_UID, sub];
@@ -420,10 +443,12 @@ impl ProcessorMessage {
coordinator::ProcessorMessage::SubstrateBlockAck { network, block, .. } => {
(0, (network, block).encode())
}
// Unique since BatchSignId
// Unique since SubstrateSignId
coordinator::ProcessorMessage::InvalidParticipant { id, .. } => (1, id.encode()),
coordinator::ProcessorMessage::BatchPreprocess { id, .. } => (2, id.encode()),
coordinator::ProcessorMessage::BatchShare { id, .. } => (3, id.encode()),
coordinator::ProcessorMessage::CosignPreprocess { id, .. } => (2, id.encode()),
coordinator::ProcessorMessage::BatchPreprocess { id, .. } => (3, id.encode()),
coordinator::ProcessorMessage::SubstrateShare { id, .. } => (4, id.encode()),
coordinator::ProcessorMessage::CosignedBlock { block, .. } => (5, block.encode()),
};
let mut res = vec![PROCESSSOR_UID, TYPE_COORDINATOR_UID, sub];

286
processor/src/cosigner.rs Normal file
View File

@@ -0,0 +1,286 @@
use core::fmt;
use std::collections::HashMap;
use rand_core::OsRng;
use ciphersuite::group::GroupEncoding;
use frost::{
curve::Ristretto,
ThresholdKeys, FrostError,
algorithm::Algorithm,
sign::{
Writable, PreprocessMachine, SignMachine, SignatureMachine, AlgorithmMachine,
AlgorithmSignMachine, AlgorithmSignatureMachine,
},
};
use frost_schnorrkel::Schnorrkel;
use log::{info, warn};
use scale::Encode;
use messages::coordinator::*;
use crate::{Get, DbTxn, create_db};
create_db! {
CosignerDb {
Completed: (id: [u8; 32]) -> (),
Attempt: (id: [u8; 32], attempt: u32) -> ()
}
}
type Preprocess = <AlgorithmMachine<Ristretto, Schnorrkel> as PreprocessMachine>::Preprocess;
type SignatureShare = <AlgorithmSignMachine<Ristretto, Schnorrkel> as SignMachine<
<Schnorrkel as Algorithm<Ristretto>>::Signature,
>>::SignatureShare;
pub struct Cosigner {
#[allow(dead_code)] // False positive
keys: Vec<ThresholdKeys<Ristretto>>,
id: [u8; 32],
attempt: u32,
#[allow(clippy::type_complexity)]
preprocessing: Option<(Vec<AlgorithmSignMachine<Ristretto, Schnorrkel>>, Vec<Preprocess>)>,
#[allow(clippy::type_complexity)]
signing: Option<(AlgorithmSignatureMachine<Ristretto, Schnorrkel>, Vec<SignatureShare>)>,
}
impl fmt::Debug for Cosigner {
fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result {
fmt
.debug_struct("Cosigner")
.field("id", &self.id)
.field("attempt", &self.attempt)
.field("preprocessing", &self.preprocessing.is_some())
.field("signing", &self.signing.is_some())
.finish_non_exhaustive()
}
}
impl Cosigner {
pub fn new(
txn: &mut impl DbTxn,
keys: Vec<ThresholdKeys<Ristretto>>,
id: [u8; 32],
attempt: u32,
) -> Option<(Cosigner, ProcessorMessage)> {
assert!(!keys.is_empty());
if Completed::get(txn, id).is_some() {
return None;
}
if Attempt::get(txn, id, attempt).is_some() {
warn!(
"already attempted cosigning {}, attempt #{}. this is an error if we didn't reboot",
hex::encode(id),
attempt,
);
return None;
}
Attempt::set(txn, id, attempt, &());
info!("cosigning block {} with attempt #{}", hex::encode(id), attempt);
let mut machines = vec![];
let mut preprocesses = vec![];
let mut serialized_preprocesses = vec![];
for keys in &keys {
// b"substrate" is a literal from sp-core
let machine = AlgorithmMachine::new(Schnorrkel::new(b"substrate"), keys.clone());
let (machine, preprocess) = machine.preprocess(&mut OsRng);
machines.push(machine);
serialized_preprocesses.push(preprocess.serialize());
preprocesses.push(preprocess);
}
let preprocessing = Some((machines, preprocesses));
let substrate_sign_id = SubstrateSignId {
key: keys[0].group_key().to_bytes(),
id: SubstrateSignableId::CosigningSubstrateBlock(id),
attempt,
};
Some((
Cosigner { keys, id, attempt, preprocessing, signing: None },
ProcessorMessage::CosignPreprocess {
id: substrate_sign_id,
preprocesses: serialized_preprocesses,
},
))
}
#[must_use]
pub async fn handle(
&mut self,
txn: &mut impl DbTxn,
msg: CoordinatorMessage,
) -> Option<ProcessorMessage> {
match msg {
CoordinatorMessage::CosignSubstrateBlock { .. } => {
panic!("Cosigner passed CosignSubstrateBlock")
}
CoordinatorMessage::SubstratePreprocesses { id, preprocesses } => {
assert_eq!(id.key, self.keys[0].group_key().to_bytes());
let SubstrateSignableId::CosigningSubstrateBlock(block) = id.id else {
panic!("cosigner passed Batch")
};
if block != self.id {
panic!("given preprocesses for a distinct block than cosigner is signing")
}
if id.attempt != self.attempt {
panic!("given preprocesses for a distinct attempt than cosigner is signing")
}
let (machines, our_preprocesses) = match self.preprocessing.take() {
// Either rebooted or RPC error, or some invariant
None => {
warn!(
"not preprocessing for {}. this is an error if we didn't reboot",
hex::encode(block),
);
return None;
}
Some(preprocess) => preprocess,
};
let mut parsed = HashMap::new();
for l in {
let mut keys = preprocesses.keys().cloned().collect::<Vec<_>>();
keys.sort();
keys
} {
let mut preprocess_ref = preprocesses.get(&l).unwrap().as_slice();
let Ok(res) = machines[0].read_preprocess(&mut preprocess_ref) else {
return Some(ProcessorMessage::InvalidParticipant { id, participant: l });
};
if !preprocess_ref.is_empty() {
return Some(ProcessorMessage::InvalidParticipant { id, participant: l });
}
parsed.insert(l, res);
}
let preprocesses = parsed;
// Only keep a single machine as we only need one to get the signature
let mut signature_machine = None;
let mut shares = vec![];
let mut serialized_shares = vec![];
for (m, machine) in machines.into_iter().enumerate() {
let mut preprocesses = preprocesses.clone();
for (i, our_preprocess) in our_preprocesses.clone().into_iter().enumerate() {
if i != m {
assert!(preprocesses.insert(self.keys[i].params().i(), our_preprocess).is_none());
}
}
let (machine, share) = match machine.sign(preprocesses, &cosign_block_msg(self.id)) {
Ok(res) => res,
Err(e) => match e {
FrostError::InternalError(_) |
FrostError::InvalidParticipant(_, _) |
FrostError::InvalidSigningSet(_) |
FrostError::InvalidParticipantQuantity(_, _) |
FrostError::DuplicatedParticipant(_) |
FrostError::MissingParticipant(_) => unreachable!(),
FrostError::InvalidPreprocess(l) | FrostError::InvalidShare(l) => {
return Some(ProcessorMessage::InvalidParticipant { id, participant: l })
}
},
};
if m == 0 {
signature_machine = Some(machine);
}
let mut share_bytes = [0; 32];
share_bytes.copy_from_slice(&share.serialize());
serialized_shares.push(share_bytes);
shares.push(share);
}
self.signing = Some((signature_machine.unwrap(), shares));
// Broadcast our shares
Some(ProcessorMessage::SubstrateShare { id, shares: serialized_shares })
}
CoordinatorMessage::SubstrateShares { id, shares } => {
assert_eq!(id.key, self.keys[0].group_key().to_bytes());
let SubstrateSignableId::CosigningSubstrateBlock(block) = id.id else {
panic!("cosigner passed Batch")
};
if block != self.id {
panic!("given preprocesses for a distinct block than cosigner is signing")
}
if id.attempt != self.attempt {
panic!("given preprocesses for a distinct attempt than cosigner is signing")
}
let (machine, our_shares) = match self.signing.take() {
// Rebooted, RPC error, or some invariant
None => {
// If preprocessing has this ID, it means we were never sent the preprocess by the
// coordinator
if self.preprocessing.is_some() {
panic!("never preprocessed yet signing?");
}
warn!(
"not preprocessing for {}. this is an error if we didn't reboot",
hex::encode(block)
);
return None;
}
Some(signing) => signing,
};
let mut parsed = HashMap::new();
for l in {
let mut keys = shares.keys().cloned().collect::<Vec<_>>();
keys.sort();
keys
} {
let mut share_ref = shares.get(&l).unwrap().as_slice();
let Ok(res) = machine.read_share(&mut share_ref) else {
return Some(ProcessorMessage::InvalidParticipant { id, participant: l });
};
if !share_ref.is_empty() {
return Some(ProcessorMessage::InvalidParticipant { id, participant: l });
}
parsed.insert(l, res);
}
let mut shares = parsed;
for (i, our_share) in our_shares.into_iter().enumerate().skip(1) {
assert!(shares.insert(self.keys[i].params().i(), our_share).is_none());
}
let sig = match machine.complete(shares) {
Ok(res) => res,
Err(e) => match e {
FrostError::InternalError(_) |
FrostError::InvalidParticipant(_, _) |
FrostError::InvalidSigningSet(_) |
FrostError::InvalidParticipantQuantity(_, _) |
FrostError::DuplicatedParticipant(_) |
FrostError::MissingParticipant(_) => unreachable!(),
FrostError::InvalidPreprocess(l) | FrostError::InvalidShare(l) => {
return Some(ProcessorMessage::InvalidParticipant { id, participant: l })
}
},
};
info!("cosigned {} with attempt #{}", hex::encode(block), id.attempt);
Completed::set(txn, block, &());
Some(ProcessorMessage::CosignedBlock { block, signature: sig.to_bytes().to_vec() })
}
CoordinatorMessage::BatchReattempt { .. } => panic!("BatchReattempt passed to Cosigner"),
}
}
}

View File

@@ -36,7 +36,10 @@ create_db!(
// Overwriting its commitments would be accordingly poor
CommitmentsDb: (key: &KeyGenId) -> HashMap<Participant, Vec<u8>>,
GeneratedKeysDb: (set: &ValidatorSet, substrate_key: &[u8; 32], network_key: &[u8]) -> Vec<u8>,
KeysDb: (network_key: &[u8]) -> Vec<u8>
// These do assume a key is only used once across sets, which holds true so long as a single
// participant is honest in their execution of the protocol
KeysDb: (network_key: &[u8]) -> Vec<u8>,
NetworkKey: (substrate_key: [u8; 32]) -> Vec<u8>
}
);
@@ -102,6 +105,7 @@ impl KeysDb {
keys.1[0].group_key().to_bytes().as_ref(),
);
txn.put(KeysDb::key(keys.1[0].group_key().to_bytes().as_ref()), keys_vec);
NetworkKey::set(txn, key_pair.0.into(), &key_pair.1.clone().into_inner());
keys
}
@@ -115,6 +119,16 @@ impl KeysDb {
assert_eq!(&res.1[0].group_key(), network_key);
Some(res)
}
pub fn substrate_keys_by_substrate_key<N: Network>(
getter: &impl Get,
substrate_key: &[u8; 32],
) -> Option<Vec<ThresholdKeys<Ristretto>>> {
let network_key = NetworkKey::get(getter, *substrate_key)?;
let res = GeneratedKeysDb::read_keys::<N>(getter, &Self::key(&network_key))?.1;
assert_eq!(&res.0[0].group_key().to_bytes(), substrate_key);
Some(res.0)
}
}
type SecretShareMachines<N> =
@@ -152,6 +166,13 @@ impl<N: Network, D: Db> KeyGen<N, D> {
KeysDb::keys::<N>(&self.db, key)
}
pub fn substrate_keys_by_substrate_key(
&self,
substrate_key: &[u8; 32],
) -> Option<Vec<ThresholdKeys<Ristretto>>> {
KeysDb::substrate_keys_by_substrate_key::<N>(&self.db, substrate_key)
}
pub async fn handle(
&mut self,
txn: &mut D::Transaction<'_>,

View File

@@ -13,7 +13,12 @@ use serai_client::{
validator_sets::primitives::{ValidatorSet, KeyPair},
};
use messages::{coordinator::PlanMeta, CoordinatorMessage};
use messages::{
coordinator::{
SubstrateSignableId, PlanMeta, CoordinatorMessage as CoordinatorCoordinatorMessage,
},
CoordinatorMessage,
};
use serai_env as env;
@@ -44,6 +49,9 @@ use key_gen::{KeyConfirmed, KeyGen};
mod signer;
use signer::Signer;
mod cosigner;
use cosigner::Cosigner;
mod substrate_signer;
use substrate_signer::SubstrateSigner;
@@ -86,6 +94,9 @@ struct TributaryMutable<N: Network, D: Db> {
// There should only be one SubstrateSigner at a time (see #277)
substrate_signer: Option<SubstrateSigner<D>>,
// Solely mutated by the tributary.
cosigner: Option<Cosigner>,
}
// Items which are mutably borrowed by Substrate.
@@ -218,16 +229,58 @@ async fn handle_coordinator_msg<D: Db, N: Network, Co: Coordinator>(
}
CoordinatorMessage::Coordinator(msg) => {
if let Some(msg) = tributary_mutable
.substrate_signer
.as_mut()
.expect(
"coordinator told us to sign a batch when we don't have a Substrate signer at this time",
)
.handle(txn, msg)
.await
{
coordinator.send(msg).await;
let is_batch = match msg {
CoordinatorCoordinatorMessage::CosignSubstrateBlock { .. } => false,
CoordinatorCoordinatorMessage::SubstratePreprocesses { ref id, .. } => {
matches!(&id.id, SubstrateSignableId::Batch(_))
}
CoordinatorCoordinatorMessage::SubstrateShares { ref id, .. } => {
matches!(&id.id, SubstrateSignableId::Batch(_))
}
CoordinatorCoordinatorMessage::BatchReattempt { .. } => true,
};
if is_batch {
if let Some(msg) = tributary_mutable
.substrate_signer
.as_mut()
.expect(
"coordinator told us to sign a batch when we don't currently have a Substrate signer",
)
.handle(txn, msg)
.await
{
coordinator.send(msg).await;
}
} else {
match msg {
CoordinatorCoordinatorMessage::CosignSubstrateBlock { id } => {
let SubstrateSignableId::CosigningSubstrateBlock(block) = id.id else {
panic!("CosignSubstrateBlock id didn't have a CosigningSubstrateBlock")
};
let Some(keys) = tributary_mutable.key_gen.substrate_keys_by_substrate_key(&id.key)
else {
panic!("didn't have key shares for the key we were told to cosign with");
};
if let Some((cosigner, msg)) = Cosigner::new(txn, keys, block, id.attempt) {
tributary_mutable.cosigner = Some(cosigner);
coordinator.send(msg).await;
} else {
log::warn!("Cosigner::new returned None");
}
}
_ => {
if let Some(cosigner) = tributary_mutable.cosigner.as_mut() {
if let Some(msg) = cosigner.handle(txn, msg).await {
coordinator.send(msg).await;
}
} else {
log::warn!(
"received message for cosigner yet didn't have a cosigner. {}",
"this is an error if we didn't reboot",
);
}
}
}
}
}
@@ -240,6 +293,7 @@ async fn handle_coordinator_msg<D: Db, N: Network, Co: Coordinator>(
if context.network_latest_finalized_block.0 == [0; 32] {
assert!(tributary_mutable.signers.is_empty());
assert!(tributary_mutable.substrate_signer.is_none());
assert!(tributary_mutable.cosigner.is_none());
// We can't check this as existing is no longer pub
// assert!(substrate_mutable.existing.as_ref().is_none());
@@ -337,7 +391,7 @@ async fn handle_coordinator_msg<D: Db, N: Network, Co: Coordinator>(
}
}
// Since this block was acknowledged, we no longer have to sign the batches for it
// Since this block was acknowledged, we no longer have to sign the batches within it
if let Some(substrate_signer) = tributary_mutable.substrate_signer.as_mut() {
for batch_id in batches {
substrate_signer.batch_signed(txn, batch_id);
@@ -480,7 +534,11 @@ async fn boot<N: Network, D: Db, Co: Coordinator>(
// This hedges against being dropped due to full mempools, temporarily too low of a fee...
tokio::spawn(Signer::<N, D>::rebroadcast_task(raw_db.clone(), network.clone()));
(main_db, TributaryMutable { key_gen, substrate_signer, signers }, multisig_manager)
(
main_db,
TributaryMutable { key_gen, substrate_signer, cosigner: None, signers },
multisig_manager,
)
}
#[allow(clippy::await_holding_lock)] // Needed for txn, unfortunately can't be down-scoped
@@ -553,6 +611,7 @@ async fn run<N: Network, D: Db, Co: Coordinator>(mut raw_db: D, network: N, mut
for batch in batches {
info!("created batch {} ({} instructions)", batch.id, batch.instructions.len());
// The coordinator expects BatchPreprocess to immediately follow Batch
coordinator.send(
messages::substrate::ProcessorMessage::Batch { batch: batch.clone() }
).await;

View File

@@ -48,14 +48,14 @@ impl<D: Db> SubstrateSignerDb<D> {
getter.get(Self::completed_key(id)).is_some()
}
fn attempt_key(id: &BatchSignId) -> Vec<u8> {
Self::sign_key(b"attempt", id.encode())
fn attempt_key(id: [u8; 5], attempt: u32) -> Vec<u8> {
Self::sign_key(b"attempt", (id, attempt).encode())
}
fn attempt(txn: &mut D::Transaction<'_>, id: &BatchSignId) {
txn.put(Self::attempt_key(id), []);
fn attempt(txn: &mut D::Transaction<'_>, id: [u8; 5], attempt: u32) {
txn.put(Self::attempt_key(id, attempt), []);
}
fn has_attempt<G: Get>(getter: &G, id: &BatchSignId) -> bool {
getter.get(Self::attempt_key(id)).is_some()
fn has_attempt<G: Get>(getter: &G, id: [u8; 5], attempt: u32) -> bool {
getter.get(Self::attempt_key(id, attempt)).is_some()
}
fn save_batch(txn: &mut D::Transaction<'_>, batch: &SignedBatch) {
@@ -68,6 +68,7 @@ type SignatureShare = <AlgorithmSignMachine<Ristretto, Schnorrkel> as SignMachin
<Schnorrkel as Algorithm<Ristretto>>::Signature,
>>::SignatureShare;
// TODO: Rename BatchSigner
pub struct SubstrateSigner<D: Db> {
db: PhantomData<D>,
@@ -110,22 +111,27 @@ impl<D: Db> SubstrateSigner<D> {
}
}
fn verify_id(&self, id: &BatchSignId) -> Result<(), ()> {
fn verify_id(&self, id: &SubstrateSignId) -> Result<([u8; 32], [u8; 5], u32), ()> {
let SubstrateSignId { key, id, attempt } = id;
let SubstrateSignableId::Batch(id) = id else { panic!("SubstrateSigner handed non-Batch") };
assert_eq!(key, &self.keys[0].group_key().to_bytes());
// Check the attempt lines up
match self.attempt.get(&id.id) {
match self.attempt.get(id) {
// If we don't have an attempt logged, it's because the coordinator is faulty OR because we
// rebooted OR we detected the signed batch on chain
// The latter is the expected flow for batches not actively being participated in
None => {
warn!("not attempting batch {} #{}", hex::encode(id.id), id.attempt);
warn!("not attempting batch {} #{}", hex::encode(id), attempt);
Err(())?;
}
Some(attempt) => {
if attempt != &id.attempt {
Some(our_attempt) => {
if attempt != our_attempt {
warn!(
"sent signing data for batch {} #{} yet we have attempt #{}",
hex::encode(id.id),
id.attempt,
hex::encode(id),
attempt,
attempt
);
Err(())?;
@@ -133,7 +139,7 @@ impl<D: Db> SubstrateSigner<D> {
}
}
Ok(())
Ok((*key, *id, *attempt))
}
#[must_use]
@@ -176,8 +182,7 @@ impl<D: Db> SubstrateSigner<D> {
// Update the attempt number
self.attempt.insert(id, attempt);
let id = BatchSignId { key: self.keys[0].group_key().to_bytes(), id, attempt };
info!("signing batch {} #{}", hex::encode(id.id), id.attempt);
info!("signing batch {} #{}", hex::encode(id), attempt);
// If we reboot mid-sign, the current design has us abort all signs and wait for latter
// attempts/new signing protocols
@@ -192,16 +197,15 @@ impl<D: Db> SubstrateSigner<D> {
//
// Only run if this hasn't already been attempted
// TODO: This isn't complete as this txn may not be committed with the expected timing
if SubstrateSignerDb::<D>::has_attempt(txn, &id) {
if SubstrateSignerDb::<D>::has_attempt(txn, id, attempt) {
warn!(
"already attempted batch {}, attempt #{}. this is an error if we didn't reboot",
hex::encode(id.id),
id.attempt
hex::encode(id),
attempt
);
return None;
}
SubstrateSignerDb::<D>::attempt(txn, &id);
SubstrateSignerDb::<D>::attempt(txn, id, attempt);
let mut machines = vec![];
let mut preprocesses = vec![];
@@ -215,7 +219,13 @@ impl<D: Db> SubstrateSigner<D> {
serialized_preprocesses.push(preprocess.serialize());
preprocesses.push(preprocess);
}
self.preprocessing.insert(id.id, (machines, preprocesses));
self.preprocessing.insert(id, (machines, preprocesses));
let id = SubstrateSignId {
key: self.keys[0].group_key().to_bytes(),
id: SubstrateSignableId::Batch(id),
attempt,
};
// Broadcast our preprocesses
Some(ProcessorMessage::BatchPreprocess { id, block, preprocesses: serialized_preprocesses })
@@ -246,17 +256,22 @@ impl<D: Db> SubstrateSigner<D> {
msg: CoordinatorMessage,
) -> Option<messages::ProcessorMessage> {
match msg {
CoordinatorMessage::BatchPreprocesses { id, preprocesses } => {
if self.verify_id(&id).is_err() {
return None;
}
CoordinatorMessage::CosignSubstrateBlock { .. } => {
panic!("SubstrateSigner passed CosignSubstrateBlock")
}
let (machines, our_preprocesses) = match self.preprocessing.remove(&id.id) {
CoordinatorMessage::SubstratePreprocesses { id, preprocesses } => {
let (key, id, attempt) = self.verify_id(&id).ok()?;
let substrate_sign_id =
SubstrateSignId { key, id: SubstrateSignableId::Batch(id), attempt };
let (machines, our_preprocesses) = match self.preprocessing.remove(&id) {
// Either rebooted or RPC error, or some invariant
None => {
warn!(
"not preprocessing for {}. this is an error if we didn't reboot",
hex::encode(id.id),
hex::encode(id),
);
return None;
}
@@ -271,10 +286,16 @@ impl<D: Db> SubstrateSigner<D> {
} {
let mut preprocess_ref = preprocesses.get(&l).unwrap().as_slice();
let Ok(res) = machines[0].read_preprocess(&mut preprocess_ref) else {
return Some((ProcessorMessage::InvalidParticipant { id, participant: l }).into());
return Some(
(ProcessorMessage::InvalidParticipant { id: substrate_sign_id, participant: l })
.into(),
);
};
if !preprocess_ref.is_empty() {
return Some((ProcessorMessage::InvalidParticipant { id, participant: l }).into());
return Some(
(ProcessorMessage::InvalidParticipant { id: substrate_sign_id, participant: l })
.into(),
);
}
parsed.insert(l, res);
}
@@ -292,22 +313,26 @@ impl<D: Db> SubstrateSigner<D> {
}
}
let (machine, share) =
match machine.sign(preprocesses, &batch_message(&self.signable[&id.id])) {
Ok(res) => res,
Err(e) => match e {
FrostError::InternalError(_) |
FrostError::InvalidParticipant(_, _) |
FrostError::InvalidSigningSet(_) |
FrostError::InvalidParticipantQuantity(_, _) |
FrostError::DuplicatedParticipant(_) |
FrostError::MissingParticipant(_) => unreachable!(),
let (machine, share) = match machine
.sign(preprocesses, &batch_message(&self.signable[&id]))
{
Ok(res) => res,
Err(e) => match e {
FrostError::InternalError(_) |
FrostError::InvalidParticipant(_, _) |
FrostError::InvalidSigningSet(_) |
FrostError::InvalidParticipantQuantity(_, _) |
FrostError::DuplicatedParticipant(_) |
FrostError::MissingParticipant(_) => unreachable!(),
FrostError::InvalidPreprocess(l) | FrostError::InvalidShare(l) => {
return Some((ProcessorMessage::InvalidParticipant { id, participant: l }).into())
}
},
};
FrostError::InvalidPreprocess(l) | FrostError::InvalidShare(l) => {
return Some(
(ProcessorMessage::InvalidParticipant { id: substrate_sign_id, participant: l })
.into(),
)
}
},
};
if m == 0 {
signature_machine = Some(machine);
}
@@ -318,29 +343,33 @@ impl<D: Db> SubstrateSigner<D> {
shares.push(share);
}
self.signing.insert(id.id, (signature_machine.unwrap(), shares));
self.signing.insert(id, (signature_machine.unwrap(), shares));
// Broadcast our shares
Some((ProcessorMessage::BatchShare { id, shares: serialized_shares }).into())
Some(
(ProcessorMessage::SubstrateShare { id: substrate_sign_id, shares: serialized_shares })
.into(),
)
}
CoordinatorMessage::BatchShares { id, shares } => {
if self.verify_id(&id).is_err() {
return None;
}
CoordinatorMessage::SubstrateShares { id, shares } => {
let (key, id, attempt) = self.verify_id(&id).ok()?;
let (machine, our_shares) = match self.signing.remove(&id.id) {
let substrate_sign_id =
SubstrateSignId { key, id: SubstrateSignableId::Batch(id), attempt };
let (machine, our_shares) = match self.signing.remove(&id) {
// Rebooted, RPC error, or some invariant
None => {
// If preprocessing has this ID, it means we were never sent the preprocess by the
// coordinator
if self.preprocessing.contains_key(&id.id) {
if self.preprocessing.contains_key(&id) {
panic!("never preprocessed yet signing?");
}
warn!(
"not preprocessing for {}. this is an error if we didn't reboot",
hex::encode(id.id)
hex::encode(id)
);
return None;
}
@@ -355,10 +384,16 @@ impl<D: Db> SubstrateSigner<D> {
} {
let mut share_ref = shares.get(&l).unwrap().as_slice();
let Ok(res) = machine.read_share(&mut share_ref) else {
return Some((ProcessorMessage::InvalidParticipant { id, participant: l }).into());
return Some(
(ProcessorMessage::InvalidParticipant { id: substrate_sign_id, participant: l })
.into(),
);
};
if !share_ref.is_empty() {
return Some((ProcessorMessage::InvalidParticipant { id, participant: l }).into());
return Some(
(ProcessorMessage::InvalidParticipant { id: substrate_sign_id, participant: l })
.into(),
);
}
parsed.insert(l, res);
}
@@ -379,30 +414,36 @@ impl<D: Db> SubstrateSigner<D> {
FrostError::MissingParticipant(_) => unreachable!(),
FrostError::InvalidPreprocess(l) | FrostError::InvalidShare(l) => {
return Some((ProcessorMessage::InvalidParticipant { id, participant: l }).into())
return Some(
(ProcessorMessage::InvalidParticipant { id: substrate_sign_id, participant: l })
.into(),
)
}
},
};
info!("signed batch {} with attempt #{}", hex::encode(id.id), id.attempt);
info!("signed batch {} with attempt #{}", hex::encode(id), attempt);
let batch =
SignedBatch { batch: self.signable.remove(&id.id).unwrap(), signature: sig.into() };
SignedBatch { batch: self.signable.remove(&id).unwrap(), signature: sig.into() };
// Save the batch in case it's needed for recovery
SubstrateSignerDb::<D>::save_batch(txn, &batch);
SubstrateSignerDb::<D>::complete(txn, id.id);
SubstrateSignerDb::<D>::complete(txn, id);
// Stop trying to sign for this batch
assert!(self.attempt.remove(&id.id).is_some());
assert!(self.preprocessing.remove(&id.id).is_none());
assert!(self.signing.remove(&id.id).is_none());
assert!(self.attempt.remove(&id).is_some());
assert!(self.preprocessing.remove(&id).is_none());
assert!(self.signing.remove(&id).is_none());
Some((messages::substrate::ProcessorMessage::SignedBatch { batch }).into())
}
CoordinatorMessage::BatchReattempt { id } => {
self.attempt(txn, id.id, id.attempt).await.map(Into::into)
let SubstrateSignableId::Batch(batch_id) = id.id else {
panic!("BatchReattempt passed non-Batch ID")
};
self.attempt(txn, batch_id, id.attempt).await.map(Into::into)
}
}
}

View File

@@ -0,0 +1,126 @@
use std::collections::HashMap;
use rand_core::{RngCore, OsRng};
use ciphersuite::group::GroupEncoding;
use frost::{
curve::Ristretto,
Participant,
dkg::tests::{key_gen, clone_without},
};
use sp_application_crypto::{RuntimePublic, sr25519::Public};
use serai_db::{DbTxn, Db, MemDb};
use serai_client::primitives::*;
use messages::coordinator::*;
use crate::cosigner::Cosigner;
#[tokio::test]
async fn test_cosigner() {
let keys = key_gen::<_, Ristretto>(&mut OsRng);
let participant_one = Participant::new(1).unwrap();
let block = [0xaa; 32];
let actual_id = SubstrateSignId {
key: keys.values().next().unwrap().group_key().to_bytes(),
id: SubstrateSignableId::CosigningSubstrateBlock(block),
attempt: (OsRng.next_u64() >> 32).try_into().unwrap(),
};
let mut signing_set = vec![];
while signing_set.len() < usize::from(keys.values().next().unwrap().params().t()) {
let candidate = Participant::new(
u16::try_from((OsRng.next_u64() % u64::try_from(keys.len()).unwrap()) + 1).unwrap(),
)
.unwrap();
if signing_set.contains(&candidate) {
continue;
}
signing_set.push(candidate);
}
let mut signers = HashMap::new();
let mut dbs = HashMap::new();
let mut preprocesses = HashMap::new();
for i in 1 ..= keys.len() {
let i = Participant::new(u16::try_from(i).unwrap()).unwrap();
let keys = keys.get(&i).unwrap().clone();
let mut db = MemDb::new();
let mut txn = db.txn();
let (signer, preprocess) =
Cosigner::new(&mut txn, vec![keys], block, actual_id.attempt).unwrap();
match preprocess {
// All participants should emit a preprocess
ProcessorMessage::CosignPreprocess { id, preprocesses: mut these_preprocesses } => {
assert_eq!(id, actual_id);
assert_eq!(these_preprocesses.len(), 1);
if signing_set.contains(&i) {
preprocesses.insert(i, these_preprocesses.swap_remove(0));
}
}
_ => panic!("didn't get preprocess back"),
}
txn.commit();
signers.insert(i, signer);
dbs.insert(i, db);
}
let mut shares = HashMap::new();
for i in &signing_set {
let mut txn = dbs.get_mut(i).unwrap().txn();
match signers
.get_mut(i)
.unwrap()
.handle(
&mut txn,
CoordinatorMessage::SubstratePreprocesses {
id: actual_id.clone(),
preprocesses: clone_without(&preprocesses, i),
},
)
.await
.unwrap()
{
ProcessorMessage::SubstrateShare { id, shares: mut these_shares } => {
assert_eq!(id, actual_id);
assert_eq!(these_shares.len(), 1);
shares.insert(*i, these_shares.swap_remove(0));
}
_ => panic!("didn't get share back"),
}
txn.commit();
}
for i in &signing_set {
let mut txn = dbs.get_mut(i).unwrap().txn();
match signers
.get_mut(i)
.unwrap()
.handle(
&mut txn,
CoordinatorMessage::SubstrateShares {
id: actual_id.clone(),
shares: clone_without(&shares, i),
},
)
.await
.unwrap()
{
ProcessorMessage::CosignedBlock { block: signed_block, signature } => {
assert_eq!(signed_block, block);
assert!(Public::from_raw(keys[&participant_one].group_key().to_bytes())
.verify(&cosign_block_msg(block), &Signature(signature.try_into().unwrap())));
}
_ => panic!("didn't get cosigned block back"),
}
txn.commit();
}
}

View File

@@ -7,6 +7,7 @@ pub(crate) use scanner::{test_scanner, test_no_deadlock_in_multisig_completed};
mod signer;
pub(crate) use signer::{sign, test_signer};
mod cosigner;
mod substrate_signer;
mod wallet;

View File

@@ -18,7 +18,7 @@ use serai_client::{primitives::*, in_instructions::primitives::*};
use messages::{
substrate,
coordinator::{self, BatchSignId, CoordinatorMessage},
coordinator::{self, SubstrateSignableId, SubstrateSignId, CoordinatorMessage},
ProcessorMessage,
};
use crate::substrate_signer::SubstrateSigner;
@@ -48,9 +48,9 @@ async fn test_substrate_signer() {
],
};
let actual_id = BatchSignId {
let actual_id = SubstrateSignId {
key: keys.values().next().unwrap().group_key().to_bytes(),
id: (batch.network, batch.id).encode().try_into().unwrap(),
id: SubstrateSignableId::Batch((batch.network, batch.id).encode().try_into().unwrap()),
attempt: 0,
};
@@ -107,7 +107,7 @@ async fn test_substrate_signer() {
.unwrap()
.handle(
&mut txn,
CoordinatorMessage::BatchPreprocesses {
CoordinatorMessage::SubstratePreprocesses {
id: actual_id.clone(),
preprocesses: clone_without(&preprocesses, i),
},
@@ -115,7 +115,7 @@ async fn test_substrate_signer() {
.await
.unwrap()
{
ProcessorMessage::Coordinator(coordinator::ProcessorMessage::BatchShare {
ProcessorMessage::Coordinator(coordinator::ProcessorMessage::SubstrateShare {
id,
shares: mut these_shares,
}) => {
@@ -135,7 +135,7 @@ async fn test_substrate_signer() {
.unwrap()
.handle(
&mut txn,
CoordinatorMessage::BatchShares {
CoordinatorMessage::SubstrateShares {
id: actual_id.clone(),
shares: clone_without(&shares, i),
},