Ported more integration tests to the new net simulator (#387)

* Ported threshold_sign test to the new framework

* Ported the first three broadcast tests to the new framework

* Extracted messages storting and random swapping to reusable functions

Used to compose ProposeAdversary's behavior without having to duplicate code

* Implemented ProposeAdversary for the new integration testing framework

Added "id()" function to the "NodeMutHandle", required for sending messages to all nodes

ProposeAdversary needs access to all faulty node's netinfo. We follow the example of the binary_agreement_mitm integration test of using an reference counted Mutex to make netinfo available on both Consensus Protocol construction and in the Adversary implementation.
Unlike binary_agreement_mitm every faulty node needs to use its own netinfo for the broadcast algorithm, so we store all nodeinfo structures in a Map instead of just the nodeinfo of the first node.
Ideallly the new network simulation library should provide netinfo similar to the old library to avoid these hideous workarounds.

* Migrated test_broadcast_random_delivery_adv_propose to the new network simulator

Refactored the implementation of ProposeAdversary to closely resemble the behavior in the old network simulator library.
Implemented a pick_random_node function to sort messages for a random node id. Switched from using "inject_message" to joining messages generated by adversaries' temporary Broadcast Consensus Protocols with the Step generated by regular operation.

* Ported RandomAdversary to the new network simulator library

Ported all broadcast integration tests and replaced the old tests with the new.

* Eliminated the old broadcast integration test, replaced with the new

* Ported subset test to the new framework

Adjusted message queue size as suggested by Andreas

* Ported the first three honey_badger tests to the new framework

* Re-implemented FaultyShareAdversary for the new framework

Eliminated the old honey_badger integration tests, replaced with implementations using the new net simulator framework

* Fixed issues reported by clippy

* Fixed issues reported on code review

* Fixed issues reported by clippy

* Implemented a broadcast test dropping messages similar to the tests written in the old framework

* Picking the proposer id at random, verifying all possible output cases

If the proposer is faulty the message queue may starve, but the outputs of all correct nodes need to be empty, if the broadcast protocol produces output nonetheless all correct nodes need to have the same output.
If the proposer was correct all correct nodes need to output its proposed value.

* Eliminated duplicated semicolon

* Consistently using TestRng and proptest in all newly ported tests

* Increased the drop_and_re_add test message limit to 20k per node

* Removed unnecessary closure

* Increased the tolerance for deviations from the expected value range to eliminate random test failures
This commit is contained in:
David Forstenlechner 2019-03-14 14:41:23 +01:00 committed by Vladimir Komendantskiy
parent 57455d47ae
commit c1e634ecf2
7 changed files with 776 additions and 310 deletions

View File

@ -1,164 +1,327 @@
#![deny(unused_must_use)]
//! Integration test of the reliable broadcast protocol.
mod network;
pub mod net;
use std::collections::BTreeMap;
use std::iter::once;
use std::sync::Arc;
use std::sync::{Arc, Mutex};
use log::info;
use rand::Rng;
use proptest::{prelude::ProptestConfig, proptest, proptest_helper};
use rand::{Rng, SeedableRng};
use crate::network::{
Adversary, MessageScheduler, MessageWithSender, NodeId, RandomAdversary, SilentAdversary,
TestNetwork, TestNode,
use hbbft::{broadcast::Broadcast, util, ConsensusProtocol, CpStep, NetworkInfo};
use crate::net::adversary::{
sort_ascending, swap_random, Adversary, NetMutHandle, NodeOrderAdversary, RandomAdversary,
ReorderingAdversary,
};
use hbbft::broadcast::{Broadcast, Message};
use hbbft::{util, ConsensusProtocol, NetworkInfo, Target, TargetedMessage};
use crate::net::proptest::{gen_seed, TestRng, TestRngSeed};
use crate::net::{CrankError, NetBuilder, NetMessage, NewNodeInfo, VirtualNet};
/// An adversary that inputs an alternate value.
struct ProposeAdversary {
scheduler: MessageScheduler,
adv_nodes: BTreeMap<NodeId, Arc<NetworkInfo<NodeId>>>,
type NodeId = u16;
type NetworkInfoMap = BTreeMap<NodeId, Arc<NetworkInfo<NodeId>>>;
/// A strategy for picking the next node to handle a message.
/// The sorting algorithm used is stable - preserves message
/// order relative to the node id.
pub enum MessageSorting {
/// Picks a random node and swaps its messages to the front of the queue
RandomPick,
/// Sorts the message queue by receiving node id
SortAscending,
}
/// For each adversarial node does the following, but only once:
///
/// * Creates a *new* instance of the Broadcast ConsensusProtocol,
/// with the adversarial node ID as proposer
/// * Lets it handle a "Fake News" input
/// * Records the returned step's messages
/// * Injects the messages to the queue
pub struct ProposeAdversary {
message_strategy: MessageSorting,
has_sent: bool,
drop_messages: bool,
// TODO this is really hacky but there's no better way to get this value
// Solution taken from binary_agreement_mitm test - ideally the new network simulator
// should be altered to store the netinfo structure alongside nodes similar to
// the way the old network simulator did it.
netinfo_mutex: Arc<Mutex<NetworkInfoMap>>,
}
impl ProposeAdversary {
/// Creates a new replay adversary with the given message scheduler.
fn new(
scheduler: MessageScheduler,
adv_nodes: BTreeMap<NodeId, Arc<NetworkInfo<NodeId>>>,
) -> ProposeAdversary {
/// Creates a new `ProposeAdversary`.
#[inline]
pub fn new(
message_strategy: MessageSorting,
netinfo_mutex: Arc<Mutex<NetworkInfoMap>>,
drop_messages: bool,
) -> Self {
ProposeAdversary {
scheduler,
adv_nodes,
message_strategy,
has_sent: false,
drop_messages,
netinfo_mutex,
}
}
}
impl Adversary<Broadcast<NodeId>> for ProposeAdversary {
fn pick_node(&self, nodes: &BTreeMap<NodeId, TestNode<Broadcast<NodeId>>>) -> NodeId {
self.scheduler.pick_node(nodes)
}
fn push_message(&mut self, _: NodeId, _: TargetedMessage<Message, NodeId>) {
// All messages are ignored.
}
fn step(&mut self) -> Vec<MessageWithSender<Broadcast<NodeId>>> {
let mut rng = rand::thread_rng();
if self.has_sent {
return vec![];
#[inline]
fn pre_crank<R: Rng>(
&mut self,
mut net: NetMutHandle<'_, Broadcast<NodeId>, Self>,
rng: &mut R,
) {
match self.message_strategy {
MessageSorting::RandomPick => swap_random(&mut net, rng),
MessageSorting::SortAscending => sort_ascending(&mut net),
}
self.has_sent = true;
self.adv_nodes
.iter()
.flat_map(|(&id, netinfo)| {
Broadcast::new(netinfo.clone(), id)
}
#[inline]
fn tamper<R: Rng>(
&mut self,
mut net: NetMutHandle<'_, Broadcast<NodeId>, Self>,
msg: NetMessage<Broadcast<NodeId>>,
mut rng: &mut R,
) -> Result<CpStep<Broadcast<NodeId>>, CrankError<Broadcast<NodeId>>> {
let mut step = net.dispatch_message(msg, rng)?;
// optionally drop all messages other than the fake broadcasts
if self.drop_messages {
step.messages.clear();
}
if !self.has_sent {
self.has_sent = true;
// Get adversarial nodes
let faulty_nodes = net.faulty_nodes_mut();
// Instantiate a temporary broadcast consensus protocol for each faulty node
// and add the generated messages to the current step.
for faulty_node in faulty_nodes {
let netinfo = self
.netinfo_mutex
.lock()
.unwrap()
.get(faulty_node.id())
.cloned()
.expect("Adversary netinfo mutex not populated");
let fake_step = Broadcast::new(netinfo, *faulty_node.id())
.expect("broadcast instance")
.handle_input(b"Fake news".to_vec(), &mut rng)
.expect("propose")
.messages
.into_iter()
.map(move |msg| MessageWithSender::new(id, msg))
})
.collect()
.expect("propose");
step.messages.extend(fake_step.messages);
}
}
Ok(step)
}
}
/// Broadcasts a value from node 0 and expects all good nodes to receive it.
fn test_broadcast<A: Adversary<Broadcast<NodeId>>>(
mut network: TestNetwork<A, Broadcast<NodeId>>,
mut net: VirtualNet<Broadcast<NodeId>, A>,
proposed_value: &[u8],
rng: &mut TestRng,
proposer_id: NodeId,
) {
// This returns an error in all but the first test.
let _ = env_logger::try_init();
let proposer_is_faulty = net.get(proposer_id).unwrap().is_faulty();
// Make node 0 propose the value.
network.input(NodeId(0), proposed_value.to_vec());
let _step = net
.send_input(proposer_id, proposed_value.to_vec(), rng)
.expect("Setting input failed");
// Handle messages in random order until all nodes have output the proposed value.
while !network.nodes.values().all(TestNode::terminated) {
network.step();
// Handle messages until all good nodes have terminated.
// If the proposer is faulty it is legal for the queue to starve
while !net.nodes().all(|node| node.algorithm().terminated()) {
if proposer_is_faulty && net.messages_len() == 0 {
info!("Expected starvation of messages with a faulty proposer");
// The output of all correct nodes needs to be empty in this case.
// We check for the output of the first node to be empty and
// rely on the identity checks at the end of this function to
// verify that all other correct nodes have empty output as well.
let first = net
.correct_nodes()
.nth(0)
.expect("At least one correct node needs to exist");
assert!(first.outputs().is_empty());
break;
}
let _ = net.crank_expect(rng);
}
// Verify that all instances output the proposed value.
for node in network.nodes.values() {
assert!(once(&proposed_value.to_vec()).eq(node.outputs()));
if proposer_is_faulty {
// If the proposer was faulty it is sufficient for all correct nodes having the same value.
let first = net.correct_nodes().nth(0).unwrap().outputs();
assert!(net.nodes().all(|node| node.outputs() == first));
} else {
// In the case where the proposer was valid it must be the value it proposed.
assert!(net
.nodes()
.all(|node| once(&proposed_value.to_vec()).eq(node.outputs())));
}
assert!(once(&proposed_value.to_vec()).eq(network.observer.outputs()));
}
fn new_broadcast(netinfo: Arc<NetworkInfo<NodeId>>) -> Broadcast<NodeId> {
Broadcast::new(netinfo, NodeId(0)).expect("Instantiate broadcast")
}
fn test_broadcast_different_sizes<A, F>(new_adversary: F, proposed_value: &[u8])
where
fn test_broadcast_different_sizes<A, F>(
new_adversary: F,
proposed_value: &[u8],
seed: TestRngSeed,
adversary_netinfo: &Arc<Mutex<NetworkInfoMap>>,
) where
A: Adversary<Broadcast<NodeId>>,
F: Fn(BTreeMap<NodeId, Arc<NetworkInfo<NodeId>>>) -> A,
F: Fn() -> A,
{
let mut rng = rand::thread_rng();
let mut rng: TestRng = TestRng::from_seed(seed);
let sizes = (1..6)
.chain(once(rng.gen_range(6, 20)))
.chain(once(rng.gen_range(30, 50)));
for size in sizes {
// cloning since it gets moved into a closure
let cloned_netinfo_map = adversary_netinfo.clone();
let num_faulty_nodes = util::max_faulty(size);
let num_good_nodes = size - num_faulty_nodes;
info!(
"Network size: {} good nodes, {} faulty nodes",
num_good_nodes, num_faulty_nodes
size - num_faulty_nodes,
num_faulty_nodes
);
let adversary = |adv_nodes| new_adversary(adv_nodes);
let network = TestNetwork::new(num_good_nodes, num_faulty_nodes, adversary, new_broadcast);
test_broadcast(network, proposed_value);
let proposer_id = rng.gen_range(0, size) as NodeId;
let (net, _) = NetBuilder::new(0..size as u16)
.num_faulty(num_faulty_nodes as usize)
.message_limit(10_000 * size as usize)
.no_time_limit()
.adversary(new_adversary())
.using(move |info| {
let netinfo = Arc::new(info.netinfo);
cloned_netinfo_map
.lock()
.unwrap()
.insert(info.id, netinfo.clone());
Broadcast::new(netinfo, proposer_id)
.expect("Failed to create a Broadcast instance.")
})
.build(&mut rng)
.expect("Could not construct test network.");
test_broadcast(net, proposed_value, &mut rng, proposer_id);
}
}
#[test]
fn test_8_broadcast_equal_leaves_silent() {
let adversary = |_| SilentAdversary::new(MessageScheduler::Random);
proptest! {
#![proptest_config(ProptestConfig {
cases: 1, .. ProptestConfig::default()
})]
#[test]
#[allow(clippy::unnecessary_operation)]
fn test_8_broadcast_equal_leaves_silent(seed in gen_seed()) {
do_test_8_broadcast_equal_leaves_silent(seed)
}
#[test]
#[allow(clippy::unnecessary_operation)]
fn test_broadcast_random_delivery_silent(seed in gen_seed()) {
do_test_broadcast_random_delivery_silent(seed)
}
#[test]
#[allow(clippy::unnecessary_operation)]
fn test_broadcast_first_delivery_silent(seed in gen_seed()) {
do_test_broadcast_first_delivery_silent(seed)
}
#[test]
#[allow(clippy::unnecessary_operation)]
fn test_broadcast_first_delivery_adv_propose(seed in gen_seed()) {
do_test_broadcast_first_delivery_adv_propose(seed)
}
#[test]
#[allow(clippy::unnecessary_operation)]
fn test_broadcast_random_delivery_adv_propose(seed in gen_seed()) {
do_test_broadcast_random_delivery_adv_propose(seed)
}
#[test]
#[allow(clippy::unnecessary_operation)]
fn test_broadcast_random_delivery_adv_propose_and_drop(seed in gen_seed()) {
do_test_broadcast_random_delivery_adv_propose_and_drop(seed)
}
#[test]
#[allow(clippy::unnecessary_operation)]
fn test_broadcast_random_adversary(seed in gen_seed()) {
do_test_broadcast_random_adversary(seed)
}
}
fn do_test_8_broadcast_equal_leaves_silent(seed: TestRngSeed) {
let mut rng: TestRng = TestRng::from_seed(seed);
let size = 8;
let num_faulty = 0;
let proposer_id = rng.gen_range(0, size);
let (net, _) = NetBuilder::new(0..size as u16)
.num_faulty(num_faulty as usize)
.message_limit(10_000 * size as usize)
.no_time_limit()
.adversary(ReorderingAdversary::new())
.using(move |node_info: NewNodeInfo<_>| {
Broadcast::new(Arc::new(node_info.netinfo), proposer_id)
.expect("Failed to create a Broadcast instance.")
})
.build(&mut rng)
.expect("Could not construct test network.");
// Space is ASCII character 32. So 32 spaces will create shards that are all equal, even if the
// length of the value is inserted.
test_broadcast(
TestNetwork::new(8, 0, adversary, new_broadcast),
&[b' '; 32],
);
test_broadcast(net, &[b' '; 32], &mut rng, proposer_id);
}
#[test]
fn test_broadcast_random_delivery_silent() {
let new_adversary = |_| SilentAdversary::new(MessageScheduler::Random);
test_broadcast_different_sizes(new_adversary, b"Foo");
fn do_test_broadcast_random_delivery_silent(seed: TestRngSeed) {
test_broadcast_different_sizes(ReorderingAdversary::new, b"Foo", seed, &Default::default());
}
#[test]
fn test_broadcast_first_delivery_silent() {
let new_adversary = |_| SilentAdversary::new(MessageScheduler::First);
test_broadcast_different_sizes(new_adversary, b"Foo");
fn do_test_broadcast_first_delivery_silent(seed: TestRngSeed) {
test_broadcast_different_sizes(NodeOrderAdversary::new, b"Foo", seed, &Default::default());
}
#[test]
fn test_broadcast_random_delivery_adv_propose() {
let new_adversary = |adv_nodes| ProposeAdversary::new(MessageScheduler::Random, adv_nodes);
test_broadcast_different_sizes(new_adversary, b"Foo");
}
#[test]
fn test_broadcast_first_delivery_adv_propose() {
let new_adversary = |adv_nodes| ProposeAdversary::new(MessageScheduler::First, adv_nodes);
test_broadcast_different_sizes(new_adversary, b"Foo");
}
#[test]
fn test_broadcast_random_adversary() {
let new_adversary = |_| {
// Note: Set this to 0.8 to watch 30 gigs of RAM disappear.
RandomAdversary::new(0.2, 0.2, || TargetedMessage {
target: Target::All,
message: rand::random(),
})
fn do_test_broadcast_first_delivery_adv_propose(seed: TestRngSeed) {
let adversary_netinfo: Arc<Mutex<NetworkInfoMap>> = Default::default();
let new_adversary = || {
ProposeAdversary::new(
MessageSorting::SortAscending,
adversary_netinfo.clone(),
false,
)
};
test_broadcast_different_sizes(new_adversary, b"RandomFoo");
test_broadcast_different_sizes(new_adversary, b"Foo", seed, &adversary_netinfo);
}
fn do_test_broadcast_random_delivery_adv_propose(seed: TestRngSeed) {
let adversary_netinfo: Arc<Mutex<NetworkInfoMap>> = Default::default();
let new_adversary =
|| ProposeAdversary::new(MessageSorting::RandomPick, adversary_netinfo.clone(), false);
test_broadcast_different_sizes(new_adversary, b"Foo", seed, &adversary_netinfo);
}
fn do_test_broadcast_random_delivery_adv_propose_and_drop(seed: TestRngSeed) {
let adversary_netinfo: Arc<Mutex<NetworkInfoMap>> = Default::default();
let new_adversary =
|| ProposeAdversary::new(MessageSorting::RandomPick, adversary_netinfo.clone(), true);
test_broadcast_different_sizes(new_adversary, b"Foo", seed, &adversary_netinfo);
}
fn do_test_broadcast_random_adversary(seed: TestRngSeed) {
let new_adversary = || RandomAdversary::new(0.2, 0.2);
test_broadcast_different_sizes(new_adversary, b"RandomFoo", seed, &Default::default());
}

View File

@ -1,162 +1,180 @@
#![deny(unused_must_use)]
//! Network tests for Honey Badger.
mod network;
pub mod net;
use std::collections::BTreeMap;
use std::iter;
use std::sync::Arc;
use std::sync::{Arc, Mutex};
use itertools::Itertools;
use log::info;
use rand::{seq::SliceRandom, Rng};
use proptest::{prelude::ProptestConfig, proptest, proptest_helper};
use rand::{seq::SliceRandom, Rng, SeedableRng};
use hbbft::honey_badger::{Batch, EncryptionSchedule, HoneyBadger, MessageContent};
use hbbft::sender_queue::{self, SenderQueue, Step};
use hbbft::transaction_queue::TransactionQueue;
use hbbft::{threshold_decrypt, util, ConsensusProtocol, NetworkInfo, Target, TargetedMessage};
use hbbft::{threshold_decrypt, util, CpStep, NetworkInfo, Target};
use crate::network::{
Adversary, MessageScheduler, MessageWithSender, NodeId, RandomAdversary, SilentAdversary,
TestNetwork, TestNode,
use crate::net::adversary::{
sort_by_random_node, Adversary, NetMutHandle, NodeOrderAdversary, RandomAdversary,
ReorderingAdversary,
};
use crate::net::proptest::{gen_seed, TestRng, TestRngSeed};
use crate::net::{CrankError, NetBuilder, NetMessage, NewNodeInfo, Node, VirtualNet};
type NodeId = u16;
type NetworkInfoMap = BTreeMap<NodeId, Arc<NetworkInfo<NodeId>>>;
type UsizeHoneyBadger = SenderQueue<HoneyBadger<Vec<usize>, NodeId>>;
type HoneyBadgerMessage = NetMessage<UsizeHoneyBadger>;
/// An adversary whose nodes only send messages with incorrect decryption shares.
#[derive(Clone, Debug, Default)]
pub struct FaultyShareAdversary {
num_good: usize,
num_adv: usize,
adv_nodes: BTreeMap<NodeId, Arc<NetworkInfo<NodeId>>>,
scheduler: MessageScheduler,
share_triggers: BTreeMap<u64, bool>,
// TODO this is really hacky but there's no better way to get this value
// Solution taken from binary_agreement_mitm test - ideally the new network simulator
// should be altered to store the netinfo structure alongside nodes similar to
// the way the old network simulator did it.
netinfo_mutex: Arc<Mutex<NetworkInfoMap>>,
}
impl FaultyShareAdversary {
/// Creates a new silent adversary with the given message scheduler.
pub fn new(
num_good: usize,
num_adv: usize,
adv_nodes: BTreeMap<NodeId, Arc<NetworkInfo<NodeId>>>,
scheduler: MessageScheduler,
) -> FaultyShareAdversary {
/// Creates a new adversary with the necessary network info instances
pub fn new(netinfo_mutex: Arc<Mutex<NetworkInfoMap>>) -> Self {
FaultyShareAdversary {
num_good,
num_adv,
scheduler,
share_triggers: BTreeMap::new(),
adv_nodes,
netinfo_mutex,
}
}
}
impl Adversary<UsizeHoneyBadger> for FaultyShareAdversary {
fn pick_node(&self, nodes: &BTreeMap<NodeId, TestNode<UsizeHoneyBadger>>) -> NodeId {
self.scheduler.pick_node(nodes)
}
fn push_message(
#[inline]
fn pre_crank<R: Rng>(
&mut self,
sender_id: NodeId,
msg: TargetedMessage<<UsizeHoneyBadger as ConsensusProtocol>::Message, NodeId>,
mut net: NetMutHandle<'_, UsizeHoneyBadger, Self>,
rng: &mut R,
) {
let NodeId(sender_id) = sender_id;
if sender_id < self.num_good {
if let TargetedMessage {
target: Target::All,
message: sender_queue::Message::Algo(hb_msg),
} = msg
{
let epoch = hb_msg.epoch();
// Set the trigger to simulate decryption share messages.
self.share_triggers.entry(epoch).or_insert(true);
}
}
sort_by_random_node(&mut net, rng);
}
fn step(&mut self) -> Vec<MessageWithSender<UsizeHoneyBadger>> {
let mut outgoing = vec![];
let fake_proposal = &Vec::from("X marks the spot");
#[inline]
fn tamper<R: Rng>(
&mut self,
mut net: NetMutHandle<'_, UsizeHoneyBadger, Self>,
msg: HoneyBadgerMessage,
rng: &mut R,
) -> Result<CpStep<UsizeHoneyBadger>, CrankError<UsizeHoneyBadger>> {
if let sender_queue::Message::Algo(hb_msg) = msg.payload() {
let epoch = hb_msg.epoch();
// Set the trigger to simulate decryption share messages
// if epoch has not been encountered yet.
self.share_triggers.entry(epoch).or_insert(true);
}
let mut step = net.dispatch_message(msg, rng)?;
let fake_proposal = &Vec::from("X marks the spot");
// For each untriggered epoch, send fake shares
for (epoch, trigger_set) in &mut self.share_triggers {
if *trigger_set {
// Unset the trigger.
*trigger_set = false;
// Get node id vectors up-front to avoid borrow issues
let faulty_node_ids: Vec<NodeId> =
net.faulty_nodes_mut().map(|node| *node.id()).collect();
let all_node_ids: Vec<NodeId> = net.nodes_mut().map(|node| node.id()).collect();
// Broadcast fake decryption shares from all adversarial nodes.
for sender_id in self.num_good..self.num_adv {
let adv_node = &self.adv_nodes[&NodeId(sender_id)];
let fake_ciphertext = (*adv_node)
for faulty_node_id in faulty_node_ids {
// get the adversarial's net info
let netinfo = self
.netinfo_mutex
.lock()
.unwrap()
.get(&faulty_node_id)
.cloned()
.expect("Adversary netinfo mutex not populated");
// encrypt false share
let fake_ciphertext = (*netinfo)
.public_key_set()
.public_key()
.encrypt(fake_proposal);
let share = adv_node
let share = netinfo
.secret_key_share()
.expect("missing adversary key share")
.decrypt_share(&fake_ciphertext)
.expect("decryption share");
// Send the share to remote nodes.
for proposer_id in 0..self.num_good + self.num_adv {
outgoing.push(MessageWithSender::new(
NodeId(sender_id),
for proposer_id in &all_node_ids {
step.messages.push(
Target::All.message(sender_queue::Message::Algo(
MessageContent::DecryptionShare {
proposer_id: NodeId(proposer_id),
proposer_id: *proposer_id,
share: threshold_decrypt::Message(share.clone()),
}
.with_epoch(*epoch),
)),
))
);
}
}
}
}
outgoing
Ok(step)
}
}
/// Proposes `num_txs` values and expects nodes to output and order them.
fn test_honey_badger<A>(mut network: TestNetwork<A, UsizeHoneyBadger>, num_txs: usize)
where
fn test_honey_badger<A>(
mut net: VirtualNet<UsizeHoneyBadger, A>,
num_txs: usize,
mut rng: &mut TestRng,
) where
A: Adversary<UsizeHoneyBadger>,
{
let new_queue = |id: &NodeId| (*id, (0..num_txs).collect::<Vec<usize>>());
let mut queues: BTreeMap<_, _> = network.nodes.keys().map(new_queue).collect();
let mut queues: BTreeMap<_, _> = net
.correct_nodes()
.map(|node| (*node.id(), (0..num_txs).collect::<Vec<usize>>()))
.collect();
// 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<UsizeHoneyBadger>| {
let node_busy = |node: &Node<UsizeHoneyBadger>| {
node.outputs().iter().flat_map(Batch::iter).unique().count() < num_txs
};
let mut rng = rand::thread_rng();
// Handle messages in random order until all nodes have output all transactions.
while network.nodes.values_mut().any(node_busy) {
while net.correct_nodes().any(node_busy) {
// If a node is expecting input, take it from the queue. Otherwise handle a message.
let input_ids: Vec<_> = network
.nodes
.iter()
.filter(|(_, node)| !node.instance().algo().has_input())
.map(|(id, _)| *id)
let input_ids: Vec<_> = net
.correct_nodes()
.filter(|node| !node.algorithm().algo().has_input())
.map(|node| *node.id())
.collect();
if let Some(id) = input_ids[..].choose(&mut rng) {
let queue = queues.get_mut(id).unwrap();
queue.remove_multiple(network.nodes[id].outputs().iter().flat_map(Batch::iter));
network.input(*id, queue.choose(&mut rng, 3, 10));
queue.remove_multiple(net.get(*id).unwrap().outputs().iter().flat_map(Batch::iter));
let _ = net.send_input(*id, queue.choose(&mut rng, 3, 10), &mut rng);
} else {
network.step();
let _ = net.crank_expect(&mut rng);
}
}
verify_output_sequence(&network);
verify_output_sequence(&net);
}
/// Verifies that all instances output the same sequence of batches.
fn verify_output_sequence<A>(network: &TestNetwork<A, UsizeHoneyBadger>)
fn verify_output_sequence<A>(network: &VirtualNet<UsizeHoneyBadger, A>)
where
A: Adversary<UsizeHoneyBadger>,
{
let mut expected: Option<BTreeMap<u64, &_>> = None;
for node in network.nodes.values() {
for node in network.correct_nodes() {
assert!(!node.outputs().is_empty());
let outputs: BTreeMap<u64, &BTreeMap<NodeId, Vec<usize>>> = node
.outputs()
@ -175,71 +193,107 @@ fn new_honey_badger(
netinfo: Arc<NetworkInfo<NodeId>>,
) -> (UsizeHoneyBadger, Step<HoneyBadger<Vec<usize>, NodeId>>) {
let our_id = *netinfo.our_id();
let observer = NodeId(netinfo.num_nodes());
let nc = netinfo.clone();
let peer_ids = nc
.all_ids()
.filter(|&&them| them != our_id)
.cloned()
.chain(iter::once(observer));
let peer_ids = nc.all_ids().filter(|&&them| them != our_id).cloned();
let hb = HoneyBadger::builder(netinfo)
.encryption_schedule(EncryptionSchedule::EveryNthEpoch(2))
.build();
SenderQueue::builder(hb, peer_ids).build(our_id)
}
fn test_honey_badger_different_sizes<A, F>(new_adversary: F, num_txs: usize)
where
fn test_honey_badger_different_sizes<A, F>(
new_adversary: F,
num_txs: usize,
seed: TestRngSeed,
adversary_netinfo: &Arc<Mutex<NetworkInfoMap>>,
) where
A: Adversary<UsizeHoneyBadger>,
F: Fn(usize, usize, BTreeMap<NodeId, Arc<NetworkInfo<NodeId>>>) -> A,
F: Fn() -> A,
{
// This returns an error in all but the first test.
let _ = env_logger::try_init();
let mut rng = rand::thread_rng();
let mut rng: TestRng = TestRng::from_seed(seed);
let sizes = vec![1, 2, 3, 5, rng.gen_range(6, 10)];
for size in sizes {
// cloning since it gets moved into a closure
let cloned_netinfo_map = adversary_netinfo.clone();
let num_adv_nodes = util::max_faulty(size);
let num_good_nodes = size - num_adv_nodes;
info!(
"Network size: {} good nodes, {} faulty nodes",
num_good_nodes, num_adv_nodes
);
let adversary = |adv_nodes| new_adversary(num_good_nodes, num_adv_nodes, adv_nodes);
let network =
TestNetwork::new_with_step(num_good_nodes, num_adv_nodes, adversary, new_honey_badger);
test_honey_badger(network, num_txs);
let (net, _) = NetBuilder::new(0..size as u16)
.num_faulty(num_adv_nodes as usize)
.message_limit(10_000 * size as usize)
.no_time_limit()
.adversary(new_adversary())
.using_step(move |info: NewNodeInfo<_>| {
let netinfo = Arc::new(info.netinfo);
cloned_netinfo_map
.lock()
.unwrap()
.insert(info.id, netinfo.clone());
new_honey_badger(netinfo)
})
.build(&mut rng)
.expect("Could not construct test network.");
test_honey_badger(net, num_txs, &mut rng);
}
}
#[test]
fn test_honey_badger_random_delivery_silent() {
let new_adversary = |_: usize, _: usize, _| SilentAdversary::new(MessageScheduler::Random);
test_honey_badger_different_sizes(new_adversary, 30);
proptest! {
#![proptest_config(ProptestConfig {
cases: 1, .. ProptestConfig::default()
})]
#[test]
#[allow(clippy::unnecessary_operation)]
fn test_honey_badger_random_delivery_silent(seed in gen_seed()) {
do_test_honey_badger_random_delivery_silent(seed)
}
#[test]
#[allow(clippy::unnecessary_operation)]
fn test_honey_badger_first_delivery_silent(seed in gen_seed()) {
do_test_honey_badger_first_delivery_silent(seed)
}
#[test]
#[allow(clippy::unnecessary_operation)]
fn test_honey_badger_faulty_share(seed in gen_seed()) {
do_test_honey_badger_faulty_share(seed)
}
#[test]
#[allow(clippy::unnecessary_operation)]
fn test_honey_badger_random_adversary(seed in gen_seed()) {
do_test_honey_badger_random_adversary(seed)
}
}
#[test]
fn test_honey_badger_first_delivery_silent() {
let new_adversary = |_: usize, _: usize, _| SilentAdversary::new(MessageScheduler::First);
test_honey_badger_different_sizes(new_adversary, 30);
fn do_test_honey_badger_random_delivery_silent(seed: TestRngSeed) {
test_honey_badger_different_sizes(ReorderingAdversary::new, 30, seed, &Default::default());
}
#[test]
fn test_honey_badger_faulty_share() {
let new_adversary = |num_good: usize, num_adv: usize, adv_nodes| {
FaultyShareAdversary::new(num_good, num_adv, adv_nodes, MessageScheduler::Random)
};
test_honey_badger_different_sizes(new_adversary, 8);
fn do_test_honey_badger_first_delivery_silent(seed: TestRngSeed) {
test_honey_badger_different_sizes(NodeOrderAdversary::new, 30, seed, &Default::default());
}
#[test]
fn test_honey_badger_random_adversary() {
let new_adversary = |_, _, _| {
fn do_test_honey_badger_faulty_share(seed: TestRngSeed) {
let adversary_netinfo: Arc<Mutex<NetworkInfoMap>> = Default::default();
let new_adversary = || FaultyShareAdversary::new(adversary_netinfo.clone());
test_honey_badger_different_sizes(new_adversary, 8, seed, &adversary_netinfo);
}
fn do_test_honey_badger_random_adversary(seed: TestRngSeed) {
let new_adversary = || {
// A 10% injection chance is roughly ~13k extra messages added.
RandomAdversary::new(0.1, 0.1, || TargetedMessage {
target: Target::All,
message: rand::random(),
})
RandomAdversary::new(0.1, 0.1)
};
test_honey_badger_different_sizes(new_adversary, 8);
test_honey_badger_different_sizes(new_adversary, 8, seed, &Default::default());
}

View File

@ -40,7 +40,8 @@ use rand::Rng;
use hbbft::{ConsensusProtocol, CpStep};
use crate::net::{CrankError, NetMessage, Node, VirtualNet};
use crate::net::util::randomly;
use crate::net::{CrankError, NetMessage, NetworkMessage, Node, VirtualNet};
/// Immutable network handle.
///
@ -297,6 +298,12 @@ where
NodeMutHandle(inner)
}
/// Return node ID.
#[inline]
pub fn id(&self) -> D::NodeId {
self.0.id().clone()
}
/// Returns a mutable reference to the faulty node.
///
/// # Panics
@ -360,6 +367,89 @@ where
}
}
/// Utility function to sort messages in the given net handle ascending by receiving node id
#[inline]
pub fn sort_ascending<D, A>(net: &mut NetMutHandle<'_, D, A>)
where
D: ConsensusProtocol,
D::Message: Clone,
D::Output: Clone,
A: Adversary<D>,
{
net.sort_messages_by(|a, b| a.to().cmp(&b.to()))
}
/// Utility function to swap the topmost message with a random message in the queue
#[inline]
pub fn swap_random<R, D, A>(net: &mut NetMutHandle<'_, D, A>, rng: &mut R)
where
R: Rng,
D: ConsensusProtocol,
D::Message: Clone,
D::Output: Clone,
A: Adversary<D>,
{
let l = net.get_messages().len();
if l > 0 {
net.swap_messages(0, rng.gen_range(0, l));
}
}
/// Selects a random node and returns its id
#[inline]
pub fn random_node<R, D, A>(net: &mut NetMutHandle<'_, D, A>, rng: &mut R) -> Option<D::NodeId>
where
R: Rng,
D: ConsensusProtocol,
D::Message: Clone,
D::Output: Clone,
A: Adversary<D>,
{
let l = net.nodes_mut().count();
if l > 0 {
// Pick a node id at random
return Some(
net.nodes_mut()
.nth(rng.gen_range(0, l))
.expect("nodes list changed since last call")
.id(),
);
}
None
}
/// Utility function to sweep the messages for a randomly chosen node to the top
/// of the message queue. Relative message order is preserved by using
/// the available stable sort algorithm.
#[inline]
pub fn sort_by_random_node<R, D, A>(net: &mut NetMutHandle<'_, D, A>, rng: &mut R)
where
R: Rng,
D: ConsensusProtocol,
D::Message: Clone,
D::Output: Clone,
A: Adversary<D>,
{
if let Some(picked_node) = random_node(net, rng) {
// To make the picked node's messages sorted to the top of the queue
// it always has to be less than the others, regardless of its actual id.
net.sort_messages_by(|a, b| {
let a = a.to().clone();
let b = b.to().clone();
if a == b {
cmp::Ordering::Equal
} else if a == picked_node {
cmp::Ordering::Less
} else if b == picked_node {
cmp::Ordering::Greater
} else {
a.cmp(&b)
}
});
}
}
/// Passive adversary.
///
/// The `NullAdversary` does not interfere with operation in any way, it neither reorders messages
@ -409,7 +499,7 @@ where
#[inline]
fn pre_crank<R: Rng>(&mut self, mut net: NetMutHandle<'_, D, Self>, _rng: &mut R) {
// Message are sorted by NodeID on each step.
net.sort_messages_by(|a, b| a.to.cmp(&b.to))
sort_ascending(&mut net);
}
}
@ -435,9 +525,83 @@ where
{
#[inline]
fn pre_crank<R: Rng>(&mut self, mut net: NetMutHandle<'_, D, Self>, rng: &mut R) {
let l = net.0.messages_len();
if l > 0 {
net.swap_messages(0, rng.gen_range(0, l));
}
swap_random(&mut net, rng);
}
}
/// An adversary that performs naive replay attacks.
///
/// The adversary will randomly take a message that is sent to one of its nodes
/// and re-send it to a different node.
///
/// Additionally it will broadcast unrelated random messages,
/// and the message queue by a random node id.
#[derive(Copy, Clone, Debug, Default)]
pub struct RandomAdversary {
/// Probability of a message replay
p_replay: f32,
/// Probability of a message injection
p_inject: f32,
}
impl RandomAdversary {
pub fn new(p_replay: f32, p_inject: f32) -> Self {
RandomAdversary { p_replay, p_inject }
}
}
impl<D> Adversary<D> for RandomAdversary
where
D: ConsensusProtocol,
D::Message: Clone,
D::Output: Clone,
rand::distributions::Standard:
rand::distributions::Distribution<<D as ConsensusProtocol>::Message>,
{
#[inline]
fn pre_crank<R: Rng>(&mut self, mut net: NetMutHandle<'_, D, Self>, rng: &mut R) {
sort_by_random_node(&mut net, rng);
}
#[inline]
fn tamper<R: Rng>(
&mut self,
mut net: NetMutHandle<'_, D, Self>,
msg: NetMessage<D>,
rng: &mut R,
) -> Result<CpStep<D>, CrankError<D>> {
// only replay a message in some cases
if randomly(self.p_replay, rng) {
// randomly choose a target to send the message to
if let Some(picked_node) = random_node(&mut net, rng) {
let mut new_msg = msg.clone();
new_msg.from = new_msg.to;
new_msg.to = picked_node;
net.inject_message(QueuePosition::Back, new_msg);
}
}
// Possibly inject more messages
while randomly(self.p_inject, rng) {
// Messages must originate from the current node
let sender = msg.to.clone();
// Generate a random message
let message: D::Message = rand::random();
// Collect receivers
let node_ids: Vec<<D as ConsensusProtocol>::NodeId> = net
.nodes_mut()
.map(|node| node.id())
.filter(|node_id| *node_id != sender)
.collect();
// Broadcast message
for node_id in node_ids {
let new_msg = NetworkMessage::new(sender.clone(), message.clone(), node_id);
net.inject_message(QueuePosition::Back, new_msg);
}
}
net.dispatch_message(msg, rng)
}
}

View File

@ -3,6 +3,8 @@
//! A collection of miscellaneous functions that are used in the tests, but are generic enough to be
//! factored out.
use rand::Rng;
/// Try-return a result, wrapped in `Some`.
///
/// Like `try!`, but wraps into an `Option::Some` as well. Useful for iterators
@ -16,3 +18,10 @@ macro_rules! try_some {
}
};
}
/// Return true with a certain `probability` ([0 .. 1.0]).
pub fn randomly<R: Rng>(probability: f32, rng: &mut R) -> bool {
assert!(probability <= 1.0);
assert!(probability >= 0.0);
rng.gen_range(0.0, 1.0) <= probability
}

View File

@ -95,8 +95,8 @@ fn do_drop_and_re_add(cfg: TestConfig) {
let num_faulty = cfg.dimension.faulty();
let (net, _) = NetBuilder::new(0..cfg.dimension.size())
.num_faulty(num_faulty)
// Limited to 15k messages per node.
.message_limit(15_000 * cfg.dimension.size() as usize)
// Limited to 20k messages per node.
.message_limit(20_000 * cfg.dimension.size() as usize)
// 30 secs per node.
.time_limit(time::Duration::from_secs(30 * cfg.dimension.size() as u64))
.adversary(ReorderingAdversary::new())

View File

@ -1,39 +1,57 @@
#![deny(unused_must_use)]
//! Integration tests of the Subset protocol.
mod network;
pub mod net;
use std::collections::{BTreeMap, BTreeSet};
use std::iter::once;
use std::sync::Arc;
use proptest::{prelude::ProptestConfig, proptest, proptest_helper};
use rand::SeedableRng;
use hbbft::subset::{Subset, SubsetOutput};
use hbbft::NetworkInfo;
use hbbft::ConsensusProtocol;
use crate::network::{Adversary, MessageScheduler, NodeId, SilentAdversary, TestNetwork, TestNode};
use crate::net::adversary::{Adversary, NodeOrderAdversary, ReorderingAdversary};
use crate::net::proptest::{gen_seed, TestRng, TestRngSeed};
use crate::net::{NetBuilder, NewNodeInfo, VirtualNet};
type NodeId = u16;
type ProposedValue = Vec<u8>;
fn test_subset<A: Adversary<Subset<NodeId, u8>>>(
mut network: TestNetwork<A, Subset<NodeId, u8>>,
fn test_subset<A>(
mut net: VirtualNet<Subset<NodeId, u8>, A>,
inputs: &BTreeMap<NodeId, ProposedValue>,
) {
let ids: Vec<NodeId> = network.nodes.keys().cloned().collect();
mut rng: &mut TestRng,
) where
A: Adversary<Subset<NodeId, u8>>,
{
let ids: Vec<NodeId> = net.nodes().map(|node| *node.id()).collect();
for id in ids {
if let Some(value) = inputs.get(&id) {
network.input(id, value.to_owned());
let _ = net.send_input(id, value.to_owned(), &mut rng);
}
}
// Terminate when all good nodes do.
while !network.nodes.values().all(TestNode::terminated) {
network.step();
// Handle messages until all good nodes have terminated.
while !net.nodes().all(|node| node.algorithm().terminated()) {
let _ = net.crank_expect(&mut rng);
}
// Verify that all instances output the same set.
let observer: BTreeSet<_> = network.observer.outputs().iter().cloned().collect();
for node in network.nodes.values() {
// Get reference value from the first correct node.
// TODO: Revisit when observers are available in the new net simulator
// or drop this TODO if we decide to abandon that concept.
let expected_value: BTreeSet<_> = net
.correct_nodes()
.nth(0)
.unwrap()
.outputs()
.iter()
.cloned()
.collect();
// Verify that all correct nodes output the same value.
for node in net.correct_nodes() {
let outputs = node.outputs();
let mut actual = BTreeMap::default();
@ -56,42 +74,78 @@ fn test_subset<A: Adversary<Subset<NodeId, u8>>>(
assert_eq!(&inputs[id], value);
}
assert_eq!(outputs.iter().cloned().collect::<BTreeSet<_>>(), observer);
assert_eq!(
outputs.iter().cloned().collect::<BTreeSet<_>>(),
expected_value
);
}
}
fn new_network<A, F>(
good_num: usize,
bad_num: usize,
mut rng: &mut TestRng,
adversary: F,
) -> TestNetwork<A, Subset<NodeId, u8>>
) -> VirtualNet<Subset<NodeId, u8>, A>
where
A: Adversary<Subset<NodeId, u8>>,
F: Fn(BTreeMap<NodeId, Arc<NetworkInfo<NodeId>>>) -> A,
F: Fn() -> A,
{
// This returns an error in all but the first test.
let _ = env_logger::try_init();
let new_subset =
|netinfo: Arc<NetworkInfo<NodeId>>| Subset::new(netinfo, 0).expect("new Subset instance");
TestNetwork::new(good_num, bad_num, adversary, new_subset)
let size = good_num + bad_num;
let (net, _) = NetBuilder::new(0..size as u16)
.num_faulty(bad_num as usize)
.message_limit(10_000 * size as usize)
.no_time_limit()
.adversary(adversary())
.using(move |node_info: NewNodeInfo<_>| {
Subset::new(Arc::new(node_info.netinfo), 0).expect("new Subset instance")
})
.build(&mut rng)
.expect("Could not construct test network.");
net
}
#[test]
fn test_subset_3_out_of_4_nodes_propose() {
proptest! {
#![proptest_config(ProptestConfig {
cases: 1, .. ProptestConfig::default()
})]
#[test]
#[allow(clippy::unnecessary_operation)]
fn test_subset_3_out_of_4_nodes_propose(seed in gen_seed()) {
do_test_subset_3_out_of_4_nodes_propose(seed)
}
#[test]
#[allow(clippy::unnecessary_operation)]
fn test_subset_5_nodes_different_proposed_values(seed in gen_seed()) {
do_test_subset_5_nodes_different_proposed_values(seed)
}
#[test]
#[allow(clippy::unnecessary_operation)]
fn test_subset_1_node(seed in gen_seed()) {
do_test_subset_1_node(seed)
}
}
fn do_test_subset_3_out_of_4_nodes_propose(seed: TestRngSeed) {
let proposed_value = Vec::from("Fake news");
let proposing_ids: BTreeSet<NodeId> = (0..3).map(NodeId).collect();
let proposing_ids: BTreeSet<NodeId> = (0..3).collect();
let proposals: BTreeMap<NodeId, ProposedValue> = proposing_ids
.iter()
.map(|id| (*id, proposed_value.clone()))
.collect();
let adversary = |_| SilentAdversary::new(MessageScheduler::First);
let network = new_network(3, 1, adversary);
test_subset(network, &proposals);
let mut rng: TestRng = TestRng::from_seed(seed);
let net = new_network(3, 1, &mut rng, NodeOrderAdversary::new);
test_subset(net, &proposals, &mut rng);
}
#[test]
fn test_subset_5_nodes_different_proposed_values() {
fn do_test_subset_5_nodes_different_proposed_values(seed: TestRngSeed) {
let proposed_values = vec![
Vec::from("Alpha"),
Vec::from("Bravo"),
@ -99,18 +153,16 @@ fn test_subset_5_nodes_different_proposed_values() {
Vec::from("Delta"),
Vec::from("Echo"),
];
let proposals: BTreeMap<NodeId, ProposedValue> =
(0..5).map(NodeId).zip(proposed_values).collect();
let adversary = |_| SilentAdversary::new(MessageScheduler::Random);
let network = new_network(5, 0, adversary);
test_subset(network, &proposals);
let proposals: BTreeMap<NodeId, ProposedValue> = (0..5).zip(proposed_values).collect();
let mut rng: TestRng = TestRng::from_seed(seed);
let net = new_network(5, 0, &mut rng, ReorderingAdversary::new);
test_subset(net, &proposals, &mut rng);
}
#[test]
fn test_subset_1_node() {
fn do_test_subset_1_node(seed: TestRngSeed) {
let proposals: BTreeMap<NodeId, ProposedValue> =
once((NodeId(0), Vec::from("Node 0 is the greatest!"))).collect();
let adversary = |_| SilentAdversary::new(MessageScheduler::Random);
let network = new_network(1, 0, adversary);
test_subset(network, &proposals);
once((0, Vec::from("Node 0 is the greatest!"))).collect();
let mut rng: TestRng = TestRng::from_seed(seed);
let net = new_network(1, 0, &mut rng, ReorderingAdversary::new);
test_subset(net, &proposals, &mut rng);
}

View File

@ -1,46 +1,46 @@
#![deny(unused_must_use)]
//! Threshold signing tests
//! Non-deterministic tests for the ThresholdSign protocol
mod network;
pub mod net;
use std::iter::once;
use std::sync::Arc;
use log::info;
use rand::Rng;
use proptest::{prelude::ProptestConfig, proptest, proptest_helper};
use rand::{Rng, SeedableRng};
use hbbft::{crypto::Signature, threshold_sign::ThresholdSign, util};
use hbbft::{crypto::Signature, threshold_sign::ThresholdSign, util, ConsensusProtocol};
use crate::network::{Adversary, MessageScheduler, NodeId, SilentAdversary, TestNetwork, TestNode};
use crate::net::adversary::{Adversary, NodeOrderAdversary, ReorderingAdversary};
use crate::net::proptest::{gen_seed, TestRng, TestRngSeed};
use crate::net::{NetBuilder, NewNodeInfo, VirtualNet};
type NodeId = u16;
/// Tests a network of threshold signing instances with an optional expected value. Outputs the
/// computed signature if the test is successful.
fn test_threshold_sign<A>(mut network: TestNetwork<A, ThresholdSign<NodeId>>) -> Signature
fn test_threshold_sign<A>(
mut net: VirtualNet<ThresholdSign<NodeId>, A>,
mut rng: &mut TestRng,
) -> Signature
where
A: Adversary<ThresholdSign<NodeId>>,
{
let mut rng = rand::thread_rng();
network.input_all(());
network.observer.handle_input((), &mut rng); // Observer will only return after `input` was called.
net.broadcast_input(&(), &mut rng)
.expect("threshold sign input failed");
// Handle messages until all good nodes have terminated.
while !network.nodes.values().all(TestNode::terminated) {
network.step();
while !net.nodes().all(|node| node.algorithm().terminated()) {
let _ = net.crank_expect(&mut rng);
}
let mut expected = None;
// Verify that all instances output the same value.
for node in network.nodes.values() {
if let Some(ref b) = expected {
assert!(once(b).eq(node.outputs()));
} else {
assert_eq!(1, node.outputs().len());
expected = Some(node.outputs()[0].clone());
}
}
// Now `expected` is the unique output of all good nodes.
assert!(expected.iter().eq(network.observer.outputs()));
expected.unwrap()
let first = net.correct_nodes().nth(0).unwrap().outputs();
// TODO: Verify if signature is valid
assert!(!first.is_empty());
assert!(net.nodes().all(|node| node.outputs() == first));
first[0].clone()
}
const GOOD_SAMPLE_SET: f64 = 400.0;
@ -50,7 +50,7 @@ const GOOD_SAMPLE_SET: f64 = 400.0;
/// size.
fn check_coin_distribution(num_samples: usize, count_true: usize, count_false: usize) {
// Maximum 40% expectation in case of 400 samples or more.
const EXPECTED_SHARE: f64 = 0.4;
const EXPECTED_SHARE: f64 = 0.33;
let max_gain = GOOD_SAMPLE_SET.log2();
let num_samples_f64 = num_samples as f64;
let gain = num_samples_f64.log2().min(max_gain);
@ -64,17 +64,20 @@ fn check_coin_distribution(num_samples: usize, count_true: usize, count_false: u
assert!(count_false > min_throws);
}
fn test_threshold_sign_different_sizes<A, F>(new_adversary: F, num_samples: usize)
where
fn test_threshold_sign_different_sizes<A, F>(
new_adversary: F,
num_samples: usize,
seed: TestRngSeed,
) where
A: Adversary<ThresholdSign<NodeId>>,
F: Fn(usize, usize) -> A,
F: Fn() -> A,
{
assert!(num_samples > 0);
// This returns an error in all but the first test.
let _ = env_logger::try_init();
let mut rng = rand::thread_rng();
let mut rng: TestRng = TestRng::from_seed(seed);
let mut last_size = 1;
let mut sizes = vec![last_size];
@ -86,24 +89,29 @@ where
for size in sizes {
let num_faulty_nodes = util::max_faulty(size);
let num_good_nodes = size - num_faulty_nodes;
info!(
"Network size: {} good nodes, {} faulty nodes",
num_good_nodes, num_faulty_nodes
size - num_faulty_nodes,
num_faulty_nodes
);
let unique_id: u64 = rng.gen();
let mut count_true = 0;
let mut count_false = 0;
for i in 0..num_samples {
let adversary = |_| new_adversary(num_good_nodes, num_faulty_nodes);
let nonce = format!("My very unique nonce {:x}:{}", unique_id, i);
info!("Nonce: {}", nonce);
let new_coin = |netinfo: _| {
ThresholdSign::new_with_document(netinfo, nonce.clone())
.expect("Failed to set the new coin's ID")
};
let network = TestNetwork::new(num_good_nodes, num_faulty_nodes, adversary, new_coin);
let coin = test_threshold_sign(network).parity();
let (net, _) = NetBuilder::new(0..size as u16)
.num_faulty(num_faulty_nodes as usize)
.message_limit(size * (size - 1))
.no_time_limit()
.adversary(new_adversary())
.using(move |node_info: NewNodeInfo<_>| {
ThresholdSign::new_with_document(Arc::new(node_info.netinfo), nonce.clone())
.expect("Failed to create a ThresholdSign instance.")
})
.build(&mut rng)
.expect("Could not construct test network.");
let coin = test_threshold_sign(net, &mut rng).parity();
if coin {
count_true += 1;
} else {
@ -114,14 +122,30 @@ where
}
}
#[test]
fn test_threshold_sign_random_silent_200_samples() {
let new_adversary = |_: usize, _: usize| SilentAdversary::new(MessageScheduler::Random);
test_threshold_sign_different_sizes(new_adversary, 200);
proptest! {
#![proptest_config(ProptestConfig {
cases: 1, .. ProptestConfig::default()
})]
#[test]
#[allow(clippy::unnecessary_operation)]
fn test_threshold_sign_random_silent_200_samples(seed in gen_seed()) {
do_test_threshold_sign_random_silent_200_samples(seed)
}
#[test]
#[allow(clippy::unnecessary_operation)]
fn test_threshold_sign_first_silent_50_samples(seed in gen_seed()) {
do_test_threshold_sign_first_silent_50_samples(seed)
}
}
#[test]
fn test_threshold_sign_first_silent_50_samples() {
let new_adversary = |_: usize, _: usize| SilentAdversary::new(MessageScheduler::First);
test_threshold_sign_different_sizes(new_adversary, 50);
fn do_test_threshold_sign_random_silent_200_samples(seed: TestRngSeed) {
let new_adversary = || ReorderingAdversary::new();
test_threshold_sign_different_sizes(new_adversary, 200, seed);
}
fn do_test_threshold_sign_first_silent_50_samples(seed: TestRngSeed) {
let new_adversary = || NodeOrderAdversary::new();
test_threshold_sign_different_sizes(new_adversary, 50, seed);
}