Skip to content

Commit

Permalink
wip: get keygen module hooked-in
Browse files Browse the repository at this point in the history
  • Loading branch information
Thomas Braun committed Sep 18, 2023
1 parent 3e66bad commit 06ad94c
Show file tree
Hide file tree
Showing 8 changed files with 175 additions and 69 deletions.
7 changes: 0 additions & 7 deletions dkg-gadget/src/async_protocols/blockchain_interface.rs
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,6 @@ pub trait BlockchainInterface: Send + Sync + Unpin {
key: LocalKey<Secp256k1>,
session_id: SessionId,
) -> Result<(), DKGError>;
fn get_authority_set(&self) -> Vec<(KeygenPartyId, Public)>;
fn get_gossip_engine(&self) -> Option<&Self::GossipEngine>;
/// Returns the present time
fn now(&self) -> Self::Clock;
Expand All @@ -106,8 +105,6 @@ pub struct DKGProtocolEngine<
pub db: Arc<dyn crate::db::DKGDbBackend>,
pub gossip_engine: Arc<GE>,
pub aggregated_public_keys: Arc<RwLock<HashMap<SessionId, AggregatedPublicKeys>>>,
pub best_authorities: Arc<Vec<(KeygenPartyId, Public)>>,
pub authority_public_key: Arc<Public>,
pub vote_results: Arc<
RwLock<
HashMap<
Expand Down Expand Up @@ -390,10 +387,6 @@ impl<B, BE, C, GE> BlockchainInterface
self.db.store_local_key(session_id, key)
}

fn get_authority_set(&self) -> Vec<(KeygenPartyId, Public)> {
(*self.best_authorities).clone()
}

fn get_gossip_engine(&self) -> Option<&Self::GossipEngine> {
Some(&self.gossip_engine)
}
Expand Down
16 changes: 7 additions & 9 deletions dkg-gadget/src/async_protocols/frost/keygen/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,9 @@ use crate::dkg_modules::wt_frost::{FrostMessage, NetInterface};
use crate::gossip_engine::GossipEngineIface;
use crate::worker::DKGWorker;

pub struct FrostKeygen<B, BE, C, GE, BI>
pub mod proto;

pub struct FrostKeygenNetworkWrapper<B, BE, C, GE, BI>
where
B: Block,
BE: Backend<B>,
Expand All @@ -24,29 +26,25 @@ pub struct FrostKeygen<B, BE, C, GE, BI>
pub message_receiver: UnboundedReceiver<SignedDKGMessage<AuthorityId>>,
pub authority_id: AuthorityId,
pub keygen_protocol_hash: [u8; 32],
pub received_messages: HashSet<[u8;32]>
pub received_messages: HashSet<[u8;32]>,
pub engine: BI
}

