Add node transaction messages to Dynamic HB.

This commit is contained in:
Andreas Fackler 2018-06-27 13:45:25 +02:00
parent 15caa4f949
commit 7a0b397233
7 changed files with 248 additions and 147 deletions

View File

@ -456,9 +456,11 @@ impl<NodeUid: Clone + Debug + Ord> Agreement<NodeUid> {
self.output = Some(b);
// Latch the decided state.
self.decision = Some(b);
self.messages
.push_back(AgreementContent::Term(b).with_epoch(self.epoch));
self.received_term.insert(self.netinfo.our_uid().clone(), b);
if self.netinfo.is_peer() {
self.messages
.push_back(AgreementContent::Term(b).with_epoch(self.epoch));
self.received_term.insert(self.netinfo.our_uid().clone(), b);
}
self.terminated = true;
debug!(
"Agreement instance {:?} decided: {}",
@ -482,6 +484,9 @@ impl<NodeUid: Clone + Debug + Ord> Agreement<NodeUid> {
}
fn send_aux(&mut self, b: bool) -> AgreementResult<()> {
if !self.netinfo.is_peer() {
return Ok(());
}
// Multicast `Aux`.
self.messages
.push_back(AgreementContent::Aux(b).with_epoch(self.epoch));

View File

@ -29,7 +29,7 @@ const CHACHA_RNG_SEED_SIZE: usize = 8;
const ERR_OS_RNG: &str = "could not initialize the OS random number generator";
/// A public key, or a public key share.
#[derive(Deserialize, Serialize, Clone, Debug, PartialEq, Eq)]
#[derive(Deserialize, Serialize, Clone, PartialEq, Eq)]
pub struct PublicKey(#[serde(with = "serde_impl::projective")] G1);
impl Hash for PublicKey {
@ -38,6 +38,14 @@ impl Hash for PublicKey {
}
}
impl fmt::Debug for PublicKey {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
let uncomp = self.0.into_affine().into_uncompressed();
let bytes = uncomp.as_ref();
write!(f, "PublicKey({:?})", HexBytes(bytes))
}
}
impl PublicKey {
/// Returns `true` if the signature matches the element of `G2`.
pub fn verify_g2<H: Into<G2Affine>>(&self, sig: &Signature, hash: H) -> bool {
@ -82,7 +90,7 @@ impl fmt::Debug for Signature {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
let uncomp = self.0.into_affine().into_uncompressed();
let bytes = uncomp.as_ref();
write!(f, "{:?}", HexBytes(bytes))
write!(f, "Signature({:?})", HexBytes(bytes))
}
}
@ -104,9 +112,17 @@ impl Signature {
}
/// A secret key, or a secret key share.
#[derive(Debug, Clone, PartialEq, Eq)]
#[derive(Clone, PartialEq, Eq)]
pub struct SecretKey(Fr);
impl fmt::Debug for SecretKey {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
let uncomp = self.public_key().0.into_affine().into_uncompressed();
let bytes = uncomp.as_ref();
write!(f, "SecretKey({:?})", HexBytes(bytes))
}
}
impl Default for SecretKey {
fn default() -> Self {
SecretKey(Fr::zero())

View File

@ -10,7 +10,7 @@ use serde::{Deserialize, Serialize};
use crypto::{PublicKey, PublicKeySet, SecretKey, Signature};
use honey_badger::{self, HoneyBadger};
use messaging::{DistAlgorithm, NetworkInfo, TargetedMessage};
use messaging::{DistAlgorithm, NetworkInfo, Target, TargetedMessage};
use sync_key_gen::{Accept, Propose, SyncKeyGen};
error_chain!{
@ -46,8 +46,6 @@ pub enum Input<Tx, 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,
@ -67,11 +65,13 @@ where
/// 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>)>,
incoming_queue: Vec<(NodeUid, 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>>,
/// The current candidate node trying to join the peers.
candidate: Option<(NodeUid, PublicKey)>,
}
impl<Tx, NodeUid> DistAlgorithm for DynamicHoneyBadger<Tx, NodeUid>
@ -86,16 +86,29 @@ where
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()
match input {
Input::User(tx) => {
self.honey_badger.input(Transaction::User(tx))?;
self.process_output()
}
Input::Change(change) => self.send_transaction(NodeTransaction::Change(change)),
}
}
fn handle_message(&mut self, sender_id: &NodeUid, message: Self::Message) -> Result<()> {
let epoch = message.epoch();
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 = (sender_id.clone(), message);
self.incoming_queue.push(entry);
return Ok(());
}
match message {
Message::HoneyBadger(start_epoch, hb_msg) => {
self.handle_honey_badger_message(sender_id, start_epoch, hb_msg)
}
Message::HoneyBadger(_, hb_msg) => self.handle_honey_badger_message(sender_id, hb_msg),
Message::Signed(_, node_tx, sig) => self.handle_signed_message(sender_id, node_tx, sig),
}
}
@ -134,6 +147,7 @@ where
incoming_queue: Vec::new(),
messages: MessageQueue(VecDeque::new()),
output: VecDeque::new(),
candidate: None,
};
Ok(dyn_hb)
}
@ -142,19 +156,10 @@ where
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) {
info!("Unknown sender {:?} of message {:?}", sender_id, message);
return Err(ErrorKind::UnknownSender.into());
}
// Handle the message and put the outgoing messages into the queue.
@ -163,6 +168,19 @@ where
Ok(())
}
/// Handles a message for the `HoneyBadger` instance.
fn handle_signed_message(
&mut self,
sender_id: &NodeUid,
node_tx: NodeTransaction<NodeUid>,
sig: Box<Signature>,
) -> Result<()> {
self.verify_signature(sender_id, &*sig, &node_tx)?;
let tx = Transaction::Signed(self.start_epoch, sender_id.clone(), node_tx, sig);
self.honey_badger.input(tx)?;
self.process_output()
}
/// Processes all pending batches output by Honey Badger.
fn process_output(&mut self) -> Result<()> {
let mut changed = false;
@ -178,23 +196,35 @@ where
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)?,
Transaction::User(tx) => id_txs.push(tx),
Transaction::Signed(epoch, s_id, node_tx, sig) => {
if epoch < self.start_epoch {
info!("Obsolete node transaction: {:?}.", node_tx);
continue;
}
if !self.verify_signature(&s_id, &sig, &node_tx)? {
info!("Invalid signature from {:?} for: {:?}.", s_id, node_tx);
continue;
}
use self::NodeTransaction::*;
match node_tx {
Change(change) => self.handle_vote(s_id, change),
Propose(propose) => self.handle_propose(&s_id, propose)?,
Accept(accept) => self.handle_accept(&s_id, accept)?,
}
}
}
}
}
// If DKG completed, apply the change.
if let Some(ref change) = change {
if let Some((pub_key_set, sk)) = self.get_key_gen_output() {
debug!("{:?} DKG for {:?} complete!", self.our_id(), change);
let sk = sk.unwrap_or_else(|| {
ClearOnDrop::new(Box::new(self.netinfo.secret_key().clone()))
});
self.start_epoch = hb_batch.epoch + 1;
self.start_epoch += hb_batch.epoch + 1;
self.apply_change(change, pub_key_set, sk)?;
batch.change = Some(change.clone());
changed = true;
@ -203,11 +233,13 @@ where
// 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()));
self.candidate = Some((node_id.clone(), pub_key.clone()));
batch.candidate = self.candidate.clone();
}
// If a new change has a majority, restart DKG.
if new_change != change {
if let Some(change) = new_change {
debug!("{:?} Initiating DKG for {:?}.", self.our_id(), change);
self.start_key_gen(change)?;
} else {
self.key_gen = None;
@ -220,25 +252,13 @@ where
// 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)?;
for (sender_id, msg) in queue {
self.handle_message(&sender_id, 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,
@ -253,12 +273,13 @@ where
Change::Remove(ref id) => all_uids.remove(id),
Change::Add(ref id, _) => all_uids.insert(id.clone()),
} {
debug!("No-op change: {:?}", change);
info!("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)?;
let old_buffer = self.honey_badger.drain_buffer().into_iter();
let new_buffer = old_buffer.filter(Transaction::is_user);
self.honey_badger = HoneyBadger::new(Rc::new(netinfo), self.batch_size, new_buffer)?;
Ok(())
}
@ -270,7 +291,7 @@ where
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());
info!("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.
@ -280,92 +301,90 @@ where
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)?;
self.send_transaction(NodeTransaction::Propose(propose))?;
}
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(());
}
fn handle_propose(&mut self, sender_id: &NodeUid, propose: Propose) -> Result<()> {
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(())
match self.key_gen.as_mut().and_then(handle) {
Some(accept) => self.send_transaction(NodeTransaction::Accept(accept)),
None => 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);
}
fn handle_accept(&mut self, sender_id: &NodeUid, accept: Accept) -> Result<()> {
if let Some(key_gen) = self.key_gen.as_mut() {
key_gen.handle_accept(&sender_id, accept);
}
Ok(())
}
/// Signs and sends a `NodeTransaction` and also tries to commit it.
fn send_transaction(&mut self, node_tx: NodeTransaction<NodeUid>) -> Result<()> {
let sig = self.sign(&node_tx)?;
let msg = Message::Signed(self.start_epoch, node_tx.clone(), sig.clone());
self.messages.push_back(Target::All.message(msg));
if !self.netinfo.is_peer() {
return Ok(());
}
let our_uid = self.netinfo.our_uid().clone();
let hb_tx = Transaction::Signed(self.start_epoch, our_uid, node_tx, sig);
self.honey_badger.input(hb_tx)?;
self.process_output()
}
/// 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())
.filter(|key_gen| {
let candidate_ready = |&(ref id, _): &(NodeUid, _)| key_gen.is_node_ready(id);
key_gen.is_ready() && self.candidate.as_ref().map_or(true, candidate_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)?;
/// Returns a signature of `node_tx`, or an error if serialization fails.
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 the payload by the node with the specified ID is valid.
/// 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.
fn verify_signature<T: Serialize>(
fn verify_signature(
&self,
id: &NodeUid,
node_id: &NodeUid,
sig: &Signature,
payload: &T,
node_tx: &NodeTransaction<NodeUid>,
) -> Result<bool> {
let ser = bincode::serialize(payload)?;
let pk_opt = self.netinfo.public_key_share(&id);
let ser = bincode::serialize(node_tx)?;
let pk_opt = self.netinfo.public_key_share(node_id).or_else(|| {
self.candidate
.iter()
.filter(|&&(ref id, _)| id == node_id)
.map(|&(_, ref pk)| pk)
.next()
});
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)? {
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);
}
Ok(())
}
/// Returns the change that currently has a majority of votes, if any.
@ -389,16 +408,22 @@ where
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 signed internal message that gets committed via Honey Badger to communicate synchronously.
Signed(u64, NodeUid, NodeTransaction<NodeUid>, Box<Signature>),
}
impl<Tx, NodeUid> Transaction<Tx, NodeUid> {
/// Returns `true` if this is a user transaction.
fn is_user(&self) -> bool {
match *self {
Transaction::User(_) => true,
Transaction::Signed(_, _, _, _) => false,
}
}
}
/// A batch of transactions the algorithm has output.
#[derive(Clone)]
#[derive(Clone, Debug, Eq, PartialEq)]
pub struct Batch<Tx, NodeUid> {
/// The sequence number: there is exactly one batch in each epoch.
pub epoch: u64,
@ -444,12 +469,34 @@ impl<Tx, NodeUid: Ord> Batch<Tx, NodeUid> {
}
}
/// An internal message that gets committed via Honey Badger to communicate synchronously.
#[derive(Eq, PartialEq, Debug, Serialize, Deserialize, Hash, Clone)]
pub enum NodeTransaction<NodeUid> {
/// A vote to add or remove a node.
Change(Change<NodeUid>),
/// A `SyncKeyGen::Propose` message for key generation.
Propose(Propose),
/// A `SyncKeyGen::Accept` message for key generation.
Accept(Accept),
}
/// 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>),
/// A transaction to be committed, signed by a node.
Signed(u64, NodeTransaction<NodeUid>, Box<Signature>),
}
impl<NodeUid> Message<NodeUid> {
pub fn epoch(&self) -> u64 {
match *self {
Message::HoneyBadger(epoch, _) => epoch,
Message::Signed(epoch, _, _) => epoch,
}
}
}
/// The queue of outgoing messages in a `HoneyBadger` instance.

View File

@ -28,7 +28,6 @@ error_chain!{
errors {
UnknownSender
ObserverCannotPropose
}
}
@ -73,7 +72,8 @@ where
type Error = Error;
fn input(&mut self, input: Self::Input) -> HoneyBadgerResult<()> {
self.add_transactions(iter::once(input))
self.add_transactions(iter::once(input));
Ok(())
}
fn handle_message(
@ -147,16 +147,8 @@ where
}
/// Adds transactions into the buffer.
pub fn add_transactions<I: IntoIterator<Item = Tx>>(
&mut self,
txs: I,
) -> HoneyBadgerResult<()> {
if self.netinfo.is_peer() {
self.buffer.extend(txs);
Ok(())
} else {
Err(ErrorKind::ObserverCannotPropose.into())
}
pub fn add_transactions<I: IntoIterator<Item = Tx>>(&mut self, txs: I) {
self.buffer.extend(txs);
}
/// Empties and returns the transaction buffer.

View File

@ -34,7 +34,7 @@
use std::collections::btree_map::Entry;
use std::collections::{BTreeMap, BTreeSet};
use std::fmt::Debug;
use std::fmt::{self, Debug, Formatter};
use bincode;
use clear_on_drop::ClearOnDrop;
@ -49,14 +49,28 @@ use crypto::{Ciphertext, PublicKey, PublicKeySet, SecretKey};
// TODO: No need to send our own row and value to ourselves.
/// A commitment to a bivariate polynomial, and for each node, an encrypted row of values.
#[derive(Deserialize, Serialize, Debug, Clone, Hash, Eq, PartialEq)]
#[derive(Deserialize, Serialize, Clone, Hash, Eq, PartialEq)]
pub struct Propose(BivarCommitment, Vec<Ciphertext>);
impl Debug for Propose {
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)
}
}
/// A confirmation that we have received a node's proposal and verified our row against the
/// commitment. For each node, it contains one encrypted value of our row.
#[derive(Deserialize, Serialize, Debug, Clone, Hash, Eq, PartialEq)]
#[derive(Deserialize, Serialize, Clone, Hash, Eq, PartialEq)]
pub struct Accept(u64, Vec<Ciphertext>);
impl Debug for Accept {
fn fmt(&self, f: &mut Formatter) -> fmt::Result {
write!(f, "Accept({}, <{} values>", self.0, self.1.len())
}
}
/// The information needed to track a single proposer's secret sharing process.
struct ProposalState {
/// The proposer's commitment.
@ -188,9 +202,9 @@ impl<NodeUid: Ord + Debug> SyncKeyGen<NodeUid> {
}
/// Returns `true` if the proposal of the given node is complete.
pub fn is_node_ready(&self, proposer_idx: u64) -> bool {
self.proposals
.get(&proposer_idx)
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))
}

