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

Add node transaction messages to Dynamic HB.
This commit is contained in:
Andreas Fackler 2018-07-02 15:14:38 +02:00 committed by GitHub
commit 38178af124
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
12 changed files with 517 additions and 255 deletions

View File

@ -21,9 +21,13 @@ Honey Badger is a modular library composed of several independent algorithms. T
In an optimal networking environment, output includes data sent from each node. In an adverse environment, the output is an agreed upon subset of data. Either way, the resulting output contains a batch of transactions which is guaranteed to be consistent across all nodes.
In addition to **validators**, the algorithms support **observers**: These don't actively participate, and don't need to be trusted, but they receive the output as well, and are able to verify it under the assumption that more than two thirds of the validators are correct.
## Algorithms
- [ ] **[Honey Badger](https://github.com/poanetwork/hbbft/blob/master/src/honey_badger.rs):** The top level protocol proceeds in epochs using the protocols below.
- [x] **[Honey Badger](https://github.com/poanetwork/hbbft/blob/master/src/honey_badger.rs):** Each node inputs transactions. The protocol outputs a sequence of batches of transactions.
- [ ] **[Dynamic Honey Badger](https://github.com/poanetwork/hbbft/blob/master/src/dynamic_honey_badger.rs):** A modified Honey Badger where nodes can dynamically add and remove other nodes to/from the network.
- [x] **[Subset](https://github.com/poanetwork/hbbft/blob/master/src/common_subset.rs):** Each node inputs data. The nodes agree on a subset of suggested data.

View File

@ -307,7 +307,7 @@ impl<NodeUid: Clone + Debug + Ord> Agreement<NodeUid> {
}
fn send_bval(&mut self, b: bool) -> AgreementResult<()> {
if !self.netinfo.is_peer() {
if !self.netinfo.is_validator() {
return Ok(());
}
// Record the value `b` as sent.
@ -329,7 +329,7 @@ impl<NodeUid: Clone + Debug + Ord> Agreement<NodeUid> {
// Trigger the start of the `Conf` round.
self.conf_round = true;
if !self.netinfo.is_peer() {
if !self.netinfo.is_validator() {
return Ok(());
}
@ -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_validator() {
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_validator() {
return Ok(());
}
// Multicast `Aux`.
self.messages
.push_back(AgreementContent::Aux(b).with_epoch(self.epoch));

View File

@ -355,7 +355,7 @@ impl<NodeUid: Debug + Clone + Ord> Broadcast<NodeUid> {
/// Sends an `Echo` message and handles it. Does nothing if we are only an observer.
fn send_echo(&mut self, p: Proof<Vec<u8>>) -> BroadcastResult<()> {
self.echo_sent = true;
if !self.netinfo.is_peer() {
if !self.netinfo.is_validator() {
return Ok(());
}
let echo_msg = Target::All.message(BroadcastMessage::Echo(p.clone()));
@ -367,7 +367,7 @@ impl<NodeUid: Debug + Clone + Ord> Broadcast<NodeUid> {
/// Sends a `Ready` message and handles it. Does nothing if we are only an observer.
fn send_ready(&mut self, hash: &[u8]) -> BroadcastResult<()> {
self.ready_sent = true;
if !self.netinfo.is_peer() {
if !self.netinfo.is_validator() {
return Ok(());
}
let ready_msg = Target::All.message(BroadcastMessage::Ready(hash.to_vec()));

View File

@ -126,7 +126,7 @@ where
}
fn get_coin(&mut self) -> Result<()> {
if !self.netinfo.is_peer() {
if !self.netinfo.is_validator() {
return self.try_output();
}
let share = self.netinfo.secret_key().sign(&self.nonce);

View File

@ -180,7 +180,7 @@ impl<NodeUid: Clone + Debug + Ord> CommonSubset<NodeUid> {
/// Common Subset input message handler. It receives a value for broadcast
/// and redirects it to the corresponding broadcast instance.
pub fn send_proposed_value(&mut self, value: ProposedValue) -> CommonSubsetResult<()> {
if !self.netinfo.is_peer() {
if !self.netinfo.is_validator() {
return Ok(());
}
let uid = self.netinfo.our_uid().clone();

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

@ -1,6 +1,54 @@
//! # Dynamic Honey Badger
//!
//! Like Honey Badger, this protocol allows a network of `N` nodes with at most `f` faulty ones,
//! where `3 * f < N`, to input "transactions" - any kind of data -, and to agree on a sequence of
//! _batches_ of transactions. The protocol proceeds in _epochs_, starting at number 0, and outputs
//! one batch in each epoch. It never terminates: It handles a continuous stream of incoming
//! transactions and keeps producing new batches from them. All correct nodes will output the same
//! batch for each epoch.
//!
//! Unlike Honey Badger, this algorithm allows dynamically adding new validators from the pool of
//! observer nodes, and turning validators back into observers. As a signal to initiate that
//! process, it defines a special `Change` input variant, which contains either a vote
//! `Add(node_id, public_key)`, to add a new validator, or `Remove(node_id)` to remove it. Each
//! validator can have at most one active vote, and casting another vote revokes the previous one.
//! Once a simple majority of validators has the same active vote, a reconfiguration process begins
//! (they need to create new cryptographic key shares for the new composition).
//!
//! The state of that process after each epoch is communicated via the `Batch::change` field. When
//! this contains an `InProgress(Add(..))` value, all nodes need to send every future `Target::All`
//! message to the new node, too. Once the value is `Complete`, the votes will be reset, and the
//! next epoch will run using the new set of validators.
//!
//! New observers can also be added by starting them with an appropriate `start_epoch` parameter
//! and ensuring that they receive all `Target::All` messages from that epoch on. Together with the
//! above mechanism, this allows the network to change dynamically. You can introduce a new node to
//! the network and make it a validator, you can demote validators to observers, and you can remove
//! observers at any time.
//!
//! ## How it works
//!
//! Dynamic Honey Badger runs a regular Honey Badger instance internally, which in addition to the
//! user's transactions contains special transactions for the change votes and the key generation
//! messages: Running votes and key generation "on-chain", as transactions, greatly simplifies
//! these processes, since it is guaranteed that every node will see the same sequence of votes and
//! messages.
//!
//! Every time Honey Badger outputs a new batch, Dynamic Honey Badger outputs the user transactions
//! in its own batch. The other transactions are processed: votes are counted and key generation
//! messages are passed into a `SyncKeyGen` instance.
//!
//! If after an epoch key generation has completed, the Honey Badger instance (including all
//! pending batches) is dropped, and replaced by a new one with the new set of participants.
//!
//! Otherwise we check if the majority of votes has changed. If a new change has a majority, the
//! `SyncKeyGen` instance is dropped, and a new one is started to create keys according to the new
//! pending change.
use std::collections::{BTreeMap, HashMap, VecDeque};
use std::fmt::Debug;
use std::hash::Hash;
use std::marker::PhantomData;
use std::mem;
use std::rc::Rc;
@ -9,10 +57,12 @@ 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 honey_badger::{self, Batch as HbBatch, HoneyBadger, Message as HbMessage};
use messaging::{DistAlgorithm, NetworkInfo, Target, TargetedMessage};
use sync_key_gen::{Accept, Propose, SyncKeyGen};
type KeyGenOutput = (PublicKeySet, Option<ClearOnDrop<Box<SecretKey>>>);
error_chain!{
links {
HoneyBadger(honey_badger::Error, honey_badger::ErrorKind);
@ -36,6 +86,29 @@ pub enum Change<NodeUid> {
Remove(NodeUid),
}
impl<NodeUid> Change<NodeUid> {
/// Returns the ID of the current candidate for being added, if any.
fn candidate(&self) -> Option<&NodeUid> {
match *self {
Change::Add(ref id, _) => Some(id),
Change::Remove(_) => None,
}
}
}
/// A change status: whether a node addition or removal is currently in progress or completed.
#[derive(Clone, Eq, PartialEq, Serialize, Deserialize, Hash, Debug)]
pub enum ChangeState<NodeUid> {
/// No node is currently being considered for addition or removal.
None,
/// A change is currently in progress. If it is an addition, all broadcast messages must be
/// sent to the new node, too.
InProgress(Change<NodeUid>),
/// A change has been completed in this epoch. From the next epoch on, the new composition of
/// the network will perform the consensus process.
Complete(Change<NodeUid>),
}
/// The user input for `DynamicHoneyBadger`.
#[derive(Clone, Debug)]
pub enum Input<Tx, NodeUid> {
@ -45,9 +118,70 @@ pub enum Input<Tx, NodeUid> {
Change(Change<NodeUid>),
}
/// A Dynamic Honey Badger builder, to configure the parameters and create new instances of
/// `DynamicHoneyBadger`.
pub struct DynamicHoneyBadgerBuilder<Tx, NodeUid> {
/// Shared network data.
netinfo: NetworkInfo<NodeUid>,
/// The target number of transactions to be included in each batch.
batch_size: usize,
/// The epoch at which to join the network.
start_epoch: u64,
_phantom: PhantomData<Tx>,
}
impl<Tx, NodeUid> DynamicHoneyBadgerBuilder<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 `DynamicHoneyBadgerBuilder` configured to use the node IDs and cryptographic
/// keys specified by `netinfo`.
pub fn new(netinfo: NetworkInfo<NodeUid>) -> Self {
// TODO: Use the defaults from `HoneyBadgerBuilder`.
DynamicHoneyBadgerBuilder {
netinfo,
batch_size: 100,
start_epoch: 0,
_phantom: PhantomData,
}
}
/// Sets the target number of transactions per batch.
pub fn batch_size(&mut self, batch_size: usize) -> &mut Self {
self.batch_size = batch_size;
self
}
/// Sets the epoch at which to join the network as an observer. This requires the node to
/// receive all broadcast messages for `start_epoch` and later.
pub fn start_epoch(&mut self, start_epoch: u64) -> &mut Self {
self.start_epoch = start_epoch;
self
}
/// Creates a new Dynamic Honey Badger instance with an empty buffer.
pub fn build(&self) -> Result<DynamicHoneyBadger<Tx, NodeUid>>
where
Tx: Serialize + for<'r> Deserialize<'r> + Debug + Hash + Eq,
{
let honey_badger = HoneyBadger::new(Rc::new(self.netinfo.clone()), self.batch_size, None)?;
let dyn_hb = DynamicHoneyBadger {
netinfo: self.netinfo.clone(),
batch_size: self.batch_size,
start_epoch: self.start_epoch,
votes: BTreeMap::new(),
honey_badger,
key_gen: None,
incoming_queue: Vec::new(),
messages: MessageQueue(VecDeque::new()),
output: VecDeque::new(),
};
Ok(dyn_hb)
}
}
/// 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,
@ -60,14 +194,15 @@ where
/// 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.
/// vote revokes the previous one. Resets whenever the set of validators 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>>,
/// The current key generation process, and the change it applies to.
key_gen: Option<(SyncKeyGen<NodeUid>, Change<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.
@ -86,16 +221,31 @@ 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()
// User transactions are forwarded to `HoneyBadger` right away. Internal messages are
// in addition signed and broadcast.
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),
}
}
@ -121,96 +271,89 @@ 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)
/// Returns a new `DynamicHoneyBadgerBuilder` configured to use the node IDs and cryptographic
/// keys specified by `netinfo`.
pub fn builder(netinfo: NetworkInfo<NodeUid>) -> DynamicHoneyBadgerBuilder<Tx, NodeUid> {
DynamicHoneyBadgerBuilder::new(netinfo)
}
/// Handles a message for the `HoneyBadger` instance.
fn handle_honey_badger_message(
&mut self,
sender_id: &NodeUid,
epoch: u64,
message: honey_badger::Message<NodeUid>,
message: HbMessage<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.
self.honey_badger.handle_message(sender_id, message)?;
self.process_output()?;
Ok(())
self.process_output()
}
/// 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.
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;
let start_epoch = self.start_epoch;
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.
// `hb_batch`, and the current change state.
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)?,
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() {
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;
if let Some(((pub_key_set, sk), change)) = self.take_key_gen_output() {
// If DKG completed, apply the change.
debug!("{:?} DKG for {:?} complete!", self.our_id(), change);
// If we are a validator, we received a new secret key. Otherwise keep the old one.
let sk = sk.unwrap_or_else(|| {
ClearOnDrop::new(Box::new(self.netinfo.secret_key().clone()))
});
// 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)?;
batch.change = ChangeState::Complete(change);
} else {
// If the majority changed, restart DKG. Inform the user about the current change.
self.update_key_gen(batch.epoch + 1)?;
if let Some((_, ref change)) = self.key_gen {
batch.change = ChangeState::InProgress(change.clone());
}
}
self.output.push_back(batch);
@ -218,33 +361,22 @@ where
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 {
if start_epoch < self.start_epoch {
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,
change: &Change<NodeUid>,
pub_key_set: PublicKeySet,
sk: ClearOnDrop<Box<SecretKey>>,
epoch: u64,
) -> Result<()> {
self.votes.clear();
self.key_gen = None;
@ -253,24 +385,39 @@ 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)?;
Ok(())
self.netinfo = netinfo;
self.restart_honey_badger(epoch)
}
/// Starts Key Generation for the set of nodes implied by the `change`.
fn start_key_gen(&mut self, change: Change<NodeUid>) -> Result<()> {
/// If the majority of votes has changed, restarts Key Generation for the set of nodes implied
/// by the current change.
fn update_key_gen(&mut self, epoch: u64) -> Result<()> {
let change = match current_majority(&self.votes, &self.netinfo) {
None => {
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);
// Use the existing key shares - with the change applied - as keys for DKG.
let mut pub_keys = self.netinfo.public_key_map().clone();
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(),
Change::Remove(ref id) => pub_keys.remove(id).is_none(),
Change::Add(ref id, ref pk) => pub_keys.insert(id.clone(), pk.clone()).is_some(),
} {
debug!("No-op change: {:?}", self.current_majority().unwrap());
info!("{:?} No-op change: {:?}", self.our_id(), change);
}
if change.candidate().is_some() {
self.restart_honey_badger(epoch)?;
}
// 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.
@ -278,138 +425,167 @@ where
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);
self.key_gen = Some((key_gen, change));
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(())
}
/// Starts a new `HoneyBadger` instance and inputs the user transactions from the existing
/// one's buffer and pending outputs.
fn restart_honey_badger(&mut self, epoch: u64) -> Result<()> {
// TODO: Filter out the messages for `epoch` and later.
self.messages
.extend_with_epoch(self.start_epoch, &mut self.honey_badger);
self.start_epoch = epoch;
let honey_badger = {
let netinfo = Rc::new(self.netinfo.clone());
let old_buf = self.honey_badger.drain_buffer();
let outputs = (self.honey_badger.output_iter()).flat_map(HbBatch::into_tx_iter);
let buffer = outputs.chain(old_buf).filter(Transaction::is_user);
HoneyBadger::new(netinfo, self.batch_size, buffer)?
};
self.honey_badger = honey_badger;
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(()),
fn handle_propose(&mut self, sender_id: &NodeUid, propose: Propose) -> Result<()> {
let handle = |&mut (ref mut key_gen, _): &mut (SyncKeyGen<NodeUid>, _)| {
key_gen.handle_propose(&sender_id, propose)
};
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(&mut (ref mut 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()
/// 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_validator() {
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()
}
/// 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)?;
/// If the current Key Generation process is ready, returns the generated key set.
///
/// We require the minimum number of completed proposals (`SyncKeyGen::is_ready`) and if a new
/// node is joining, we require in addition that the new node's proposal is complete. That way
/// the new node knows that it's key is secret, without having to trust any number of nodes.
fn take_key_gen_output(&mut self) -> Option<(KeyGenOutput, Change<NodeUid>)> {
let is_ready = |&(ref key_gen, ref change): &(SyncKeyGen<_>, Change<_>)| {
let candidate_ready = |id: &NodeUid| key_gen.is_node_ready(id);
key_gen.is_ready() && change.candidate().map_or(true, candidate_ready)
};
if self.key_gen.as_ref().map_or(false, is_ready) {
let generate = |(key_gen, change): (SyncKeyGen<_>, _)| (key_gen.generate(), change);
self.key_gen.take().map(generate)
} else {
None
}
}
/// 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.key_gen
.iter()
.filter_map(|&(_, ref change): &(_, Change<_>)| match *change {
Change::Add(ref id, ref pk) if id == node_id => Some(pk),
Change::Add(_, _) | Change::Remove(_) => None,
})
.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.
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
}
}
/// 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 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.
/// "regular" transactions as well as internal signed messages.
#[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 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,
/// 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)>,
/// The current state of adding or removing a node: whether any is in progress, or completed
/// this epoch.
pub change: ChangeState<NodeUid>,
}
impl<Tx, NodeUid: Ord> Batch<Tx, NodeUid> {
@ -418,8 +594,7 @@ impl<Tx, NodeUid: Ord> Batch<Tx, NodeUid> {
Batch {
epoch,
transactions: BTreeMap::new(),
change: None,
candidate: None,
change: ChangeState::None,
}
}
@ -438,18 +613,43 @@ impl<Tx, NodeUid: Ord> Batch<Tx, NodeUid> {
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()
/// Returns whether any change to the set of participating nodes is in progress or was
/// completed in this epoch.
pub fn change(&self) -> &ChangeState<NodeUid> {
&self.change
}
}
/// 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
/// a batch, so that all nodes see these messages in the same order.
#[derive(Eq, PartialEq, Debug, Serialize, Deserialize, Hash, Clone)]
pub enum NodeTransaction<NodeUid> {
/// A vote to add or remove a validator.
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>),
HoneyBadger(u64, HbMessage<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.
@ -465,7 +665,7 @@ where
where
Tx: Eq + Serialize + for<'r> Deserialize<'r> + Debug + Hash,
{
let convert = |msg: TargetedMessage<honey_badger::Message<NodeUid>, NodeUid>| {
let convert = |msg: TargetedMessage<HbMessage<NodeUid>, NodeUid>| {
msg.map(|hb_msg| Message::HoneyBadger(epoch, hb_msg))
};
self.extend(hb.message_iter().map(convert));

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.
@ -166,7 +158,7 @@ where
/// Proposes a new batch in the current epoch.
fn propose(&mut self) -> HoneyBadgerResult<()> {
if !self.netinfo.is_peer() {
if !self.netinfo.is_validator() {
return Ok(());
}
let proposal = self.choose_transactions()?;
@ -449,7 +441,7 @@ where
proposer_id: &NodeUid,
ciphertext: &Ciphertext,
) -> HoneyBadgerResult<bool> {
if !self.netinfo.is_peer() {
if !self.netinfo.is_validator() {
return Ok(ciphertext.verify());
}
let share = match self.netinfo.secret_key().decrypt_share(&ciphertext) {
@ -553,11 +545,16 @@ pub struct Batch<Tx, NodeUid> {
}
impl<Tx, NodeUid: Ord> Batch<Tx, NodeUid> {
/// Returns an iterator over all transactions included in the batch.
/// Returns an iterator over references to all transactions included in the batch.
pub fn iter(&self) -> impl Iterator<Item = &Tx> {
self.transactions.values().flat_map(|vec| vec)
}
/// Returns an iterator over all transactions included in the batch. Consumes the batch.
pub fn into_tx_iter(self) -> impl Iterator<Item = Tx> {
self.transactions.into_iter().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()

View File

@ -142,7 +142,7 @@ pub struct NetworkInfo<NodeUid> {
all_uids: BTreeSet<NodeUid>,
num_nodes: usize,
num_faulty: usize,
is_peer: bool,
is_validator: bool,
secret_key: ClearOnDrop<Box<SecretKey>>,
public_key_set: PublicKeySet,
public_keys: BTreeMap<NodeUid, PublicKey>,
@ -157,7 +157,7 @@ impl<NodeUid: Clone + Ord> NetworkInfo<NodeUid> {
public_key_set: PublicKeySet,
) -> Self {
let num_nodes = all_uids.len();
let is_peer = all_uids.contains(&our_uid);
let is_validator = all_uids.contains(&our_uid);
let node_indices: BTreeMap<NodeUid, usize> = all_uids
.iter()
.enumerate()
@ -172,7 +172,7 @@ impl<NodeUid: Clone + Ord> NetworkInfo<NodeUid> {
all_uids,
num_nodes,
num_faulty: (num_nodes - 1) / 3,
is_peer,
is_validator,
secret_key,
public_key_set,
public_keys,
@ -236,7 +236,7 @@ impl<NodeUid: Clone + Ord> NetworkInfo<NodeUid> {
/// Returns `true` if this node takes part in the consensus itself. If not, it is only an
/// observer.
pub fn is_peer(&self) -> bool {
self.is_peer
pub fn is_validator(&self) -> bool {
self.is_validator
}
}

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, ChangeState, DynamicHoneyBadger, Input};
use hbbft::messaging::NetworkInfo;
use network::{Adversary, MessageScheduler, NodeUid, SilentAdversary, TestNetwork, TestNode};
@ -35,15 +36,23 @@ fn test_dynamic_honey_badger<A>(
network.input_all(Input::User(tx));
}
fn has_remove(node: &TestNode<DynamicHoneyBadger<usize, NodeUid>>) -> bool {
node.outputs()
.iter()
.any(|batch| batch.change == ChangeState::Complete(Change::Remove(NodeUid(0))))
}
fn has_add(node: &TestNode<DynamicHoneyBadger<usize, NodeUid>>) -> bool {
node.outputs().iter().any(|batch| match batch.change {
ChangeState::Complete(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 !node
.outputs()
.iter()
.filter_map(|batch| batch.change())
.any(|change| *change == Change::Remove(NodeUid(0)))
{
if !has_remove(node) || !has_add(node) {
return true;
}
let mut min_missing = 0;
@ -59,35 +68,50 @@ 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.
#[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")
DynamicHoneyBadger::builder((*netinfo).clone())
.batch_size(12)
.build()
.expect("Instantiate dynamic_honey_badger")
}
fn test_dynamic_honey_badger_different_sizes<A, F>(new_adversary: F, num_txs: usize)

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