Merge pull request #288 from logannc/optional_threshold_encryption_209

Optional threshold encryption 209
This commit is contained in:
Vladimir Komendantskiy 2018-11-01 21:45:02 +00:00 committed by GitHub
commit ad35ebafd5
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
15 changed files with 246 additions and 80 deletions

View File

@ -1,6 +1,7 @@
use std::collections::BTreeMap;
use std::sync::Arc;
use super::EncryptionSchedule;
use super::{ChangeState, JoinPlan};
use {NetworkInfo, NodeIdT};
@ -16,6 +17,8 @@ pub struct Batch<C, N> {
pub(super) change: ChangeState<N>,
/// The network info that applies to the _next_ epoch.
pub(super) netinfo: Arc<NetworkInfo<N>>,
/// The current encryption schedule for threshold cryptography.
pub(super) encryption_schedule: EncryptionSchedule,
}
impl<C, N: NodeIdT> Batch<C, N> {
@ -90,6 +93,7 @@ impl<C, N: NodeIdT> Batch<C, N> {
change: self.change.clone(),
pub_key_set: self.netinfo.public_key_set().clone(),
pub_keys: self.netinfo.public_key_map().clone(),
encryption_schedule: self.encryption_schedule,
})
}
@ -104,5 +108,6 @@ impl<C, N: NodeIdT> Batch<C, N> {
&& self.change == other.change
&& self.netinfo.public_key_set() == other.netinfo.public_key_set()
&& self.netinfo.public_key_map() == other.netinfo.public_key_map()
&& self.encryption_schedule == other.encryption_schedule
}
}

View File