View File

@ -11,13 +11,14 @@ extern crate serde_derive;
mod network;
use std::cmp;
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::dynamic_honey_badger::{Change, DynamicHoneyBadger, Input};
use hbbft::messaging::NetworkInfo;
use network::{Adversary, MessageScheduler, NodeUid, SilentAdversary, TestNetwork, TestNode};
@ -35,15 +36,27 @@ fn test_dynamic_honey_badger<A>(
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()
fn has_remove(node: &TestNode<DynamicHoneyBadger<usize, NodeUid>>) -> bool {
node.outputs()
.iter()
.filter_map(|batch| batch.change())
.any(|change| *change == Change::Remove(NodeUid(0)))
{
}
fn has_add(node: &TestNode<DynamicHoneyBadger<usize, NodeUid>>) -> bool {
node.outputs()
.iter()
.filter_map(|batch| batch.change())
.any(|change| match *change {
Change::Add(ref id, _) => *id == NodeUid(0),
_ => false,
})
}
// 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 !has_remove(node) || !has_add(node) {
return true;
}
let mut min_missing = 0;
@ -59,29 +72,41 @@ fn test_dynamic_honey_badger<A>(
}
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,
});
node.queue.retain(|(_, ref msg)| msg.epoch() < last);
}
false
};
let mut input_add = 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);
network.step();
if !input_add && network.nodes.values().all(has_remove) {
for tx in (num_txs / 2)..num_txs {
network.input_all(Input::User(tx));
}
let pk = network.pk_set.public_key_share(0);
network.input_all(Input::Change(Change::Add(NodeUid(0), pk)));
info!("Input!");
input_add = true;
}
}
// TODO: Verify that all nodes output the same epochs.
verify_output_sequence(&network);
}
/// Verifies that all instances output the same sequence of batches. We already know that all of
/// them have output all transactions and events, but some may have advanced a few empty batches
/// more than others, so we ignore those.
fn verify_output_sequence<A>(network: &TestNetwork<A, DynamicHoneyBadger<usize, NodeUid>>)
where
A: Adversary<DynamicHoneyBadger<usize, NodeUid>>,
{
let expected = network.nodes[&NodeUid(0)].outputs().to_vec();
assert!(!expected.is_empty());
for node in network.nodes.values() {
let len = cmp::min(expected.len(), node.outputs().len());
assert_eq!(&expected[..len], &node.outputs()[..len]);
}
}
// Allow passing `netinfo` by value. `TestNetwork` expects this function signature.

View File

@ -5,7 +5,7 @@ use std::rc::Rc;
use rand::{self, Rng};
use hbbft::crypto::SecretKeySet;
use hbbft::crypto::{PublicKeySet, SecretKeySet};
use hbbft::messaging::{DistAlgorithm, NetworkInfo, Target, TargetedMessage};
/// A node identifier. In the tests, nodes are simply numbered.
@ -150,6 +150,7 @@ where
pub nodes: BTreeMap<D::NodeUid, TestNode<D>>,
pub observer: TestNode<D>,
pub adv_nodes: BTreeMap<D::NodeUid, Rc<NetworkInfo<D::NodeUid>>>,
pub pk_set: PublicKeySet,
adversary: A,
}
@ -205,6 +206,7 @@ where
nodes: (0..good_num).map(NodeUid).map(new_node_by_id).collect(),
observer: new_node_by_id(NodeUid(good_num + adv_num)).1,
adversary: adversary(adv_nodes.clone()),
pk_set: pk_set.clone(),
adv_nodes,
};
let msgs = network.adversary.step();