Extract vote counter from DHB.

This commit is contained in:
Andreas Fackler 2018-07-11 16:18:32 +02:00
parent 0e39bfed40
commit 313fe1da75
5 changed files with 364 additions and 124 deletions

View File

@ -1,4 +1,4 @@
use std::collections::{BTreeMap, VecDeque}; use std::collections::VecDeque;
use std::fmt::Debug; use std::fmt::Debug;
use std::hash::Hash; use std::hash::Hash;
use std::marker::PhantomData; use std::marker::PhantomData;
@ -6,7 +6,7 @@ use std::sync::Arc;
use serde::{Deserialize, Serialize}; use serde::{Deserialize, Serialize};
use super::{DynamicHoneyBadger, MessageQueue}; use super::{DynamicHoneyBadger, MessageQueue, VoteCounter};
use honey_badger::HoneyBadger; use honey_badger::HoneyBadger;
use messaging::NetworkInfo; use messaging::NetworkInfo;
@ -54,15 +54,16 @@ where
/// Creates a new Dynamic Honey Badger instance with an empty buffer. /// Creates a new Dynamic Honey Badger instance with an empty buffer.
pub fn build(&self) -> DynamicHoneyBadger<C, NodeUid> { pub fn build(&self) -> DynamicHoneyBadger<C, NodeUid> {
let honey_badger = HoneyBadger::builder(Arc::new(self.netinfo.clone())) let netinfo = Arc::new(self.netinfo.clone());
let honey_badger = HoneyBadger::builder(netinfo.clone())
.max_future_epochs(self.max_future_epochs) .max_future_epochs(self.max_future_epochs)
.build(); .build();
DynamicHoneyBadger { DynamicHoneyBadger {
netinfo: self.netinfo.clone(), netinfo: self.netinfo.clone(),
max_future_epochs: self.max_future_epochs, max_future_epochs: self.max_future_epochs,
start_epoch: self.start_epoch, start_epoch: self.start_epoch,
votes: BTreeMap::new(), vote_counter: VoteCounter::new(netinfo, self.start_epoch),
node_tx_buffer: Vec::new(), key_gen_msg_buffer: Vec::new(),
honey_badger, honey_badger,
key_gen: None, key_gen: None,
incoming_queue: Vec::new(), incoming_queue: Vec::new(),

View File

@ -44,7 +44,7 @@
//! `SyncKeyGen` instance is dropped, and a new one is started to create keys according to the new //! `SyncKeyGen` instance is dropped, and a new one is started to create keys according to the new
//! pending change. //! pending change.
use std::collections::{BTreeMap, HashMap, VecDeque}; use std::collections::VecDeque;
use std::fmt::Debug; use std::fmt::Debug;
use std::hash::Hash; use std::hash::Hash;
use std::mem; use std::mem;
@ -54,6 +54,7 @@ use bincode;
use clear_on_drop::ClearOnDrop; use clear_on_drop::ClearOnDrop;
use serde::{Deserialize, Serialize}; use serde::{Deserialize, Serialize};
use self::votes::{SignedVote, VoteCounter};
use crypto::{PublicKeySet, SecretKey, Signature}; use crypto::{PublicKeySet, SecretKey, Signature};
use fault_log::{FaultKind, FaultLog}; use fault_log::{FaultKind, FaultLog};
use honey_badger::{HoneyBadger, Message as HbMessage}; use honey_badger::{HoneyBadger, Message as HbMessage};
@ -63,12 +64,13 @@ use sync_key_gen::{Accept, Propose, ProposeOutcome, SyncKeyGen};
pub use self::batch::Batch; pub use self::batch::Batch;
pub use self::builder::DynamicHoneyBadgerBuilder; pub use self::builder::DynamicHoneyBadgerBuilder;
pub use self::change::{Change, ChangeState}; pub use self::change::{Change, ChangeState};
pub use self::err::{Error, ErrorKind, Result}; pub use self::error::{Error, ErrorKind, Result};
mod batch; mod batch;
mod builder; mod builder;
mod change; mod change;
mod err; mod error;
mod votes;
type KeyGenOutput = (PublicKeySet, Option<ClearOnDrop<Box<SecretKey>>>); type KeyGenOutput = (PublicKeySet, Option<ClearOnDrop<Box<SecretKey>>>);
@ -93,12 +95,10 @@ where
max_future_epochs: usize, max_future_epochs: usize,
/// The first epoch after the latest node change. /// The first epoch after the latest node change.
start_epoch: u64, start_epoch: u64,
/// Collected votes for adding or removing nodes. Each node has one vote, and casting another /// The buffer and counter for the pending and committed change votes.
/// vote revokes the previous one. Resets whenever the set of validators is successfully vote_counter: VoteCounter<NodeUid>,
/// changed.
votes: BTreeMap<NodeUid, Change<NodeUid>>,
/// Pending node transactions that we will propose in the next epoch. /// Pending node transactions that we will propose in the next epoch.
node_tx_buffer: Vec<SignedTransaction<NodeUid>>, key_gen_msg_buffer: Vec<SignedKeyGenMsg<NodeUid>>,
/// The `HoneyBadger` instance with the current set of nodes. /// The `HoneyBadger` instance with the current set of nodes.
honey_badger: HoneyBadger<InternalContrib<C, NodeUid>, NodeUid>, honey_badger: HoneyBadger<InternalContrib<C, NodeUid>, NodeUid>,
/// The current key generation process, and the change it applies to. /// The current key generation process, and the change it applies to.
@ -126,16 +126,9 @@ where
// User contributions are forwarded to `HoneyBadger` right away. Votes are signed and // User contributions are forwarded to `HoneyBadger` right away. Votes are signed and
// broadcast. // broadcast.
match input { match input {
Input::User(contrib) => { Input::User(contrib) => self.propose(contrib),
let mut fault_log = self.honey_badger.input(InternalContrib {
contrib,
transactions: self.node_tx_buffer.clone(),
})?;
self.process_output()?.merge_into(&mut fault_log);
Ok(fault_log)
}
Input::Change(change) => { Input::Change(change) => {
self.send_transaction(NodeTransaction::Change(change))?; self.vote_for(change)?;
Ok(FaultLog::new()) Ok(FaultLog::new())
} }
} }
@ -158,8 +151,10 @@ where
} }
match message { match message {
Message::HoneyBadger(_, hb_msg) => self.handle_honey_badger_message(sender_id, hb_msg), Message::HoneyBadger(_, hb_msg) => self.handle_honey_badger_message(sender_id, hb_msg),
Message::Signed(_, node_tx, sig) => { Message::KeyGen(_, kg_msg, sig) => self.handle_key_gen_message(sender_id, kg_msg, *sig),
self.handle_signed_message(sender_id, node_tx, *sig) Message::SignedVote(signed_vote) => {
self.vote_counter.add_pending_vote(signed_vote)?;
Ok(FaultLog::new())
} }
} }
} }
@ -181,14 +176,14 @@ where
} }
} }
impl<Tx, NodeUid> DynamicHoneyBadger<Tx, NodeUid> impl<C, NodeUid> DynamicHoneyBadger<C, NodeUid>
where where
Tx: Eq + Serialize + for<'r> Deserialize<'r> + Debug + Hash, C: Eq + Serialize + for<'r> Deserialize<'r> + Debug + Hash,
NodeUid: Eq + Ord + Clone + Debug + Serialize + for<'r> Deserialize<'r> + Hash, NodeUid: Eq + Ord + Clone + Debug + Serialize + for<'r> Deserialize<'r> + Hash,
{ {
/// Returns a new `DynamicHoneyBadgerBuilder` configured to use the node IDs and cryptographic /// Returns a new `DynamicHoneyBadgerBuilder` configured to use the node IDs and cryptographic
/// keys specified by `netinfo`. /// keys specified by `netinfo`.
pub fn builder(netinfo: NetworkInfo<NodeUid>) -> DynamicHoneyBadgerBuilder<Tx, NodeUid> { pub fn builder(netinfo: NetworkInfo<NodeUid>) -> DynamicHoneyBadgerBuilder<C, NodeUid> {
DynamicHoneyBadgerBuilder::new(netinfo) DynamicHoneyBadgerBuilder::new(netinfo)
} }
@ -197,6 +192,28 @@ where
self.honey_badger.has_input() self.honey_badger.has_input()
} }
/// Proposes a contribution in the current epoch.
pub fn propose(&mut self, contrib: C) -> Result<FaultLog<NodeUid>> {
let mut fault_log = self.honey_badger.input(InternalContrib {
contrib,
key_gen_messages: self.key_gen_msg_buffer.clone(),
votes: self.vote_counter.pending_votes().cloned().collect(),
})?;
self.process_output()?.merge_into(&mut fault_log);
Ok(fault_log)
}
/// Cast a vote to change the set of validators.
pub fn vote_for(&mut self, change: Change<NodeUid>) -> Result<()> {
if !self.netinfo.is_validator() {
return Ok(()); // TODO: Return an error?
}
let signed_vote = self.vote_counter.sign_vote_for(change)?.clone();
let msg = Message::SignedVote(signed_vote);
self.messages.push_back(Target::All.message(msg));
Ok(())
}
/// Returns the information about the node IDs in the network, and the cryptographic keys. /// Returns the information about the node IDs in the network, and the cryptographic keys.
pub fn netinfo(&self) -> &NetworkInfo<NodeUid> { pub fn netinfo(&self) -> &NetworkInfo<NodeUid> {
&self.netinfo &self.netinfo
@ -220,15 +237,15 @@ where
/// Handles a vote or key generation message and tries to commit it as a transaction. These /// Handles a vote or key generation message and tries to commit it as a transaction. These
/// messages are only handled once they appear in a batch output from Honey Badger. /// messages are only handled once they appear in a batch output from Honey Badger.
fn handle_signed_message( fn handle_key_gen_message(
&mut self, &mut self,
sender_id: &NodeUid, sender_id: &NodeUid,
node_tx: NodeTransaction<NodeUid>, kg_msg: KeyGenMessage,
sig: Signature, sig: Signature,
) -> Result<FaultLog<NodeUid>> { ) -> Result<FaultLog<NodeUid>> {
self.verify_signature(sender_id, &sig, &node_tx)?; self.verify_signature(sender_id, &sig, &kg_msg)?;
let tx = SignedTransaction(self.start_epoch, sender_id.clone(), node_tx, sig); let tx = SignedKeyGenMsg(self.start_epoch, sender_id.clone(), kg_msg, sig);
self.node_tx_buffer.push(tx); self.key_gen_msg_buffer.push(tx);
self.process_output() self.process_output()
} }
@ -242,28 +259,23 @@ where
let mut batch = Batch::new(hb_batch.epoch + self.start_epoch); let mut batch = Batch::new(hb_batch.epoch + self.start_epoch);
// Add the user transactions to `batch` and handle votes and DKG messages. // Add the user transactions to `batch` and handle votes and DKG messages.
for (id, int_contrib) in hb_batch.contributions { for (id, int_contrib) in hb_batch.contributions {
self.vote_counter.add_committed_votes(int_contrib.votes)?;
batch.contributions.insert(id, int_contrib.contrib); batch.contributions.insert(id, int_contrib.contrib);
for SignedTransaction(epoch, s_id, node_tx, sig) in int_contrib.transactions { for SignedKeyGenMsg(epoch, s_id, kg_msg, sig) in int_contrib.key_gen_messages {
if epoch < self.start_epoch { if epoch < self.start_epoch {
info!("Obsolete node transaction: {:?}.", node_tx); info!("Obsolete key generation message: {:?}.", kg_msg);
continue; continue;
} }
if !self.verify_signature(&s_id, &sig, &node_tx)? { if !self.verify_signature(&s_id, &sig, &kg_msg)? {
info!("Invalid signature from {:?} for: {:?}.", s_id, node_tx); info!("Invalid signature from {:?} for: {:?}.", s_id, kg_msg);
let fault_kind = FaultKind::InvalidNodeTransactionSignature; let fault_kind = FaultKind::InvalidKeyGenMessageSignature;
fault_log.append(s_id.clone(), fault_kind); fault_log.append(s_id.clone(), fault_kind);
continue; continue;
} }
use self::NodeTransaction::*; match kg_msg {
match node_tx { KeyGenMessage::Propose(propose) => self.handle_propose(&s_id, propose)?,
Change(change) => self.handle_vote(s_id, change), KeyGenMessage::Accept(accept) => self.handle_accept(&s_id, accept)?,
Propose(propose) => self }.merge_into(&mut fault_log);
.handle_propose(&s_id, propose)?
.merge_into(&mut fault_log),
Accept(accept) => self
.handle_accept(&s_id, accept)?
.merge_into(&mut fault_log),
}
} }
} }
if let Some(((pub_key_set, sk), change)) = self.take_key_gen_output() { if let Some(((pub_key_set, sk), change)) = self.take_key_gen_output() {
@ -276,9 +288,9 @@ where
// Restart Honey Badger in the next epoch, and inform the user about the change. // Restart Honey Badger in the next epoch, and inform the user about the change.
self.apply_change(&change, pub_key_set, sk, batch.epoch + 1)?; self.apply_change(&change, pub_key_set, sk, batch.epoch + 1)?;
batch.change = ChangeState::Complete(change); batch.change = ChangeState::Complete(change);
} else { } else if let Some(change) = self.vote_counter.compute_majority().cloned() {
// If the majority changed, restart DKG. Inform the user about the current change. // If there is a majority, restart DKG. Inform the user about the current change.
self.update_key_gen(batch.epoch + 1)?; self.update_key_gen(batch.epoch + 1, change)?;
if let Some((_, ref change)) = self.key_gen { if let Some((_, ref change)) = self.key_gen {
batch.change = ChangeState::InProgress(change.clone()); batch.change = ChangeState::InProgress(change.clone());
} }
@ -306,7 +318,6 @@ where
sk: ClearOnDrop<Box<SecretKey>>, sk: ClearOnDrop<Box<SecretKey>>,
epoch: u64, epoch: u64,
) -> Result<()> { ) -> Result<()> {
self.votes.clear();
self.key_gen = None; self.key_gen = None;
let mut all_uids = self.netinfo.all_uids().clone(); let mut all_uids = self.netinfo.all_uids().clone();
if !match *change { if !match *change {
@ -322,19 +333,10 @@ where
/// If the majority of votes has changed, restarts Key Generation for the set of nodes implied /// If the majority of votes has changed, restarts Key Generation for the set of nodes implied
/// by the current change. /// by the current change.
fn update_key_gen(&mut self, epoch: u64) -> Result<()> { fn update_key_gen(&mut self, epoch: u64, change: Change<NodeUid>) -> Result<()> {
let change = match current_majority(&self.votes, &self.netinfo) { if self.key_gen.as_ref().map(|&(_, ref ch)| ch) == Some(&change) {
None => { return Ok(()); // The change is the same as before. Continue DKG as is.
self.key_gen = None;
return Ok(());
} }
Some(change) => {
if self.key_gen.as_ref().map(|&(_, ref ch)| ch) == Some(change) {
return Ok(()); // The change is the same as last epoch. Continue DKG as is.
}
change.clone()
}
};
debug!("{:?} Restarting DKG for {:?}.", self.our_id(), change); debug!("{:?} Restarting DKG for {:?}.", self.our_id(), change);
// Use the existing key shares - with the change applied - as keys for DKG. // Use the existing key shares - with the change applied - as keys for DKG.
let mut pub_keys = self.netinfo.public_key_map().clone(); let mut pub_keys = self.netinfo.public_key_map().clone();
@ -344,9 +346,7 @@ where
} { } {
info!("{:?} No-op change: {:?}", self.our_id(), change); info!("{:?} No-op change: {:?}", self.our_id(), change);
} }
if change.candidate().is_some() {
self.restart_honey_badger(epoch)?; self.restart_honey_badger(epoch)?;
}
// TODO: This needs to be the same as `num_faulty` will be in the _new_ // 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. // `NetworkInfo` if the change goes through. It would be safer to deduplicate.
let threshold = (pub_keys.len() - 1) / 3; let threshold = (pub_keys.len() - 1) / 3;
@ -355,19 +355,21 @@ where
let (key_gen, propose) = SyncKeyGen::new(&our_uid, sk, pub_keys, threshold); let (key_gen, propose) = SyncKeyGen::new(&our_uid, sk, pub_keys, threshold);
self.key_gen = Some((key_gen, change)); self.key_gen = Some((key_gen, change));
if let Some(propose) = propose { if let Some(propose) = propose {
self.send_transaction(NodeTransaction::Propose(propose))?; self.send_transaction(KeyGenMessage::Propose(propose))?;
} }
Ok(()) Ok(())
} }
/// Starts a new `HoneyBadger` instance and inputs the user transactions from the existing /// Starts a new `HoneyBadger` instance and resets the vote counter.
/// one's buffer and pending outputs.
fn restart_honey_badger(&mut self, epoch: u64) -> Result<()> { fn restart_honey_badger(&mut self, epoch: u64) -> Result<()> {
// TODO: Filter out the messages for `epoch` and later. // TODO: Filter out the messages for `epoch` and later.
self.messages self.messages
.extend_with_epoch(self.start_epoch, &mut self.honey_badger); .extend_with_epoch(self.start_epoch, &mut self.honey_badger);
self.start_epoch = epoch; self.start_epoch = epoch;
self.honey_badger = HoneyBadger::builder(Arc::new(self.netinfo.clone())) let netinfo = Arc::new(self.netinfo.clone());
let counter = VoteCounter::new(netinfo.clone(), epoch);
mem::replace(&mut self.vote_counter, counter);
self.honey_badger = HoneyBadger::builder(netinfo)
.max_future_epochs(self.max_future_epochs) .max_future_epochs(self.max_future_epochs)
.build(); .build();
Ok(()) Ok(())
@ -384,7 +386,7 @@ where
}; };
match self.key_gen.as_mut().and_then(handle) { match self.key_gen.as_mut().and_then(handle) {
Some(ProposeOutcome::Valid(accept)) => { Some(ProposeOutcome::Valid(accept)) => {
self.send_transaction(NodeTransaction::Accept(accept))?; self.send_transaction(KeyGenMessage::Accept(accept))?;
Ok(FaultLog::new()) Ok(FaultLog::new())
} }
Some(ProposeOutcome::Invalid(fault_log)) => Ok(fault_log), Some(ProposeOutcome::Invalid(fault_log)) => Ok(fault_log),
@ -401,17 +403,18 @@ where
} }
} }
/// Signs and sends a `NodeTransaction` and also tries to commit it. /// Signs and sends a `KeyGenMessage` and also tries to commit it.
fn send_transaction(&mut self, node_tx: NodeTransaction<NodeUid>) -> Result<()> { fn send_transaction(&mut self, kg_msg: KeyGenMessage) -> Result<()> {
let sig = self.sign(&node_tx)?; let ser = bincode::serialize(&kg_msg)?;
let msg = Message::Signed(self.start_epoch, node_tx.clone(), sig.clone()); let sig = Box::new(self.netinfo.secret_key().sign(ser));
let msg = Message::KeyGen(self.start_epoch, kg_msg.clone(), sig.clone());
self.messages.push_back(Target::All.message(msg)); self.messages.push_back(Target::All.message(msg));
if !self.netinfo.is_validator() { if !self.netinfo.is_validator() {
return Ok(()); return Ok(());
} }
let our_uid = self.netinfo.our_uid().clone(); let our_uid = self.netinfo.our_uid().clone();
self.node_tx_buffer let signed_msg = SignedKeyGenMsg(self.start_epoch, our_uid, kg_msg, *sig);
.push(SignedTransaction(self.start_epoch, our_uid, node_tx, *sig)); self.key_gen_msg_buffer.push(signed_msg);
Ok(()) Ok(())
} }
@ -433,21 +436,15 @@ where
} }
} }
/// Returns a signature of `node_tx`, or an error if serialization fails. /// Returns `true` if the signature of `kg_msg` by the node with the specified ID is valid.
fn sign(&self, node_tx: &NodeTransaction<NodeUid>) -> Result<Box<Signature>> {
let ser = bincode::serialize(node_tx)?;
Ok(Box::new(self.netinfo.secret_key().sign(ser)))
}
/// Returns `true` if the signature of `node_tx` by the node with the specified ID is valid.
/// Returns an error if the payload fails to serialize. /// Returns an error if the payload fails to serialize.
fn verify_signature( fn verify_signature(
&self, &self,
node_id: &NodeUid, node_id: &NodeUid,
sig: &Signature, sig: &Signature,
node_tx: &NodeTransaction<NodeUid>, kg_msg: &KeyGenMessage,
) -> Result<bool> { ) -> Result<bool> {
let ser = bincode::serialize(node_tx)?; let ser = bincode::serialize(kg_msg)?;
let pk_opt = (self.netinfo.public_key_share(node_id)).or_else(|| { let pk_opt = (self.netinfo.public_key_share(node_id)).or_else(|| {
self.key_gen self.key_gen
.iter() .iter()
@ -459,33 +456,6 @@ where
}); });
Ok(pk_opt.map_or(false, |pk| pk.verify(&sig, ser))) 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<NodeUid>) {
let obsolete = match change {
Change::Add(ref id, _) => self.netinfo.all_uids().contains(id),
Change::Remove(ref id) => !self.netinfo.all_uids().contains(id),
};
if !obsolete {
self.votes.insert(sender_id, change);
}
}
}
/// Returns the change that currently has a majority of votes, if any.
fn current_majority<'a, NodeUid: Ord + Clone + Hash + Eq>(
votes: &'a BTreeMap<NodeUid, Change<NodeUid>>,
netinfo: &'a NetworkInfo<NodeUid>,
) -> Option<&'a Change<NodeUid>> {
let mut vote_counts: HashMap<&Change<NodeUid>, usize> = HashMap::new();
for change in votes.values() {
let entry = vote_counts.entry(change).or_insert(0);
*entry += 1;
if *entry * 2 > netinfo.num_nodes() {
return Some(change);
}
}
None
} }
/// The contribution for the internal `HoneyBadger` instance: this includes a user-defined /// The contribution for the internal `HoneyBadger` instance: this includes a user-defined
@ -494,21 +464,21 @@ fn current_majority<'a, NodeUid: Ord + Clone + Hash + Eq>(
struct InternalContrib<C, NodeUid> { struct InternalContrib<C, NodeUid> {
/// A user-defined contribution. /// A user-defined contribution.
contrib: C, contrib: C,
/// Signed internal messages that get committed via Honey Badger to communicate synchronously. /// Key generation messages that get committed via Honey Badger to communicate synchronously.
transactions: Vec<SignedTransaction<NodeUid>>, key_gen_messages: Vec<SignedKeyGenMsg<NodeUid>>,
/// Signed votes for validator set changes.
votes: Vec<SignedVote<NodeUid>>,
} }
/// A signed internal message. /// A signed internal message.
#[derive(Eq, PartialEq, Debug, Serialize, Deserialize, Hash, Clone)] #[derive(Eq, PartialEq, Debug, Serialize, Deserialize, Hash, Clone)]
struct SignedTransaction<NodeUid>(u64, NodeUid, NodeTransaction<NodeUid>, Signature); struct SignedKeyGenMsg<NodeUid>(u64, NodeUid, KeyGenMessage, Signature);
/// An internal message containing a vote for adding or removing a validator, or a message for key /// An internal message containing a vote for adding or removing a validator, or a message for key
/// generation. It gets committed via Honey Badger and is only handled after it has been output in /// generation. It gets committed via Honey Badger and is only handled after it has been output in
/// a batch, so that all nodes see these messages in the same order. /// a batch, so that all nodes see these messages in the same order.
#[derive(Eq, PartialEq, Debug, Serialize, Deserialize, Hash, Clone)] #[derive(Eq, PartialEq, Debug, Serialize, Deserialize, Hash, Clone)]
pub enum NodeTransaction<NodeUid> { pub enum KeyGenMessage {
/// A vote to add or remove a validator.
Change(Change<NodeUid>),
/// A `SyncKeyGen::Propose` message for key generation. /// A `SyncKeyGen::Propose` message for key generation.
Propose(Propose), Propose(Propose),
/// A `SyncKeyGen::Accept` message for key generation. /// A `SyncKeyGen::Accept` message for key generation.
@ -521,14 +491,17 @@ pub enum Message<NodeUid> {
/// A message belonging to the `HoneyBadger` algorithm started in the given epoch. /// A message belonging to the `HoneyBadger` algorithm started in the given epoch.
HoneyBadger(u64, HbMessage<NodeUid>), HoneyBadger(u64, HbMessage<NodeUid>),
/// A transaction to be committed, signed by a node. /// A transaction to be committed, signed by a node.
Signed(u64, NodeTransaction<NodeUid>, Box<Signature>), KeyGen(u64, KeyGenMessage, Box<Signature>),
/// A vote to be committed, signed by a validator.
SignedVote(SignedVote<NodeUid>),
} }
impl<NodeUid> Message<NodeUid> { impl<NodeUid> Message<NodeUid> {
pub fn epoch(&self) -> u64 { pub fn epoch(&self) -> u64 {
match *self { match *self {
Message::HoneyBadger(epoch, _) => epoch, Message::HoneyBadger(epoch, _) => epoch,
Message::Signed(epoch, _, _) => epoch, Message::KeyGen(epoch, _, _) => epoch,
Message::SignedVote(ref signed_vote) => signed_vote.era(),
} }
} }
} }

View File

@ -0,0 +1,266 @@
use messaging::NetworkInfo;
use std::collections::{btree_map, BTreeMap, HashMap};
use std::fmt::Debug;
use std::hash::Hash;
use std::sync::Arc;
use bincode;
use serde::{Deserialize, Serialize};
use super::{Change, Result};
use crypto::Signature;
/// A buffer and counter collecting pending and committed votes for validator set changes.
///
/// This is reset whenever the set of validators changes or a change reaches a majority. We call
/// the epochs since the last reset the current _era_.
pub struct VoteCounter<NodeUid> {
/// Shared network data.
netinfo: Arc<NetworkInfo<NodeUid>>,
/// The epoch when voting was reset.
era: u64,
/// Pending node transactions that we will propose in the next epoch.
pending: BTreeMap<NodeUid, SignedVote<NodeUid>>,
/// Collected votes for adding or removing nodes. Each node has one vote, and casting another
/// vote revokes the previous one.
committed: BTreeMap<NodeUid, Vote<NodeUid>>,
}
impl<NodeUid> VoteCounter<NodeUid>
where
NodeUid: Eq + Hash + Ord + Clone + Debug + Serialize + for<'r> Deserialize<'r>,
{
/// Creates a new `VoteCounter` object with empty buffer and counter.
pub fn new(netinfo: Arc<NetworkInfo<NodeUid>>, era: u64) -> Self {
VoteCounter {
era,
netinfo,
pending: BTreeMap::new(),
committed: BTreeMap::new(),
}
}
/// Creates a signed vote for the given change, and inserts it into the pending votes buffer.
pub fn sign_vote_for(&mut self, change: Change<NodeUid>) -> Result<&SignedVote<NodeUid>> {
let voter = self.netinfo.our_uid().clone();
let vote = Vote {
change,
era: self.era,
num: self.pending.get(&voter).map_or(0, |sv| sv.vote.num + 1),
};
let ser_vote = bincode::serialize(&vote)?;
let signed_vote = SignedVote {
vote,
voter: voter.clone(),
sig: self.netinfo.secret_key().sign(ser_vote),
};
self.pending.insert(voter.clone(), signed_vote);
Ok(self.pending.get(&voter).expect("entry was just inserted"))
}
/// Inserts a pending vote into the buffer, if it has a higher number than the existing one.
// TODO: Detect and report double votes where feasible.
pub fn add_pending_vote(&mut self, signed_vote: SignedVote<NodeUid>) -> Result<()> {
if !self.validate(&signed_vote)? {
return Ok(());
}
if signed_vote.vote.era != self.era {
return Ok(()); // The vote is obsolete.
}
match self.pending.entry(signed_vote.voter.clone()) {
btree_map::Entry::Vacant(entry) => {
entry.insert(signed_vote);
}
btree_map::Entry::Occupied(mut entry) => {
if entry.get().vote.num < signed_vote.vote.num {
entry.insert(signed_vote);
}
}
}
Ok(())
}
/// Returns an iterator over all pending votes that are newer than their voter's committed
/// vote.
pub fn pending_votes<'a>(&'a self) -> impl Iterator<Item = &'a SignedVote<NodeUid>> {
self.pending.values().filter(move |signed_vote| {
self.committed
.get(&signed_vote.voter)
.map_or(true, |vote| vote.num < signed_vote.vote.num)
})
}
// TODO: Document and return fault logs?
pub fn add_committed_votes<I>(&mut self, signed_votes: I) -> Result<()>
where
I: IntoIterator<Item = SignedVote<NodeUid>>,
{
for signed_vote in signed_votes {
self.add_committed_vote(signed_vote)?;
}
Ok(())
}
/// Inserts a committed vote into the counter, if it has a higher number than the existing one.
pub fn add_committed_vote(&mut self, signed_vote: SignedVote<NodeUid>) -> Result<()> {
if !self.validate(&signed_vote)? || signed_vote.vote.era != self.era {
return Ok(());
}
match self.committed.entry(signed_vote.voter.clone()) {
btree_map::Entry::Vacant(entry) => {
entry.insert(signed_vote.vote);
}
btree_map::Entry::Occupied(mut entry) => {
if entry.get().num < signed_vote.vote.num {
entry.insert(signed_vote.vote);
}
}
}
Ok(())
}
/// Returns the change that has a majority of votes, if any.
pub fn compute_majority(&self) -> Option<&Change<NodeUid>> {
let mut vote_counts: HashMap<&Change<NodeUid>, usize> = HashMap::new();
for vote in self.committed.values() {
let change = &vote.change;
let entry = vote_counts.entry(change).or_insert(0);
*entry += 1;
if *entry * 2 > self.netinfo.num_nodes() {
return Some(change);
}
}
None
}
/// Returns `true` if the signature is valid.
fn validate(&self, signed_vote: &SignedVote<NodeUid>) -> Result<bool> {
let ser_vote = bincode::serialize(&signed_vote.vote)?;
let pk_opt = self.netinfo.public_key_share(&signed_vote.voter);
Ok(pk_opt.map_or(false, |pk| pk.verify(&signed_vote.sig, ser_vote)))
}
}
/// A vote fore removing or adding a validator.
#[derive(Eq, PartialEq, Debug, Serialize, Deserialize, Hash, Clone)]
struct Vote<NodeUid> {
/// The change this vote is for.
change: Change<NodeUid>,
/// The epoch in which the current era began.
era: u64,
/// The vote number: VoteCounter can be changed by casting another vote with a higher number.
num: u64,
}
/// A signed vote for removing or adding a validator.
#[derive(Eq, PartialEq, Debug, Serialize, Deserialize, Hash, Clone)]
pub struct SignedVote<NodeUid> {
vote: Vote<NodeUid>,
voter: NodeUid,
sig: Signature,
}
impl<NodeUid> SignedVote<NodeUid> {
pub fn era(&self) -> u64 {
self.vote.era
}
}
#[cfg(test)]
mod tests {
use std::collections::BTreeSet;
use std::sync::Arc;
use rand;
use super::{Change, SignedVote, VoteCounter};
use crypto::SecretKeySet;
use messaging::NetworkInfo;
/// Returns a vector of `node_num` `VoteCounter`s, and some signed example votes.
///
/// If `signed_votes` is the second entry of the return value, then `signed_votes[i][j]` is the
/// the vote for `Remove(j)` by node `i`. Each node signed `Remove(0)`, `Remove(1)`, ... in
/// order.
fn setup(node_num: usize, era: u64) -> (Vec<VoteCounter<usize>>, Vec<Vec<SignedVote<usize>>>) {
let mut rng = rand::thread_rng();
let sk_set = SecretKeySet::random(3, &mut rng);
let ids: BTreeSet<usize> = (0..node_num).collect();
let pk_set = sk_set.public_keys();
let create_counter = |id: usize| {
let sk = sk_set.secret_key_share(id as u64);
let netinfo = NetworkInfo::new(id, ids.clone(), sk, pk_set.clone());
VoteCounter::new(Arc::new(netinfo), era)
};
let mut counters: Vec<_> = (0..node_num).map(create_counter).collect();
let sign_votes = |counter: &mut VoteCounter<usize>| {
(0..node_num)
.map(Change::Remove)
.map(|change| counter.sign_vote_for(change).expect("sign vote").clone())
.collect::<Vec<_>>()
};
let signed_votes: Vec<_> = counters.iter_mut().map(sign_votes).collect();
(counters, signed_votes)
}
#[test]
fn test_pending_votes() {
let node_num = 4;
let era = 5;
// Create the counter instances and the matrix of signed votes.
let (mut counters, sv) = setup(node_num, era);
// We will only use counter number 0.
let ct = &mut counters[0];
// Node 0 already contains its own vote for `Remove(3)`. Add two more.
ct.add_pending_vote(sv[1][2].clone()).expect("add pending");
ct.add_pending_vote(sv[2][1].clone()).expect("add pending");
// Include a vote with a wrong signature.
ct.add_pending_vote(SignedVote {
sig: sv[2][1].sig.clone(),
..sv[3][1].clone()
}).expect("add pending");
assert_eq!(
ct.pending_votes().collect::<Vec<_>>(),
vec![&sv[0][3], &sv[1][2], &sv[2][1]]
);
// Now add an older vote by node 1 and a newer one by node 2. Only the latter should be
// included.
ct.add_pending_vote(sv[1][1].clone()).expect("add pending");
ct.add_pending_vote(sv[2][2].clone()).expect("add pending");
assert_eq!(
ct.pending_votes().collect::<Vec<_>>(),
vec![&sv[0][3], &sv[1][2], &sv[2][2]]
);
// Adding a committed vote removes it from the pending ones, unless it is older.
let vote_batch = vec![sv[1][3].clone(), sv[2][1].clone(), sv[0][3].clone()];
ct.add_committed_votes(vote_batch).expect("add committed");
assert_eq!(ct.pending_votes().collect::<Vec<_>>(), vec![&sv[2][2]]);
}
#[test]
fn test_committed_votes() {
let node_num = 4;
let era = 5;
// Create the counter instances and the matrix of signed votes.
let (mut counters, sv) = setup(node_num, era);
// We will only use counter number 0.
let ct = &mut counters[0];
let mut vote_batch = vec![sv[1][1].clone(), sv[2][1].clone()];
// Include a vote with a wrong signature.
vote_batch.push(SignedVote {
sig: sv[2][1].sig.clone(),
..sv[3][1].clone()
});
ct.add_committed_votes(vote_batch).expect("add committed");
assert_eq!(ct.compute_majority(), None);
// Adding the third vote for `Remove(1)` should return the change: It has the majority.
ct.add_committed_vote(sv[3][1].clone())
.expect("add committed");
assert_eq!(ct.compute_majority(), Some(&Change::Remove(1)));
}
}

View File

@ -23,9 +23,9 @@ pub enum FaultKind {
/// `HoneyBadger` could not deserialize bytes (i.e. a serialized Batch) /// `HoneyBadger` could not deserialize bytes (i.e. a serialized Batch)
/// from a given proposer into a vector of transactions. /// from a given proposer into a vector of transactions.
BatchDeserializationFailed, BatchDeserializationFailed,
/// `DynamicHoneyBadger` received a node transaction with an invalid /// `DynamicHoneyBadger` received a key generation message with an invalid
/// signature. /// signature.
InvalidNodeTransactionSignature, InvalidKeyGenMessageSignature,
/// `DynamicHoneyBadger` received a message (Accept, Propose, or Change) /// `DynamicHoneyBadger` received a message (Accept, Propose, or Change)
/// with an invalid signature. /// with an invalid signature.
IncorrectPayloadSignature, IncorrectPayloadSignature,