@ -7,8 +7,9 @@ use crypto::{SecretKey, SecretKeySet, SecretKeyShare};
use rand::{self, Rand, Rng};
use serde::{de::DeserializeOwned, Serialize};
use super::{ChangeState, DynamicHoneyBadger, JoinPlan, Result, Step, VoteCounter};
use super::{Change, ChangeState, DynamicHoneyBadger, JoinPlan, Result, Step, VoteCounter};
use honey_badger::{HoneyBadger, SubsetHandlingStrategy};
use threshold_decryption::EncryptionSchedule;
use util::SubRng;
use {Contribution, NetworkInfo, NodeIdT};
@ -24,6 +25,8 @@ pub struct DynamicHoneyBadgerBuilder<C, N> {
rng: Box<dyn rand::Rng>,
/// Strategy used to handle the output of the `Subset` algorithm.
subset_handling_strategy: SubsetHandlingStrategy,
/// Schedule for adding threshold encryption to some percentage of rounds
encryption_schedule: EncryptionSchedule,
_phantom: PhantomData<(C, N)>,
}
@ -35,6 +38,7 @@ impl<C, N> Default for DynamicHoneyBadgerBuilder<C, N> {
max_future_epochs: 3,
rng: Box::new(rand::thread_rng()),
subset_handling_strategy: SubsetHandlingStrategy::Incremental,
encryption_schedule: EncryptionSchedule::Always,
_phantom: PhantomData,
}
}
@ -78,6 +82,12 @@ where
self
}
/// Sets the schedule to use for threshold encryption.
pub fn encryption_schedule(&mut self, encryption_schedule: EncryptionSchedule) -> &mut Self {
self.encryption_schedule = encryption_schedule;
self
}
/// Creates a new Dynamic Honey Badger instance with an empty buffer.
pub fn build(&mut self, netinfo: NetworkInfo<N>) -> DynamicHoneyBadger<C, N> {
let DynamicHoneyBadgerBuilder {
@ -85,6 +95,7 @@ where
max_future_epochs,
rng,
subset_handling_strategy,
encryption_schedule,
_phantom,
} = self;
let epoch = *epoch;
@ -95,6 +106,7 @@ where
.max_future_epochs(max_future_epochs)
.rng(rng.sub_rng())
.subset_handling_strategy(subset_handling_strategy.clone())
.encryption_schedule(*encryption_schedule)
.build();
DynamicHoneyBadger {
netinfo,
@ -138,6 +150,7 @@ where
let arc_netinfo = Arc::new(netinfo.clone());
let honey_badger = HoneyBadger::builder(arc_netinfo.clone())
.max_future_epochs(self.max_future_epochs)
.encryption_schedule(join_plan.encryption_schedule)
.build();
let mut dhb = DynamicHoneyBadger {
netinfo,
@ -151,7 +164,10 @@ where
rng: Box::new(self.rng.sub_rng()),
};
let step = match join_plan.change {
ChangeState::InProgress(ref change) => dhb.update_key_gen(join_plan.epoch, change)?,
ChangeState::InProgress(ref change) => match change {
Change::NodeChange(change) => dhb.update_key_gen(join_plan.epoch, change)?,
_ => Step::default(),
},
ChangeState::None | ChangeState::Complete(..) => Step::default(),
};
Ok((dhb, step))

View File

@ -1,31 +1,51 @@
use crypto::PublicKey;
use serde_derive::{Deserialize, Serialize};
use threshold_decryption::EncryptionSchedule;
/// A node change action: adding or removing a node.
#[derive(Clone, Eq, PartialEq, Serialize, Deserialize, Hash, Debug)]
pub enum Change<N> {
#[derive(Clone, Copy, Eq, PartialEq, Serialize, Deserialize, Hash, Debug)]
pub enum NodeChange<N> {
/// Add a node. The public key is used only temporarily, for key generation.
Add(N, PublicKey),
/// Remove a node.
Remove(N),
}
impl<N> Change<N> {
impl<N> NodeChange<N> {
/// Returns the ID of the current candidate for being added, if any.
pub fn candidate(&self) -> Option<&N> {
match *self {
Change::Add(ref id, _) => Some(id),
Change::Remove(_) => None,
NodeChange::Add(ref id, _) => Some(id),
NodeChange::Remove(_) => None,
}
}
}
/// A change status: whether a node addition or removal is currently in progress or completed.
/// A node change action: adding or removing a node.
#[derive(Clone, Eq, PartialEq, Serialize, Deserialize, Hash, Debug)]
pub enum Change<N> {
// Add or Remove a node from the set of validators
NodeChange(NodeChange<N>),
/// Change the threshold encryption schedule.
/// Increase frequency to prevent censorship or decrease frequency for increased throughput.
EncryptionSchedule(EncryptionSchedule),
}
impl<N> Change<N> {
/// Returns the ID of the current candidate for being added, if any.
pub fn candidate(&self) -> Option<&N> {
match self {
Change::NodeChange(node_change) => node_change.candidate(),
_ => None,
}
}
}
/// A change status: whether a change to the network is currently in progress or completed.
#[derive(Clone, Eq, PartialEq, Serialize, Deserialize, Hash, Debug)]
pub enum ChangeState<N> {
/// No node is currently being considered for addition or removal.
/// No change is currently being considered.
None,
/// A change is currently in progress. If it is an addition, all broadcast messages must be
/// A change is currently in progress. If it is a node addition, all broadcast messages must be
/// sent to the new node, too.
InProgress(Change<N>),
/// A change has been completed in this epoch. From the next epoch on, the new composition of

View File

@ -12,11 +12,14 @@ use serde::{de::DeserializeOwned, Serialize};
use super::votes::{SignedVote, VoteCounter};
use super::{
Batch, Change, ChangeState, DynamicHoneyBadgerBuilder, Error, ErrorKind, Input,
InternalContrib, KeyGenMessage, KeyGenState, Message, Result, SignedKeyGenMsg, Step,
InternalContrib, KeyGenMessage, KeyGenState, Message, NodeChange, Result, SignedKeyGenMsg,
Step,
};
use fault_log::{Fault, FaultKind, FaultLog};
use honey_badger::{self, HoneyBadger, Message as HbMessage};
use sync_key_gen::{Ack, AckOutcome, Part, PartOutcome, SyncKeyGen};
use threshold_decryption::EncryptionSchedule;
use util::{self, SubRng};
use {Contribution, DistAlgorithm, NetworkInfo, NodeIdT, Target};
@ -290,12 +293,20 @@ where
// If DKG completed, apply the change, restart Honey Badger, and inform the user.
debug!("{:?} DKG for {:?} complete!", self.our_id(), kgs.change);
self.netinfo = kgs.key_gen.into_network_info()?;
self.restart_honey_badger(batch_epoch + 1);
ChangeState::Complete(kgs.change)
self.restart_honey_badger(batch_epoch + 1, None);
ChangeState::Complete(Change::NodeChange(kgs.change))
} else if let Some(change) = self.vote_counter.compute_winner().cloned() {
// If there is a new change, restart DKG. Inform the user about the current change.
step.extend(self.update_key_gen(batch_epoch + 1, &change)?);
ChangeState::InProgress(change)
step.extend(match &change {
Change::NodeChange(change) => self.update_key_gen(batch_epoch + 1, &change)?,
Change::EncryptionSchedule(schedule) => {
self.update_encryption_schedule(batch_epoch + 1, *schedule)?
}
});
match change {
Change::NodeChange(_) => ChangeState::InProgress(change),
Change::EncryptionSchedule(_) => ChangeState::Complete(change),
}
} else {
ChangeState::None
};
@ -304,6 +315,7 @@ where
change,
netinfo: Arc::new(self.netinfo.clone()),
contributions: batch_contributions,
encryption_schedule: self.honey_badger.get_encryption_schedule(),
});
}
// If `start_epoch` changed, we can now handle some queued messages.
@ -316,9 +328,22 @@ where
Ok(step)
}
pub(super) fn update_encryption_schedule(
&mut self,
epoch: u64,
encryption_schedule: EncryptionSchedule,
) -> Result<Step<C, N>> {
self.restart_honey_badger(epoch, Some(encryption_schedule));
Ok(Step::default())
}
/// If the winner of the vote has changed, restarts Key Generation for the set of nodes implied
/// by the current change.
pub(super) fn update_key_gen(&mut self, epoch: u64, change: &Change<N>) -> Result<Step<C, N>> {
pub(super) fn update_key_gen(
&mut self,
epoch: u64,
change: &NodeChange<N>,
) -> Result<Step<C, N>> {
if self.key_gen_state.as_ref().map(|kgs| &kgs.change) == Some(change) {
return Ok(Step::default()); // The change is the same as before. Continue DKG as is.
}
@ -326,12 +351,12 @@ where
// 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(ref id) => pub_keys.remove(id).is_none(),
Change::Add(ref id, ref pk) => pub_keys.insert(id.clone(), pk.clone()).is_some(),
NodeChange::Remove(ref id) => pub_keys.remove(id).is_none(),
NodeChange::Add(ref id, ref pk) => pub_keys.insert(id.clone(), pk.clone()).is_some(),
} {
info!("{:?} No-op change: {:?}", self.our_id(), change);
}
self.restart_honey_badger(epoch);
self.restart_honey_badger(epoch, None);
// 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;
@ -347,7 +372,11 @@ where
}
/// Starts a new `HoneyBadger` instance and resets the vote counter.
fn restart_honey_badger(&mut self, epoch: u64) {
fn restart_honey_badger(
&mut self,
epoch: u64,
encryption_schedule: Option<EncryptionSchedule>,
) {
self.start_epoch = epoch;
self.key_gen_msg_buffer.retain(|kg_msg| kg_msg.0 >= epoch);
let netinfo = Arc::new(self.netinfo.clone());
@ -356,7 +385,9 @@ where
.session_id(epoch)
.max_future_epochs(self.max_future_epochs)
.rng(self.rng.sub_rng())
.build();
.encryption_schedule(
encryption_schedule.unwrap_or_else(|| self.honey_badger.get_encryption_schedule()),
).build();
}
/// Handles a `Part` message that was output by Honey Badger.

View File

@ -68,13 +68,14 @@ use serde_derive::{Deserialize, Serialize};
use std::collections::BTreeMap;
use self::votes::{SignedVote, VoteCounter};
use super::threshold_decryption::EncryptionSchedule;
use honey_badger::Message as HbMessage;
use sync_key_gen::{Ack, Part, SyncKeyGen};
use NodeIdT;
pub use self::batch::Batch;
pub use self::builder::DynamicHoneyBadgerBuilder;
pub use self::change::{Change, ChangeState};
pub use self::change::{Change, ChangeState, NodeChange};
pub use self::dynamic_honey_badger::DynamicHoneyBadger;
pub use self::error::{Error, ErrorKind, Result};
@ -142,6 +143,8 @@ pub struct JoinPlan<N: Ord> {
pub_key_set: PublicKeySet,
/// The public keys of the nodes taking part in key generation.
pub_keys: BTreeMap<N, PublicKey>,
/// The current encryption schedule for threshold cryptography.
encryption_schedule: EncryptionSchedule,
}
/// The ongoing key generation, together with information about the validator change.
@ -150,14 +153,14 @@ struct KeyGenState<N> {
/// The key generation instance.
key_gen: SyncKeyGen<N>,
/// The change for which key generation is performed.
change: Change<N>,
change: NodeChange<N>,
/// The number of key generation messages received from each peer. At most _N + 1_ are
/// accepted.
msg_count: BTreeMap<N, usize>,
}
impl<N: NodeIdT> KeyGenState<N> {
fn new(key_gen: SyncKeyGen<N>, change: Change<N>) -> Self {
fn new(key_gen: SyncKeyGen<N>, change: NodeChange<N>) -> Self {
KeyGenState {
key_gen,
change,
@ -175,8 +178,8 @@ impl<N: NodeIdT> KeyGenState<N> {
/// If the node `node_id` is the currently joining candidate, returns its public key.
fn candidate_key(&self, node_id: &N) -> Option<&PublicKey> {
match self.change {
Change::Add(ref id, ref pk) if id == node_id => Some(pk),
Change::Add(_, _) | Change::Remove(_) => None,
NodeChange::Add(ref id, ref pk) if id == node_id => Some(pk),
NodeChange::Add(_, _) | NodeChange::Remove(_) => None,
}
}

View File

@ -189,10 +189,10 @@ impl<N> SignedVote<N> {
mod tests {
use std::sync::Arc;
use rand;
use super::super::NodeChange;
use super::{Change, SignedVote, VoteCounter};
use fault_log::{FaultKind, FaultLog};
use rand;
use NetworkInfo;
/// Returns a vector of `node_num` `VoteCounter`s, and some signed example votes.
@ -213,9 +213,13 @@ mod tests {
// Sign a few votes.
let sign_votes = |counter: &mut VoteCounter<usize>| {
(0..node_num)
.map(Change::Remove)
.map(|change| counter.sign_vote_for(change).expect("sign vote").clone())
.collect::<Vec<_>>()
.map(NodeChange::Remove)
.map(|change| {
counter
.sign_vote_for(Change::NodeChange(change))
.expect("sign vote")
.clone()
}).collect::<Vec<_>>()
};
let signed_votes: Vec<_> = counters.iter_mut().map(sign_votes).collect();
(counters, signed_votes)
@ -301,6 +305,9 @@ mod tests {
.add_committed_vote(&1, sv[2][1].clone())
.expect("add committed");
assert!(faults.is_empty());
assert_eq!(ct.compute_winner(), Some(&Change::Remove(1)));
assert_eq!(
ct.compute_winner(),
Some(&Change::NodeChange(NodeChange::Remove(1)))
);
}
}

View File

@ -7,6 +7,7 @@ use serde::{de::DeserializeOwned, Serialize};
use super::HoneyBadger;
use honey_badger::SubsetHandlingStrategy;
use threshold_decryption::EncryptionSchedule;
use util::SubRng;
use {Contribution, NetworkInfo, NodeIdT};
@ -25,6 +26,8 @@ pub struct HoneyBadgerBuilder<C, N> {
rng: Box<dyn Rng>,
/// Strategy used to handle the output of the `Subset` algorithm.
subset_handling_strategy: SubsetHandlingStrategy,
/// Schedule for adding threshold encryption to some percentage of rounds
encryption_schedule: EncryptionSchedule,
_phantom: PhantomData<C>,
}
@ -43,6 +46,7 @@ where
max_future_epochs: 3,
rng: Box::new(rand::thread_rng()),
subset_handling_strategy: SubsetHandlingStrategy::Incremental,
encryption_schedule: EncryptionSchedule::Always,
_phantom: PhantomData,
}
}
@ -83,6 +87,12 @@ where
self
}
/// Sets the schedule to use for threshold encryption.
pub fn encryption_schedule(&mut self, encryption_schedule: EncryptionSchedule) -> &mut Self {
self.encryption_schedule = encryption_schedule;
self
}
/// Creates a new Honey Badger instance.
pub fn build(&mut self) -> HoneyBadger<C, N> {
HoneyBadger {
@ -95,6 +105,7 @@ where
incoming_queue: BTreeMap::new(),
rng: Box::new(self.rng.sub_rng()),
subset_handling_strategy: self.subset_handling_strategy.clone(),
encryption_schedule: self.encryption_schedule,
}
}
}

View File

@ -9,7 +9,7 @@ use std::sync::Arc;
use bincode;
use crypto::Ciphertext;
use log::{debug, error, warn};
use rand::Rand;
use rand::{Rand, Rng};
use serde::{de::DeserializeOwned, Serialize};
use serde_derive::Serialize;
@ -189,6 +189,7 @@ pub struct EpochState<C, N: Rand> {
accepted_proposers: BTreeSet<N>,
/// Determines the behavior upon receiving proposals from `subset`.
subset_handler: SubsetHandler<N>,
require_decryption: bool,
_phantom: PhantomData<C>,
}
@ -203,6 +204,7 @@ where
hb_id: u64,
epoch: u64,
subset_handling_strategy: SubsetHandlingStrategy,
require_decryption: bool,
) -> Result<Self> {
let epoch_id = EpochId { hb_id, epoch };
let cs = Subset::new(netinfo.clone(), &epoch_id).map_err(ErrorKind::CreateSubset)?;
@ -213,15 +215,25 @@ where
decryption: BTreeMap::default(),
accepted_proposers: Default::default(),
subset_handler: subset_handling_strategy.into(),
require_decryption,
_phantom: PhantomData,
})
}
/// If the instance hasn't terminated yet, inputs our encrypted contribution.
pub fn propose(&mut self, ciphertext: &Ciphertext) -> Result<Step<C, N>> {
let ser_ct =
bincode::serialize(ciphertext).map_err(|err| ErrorKind::ProposeBincode(*err))?;
let cs_step = self.subset.handle_input(ser_ct)?;
pub fn propose<R: Rng>(&mut self, proposal: &C, rng: &mut R) -> Result<Step<C, N>> {
let ser_prop =
bincode::serialize(&proposal).map_err(|err| ErrorKind::ProposeBincode(*err))?;
let cs_step = self.subset.handle_input(if self.require_decryption {
let ciphertext = self
.netinfo
.public_key_set()
.public_key()
.encrypt_with_rng(rng, ser_prop);
bincode::serialize(&ciphertext).map_err(|err| ErrorKind::ProposeBincode(*err))?
} else {
ser_prop
})?;
self.process_subset(cs_step)
}
@ -315,7 +327,13 @@ where
} = self.subset_handler.handle(cs_output);
for (k, v) in contributions {
step.extend(self.send_decryption_share(k.clone(), &v)?);
step.extend(if self.require_decryption {
self.send_decryption_share(k.clone(), &v)?
} else {
self.decryption
.insert(k.clone(), DecryptionState::Complete(v));
Step::default()
});
self.accepted_proposers.insert(k);
}

View File

@ -2,7 +2,6 @@ use std::collections::btree_map::Entry;
use std::collections::BTreeMap;
use std::sync::Arc;
use bincode;
use derivative::Derivative;
use rand::{Rand, Rng};
use serde::{de::DeserializeOwned, Serialize};
@ -12,6 +11,7 @@ use super::{Batch, Error, ErrorKind, HoneyBadgerBuilder, Message, MessageContent
use {util, Contribution, DistAlgorithm, NetworkInfo, NodeIdT};
pub use super::epoch_state::SubsetHandlingStrategy;
use threshold_decryption::EncryptionSchedule;
/// An instance of the Honey Badger Byzantine fault tolerant consensus algorithm.
#[derive(Derivative)]
@ -38,6 +38,8 @@ pub struct HoneyBadger<C, N: Rand> {
pub(super) rng: Box<dyn Rng + Send + Sync>,
/// Represents the optimization strategy to use for output of the `Subset` algorithm.
pub(super) subset_handling_strategy: SubsetHandlingStrategy,
/// The schedule for which rounds we should use threshold encryption.
pub(super) encryption_schedule: EncryptionSchedule,
}
pub type Step<C, N> = ::Step<HoneyBadger<C, N>>;
@ -92,15 +94,17 @@ where
return Ok(Step::default());
}
self.has_input = true;
let ser_prop =
bincode::serialize(&proposal).map_err(|err| ErrorKind::ProposeBincode(*err))?;
let ciphertext = self
.netinfo
.public_key_set()
.public_key()
.encrypt_with_rng(&mut self.rng, ser_prop);
let epoch = self.epoch;
let step = self.epoch_state_mut(epoch)?.propose(&ciphertext)?;
let step = {
let epoch_state = {
self.epoch_state_mut(epoch)?;
self.epochs.get_mut(&epoch).expect(
"We created the epoch_state in `self.epoch_state_mut(...)` just a moment ago.",
)
};
let rng = &mut self.rng;
epoch_state.propose(proposal, rng)?
};
Ok(step.join(self.try_output_batches()?))
}
@ -132,6 +136,10 @@ where
!self.netinfo.is_validator() || self.has_input
}
pub fn get_encryption_schedule(&self) -> EncryptionSchedule {
self.encryption_schedule
}
/// Returns the number of validators from which we have already received a proposal for the
/// current epoch.
pub(crate) fn received_proposals(&self) -> usize {
@ -183,6 +191,7 @@ where
self.session_id,
epoch,
self.subset_handling_strategy.clone(),
self.encryption_schedule.use_on_epoch(epoch),
)?),
})
}

