diff --git a/src/dynamic_honey_badger.rs b/src/dynamic_honey_badger.rs new file mode 100644 index 0000000..024193e --- /dev/null +++ b/src/dynamic_honey_badger.rs @@ -0,0 +1,473 @@ +use std::collections::{BTreeMap, HashMap, VecDeque}; +use std::fmt::Debug; +use std::hash::Hash; +use std::mem; +use std::rc::Rc; + +use bincode; +use clear_on_drop::ClearOnDrop; +use serde::{Deserialize, Serialize}; + +use crypto::{PublicKey, PublicKeySet, SecretKey, Signature}; +use honey_badger::{self, HoneyBadger}; +use messaging::{DistAlgorithm, NetworkInfo, TargetedMessage}; +use sync_key_gen::{Accept, Propose, SyncKeyGen}; + +error_chain!{ + links { + HoneyBadger(honey_badger::Error, honey_badger::ErrorKind); + } + + foreign_links { + Bincode(Box); + } + + errors { + UnknownSender + } +} + +/// A node change action: adding or removing a node. +#[derive(Clone, Eq, PartialEq, Serialize, Deserialize, Hash, Debug)] +pub enum Change { + /// Add a node. The public key is used only temporarily, for key generation. + Add(NodeUid, PublicKey), + /// Remove a node. + Remove(NodeUid), +} + +/// The user input for `DynamicHoneyBadger`. +#[derive(Clone, Debug)] +pub enum Input { + /// A user-defined transaction. + User(Tx), + /// A vote to change the set of nodes. + Change(Change), +} + +/// A Honey Badger instance that can handle adding and removing nodes. +// TODO: Handle the joining process correctly in the new node. Allow the new node to contribute its +// key generation input. +pub struct DynamicHoneyBadger +where + Tx: Eq + Serialize + for<'r> Deserialize<'r> + Debug + Hash, + NodeUid: Ord + Clone + Serialize + for<'r> Deserialize<'r> + Debug, +{ + /// Shared network data. + netinfo: NetworkInfo, + /// The target number of transactions per batch. + batch_size: usize, + /// The first epoch after the latest node change. + start_epoch: u64, + /// Collected votes for adding or removing nodes. Each node has one vote, and casting another + /// vote revokes the previous one. Resets whenever the set of peers is successfully changed. + votes: BTreeMap>, + /// The `HoneyBadger` instance with the current set of nodes. + honey_badger: HoneyBadger, NodeUid>, + /// The current key generation process. + key_gen: Option>, + /// A queue for messages from future epochs that cannot be handled yet. + incoming_queue: Vec<(u64, NodeUid, honey_badger::Message)>, + /// The messages that need to be sent to other nodes. + messages: MessageQueue, + /// The outputs from completed epochs. + output: VecDeque>, +} + +impl DistAlgorithm for DynamicHoneyBadger +where + Tx: Eq + Serialize + for<'r> Deserialize<'r> + Debug + Hash, + NodeUid: Eq + Ord + Clone + Serialize + for<'r> Deserialize<'r> + Debug + Hash, +{ + type NodeUid = NodeUid; + type Input = Input; + type Output = Batch; + type Message = Message; + type Error = Error; + + fn input(&mut self, input: Self::Input) -> Result<()> { + let tx = self.input_to_tx(input)?; + self.honey_badger.input(tx)?; + self.process_output() + } + + fn handle_message(&mut self, sender_id: &NodeUid, message: Self::Message) -> Result<()> { + match message { + Message::HoneyBadger(start_epoch, hb_msg) => { + self.handle_honey_badger_message(sender_id, start_epoch, hb_msg) + } + } + } + + fn next_message(&mut self) -> Option> { + self.messages.pop_front() + } + + fn next_output(&mut self) -> Option { + self.output.pop_front() + } + + fn terminated(&self) -> bool { + false + } + + fn our_id(&self) -> &NodeUid { + self.netinfo.our_uid() + } +} + +impl DynamicHoneyBadger +where + Tx: Eq + Serialize + for<'r> Deserialize<'r> + Debug + Hash, + NodeUid: Eq + Ord + Clone + Debug + Serialize + for<'r> Deserialize<'r> + Hash, +{ + /// Returns a new instance with the given parameters, starting at epoch `0`. + pub fn new(netinfo: NetworkInfo, batch_size: usize) -> Result { + let honey_badger = HoneyBadger::new(Rc::new(netinfo.clone()), batch_size, None)?; + let dyn_hb = DynamicHoneyBadger { + netinfo, + batch_size, + start_epoch: 0, + votes: BTreeMap::new(), + honey_badger, + key_gen: None, + incoming_queue: Vec::new(), + messages: MessageQueue(VecDeque::new()), + output: VecDeque::new(), + }; + Ok(dyn_hb) + } + + /// Handles a message for the `HoneyBadger` instance. + fn handle_honey_badger_message( + &mut self, + sender_id: &NodeUid, + epoch: u64, + message: honey_badger::Message, + ) -> Result<()> { + if epoch < self.start_epoch { + return Ok(()); // Obsolete message. + } + if epoch > self.start_epoch { + // Message cannot be handled yet. Save it for later. + let entry = (epoch, sender_id.clone(), message); + self.incoming_queue.push(entry); + return Ok(()); + } + if !self.netinfo.all_uids().contains(sender_id) { + return Err(ErrorKind::UnknownSender.into()); + } + // Handle the message and put the outgoing messages into the queue. + self.honey_badger.handle_message(sender_id, message)?; + self.process_output()?; + Ok(()) + } + + /// Processes all pending batches output by Honey Badger. + fn process_output(&mut self) -> Result<()> { + let mut changed = false; + while let Some(hb_batch) = self.honey_badger.next_output() { + // Create the batch we output ourselves. It will contain the _user_ transactions of + // `hb_batch`, and the applied change, if any. + let mut batch = Batch::new(hb_batch.epoch + self.start_epoch); + // The change that currently has a majority. All key generation messages in this batch + // are related to this change. + let change = self.current_majority(); + // Add the user transactions to `batch` and handle votes and DKG messages. + for (id, tx_vec) in hb_batch.transactions { + let entry = batch.transactions.entry(id); + let id_txs = entry.or_insert_with(Vec::new); + for tx in tx_vec { + use self::Transaction::*; + info!("{:?} output {:?}.", self.netinfo.our_uid(), tx); + match tx { + User(tx) => id_txs.push(tx), + Change(s_id, change, sig) => self.handle_vote(s_id, change, &sig)?, + Propose(s_id, propose, sig) => self.handle_propose(&s_id, propose, &*sig)?, + Accept(s_id, accept, sig) => self.handle_accept(&s_id, accept, &*sig)?, + } + } + } + // If DKG completed, apply the change. + if let Some(ref change) = change { + if let Some((pub_key_set, sk)) = self.get_key_gen_output() { + let sk = sk.unwrap_or_else(|| { + ClearOnDrop::new(Box::new(self.netinfo.secret_key().clone())) + }); + self.start_epoch = hb_batch.epoch + 1; + self.apply_change(change, pub_key_set, sk)?; + batch.change = Some(change.clone()); + changed = true; + } + } + // If a node is in the process of joining, inform the user. + let new_change = self.current_majority(); + if let Some(Change::Add(ref node_id, ref pub_key)) = new_change { + batch.candidate = Some((node_id.clone(), pub_key.clone())); + } + // If a new change has a majority, restart DKG. + if new_change != change { + if let Some(change) = new_change { + self.start_key_gen(change)?; + } else { + self.key_gen = None; + } + } + self.output.push_back(batch); + } + self.messages + .extend_with_epoch(self.start_epoch, &mut self.honey_badger); + // If `start_epoch` changed, we can now handle some queued messages. + if changed { + let queue = mem::replace(&mut self.incoming_queue, Vec::new()); + for (epoch, sender_id, msg) in queue { + self.handle_honey_badger_message(&sender_id, epoch, msg)?; + } + } + Ok(()) + } + + /// Converts the input into a transaction to commit via Honey Badger. + fn input_to_tx(&self, input: Input) -> Result> { + Ok(match input { + Input::User(tx) => Transaction::User(tx), + Input::Change(change) => { + let our_id = self.our_id().clone(); + let sig = self.sign(&change)?; + Transaction::Change(our_id, change, sig) + } + }) + } + + /// Restarts Honey Badger with a new set of nodes, and resets the Key Generation. + fn apply_change( + &mut self, + change: &Change, + pub_key_set: PublicKeySet, + sk: ClearOnDrop>, + ) -> Result<()> { + self.votes.clear(); + self.key_gen = None; + let mut all_uids = self.netinfo.all_uids().clone(); + if !match *change { + Change::Remove(ref id) => all_uids.remove(id), + Change::Add(ref id, _) => all_uids.insert(id.clone()), + } { + debug!("No-op change: {:?}", change); + } + let netinfo = NetworkInfo::new(self.our_id().clone(), all_uids, sk, pub_key_set); + self.netinfo = netinfo.clone(); + let buffer = self.honey_badger.drain_buffer(); + self.honey_badger = HoneyBadger::new(Rc::new(netinfo), self.batch_size, buffer)?; + Ok(()) + } + + /// Starts Key Generation for the set of nodes implied by the `change`. + fn start_key_gen(&mut self, change: Change) -> Result<()> { + // Use the existing key shares - with the change applied - as keys for DKG. + let mut pub_keys = self.netinfo.public_key_map(); + if match change { + Change::Remove(id) => pub_keys.remove(&id).is_none(), + Change::Add(id, pub_key) => pub_keys.insert(id, pub_key).is_some(), + } { + debug!("No-op change: {:?}", self.current_majority().unwrap()); + } + // TODO: This needs to be the same as `num_faulty` will be in the _new_ + // `NetworkInfo` if the change goes through. It would be safer to deduplicate. + 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); + self.key_gen = Some(key_gen); + if let Some(propose) = propose { + let sig = self.sign(&propose)?; + let tx = Transaction::Propose(our_uid, propose, sig); + self.honey_badger.input(tx)?; + } + Ok(()) + } + + /// Handles a `Propose` message that was output by Honey Badger. + fn handle_propose( + &mut self, + sender_id: &NodeUid, + propose: Propose, + sig: &Signature, + ) -> Result<()> { + if !self.verify_signature(sender_id, sig, &propose)? { + debug!("Invalid signature from {:?} for: {:?}.", sender_id, propose); + return Ok(()); + } + let handle = + |key_gen: &mut SyncKeyGen| key_gen.handle_propose(&sender_id, propose); + let accept = match self.key_gen.as_mut().and_then(handle) { + Some(accept) => accept, + None => return Ok(()), + }; + let our_id = self.our_id().clone(); + let sig = self.sign(&accept)?; + let tx = Transaction::Accept(our_id, accept, sig); + self.honey_badger.input(tx)?; + Ok(()) + } + + /// Handles an `Accept` message that was output by Honey Badger. + fn handle_accept( + &mut self, + sender_id: &NodeUid, + accept: Accept, + sig: &Signature, + ) -> Result<()> { + if self.verify_signature(sender_id, sig, &accept)? { + if let Some(key_gen) = self.key_gen.as_mut() { + key_gen.handle_accept(&sender_id, accept); + } + } + Ok(()) + } + + /// If the current Key Generation process is ready, returns the generated key set. + fn get_key_gen_output(&self) -> Option<(PublicKeySet, Option>>)> { + // TODO: Once we've upgraded to Rust 1.27.0, we can use `Option::filter` here. + self.key_gen + .iter() + .filter(|key_gen| key_gen.is_ready()) + .map(SyncKeyGen::generate) + .next() + } + + /// Returns a signature of `payload`, or an error if serialization fails. + fn sign(&self, payload: &T) -> Result> { + let ser = bincode::serialize(payload)?; + Ok(Box::new(self.netinfo.secret_key().sign(ser))) + } + + /// Returns `true` if the signature of the payload by the node with the specified ID is valid. + /// Returns an error if the payload fails to serialize. + fn verify_signature( + &self, + id: &NodeUid, + sig: &Signature, + payload: &T, + ) -> Result { + let ser = bincode::serialize(payload)?; + let pk_opt = self.netinfo.public_key_share(&id); + Ok(pk_opt.map_or(false, |pk| pk.verify(&sig, ser))) + } + + /// Adds a vote for a node change by the node with `id`. + fn handle_vote( + &mut self, + sender_id: NodeUid, + change: Change, + sig: &Signature, + ) -> Result<()> { + if self.verify_signature(&sender_id, sig, &change)? { + self.votes.insert(sender_id, change); + } + Ok(()) + } + + /// Returns the change that currently has a majority of votes, if any. + fn current_majority(&self) -> Option> { + let mut vote_counts: HashMap<&Change, usize> = HashMap::new(); + for change in self.votes.values() { + let entry = vote_counts.entry(change).or_insert(0); + *entry += 1; + if *entry * 2 > self.netinfo.num_nodes() { + return Some(change.clone()); + } + } + None + } +} + +/// The transactions for the internal `HoneyBadger` instance: this includes both user-defined +/// "regular" transactions as well as internal transactions for coordinating node additions and +/// removals and key generation. +#[derive(Eq, PartialEq, Debug, Serialize, Deserialize, Hash)] +enum Transaction { + /// A user-defined transaction. + User(Tx), + /// A vote by an existing node to add or remove a node. + Change(NodeUid, Change, Box), + /// A proposal message for key generation. + Propose(NodeUid, Propose, Box), + /// An accept message for key generation. + Accept(NodeUid, Accept, Box), +} + +/// A batch of transactions the algorithm has output. +#[derive(Clone)] +pub struct Batch { + /// The sequence number: there is exactly one batch in each epoch. + pub epoch: u64, + /// The user transactions committed in this epoch. + pub transactions: BTreeMap>, + /// Information about a newly added or removed node. This is `Some` if the set of nodes taking + /// part in the consensus process has changed. + pub change: Option>, + /// The current candidate for joining the consensus nodes. All future broadcast messages must + /// be delivered to this node, too. + pub candidate: Option<(NodeUid, PublicKey)>, +} + +impl Batch { + /// Returns a new, empty batch with the given epoch. + pub fn new(epoch: u64) -> Self { + Batch { + epoch, + transactions: BTreeMap::new(), + change: None, + candidate: None, + } + } + + /// Returns an iterator over all transactions included in the batch. + pub fn iter(&self) -> impl Iterator { + self.transactions.values().flat_map(|vec| vec) + } + + /// Returns the number of transactions in the batch (without detecting duplicates). + pub fn len(&self) -> usize { + self.transactions.values().map(Vec::len).sum() + } + + /// Returns `true` if the batch contains no transactions. + pub fn is_empty(&self) -> bool { + self.transactions.values().all(Vec::is_empty) + } + + /// Returns the change to the set of participating nodes, if any. + pub fn change(&self) -> Option<&Change> { + self.change.as_ref() + } +} + +/// A message sent to or received from another node's Honey Badger instance. +#[cfg_attr(feature = "serialization-serde", derive(Serialize, Deserialize))] +#[derive(Debug, Clone)] +pub enum Message { + /// A message belonging to the `HoneyBadger` algorithm started in the given epoch. + HoneyBadger(u64, honey_badger::Message), +} + +/// The queue of outgoing messages in a `HoneyBadger` instance. +#[derive(Deref, DerefMut)] +struct MessageQueue(VecDeque, NodeUid>>); + +impl MessageQueue +where + NodeUid: Eq + Hash + Ord + Clone + Debug + Serialize + for<'r> Deserialize<'r>, +{ + /// Appends to the queue the messages from `hb`, wrapped with `epoch`. + fn extend_with_epoch(&mut self, epoch: u64, hb: &mut HoneyBadger) + where + Tx: Eq + Serialize + for<'r> Deserialize<'r> + Debug + Hash, + { + let convert = |msg: TargetedMessage, NodeUid>| { + msg.map(|hb_msg| Message::HoneyBadger(epoch, hb_msg)) + }; + self.extend(hb.message_iter().map(convert)); + } +} diff --git a/src/honey_badger.rs b/src/honey_badger.rs index ebc8345..abc6769 100644 --- a/src/honey_badger.rs +++ b/src/honey_badger.rs @@ -116,7 +116,6 @@ where } } -// TODO: Use a threshold encryption scheme to encrypt the proposed transactions. impl HoneyBadger where Tx: Serialize + for<'r> Deserialize<'r> + Debug + Hash + Eq, @@ -569,9 +568,6 @@ impl Batch { } /// The content of a `HoneyBadger` message. It should be further annotated with an epoch. -/// -/// TODO: Store a pointer to the `share` and dereference it at the time of serialization to avoid -/// cloning shares. #[derive(Clone, Debug, Deserialize, Serialize)] pub enum MessageContent { /// A message belonging to the common subset algorithm in the given epoch. diff --git a/src/lib.rs b/src/lib.rs index 288cc81..6b263f8 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -120,6 +120,7 @@ pub mod broadcast; pub mod common_coin; pub mod common_subset; pub mod crypto; +pub mod dynamic_honey_badger; mod fmt; pub mod honey_badger; pub mod messaging; diff --git a/src/messaging.rs b/src/messaging.rs index b0c4152..09714b4 100644 --- a/src/messaging.rs +++ b/src/messaging.rs @@ -3,7 +3,7 @@ use std::fmt::Debug; use clear_on_drop::ClearOnDrop; -use crypto::{PublicKeySet, SecretKey}; +use crypto::{PublicKey, PublicKeySet, SecretKey}; /// Message sent by a given source. #[derive(Clone, Debug)] @@ -204,6 +204,21 @@ impl NetworkInfo { &self.public_key_set } + /// Returns the public key share if a node with that ID exists, otherwise `None`. + pub fn public_key_share(&self, id: &NodeUid) -> Option { + self.node_index(id) + .map(|idx| self.public_key_set.public_key_share(*idx as u64)) + } + + /// Returns a map of all node IDs to their public key shares. + pub fn public_key_map(&self) -> BTreeMap { + let to_pair = |(idx, id): (usize, &NodeUid)| { + let pub_key = self.public_key_set().public_key_share(idx as u64); + (id.clone(), pub_key) + }; + self.all_uids().iter().enumerate().map(to_pair).collect() + } + /// The index of a node in a canonical numbering of all nodes. pub fn node_index(&self, id: &NodeUid) -> Option<&usize> { self.node_indices.get(id) diff --git a/src/sync_key_gen.rs b/src/sync_key_gen.rs index 67dbc06..c1d425b 100644 --- a/src/sync_key_gen.rs +++ b/src/sync_key_gen.rs @@ -88,7 +88,7 @@ impl ProposalState { /// It requires that all nodes handle all messages in the exact same order. pub struct SyncKeyGen { /// Our node index. - our_idx: u64, + our_idx: Option, /// Our secret key. sec_key: SecretKey, /// The public keys of all nodes, by node index. @@ -101,29 +101,17 @@ pub struct SyncKeyGen { impl SyncKeyGen { /// Creates a new `SyncKeyGen` instance, together with the `Propose` message that should be - /// broadcast. + /// broadcast, if we are a peer. pub fn new( our_uid: &NodeUid, sec_key: SecretKey, pub_keys: BTreeMap, threshold: usize, - ) -> (SyncKeyGen, Propose) { + ) -> (SyncKeyGen, Option) { let our_idx = pub_keys .keys() .position(|uid| uid == our_uid) - .expect("missing pub key for own ID") as u64; - let mut rng = OsRng::new().expect("OS random number generator"); - let our_proposal = BivarPoly::random(threshold, &mut rng); - let commit = our_proposal.commitment(); - let rows: Vec<_> = pub_keys - .values() - .enumerate() - .map(|(i, pk)| { - let row = our_proposal.row(i as u64 + 1); - let bytes = bincode::serialize(&row).expect("failed to serialize row"); - pk.encrypt(&bytes) - }) - .collect(); + .map(|idx| idx as u64); let key_gen = SyncKeyGen { our_idx, sec_key, @@ -131,7 +119,19 @@ impl SyncKeyGen { proposals: BTreeMap::new(), threshold, }; - (key_gen, Propose(commit, rows)) + if our_idx.is_none() { + return (key_gen, None); // No proposal: 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 encrypt = |(i, pk): (usize, &PublicKey)| { + let row = our_proposal.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))) } /// Handles a `Propose` message. If it is valid, returns an `Accept` message to be broadcast. @@ -140,52 +140,41 @@ impl SyncKeyGen { sender_id: &NodeUid, Propose(commit, rows): Propose, ) -> Option { - let sender_idx = - if let Some(sender_idx) = self.pub_keys.keys().position(|uid| uid == sender_id) { - sender_idx as u64 - } else { - debug!("Unknown sender {:?}", sender_id); - return None; - }; - let commit_row = commit.row(self.our_idx + 1); + 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. Entry::Vacant(entry) => { entry.insert(ProposalState::new(commit)); } } - let ser_row = self.sec_key.decrypt(rows.get(self.our_idx as usize)?)?; + // If we are only an observer, return `None`. We don't need to send `Accept`. + 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)?)?; let row: Poly = bincode::deserialize(&ser_row).ok()?; // Ignore invalid messages. if row.commitment() != commit_row { debug!("Invalid proposal from node {}.", sender_idx); return None; } // The row is valid: now encrypt one value for each node. - let values = self - .pub_keys - .values() - .enumerate() - .map(|(idx, pk)| { - let val = row.evaluate(idx as u64 + 1); - let ser_val = - bincode::serialize(&FieldWrap::new(val)).expect("failed to serialize value"); - pk.encrypt(ser_val) - }) - .collect(); + let encrypt = |(idx, pk): (usize, &PublicKey)| { + let val = row.evaluate(idx as u64 + 1); + let wrap = FieldWrap::new(val); + // TODO: Handle errors. + let ser_val = bincode::serialize(&wrap).expect("failed to serialize value"); + pk.encrypt(ser_val) + }; + let values = self.pub_keys.values().enumerate().map(encrypt).collect(); Some(Accept(sender_idx, values)) } /// Handles an `Accept` message. pub fn handle_accept(&mut self, sender_id: &NodeUid, accept: Accept) { - let sender_idx = - if let Some(sender_idx) = self.pub_keys.keys().position(|uid| uid == sender_id) { - sender_idx as u64 - } else { - debug!("Unknown sender {:?}", sender_id); - return; - }; - if let Err(err) = self.handle_accept_or_err(sender_idx, accept) { - debug!("Invalid accept from node {}: {}", sender_idx, err); + 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); + } } } @@ -214,20 +203,20 @@ impl SyncKeyGen { /// /// These are only secure if `is_ready` returned `true`. Otherwise it is not guaranteed that /// none of the nodes knows the secret master key. - pub fn generate(&self) -> (PublicKeySet, ClearOnDrop>) { + pub fn generate(&self) -> (PublicKeySet, Option>>) { let mut pk_commit = Poly::zero().commitment(); - let mut sk_val = Fr::zero(); - for proposal in self - .proposals - .values() - .filter(|proposal| proposal.is_complete(self.threshold)) - { + 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 row: Poly = Poly::interpolate(proposal.values.iter().take(self.threshold + 1)); - sk_val.add_assign(&row.evaluate(0)); + if let Some(sk_val) = opt_sk_val.as_mut() { + let row: Poly = Poly::interpolate(proposal.values.iter().take(self.threshold + 1)); + sk_val.add_assign(&row.evaluate(0)); + } } - let sk = ClearOnDrop::new(Box::new(SecretKey::from_value(sk_val))); - (pk_commit.into(), sk) + let opt_sk = + opt_sk_val.map(|sk_val| ClearOnDrop::new(Box::new(SecretKey::from_value(sk_val)))); + (pk_commit.into(), opt_sk) } /// Handles an `Accept` message or returns an error string. @@ -236,6 +225,9 @@ impl SyncKeyGen { sender_idx: u64, Accept(proposer_idx, values): Accept, ) -> Result<(), String> { + if values.len() != self.pub_keys.len() { + return Err("wrong node count".to_string()); + } let proposal = self .proposals .get_mut(&proposer_idx) @@ -243,20 +235,31 @@ impl SyncKeyGen { if !proposal.accepts.insert(sender_idx) { return Err("duplicate accept".to_string()); } - if values.len() != self.pub_keys.len() { - return Err("wrong node count".to_string()); - } + let our_idx = match self.our_idx { + Some(our_idx) => our_idx, + None => return Ok(()), // We are only an observer. Nothing to decrypt for us. + }; let ser_val: Vec = self .sec_key - .decrypt(&values[self.our_idx as usize]) + .decrypt(&values[our_idx as usize]) .ok_or_else(|| "value decryption failed".to_string())?; let val = bincode::deserialize::>(&ser_val) .map_err(|err| format!("deserialization failed: {:?}", err))? .into_inner(); - if proposal.commit.evaluate(self.our_idx + 1, sender_idx + 1) != G1Affine::one().mul(val) { + if proposal.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); Ok(()) } + + /// Returns the index of the node, or `None` if it is unknown. + fn node_index(&self, node_id: &NodeUid) -> Option { + if let Some(node_idx) = self.pub_keys.keys().position(|uid| uid == node_id) { + Some(node_idx as u64) + } else { + debug!("Unknown node {:?}", node_id); + None + } + } } diff --git a/tests/dynamic_honey_badger.rs b/tests/dynamic_honey_badger.rs new file mode 100644 index 0000000..cccebb6 --- /dev/null +++ b/tests/dynamic_honey_badger.rs @@ -0,0 +1,127 @@ +//! Network tests for Dynamic Honey Badger. + +extern crate hbbft; +#[macro_use] +extern crate log; +extern crate env_logger; +extern crate pairing; +extern crate rand; +#[macro_use] +extern crate serde_derive; + +mod network; + +use std::collections::BTreeMap; +use std::iter::once; +use std::rc::Rc; + +use rand::Rng; + +use hbbft::dynamic_honey_badger::{Change, DynamicHoneyBadger, Input, Message}; +use hbbft::messaging::NetworkInfo; + +use network::{Adversary, MessageScheduler, NodeUid, SilentAdversary, TestNetwork, TestNode}; + +/// Proposes `num_txs` values and expects nodes to output and order them. +fn test_dynamic_honey_badger( + mut network: TestNetwork>, + num_txs: usize, +) where + A: Adversary>, +{ + // The second half of the transactions will be input only after a node has been removed. + network.input_all(Input::Change(Change::Remove(NodeUid(0)))); + for tx in 0..(num_txs / 2) { + network.input_all(Input::User(tx)); + } + + // Returns `true` if the node has not output all transactions yet. + // If it has, and has advanced another epoch, it clears all messages for later epochs. + let node_busy = |node: &mut TestNode>| { + if !node + .outputs() + .iter() + .filter_map(|batch| batch.change()) + .any(|change| *change == Change::Remove(NodeUid(0))) + { + return true; + } + let mut min_missing = 0; + for batch in node.outputs() { + for tx in batch.iter() { + if *tx >= min_missing { + min_missing = tx + 1; + } + } + } + if min_missing < num_txs { + return true; + } + if node.outputs().last().unwrap().is_empty() { + let last = node.outputs().last().unwrap().epoch; + node.queue.retain(|(_, ref msg)| match msg { + Message::HoneyBadger(_, hb_msg) => hb_msg.epoch() < last, + }); + } + false + }; + + // Handle messages in random order until all nodes have output all transactions. + while network.nodes.values_mut().any(node_busy) { + let id = network.step(); + if network.nodes[&id] + .outputs() + .last() + .and_then(|batch| batch.change()) + .map_or(false, |change| *change == Change::Remove(id)) + { + network.nodes.remove(&id); + for tx in (num_txs / 2)..num_txs { + network.input_all(Input::User(tx)); + } + } + } + // TODO: Verify that all nodes output the same epochs. +} + +// Allow passing `netinfo` by value. `TestNetwork` expects this function signature. +#[cfg_attr(feature = "cargo-clippy", allow(needless_pass_by_value))] +fn new_dynamic_hb(netinfo: Rc>) -> DynamicHoneyBadger { + DynamicHoneyBadger::new((*netinfo).clone(), 12).expect("Instantiate dynamic_honey_badger") +} + +fn test_dynamic_honey_badger_different_sizes(new_adversary: F, num_txs: usize) +where + A: Adversary>, + F: Fn(usize, usize, BTreeMap>>) -> A, +{ + // This returns an error in all but the first test. + let _ = env_logger::try_init(); + + let mut rng = rand::thread_rng(); + let sizes = (3..5).chain(once(rng.gen_range(6, 10))); + for size in sizes { + // The test is removing one correct node, so we allow fewer faulty ones. + let num_adv_nodes = (size - 2) / 3; + let num_good_nodes = size - num_adv_nodes; + info!( + "Network size: {} good nodes, {} faulty nodes", + num_good_nodes, num_adv_nodes + ); + let adversary = |adv_nodes| new_adversary(num_good_nodes, num_adv_nodes, adv_nodes); + let network = TestNetwork::new(num_good_nodes, num_adv_nodes, adversary, new_dynamic_hb); + test_dynamic_honey_badger(network, num_txs); + } +} + +#[test] +fn test_dynamic_honey_badger_random_delivery_silent() { + let new_adversary = |_: usize, _: usize, _| SilentAdversary::new(MessageScheduler::Random); + test_dynamic_honey_badger_different_sizes(new_adversary, 10); +} + +#[test] +fn test_dynamic_honey_badger_first_delivery_silent() { + let new_adversary = |_: usize, _: usize, _| SilentAdversary::new(MessageScheduler::First); + test_dynamic_honey_badger_different_sizes(new_adversary, 10); +} diff --git a/tests/network/mod.rs b/tests/network/mod.rs index cade247..57ff304 100644 --- a/tests/network/mod.rs +++ b/tests/network/mod.rs @@ -242,12 +242,13 @@ where Target::Node(to_id) => { if self.adv_nodes.contains_key(&to_id) { self.adversary.push_message(sender_id, msg); + } else if let Some(node) = self.nodes.get_mut(&to_id) { + node.queue.push_back((sender_id, msg.message)); } else { - self.nodes - .get_mut(&to_id) - .unwrap() - .queue - .push_back((sender_id, msg.message)); + warn!( + "Unknown recipient {:?} for message: {:?}", + to_id, msg.message + ); } } } diff --git a/tests/sync_key_gen.rs b/tests/sync_key_gen.rs index 6b6f46f..686ecd2 100644 --- a/tests/sync_key_gen.rs +++ b/tests/sync_key_gen.rs @@ -38,7 +38,7 @@ fn test_sync_key_gen_with(threshold: usize, node_num: usize) { for (sender_id, proposal) in proposals[..=threshold].iter().enumerate() { for (node_id, node) in nodes.iter_mut().enumerate() { let accept = node - .handle_propose(&sender_id, proposal.clone()) + .handle_propose(&sender_id, proposal.clone().expect("proposal")) .expect("valid proposal"); // Only the first `threshold + 1` manage to commit their `Accept`s. if node_id <= 2 * threshold { @@ -63,7 +63,8 @@ fn test_sync_key_gen_with(threshold: usize, node_num: usize) { .enumerate() .map(|(idx, node)| { assert!(node.is_ready()); - let (pks, sk) = node.generate(); + let (pks, opt_sk) = node.generate(); + let sk = opt_sk.expect("new secret key"); assert_eq!(pks, pub_key_set); let sig = sk.sign(msg); assert!(pks.public_key_share(idx as u64).verify(&sig, msg));