Add a LibP2P instantiation to coordinator

It's largely unoptimized, and not yet exclusive to validators, yet has basic
sanity (using message content for ID instead of sender + index).

Fixes bugs as found. Notably, we used a time in milliseconds where the
Tributary expected  seconds.

Also has Tributary::new jump to the presumed round number. This reduces slashes
when starting new chains (whose times will be before the current time) and was
the only way I was able to observe successful confirmations given current
surrounding infrastructure.
This commit is contained in:
Luke Parker
2023-08-08 15:12:47 -04:00
parent 0dd8aed134
commit f6f945e747
18 changed files with 376 additions and 50 deletions

View File

@@ -114,6 +114,8 @@ pub async fn scan_substrate<D: Db, Pro: Processors>(
&mut db,
&key,
|db: &mut D, spec: TributarySpec| {
log::info!("creating new tributary for {:?}", spec.set());
// Save it to the database
MainDb::new(db).add_active_tributary(&spec);
@@ -216,7 +218,17 @@ pub async fn heartbeat_tributaries<D: Db, P: P2p>(
// Only trigger syncing if the block is more than a minute behind
if SystemTime::now() > (block_time + Duration::from_secs(60)) {
log::warn!("last known tributary block was over a minute ago");
P2p::broadcast(&p2p, P2pMessageKind::Heartbeat(tributary.genesis()), tip.to_vec()).await;
let mut msg = tip.to_vec();
// Also include the timestamp so LibP2p doesn't flag this as an old message re-circulating
let timestamp = SystemTime::now()
.duration_since(SystemTime::UNIX_EPOCH)
.expect("system clock is wrong")
.as_secs();
// Divide by the block time so if multiple parties send a Heartbeat, they're more likely to
// overlap
let time_unit = timestamp / u64::from(Tributary::<D, Transaction, P>::block_time());
msg.extend(time_unit.to_le_bytes());
P2p::broadcast(&p2p, P2pMessageKind::Heartbeat(tributary.genesis()), msg).await;
}
}
@@ -240,14 +252,15 @@ pub async fn handle_p2p<D: Db, P: P2p>(
continue;
};
log::trace!("handling message for tributary {:?}", tributary.spec.set());
if tributary.tributary.write().await.handle_message(&msg.msg).await {
P2p::broadcast(&p2p, msg.kind, msg.msg).await;
}
}
// TODO2: Rate limit this per validator
// TODO2: Rate limit this per timestamp
P2pMessageKind::Heartbeat(genesis) => {
if msg.msg.len() != 32 {
if msg.msg.len() != 40 {
log::error!("validator sent invalid heartbeat");
continue;
}
@@ -273,7 +286,7 @@ pub async fn handle_p2p<D: Db, P: P2p>(
// Decide which nodes will respond by using the latest block's hash as a mutually agreed
// upon entropy source
// THis isn't a secure source of entropy, yet it's fine for this
// This isn't a secure source of entropy, yet it's fine for this
let entropy = u64::from_le_bytes(tributary_read.tip().await[.. 8].try_into().unwrap());
// If n = 10, responders = 3, we want start to be 0 ..= 7 (so the highest is 7, 8, 9)
// entropy % (10 + 1) - 3 = entropy % 8 = 0 ..= 7
@@ -298,10 +311,12 @@ pub async fn handle_p2p<D: Db, P: P2p>(
let reader = tributary_read.reader();
drop(tributary_read);
let mut latest = msg.msg.try_into().unwrap();
let mut latest = msg.msg[.. 32].try_into().unwrap();
while let Some(next) = reader.block_after(&latest) {
let mut res = reader.block(&next).unwrap().serialize();
res.extend(reader.commit(&next).unwrap());
// Also include the timestamp used within the Heartbeat
res.extend(&msg.msg[32 .. 40]);
p2p.send(msg.sender, P2pMessageKind::Block(tributary.spec.genesis()), res).await;
latest = next;
}
@@ -315,6 +330,7 @@ pub async fn handle_p2p<D: Db, P: P2p>(
};
// Get just the commit
msg.msg.drain(.. (msg.msg.len() - msg_ref.len()));
msg.msg.drain((msg.msg.len() - 8) ..);
// Spawn a dedicated task to add this block, as it may take a notable amount of time
// While we could use a long-lived task to add each block, that task would only add one
@@ -341,6 +357,10 @@ pub async fn handle_p2p<D: Db, P: P2p>(
return;
};
// TODO: Add a check which doesn't require write to see if this is the next block in
// line
// If it's in the future, hold it for up to T time
let res = tributary.tributary.write().await.sync_block(block, msg.msg).await;
log::debug!("received block from {:?}, sync_block returned {}", msg.sender, res);
}
@@ -699,7 +719,7 @@ async fn main() {
key_bytes.zeroize();
key
};
let p2p = LocalP2p::new(1).swap_remove(0); // TODO
let p2p = LibP2p::new();
let processors = Arc::new(MessageQueue::from_env(Service::Coordinator));

View File

@@ -1,12 +1,29 @@
use core::fmt::Debug;
use std::{sync::Arc, io::Read, collections::VecDeque};
use core::{time::Duration, fmt, task::Poll};
use std::{sync::Arc, collections::VecDeque, io::Read};
use async_trait::async_trait;
use tokio::sync::RwLock;
use tokio::{sync::Mutex, time::sleep};
use libp2p::{
futures::StreamExt,
identity::Keypair,
PeerId, Transport,
core::upgrade,
tcp::{Config, tokio as libp2p_tokio},
noise, yamux,
gossipsub::{
IdentTopic, FastMessageId, MessageId, MessageAuthenticity, ValidationMode, ConfigBuilder,
IdentityTransform, AllowAllSubscriptionFilter, Event as GsEvent, PublishError,
Behaviour as GsBehavior,
},
swarm::{NetworkBehaviour, SwarmBuilder, SwarmEvent, Swarm},
};
pub use tributary::P2p as TributaryP2p;
const LIBP2P_TOPIC: &str = "serai-coordinator";
#[derive(Clone, Copy, PartialEq, Eq, Hash, Debug)]
pub enum P2pMessageKind {
Tributary([u8; 32]),
@@ -67,8 +84,8 @@ pub struct Message<P: P2p> {
}
#[async_trait]
pub trait P2p: Send + Sync + Clone + Debug + TributaryP2p {
type Id: Send + Sync + Clone + Copy + Debug;
pub trait P2p: Send + Sync + Clone + fmt::Debug + TributaryP2p {
type Id: Send + Sync + Clone + Copy + fmt::Debug;
async fn send_raw(&self, to: Self::Id, msg: Vec<u8>);
async fn broadcast_raw(&self, msg: Vec<u8>);
@@ -82,6 +99,7 @@ pub trait P2p: Send + Sync + Clone + Debug + TributaryP2p {
async fn broadcast(&self, kind: P2pMessageKind, msg: Vec<u8>) {
let mut actual_msg = kind.serialize();
actual_msg.extend(msg);
log::trace!("broadcasting p2p message (kind {kind:?})");
self.broadcast_raw(actual_msg).await;
}
async fn receive(&self) -> Message<Self> {
@@ -99,56 +117,187 @@ pub trait P2p: Send + Sync + Clone + Debug + TributaryP2p {
};
break (sender, kind, msg_ref.to_vec());
};
log::trace!("received p2p message (kind {kind:?})");
Message { sender, kind, msg }
}
}
// TODO: Move this to tests
#[allow(clippy::type_complexity)]
#[derive(Clone, Debug)]
pub struct LocalP2p(usize, pub Arc<RwLock<Vec<VecDeque<(usize, Vec<u8>)>>>>);
#[derive(NetworkBehaviour)]
struct Behavior {
gossipsub: GsBehavior,
//#[cfg(debug_assertions)]
mdns: libp2p::mdns::tokio::Behaviour,
}
impl LocalP2p {
pub fn new(validators: usize) -> Vec<LocalP2p> {
let shared = Arc::new(RwLock::new(vec![VecDeque::new(); validators]));
let mut res = vec![];
for i in 0 .. validators {
res.push(LocalP2p(i, shared.clone()));
}
#[allow(clippy::type_complexity)]
#[derive(Clone)]
pub struct LibP2p(Arc<Mutex<Swarm<Behavior>>>, Arc<Mutex<VecDeque<(PeerId, Vec<u8>)>>>);
impl fmt::Debug for LibP2p {
fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result {
fmt.debug_struct("LibP2p").finish_non_exhaustive()
}
}
impl LibP2p {
#[allow(clippy::new_without_default)]
pub fn new() -> Self {
log::info!("creating a libp2p instance");
let throwaway_key_pair = Keypair::generate_ed25519();
let throwaway_peer_id = PeerId::from(throwaway_key_pair.public());
// Uses noise for authentication, yamux for multiplexing
// TODO: Do we want to add a custom authentication protocol to only accept connections from
// fellow validators? Doing so would reduce the potential for spam
let transport = libp2p_tokio::Transport::new(Config::default())
.upgrade(upgrade::Version::V1)
.authenticate(noise::Config::new(&throwaway_key_pair).unwrap())
.multiplex(yamux::Config::default())
.boxed();
let behavior = Behavior {
gossipsub: {
// Block size limit + 1 KB of space for signatures/metadata
const MAX_LIBP2P_MESSAGE_SIZE: usize = tributary::BLOCK_SIZE_LIMIT + 1024;
use blake2::{Digest, Blake2s256};
let config = ConfigBuilder::default()
.max_transmit_size(MAX_LIBP2P_MESSAGE_SIZE)
.validation_mode(ValidationMode::Strict)
// Uses a content based message ID to avoid duplicates as much as possible
.message_id_fn(|msg| {
MessageId::new(&Blake2s256::digest([msg.topic.as_str().as_bytes(), &msg.data].concat()))
})
// Re-defines for fast ID to prevent needing to convert into a Message to run
// message_id_fn
// This function is valid for both
.fast_message_id_fn(|msg| {
FastMessageId::new(&Blake2s256::digest(
[msg.topic.as_str().as_bytes(), &msg.data].concat(),
))
})
.build();
let mut gossipsub = GsBehavior::<IdentityTransform, AllowAllSubscriptionFilter>::new(
MessageAuthenticity::Signed(throwaway_key_pair),
config.unwrap(),
)
.unwrap();
// Uses a single topic to prevent being a BTC validator only connected to ETH validators,
// unable to communicate with other BTC validators
let topic = IdentTopic::new(LIBP2P_TOPIC);
gossipsub.subscribe(&topic).unwrap();
gossipsub
},
// Only use MDNS in debug environments, as it should have no value in a release build
// TODO: We do tests on release binaries as of right now...
//#[cfg(debug_assertions)]
mdns: {
log::info!("spawning mdns");
libp2p::mdns::tokio::Behaviour::new(libp2p::mdns::Config::default(), throwaway_peer_id)
.unwrap()
},
};
let mut swarm =
SwarmBuilder::with_tokio_executor(transport, behavior, throwaway_peer_id).build();
const PORT: u16 = 30563; // 5132 ^ (('c' << 8) | 'o')
swarm.listen_on(format!("/ip4/0.0.0.0/tcp/{PORT}").parse().unwrap()).unwrap();
let res = LibP2p(Arc::new(Mutex::new(swarm)), Arc::new(Mutex::new(VecDeque::new())));
tokio::spawn({
let p2p = res.clone();
async move {
// Run this task ad-infinitum
loop {
// Maintain this lock until it's out of events
let mut p2p_lock = p2p.0.lock().await;
loop {
match futures::poll!(p2p_lock.next()) {
//#[cfg(debug_assertions)]
Poll::Ready(Some(SwarmEvent::Behaviour(BehaviorEvent::Mdns(
libp2p::mdns::Event::Discovered(list),
)))) => {
for (peer, mut addr) in list {
if addr.pop() == Some(libp2p::multiaddr::Protocol::Tcp(PORT)) {
log::info!("found peer via mdns");
p2p_lock.behaviour_mut().gossipsub.add_explicit_peer(&peer);
}
}
}
//#[cfg(debug_assertions)]
Poll::Ready(Some(SwarmEvent::Behaviour(BehaviorEvent::Mdns(
libp2p::mdns::Event::Expired(list),
)))) => {
for (peer, _) in list {
log::info!("disconnecting peer due to mdns");
p2p_lock.behaviour_mut().gossipsub.remove_explicit_peer(&peer);
}
}
Poll::Ready(Some(SwarmEvent::Behaviour(BehaviorEvent::Gossipsub(
GsEvent::Message { propagation_source, message, .. },
)))) => {
p2p.1.lock().await.push_back((propagation_source, message.data));
}
Poll::Ready(Some(_)) => {}
_ => {
drop(p2p_lock);
sleep(Duration::from_millis(100)).await;
break;
}
}
}
}
}
});
res
}
}
#[async_trait]
impl P2p for LocalP2p {
type Id = usize;
impl P2p for LibP2p {
type Id = PeerId;
async fn send_raw(&self, to: Self::Id, msg: Vec<u8>) {
self.1.write().await[to].push_back((self.0, msg));
async fn send_raw(&self, _: Self::Id, msg: Vec<u8>) {
self.broadcast_raw(msg).await;
}
async fn broadcast_raw(&self, msg: Vec<u8>) {
for (i, msg_queue) in self.1.write().await.iter_mut().enumerate() {
if i == self.0 {
continue;
match self
.0
.lock()
.await
.behaviour_mut()
.gossipsub
.publish(IdentTopic::new(LIBP2P_TOPIC), msg.clone())
{
Err(PublishError::SigningError(e)) => panic!("signing error when broadcasting: {e}"),
Err(PublishError::InsufficientPeers) => {
log::warn!("failed to send p2p message due to insufficient peers")
}
msg_queue.push_back((self.0, msg.clone()));
}
Err(PublishError::MessageTooLarge) => {
panic!("tried to send a too large message: {}", hex::encode(msg))
}
Err(PublishError::TransformFailed(e)) => panic!("IdentityTransform failed: {e}"),
Err(PublishError::Duplicate) | Ok(_) => {}
};
}
async fn receive_raw(&self) -> (Self::Id, Vec<u8>) {
// This is a cursed way to implement an async read from a Vec
loop {
if let Some(res) = self.1.write().await[self.0].pop_front() {
if let Some(res) = self.1.lock().await.pop_front() {
return res;
}
tokio::time::sleep(std::time::Duration::from_millis(100)).await;
sleep(Duration::from_millis(100)).await;
}
}
}
#[async_trait]
impl TributaryP2p for LocalP2p {
impl TributaryP2p for LibP2p {
async fn broadcast(&self, genesis: [u8; 32], msg: Vec<u8>) {
<Self as P2p>::broadcast(self, P2pMessageKind::Tributary(genesis), msg).await
}

View File

@@ -72,8 +72,10 @@ async fn handle_new_set<
break res.time().unwrap();
}
};
// The block time is in milliseconds yet the Tributary is in seconds
let time = time / 1000;
let spec = TributarySpec::new(block.hash(), time, set, set_data);
create_new_tributary(db, spec.clone());
create_new_tributary(db, spec.clone()).await;
// Trigger a DKG
// TODO: Check how the processor handles this being fired multiple times

View File

@@ -1,3 +1,4 @@
use core::fmt::Debug;
use std::{
sync::Arc,
collections::{VecDeque, HashMap},
@@ -7,9 +8,14 @@ use serai_client::primitives::NetworkId;
use processor_messages::CoordinatorMessage;
use async_trait::async_trait;
use tokio::sync::RwLock;
use crate::processors::{Message, Processors};
use crate::{
processors::{Message, Processors},
TributaryP2p, P2pMessageKind, P2p,
};
pub mod tributary;
@@ -36,3 +42,53 @@ impl Processors for MemProcessors {
todo!()
}
}
#[allow(clippy::type_complexity)]
#[derive(Clone, Debug)]
pub struct LocalP2p(usize, pub Arc<RwLock<Vec<VecDeque<(usize, Vec<u8>)>>>>);
impl LocalP2p {
pub fn new(validators: usize) -> Vec<LocalP2p> {
let shared = Arc::new(RwLock::new(vec![VecDeque::new(); validators]));
let mut res = vec![];
for i in 0 .. validators {
res.push(LocalP2p(i, shared.clone()));
}
res
}
}
#[async_trait]
impl P2p for LocalP2p {
type Id = usize;
async fn send_raw(&self, to: Self::Id, msg: Vec<u8>) {
self.1.write().await[to].push_back((self.0, msg));
}
async fn broadcast_raw(&self, msg: Vec<u8>) {
for (i, msg_queue) in self.1.write().await.iter_mut().enumerate() {
if i == self.0 {
continue;
}
msg_queue.push_back((self.0, msg.clone()));
}
}
async fn receive_raw(&self) -> (Self::Id, Vec<u8>) {
// This is a cursed way to implement an async read from a Vec
loop {
if let Some(res) = self.1.write().await[self.0].pop_front() {
return res;
}
tokio::time::sleep(std::time::Duration::from_millis(100)).await;
}
}
}
#[async_trait]
impl TributaryP2p for LocalP2p {
async fn broadcast(&self, genesis: [u8; 32], msg: Vec<u8>) {
<Self as P2p>::broadcast(self, P2pMessageKind::Tributary(genesis), msg).await
}
}

View File

@@ -23,8 +23,9 @@ use serai_db::MemDb;
use tributary::{Transaction as TransactionTrait, Tributary};
use crate::{
P2pMessageKind, P2p, LocalP2p,
P2pMessageKind, P2p,
tributary::{Transaction, TributarySpec},
tests::LocalP2p,
};
pub fn new_keys<R: RngCore + CryptoRng>(

View File

@@ -19,10 +19,9 @@ use processor_messages::{
use tributary::{Transaction as TransactionTrait, Tributary};
use crate::{
LocalP2p,
tributary::{TributaryDb, Transaction, TributarySpec, scanner::handle_new_blocks},
tests::{
MemProcessors,
MemProcessors, LocalP2p,
tributary::{new_keys, new_spec, new_tributaries, run_tributaries, wait_for_tx_inclusion},
},
};

View File

@@ -13,8 +13,11 @@ use tributary::Tributary;
use crate::{
tributary::Transaction,
LocalP2p, ActiveTributary, handle_p2p,
tests::tributary::{new_keys, new_spec, new_tributaries},
ActiveTributary, handle_p2p,
tests::{
LocalP2p,
tributary::{new_keys, new_spec, new_tributaries},
},
};
#[tokio::test]

View File

@@ -16,8 +16,11 @@ use tributary::Tributary;
use crate::{
tributary::Transaction,
LocalP2p, ActiveTributary, handle_p2p, heartbeat_tributaries,
tests::tributary::{new_keys, new_spec, new_tributaries},
ActiveTributary, handle_p2p, heartbeat_tributaries,
tests::{
LocalP2p,
tributary::{new_keys, new_spec, new_tributaries},
},
};
#[tokio::test]

View File

@@ -9,9 +9,11 @@ use serai_db::MemDb;
use tributary::{Transaction as TransactionTrait, Tributary};
use crate::{
LocalP2p,
tributary::Transaction,
tests::tributary::{new_keys, new_spec, new_tributaries, run_tributaries, wait_for_tx_inclusion},
tests::{
LocalP2p,
tributary::{new_keys, new_spec, new_tributaries, run_tributaries, wait_for_tx_inclusion},
},
};
#[tokio::test]

View File

@@ -39,6 +39,8 @@ async fn handle_block<D: Db, Pro: Processors>(
spec: &TributarySpec,
block: Block<Transaction>,
) {
log::info!("found block for Tributary {:?}", spec.set());
let genesis = spec.genesis();
let hash = block.hash();