View File

@ -35,7 +35,7 @@ use dynamic_honey_badger::{self, Batch as DhbBatch, DynamicHoneyBadger, Message}
use transaction_queue::TransactionQueue;
use {util, Contribution, DistAlgorithm, NodeIdT};
pub use dynamic_honey_badger::{Change, ChangeState, Input};
pub use dynamic_honey_badger::{Change, ChangeState, Input, NodeChange};
/// Queueing honey badger error variants.
#[derive(Debug, Fail)]

View File

@ -21,6 +21,27 @@ use serde_derive::{Deserialize, Serialize};
use fault_log::{Fault, FaultKind, FaultLog};
use {DistAlgorithm, NetworkInfo, NodeIdT, Target};
/// How frequently Threshold Encryption should be used.
#[derive(Clone, Copy, Eq, PartialEq, Serialize, Deserialize, Hash, Debug)]
pub enum EncryptionSchedule {
Always,
Never,
EveryNthEpoch(u32),
/// How many with encryption, followed by how many without encryption.
TickTock(u32, u32),
}
impl EncryptionSchedule {
pub fn use_on_epoch(self, epoch: u64) -> bool {
match self {
EncryptionSchedule::Always => true,
EncryptionSchedule::Never => false,
EncryptionSchedule::EveryNthEpoch(n) => (epoch % u64::from(n)) == 0,
EncryptionSchedule::TickTock(on, off) => (epoch % u64::from(on + off)) <= u64::from(on),
}
}
}
/// A threshold decryption error.
#[derive(Clone, Eq, PartialEq, Debug, Fail)]
pub enum Error {

View File

@ -19,7 +19,9 @@ use itertools::Itertools;
use log::info;
use rand::{Isaac64Rng, Rng};
use hbbft::dynamic_honey_badger::{Batch, Change, ChangeState, DynamicHoneyBadger, Input};
use hbbft::dynamic_honey_badger::{
Batch, Change, ChangeState, DynamicHoneyBadger, Input, NodeChange,
};
use hbbft::transaction_queue::TransactionQueue;
use hbbft::NetworkInfo;
@ -39,17 +41,22 @@ where
network.input(*id, Input::User(queue.choose(&mut rng, 3, 10)));
}
network.input_all(Input::Change(Change::Remove(NodeId(0))));
network.input_all(Input::Change(Change::NodeChange(NodeChange::Remove(
NodeId(0),
))));
fn has_remove(node: &TestNode<UsizeDhb>) -> bool {
node.outputs()
.iter()
.any(|batch| *batch.change() == ChangeState::Complete(Change::Remove(NodeId(0))))
node.outputs().iter().any(|batch| {
*batch.change()
== ChangeState::Complete(Change::NodeChange(NodeChange::Remove(NodeId(0))))
})
}
fn has_add(node: &TestNode<UsizeDhb>) -> bool {
node.outputs().iter().any(|batch| match *batch.change() {
ChangeState::Complete(Change::Add(ref id, _)) => *id == NodeId(0),
ChangeState::Complete(Change::NodeChange(NodeChange::Add(ref id, _))) => {
*id == NodeId(0)
}
_ => false,
})
}
@ -95,7 +102,10 @@ where
.netinfo()
.secret_key()
.public_key();
network.input_all(Input::Change(Change::Add(NodeId(0), pk)));
network.input_all(Input::Change(Change::NodeChange(NodeChange::Add(
NodeId(0),
pk,
))));
input_add = true;
}
}

