1
0
mirror of synced 2025-01-09 23:35:46 +00:00

Mix: time abstract from streams (#906)

* Extract temporal async stream

* Make stuff public

* Extract persistent transmission scheduler

* Fix tests

* Push temporal scheduler one layer up

* fix compile errors

* return Poll::Pending instead of Poll::Ready(None) when queue is empty

---------

Co-authored-by: Youngjoon Lee <5462944+youngjoon-lee@users.noreply.github.com>
This commit is contained in:
Daniel Sanchez 2024-11-06 06:08:52 +01:00 committed by GitHub
parent c84a29db31
commit 9b29c17e2f
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
7 changed files with 142 additions and 92 deletions

View File

@ -8,7 +8,7 @@ use rand::RngCore;
use std::marker::PhantomData; use std::marker::PhantomData;
use std::pin::Pin; use std::pin::Pin;
use std::task::{Context, Poll}; use std::task::{Context, Poll};
pub use temporal::TemporalProcessorSettings; pub use temporal::TemporalSchedulerSettings;
use crate::membership::Membership; use crate::membership::Membership;
use crate::message_blend::crypto::CryptographicProcessor; use crate::message_blend::crypto::CryptographicProcessor;
@ -28,13 +28,13 @@ where
M::PrivateKey: Serialize + DeserializeOwned, M::PrivateKey: Serialize + DeserializeOwned,
{ {
pub cryptographic_processor: CryptographicProcessorSettings<M::PrivateKey>, pub cryptographic_processor: CryptographicProcessorSettings<M::PrivateKey>,
pub temporal_processor: TemporalProcessorSettings, pub temporal_processor: TemporalSchedulerSettings,
} }
/// [`MessageBlendStream`] handles the entire mixing tiers process /// [`MessageBlendStream`] handles the entire mixing tiers process
/// - Unwraps incoming messages received from network using [`CryptographicProcessor`] /// - Unwraps incoming messages received from network using [`CryptographicProcessor`]
/// - Pushes unwrapped messages to [`TemporalProcessor`] /// - Pushes unwrapped messages to [`TemporalProcessor`]
pub struct MessageBlendStream<S, Rng, M> pub struct MessageBlendStream<S, Rng, M, Scheduler>
where where
M: MixMessage, M: MixMessage,
{ {
@ -43,22 +43,24 @@ where
temporal_sender: UnboundedSender<MixOutgoingMessage>, temporal_sender: UnboundedSender<MixOutgoingMessage>,
cryptographic_processor: CryptographicProcessor<Rng, M>, cryptographic_processor: CryptographicProcessor<Rng, M>,
_rng: PhantomData<Rng>, _rng: PhantomData<Rng>,
_scheduler: PhantomData<Scheduler>,
} }
impl<S, Rng, M> MessageBlendStream<S, Rng, M> impl<S, Rng, M, Scheduler> MessageBlendStream<S, Rng, M, Scheduler>
where where
S: Stream<Item = Vec<u8>>, S: Stream<Item = Vec<u8>>,
Rng: RngCore + Unpin + Send + 'static, Rng: RngCore + Unpin + Send + 'static,
M: MixMessage, M: MixMessage,
M::PrivateKey: Serialize + DeserializeOwned, M::PrivateKey: Serialize + DeserializeOwned,
M::PublicKey: Clone + PartialEq, M::PublicKey: Clone + PartialEq,
Scheduler: Stream<Item = ()> + Unpin + Send + 'static,
{ {
pub fn new( pub fn new(
input_stream: S, input_stream: S,
settings: MessageBlendSettings<M>, settings: MessageBlendSettings<M>,
membership: Membership<M>, membership: Membership<M>,
scheduler: Scheduler,
cryptographic_processor_rng: Rng, cryptographic_processor_rng: Rng,
temporal_processor_rng: Rng,
) -> Self { ) -> Self {
let cryptographic_processor = CryptographicProcessor::new( let cryptographic_processor = CryptographicProcessor::new(
settings.cryptographic_processor, settings.cryptographic_processor,
@ -67,7 +69,7 @@ where
); );
let (temporal_sender, temporal_receiver) = mpsc::unbounded_channel(); let (temporal_sender, temporal_receiver) = mpsc::unbounded_channel();
let output_stream = UnboundedReceiverStream::new(temporal_receiver) let output_stream = UnboundedReceiverStream::new(temporal_receiver)
.temporal_stream(settings.temporal_processor, temporal_processor_rng) .temporal_stream(scheduler)
.boxed(); .boxed();
Self { Self {
input_stream, input_stream,
@ -75,6 +77,7 @@ where
temporal_sender, temporal_sender,
cryptographic_processor, cryptographic_processor,
_rng: Default::default(), _rng: Default::default(),
_scheduler: Default::default(),
} }
} }
@ -100,13 +103,14 @@ where
} }
} }
impl<S, Rng, M> Stream for MessageBlendStream<S, Rng, M> impl<S, Rng, M, Scheduler> Stream for MessageBlendStream<S, Rng, M, Scheduler>
where where
S: Stream<Item = Vec<u8>> + Unpin, S: Stream<Item = Vec<u8>> + Unpin,
Rng: RngCore + Unpin + Send + 'static, Rng: RngCore + Unpin + Send + 'static,
M: MixMessage + Unpin, M: MixMessage + Unpin,
M::PrivateKey: Serialize + DeserializeOwned + Unpin, M::PrivateKey: Serialize + DeserializeOwned + Unpin,
M::PublicKey: Clone + PartialEq + Unpin, M::PublicKey: Clone + PartialEq + Unpin,
Scheduler: Stream<Item = ()> + Unpin + Send + 'static,
{ {
type Item = MixOutgoingMessage; type Item = MixOutgoingMessage;
@ -118,20 +122,21 @@ where
} }
} }
pub trait MessageBlendExt<Rng, M>: Stream<Item = Vec<u8>> pub trait MessageBlendExt<Rng, M, Scheduler>: Stream<Item = Vec<u8>>
where where
Rng: RngCore + Send + Unpin + 'static, Rng: RngCore + Send + Unpin + 'static,
M: MixMessage, M: MixMessage,
M::PrivateKey: Serialize + DeserializeOwned, M::PrivateKey: Serialize + DeserializeOwned,
M::PublicKey: Clone + PartialEq, M::PublicKey: Clone + PartialEq,
Scheduler: Stream<Item = ()> + Unpin + Send + 'static,
{ {
fn blend( fn blend(
self, self,
message_blend_settings: MessageBlendSettings<M>, message_blend_settings: MessageBlendSettings<M>,
membership: Membership<M>, membership: Membership<M>,
scheduler: Scheduler,
cryptographic_processor_rng: Rng, cryptographic_processor_rng: Rng,
temporal_processor_rng: Rng, ) -> MessageBlendStream<Self, Rng, M, Scheduler>
) -> MessageBlendStream<Self, Rng, M>
where where
Self: Sized + Unpin, Self: Sized + Unpin,
{ {
@ -139,18 +144,19 @@ where
self, self,
message_blend_settings, message_blend_settings,
membership, membership,
scheduler,
cryptographic_processor_rng, cryptographic_processor_rng,
temporal_processor_rng,
) )
} }
} }
impl<T, Rng, M> MessageBlendExt<Rng, M> for T impl<T, Rng, M, S> MessageBlendExt<Rng, M, S> for T
where where
T: Stream<Item = Vec<u8>>, T: Stream<Item = Vec<u8>>,
Rng: RngCore + Unpin + Send + 'static, Rng: RngCore + Unpin + Send + 'static,
M: MixMessage, M: MixMessage,
M::PrivateKey: Clone + Serialize + DeserializeOwned + PartialEq, M::PrivateKey: Clone + Serialize + DeserializeOwned + PartialEq,
M::PublicKey: Clone + Serialize + DeserializeOwned + PartialEq, M::PublicKey: Clone + Serialize + DeserializeOwned + PartialEq,
S: Stream<Item = ()> + Unpin + Send + 'static,
{ {
} }

