Switch from Carnot to Cryptarchia (#624)

* Switch from Carnot to Cryptarchia

* Do not delete old coins

* Update tests/src/lib.rs

Co-authored-by: davidrusu <davidrusu.me@gmail.com>

* clippy happy

---------

Co-authored-by: davidrusu <davidrusu.me@gmail.com>
This commit is contained in:
Giacomo Pasini 2024-03-28 12:56:04 +01:00 committed by GitHub
parent 16444cf32a
commit 7e4d00cc78
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
86 changed files with 595 additions and 11263 deletions

View File

@ -6,7 +6,6 @@ members = [
"nomos-services/log",
"nomos-services/network",
"nomos-services/storage",
"nomos-services/carnot-consensus",
"nomos-services/cryptarchia-consensus",
"nomos-services/mempool",
"nomos-services/metrics",
@ -19,7 +18,6 @@ members = [
"nomos-utils",
"nodes/nomos-node",
"mixnet",
"simulations",
"consensus/carnot-engine",
"consensus/cryptarchia-engine",
"ledger/cryptarchia-ledger",

View File

@ -30,6 +30,21 @@ pub struct Branch<Id> {
length: u64,
}
impl<Id: Copy> Branch<Id> {
pub fn id(&self) -> Id {
self.id
}
pub fn parent(&self) -> Id {
self.parent
}
pub fn slot(&self) -> Slot {
self.slot
}
pub fn length(&self) -> u64 {
self.length
}
}
impl<Id> Branches<Id>
where
Id: Eq + std::hash::Hash + Copy,

View File

@ -18,6 +18,7 @@ impl From<u32> for Value {
// This implementatio is only a stub
// see https://github.com/logos-co/nomos-specs/blob/master/cryptarchia/cryptarchia.py for a spec
#[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))]
#[derive(Debug, Clone, PartialEq, Eq, Copy)]
pub struct Coin {
sk: SecretKey,
@ -30,6 +31,10 @@ impl Coin {
Self { sk, nonce, value }
}
pub fn value(&self) -> Value {
self.value
}
pub fn evolve(&self) -> Self {
let mut h = Blake2b::new();
h.update("coin-evolve");

View File

@ -77,6 +77,10 @@ impl EpochState {
pub fn epoch(&self) -> Epoch {
self.epoch
}
pub fn total_stake(&self) -> Value {
self.total_stake
}
}
#[derive(Clone, Debug, PartialEq)]
@ -323,7 +327,10 @@ impl LedgerState {
}
}
pub fn from_commitments(commitments: impl IntoIterator<Item = Commitment>) -> Self {
pub fn from_commitments(
commitments: impl IntoIterator<Item = Commitment>,
total_stake: Value,
) -> Self {
let commitments = commitments.into_iter().collect::<HashTrieSet<_>>();
Self {
lead_commitments: commitments.clone(),
@ -335,13 +342,13 @@ impl LedgerState {
epoch: 1.into(),
nonce: [0; 32].into(),
commitments: Default::default(),
total_stake: 1.into(),
total_stake,
},
epoch_state: EpochState {
epoch: 0.into(),
nonce: [0; 32].into(),
commitments: Default::default(),
total_stake: 1.into(),
total_stake,
},
}
}

View File

@ -29,18 +29,15 @@ nomos-mempool = { path = "../../nomos-services/mempool", features = [
] }
nomos-metrics = { path = "../../nomos-services/metrics" }
nomos-storage = { path = "../../nomos-services/storage", features = ["rocksdb"] }
carnot-consensus = { path = "../../nomos-services/carnot-consensus", features = [
"libp2p",
] }
cryptarchia-consensus = { path = "../../nomos-services/cryptarchia-consensus", features = ["libp2p"] }
nomos-libp2p = { path = "../../nomos-libp2p" }
nomos-da = { path = "../../nomos-services/data-availability", features = [
"libp2p",
] }
nomos-system-sig = { path = "../../nomos-services/system-sig" }
tracing-subscriber = "0.3"
carnot-engine = { path = "../../consensus/carnot-engine" }
cryptarchia-engine = { path = "../../consensus/cryptarchia-engine" }
cryptarchia-ledger = { path = "../../ledger/cryptarchia-ledger" }
tokio = { version = "1.24", features = ["sync"] }
serde_json = "1.0"
serde_yaml = "0.9"
@ -54,8 +51,8 @@ utoipa-swagger-ui = { version = "4.0" }
axum = { version = "0.6" }
hyper = { version = "0.14", features = ["full"] }
tower-http = { version = "0.4", features = ["cors", "trace"] }
time = "0.3"
[features]
default = []
mixnet = ["nomos-network/mixnet"]
metrics = []
metrics = []

View File

@ -122,8 +122,14 @@ where
.route("/da/blobs", routing::post(da_blobs))
.route("/cl/metrics", routing::get(cl_metrics::<T>))
.route("/cl/status", routing::post(cl_status::<T>))
.route("/carnot/info", routing::get(carnot_info::<T, S, SIZE>))
.route("/carnot/blocks", routing::get(carnot_blocks::<T, S, SIZE>))
.route(
"/cryptarchia/info",
routing::get(cryptarchia_info::<T, S, SIZE>),
)
.route(
"/cryptarchia/headers",
routing::get(cryptarchia_headers::<T, S, SIZE>),
)
.route("/network/info", routing::get(libp2p_info))
.route("/storage/block", routing::post(block::<S, T>))
.route("/mempool/add/tx", routing::post(add_tx::<T>))
@ -237,25 +243,8 @@ where
{
make_request_and_return_response!(cl::cl_mempool_status::<T>(&handle, items))
}
#[utoipa::path(
get,
path = "/carnot/info",
responses(
(status = 200, description = "Query the carnot information", body = nomos_consensus::CarnotInfo),
(status = 500, description = "Internal server error", body = String),
)
)]
async fn carnot_info<Tx, SS, const SIZE: usize>(State(handle): State<OverwatchHandle>) -> Response
where
Tx: Transaction + Clone + Debug + Hash + Serialize + DeserializeOwned + Send + Sync + 'static,
<Tx as Transaction>::Hash: std::cmp::Ord + Debug + Send + Sync + 'static,
SS: StorageSerde + Send + Sync + 'static,
{
make_request_and_return_response!(consensus::carnot_info::<Tx, SS, SIZE>(&handle))
}
#[derive(Deserialize)]
#[allow(dead_code)]
struct QueryParams {
from: Option<HeaderId>,
to: Option<HeaderId>,
@ -263,23 +252,62 @@ struct QueryParams {
#[utoipa::path(
get,
path = "/carnot/blocks",
path = "/cryptarchia/info",
responses(
(status = 200, description = "Query the block information", body = Vec<consensus_engine::Block>),
(status = 200, description = "Query consensus information", body = nomos_consensus::CryptarchiaInfo),
(status = 500, description = "Internal server error", body = String),
)
)]
async fn carnot_blocks<Tx, SS, const SIZE: usize>(
async fn cryptarchia_info<Tx, SS, const SIZE: usize>(
State(handle): State<OverwatchHandle>,
) -> Response
where
Tx: Transaction
+ Clone
+ Eq
+ Debug
+ Hash
+ Serialize
+ DeserializeOwned
+ Send
+ Sync
+ 'static,
<Tx as Transaction>::Hash: std::cmp::Ord + Debug + Send + Sync + 'static,
SS: StorageSerde + Send + Sync + 'static,
{
make_request_and_return_response!(consensus::cryptarchia_info::<Tx, SS, SIZE>(&handle))
}
#[utoipa::path(
get,
path = "/cryptarchia/headers",
responses(
(status = 200, description = "Query header ids", body = Vec<HeaderId>),
(status = 500, description = "Internal server error", body = String),
)
)]
async fn cryptarchia_headers<Tx, SS, const SIZE: usize>(
State(store): State<OverwatchHandle>,
Query(query): Query<QueryParams>,
) -> Response
where
Tx: Transaction + Clone + Debug + Hash + Serialize + DeserializeOwned + Send + Sync + 'static,
Tx: Transaction
+ Eq
+ Clone
+ Debug
+ Hash
+ Serialize
+ DeserializeOwned
+ Send
+ Sync
+ 'static,
<Tx as Transaction>::Hash: std::cmp::Ord + Debug + Send + Sync + 'static,
SS: StorageSerde + Send + Sync + 'static,
{
let QueryParams { from, to } = query;
make_request_and_return_response!(consensus::carnot_blocks::<Tx, SS, SIZE>(&store, from, to))
make_request_and_return_response!(consensus::cryptarchia_headers::<Tx, SS, SIZE>(
&store, from, to
))
}
#[utoipa::path(

View File

@ -1,14 +1,13 @@
use std::{
net::{IpAddr, SocketAddr},
path::PathBuf,
time::Duration,
};
use crate::api::AxumBackend;
use crate::DataAvailability;
use crate::{Carnot, Tx, Wire, MB16};
use crate::{Tx, Wire, MB16};
use clap::{Parser, ValueEnum};
use color_eyre::eyre::{self, eyre, Result};
use color_eyre::eyre::{eyre, Result};
use hex::FromHex;
use nomos_api::ApiService;
use nomos_libp2p::{secp256k1::SecretKey, Multiaddr};
@ -78,33 +77,12 @@ pub struct HttpArgs {
}
#[derive(Parser, Debug, Clone)]
pub struct ConsensusArgs {
#[clap(long = "consensus-priv-key", env = "CONSENSUS_PRIV_KEY")]
consensus_priv_key: Option<String>,
pub struct CryptarchiaArgs {
#[clap(long = "consensus-chain-start", env = "CONSENSUS_CHAIN_START")]
chain_start_time: Option<i64>,
#[clap(long = "consensus-timeout-secs", env = "CONSENSUS_TIMEOUT_SECS")]
consensus_timeout_secs: Option<String>,
}
#[derive(Parser, Debug, Clone)]
pub struct OverlayArgs {
#[clap(long = "overlay-nodes", env = "OVERLAY_NODES", num_args = 1.., value_delimiter = ',')]
pub overlay_nodes: Option<Vec<String>>,
#[clap(long = "overlay-leader", env = "OVERLAY_LEADER")]
pub overlay_leader: Option<String>,
#[clap(
long = "overlay-number-of-committees",
env = "OVERLAY_NUMBER_OF_COMMITTEES"
)]
pub overlay_number_of_committees: Option<usize>,
#[clap(
long = "overlay-super-majority-threshold",
env = "OVERLAY_SUPER_MAJORITY_THRESHOLD"
)]
pub overlay_super_majority_threshold: Option<f32>,
#[clap(long = "consensus-slot-duration", env = "CONSENSUS_SLOT_DURATION")]
slot_duration: Option<u64>,
}
#[derive(Parser, Debug, Clone)]
@ -124,7 +102,7 @@ pub struct Config {
pub log: <Logger as ServiceData>::Settings,
pub network: <NetworkService<NetworkBackend> as ServiceData>::Settings,
pub http: <ApiService<AxumBackend<Tx, Wire, MB16>> as ServiceData>::Settings,
pub consensus: <Carnot as ServiceData>::Settings,
pub cryptarchia: <crate::Cryptarchia as ServiceData>::Settings,
pub da: <DataAvailability as ServiceData>::Settings,
}
@ -222,56 +200,19 @@ impl Config {
Ok(self)
}
pub fn update_consensus(mut self, consensus_args: ConsensusArgs) -> Result<Self> {
let ConsensusArgs {
consensus_priv_key,
consensus_timeout_secs,
pub fn update_cryptarchia_consensus(mut self, consensus_args: CryptarchiaArgs) -> Result<Self> {
let CryptarchiaArgs {
chain_start_time,
slot_duration,
} = consensus_args;
if let Some(private_key) = consensus_priv_key {
let bytes = <[u8; 32]>::from_hex(private_key)?;
self.consensus.private_key = bytes;
if let Some(start_time) = chain_start_time {
self.cryptarchia.time.chain_start_time =
time::OffsetDateTime::from_unix_timestamp(start_time)?;
}
if let Some(timeout) = consensus_timeout_secs {
let secs = timeout.parse::<u64>()?;
self.consensus.timeout = Duration::from_secs(secs);
}
Ok(self)
}
pub fn update_overlay(mut self, overlay_args: OverlayArgs) -> Result<Self> {
let OverlayArgs {
overlay_nodes,
overlay_leader,
overlay_number_of_committees,
overlay_super_majority_threshold,
} = overlay_args;
if let Some(nodes) = overlay_nodes {
self.consensus.overlay_settings.nodes = nodes
.iter()
.map(|n| {
<[u8; 32]>::from_hex(n)
.map_err(|e| eyre::eyre!("Failed to decode hex: {}", e))
.map(|b| b.into())
})
.collect::<Result<Vec<_>, eyre::Report>>()?;
}
if let Some(leader) = overlay_leader {
let bytes = <[u8; 32]>::from_hex(leader)?;
self.consensus.overlay_settings.current_leader = bytes.into();
}
if let Some(committees) = overlay_number_of_committees {
self.consensus.overlay_settings.number_of_committees = committees;
}
if let Some(super_majority_threshold) = overlay_super_majority_threshold {
self.consensus.overlay_settings.super_majority_threshold =
Some(super_majority_threshold.into());
if let Some(duration) = slot_duration {
self.cryptarchia.time.slot_duration = std::time::Duration::from_secs(duration);
}
Ok(self)

View File

@ -2,8 +2,6 @@ pub mod api;
mod config;
mod tx;
use carnot_consensus::network::adapters::libp2p::Libp2pAdapter as ConsensusNetworkAdapter;
use carnot_engine::overlay::{RandomBeaconState, RoundRobin, TreeOverlay};
use color_eyre::eyre::Result;
use full_replication::Certificate;
use full_replication::{AbsoluteNumber, Attestation, Blob, FullReplication};
@ -12,7 +10,7 @@ use metrics::{backend::map::MapMetricsBackend, types::MetricsData, MetricsServic
use api::AxumBackend;
use bytes::Bytes;
use carnot_consensus::CarnotConsensus;
pub use config::{Config, CryptarchiaArgs, DaArgs, HttpArgs, LogArgs, MetricsArgs, NetworkArgs};
use nomos_api::ApiService;
use nomos_core::{
da::{blob, certificate},
@ -38,10 +36,9 @@ use nomos_storage::{
StorageService,
};
pub use config::{
Config, ConsensusArgs, DaArgs, HttpArgs, LogArgs, MetricsArgs, NetworkArgs, OverlayArgs,
};
use nomos_core::{
#[cfg(feature = "carnot")]
use carnot_engine::overlay::{RandomBeaconState, RoundRobin, TreeOverlay};
pub use nomos_core::{
da::certificate::select::FillSize as FillSizeWithBlobsCertificate,
tx::select::FillSize as FillSizeWithTx,
};
@ -57,8 +54,8 @@ pub const CL_TOPIC: &str = "cl";
pub const DA_TOPIC: &str = "da";
const MB16: usize = 1024 * 1024 * 16;
pub type Carnot = CarnotConsensus<
ConsensusNetworkAdapter,
pub type Cryptarchia = cryptarchia_consensus::CryptarchiaConsensus<
cryptarchia_consensus::network::adapters::libp2p::LibP2pAdapter<Tx, Certificate>,
MockPool<HeaderId, Tx, <Tx as Transaction>::Hash>,
MempoolNetworkAdapter<Tx, <Tx as Transaction>::Hash>,
MockPool<
@ -70,7 +67,6 @@ pub type Carnot = CarnotConsensus<
Certificate,
<<Certificate as certificate::Certificate>::Blob as blob::Blob>::Hash,
>,
TreeOverlay<RoundRobin, RandomBeaconState>,
FillSizeWithTx<MB16, Tx>,
FillSizeWithBlobsCertificate<MB16, Certificate>,
RocksBackend<Wire>,
@ -96,7 +92,7 @@ pub struct Nomos {
CertDiscriminant,
>,
>,
consensus: ServiceHandle<Carnot>,
cryptarchia: ServiceHandle<Cryptarchia>,
http: ServiceHandle<ApiService<AxumBackend<Tx, Wire, MB16>>>,
da: ServiceHandle<DataAvailability>,
storage: ServiceHandle<StorageService<RocksBackend<Wire>>>,

View File

@ -2,8 +2,8 @@ use full_replication::{Blob, Certificate};
#[cfg(feature = "metrics")]
use nomos_metrics::MetricsSettings;
use nomos_node::{
Config, ConsensusArgs, DaArgs, HttpArgs, LogArgs, MetricsArgs, NetworkArgs, Nomos,
NomosServiceSettings, OverlayArgs, Tx,
Config, CryptarchiaArgs, DaArgs, HttpArgs, LogArgs, MetricsArgs, NetworkArgs, Nomos,
NomosServiceSettings, Tx,
};
use clap::Parser;
@ -33,12 +33,8 @@ struct Args {
/// Overrides http config.
#[clap(flatten)]
http_args: HttpArgs,
/// Overrides consensus config.
#[clap(flatten)]
consensus_args: ConsensusArgs,
/// Overrides overlay config.
#[clap(flatten)]
overlay_args: OverlayArgs,
cryptarchia_args: CryptarchiaArgs,
/// Overrides da config.
#[clap(flatten)]
da_args: DaArgs,
@ -54,17 +50,15 @@ fn main() -> Result<()> {
log_args,
http_args,
network_args,
consensus_args,
overlay_args,
cryptarchia_args,
metrics_args,
} = Args::parse();
let config = serde_yaml::from_reader::<_, Config>(std::fs::File::open(config)?)?
.update_da(da_args)?
.update_log(log_args)?
.update_http(http_args)?
.update_consensus(consensus_args)?
.update_overlay(overlay_args)?
.update_network(network_args)?;
.update_network(network_args)?
.update_cryptarchia_consensus(cryptarchia_args)?;
let registry = cfg!(feature = "metrics")
.then(|| {
@ -95,7 +89,7 @@ fn main() -> Result<()> {
},
registry: registry.clone(),
},
consensus: config.consensus,
cryptarchia: config.cryptarchia,
#[cfg(feature = "metrics")]
metrics: MetricsSettings { registry },
da: config.da,

View File

@ -12,7 +12,6 @@ tracing = "0.1"
tracing-subscriber = "0.3"
async-trait = "0.1"
clap = { version = "4", features = ["derive"] }
carnot-engine = { path = "../consensus/carnot-engine" }
serde_yaml = "0.9"
futures = "0.3"
tokio = { version = "1", features = ["sync"] }
@ -22,7 +21,7 @@ nomos-network = { path = "../nomos-services/network", features = ["libp2p"] }
nomos-da = { path = "../nomos-services/data-availability", features = [
"libp2p",
] }
carnot-consensus = { path = "../nomos-services/carnot-consensus" }
cryptarchia-consensus = { path = "../nomos-services/cryptarchia-consensus" }
nomos-log = { path = "../nomos-services/log" }
nomos-libp2p = { path = "../nomos-libp2p" }
nomos-core = { path = "../nomos-core" }

View File

@ -1,26 +1,25 @@
use super::CLIENT;
use carnot_consensus::CarnotInfo;
use carnot_engine::Block;
use cryptarchia_consensus::CryptarchiaInfo;
use nomos_core::header::HeaderId;
use reqwest::Url;
pub async fn carnot_info(node: &Url) -> Result<CarnotInfo, reqwest::Error> {
const NODE_CARNOT_INFO_PATH: &str = "carnot/info";
pub async fn cryptarchia_info(node: &Url) -> Result<CryptarchiaInfo, reqwest::Error> {
const NODE_CRYPTARCHIA_INFO_PATH: &str = "cryptarchia/info";
CLIENT
.get(node.join(NODE_CARNOT_INFO_PATH).unwrap())
.get(node.join(NODE_CRYPTARCHIA_INFO_PATH).unwrap())
.send()
.await?
.json::<CarnotInfo>()
.json::<CryptarchiaInfo>()
.await
}
pub async fn get_blocks_info(
pub async fn get_headers_info(
node: &Url,
from: Option<HeaderId>,
to: Option<HeaderId>,
) -> Result<Vec<Block<HeaderId>>, reqwest::Error> {
const NODE_CARNOT_INFO_PATH: &str = "carnot/blocks";
let mut req = CLIENT.get(node.join(NODE_CARNOT_INFO_PATH).unwrap());
) -> Result<Vec<HeaderId>, reqwest::Error> {
const NODE_CRYPTARCHIA_HEADERS_PATH: &str = "cryptarchia/headers";
let mut req = CLIENT.get(node.join(NODE_CRYPTARCHIA_HEADERS_PATH).unwrap());
if let Some(from) = from {
req = req.query(&[("from", from)]);
}

View File

@ -5,7 +5,7 @@
mod ui;
use crate::{
api::consensus::get_blocks_info,
api::consensus::get_headers_info,
da::{
disseminate::{
DaProtocolChoice, DisseminateApp, DisseminateAppServiceSettings, Settings, Status,
@ -309,10 +309,9 @@ async fn fetch_new_messages(
node: &Url,
) -> Result<(HeaderId, Vec<ChatMessage>), Box<dyn std::error::Error>> {
// By only specifying the 'to' parameter we get all the blocks since the last tip
let mut new_blocks = get_blocks_info(node, None, Some(*last_tip))
let mut new_blocks = get_headers_info(node, None, Some(*last_tip))
.await?
.into_iter()
.map(|block| block.id)
.collect::<Vec<_>>();
// The first block is the most recent one.

View File

@ -12,10 +12,8 @@ async-trait = "0.1"
overwatch-rs = { git = "https://github.com/logos-co/Overwatch", rev = "2f70806" }
overwatch-derive = { git = "https://github.com/logos-co/Overwatch", rev = "ac28d01" }
tracing = "0.1"
carnot-engine = { path = "../../consensus/carnot-engine" }
nomos-core = { path = "../../nomos-core" }
carnot-consensus = { path = "../carnot-consensus" }
cryptarchia-consensus = { path = "../cryptarchia-consensus" }
nomos-network = { path = "../../nomos-services/network" }
nomos-da = { path = "../../nomos-services/data-availability" }
nomos-mempool = { path = "../../nomos-services/mempool", features = [

View File

@ -4,13 +4,10 @@ use overwatch_rs::overwatch::handle::OverwatchHandle;
use serde::{de::DeserializeOwned, Serialize};
use tokio::sync::oneshot;
use carnot_consensus::{
network::adapters::libp2p::Libp2pAdapter as ConsensusNetworkAdapter, CarnotConsensus,
CarnotInfo, ConsensusMsg,
};
use carnot_engine::{
overlay::{RandomBeaconState, RoundRobin, TreeOverlay},
Block,
use crate::http::DynError;
use cryptarchia_consensus::{
network::adapters::libp2p::LibP2pAdapter as ConsensusNetworkAdapter, ConsensusMsg,
CryptarchiaConsensus, CryptarchiaInfo,
};
use full_replication::Certificate;
use nomos_core::{
@ -26,8 +23,8 @@ use nomos_mempool::{
};
use nomos_storage::backends::{rocksdb::RocksBackend, StorageSerde};
pub type Carnot<Tx, SS, const SIZE: usize> = CarnotConsensus<
ConsensusNetworkAdapter,
pub type Cryptarchia<Tx, SS, const SIZE: usize> = CryptarchiaConsensus<
ConsensusNetworkAdapter<Tx, Certificate>,
MockPool<HeaderId, Tx, <Tx as Transaction>::Hash>,
MempoolNetworkAdapter<Tx, <Tx as Transaction>::Hash>,
MockPool<
@ -39,21 +36,32 @@ pub type Carnot<Tx, SS, const SIZE: usize> = CarnotConsensus<
Certificate,
<<Certificate as certificate::Certificate>::Blob as blob::Blob>::Hash,
>,
TreeOverlay<RoundRobin, RandomBeaconState>,
FillSizeWithTx<SIZE, Tx>,
FillSizeWithBlobsCertificate<SIZE, Certificate>,
RocksBackend<SS>,
>;
pub async fn carnot_info<Tx, SS, const SIZE: usize>(
pub async fn cryptarchia_info<Tx, SS, const SIZE: usize>(
handle: &OverwatchHandle,
) -> Result<CarnotInfo, super::DynError>
) -> Result<CryptarchiaInfo, DynError>
where
Tx: Transaction + Clone + Debug + Hash + Serialize + DeserializeOwned + Send + Sync + 'static,
Tx: Transaction
+ Eq
+ Clone
+ Debug
+ Hash
+ Serialize
+ DeserializeOwned
+ Send
+ Sync
+ 'static,
<Tx as Transaction>::Hash: std::cmp::Ord + Debug + Send + Sync + 'static,
SS: StorageSerde + Send + Sync + 'static,
{
let relay = handle.relay::<Carnot<Tx, SS, SIZE>>().connect().await?;
let relay = handle
.relay::<Cryptarchia<Tx, SS, SIZE>>()
.connect()
.await?;
let (sender, receiver) = oneshot::channel();
relay
.send(ConsensusMsg::Info { tx: sender })
@ -63,20 +71,32 @@ where
Ok(receiver.await?)
}
pub async fn carnot_blocks<Tx, SS, const SIZE: usize>(
pub async fn cryptarchia_headers<Tx, SS, const SIZE: usize>(
handle: &OverwatchHandle,
from: Option<HeaderId>,
to: Option<HeaderId>,
) -> Result<Vec<Block<HeaderId>>, super::DynError>
) -> Result<Vec<HeaderId>, DynError>
where
Tx: Transaction + Clone + Debug + Hash + Serialize + DeserializeOwned + Send + Sync + 'static,
Tx: Transaction
+ Clone
+ Debug
+ Eq
+ Hash
+ Serialize
+ DeserializeOwned
+ Send
+ Sync
+ 'static,
<Tx as Transaction>::Hash: std::cmp::Ord + Debug + Send + Sync + 'static,
SS: StorageSerde + Send + Sync + 'static,
{
let relay = handle.relay::<Carnot<Tx, SS, SIZE>>().connect().await?;
let relay = handle
.relay::<Cryptarchia<Tx, SS, SIZE>>()
.connect()
.await?;
let (sender, receiver) = oneshot::channel();
relay
.send(ConsensusMsg::GetBlocks {
.send(ConsensusMsg::GetHeaders {
from,
to,
tx: sender,

View File

@ -0,0 +1,2 @@
mod cryptarchia;
pub use cryptarchia::{cryptarchia_headers, cryptarchia_info};

View File

@ -1,42 +0,0 @@
[package]
name = "carnot-consensus"
version = "0.1.0"
edition = "2021"
# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html
[dependencies]
async-trait = "0.1"
bytes = "1.3"
chrono = "0.4"
carnot-engine = { path = "../../consensus/carnot-engine", features = ["serde"] }
futures = "0.3"
nomos-network = { path = "../network" }
nomos-mempool = { path = "../mempool" }
nomos-core = { path = "../../nomos-core" }
overwatch-rs = { git = "https://github.com/logos-co/Overwatch", rev = "2f70806" }
nomos-storage = { path = "../storage" }
rand_chacha = "0.3"
rand = "0.8"
serde = { version = "1", features = ["derive"] }
thiserror = "1.0"
tokio = { version = "1", features = ["sync"] }
tokio-stream = "0.1"
tokio-util = "0.7"
tracing = "0.1"
bls-signatures = "0.14"
serde_with = "3.0.0"
nomos-libp2p = { path = "../../nomos-libp2p", optional = true }
blake2 = "0.10"
utoipa = { version = "4.0", optional = true }
serde_json = { version = "1", optional = true }
[features]
default = []
mock = ["nomos-network/mock"]
libp2p = ["nomos-network/libp2p", "nomos-libp2p"]
openapi = ["dep:utoipa", "serde_json"]
[dev-dependencies]
serde_json = "1.0.96"

View File

@ -1,56 +0,0 @@
use std::convert::Infallible;
// std
use std::error::Error;
use std::hash::Hash;
// crates
// internal
use crate::TimeoutQc;
use carnot_engine::overlay::{
CommitteeMembership, Error as RandomBeaconError, FreezeMembership, RandomBeaconState,
};
use nomos_core::block::Block;
pub trait UpdateableCommitteeMembership: CommitteeMembership {
type Error: Error;
fn on_new_block_received<Tx: Hash + Clone + Eq, Blob: Clone + Eq + Hash>(
&self,
block: &Block<Tx, Blob>,
) -> Result<Self, Self::Error>;
fn on_timeout_qc_received(&self, qc: &TimeoutQc) -> Result<Self, Self::Error>;
}
impl UpdateableCommitteeMembership for FreezeMembership {
type Error = Infallible;
fn on_new_block_received<Tx: Hash + Clone + Eq, Blob: Clone + Eq + Hash>(
&self,
_block: &Block<Tx, Blob>,
) -> Result<Self, Self::Error> {
Ok(Self)
}
fn on_timeout_qc_received(&self, _qc: &TimeoutQc) -> Result<Self, Self::Error> {
Ok(Self)
}
}
impl UpdateableCommitteeMembership for RandomBeaconState {
type Error = RandomBeaconError;
fn on_new_block_received<Tx: Hash + Clone + Eq, Blob: Clone + Eq + Hash>(
&self,
block: &Block<Tx, Blob>,
) -> Result<Self, Self::Error> {
self.check_advance_happy(
block.header().carnot().beacon().clone(),
block.header().carnot().parent_qc().view(),
)
}
fn on_timeout_qc_received(&self, qc: &TimeoutQc) -> Result<Self, Self::Error> {
Ok(Self::generate_sad(qc.view(), self))
}
}

View File

@ -1,49 +0,0 @@
use crate::TimeoutQc;
use carnot_engine::overlay::RoundRobin;
use carnot_engine::overlay::{Error as RandomBeaconError, LeaderSelection, RandomBeaconState};
use nomos_core::block::Block;
use std::{convert::Infallible, error::Error, hash::Hash};
pub trait UpdateableLeaderSelection: LeaderSelection {
type Error: Error;
fn on_new_block_received<Tx: Hash + Clone + Eq, Blob: Clone + Eq + Hash>(
&self,
block: &Block<Tx, Blob>,
) -> Result<Self, Self::Error>;
fn on_timeout_qc_received(&self, qc: &TimeoutQc) -> Result<Self, Self::Error>;
}
impl UpdateableLeaderSelection for RoundRobin {
type Error = Infallible;
fn on_new_block_received<Tx: Hash + Clone + Eq, Blob: Clone + Eq + Hash>(
&self,
_block: &Block<Tx, Blob>,
) -> Result<Self, Self::Error> {
Ok(self.advance())
}
fn on_timeout_qc_received(&self, _qc: &TimeoutQc) -> Result<Self, Self::Error> {
Ok(self.advance())
}
}
impl UpdateableLeaderSelection for RandomBeaconState {
type Error = RandomBeaconError;
fn on_new_block_received<Tx: Hash + Clone + Eq, Blob: Clone + Eq + Hash>(
&self,
block: &Block<Tx, Blob>,
) -> Result<Self, Self::Error> {
self.check_advance_happy(
block.header().carnot().beacon().clone(),
block.header().carnot().parent_qc().view(),
)
// TODO: check random beacon public keys is leader id
}
fn on_timeout_qc_received(&self, qc: &TimeoutQc) -> Result<Self, Self::Error> {
Ok(Self::generate_sad(qc.view(), self))
}
}

File diff suppressed because it is too large Load Diff

View File

@ -1,381 +0,0 @@
// std
use std::collections::{BTreeMap, HashMap};
use std::ops::DerefMut;
use std::sync::{Arc, Mutex};
// crates
use serde::{Deserialize, Serialize};
use tokio::sync::broadcast::error::RecvError;
use tokio::sync::mpsc::{error::TrySendError, Receiver, Sender};
use tokio_stream::wrappers::ReceiverStream;
// internal
use crate::network::messages::{NewViewMsg, TimeoutMsg, TimeoutQcMsg};
use crate::network::{
messages::{NetworkMessage, ProposalMsg, VoteMsg},
BoxedStream, NetworkAdapter,
};
use carnot_engine::{Committee, CommitteeId, View};
use nomos_core::{header::HeaderId, wire};
use nomos_network::{
backends::libp2p::{Command, Event, EventKind, Libp2p},
NetworkMsg, NetworkService,
};
use overwatch_rs::services::{relay::OutboundRelay, ServiceData};
const TOPIC: &str = "/carnot/proto";
// TODO: this could be tailored per message (e.g. we need to store only a few proposals per view but might need a lot of votes)
const BUFFER_SIZE: usize = 500;
type Relay<T> = OutboundRelay<<NetworkService<T> as ServiceData>::Message>;
/// Due to network effects, latencies, or other factors, it is possible that a node may receive messages
/// out of order, or simply messages that are relevant to future views.
/// Since the implementation only starts listening for a message when it is needed, we need to store
/// messages so that they can be returned when needed.
///
/// Synched nodes can't fall more than a view behind the leader, and in a healthy network we expect the difference
/// between a node's view and the leader's view to be small. Given this, we can limit the size of the cache to a few
/// views and automatically clear it when the node's view is updated.
/// Messages that fall out of the cache (either evicted or never inserted because of view limits) will be discarded and
/// will have to be requested again from the network.
#[derive(Clone)]
struct MessageCache {
// This will always contain VIEW_SIZE_LIMIT consecutive entries
cache: Arc<Mutex<BTreeMap<View, Messages>>>,
}
// This is essentially a synchronization for a single consumer/single producer where the producer must be able to
// buffer messages even if no consumer showed up yet.
// Lock-free thread safe ring buffer exists but haven't found a good implementation for rust yet so let's just use
// channels for now.
struct Spsc<T> {
sender: Sender<T>,
receiver: Option<Receiver<T>>,
}
impl<T> Default for Spsc<T> {
fn default() -> Self {
let (sender, receiver) = tokio::sync::mpsc::channel(BUFFER_SIZE);
Self {
sender,
receiver: Some(receiver),
}
}
}
impl<T> Spsc<T> {
fn recv_or_restore(&mut self) -> Receiver<T> {
match self.receiver.take() {
Some(recv) => recv,
None => {
// somebody already requested the receiver, let's create a new channel
let (sender, receiver) = tokio::sync::mpsc::channel(BUFFER_SIZE);
self.sender = sender;
receiver
}
}
}
fn try_send(&mut self, message: T) {
match self.sender.try_send(message) {
Ok(()) => {}
Err(TrySendError::Closed(message)) => {
let (sender, receiver) = tokio::sync::mpsc::channel(BUFFER_SIZE);
self.sender = sender;
self.receiver = Some(receiver);
self.sender
.try_send(message)
.expect("new channel should be empty");
}
Err(TrySendError::Full(_)) => tracing::error!("full channel, dropping message"),
}
}
}
#[derive(Default)]
struct Messages {
proposal_chunks: Spsc<ProposalMsg>,
votes: HashMap<CommitteeId, HashMap<HeaderId, Spsc<VoteMsg>>>,
new_views: HashMap<CommitteeId, Spsc<NewViewMsg>>,
timeouts: HashMap<CommitteeId, Spsc<TimeoutMsg>>,
timeout_qcs: Spsc<TimeoutQcMsg>,
}
/// Requesting the same stream type multiple times will re-initialize it and new items will only be forwarded to the latest one.
/// It's required for the consumer to keep the stream around for the time it's necessary
#[derive(Clone)]
pub struct Libp2pAdapter {
network_relay: OutboundRelay<<NetworkService<Libp2p> as ServiceData>::Message>,
message_cache: MessageCache,
}
impl MessageCache {
/// The number of views a node will cache messages for, from current_view to current_view + VIEW_SIZE_LIMIT.
/// Messages for views outside [current_view, current_view + VIEW_SIZE_LIMIT] will be discarded.
const VIEW_SIZE_LIMIT: View = View::new(5);
fn new() -> Self {
let cache = (0..Self::VIEW_SIZE_LIMIT.into())
.map(|v| (v.into(), Default::default()))
.collect::<BTreeMap<View, Messages>>();
Self {
cache: Arc::new(Mutex::new(cache)),
}
}
// treat view as the current view
fn advance(mut cache: impl DerefMut<Target = BTreeMap<View, Messages>>, view: View) {
if cache.remove(&(view - 1.into())).is_some() {
cache.insert(view + Self::VIEW_SIZE_LIMIT - 1.into(), Messages::default());
}
}
// This will also advance the cache to use view - 1 as the current view
fn get_proposals(&self, view: View) -> Option<Receiver<ProposalMsg>> {
let mut cache = self.cache.lock().unwrap();
let res = cache
.get_mut(&view)
.map(|m| m.proposal_chunks.recv_or_restore());
Self::advance(cache, view - 1.into());
res
}
// This will also advance the cache to use view as the current view
fn get_timeout_qcs(&self, view: View) -> Option<Receiver<TimeoutQcMsg>> {
let mut cache = self.cache.lock().unwrap();
let res = cache
.get_mut(&view)
.map(|m| m.timeout_qcs.recv_or_restore());
Self::advance(cache, view);
res
}
fn get_votes(
&self,
view: View,
committee_id: CommitteeId,
proposal_id: HeaderId,
) -> Option<Receiver<VoteMsg>> {
self.cache.lock().unwrap().get_mut(&view).map(|m| {
m.votes
.entry(committee_id)
.or_default()
.entry(proposal_id)
.or_default()
.recv_or_restore()
})
}
fn get_new_views(&self, view: View, committee_id: CommitteeId) -> Option<Receiver<NewViewMsg>> {
self.cache.lock().unwrap().get_mut(&view).map(|m| {
m.new_views
.entry(committee_id)
.or_default()
.recv_or_restore()
})
}
fn get_timeouts(&self, view: View, committee_id: CommitteeId) -> Option<Receiver<TimeoutMsg>> {
self.cache.lock().unwrap().get_mut(&view).map(|m| {
m.timeouts
.entry(committee_id)
.or_default()
.recv_or_restore()
})
}
}
/// A message published via libp2p gossipsub.
/// If `to` is [`None`], it means that the `message` is propagated to all committees.
#[derive(Serialize, Deserialize)]
struct GossipsubMessage {
to: Option<CommitteeId>,
message: NetworkMessage,
}
impl GossipsubMessage {
pub fn as_bytes(&self) -> Box<[u8]> {
wire::serialize(self).unwrap().into_boxed_slice()
}
}
impl Libp2pAdapter {
async fn broadcast(&self, message: GossipsubMessage, topic: &str) {
if let Err((e, message)) = self
.network_relay
.send(NetworkMsg::Process(Command::Broadcast {
message: message.as_bytes(),
topic: topic.into(),
}))
.await
{
tracing::error!("error broadcasting {message:?}: {e}");
};
}
async fn subscribe(relay: &Relay<Libp2p>, topic: &str) {
if let Err((e, _)) = relay
.send(NetworkMsg::Process(Command::Subscribe(topic.into())))
.await
{
tracing::error!("error subscribing to {topic}: {e}");
};
}
}
#[async_trait::async_trait]
impl NetworkAdapter for Libp2pAdapter {
type Backend = Libp2p;
async fn new(network_relay: Relay<Self::Backend>) -> Self {
let message_cache = MessageCache::new();
let cache = message_cache.clone();
let relay = network_relay.clone();
Self::subscribe(&relay, TOPIC).await;
tracing::debug!("Starting up...");
// this wait seems to be helpful in some cases since we give the time
// to the network to establish connections before we start sending messages
tokio::time::sleep(std::time::Duration::from_secs(1)).await;
// TODO: maybe we need the runtime handle here?
tokio::spawn(async move {
let (sender, receiver) = tokio::sync::oneshot::channel();
if let Err((e, _)) = relay
.send(NetworkMsg::Subscribe {
kind: EventKind::Message,
sender,
})
.await
{
tracing::error!("error subscribing to incoming messages: {e}");
}
let mut incoming_messages = receiver.await.unwrap();
loop {
match incoming_messages.recv().await {
Ok(Event::Message(message)) => {
match nomos_core::wire::deserialize(&message.data) {
Ok(GossipsubMessage { to, message }) => match message {
NetworkMessage::Proposal(msg) => {
tracing::debug!("received proposal chunk");
let mut cache = cache.cache.lock().unwrap();
let view = msg.view;
if let Some(messages) = cache.get_mut(&view) {
messages.proposal_chunks.try_send(msg);
}
}
NetworkMessage::Vote(msg) => {
tracing::debug!("received vote {:?}", msg);
let mut cache = cache.cache.lock().unwrap();
let view = msg.vote.view;
if let Some(messages) = cache.get_mut(&view) {
messages
.votes
.entry(to.unwrap())
.or_default()
.entry(msg.vote.block)
.or_default()
.try_send(msg);
}
}
NetworkMessage::Timeout(msg) => {
tracing::debug!("received timeout");
let mut cache = cache.cache.lock().unwrap();
let view = msg.vote.view;
if let Some(messages) = cache.get_mut(&view) {
messages
.timeouts
.entry(to.unwrap())
.or_default()
.try_send(msg);
}
}
NetworkMessage::TimeoutQc(msg) => {
tracing::debug!("received timeout_qc");
let mut cache = cache.cache.lock().unwrap();
let view = msg.qc.view();
if let Some(messages) = cache.get_mut(&view) {
messages.timeout_qcs.try_send(msg);
}
}
NetworkMessage::NewView(msg) => {
tracing::debug!("received new_view");
let mut cache = cache.cache.lock().unwrap();
let view = msg.vote.view;
if let Some(messages) = cache.get_mut(&view) {
messages
.new_views
.entry(to.unwrap())
.or_default()
.try_send(msg);
}
}
},
_ => tracing::debug!("unrecognized gossipsub message"),
}
}
Err(RecvError::Lagged(n)) => {
tracing::error!("lagged messages: {n}")
}
Err(RecvError::Closed) => unreachable!(),
}
}
});
Self {
network_relay,
message_cache,
}
}
async fn proposal_chunks_stream(&self, view: View) -> BoxedStream<ProposalMsg> {
self.message_cache
.get_proposals(view)
.map::<BoxedStream<ProposalMsg>, _>(|stream| Box::new(ReceiverStream::new(stream)))
.unwrap_or_else(|| Box::new(tokio_stream::empty()))
}
async fn broadcast(&self, message: NetworkMessage) {
let message = GossipsubMessage { to: None, message };
self.broadcast(message, TOPIC).await;
}
async fn timeout_stream(&self, committee: &Committee, view: View) -> BoxedStream<TimeoutMsg> {
self.message_cache
.get_timeouts(view, committee.id::<blake2::Blake2s256>())
.map::<BoxedStream<TimeoutMsg>, _>(|stream| Box::new(ReceiverStream::new(stream)))
.unwrap_or_else(|| Box::new(tokio_stream::empty()))
}
async fn timeout_qc_stream(&self, view: View) -> BoxedStream<TimeoutQcMsg> {
self.message_cache
.get_timeout_qcs(view)
.map::<BoxedStream<TimeoutQcMsg>, _>(|stream| Box::new(ReceiverStream::new(stream)))
.unwrap_or_else(|| Box::new(tokio_stream::empty()))
}
async fn votes_stream(
&self,
committee: &Committee,
view: View,
proposal_id: HeaderId,
) -> BoxedStream<VoteMsg> {
self.message_cache
.get_votes(view, committee.id::<blake2::Blake2s256>(), proposal_id)
.map::<BoxedStream<VoteMsg>, _>(|stream| Box::new(ReceiverStream::new(stream)))
.unwrap_or_else(|| Box::new(tokio_stream::empty()))
}
async fn new_view_stream(&self, committee: &Committee, view: View) -> BoxedStream<NewViewMsg> {
self.message_cache
.get_new_views(view, committee.id::<blake2::Blake2s256>())
.map::<BoxedStream<NewViewMsg>, _>(|stream| Box::new(ReceiverStream::new(stream)))
.unwrap_or_else(|| Box::new(tokio_stream::empty()))
}
async fn send(&self, message: NetworkMessage, committee: &Committee) {
let message = GossipsubMessage {
to: Some(committee.id::<blake2::Blake2s256>()),
message,
};
self.broadcast(message, TOPIC).await;
}
}

View File

@ -1,152 +0,0 @@
use futures::StreamExt;
use nomos_network::{
backends::mock::{
EventKind, Mock, MockBackendMessage, MockContentTopic, MockMessage, NetworkEvent,
},
NetworkMsg, NetworkService,
};
use overwatch_rs::services::{relay::OutboundRelay, ServiceData};
use tokio_stream::wrappers::BroadcastStream;
use crate::network::messages::{NetworkMessage, NewViewMsg, TimeoutMsg, TimeoutQcMsg};
use crate::network::{
messages::{ProposalMsg, VoteMsg},
BoxedStream, NetworkAdapter,
};
use consensus_engine::{BlockId, Committee, View};
const MOCK_PUB_SUB_TOPIC: &str = "MockPubSubTopic";
const MOCK_BLOCK_CONTENT_TOPIC: MockContentTopic = MockContentTopic::new("MockSim", 1, "MockBlock");
const MOCK_APPROVAL_CONTENT_TOPIC: MockContentTopic =
MockContentTopic::new("MockSim", 1, "MockApproval");
#[derive(Clone)]
pub struct MockAdapter {
network_relay: OutboundRelay<<NetworkService<Mock> as ServiceData>::Message>,
}
impl MockAdapter {
async fn message_subscriber_channel(
&self,
) -> Result<
tokio::sync::broadcast::Receiver<NetworkEvent>,
tokio::sync::oneshot::error::RecvError,
> {
let (sender, receiver) = tokio::sync::oneshot::channel();
if let Err((e, _e)) = self
.network_relay
.send(NetworkMsg::Subscribe {
kind: EventKind::Message,
sender,
})
.await
{
tracing::error!("error subscribing to network messages: {:?}", e);
};
receiver.await
}
}
#[async_trait::async_trait]
impl NetworkAdapter for MockAdapter {
type Backend = Mock;
async fn new(
network_relay: OutboundRelay<<NetworkService<Self::Backend> as ServiceData>::Message>,
) -> Self {
Self { network_relay }
}
async fn proposal_chunks_stream(&self, _view: View) -> BoxedStream<ProposalMsg> {
let stream_channel = self
.message_subscriber_channel()
.await
.map_err(|e| tracing::error!("handle error {e:?}"))
.unwrap();
Box::new(BroadcastStream::new(stream_channel).filter_map(|msg| {
Box::pin(async move {
match msg {
Ok(event) => match event {
NetworkEvent::RawMessage(message) => {
if MOCK_BLOCK_CONTENT_TOPIC.content_topic_name
== message.content_topic().content_topic_name
{
let payload = message.payload();
Some(ProposalMsg::from_bytes(payload.as_bytes()))
} else {
None
}
}
},
Err(_e) => None,
}
})
}))
}
async fn broadcast(&self, message: NetworkMessage) {
self.send(message, &Committee::default()).await
}
#[allow(clippy::diverging_sub_expression)]
async fn timeout_stream(&self, _committee: &Committee, _view: View) -> BoxedStream<TimeoutMsg> {
todo!()
}
#[allow(clippy::diverging_sub_expression)]
async fn timeout_qc_stream(&self, _view: View) -> BoxedStream<TimeoutQcMsg> {
todo!()
}
async fn votes_stream(&self, _: &Committee, _: View, _: BlockId) -> BoxedStream<VoteMsg> {
let stream_channel = self
.message_subscriber_channel()
.await
.unwrap_or_else(|_e| todo!("handle error"));
Box::new(Box::pin(BroadcastStream::new(stream_channel).filter_map(
|msg| async move {
match msg {
Ok(event) => match event {
NetworkEvent::RawMessage(message) => {
if MOCK_APPROVAL_CONTENT_TOPIC.content_topic_name
== message.content_topic().content_topic_name
{
let payload = message.payload();
Some(VoteMsg::from_bytes(payload.as_bytes()))
} else {
None
}
}
},
Err(_e) => None,
}
},
)))
}
#[allow(clippy::diverging_sub_expression)]
async fn new_view_stream(&self, _: &Committee, _view: View) -> BoxedStream<NewViewMsg> {
todo!()
}
async fn send(&self, message: NetworkMessage, _committee: &Committee) {
let message = MockMessage::new(
String::from_utf8_lossy(&message.as_bytes()).to_string(),
MOCK_APPROVAL_CONTENT_TOPIC,
1,
chrono::Utc::now()
.timestamp_nanos_opt()
.expect("timestamp should be in valid range") as usize,
);
if let Err((e, _e)) = self
.network_relay
.send(NetworkMsg::Process(MockBackendMessage::Broadcast {
msg: message,
topic: MOCK_PUB_SUB_TOPIC.to_string(),
}))
.await
{
tracing::error!("Failed to forward approval: {:?}", e);
};
}
}

View File

@ -1,4 +0,0 @@
#[cfg(feature = "libp2p")]
pub mod libp2p;
#[cfg(feature = "mock")]
pub mod mock;

View File

@ -1,105 +0,0 @@
// std
// crates
use serde::{Deserialize, Serialize};
// internal
use crate::NodeId;
use crate::{NewView, Qc, Timeout, TimeoutQc, Vote};
use carnot_engine::View;
use nomos_core::header::HeaderId;
use nomos_core::wire;
#[derive(Clone, Serialize, Deserialize, Debug, Eq, PartialEq, Hash)]
pub struct ProposalMsg {
pub data: Box<[u8]>,
pub proposal: HeaderId,
pub view: View,
}
impl ProposalMsg {
pub fn as_bytes(&self) -> Box<[u8]> {
wire::serialize(self).unwrap().into_boxed_slice()
}
pub fn from_bytes(data: &[u8]) -> Self {
wire::deserialize(data).unwrap()
}
}
#[derive(Debug, Eq, PartialEq, Hash, Serialize, Deserialize, Clone)]
pub struct VoteMsg {
pub voter: NodeId,
pub vote: Vote,
pub qc: Option<Qc>,
}
impl VoteMsg {
pub fn as_bytes(&self) -> Box<[u8]> {
wire::serialize(self).unwrap().into_boxed_slice()
}
pub fn from_bytes(data: &[u8]) -> Self {
wire::deserialize(data).unwrap()
}
}
#[derive(Debug, Serialize, Deserialize, Hash, PartialEq, Eq, Clone)]
pub struct NewViewMsg {
pub voter: NodeId,
pub vote: NewView,
}
impl NewViewMsg {
pub fn as_bytes(&self) -> Box<[u8]> {
wire::serialize(self).unwrap().into_boxed_slice()
}
pub fn from_bytes(data: &[u8]) -> Self {
wire::deserialize(data).unwrap()
}
}
#[derive(Debug, Serialize, Deserialize, Hash, PartialEq, Eq, Clone)]
pub struct TimeoutMsg {
pub voter: NodeId,
pub vote: Timeout,
}
impl TimeoutMsg {
pub fn as_bytes(&self) -> Box<[u8]> {
wire::serialize(self).unwrap().into_boxed_slice()
}
pub fn from_bytes(data: &[u8]) -> Self {
wire::deserialize(data).unwrap()
}
}
#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Hash, Clone)]
pub struct TimeoutQcMsg {
pub source: NodeId,
pub qc: TimeoutQc,
}
impl TimeoutQcMsg {
pub fn as_bytes(&self) -> Box<[u8]> {
wire::serialize(self).unwrap().into_boxed_slice()
}
pub fn from_bytes(data: &[u8]) -> Self {
wire::deserialize(data).unwrap()
}
}
#[derive(Debug, Serialize, Deserialize)]
pub enum NetworkMessage {
Timeout(TimeoutMsg),
TimeoutQc(TimeoutQcMsg),
Vote(VoteMsg),
NewView(NewViewMsg),
Proposal(ProposalMsg),
}
impl NetworkMessage {
pub fn as_bytes(&self) -> Box<[u8]> {
wire::serialize(self).unwrap().into_boxed_slice()
}
pub fn from_bytes(data: &[u8]) -> Self {
wire::deserialize(data).unwrap()
}
}

View File

@ -1,41 +0,0 @@
pub mod adapters;
pub mod messages;
// std
// crates
use futures::Stream;
use nomos_core::header::HeaderId;
// internal
use crate::network::messages::{
NetworkMessage, NewViewMsg, ProposalMsg, TimeoutMsg, TimeoutQcMsg, VoteMsg,
};
use carnot_engine::{Committee, View};
use nomos_network::backends::NetworkBackend;
use nomos_network::NetworkService;
use overwatch_rs::services::relay::OutboundRelay;
use overwatch_rs::services::ServiceData;
type BoxedStream<T> = Box<dyn Stream<Item = T> + Send + Sync + Unpin>;
#[async_trait::async_trait]
pub trait NetworkAdapter {
type Backend: NetworkBackend + 'static;
async fn new(
network_relay: OutboundRelay<<NetworkService<Self::Backend> as ServiceData>::Message>,
) -> Self;
async fn proposal_chunks_stream(
&self,
view: View,
) -> Box<dyn Stream<Item = ProposalMsg> + Send + Sync + Unpin>;
async fn broadcast(&self, message: NetworkMessage);
async fn timeout_stream(&self, committee: &Committee, view: View) -> BoxedStream<TimeoutMsg>;
async fn timeout_qc_stream(&self, view: View) -> BoxedStream<TimeoutQcMsg>;
async fn votes_stream(
&self,
committee: &Committee,
view: View,
proposal_id: HeaderId,
) -> BoxedStream<VoteMsg>;
async fn new_view_stream(&self, committee: &Committee, view: View) -> BoxedStream<NewViewMsg>;
async fn send(&self, message: NetworkMessage, committee: &Committee);
}

View File

@ -1,89 +0,0 @@
#![allow(dead_code)]
// TODO: Well, remove this when we actually use the fields from the specification
// std
use std::collections::HashSet;
// crates
use futures::{Stream, StreamExt};
use nomos_core::header::HeaderId;
// internal
use super::CarnotTallySettings;
use crate::network::messages::VoteMsg;
use crate::{Qc, StandardQc, Vote};
use nomos_core::crypto::PublicKey;
use nomos_core::vote::Tally;
pub type NodeId = PublicKey;
type Block = carnot_engine::Block<HeaderId>;
#[derive(thiserror::Error, Debug)]
pub enum CarnotTallyError {
#[error("Received invalid vote: {0}")]
InvalidVote(String),
#[error("Did not receive enough votes")]
InsufficientVotes,
#[error("The vote stream ended without tally")]
StreamEnded,
}
#[derive(Clone, Debug)]
pub struct CarnotTally {
settings: CarnotTallySettings,
}
#[async_trait::async_trait]
impl Tally for CarnotTally {
type Vote = VoteMsg;
type Qc = Qc;
type Subject = Block;
type Outcome = HashSet<Vote>;
type TallyError = CarnotTallyError;
type Settings = CarnotTallySettings;
fn new(settings: Self::Settings) -> Self {
Self { settings }
}
async fn tally<S: Stream<Item = Self::Vote> + Unpin + Send>(
&self,
block: Block,
mut vote_stream: S,
) -> Result<(Self::Qc, Self::Outcome), Self::TallyError> {
let mut seen = HashSet::new();
let mut outcome = HashSet::new();
// return early for leaf nodes
if self.settings.threshold == 0 {
return Ok((
Qc::Standard(StandardQc {
view: block.view,
id: block.id,
}),
outcome,
));
}
while let Some(vote) = vote_stream.next().await {
// check vote view is valid
if vote.vote.view != block.view || vote.vote.block != block.id {
continue;
}
// check for individual nodes votes
if !self.settings.participating_nodes.contains(&vote.voter) {
continue;
}
seen.insert(vote.voter);
outcome.insert(vote.vote.clone());
if seen.len() >= self.settings.threshold {
return Ok((
Qc::Standard(StandardQc {
view: vote.vote.view,
id: vote.vote.block,
}),
outcome,
));
}
}
Err(CarnotTallyError::StreamEnded)
}
}

View File

@ -1,18 +0,0 @@
pub mod happy;
pub mod timeout;
pub mod unhappy;
// std
// crates
use serde::{Deserialize, Serialize};
// internal
use carnot_engine::Committee;
#[derive(Clone, Debug, Serialize, Deserialize)]
pub struct CarnotTallySettings {
pub threshold: usize,
// TODO: this probably should be dynamic and should change with the view (?)
pub participating_nodes: Committee,
}

View File

@ -1,56 +0,0 @@
// std
use std::{collections::HashSet, convert::Infallible};
// crates
use futures::{Stream, StreamExt};
// internal
use super::CarnotTallySettings;
use crate::network::messages::TimeoutMsg;
use crate::Timeout;
use carnot_engine::View;
use nomos_core::vote::Tally;
#[derive(Clone, Debug)]
pub struct TimeoutTally {
settings: CarnotTallySettings,
}
#[async_trait::async_trait]
impl Tally for TimeoutTally {
type Vote = TimeoutMsg;
type Qc = ();
type Subject = View;
type Outcome = HashSet<Timeout>;
type TallyError = Infallible;
type Settings = CarnotTallySettings;
fn new(settings: Self::Settings) -> Self {
Self { settings }
}
async fn tally<S: Stream<Item = Self::Vote> + Unpin + Send>(
&self,
view: View,
mut vote_stream: S,
) -> Result<(Self::Qc, Self::Outcome), Self::TallyError> {
let mut seen = HashSet::new();
let mut outcome = HashSet::new();
while let Some(vote) = vote_stream.next().await {
// check timeout view is valid
if vote.vote.view != view {
continue;
}
// check for individual nodes votes
if !self.settings.participating_nodes.contains(&vote.voter) {
continue;
}
seen.insert(vote.voter);
outcome.insert(vote.vote.clone());
if seen.len() >= self.settings.threshold {
return Ok(((), outcome));
}
}
unreachable!()
}
}

View File

@ -1,67 +0,0 @@
// std
use std::collections::HashSet;
// crates
use futures::{Stream, StreamExt};
use serde::{Deserialize, Serialize};
// internal
use super::CarnotTallySettings;
use crate::network::messages::NewViewMsg;
use nomos_core::vote::Tally;
use crate::{NewView, TimeoutQc};
#[derive(thiserror::Error, Debug)]
pub enum NewViewTallyError {
#[error("Did not receive enough votes")]
InsufficientVotes,
}
#[derive(Clone, Debug, Serialize, Deserialize)]
pub struct NewViewTally {
settings: CarnotTallySettings,
}
#[async_trait::async_trait]
impl Tally for NewViewTally {
type Vote = NewViewMsg;
type Qc = ();
type Subject = TimeoutQc;
type Outcome = HashSet<NewView>;
type TallyError = NewViewTallyError;
type Settings = CarnotTallySettings;
fn new(settings: Self::Settings) -> Self {
Self { settings }
}
async fn tally<S: Stream<Item = Self::Vote> + Unpin + Send>(
&self,
timeout_qc: TimeoutQc,
mut vote_stream: S,
) -> Result<(Self::Qc, Self::Outcome), Self::TallyError> {
let mut seen = HashSet::new();
let mut outcome = HashSet::new();
// return early for leaf nodes
if self.settings.threshold == 0 {
return Ok(((), outcome));
}
while let Some(vote) = vote_stream.next().await {
// check vote view is valid
if vote.vote.view != timeout_qc.view().next() {
continue;
}
// check for individual nodes votes
if !self.settings.participating_nodes.contains(&vote.voter) {
continue;
}
seen.insert(vote.voter);
outcome.insert(vote.vote.clone());
if seen.len() >= self.settings.threshold {
return Ok(((), outcome));
}
}
Err(NewViewTallyError::InsufficientVotes)
}
}

View File

@ -1,121 +0,0 @@
use futures::{stream::FuturesUnordered, Future, Stream, StreamExt};
use std::collections::HashMap;
use std::hash::Hash;
use std::pin::Pin;
use tokio::select;
use tokio_util::sync::CancellationToken;
pub struct TaskManager<Group, Out> {
tasks: FuturesUnordered<Pin<Box<dyn Future<Output = Option<Out>> + Send>>>,
cancel_cache: CancelCache<Group>,
}
impl<Group, Out> TaskManager<Group, Out>
where
Group: Eq + PartialEq + Hash + 'static,
Out: 'static,
{
pub fn new() -> Self {
Self {
tasks: FuturesUnordered::new(),
cancel_cache: CancelCache::new(),
}
}
pub fn push(&mut self, group: Group, task: impl Future<Output = Out> + Send + 'static) {
self.tasks.push(Box::pin(
self.cancel_cache.cancelable_event_future(group, task),
));
}
pub fn cancel(&mut self, group: Group) {
self.cancel_cache.cancel(group);
}
}
impl<Group: Unpin, Out> Stream for TaskManager<Group, Out> {
type Item = Out;
fn poll_next(
self: Pin<&mut Self>,
cx: &mut std::task::Context<'_>,
) -> std::task::Poll<Option<Self::Item>> {
use std::task::Poll;
let tasks = &mut self.get_mut().tasks;
loop {
match tasks.poll_next_unpin(cx) {
// we need to remove the outer Option that was inserted by the cancelabl future
Poll::Ready(Some(Some(event))) => return Poll::Ready(Some(event)),
// an empty output means the task was cancelled, ignore it
Poll::Ready(Some(None)) => {}
Poll::Ready(None) => return Poll::Ready(None),
Poll::Pending => return Poll::Pending,
}
}
}
}
pub struct GroupCancel(CancellationToken);
impl GroupCancel {
pub fn new() -> Self {
Self(CancellationToken::new())
}
pub fn cancel(&self) {
self.0.cancel();
}
pub fn cancel_token(&self) -> CancellationToken {
self.0.clone()
}
}
impl Drop for GroupCancel {
fn drop(&mut self) {
if !self.0.is_cancelled() {
self.cancel();
}
}
}
pub struct CancelCache<Group> {
cancels: HashMap<Group, GroupCancel>,
}
impl<Group: Eq + PartialEq + Hash> CancelCache<Group> {
pub fn new() -> Self {
Self {
cancels: HashMap::new(),
}
}
pub fn cancel(&mut self, group: Group) {
if let Some(cancel) = self.cancels.remove(&group) {
cancel.cancel();
}
}
pub fn cancel_token(&mut self, group: Group) -> CancellationToken {
self.cancels
.entry(group)
.or_insert_with(GroupCancel::new)
.cancel_token()
}
pub(crate) fn cancelable_event_future<Out, F: Future<Output = Out>>(
&mut self,
group: Group,
f: F,
) -> impl Future<Output = Option<Out>> {
let token = self.cancel_token(group);
async move {
select! {
event = f => Some(event),
_ = token.cancelled() => None,
}
}
}
}

View File

@ -1,125 +0,0 @@
use crate::network::messages::*;
use crate::overlay::committees::*;
use crate::overlay::*;
use crate::*;
use async_trait::async_trait;
use bytes::Bytes;
use futures::Stream;
use nomos_core::fountain::FountainError;
use nomos_core::fountain::{mock::MockFountain, FountainCode};
use nomos_core::vote::mock::{MockQc, MockTally, MockTallySettings};
use nomos_network::backends::NetworkBackend;
use nomos_network::NetworkService;
use overwatch_rs::services::relay::*;
use serde::de::DeserializeOwned;
use tokio::sync::broadcast::Receiver;
struct DummyOverlay;
struct DummyAdapter;
struct DummyBackend;
#[async_trait]
impl<N: NetworkAdapter + Sync, F: FountainCode + Sync> Overlay<N, F, MockTally> for DummyOverlay {
fn new(_: &View, _: NodeId) -> Self {
DummyOverlay
}
async fn reconstruct_proposal_block(
&self,
_view: &View,
_adapter: &N,
_fountain: &F,
) -> Result<Block, FountainError> {
Ok(Block)
}
async fn broadcast_block(&self, _view: &View, _block: Block, _adapter: &N, _fountain: &F) {}
async fn approve_and_forward(
&self,
_view: &View,
_block: &Block,
_adapter: &N,
_vote_tally: &MockTally,
_next_view: &View,
) -> Result<(), Box<dyn Error>> {
Ok(())
}
async fn build_qc(&self, view: &View, _adapter: &N, _vote_tally: &MockTally) -> MockQc {
// TODO: mock the total votes
MockQc::new(0)
}
}
#[async_trait]
impl NetworkAdapter for DummyAdapter {
type Backend = DummyBackend;
async fn new(
_: OutboundRelay<<NetworkService<Self::Backend> as ServiceData>::Message>,
) -> Self {
DummyAdapter
}
async fn proposal_chunks_stream(
&self,
_: Committee,
_: &View,
) -> Box<dyn Stream<Item = Bytes> + Send + Sync + Unpin> {
unimplemented!()
}
async fn broadcast_block_chunk(&self, _: Committee, _: &View, _: ProposalChunkMsg) {
unimplemented!()
}
async fn votes_stream<Vote: DeserializeOwned>(
&self,
_committee: Committee,
_view: &View,
) -> Box<dyn Stream<Item = Vote> + Send> {
unimplemented!()
}
async fn forward_approval<Vote: Serialize + Send>(
&self,
_committee: Committee,
_view: &View,
_approval: VoteMsg<Vote>,
) {
unimplemented!()
}
}
#[async_trait]
impl NetworkBackend for DummyBackend {
type Settings = ();
type State = NoState<()>;
type Message = ();
type EventKind = ();
type NetworkEvent = ();
fn new(_config: Self::Settings) -> Self {
Self
}
async fn process(&self, _: Self::Message) {}
async fn subscribe(&mut self, _: Self::EventKind) -> Receiver<Self::NetworkEvent> {
unimplemented!()
}
}
#[tokio::test]
async fn test_single_round_non_leader() {
let view = View {
seed: [0; 32],
staking_keys: BTreeMap::new(),
view_n: 0,
};
let mock_tally = MockTally::new(MockTallySettings { threshold: 0 });
let (_, next_view) = view
.resolve_non_leader::<DummyAdapter, DummyOverlay, MockFountain, MockTally>(
[0; 32],
&DummyAdapter,
&MockFountain,
&mock_tally,
)
.await
.unwrap();
assert!(next_view.view_n == 1);
}

View File

@ -1,20 +1,57 @@
use cryptarchia_engine::Slot;
use cryptarchia_ledger::{Coin, Config, EpochState, LeaderProof};
use cryptarchia_ledger::{Coin, Commitment, Config, EpochState, LeaderProof};
use nomos_core::header::HeaderId;
use std::collections::HashMap;
pub struct Leader {
coins: Vec<Coin>,
coins: HashMap<HeaderId, Vec<Coin>>,
config: cryptarchia_ledger::Config,
}
impl Leader {
pub fn new(coins: Vec<Coin>, config: Config) -> Self {
Leader { coins, config }
pub fn new(genesis: HeaderId, coins: Vec<Coin>, config: Config) -> Self {
Leader {
coins: HashMap::from([(genesis, coins)]),
config,
}
}
pub fn build_proof_for(&mut self, epoch_state: &EpochState, slot: Slot) -> Option<LeaderProof> {
for coin in &self.coins {
// Signal that the chain extended with a new header, possibly evolving a leader coin in the process
// FIXME: this implementation does not delete old coins and will attempt to re-use a coin in different forks,
// we should use the orphan proofs mechanism to handle this.
pub fn follow_chain(&mut self, parent_id: HeaderId, id: HeaderId, to_evolve: Commitment) {
if let Some(coins) = self.coins.get(&parent_id) {
let coins = coins
.iter()
.map(|coin| {
if coin.commitment() == to_evolve {
coin.evolve()
} else {
*coin
}
})
.collect();
self.coins.insert(id, coins);
}
}
pub fn build_proof_for(
&self,
epoch_state: &EpochState,
slot: Slot,
parent: HeaderId,
) -> Option<LeaderProof> {
let coins = self.coins.get(&parent)?;
for coin in coins {
if coin.is_slot_leader(epoch_state, slot, &self.config.consensus_config) {
return Some(coin.to_proof(slot));
tracing::debug!(
"leader for slot {:?}, {:?}/{:?}",
slot,
coin.value(),
epoch_state.total_stake()
);
let proof = coin.to_proof(slot);
return Some(proof);
}
}
None

View File

@ -4,7 +4,7 @@ mod time;
use core::fmt::Debug;
use cryptarchia_engine::Slot;
use cryptarchia_ledger::{LeaderProof, LedgerState};
use cryptarchia_ledger::{Coin, LeaderProof, LedgerState};
use futures::StreamExt;
use network::{messages::NetworkMessage, NetworkAdapter};
use nomos_core::da::certificate::{BlobCertificateSelect, Certificate};
@ -31,11 +31,15 @@ use serde::{de::DeserializeOwned, Deserialize, Serialize};
use serde_with::serde_as;
use std::hash::Hash;
use thiserror::Error;
pub use time::Config as TimeConfig;
use tokio::sync::oneshot::Sender;
use tokio::sync::{broadcast, oneshot};
use tokio_stream::wrappers::IntervalStream;
use tracing::{error, instrument};
// Limit the number of blocks returned by GetHeaders
const HEADERS_LIMIT: usize = 512;
#[derive(Debug, Clone, Error)]
pub enum Error {
#[error("Ledger error: {0}")]
@ -54,6 +58,10 @@ impl Cryptarchia {
self.consensus.tip()
}
fn genesis(&self) -> HeaderId {
self.consensus.genesis()
}
fn try_apply_header(&self, header: &Header) -> Result<Self, Error> {
let id = header.id();
let parent = header.parent();
@ -95,7 +103,8 @@ pub struct CryptarchiaSettings<Ts, Bs> {
pub blob_selector_settings: Bs,
pub config: cryptarchia_ledger::Config,
pub genesis_state: LedgerState,
pub time: time::Config,
pub time: TimeConfig,
pub coins: Vec<Coin>,
}
impl<Ts, Bs> CryptarchiaSettings<Ts, Bs> {
@ -105,7 +114,8 @@ impl<Ts, Bs> CryptarchiaSettings<Ts, Bs> {
blob_selector_settings: Bs,
config: cryptarchia_ledger::Config,
genesis_state: LedgerState,
time: time::Config,
time: TimeConfig,
coins: Vec<Coin>,
) -> Self {
Self {
transaction_selector_settings,
@ -113,6 +123,7 @@ impl<Ts, Bs> CryptarchiaSettings<Ts, Bs> {
config,
genesis_state,
time,
coins,
}
}
}
@ -257,6 +268,7 @@ where
transaction_selector_settings,
blob_selector_settings,
time,
coins,
} = self.service_state.settings_reader.get_updated_settings();
let genesis_id = HeaderId::from([0; 32]);
@ -276,7 +288,7 @@ where
let blob_selector = BS::new(blob_selector_settings);
let mut incoming_blocks = adapter.blocks_stream().await;
let mut leader = leadership::Leader::new(vec![], config);
let mut leader = leadership::Leader::new(genesis_id, coins, config);
let timer = time::Timer::new(time);
let mut slot_timer = IntervalStream::new(timer.slot_interval());
@ -287,6 +299,7 @@ where
Some(block) = incoming_blocks.next() => {
cryptarchia = Self::process_block(
cryptarchia,
&mut leader,
block,
storage_relay.clone(),
cl_mempool_relay.clone(),
@ -299,12 +312,14 @@ where
_ = slot_timer.next() => {
let slot = timer.current_slot();
let parent = cryptarchia.tip();
tracing::debug!("ticking for slot {}", u64::from(slot));
let Some(epoch_state) = cryptarchia.epoch_state_for_slot(slot) else {
tracing::error!("trying to propose a block for slot {} but epoch state is not available", u64::from(slot));
continue;
};
if let Some(proof) = leader.build_proof_for(epoch_state, slot) {
if let Some(proof) = leader.build_proof_for(epoch_state, slot, parent) {
tracing::debug!("proposing block...");
// TODO: spawn as a separate task?
let block = Self::propose_block(
parent,
@ -316,7 +331,7 @@ where
).await;
if let Some(block) = block {
let _ = adapter.broadcast(NetworkMessage::Block(block)).await;
adapter.broadcast(NetworkMessage::Block(block)).await;
}
}
}
@ -395,6 +410,17 @@ where
ConsensusMsg::Info { tx } => {
let info = CryptarchiaInfo {
tip: cryptarchia.tip(),
slot: cryptarchia
.ledger
.state(&cryptarchia.tip())
.expect("tip state not available")
.slot(),
height: cryptarchia
.consensus
.branches()
.get(&cryptarchia.tip())
.expect("tip branch not available")
.length(),
};
tx.send(info).unwrap_or_else(|e| {
tracing::error!("Could not send consensus info through channel: {:?}", e)
@ -405,16 +431,39 @@ where
tracing::error!("Could not subscribe to block subscription channel")
});
}
ConsensusMsg::GetHeaders { from, to, tx } => {
// default to tip block if not present
let from = from.unwrap_or(cryptarchia.tip());
// default to genesis block if not present
let to = to.unwrap_or(cryptarchia.genesis());
let mut res = Vec::new();
let mut cur = from;
let branches = cryptarchia.consensus.branches();
while let Some(h) = branches.get(&cur) {
res.push(h.id());
// limit the response size
if cur == to || cur == cryptarchia.genesis() || res.len() >= HEADERS_LIMIT {
break;
}
cur = h.parent();
}
tx.send(res)
.unwrap_or_else(|_| tracing::error!("could not send blocks through channel"));
}
}
}
#[allow(clippy::type_complexity, clippy::too_many_arguments)]
#[instrument(
level = "debug",
skip(cryptarchia, storage_relay, cl_mempool_relay, da_mempool_relay)
skip(cryptarchia, storage_relay, cl_mempool_relay, da_mempool_relay, leader)
)]
async fn process_block(
mut cryptarchia: Cryptarchia,
leader: &mut leadership::Leader,
block: Block<ClPool::Item, DaPool::Item>,
storage_relay: OutboundRelay<StorageMsg<Storage>>,
cl_mempool_relay: OutboundRelay<MempoolMsg<HeaderId, ClPool::Item, ClPool::Key>>,
@ -425,10 +474,13 @@ where
// TODO: filter on time?
let header = block.header();
let header = block.header().cryptarchia();
let id = header.id();
match cryptarchia.try_apply_header(block.header().cryptarchia()) {
match cryptarchia.try_apply_header(header) {
Ok(new_state) => {
// update leader
leader.follow_chain(header.parent(), id, *header.leader_proof().commitment());
// remove included content from mempool
mark_in_block(
cl_mempool_relay,
@ -487,6 +539,7 @@ where
else {
panic!("Proposal block should always succeed to be built")
};
tracing::debug!("proposed block with id {:?}", block.header().id());
output = Some(block);
}
(Err(_), _) => tracing::error!("Could not fetch block cl transactions"),
@ -504,6 +557,11 @@ pub enum ConsensusMsg<Block> {
BlockSubscribe {
sender: oneshot::Sender<broadcast::Receiver<Block>>,
},
GetHeaders {
from: Option<HeaderId>,
to: Option<HeaderId>,
tx: Sender<Vec<HeaderId>>,
},
}
impl<Block: 'static> RelayMessage for ConsensusMsg<Block> {}
@ -513,6 +571,8 @@ impl<Block: 'static> RelayMessage for ConsensusMsg<Block> {}
#[cfg_attr(feature = "openapi", derive(utoipa::ToSchema))]
pub struct CryptarchiaInfo {
pub tip: HeaderId,
pub slot: Slot,
pub height: u64,
}
async fn get_mempool_contents<Item, Key>(

View File

@ -1,48 +0,0 @@
[package]
name = "simulations"
version = "0.1.0"
edition = "2021"
[[bin]]
name = "simulation"
path = "src/bin/app/main.rs"
# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html
[dependencies]
anyhow = "1"
blake2 = "0.10"
bls-signatures = "0.14"
digest = "0.10"
csv = "1"
clap = { version = "4", features = ["derive"] }
ctrlc = "3.4"
chrono = { version = "0.4", features = ["serde"] }
crc32fast = "1.3"
crossbeam = { version = "0.8.2", features = ["crossbeam-channel"] }
carnot-engine = { path = "../consensus/carnot-engine", features = ["simulation"] }
fixed-slice-deque = "0.1.0-beta2"
futures = "0.3"
humantime = "2.1"
humantime-serde = "1"
nomos-core = { path = "../nomos-core" }
carnot-consensus = { path = "../nomos-services/carnot-consensus" }
once_cell = "1.17"
parking_lot = "0.12"
polars = { version = "0.27", features = ["serde", "object", "json", "csv-file", "parquet", "dtype-struct"], optional = true }
rand = { version = "0.8", features = ["small_rng"] }
rayon = "1.8"
scopeguard = "1"
serde = { version = "1.0", features = ["derive", "rc"] }
serde_with = "2.3"
serde_json = "1.0"
thiserror = "1"
tracing = { version = "0.1", default-features = false, features = ["log", "attributes"] }
tracing-subscriber = { version = "0.3", features = ["json", "env-filter", "tracing-log"]}
nomos-utils = { path = "../nomos-utils" }
[target.'cfg(target_arch = "wasm32")'.dependencies]
getrandom = { version = "0.2", features = ["js"] }
[features]
polars = ["dep:polars"]

View File

@ -1,52 +0,0 @@
{
"network_settings": {
"network_behaviors": {
"north america:north america": "50ms",
"north america:europe": "100ms",
"north america:asia": "120ms",
"europe:europe": "50ms",
"europe:asia": "100ms",
"europe:north america": "120ms",
"asia:north america": "100ms",
"asia:europe": "120ms",
"asia:asia": "40ms"
},
"regions": {
"north america": 0.4,
"europe": 0.4,
"asia": 0.3
}
},
"overlay_settings": {
"number_of_committees": 3
},
"node_settings": {
"timeout": "1000ms"
},
"step_time": "10ms",
"runner_settings": "Sync",
"stream_settings": {
"path": "test.csv"
},
"node_count": 3000,
"views_count": 3,
"leaders_count": 1,
"seed": 0,
"wards": [
{"max_view": 1}
],
"record_settings": {
"node_id": true,
"current_view": true,
"highest_voted_view": true,
"local_high_qc": true,
"safe_blocks": true,
"last_view_timeout_qc": true,
"latest_committed_block": true,
"latest_committed_view": true,
"root_committee": true,
"parent_committee": true,
"child_committees": true,
"committed_blocks": true
}
}

View File

@ -1,62 +0,0 @@
{
"network_settings": {
"network_behaviors": {
"north america:north america": "10ms",
"north america:europe": "150ms",
"north america:asia": "250ms",
"europe:europe": "10ms",
"europe:asia": "200ms",
"europe:north america": "150ms",
"asia:north america": "250ms",
"asia:europe": "200ms",
"asia:asia": "10ms"
},
"regions": {
"north america": 0.4,
"europe": 0.3,
"asia": 0.3
}
},
"overlay_settings": {
"number_of_committees": 7
},
"node_settings": {
"network_capacity_kbps": 10000024,
"timeout": "10000ms"
},
"step_time": "100ms",
"runner_settings": "Sync",
"stream_settings": {
"path": "tree_500_7_view_1_default.csv",
"format": "csv"
},
"node_count": 500,
"views_count": 10,
"leaders_count": 1,
"seed": 0,
"wards": [
{
"max_view": 1
},
{
"stalled_view": {
"consecutive_viewed_checkpoint": null,
"criterion": 0,
"threshold": 100
}
}
],
"record_settings": {
"current_view": true,
"highest_voted_view": true,
"local_high_qc": true,
"safe_blocks": false,
"last_view_timeout_qc": true,
"latest_committed_block": true,
"latest_committed_view": true,
"root_committee": false,
"parent_committee": false,
"child_committees": false,
"committed_blocks": false
}
}

View File

@ -1,86 +0,0 @@
use std::{
fs::File,
io::{stderr, stdout},
path::PathBuf,
str::FromStr,
};
use tracing_subscriber::fmt::{format::Format, FormatEvent, FormatFields, SubscriberBuilder};
#[derive(Default, Copy, Clone)]
pub enum LogFormat {
#[default]
Plain,
Json,
}
impl FromStr for LogFormat {
type Err = anyhow::Error;
fn from_str(s: &str) -> Result<Self, Self::Err> {
match s.trim().to_ascii_lowercase().as_str() {
"json" => Ok(LogFormat::Json),
"plain" => Ok(LogFormat::Plain),
_ => Err(anyhow::anyhow!("Unknown log format")),
}
}
}
#[derive(Default, Clone)]
pub enum LogOutput {
#[default]
StdOut,
StdErr,
File(PathBuf),
}
impl FromStr for LogOutput {
type Err = anyhow::Error;
fn from_str(s: &str) -> Result<Self, Self::Err> {
match s.trim().to_ascii_lowercase().as_str() {
"stdout" => Ok(Self::StdOut),
"stderr" => Ok(Self::StdErr),
path => Ok(Self::File(PathBuf::from(path))),
}
}
}
pub fn config_tracing(fmt: LogFormat, file: &LogOutput) {
let filter = std::env::var("SIMULATION_LOG").unwrap_or_else(|_| "info".to_owned());
let subscriber = tracing_subscriber::fmt::fmt()
.without_time()
.with_line_number(true)
.with_env_filter(filter)
.with_file(false)
.with_target(true);
if let LogFormat::Json = fmt {
set_global(subscriber.json(), file);
} else {
set_global(subscriber, file);
}
}
fn set_global<N, L, T>(
subscriber: SubscriberBuilder<N, Format<L, T>, tracing_subscriber::EnvFilter>,
output: &LogOutput,
) where
N: for<'writer> FormatFields<'writer> + 'static + Send + Sync,
Format<L, T>: FormatEvent<tracing_subscriber::Registry, N>,
L: Send + Sync + 'static,
T: Send + Sync + 'static,
{
use tracing::subscriber::set_global_default;
match output {
LogOutput::StdOut => set_global_default(subscriber.with_writer(stdout).finish()),
LogOutput::StdErr => set_global_default(subscriber.with_writer(stderr).finish()),
LogOutput::File(path) => set_global_default(
subscriber
.with_ansi(false)
.with_writer(File::create(path).expect("Unable to create log file"))
.finish(),
),
}
.expect("Unable to set global default subscriber")
}

View File

@ -1,245 +0,0 @@
// std
use std::fs::File;
use std::path::{Path, PathBuf};
use std::sync::Arc;
use std::time::{Duration, SystemTime, UNIX_EPOCH};
// crates
use anyhow::Ok;
use carnot_engine::overlay::RandomBeaconState;
use carnot_engine::{Block, LeaderProof, View};
use clap::Parser;
use crossbeam::channel;
use nomos_core::block::builder::BlockBuilder;
use parking_lot::Mutex;
use rand::rngs::SmallRng;
use rand::seq::SliceRandom;
use rand::SeedableRng;
use rayon::prelude::{IntoParallelRefIterator, ParallelIterator};
use serde::de::DeserializeOwned;
use serde::Serialize;
use simulations::network::behaviour::create_behaviours;
use simulations::network::regions::{create_regions, RegionsData};
use simulations::network::{InMemoryNetworkInterface, Network};
use simulations::node::carnot::{CarnotRecord, CarnotSettings, CarnotState};
use simulations::node::{NodeId, NodeIdExt};
use simulations::output_processors::Record;
use simulations::runner::{BoxedNode, SimulationRunnerHandle};
#[cfg(feature = "polars")]
use simulations::streaming::polars::PolarsSubscriber;
use simulations::streaming::{io::IOSubscriber, naive::NaiveSubscriber, StreamType};
// internal
use simulations::{runner::SimulationRunner, settings::SimulationSettings};
mod log;
mod overlay_node;
/// Main simulation wrapper
/// Pipes together the cli arguments with the execution
#[derive(Parser)]
pub struct SimulationApp {
/// Json file path, on `SimulationSettings` format
#[clap(long, short)]
input_settings: PathBuf,
#[clap(long)]
stream_type: Option<StreamType>,
#[clap(long, default_value = "plain")]
log_format: log::LogFormat,
#[clap(long, default_value = "stdout")]
log_to: log::LogOutput,
#[clap(long)]
dump_overlay_info: bool,
#[clap(long)]
no_netcap: bool,
}
impl SimulationApp {
pub fn run(self) -> anyhow::Result<()> {
let Self {
input_settings,
stream_type,
log_format: _,
log_to: _,
dump_overlay_info,
no_netcap,
} = self;
let simulation_settings: SimulationSettings = load_json_from_file(&input_settings)?;
let seed = simulation_settings.seed.unwrap_or_else(|| {
SystemTime::now()
.duration_since(UNIX_EPOCH)
.expect("Time went backwards")
.as_secs()
});
let mut rng = SmallRng::seed_from_u64(seed);
let mut node_ids: Vec<NodeId> = (0..simulation_settings.node_count)
.map(NodeId::from_index)
.collect();
node_ids.shuffle(&mut rng);
let regions = create_regions(&node_ids, &mut rng, &simulation_settings.network_settings);
let behaviours = create_behaviours(&simulation_settings.network_settings);
let regions_data = RegionsData::new(regions, behaviours);
let ids = node_ids.clone();
let network = Arc::new(Mutex::new(Network::new(regions_data, seed)));
if dump_overlay_info {
dump_json_to_file(
Path::new("overlay_info.json"),
&overlay_node::overlay_info(
node_ids.clone(),
node_ids.first().copied().unwrap(),
&simulation_settings.overlay_settings,
),
)?;
}
let nodes: Vec<BoxedNode<CarnotSettings, CarnotState>> = node_ids
.par_iter()
.copied()
.map(|node_id| {
let (node_message_broadcast_sender, node_message_broadcast_receiver) =
channel::unbounded();
let (node_message_sender, node_message_receiver) = channel::unbounded();
// Dividing milliseconds in second by milliseconds in the step.
let step_time_as_second_fraction =
simulation_settings.step_time.subsec_millis() as f32 / 1_000_000_f32;
let capacity_bps = if no_netcap {
None
} else {
simulation_settings
.node_settings
.network_capacity_kbps
.map(|c| (c as f32 * 1024.0 * step_time_as_second_fraction) as u32)
};
let network_message_receiver = {
let mut network = network.lock();
network.connect(
node_id,
capacity_bps,
node_message_receiver,
node_message_broadcast_receiver,
)
};
let network_interface = InMemoryNetworkInterface::new(
node_id,
node_message_broadcast_sender,
node_message_sender,
network_message_receiver,
);
let nodes: Vec<NodeId> = ids.clone().into_iter().map(Into::into).collect();
let leader = nodes.first().copied().unwrap();
// FIXME: Actually use a proposer and a key to generate random beacon state
let genesis = <BlockBuilder<_, _, (), ()>>::empty_carnot(
RandomBeaconState::Sad {
entropy: Box::new([0; 32]),
},
View::new(0),
Block::genesis([0; 32].into()).parent_qc,
LeaderProof::LeaderId { leader_id: leader },
);
let mut rng = SmallRng::seed_from_u64(seed);
overlay_node::to_overlay_node(
node_id,
nodes,
leader,
network_interface,
genesis,
&mut rng,
&simulation_settings,
)
})
.collect();
let network = Arc::try_unwrap(network)
.expect("network is not used anywhere else")
.into_inner();
run::<_, _, _>(network, nodes, simulation_settings, stream_type)?;
Ok(())
}
}
fn run<M: std::fmt::Debug, S, T>(
network: Network<M>,
nodes: Vec<BoxedNode<S, T>>,
settings: SimulationSettings,
stream_type: Option<StreamType>,
) -> anyhow::Result<()>
where
M: Clone + Send + Sync + 'static,
S: 'static,
T: Serialize + Clone + 'static,
{
let stream_settings = settings.stream_settings.clone();
let runner = SimulationRunner::<_, CarnotRecord, S, T>::new(
network,
nodes,
Default::default(),
settings,
)?;
let handle = match stream_type {
Some(StreamType::Naive) => {
let settings = stream_settings.unwrap_naive();
runner.simulate_and_subscribe::<NaiveSubscriber<CarnotRecord>>(settings)?
}
Some(StreamType::IO) => {
let settings = stream_settings.unwrap_io();
runner.simulate_and_subscribe::<IOSubscriber<CarnotRecord>>(settings)?
}
#[cfg(feature = "polars")]
Some(StreamType::Polars) => {
let settings = stream_settings.unwrap_polars();
runner.simulate_and_subscribe::<PolarsSubscriber<CarnotRecord>>(settings)?
}
None => runner.simulate()?,
};
signal(handle)
}
fn signal<R: Record>(handle: SimulationRunnerHandle<R>) -> anyhow::Result<()> {
let handle = Arc::new(handle);
let (tx, rx) = crossbeam::channel::bounded(1);
ctrlc::set_handler(move || {
tx.send(()).unwrap();
})?;
loop {
crossbeam::select! {
recv(rx) -> _ => {
handle.stop()?;
tracing::info!("gracefully shutdown the simulation app");
break;
},
default => {
if handle.is_finished() {
handle.shutdown()?;
break;
}
std::thread::sleep(Duration::from_millis(50));
}
}
}
Ok(())
}
/// Generically load a json file
fn load_json_from_file<T: DeserializeOwned>(path: &Path) -> anyhow::Result<T> {
let f = File::open(path).map_err(Box::new)?;
Ok(serde_json::from_reader(f)?)
}
fn dump_json_to_file<T: Serialize>(path: &Path, data: &T) -> anyhow::Result<()> {
let f = File::create(path).map_err(Box::new)?;
Ok(serde_json::to_writer(f, data)?)
}
fn main() -> anyhow::Result<()> {
let app: SimulationApp = SimulationApp::parse();
log::config_tracing(app.log_format, &app.log_to);
if let Err(e) = app.run() {
tracing::error!("error: {}", e);
std::process::exit(1);
}
Ok(())
}

View File

@ -1,145 +0,0 @@
use carnot_engine::overlay::{BranchOverlay, RandomBeaconState};
use carnot_engine::Overlay;
use carnot_engine::{
overlay::{FlatOverlay, FreezeMembership, RoundRobin, TreeOverlay},
NodeId,
};
use nomos_utils::fisheryates::FisherYatesShuffle;
use rand::Rng;
use simulations::overlay::overlay_info::{OverlayInfo, OverlayInfoExt};
use simulations::settings::OverlaySettings;
use simulations::{
network::InMemoryNetworkInterface,
node::carnot::{messages::CarnotMessage, CarnotNode, CarnotSettings, CarnotState},
runner::BoxedNode,
settings::SimulationSettings,
};
pub fn overlay_info(
nodes: Vec<NodeId>,
leader: NodeId,
overlay_settings: &OverlaySettings,
) -> OverlayInfo {
match &overlay_settings {
simulations::settings::OverlaySettings::Flat => {
FlatOverlay::<RoundRobin, FisherYatesShuffle>::new(
carnot_engine::overlay::FlatOverlaySettings {
nodes: nodes.to_vec(),
leader: RoundRobin::new(),
leader_super_majority_threshold: None,
},
)
.info()
}
simulations::settings::OverlaySettings::Tree(tree_settings) => {
TreeOverlay::new(carnot_engine::overlay::TreeOverlaySettings {
nodes,
current_leader: leader,
number_of_committees: tree_settings.number_of_committees,
leader: RoundRobin::new(),
committee_membership: RandomBeaconState::initial_sad_from_entropy([0; 32]),
super_majority_threshold: None,
})
.info()
}
simulations::settings::OverlaySettings::Branch(branch_settings) => {
BranchOverlay::new(carnot_engine::overlay::BranchOverlaySettings {
nodes,
current_leader: leader,
branch_depth: branch_settings.branch_depth,
leader: RoundRobin::new(),
committee_membership: RandomBeaconState::initial_sad_from_entropy([0; 32]),
})
.info()
}
}
}
pub fn to_overlay_node<R: Rng>(
node_id: NodeId,
nodes: Vec<NodeId>,
leader: NodeId,
network_interface: InMemoryNetworkInterface<CarnotMessage>,
genesis: nomos_core::block::Block<[u8; 32], Box<[u8]>>,
mut rng: R,
settings: &SimulationSettings,
) -> BoxedNode<CarnotSettings, CarnotState> {
let fmt = match &settings.stream_settings {
simulations::streaming::StreamSettings::Naive(n) => n.format,
simulations::streaming::StreamSettings::IO(_) => {
simulations::streaming::SubscriberFormat::Csv
}
#[cfg(feature = "polars")]
simulations::streaming::StreamSettings::Polars(p) => p.format,
};
match &settings.overlay_settings {
simulations::settings::OverlaySettings::Flat => {
let overlay_settings = carnot_engine::overlay::FlatOverlaySettings {
nodes: nodes.to_vec(),
leader: RoundRobin::new(),
leader_super_majority_threshold: None,
};
Box::new(
CarnotNode::<FlatOverlay<RoundRobin, FreezeMembership>>::new(
node_id,
CarnotSettings::new(
settings.node_settings.timeout,
settings.record_settings.clone(),
fmt,
),
overlay_settings,
genesis,
network_interface,
&mut rng,
),
)
}
simulations::settings::OverlaySettings::Tree(tree_settings) => {
let overlay_settings = carnot_engine::overlay::TreeOverlaySettings {
nodes,
current_leader: leader,
number_of_committees: tree_settings.number_of_committees,
leader: RoundRobin::new(),
committee_membership: RandomBeaconState::initial_sad_from_entropy([0; 32]),
super_majority_threshold: None,
};
Box::new(
CarnotNode::<TreeOverlay<RoundRobin, RandomBeaconState>>::new(
node_id,
CarnotSettings::new(
settings.node_settings.timeout,
settings.record_settings.clone(),
fmt,
),
overlay_settings,
genesis,
network_interface,
&mut rng,
),
)
}
simulations::settings::OverlaySettings::Branch(branch_settings) => {
let overlay_settings = carnot_engine::overlay::BranchOverlaySettings {
nodes,
current_leader: leader,
branch_depth: branch_settings.branch_depth,
leader: RoundRobin::new(),
committee_membership: RandomBeaconState::initial_sad_from_entropy([0; 32]),
};
Box::new(
CarnotNode::<BranchOverlay<RoundRobin, RandomBeaconState>>::new(
node_id,
CarnotSettings::new(
settings.node_settings.timeout,
settings.record_settings.clone(),
fmt,
),
overlay_settings,
genesis,
network_interface,
&mut rng,
),
)
}
}
}

View File

@ -1,11 +0,0 @@
pub mod network;
pub mod node;
pub mod output_processors;
pub mod overlay;
pub mod runner;
pub mod settings;
pub mod streaming;
pub mod warding;
static START_TIME: once_cell::sync::Lazy<std::time::Instant> =
once_cell::sync::Lazy::new(std::time::Instant::now);

View File

@ -1,40 +0,0 @@
// std
use std::{collections::HashMap, time::Duration};
// crates
use rand::Rng;
use serde::{Deserialize, Serialize};
use super::{NetworkBehaviourKey, NetworkSettings};
// internal
#[derive(Default, Debug, Clone, Serialize, Deserialize)]
pub struct NetworkBehaviour {
pub delay: Duration,
pub drop: f64,
}
impl NetworkBehaviour {
pub fn new(delay: Duration, drop: f64) -> Self {
Self { delay, drop }
}
pub fn delay(&self) -> Duration {
self.delay
}
pub fn should_drop<R: Rng>(&self, rng: &mut R) -> bool {
rng.gen_bool(self.drop)
}
}
// Takes a reference to the simulation_settings and returns a HashMap representing the
// network behaviors for pairs of NodeIds.
pub fn create_behaviours(
network_settings: &NetworkSettings,
) -> HashMap<NetworkBehaviourKey, NetworkBehaviour> {
network_settings
.network_behaviors
.iter()
.map(|(k, d)| (*k, NetworkBehaviour::new(*d, 0.0)))
.collect()
}

View File

@ -1,814 +0,0 @@
// std
use std::{
collections::HashMap,
ops::Add,
str::FromStr,
sync::{
atomic::{AtomicU32, Ordering},
Arc,
},
time::{Duration, Instant},
};
// crates
use crossbeam::channel::{self, Receiver, Sender};
use parking_lot::Mutex;
use rand::{rngs::SmallRng, Rng, SeedableRng};
use rayon::prelude::*;
use serde::{Deserialize, Serialize};
// internal
use crate::node::NodeId;
pub mod behaviour;
pub mod regions;
type NetworkTime = Instant;
#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)]
pub struct NetworkBehaviourKey {
pub from: regions::Region,
pub to: regions::Region,
}
impl NetworkBehaviourKey {
pub fn new(from: regions::Region, to: regions::Region) -> Self {
Self { from, to }
}
}
impl Serialize for NetworkBehaviourKey {
fn serialize<S: serde::Serializer>(&self, serializer: S) -> Result<S::Ok, S::Error> {
let s = format!("{}:{}", self.from, self.to);
serializer.serialize_str(&s)
}
}
impl<'de> Deserialize<'de> for NetworkBehaviourKey {
fn deserialize<D: serde::Deserializer<'de>>(deserializer: D) -> Result<Self, D::Error> {
let s = String::deserialize(deserializer)?;
let mut split = s.split(':');
let from = split.next().ok_or(serde::de::Error::custom(
"NetworkBehaviourKey should be in the form of `from_region:to_region`",
))?;
let to = split.next().ok_or(serde::de::Error::custom(
"NetworkBehaviourKey should be in the form of `from_region:to_region`",
))?;
Ok(Self::new(
regions::Region::from_str(from).map_err(serde::de::Error::custom)?,
regions::Region::from_str(to).map_err(serde::de::Error::custom)?,
))
}
}
#[derive(Clone, Debug, Serialize, Deserialize, Default)]
pub struct NetworkSettings {
#[serde(with = "network_behaviors_serde")]
pub network_behaviors: HashMap<NetworkBehaviourKey, Duration>,
/// Represents node distribution in the simulated regions.
/// The sum of distributions should be 1.
pub regions: HashMap<regions::Region, f32>,
}
/// Ser/Deser `HashMap<NetworkBehaviourKey, Duration>` to humantime format.
mod network_behaviors_serde {
use super::{Deserialize, Duration, HashMap, NetworkBehaviourKey};
/// Have to implement this manually because of the `serde_json` will panic if the key of map
/// is not a string.
pub fn serialize<S>(
vals: &HashMap<NetworkBehaviourKey, Duration>,
serializer: S,
) -> Result<S::Ok, S::Error>
where
S: serde::Serializer,
{
use serde::ser::SerializeMap;
let mut ser = serializer.serialize_map(Some(vals.len()))?;
for (k, v) in vals {
ser.serialize_key(&k)?;
ser.serialize_value(&humantime::format_duration(*v).to_string())?;
}
ser.end()
}
pub fn deserialize<'de, D>(
deserializer: D,
) -> Result<HashMap<NetworkBehaviourKey, Duration>, D::Error>
where
D: serde::Deserializer<'de>,
{
let map = HashMap::<NetworkBehaviourKey, String>::deserialize(deserializer)?;
map.into_iter()
.map(|(k, v)| {
let v = humantime::parse_duration(&v).map_err(serde::de::Error::custom)?;
Ok((k, v))
})
.collect::<Result<HashMap<_, _>, _>>()
}
}
/// Represents node network capacity and current load in bytes.
#[derive(Debug)]
struct NodeNetworkCapacity {
capacity_bps: Option<u32>,
current_load: Mutex<u32>,
load_to_flush: AtomicU32,
}
impl NodeNetworkCapacity {
fn new(capacity_bps: Option<u32>) -> Self {
Self {
capacity_bps,
current_load: Mutex::new(0),
load_to_flush: AtomicU32::new(0),
}
}
fn increase_load(&self, load: u32) -> bool {
if let Some(capacity_bps) = self.capacity_bps {
let mut current_load = self.current_load.lock();
if *current_load + load <= capacity_bps {
*current_load += load;
true
} else {
false
}
} else {
true
}
}
fn decrease_load(&self, load: u32) {
self.load_to_flush.fetch_add(load, Ordering::Relaxed);
}
fn flush_load(&self) {
if self.capacity_bps.is_none() {
return;
}
let mut s = self.current_load.lock();
*s -= self.load_to_flush.load(Ordering::Relaxed);
self.load_to_flush.store(0, Ordering::Relaxed);
}
}
#[derive(Debug)]
pub struct Network<M: std::fmt::Debug> {
pub regions: regions::RegionsData,
network_time: NetworkTime,
messages: Vec<(NetworkTime, NetworkMessage<M>)>,
node_network_capacity: HashMap<NodeId, NodeNetworkCapacity>,
from_node_receivers: HashMap<NodeId, Receiver<NetworkMessage<M>>>,
from_node_broadcast_receivers: HashMap<NodeId, Receiver<NetworkMessage<M>>>,
to_node_senders: HashMap<NodeId, Sender<NetworkMessage<M>>>,
seed: u64,
}
impl<M> Network<M>
where
M: std::fmt::Debug + Send + Sync + Clone,
{
pub fn new(regions: regions::RegionsData, seed: u64) -> Self {
Self {
regions,
network_time: Instant::now(),
messages: Vec::new(),
node_network_capacity: HashMap::new(),
from_node_receivers: HashMap::new(),
from_node_broadcast_receivers: HashMap::new(),
to_node_senders: HashMap::new(),
seed,
}
}
fn send_message_cost<R: Rng>(
&self,
rng: &mut R,
node_a: NodeId,
node_b: NodeId,
) -> Option<Duration> {
let network_behaviour = self.regions.network_behaviour(node_a, node_b);
(!network_behaviour.should_drop(rng))
// TODO: use a delay range
.then(|| network_behaviour.delay())
}
pub fn connect(
&mut self,
node_id: NodeId,
capacity_bps: Option<u32>,
node_message_receiver: Receiver<NetworkMessage<M>>,
node_message_broadcast_receiver: Receiver<NetworkMessage<M>>,
) -> Receiver<NetworkMessage<M>> {
self.node_network_capacity
.insert(node_id, NodeNetworkCapacity::new(capacity_bps));
let (to_node_sender, from_network_receiver) = channel::unbounded();
self.from_node_receivers
.insert(node_id, node_message_receiver);
self.from_node_broadcast_receivers
.insert(node_id, node_message_broadcast_receiver);
self.to_node_senders.insert(node_id, to_node_sender);
from_network_receiver
}
/// Collects and dispatches messages to connected interfaces.
pub fn step(&mut self, time_passed: Duration) {
self.collect_messages();
self.dispatch_after(time_passed);
}
/// Receive and store all messages from nodes.
pub fn collect_messages(&mut self) {
let mut adhoc_messages = self
.from_node_receivers
.par_iter()
.flat_map(|(_, from_node)| {
from_node
.try_iter()
.map(|msg| (self.network_time, msg))
.collect::<Vec<_>>()
})
.collect();
self.messages.append(&mut adhoc_messages);
let mut broadcast_messages = self
.from_node_broadcast_receivers
.iter()
.flat_map(|(_, from_node)| {
from_node.try_iter().flat_map(|msg| {
self.to_node_senders.keys().map(move |recipient| {
let mut m = msg.clone();
m.to = Some(*recipient);
m
})
})
})
.map(|m| (self.network_time, m))
.collect::<Vec<_>>();
self.messages.append(&mut broadcast_messages);
}
/// Reiterate all messages and send to appropriate nodes if simulated
/// delay has passed.
pub fn dispatch_after(&mut self, time_passed: Duration) {
self.network_time += time_passed;
let delayed = self
.messages
.par_iter()
.filter(|(network_time, message)| {
let mut rng = SmallRng::seed_from_u64(self.seed);
self.send_or_drop_message(&mut rng, network_time, message)
})
.cloned()
.collect();
for (_, c) in self.node_network_capacity.iter() {
c.flush_load();
}
self.messages = delayed;
}
/// Returns true if message needs to be delayed and be dispatched in future.
fn send_or_drop_message<R: Rng>(
&self,
rng: &mut R,
network_time: &NetworkTime,
message: &NetworkMessage<M>,
) -> bool {
let to = message.to.expect("adhoc message has recipient");
if let Some(delay) = self.send_message_cost(rng, message.from, to) {
let node_capacity = self.node_network_capacity.get(&to).unwrap();
let should_send = network_time.add(delay) <= self.network_time;
let remaining_size = message.remaining_size();
if should_send && node_capacity.increase_load(remaining_size) {
let to_node = self.to_node_senders.get(&to).unwrap();
to_node
.send(message.clone())
.expect("node should have connection");
node_capacity.decrease_load(remaining_size);
return false;
} else {
// if we do not need to delay, then we should check if the msg is too large
// if so, we mock the partial sending message behavior
if should_send {
// if remaining is 0, we should send without delay
return self.try_partial_send(node_capacity, message, &to) != 0;
}
return true;
}
}
false
}
/// Try to apply partial send logic, returns the remaining size of the message
fn try_partial_send(
&self,
node_capacity: &NodeNetworkCapacity,
message: &NetworkMessage<M>,
to: &NodeId,
) -> u32 {
if let Some(capacity_bps) = node_capacity.capacity_bps {
let mut cap = node_capacity.current_load.lock();
let sent = capacity_bps - *cap;
*cap = capacity_bps;
let remaining = message.partial_send(sent);
// Message is partially sent, the node capacity needs to be flushed at the end of step even
// if the whole message is not sent.
node_capacity.decrease_load(sent);
if remaining == 0 {
let to_node = self.to_node_senders.get(to).unwrap();
to_node
.send(message.clone())
.expect("node should have connection");
}
remaining
} else {
0
}
}
}
#[derive(Clone, Debug)]
pub struct NetworkMessage<M> {
pub from: NodeId,
pub to: Option<NodeId>,
pub payload: M,
pub remaining: Arc<AtomicU32>,
}
impl<M> NetworkMessage<M> {
pub fn new(from: NodeId, to: Option<NodeId>, payload: M, size_bytes: u32) -> Self {
Self {
from,
to,
payload,
remaining: Arc::new(AtomicU32::new(size_bytes)),
}
}
pub fn payload(&self) -> &M {
&self.payload
}
pub fn into_payload(self) -> M {
self.payload
}
fn remaining_size(&self) -> u32 {
self.remaining.load(Ordering::SeqCst)
}
/// Mock the partial sending of a message behavior, returning the remaining message size.
fn partial_send(&self, size: u32) -> u32 {
self.remaining
.fetch_sub(size, Ordering::SeqCst)
.saturating_sub(size)
}
}
pub trait PayloadSize {
fn size_bytes(&self) -> u32;
}
pub trait NetworkInterface {
type Payload;
fn broadcast(&self, message: Self::Payload);
fn send_message(&self, address: NodeId, message: Self::Payload);
fn receive_messages(&self) -> Vec<NetworkMessage<Self::Payload>>;
}
pub struct InMemoryNetworkInterface<M> {
id: NodeId,
broadcast: Sender<NetworkMessage<M>>,
sender: Sender<NetworkMessage<M>>,
receiver: Receiver<NetworkMessage<M>>,
}
impl<M> InMemoryNetworkInterface<M> {
pub fn new(
id: NodeId,
broadcast: Sender<NetworkMessage<M>>,
sender: Sender<NetworkMessage<M>>,
receiver: Receiver<NetworkMessage<M>>,
) -> Self {
Self {
id,
broadcast,
sender,
receiver,
}
}
}
impl<M: PayloadSize> NetworkInterface for InMemoryNetworkInterface<M> {
type Payload = M;
fn broadcast(&self, message: Self::Payload) {
let size = message.size_bytes();
let message = NetworkMessage::new(self.id, None, message, size);
self.broadcast.send(message).unwrap();
}
fn send_message(&self, address: NodeId, message: Self::Payload) {
let size = message.size_bytes();
let message = NetworkMessage::new(self.id, Some(address), message, size);
self.sender.send(message).unwrap();
}
fn receive_messages(&self) -> Vec<crate::network::NetworkMessage<Self::Payload>> {
self.receiver.try_iter().collect()
}
}
#[cfg(test)]
mod tests {
use super::{
behaviour::NetworkBehaviour,
regions::{Region, RegionsData},
Network, NetworkInterface, NetworkMessage,
};
use crate::{
network::NetworkBehaviourKey,
node::{NodeId, NodeIdExt},
};
use crossbeam::channel::{self, Receiver, Sender};
use std::{collections::HashMap, time::Duration};
struct MockNetworkInterface {
id: NodeId,
broadcast: Sender<NetworkMessage<()>>,
sender: Sender<NetworkMessage<()>>,
receiver: Receiver<NetworkMessage<()>>,
message_size: u32,
}
impl MockNetworkInterface {
pub fn new(
id: NodeId,
broadcast: Sender<NetworkMessage<()>>,
sender: Sender<NetworkMessage<()>>,
receiver: Receiver<NetworkMessage<()>>,
message_size: u32,
) -> Self {
Self {
id,
broadcast,
sender,
receiver,
message_size,
}
}
}
impl NetworkInterface for MockNetworkInterface {
type Payload = ();
fn broadcast(&self, message: Self::Payload) {
let message = NetworkMessage::new(self.id, None, message, self.message_size);
self.broadcast.send(message).unwrap();
}
fn send_message(&self, address: NodeId, message: Self::Payload) {
let message = NetworkMessage::new(self.id, Some(address), message, self.message_size);
self.sender.send(message).unwrap();
}
fn receive_messages(&self) -> Vec<crate::network::NetworkMessage<Self::Payload>> {
self.receiver.try_iter().collect()
}
}
#[test]
fn send_receive_messages() {
let node_a = NodeId::from_index(0);
let node_b = NodeId::from_index(1);
let regions = HashMap::from([(Region::Europe, vec![node_a, node_b])]);
let behaviour = HashMap::from([(
NetworkBehaviourKey::new(Region::Europe, Region::Europe),
NetworkBehaviour::new(Duration::from_millis(100), 0.0),
)]);
let regions_data = RegionsData::new(regions, behaviour);
let mut network = Network::new(regions_data, 0);
let (from_a_sender, from_a_receiver) = channel::unbounded();
let (from_a_broadcast_sender, from_a_broadcast_receiver) = channel::unbounded();
let to_a_receiver =
network.connect(node_a, Some(3), from_a_receiver, from_a_broadcast_receiver);
let a = MockNetworkInterface::new(
node_a,
from_a_broadcast_sender,
from_a_sender,
to_a_receiver,
1,
);
let (from_b_sender, from_b_receiver) = channel::unbounded();
let (from_b_broadcast_sender, from_b_broadcast_receiver) = channel::unbounded();
let to_b_receiver =
network.connect(node_b, Some(3), from_b_receiver, from_b_broadcast_receiver);
let b = MockNetworkInterface::new(
node_b,
from_b_broadcast_sender,
from_b_sender,
to_b_receiver,
1,
);
a.send_message(node_b, ());
network.collect_messages();
assert_eq!(a.receive_messages().len(), 0);
assert_eq!(b.receive_messages().len(), 0);
network.step(Duration::from_millis(0));
assert_eq!(a.receive_messages().len(), 0);
assert_eq!(b.receive_messages().len(), 0);
network.step(Duration::from_millis(100));
assert_eq!(a.receive_messages().len(), 0);
assert_eq!(b.receive_messages().len(), 1);
network.step(Duration::from_millis(100));
assert_eq!(a.receive_messages().len(), 0);
assert_eq!(b.receive_messages().len(), 0);
b.send_message(node_a, ());
b.send_message(node_a, ());
b.send_message(node_a, ());
network.collect_messages();
network.dispatch_after(Duration::from_millis(100));
assert_eq!(a.receive_messages().len(), 3);
assert_eq!(b.receive_messages().len(), 0);
}
#[test]
fn regions_send_receive_messages() {
let node_a = NodeId::from_index(0);
let node_b = NodeId::from_index(1);
let node_c = NodeId::from_index(2);
let regions = HashMap::from([
(Region::Asia, vec![node_a, node_b]),
(Region::Europe, vec![node_c]),
]);
let behaviour = HashMap::from([
(
NetworkBehaviourKey::new(Region::Asia, Region::Asia),
NetworkBehaviour::new(Duration::from_millis(100), 0.0),
),
(
NetworkBehaviourKey::new(Region::Asia, Region::Europe),
NetworkBehaviour::new(Duration::from_millis(500), 0.0),
),
(
NetworkBehaviourKey::new(Region::Europe, Region::Europe),
NetworkBehaviour::new(Duration::from_millis(100), 0.0),
),
]);
let regions_data = RegionsData::new(regions, behaviour);
let mut network = Network::new(regions_data, 0);
let (from_a_sender, from_a_receiver) = channel::unbounded();
let (from_a_broadcast_sender, from_a_broadcast_receiver) = channel::unbounded();
let to_a_receiver =
network.connect(node_a, Some(2), from_a_receiver, from_a_broadcast_receiver);
let a = MockNetworkInterface::new(
node_a,
from_a_broadcast_sender,
from_a_sender,
to_a_receiver,
1,
);
let (from_b_sender, from_b_receiver) = channel::unbounded();
let (from_b_broadcast_sender, from_b_broadcast_receiver) = channel::unbounded();
let to_b_receiver =
network.connect(node_b, Some(2), from_b_receiver, from_b_broadcast_receiver);
let b = MockNetworkInterface::new(
node_b,
from_b_broadcast_sender,
from_b_sender,
to_b_receiver,
1,
);
let (from_c_sender, from_c_receiver) = channel::unbounded();
let (from_c_broadcast_sender, from_c_broadcast_receiver) = channel::unbounded();
let to_c_receiver =
network.connect(node_c, Some(2), from_c_receiver, from_c_broadcast_receiver);
let c = MockNetworkInterface::new(
node_c,
from_c_broadcast_sender,
from_c_sender,
to_c_receiver,
1,
);
a.send_message(node_b, ());
a.send_message(node_c, ());
network.collect_messages();
b.send_message(node_a, ());
b.send_message(node_c, ());
network.collect_messages();
c.send_message(node_a, ());
c.send_message(node_b, ());
network.collect_messages();
network.dispatch_after(Duration::from_millis(100));
assert_eq!(a.receive_messages().len(), 1);
assert_eq!(b.receive_messages().len(), 1);
assert_eq!(c.receive_messages().len(), 0);
a.send_message(node_b, ());
b.send_message(node_c, ());
network.collect_messages();
network.dispatch_after(Duration::from_millis(400));
assert_eq!(a.receive_messages().len(), 1); // c to a
assert_eq!(b.receive_messages().len(), 2); // c to b && a to b
assert_eq!(c.receive_messages().len(), 2); // a to c && b to c
network.dispatch_after(Duration::from_millis(100));
assert_eq!(a.receive_messages().len(), 0);
assert_eq!(b.receive_messages().len(), 0);
assert_eq!(c.receive_messages().len(), 1); // b to c
}
#[test]
fn node_network_capacity_limit() {
let node_a = NodeId::from_index(0);
let node_b = NodeId::from_index(1);
let regions = HashMap::from([(Region::Europe, vec![node_a, node_b])]);
let behaviour = HashMap::from([(
NetworkBehaviourKey::new(Region::Europe, Region::Europe),
NetworkBehaviour::new(Duration::from_millis(100), 0.0),
)]);
let regions_data = RegionsData::new(regions, behaviour);
let mut network = Network::new(regions_data, 0);
let (from_a_sender, from_a_receiver) = channel::unbounded();
let (from_a_broadcast_sender, from_a_broadcast_receiver) = channel::unbounded();
let to_a_receiver =
network.connect(node_a, Some(3), from_a_receiver, from_a_broadcast_receiver);
let a = MockNetworkInterface::new(
node_a,
from_a_broadcast_sender,
from_a_sender,
to_a_receiver,
1,
);
let (from_b_sender, from_b_receiver) = channel::unbounded();
let (from_b_broadcast_sender, from_b_broadcast_receiver) = channel::unbounded();
let to_b_receiver =
network.connect(node_b, Some(2), from_b_receiver, from_b_broadcast_receiver);
let b = MockNetworkInterface::new(
node_b,
from_b_broadcast_sender,
from_b_sender,
to_b_receiver,
1,
);
for _ in 0..6 {
a.send_message(node_b, ());
b.send_message(node_a, ());
}
network.step(Duration::from_millis(100));
assert_eq!(a.receive_messages().len(), 3);
assert_eq!(b.receive_messages().len(), 2);
network.step(Duration::from_millis(100));
assert_eq!(a.receive_messages().len(), 3);
assert_eq!(b.receive_messages().len(), 2);
network.step(Duration::from_millis(100));
assert_eq!(a.receive_messages().len(), 0);
assert_eq!(b.receive_messages().len(), 2);
}
#[test]
fn node_network_capacity_no_limit() {
let node_a = NodeId::from_index(0);
let node_b = NodeId::from_index(1);
let regions = HashMap::from([(Region::Europe, vec![node_a, node_b])]);
let behaviour = HashMap::from([(
NetworkBehaviourKey::new(Region::Europe, Region::Europe),
NetworkBehaviour::new(Duration::from_millis(100), 0.0),
)]);
let regions_data = RegionsData::new(regions, behaviour);
let mut network = Network::new(regions_data, 0);
let (from_a_sender, from_a_receiver) = channel::unbounded();
let (from_a_broadcast_sender, from_a_broadcast_receiver) = channel::unbounded();
let to_a_receiver =
network.connect(node_a, None, from_a_receiver, from_a_broadcast_receiver);
let a = MockNetworkInterface::new(
node_a,
from_a_broadcast_sender,
from_a_sender,
to_a_receiver,
1000,
);
let (from_b_sender, from_b_receiver) = channel::unbounded();
let (from_b_broadcast_sender, from_b_broadcast_receiver) = channel::unbounded();
let to_b_receiver =
network.connect(node_b, None, from_b_receiver, from_b_broadcast_receiver);
let b = MockNetworkInterface::new(
node_b,
from_b_broadcast_sender,
from_b_sender,
to_b_receiver,
100,
);
for _ in 0..6 {
a.send_message(node_b, ());
b.send_message(node_a, ());
}
network.step(Duration::from_millis(100));
assert_eq!(a.receive_messages().len(), 6);
assert_eq!(b.receive_messages().len(), 6);
}
#[test]
fn node_network_message_partial_send() {
let node_a = NodeId::from_index(0);
let node_b = NodeId::from_index(1);
let regions = HashMap::from([(Region::Europe, vec![node_a, node_b])]);
let behaviour = HashMap::from([(
NetworkBehaviourKey::new(Region::Europe, Region::Europe),
NetworkBehaviour::new(Duration::from_millis(100), 0.0),
)]);
let regions_data = RegionsData::new(regions, behaviour);
let mut network = Network::new(regions_data, 0);
let (from_a_sender, from_a_receiver) = channel::unbounded();
let (from_a_broadcast_sender, from_a_broadcast_receiver) = channel::unbounded();
// Node A is connected to the network with throuput of 5.
let to_a_receiver =
network.connect(node_a, Some(5), from_a_receiver, from_a_broadcast_receiver);
// Every message sent **from** Node A will be of size 15.
let a = MockNetworkInterface::new(
node_a,
from_a_broadcast_sender,
from_a_sender,
to_a_receiver,
2,
);
let (from_b_sender, from_b_receiver) = channel::unbounded();
let (from_b_broadcast_sender, from_b_broadcast_receiver) = channel::unbounded();
// Node B is connected to the network with throuput of 1.
let to_b_receiver =
network.connect(node_b, Some(1), from_b_receiver, from_b_broadcast_receiver);
// Every message sent **from** Node B will be of size 2.
let b = MockNetworkInterface::new(
node_b,
from_b_broadcast_sender,
from_b_sender,
to_b_receiver,
15,
);
// Node A sends message of size 2 to Node B.
a.send_message(node_b, ());
// Node B sends message of size 15 to Node A.
b.send_message(node_a, ());
// Step duration matches the latency between nodes, thus Node A can receive 5 units of a
// message, Node B - 1 unit of a message during the step.
network.step(Duration::from_millis(100));
assert_eq!(a.receive_messages().len(), 0);
assert_eq!(b.receive_messages().len(), 0);
// Node B should receive a message during the second step, because it's throughput during the
// step is 1, but the message size it receives is 2.
network.step(Duration::from_millis(100));
assert_eq!(a.receive_messages().len(), 0);
assert_eq!(b.receive_messages().len(), 1);
// Node A should receive a message during the third step, because it's throughput during the
// step is 5, but the message it receives is of size 15.
network.step(Duration::from_millis(100));
assert_eq!(a.receive_messages().len(), 1);
assert_eq!(b.receive_messages().len(), 0);
}
}

View File

@ -1,236 +0,0 @@
use carnot_engine::NodeId;
// std
use rand::{seq::SliceRandom, Rng};
use std::{collections::HashMap, str::FromStr};
// crates
use serde::{Deserialize, Serialize};
// internal
use crate::network::behaviour::NetworkBehaviour;
use super::{NetworkBehaviourKey, NetworkSettings};
#[derive(Debug, Copy, Clone, Eq, PartialEq, Hash)]
pub enum Region {
NorthAmerica,
Europe,
Asia,
Africa,
SouthAmerica,
Australia,
}
impl core::fmt::Display for Region {
fn fmt(&self, f: &mut core::fmt::Formatter<'_>) -> core::fmt::Result {
let s = match self {
Self::NorthAmerica => "NorthAmerica",
Self::Europe => "Europe",
Self::Asia => "Asia",
Self::Africa => "Africa",
Self::SouthAmerica => "SouthAmerica",
Self::Australia => "Australia",
};
write!(f, "{s}")
}
}
impl FromStr for Region {
type Err = String;
fn from_str(s: &str) -> Result<Self, Self::Err> {
match s
.trim()
.to_lowercase()
.replace(['-', '_', ' '], "")
.as_str()
{
"northamerica" | "na" => Ok(Self::NorthAmerica),
"europe" | "eu" => Ok(Self::Europe),
"asia" | "as" => Ok(Self::Asia),
"africa" | "af" => Ok(Self::Africa),
"southamerica" | "sa" => Ok(Self::SouthAmerica),
"australia" | "au" => Ok(Self::Australia),
_ => Err(format!("Unknown region: {s}")),
}
}
}
impl Serialize for Region {
fn serialize<S: serde::Serializer>(&self, serializer: S) -> Result<S::Ok, S::Error> {
let s = match self {
Self::NorthAmerica => "North America",
Self::Europe => "Europe",
Self::Asia => "Asia",
Self::Africa => "Africa",
Self::SouthAmerica => "South America",
Self::Australia => "Australia",
};
serializer.serialize_str(s)
}
}
impl<'de> Deserialize<'de> for Region {
fn deserialize<D: serde::Deserializer<'de>>(deserializer: D) -> Result<Self, D::Error> {
let s = String::deserialize(deserializer)?;
Self::from_str(&s).map_err(serde::de::Error::custom)
}
}
#[derive(Debug, Clone, Serialize, Deserialize)]
pub struct RegionsData {
pub regions: HashMap<Region, Vec<NodeId>>,
#[serde(skip)]
pub node_region: HashMap<NodeId, Region>,
pub region_network_behaviour: HashMap<NetworkBehaviourKey, NetworkBehaviour>,
}
impl RegionsData {
pub fn new(
regions: HashMap<Region, Vec<NodeId>>,
region_network_behaviour: HashMap<NetworkBehaviourKey, NetworkBehaviour>,
) -> Self {
let node_region = regions
.iter()
.flat_map(|(region, nodes)| nodes.iter().copied().map(|node| (node, *region)))
.collect();
Self {
regions,
node_region,
region_network_behaviour,
}
}
pub fn node_region(&self, node_id: NodeId) -> Region {
self.node_region[&node_id]
}
pub fn network_behaviour(&self, node_a: NodeId, node_b: NodeId) -> &NetworkBehaviour {
let region_a = self.node_region[&node_a];
let region_b = self.node_region[&node_b];
let k = NetworkBehaviourKey::new(region_a, region_b);
let k_rev = NetworkBehaviourKey::new(region_b, region_a);
self.region_network_behaviour
.get(&k)
.or(self.region_network_behaviour.get(&k_rev))
.expect("Network behaviour not found for the given regions")
}
pub fn region_nodes(&self, region: Region) -> &[NodeId] {
&self.regions[&region]
}
}
// Takes a reference to the node_ids and simulation_settings and returns a HashMap
// representing the regions and their associated node IDs.
pub fn create_regions<R: Rng>(
node_ids: &[NodeId],
rng: &mut R,
network_settings: &NetworkSettings,
) -> HashMap<Region, Vec<NodeId>> {
let mut region_nodes = node_ids.to_vec();
region_nodes.shuffle(rng);
let regions = network_settings
.regions
.clone()
.into_iter()
.collect::<Vec<_>>();
let last_region_index = regions.len() - 1;
regions
.iter()
.enumerate()
.map(|(i, (region, distribution))| {
if i < last_region_index {
let node_count = (node_ids.len() as f32 * distribution).round() as usize;
let nodes = region_nodes.drain(..node_count).collect::<Vec<_>>();
(*region, nodes)
} else {
// Assign the remaining nodes to the last region.
(*region, region_nodes.clone())
}
})
.collect()
}
#[cfg(test)]
mod tests {
use std::collections::HashMap;
use carnot_engine::NodeId;
use rand::rngs::mock::StepRng;
use crate::{
network::{
regions::{create_regions, Region},
NetworkSettings,
},
node::NodeIdExt,
};
#[test]
fn create_regions_precision() {
struct TestCase {
node_count: usize,
distributions: Vec<f32>,
}
let test_cases = vec![
TestCase {
node_count: 10,
distributions: vec![0.5, 0.3, 0.2],
},
TestCase {
node_count: 7,
distributions: vec![0.6, 0.4],
},
TestCase {
node_count: 20,
distributions: vec![0.4, 0.3, 0.2, 0.1],
},
TestCase {
node_count: 23,
distributions: vec![0.4, 0.3, 0.3],
},
TestCase {
node_count: 111,
distributions: vec![0.3, 0.3, 0.3, 0.1],
},
TestCase {
node_count: 73,
distributions: vec![0.3, 0.2, 0.2, 0.2, 0.1],
},
];
let mut rng = StepRng::new(1, 0);
for tcase in test_cases.iter() {
let nodes = (0..tcase.node_count)
.map(NodeId::from_index)
.collect::<Vec<NodeId>>();
let available_regions = [
Region::NorthAmerica,
Region::Europe,
Region::Asia,
Region::Africa,
Region::SouthAmerica,
Region::Australia,
];
let mut region_distribution = HashMap::new();
for (region, &dist) in available_regions.iter().zip(&tcase.distributions) {
region_distribution.insert(*region, dist);
}
let settings = NetworkSettings {
network_behaviors: HashMap::new(),
regions: region_distribution,
};
let regions = create_regions(&nodes, &mut rng, &settings);
let total_nodes_in_regions = regions.values().map(|v| v.len()).sum::<usize>();
assert_eq!(total_nodes_in_regions, nodes.len());
}
}
}

View File

@ -1,257 +0,0 @@
use crate::node::carnot::{messages::CarnotMessage, tally::Tally, timeout::TimeoutHandler};
use crate::node::carnot::{AggregateQc, Carnot, NewView, Qc, StandardQc, Timeout, TimeoutQc, Vote};
use carnot_consensus::network::messages::{NewViewMsg, TimeoutMsg, VoteMsg};
use carnot_consensus::NodeId;
use carnot_engine::{Overlay, View};
use nomos_core::block::Block;
use nomos_core::header::HeaderId;
use std::collections::HashSet;
use std::hash::Hash;
use std::time::Duration;
pub type CarnotTx = [u8; 32];
pub type CarnotBlob = Box<[u8]>;
pub(crate) struct EventBuilder {
id: NodeId,
leader_vote_message: Tally<VoteMsg>,
vote_message: Tally<VoteMsg>,
timeout_message: Tally<TimeoutMsg>,
leader_new_view_message: Tally<NewViewMsg>,
new_view_message: Tally<NewViewMsg>,
timeout_handler: TimeoutHandler,
pub(crate) current_view: View,
}
impl EventBuilder {
pub fn new(id: NodeId, timeout: Duration) -> Self {
Self {
vote_message: Tally::new(),
leader_vote_message: Tally::new(),
timeout_message: Tally::new(),
leader_new_view_message: Tally::new(),
new_view_message: Tally::new(),
current_view: View::default(),
id,
timeout_handler: TimeoutHandler::new(timeout),
}
}
/// At some point we need to clear old votes or RAM just goes brrrrrr
pub fn prune_by_view(&mut self, view: View) {
self.vote_message.prune(view);
self.leader_vote_message.prune(view);
self.timeout_message.prune(view);
self.leader_new_view_message.prune(view);
self.new_view_message.prune(view);
}
fn local_timeout(&mut self, view: View, elapsed: Duration) -> bool {
if self.timeout_handler.step(view, elapsed) {
self.timeout_handler.prune_by_view(view);
true
} else {
false
}
}
pub fn step<O: Overlay>(
&mut self,
mut messages: Vec<CarnotMessage>,
engine: &Carnot<O>,
elapsed: Duration,
) -> Vec<Event<CarnotTx>> {
let mut events = Vec::new();
// check timeout and exit
if self.local_timeout(engine.current_view(), elapsed) {
events.push(Event::LocalTimeout);
// if we timeout discard incoming current view messages
messages.retain(|msg| {
matches!(
msg,
CarnotMessage::Proposal(_) | CarnotMessage::TimeoutQc(_)
)
});
}
// only run when the engine is in the genesis view
if engine.highest_voted_view() == View::new(-1)
&& engine.overlay().is_member_of_leaf_committee(self.id)
{
tracing::info!(node = %self.id, "voting genesis",);
let genesis = engine.genesis_block();
events.push(Event::Approve {
qc: Qc::Standard(StandardQc {
view: genesis.view,
id: genesis.id,
}),
block: genesis,
votes: HashSet::new(),
})
}
for message in messages {
match message {
CarnotMessage::Proposal(msg) => {
let block = Block::from_bytes(&msg.data);
tracing::info!(
node=%self.id,
current_view = %engine.current_view(),
block_view=%block.header().carnot().view(),
block=?block.header().id(),
parent_block=?block.header().parent(),
"receive proposal message",
);
events.push(Event::Proposal { block })
}
CarnotMessage::TimeoutQc(msg) => {
let timeout_qc = msg.qc.clone();
events.push(Event::TimeoutQc { timeout_qc: msg.qc });
if engine.overlay().is_member_of_leaf_committee(self.id) {
events.push(Event::NewView {
timeout_qc,
new_views: Default::default(),
});
}
}
CarnotMessage::Vote(msg) => {
let msg_view = msg.vote.view;
let block_id = msg.vote.block;
let voter = msg.voter;
let is_next_view_leader = engine.is_next_leader();
let is_message_from_root_committee =
engine.overlay().is_member_of_root_committee(voter);
let tally = if is_message_from_root_committee {
&mut self.leader_vote_message
} else {
&mut self.vote_message
};
let Some(qc) = msg.qc.clone() else {
tracing::warn!(node=%self.id, current_view = %engine.current_view(), "received vote without QC");
continue;
};
// if the message comes from the root committee, then use the leader threshold, otherwise use the leaf threshold
let threshold = if is_message_from_root_committee {
engine.leader_super_majority_threshold()
} else {
engine.super_majority_threshold()
};
if let Some(votes) = tally.tally_by(msg_view, msg, threshold) {
if let Some(block) = engine
.blocks_in_view(msg_view)
.iter()
.find(|block| block.id == block_id)
.cloned()
{
tracing::info!(
node=%self.id,
votes=votes.len(),
current_view = %engine.current_view(),
block_view=%block.view,
block=%block.id,
"approve block",
);
if is_next_view_leader && is_message_from_root_committee {
events.push(Event::ProposeBlock {
qc: Qc::Standard(StandardQc {
view: block.view,
id: block.id,
}),
});
} else {
events.push(Event::Approve {
qc,
block,
votes: votes.into_iter().map(|v| v.vote).collect(),
});
}
}
}
}
CarnotMessage::Timeout(msg) => {
let msg_view = msg.vote.view;
if let Some(timeouts) = self.timeout_message.tally_by(
msg_view,
msg,
engine.overlay().super_majority_threshold(self.id),
) {
events.push(Event::RootTimeout {
timeouts: timeouts.into_iter().map(|v| v.vote).collect(),
})
}
}
CarnotMessage::NewView(msg) => {
let msg_view = msg.vote.view;
let voter = msg.voter;
let timeout_qc = msg.vote.timeout_qc.clone();
let is_next_view_leader = engine.is_next_leader();
let is_message_from_root_committee =
engine.overlay().is_member_of_root_committee(voter);
let tally = if is_message_from_root_committee {
&mut self.leader_new_view_message
} else {
&mut self.new_view_message
};
// if the message comes from the root committee, then use the leader threshold, otherwise use the leaf threshold
let threshold = if is_message_from_root_committee {
engine.leader_super_majority_threshold()
} else {
engine.super_majority_threshold()
};
if let Some(votes) = tally.tally_by(msg_view, msg, threshold) {
if is_next_view_leader && is_message_from_root_committee {
let high_qc = engine.high_qc();
events.push(Event::ProposeBlock {
qc: Qc::Aggregated(AggregateQc {
high_qc,
view: msg_view.next(),
}),
});
} else {
events.push(Event::NewView {
timeout_qc,
new_views: votes.into_iter().map(|v| v.vote).collect(),
});
}
}
}
}
}
events
}
}
pub enum Event<Tx: Clone + Hash + Eq> {
Proposal {
block: Block<Tx, Box<[u8]>>,
},
#[allow(dead_code)]
Approve {
qc: Qc,
block: carnot_engine::Block<HeaderId>,
votes: HashSet<Vote>,
},
ProposeBlock {
qc: Qc,
},
LocalTimeout,
NewView {
timeout_qc: TimeoutQc,
new_views: HashSet<NewView>,
},
TimeoutQc {
timeout_qc: TimeoutQc,
},
RootTimeout {
timeouts: HashSet<Timeout>,
},
None,
}

View File

@ -1,30 +0,0 @@
use crate::node::carnot::messages::CarnotMessage;
use carnot_engine::View;
use std::collections::HashMap;
pub(crate) struct MessageCache {
cache: HashMap<View, Vec<CarnotMessage>>,
}
impl MessageCache {
pub fn new() -> Self {
Self {
cache: Default::default(),
}
}
pub fn update<I: IntoIterator<Item = CarnotMessage>>(&mut self, messages: I) {
for message in messages {
let entry = self.cache.entry(message.view()).or_default();
entry.push(message);
}
}
pub fn prune(&mut self, view: View) {
self.cache.retain(|v, _| v > &view);
}
pub fn retrieve(&mut self, view: View) -> Vec<CarnotMessage> {
self.cache.remove(&view).unwrap_or_default()
}
}

View File

@ -1,41 +0,0 @@
use carnot_consensus::network::messages::{
NewViewMsg, ProposalMsg, TimeoutMsg, TimeoutQcMsg, VoteMsg,
};
use carnot_engine::View;
use crate::network::PayloadSize;
#[derive(Debug, Eq, PartialEq, Hash, Clone)]
pub enum CarnotMessage {
Proposal(ProposalMsg),
Vote(VoteMsg),
TimeoutQc(TimeoutQcMsg),
Timeout(TimeoutMsg),
NewView(NewViewMsg),
}
impl CarnotMessage {
pub fn view(&self) -> View {
match self {
CarnotMessage::Proposal(msg) => msg.view,
CarnotMessage::Vote(msg) => msg.vote.view,
CarnotMessage::TimeoutQc(msg) => msg.qc.view(),
CarnotMessage::Timeout(msg) => msg.vote.view,
CarnotMessage::NewView(msg) => msg.vote.view,
}
}
}
impl PayloadSize for CarnotMessage {
fn size_bytes(&self) -> u32 {
match self {
CarnotMessage::Proposal(p) => {
(std::mem::size_of::<ProposalMsg>() + p.data.len()) as u32
}
CarnotMessage::Vote(_) => std::mem::size_of::<VoteMsg>() as u32,
CarnotMessage::TimeoutQc(_) => std::mem::size_of::<TimeoutQcMsg>() as u32,
CarnotMessage::Timeout(_) => std::mem::size_of::<TimeoutMsg>() as u32,
CarnotMessage::NewView(_) => std::mem::size_of::<NewViewMsg>() as u32,
}
}
}

View File

@ -1,464 +0,0 @@
#![allow(dead_code)]
mod event_builder;
mod message_cache;
pub mod messages;
mod state;
use nomos_core::block::builder::BlockBuilder;
use nomos_core::header::HeaderId;
pub use state::*;
mod serde_util;
mod tally;
mod timeout;
use std::any::Any;
use std::collections::BTreeMap;
// std
use std::hash::Hash;
use std::time::Instant;
use std::{collections::HashMap, time::Duration};
// crates
use bls_signatures::PrivateKey;
use rand::Rng;
use serde::Deserialize;
// internal
use self::messages::CarnotMessage;
use super::{Node, NodeId};
use crate::network::{InMemoryNetworkInterface, NetworkInterface, NetworkMessage};
use crate::node::carnot::event_builder::{CarnotBlob, CarnotTx, Event};
use crate::node::carnot::message_cache::MessageCache;
use crate::output_processors::{Record, RecordType, Runtime};
use crate::settings::SimulationSettings;
use crate::streaming::SubscriberFormat;
use crate::warding::SimulationState;
use carnot_consensus::committee_membership::UpdateableCommitteeMembership;
use carnot_consensus::network::messages::{ProposalMsg, TimeoutQcMsg};
use carnot_consensus::{
leader_selection::UpdateableLeaderSelection,
network::messages::{NewViewMsg, TimeoutMsg, VoteMsg},
};
use carnot_engine::overlay::RandomBeaconState;
use carnot_engine::{Committee, LeaderProof, Overlay, View};
type Block = carnot_engine::Block<HeaderId>;
type AggregateQc = carnot_engine::AggregateQc<HeaderId>;
type Carnot<O> = carnot_engine::Carnot<O, HeaderId>;
type Payload = carnot_engine::Payload<HeaderId>;
type TimeoutQc = carnot_engine::TimeoutQc<HeaderId>;
type Vote = carnot_engine::Vote<HeaderId>;
type Qc = carnot_engine::Qc<HeaderId>;
type StandardQc = carnot_engine::StandardQc<HeaderId>;
type NewView = carnot_engine::NewView<HeaderId>;
type Timeout = carnot_engine::Timeout<HeaderId>;
static RECORD_SETTINGS: std::sync::OnceLock<BTreeMap<String, bool>> = std::sync::OnceLock::new();
#[derive(Clone, Default, Deserialize)]
pub struct CarnotSettings {
timeout: Duration,
record_settings: BTreeMap<String, bool>,
#[serde(default)]
format: SubscriberFormat,
}
impl CarnotSettings {
pub fn new(
timeout: Duration,
record_settings: BTreeMap<String, bool>,
format: SubscriberFormat,
) -> Self {
Self {
timeout,
record_settings,
format,
}
}
}
#[allow(dead_code)] // TODO: remove when handling settings
pub struct CarnotNode<O: Overlay> {
id: carnot_engine::NodeId,
state: CarnotState,
/// A step counter
current_step: usize,
settings: CarnotSettings,
network_interface: InMemoryNetworkInterface<CarnotMessage>,
message_cache: MessageCache,
event_builder: event_builder::EventBuilder,
engine: Carnot<O>,
random_beacon_pk: PrivateKey,
step_duration: Duration,
}
impl<
L: UpdateableLeaderSelection,
M: UpdateableCommitteeMembership,
O: Overlay<LeaderSelection = L, CommitteeMembership = M>,
> CarnotNode<O>
{
pub fn new<R: Rng>(
id: carnot_engine::NodeId,
settings: CarnotSettings,
overlay_settings: O::Settings,
genesis: nomos_core::block::Block<CarnotTx, CarnotBlob>,
network_interface: InMemoryNetworkInterface<CarnotMessage>,
rng: &mut R,
) -> Self {
let overlay = O::new(overlay_settings);
let engine = Carnot::from_genesis(id, genesis.header().carnot().to_carnot_block(), overlay);
let state = CarnotState::from(&engine);
let timeout = settings.timeout;
RECORD_SETTINGS.get_or_init(|| settings.record_settings.clone());
// pk is generated in an insecure way, but for simulation purpouses using a rng like smallrng is more useful
let mut pk_buff = [0; 32];
rng.fill_bytes(&mut pk_buff);
let random_beacon_pk = PrivateKey::new(pk_buff);
let mut this = Self {
id,
state,
settings,
network_interface,
message_cache: MessageCache::new(),
event_builder: event_builder::EventBuilder::new(id, timeout),
engine,
random_beacon_pk,
step_duration: Duration::ZERO,
current_step: 0,
};
this.state = CarnotState::from(&this.engine);
this.state.format = this.settings.format;
this
}
fn handle_output(&self, output: Output<CarnotTx, CarnotBlob>) {
match output {
Output::Send(carnot_engine::Send {
to,
payload: Payload::Vote(vote),
}) => {
for node in to {
self.network_interface.send_message(
node,
CarnotMessage::Vote(VoteMsg {
voter: self.id,
vote: vote.clone(),
qc: Some(Qc::Standard(StandardQc {
view: vote.view,
id: vote.block,
})),
}),
);
}
}
Output::Send(carnot_engine::Send {
to,
payload: Payload::NewView(new_view),
}) => {
for node in to {
self.network_interface.send_message(
node,
CarnotMessage::NewView(NewViewMsg {
voter: node,
vote: new_view.clone(),
}),
);
}
}
Output::Send(carnot_engine::Send {
to,
payload: Payload::Timeout(timeout),
}) => {
for node in to {
self.network_interface.send_message(
node,
CarnotMessage::Timeout(TimeoutMsg {
voter: node,
vote: timeout.clone(),
}),
);
}
}
Output::BroadcastTimeoutQc { timeout_qc } => {
self.network_interface
.broadcast(CarnotMessage::TimeoutQc(TimeoutQcMsg {
source: self.id,
qc: timeout_qc,
}));
}
Output::BroadcastProposal { proposal } => {
self.network_interface
.broadcast(CarnotMessage::Proposal(ProposalMsg {
data: proposal.as_bytes().to_vec().into(),
proposal: proposal.header().id(),
view: proposal.header().carnot().view(),
}))
}
}
}
fn process_event(&mut self, event: Event<[u8; 32]>) {
let mut output = None;
match event {
Event::Proposal { block } => {
let current_view = self.engine.current_view();
tracing::info!(
node=%self.id,
last_committed_view=%self.engine.latest_committed_view(),
current_view = %current_view,
block_view = %block.header().carnot().view(),
block = %block.header().id(),
parent_block=%block.header().parent(),
"receive block proposal",
);
match self
.engine
.receive_block(block.header().carnot().to_carnot_block())
{
Ok(mut new) => {
if self.engine.current_view() != new.current_view() {
new = Self::update_overlay_with_block(new, &block);
self.engine = new;
}
}
Err(_) => {
tracing::error!(
node = %self.id,
current_view = %self.engine.current_view(),
block_view = %block.header().carnot().view(), block = %block.header().id(),
"receive block proposal, but is invalid",
);
}
}
if self.engine.overlay().is_member_of_leaf_committee(self.id) {
// Check if we are also a member of the parent committee, this is a special case for the flat committee
let to = if self.engine.overlay().is_member_of_root_committee(self.id) {
[self.engine.overlay().next_leader()].into_iter().collect()
} else {
self.engine.parent_committee().expect(
"Parent committee of non root committee members should be present",
)
};
output = Some(Output::Send(carnot_engine::Send {
to,
payload: Payload::Vote(Vote {
view: self.engine.current_view(),
block: block.header().id(),
}),
}))
}
}
// This branch means we already get enough votes for this block
// So we can just call approve_block
Event::Approve { block, .. } => {
tracing::info!(
node = %self.id,
current_view = %self.engine.current_view(),
block_view = %block.view,
block = %block.id,
parent_block=%block.parent(),
"receive approve message"
);
let block_grandparent_view = match &block.parent_qc {
Qc::Standard(qc) => qc.view,
Qc::Aggregated(qc) => {
self.engine
.safe_blocks()
.get(&qc.high_qc.id)
.expect("Parent block must be present")
.view
}
} - View::new(3);
let (mut new, out) = self.engine.approve_block(block);
tracing::info!(vote=?out, node=%self.id);
// pruning old blocks older than the grandparent block needed to check validity
new.prune_older_blocks_by_view(block_grandparent_view);
output = Some(Output::Send(out));
self.engine = new;
}
Event::ProposeBlock { qc } => {
output = Some(Output::BroadcastProposal {
proposal: <BlockBuilder<_, _, (), ()>>::empty_carnot(
RandomBeaconState::generate_happy(qc.view().next(), &self.random_beacon_pk),
qc.view().next(),
qc,
LeaderProof::LeaderId {
leader_id: [0; 32].into(),
},
),
});
}
// This branch means we already get enough new view msgs for this qc
// So we can just call approve_new_view
Event::NewView {
timeout_qc,
new_views,
} => {
tracing::info!(
node = %self.id,
current_view = %self.engine.current_view(),
timeout_view = %timeout_qc.view(),
"receive new view message"
);
// just process timeout if node have not already process it
if timeout_qc.view() == self.engine.current_view() {
let (new, out) = self.engine.approve_new_view(timeout_qc, new_views);
output = Some(Output::Send(out));
self.engine = new;
}
}
Event::TimeoutQc { timeout_qc } => {
tracing::info!(
node = %self.id,
current_view = %self.engine.current_view(),
timeout_view = %timeout_qc.view(),
"receive timeout qc message"
);
let new = self.engine.receive_timeout_qc(timeout_qc.clone());
self.engine = Self::update_overlay_with_timeout_qc(new, &timeout_qc);
}
Event::RootTimeout { timeouts } => {
tracing::debug!("root timeout {:?}", timeouts);
if self.engine.is_member_of_root_committee() {
assert!(timeouts
.iter()
.all(|t| t.view == self.engine.current_view()));
let high_qc = timeouts
.iter()
.map(|t| &t.high_qc)
.chain(std::iter::once(&self.engine.high_qc()))
.max_by_key(|qc| qc.view)
.expect("empty root committee")
.clone();
let timeout_qc =
TimeoutQc::new(timeouts.iter().next().unwrap().view, high_qc, self.id);
output = Some(Output::BroadcastTimeoutQc { timeout_qc });
}
}
Event::LocalTimeout => {
tracing::info!(
node = %self.id,
current_view = %self.engine.current_view(),
"receive local timeout message"
);
let (new, out) = self.engine.local_timeout();
self.engine = new;
output = out.map(Output::Send);
}
Event::None => {
tracing::error!("unimplemented none branch");
unreachable!("none event will never be constructed")
}
}
if let Some(event) = output {
self.handle_output(event);
}
}
fn update_overlay_with_block<Tx: Clone + Eq + Hash>(
state: Carnot<O>,
block: &nomos_core::block::Block<Tx, CarnotBlob>,
) -> Carnot<O> {
state
.update_overlay(|overlay| {
overlay
.update_leader_selection(|leader_selection| {
leader_selection.on_new_block_received(block)
})
.expect("Leader selection update should succeed")
.update_committees(|committee_membership| {
committee_membership.on_new_block_received(block)
})
})
.unwrap_or(state)
}
fn update_overlay_with_timeout_qc(state: Carnot<O>, qc: &TimeoutQc) -> Carnot<O> {
state
.update_overlay(|overlay| {
overlay
.update_leader_selection(|leader_selection| {
leader_selection.on_timeout_qc_received(qc)
})
.expect("Leader selection update should succeed")
.update_committees(|committee_membership| {
committee_membership.on_timeout_qc_received(qc)
})
})
.unwrap_or(state)
}
}
impl<
L: UpdateableLeaderSelection,
M: UpdateableCommitteeMembership,
O: Overlay<LeaderSelection = L, CommitteeMembership = M>,
> Node for CarnotNode<O>
{
type Settings = CarnotSettings;
type State = CarnotState;
fn id(&self) -> NodeId {
self.id
}
fn current_view(&self) -> View {
self.engine.current_view()
}
fn state(&self) -> &CarnotState {
&self.state
}
fn step(&mut self, elapsed: Duration) {
let step_duration = Instant::now();
// split messages per view, we just want to process the current engine processing view or proposals or timeoutqcs
let (mut current_view_messages, other_view_messages): (Vec<_>, Vec<_>) = self
.network_interface
.receive_messages()
.into_iter()
.map(NetworkMessage::into_payload)
// do not care for older view messages
.filter(|m| m.view() >= self.engine.current_view())
.partition(|m| {
m.view() == self.engine.current_view()
|| matches!(m, CarnotMessage::Proposal(_) | CarnotMessage::TimeoutQc(_))
});
self.message_cache.prune(self.engine.current_view().prev());
self.event_builder
.prune_by_view(self.engine.current_view().prev());
self.message_cache.update(other_view_messages);
current_view_messages.append(&mut self.message_cache.retrieve(self.engine.current_view()));
let events = self
.event_builder
.step(current_view_messages, &self.engine, elapsed);
for event in events {
self.process_event(event);
}
// update state
self.state = CarnotState::new(
self.current_step,
step_duration.elapsed(),
self.settings.format,
&self.engine,
);
self.current_step += 1;
}
}
#[derive(Debug)]
#[allow(clippy::large_enum_variant)]
enum Output<Tx: Clone + Eq + Hash, Blob: Clone + Eq + Hash> {
Send(carnot_engine::Send<HeaderId>),
BroadcastTimeoutQc {
timeout_qc: TimeoutQc,
},
BroadcastProposal {
proposal: nomos_core::block::Block<Tx, Blob>,
},
}

View File

@ -1,359 +0,0 @@
use std::{collections::HashMap, time::Duration};
use serde::{
ser::{SerializeSeq, Serializer},
Deserialize, Serialize,
};
use self::{
serde_id::{BlockIdHelper, NodeIdHelper},
standard_qc::StandardQcHelper,
timeout_qc::TimeoutQcHelper,
};
use crate::node::carnot::{AggregateQc, Block, Committee, Qc, StandardQc, TimeoutQc};
use carnot_engine::View;
const NODE_ID: &str = "node_id";
const CURRENT_VIEW: &str = "current_view";
const HIGHEST_VOTED_VIEW: &str = "highest_voted_view";
const LOCAL_HIGH_QC: &str = "local_high_qc";
const SAFE_BLOCKS: &str = "safe_blocks";
const LAST_VIEW_TIMEOUT_QC: &str = "last_view_timeout_qc";
const LATEST_COMMITTED_BLOCK: &str = "latest_committed_block";
const LATEST_COMMITTED_VIEW: &str = "latest_committed_view";
const ROOT_COMMITTEE: &str = "root_committee";
const PARENT_COMMITTEE: &str = "parent_committee";
const CHILD_COMMITTEES: &str = "child_committees";
const COMMITTED_BLOCKS: &str = "committed_blocks";
const STEP_DURATION: &str = "step_duration";
pub const CARNOT_RECORD_KEYS: &[&str] = &[
CHILD_COMMITTEES,
COMMITTED_BLOCKS,
CURRENT_VIEW,
HIGHEST_VOTED_VIEW,
LAST_VIEW_TIMEOUT_QC,
LATEST_COMMITTED_BLOCK,
LATEST_COMMITTED_VIEW,
LOCAL_HIGH_QC,
NODE_ID,
PARENT_COMMITTEE,
ROOT_COMMITTEE,
SAFE_BLOCKS,
STEP_DURATION,
];
macro_rules! serializer {
($name: ident) => {
#[serde_with::skip_serializing_none]
#[serde_with::serde_as]
#[derive(Serialize, Default)]
pub(crate) struct $name<'a> {
step_id: usize,
child_committees: Option<CommitteesHelper<'a>>,
committed_blocks: Option<CommittedBlockHelper<'a>>,
current_view: Option<View>,
highest_voted_view: Option<View>,
last_view_timeout_qc: Option<Option<TimeoutQcHelper<'a>>>,
latest_committed_block: Option<BlockHelper<'a>>,
latest_committed_view: Option<View>,
local_high_qc: Option<LocalHighQcHelper<'a>>,
node_id: Option<NodeIdHelper<'a>>,
parent_committee: Option<Option<CommitteeHelper<'a>>>,
root_committee: Option<CommitteeHelper<'a>>,
safe_blocks: Option<SafeBlocksHelper<'a>>,
#[serde_as(as = "Option<serde_with::DurationMilliSeconds>")]
step_duration: Option<Duration>,
}
impl<'a> $name<'a> {
pub(crate) fn serialize_state<S: serde::ser::Serializer>(
&mut self,
keys: Vec<&String>,
state: &'a super::super::CarnotState,
serializer: S,
) -> Result<S::Ok, S::Error> {
self.step_id = state.step_id;
for k in keys {
match k.trim() {
NODE_ID => {
self.node_id = Some((&state.node_id).into());
}
CURRENT_VIEW => {
self.current_view = Some(state.current_view);
}
HIGHEST_VOTED_VIEW => {
self.highest_voted_view = Some(state.highest_voted_view);
}
LOCAL_HIGH_QC => {
self.local_high_qc = Some((&state.local_high_qc).into());
}
SAFE_BLOCKS => {
self.safe_blocks = Some((&state.safe_blocks).into());
}
LAST_VIEW_TIMEOUT_QC => {
self.last_view_timeout_qc =
Some(state.last_view_timeout_qc.as_ref().map(From::from));
}
LATEST_COMMITTED_BLOCK => {
self.latest_committed_block =
Some((&state.latest_committed_block).into());
}
LATEST_COMMITTED_VIEW => {
self.latest_committed_view = Some(state.latest_committed_view);
}
ROOT_COMMITTEE => {
self.root_committee = Some((&state.root_committee).into());
}
PARENT_COMMITTEE => {
self.parent_committee =
Some(state.parent_committee.as_ref().map(From::from));
}
CHILD_COMMITTEES => {
self.child_committees = Some(state.child_committees.as_slice().into());
}
COMMITTED_BLOCKS => {
self.committed_blocks = Some(state.committed_blocks.as_slice().into());
}
STEP_DURATION => {
self.step_duration = Some(state.step_duration);
}
_ => {}
}
}
self.serialize(serializer)
}
}
};
}
mod csv;
mod json;
pub(super) use self::csv::CarnotStateCsvSerializer;
pub(super) use json::CarnotStateJsonSerializer;
pub(crate) mod standard_qc {
use super::*;
#[derive(Serialize)]
pub(crate) struct StandardQcHelper<'a> {
view: View,
id: serde_id::BlockIdHelper<'a>,
}
impl<'a> From<&'a StandardQc> for StandardQcHelper<'a> {
fn from(val: &'a StandardQc) -> Self {
Self {
view: val.view,
id: (&val.id).into(),
}
}
}
pub fn serialize<S: Serializer>(t: &StandardQc, serializer: S) -> Result<S::Ok, S::Error> {
StandardQcHelper::from(t).serialize(serializer)
}
}
pub(crate) mod aggregate_qc {
use super::*;
#[derive(Serialize)]
pub(crate) struct AggregateQcHelper<'a> {
#[serde(serialize_with = "standard_qc::serialize")]
high_qc: &'a StandardQc,
view: View,
}
impl<'a> From<&'a AggregateQc> for AggregateQcHelper<'a> {
fn from(t: &'a AggregateQc) -> Self {
Self {
high_qc: &t.high_qc,
view: t.view,
}
}
}
pub fn serialize<S: serde::Serializer>(
t: &AggregateQc,
serializer: S,
) -> Result<S::Ok, S::Error> {
AggregateQcHelper::from(t).serialize(serializer)
}
}
pub(crate) mod qc {
use super::*;
#[derive(Serialize)]
#[serde(untagged)]
pub(crate) enum QcHelper<'a> {
Standard(#[serde(serialize_with = "standard_qc::serialize")] &'a StandardQc),
Aggregate(aggregate_qc::AggregateQcHelper<'a>),
}
impl<'a> From<&'a Qc> for QcHelper<'a> {
fn from(value: &'a Qc) -> Self {
match value {
Qc::Standard(s) => Self::Standard(s),
Qc::Aggregated(a) => Self::Aggregate(a.into()),
}
}
}
pub fn serialize<S: serde::Serializer>(t: &Qc, serializer: S) -> Result<S::Ok, S::Error> {
QcHelper::from(t).serialize(serializer)
}
}
pub(crate) mod timeout_qc {
use super::*;
use carnot_engine::NodeId;
#[derive(Serialize)]
pub(crate) struct TimeoutQcHelper<'a> {
view: View,
#[serde(serialize_with = "standard_qc::serialize")]
high_qc: &'a StandardQc,
#[serde(serialize_with = "serde_id::serialize_node_id")]
sender: NodeId,
}
impl<'a> From<&'a TimeoutQc> for TimeoutQcHelper<'a> {
fn from(value: &'a TimeoutQc) -> Self {
Self {
view: value.view(),
high_qc: value.high_qc(),
sender: value.sender(),
}
}
}
pub fn serialize<S: serde::Serializer>(
t: &TimeoutQc,
serializer: S,
) -> Result<S::Ok, S::Error> {
TimeoutQcHelper::from(t).serialize(serializer)
}
}
pub(crate) mod serde_id {
use carnot_engine::NodeId;
use nomos_core::header::HeaderId;
use super::*;
#[derive(Serialize)]
pub(crate) struct BlockIdHelper<'a> {
#[serde(with = "serde_array32")]
header: [u8; 32],
_marker: std::marker::PhantomData<&'a HeaderId>,
}
impl<'a> From<&'a HeaderId> for BlockIdHelper<'a> {
fn from(val: &'a HeaderId) -> Self {
Self {
header: (*val).into(),
_marker: std::marker::PhantomData,
}
}
}
#[derive(Serialize)]
pub(crate) struct NodeIdHelper<'a>(#[serde(with = "serde_array32")] &'a [u8; 32]);
impl<'a> From<&'a NodeId> for NodeIdHelper<'a> {
fn from(val: &'a NodeId) -> Self {
Self(val.into())
}
}
pub fn serialize_node_id<S: serde::Serializer>(
t: &NodeId,
serializer: S,
) -> Result<S::Ok, S::Error> {
NodeIdHelper::from(t).serialize(serializer)
}
pub(crate) mod serde_array32 {
use super::*;
use std::cell::RefCell;
const MAX_SERIALIZATION_LENGTH: usize = 32 * 2 + 2;
thread_local! {
static STRING_BUFFER: RefCell<String> = RefCell::new(String::with_capacity(MAX_SERIALIZATION_LENGTH));
}
pub fn serialize<S: serde::Serializer>(
t: &[u8; 32],
serializer: S,
) -> Result<S::Ok, S::Error> {
if serializer.is_human_readable() {
STRING_BUFFER.with(|s| {
let mut s = s.borrow_mut();
s.clear();
s.push_str("0x");
for v in t {
std::fmt::write(&mut *s, format_args!("{:02x}", v)).unwrap();
}
s.serialize(serializer)
})
} else {
t.serialize(serializer)
}
}
pub fn deserialize<'de, D>(deserializer: D) -> Result<[u8; 32], D::Error>
where
D: serde::Deserializer<'de>,
{
if deserializer.is_human_readable() {
<&str>::deserialize(deserializer).and_then(|s| {
super::parse_hex_from_str::<32>(s)
.map_err(<D::Error as serde::de::Error>::custom)
})
} else {
let x = <&[u8]>::deserialize(deserializer)?;
<[u8; 32]>::try_from(x).map_err(<D::Error as serde::de::Error>::custom)
}
}
}
#[derive(Debug, thiserror::Error)]
enum DecodeError {
#[error("expected str of length {expected} but got length {actual}")]
UnexpectedSize { expected: usize, actual: usize },
#[error("invalid character pair '{0}{1}'")]
InvalidCharacter(char, char),
}
fn parse_hex_from_str<const N: usize>(mut s: &str) -> Result<[u8; N], DecodeError> {
// check if we start with 0x or not
let prefix_len = if s.starts_with("0x") || s.starts_with("0X") {
s = &s[2..];
2
} else {
0
};
if s.len() != N * 2 {
return Err(DecodeError::UnexpectedSize {
expected: N * 2 + prefix_len,
actual: s.len(),
});
}
let mut output = [0; N];
for (chars, byte) in s.as_bytes().chunks_exact(2).zip(output.iter_mut()) {
let (l, r) = (chars[0] as char, chars[1] as char);
match (l.to_digit(16), r.to_digit(16)) {
(Some(l), Some(r)) => *byte = (l as u8) << 4 | r as u8,
(_, _) => return Err(DecodeError::InvalidCharacter(l, r)),
};
}
Ok(output)
}
}

View File

@ -1,166 +0,0 @@
use super::*;
use nomos_core::header::HeaderId;
use serde_block::BlockHelper;
serializer!(CarnotStateCsvSerializer);
pub(crate) mod serde_block {
use carnot_engine::LeaderProof;
use super::{qc::QcHelper, *};
#[derive(Serialize)]
#[serde(untagged)]
enum LeaderProofHelper<'a> {
LeaderId { leader_id: NodeIdHelper<'a> },
}
impl<'a> From<&'a LeaderProof> for LeaderProofHelper<'a> {
fn from(value: &'a LeaderProof) -> Self {
match value {
LeaderProof::LeaderId { leader_id } => Self::LeaderId {
leader_id: leader_id.into(),
},
}
}
}
pub(super) struct BlockHelper<'a>(BlockHelperInner<'a>);
#[derive(Serialize)]
struct BlockHelperInner<'a> {
view: View,
id: BlockIdHelper<'a>,
parent_qc: QcHelper<'a>,
leader_proof: LeaderProofHelper<'a>,
}
impl<'a> From<&'a Block> for BlockHelper<'a> {
fn from(val: &'a Block) -> Self {
Self(BlockHelperInner {
view: val.view,
id: (&val.id).into(),
parent_qc: (&val.parent_qc).into(),
leader_proof: (&val.leader_proof).into(),
})
}
}
impl<'a> serde::Serialize for BlockHelper<'a> {
fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
where
S: Serializer,
{
serde_json::to_string(&self.0)
.map_err(<S::Error as serde::ser::Error>::custom)
.and_then(|s| serializer.serialize_str(s.as_str()))
}
}
}
pub(super) struct LocalHighQcHelper<'a>(StandardQcHelper<'a>);
impl<'a> Serialize for LocalHighQcHelper<'a> {
fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
where
S: Serializer,
{
serde_json::to_string(&self.0)
.map_err(<S::Error as serde::ser::Error>::custom)
.and_then(|s| serializer.serialize_str(s.as_str()))
}
}
impl<'a> From<&'a StandardQc> for LocalHighQcHelper<'a> {
fn from(value: &'a StandardQc) -> Self {
Self(From::from(value))
}
}
struct SafeBlocksHelper<'a>(&'a HashMap<HeaderId, Block>);
impl<'a> From<&'a HashMap<HeaderId, Block>> for SafeBlocksHelper<'a> {
fn from(val: &'a HashMap<HeaderId, Block>) -> Self {
Self(val)
}
}
impl<'a> Serialize for SafeBlocksHelper<'a> {
fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
where
S: Serializer,
{
self.0
.values()
.map(|b| serde_json::to_string(&BlockHelper::from(b)))
.collect::<Result<Vec<_>, _>>()
.map_err(<S::Error as serde::ser::Error>::custom)
.and_then(|val| serializer.serialize_str(&format!("[{}]", val.join(","))))
}
}
struct CommitteeHelper<'a>(&'a Committee);
impl<'a> From<&'a Committee> for CommitteeHelper<'a> {
fn from(val: &'a Committee) -> Self {
Self(val)
}
}
impl<'a> Serialize for CommitteeHelper<'a> {
fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
where
S: Serializer,
{
self.0
.iter()
.map(|b| serde_json::to_string(&NodeIdHelper::from(b)))
.collect::<Result<Vec<_>, _>>()
.map_err(<S::Error as serde::ser::Error>::custom)
.and_then(|val| serializer.serialize_str(&format!("[{}]", val.join(","))))
}
}
struct CommitteesHelper<'a>(&'a [Committee]);
impl<'a> From<&'a [Committee]> for CommitteesHelper<'a> {
fn from(val: &'a [Committee]) -> Self {
Self(val)
}
}
impl<'a> Serialize for CommitteesHelper<'a> {
fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
where
S: Serializer,
{
self.0
.iter()
.map(|b| serde_json::to_string(&CommitteeHelper::from(b)))
.collect::<Result<Vec<_>, _>>()
.map_err(<S::Error as serde::ser::Error>::custom)
.and_then(|val| serializer.serialize_str(&format!("[{}]", val.join(","))))
}
}
struct CommittedBlockHelper<'a>(&'a [HeaderId]);
impl<'a> From<&'a [HeaderId]> for CommittedBlockHelper<'a> {
fn from(val: &'a [HeaderId]) -> Self {
Self(val)
}
}
impl<'a> Serialize for CommittedBlockHelper<'a> {
fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
where
S: Serializer,
{
self.0
.iter()
.map(|b| serde_json::to_string(&BlockIdHelper::from(b)))
.collect::<Result<Vec<_>, _>>()
.map_err(<S::Error as serde::ser::Error>::custom)
.and_then(|val| serializer.serialize_str(&format!("[{}]", val.join(","))))
}
}

View File

@ -1,138 +0,0 @@
use super::*;
use nomos_core::header::HeaderId;
use serde_block::BlockHelper;
serializer!(CarnotStateJsonSerializer);
pub(super) type LocalHighQcHelper<'a> = super::standard_qc::StandardQcHelper<'a>;
pub(crate) mod serde_block {
use carnot_engine::LeaderProof;
use super::{qc::QcHelper, *};
#[derive(Serialize)]
#[serde(untagged)]
enum LeaderProofHelper<'a> {
LeaderId { leader_id: NodeIdHelper<'a> },
}
impl<'a> From<&'a LeaderProof> for LeaderProofHelper<'a> {
fn from(value: &'a LeaderProof) -> Self {
match value {
LeaderProof::LeaderId { leader_id } => Self::LeaderId {
leader_id: leader_id.into(),
},
}
}
}
#[derive(Serialize)]
pub(crate) struct BlockHelper<'a> {
view: View,
id: BlockIdHelper<'a>,
parent_qc: QcHelper<'a>,
leader_proof: LeaderProofHelper<'a>,
}
impl<'a> From<&'a Block> for BlockHelper<'a> {
fn from(val: &'a Block) -> Self {
Self {
view: val.view,
id: (&val.id).into(),
parent_qc: (&val.parent_qc).into(),
leader_proof: (&val.leader_proof).into(),
}
}
}
pub fn serialize<S: serde::Serializer>(t: &Block, serializer: S) -> Result<S::Ok, S::Error> {
BlockHelper::from(t).serialize(serializer)
}
}
struct SafeBlocksHelper<'a>(&'a HashMap<HeaderId, Block>);
impl<'a> From<&'a HashMap<HeaderId, Block>> for SafeBlocksHelper<'a> {
fn from(val: &'a HashMap<HeaderId, Block>) -> Self {
Self(val)
}
}
impl<'a> Serialize for SafeBlocksHelper<'a> {
fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
where
S: Serializer,
{
let iter = self.0.values();
let mut s = serializer.serialize_seq(Some(iter.size_hint().0))?;
for b in iter {
s.serialize_element(&BlockHelper::from(b))?;
}
s.end()
}
}
struct CommitteeHelper<'a>(&'a Committee);
impl<'a> From<&'a Committee> for CommitteeHelper<'a> {
fn from(val: &'a Committee) -> Self {
Self(val)
}
}
impl<'a> Serialize for CommitteeHelper<'a> {
fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
where
S: Serializer,
{
let iter = self.0.iter();
let mut s = serializer.serialize_seq(Some(iter.size_hint().0))?;
for id in iter {
s.serialize_element(&NodeIdHelper::from(id))?;
}
s.end()
}
}
struct CommitteesHelper<'a>(&'a [Committee]);
impl<'a> From<&'a [Committee]> for CommitteesHelper<'a> {
fn from(val: &'a [Committee]) -> Self {
Self(val)
}
}
impl<'a> Serialize for CommitteesHelper<'a> {
fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
where
S: Serializer,
{
let mut s = serializer.serialize_seq(Some(self.0.len()))?;
for c in self.0 {
s.serialize_element(&CommitteeHelper::from(c))?;
}
s.end()
}
}
struct CommittedBlockHelper<'a>(&'a [HeaderId]);
impl<'a> From<&'a [HeaderId]> for CommittedBlockHelper<'a> {
fn from(val: &'a [HeaderId]) -> Self {
Self(val)
}
}
impl<'a> Serialize for CommittedBlockHelper<'a> {
fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
where
S: Serializer,
{
let mut s = serializer.serialize_seq(Some(self.0.len()))?;
for c in self.0 {
s.serialize_element(&BlockIdHelper::from(c))?;
}
s.end()
}
}

View File

@ -1,162 +0,0 @@
use serde::Serialize;
use super::*;
#[derive(Debug, Clone)]
pub struct CarnotState {
pub(crate) node_id: NodeId,
pub(crate) current_view: View,
pub(crate) highest_voted_view: View,
pub(crate) local_high_qc: StandardQc,
pub(crate) safe_blocks: HashMap<HeaderId, Block>,
pub(crate) last_view_timeout_qc: Option<TimeoutQc>,
pub(crate) latest_committed_block: Block,
pub(crate) latest_committed_view: View,
pub(crate) root_committee: Committee,
pub(crate) parent_committee: Option<Committee>,
pub(crate) child_committees: Vec<Committee>,
pub(crate) committed_blocks: Vec<HeaderId>,
pub(super) step_duration: Duration,
/// Step id for this state
pub(super) step_id: usize,
/// does not serialize this field, this field is used to check
/// how to serialize other fields because csv format does not support
/// nested map or struct, we have to do some customize.
pub(super) format: SubscriberFormat,
}
impl CarnotState {
pub(super) fn new<O: Overlay>(
step_id: usize,
step_duration: Duration,
fmt: SubscriberFormat,
engine: &Carnot<O>,
) -> Self {
let mut this = Self::from(engine);
this.step_id = step_id;
this.step_duration = step_duration;
this.format = fmt;
this
}
}
#[derive(Serialize)]
#[serde(untagged)]
pub enum CarnotRecord {
Runtime(Runtime),
Settings(Box<SimulationSettings>),
Data(Vec<Box<CarnotState>>),
}
impl From<Runtime> for CarnotRecord {
fn from(value: Runtime) -> Self {
Self::Runtime(value)
}
}
impl From<SimulationSettings> for CarnotRecord {
fn from(value: SimulationSettings) -> Self {
Self::Settings(Box::new(value))
}
}
impl Record for CarnotRecord {
type Data = CarnotState;
fn record_type(&self) -> RecordType {
match self {
CarnotRecord::Runtime(_) => RecordType::Meta,
CarnotRecord::Settings(_) => RecordType::Settings,
CarnotRecord::Data(_) => RecordType::Data,
}
}
fn data(&self) -> Vec<&CarnotState> {
match self {
CarnotRecord::Data(d) => d.iter().map(AsRef::as_ref).collect(),
_ => vec![],
}
}
}
impl<S, T: Clone + Serialize + 'static> TryFrom<&SimulationState<S, T>> for CarnotRecord {
type Error = anyhow::Error;
fn try_from(state: &SimulationState<S, T>) -> Result<Self, Self::Error> {
let Ok(states) = state
.nodes
.read()
.iter()
.map(|n| Box::<dyn Any + 'static>::downcast(Box::new(n.state().clone())))
.collect::<Result<Vec<_>, _>>()
else {
return Err(anyhow::anyhow!("use carnot record on other node"));
};
Ok(Self::Data(states))
}
}
impl serde::Serialize for CarnotState {
fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
where
S: serde::Serializer,
{
if let Some(rs) = RECORD_SETTINGS.get() {
let keys = rs
.iter()
.filter_map(|(k, v)| {
if serde_util::CARNOT_RECORD_KEYS.contains(&k.trim()) && *v {
Some(k)
} else {
None
}
})
.collect::<Vec<_>>();
match self.format {
SubscriberFormat::Json => serde_util::CarnotStateJsonSerializer::default()
.serialize_state(keys, self, serializer),
SubscriberFormat::Csv => serde_util::CarnotStateCsvSerializer::default()
.serialize_state(keys, self, serializer),
SubscriberFormat::Parquet => unreachable!(),
}
} else {
serializer.serialize_none()
}
}
}
impl CarnotState {
const fn keys() -> &'static [&'static str] {
serde_util::CARNOT_RECORD_KEYS
}
}
impl<O: Overlay> From<&Carnot<O>> for CarnotState {
fn from(value: &Carnot<O>) -> Self {
let node_id = value.id();
let current_view = value.current_view();
Self {
node_id,
current_view,
local_high_qc: value.high_qc(),
parent_committee: value.parent_committee(),
root_committee: value.root_committee(),
child_committees: value.child_committees(),
latest_committed_block: value.latest_committed_block(),
latest_committed_view: value.latest_committed_view(),
safe_blocks: value
.blocks_in_view(current_view)
.into_iter()
.map(|b| (b.id, b))
.collect(),
last_view_timeout_qc: value.last_view_timeout_qc(),
committed_blocks: value.latest_committed_blocks(None),
highest_voted_view: Default::default(),
step_duration: Default::default(),
format: SubscriberFormat::Csv,
step_id: 0,
}
}
}

View File

@ -1,29 +0,0 @@
use carnot_engine::View;
use std::collections::{HashMap, HashSet};
pub(crate) struct Tally<T: core::hash::Hash + Eq + Clone> {
cache: HashMap<View, HashSet<T>>,
}
impl<T: core::hash::Hash + Eq + Clone> Tally<T> {
pub fn new() -> Self {
Self {
cache: Default::default(),
}
}
pub fn tally_by(&mut self, view: View, message: T, threshold: usize) -> Option<HashSet<T>> {
let entries = self.cache.entry(view).or_default();
entries.insert(message);
let entries_len = entries.len();
if entries_len == threshold {
Some(entries.clone())
} else {
None
}
}
pub fn prune(&mut self, view: View) {
self.cache.retain(|v, _| v > &view);
}
}

View File

@ -1,37 +0,0 @@
use carnot_engine::View;
#[cfg(feature = "polars")]
use polars::export::ahash::HashMap;
#[cfg(not(feature = "polars"))]
use std::collections::HashMap;
use std::time::Duration;
pub(crate) struct TimeoutHandler {
pub timeout: Duration,
pub per_view: HashMap<View, Duration>,
}
impl TimeoutHandler {
pub fn new(timeout: Duration) -> Self {
Self {
timeout,
per_view: Default::default(),
}
}
pub fn step(&mut self, view: View, elapsed: Duration) -> bool {
let timeout = self.per_view.entry(view).or_insert(self.timeout);
*timeout = timeout.saturating_sub(elapsed);
*timeout == Duration::ZERO
}
pub fn is_timeout(&self, view: View) -> bool {
self.per_view
.get(&view)
.map(|t| t.is_zero())
.unwrap_or(false)
}
pub fn prune_by_view(&mut self, view: View) {
self.per_view.retain(|entry, _| entry > &view);
}
}

View File

@ -1,933 +0,0 @@
// std
use carnot_engine::View;
use std::collections::{BTreeMap, BTreeSet};
use std::time::Duration;
// crates
use crate::network::PayloadSize;
use serde::{Deserialize, Serialize};
// internal
use crate::{
network::{InMemoryNetworkInterface, NetworkInterface, NetworkMessage},
node::{Node, NodeId},
};
use super::{CommitteeId, OverlayGetter, OverlayState, SharedState, ViewOverlay};
#[derive(Debug, Default, Clone, Serialize)]
pub struct DummyState {
pub current_view: View,
pub message_count: usize,
pub view_state: BTreeMap<View, DummyViewState>,
}
#[derive(Debug, Default, Copy, Clone, Serialize)]
pub struct DummyViewState {
proposal_received: bool,
vote_received_count: usize,
vote_sent: bool,
}
#[derive(Clone, Default, Deserialize)]
pub struct DummySettings {}
/// Helper intent to distinguish between votes ment for different roles in the tree
/// because dummy node does not compute QC.
#[derive(Debug, Default, Clone, PartialEq, Eq)]
pub enum Intent {
FromRootToLeader,
FromInternalToInternal,
#[default]
FromLeafToInternal,
}
#[derive(Debug, Clone)]
pub struct Vote {
pub view: View,
pub intent: Intent,
}
impl Vote {
pub fn new(id: View, intent: Intent) -> Self {
Self { view: id, intent }
}
pub fn upgrade(&self, intent: Intent) -> Self {
Self {
view: self.view,
intent,
}
}
}
impl From<View> for Vote {
fn from(id: View) -> Self {
Self {
view: id,
intent: Default::default(),
}
}
}
#[derive(Debug, Clone)]
pub struct Block {
pub view: View,
}
impl Block {
pub fn new(id: View) -> Self {
Self { view: id }
}
}
impl From<View> for Block {
fn from(id: View) -> Self {
Self { view: id }
}
}
#[derive(Debug, Clone)]
pub enum DummyMessage {
Vote(Vote),
Proposal(Block),
}
impl PayloadSize for DummyMessage {
fn size_bytes(&self) -> u32 {
0
}
}
struct LocalView {
pub next_view_leaders: Vec<NodeId>,
pub current_roots: Option<BTreeSet<NodeId>>,
pub children: Option<BTreeSet<NodeId>>,
pub parents: Option<BTreeSet<NodeId>>,
pub roles: Vec<DummyRole>,
}
impl LocalView {
pub fn new<O: OverlayGetter>(node_id: NodeId, view_id: View, overlays: O) -> Self {
let view = overlays
.get_view(view_id)
.expect("simulation generated enough views");
let next_view = overlays
.get_view(view_id.next())
.expect("simulation generated enough views");
let parents = get_parent_nodes(node_id, &view);
let children = get_child_nodes(node_id, &view);
let roles = get_roles(node_id, &next_view, &parents, &children);
// In tree layout CommitteeId(0) is always root committee.
let current_roots = view
.layout
.committees
.get(&CommitteeId(0))
.map(|c| c.nodes.clone());
Self {
next_view_leaders: next_view.leaders,
current_roots,
children,
parents,
roles,
}
}
}
pub struct DummyNode {
node_id: NodeId,
state: DummyState,
_settings: DummySettings,
overlay_state: SharedState<OverlayState>,
network_interface: InMemoryNetworkInterface<DummyMessage>,
local_view: LocalView,
// Node in current view might be a leader in the next view.
// To prevent two states for different roles colliding, temp_leader_state
// is used only for leaders. It is overridden when current_view is updated.
temp_leader_state: DummyViewState,
}
#[derive(Debug, Clone, PartialEq, Eq)]
pub enum DummyRole {
Leader,
Root,
Internal,
Leaf,
Unknown,
}
impl DummyNode {
pub fn new(
node_id: NodeId,
view_id: View,
overlay_state: SharedState<OverlayState>,
network_interface: InMemoryNetworkInterface<DummyMessage>,
) -> Self {
Self {
node_id,
state: Default::default(),
_settings: Default::default(),
overlay_state: overlay_state.clone(),
network_interface,
local_view: LocalView::new(node_id, view_id, overlay_state),
temp_leader_state: Default::default(),
}
}
pub fn send_message(&self, address: NodeId, message: DummyMessage) {
self.network_interface.send_message(address, message);
}
fn broadcast(&self, addresses: &[NodeId], message: DummyMessage) {
addresses
.iter()
.for_each(|address| self.send_message(*address, message.clone()))
}
fn update_view(&mut self, view: View) {
self.state.view_state.insert(
view,
DummyViewState {
proposal_received: true,
..Default::default()
},
);
self.temp_leader_state = Default::default();
self.state.current_view = view;
self.local_view = LocalView::new(self.id(), view, self.overlay_state.clone());
}
fn is_vote_sent(&self, view: View) -> bool {
self.state
.view_state
.get(&view)
.expect("view state created")
.vote_sent
}
fn set_vote_sent(&mut self, view: View) {
let view_state = self
.state
.view_state
.get_mut(&view)
.expect("view state created");
view_state.vote_sent = true;
}
fn get_vote_count(&self, view: View) -> usize {
self.state
.view_state
.get(&view)
.expect("view state created")
.vote_received_count
}
fn increment_vote_count(&mut self, view: View) {
let view_state = self
.state
.view_state
.get_mut(&view)
.expect("view state created");
view_state.vote_received_count += 1;
}
fn has_enough_votes(&self, count: usize, children: &Option<BTreeSet<NodeId>>) -> bool {
let children = children.as_ref().expect("has children");
// TODO: Get percentage from the node settings.
let enough = children.len() as f32 * 0.66;
count > enough as usize
}
// Assumptions:
// - Leader gets vote from root nodes that are in previous overlay.
// - Leader sends NewView message to all it's view nodes if it receives votes from all root
// nodes.
fn handle_leader(&mut self, payload: &DummyMessage) {
if let DummyMessage::Vote(vote) = payload {
// Internal node can be a leader in the next view, check if the vote traversed the
// whole tree.
if vote.intent != Intent::FromRootToLeader || vote.view < self.current_view() {
return;
}
self.temp_leader_state.vote_received_count += 1;
if !self.temp_leader_state.vote_sent
&& self.has_enough_votes(
self.temp_leader_state.vote_received_count,
&self.local_view.current_roots,
)
{
let new_view_id = self.current_view().next();
self.broadcast(
&self.overlay_state.get_all_nodes(),
DummyMessage::Proposal(new_view_id.into()),
);
self.temp_leader_state.vote_sent = true;
}
}
}
fn handle_root(&mut self, payload: &DummyMessage) {
if let DummyMessage::Vote(vote) = payload {
// Root node can be a leader in the next view, check if the vote traversed the
// whole tree.
if vote.intent != Intent::FromInternalToInternal || vote.view != self.current_view() {
return;
}
self.increment_vote_count(vote.view);
if !self.is_vote_sent(vote.view)
&& self.has_enough_votes(self.get_vote_count(vote.view), &self.local_view.children)
{
self.broadcast(
&self.local_view.next_view_leaders,
DummyMessage::Vote(vote.upgrade(Intent::FromRootToLeader)),
);
self.set_vote_sent(vote.view);
}
}
}
fn handle_internal(&mut self, message: &DummyMessage) {
if let DummyMessage::Vote(vote) = &message {
// Internal node can be a leader in the next view, check if the vote traversed the
// whole tree.
if vote.intent != Intent::FromLeafToInternal
&& vote.intent != Intent::FromInternalToInternal
|| vote.view != self.current_view()
{
return;
}
self.increment_vote_count(vote.view);
if !self.is_vote_sent(vote.view)
&& self.has_enough_votes(self.get_vote_count(vote.view), &self.local_view.children)
{
let parents = self
.local_view
.parents
.as_ref()
.expect("internal has parents");
parents.iter().for_each(|node_id| {
self.send_message(
*node_id,
DummyMessage::Vote(vote.upgrade(Intent::FromInternalToInternal)),
)
});
self.set_vote_sent(vote.view);
}
}
}
fn handle_leaf(&mut self, payload: &DummyMessage) {
if let DummyMessage::Proposal(block) = &payload {
if !self.is_vote_sent(block.view) {
let parents = &self.local_view.parents.as_ref().expect("leaf has parents");
parents.iter().for_each(|node_id| {
self.send_message(*node_id, DummyMessage::Vote(block.view.into()))
});
self.set_vote_sent(block.view);
}
}
}
fn handle_message(&mut self, message: &NetworkMessage<DummyMessage>) {
// The view can change on any message, node needs to change its position
// and roles if the view changes during the message processing.
if let DummyMessage::Proposal(block) = &message.payload {
if block.view > self.current_view() {
self.update_view(block.view);
}
}
let roles = self.local_view.roles.clone();
for role in roles.iter() {
match role {
DummyRole::Leader => self.handle_leader(&message.payload),
DummyRole::Root => self.handle_root(&message.payload),
DummyRole::Internal => self.handle_internal(&message.payload),
DummyRole::Leaf => self.handle_leaf(&message.payload),
DummyRole::Unknown => (),
}
}
}
}
impl Node for DummyNode {
type Settings = DummySettings;
type State = DummyState;
fn id(&self) -> NodeId {
self.node_id
}
fn current_view(&self) -> View {
self.state.current_view
}
fn state(&self) -> &DummyState {
&self.state
}
fn step(&mut self, _: Duration) {
let incoming_messages = self.network_interface.receive_messages();
self.state.message_count += incoming_messages.len();
incoming_messages
.iter()
.for_each(|m| self.handle_message(m));
}
}
fn get_parent_nodes(node_id: NodeId, view: &ViewOverlay) -> Option<BTreeSet<NodeId>> {
let committee_id = view.layout.committee(node_id)?;
view.layout.parent_nodes(committee_id).map(|c| c.nodes)
}
fn get_child_nodes(node_id: NodeId, view: &ViewOverlay) -> Option<BTreeSet<NodeId>> {
let committee_id = view.layout.committee(node_id)?;
let child_nodes: BTreeSet<NodeId> = view
.layout
.children_nodes(committee_id)
.iter()
.flat_map(|c| c.nodes.clone())
.collect();
match child_nodes.len() {
0 => None,
_ => Some(child_nodes),
}
}
fn get_roles(
node_id: NodeId,
next_view: &ViewOverlay,
parents: &Option<BTreeSet<NodeId>>,
children: &Option<BTreeSet<NodeId>>,
) -> Vec<DummyRole> {
let mut roles = Vec::new();
if next_view.leaders.contains(&node_id) {
roles.push(DummyRole::Leader);
}
match (parents, children) {
(None, Some(_)) => roles.push(DummyRole::Root),
(Some(_), Some(_)) => roles.push(DummyRole::Internal),
(Some(_), None) => roles.push(DummyRole::Leaf),
(None, None) => {
//roles.push(DummyRole::Root);
//roles.push(DummyRole::Leaf);
roles.push(DummyRole::Unknown);
}
};
roles
}
#[cfg(test)]
mod tests {
use std::{
collections::{BTreeMap, BTreeSet, HashMap},
sync::Arc,
time::{Duration, SystemTime, UNIX_EPOCH},
};
use carnot_engine::View;
use crossbeam::channel;
use parking_lot::RwLock;
use rand::{
rngs::{mock::StepRng, SmallRng},
Rng, SeedableRng,
};
use rayon::prelude::*;
use crate::{
network::{
behaviour::NetworkBehaviour,
regions::{Region, RegionsData},
InMemoryNetworkInterface, Network, NetworkBehaviourKey,
},
node::{
dummy::{get_child_nodes, get_parent_nodes, get_roles, DummyRole},
Node, NodeId, NodeIdExt, OverlayState, SharedState, SimulationOverlay, ViewOverlay,
},
overlay::tests::{
tree::{TreeOverlay, TreeSettings},
Overlay,
},
};
use super::{DummyMessage, DummyNode, Intent, Vote};
fn init_network(node_ids: &[NodeId]) -> Network<DummyMessage> {
let regions = HashMap::from([(Region::Europe, node_ids.to_vec())]);
let behaviour = HashMap::from([(
NetworkBehaviourKey::new(Region::Europe, Region::Europe),
NetworkBehaviour::new(Duration::from_millis(100), 0.0),
)]);
let regions_data = RegionsData::new(regions, behaviour);
Network::new(regions_data, 0)
}
fn init_dummy_nodes(
node_ids: &[NodeId],
network: &mut Network<DummyMessage>,
overlay_state: SharedState<OverlayState>,
) -> HashMap<NodeId, DummyNode> {
node_ids
.iter()
.map(|node_id| {
let (node_message_sender, node_message_receiver) = channel::unbounded();
let (node_message_broadcast_sender, node_message_broadcast_receiver) =
channel::unbounded();
let network_message_receiver = network.connect(
*node_id,
None,
node_message_receiver,
node_message_broadcast_receiver,
);
let network_interface = InMemoryNetworkInterface::new(
*node_id,
node_message_broadcast_sender,
node_message_sender,
network_message_receiver,
);
(
*node_id,
DummyNode::new(
*node_id,
View::new(0),
overlay_state.clone(),
network_interface,
),
)
})
.collect()
}
fn generate_overlays<R: Rng>(
node_ids: &[NodeId],
overlay: &SimulationOverlay,
overlay_count: usize,
leader_count: usize,
rng: &mut R,
) -> BTreeMap<View, ViewOverlay> {
(0..overlay_count)
.map(|view_id| {
(
View::new(view_id as i64),
ViewOverlay {
leaders: overlay.leaders(node_ids, leader_count, rng).collect(),
layout: overlay.layout(node_ids, rng),
},
)
})
.collect()
}
fn send_initial_votes(
overlays: &BTreeMap<View, ViewOverlay>,
committee_size: usize,
nodes: &HashMap<NodeId, DummyNode>,
) {
let initial_vote = Vote::new(View::new(1), Intent::FromRootToLeader);
overlays
.get(&View::new(1))
.unwrap()
.leaders
.iter()
.for_each(|leader_id| {
for _ in 0..committee_size {
nodes
.get(&NodeId::from_index(0))
.unwrap()
.send_message(*leader_id, DummyMessage::Vote(initial_vote.clone()));
}
});
}
#[test]
fn send_receive_tree_overlay_steprng() {
let mut rng = StepRng::new(1, 0);
// 0
// 1 2
// 3 4 5 6
let overlay = TreeOverlay::new(TreeSettings {
tree_type: Default::default(),
depth: 3,
committee_size: 1,
});
let node_ids: Vec<NodeId> = overlay.nodes();
let mut network = init_network(&node_ids);
let view = ViewOverlay {
leaders: vec![
NodeId::from_index(0),
NodeId::from_index(1),
NodeId::from_index(2),
],
layout: overlay.layout(&node_ids, &mut rng),
};
let overlay_state = Arc::new(RwLock::new(OverlayState {
all_nodes: node_ids.clone(),
overlay: SimulationOverlay::Tree(overlay),
overlays: BTreeMap::from([
(View::new(0), view.clone()),
(View::new(1), view.clone()),
(View::new(2), view.clone()),
(View::new(3), view),
]),
}));
let mut nodes = init_dummy_nodes(&node_ids, &mut network, overlay_state);
let initial_vote = Vote::new(View::new(1), Intent::FromRootToLeader);
// Using any node as the sender for initial proposal to leader nodes.
nodes[&NodeId::from_index(0)].send_message(
NodeId::from_index(0),
DummyMessage::Vote(initial_vote.clone()),
);
nodes[&NodeId::from_index(0)].send_message(
NodeId::from_index(1),
DummyMessage::Vote(initial_vote.clone()),
);
nodes[&NodeId::from_index(0)]
.send_message(NodeId::from_index(2), DummyMessage::Vote(initial_vote));
network.collect_messages();
for (_, node) in nodes.iter() {
assert_eq!(node.current_view(), View::new(0));
}
let elapsed = Duration::from_millis(100);
// 1. Leaders receive vote and broadcast new Proposal(Block) to all nodes.
network.dispatch_after(elapsed);
nodes.iter_mut().for_each(|(_, node)| {
node.step(elapsed);
});
network.collect_messages();
// 2. a) All nodes received proposal block.
// b) Leaf nodes send vote to internal nodes.
network.dispatch_after(elapsed);
nodes.iter_mut().for_each(|(_, node)| {
node.step(elapsed);
});
network.collect_messages();
// All nodes should be updated to the proposed blocks view.
for (_, node) in nodes.iter() {
assert_eq!(node.current_view(), View::new(1));
}
// Root and Internal haven't sent their votes yet.
assert!(!nodes[&NodeId::from_index(0)].state().view_state[&View::new(1)].vote_sent); // Root
assert!(!nodes[&NodeId::from_index(1)].state().view_state[&View::new(1)].vote_sent); // Internal
assert!(!nodes[&NodeId::from_index(2)].state().view_state[&View::new(1)].vote_sent); // Internal
// Leaves should have their vote sent.
assert!(nodes[&NodeId::from_index(3)].state().view_state[&View::new(1)].vote_sent); // Leaf
assert!(nodes[&NodeId::from_index(4)].state().view_state[&View::new(1)].vote_sent); // Leaf
assert!(nodes[&NodeId::from_index(5)].state().view_state[&View::new(1)].vote_sent); // Leaf
assert!(nodes[&NodeId::from_index(6)].state().view_state[&View::new(1)].vote_sent); // Leaf
// 3. Internal nodes send vote to root node.
network.dispatch_after(elapsed);
nodes.iter_mut().for_each(|(_, node)| {
node.step(elapsed);
});
network.collect_messages();
// Root hasn't sent its votes yet.
assert!(!nodes[&NodeId::from_index(0)].state().view_state[&View::new(1)].vote_sent); // Root
// Internal and leaves should have their vote sent.
assert!(nodes[&NodeId::from_index(1)].state().view_state[&View::new(1)].vote_sent); // Internal
assert!(nodes[&NodeId::from_index(2)].state().view_state[&View::new(1)].vote_sent); // Internal
assert!(nodes[&NodeId::from_index(3)].state().view_state[&View::new(1)].vote_sent); // Leaf
assert!(nodes[&NodeId::from_index(4)].state().view_state[&View::new(1)].vote_sent); // Leaf
assert!(nodes[&NodeId::from_index(5)].state().view_state[&View::new(1)].vote_sent); // Leaf
assert!(nodes[&NodeId::from_index(6)].state().view_state[&View::new(1)].vote_sent); // Leaf
// 4. Root node send vote to next view leader nodes.
network.dispatch_after(elapsed);
nodes.iter_mut().for_each(|(_, node)| {
node.step(elapsed);
});
network.collect_messages();
// Root has sent its votes.
assert!(nodes[&NodeId::from_index(0)].state().view_state[&View::new(1)].vote_sent); // Root
assert!(nodes[&NodeId::from_index(1)].state().view_state[&View::new(1)].vote_sent); // Internal
assert!(nodes[&NodeId::from_index(2)].state().view_state[&View::new(1)].vote_sent); // Internal
assert!(nodes[&NodeId::from_index(3)].state().view_state[&View::new(1)].vote_sent); // Leaf
assert!(nodes[&NodeId::from_index(4)].state().view_state[&View::new(1)].vote_sent); // Leaf
assert!(nodes[&NodeId::from_index(5)].state().view_state[&View::new(1)].vote_sent); // Leaf
assert!(nodes[&NodeId::from_index(6)].state().view_state[&View::new(1)].vote_sent); // Leaf
// 5. Leaders receive vote and broadcast new Proposal(Block) to all nodes.
network.dispatch_after(elapsed);
nodes.iter_mut().for_each(|(_, node)| {
node.step(elapsed);
});
network.collect_messages();
// All nodes should be in an old view
for (_, node) in nodes.iter() {
assert_eq!(node.current_view(), View::new(1)); // old
}
// 6. a) All nodes received proposal block.
// b) Leaf nodes send vote to internal nodes.
network.dispatch_after(elapsed);
nodes.iter_mut().for_each(|(_, node)| {
node.step(elapsed);
});
network.collect_messages();
// All nodes should be updated to the proposed blocks view.
for (_, node) in nodes.iter() {
assert_eq!(node.current_view(), View::new(2)); // new
}
// Root and Internal haven't sent their votes yet.
assert!(!nodes[&NodeId::from_index(0)].state().view_state[&View::new(2)].vote_sent); // Root
assert!(!nodes[&NodeId::from_index(1)].state().view_state[&View::new(2)].vote_sent); // Internal
assert!(!nodes[&NodeId::from_index(2)].state().view_state[&View::new(2)].vote_sent); // Internal
// Leaves should have their vote sent.
assert!(nodes[&NodeId::from_index(3)].state().view_state[&View::new(2)].vote_sent); // Leaf
assert!(nodes[&NodeId::from_index(4)].state().view_state[&View::new(2)].vote_sent); // Leaf
assert!(nodes[&NodeId::from_index(5)].state().view_state[&View::new(2)].vote_sent); // Leaf
assert!(nodes[&NodeId::from_index(6)].state().view_state[&View::new(2)].vote_sent);
// Leaf
}
#[test]
fn send_receive_tree_overlay_smallrng() {
// Nodes should progress to second view in 6 steps with random nodes in tree overlay if
// network conditions are the same as in `send_receive_tree_overlay` test.
let timestamp = SystemTime::now()
.duration_since(UNIX_EPOCH)
.expect("valid unix timestamp")
.as_secs();
let mut rng = SmallRng::seed_from_u64(timestamp);
let committee_size = 1;
let overlay = SimulationOverlay::Tree(TreeOverlay::new(TreeSettings {
tree_type: Default::default(),
depth: 3,
committee_size,
}));
// There are more nodes in the network than in a tree overlay.
let node_ids: Vec<NodeId> = (0..100).map(NodeId::from_index).collect();
let mut network = init_network(&node_ids);
let overlays = generate_overlays(&node_ids, &overlay, 4, 3, &mut rng);
let overlay_state = Arc::new(RwLock::new(OverlayState {
all_nodes: node_ids.clone(),
overlay,
overlays: overlays.clone(),
}));
let mut nodes = init_dummy_nodes(&node_ids, &mut network, overlay_state);
// Using any node as the sender for initial proposal to leader nodes.
send_initial_votes(&overlays, committee_size, &nodes);
network.collect_messages();
for (_, node) in nodes.iter() {
assert_eq!(node.current_view(), View::new(0));
}
let elapsed = Duration::from_millis(100);
for _ in 0..7 {
network.dispatch_after(elapsed);
nodes.iter_mut().for_each(|(_, node)| {
node.step(elapsed);
});
network.collect_messages();
}
for (_, node) in nodes.iter() {
assert_eq!(node.current_view(), View::new(2));
}
}
#[test]
#[ignore]
fn send_receive_tree_overlay_medium_network() {
let timestamp = SystemTime::now()
.duration_since(UNIX_EPOCH)
.expect("valid unix timestamp")
.as_secs();
let mut rng = SmallRng::seed_from_u64(timestamp);
let committee_size = 100;
let overlay = SimulationOverlay::Tree(TreeOverlay::new(TreeSettings {
tree_type: Default::default(),
depth: 3,
committee_size,
}));
// There are more nodes in the network than in a tree overlay.
let node_ids: Vec<NodeId> = (0..10000).map(NodeId::from_index).collect();
let mut network = init_network(&node_ids);
let overlays = generate_overlays(&node_ids, &overlay, 4, 100, &mut rng);
let overlay_state = Arc::new(RwLock::new(OverlayState {
all_nodes: node_ids.clone(),
overlay,
overlays: overlays.clone(),
}));
let mut nodes = init_dummy_nodes(&node_ids, &mut network, overlay_state);
// Using any node as the sender for initial proposal to leader nodes.
send_initial_votes(&overlays, committee_size, &nodes);
network.collect_messages();
for (_, node) in nodes.iter() {
assert_eq!(node.current_view(), View::new(0));
}
let elapsed = Duration::from_millis(100);
for _ in 0..7 {
network.dispatch_after(elapsed);
nodes.iter_mut().for_each(|(_, node)| {
node.step(elapsed);
});
network.collect_messages();
}
for (_, node) in nodes.iter() {
assert_eq!(node.current_view(), View::new(2));
}
}
#[test]
#[ignore]
fn send_receive_tree_overlay_large_network() {
let timestamp = SystemTime::now()
.duration_since(UNIX_EPOCH)
.expect("valid unix timestamp")
.as_secs();
let mut rng = SmallRng::seed_from_u64(timestamp);
let committee_size = 500;
let overlay = SimulationOverlay::Tree(TreeOverlay::new(TreeSettings {
tree_type: Default::default(),
depth: 5,
committee_size,
}));
// There are more nodes in the network than in a tree overlay.
let node_ids: Vec<NodeId> = (0..100000).map(NodeId::from_index).collect();
let mut network = init_network(&node_ids);
let overlays = generate_overlays(&node_ids, &overlay, 4, 1000, &mut rng);
let overlay_state = Arc::new(RwLock::new(OverlayState {
all_nodes: node_ids.clone(),
overlay,
overlays: overlays.clone(),
}));
let nodes = init_dummy_nodes(&node_ids, &mut network, overlay_state);
// Using any node as the sender for initial proposal to leader nodes.
send_initial_votes(&overlays, committee_size, &nodes);
network.collect_messages();
let nodes = Arc::new(RwLock::new(nodes));
let elapsed = Duration::from_millis(100);
for _ in 0..9 {
network.dispatch_after(elapsed);
nodes.write().par_iter_mut().for_each(|(_, node)| {
node.step(elapsed);
});
network.collect_messages();
}
for (_, node) in nodes.read().iter() {
assert_eq!(node.current_view(), View::new(2));
}
}
#[test]
fn get_related_nodes() {
// 0
// 1 2
// 3 4 5 6
let test_cases = vec![
(
0,
None,
Some(BTreeSet::from([
NodeId::from_index(1),
NodeId::from_index(2),
])),
vec![DummyRole::Root],
),
(
1,
Some(BTreeSet::from([NodeId::from_index(0)])),
Some(BTreeSet::from([
NodeId::from_index(3),
NodeId::from_index(4),
])),
vec![DummyRole::Internal],
),
(
2,
Some(BTreeSet::from([NodeId::from_index(0)])),
Some(BTreeSet::from([
NodeId::from_index(5),
NodeId::from_index(6),
])),
vec![DummyRole::Internal],
),
(
3,
Some(BTreeSet::from([NodeId::from_index(1)])),
None,
vec![DummyRole::Leaf],
),
(
4,
Some(BTreeSet::from([NodeId::from_index(1)])),
None,
vec![DummyRole::Leaf],
),
(
5,
Some(BTreeSet::from([NodeId::from_index(2)])),
None,
vec![DummyRole::Leaf],
),
(
6,
Some(BTreeSet::from([NodeId::from_index(2)])),
None,
vec![DummyRole::Leader, DummyRole::Leaf],
),
];
let mut rng = StepRng::new(1, 0);
let overlay = TreeOverlay::new(TreeSettings {
tree_type: Default::default(),
depth: 3,
committee_size: 1,
});
let node_ids: Vec<NodeId> = overlay.nodes();
let leaders = vec![NodeId::from_index(6)];
let layout = overlay.layout(&node_ids, &mut rng);
let view = ViewOverlay { leaders, layout };
for (nid, expected_parents, expected_children, expected_roles) in test_cases {
let node_id = NodeId::from_index(nid);
let parents = get_parent_nodes(node_id, &view);
let children = get_child_nodes(node_id, &view);
let role = get_roles(node_id, &view, &parents, &children);
assert_eq!(parents, expected_parents);
assert_eq!(children, expected_children);
assert_eq!(role, expected_roles);
}
}
}

View File

@ -1,50 +0,0 @@
use carnot_engine::View;
use serde::{Deserialize, Serialize};
use std::time::Duration;
use super::{Node, NodeId};
#[derive(Debug, Default, Copy, Clone, Serialize, Deserialize)]
pub struct DummyStreamingState {
pub current_view: View,
}
/// This node implementation only used for testing different streaming implementation purposes.
pub struct DummyStreamingNode<S> {
id: NodeId,
state: DummyStreamingState,
#[allow(dead_code)]
settings: S,
}
impl<S> DummyStreamingNode<S> {
pub fn new(id: NodeId, settings: S) -> Self {
Self {
id,
state: DummyStreamingState::default(),
settings,
}
}
}
impl<S> Node for DummyStreamingNode<S> {
type Settings = S;
type State = DummyStreamingState;
fn id(&self) -> NodeId {
self.id
}
fn current_view(&self) -> View {
self.state.current_view
}
fn state(&self) -> &Self::State {
&self.state
}
fn step(&mut self, _: Duration) {
self.state.current_view = self.state.current_view.next();
}
}

View File

@ -1,203 +0,0 @@
pub mod carnot;
pub mod dummy;
#[cfg(test)]
pub mod dummy_streaming;
// std
use carnot_engine::View;
use std::{
collections::BTreeMap,
ops::{Deref, DerefMut},
sync::Arc,
time::Duration,
};
// crates
use parking_lot::RwLock;
use serde::{Deserialize, Serialize};
// internal
use crate::overlay::tests::{Layout, OverlaySettings, SimulationOverlay};
pub use carnot_engine::NodeId;
#[derive(Copy, Clone, Debug, PartialEq, Eq, PartialOrd, Ord, Hash, Serialize, Deserialize)]
#[serde(transparent)]
pub struct CommitteeId(usize);
impl CommitteeId {
#[inline]
pub const fn new(id: usize) -> Self {
Self(id)
}
}
impl From<usize> for CommitteeId {
fn from(id: usize) -> Self {
Self(id)
}
}
#[serde_with::serde_as]
#[derive(Copy, Clone, Debug, PartialEq, Eq, PartialOrd, Ord, Hash, Serialize, Deserialize)]
#[serde(transparent)]
pub struct StepTime(#[serde_as(as = "serde_with::DurationMilliSeconds")] Duration);
impl From<Duration> for StepTime {
fn from(duration: Duration) -> Self {
Self(duration)
}
}
impl StepTime {
#[inline]
pub const fn new(duration: Duration) -> Self {
Self(duration)
}
#[inline]
pub const fn into_inner(&self) -> Duration {
self.0
}
#[inline]
pub const fn from_millis(millis: u64) -> Self {
Self(Duration::from_millis(millis))
}
#[inline]
pub const fn from_secs(secs: u64) -> Self {
Self(Duration::from_secs(secs))
}
}
impl Deref for StepTime {
type Target = Duration;
fn deref(&self) -> &Self::Target {
&self.0
}
}
impl DerefMut for StepTime {
fn deref_mut(&mut self) -> &mut Self::Target {
&mut self.0
}
}
impl core::iter::Sum<Self> for StepTime {
fn sum<I: Iterator<Item = Self>>(iter: I) -> Self {
Self(iter.into_iter().map(|s| s.0).sum())
}
}
impl core::iter::Sum<Duration> for StepTime {
fn sum<I: Iterator<Item = Duration>>(iter: I) -> Self {
Self(iter.into_iter().sum())
}
}
impl core::iter::Sum<StepTime> for Duration {
fn sum<I: Iterator<Item = StepTime>>(iter: I) -> Self {
iter.into_iter().map(|s| s.0).sum()
}
}
#[derive(Clone, Debug)]
pub struct ViewOverlay {
pub leaders: Vec<NodeId>,
pub layout: Layout,
}
impl From<OverlaySettings> for ViewOverlay {
fn from(value: OverlaySettings) -> Self {
match value {
OverlaySettings::Flat => {
todo!()
}
OverlaySettings::Tree(_) => {
todo!()
}
}
}
}
pub type SharedState<S> = Arc<RwLock<S>>;
/// A state that represents how nodes are interconnected in the network.
pub struct OverlayState {
pub all_nodes: Vec<NodeId>,
pub overlay: SimulationOverlay,
pub overlays: BTreeMap<View, ViewOverlay>,
}
pub trait OverlayGetter {
fn get_view(&self, index: View) -> Option<ViewOverlay>;
fn get_all_nodes(&self) -> Vec<NodeId>;
}
impl OverlayGetter for SharedState<OverlayState> {
fn get_view(&self, index: View) -> Option<ViewOverlay> {
let overlay_state = self.read();
overlay_state.overlays.get(&index).cloned()
}
fn get_all_nodes(&self) -> Vec<NodeId> {
let overlay_state = self.read();
overlay_state.all_nodes.clone()
}
}
pub trait Node {
type Settings;
type State;
fn id(&self) -> NodeId;
fn current_view(&self) -> View;
fn state(&self) -> &Self::State;
fn step(&mut self, elapsed: Duration);
}
#[cfg(test)]
impl Node for usize {
type Settings = ();
type State = Self;
fn id(&self) -> NodeId {
NodeId::from_index(*self)
}
fn current_view(&self) -> View {
View::new(*self as i64)
}
fn state(&self) -> &Self::State {
self
}
fn step(&mut self, _: Duration) {
use std::ops::AddAssign;
self.add_assign(1);
}
}
pub trait NodeIdExt {
fn index(&self) -> usize;
fn from_index(idx: usize) -> Self;
}
impl NodeIdExt for NodeId {
fn index(&self) -> usize {
const SIZE: usize = core::mem::size_of::<usize>();
let mut bytes = [0u8; SIZE];
let src: [u8; 32] = (*self).into();
bytes.copy_from_slice(&src[..SIZE]);
usize::from_be_bytes(bytes)
}
fn from_index(idx: usize) -> Self {
let mut bytes = [0u8; 32];
bytes[..core::mem::size_of::<usize>()].copy_from_slice(&idx.to_be_bytes());
NodeId::new(bytes)
}
}

View File

@ -1,132 +0,0 @@
use std::time::Duration;
use chrono::{DateTime, Utc};
use serde::Serialize;
use crate::settings::SimulationSettings;
use crate::warding::SimulationState;
#[derive(Debug, Copy, Clone, PartialEq, Eq)]
pub enum RecordType {
Meta,
Settings,
Data,
}
pub trait Record: From<Runtime> + From<SimulationSettings> + Send + Sync + 'static {
type Data: serde::Serialize;
fn record_type(&self) -> RecordType;
fn is_settings(&self) -> bool {
self.record_type() == RecordType::Settings
}
fn is_meta(&self) -> bool {
self.record_type() == RecordType::Meta
}
fn is_data(&self) -> bool {
self.record_type() == RecordType::Data
}
fn data(&self) -> Vec<&Self::Data>;
}
pub type SerializedNodeState = serde_json::Value;
#[derive(Serialize)]
pub struct Runtime {
start: DateTime<Utc>,
end: DateTime<Utc>,
elapsed: Duration,
}
impl Runtime {
pub(crate) fn load() -> anyhow::Result<Self> {
let elapsed = crate::START_TIME.elapsed();
let end = Utc::now();
Ok(Self {
start: end
.checked_sub_signed(chrono::Duration::from_std(elapsed)?)
.unwrap(),
end,
elapsed,
})
}
}
#[derive(Serialize)]
#[serde(untagged)]
pub enum OutData {
Runtime(Runtime),
Settings(Box<SimulationSettings>),
Data(SerializedNodeState),
}
impl From<Runtime> for OutData {
fn from(runtime: Runtime) -> Self {
Self::Runtime(runtime)
}
}
impl From<SimulationSettings> for OutData {
fn from(settings: SimulationSettings) -> Self {
Self::Settings(Box::new(settings))
}
}
impl From<SerializedNodeState> for OutData {
fn from(state: SerializedNodeState) -> Self {
Self::Data(state)
}
}
impl Record for OutData {
type Data = SerializedNodeState;
fn record_type(&self) -> RecordType {
match self {
Self::Runtime(_) => RecordType::Meta,
Self::Settings(_) => RecordType::Settings,
Self::Data(_) => RecordType::Data,
}
}
fn data(&self) -> Vec<&SerializedNodeState> {
match self {
Self::Data(d) => vec![d],
_ => unreachable!(),
}
}
}
impl OutData {
#[inline]
pub const fn new(state: SerializedNodeState) -> Self {
Self::Data(state)
}
}
impl<S, T: Serialize + Clone> TryFrom<&SimulationState<S, T>> for OutData {
type Error = anyhow::Error;
fn try_from(state: &SimulationState<S, T>) -> Result<Self, Self::Error> {
serde_json::to_value(
state
.nodes
.read()
.iter()
.map(|n| n.state())
.collect::<Vec<_>>(),
)
.map(OutData::new)
.map_err(From::from)
}
}
pub trait NodeStateRecord {
fn get_serialized_state_record(&self) -> SerializedNodeState {
SerializedNodeState::Null
}
}

View File

@ -1,2 +0,0 @@
pub mod overlay_info;
pub mod tests;

View File

@ -1,181 +0,0 @@
use carnot_engine::{CommitteeId, NodeId, Overlay};
use serde::Serialize;
use std::collections::{BTreeSet, HashMap, VecDeque};
pub type Blake2bU32 = blake2::Blake2b<digest::typenum::U32>;
#[derive(Debug, Serialize)]
pub struct OverlayInfo {
pub committees: BTreeSet<CommitteeId>,
pub committee_sizes: HashMap<CommitteeId, usize>,
pub edges: Vec<(CommitteeId, CommitteeId)>,
pub next_leader: NodeId,
pub root_id: CommitteeId,
}
pub trait OverlayInfoExt {
fn info(&self) -> OverlayInfo;
}
impl<T: Overlay> OverlayInfoExt for T {
fn info(&self) -> OverlayInfo {
let mut committees = BTreeSet::new();
let mut edges = Vec::new();
let mut committee_sizes = HashMap::new();
let next_leader = self.next_leader();
let root = self.root_committee();
let root_id = root.id::<Blake2bU32>();
committees.insert(root_id);
committee_sizes.insert(root_id, root.len());
let mut queue = VecDeque::new();
queue.push_back(root);
while let Some(current_committee) = queue.pop_front() {
let current_id = current_committee.id::<Blake2bU32>();
if let Some(committee_node) = current_committee.iter().next() {
for child in self.child_committees(*committee_node) {
let child_id = child.id::<Blake2bU32>();
committees.insert(child_id);
committee_sizes.insert(child_id, child.len());
edges.push((current_id, child_id));
queue.push_back(child);
}
}
}
OverlayInfo {
committees,
committee_sizes,
edges,
next_leader,
root_id,
}
}
}
#[cfg(test)]
mod tests {
use carnot_engine::{
overlay::{
BranchOverlay, BranchOverlaySettings, RoundRobin, TreeOverlay, TreeOverlaySettings,
},
NodeId, Overlay,
};
use nomos_utils::fisheryates::FisherYatesShuffle;
use super::*;
const ENTROPY: [u8; 32] = [0; 32];
#[test]
fn tree_overlay_info() {
let nodes: Vec<_> = (0..7).map(|i| NodeId::new([i as u8; 32])).collect();
let overlay = TreeOverlay::new(TreeOverlaySettings {
nodes: nodes.clone(),
current_leader: nodes[0],
number_of_committees: 3,
leader: RoundRobin::new(),
committee_membership: FisherYatesShuffle::new(ENTROPY),
super_majority_threshold: None,
});
let root_committee = overlay.root_committee();
let root_node = root_committee.iter().next().unwrap();
let child_committees = overlay.child_committees(*root_node);
let child1 = child_committees[0].clone();
let child2 = child_committees[1].clone();
let info = overlay.info();
let info_children: Vec<&(CommitteeId, CommitteeId)> = info
.edges
.iter()
.filter(|(p, _)| *p == info.root_id)
.collect();
assert_eq!(info.committees.len(), 3);
assert_eq!(root_committee.id::<Blake2bU32>(), info.root_id);
let mut info_child_iter = info_children.iter();
let info_child1 = info_child_iter.next().map(|(_, c)| c).unwrap();
let info_child2 = info_child_iter.next().map(|(_, c)| c).unwrap();
assert_eq!(child1.id::<Blake2bU32>(), *info_child1);
assert_eq!(child2.id::<Blake2bU32>(), *info_child2);
assert_eq!(
child1.len(),
*info.committee_sizes.get(info_child1).unwrap()
);
assert_eq!(
child2.len(),
*info.committee_sizes.get(info_child2).unwrap()
);
}
#[test]
fn branch_overlay_info() {
let nodes: Vec<_> = (0..7).map(|i| NodeId::new([i as u8; 32])).collect();
let overlay = BranchOverlay::new(BranchOverlaySettings {
nodes: nodes.clone(),
current_leader: nodes[0],
branch_depth: 3,
leader: RoundRobin::new(),
committee_membership: FisherYatesShuffle::new(ENTROPY),
});
let root_committee = overlay.root_committee();
let root_node = root_committee.iter().next().unwrap();
let info = overlay.info();
assert_eq!(info.committees.len(), 3);
assert_eq!(root_committee.id::<Blake2bU32>(), info.root_id);
assert_eq!(overlay.child_committees(*root_node).len(), 1);
let layer1 = overlay
.child_committees(*root_node)
.first()
.unwrap()
.clone();
let layer1_node = layer1.iter().next().unwrap();
assert_eq!(overlay.child_committees(*layer1_node).len(), 1);
let info_layer1: Vec<&(CommitteeId, CommitteeId)> = info
.edges
.iter()
.filter(|(p, _)| *p == info.root_id)
.collect();
assert_eq!(info_layer1.len(), 1);
let info_layer1 = info_layer1.first().map(|(_, c)| c).unwrap();
assert_eq!(layer1.id::<Blake2bU32>(), *info_layer1);
assert_eq!(
layer1.len(),
*info.committee_sizes.get(info_layer1).unwrap()
);
let layer2 = overlay
.child_committees(*layer1_node)
.first()
.unwrap()
.clone();
let layer2_node = layer2.iter().next().unwrap();
assert_eq!(overlay.child_committees(*layer2_node).len(), 0);
let info_layer2: Vec<&(CommitteeId, CommitteeId)> = info
.edges
.iter()
.filter(|(p, _)| *p == layer1.id::<Blake2bU32>())
.collect();
assert_eq!(info_layer2.len(), 1);
let info_layer2 = info_layer2.first().map(|(_, c)| c).unwrap();
assert_eq!(layer2.id::<Blake2bU32>(), *info_layer2);
assert_eq!(
layer2.len(),
*info.committee_sizes.get(info_layer2).unwrap()
);
}
}

View File

@ -1,51 +0,0 @@
use carnot_engine::NodeId;
// std
// crates
use rand::prelude::IteratorRandom;
use rand::Rng;
// internal
use super::{Committee, Layout, Overlay};
use crate::node::NodeIdExt;
pub struct FlatOverlay;
impl FlatOverlay {
pub fn new() -> Self {
Self
}
}
impl Default for FlatOverlay {
fn default() -> Self {
Self::new()
}
}
impl Overlay for FlatOverlay {
fn nodes(&self) -> Vec<NodeId> {
(0..10).map(NodeId::from_index).collect()
}
fn leaders<R: Rng>(
&self,
nodes: &[NodeId],
size: usize,
rng: &mut R,
) -> Box<dyn Iterator<Item = NodeId>> {
let leaders = nodes.iter().copied().choose_multiple(rng, size).into_iter();
Box::new(leaders)
}
fn layout<R: Rng>(&self, nodes: &[NodeId], _rng: &mut R) -> Layout {
let committees = std::iter::once((
0.into(),
Committee {
nodes: nodes.iter().copied().collect(),
},
))
.collect();
let parent = std::iter::once((0.into(), 0.into())).collect();
let children = std::iter::once((0.into(), vec![0.into()])).collect();
let layers = std::iter::once((0.into(), vec![0.into()])).collect();
Layout::new(committees, parent, children, layers)
}
}

View File

@ -1,178 +0,0 @@
pub mod flat;
pub mod tree;
// std
use std::collections::{BTreeSet, HashMap};
// crates
use rand::Rng;
use serde::{Deserialize, Serialize};
// internal
use crate::node::{CommitteeId, NodeId};
use self::tree::TreeSettings;
#[derive(Debug, Clone, serde::Serialize, serde::Deserialize)]
pub struct Committee {
pub nodes: BTreeSet<NodeId>,
}
impl Committee {
#[must_use]
pub fn is_empty(&self) -> bool {
self.nodes.len() == 0
}
}
pub type Leaders = BTreeSet<NodeId>;
#[derive(Debug, Clone, serde::Serialize, serde::Deserialize)]
pub struct Layout {
pub committees: HashMap<CommitteeId, Committee>,
pub from_committee: HashMap<NodeId, CommitteeId>,
pub parent: HashMap<CommitteeId, CommitteeId>,
pub children: HashMap<CommitteeId, Vec<CommitteeId>>,
pub layers: HashMap<CommitteeId, Vec<CommitteeId>>,
}
impl Layout {
pub fn new(
committees: HashMap<CommitteeId, Committee>,
parent: HashMap<CommitteeId, CommitteeId>,
children: HashMap<CommitteeId, Vec<CommitteeId>>,
layers: HashMap<CommitteeId, Vec<CommitteeId>>,
) -> Self {
let from_committee = committees
.iter()
.flat_map(|(&committee_id, committee)| {
committee
.nodes
.iter()
.map(move |&node_id| (node_id, committee_id))
})
.collect();
Self {
committees,
from_committee,
parent,
children,
layers,
}
}
pub fn committee(&self, node_id: NodeId) -> Option<CommitteeId> {
self.from_committee.get(&node_id).copied()
}
pub fn committee_nodes(&self, committee_id: CommitteeId) -> &Committee {
&self.committees[&committee_id]
}
pub fn parent(&self, committee_id: CommitteeId) -> Option<CommitteeId> {
self.parent.get(&committee_id).copied()
}
pub fn parent_nodes(&self, committee_id: CommitteeId) -> Option<Committee> {
self.parent(committee_id)
.map(|c| self.committees[&c].clone())
}
pub fn children(&self, committee_id: CommitteeId) -> Option<&Vec<CommitteeId>> {
self.children.get(&committee_id)
}
pub fn children_nodes(&self, committee_id: CommitteeId) -> Vec<&Committee> {
self.children(committee_id)
.iter()
.flat_map(|&committees| committees.iter().map(|c| &self.committees[c]))
.collect()
}
pub fn node_ids(&self) -> impl Iterator<Item = NodeId> + '_ {
self.from_committee.keys().copied()
}
}
pub enum SimulationOverlay {
Flat(flat::FlatOverlay),
Tree(tree::TreeOverlay),
}
#[derive(Clone, Debug, Serialize, Deserialize)]
pub enum OverlaySettings {
Flat,
Tree(TreeSettings),
}
impl Default for OverlaySettings {
fn default() -> Self {
Self::Tree(Default::default())
}
}
impl From<TreeSettings> for OverlaySettings {
fn from(settings: TreeSettings) -> OverlaySettings {
OverlaySettings::Tree(settings)
}
}
impl TryInto<TreeSettings> for OverlaySettings {
type Error = String;
fn try_into(self) -> Result<TreeSettings, Self::Error> {
if let Self::Tree(settings) = self {
Ok(settings)
} else {
Err("unable to convert to tree settings".into())
}
}
}
impl Overlay for SimulationOverlay {
fn nodes(&self) -> Vec<NodeId> {
match self {
SimulationOverlay::Flat(overlay) => overlay.nodes(),
SimulationOverlay::Tree(overlay) => overlay.nodes(),
}
}
fn leaders<R: Rng>(
&self,
nodes: &[NodeId],
size: usize,
rng: &mut R,
) -> Box<dyn Iterator<Item = NodeId>> {
match self {
SimulationOverlay::Flat(overlay) => overlay.leaders(nodes, size, rng),
SimulationOverlay::Tree(overlay) => overlay.leaders(nodes, size, rng),
}
}
fn layout<R: Rng>(&self, nodes: &[NodeId], rng: &mut R) -> Layout {
match self {
SimulationOverlay::Flat(overlay) => overlay.layout(nodes, rng),
SimulationOverlay::Tree(overlay) => overlay.layout(nodes, rng),
}
}
}
pub trait Overlay {
fn nodes(&self) -> Vec<NodeId>;
fn leaders<R: Rng>(
&self,
nodes: &[NodeId],
size: usize,
rng: &mut R,
) -> Box<dyn Iterator<Item = NodeId>>;
fn layout<R: Rng>(&self, nodes: &[NodeId], rng: &mut R) -> Layout;
}
// Takes a reference to the simulation_settings and returns a SimulationOverlay instance based
// on the overlay settings specified in simulation_settings.
pub fn create_overlay(overlay_settings: &OverlaySettings) -> SimulationOverlay {
match &overlay_settings {
OverlaySettings::Flat => SimulationOverlay::Flat(flat::FlatOverlay::new()),
OverlaySettings::Tree(settings) => {
SimulationOverlay::Tree(tree::TreeOverlay::new(settings.clone()))
}
}
}

View File

@ -1,270 +0,0 @@
// std
use carnot_engine::NodeId;
use std::collections::HashMap;
// crates
use rand::seq::IteratorRandom;
use serde::{Deserialize, Serialize};
// internal
use super::{Committee, Layout, Overlay};
use crate::node::{CommitteeId, NodeIdExt};
#[derive(Clone, Debug, Default, Serialize, Deserialize)]
pub enum TreeType {
#[default]
FullBinaryTree,
}
#[derive(Clone, Debug, Serialize, Deserialize)]
pub struct TreeSettings {
pub tree_type: TreeType,
pub committee_size: usize,
pub depth: usize,
}
impl Default for TreeSettings {
fn default() -> Self {
Self {
tree_type: TreeType::default(),
committee_size: 1,
depth: 3, // Root, internal and leaf layers.
}
}
}
pub struct TreeOverlay {
settings: TreeSettings,
}
struct TreeProperties {
committee_count: usize,
node_count: usize,
}
impl TreeOverlay {
pub fn new(settings: TreeSettings) -> Self {
Self { settings }
}
fn build_full_binary_tree<R: rand::Rng>(
node_id: &[NodeId],
rng: &mut R,
settings: &TreeSettings,
) -> Layout {
let properties = get_tree_properties(settings);
// For full binary tree to be formed from existing nodes
// a certain unique node count needs to be provided.
assert!(properties.node_count <= node_id.len());
let mut committees = HashMap::new();
let mut parents = HashMap::new();
let mut children = HashMap::new();
let mut layers = HashMap::new();
for (committee_id, nodes) in node_id
.iter()
.choose_multiple(rng, properties.node_count)
.chunks(settings.committee_size)
.enumerate()
{
// TODO: Why do we have has_children here?
let mut _has_children = false;
let left_child_id = 2 * committee_id + 1;
let right_child_id = left_child_id + 1;
// Check for leaf nodes.
if right_child_id <= properties.committee_count {
children.insert(
committee_id.into(),
vec![left_child_id.into(), right_child_id.into()],
);
_has_children = true;
}
// Root node has no parent.
if committee_id > 0 {
let parent_id = get_parent_id(committee_id);
parents.insert(committee_id.into(), parent_id.into());
}
let committee = Committee {
nodes: nodes.iter().copied().copied().collect(),
};
committees.insert(committee_id.into(), committee);
layers
.entry(get_layer(committee_id))
.or_insert_with(Vec::new)
.push(committee_id.into());
}
Layout::new(committees, parents, children, layers)
}
}
impl Overlay for TreeOverlay {
fn nodes(&self) -> Vec<NodeId> {
let properties = get_tree_properties(&self.settings);
(0..properties.node_count).map(NodeId::from_index).collect()
}
fn leaders<R: rand::Rng>(
&self,
nodes: &[NodeId],
size: usize,
rng: &mut R,
) -> Box<dyn Iterator<Item = NodeId>> {
let leaders = nodes.iter().copied().choose_multiple(rng, size).into_iter();
Box::new(leaders)
}
fn layout<R: rand::Rng>(&self, nodes: &[NodeId], rng: &mut R) -> Layout {
match self.settings.tree_type {
TreeType::FullBinaryTree => Self::build_full_binary_tree(nodes, rng, &self.settings),
}
}
}
fn get_tree_properties(settings: &TreeSettings) -> TreeProperties {
let committee_count = committee_count(settings.depth);
let node_count = committee_count * settings.committee_size;
TreeProperties {
committee_count,
node_count,
}
}
/// Returns the number of nodes in the whole tree.
/// `depth` parameter assumes that roots is included.
fn committee_count(depth: usize) -> usize {
(1 << depth) - 1
}
fn get_parent_id(id: usize) -> usize {
(id - 1 + id % 2) / 2
}
/// Get a layer in a tree of a given committee id.
fn get_layer(id: usize) -> CommitteeId {
CommitteeId::new((id as f64 + 1.).log2().floor() as usize)
}
#[cfg(test)]
mod tests {
use super::*;
use rand::rngs::mock::StepRng;
#[test]
fn build_full_depth_1() {
let mut rng = StepRng::new(1, 0);
let overlay = TreeOverlay::new(TreeSettings {
tree_type: TreeType::FullBinaryTree,
depth: 1,
committee_size: 1,
});
let nodes = overlay.nodes();
let layout = overlay.layout(&nodes, &mut rng);
assert_eq!(layout.committees.len(), 1);
assert!(layout.children.is_empty());
assert!(layout.parent.is_empty());
}
#[test]
fn build_full_depth_3() {
let mut rng = StepRng::new(1, 0);
let overlay = TreeOverlay::new(TreeSettings {
tree_type: TreeType::FullBinaryTree,
depth: 3,
committee_size: 1,
});
let nodes = overlay.nodes();
let layout = overlay.layout(&nodes, &mut rng);
assert_eq!(
layout.children[&CommitteeId::new(0)],
vec![1.into(), 2.into()]
);
assert_eq!(layout.parent[&CommitteeId::new(1)], 0.into());
assert_eq!(layout.parent[&CommitteeId::new(2)], 0.into());
assert_eq!(
layout.children[&CommitteeId::new(1)],
vec![3.into(), 4.into()]
);
assert_eq!(
layout.children[&CommitteeId::new(2)],
vec![5.into(), 6.into()]
);
assert_eq!(layout.parent[&CommitteeId::new(3)], 1.into());
assert_eq!(layout.children.get(&CommitteeId::new(3)), None);
assert_eq!(layout.parent[&CommitteeId::new(4)], 1.into());
assert_eq!(layout.children.get(&CommitteeId::new(4)), None);
assert_eq!(layout.parent[&CommitteeId::new(5)], 2.into());
assert_eq!(layout.children.get(&CommitteeId::new(5)), None);
assert_eq!(layout.parent[&CommitteeId::new(6)], 2.into());
assert_eq!(layout.children.get(&CommitteeId::new(6)), None);
}
#[test]
fn build_full_committee_size() {
let mut rng = StepRng::new(1, 0);
let overlay = TreeOverlay::new(TreeSettings {
tree_type: TreeType::FullBinaryTree,
depth: 10,
committee_size: 10,
});
let nodes = overlay.nodes();
let layout = overlay.layout(&nodes, &mut rng);
// 2^h - 1
assert_eq!(layout.committees.len(), 1023);
let root_nodes = &layout.committees[&CommitteeId::new(0)].nodes;
assert_eq!(root_nodes.len(), 10);
assert_eq!(root_nodes.first(), Some(&NodeId::from_index(0)));
assert_eq!(root_nodes.last(), Some(&NodeId::from_index(9)));
let last_nodes = &layout.committees[&CommitteeId::new(1022)].nodes;
assert_eq!(last_nodes.len(), 10);
assert_eq!(last_nodes.first(), Some(&NodeId::from_index(10220)));
assert_eq!(last_nodes.last(), Some(&NodeId::from_index(10229)));
}
#[test]
fn check_committee_role() {
let mut rng = StepRng::new(1, 0);
let overlay = TreeOverlay::new(TreeSettings {
tree_type: TreeType::FullBinaryTree,
depth: 3,
committee_size: 1,
});
let nodes = overlay.nodes();
let _layout = overlay.layout(&nodes, &mut rng);
}
#[test]
fn check_layers() {
let mut rng = StepRng::new(1, 0);
let overlay = TreeOverlay::new(TreeSettings {
tree_type: TreeType::FullBinaryTree,
depth: 4,
committee_size: 1,
});
let nodes = overlay.nodes();
let layout = overlay.layout(&nodes, &mut rng);
assert_eq!(layout.layers[&CommitteeId::new(0)], vec![0.into()]);
assert_eq!(
layout.layers[&CommitteeId::new(1)],
vec![1.into(), 2.into()]
);
assert_eq!(
layout.layers[&CommitteeId::new(2)],
vec![3.into(), 4.into(), 5.into(), 6.into()]
);
}
}

View File

@ -1,75 +0,0 @@
use crate::node::NodeId;
use crate::output_processors::Record;
use crate::runner::SimulationRunner;
use crate::warding::SimulationState;
use crossbeam::channel::bounded;
use crossbeam::select;
use rand::prelude::SliceRandom;
use rayon::prelude::*;
use std::collections::HashSet;
use std::sync::Arc;
use std::time::Duration;
use super::SimulationRunnerHandle;
/// Simulate with sending the network state to any subscriber
pub fn simulate<M, R, S, T>(
runner: SimulationRunner<M, R, S, T>,
chunk_size: usize,
step_time: Duration,
) -> anyhow::Result<SimulationRunnerHandle<R>>
where
M: std::fmt::Debug + Clone + Send + Sync + 'static,
R: Record
+ for<'a> TryFrom<&'a SimulationState<S, T>, Error = anyhow::Error>
+ Send
+ Sync
+ 'static,
S: 'static,
T: 'static,
{
let simulation_state = SimulationState {
nodes: Arc::clone(&runner.nodes),
};
let mut node_ids: Vec<NodeId> = runner.nodes.read().iter().map(|n| n.id()).collect();
let mut inner_runner = runner.inner;
let nodes = runner.nodes;
let (stop_tx, stop_rx) = bounded(1);
let p = runner.producer.clone();
let p1 = runner.producer;
let handle = std::thread::spawn(move || {
loop {
select! {
recv(stop_rx) -> _ => {
return Ok(());
}
default => {
node_ids.shuffle(&mut inner_runner.rng);
for ids_chunk in node_ids.chunks(chunk_size) {
let ids: HashSet<NodeId> = ids_chunk.iter().copied().collect();
nodes
.write()
.par_iter_mut()
.filter(|n| ids.contains(&n.id()))
.for_each(|node| node.step(step_time));
p.send(R::try_from(
&simulation_state,
)?)?;
}
// check if any condition makes the simulation stop
if inner_runner.check_wards(&simulation_state) {
return Ok(());
}
}
}
}
});
Ok(SimulationRunnerHandle {
producer: p1,
stop_tx,
handle,
})
}

View File

@ -1,91 +0,0 @@
use crate::node::{Node, NodeId, NodeIdExt};
use crate::output_processors::Record;
use crate::runner::SimulationRunner;
use crate::warding::SimulationState;
use crossbeam::channel::bounded;
use crossbeam::select;
use rand::prelude::IteratorRandom;
use std::collections::BTreeSet;
use std::sync::Arc;
use std::time::Duration;
use super::SimulationRunnerHandle;
/// Simulate with sending the network state to any subscriber.
///
/// [Glauber dynamics simulation](https://en.wikipedia.org/wiki/Glauber_dynamics)
pub fn simulate<M, R, S, T>(
runner: SimulationRunner<M, R, S, T>,
update_rate: usize,
maximum_iterations: usize,
step_time: Duration,
) -> anyhow::Result<SimulationRunnerHandle<R>>
where
M: std::fmt::Debug + Send + Sync + Clone + 'static,
R: Record
+ for<'a> TryFrom<&'a SimulationState<S, T>, Error = anyhow::Error>
+ Send
+ Sync
+ 'static,
S: 'static,
T: 'static,
{
let simulation_state = SimulationState {
nodes: Arc::clone(&runner.nodes),
};
let mut inner_runner = runner.inner;
let nodes = runner.nodes;
let nodes_remaining: BTreeSet<NodeId> =
(0..nodes.read().len()).map(NodeId::from_index).collect();
let iterations: Vec<_> = (0..maximum_iterations).collect();
let (stop_tx, stop_rx) = bounded(1);
let p = runner.producer.clone();
let p1 = runner.producer;
let handle = std::thread::spawn(move || {
'main: for chunk in iterations.chunks(update_rate) {
select! {
recv(stop_rx) -> _ => break 'main,
default => {
for _ in chunk {
if nodes_remaining.is_empty() {
break 'main;
}
let node_id = *nodes_remaining.iter().choose(&mut inner_runner.rng).expect(
"Some id to be selected as it should be impossible for the set to be empty here",
);
{
let mut shared_nodes = nodes.write();
let node: &mut dyn Node<Settings = S, State = T> = &mut **shared_nodes
.get_mut(node_id.index())
.expect("Node should be present");
node.step(step_time);
}
// check if any condition makes the simulation stop
if inner_runner.check_wards(&simulation_state) {
// we break the outer main loop, so we need to dump it before the breaking
p.send(R::try_from(
&simulation_state,
)?)?;
break 'main;
}
}
// update_rate iterations reached, so dump state
p.send(R::try_from(
&simulation_state,
)?)?;
}
}
}
Ok(())
});
Ok(SimulationRunnerHandle {
producer: p1,
stop_tx,
handle,
})
}

View File

@ -1,179 +0,0 @@
//! # Layered simulation runner
//!
//! A revision of the [`glauber`](super::glauber_runner) simulation runner.
//!
//! **`glauber`** simulations have some drawbacks:
//!
//! * Completely random, difficult to control
//! * Not close to how real nodes would perform in reality
//! * Difficult to analise recorded data, as data it is updated by chunks of iterations
//!
//! To solve this we can use a concept of layered *glauber* executions.
//! The algorithm roughly works as follows:
//!
//! ```python
//! nodes <- [nodes]
//! layers <- [[nodes_ids], [], ...]
//! while nodes_to_compute(layers):
//! layer_index <- pick_rand_layer(layers)
//! node_index <- pop_rand_node(rand_layer)
//! step(nodes[node_index])
//! if not node_decided(node):
//! push(layers[layer_index+1], node_index)
//! ```
//!
//! From within this, controlling the *number of layers*, and *weighting* them (how often are they picked),
//! we can control the flow of the simulations.
//! Also we can consider that once the bottom layer is empty a fully step have been concluded and we can record
//! the data of that step simulation.
// std
use crossbeam::channel::bounded;
use crossbeam::select;
use std::collections::BTreeSet;
use std::ops::Not;
use std::sync::Arc;
use std::time::Duration;
// crates
use fixed_slice_deque::FixedSliceDeque;
use rand::prelude::{IteratorRandom, SliceRandom};
use rand::rngs::SmallRng;
// internal
use crate::node::{Node, NodeId, NodeIdExt};
use crate::output_processors::Record;
use crate::runner::SimulationRunner;
use crate::warding::SimulationState;
use super::SimulationRunnerHandle;
/// Simulate with sending the network state to any subscriber
pub fn simulate<M, R, S, T>(
runner: SimulationRunner<M, R, S, T>,
gap: usize,
distribution: Option<Vec<f32>>,
step_time: Duration,
) -> anyhow::Result<SimulationRunnerHandle<R>>
where
M: std::fmt::Debug + Send + Sync + Clone + 'static,
R: Record
+ for<'a> TryFrom<&'a SimulationState<S, T>, Error = anyhow::Error>
+ Send
+ Sync
+ 'static,
S: 'static,
T: 'static,
{
let distribution =
distribution.unwrap_or_else(|| std::iter::repeat(1.0f32).take(gap).collect());
let layers: Vec<usize> = (0..gap).collect();
let mut deque = build_node_ids_deque::<M, R, S, T>(gap, &runner);
let simulation_state = SimulationState {
nodes: Arc::clone(&runner.nodes),
};
let mut inner_runner = runner.inner;
let nodes = runner.nodes;
let (stop_tx, stop_rx) = bounded(1);
let p = runner.producer.clone();
let p1 = runner.producer;
let handle = std::thread::spawn(move || {
loop {
select! {
recv(stop_rx) -> _ => {
break;
}
default => {
let (group_index, node_id) =
choose_random_layer_and_node_id(&mut inner_runner.rng, &distribution, &layers, &mut deque);
// remove node_id from group
deque.get_mut(group_index).unwrap().remove(&node_id);
{
let mut shared_nodes = nodes.write();
let node: &mut dyn Node<Settings = S, State = T> = &mut **shared_nodes
.get_mut(node_id.index())
.expect("Node should be present");
let prev_view = node.current_view();
node.step(step_time);
let after_view = node.current_view();
if after_view > prev_view {
// pass node to next step group
deque.get_mut(group_index + 1).unwrap().insert(node_id);
}
}
// check if any condition makes the simulation stop
if inner_runner.check_wards(&simulation_state) {
break;
}
// if initial is empty then we finished a full round, append a new set to the end so we can
// compute the most advanced nodes again
if deque.first().unwrap().is_empty() {
let _ = deque.push_back(BTreeSet::default());
p.send(R::try_from(
&simulation_state,
)?)?;
}
// if no more nodes to compute
if deque.iter().all(BTreeSet::is_empty) {
break;
}
}
}
}
// write latest state
p.send(R::try_from(&simulation_state)?)?;
Ok(())
});
Ok(SimulationRunnerHandle {
producer: p1,
stop_tx,
handle,
})
}
fn choose_random_layer_and_node_id(
rng: &mut SmallRng,
distribution: &[f32],
layers: &[usize],
deque: &mut FixedSliceDeque<BTreeSet<NodeId>>,
) -> (usize, NodeId) {
let i = *layers
.iter()
// filter out empty round groups
.filter_map(|&i| {
let g = deque.get(i).unwrap();
g.is_empty().not().then_some(i)
})
// intermediate collect necessary for choose_weighted
.collect::<Vec<_>>()
.choose_weighted(rng, |&i| distribution.get(i).unwrap())
.expect("Distribution choose to work");
let group: &mut BTreeSet<NodeId> = deque.get_mut(i).unwrap();
let node_id = group.iter().choose(rng).unwrap();
(i, *node_id)
}
fn build_node_ids_deque<M: std::fmt::Debug, R, S, T>(
gap: usize,
runner: &SimulationRunner<M, R, S, T>,
) -> FixedSliceDeque<BTreeSet<NodeId>> {
// add a +1 so we always have
let mut deque = FixedSliceDeque::new(gap + 1);
// push first layer
let node_ids: BTreeSet<NodeId> = runner.nodes.write().iter().map(|node| node.id()).collect();
deque.push_back(node_ids);
// allocate default sets
while deque.try_push_back(BTreeSet::new()).is_ok() {}
deque
}

View File

@ -1,225 +0,0 @@
mod async_runner;
mod glauber_runner;
mod layered_runner;
mod sync_runner;
// std
use std::sync::Arc;
use std::time::Duration;
use crate::output_processors::Record;
// crates
use crate::streaming::{
runtime_subscriber::RuntimeSubscriber, settings_subscriber::SettingsSubscriber, StreamProducer,
Subscriber, SubscriberHandle,
};
use crossbeam::channel::Sender;
use parking_lot::RwLock;
use rand::rngs::SmallRng;
use rand::{RngCore, SeedableRng};
use rayon::prelude::*;
use serde::Serialize;
// internal
use crate::network::Network;
use crate::node::Node;
use crate::settings::{RunnerSettings, SimulationSettings};
use crate::warding::{SimulationState, SimulationWard, Ward};
pub type BoxedNode<S, T> = Box<dyn Node<Settings = S, State = T> + Send + Sync>;
pub struct SimulationRunnerHandle<R> {
producer: StreamProducer<R>,
stop_tx: Sender<()>,
handle: std::thread::JoinHandle<anyhow::Result<()>>,
}
impl<R: Record> SimulationRunnerHandle<R> {
pub fn stop_after(self, duration: Duration) -> anyhow::Result<()> {
std::thread::sleep(duration);
self.stop()
}
pub fn stop(&self) -> anyhow::Result<()> {
if !self.handle.is_finished() {
self.stop_tx.send(())?;
self.shutdown()?;
}
Ok(())
}
pub fn subscribe<S: Subscriber<Record = R>>(
&self,
settings: S::Settings,
) -> anyhow::Result<SubscriberHandle<S>> {
self.producer.subscribe(settings)
}
pub fn is_finished(&self) -> bool {
self.handle.is_finished()
}
pub fn shutdown(&self) -> anyhow::Result<()> {
self.producer.stop()
}
pub fn join(self) -> anyhow::Result<()> {
self.handle.join().expect("Join simulation thread")
}
}
pub(crate) struct SimulationRunnerInner<M: std::fmt::Debug> {
network: Network<M>,
wards: Vec<Ward>,
rng: SmallRng,
}
impl<M> SimulationRunnerInner<M>
where
M: std::fmt::Debug + Send + Sync + Clone,
{
fn check_wards<S, T>(&mut self, state: &SimulationState<S, T>) -> bool {
self.wards
.par_iter_mut()
.map(|ward| ward.analyze(state))
.any(|x| x)
}
fn step<S, T>(&mut self, nodes: &mut [BoxedNode<S, T>], elapsed: Duration) {
self.network.dispatch_after(elapsed);
nodes.par_iter_mut().for_each(|node| {
node.step(elapsed);
});
self.network.collect_messages();
}
}
/// Encapsulation solution for the simulations runner
/// Holds the network state, the simulating nodes and the simulation settings.
pub struct SimulationRunner<M: std::fmt::Debug, R, S, T> {
inner: SimulationRunnerInner<M>,
nodes: Arc<RwLock<Vec<BoxedNode<S, T>>>>,
runner_settings: RunnerSettings,
producer: StreamProducer<R>,
step_time: Duration,
}
impl<M, R, S, T> SimulationRunner<M, R, S, T>
where
M: std::fmt::Debug + Clone + Send + Sync + 'static,
R: Record
+ for<'a> TryFrom<&'a SimulationState<S, T>, Error = anyhow::Error>
+ Send
+ Sync
+ 'static,
S: 'static,
T: Serialize + Clone + 'static,
{
pub fn new(
network: Network<M>,
nodes: Vec<BoxedNode<S, T>>,
producer: StreamProducer<R>,
mut settings: SimulationSettings,
) -> anyhow::Result<Self> {
let seed = settings
.seed
.unwrap_or_else(|| rand::thread_rng().next_u64());
settings
.seed
.get_or_insert_with(|| rand::thread_rng().next_u64());
// Store the settings to the producer so that we can collect them later
producer.send(R::from(settings.clone()))?;
let rng = SmallRng::seed_from_u64(seed);
let nodes = Arc::new(RwLock::new(nodes));
let SimulationSettings {
wards,
overlay_settings: _,
node_settings: _,
runner_settings,
stream_settings: _,
node_count: _,
seed: _,
views_count: _,
leaders_count: _,
network_settings: _,
step_time,
record_settings: _,
} = settings;
Ok(Self {
runner_settings,
inner: SimulationRunnerInner {
network,
rng,
wards,
},
nodes,
producer,
step_time,
})
}
pub fn simulate(self) -> anyhow::Result<SimulationRunnerHandle<R>> {
// init the start time
let _ = *crate::START_TIME;
let step_time = self.step_time;
match self.runner_settings.clone() {
RunnerSettings::Sync => sync_runner::simulate(self, step_time),
RunnerSettings::Async { chunks } => async_runner::simulate(self, chunks, step_time),
RunnerSettings::Glauber {
maximum_iterations,
update_rate,
} => glauber_runner::simulate(self, update_rate, maximum_iterations, step_time),
RunnerSettings::Layered {
rounds_gap,
distribution,
} => layered_runner::simulate(self, rounds_gap, distribution, step_time),
}
}
}
impl<M, R, S, T> SimulationRunner<M, R, S, T>
where
M: std::fmt::Debug + Clone + Send + Sync + 'static,
R: Record
+ serde::Serialize
+ for<'a> TryFrom<&'a SimulationState<S, T>, Error = anyhow::Error>
+ Send
+ Sync
+ 'static,
S: 'static,
T: Serialize + Clone + 'static,
{
pub fn simulate_and_subscribe<B>(
self,
settings: B::Settings,
) -> anyhow::Result<SimulationRunnerHandle<R>>
where
B: Subscriber<Record = R> + Send + Sync + 'static,
{
let handle = self.simulate()?;
let mut data_subscriber_handle = handle.subscribe::<B>(settings)?;
let mut runtime_subscriber_handle =
handle.subscribe::<RuntimeSubscriber<R>>(Default::default())?;
let mut settings_subscriber_handle =
handle.subscribe::<SettingsSubscriber<R>>(Default::default())?;
std::thread::scope(|s| {
s.spawn(move || {
data_subscriber_handle.run();
});
s.spawn(move || {
runtime_subscriber_handle.run();
});
s.spawn(move || {
settings_subscriber_handle.run();
});
});
Ok(handle)
}
}

View File

@ -1,244 +0,0 @@
use super::{SimulationRunner, SimulationRunnerHandle};
use crate::output_processors::Record;
use crate::warding::SimulationState;
use crossbeam::channel::{bounded, select};
use std::sync::Arc;
use std::time::Duration;
/// Simulate with sending the network state to any subscriber
pub fn simulate<M, R, S, T>(
runner: SimulationRunner<M, R, S, T>,
step_time: Duration,
) -> anyhow::Result<SimulationRunnerHandle<R>>
where
M: std::fmt::Debug + Send + Sync + Clone + 'static,
R: Record
+ for<'a> TryFrom<&'a SimulationState<S, T>, Error = anyhow::Error>
+ Send
+ Sync
+ 'static,
S: 'static,
T: 'static,
{
let state = SimulationState {
nodes: Arc::clone(&runner.nodes),
};
let mut inner_runner = runner.inner;
let nodes = runner.nodes;
let (stop_tx, stop_rx) = bounded(1);
let p = runner.producer.clone();
let p1 = runner.producer;
let handle = std::thread::spawn(move || {
p.send(R::try_from(&state)?)?;
loop {
select! {
recv(stop_rx) -> _ => {
return Ok(());
}
default => {
// we must use a code block to make sure once the step call is finished then the write lock will be released, because in Record::try_from(&state),
// we need to call the read lock, if we do not release the write lock,
// then dead lock will occur
{
let mut nodes = nodes.write();
inner_runner.step(&mut nodes, step_time);
}
p.send(R::try_from(&state)?)?;
// check if any condition makes the simulation stop
if inner_runner.check_wards(&state) {
return Ok(());
}
}
}
}
});
Ok(SimulationRunnerHandle {
producer: p1,
stop_tx,
handle,
})
}
#[cfg(test)]
mod tests {
use crate::{
network::{
behaviour::NetworkBehaviour,
regions::{Region, RegionsData},
InMemoryNetworkInterface, Network, NetworkBehaviourKey,
},
node::{
dummy::{DummyMessage, DummyNode, DummySettings, DummyState},
Node, NodeId, NodeIdExt, OverlayState, SharedState, ViewOverlay,
},
output_processors::OutData,
overlay::tests::{
tree::{TreeOverlay, TreeSettings},
Overlay, SimulationOverlay,
},
runner::SimulationRunner,
settings::SimulationSettings,
streaming::StreamProducer,
};
use carnot_engine::View;
use crossbeam::channel;
use parking_lot::RwLock;
use rand::rngs::mock::StepRng;
use std::{
collections::{BTreeMap, HashMap},
sync::Arc,
time::Duration,
};
fn init_network(node_ids: &[NodeId]) -> Network<DummyMessage> {
let regions = HashMap::from([(Region::Europe, node_ids.to_vec())]);
let behaviour = HashMap::from([(
NetworkBehaviourKey::new(Region::Europe, Region::Europe),
NetworkBehaviour::new(Duration::from_millis(100), 0.0),
)]);
let regions_data = RegionsData::new(regions, behaviour);
Network::new(regions_data, 0)
}
fn init_dummy_nodes(
node_ids: &[NodeId],
network: &mut Network<DummyMessage>,
overlay_state: SharedState<OverlayState>,
) -> Vec<DummyNode> {
node_ids
.iter()
.map(|node_id| {
let (node_message_sender, node_message_receiver) = channel::unbounded();
let (node_message_broadcast_sender, node_message_broadcast_receiver) =
channel::unbounded();
let network_message_receiver = network.connect(
*node_id,
Some(1),
node_message_receiver,
node_message_broadcast_receiver,
);
let network_interface = InMemoryNetworkInterface::new(
*node_id,
node_message_broadcast_sender,
node_message_sender,
network_message_receiver,
);
DummyNode::new(
*node_id,
View::new(0),
overlay_state.clone(),
network_interface,
)
})
.collect()
}
#[test]
fn runner_one_step() {
let settings = SimulationSettings {
node_count: 10,
..Default::default()
};
let mut rng = StepRng::new(1, 0);
let node_ids: Vec<NodeId> = (0..settings.node_count).map(NodeId::from_index).collect();
let overlay = TreeOverlay::new(TreeSettings::default());
let mut network = init_network(&node_ids);
let view = ViewOverlay {
leaders: overlay.leaders(&node_ids, 1, &mut rng).collect(),
layout: overlay.layout(&node_ids, &mut rng),
};
let overlay_state = Arc::new(RwLock::new(OverlayState {
all_nodes: node_ids.clone(),
overlay: SimulationOverlay::Tree(overlay),
overlays: BTreeMap::from([(View::new(0), view.clone()), (View::new(1), view)]),
}));
let nodes = init_dummy_nodes(&node_ids, &mut network, overlay_state)
.into_iter()
.map(|n| {
Box::new(n)
as Box<
dyn Node<State = DummyState, Settings = DummySettings>
+ std::marker::Send
+ Sync,
>
})
.collect();
let producer = StreamProducer::default();
let mut runner: SimulationRunner<DummyMessage, OutData, DummySettings, DummyState> =
SimulationRunner::<_, OutData, DummySettings, DummyState>::new(
network, nodes, producer, settings,
)
.unwrap();
let mut nodes = runner.nodes.write();
runner.inner.step(&mut nodes, Duration::from_millis(100));
drop(nodes);
let nodes = runner.nodes.read();
for node in nodes.iter() {
assert_eq!(node.current_view(), View::new(0));
}
}
#[test]
fn runner_send_receive() {
let settings = SimulationSettings {
node_count: 10,
..Default::default()
};
let mut rng = StepRng::new(1, 0);
let node_ids: Vec<NodeId> = (0..settings.node_count).map(NodeId::from_index).collect();
let overlay = TreeOverlay::new(TreeSettings::default());
let mut network = init_network(&node_ids);
let view = ViewOverlay {
leaders: overlay.leaders(&node_ids, 1, &mut rng).collect(),
layout: overlay.layout(&node_ids, &mut rng),
};
let overlay_state = Arc::new(RwLock::new(OverlayState {
all_nodes: node_ids.clone(),
overlay: SimulationOverlay::Tree(overlay),
overlays: BTreeMap::from([
(View::new(0), view.clone()),
(View::new(1), view.clone()),
(View::new(42), view.clone()),
(View::new(43), view),
]),
}));
let nodes = init_dummy_nodes(&node_ids, &mut network, overlay_state.clone());
for node in nodes.iter() {
// All nodes send one message to NodeId(1).
// Nodes can send messages to themselves.
node.send_message(node_ids[1], DummyMessage::Proposal(View::new(42).into()));
}
network.collect_messages();
let nodes = init_dummy_nodes(&node_ids, &mut network, overlay_state)
.into_iter()
.map(|n| {
Box::new(n)
as Box<
dyn Node<State = DummyState, Settings = DummySettings>
+ std::marker::Send
+ Sync,
>
})
.collect();
let mut runner: SimulationRunner<DummyMessage, OutData, DummySettings, DummyState> =
SimulationRunner::new(network, nodes, Default::default(), settings).unwrap();
let mut nodes = runner.nodes.write();
runner.inner.step(&mut nodes, Duration::from_millis(100));
drop(nodes);
let nodes = runner.nodes.read();
let state = nodes[1].state();
assert_eq!(state.message_count, 10);
}
}

View File

@ -1,69 +0,0 @@
use std::collections::BTreeMap;
use crate::network::NetworkSettings;
use crate::streaming::StreamSettings;
use crate::warding::Ward;
use serde::{Deserialize, Serialize};
#[derive(Clone, Debug, Serialize, Deserialize, Default)]
pub enum RunnerSettings {
#[default]
Sync,
Async {
chunks: usize,
},
Glauber {
maximum_iterations: usize,
update_rate: usize,
},
Layered {
rounds_gap: usize,
distribution: Option<Vec<f32>>,
},
}
#[derive(Clone, Debug, Default, Serialize, Deserialize)]
#[serde(untagged)]
pub enum OverlaySettings {
#[default]
Flat,
Tree(TreeSettings),
Branch(BranchSettings),
}
#[derive(Clone, Debug, Serialize, Deserialize)]
pub struct TreeSettings {
pub number_of_committees: usize,
}
#[derive(Clone, Debug, Serialize, Deserialize)]
pub struct BranchSettings {
pub branch_depth: usize,
}
#[derive(Clone, Debug, Serialize, Deserialize, Default)]
pub struct NodeSettings {
pub network_capacity_kbps: Option<u32>,
#[serde(with = "humantime_serde")]
pub timeout: std::time::Duration,
}
#[derive(Clone, Default, Debug, Serialize, Deserialize)]
pub struct SimulationSettings {
#[serde(default)]
pub wards: Vec<Ward>,
#[serde(default)]
pub record_settings: BTreeMap<String, bool>,
pub network_settings: NetworkSettings,
pub overlay_settings: OverlaySettings,
pub node_settings: NodeSettings,
#[serde(default)]
pub runner_settings: RunnerSettings,
pub stream_settings: StreamSettings,
#[serde(with = "humantime_serde")]
pub step_time: std::time::Duration,
pub node_count: usize,
pub views_count: usize,
pub leaders_count: usize,
pub seed: Option<u64>,
}

View File

@ -1,235 +0,0 @@
use std::{any::Any, io::stdout, sync::Arc};
use super::{Receivers, StreamSettings, Subscriber};
use crate::output_processors::{RecordType, Runtime};
use crossbeam::channel::{Receiver, Sender};
use parking_lot::Mutex;
use serde::{Deserialize, Serialize};
#[derive(Debug, Clone, Default, Serialize, Deserialize)]
pub struct IOStreamSettings {
#[serde(rename = "type")]
pub writer_type: WriteType,
}
#[derive(Debug, Clone, Default, Serialize, Deserialize)]
#[serde(rename_all = "snake_case")]
pub enum WriteType {
#[default]
Stdout,
}
pub trait ToWriter<W: std::io::Write + Send + Sync + 'static> {
fn to_writer(&self) -> anyhow::Result<W>;
}
impl<W: std::io::Write + Send + Sync + 'static> ToWriter<W> for WriteType {
fn to_writer(&self) -> anyhow::Result<W> {
match self {
WriteType::Stdout => {
let boxed_any = Box::new(stdout()) as Box<dyn Any + Send + Sync>;
Ok(boxed_any.downcast::<W>().map(|boxed| *boxed).map_err(|_| {
std::io::Error::new(std::io::ErrorKind::Other, "Writer type mismatch")
})?)
}
}
}
}
impl TryFrom<StreamSettings> for IOStreamSettings {
type Error = String;
fn try_from(settings: StreamSettings) -> Result<Self, Self::Error> {
match settings {
StreamSettings::IO(settings) => Ok(settings),
_ => Err("io settings can't be created".into()),
}
}
}
#[derive(Debug)]
pub struct IOSubscriber<R, W = std::io::Stdout> {
recvs: Arc<Receivers<R>>,
writer: Arc<Mutex<W>>,
}
impl<W, R> Subscriber for IOSubscriber<R, W>
where
W: std::io::Write + Send + Sync + 'static,
R: crate::output_processors::Record + Serialize,
{
type Record = R;
type Settings = IOStreamSettings;
fn new(
record_recv: Receiver<Arc<Self::Record>>,
stop_recv: Receiver<Sender<()>>,
settings: Self::Settings,
) -> anyhow::Result<Self>
where
Self: Sized,
{
Ok(Self {
recvs: Arc::new(Receivers {
stop_rx: stop_recv,
recv: record_recv,
}),
writer: Arc::new(Mutex::new(settings.writer_type.to_writer()?)),
})
}
fn next(&self) -> Option<anyhow::Result<Arc<Self::Record>>> {
Some(self.recvs.recv.recv().map_err(From::from))
}
fn run(self) -> anyhow::Result<()> {
loop {
crossbeam::select! {
recv(self.recvs.stop_rx) -> finish_tx => {
// Flush remaining messages after stop signal.
while let Ok(msg) = self.recvs.recv.try_recv() {
self.sink(msg)?;
}
// collect the run time meta
self.sink(Arc::new(R::from(Runtime::load()?)))?;
finish_tx?.send(())?
}
recv(self.recvs.recv) -> msg => {
self.sink(msg?)?;
}
}
}
}
fn sink(&self, state: Arc<Self::Record>) -> anyhow::Result<()> {
serde_json::to_writer(&mut *self.writer.lock(), &state)?;
Ok(())
}
fn subscribe_data_type() -> RecordType {
RecordType::Data
}
}
#[cfg(test)]
mod tests {
use std::{collections::HashMap, time::Duration};
use carnot_engine::View;
use crate::{
network::{
behaviour::NetworkBehaviour,
regions::{Region, RegionsData},
Network, NetworkBehaviourKey,
},
node::{
dummy_streaming::{DummyStreamingNode, DummyStreamingState},
Node, NodeId, NodeIdExt,
},
output_processors::OutData,
runner::SimulationRunner,
warding::SimulationState,
};
use super::*;
#[derive(Debug, Clone, Serialize)]
struct IORecord {
states: HashMap<NodeId, View>,
}
impl<S, T: Serialize> TryFrom<&SimulationState<S, T>> for IORecord {
type Error = anyhow::Error;
fn try_from(value: &SimulationState<S, T>) -> Result<Self, Self::Error> {
let nodes = value.nodes.read();
Ok(Self {
states: nodes
.iter()
.map(|node| (node.id(), node.current_view()))
.collect(),
})
}
}
#[test]
fn test_streaming() {
let simulation_settings = crate::settings::SimulationSettings {
seed: Some(1),
..Default::default()
};
let nodes = (0..6)
.map(|idx| {
Box::new(DummyStreamingNode::new(NodeId::from_index(idx), ()))
as Box<
dyn Node<State = DummyStreamingState, Settings = ()>
+ std::marker::Send
+ Sync,
>
})
.collect::<Vec<_>>();
let network = Network::new(
RegionsData {
regions: (0..6)
.map(|idx| {
let region = match idx % 6 {
0 => Region::Europe,
1 => Region::NorthAmerica,
2 => Region::SouthAmerica,
3 => Region::Asia,
4 => Region::Africa,
5 => Region::Australia,
_ => unreachable!(),
};
(region, vec![NodeId::from_index(idx)])
})
.collect(),
node_region: (0..6)
.map(|idx| {
let region = match idx % 6 {
0 => Region::Europe,
1 => Region::NorthAmerica,
2 => Region::SouthAmerica,
3 => Region::Asia,
4 => Region::Africa,
5 => Region::Australia,
_ => unreachable!(),
};
(NodeId::from_index(idx), region)
})
.collect(),
region_network_behaviour: (0..6)
.map(|idx| {
let region = match idx % 6 {
0 => Region::Europe,
1 => Region::NorthAmerica,
2 => Region::SouthAmerica,
3 => Region::Asia,
4 => Region::Africa,
5 => Region::Australia,
_ => unreachable!(),
};
(
NetworkBehaviourKey::new(region, region),
NetworkBehaviour {
delay: Duration::from_millis(100),
drop: 0.0,
},
)
})
.collect(),
},
0,
);
let simulation_runner: SimulationRunner<(), OutData, (), DummyStreamingState> =
SimulationRunner::new(network, nodes, Default::default(), simulation_settings).unwrap();
simulation_runner
.simulate()
.unwrap()
.stop_after(Duration::from_millis(100))
.unwrap();
}
}

View File

@ -1,367 +0,0 @@
use std::{
str::FromStr,
sync::{Arc, Mutex},
time::Duration,
};
use crossbeam::channel::{bounded, unbounded, Receiver, Sender};
use serde::{Deserialize, Serialize};
use crate::output_processors::{Record, RecordType, Runtime};
pub mod io;
pub mod naive;
#[cfg(feature = "polars")]
pub mod polars;
pub mod runtime_subscriber;
pub mod settings_subscriber;
#[derive(Debug, Default, Clone, Copy, Serialize, PartialEq, Eq)]
pub enum SubscriberFormat {
Json,
#[default]
Csv,
Parquet,
}
impl SubscriberFormat {
pub const fn csv() -> Self {
Self::Csv
}
pub const fn json() -> Self {
Self::Json
}
pub const fn parquet() -> Self {
Self::Parquet
}
pub fn is_csv(&self) -> bool {
matches!(self, Self::Csv)
}
}
impl FromStr for SubscriberFormat {
type Err = String;
fn from_str(s: &str) -> Result<Self, Self::Err> {
match s.trim().to_ascii_lowercase().as_str() {
"json" => Ok(Self::Json),
"csv" => Ok(Self::Csv),
"parquet" => Ok(Self::Parquet),
tag => Err(format!(
"Invalid {tag} format, only [json, csv, parquet] are supported",
)),
}
}
}
impl<'de> Deserialize<'de> for SubscriberFormat {
fn deserialize<D>(deserializer: D) -> Result<Self, D::Error>
where
D: serde::Deserializer<'de>,
{
let s = String::deserialize(deserializer)?;
SubscriberFormat::from_str(&s).map_err(serde::de::Error::custom)
}
}
pub enum SubscriberType {
Meta,
Settings,
Data,
}
#[derive(Debug)]
struct Receivers<R> {
stop_rx: Receiver<Sender<()>>,
recv: Receiver<Arc<R>>,
}
#[derive(Debug, Default, Clone, Copy, PartialEq, Eq, Serialize)]
pub enum StreamType {
#[default]
IO,
Naive,
#[cfg(feature = "polars")]
Polars,
}
impl FromStr for StreamType {
type Err = String;
fn from_str(s: &str) -> Result<Self, Self::Err> {
match s.trim().to_ascii_lowercase().as_str() {
"io" => Ok(Self::IO),
"naive" => Ok(Self::Naive),
#[cfg(feature = "polars")]
"polars" => Ok(Self::Polars),
tag => Err(format!(
"Invalid {tag} streaming type, only [naive, polars] are supported",
)),
}
}
}
impl<'de> serde::Deserialize<'de> for StreamType {
fn deserialize<D>(deserializer: D) -> Result<Self, D::Error>
where
D: serde::Deserializer<'de>,
{
let s = String::deserialize(deserializer)?;
StreamType::from_str(&s).map_err(serde::de::Error::custom)
}
}
#[derive(Debug, Serialize, Deserialize, Clone)]
#[serde(rename_all = "camelCase", untagged)]
pub enum StreamSettings {
Naive(naive::NaiveSettings),
IO(io::IOStreamSettings),
#[cfg(feature = "polars")]
Polars(polars::PolarsSettings),
}
impl Default for StreamSettings {
fn default() -> Self {
Self::IO(Default::default())
}
}
impl StreamSettings {
pub fn unwrap_naive(self) -> naive::NaiveSettings {
match self {
StreamSettings::Naive(settings) => settings,
_ => panic!("unwrap naive failed"),
}
}
pub fn unwrap_io(self) -> io::IOStreamSettings {
match self {
StreamSettings::IO(settings) => settings,
_ => panic!("unwrap io failed"),
}
}
#[cfg(feature = "polars")]
pub fn unwrap_polars(self) -> polars::PolarsSettings {
match self {
StreamSettings::Polars(settings) => settings,
_ => panic!("unwrap polars failed"),
}
}
}
pub struct SubscriberHandle<S> {
handle: Option<std::thread::JoinHandle<anyhow::Result<()>>>,
stop_tx: Sender<Sender<()>>,
subscriber: Option<S>,
}
impl<S> SubscriberHandle<S>
where
S: Subscriber + Send + 'static,
{
pub fn run(&mut self) {
if self.handle.is_some() {
return;
}
// unwrap safe here, because if handld is none, then we must have not booted the subscriber.
let subscriber = self.subscriber.take().unwrap();
let handle = std::thread::spawn(move || subscriber.run());
self.handle = Some(handle);
}
pub fn stop_after(self, duration: Duration) -> anyhow::Result<()> {
std::thread::sleep(duration);
self.stop()
}
pub fn stop(self) -> anyhow::Result<()> {
if let Some(handle) = self.handle {
// if we have a handle, and the handle is not finished
if !handle.is_finished() {
let (finish_tx, finish_rx) = bounded(1);
self.stop_tx.send(finish_tx)?;
finish_rx.recv()?;
} else {
// we are sure the handle is finished, so we can join it and try to get the result.
// if we have any error on subscriber side, return the error.
match handle.join() {
Ok(rst) => rst?,
Err(_) => {
tracing::error!("Error joining subscriber thread");
}
}
}
Ok(())
} else {
// if we do not have a handle, then we have not booted the subscriber yet.
// we can just return immediately
Ok(())
}
}
}
#[derive(Debug)]
struct Senders<R> {
record_ty: RecordType,
record_sender: Sender<Arc<R>>,
stop_sender: Sender<Sender<()>>,
}
#[derive(Debug)]
struct StreamProducerInner<R> {
/// senders is used to send messages to subscribers.
senders: Vec<Senders<R>>,
/// record_cache is used to cache messsages when there are no subscribers.
record_cache: Vec<Arc<R>>,
}
impl<R> Default for StreamProducerInner<R> {
fn default() -> Self {
Self {
senders: Vec::new(),
record_cache: Vec::new(),
}
}
}
#[derive(Debug)]
pub struct StreamProducer<R> {
inner: Arc<Mutex<StreamProducerInner<R>>>,
}
impl<R> Default for StreamProducer<R> {
fn default() -> Self {
Self {
inner: Arc::new(Mutex::new(StreamProducerInner::default())),
}
}
}
impl<R> Clone for StreamProducer<R> {
fn clone(&self) -> Self {
Self {
inner: Arc::clone(&self.inner),
}
}
}
impl<R> StreamProducer<R> {
pub fn new() -> Self {
Self::default()
}
}
impl<R> StreamProducer<R>
where
R: Record + Send + Sync + 'static,
{
pub fn send(&self, record: R) -> anyhow::Result<()> {
let mut inner = self.inner.lock().unwrap();
if inner.senders.is_empty() {
inner.record_cache.push(Arc::new(record));
Ok(())
} else {
let record = Arc::new(record);
// cache record for new subscriber
inner.record_cache.push(record.clone());
// if a send fails, then it means the corresponding subscriber is dropped,
// we just remove the sender from the list of senders.
inner.senders.retain(|tx| {
if tx.record_ty != record.record_type() {
true
} else {
tx.record_sender.send(Arc::clone(&record)).is_ok()
}
});
Ok(())
}
}
pub fn subscribe<S: Subscriber<Record = R>>(
&self,
settings: S::Settings,
) -> anyhow::Result<SubscriberHandle<S>> {
let (tx, rx) = unbounded();
let (stop_tx, stop_rx) = bounded(1);
let mut inner = self.inner.lock().unwrap();
// send all previous records to the new subscriber
for record in inner.record_cache.iter() {
if S::subscribe_data_type() == record.record_type() {
tx.send(Arc::clone(record))?;
}
}
inner.senders.push(Senders {
record_sender: tx,
stop_sender: stop_tx.clone(),
record_ty: S::subscribe_data_type(),
});
Ok(SubscriberHandle {
handle: None,
stop_tx,
subscriber: Some(S::new(rx, stop_rx, settings)?),
})
}
pub fn stop(&self) -> anyhow::Result<()> {
let meta_record = Arc::new(R::from(Runtime::load()?));
let inner = self.inner.lock().unwrap();
// send runtime record to runtime subscribers
inner.senders.iter().for_each(|tx| {
if tx.record_ty == meta_record.record_type() {
if let Err(e) = tx.record_sender.send(Arc::clone(&meta_record)) {
tracing::error!("Error sending meta record: {e}");
}
}
});
// send stop signal to all subscribers
inner.senders.iter().for_each(|tx| {
let (finish_tx, finish_rx) = bounded(1);
if let Err(e) = tx.stop_sender.send(finish_tx) {
tracing::error!("Error stopping subscriber: {e}");
} else if let Err(e) = finish_rx.recv() {
tracing::error!("Error finilizing subscriber: {e}");
}
});
Ok(())
}
}
pub trait Subscriber {
type Settings;
type Record: crate::output_processors::Record + Serialize;
fn new(
record_recv: Receiver<Arc<Self::Record>>,
stop_recv: Receiver<Sender<()>>,
settings: Self::Settings,
) -> anyhow::Result<Self>
where
Self: Sized;
fn next(&self) -> Option<anyhow::Result<Arc<Self::Record>>>;
fn run(self) -> anyhow::Result<()>
where
Self: Sized,
{
while let Some(state) = self.next() {
self.sink(state?)?;
}
Ok(())
}
fn sink(&self, state: Arc<Self::Record>) -> anyhow::Result<()>;
fn subscribe_data_type() -> RecordType;
}

View File

@ -1,311 +0,0 @@
use super::{Receivers, StreamSettings, Subscriber, SubscriberFormat};
use crate::output_processors::{Record, RecordType, Runtime};
use crossbeam::channel::{Receiver, Sender};
use parking_lot::Mutex;
use serde::{Deserialize, Serialize};
use std::{
fs::{File, OpenOptions},
io::{Seek, Write},
path::PathBuf,
sync::{
atomic::{AtomicBool, Ordering},
Arc,
},
};
#[derive(Debug, Clone, Serialize, Deserialize)]
pub struct NaiveSettings {
pub path: PathBuf,
#[serde(default = "SubscriberFormat::csv")]
pub format: SubscriberFormat,
}
impl TryFrom<StreamSettings> for NaiveSettings {
type Error = String;
fn try_from(settings: StreamSettings) -> Result<Self, Self::Error> {
match settings {
StreamSettings::Naive(settings) => Ok(settings),
_ => Err("naive settings can't be created".into()),
}
}
}
impl Default for NaiveSettings {
fn default() -> Self {
let mut tmp = std::env::temp_dir();
tmp.push("simulation");
tmp.set_extension("data");
Self {
path: tmp,
format: SubscriberFormat::Csv,
}
}
}
#[derive(Debug)]
pub struct NaiveSubscriber<R> {
file: Mutex<File>,
recvs: Receivers<R>,
initialized: AtomicBool,
format: SubscriberFormat,
}
impl<R> Subscriber for NaiveSubscriber<R>
where
R: crate::output_processors::Record + Serialize,
{
type Record = R;
type Settings = NaiveSettings;
fn new(
record_recv: Receiver<Arc<Self::Record>>,
stop_recv: Receiver<Sender<()>>,
settings: Self::Settings,
) -> anyhow::Result<Self>
where
Self: Sized,
{
let mut opts = OpenOptions::new();
let recvs = Receivers {
stop_rx: stop_recv,
recv: record_recv,
};
let this = NaiveSubscriber {
file: Mutex::new(
opts.truncate(true)
.create(true)
.read(true)
.write(true)
.open(&settings.path)?,
),
recvs,
initialized: AtomicBool::new(false),
format: settings.format,
};
tracing::info!(
target = "simulation",
"subscribed to {}",
settings.path.display()
);
Ok(this)
}
fn next(&self) -> Option<anyhow::Result<Arc<Self::Record>>> {
Some(self.recvs.recv.recv().map_err(From::from))
}
fn run(self) -> anyhow::Result<()> {
loop {
crossbeam::select! {
recv(self.recvs.stop_rx) -> finish_tx => {
// Flush remaining messages after stop signal.
while let Ok(msg) = self.recvs.recv.try_recv() {
self.sink(msg)?;
}
// collect the run time meta
self.sink(Arc::new(R::from(Runtime::load()?)))?;
finish_tx?.send(())?
}
recv(self.recvs.recv) -> msg => {
self.sink(msg?)?;
}
}
}
}
fn sink(&self, state: Arc<Self::Record>) -> anyhow::Result<()> {
let mut file = self.file.lock();
match self.format {
SubscriberFormat::Json => {
write_json_record(&mut *file, &self.initialized, &*state)?;
}
SubscriberFormat::Csv => {
write_csv_record(&mut *file, &self.initialized, &*state)?;
}
SubscriberFormat::Parquet => {
panic!("native subscriber does not support parquet format")
}
}
Ok(())
}
fn subscribe_data_type() -> RecordType {
RecordType::Data
}
}
impl<R> Drop for NaiveSubscriber<R> {
fn drop(&mut self) {
if SubscriberFormat::Json == self.format {
let mut file = self.file.lock();
// To construct a valid json format, we need to overwrite the last comma
if let Err(e) = file
.seek(std::io::SeekFrom::End(-1))
.and_then(|_| file.write_all(b"]}"))
{
tracing::error!(target="simulations", err=%e, "fail to close json format");
}
}
}
}
fn write_json_record<W: std::io::Write, R: Record>(
mut w: W,
initialized: &AtomicBool,
record: &R,
) -> std::io::Result<()> {
if !initialized.load(Ordering::Acquire) {
w.write_all(b"{\"records\": [")?;
initialized.store(true, Ordering::Release);
}
for data in record.data() {
serde_json::to_writer(&mut w, data)?;
w.write_all(b",")?;
}
Ok(())
}
fn write_csv_record<W: std::io::Write, R: Record>(
w: &mut W,
initialized: &AtomicBool,
record: &R,
) -> csv::Result<()> {
// If have not write csv header, then write it
let mut w = if !initialized.load(Ordering::Acquire) {
initialized.store(true, Ordering::Release);
csv::WriterBuilder::new().has_headers(true).from_writer(w)
} else {
csv::WriterBuilder::new().has_headers(false).from_writer(w)
};
for data in record.data() {
w.serialize(data).map_err(|e| {
tracing::error!(target = "simulations", err = %e, "fail to write CSV record");
e
})?;
w.flush()?;
}
Ok(())
}
#[cfg(test)]
mod tests {
use std::{collections::HashMap, time::Duration};
use carnot_engine::View;
use crate::{
network::{
behaviour::NetworkBehaviour,
regions::{Region, RegionsData},
Network, NetworkBehaviourKey,
},
node::{
dummy_streaming::{DummyStreamingNode, DummyStreamingState},
Node, NodeId, NodeIdExt,
},
output_processors::OutData,
runner::SimulationRunner,
warding::SimulationState,
};
use super::*;
#[derive(Debug, Clone, Serialize)]
struct NaiveRecord {
states: HashMap<NodeId, View>,
}
impl<S, T: Serialize> TryFrom<&SimulationState<S, T>> for NaiveRecord {
type Error = anyhow::Error;
fn try_from(value: &SimulationState<S, T>) -> Result<Self, Self::Error> {
Ok(Self {
states: value
.nodes
.read()
.iter()
.map(|node| (node.id(), node.current_view()))
.collect(),
})
}
}
#[test]
fn test_streaming() {
let simulation_settings = crate::settings::SimulationSettings {
seed: Some(1),
..Default::default()
};
let nodes = (0..6)
.map(|idx| {
Box::new(DummyStreamingNode::new(NodeId::from_index(idx), ()))
as Box<
dyn Node<State = DummyStreamingState, Settings = ()>
+ std::marker::Send
+ Sync,
>
})
.collect::<Vec<_>>();
let network = Network::new(
RegionsData {
regions: (0..6)
.map(|idx| {
let region = match idx % 6 {
0 => Region::Europe,
1 => Region::NorthAmerica,
2 => Region::SouthAmerica,
3 => Region::Asia,
4 => Region::Africa,
5 => Region::Australia,
_ => unreachable!(),
};
(region, vec![NodeId::from_index(idx)])
})
.collect(),
node_region: (0..6)
.map(|idx| {
let region = match idx % 6 {
0 => Region::Europe,
1 => Region::NorthAmerica,
2 => Region::SouthAmerica,
3 => Region::Asia,
4 => Region::Africa,
5 => Region::Australia,
_ => unreachable!(),
};
(NodeId::from_index(idx), region)
})
.collect(),
region_network_behaviour: (0..6)
.map(|idx| {
let region = match idx % 6 {
0 => Region::Europe,
1 => Region::NorthAmerica,
2 => Region::SouthAmerica,
3 => Region::Asia,
4 => Region::Africa,
5 => Region::Australia,
_ => unreachable!(),
};
(
NetworkBehaviourKey::new(region, region),
NetworkBehaviour {
delay: Duration::from_millis(100),
drop: 0.0,
},
)
})
.collect(),
},
0,
);
let simulation_runner: SimulationRunner<(), OutData, (), DummyStreamingState> =
SimulationRunner::new(network, nodes, Default::default(), simulation_settings).unwrap();
simulation_runner.simulate().unwrap();
}
}

View File

@ -1,156 +0,0 @@
use super::{Receivers, StreamSettings, SubscriberFormat};
use crate::output_processors::{RecordType, Runtime};
use crossbeam::channel::{Receiver, Sender};
use parking_lot::Mutex;
use polars::prelude::*;
use serde::{Deserialize, Serialize};
use std::{
fs::File,
io::Cursor,
path::{Path, PathBuf},
};
#[derive(Debug, Clone, Serialize, Deserialize)]
pub struct PolarsSettings {
pub format: SubscriberFormat,
#[serde(skip_serializing_if = "Option::is_none")]
pub path: Option<PathBuf>,
}
impl TryFrom<StreamSettings> for PolarsSettings {
type Error = String;
fn try_from(settings: StreamSettings) -> Result<Self, Self::Error> {
match settings {
StreamSettings::Polars(settings) => Ok(settings),
_ => Err("polars settings can't be created".into()),
}
}
}
#[derive(Debug)]
pub struct PolarsSubscriber<R> {
data: Mutex<Vec<Arc<R>>>,
path: PathBuf,
format: SubscriberFormat,
recvs: Receivers<R>,
}
impl<R> PolarsSubscriber<R>
where
R: Serialize,
{
fn persist(&self) -> anyhow::Result<()> {
let data = self.data.lock();
let mut cursor = Cursor::new(Vec::new());
serde_json::to_writer(&mut cursor, &*data).expect("Dump data to json ");
let mut data = JsonReader::new(cursor)
.finish()
.expect("Load dataframe from intermediary json");
data.unnest(["state"])?;
match self.format {
SubscriberFormat::Json => dump_dataframe_to_json(&mut data, self.path.as_path()),
SubscriberFormat::Csv => dump_dataframe_to_csv(&mut data, self.path.as_path()),
SubscriberFormat::Parquet => dump_dataframe_to_parquet(&mut data, self.path.as_path()),
}
}
}
impl<R> super::Subscriber for PolarsSubscriber<R>
where
R: crate::output_processors::Record + Serialize,
{
type Record = R;
type Settings = PolarsSettings;
fn new(
record_recv: Receiver<Arc<Self::Record>>,
stop_recv: Receiver<Sender<()>>,
settings: Self::Settings,
) -> anyhow::Result<Self>
where
Self: Sized,
{
let recvs = Receivers {
stop_rx: stop_recv,
recv: record_recv,
};
let this = PolarsSubscriber {
data: Mutex::new(Vec::new()),
recvs,
path: settings.path.clone().unwrap_or_else(|| {
let mut p = std::env::temp_dir().join("polars");
match settings.format {
SubscriberFormat::Json => p.set_extension("json"),
SubscriberFormat::Csv => p.set_extension("csv"),
SubscriberFormat::Parquet => p.set_extension("parquet"),
};
p
}),
format: settings.format,
};
tracing::info!(
target = "simulation",
"subscribed to {}",
this.path.display()
);
Ok(this)
}
fn next(&self) -> Option<anyhow::Result<Arc<Self::Record>>> {
Some(self.recvs.recv.recv().map_err(From::from))
}
fn run(self) -> anyhow::Result<()> {
loop {
crossbeam::select! {
recv(self.recvs.stop_rx) -> finish_tx => {
// Flush remaining messages after stop signal.
while let Ok(msg) = self.recvs.recv.try_recv() {
self.sink(msg)?;
}
// collect the run time meta
self.sink(Arc::new(R::from(Runtime::load()?)))?;
finish_tx?.send(())?;
return self.persist();
}
recv(self.recvs.recv) -> msg => {
self.sink(msg?)?;
}
}
}
}
fn sink(&self, state: Arc<Self::Record>) -> anyhow::Result<()> {
self.data.lock().push(state);
Ok(())
}
fn subscribe_data_type() -> RecordType {
RecordType::Data
}
}
fn dump_dataframe_to_json(data: &mut DataFrame, out_path: &Path) -> anyhow::Result<()> {
let out_path = out_path.with_extension("json");
let f = File::create(out_path)?;
let mut writer = polars::prelude::JsonWriter::new(f);
Ok(writer.finish(data)?)
}
fn dump_dataframe_to_csv(data: &mut DataFrame, out_path: &Path) -> anyhow::Result<()> {
let out_path = out_path.with_extension("csv");
let f = File::create(out_path)?;
let mut writer = polars::prelude::CsvWriter::new(f);
Ok(writer.finish(data)?)
}
fn dump_dataframe_to_parquet(data: &mut DataFrame, out_path: &Path) -> anyhow::Result<()> {
let out_path = out_path.with_extension("parquet");
let f = File::create(out_path)?;
let writer = polars::prelude::ParquetWriter::new(f);
Ok(writer.finish(data).map(|_| ())?)
}

View File

@ -1,218 +0,0 @@
use super::{Receivers, Subscriber};
use crate::output_processors::{RecordType, Runtime};
use crossbeam::channel::{Receiver, Sender};
use serde::{Deserialize, Serialize};
use std::{
fs::{File, OpenOptions},
io::Write,
path::PathBuf,
sync::{Arc, Mutex},
};
#[derive(Debug, Clone, Serialize, Deserialize)]
pub struct RuntimeSettings {
pub path: PathBuf,
}
impl Default for RuntimeSettings {
fn default() -> Self {
let mut tmp = std::env::temp_dir();
tmp.push("simulation");
tmp.set_extension("runtime");
Self { path: tmp }
}
}
#[derive(Debug)]
pub struct RuntimeSubscriber<R> {
file: Arc<Mutex<File>>,
recvs: Arc<Receivers<R>>,
}
impl<R> Subscriber for RuntimeSubscriber<R>
where
R: crate::output_processors::Record + Serialize,
{
type Record = R;
type Settings = RuntimeSettings;
fn new(
record_recv: Receiver<Arc<Self::Record>>,
stop_recv: Receiver<Sender<()>>,
settings: Self::Settings,
) -> anyhow::Result<Self>
where
Self: Sized,
{
let mut opts = OpenOptions::new();
let recvs = Receivers {
stop_rx: stop_recv,
recv: record_recv,
};
let this = RuntimeSubscriber {
file: Arc::new(Mutex::new(
opts.truncate(true)
.create(true)
.read(true)
.write(true)
.open(&settings.path)?,
)),
recvs: Arc::new(recvs),
};
tracing::info!(
taget = "simulation",
"subscribed to {}",
settings.path.display()
);
Ok(this)
}
fn next(&self) -> Option<anyhow::Result<Arc<Self::Record>>> {
Some(self.recvs.recv.recv().map_err(From::from))
}
fn run(self) -> anyhow::Result<()> {
crossbeam::select! {
recv(self.recvs.stop_rx) -> finish_tx => {
// collect the run time meta
self.sink(Arc::new(R::from(Runtime::load()?)))?;
finish_tx?.send(())?;
}
recv(self.recvs.recv) -> msg => {
self.sink(msg?)?;
}
}
Ok(())
}
fn sink(&self, state: Arc<Self::Record>) -> anyhow::Result<()> {
let mut file = self.file.lock().expect("failed to lock file");
serde_json::to_writer(&mut *file, &state)?;
file.write_all(b",\n")?;
Ok(())
}
fn subscribe_data_type() -> RecordType {
RecordType::Data
}
}
#[cfg(test)]
mod tests {
use std::{collections::HashMap, time::Duration};
use carnot_engine::View;
use crate::{
network::{
behaviour::NetworkBehaviour,
regions::{Region, RegionsData},
Network, NetworkBehaviourKey,
},
node::{
dummy_streaming::{DummyStreamingNode, DummyStreamingState},
Node, NodeId, NodeIdExt,
},
output_processors::OutData,
runner::SimulationRunner,
warding::SimulationState,
};
use super::*;
#[derive(Debug, Clone, Serialize)]
struct RuntimeRecord {
states: HashMap<NodeId, View>,
}
impl<S, T: Serialize> TryFrom<&SimulationState<S, T>> for RuntimeRecord {
type Error = anyhow::Error;
fn try_from(value: &SimulationState<S, T>) -> Result<Self, Self::Error> {
Ok(Self {
states: value
.nodes
.read()
.iter()
.map(|node| (node.id(), node.current_view()))
.collect(),
})
}
}
#[test]
fn test_streaming() {
let simulation_settings = crate::settings::SimulationSettings {
seed: Some(1),
..Default::default()
};
let nodes = (0..6)
.map(|idx| {
Box::new(DummyStreamingNode::new(NodeId::from_index(idx), ()))
as Box<
dyn Node<State = DummyStreamingState, Settings = ()>
+ std::marker::Send
+ Sync,
>
})
.collect::<Vec<_>>();
let network = Network::new(
RegionsData {
regions: (0..6)
.map(|idx| {
let region = match idx % 6 {
0 => Region::Europe,
1 => Region::NorthAmerica,
2 => Region::SouthAmerica,
3 => Region::Asia,
4 => Region::Africa,
5 => Region::Australia,
_ => unreachable!(),
};
(region, vec![NodeId::from_index(idx)])
})
.collect(),
node_region: (0..6)
.map(|idx| {
let region = match idx % 6 {
0 => Region::Europe,
1 => Region::NorthAmerica,
2 => Region::SouthAmerica,
3 => Region::Asia,
4 => Region::Africa,
5 => Region::Australia,
_ => unreachable!(),
};
(NodeId::from_index(idx), region)
})
.collect(),
region_network_behaviour: (0..6)
.map(|idx| {
let region = match idx % 6 {
0 => Region::Europe,
1 => Region::NorthAmerica,
2 => Region::SouthAmerica,
3 => Region::Asia,
4 => Region::Africa,
5 => Region::Australia,
_ => unreachable!(),
};
(
NetworkBehaviourKey::new(region, region),
NetworkBehaviour {
delay: Duration::from_millis(100),
drop: 0.0,
},
)
})
.collect(),
},
0,
);
let simulation_runner: SimulationRunner<(), OutData, (), DummyStreamingState> =
SimulationRunner::new(network, nodes, Default::default(), simulation_settings).unwrap();
simulation_runner.simulate().unwrap();
}
}

View File

@ -1,218 +0,0 @@
use super::{Receivers, Subscriber};
use crate::output_processors::{RecordType, Runtime};
use crossbeam::channel::{Receiver, Sender};
use serde::{Deserialize, Serialize};
use std::{
fs::{File, OpenOptions},
io::Write,
path::PathBuf,
sync::{Arc, Mutex},
};
#[derive(Debug, Clone, Serialize, Deserialize)]
pub struct SettingsSubscriberSettings {
pub path: PathBuf,
}
impl Default for SettingsSubscriberSettings {
fn default() -> Self {
let mut tmp = std::env::temp_dir();
tmp.push("simulation");
tmp.set_extension("conf");
Self { path: tmp }
}
}
#[derive(Debug)]
pub struct SettingsSubscriber<R> {
file: Arc<Mutex<File>>,
recvs: Arc<Receivers<R>>,
}
impl<R> Subscriber for SettingsSubscriber<R>
where
R: crate::output_processors::Record + Serialize,
{
type Record = R;
type Settings = SettingsSubscriberSettings;
fn new(
record_recv: Receiver<Arc<Self::Record>>,
stop_recv: Receiver<Sender<()>>,
settings: Self::Settings,
) -> anyhow::Result<Self>
where
Self: Sized,
{
let mut opts = OpenOptions::new();
let recvs = Receivers {
stop_rx: stop_recv,
recv: record_recv,
};
let this = SettingsSubscriber {
file: Arc::new(Mutex::new(
opts.truncate(true)
.create(true)
.read(true)
.write(true)
.open(&settings.path)?,
)),
recvs: Arc::new(recvs),
};
tracing::info!(
target = "simulation",
"subscribed to {}",
settings.path.display()
);
Ok(this)
}
fn next(&self) -> Option<anyhow::Result<Arc<Self::Record>>> {
Some(self.recvs.recv.recv().map_err(From::from))
}
fn run(self) -> anyhow::Result<()> {
crossbeam::select! {
recv(self.recvs.stop_rx) -> finish_tx => {
// collect the run time meta
self.sink(Arc::new(R::from(Runtime::load()?)))?;
finish_tx?.send(())?;
}
recv(self.recvs.recv) -> msg => {
self.sink(msg?)?;
}
}
Ok(())
}
fn sink(&self, state: Arc<Self::Record>) -> anyhow::Result<()> {
let mut file = self.file.lock().expect("failed to lock file");
serde_json::to_writer(&mut *file, &state)?;
file.write_all(b",\n")?;
Ok(())
}
fn subscribe_data_type() -> RecordType {
RecordType::Data
}
}
#[cfg(test)]
mod tests {
use std::{collections::HashMap, time::Duration};
use carnot_engine::View;
use crate::{
network::{
behaviour::NetworkBehaviour,
regions::{Region, RegionsData},
Network, NetworkBehaviourKey,
},
node::{
dummy_streaming::{DummyStreamingNode, DummyStreamingState},
Node, NodeId, NodeIdExt,
},
output_processors::OutData,
runner::SimulationRunner,
warding::SimulationState,
};
use super::*;
#[derive(Debug, Clone, Serialize)]
struct SettingsRecord {
states: HashMap<NodeId, View>,
}
impl<S, T: Serialize> TryFrom<&SimulationState<S, T>> for SettingsRecord {
type Error = anyhow::Error;
fn try_from(value: &SimulationState<S, T>) -> Result<Self, Self::Error> {
Ok(Self {
states: value
.nodes
.read()
.iter()
.map(|node| (node.id(), node.current_view()))
.collect(),
})
}
}
#[test]
fn test_streaming() {
let simulation_settings = crate::settings::SimulationSettings {
seed: Some(1),
..Default::default()
};
let nodes = (0..6)
.map(|idx| {
Box::new(DummyStreamingNode::new(NodeId::from_index(idx), ()))
as Box<
dyn Node<State = DummyStreamingState, Settings = ()>
+ std::marker::Send
+ Sync,
>
})
.collect::<Vec<_>>();
let network = Network::new(
RegionsData {
regions: (0..6)
.map(|idx| {
let region = match idx % 6 {
0 => Region::Europe,
1 => Region::NorthAmerica,
2 => Region::SouthAmerica,
3 => Region::Asia,
4 => Region::Africa,
5 => Region::Australia,
_ => unreachable!(),
};
(region, vec![NodeId::from_index(idx)])
})
.collect(),
node_region: (0..6)
.map(|idx| {
let region = match idx % 6 {
0 => Region::Europe,
1 => Region::NorthAmerica,
2 => Region::SouthAmerica,
3 => Region::Asia,
4 => Region::Africa,
5 => Region::Australia,
_ => unreachable!(),
};
(NodeId::from_index(idx), region)
})
.collect(),
region_network_behaviour: (0..6)
.map(|idx| {
let region = match idx % 6 {
0 => Region::Europe,
1 => Region::NorthAmerica,
2 => Region::SouthAmerica,
3 => Region::Asia,
4 => Region::Africa,
5 => Region::Australia,
_ => unreachable!(),
};
(
NetworkBehaviourKey::new(region, region),
NetworkBehaviour {
delay: Duration::from_millis(100),
drop: 0.0,
},
)
})
.collect(),
},
0,
);
let simulation_runner: SimulationRunner<(), OutData, (), DummyStreamingState> =
SimulationRunner::new(network, nodes, Default::default(), simulation_settings).unwrap();
simulation_runner.simulate().unwrap();
}
}

View File

@ -1,52 +0,0 @@
use crate::warding::{SimulationState, SimulationWard};
use carnot_engine::View;
use serde::{Deserialize, Serialize};
/// MinMaxView. It monitors the gap between a min view and max view, triggers when surpassing
/// the max view - min view is larger than a gap.
#[derive(Debug, Serialize, Deserialize, Copy, Clone)]
#[serde(transparent)]
pub struct MinMaxViewWard {
max_gap: View,
}
impl<S, T> SimulationWard<S, T> for MinMaxViewWard {
type SimulationState = SimulationState<S, T>;
fn analyze(&mut self, state: &Self::SimulationState) -> bool {
let mut min = View::new(i64::MAX);
let mut max = View::new(0);
let nodes = state.nodes.read();
for node in nodes.iter() {
let view = node.current_view();
min = min.min(view);
max = max.max(view);
}
max - min >= self.max_gap
}
}
#[cfg(test)]
mod test {
use crate::warding::minmax::MinMaxViewWard;
use crate::warding::{SimulationState, SimulationWard};
use carnot_engine::View;
use parking_lot::RwLock;
use std::sync::Arc;
#[test]
fn rebase_threshold() {
let mut minmax = MinMaxViewWard {
max_gap: View::new(5),
};
let state = SimulationState {
nodes: Arc::new(RwLock::new(vec![Box::new(10)])),
};
// we only have one node, so always false
assert!(!minmax.analyze(&state));
// push a new node with 10
state.nodes.write().push(Box::new(20));
// we now have two nodes and the max - min is 10 > max_gap 5, so true
assert!(minmax.analyze(&state));
}
}

View File

@ -1,60 +0,0 @@
// std
use std::sync::Arc;
// crates
use parking_lot::RwLock;
use serde::{Deserialize, Serialize};
// internal
use crate::runner::BoxedNode;
mod minmax;
mod stalled;
mod ttf;
pub struct SimulationState<S, T> {
pub nodes: Arc<RwLock<Vec<BoxedNode<S, T>>>>,
}
impl<S, T> SimulationState<S, T> {
#[inline]
pub fn new(nodes: Vec<BoxedNode<S, T>>) -> Self {
Self {
nodes: Arc::new(RwLock::new(nodes)),
}
}
}
/// A ward is a computation over the `NetworkState`, it must return true if the state satisfies
/// the warding conditions. It is used to stop the consensus simulation if such condition is reached.
pub trait SimulationWard<S, T> {
type SimulationState;
fn analyze(&mut self, state: &Self::SimulationState) -> bool;
}
/// Ward dispatcher
/// Enum to avoid Boxing (Box<dyn SimulationWard>) wards.
#[derive(Debug, Clone, Serialize, Deserialize)]
#[serde(rename_all = "snake_case")]
pub enum Ward {
MaxView(ttf::MaxViewWard),
MinMaxView(minmax::MinMaxViewWard),
StalledView(stalled::StalledViewWard),
}
impl Ward {
pub fn simulation_ward_mut<S, T>(
&mut self,
) -> &mut dyn SimulationWard<S, T, SimulationState = SimulationState<S, T>> {
match self {
Ward::MaxView(ward) => ward,
Ward::MinMaxView(ward) => ward,
Ward::StalledView(ward) => ward,
}
}
}
impl<S, T> SimulationWard<S, T> for Ward {
type SimulationState = SimulationState<S, T>;
fn analyze(&mut self, state: &Self::SimulationState) -> bool {
self.simulation_ward_mut().analyze(state)
}
}

View File

@ -1,88 +0,0 @@
use crate::runner::BoxedNode;
use crate::warding::{SimulationState, SimulationWard};
use serde::{Deserialize, Serialize};
/// StalledView. Track stalled nodes (e.g incoming queue is empty, the node doesn't write to other queues)
#[derive(Debug, Serialize, Deserialize, Clone)]
pub struct StalledViewWard {
// the hash checksum
consecutive_viewed_checkpoint: Option<u32>,
// use to check if the node is stalled
criterion: usize,
threshold: usize,
}
impl StalledViewWard {
fn update_state(&mut self, cks: u32) {
match &mut self.consecutive_viewed_checkpoint {
Some(cp) => {
if cks == *cp {
self.criterion += 1;
} else {
*cp = cks;
// reset the criterion
self.criterion = 0;
}
}
None => {
self.consecutive_viewed_checkpoint = Some(cks);
}
}
}
}
impl<S, T> SimulationWard<S, T> for StalledViewWard {
type SimulationState = SimulationState<S, T>;
fn analyze(&mut self, state: &Self::SimulationState) -> bool {
let nodes = state.nodes.read();
self.update_state(checksum(nodes.as_slice()));
self.criterion >= self.threshold
}
}
#[inline]
fn checksum<S, T>(nodes: &[BoxedNode<S, T>]) -> u32 {
let mut hash = crc32fast::Hasher::new();
for node in nodes.iter() {
let view: i64 = node.current_view().into();
hash.update(&(view as usize).to_be_bytes());
// TODO: hash messages in the node
}
hash.finalize()
}
#[cfg(test)]
mod test {
use super::*;
use parking_lot::RwLock;
use std::sync::Arc;
#[test]
fn rebase_threshold() {
let mut stalled = StalledViewWard {
consecutive_viewed_checkpoint: None,
criterion: 0,
threshold: 2,
};
let state = SimulationState {
nodes: Arc::new(RwLock::new(vec![Box::new(10)])),
};
// increase the criterion, 1
assert!(!stalled.analyze(&state));
// increase the criterion, 2
assert!(!stalled.analyze(&state));
// increase the criterion, 3 > threshold 2, so true
assert!(stalled.analyze(&state));
// push a new one, so the criterion is reset to 0
state.nodes.write().push(Box::new(20));
assert!(!stalled.analyze(&state));
// increase the criterion, 2
assert!(!stalled.analyze(&state));
// increase the criterion, 3 > threshold 2, so true
assert!(stalled.analyze(&state));
}
}

View File

@ -1,47 +0,0 @@
use crate::warding::{SimulationState, SimulationWard};
use carnot_engine::View;
use serde::{Deserialize, Serialize};
/// Time to finality ward. It monitors the amount of rounds of the simulations, triggers when surpassing
/// the set threshold.
#[derive(Debug, Serialize, Deserialize, Copy, Clone)]
#[serde(transparent)]
pub struct MaxViewWard {
max_view: View,
}
impl<S, T> SimulationWard<S, T> for MaxViewWard {
type SimulationState = SimulationState<S, T>;
fn analyze(&mut self, state: &Self::SimulationState) -> bool {
state
.nodes
.read()
.iter()
.all(|n| n.current_view() >= self.max_view)
}
}
#[cfg(test)]
mod test {
use crate::warding::ttf::MaxViewWard;
use crate::warding::{SimulationState, SimulationWard};
use carnot_engine::View;
use parking_lot::RwLock;
use std::sync::Arc;
#[test]
fn rebase_threshold() {
let mut ttf = MaxViewWard {
max_view: View::new(10),
};
let node = 11;
let state = SimulationState {
nodes: Arc::new(RwLock::new(vec![Box::new(node)])),
};
assert!(ttf.analyze(&state));
state.nodes.write().push(Box::new(9));
assert!(!ttf.analyze(&state));
}
}

View File

@ -6,18 +6,16 @@ publish = false
[dependencies]
nomos-node = { path = "../nodes/nomos-node", default-features = false }
carnot-consensus = { path = "../nomos-services/carnot-consensus" }
nomos-network = { path = "../nomos-services/network", features = ["libp2p"] }
cryptarchia-consensus = { path = "../nomos-services/cryptarchia-consensus" }
mixnet = { path = "../mixnet" }
nomos-log = { path = "../nomos-services/log" }
nomos-api = { path = "../nomos-services/api" }
overwatch-rs = { git = "https://github.com/logos-co/Overwatch", rev = "2f70806" }
nomos-core = { path = "../nomos-core" }
carnot-engine = { path = "../consensus/carnot-engine", features = ["serde"] }
nomos-mempool = { path = "../nomos-services/mempool", features = [
"mock",
"libp2p",
] }
cryptarchia-engine = { path = "../consensus/cryptarchia-engine", features = ["serde"] }
cryptarchia-ledger = { path = "../ledger/cryptarchia-ledger", features = ["serde"] }
nomos-mempool = { path = "../nomos-services/mempool", features = ["mock", "libp2p"] }
nomos-da = { path = "../nomos-services/data-availability" }
full-replication = { path = "../nomos-da/full-replication" }
rand = "0.8"
@ -36,14 +34,11 @@ fraction = "0.13"
ntest = "0.9.0"
criterion = { version = "0.5", features = ["async_tokio"] }
nomos-cli = { path = "../nomos-cli" }
time = "0.3"
[[test]]
name = "test_consensus_happy_path"
path = "src/tests/happy.rs"
[[test]]
name = "test_consensus_unhappy_path"
path = "src/tests/unhappy.rs"
name = "test_cryptarchia_happy_path"
path = "src/tests/cryptarchia/happy.rs"
[[test]]
name = "test_cli"
@ -52,3 +47,4 @@ path = "src/tests/cli.rs"
[features]
mixnet = ["nomos-network/mixnet"]
metrics = ["nomos-node/metrics"]

View File

@ -10,7 +10,8 @@ use std::time::Duration;
use std::{fmt::Debug, sync::Mutex};
//crates
use fraction::{Fraction, One};
use nomos_libp2p::{Multiaddr, Swarm};
use nomos_node::Config;
use rand::{thread_rng, Rng};
static NET_PORT: Lazy<Mutex<u16>> = Lazy::new(|| Mutex::new(thread_rng().gen_range(8000..10000)));
@ -38,8 +39,47 @@ pub fn adjust_timeout(d: Duration) -> Duration {
#[async_trait::async_trait]
pub trait Node: Sized {
type ConsensusInfo: Debug + Clone + PartialEq;
async fn spawn_nodes(config: SpawnConfig) -> Vec<Self>;
fn node_configs(config: SpawnConfig) -> Vec<nomos_node::Config>;
async fn spawn(mut config: Config) -> Self;
async fn spawn_nodes(config: SpawnConfig) -> Vec<Self> {
let mut nodes = Vec::new();
for conf in Self::node_configs(config) {
nodes.push(Self::spawn(conf).await);
}
nodes
}
fn node_configs(config: SpawnConfig) -> Vec<Config> {
match config {
SpawnConfig::Star { consensus } => {
let mut configs = Self::create_node_configs(consensus);
let next_leader_config = configs.remove(0);
let first_node_addr = node_address(&next_leader_config);
let mut node_configs = vec![next_leader_config];
for mut conf in configs {
conf.network
.backend
.initial_peers
.push(first_node_addr.clone());
node_configs.push(conf);
}
node_configs
}
SpawnConfig::Chain { consensus } => {
let mut configs = Self::create_node_configs(consensus);
let next_leader_config = configs.remove(0);
let mut prev_node_addr = node_address(&next_leader_config);
let mut node_configs = vec![next_leader_config];
for mut conf in configs {
conf.network.backend.initial_peers.push(prev_node_addr);
prev_node_addr = node_address(&conf);
node_configs.push(conf);
}
node_configs
}
}
}
fn create_node_configs(consensus: ConsensusConfig) -> Vec<Config>;
async fn consensus_info(&self) -> Self::ConsensusInfo;
fn stop(&mut self);
}
@ -58,20 +98,41 @@ impl SpawnConfig {
Self::Chain {
consensus: ConsensusConfig {
n_participants,
// All nodes are expected to be responsive in happy-path tests.
threshold: Fraction::one(),
// Set the timeout conservatively
// since nodes should be spawned sequentially in the chain topology
// and it takes 1+ secs for each nomos-node to be started.
timeout: adjust_timeout(Duration::from_millis(n_participants as u64 * 2500)),
// by setting the active slot coeff close to 1, we also increase the probability of multiple blocks (forks)
// being produced in the same slot (epoch). Setting the security parameter to some value > 1
// ensures nodes have some time to sync before deciding on the longest chain.
security_param: 10,
// a block should be produced (on average) every slot
active_slot_coeff: 0.9,
},
}
}
pub fn star_happy(n_participants: usize) -> Self {
Self::Star {
consensus: ConsensusConfig {
n_participants,
// by setting the slot coeff to 1, we also increase the probability of multiple blocks (forks)
// being produced in the same slot (epoch). Setting the security parameter to some value > 1
// ensures nodes have some time to sync before deciding on the longest chain.
security_param: 10,
// a block should be produced (on average) every slot
active_slot_coeff: 0.9,
},
}
}
}
fn node_address(config: &Config) -> Multiaddr {
Swarm::multiaddr(
std::net::Ipv4Addr::new(127, 0, 0, 1),
config.network.backend.inner.port,
)
}
#[derive(Clone)]
pub struct ConsensusConfig {
pub n_participants: usize,
pub threshold: Fraction,
pub timeout: Duration,
pub security_param: u32,
pub active_slot_coeff: f64,
}

View File

@ -1,6 +1,6 @@
pub mod nomos;
pub use nomos::NomosNode;
use tempfile::TempDir;
const LOGS_PREFIX: &str = "__logs";

View File

@ -2,17 +2,11 @@
use std::net::SocketAddr;
use std::process::{Child, Command, Stdio};
use std::time::Duration;
#[cfg(feature = "mixnet")]
use std::{
net::{IpAddr, Ipv4Addr},
num::NonZeroU8,
};
// internal
use super::{create_tempdir, persist_tempdir, LOGS_PREFIX};
use crate::{adjust_timeout, get_available_port, ConsensusConfig, Node, SpawnConfig};
use carnot_consensus::{CarnotInfo, CarnotSettings};
use carnot_engine::overlay::{RandomBeaconState, RoundRobin, TreeOverlay, TreeOverlaySettings};
use carnot_engine::{NodeId, Overlay};
use crate::{adjust_timeout, get_available_port, ConsensusConfig, Node};
use cryptarchia_consensus::{CryptarchiaInfo, CryptarchiaSettings, TimeConfig};
use cryptarchia_ledger::{Coin, LedgerState};
use full_replication::Certificate;
#[cfg(feature = "mixnet")]
use mixnet::{
@ -22,7 +16,6 @@ use mixnet::{
topology::{MixNodeInfo, MixnetTopology},
};
use nomos_core::{block::Block, header::HeaderId};
use nomos_libp2p::{Multiaddr, Swarm};
use nomos_log::{LoggerBackend, LoggerFormat};
use nomos_mempool::MempoolMetrics;
#[cfg(feature = "mixnet")]
@ -30,17 +23,19 @@ use nomos_network::backends::libp2p::mixnet::MixnetConfig;
use nomos_network::{backends::libp2p::Libp2pConfig, NetworkConfig};
use nomos_node::{api::AxumBackendSettings, Config, Tx};
// crates
use fraction::Fraction;
use once_cell::sync::Lazy;
use rand::{thread_rng, Rng};
use reqwest::{Client, Url};
use tempfile::NamedTempFile;
use time::OffsetDateTime;
static CLIENT: Lazy<Client> = Lazy::new(Client::new);
const CRYPTARCHIA_INFO_API: &str = "cryptarchia/info";
const GET_HEADERS_INFO: &str = "cryptarchia/headers";
const NOMOS_BIN: &str = "../target/debug/nomos-node";
const CARNOT_INFO_API: &str = "carnot/info";
const STORAGE_BLOCKS_API: &str = "storage/block";
const GET_BLOCKS_INFO: &str = "carnot/blocks";
#[cfg(feature = "mixnet")]
const NUM_MIXNODE_CANDIDATES: usize = 2;
pub struct NomosNode {
addr: SocketAddr,
@ -62,13 +57,8 @@ impl Drop for NomosNode {
}
}
}
impl NomosNode {
pub fn id(&self) -> NodeId {
NodeId::from(self.config.consensus.private_key)
}
pub async fn spawn(mut config: Config) -> Self {
pub async fn spawn_inner(mut config: Config) -> Self {
// Waku stores the messages in a db file in the current dir, we need a different
// directory for each node to avoid conflicts
let dir = create_tempdir().unwrap();
@ -111,9 +101,13 @@ impl NomosNode {
.await
}
pub fn url(&self) -> Url {
format!("http://{}", self.addr).parse().unwrap()
}
async fn wait_online(&self) {
loop {
let res = self.get(CARNOT_INFO_API).await;
let res = self.get("da/metrics").await;
if res.is_ok() && res.unwrap().status().is_success() {
break;
}
@ -121,10 +115,6 @@ impl NomosNode {
}
}
pub fn url(&self) -> Url {
format!("http://{}", self.addr).parse().unwrap()
}
pub async fn get_block(&self, id: HeaderId) -> Option<Block<Tx, Certificate>> {
CLIENT
.post(&format!("http://{}/{}", self.addr, STORAGE_BLOCKS_API))
@ -157,29 +147,6 @@ impl NomosNode {
}
}
pub async fn get_blocks_info(
&self,
from: Option<HeaderId>,
to: Option<HeaderId>,
) -> Vec<carnot_engine::Block<HeaderId>> {
let mut req = CLIENT.get(format!("http://{}/{}", self.addr, GET_BLOCKS_INFO));
if let Some(from) = from {
req = req.query(&[("from", from)]);
}
if let Some(to) = to {
req = req.query(&[("to", to)]);
}
req.send()
.await
.unwrap()
.json::<Vec<carnot_engine::Block<_>>>()
.await
.unwrap()
}
// not async so that we can use this in `Drop`
pub fn get_logs_from_file(&self) -> String {
println!(
@ -205,141 +172,182 @@ impl NomosNode {
pub fn config(&self) -> &Config {
&self.config
}
pub async fn get_headers(&self, from: Option<HeaderId>, to: Option<HeaderId>) -> Vec<HeaderId> {
let mut req = CLIENT.get(format!("http://{}/{}", self.addr, GET_HEADERS_INFO));
if let Some(from) = from {
req = req.query(&[("from", from)]);
}
if let Some(to) = to {
req = req.query(&[("to", to)]);
}
let res = req.send().await;
println!("res: {res:?}");
res.unwrap().json::<Vec<HeaderId>>().await.unwrap()
}
}
#[async_trait::async_trait]
impl Node for NomosNode {
type ConsensusInfo = CarnotInfo;
type ConsensusInfo = CryptarchiaInfo;
/// Spawn nodes sequentially.
/// After one node is spawned successfully, the next node is spawned.
async fn spawn_nodes(config: SpawnConfig) -> Vec<Self> {
let mut nodes = Vec::new();
for conf in Self::node_configs(config) {
nodes.push(Self::spawn(conf).await);
}
nodes
}
fn node_configs(config: SpawnConfig) -> Vec<nomos_node::Config> {
match config {
SpawnConfig::Star { consensus } => {
let (next_leader_config, configs) = create_node_configs(consensus);
let first_node_addr = node_address(&next_leader_config);
let mut node_configs = vec![next_leader_config];
for mut conf in configs {
conf.network
.backend
.initial_peers
.push(first_node_addr.clone());
node_configs.push(conf);
}
node_configs
}
SpawnConfig::Chain { consensus } => {
let (next_leader_config, configs) = create_node_configs(consensus);
let mut prev_node_addr = node_address(&next_leader_config);
let mut node_configs = vec![next_leader_config];
for mut conf in configs {
conf.network
.backend
.initial_peers
.push(prev_node_addr.clone());
prev_node_addr = node_address(&conf);
node_configs.push(conf);
}
node_configs
}
}
async fn spawn(config: Config) -> Self {
Self::spawn_inner(config).await
}
async fn consensus_info(&self) -> Self::ConsensusInfo {
let res = self.get(CARNOT_INFO_API).await;
let res = self.get(CRYPTARCHIA_INFO_API).await;
println!("{:?}", res);
res.unwrap().json().await.unwrap()
}
fn stop(&mut self) {
self.child.kill().unwrap();
}
/// Depending on the network topology, the next leader must be spawned first,
/// so the leader can receive votes from all other nodes that will be subsequently spawned.
/// If not, the leader will miss votes from nodes spawned before itself.
/// This issue will be resolved by devising the block catch-up mechanism in the future.
fn create_node_configs(consensus: ConsensusConfig) -> Vec<Config> {
// we use the same random bytes for:
// * da id
// * coin sk
// * coin nonce
let mut ids = vec![[0; 32]; consensus.n_participants];
for id in &mut ids {
thread_rng().fill(id);
}
#[cfg(feature = "mixnet")]
let (mixclient_config, mixnode_configs) = create_mixnet_config(&ids);
let coins = ids
.iter()
.map(|&id| Coin::new(id, id.into(), 1.into()))
.collect::<Vec<_>>();
// no commitments for now, proofs are not checked anyway
let genesis_state = LedgerState::from_commitments(
coins.iter().map(|c| c.commitment()),
(ids.len() as u32).into(),
);
let ledger_config = cryptarchia_ledger::Config {
epoch_stake_distribution_stabilization: 3,
epoch_period_nonce_buffer: 3,
epoch_period_nonce_stabilization: 4,
consensus_config: cryptarchia_engine::Config {
security_param: consensus.security_param,
active_slot_coeff: consensus.active_slot_coeff,
},
};
let time_config = TimeConfig {
slot_duration: Duration::from_secs(1),
chain_start_time: OffsetDateTime::now_utc(),
};
#[allow(unused_mut)]
let mut configs = ids
.into_iter()
.zip(coins)
.enumerate()
.map(|(_i, (da_id, coin))| {
create_node_config(
da_id,
genesis_state.clone(),
ledger_config.clone(),
vec![coin],
time_config.clone(),
#[cfg(feature = "mixnet")]
MixnetConfig {
mixclient: mixclient_config.clone(),
mixnode: mixnode_configs[_i].clone(),
},
)
})
.collect::<Vec<_>>();
#[cfg(feature = "mixnet")]
{
// Build a topology using only a subset of nodes.
let mixnode_candidates = configs
.iter()
.take(NUM_MIXNODE_CANDIDATES)
.collect::<Vec<_>>();
let topology = build_mixnet_topology(&mixnode_candidates);
// Set the topology to all configs
for config in &mut configs {
config.network.backend.mixnet.mixclient.topology = topology.clone();
}
configs
}
#[cfg(not(feature = "mixnet"))]
configs
}
}
pub enum Pool {
Da,
Cl,
}
#[cfg(feature = "mixnet")]
const NUM_MIXNODE_CANDIDATES: usize = 2;
fn create_mixnet_config(ids: &[[u8; 32]]) -> (MixClientConfig, Vec<MixNodeConfig>) {
use std::num::NonZeroU8;
/// Returns the config of the next leader and all other nodes.
///
/// Depending on the network topology, the next leader must be spawned first,
/// so the leader can receive votes from all other nodes that will be subsequently spawned.
/// If not, the leader will miss votes from nodes spawned before itself.
/// This issue will be resolved by devising the block catch-up mechanism in the future.
fn create_node_configs(consensus: ConsensusConfig) -> (Config, Vec<Config>) {
let mut ids = vec![[0; 32]; consensus.n_participants];
for id in &mut ids {
thread_rng().fill(id);
}
#[cfg(feature = "mixnet")]
let (mixclient_config, mixnode_configs) = create_mixnet_config(&ids);
let mut configs = ids
let mixnode_configs: Vec<MixNodeConfig> = ids
.iter()
.enumerate()
.map(|(_i, id)| {
create_node_config(
ids.iter().copied().map(NodeId::new).collect(),
*id,
consensus.threshold,
consensus.timeout,
#[cfg(feature = "mixnet")]
MixnetConfig {
mixclient: mixclient_config.clone(),
mixnode: mixnode_configs[_i].clone(),
},
.map(|id| MixNodeConfig {
encryption_private_key: *id,
delay_rate_per_min: 100000000.0,
})
.collect();
// Build an empty topology because it will be constructed with meaningful node infos later
let topology = MixnetTopology::new(Vec::new(), 0, 0, [1u8; 32]).unwrap();
(
MixClientConfig {
topology,
emission_rate_per_min: 120.0,
redundancy: NonZeroU8::new(1).unwrap(),
},
mixnode_configs,
)
}
#[cfg(feature = "mixnet")]
fn build_mixnet_topology(mixnode_candidates: &[&Config]) -> MixnetTopology {
use mixnet::crypto::public_key_from;
use std::net::{IpAddr, Ipv4Addr};
let candidates = mixnode_candidates
.iter()
.map(|config| {
MixNodeInfo::new(
NodeAddress::from(SocketAddr::new(
IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)),
config.network.backend.inner.port,
)),
public_key_from(config.network.backend.mixnet.mixnode.encryption_private_key),
)
.unwrap()
})
.collect::<Vec<_>>();
let overlay = TreeOverlay::new(configs[0].consensus.overlay_settings.clone());
let next_leader = overlay.next_leader();
let next_leader_idx = ids
.iter()
.position(|&id| NodeId::from(id) == next_leader)
.unwrap();
#[cfg(not(feature = "mixnet"))]
{
let next_leader_config = configs.swap_remove(next_leader_idx);
(next_leader_config, configs)
}
#[cfg(feature = "mixnet")]
{
let mut next_leader_config = configs.swap_remove(next_leader_idx);
// Build a topology using only a subset of nodes.
let mut mixnode_candidates = vec![&next_leader_config];
configs
.iter()
.take(NUM_MIXNODE_CANDIDATES - 1)
.for_each(|config| mixnode_candidates.push(config));
let topology = build_mixnet_topology(&mixnode_candidates);
// Set the topology to all configs
next_leader_config.network.backend.mixnet.mixclient.topology = topology.clone();
configs.iter_mut().for_each(|config| {
config.network.backend.mixnet.mixclient.topology = topology.clone();
});
(next_leader_config, configs)
}
let num_layers = candidates.len();
MixnetTopology::new(candidates, num_layers, 1, [1u8; 32]).unwrap()
}
fn create_node_config(
nodes: Vec<NodeId>,
id: [u8; 32],
threshold: Fraction,
timeout: Duration,
genesis_state: LedgerState,
config: cryptarchia_ledger::Config,
coins: Vec<Coin>,
time: TimeConfig,
#[cfg(feature = "mixnet")] mixnet_config: MixnetConfig,
) -> Config {
let mut config = Config {
@ -351,20 +359,11 @@ fn create_node_config(
mixnet: mixnet_config,
},
},
consensus: CarnotSettings {
private_key: id,
overlay_settings: TreeOverlaySettings {
nodes,
leader: RoundRobin::new(),
current_leader: [0; 32].into(),
number_of_committees: 1,
committee_membership: RandomBeaconState::initial_sad_from_entropy([0; 32]),
// By setting the threshold to 1 we ensure that all nodes come
// online before progressing. This is only necessary until we add a way
// to recover poast blocks from other nodes.
super_majority_threshold: Some(threshold),
},
timeout,
cryptarchia: CryptarchiaSettings {
coins,
config,
genesis_state,
time,
transaction_selector_settings: (),
blob_selector_settings: (),
},
@ -393,58 +392,3 @@ fn create_node_config(
config
}
#[cfg(feature = "mixnet")]
fn create_mixnet_config(ids: &[[u8; 32]]) -> (MixClientConfig, Vec<MixNodeConfig>) {
let mixnode_configs: Vec<MixNodeConfig> = ids
.iter()
.map(|id| MixNodeConfig {
encryption_private_key: *id,
delay_rate_per_min: 100000000.0,
})
.collect();
// Build an empty topology because it will be constructed with meaningful node infos later
let topology = MixnetTopology::new(Vec::new(), 0, 0, [1u8; 32]).unwrap();
(
MixClientConfig {
topology,
emission_rate_per_min: 120.0,
redundancy: NonZeroU8::new(1).unwrap(),
},
mixnode_configs,
)
}
#[cfg(feature = "mixnet")]
fn build_mixnet_topology(mixnode_candidates: &[&Config]) -> MixnetTopology {
use mixnet::crypto::public_key_from;
let candidates = mixnode_candidates
.iter()
.map(|config| {
MixNodeInfo::new(
NodeAddress::from(SocketAddr::new(
IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)),
config.network.backend.inner.port,
)),
public_key_from(config.network.backend.mixnet.mixnode.encryption_private_key),
)
.unwrap()
})
.collect::<Vec<_>>();
let num_layers = candidates.len();
MixnetTopology::new(candidates, num_layers, 1, [1u8; 32]).unwrap()
}
fn node_address(config: &Config) -> Multiaddr {
Swarm::multiaddr(
std::net::Ipv4Addr::new(127, 0, 0, 1),
config.network.backend.inner.port,
)
}
pub enum Pool {
Da,
Cl,
}

View File

@ -7,7 +7,8 @@ use nomos_cli::{
use nomos_core::da::{blob::Blob as _, DaProtocol};
use std::{io::Write, time::Duration};
use tempfile::NamedTempFile;
use tests::{adjust_timeout, nodes::nomos::Pool, Node, NomosNode, SpawnConfig};
use tests::{adjust_timeout, nodes::nomos::Pool, SpawnConfig};
use tests::{Node, NomosNode};
const CLI_BIN: &str = "../target/debug/nomos-cli";

View File

@ -0,0 +1,57 @@
use futures::stream::{self, StreamExt};
use std::collections::HashSet;
use std::time::Duration;
use tests::{adjust_timeout, Node, NomosNode, SpawnConfig};
// how many times more than the expected time to produce a predefined number of blocks we wait before timing out
const TIMEOUT_MULTIPLIER: f64 = 3.0;
// how long we let the chain grow before checking the block at tip - k is the same in all chains
const CHAIN_LENGTH_MULTIPLIER: u32 = 2;
async fn happy_test(nodes: &[NomosNode]) {
let config = nodes[0].config();
let security_param = config.cryptarchia.config.consensus_config.security_param;
let n_blocks = security_param * CHAIN_LENGTH_MULTIPLIER;
println!("waiting for {n_blocks} blocks");
let timeout = (n_blocks as f64 / config.cryptarchia.config.consensus_config.active_slot_coeff
* config.cryptarchia.time.slot_duration.as_secs() as f64
* TIMEOUT_MULTIPLIER)
.floor() as u64;
let timeout = adjust_timeout(Duration::from_secs(timeout));
let timeout = tokio::time::sleep(timeout);
tokio::select! {
_ = timeout => panic!("timed out waiting for nodes to produce {} blocks", n_blocks),
_ = async { while stream::iter(nodes)
.any(|n| async move { (n.consensus_info().await.height as u32) < n_blocks })
.await
{
println!(
"waiting... {}",
stream::iter(nodes)
.then(|n| async move { format!("{}", n.consensus_info().await.height) })
.collect::<Vec<_>>()
.await
.join(" | ")
);
tokio::time::sleep(std::time::Duration::from_millis(100)).await;
}
} => {}
};
let last_committed_block_height = n_blocks - security_param;
println!("{:?}", nodes[0].consensus_info().await);
let infos = stream::iter(nodes)
.then(|n| async move { n.get_headers(None, None).await })
.map(|blocks| blocks[last_committed_block_height as usize])
.collect::<HashSet<_>>()
.await;
assert_eq!(infos.len(), 1, "consensus not reached");
}
#[tokio::test]
async fn two_nodes_happy() {
let nodes = NomosNode::spawn_nodes(SpawnConfig::star_happy(2)).await;
happy_test(&nodes).await;
}

View File

@ -1,97 +0,0 @@
use carnot_engine::{Qc, View};
use futures::stream::{self, StreamExt};
use std::collections::HashSet;
use std::time::Duration;
use tests::{adjust_timeout, Node, NomosNode, SpawnConfig};
const TARGET_VIEW: View = View::new(20);
#[derive(serde::Serialize)]
struct Info {
node_id: String,
block_id: String,
view: View,
}
async fn happy_test(nodes: &[NomosNode]) {
let timeout = adjust_timeout(Duration::from_secs(60));
let timeout = tokio::time::sleep(timeout);
tokio::select! {
_ = timeout => panic!("timed out waiting for nodes to reach view {}", TARGET_VIEW),
_ = async { while stream::iter(nodes)
.any(|n| async move { n.consensus_info().await.current_view < TARGET_VIEW })
.await
{
println!(
"waiting... {}",
stream::iter(nodes)
.then(|n| async move { format!("{}", n.consensus_info().await.current_view) })
.collect::<Vec<_>>()
.await
.join(" | ")
);
tokio::time::sleep(std::time::Duration::from_millis(100)).await;
}
} => {}
};
let infos = stream::iter(nodes)
.then(|n| async move { n.get_blocks_info(None, None).await })
.collect::<Vec<_>>()
.await;
// check that they have the same block
let blocks = infos
.iter()
.map(|i| i.iter().find(|b| b.view == TARGET_VIEW).unwrap())
.collect::<HashSet<_>>();
// try to see if we have invalid blocks
let invalid_blocks = infos
.iter()
.zip(nodes.iter())
.flat_map(|(blocks, node)| {
blocks.iter().filter_map(|b| match &b.parent_qc {
Qc::Standard(_) => None,
Qc::Aggregated(_) => Some(Info {
node_id: node.id().to_string(),
block_id: b.id.to_string(),
view: b.view,
}),
})
})
.collect::<Vec<_>>();
assert!(
invalid_blocks.is_empty(),
"{}",
serde_json::to_string_pretty(&invalid_blocks).unwrap()
);
assert_eq!(blocks.len(), 1);
}
#[tokio::test]
async fn two_nodes_happy() {
let nodes = NomosNode::spawn_nodes(SpawnConfig::chain_happy(2)).await;
happy_test(&nodes).await;
}
#[tokio::test]
async fn ten_nodes_happy() {
let nodes = NomosNode::spawn_nodes(SpawnConfig::chain_happy(10)).await;
happy_test(&nodes).await;
}
#[tokio::test]
async fn test_get_block() {
let nodes = NomosNode::spawn_nodes(SpawnConfig::chain_happy(2)).await;
happy_test(&nodes).await;
let id = nodes[0].consensus_info().await.last_committed_block.id;
tokio::time::timeout(Duration::from_secs(10), async {
while nodes[0].get_block(id).await.is_none() {
tokio::time::sleep(Duration::from_millis(100)).await;
println!("trying...");
}
})
.await
.unwrap();
}

View File

@ -1,126 +0,0 @@
use carnot_consensus::CarnotInfo;
use carnot_engine::{NodeId, View};
use fraction::Fraction;
use futures::stream::{self, StreamExt};
use nomos_core::header::HeaderId;
use std::{collections::HashSet, time::Duration};
use tests::{adjust_timeout, ConsensusConfig, Node, NomosNode, SpawnConfig};
const TARGET_VIEW: View = View::new(20);
const DUMMY_NODE_ID: NodeId = NodeId::new([0u8; 32]);
type Block = carnot_engine::Block<HeaderId>;
type TimeoutQc = carnot_engine::TimeoutQc<HeaderId>;
#[tokio::test]
async fn ten_nodes_one_down() {
let mut nodes = NomosNode::spawn_nodes(SpawnConfig::Chain {
consensus: ConsensusConfig {
n_participants: 10,
threshold: Fraction::new(9u32, 10u32),
timeout: std::time::Duration::from_secs(5),
},
})
.await;
let mut failed_node = nodes.pop().unwrap();
failed_node.stop();
let timeout = adjust_timeout(Duration::from_secs(120));
let timeout = tokio::time::sleep(timeout);
tokio::select! {
_ = timeout => panic!("timed out waiting for nodes to reach view {}", TARGET_VIEW),
_ = async { while stream::iter(&nodes)
.any(|n| async move { n.consensus_info().await.current_view < TARGET_VIEW })
.await
{
println!(
"waiting... {}",
stream::iter(&nodes)
.then(|n| async move { format!("{}", n.consensus_info().await.current_view) })
.collect::<Vec<_>>()
.await
.join(" | ")
);
tokio::time::sleep(std::time::Duration::from_millis(100)).await;
}
} => {}
};
let (infos, blocks): (Vec<_>, Vec<_>) = stream::iter(nodes)
.then(|n| async move {
(
n.consensus_info().await,
n.get_blocks_info(None, None).await,
)
})
.unzip()
.await;
let target_block = assert_block_consensus(&blocks, TARGET_VIEW);
// If no node has the target block, check that TARGET_VIEW was reached by timeout_qc.
if target_block.is_none() {
println!("No node has the block with {TARGET_VIEW:?}. Checking timeout_qcs...");
assert_timeout_qc_consensus(&infos, TARGET_VIEW.prev());
}
}
// Check if all nodes have the same block at the specific view.
fn assert_block_consensus<'a>(blocks: &[Vec<Block>], view: View) -> Option<Block> {
let blocks = blocks
.into_iter()
.map(|b| b.iter().find(|b| b.view == view))
.collect::<HashSet<_>>();
// Every nodes must have the same target block (Some(block))
// , or no node must have it (None).
assert_eq!(
blocks.len(),
1,
"multiple blocks found at {:?}: {:?}",
view,
blocks
);
blocks.iter().next().unwrap().cloned()
}
// Check if all nodes have the same timeout_qc at the specific view.
fn assert_timeout_qc_consensus<'a>(
consensus_infos: impl IntoIterator<Item = &'a CarnotInfo>,
view: View,
) -> TimeoutQc {
let timeout_qcs = consensus_infos
.into_iter()
.map(|i| {
i.last_view_timeout_qc.clone().map(|timeout_qc| {
// Masking the `sender` field because we want timeout_qcs from different
// senders to be considered the same if all other fields are the same.
TimeoutQc::new(
timeout_qc.view(),
timeout_qc.high_qc().clone(),
DUMMY_NODE_ID,
)
})
})
.collect::<HashSet<_>>();
assert_eq!(
timeout_qcs.len(),
1,
"multiple timeout_qcs found at {:?}: {:?}",
view,
timeout_qcs
);
let timeout_qc = timeout_qcs
.iter()
.next()
.unwrap()
.clone()
.expect("collected timeout_qc shouldn't be None");
// NOTE: This check could be failed if other timeout_qcs had occurred
// before `consensus_infos` were gathered.
// But it should be okay as long as the `timeout` is not too short.
assert_eq!(timeout_qc.view(), view);
timeout_qc
}