View File

@ -228,17 +228,19 @@ where
));
}
// Broadcast messages get expanded into multiple direct messages.
hbbft::Target::All => for to in nodes.keys().filter(|&to| to != &sender) {
if !faulty {
message_count = message_count.saturating_add(1);
}
hbbft::Target::All => {
for to in nodes.keys().filter(|&to| to != &sender) {
if !faulty {
message_count = message_count.saturating_add(1);
}
dest.push_back(NetworkMessage::new(
sender.clone(),
tmsg.message.clone(),
to.clone(),
));
},
dest.push_back(NetworkMessage::new(
sender.clone(),
tmsg.message.clone(),
to.clone(),
));
}
}
}
}

View File

@ -9,7 +9,7 @@ pub mod net;
use std::{collections, time};
use hbbft::dynamic_honey_badger::{Change, ChangeState, DynamicHoneyBadger, Input};
use hbbft::dynamic_honey_badger::{Change, ChangeState, DynamicHoneyBadger, Input, NodeChange};
use hbbft::DistAlgorithm;
use net::adversary::ReorderingAdversary;
use net::proptest::{gen_seed, NetworkDimension, TestRng, TestRngSeed};
@ -138,8 +138,9 @@ fn do_drop_and_readd(cfg: TestConfig) {
}
// Afterwards, remove a specific node from the dynamic honey badger network.
net.broadcast_input(&Input::Change(Change::Remove(pivot_node_id)))
.expect("broadcasting failed");
net.broadcast_input(&Input::Change(Change::NodeChange(NodeChange::Remove(
pivot_node_id,
)))).expect("broadcasting failed");
// We are tracking (correct) nodes' state through the process by ticking them off individually.
let mut awaiting_removal: collections::BTreeSet<_> =
@ -157,7 +158,7 @@ fn do_drop_and_readd(cfg: TestConfig) {
for change in step.output.iter().map(|output| output.change()) {
match change {
ChangeState::Complete(Change::Remove(pivot_node_id)) => {
ChangeState::Complete(Change::NodeChange(NodeChange::Remove(pivot_node_id))) => {
println!("Node {:?} done removing.", node_id);
// Removal complete, tally:
awaiting_removal.remove(&node_id);
@ -170,11 +171,13 @@ fn do_drop_and_readd(cfg: TestConfig) {
.public_key();
let _ = net[node_id]
.algorithm_mut()
.handle_input(Input::Change(Change::Add(*pivot_node_id, pk)))
.expect("failed to send `Add` input");
.handle_input(Input::Change(Change::NodeChange(NodeChange::Add(
*pivot_node_id,
pk,
)))).expect("failed to send `Add` input");
}
ChangeState::Complete(Change::Add(pivot_node_id, _)) => {
ChangeState::Complete(Change::NodeChange(NodeChange::Add(pivot_node_id, _))) => {
println!("Node {:?} done adding.", node_id);
// Node added, ensure it has been removed first.
if awaiting_removal.contains(&node_id) {

View File

@ -20,7 +20,9 @@ use log::info;
use rand::{Isaac64Rng, Rng};
use hbbft::dynamic_honey_badger::DynamicHoneyBadger;
use hbbft::queueing_honey_badger::{Batch, Change, ChangeState, Input, QueueingHoneyBadger, Step};
use hbbft::queueing_honey_badger::{
Batch, Change, ChangeState, Input, NodeChange, QueueingHoneyBadger, Step,
};
use hbbft::NetworkInfo;
use network::{Adversary, MessageScheduler, NodeId, SilentAdversary, TestNetwork, TestNode};
@ -33,20 +35,25 @@ where
A: Adversary<QHB>,
{
// The second half of the transactions will be input only after a node has been removed.
network.input_all(Input::Change(Change::Remove(NodeId(0))));
network.input_all(Input::Change(Change::NodeChange(NodeChange::Remove(
NodeId(0),
))));
for tx in 0..(num_txs / 2) {
network.input_all(Input::User(tx));
}
fn has_remove(node: &TestNode<QHB>) -> bool {
node.outputs()
.iter()
.any(|batch| *batch.change() == ChangeState::Complete(Change::Remove(NodeId(0))))
node.outputs().iter().any(|batch| {
*batch.change()
== ChangeState::Complete(Change::NodeChange(NodeChange::Remove(NodeId(0))))
})
}
fn has_add(node: &TestNode<QHB>) -> bool {
node.outputs().iter().any(|batch| match *batch.change() {
ChangeState::Complete(Change::Add(ref id, _)) => *id == NodeId(0),
ChangeState::Complete(Change::NodeChange(NodeChange::Add(ref id, _))) => {
*id == NodeId(0)
}
_ => false,
})
}
@ -77,7 +84,10 @@ where
.netinfo()
.secret_key()
.public_key();
network.input_all(Input::Change(Change::Add(NodeId(0), pk)));
network.input_all(Input::Change(Change::NodeChange(NodeChange::Add(
NodeId(0),
pk,
))));
input_add = true;
}
}