Merge pull request #86 from poanetwork/afck-dyn-hb

Add DynamicHoneyBadger (part 1).
This commit is contained in:
Vladimir Komendantskiy 2018-06-27 11:20:20 +01:00 committed by GitHub
commit f7dd8347bf
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 691 additions and 74 deletions

473
src/dynamic_honey_badger.rs Normal file
View File

@ -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<bincode::ErrorKind>);
}
errors {
UnknownSender
}
}
/// A node change action: adding or removing a node.
#[derive(Clone, Eq, PartialEq, Serialize, Deserialize, Hash, Debug)]
pub enum Change<NodeUid> {
/// 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<Tx, NodeUid> {
/// A user-defined transaction.
User(Tx),
/// A vote to change the set of nodes.
Change(Change<NodeUid>),
}
/// 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<Tx, NodeUid>
where
Tx: Eq + Serialize + for<'r> Deserialize<'r> + Debug + Hash,
NodeUid: Ord + Clone + Serialize + for<'r> Deserialize<'r> + Debug,
{
/// Shared network data.
netinfo: NetworkInfo<NodeUid>,
/// 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<NodeUid, Change<NodeUid>>,
/// The `HoneyBadger` instance with the current set of nodes.
honey_badger: HoneyBadger<Transaction<Tx, NodeUid>, NodeUid>,
/// The current key generation process.
key_gen: Option<SyncKeyGen<NodeUid>>,
/// A queue for messages from future epochs that cannot be handled yet.
incoming_queue: Vec<(u64, NodeUid, honey_badger::Message<NodeUid>)>,
/// The messages that need to be sent to other nodes.
messages: MessageQueue<NodeUid>,
/// The outputs from completed epochs.
output: VecDeque<Batch<Tx, NodeUid>>,
}
impl<Tx, NodeUid> DistAlgorithm for DynamicHoneyBadger<Tx, NodeUid>
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<Tx, NodeUid>;
type Output = Batch<Tx, NodeUid>;
type Message = Message<NodeUid>;
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<TargetedMessage<Self::Message, NodeUid>> {
self.messages.pop_front()
}
fn next_output(&mut self) -> Option<Self::Output> {
self.output.pop_front()
}
fn terminated(&self) -> bool {
false
}
fn our_id(&self) -> &NodeUid {
self.netinfo.our_uid()
}
}
impl<Tx, NodeUid> DynamicHoneyBadger<Tx, NodeUid>
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<NodeUid>, batch_size: usize) -> Result<Self> {
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<NodeUid>,
) -> 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<Tx, NodeUid>) -> Result<Transaction<Tx, NodeUid>> {
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<NodeUid>,
pub_key_set: PublicKeySet,
sk: ClearOnDrop<Box<SecretKey>>,
) -> 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<NodeUid>) -> 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<NodeUid>| 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<ClearOnDrop<Box<SecretKey>>>)> {
// 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<T: Serialize>(&self, payload: &T) -> Result<Box<Signature>> {
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<T: Serialize>(
&self,
id: &NodeUid,
sig: &Signature,
payload: &T,
) -> Result<bool> {
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<NodeUid>,
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<Change<NodeUid>> {
let mut vote_counts: HashMap<&Change<NodeUid>, 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<Tx, NodeUid> {
/// A user-defined transaction.
User(Tx),
/// A vote by an existing node to add or remove a node.
Change(NodeUid, Change<NodeUid>, Box<Signature>),
/// A proposal message for key generation.
Propose(NodeUid, Propose, Box<Signature>),
/// An accept message for key generation.
Accept(NodeUid, Accept, Box<Signature>),
}
/// A batch of transactions the algorithm has output.
#[derive(Clone)]
pub struct Batch<Tx, NodeUid> {
/// The sequence number: there is exactly one batch in each epoch.
pub epoch: u64,
/// The user transactions committed in this epoch.
pub transactions: BTreeMap<NodeUid, Vec<Tx>>,
/// 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<Change<NodeUid>>,
/// 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<Tx, NodeUid: Ord> Batch<Tx, NodeUid> {
/// 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<Item = &Tx> {
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<NodeUid>> {
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<NodeUid> {
/// A message belonging to the `HoneyBadger` algorithm started in the given epoch.
HoneyBadger(u64, honey_badger::Message<NodeUid>),
}
/// The queue of outgoing messages in a `HoneyBadger` instance.
#[derive(Deref, DerefMut)]
struct MessageQueue<NodeUid>(VecDeque<TargetedMessage<Message<NodeUid>, NodeUid>>);
impl<NodeUid> MessageQueue<NodeUid>
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<Tx>(&mut self, epoch: u64, hb: &mut HoneyBadger<Tx, NodeUid>)
where
Tx: Eq + Serialize + for<'r> Deserialize<'r> + Debug + Hash,
{
let convert = |msg: TargetedMessage<honey_badger::Message<NodeUid>, NodeUid>| {
msg.map(|hb_msg| Message::HoneyBadger(epoch, hb_msg))
};
self.extend(hb.message_iter().map(convert));
}
}

View File

@ -116,7 +116,6 @@ where
}
}
// TODO: Use a threshold encryption scheme to encrypt the proposed transactions.
impl<Tx, NodeUid> HoneyBadger<Tx, NodeUid>
where
Tx: Serialize + for<'r> Deserialize<'r> + Debug + Hash + Eq,
@ -569,9 +568,6 @@ impl<Tx, NodeUid: Ord> Batch<Tx, NodeUid> {
}
/// 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<NodeUid> {
/// A message belonging to the common subset algorithm in the given epoch.

View File

@ -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;

View File

@ -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<NodeUid: Clone + Ord> NetworkInfo<NodeUid> {
&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<PublicKey> {
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<NodeUid, PublicKey> {
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)

View File

@ -88,7 +88,7 @@ impl ProposalState {
/// It requires that all nodes handle all messages in the exact same order.
pub struct SyncKeyGen<NodeUid> {
/// Our node index.
our_idx: u64,
our_idx: Option<u64>,
/// Our secret key.
sec_key: SecretKey,
/// The public keys of all nodes, by node index.
@ -101,29 +101,17 @@ pub struct SyncKeyGen<NodeUid> {
impl<NodeUid: Ord + Debug> SyncKeyGen<NodeUid> {
/// 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<NodeUid, PublicKey>,
threshold: usize,
) -> (SyncKeyGen<NodeUid>, Propose) {
) -> (SyncKeyGen<NodeUid>, Option<Propose>) {
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<NodeUid: Ord + Debug> SyncKeyGen<NodeUid> {
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,54 +140,43 @@ impl<NodeUid: Ord + Debug> SyncKeyGen<NodeUid> {
sender_id: &NodeUid,
Propose(commit, rows): Propose,
) -> Option<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 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 encrypt = |(idx, pk): (usize, &PublicKey)| {
let val = row.evaluate(idx as u64 + 1);
let ser_val =
bincode::serialize(&FieldWrap::new(val)).expect("failed to serialize value");
let wrap = FieldWrap::new(val);
// TODO: Handle errors.
let ser_val = bincode::serialize(&wrap).expect("failed to serialize value");
pk.encrypt(ser_val)
})
.collect();
};
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 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);
}
}
}
/// Returns the number of complete proposals. If this is at least `threshold + 1`, the keys can
/// be generated, but it is possible to wait for more to increase security.
@ -214,20 +203,20 @@ impl<NodeUid: Ord + Debug> SyncKeyGen<NodeUid> {
///
/// 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<Box<SecretKey>>) {
pub fn generate(&self) -> (PublicKeySet, Option<ClearOnDrop<Box<SecretKey>>>) {
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);
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<NodeUid: Ord + Debug> SyncKeyGen<NodeUid> {
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<NodeUid: Ord + Debug> SyncKeyGen<NodeUid> {
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<u8> = 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::<FieldWrap<Fr, Fr>>(&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<u64> {
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
}
}
}

View File

@ -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<A>(
mut network: TestNetwork<A, DynamicHoneyBadger<usize, NodeUid>>,
num_txs: usize,
) where
A: Adversary<DynamicHoneyBadger<usize, NodeUid>>,
{
// 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<DynamicHoneyBadger<usize, NodeUid>>| {
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<NetworkInfo<NodeUid>>) -> DynamicHoneyBadger<usize, NodeUid> {
DynamicHoneyBadger::new((*netinfo).clone(), 12).expect("Instantiate dynamic_honey_badger")
}
fn test_dynamic_honey_badger_different_sizes<A, F>(new_adversary: F, num_txs: usize)
where
A: Adversary<DynamicHoneyBadger<usize, NodeUid>>,
F: Fn(usize, usize, BTreeMap<NodeUid, Rc<NetworkInfo<NodeUid>>>) -> 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);
}

View File

@ -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
);
}
}
}

View File

@ -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));