mirror of https://github.com/poanetwork/hbbft.git
replaced lockable scoped channel variables with lock-free ones
This commit is contained in:
parent
e01a80dfa7
commit
186a855d2f
|
@ -6,17 +6,14 @@ authors = ["Vladimir Komendantskiy <komendantsky@gmail.com>"]
|
|||
[dependencies]
|
||||
log = "0.4.1"
|
||||
simple_logger = "0.5"
|
||||
tokio = "0.1"
|
||||
#tokio-io = "0.1"
|
||||
#tokio-timer = "0.1"
|
||||
reed-solomon-erasure = "3.0"
|
||||
merkle = { git = "https://github.com/vkomenda/merkle.rs", branch = "public-proof" }
|
||||
ring = "^0.12"
|
||||
rand = "*"
|
||||
error-chain = "0.11"
|
||||
protobuf = "1.4.4"
|
||||
spmc = "0.2.2"
|
||||
crossbeam = "0.3.2"
|
||||
crossbeam-channel = "0.1"
|
||||
|
||||
[build-dependencies]
|
||||
protoc-rust = "1.4.4"
|
||||
|
|
|
@ -1,15 +1,14 @@
|
|||
//! Reliable broadcast algorithm.
|
||||
//! Reliable broadcast algorithm instance.
|
||||
use std::fmt::Debug;
|
||||
use std::hash::Hash;
|
||||
use std::sync::{Arc, Mutex};
|
||||
use std::sync::mpsc;
|
||||
use spmc;
|
||||
use crossbeam;
|
||||
use proto::*;
|
||||
use std::marker::{Send, Sync};
|
||||
use merkle::MerkleTree;
|
||||
use merkle::proof::{Proof, Lemma, Positioned};
|
||||
use reed_solomon_erasure::ReedSolomon;
|
||||
use crossbeam_channel as channel;
|
||||
|
||||
/// Temporary placeholders for the number of participants and the maximum
|
||||
/// envisaged number of faulty nodes. Only one is required since N >= 3f +
|
||||
|
@ -25,31 +24,30 @@ const PLACEHOLDER_F: usize = 2;
|
|||
/// Broadcast stage. See the TODO note below!
|
||||
///
|
||||
/// TODO: The ACS algorithm will require multiple broadcast instances running
|
||||
/// asynchronously, see Figure 4 in the HBBFT paper. So, it's likely that the
|
||||
/// broadcast *stage* has to be replaced with N asynchronous threads, each
|
||||
/// responding to values from one particular remote node. The 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 has to become
|
||||
/// part of the message for this to work.
|
||||
pub struct Stage<T: Send + Sync> {
|
||||
/// asynchronously, see Figure 4 in the HBBFT paper. Those are N asynchronous
|
||||
/// threads, each responding to values from one particular remote node. The
|
||||
/// 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.
|
||||
pub tx: Arc<Mutex<spmc::Sender<Message<T>>>>,
|
||||
pub tx: &'a channel::Sender<Message<T>>,
|
||||
/// The receive side of the multiple producer channel from comms threads.
|
||||
pub rx: Arc<Mutex<mpsc::Receiver<Message<T>>>>,
|
||||
pub rx: &'a channel::Receiver<Message<T>>,
|
||||
/// Value to be broadcast.
|
||||
pub broadcast_value: Option<T>,
|
||||
}
|
||||
|
||||
impl<T: Clone + Debug + Eq + Hash + Send + Sync + Into<Vec<u8>>
|
||||
impl<'a, T: Clone + Debug + Eq + Hash + Send + Sync + Into<Vec<u8>>
|
||||
+ From<Vec<u8>> + AsRef<[u8]>>
|
||||
Stage<T>
|
||||
Instance<'a, T>
|
||||
where Vec<u8>: From<T>
|
||||
{
|
||||
pub fn new(tx: Arc<Mutex<spmc::Sender<Message<T>>>>,
|
||||
rx: Arc<Mutex<mpsc::Receiver<Message<T>>>>,
|
||||
pub fn new(tx: &'a channel::Sender<Message<T>>,
|
||||
rx: &'a channel::Receiver<Message<T>>,
|
||||
broadcast_value: Option<T>) -> Self
|
||||
{
|
||||
Stage {
|
||||
Instance {
|
||||
tx: tx,
|
||||
rx: rx,
|
||||
broadcast_value: broadcast_value,
|
||||
|
@ -62,12 +60,6 @@ where Vec<u8>: From<T>
|
|||
/// TODO: Detailed error status.
|
||||
pub fn run(&mut self) -> Result<T, BroadcastError> {
|
||||
// Broadcast state machine thread.
|
||||
//
|
||||
// rx cannot be cloned due to its type constraint but can be used inside
|
||||
// a thread with the help of an `Arc` (`Rc` wouldn't work for the same
|
||||
// reason). A `Mutex` is used to grant write access.
|
||||
let rx = self.rx.to_owned();
|
||||
let tx = self.tx.to_owned();
|
||||
let bvalue = self.broadcast_value.to_owned();
|
||||
let result: Result<T, BroadcastError>;
|
||||
let result_r = Arc::new(Mutex::new(None));
|
||||
|
@ -76,7 +68,7 @@ where Vec<u8>: From<T>
|
|||
crossbeam::scope(|scope| {
|
||||
scope.spawn(move || {
|
||||
*result_r_scoped.lock().unwrap() =
|
||||
Some(inner_run(tx, rx, bvalue));
|
||||
Some(inner_run(self.tx, self.rx, bvalue));
|
||||
});
|
||||
});
|
||||
if let Some(ref r) = *result_r.lock().unwrap() {
|
||||
|
@ -98,8 +90,8 @@ pub enum BroadcastError {
|
|||
|
||||
/// Breaks the input value into shards of equal length and encodes them -- and
|
||||
/// some extra parity shards -- with a Reed-Solomon erasure coding scheme.
|
||||
fn send_shards<T>(value: T,
|
||||
tx: Arc<Mutex<spmc::Sender<Message<T>>>>,
|
||||
fn send_shards<'a, T>(value: T,
|
||||
tx: &'a channel::Sender<Message<T>>,
|
||||
coding: &ReedSolomon,
|
||||
data_shard_num: usize,
|
||||
parity_shard_num: usize)
|
||||
|
@ -150,15 +142,15 @@ where T: Clone + Debug + Send + Sync + Into<Vec<u8>>
|
|||
for leaf_value in mtree.iter().cloned() {
|
||||
let proof = mtree.gen_proof(leaf_value);
|
||||
if let Some(proof) = proof {
|
||||
tx.lock().unwrap().send(Message::Broadcast(
|
||||
tx.send(Message::Broadcast(
|
||||
BroadcastMessage::Value(proof))).unwrap();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// The main loop of the broadcast task.
|
||||
fn inner_run<T>(tx: Arc<Mutex<spmc::Sender<Message<T>>>>,
|
||||
rx: Arc<Mutex<mpsc::Receiver<Message<T>>>>,
|
||||
fn inner_run<'a, T>(tx: &'a channel::Sender<Message<T>>,
|
||||
rx: &'a channel::Receiver<Message<T>>,
|
||||
broadcast_value: Option<T>) -> Result<T, BroadcastError>
|
||||
where T: Clone + Debug + Eq + Hash + Send + Sync + Into<Vec<u8>>
|
||||
+ From<Vec<u8>> + AsRef<[u8]>
|
||||
|
@ -175,7 +167,7 @@ where T: Clone + Debug + Eq + Hash + Send + Sync + Into<Vec<u8>>
|
|||
//
|
||||
// FIXME: Does the node send a proof to itself?
|
||||
if let Some(v) = broadcast_value {
|
||||
send_shards(v, tx.clone(), &coding, data_shard_num, parity_shard_num);
|
||||
send_shards(v, tx, &coding, data_shard_num, parity_shard_num);
|
||||
}
|
||||
|
||||
// currently known leaf values
|
||||
|
@ -197,7 +189,7 @@ where T: Clone + Debug + Eq + Hash + Send + Sync + Into<Vec<u8>>
|
|||
// TODO: handle exit conditions
|
||||
while result == None {
|
||||
// Receive a message from the socket IO task.
|
||||
let message = rx.lock().unwrap().recv().unwrap();
|
||||
let message = rx.recv().unwrap();
|
||||
if let Message::Broadcast(message) = message {
|
||||
match message {
|
||||
// A value received. Record the value and multicast an echo.
|
||||
|
@ -220,9 +212,7 @@ where T: Clone + Debug + Eq + Hash + Send + Sync + Into<Vec<u8>>
|
|||
}
|
||||
}
|
||||
// Broadcast an echo of this proof.
|
||||
tx.lock().unwrap()
|
||||
.send(Message::Broadcast(
|
||||
BroadcastMessage::Echo(p)))
|
||||
tx.send(Message::Broadcast(BroadcastMessage::Echo(p)))
|
||||
.unwrap()
|
||||
},
|
||||
|
||||
|
@ -266,7 +256,7 @@ where T: Clone + Debug + Eq + Hash + Send + Sync + Into<Vec<u8>>
|
|||
// Ready
|
||||
if !ready_sent {
|
||||
ready_sent = true;
|
||||
tx.lock().unwrap().send(Message::Broadcast(
|
||||
tx.send(Message::Broadcast(
|
||||
BroadcastMessage::Ready(h.to_owned())))
|
||||
.unwrap();
|
||||
}
|
||||
|
@ -291,7 +281,7 @@ where T: Clone + Debug + Eq + Hash + Send + Sync + Into<Vec<u8>>
|
|||
if (ready_num == PLACEHOLDER_F + 1) &&
|
||||
!ready_sent
|
||||
{
|
||||
tx.lock().unwrap().send(Message::Broadcast(
|
||||
tx.send(Message::Broadcast(
|
||||
BroadcastMessage::Ready(h.to_vec()))).unwrap();
|
||||
}
|
||||
|
||||
|
|
|
@ -3,33 +3,33 @@
|
|||
//! `spmc::channel()` and `mpsc::channel()`.
|
||||
use std::fmt::Debug;
|
||||
use std::sync::{Arc, Mutex};
|
||||
use std::sync::mpsc;
|
||||
use spmc;
|
||||
use crossbeam;
|
||||
use crossbeam_channel as channel;
|
||||
|
||||
use proto::Message;
|
||||
use task;
|
||||
|
||||
/// A communication task connects a remote node to the thread that manages the
|
||||
/// consensus algorithm.
|
||||
pub struct CommsTask<'a, T: Send + Sync + From<Vec<u8>> + Into<Vec<u8>>>
|
||||
pub struct CommsTask<'a, 'b, T: 'a + 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: mpsc::Sender<Message<T>>,
|
||||
tx: &'a channel::Sender<Message<T>>,
|
||||
/// The receive side of the multiple consumer channel to comms threads.
|
||||
rx: spmc::Receiver<Message<T>>,
|
||||
rx: &'a channel::Receiver<Message<T>>,
|
||||
/// The socket IO task.
|
||||
task: task::Task<'a>
|
||||
task: task::Task<'b>
|
||||
}
|
||||
|
||||
impl<'a, T: Debug + Send + Sync + From<Vec<u8>> + Into<Vec<u8>>>
|
||||
CommsTask<'a, T>
|
||||
impl<'a, 'b, T: Debug + Send + Sync + From<Vec<u8>> + Into<Vec<u8>>>
|
||||
CommsTask<'a, 'b, T>
|
||||
where Vec<u8>: From<T>
|
||||
{
|
||||
pub fn new(tx: mpsc::Sender<Message<T>>,
|
||||
rx: spmc::Receiver<Message<T>>,
|
||||
stream: &'a ::std::net::TcpStream) -> Self {
|
||||
pub fn new(tx: &'a channel::Sender<Message<T>>,
|
||||
rx: &'a channel::Receiver<Message<T>>,
|
||||
stream: &'b ::std::net::TcpStream) -> Self {
|
||||
CommsTask {
|
||||
tx: tx,
|
||||
rx: rx,
|
||||
|
@ -41,8 +41,8 @@ where Vec<u8>: From<T>
|
|||
/// thread requests.
|
||||
pub fn run(&mut self) {
|
||||
// Borrow parts of `self` before entering the thread binding scope.
|
||||
let tx = Arc::new(&self.tx);
|
||||
let rx = Arc::new(&self.rx);
|
||||
let tx = Arc::new(self.tx);
|
||||
let rx = Arc::new(self.rx);
|
||||
let task = Arc::new(Mutex::new(&mut self.task));
|
||||
|
||||
crossbeam::scope(|scope| {
|
||||
|
@ -62,7 +62,7 @@ where Vec<u8>: From<T>
|
|||
// Remote comms receive loop.
|
||||
loop {
|
||||
match task.lock().unwrap().receive_message() {
|
||||
Ok(message) => // self.on_message_received(message),
|
||||
Ok(message) =>
|
||||
tx.send(message).unwrap(),
|
||||
Err(task::Error::ProtobufError(e)) =>
|
||||
warn!("Protobuf error {}", e),
|
||||
|
@ -75,10 +75,4 @@ where Vec<u8>: From<T>
|
|||
});
|
||||
|
||||
}
|
||||
|
||||
/// Handler of a received message.
|
||||
fn on_message_received(&mut self, message: Message<T>) {
|
||||
// Forward the message to the manager thread.
|
||||
self.tx.send(message).unwrap();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -2,7 +2,7 @@
|
|||
|
||||
use std::collections::HashSet;
|
||||
use std::fmt::Debug;
|
||||
use std::io::{Read, Write, BufReader};
|
||||
use std::io::BufReader;
|
||||
use std::net::{TcpStream, TcpListener, SocketAddr};
|
||||
|
||||
#[derive(Debug)]
|
||||
|
|
|
@ -42,9 +42,8 @@ extern crate log;
|
|||
extern crate protobuf;
|
||||
extern crate ring;
|
||||
extern crate merkle;
|
||||
//extern crate futures;
|
||||
extern crate spmc;
|
||||
extern crate crossbeam;
|
||||
extern crate crossbeam_channel;
|
||||
extern crate reed_solomon_erasure;
|
||||
|
||||
mod connection;
|
||||
|
|
72
src/node.rs
72
src/node.rs
|
@ -3,11 +3,9 @@ use std::collections::HashSet;
|
|||
use std::fmt::Debug;
|
||||
use std::hash::Hash;
|
||||
use std::marker::{Send, Sync};
|
||||
use std::net::{TcpListener, SocketAddr};
|
||||
use std::sync::{Arc, Mutex};
|
||||
use std::sync::mpsc;
|
||||
use spmc;
|
||||
use std::net::SocketAddr;
|
||||
use crossbeam;
|
||||
use crossbeam_channel as channel;
|
||||
|
||||
use connection;
|
||||
use broadcast;
|
||||
|
@ -39,38 +37,64 @@ where Vec<u8>: From<T>
|
|||
/// Consensus node procedure implementing HoneyBadgerBFT.
|
||||
pub fn run(&self) -> Result<T, ()>
|
||||
{
|
||||
// Multicast channel from the manager task to comms tasks.
|
||||
let (stx, srx): (spmc::Sender<Message<T>>,
|
||||
spmc::Receiver<Message<T>>) = spmc::channel();
|
||||
// Unicast channel from comms tasks to the manager task.
|
||||
let (mtx, mrx): (mpsc::Sender<Message<T>>,
|
||||
mpsc::Receiver<Message<T>>) = mpsc::channel();
|
||||
// Multiple-producer, multiple-consumer channel from comms tasks to
|
||||
// algorithm actor tasks such as Reliable Broadcast.
|
||||
let (from_comms_tx, from_comms_rx):
|
||||
(
|
||||
channel::Sender<Message<T>>,
|
||||
channel::Receiver<Message<T>>
|
||||
) = channel::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 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);
|
||||
|
||||
let broadcast_value = self.value.to_owned();
|
||||
let connections = connection::make(&self.addr, &self.remotes);
|
||||
|
||||
// All spawned threads will have exited by the end of the scope.
|
||||
crossbeam::scope(|scope| {
|
||||
// FIXME: Compute [i <- connections | v_i].
|
||||
|
||||
// Start a comms task for each connection.
|
||||
for c in connections.iter() {
|
||||
info!("Creating a comms task for {:?}",
|
||||
for (i, c) in connections.iter().enumerate() {
|
||||
// FIXME:
|
||||
//
|
||||
// - Connect the comms task to the broadcast instance.
|
||||
//
|
||||
// - Broadcast v_i through the broadcast instance?
|
||||
|
||||
info!("Creating a comms task #{} for {:?}", i,
|
||||
c.stream.peer_addr().unwrap());
|
||||
let tx = mtx.clone();
|
||||
let rx = srx.clone();
|
||||
scope.spawn(move || {
|
||||
commst::CommsTask::new(tx, rx, &c.stream).run();
|
||||
commst::CommsTask::new(from_comms_tx,
|
||||
to_comms_rx,
|
||||
&c.stream)
|
||||
.run();
|
||||
});
|
||||
|
||||
// Associate a broadcast instance to the above comms task.
|
||||
scope.spawn(move || {
|
||||
match broadcast::Instance::new(to_comms_tx,
|
||||
from_comms_rx,
|
||||
// FIXME
|
||||
None)
|
||||
.run()
|
||||
{
|
||||
Ok(_) => debug!("Broadcast instance #{} succeeded", i),
|
||||
Err(_) => error!("Broadcast instance #{} failed", i)
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
// broadcast stage
|
||||
let (tx, rx) = (Arc::new(Mutex::new(stx)),
|
||||
Arc::new(Mutex::new(mrx)));
|
||||
match broadcast::Stage::new(tx, rx, broadcast_value).run() {
|
||||
Ok(_) => debug!("Broadcast stage succeeded"),
|
||||
Err(_) => error!("Broadcast stage failed")
|
||||
}
|
||||
|
||||
// TODO: other stages
|
||||
// TODO: continue the implementation of the asynchronous common
|
||||
// subset algorithm.
|
||||
|
||||
}); // end of thread scope
|
||||
|
||||
|
|
Loading…
Reference in New Issue