Rename DKG messages to Part and Ack.

This commit is contained in:
Andreas Fackler 2018-07-19 12:28:15 +02:00
parent 9159b42150
commit d787173dd8
4 changed files with 131 additions and 135 deletions

View File

@ -70,7 +70,7 @@ use crypto::{PublicKey, PublicKeySet, SecretKey, Signature};
use fault_log::{FaultKind, FaultLog};
use honey_badger::{HoneyBadger, HoneyBadgerStep, Message as HbMessage};
use messaging::{DistAlgorithm, NetworkInfo, Step, Target, TargetedMessage};
use sync_key_gen::{Accept, Propose, ProposeOutcome, SyncKeyGen};
use sync_key_gen::{Ack, Part, PartOutcome, SyncKeyGen};
pub use self::batch::Batch;
pub use self::builder::DynamicHoneyBadgerBuilder;
@ -309,8 +309,8 @@ where
continue;
}
match kg_msg {
KeyGenMessage::Propose(propose) => self.handle_propose(&s_id, propose)?,
KeyGenMessage::Accept(accept) => self.handle_accept(&s_id, accept)?,
KeyGenMessage::Part(part) => self.handle_part(&s_id, part)?,
KeyGenMessage::Ack(ack) => self.handle_ack(&s_id, ack)?,
}.merge_into(&mut fault_log);
}
}
@ -363,10 +363,10 @@ where
let threshold = (pub_keys.len() - 1) / 3;
let sk = self.netinfo.secret_key().clone();
let our_uid = self.our_id().clone();
let (key_gen, propose) = SyncKeyGen::new(our_uid, sk, pub_keys, threshold);
let (key_gen, part) = SyncKeyGen::new(our_uid, sk, pub_keys, threshold);
self.key_gen = Some((key_gen, change.clone()));
if let Some(propose) = propose {
self.send_transaction(KeyGenMessage::Propose(propose))?;
if let Some(part) = part {
self.send_transaction(KeyGenMessage::Part(part))?;
}
Ok(())
}
@ -386,29 +386,25 @@ where
.build();
}
/// Handles a `Propose` message that was output by Honey Badger.
fn handle_propose(
&mut self,
sender_id: &NodeUid,
propose: Propose,
) -> Result<FaultLog<NodeUid>> {
/// Handles a `Part` message that was output by Honey Badger.
fn handle_part(&mut self, sender_id: &NodeUid, part: Part) -> Result<FaultLog<NodeUid>> {
let handle = |&mut (ref mut key_gen, _): &mut (SyncKeyGen<NodeUid>, _)| {
key_gen.handle_propose(&sender_id, propose)
key_gen.handle_part(&sender_id, part)
};
match self.key_gen.as_mut().and_then(handle) {
Some(ProposeOutcome::Valid(accept)) => {
self.send_transaction(KeyGenMessage::Accept(accept))?;
Some(PartOutcome::Valid(ack)) => {
self.send_transaction(KeyGenMessage::Ack(ack))?;
Ok(FaultLog::new())
}
Some(ProposeOutcome::Invalid(fault_log)) => Ok(fault_log),
Some(PartOutcome::Invalid(fault_log)) => Ok(fault_log),
None => Ok(FaultLog::new()),
}
}
/// Handles an `Accept` message that was output by Honey Badger.
fn handle_accept(&mut self, sender_id: &NodeUid, accept: Accept) -> Result<FaultLog<NodeUid>> {
/// Handles an `Ack` message that was output by Honey Badger.
fn handle_ack(&mut self, sender_id: &NodeUid, ack: Ack) -> Result<FaultLog<NodeUid>> {
if let Some(&mut (ref mut key_gen, _)) = self.key_gen.as_mut() {
Ok(key_gen.handle_accept(&sender_id, accept))
Ok(key_gen.handle_ack(&sender_id, ack))
} else {
Ok(FaultLog::new())
}
@ -489,10 +485,10 @@ struct SignedKeyGenMsg<NodeUid>(u64, NodeUid, KeyGenMessage, Signature);
/// a batch, so that all nodes see these messages in the same order.
#[derive(Eq, PartialEq, Debug, Serialize, Deserialize, Hash, Clone)]
pub enum KeyGenMessage {
/// A `SyncKeyGen::Propose` message for key generation.
Propose(Propose),
/// A `SyncKeyGen::Accept` message for key generation.
Accept(Accept),
/// A `SyncKeyGen::Part` message for key generation.
Part(Part),
/// A `SyncKeyGen::Ack` message for key generation.
Ack(Ack),
}
/// A message sent to or received from another node's Honey Badger instance.

View File

@ -29,10 +29,10 @@ pub enum FaultKind {
/// `DynamicHoneyBadger` received a message (Accept, Propose, or Change)
/// with an invalid signature.
IncorrectPayloadSignature,
/// `DynamicHoneyBadger`/`SyncKeyGen` received an invalid Accept message.
InvalidAcceptMessage,
/// `DynamicHoneyBadger`/`SyncKeyGen` received an invalid Propose message.
InvalidProposeMessage,
/// `DynamicHoneyBadger`/`SyncKeyGen` received an invalid Ack message.
InvalidAckMessage,
/// `DynamicHoneyBadger`/`SyncKeyGen` received an invalid Part message.
InvalidPartMessage,
/// `DynamicHoneyBadger` received a change vote with an invalid signature.
InvalidVoteSignature,
/// A validator committed an invalid vote in `DynamicHoneyBadger`.

View File

@ -15,12 +15,12 @@
//!
//! Before beginning the threshold key generation process, each validator needs to generate a
//! regular (non-threshold) key pair and multicast its public key. `SyncKeyGen::new` returns the
//! instance itself and a `Propose` message, containing a contribution to the new threshold keys.
//! It needs to be sent to all nodes. `SyncKeyGen::handle_propose` in turn produces an `Accept`
//! instance itself and a `Part` message, containing a contribution to the new threshold keys.
//! It needs to be sent to all nodes. `SyncKeyGen::handle_part` in turn produces an `Ack`
//! message, which is also multicast.
//!
//! All nodes must handle the exact same set of `Propose` and `Accept` messages. In this sense the
//! algorithm is synchronous: If Alice's `Accept` was handled by Bob but not by Carol, Bob and
//! All nodes must handle the exact same set of `Part` and `Ack` messages. In this sense the
//! algorithm is synchronous: If Alice's `Ack` was handled by Bob but not by Carol, Bob and
//! Carol could receive different public key sets, and secret key shares that don't match. One way
//! to ensure this is to commit the messages to a public ledger before handling them, e.g. by
//! feeding them to a preexisting instance of Honey Badger. The messages will then appear in the
@ -30,20 +30,20 @@
//! public key set.
//!
//! While not asynchronous, the algorithm is fault tolerant: It is not necessary to handle a
//! `Propose` and all `Accept` messages from every validator. A `Propose` is _complete_ if it
//! received at least _2 t + 1_ valid `Accept`s. Only complete `Propose`s are used for key
//! generation in the end, and as long as at least one complete `Propose` is from a correct node,
//! `Part` and all `Ack` messages from every validator. A `Part` is _complete_ if it
//! received at least _2 t + 1_ valid `Ack`s. Only complete `Part`s are used for key
//! generation in the end, and as long as at least one complete `Part` is from a correct node,
//! the new key set is secure. You can use `SyncKeyGen::is_ready` to check whether at least
//! _t + 1_ `Propose`s are complete. So all nodes can call `generate` as soon as `is_ready` returns
//! _t + 1_ `Part`s are complete. So all nodes can call `generate` as soon as `is_ready` returns
//! `true`.
//!
//! Alternatively, you can use any stronger criterion, too, as long as all validators call
//! `generate` at the same point, i.e. after handling the same set of messages.
//! `SyncKeyGen::count_complete` returns the number of complete `Propose` messages. And
//! `SyncKeyGen::is_node_ready` can be used to check whether a particluar node's `Propose` is
//! `SyncKeyGen::count_complete` returns the number of complete `Part` messages. And
//! `SyncKeyGen::is_node_ready` can be used to check whether a particluar node's `Part` is
//! complete.
//!
//! Finally, observer nodes can also use `SyncKeyGen`. For observers, no `Propose` and `Accept`
//! Finally, observer nodes can also use `SyncKeyGen`. For observers, no `Part` and `Ack`
//! messages will be created and they do not need to send anything. On completion, they will only
//! receive the public key set, but no secret key share.
//!
@ -56,7 +56,7 @@
//! use std::collections::BTreeMap;
//!
//! use hbbft::crypto::{PublicKey, SecretKey, SignatureShare};
//! use hbbft::sync_key_gen::{ProposeOutcome, SyncKeyGen};
//! use hbbft::sync_key_gen::{PartOutcome, SyncKeyGen};
//!
//! // Two out of four shares will suffice to sign or encrypt something.
//! let (threshold, node_num) = (1, 4);
@ -69,32 +69,32 @@
//! .enumerate()
//! .collect();
//!
//! // Create the `SyncKeyGen` instances. The constructor also outputs the proposal that needs to
//! // be sent to all other participants, so we save the proposals together with their sender ID.
//! // Create the `SyncKeyGen` instances. The constructor also outputs the part that needs to
//! // be sent to all other participants, so we save the parts together with their sender ID.
//! let mut nodes = BTreeMap::new();
//! let mut proposals = Vec::new();
//! let mut parts = Vec::new();
//! for (id, sk) in sec_keys.into_iter().enumerate() {
//! let (sync_key_gen, opt_proposal) = SyncKeyGen::new(id, sk, pub_keys.clone(), threshold);
//! let (sync_key_gen, opt_part) = SyncKeyGen::new(id, sk, pub_keys.clone(), threshold);
//! nodes.insert(id, sync_key_gen);
//! proposals.push((id, opt_proposal.unwrap())); // Would be `None` for observer nodes.
//! parts.push((id, opt_part.unwrap())); // Would be `None` for observer nodes.
//! }
//!
//! // All nodes now handle the proposals and send the resulting `Accept` messages.
//! let mut accepts = Vec::new();
//! for (sender_id, proposal) in proposals {
//! // All nodes now handle the parts and send the resulting `Ack` messages.
//! let mut acks = Vec::new();
//! for (sender_id, part) in parts {
//! for (&id, node) in &mut nodes {
//! match node.handle_propose(&sender_id, proposal.clone()) {
//! Some(ProposeOutcome::Valid(accept)) => accepts.push((id, accept)),
//! Some(ProposeOutcome::Invalid(faults)) => panic!("Invalid proposal: {:?}", faults),
//! None => panic!("We are not an observer, so we should send Accept."),
//! match node.handle_part(&sender_id, part.clone()) {
//! Some(PartOutcome::Valid(ack)) => acks.push((id, ack)),
//! Some(PartOutcome::Invalid(faults)) => panic!("Invalid part: {:?}", faults),
//! None => panic!("We are not an observer, so we should send Ack."),
//! }
//! }
//! }
//!
//! // Finally, we handle all the `Accept`s.
//! for (sender_id, accept) in accepts {
//! // Finally, we handle all the `Ack`s.
//! for (sender_id, ack) in acks {
//! for node in nodes.values_mut() {
//! node.handle_accept(&sender_id, accept.clone());
//! node.handle_ack(&sender_id, ack.clone());
//! }
//! }
//!
@ -172,30 +172,30 @@ use messaging::NetworkInfo;
/// nodes and handled by all of them, including the one that produced it.
///
/// The message contains a commitment to a bivariate polynomial, and for each node, an encrypted
/// row of values. If this message receives enough `Accept`s, it will be used as summand to produce
/// row of values. If this message receives enough `Ack`s, it will be used as summand to produce
/// the the key set in the end.
#[derive(Deserialize, Serialize, Clone, Hash, Eq, PartialEq)]
pub struct Propose(BivarCommitment, Vec<Ciphertext>);
pub struct Part(BivarCommitment, Vec<Ciphertext>);
impl Debug for Propose {
impl Debug for Part {
fn fmt(&self, f: &mut Formatter) -> fmt::Result {
let deg = self.0.degree();
let len = self.1.len();
write!(f, "Propose(<degree {}>, <{} rows>)", deg, len)
write!(f, "Part(<degree {}>, <{} rows>)", deg, len)
}
}
/// A confirmation that we have received and verified a validator's proposal. It must be sent to
/// A confirmation that we have received and verified a validator's part. It must be sent to
/// all participating nodes and handled by all of them, including ourselves.
///
/// The message is only produced after we verified our row against the commitment in the `Propose`.
/// The message is only produced after we verified our row against the commitment in the `Part`.
/// For each node, it contains one encrypted value of that row.
#[derive(Deserialize, Serialize, Clone, Hash, Eq, PartialEq)]
pub struct Accept(u64, Vec<Ciphertext>);
pub struct Ack(u64, Vec<Ciphertext>);
impl Debug for Accept {
impl Debug for Ack {
fn fmt(&self, f: &mut Formatter) -> fmt::Result {
write!(f, "Accept({}, <{} values>", self.0, self.1.len())
write!(f, "Ack({}, <{} values>", self.0, self.1.len())
}
}
@ -203,37 +203,37 @@ impl Debug for Accept {
struct ProposalState {
/// The proposer's commitment.
commit: BivarCommitment,
/// The verified values we received from `Accept` messages.
/// The verified values we received from `Ack` messages.
values: BTreeMap<u64, Fr>,
/// The nodes which have accepted this proposal, valid or not.
accepts: BTreeSet<u64>,
/// The nodes which have acked this part, valid or not.
acks: BTreeSet<u64>,
}
impl ProposalState {
/// Creates a new proposal state with a commitment.
/// Creates a new part state with a commitment.
fn new(commit: BivarCommitment) -> ProposalState {
ProposalState {
commit,
values: BTreeMap::new(),
accepts: BTreeSet::new(),
acks: BTreeSet::new(),
}
}
/// Returns `true` if at least `2 * threshold + 1` nodes have accepted.
/// Returns `true` if at least `2 * threshold + 1` nodes have acked.
fn is_complete(&self, threshold: usize) -> bool {
self.accepts.len() > 2 * threshold
self.acks.len() > 2 * threshold
}
}
/// The outcome of handling and verifying a `Propose` message.
pub enum ProposeOutcome<NodeUid: Clone> {
/// The outcome of handling and verifying a `Part` message.
pub enum PartOutcome<NodeUid: Clone> {
/// The message was valid: the part of it that was encrypted to us matched the public
/// commitment, so we can multicast an `Accept` message for it.
Valid(Accept),
// If the Propose message passed to `handle_propose()` is invalid, the
/// commitment, so we can multicast an `Ack` message for it.
Valid(Ack),
// If the Part message passed to `handle_part()` is invalid, the
// fault is logged and passed onto the caller.
/// The message was invalid: the part encrypted to us was malformed or didn't match the
/// commitment. We now know that the proposer is faulty, and dont' send an `Accept`.
/// commitment. We now know that the proposer is faulty, and dont' send an `Ack`.
Invalid(FaultLog<NodeUid>),
}
@ -249,24 +249,24 @@ pub struct SyncKeyGen<NodeUid> {
sec_key: SecretKey,
/// The public keys of all nodes, by node index.
pub_keys: BTreeMap<NodeUid, PublicKey>,
/// Proposed bivariate polynomial.
proposals: BTreeMap<u64, ProposalState>,
/// Proposed bivariate polynomials.
parts: BTreeMap<u64, ProposalState>,
/// The degree of the generated polynomial.
threshold: usize,
}
impl<NodeUid: Ord + Clone + Debug> SyncKeyGen<NodeUid> {
/// Creates a new `SyncKeyGen` instance, together with the `Propose` message that should be
/// Creates a new `SyncKeyGen` instance, together with the `Part` message that should be
/// multicast to all nodes.
///
/// If we are not a validator but only an observer, no `Propose` message is produced and no
/// If we are not a validator but only an observer, no `Part` message is produced and no
/// messages need to be sent.
pub fn new(
our_uid: NodeUid,
sec_key: SecretKey,
pub_keys: BTreeMap<NodeUid, PublicKey>,
threshold: usize,
) -> (SyncKeyGen<NodeUid>, Option<Propose>) {
) -> (SyncKeyGen<NodeUid>, Option<Part>) {
let our_idx = pub_keys
.keys()
.position(|uid| *uid == our_uid)
@ -276,41 +276,41 @@ impl<NodeUid: Ord + Clone + Debug> SyncKeyGen<NodeUid> {
our_idx,
sec_key,
pub_keys,
proposals: BTreeMap::new(),
parts: BTreeMap::new(),
threshold,
};
if our_idx.is_none() {
return (key_gen, None); // No proposal: we are an observer.
return (key_gen, None); // No part: we are an observer.
}
let mut rng = OsRng::new().expect("OS random number generator");
let our_proposal = BivarPoly::random(threshold, &mut rng);
let commit = our_proposal.commitment();
let our_part = BivarPoly::random(threshold, &mut rng);
let commit = our_part.commitment();
let encrypt = |(i, pk): (usize, &PublicKey)| {
let row = our_proposal.row(i as u64 + 1);
let row = our_part.row(i as u64 + 1);
let bytes = bincode::serialize(&row).expect("failed to serialize row");
pk.encrypt(&bytes)
};
let rows: Vec<_> = key_gen.pub_keys.values().enumerate().map(encrypt).collect();
(key_gen, Some(Propose(commit, rows)))
(key_gen, Some(Part(commit, rows)))
}
/// Handles a `Propose` message. If it is valid, returns an `Accept` message to be broadcast.
/// Handles a `Part` message. If it is valid, returns an `Ack` message to be broadcast.
///
/// If we are only an observer, `None` is returned instead and no messages need to be sent.
pub fn handle_propose(
pub fn handle_part(
&mut self,
sender_id: &NodeUid,
Propose(commit, rows): Propose,
) -> Option<ProposeOutcome<NodeUid>> {
Part(commit, rows): Part,
) -> Option<PartOutcome<NodeUid>> {
let sender_idx = self.node_index(sender_id)?;
let opt_commit_row = self.our_idx.map(|idx| commit.row(idx + 1));
match self.proposals.entry(sender_idx) {
Entry::Occupied(_) => return None, // Ignore multiple proposals.
match self.parts.entry(sender_idx) {
Entry::Occupied(_) => return None, // Ignore multiple parts.
Entry::Vacant(entry) => {
entry.insert(ProposalState::new(commit));
}
}
// If we are only an observer, return `None`. We don't need to send `Accept`.
// If we are only an observer, return `None`. We don't need to send `Ack`.
let our_idx = self.our_idx?;
let commit_row = opt_commit_row?;
let ser_row = self.sec_key.decrypt(rows.get(our_idx as usize)?)?;
@ -318,13 +318,13 @@ impl<NodeUid: Ord + Clone + Debug> SyncKeyGen<NodeUid> {
row
} else {
// Log the faulty node and ignore invalid messages.
let fault_log = FaultLog::init(sender_id.clone(), FaultKind::InvalidProposeMessage);
return Some(ProposeOutcome::Invalid(fault_log));
let fault_log = FaultLog::init(sender_id.clone(), FaultKind::InvalidPartMessage);
return Some(PartOutcome::Invalid(fault_log));
};
if row.commitment() != commit_row {
debug!("Invalid proposal from node {}.", sender_idx);
let fault_log = FaultLog::init(sender_id.clone(), FaultKind::InvalidProposeMessage);
return Some(ProposeOutcome::Invalid(fault_log));
debug!("Invalid part from node {}.", sender_idx);
let fault_log = FaultLog::init(sender_id.clone(), FaultKind::InvalidPartMessage);
return Some(PartOutcome::Invalid(fault_log));
}
// The row is valid: now encrypt one value for each node.
let encrypt = |(idx, pk): (usize, &PublicKey)| {
@ -335,38 +335,38 @@ impl<NodeUid: Ord + Clone + Debug> SyncKeyGen<NodeUid> {
pk.encrypt(ser_val)
};
let values = self.pub_keys.values().enumerate().map(encrypt).collect();
Some(ProposeOutcome::Valid(Accept(sender_idx, values)))
Some(PartOutcome::Valid(Ack(sender_idx, values)))
}
/// Handles an `Accept` message.
pub fn handle_accept(&mut self, sender_id: &NodeUid, accept: Accept) -> FaultLog<NodeUid> {
/// Handles an `Ack` message.
pub fn handle_ack(&mut self, sender_id: &NodeUid, ack: Ack) -> FaultLog<NodeUid> {
let mut fault_log = FaultLog::new();
if let Some(sender_idx) = self.node_index(sender_id) {
if let Err(err) = self.handle_accept_or_err(sender_idx, accept) {
debug!("Invalid accept from node {}: {}", sender_idx, err);
fault_log.append(sender_id.clone(), FaultKind::InvalidAcceptMessage);
if let Err(err) = self.handle_ack_or_err(sender_idx, ack) {
debug!("Invalid ack from node {}: {}", sender_idx, err);
fault_log.append(sender_id.clone(), FaultKind::InvalidAckMessage);
}
}
fault_log
}
/// Returns the number of complete proposals. If this is at least `threshold + 1`, the keys can
/// Returns the number of complete parts. If this is at least `threshold + 1`, the keys can
/// be generated, but it is possible to wait for more to increase security.
pub fn count_complete(&self) -> usize {
self.proposals
self.parts
.values()
.filter(|proposal| proposal.is_complete(self.threshold))
.filter(|part| part.is_complete(self.threshold))
.count()
}
/// Returns `true` if the proposal of the given node is complete.
/// Returns `true` if the part of the given node is complete.
pub fn is_node_ready(&self, proposer_id: &NodeUid) -> bool {
self.node_index(proposer_id)
.and_then(|proposer_idx| self.proposals.get(&proposer_idx))
.map_or(false, |proposal| proposal.is_complete(self.threshold))
.and_then(|proposer_idx| self.parts.get(&proposer_idx))
.map_or(false, |part| part.is_complete(self.threshold))
}
/// Returns `true` if enough proposals are complete to safely generate the new key.
/// Returns `true` if enough parts are complete to safely generate the new key.
pub fn is_ready(&self) -> bool {
self.count_complete() > self.threshold
}
@ -380,11 +380,11 @@ impl<NodeUid: Ord + Clone + Debug> SyncKeyGen<NodeUid> {
pub fn generate(&self) -> (PublicKeySet, Option<SecretKeyShare>) {
let mut pk_commit = Poly::zero().commitment();
let mut opt_sk_val = self.our_idx.map(|_| Fr::zero());
let is_complete = |proposal: &&ProposalState| proposal.is_complete(self.threshold);
for proposal in self.proposals.values().filter(is_complete) {
pk_commit += proposal.commit.row(0);
let is_complete = |part: &&ProposalState| part.is_complete(self.threshold);
for part in self.parts.values().filter(is_complete) {
pk_commit += part.commit.row(0);
if let Some(sk_val) = opt_sk_val.as_mut() {
let row: Poly = Poly::interpolate(proposal.values.iter().take(self.threshold + 1));
let row: Poly = Poly::interpolate(part.values.iter().take(self.threshold + 1));
sk_val.add_assign(&row.evaluate(0));
}
}
@ -400,21 +400,21 @@ impl<NodeUid: Ord + Clone + Debug> SyncKeyGen<NodeUid> {
NetworkInfo::new(self.our_uid, sk_share, pk_set, self.sec_key, self.pub_keys)
}
/// Handles an `Accept` message or returns an error string.
fn handle_accept_or_err(
/// Handles an `Ack` message or returns an error string.
fn handle_ack_or_err(
&mut self,
sender_idx: u64,
Accept(proposer_idx, values): Accept,
Ack(proposer_idx, values): Ack,
) -> Result<(), String> {
if values.len() != self.pub_keys.len() {
return Err("wrong node count".to_string());
}
let proposal = self
.proposals
let part = self
.parts
.get_mut(&proposer_idx)
.ok_or_else(|| "sender does not exist".to_string())?;
if !proposal.accepts.insert(sender_idx) {
return Err("duplicate accept".to_string());
if !part.acks.insert(sender_idx) {
return Err("duplicate ack".to_string());
}
let our_idx = match self.our_idx {
Some(our_idx) => our_idx,
@ -427,10 +427,10 @@ impl<NodeUid: Ord + Clone + Debug> SyncKeyGen<NodeUid> {
let val = bincode::deserialize::<FieldWrap<Fr, Fr>>(&ser_val)
.map_err(|err| format!("deserialization failed: {:?}", err))?
.into_inner();
if proposal.commit.evaluate(our_idx + 1, sender_idx + 1) != G1Affine::one().mul(val) {
if part.commit.evaluate(our_idx + 1, sender_idx + 1) != G1Affine::one().mul(val) {
return Err("wrong value".to_string());
}
proposal.values.insert(sender_idx + 1, val);
part.values.insert(sender_idx + 1, val);
Ok(())
}

View File

@ -9,7 +9,7 @@ extern crate rand;
use std::collections::BTreeMap;
use hbbft::crypto::{PublicKey, SecretKey};
use hbbft::sync_key_gen::{ProposeOutcome, SyncKeyGen};
use hbbft::sync_key_gen::{PartOutcome, SyncKeyGen};
fn test_sync_key_gen_with(threshold: usize, node_num: usize) {
// Generate individual key pairs for encryption. These are not suitable for threshold schemes.
@ -33,26 +33,26 @@ fn test_sync_key_gen_with(threshold: usize, node_num: usize) {
.collect();
// Handle the first `threshold + 1` proposals. Those should suffice for key generation.
let mut accepts = Vec::new();
let mut acks = Vec::new();
for (sender_id, proposal) in proposals[..=threshold].iter().enumerate() {
for (node_id, node) in nodes.iter_mut().enumerate() {
let proposal = proposal.clone().expect("proposal");
let accept = match node.handle_propose(&sender_id, proposal) {
Some(ProposeOutcome::Valid(accept)) => accept,
let ack = match node.handle_part(&sender_id, proposal) {
Some(PartOutcome::Valid(ack)) => ack,
_ => panic!("invalid proposal"),
};
// Only the first `threshold + 1` manage to commit their `Accept`s.
// Only the first `threshold + 1` manage to commit their `Ack`s.
if node_id <= 2 * threshold {
accepts.push((node_id, accept));
acks.push((node_id, ack));
}
}
}
// Handle the `Accept`s from `2 * threshold + 1` nodes.
for (sender_id, accept) in accepts {
// Handle the `Ack`s from `2 * threshold + 1` nodes.
for (sender_id, ack) in acks {
for node in &mut nodes {
assert!(!node.is_ready()); // Not enough `Accept`s yet.
node.handle_accept(&sender_id, accept.clone());
assert!(!node.is_ready()); // Not enough `Ack`s yet.
node.handle_ack(&sender_id, ack.clone());
}
}