Introduced a local message delivery system as a solution to message broadcasts.

This commit is contained in:
Vladimir Komendantskiy 2018-04-05 13:09:46 +01:00
parent 6b90917f79
commit 4cb80ee789
4 changed files with 149 additions and 104 deletions

View File

@ -8,7 +8,9 @@ use std::marker::{Send, Sync};
use merkle::MerkleTree;
use merkle::proof::{Proof, Lemma, Positioned};
use reed_solomon_erasure::ReedSolomon;
use crossbeam_channel as channel;
use crossbeam_channel::{Sender, Receiver};
use messaging::{Target, TargetedMessage, SourcedMessage};
/// Broadcast algorithm instance.
///
@ -18,17 +20,15 @@ use crossbeam_channel as channel;
/// paper doesn't make it clear though how other messages - Echo and Ready - are
/// distributed over the instances. Also it appears that the sender of a message
/// might become part of the message for this to work.
pub struct Instance<'a, T: 'a + Send + Sync> {
/// The transmit side of the multiple consumer channel to comms threads.
tx: &'a channel::Sender<Message<T>>,
/// The receive side of the multiple producer channel from comms threads.
rx: &'a channel::Receiver<(usize, Message<T>)>,
/// Transmit sides of private channels to comms threads.
txs_priv: &'a Vec<channel::Sender<Message<T>>>,
pub struct Instance<'a, T: 'a + Clone + Debug + Send + Sync> {
/// The transmit side of the channel to comms threads.
tx: &'a Sender<TargetedMessage<T>>,
/// The receive side of the channel from comms threads.
rx: &'a Receiver<SourcedMessage<T>>,
/// Value to be broadcast.
broadcast_value: Option<T>,
/// This instance's index for identification against its comms task.
pub node_index: usize,
node_index: usize,
/// Number of nodes participating in broadcast.
num_nodes: usize,
/// Maximum allowed number of faulty nodes.
@ -40,23 +40,20 @@ impl<'a, T: Clone + Debug + Eq + Hash + Send + Sync + Into<Vec<u8>>
Instance<'a, T>
where Vec<u8>: From<T>
{
pub fn new(tx: &'a channel::Sender<Message<T>>,
rx: &'a channel::Receiver<(usize, Message<T>)>,
txs_priv: &'a Vec<channel::Sender<Message<T>>>,
pub fn new(tx: &'a Sender<TargetedMessage<T>>,
rx: &'a Receiver<SourcedMessage<T>>,
broadcast_value: Option<T>,
num_nodes: usize,
node_index: usize) ->
Self
{
let num_remote_nodes = txs_priv.len();
Instance {
tx: tx,
rx: rx,
txs_priv: txs_priv,
broadcast_value: broadcast_value,
node_index: node_index,
num_nodes: num_remote_nodes + 1,
num_faulty_nodes: num_remote_nodes / 3
num_nodes: num_nodes,
num_faulty_nodes: (num_nodes - 1) / 3
}
}
@ -74,7 +71,7 @@ where Vec<u8>: From<T>
crossbeam::scope(|scope| {
scope.spawn(move || {
*result_r_scoped.lock().unwrap() =
Some(inner_run(self.tx, self.rx, self.txs_priv, bvalue,
Some(inner_run(self.tx, self.rx, bvalue,
self.node_index, self.num_nodes,
self.num_faulty_nodes));
});
@ -103,7 +100,7 @@ pub enum BroadcastError {
/// need to be sent anywhere. It is returned to the broadcast instance and gets
/// recorded immediately.
fn send_shards<'a, T>(value: T,
txs_priv: &'a Vec<channel::Sender<Message<T>>>,
tx: &'a Sender<TargetedMessage<T>>,
coding: &ReedSolomon) ->
Result<Proof<T>, BroadcastError>
where T: Clone + Debug + Send + Sync + Into<Vec<u8>>
@ -115,7 +112,6 @@ where T: Clone + Debug + Send + Sync + Into<Vec<u8>>
debug!("Data shards: {}, parity shards: {}",
data_shard_num, parity_shard_num);
assert_eq!(txs_priv.len() + 1, data_shard_num + parity_shard_num);
let mut v: Vec<u8> = Vec::from(value).to_owned();
// Pad the value vector with zeros to allow for shards of equal sizes.
@ -168,8 +164,12 @@ where T: Clone + Debug + Send + Sync + Into<Vec<u8>>
}
else {
// Rest of the proofs are sent to remote nodes.
txs_priv[i-1].send(Message::Broadcast(
BroadcastMessage::Value(proof))).unwrap();
tx.send(
TargetedMessage {
target: Target::Node(i + 1),
message: Message::Broadcast(
BroadcastMessage::Value(proof))
}).unwrap();
}
}
}
@ -178,11 +178,11 @@ where T: Clone + Debug + Send + Sync + Into<Vec<u8>>
}
/// The main loop of the broadcast task.
fn inner_run<'a, T>(tx: &'a channel::Sender<Message<T>>,
rx: &'a channel::Receiver<(usize, Message<T>)>,
txs_priv: &'a Vec<channel::Sender<Message<T>>>,
fn inner_run<'a, T>(tx: &'a Sender<TargetedMessage<T>>,
rx: &'a Receiver<SourcedMessage<T>>,
broadcast_value: Option<T>,
node_index: usize, num_nodes: usize,
node_index: usize,
num_nodes: usize,
num_faulty_nodes: usize) ->
Result<T, BroadcastError>
where T: Clone + Debug + Eq + Hash + Send + Sync + Into<Vec<u8>>
@ -205,7 +205,7 @@ where T: Clone + Debug + Eq + Hash + Send + Sync + Into<Vec<u8>>
// Assemble a Merkle tree from data and parity shards. Take all proofs from
// this tree and send them, each to its own node.
if let Some(v) = broadcast_value {
match send_shards(v, txs_priv, &coding) {
match send_shards(v, tx, &coding) {
// Record the first proof that was generated by the node itself.
Ok(proof) => {
let h = proof.root_hash.clone();
@ -236,7 +236,10 @@ where T: Clone + Debug + Eq + Hash + Send + Sync + Into<Vec<u8>>
while result == None {
// Receive a message from the socket IO task.
let message = rx.recv().unwrap();
if let (i, Message::Broadcast(message)) = message {
if let SourcedMessage {
source: i,
message: Message::Broadcast(message)
} = message {
match message {
// A value received. Record the value and multicast an echo.
BroadcastMessage::Value(p) => {
@ -262,8 +265,10 @@ where T: Clone + Debug + Eq + Hash + Send + Sync + Into<Vec<u8>>
}
}
// Broadcast an echo of this proof.
tx.send(Message::Broadcast(BroadcastMessage::Echo(p)))
.unwrap()
tx.send(TargetedMessage {
target: Target::All,
message: Message::Broadcast(BroadcastMessage::Echo(p))
}).unwrap()
},
// An echo received. Verify the proof it contains.
@ -308,9 +313,12 @@ where T: Clone + Debug + Eq + Hash + Send + Sync + Into<Vec<u8>>
// Ready
if !ready_sent {
ready_sent = true;
tx.send(Message::Broadcast(
BroadcastMessage::Ready(h.to_owned())))
.unwrap();
tx.send(TargetedMessage {
target: Target::All,
message: Message::Broadcast(
BroadcastMessage::Ready(
h.to_owned()))
}).unwrap();
}
}
}
@ -335,8 +343,12 @@ where T: Clone + Debug + Eq + Hash + Send + Sync + Into<Vec<u8>>
if (ready_num == num_faulty_nodes + 1) &&
!ready_sent
{
tx.send(Message::Broadcast(
BroadcastMessage::Ready(h.to_vec()))).unwrap();
tx.send(TargetedMessage {
target: Target::All,
message: Message::Broadcast(
BroadcastMessage::Ready(
h.to_vec()))
}).unwrap();
}
// Upon receiving 2f + 1 matching Ready(h) messages,

View File

@ -2,37 +2,36 @@
//! socket. Local communication with coordinating threads is made via
//! `crossbeam_channel::unbounded()`.
use std::fmt::Debug;
use std::sync::{Arc, Mutex};
use std::sync::Arc;
use crossbeam;
use crossbeam_channel as channel;
use proto::Message;
use task;
use messaging::{SourcedMessage};
/// A communication task connects a remote node to the thread that manages the
/// consensus algorithm.
pub struct CommsTask<'a, T: 'a + Send + Sync + From<Vec<u8>> + Into<Vec<u8>>>
pub struct CommsTask<'a, T: 'a + Clone + Debug + Send + Sync +
From<Vec<u8>> + Into<Vec<u8>>>
where Vec<u8>: From<T>
{
/// The transmit side of the multiple producer channel from comms threads.
tx: &'a channel::Sender<(usize, Message<T>)>,
/// The receive side of the multiple consumer channel to comms threads.
tx: &'a channel::Sender<SourcedMessage<T>>,
/// The receive side of the channel to the comms thread.
rx: &'a channel::Receiver<Message<T>>,
/// The receive side of the private channel to the comms thread.
rx_priv: &'a channel::Receiver<Message<T>>,
/// The socket IO task.
task: task::Task,
/// The index of this comms task for identification against its remote node.
pub node_index: usize
}
impl<'a, T: Debug + Send + Sync + From<Vec<u8>> + Into<Vec<u8>>>
impl<'a, T: Clone + Debug + Send + Sync + From<Vec<u8>> + Into<Vec<u8>>>
CommsTask<'a, T>
where Vec<u8>: From<T>
{
pub fn new(tx: &'a channel::Sender<(usize, Message<T>)>,
pub fn new(tx: &'a channel::Sender<SourcedMessage<T>>,
rx: &'a channel::Receiver<Message<T>>,
rx_priv: &'a channel::Receiver<Message<T>>,
stream: ::std::net::TcpStream,
node_index: usize) ->
Self
@ -43,7 +42,6 @@ where Vec<u8>: From<T>
CommsTask {
tx: tx,
rx: rx,
rx_priv: rx_priv,
task: task::Task::new(stream),
node_index: node_index
}
@ -55,7 +53,6 @@ where Vec<u8>: From<T>
// Borrow parts of `self` before entering the thread binding scope.
let tx = Arc::new(self.tx);
let rx = Arc::new(self.rx);
let rx_priv = Arc::new(self.rx_priv);
let mut task1 = self.task.try_clone().unwrap(); // FIXME: handle errors
let node_index = self.node_index;
@ -63,24 +60,12 @@ where Vec<u8>: From<T>
// Local comms receive loop thread.
scope.spawn(move || {
loop {
select_loop! {
// Receive a multicast message from the manager thread.
recv(rx, message) => {
debug!("Node {} <- {:?}", node_index, message);
// Forward the message to the remote node.
task1.send_message(message)
.unwrap();
debug!("SENT Node {}", node_index);
},
// Receive a private message from the manager thread.
recv(rx_priv, message) => {
debug!("Node {} <- {:?}", node_index, message);
// Forward the message to the remote node.
task1.send_message(message)
.unwrap();
debug!("SENT Node {}", node_index);
}
}
// Receive a multicast message from the manager thread.
let message = rx.recv().unwrap();
debug!("Node {} <- {:?}", node_index, message);
// Forward the message to the remote node.
task1.send_message(message).unwrap();
debug!("SENT Node {}", node_index);
}
});
@ -90,7 +75,12 @@ where Vec<u8>: From<T>
match self.task.receive_message() {
Ok(message) => {
debug!("Node {} -> {:?}", node_index, message);
tx.send((node_index, message)).unwrap()
tx.send(
SourcedMessage {
source: node_index,
message
})
.unwrap()
},
Err(task::Error::ProtobufError(e)) =>
warn!("Protobuf error {}", e),

View File

@ -48,11 +48,12 @@ extern crate crossbeam_channel;
extern crate reed_solomon_erasure;
mod connection;
mod messaging;
mod errors;
mod proto;
mod task;
mod commst;
mod broadcast;
mod agreement;
pub mod node;
pub mod broadcast;
pub mod agreement;

View File

@ -5,18 +5,19 @@ use std::hash::Hash;
use std::marker::{Send, Sync};
use std::net::SocketAddr;
use crossbeam;
use crossbeam_channel as channel;
use crossbeam_channel::{unbounded, Sender, Receiver};
use connection;
use broadcast;
use proto::Message;
use commst;
use messaging::{Messaging, Target, TargetedMessage, SourcedMessage};
/// This is a structure to start a consensus node.
pub struct Node<T> {
/// Incoming connection socket.
addr: SocketAddr,
/// Sockets of remote nodes. TODO.
/// Sockets of remote nodes.
remotes: HashSet<SocketAddr>,
/// Optionally, a value to be broadcast by this node.
value: Option<T>
@ -37,52 +38,92 @@ where Vec<u8>: From<T>
/// Consensus node procedure implementing HoneyBadgerBFT.
pub fn run(&self) -> Result<T, ()>
{
// Multiple-producer, multiple-consumer channel from comms tasks to
// all algorithm actor tasks such as Reliable Broadcast.
let (from_comms_tx, from_comms_rx):
(
channel::Sender<(usize, Message<T>)>,
channel::Receiver<(usize, Message<T>)>
) = channel::unbounded();
let (from_comms_tx, from_comms_rx) = (&from_comms_tx, &from_comms_rx);
// // Multiple-producer, multiple-consumer channel from comms tasks to
// // all algorithm actor tasks such as Reliable Broadcast.
// let (from_comms_tx, from_comms_rx):
// (
// Sender<(usize, Message<T>)>,
// Receiver<(usize, Message<T>)>
// ) = unbounded();
// let (from_comms_tx, from_comms_rx) = (&from_comms_tx, &from_comms_rx);
// Multiple-producer, multiple-consumer channel from algorithm actor
// tasks such as Reliable Broadcast to all comms tasks.
let (to_comms_tx, to_comms_rx):
(
channel::Sender<Message<T>>,
channel::Receiver<Message<T>>
) = channel::unbounded();
let (to_comms_tx, to_comms_rx) = (&to_comms_tx, &to_comms_rx);
// // Multiple-producer, multiple-consumer channel from algorithm actor
// // tasks such as Reliable Broadcast to all comms tasks.
// let (to_comms_tx, to_comms_rx):
// (
// Sender<Message<T>>,
// Receiver<Message<T>>
// ) = unbounded();
// let (to_comms_tx, to_comms_rx) = (&to_comms_tx, &to_comms_rx);
let value = &self.value;
let connections = connection::make(&self.addr, &self.remotes);
// Single-consumer channels from algorithm actor tasks to comms tasks.
let to_comms_1: Vec<(channel::Sender<Message<T>>,
channel::Receiver<Message<T>>)> =
(0 .. connections.len())
.map(|_| channel::unbounded())
// // Single-consumer channels from algorithm actor tasks to comms tasks.
// let to_comms_1: Vec<(channel::Sender<Message<T>>,
// channel::Receiver<Message<T>>)> =
// (0 .. connections.len())
// .map(|_| channel::unbounded())
// .collect();
// // All transmit sides of channels to comms tasks are collected together
// // for sending messages to particular remote nodes.
// let to_comms_1_txs: Vec<channel::Sender<Message<T>>> =
// to_comms_1.iter().map(|(tx, _)| tx.to_owned()).collect();
// let to_comms_1 = &to_comms_1;
// let to_comms_1_txs = &to_comms_1_txs;
// let (to_comms_tx, to_comms_rx):
// (
// channel::Sender<Message<T>>,
// channel::Receiver<Message<T>>
// ) = channel::unbounded();
// let (to_comms_tx, to_comms_rx) = (&to_comms_tx, &to_comms_rx);
let to_comms: Vec<(Sender<Message<T>>, Receiver<Message<T>>)>
= (0 .. connections.len())
.map(|_| unbounded())
.collect();
// All transmit sides of channels to comms tasks are collected together
// for sending messages to particular remote nodes.
let to_comms_1_txs: Vec<channel::Sender<Message<T>>> =
to_comms_1.iter().map(|(tx, _)| tx.to_owned()).collect();
let to_comms_1 = &to_comms_1;
let to_comms_1_txs = &to_comms_1_txs;
let to_comms_txs = &to_comms.iter()
.map(|(tx, _)| tx.to_owned())
.collect();
let to_comms_rxs: &Vec<Receiver<Message<T>>> = &to_comms.iter()
.map(|(_, rx)| rx.to_owned())
.collect();
let (from_comms_tx, from_comms_rx) = unbounded();
let (from_comms_tx, from_comms_rx) = (&from_comms_tx, &from_comms_rx);
let to_algo: Vec<(Sender<SourcedMessage<T>>,
Receiver<SourcedMessage<T>>)>
= (0 .. connections.len() + 1)
.map(|_| unbounded())
.collect();
let to_algo_txs = &to_algo.iter()
.map(|(tx, _)| tx.to_owned())
.collect();
let to_algo_rxs: &Vec<Receiver<SourcedMessage<T>>> = &to_algo.iter()
.map(|(_, rx)| rx.to_owned())
.collect();
let (from_algo_tx, from_algo_rx) = unbounded();
let (from_algo_tx, from_algo_rx) = (&from_algo_tx, &from_algo_rx);
let messaging: Messaging<T> =
Messaging::new(to_comms_txs, from_comms_rx,
to_algo_txs, from_algo_rx);
// All spawned threads will have exited by the end of the scope.
crossbeam::scope(|scope| {
let num_nodes = connections.len() + 1;
messaging.spawn(scope);
// Associate a broadcast instance with this node. This instance will
// broadcast the proposed value. There is no remote node
// corresponding to this instance, and no dedicated comms task. The
// node index is 0.
let ref to_algo_rx0 = to_algo_rxs[0];
scope.spawn(move || {
match broadcast::Instance::new(to_comms_tx,
from_comms_rx,
to_comms_1_txs,
match broadcast::Instance::new(from_algo_tx,
to_algo_rx0,
value.to_owned(),
num_nodes,
0)
.run()
{
@ -97,25 +138,26 @@ where Vec<u8>: From<T>
// Receive side of a single-consumer channel from algorithm
// actor tasks to the comms task.
let ref to_comms_1_rx = to_comms_1[i].1;
let ref to_comms_rx = to_comms_rxs[i];
let node_index = i + 1;
scope.spawn(move || {
commst::CommsTask::new(from_comms_tx,
to_comms_rx,
to_comms_1_rx,
// FIXME: handle error
c.stream.try_clone().unwrap(),
node_index)
.run();
});
// Associate a broadcast instance to the above comms task.
let ref to_algo_rx = to_algo_rxs[node_index];
scope.spawn(move || {
match broadcast::Instance::new(to_comms_tx,
from_comms_rx,
to_comms_1_txs,
match broadcast::Instance::new(from_algo_tx,
to_algo_rx,
None,
num_nodes,
node_index)
.run()
{