View File

@ -10,14 +10,8 @@ use rand::{Rng, RngCore};
use serde::{Deserialize, Serialize}; use serde::{Deserialize, Serialize};
use tokio::time; use tokio::time;
/// [`TemporalProcessor`] delays messages randomly to hide timing correlation pub struct TemporalScheduler<Rng> {
/// between incoming and outgoing messages from a node. settings: TemporalSchedulerSettings,
///
/// See the [`Stream`] implementation below for more details on how it works.
pub(crate) struct TemporalProcessor<M, Rng> {
settings: TemporalProcessorSettings,
// All scheduled messages
queue: VecDeque<M>,
/// Interval in seconds for running the lottery to release a message /// Interval in seconds for running the lottery to release a message
lottery_interval: time::Interval, lottery_interval: time::Interval,
/// To wait a few seconds after running the lottery before releasing the message. /// To wait a few seconds after running the lottery before releasing the message.
@ -27,17 +21,11 @@ pub(crate) struct TemporalProcessor<M, Rng> {
rng: Rng, rng: Rng,
} }
#[derive(Clone, Copy, Debug, Serialize, Deserialize)] impl<Rng> TemporalScheduler<Rng> {
pub struct TemporalProcessorSettings { pub fn new(settings: TemporalSchedulerSettings, rng: Rng) -> Self {
pub max_delay_seconds: u64,
}
impl<M, Rng> TemporalProcessor<M, Rng> {
pub(crate) fn new(settings: TemporalProcessorSettings, rng: Rng) -> Self {
let lottery_interval = Self::lottery_interval(settings.max_delay_seconds); let lottery_interval = Self::lottery_interval(settings.max_delay_seconds);
Self { Self {
settings, settings,
queue: VecDeque::new(),
lottery_interval, lottery_interval,
release_timer: None, release_timer: None,
rng, rng,
@ -58,12 +46,9 @@ impl<M, Rng> TemporalProcessor<M, Rng> {
fn lottery_interval_seconds(max_delay_seconds: u64) -> u64 { fn lottery_interval_seconds(max_delay_seconds: u64) -> u64 {
max_delay_seconds / 2 max_delay_seconds / 2
} }
/// Schedule a message to be released later.
pub(crate) fn push_message(&mut self, message: M) {
self.queue.push_back(message);
}
} }
impl<M, Rng> TemporalProcessor<M, Rng>
impl<Rng> TemporalScheduler<Rng>
where where
Rng: RngCore, Rng: RngCore,
{ {
@ -75,12 +60,11 @@ where
} }
} }
impl<M, Rng> Stream for TemporalProcessor<M, Rng> impl<Rng> Stream for TemporalScheduler<Rng>
where where
M: Unpin,
Rng: RngCore + Unpin, Rng: RngCore + Unpin,
{ {
type Item = M; type Item = ();
fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> { fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
// Check whether it's time to run a new lottery to determine the delay. // Check whether it's time to run a new lottery to determine the delay.
@ -94,33 +78,74 @@ where
if let Some(timer) = self.release_timer.as_mut() { if let Some(timer) = self.release_timer.as_mut() {
if timer.as_mut().poll(cx).is_ready() { if timer.as_mut().poll(cx).is_ready() {
self.release_timer.take(); // Reset timer after it's done self.release_timer.take(); // Reset timer after it's done
if let Some(msg) = self.queue.pop_front() { return Poll::Ready(Some(()));
// Release the 1st message in the queue if it exists.
return Poll::Ready(Some(msg));
} }
} }
}
Poll::Pending Poll::Pending
} }
} }
pub struct TemporalStream<S, Rng> /// [`TemporalProcessor`] delays messages randomly to hide timing correlation
where /// between incoming and outgoing messages from a node.
S: Stream, ///
Rng: RngCore, /// See the [`Stream`] implementation below for more details on how it works.
{ pub struct TemporalProcessor<M, S> {
processor: TemporalProcessor<S::Item, Rng>, // All scheduled messages
wrapped_stream: S, queue: VecDeque<M>,
scheduler: S,
} }
impl<S, Rng> Stream for TemporalStream<S, Rng> #[derive(Clone, Copy, Debug, Serialize, Deserialize)]
pub struct TemporalSchedulerSettings {
pub max_delay_seconds: u64,
}
impl<M, S> TemporalProcessor<M, S> {
pub(crate) fn new(scheduler: S) -> Self {
Self {
queue: VecDeque::new(),
scheduler,
}
}
/// Schedule a message to be released later.
pub(crate) fn push_message(&mut self, message: M) {
self.queue.push_back(message);
}
}
impl<M, S> Stream for TemporalProcessor<M, S>
where where
S: Stream + Unpin, M: Unpin,
S::Item: Unpin, S: Stream<Item = ()> + Unpin,
Rng: RngCore + Unpin,
{ {
type Item = S::Item; type Item = M;
fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
if self.scheduler.poll_next_unpin(cx).is_ready() {
if let Some(msg) = self.queue.pop_front() {
return Poll::Ready(Some(msg));
}
};
Poll::Pending
}
}
pub struct TemporalStream<WrappedStream, Scheduler>
where
WrappedStream: Stream,
Scheduler: Stream<Item = ()>,
{
processor: TemporalProcessor<WrappedStream::Item, Scheduler>,
wrapped_stream: WrappedStream,
}
impl<WrappedStream, Scheduler> Stream for TemporalStream<WrappedStream, Scheduler>
where
WrappedStream: Stream + Unpin,
WrappedStream::Item: Unpin,
Scheduler: Stream<Item = ()> + Unpin,
{
type Item = WrappedStream::Item;
fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> { fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
if let Poll::Ready(Some(item)) = self.wrapped_stream.poll_next_unpin(cx) { if let Poll::Ready(Some(item)) = self.wrapped_stream.poll_next_unpin(cx) {
@ -129,28 +154,24 @@ where
self.processor.poll_next_unpin(cx) self.processor.poll_next_unpin(cx)
} }
} }
pub trait TemporalProcessorExt<Rng>: Stream pub trait TemporalProcessorExt<Scheduler>: Stream
where where
Rng: RngCore, Scheduler: Stream<Item = ()>,
{ {
fn temporal_stream( fn temporal_stream(self, scheduler: Scheduler) -> TemporalStream<Self, Scheduler>
self,
settings: TemporalProcessorSettings,
rng: Rng,
) -> TemporalStream<Self, Rng>
where where
Self: Sized, Self: Sized,
{ {
TemporalStream { TemporalStream {
processor: TemporalProcessor::new(settings, rng), processor: TemporalProcessor::<Self::Item, Scheduler>::new(scheduler),
wrapped_stream: self, wrapped_stream: self,
} }
} }
} }
impl<T, Rng> TemporalProcessorExt<Rng> for T impl<T, S> TemporalProcessorExt<S> for T
where where
T: Stream, T: Stream,
Rng: RngCore, S: Stream<Item = ()>,
{ {
} }

View File

@ -1,4 +1,4 @@
use futures::Stream; use futures::{Stream, StreamExt};
use nomos_mix_message::MixMessage; use nomos_mix_message::MixMessage;
use rand::{distributions::Uniform, prelude::Distribution, Rng, RngCore}; use rand::{distributions::Uniform, prelude::Distribution, Rng, RngCore};
use serde::de::DeserializeOwned; use serde::de::DeserializeOwned;
@ -6,9 +6,6 @@ use serde::{Deserialize, Serialize};
use std::marker::PhantomData; use std::marker::PhantomData;
use std::pin::{pin, Pin}; use std::pin::{pin, Pin};
use std::task::{Context, Poll}; use std::task::{Context, Poll};
use std::time::Duration;
use tokio::time;
use tokio::time::Interval;
#[derive(Clone, Copy, Debug, Serialize, Deserialize)] #[derive(Clone, Copy, Debug, Serialize, Deserialize)]
pub struct PersistentTransmissionSettings { pub struct PersistentTransmissionSettings {
@ -28,57 +25,57 @@ impl Default for PersistentTransmissionSettings {
} }
/// Transmit scheduled messages with a persistent rate as a stream. /// Transmit scheduled messages with a persistent rate as a stream.
pub struct PersistentTransmissionStream<S, Rng, M> pub struct PersistentTransmissionStream<S, Rng, M, Scheduler>
where where
S: Stream, S: Stream,
Rng: RngCore, Rng: RngCore,
{ {
interval: Interval,
coin: Coin<Rng>, coin: Coin<Rng>,
stream: S, stream: S,
scheduler: Scheduler,
_mix_message: PhantomData<M>, _mix_message: PhantomData<M>,
} }
impl<S, Rng, M> PersistentTransmissionStream<S, Rng, M> impl<S, Rng, M, Scheduler> PersistentTransmissionStream<S, Rng, M, Scheduler>
where where
S: Stream, S: Stream,
Rng: RngCore, Rng: RngCore,
M: MixMessage, M: MixMessage,
Scheduler: Stream<Item = ()>,
{ {
pub fn new( pub fn new(
settings: PersistentTransmissionSettings, settings: PersistentTransmissionSettings,
stream: S, stream: S,
scheduler: Scheduler,
rng: Rng, rng: Rng,
) -> PersistentTransmissionStream<S, Rng, M> { ) -> PersistentTransmissionStream<S, Rng, M, Scheduler> {
let interval = time::interval(Duration::from_secs_f64(
1.0 / settings.max_emission_frequency,
));
let coin = Coin::<Rng>::new(rng, settings.drop_message_probability).unwrap(); let coin = Coin::<Rng>::new(rng, settings.drop_message_probability).unwrap();
Self { Self {
interval,
coin, coin,
stream, stream,
scheduler,
_mix_message: Default::default(), _mix_message: Default::default(),
} }
} }
} }
impl<S, Rng, M> Stream for PersistentTransmissionStream<S, Rng, M> impl<S, Rng, M, Scheduler> Stream for PersistentTransmissionStream<S, Rng, M, Scheduler>
where where
S: Stream<Item = Vec<u8>> + Unpin, S: Stream<Item = Vec<u8>> + Unpin,
Rng: RngCore + Unpin, Rng: RngCore + Unpin,
M: MixMessage + Unpin, M: MixMessage + Unpin,
Scheduler: Stream<Item = ()> + Unpin,
{ {
type Item = Vec<u8>; type Item = Vec<u8>;
fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> { fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
let Self { let Self {
ref mut interval, ref mut scheduler,
ref mut stream, ref mut stream,
ref mut coin, ref mut coin,
.. ..
} = self.get_mut(); } = self.get_mut();
if pin!(interval).poll_tick(cx).is_pending() { if pin!(scheduler).poll_next_unpin(cx).is_pending() {
return Poll::Pending; return Poll::Pending;
} }
if let Poll::Ready(Some(item)) = pin!(stream).poll_next(cx) { if let Poll::Ready(Some(item)) = pin!(stream).poll_next(cx) {
@ -91,29 +88,32 @@ where
} }
} }
pub trait PersistentTransmissionExt<Rng, M>: Stream pub trait PersistentTransmissionExt<Rng, M, Scheduler>: Stream
where where
Rng: RngCore, Rng: RngCore,
M: MixMessage, M: MixMessage,
Scheduler: Stream<Item = ()>,
{ {
fn persistent_transmission( fn persistent_transmission(
self, self,
settings: PersistentTransmissionSettings, settings: PersistentTransmissionSettings,
rng: Rng, rng: Rng,
) -> PersistentTransmissionStream<Self, Rng, M> scheduler: Scheduler,
) -> PersistentTransmissionStream<Self, Rng, M, Scheduler>
where where
Self: Sized + Unpin, Self: Sized + Unpin,
{ {
PersistentTransmissionStream::new(settings, self, rng) PersistentTransmissionStream::new(settings, self, scheduler, rng)
} }
} }
impl<S, Rng, M> PersistentTransmissionExt<Rng, M> for S impl<S, Rng, M, Scheduler> PersistentTransmissionExt<Rng, M, Scheduler> for S
where where
S: Stream, S: Stream,
Rng: RngCore, Rng: RngCore,
M: MixMessage, M: MixMessage,
M::PublicKey: Clone + Serialize + DeserializeOwned, M::PublicKey: Clone + Serialize + DeserializeOwned,
Scheduler: Stream<Item = ()>,
{ {
} }
@ -153,7 +153,10 @@ mod tests {
use nomos_mix_message::mock::MockMixMessage; use nomos_mix_message::mock::MockMixMessage;
use rand::SeedableRng; use rand::SeedableRng;
use rand_chacha::ChaCha8Rng; use rand_chacha::ChaCha8Rng;
use std::time::Duration;
use tokio::sync::mpsc; use tokio::sync::mpsc;
use tokio::time;
use tokio_stream::wrappers::IntervalStream;
macro_rules! assert_interval { macro_rules! assert_interval {
($last_time:expr, $lower_bound:expr, $upper_bound:expr) => { ($last_time:expr, $lower_bound:expr, $upper_bound:expr) => {
@ -193,8 +196,16 @@ mod tests {
let lower_bound = expected_emission_interval - torelance; let lower_bound = expected_emission_interval - torelance;
let upper_bound = expected_emission_interval + torelance; let upper_bound = expected_emission_interval + torelance;
// prepare stream // prepare stream
let mut persistent_transmission_stream: PersistentTransmissionStream<_, _, MockMixMessage> = let mut persistent_transmission_stream: PersistentTransmissionStream<
stream.persistent_transmission(settings, ChaCha8Rng::from_entropy()); _,
_,
MockMixMessage,
_,
> = stream.persistent_transmission(
settings,
ChaCha8Rng::from_entropy(),
IntervalStream::new(time::interval(expected_emission_interval)).map(|_| ()),
);
// Messages must be scheduled in non-blocking manner. // Messages must be scheduled in non-blocking manner.
schedule_sender.send(vec![1]).unwrap(); schedule_sender.send(vec![1]).unwrap();
schedule_sender.send(vec![2]).unwrap(); schedule_sender.send(vec![2]).unwrap();

View File

@ -3,7 +3,7 @@ use cryptarchia_consensus::LeaderConfig;
use nomos_da_network_service::backends::libp2p::common::DaNetworkBackendSettings; use nomos_da_network_service::backends::libp2p::common::DaNetworkBackendSettings;
use nomos_mix::membership::Node; use nomos_mix::membership::Node;
use nomos_mix::message_blend::{ use nomos_mix::message_blend::{
CryptographicProcessorSettings, MessageBlendSettings, TemporalProcessorSettings, CryptographicProcessorSettings, MessageBlendSettings, TemporalSchedulerSettings,
}; };
use nomos_mix_message::mock::MockMixMessage; use nomos_mix_message::mock::MockMixMessage;
use nomos_mix_message::MixMessage; use nomos_mix_message::MixMessage;
@ -226,7 +226,7 @@ pub fn new_node(
private_key: mix_config.private_key.to_bytes(), private_key: mix_config.private_key.to_bytes(),
num_mix_layers: 1, num_mix_layers: 1,
}, },
temporal_processor: TemporalProcessorSettings { temporal_processor: TemporalSchedulerSettings {
max_delay_seconds: 2, max_delay_seconds: 2,
}, },
}, },

View File

@ -8,6 +8,7 @@ use network::NetworkAdapter;
use nomos_core::wire; use nomos_core::wire;
use nomos_mix::membership::{Membership, Node}; use nomos_mix::membership::{Membership, Node};
use nomos_mix::message_blend::crypto::CryptographicProcessor; use nomos_mix::message_blend::crypto::CryptographicProcessor;
use nomos_mix::message_blend::temporal::TemporalScheduler;
use nomos_mix::message_blend::{MessageBlendExt, MessageBlendSettings}; use nomos_mix::message_blend::{MessageBlendExt, MessageBlendSettings};
use nomos_mix::persistent_transmission::{ use nomos_mix::persistent_transmission::{
PersistentTransmissionExt, PersistentTransmissionSettings, PersistentTransmissionStream, PersistentTransmissionExt, PersistentTransmissionSettings, PersistentTransmissionStream,
@ -26,8 +27,10 @@ use rand::SeedableRng;
use rand_chacha::ChaCha12Rng; use rand_chacha::ChaCha12Rng;
use serde::{de::DeserializeOwned, Deserialize, Serialize}; use serde::{de::DeserializeOwned, Deserialize, Serialize};
use std::fmt::Debug; use std::fmt::Debug;
use std::time::Duration;
use tokio::sync::mpsc; use tokio::sync::mpsc;
use tokio_stream::wrappers::UnboundedReceiverStream; use tokio::time;
use tokio_stream::wrappers::{IntervalStream, UnboundedReceiverStream};
/// A mix service that sends messages to the mix network /// A mix service that sends messages to the mix network
/// and broadcasts fully unwrapped messages through the [`NetworkService`]. /// and broadcasts fully unwrapped messages through the [`NetworkService`].
@ -108,16 +111,25 @@ where
_, _,
_, _,
MockMixMessage, MockMixMessage,
_,
> = UnboundedReceiverStream::new(persistent_receiver).persistent_transmission( > = UnboundedReceiverStream::new(persistent_receiver).persistent_transmission(
mix_config.persistent_transmission, mix_config.persistent_transmission,
ChaCha12Rng::from_entropy(), ChaCha12Rng::from_entropy(),
IntervalStream::new(time::interval(Duration::from_secs_f64(
1.0 / mix_config.persistent_transmission.max_emission_frequency,
)))
.map(|_| ()),
); );
// tier 2 blend // tier 2 blend
let temporal_scheduler = TemporalScheduler::new(
mix_config.message_blend.temporal_processor,
ChaCha12Rng::from_entropy(),
);
let mut blend_messages = backend.listen_to_incoming_messages().blend( let mut blend_messages = backend.listen_to_incoming_messages().blend(
mix_config.message_blend, mix_config.message_blend,
membership.clone(), membership.clone(),
ChaCha12Rng::from_entropy(), temporal_scheduler,
ChaCha12Rng::from_entropy(), ChaCha12Rng::from_entropy(),
); );

View File

@ -23,7 +23,7 @@ use nomos_da_verifier::DaVerifierServiceSettings;
use nomos_executor::api::backend::AxumBackendSettings; use nomos_executor::api::backend::AxumBackendSettings;
use nomos_executor::config::Config; use nomos_executor::config::Config;
use nomos_mix::message_blend::{ use nomos_mix::message_blend::{
CryptographicProcessorSettings, MessageBlendSettings, TemporalProcessorSettings, CryptographicProcessorSettings, MessageBlendSettings, TemporalSchedulerSettings,
}; };
use nomos_network::{backends::libp2p::Libp2pConfig, NetworkConfig}; use nomos_network::{backends::libp2p::Libp2pConfig, NetworkConfig};
use nomos_node::api::paths::{CL_METRICS, DA_GET_RANGE}; use nomos_node::api::paths::{CL_METRICS, DA_GET_RANGE};
@ -162,7 +162,7 @@ pub fn create_executor_config(config: GeneralConfig) -> Config {
private_key: config.mix_config.private_key.to_bytes(), private_key: config.mix_config.private_key.to_bytes(),
num_mix_layers: 1, num_mix_layers: 1,
}, },
temporal_processor: TemporalProcessorSettings { temporal_processor: TemporalSchedulerSettings {
max_delay_seconds: 2, max_delay_seconds: 2,
}, },
}, },

View File

@ -15,7 +15,7 @@ use nomos_da_verifier::storage::adapters::rocksdb::RocksAdapterSettings as Verif
use nomos_da_verifier::{backend::kzgrs::KzgrsDaVerifierSettings, DaVerifierServiceSettings}; use nomos_da_verifier::{backend::kzgrs::KzgrsDaVerifierSettings, DaVerifierServiceSettings};
use nomos_mempool::MempoolMetrics; use nomos_mempool::MempoolMetrics;
use nomos_mix::message_blend::{ use nomos_mix::message_blend::{
CryptographicProcessorSettings, MessageBlendSettings, TemporalProcessorSettings, CryptographicProcessorSettings, MessageBlendSettings, TemporalSchedulerSettings,
}; };
use nomos_network::{backends::libp2p::Libp2pConfig, NetworkConfig}; use nomos_network::{backends::libp2p::Libp2pConfig, NetworkConfig};
use nomos_node::api::paths::{ use nomos_node::api::paths::{
@ -248,7 +248,7 @@ pub fn create_validator_config(config: GeneralConfig) -> Config {
private_key: config.mix_config.private_key.to_bytes(), private_key: config.mix_config.private_key.to_bytes(),
num_mix_layers: 1, num_mix_layers: 1,
}, },
temporal_processor: TemporalProcessorSettings { temporal_processor: TemporalSchedulerSettings {
max_delay_seconds: 2, max_delay_seconds: 2,
}, },
}, },