mirror of
https://github.com/serai-dex/serai.git
synced 2025-12-09 20:59:23 +00:00
Use futures mpsc instead of tokio
This commit is contained in:
1
Cargo.lock
generated
1
Cargo.lock
generated
@@ -8893,6 +8893,7 @@ name = "tendermint-machine"
|
|||||||
version = "0.1.0"
|
version = "0.1.0"
|
||||||
dependencies = [
|
dependencies = [
|
||||||
"async-trait",
|
"async-trait",
|
||||||
|
"futures",
|
||||||
"parity-scale-codec",
|
"parity-scale-codec",
|
||||||
"sp-runtime",
|
"sp-runtime",
|
||||||
"thiserror",
|
"thiserror",
|
||||||
|
|||||||
@@ -8,7 +8,7 @@ use async_trait::async_trait;
|
|||||||
use log::{warn, error};
|
use log::{warn, error};
|
||||||
|
|
||||||
use futures::{
|
use futures::{
|
||||||
StreamExt,
|
SinkExt, StreamExt,
|
||||||
channel::mpsc::{self, UnboundedSender},
|
channel::mpsc::{self, UnboundedSender},
|
||||||
};
|
};
|
||||||
|
|
||||||
@@ -170,7 +170,7 @@ impl<T: TendermintValidator> TendermintAuthority<T> {
|
|||||||
let (gossip_tx, mut gossip_rx) = mpsc::unbounded();
|
let (gossip_tx, mut gossip_rx) = mpsc::unbounded();
|
||||||
|
|
||||||
// Create the Tendermint machine
|
// Create the Tendermint machine
|
||||||
let handle = {
|
let mut handle = {
|
||||||
// Set this struct as active
|
// Set this struct as active
|
||||||
*self.import.providers.write().await = Some(providers);
|
*self.import.providers.write().await = Some(providers);
|
||||||
self.active = Some(ActiveAuthority {
|
self.active = Some(ActiveAuthority {
|
||||||
|
|||||||
@@ -13,6 +13,7 @@ thiserror = "1"
|
|||||||
|
|
||||||
parity-scale-codec = { version = "3.2", features = ["derive"] }
|
parity-scale-codec = { version = "3.2", features = ["derive"] }
|
||||||
|
|
||||||
|
futures = "0.3"
|
||||||
tokio = { version = "1", features = ["macros", "sync", "time", "rt"] }
|
tokio = { version = "1", features = ["macros", "sync", "time", "rt"] }
|
||||||
|
|
||||||
sp-runtime = { git = "https://github.com/serai-dex/substrate", version = "6.0.0", optional = true }
|
sp-runtime = { git = "https://github.com/serai-dex/substrate", version = "6.0.0", optional = true }
|
||||||
|
|||||||
@@ -3,16 +3,14 @@ use core::fmt::Debug;
|
|||||||
use std::{
|
use std::{
|
||||||
sync::Arc,
|
sync::Arc,
|
||||||
time::{UNIX_EPOCH, SystemTime, Instant, Duration},
|
time::{UNIX_EPOCH, SystemTime, Instant, Duration},
|
||||||
collections::HashMap,
|
collections::{VecDeque, HashMap},
|
||||||
};
|
};
|
||||||
|
|
||||||
use parity_scale_codec::{Encode, Decode};
|
use parity_scale_codec::{Encode, Decode};
|
||||||
|
|
||||||
use tokio::{
|
use futures::{task::Poll, StreamExt, channel::mpsc};
|
||||||
task::{JoinHandle, yield_now},
|
|
||||||
sync::mpsc::{self, error::TryRecvError},
|
use tokio::time::sleep;
|
||||||
time::sleep,
|
|
||||||
};
|
|
||||||
|
|
||||||
/// Traits and types of the external network being integrated with to provide consensus over.
|
/// Traits and types of the external network being integrated with to provide consensus over.
|
||||||
pub mod ext;
|
pub mod ext;
|
||||||
@@ -113,10 +111,13 @@ pub struct TendermintMachine<N: Network> {
|
|||||||
start_time: Instant,
|
start_time: Instant,
|
||||||
personal_proposal: N::Block,
|
personal_proposal: N::Block,
|
||||||
|
|
||||||
queue: Vec<(
|
queue: VecDeque<(
|
||||||
bool,
|
bool,
|
||||||
Message<N::ValidatorId, N::Block, <N::SignatureScheme as SignatureScheme>::Signature>,
|
Message<N::ValidatorId, N::Block, <N::SignatureScheme as SignatureScheme>::Signature>,
|
||||||
)>,
|
)>,
|
||||||
|
msg_recv: mpsc::UnboundedReceiver<
|
||||||
|
SignedMessage<N::ValidatorId, N::Block, <N::SignatureScheme as SignatureScheme>::Signature>,
|
||||||
|
>,
|
||||||
|
|
||||||
log: MessageLog<N>,
|
log: MessageLog<N>,
|
||||||
round: Round,
|
round: Round,
|
||||||
@@ -129,15 +130,20 @@ pub struct TendermintMachine<N: Network> {
|
|||||||
timeouts: HashMap<Step, Instant>,
|
timeouts: HashMap<Step, Instant>,
|
||||||
}
|
}
|
||||||
|
|
||||||
/// A handle to an asynchronous task, along with a channel to inform of it of messages received.
|
pub type MessageSender<N> = mpsc::UnboundedSender<
|
||||||
|
SignedMessage<
|
||||||
|
<N as Network>::ValidatorId,
|
||||||
|
<N as Network>::Block,
|
||||||
|
<<N as Network>::SignatureScheme as SignatureScheme>::Signature,
|
||||||
|
>,
|
||||||
|
>;
|
||||||
|
|
||||||
|
/// A Tendermint machine and its channel to receive messages from the gossip layer over.
|
||||||
pub struct TendermintHandle<N: Network> {
|
pub struct TendermintHandle<N: Network> {
|
||||||
/// Channel to send messages received from the P2P layer.
|
/// Channel to send messages received from the P2P layer.
|
||||||
pub messages: mpsc::Sender<
|
pub messages: MessageSender<N>,
|
||||||
SignedMessage<N::ValidatorId, N::Block, <N::SignatureScheme as SignatureScheme>::Signature>,
|
/// Tendermint machine to be run on an asynchronous task.
|
||||||
>,
|
pub machine: TendermintMachine<N>,
|
||||||
/// Handle for the asynchronous task executing the machine. The task will automatically exit
|
|
||||||
/// when the channel is dropped.
|
|
||||||
pub handle: JoinHandle<()>,
|
|
||||||
}
|
}
|
||||||
|
|
||||||
impl<N: Network + 'static> TendermintMachine<N> {
|
impl<N: Network + 'static> TendermintMachine<N> {
|
||||||
@@ -175,7 +181,7 @@ impl<N: Network + 'static> TendermintMachine<N> {
|
|||||||
let step = data.step();
|
let step = data.step();
|
||||||
// 27, 33, 41, 46, 60, 64
|
// 27, 33, 41, 46, 60, 64
|
||||||
self.step = step;
|
self.step = step;
|
||||||
self.queue.push((
|
self.queue.push_back((
|
||||||
true,
|
true,
|
||||||
Message { sender: self.validator_id, number: self.number, round: self.round, data },
|
Message { sender: self.validator_id, number: self.number, round: self.round, data },
|
||||||
));
|
));
|
||||||
@@ -239,14 +245,19 @@ impl<N: Network + 'static> TendermintMachine<N> {
|
|||||||
self.round(Round(0));
|
self.round(Round(0));
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Create a new Tendermint machine, for the specified proposer, from the specified block, with
|
/// Create a new Tendermint machine, from the specified point, with the specified block as the
|
||||||
/// the specified block as the one to propose next, returning a handle for the machine.
|
/// one to propose next. This will return a channel to send messages from the gossip layer and
|
||||||
|
/// the machine itself. The machine should have `run` called from an asynchronous task.
|
||||||
#[allow(clippy::new_ret_no_self)]
|
#[allow(clippy::new_ret_no_self)]
|
||||||
pub fn new(network: N, last: (BlockNumber, u64), proposal: N::Block) -> TendermintHandle<N> {
|
pub async fn new(
|
||||||
let (msg_send, mut msg_recv) = mpsc::channel(100); // Backlog to accept. Currently arbitrary
|
network: N,
|
||||||
|
last: (BlockNumber, u64),
|
||||||
|
proposal: N::Block,
|
||||||
|
) -> TendermintHandle<N> {
|
||||||
|
let (msg_send, msg_recv) = mpsc::unbounded();
|
||||||
TendermintHandle {
|
TendermintHandle {
|
||||||
messages: msg_send,
|
messages: msg_send,
|
||||||
handle: tokio::spawn(async move {
|
machine: {
|
||||||
let last_end = UNIX_EPOCH + Duration::from_secs(last.1);
|
let last_end = UNIX_EPOCH + Duration::from_secs(last.1);
|
||||||
|
|
||||||
// If the last block hasn't ended yet, sleep until it has
|
// If the last block hasn't ended yet, sleep until it has
|
||||||
@@ -271,7 +282,7 @@ impl<N: Network + 'static> TendermintMachine<N> {
|
|||||||
let weights = Arc::new(network.weights());
|
let weights = Arc::new(network.weights());
|
||||||
let validator_id = signer.validator_id().await;
|
let validator_id = signer.validator_id().await;
|
||||||
// 01-10
|
// 01-10
|
||||||
let mut machine = TendermintMachine {
|
TendermintMachine {
|
||||||
network,
|
network,
|
||||||
signer,
|
signer,
|
||||||
validators,
|
validators,
|
||||||
@@ -284,14 +295,15 @@ impl<N: Network + 'static> TendermintMachine<N> {
|
|||||||
// The end time of the last block is the start time for this one
|
// The end time of the last block is the start time for this one
|
||||||
// The Commit explicitly contains the end time, so loading the last commit will provide
|
// The Commit explicitly contains the end time, so loading the last commit will provide
|
||||||
// this. The only exception is for the genesis block, which doesn't have a commit
|
// this. The only exception is for the genesis block, which doesn't have a commit
|
||||||
// Using the genesis time in place will cause this block to be created immediately after
|
// Using the genesis time in place will cause this block to be created immediately
|
||||||
// it, without the standard amount of separation (so their times will be equivalent or
|
// after it, without the standard amount of separation (so their times will be
|
||||||
// minimally offset)
|
// equivalent or minimally offset)
|
||||||
// For callers wishing to avoid this, they should pass (0, GENESIS + BLOCK_TIME)
|
// For callers wishing to avoid this, they should pass (0, GENESIS + BLOCK_TIME)
|
||||||
start_time: last_time,
|
start_time: last_time,
|
||||||
personal_proposal: proposal,
|
personal_proposal: proposal,
|
||||||
|
|
||||||
queue: vec![],
|
queue: VecDeque::new(),
|
||||||
|
msg_recv,
|
||||||
|
|
||||||
log: MessageLog::new(weights),
|
log: MessageLog::new(weights),
|
||||||
round: Round(0),
|
round: Round(0),
|
||||||
@@ -302,95 +314,101 @@ impl<N: Network + 'static> TendermintMachine<N> {
|
|||||||
valid: None,
|
valid: None,
|
||||||
|
|
||||||
timeouts: HashMap::new(),
|
timeouts: HashMap::new(),
|
||||||
};
|
}
|
||||||
machine.round(Round(0));
|
},
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
loop {
|
pub async fn run(mut self) {
|
||||||
// Check if any timeouts have been triggered
|
self.round(Round(0));
|
||||||
let now = Instant::now();
|
|
||||||
let (t1, t2, t3) = {
|
|
||||||
let ready = |step| machine.timeouts.get(&step).unwrap_or(&now) < &now;
|
|
||||||
(ready(Step::Propose), ready(Step::Prevote), ready(Step::Precommit))
|
|
||||||
};
|
|
||||||
|
|
||||||
// Propose timeout
|
'outer: loop {
|
||||||
if t1 && (machine.step == Step::Propose) {
|
// Check if any timeouts have been triggered
|
||||||
machine.broadcast(Data::Prevote(None));
|
let now = Instant::now();
|
||||||
}
|
let (t1, t2, t3) = {
|
||||||
|
let ready = |step| self.timeouts.get(&step).unwrap_or(&now) < &now;
|
||||||
|
(ready(Step::Propose), ready(Step::Prevote), ready(Step::Precommit))
|
||||||
|
};
|
||||||
|
|
||||||
// Prevote timeout
|
// Propose timeout
|
||||||
if t2 && (machine.step == Step::Prevote) {
|
if t1 && (self.step == Step::Propose) {
|
||||||
machine.broadcast(Data::Precommit(None));
|
self.broadcast(Data::Prevote(None));
|
||||||
}
|
}
|
||||||
|
|
||||||
// Precommit timeout
|
// Prevote timeout
|
||||||
if t3 {
|
if t2 && (self.step == Step::Prevote) {
|
||||||
machine.round(Round(machine.round.0.wrapping_add(1)));
|
self.broadcast(Data::Precommit(None));
|
||||||
}
|
}
|
||||||
|
|
||||||
// Drain the channel of messages
|
// Precommit timeout
|
||||||
let mut broken = false;
|
if t3 {
|
||||||
loop {
|
self.round(Round(self.round.0.wrapping_add(1)));
|
||||||
match msg_recv.try_recv() {
|
}
|
||||||
Ok(msg) => {
|
|
||||||
if !msg.verify_signature(&machine.validators) {
|
// Drain the channel of messages
|
||||||
continue;
|
loop {
|
||||||
}
|
match futures::poll!(self.msg_recv.next()) {
|
||||||
machine.queue.push((false, msg.msg));
|
Poll::Ready(Some(msg)) => {
|
||||||
}
|
if !msg.verify_signature(&self.validators) {
|
||||||
Err(TryRecvError::Empty) => break,
|
continue;
|
||||||
Err(TryRecvError::Disconnected) => broken = true,
|
|
||||||
}
|
}
|
||||||
|
self.queue.push_back((false, msg.msg));
|
||||||
}
|
}
|
||||||
if broken {
|
Poll::Ready(None) => {
|
||||||
|
break 'outer;
|
||||||
|
}
|
||||||
|
Poll::Pending => {
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
|
||||||
// Handle the queue
|
|
||||||
let mut queue = machine.queue.drain(..).collect::<Vec<_>>();
|
|
||||||
for (broadcast, msg) in queue.drain(..) {
|
|
||||||
let res = machine.message(msg.clone()).await;
|
|
||||||
if res.is_err() && broadcast {
|
|
||||||
panic!("honest node had invalid behavior");
|
|
||||||
}
|
|
||||||
|
|
||||||
match res {
|
|
||||||
Ok(None) => (),
|
|
||||||
Ok(Some(block)) => {
|
|
||||||
let mut validators = vec![];
|
|
||||||
let mut sigs = vec![];
|
|
||||||
for (v, sig) in machine.log.precommitted.iter().filter_map(|(k, (id, sig))| {
|
|
||||||
Some((*k, sig.clone())).filter(|_| id == &block.id())
|
|
||||||
}) {
|
|
||||||
validators.push(v);
|
|
||||||
sigs.push(sig);
|
|
||||||
}
|
|
||||||
|
|
||||||
let commit = Commit {
|
|
||||||
end_time: machine.canonical_end_time(msg.round),
|
|
||||||
validators,
|
|
||||||
signature: N::SignatureScheme::aggregate(&sigs),
|
|
||||||
};
|
|
||||||
debug_assert!(machine.network.verify_commit(block.id(), &commit));
|
|
||||||
|
|
||||||
let proposal = machine.network.add_block(block, commit).await;
|
|
||||||
machine.reset(msg.round, proposal).await;
|
|
||||||
}
|
|
||||||
Err(TendermintError::Malicious(validator)) => {
|
|
||||||
machine.network.slash(validator).await;
|
|
||||||
}
|
|
||||||
Err(TendermintError::Temporal) => (),
|
|
||||||
}
|
|
||||||
|
|
||||||
if broadcast {
|
|
||||||
let sig = machine.signer.sign(&msg.encode()).await;
|
|
||||||
machine.network.broadcast(SignedMessage { msg, sig }).await;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
yield_now().await;
|
|
||||||
}
|
}
|
||||||
}),
|
}
|
||||||
|
|
||||||
|
// Handle the queue
|
||||||
|
if let Some((broadcast, msg)) = self.queue.pop_front() {
|
||||||
|
let res = self.message(msg.clone()).await;
|
||||||
|
if res.is_err() && broadcast {
|
||||||
|
panic!("honest node had invalid behavior");
|
||||||
|
}
|
||||||
|
|
||||||
|
match res {
|
||||||
|
Ok(None) => (),
|
||||||
|
Ok(Some(block)) => {
|
||||||
|
let mut validators = vec![];
|
||||||
|
let mut sigs = vec![];
|
||||||
|
for (v, sig) in self
|
||||||
|
.log
|
||||||
|
.precommitted
|
||||||
|
.iter()
|
||||||
|
.filter_map(|(k, (id, sig))| Some((*k, sig.clone())).filter(|_| id == &block.id()))
|
||||||
|
{
|
||||||
|
validators.push(v);
|
||||||
|
sigs.push(sig);
|
||||||
|
}
|
||||||
|
|
||||||
|
let commit = Commit {
|
||||||
|
end_time: self.canonical_end_time(msg.round),
|
||||||
|
validators,
|
||||||
|
signature: N::SignatureScheme::aggregate(&sigs),
|
||||||
|
};
|
||||||
|
debug_assert!(self.network.verify_commit(block.id(), &commit));
|
||||||
|
|
||||||
|
let proposal = self.network.add_block(block, commit).await;
|
||||||
|
self.reset(msg.round, proposal).await;
|
||||||
|
}
|
||||||
|
Err(TendermintError::Malicious(validator)) => {
|
||||||
|
self.network.slash(validator).await;
|
||||||
|
}
|
||||||
|
Err(TendermintError::Temporal) => (),
|
||||||
|
}
|
||||||
|
|
||||||
|
if broadcast {
|
||||||
|
let sig = self.signer.sign(&msg.encode()).await;
|
||||||
|
self.network.broadcast(SignedMessage { msg, sig }).await;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// futures::pending here does not work
|
||||||
|
tokio::task::yield_now().await;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@@ -7,9 +7,10 @@ use async_trait::async_trait;
|
|||||||
|
|
||||||
use parity_scale_codec::{Encode, Decode};
|
use parity_scale_codec::{Encode, Decode};
|
||||||
|
|
||||||
|
use futures::SinkExt;
|
||||||
use tokio::{sync::RwLock, time::sleep};
|
use tokio::{sync::RwLock, time::sleep};
|
||||||
|
|
||||||
use tendermint_machine::{ext::*, SignedMessage, TendermintMachine, TendermintHandle};
|
use tendermint_machine::{ext::*, SignedMessage, MessageSender, TendermintMachine, TendermintHandle};
|
||||||
|
|
||||||
type TestValidatorId = u16;
|
type TestValidatorId = u16;
|
||||||
type TestBlockId = [u8; 4];
|
type TestBlockId = [u8; 4];
|
||||||
@@ -93,7 +94,7 @@ impl Block for TestBlock {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
struct TestNetwork(u16, Arc<RwLock<Vec<TendermintHandle<Self>>>>);
|
struct TestNetwork(u16, Arc<RwLock<Vec<MessageSender<Self>>>>);
|
||||||
|
|
||||||
#[async_trait]
|
#[async_trait]
|
||||||
impl Network for TestNetwork {
|
impl Network for TestNetwork {
|
||||||
@@ -117,8 +118,8 @@ impl Network for TestNetwork {
|
|||||||
}
|
}
|
||||||
|
|
||||||
async fn broadcast(&mut self, msg: SignedMessage<TestValidatorId, Self::Block, [u8; 32]>) {
|
async fn broadcast(&mut self, msg: SignedMessage<TestValidatorId, Self::Block, [u8; 32]>) {
|
||||||
for handle in self.1.write().await.iter_mut() {
|
for messages in self.1.write().await.iter_mut() {
|
||||||
handle.messages.send(msg.clone()).await.unwrap();
|
messages.send(msg.clone()).await.unwrap();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -144,17 +145,20 @@ impl Network for TestNetwork {
|
|||||||
}
|
}
|
||||||
|
|
||||||
impl TestNetwork {
|
impl TestNetwork {
|
||||||
async fn new(validators: usize) -> Arc<RwLock<Vec<TendermintHandle<Self>>>> {
|
async fn new(validators: usize) -> Arc<RwLock<Vec<MessageSender<Self>>>> {
|
||||||
let arc = Arc::new(RwLock::new(vec![]));
|
let arc = Arc::new(RwLock::new(vec![]));
|
||||||
{
|
{
|
||||||
let mut write = arc.write().await;
|
let mut write = arc.write().await;
|
||||||
for i in 0 .. validators {
|
for i in 0 .. validators {
|
||||||
let i = u16::try_from(i).unwrap();
|
let i = u16::try_from(i).unwrap();
|
||||||
write.push(TendermintMachine::new(
|
let TendermintHandle { messages, machine } = TendermintMachine::new(
|
||||||
TestNetwork(i, arc.clone()),
|
TestNetwork(i, arc.clone()),
|
||||||
(BlockNumber(1), (SystemTime::now().duration_since(UNIX_EPOCH)).unwrap().as_secs()),
|
(BlockNumber(1), (SystemTime::now().duration_since(UNIX_EPOCH)).unwrap().as_secs()),
|
||||||
TestBlock { id: 1u32.to_le_bytes(), valid: Ok(()) },
|
TestBlock { id: 1u32.to_le_bytes(), valid: Ok(()) },
|
||||||
));
|
)
|
||||||
|
.await;
|
||||||
|
tokio::task::spawn(machine.run());
|
||||||
|
write.push(messages);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
arc
|
arc
|
||||||
|
|||||||
Reference in New Issue
Block a user