Make the coordinator's P2P modules their own crates

This commit is contained in:
Luke Parker
2025-01-09 01:26:25 -05:00
parent adf20773ac
commit 465e8498c4
24 changed files with 234 additions and 63 deletions

View File

@@ -177,6 +177,8 @@ jobs:
cargo msrv verify --manifest-path coordinator/tributary/Cargo.toml
cargo msrv verify --manifest-path coordinator/cosign/Cargo.toml
cargo msrv verify --manifest-path coordinator/substrate/Cargo.toml
cargo msrv verify --manifest-path coordinator/p2p/Cargo.toml
cargo msrv verify --manifest-path coordinator/p2p/libp2p/Cargo.toml
cargo msrv verify --manifest-path coordinator/Cargo.toml
msrv-substrate:

View File

@@ -63,6 +63,8 @@ jobs:
-p tributary-chain \
-p serai-cosign \
-p serai-coordinator-substrate \
-p serai-coordinator-p2p \
-p serai-coordinator-libp2p-p2p \
-p serai-coordinator \
-p serai-orchestrator \
-p serai-docker-tests

44
Cargo.lock generated
View File

@@ -8311,7 +8311,6 @@ dependencies = [
name = "serai-coordinator"
version = "0.1.0"
dependencies = [
"async-trait",
"bitvec",
"blake2",
"borsh",
@@ -8319,9 +8318,7 @@ dependencies = [
"env_logger",
"flexible-transcript",
"frost-schnorrkel",
"futures-util",
"hex",
"libp2p",
"log",
"modular-frost",
"parity-scale-codec",
@@ -8329,6 +8326,9 @@ dependencies = [
"schnorr-signatures",
"schnorrkel",
"serai-client",
"serai-coordinator-libp2p-p2p",
"serai-coordinator-p2p",
"serai-coordinator-substrate",
"serai-cosign",
"serai-db",
"serai-env",
@@ -8337,13 +8337,49 @@ dependencies = [
"serai-task",
"sp-application-crypto",
"sp-runtime",
"tributary-chain",
"zalloc",
"zeroize",
]
[[package]]
name = "serai-coordinator-libp2p-p2p"
version = "0.1.0"
dependencies = [
"async-trait",
"blake2",
"borsh",
"futures-util",
"hex",
"libp2p",
"log",
"rand_core",
"schnorrkel",
"serai-client",
"serai-coordinator-p2p",
"serai-cosign",
"serai-task",
"tokio",
"tributary-chain",
"void",
"zalloc",
"zeroize",
]
[[package]]
name = "serai-coordinator-p2p"
version = "0.1.0"
dependencies = [
"borsh",
"futures-util",
"log",
"serai-client",
"serai-cosign",
"serai-db",
"serai-task",
"tokio",
"tributary-chain",
]
[[package]]
name = "serai-coordinator-substrate"
version = "0.1.0"

View File

@@ -100,6 +100,8 @@ members = [
"coordinator/tributary",
"coordinator/cosign",
"coordinator/substrate",
"coordinator/p2p",
"coordinator/p2p/libp2p",
"coordinator",
"substrate/primitives",

View File

@@ -18,8 +18,6 @@ rustdoc-args = ["--cfg", "docsrs"]
workspace = true
[dependencies]
async-trait = { version = "0.1", default-features = false }
zeroize = { version = "^1.5", default-features = false, features = ["std"] }
bitvec = { version = "1", default-features = false, features = ["std"] }
rand_core = { version = "0.6", default-features = false, features = ["std"] }
@@ -53,12 +51,10 @@ borsh = { version = "1", default-features = false, features = ["std", "derive",
log = { version = "0.4", default-features = false, features = ["std"] }
env_logger = { version = "0.10", default-features = false, features = ["humantime"] }
futures-util = { version = "0.3", default-features = false, features = ["std"] }
tokio = { version = "1", default-features = false, features = ["rt-multi-thread", "sync", "time", "macros"] }
void = { version = "1", default-features = false }
libp2p = { version = "0.52", default-features = false, features = ["tokio", "tcp", "noise", "yamux", "ping", "request-response", "gossipsub", "macros"] }
serai-cosign = { path = "./cosign" }
serai-coordinator-substrate = { path = "./substrate" }
serai-coordinator-p2p = { path = "./p2p" }
serai-coordinator-libp2p-p2p = { path = "./p2p/libp2p" }
[dev-dependencies]
tributary = { package = "tributary-chain", path = "./tributary", features = ["tests"] }

View File

@@ -1,6 +1,6 @@
AGPL-3.0-only license
Copyright (c) 2023-2024 Luke Parker
Copyright (c) 2023-2025 Luke Parker
This program is free software: you can redistribute it and/or modify
it under the terms of the GNU Affero General Public License Version 3 as

View File

@@ -0,0 +1,33 @@
[package]
name = "serai-coordinator-p2p"
version = "0.1.0"
description = "Serai coordinator's P2P abstraction"
license = "AGPL-3.0-only"
repository = "https://github.com/serai-dex/serai/tree/develop/coordinator/p2p"
authors = ["Luke Parker <lukeparker5132@gmail.com>"]
keywords = []
edition = "2021"
publish = false
rust-version = "1.81"
[package.metadata.docs.rs]
all-features = true
rustdoc-args = ["--cfg", "docsrs"]
[lints]
workspace = true
[dependencies]
borsh = { version = "1", default-features = false, features = ["std", "derive", "de_strict_order"] }
serai-db = { path = "../../common/db", version = "0.1" }
serai-client = { path = "../../substrate/client", default-features = false, features = ["serai", "borsh"] }
serai-cosign = { path = "../cosign" }
tributary = { package = "tributary-chain", path = "../tributary" }
futures-util = { version = "0.3", default-features = false, features = ["std"] }
tokio = { version = "1", default-features = false, features = ["sync"] }
log = { version = "0.4", default-features = false, features = ["std"] }
serai-task = { path = "../../common/task", version = "0.1" }

15
coordinator/p2p/LICENSE Normal file
View File

@@ -0,0 +1,15 @@
AGPL-3.0-only license
Copyright (c) 2023-2025 Luke Parker
This program is free software: you can redistribute it and/or modify
it under the terms of the GNU Affero General Public License Version 3 as
published by the Free Software Foundation.
This program is distributed in the hope that it will be useful,
but WITHOUT ANY WARRANTY; without even the implied warranty of
MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
GNU Affero General Public License for more details.
You should have received a copy of the GNU Affero General Public License
along with this program. If not, see <http://www.gnu.org/licenses/>.

View File

@@ -0,0 +1,3 @@
# Serai Coordinator P2P
The P2P abstraction used by Serai's coordinator.

View File

@@ -0,0 +1,43 @@
[package]
name = "serai-coordinator-libp2p-p2p"
version = "0.1.0"
description = "Serai coordinator's libp2p-based P2P backend"
license = "AGPL-3.0-only"
repository = "https://github.com/serai-dex/serai/tree/develop/coordinator/p2p/libp2p"
authors = ["Luke Parker <lukeparker5132@gmail.com>"]
keywords = []
edition = "2021"
publish = false
rust-version = "1.81"
[package.metadata.docs.rs]
all-features = true
rustdoc-args = ["--cfg", "docsrs"]
[lints]
workspace = true
[dependencies]
async-trait = { version = "0.1", default-features = false }
rand_core = { version = "0.6", default-features = false, features = ["std"] }
zeroize = { version = "^1.5", default-features = false, features = ["std"] }
blake2 = { version = "0.10", default-features = false, features = ["std"] }
schnorrkel = { version = "0.11", default-features = false, features = ["std"] }
hex = { version = "0.4", default-features = false, features = ["std"] }
borsh = { version = "1", default-features = false, features = ["std", "derive", "de_strict_order"] }
serai-client = { path = "../../../substrate/client", default-features = false, features = ["serai", "borsh"] }
serai-cosign = { path = "../../cosign" }
tributary = { package = "tributary-chain", path = "../../tributary" }
void = { version = "1", default-features = false }
futures-util = { version = "0.3", default-features = false, features = ["std"] }
tokio = { version = "1", default-features = false, features = ["sync"] }
libp2p = { version = "0.52", default-features = false, features = ["tokio", "tcp", "noise", "yamux", "ping", "request-response", "gossipsub", "macros"] }
log = { version = "0.4", default-features = false, features = ["std"] }
serai-task = { path = "../../../common/task", version = "0.1" }
serai-coordinator-p2p = { path = "../" }

View File

@@ -0,0 +1,15 @@
AGPL-3.0-only license
Copyright (c) 2023-2025 Luke Parker
This program is free software: you can redistribute it and/or modify
it under the terms of the GNU Affero General Public License Version 3 as
published by the Free Software Foundation.
This program is distributed in the hope that it will be useful,
but WITHOUT ANY WARRANTY; without even the implied warranty of
MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
GNU Affero General Public License for more details.
You should have received a copy of the GNU Affero General Public License
along with this program. If not, see <http://www.gnu.org/licenses/>.

View File

@@ -0,0 +1,14 @@
# Serai Coordinator libp2p P2P
A libp2p-backed P2P instantiation for Serai's coordinator.
The libp2p swarm is limited to validators from the Serai network. The swarm
does not maintain any of its own peer finding/routing infrastructure, instead
relying on the Serai network's connection information to dial peers. This does
limit the listening peers to only the peers immediately reachable via the same
IP address (despite the two distinct services), not hidden behind a NAT, yet is
also quite simple and gives full control of who to connect to to us.
Peers are decided via the internal `DialTask` which aims to maintain a target
amount of peers for each external network. This ensures cosigns are able to
propagate across the external networks which sign them.

View File

@@ -17,7 +17,7 @@ use libp2p::{
noise,
};
use crate::p2p::libp2p::peer_id_from_public;
use crate::peer_id_from_public;
const PROTOCOL: &str = "/serai/coordinator/validators";

View File

@@ -14,7 +14,7 @@ use libp2p::{
use serai_task::ContinuallyRan;
use crate::p2p::libp2p::{PORT, Peers, validators::Validators};
use crate::{PORT, Peers, validators::Validators};
const TARGET_PEERS_PER_NETWORK: usize = 5;
/*

View File

@@ -15,8 +15,6 @@ use serai_cosign::SignedCosign;
// Block size limit + 16 KB of space for signatures/metadata
pub(crate) const MAX_LIBP2P_GOSSIP_MESSAGE_SIZE: usize = tributary::BLOCK_SIZE_LIMIT + 16384;
const KEEP_ALIVE_INTERVAL: Duration = Duration::from_secs(80);
const LIBP2P_PROTOCOL: &str = "/serai/coordinator/gossip/1.0.0";
const BASE_TOPIC: &str = "/";

View File

@@ -1,14 +1,6 @@
//! A libp2p-based backend for P2p.
//!
//! The libp2p swarm is limited to validators from the Serai network. The swarm does not maintain
//! any of its own peer finding/routing infrastructure, instead relying on the Serai network's
//! connection information to dial peers. This does limit the listening peers to only the peers
//! immediately reachable via the same IP address (despite the two distinct services), not hidden
//! behind a NAT, yet is also quite simple and gives full control of who to connect to to us.
//!
//! Peers are decided via the `DialTask` which aims to maintain a target amount of peers from each
//! external network.
// TODO: Consider adding that infrastructure, leaving the Serai network solely for bootstrapping
#![cfg_attr(docsrs, feature(doc_auto_cfg))]
#![doc = include_str!("../README.md")]
#![deny(missing_docs)]
use core::{future::Future, time::Duration};
use std::{
@@ -43,7 +35,7 @@ use libp2p::{
SwarmBuilder,
};
use crate::p2p::TributaryBlockWithCommit;
use serai_coordinator_p2p::TributaryBlockWithCommit;
/// A struct to sync the validators from the Serai node in order to keep track of them.
mod validators;
@@ -88,11 +80,12 @@ fn peer_id_from_public(public: PublicKey) -> PeerId {
PeerId::from_multihash(Multihash::wrap(0, &public.0).unwrap()).unwrap()
}
struct Peer<'a> {
/// The representation of a peer.
pub struct Peer<'a> {
outbound_requests: &'a mpsc::UnboundedSender<(PeerId, Request, oneshot::Sender<Response>)>,
id: PeerId,
}
impl crate::p2p::Peer<'_> for Peer<'_> {
impl serai_coordinator_p2p::Peer<'_> for Peer<'_> {
fn send_heartbeat(
&self,
set: ValidatorSet,
@@ -123,6 +116,8 @@ struct Peers {
peers: Arc<RwLock<HashMap<NetworkId, HashSet<PeerId>>>>,
}
// Consider adding identify/kad/autonat/rendevous/(relay + dcutr). While we currently use the Serai
// network for peers, we could use it solely for bootstrapping/as a fallback.
#[derive(NetworkBehaviour)]
struct Behavior {
// Used to only allow Serai validators as peers
@@ -137,8 +132,13 @@ struct Behavior {
gossip: gossip::Behavior,
}
/// The libp2p-backed P2P implementation.
///
/// The P2p trait implementation does not support backpressure and is expected to be fully
/// utilized. Failure to poll the entire API will cause unbounded memory growth.
#[allow(clippy::type_complexity)]
#[derive(Clone)]
struct Libp2p {
pub struct Libp2p {
peers: Peers,
gossip: mpsc::UnboundedSender<Message>,
@@ -155,7 +155,10 @@ struct Libp2p {
}
impl Libp2p {
pub(crate) fn new(serai_key: &Zeroizing<Keypair>, serai: Serai) -> Libp2p {
/// Create a new libp2p-backed P2P instance.
///
/// This will spawn all of the internal tasks necessary for functioning.
pub fn new(serai_key: &Zeroizing<Keypair>, serai: Serai) -> Libp2p {
// Define the object we track peers with
let peers = Peers { peers: Arc::new(RwLock::new(HashMap::new())) };
@@ -320,7 +323,7 @@ impl serai_cosign::RequestNotableCosigns for Libp2p {
}
}
impl crate::p2p::P2p for Libp2p {
impl serai_coordinator_p2p::P2p for Libp2p {
type Peer<'a> = Peer<'a>;
fn peers(&self, network: NetworkId) -> impl Send + Future<Output = Vec<Self::Peer<'_>>> {
@@ -353,6 +356,9 @@ impl crate::p2p::P2p for Libp2p {
tokio::spawn({
let respond = self.inbound_request_responses.clone();
async move {
// The swarm task expects us to respond to every request. If the caller drops this
// channel, we'll receive `Err` and respond with `None`, safely satisfying that bound
// without requiring the caller send a value down this channel
let response =
if let Ok(blocks) = receiver.await { Response::Blocks(blocks) } else { Response::None };
respond

View File

@@ -15,12 +15,12 @@ pub use request_response::{RequestId, Message};
use serai_cosign::SignedCosign;
use crate::p2p::TributaryBlockWithCommit;
use serai_coordinator_p2p::TributaryBlockWithCommit;
/// The maximum message size for the request-response protocol
// This is derived from the heartbeat message size as it's our largest message
pub(crate) const MAX_LIBP2P_REQRES_MESSAGE_SIZE: usize =
(tributary::BLOCK_SIZE_LIMIT * crate::p2p::heartbeat::BLOCKS_PER_BATCH) + 1024;
(tributary::BLOCK_SIZE_LIMIT * serai_coordinator_p2p::heartbeat::BLOCKS_PER_BATCH) + 1024;
const PROTOCOL: &str = "/serai/coordinator/reqres/1.0.0";
@@ -103,7 +103,7 @@ impl CodecTrait for Codec {
}
async fn read_response<R: Send + Unpin + AsyncRead>(
&mut self,
proto: &Self::Protocol,
_: &Self::Protocol,
io: &mut R,
) -> io::Result<Response> {
Self::read(io).await
@@ -118,7 +118,7 @@ impl CodecTrait for Codec {
}
async fn write_response<W: Send + Unpin + AsyncWrite>(
&mut self,
proto: &Self::Protocol,
_: &Self::Protocol,
io: &mut W,
res: Response,
) -> io::Result<()> {

View File

@@ -21,7 +21,7 @@ use libp2p::{
swarm::{dial_opts::DialOpts, SwarmEvent, Swarm},
};
use crate::p2p::libp2p::{
use crate::{
Peers, BehaviorEvent, Behavior,
validators::{self, Validators},
ping,

View File

@@ -13,7 +13,7 @@ use libp2p::PeerId;
use futures_util::stream::{StreamExt, FuturesUnordered};
use tokio::sync::{mpsc, RwLock};
use crate::p2p::libp2p::peer_id_from_public;
use crate::peer_id_from_public;
pub(crate) struct Changes {
pub(crate) removed: HashSet<PeerId>,
@@ -157,10 +157,6 @@ impl Validators {
&self.by_network
}
pub(crate) fn contains(&self, peer_id: &PeerId) -> bool {
self.validators.contains_key(peer_id)
}
pub(crate) fn networks(&self, peer_id: &PeerId) -> Option<&HashSet<NetworkId>> {
self.validators.get(peer_id)
}

View File

@@ -5,20 +5,17 @@ use serai_client::validator_sets::primitives::ValidatorSet;
use futures_util::FutureExt;
use tributary::{ReadWrite, Block, Tributary, TributaryReader};
use tributary::{ReadWrite, TransactionTrait, Block, Tributary, TributaryReader};
use serai_db::*;
use serai_task::ContinuallyRan;
use crate::{
tributary::Transaction,
p2p::{Peer, P2p},
};
use crate::{Peer, P2p};
// Amount of blocks in a minute
const BLOCKS_PER_MINUTE: usize = (60 / (tributary::tendermint::TARGET_BLOCK_TIME / 1000)) as usize;
// Maximum amount of blocks to send in a batch of blocks
/// The maximum amount of blocks to include/included within a batch.
pub const BLOCKS_PER_BATCH: usize = BLOCKS_PER_MINUTE + 1;
/// Sends a heartbeat to other validators on regular intervals informing them of our Tributary's
@@ -26,14 +23,14 @@ pub const BLOCKS_PER_BATCH: usize = BLOCKS_PER_MINUTE + 1;
///
/// If the other validator has more blocks then we do, they're expected to inform us. This forms
/// the sync protocol for our Tributaries.
struct HeartbeatTask<TD: Db, P: P2p> {
pub struct HeartbeatTask<TD: Db, Tx: TransactionTrait, P: P2p> {
set: ValidatorSet,
tributary: Tributary<TD, Transaction, P>,
reader: TributaryReader<TD, Transaction>,
tributary: Tributary<TD, Tx, P>,
reader: TributaryReader<TD, Tx>,
p2p: P,
}
impl<TD: Db, P: P2p> ContinuallyRan for HeartbeatTask<TD, P> {
impl<TD: Db, Tx: TransactionTrait, P: P2p> ContinuallyRan for HeartbeatTask<TD, Tx, P> {
fn run_iteration(&mut self) -> impl Send + Future<Output = Result<bool, String>> {
async move {
// If our blockchain hasn't had a block in the past minute, trigger the heartbeat protocol

View File

@@ -1,3 +1,7 @@
#![cfg_attr(docsrs, feature(doc_auto_cfg))]
#![doc = include_str!("../README.md")]
#![deny(missing_docs)]
use core::future::Future;
use borsh::{BorshSerialize, BorshDeserialize};
@@ -8,20 +12,21 @@ use tokio::sync::oneshot;
use serai_cosign::SignedCosign;
/// The libp2p-backed P2P network
mod libp2p;
/// The heartbeat task, effecting sync of Tributaries
mod heartbeat;
pub mod heartbeat;
/// A tributary block and its commit.
#[derive(Clone, BorshSerialize, BorshDeserialize)]
pub(crate) struct TributaryBlockWithCommit {
pub(crate) block: Vec<u8>,
pub(crate) commit: Vec<u8>,
pub struct TributaryBlockWithCommit {
/// The serialized block.
pub block: Vec<u8>,
/// The serialized commit.
pub commit: Vec<u8>,
}
trait Peer<'a>: Send {
/// A representation of a peer.
pub trait Peer<'a>: Send {
/// Send a heartbeat to this peer.
fn send_heartbeat(
&self,
set: ValidatorSet,
@@ -29,7 +34,9 @@ trait Peer<'a>: Send {
) -> impl Send + Future<Output = Option<Vec<TributaryBlockWithCommit>>>;
}
trait P2p: Send + Sync + tributary::P2p + serai_cosign::RequestNotableCosigns {
/// The representation of the P2P network.
pub trait P2p: Send + Sync + Clone + tributary::P2p + serai_cosign::RequestNotableCosigns {
/// The representation of a peer.
type Peer<'a>: Peer<'a>;
/// Fetch the peers for this network.

View File

@@ -1,5 +1,9 @@
mod tributary;
mod p2p;
mod p2p {
use serai_coordinator_p2p::*;
pub use serai_coordinator_libp2p_p2p::Libp2p;
}
fn main() {
todo!("TODO")

View File

@@ -75,6 +75,8 @@ exceptions = [
{ allow = ["AGPL-3.0"], name = "tributary-chain" },
{ allow = ["AGPL-3.0"], name = "serai-cosign" },
{ allow = ["AGPL-3.0"], name = "serai-coordinator-substrate" },
{ allow = ["AGPL-3.0"], name = "serai-coordinator-p2p" },
{ allow = ["AGPL-3.0"], name = "serai-coordinator-libp2p-p2p" },
{ allow = ["AGPL-3.0"], name = "serai-coordinator" },
{ allow = ["AGPL-3.0"], name = "serai-coins-pallet" },