impl<B, BE, C, GE, BI: BlockchainInterface> FrostKeygen<B, BE, C, GE, BI> {
pub fn new(dkg_worker: DKGWorker<B, BE, C, GE>, engine: BI, remote: AsyncProtocolRemote<C>, authority_id: AuthorityId, retry_id: usize) -> Self {
impl<B, BE, C, GE, BI: BlockchainInterface> FrostKeygenNetworkWrapper<B, BE, C, GE, BI> {
pub fn new(dkg_worker: DKGWorker<B, BE, C, GE>, engine: BI, remote: AsyncProtocolRemote<C>, authority_id: AuthorityId, keygen_protocol_hash: [u8; 32]) -> Self {
let message_receiver = remote
.rx_keygen_signing
.lock()
.take()
.expect("rx_keygen_signing already taken");

let mut data = retry_id.to_be_bytes().to_vec();
data.extend_from_slice(&remote.session_id.to_be_bytes());

let keygen_protocol_hash = sha2_256(&data);
let received_messages = HashSet::new();

Self { dkg_worker, engine, remote, message_receiver, authority_id, keygen_protocol_hash, received_messages }
}
}

impl<B, BE, C, GE, BI: BlockchainInterface> NetInterface for FrostKeygen<B, BE, C, GE, BI>
impl<B, BE, C, GE, BI: BlockchainInterface> NetInterface for FrostKeygenNetworkWrapper<B, BE, C, GE, BI>
where
B: Block,
BE: Backend<B>,
Expand Down
38 changes: 38 additions & 0 deletions dkg-gadget/src/async_protocols/frost/keygen/proto.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,38 @@
use std::future::Future;
use std::pin::Pin;
use wsts::v2::Party;
use dkg_primitives::types::DKGError;
use dkg_runtime_primitives::gossip_messages::PublicKeyMessage;
use dkg_runtime_primitives::SessionId;
use crate::async_protocols::blockchain_interface::BlockchainInterface;
use crate::dkg_modules::wt_frost::{NetInterface, validate_parameters};

/// `party_id`: Should be in the range [0, n). For the DKG, should be our index in the best authorities starting from 0.
pub fn protocol<Net: NetInterface, BI: BlockchainInterface>(n: u32, party_id: u32, k: u32, t: u32, mut network: Net, bc: BI, session_id: SessionId) -> Pin<Box<dyn Future<Output=Result<(), DKGError>>>> {
Box::pin(async move {
validate_parameters(n, k, t)?;

let mut rng = rand::thread_rng();
let key_ids = crate::dkg_modules::wt_frost::generate_party_key_ids(n, k);
let our_key_ids = key_ids.get(party_id as usize).ok_or_else(|| DKGError::StartKeygen { reason: "Bad party_id".to_string() })?;

let mut party = Party::new(party_id, our_key_ids, n, k, t, &mut rng);
let public_key = crate::dkg_modules::wt_frost::run_dkg(&mut party, &mut rng, &mut network, n as usize).await?;

// Encode via serde_json
let public_key_bytes = serde_json::to_vec(&public_key).map_err(|err| DKGError::GenericError { reason: err.to_string() })?;

// Gossip the public key
let pkey_message = PublicKeyMessage {
session_id,
pub_key: public_key_bytes,
signature: vec![],
};

// Store and gossip the public key
bc.store_public_key(public_key, session_id)?;
bc.gossip_public_key(pkey_message)?;

Ok(())
})
}
11 changes: 10 additions & 1 deletion dkg-gadget/src/dkg_modules/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,16 @@ pub enum KeygenProtocolSetupParameters<B: Block> {
stage: ProtoStageType,
keygen_protocol_hash: [u8; 32],
},
WTFrost {},
WTFrost {
authority_id: AuthorityId,
best_authorities: Vec<(KeygenPartyId, Public)>,
authority_public_key: Public,
keygen_protocol_hash: [u8; 32],
threshold: u32,
session_id: SessionId,
associated_block: NumberFor<B>,
stage: ProtoStageType
},
}

/// Setup parameters for the Signing protocol
Expand Down
40 changes: 1 addition & 39 deletions dkg-gadget/src/dkg_modules/mp_ecdsa.rs
Original file line number Diff line number Diff line change
Expand Up @@ -62,10 +62,7 @@ where
KEYGEN_SSID,
) {
Ok(async_proto_params) => {
let err_handler_tx = self.dkg_worker.error_handler_channel.tx.clone();

let remote = async_proto_params.handle.clone();
let keygen_manager = self.dkg_worker.keygen_manager.clone();
let status = match stage {
ProtoStageType::KeygenGenesis => KeygenRound::Genesis,
ProtoStageType::KeygenStandard => KeygenRound::Next,
Expand All @@ -81,43 +78,8 @@ where
keygen_protocol_hash,
) {
Ok(meta_handler) => {
let logger = self.dkg_worker.logger.clone();
let signing_manager = self.dkg_worker.signing_manager.clone();
signing_manager.keygen_lock();
let task = async move {
match meta_handler.await {
Ok(_) => {
keygen_manager.set_state(KeygenState::KeygenCompleted {
session_completed: session_id,
});
let _ = keygen_manager
.finished_count
.fetch_add(1, Ordering::SeqCst);
signing_manager.keygen_unlock();
logger.info(
"The keygen meta handler has executed successfully"
.to_string(),
);

Ok(())
},

Err(err) => {
logger.error(format!(
"Error executing meta handler {:?}",
&err
));
keygen_manager
.set_state(KeygenState::Failed { session_id });
signing_manager.keygen_unlock();
let _ = err_handler_tx.send(err.clone());
Err(err)
},
}
};

self.dkg_worker.logger.debug(format!("Created Keygen Protocol task for session {session_id} with status {status:?}"));
return Some((remote, Box::pin(task)))
return Some((remote, Box::pin(meta_handler)))
},

Err(err) => {
Expand Down
89 changes: 78 additions & 11 deletions dkg-gadget/src/dkg_modules/wt_frost.rs
Original file line number Diff line number Diff line change
@@ -1,22 +1,26 @@
use dkg_primitives::types::DKGError;
use dkg_primitives::types::{DKGError, SSID};
use itertools::Itertools;
use rand::{CryptoRng, RngCore};
use std::{collections::HashMap, fmt::Debug};
use std::sync::Arc;
use async_trait::async_trait;
use sc_client_api::Backend;
use serde::{Deserialize, Serialize};
use sp_arithmetic::traits::SaturatedConversion;
use sp_runtime::traits::Block;
use wsts::{
common::{PolyCommitment, PublicNonce, Signature, SignatureShare},
v2,
v2::SignatureAggregator,
Scalar,
};
use crate::async_protocols::blockchain_interface::DKGProtocolEngine;
use crate::async_protocols::frost::keygen::FrostKeygenNetworkWrapper;
use crate::async_protocols::remote::AsyncProtocolRemote;
use crate::Client;
use crate::dkg_modules::{DKG, KeygenProtocolSetupParameters, ProtocolInitReturn, SigningProtocolSetupParameters};
use crate::gossip_engine::GossipEngineIface;
use crate::worker::DKGWorker;
use crate::worker::{DKGWorker, HasLatestHeader, ProtoStageType};

/// DKG module for Weighted Threshold Frost
pub struct WTFrostDKG<B, BE, C, GE>
Expand All @@ -41,12 +45,58 @@ impl<B, BE, C, GE> DKG<B> for WTFrostDKG<B, BE, C, GE>
&self,
params: KeygenProtocolSetupParameters<B>,
) -> Option<ProtocolInitReturn<B>> {
if let KeygenProtocolSetupParameters::WTFrost {} = params {
let remote = AsyncProtocolRemote::new();
let task = Box::pin(async move {

if let KeygenProtocolSetupParameters::WTFrost {
best_authorities,
authority_id,
authority_public_key,
keygen_protocol_hash,
threshold,
session_id,
associated_block,
stage
} = params {
// We must construct a remote and a task to ensure compatibility with the work manager
// and future message delivery to the service
let n = best_authorities.len() as u32;
// For now, assume each party member owns n keys
let k = n;
// The party id for frost must be in [0, n). Thus, we just find out index in the best authorities
const KEYGEN_SSID: SSID = 0;
let party_id = best_authorities
.iter()
.find_position(|r| &r.1 == &authority_public_key)
.map(|r| r.0 as u32)
.expect("Authority ID not found in best authorities");
let at = self.dkg_worker.get_latest_block_number();
let associated_block_id = associated_block.saturated_into();
let authority_mapping = Default::default(); // TODO

// Setup the remote to allow communication between the async protocol and the DKG worker
let remote = AsyncProtocolRemote::new(at, session_id, self.dkg_worker.logger.clone(), associated_block_id, KEYGEN_SSID, stage, authority_mapping);

// Setup the engine to allow communication between the async protocol and the blockchain
let bc_iface = Arc::new(DKGProtocolEngine {
backend: self.dkg_worker.backend.clone(),
latest_header: self.dkg_worker.latest_header.clone(),
client: self.dkg_worker.client.clone(),
keystore: self.dkg_worker.key_store.clone(),
db: self.dkg_worker.db.clone(),
gossip_engine: self.dkg_worker.gossip_engine.clone(),
aggregated_public_keys: self.dkg_worker.aggregated_public_keys.clone(),
current_validator_set: self.dkg_worker.current_validator_set.clone(),
local_keystore: self.dkg_worker.local_keystore.clone(),
vote_results: Arc::new(Default::default()),
is_genesis: stage == ProtoStageType::KeygenGenesis,
metrics: self.dkg_worker.metrics.clone(),
test_bundle: self.dkg_worker.test_bundle.clone(),
logger: self.dkg_worker.logger.clone(),
_pd: Default::default(),
});

let network = FrostKeygenNetworkWrapper::new(self.dkg_worker.clone(), bc_iface.clone(), remote.clone(), authority_id, keygen_protocol_hash);

let task = crate::async_protocols::frost::keygen::proto::protocol(n, party_id, k, threshold, network, bc_iface, session_id);

Some((remote, task))
} else {
None
Expand Down Expand Up @@ -144,12 +194,13 @@ pub async fn run_dkg<RNG: RngCore + CryptoRng, Net: NetInterface>(
Ok(polys)
}

/// `threshold`: Should be the number of participants in this round, since we stop looking for messages
/// after finding the first `t` messages
pub async fn run_signing<RNG: RngCore + CryptoRng, Net: NetInterface>(
signer: &mut v2::Party,
rng: &mut RNG,
msg: &[u8],
net: &mut Net,
n_signers: usize,
num_keys: u32,
threshold: u32,
public_key: Vec<PolyCommitment>,
Expand All @@ -171,7 +222,7 @@ pub async fn run_signing<RNG: RngCore + CryptoRng, Net: NetInterface>(
party_key_ids.insert(party_id, key_ids);
party_nonces.insert(party_id, nonce);

while party_nonces.len() < n_signers {
while party_nonces.len() < threshold as usize {
match net.next_message().await {
Ok(Some(FrostMessage::Sign { party_id: party_id_recv, key_ids, nonce })) => {
party_key_ids.insert(party_id_recv, key_ids);
Expand All @@ -187,7 +238,7 @@ pub async fn run_signing<RNG: RngCore + CryptoRng, Net: NetInterface>(
}

// Sort the vecs
let party_ids = (0..n_signers).into_iter().map(|r| r as u32).collect_vec();
let party_ids = (0..threshold).into_iter().collect_vec();
let party_key_ids = party_key_ids
.into_iter()
.sorted_by(|a, b| a.0.cmp(&b.0))
Expand All @@ -211,7 +262,7 @@ pub async fn run_signing<RNG: RngCore + CryptoRng, Net: NetInterface>(
signature_shares.insert(party_id, signature_share.clone());

// Receive n_signers number of shares
while signature_shares.len() < n_signers {
while signature_shares.len() < threshold as usize {
match net.next_message().await {
Ok(Some(FrostMessage::SignFinal { party_id, signature_share })) => {
signature_shares.insert(party_id, signature_share);
Expand Down Expand Up @@ -275,6 +326,22 @@ pub fn generate_party_key_ids(n: u32, k: u32) -> Vec<Vec<u32>> {
result
}

pub fn validate_parameters(n: u32, k: u32, t: u32) -> Result<(), DKGError> {
if k & n != 0 {
return Err(DKGError::GenericError { reason: "K % N != 0".to_string() })
}

if k == 0 {
return Err(DKGError::GenericError { reason: "K == 0".to_string() })
}

if n <= t {
return Err(DKGError::GenericError { reason: "N <= T".to_string() })
}

Ok(())
}

#[derive(Clone, Debug, Serialize, Deserialize)]
pub enum FrostMessage {
DKG {
Expand Down Expand Up @@ -403,7 +470,7 @@ mod tests {
signers.push(Box::pin(async move {
let mut rng = rand::thread_rng();
crate::dkg_modules::wt_frost::run_signing(
party, &mut rng, &*msg, network, T as usize, K, T, public_key,
party, &mut rng, &*msg, network, K, T, public_key,
)
.await
}));
Expand Down
41 changes: 41 additions & 0 deletions dkg-gadget/src/keygen_manager/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -475,6 +475,47 @@ where
self.work_manager.force_shutdown_all();
}

// Now, map the task to properly handle cleanup
let logger = dkg_worker.logger.clone();
let signing_manager = dkg_worker.signing_manager.clone();
let keygen_manager = dkg_worker.keygen_manager.clone();
let err_handler_tx = dkg_worker.error_handler_channel.tx.clone();

// Prevent any signing tasks from running while the keygen is running
signing_manager.keygen_lock();

let task = async move {
match task.await {
Ok(_) => {
keygen_manager.set_state(KeygenState::KeygenCompleted {
session_completed: session_id,
});
let _ = keygen_manager
.finished_count
.fetch_add(1, Ordering::SeqCst);
signing_manager.keygen_unlock();
logger.info(
"The keygen meta handler has executed successfully"
.to_string(),
);

Ok(())
},

Err(err) => {
logger.error(format!(
"Error executing meta handler {:?}",
&err
));
keygen_manager
.set_state(KeygenState::Failed { session_id });
signing_manager.keygen_unlock();
let _ = err_handler_tx.send(err.clone());
Err(err)
},
}
};

if let Err(err) = self.push_task(handle, task) {
dkg_worker.logger.error(format!(
"🕸️ PARTY {party_i} | SPAWNING KEYGEN SESSION {session_id} | ERROR: {err}"
Expand Down
Loading

0 comments on commit 06ad94c

Please sign in to comment.