From 8d51c66af69eed9785cb98ed1c973afe66d6a6fe Mon Sep 17 00:00:00 2001 From: Yashash H L Date: Tue, 11 Feb 2025 22:01:28 +0530 Subject: [PATCH] feat: Idle Watermark Implementation Inside Async Data Movement for Source and ISB (#2385) Signed-off-by: Yashash H L Signed-off-by: Vigith Maurice Co-authored-by: Vigith Maurice --- pkg/shared/idlehandler/source_idlehandler.go | 2 +- pkg/sources/forward/data_forward.go | 2 +- rust/Cargo.lock | 79 +++ rust/extns/numaflow-pulsar/src/source.rs | 7 +- rust/extns/numaflow-sqs/src/source.rs | 62 +- rust/numaflow-core/Cargo.toml | 1 + rust/numaflow-core/src/config/pipeline.rs | 65 +- rust/numaflow-core/src/config/pipeline/isb.rs | 10 + .../src/config/pipeline/watermark.rs | 23 + rust/numaflow-core/src/mapper/map.rs | 9 + .../src/mapper/map/user_defined.rs | 6 + rust/numaflow-core/src/message.rs | 49 +- rust/numaflow-core/src/metrics.rs | 39 +- rust/numaflow-core/src/pipeline.rs | 37 +- .../pipeline/forwarder/source_forwarder.rs | 2 +- .../src/pipeline/isb/jetstream/reader.rs | 46 +- .../src/pipeline/isb/jetstream/writer.rs | 243 ++++--- .../src/shared/create_components.rs | 14 +- rust/numaflow-core/src/sink.rs | 11 +- rust/numaflow-core/src/sink/blackhole.rs | 2 + rust/numaflow-core/src/sink/log.rs | 2 + rust/numaflow-core/src/sink/user_defined.rs | 2 + rust/numaflow-core/src/source.rs | 59 +- rust/numaflow-core/src/source/generator.rs | 6 +- rust/numaflow-core/src/source/pulsar.rs | 10 +- rust/numaflow-core/src/source/serving.rs | 7 +- rust/numaflow-core/src/source/sqs.rs | 28 +- rust/numaflow-core/src/source/user_defined.rs | 24 +- rust/numaflow-core/src/tracker.rs | 8 +- rust/numaflow-core/src/transformer.rs | 3 + .../src/transformer/user_defined.rs | 3 + rust/numaflow-core/src/watermark.rs | 3 + rust/numaflow-core/src/watermark/idle.rs | 18 + rust/numaflow-core/src/watermark/idle/isb.rs | 347 ++++++++++ .../src/watermark/idle/source.rs | 148 ++++ rust/numaflow-core/src/watermark/isb.rs | 302 +++++++- .../src/watermark/isb/wm_fetcher.rs | 622 +++++++++++++++-- .../src/watermark/isb/wm_publisher.rs | 160 +++-- .../src/watermark/processor/manager.rs | 93 ++- .../src/watermark/processor/timeline.rs | 11 +- rust/numaflow-core/src/watermark/source.rs | 653 +++++++++++++++++- .../src/watermark/source/source_wm_fetcher.rs | 11 +- .../watermark/source/source_wm_publisher.rs | 250 ++++++- rust/serving/src/config.rs | 14 +- 44 files changed, 3024 insertions(+), 469 deletions(-) create mode 100644 rust/numaflow-core/src/watermark/idle.rs create mode 100644 rust/numaflow-core/src/watermark/idle/isb.rs create mode 100644 rust/numaflow-core/src/watermark/idle/source.rs diff --git a/pkg/shared/idlehandler/source_idlehandler.go b/pkg/shared/idlehandler/source_idlehandler.go index bbe60dfe6d..6556c3c394 100644 --- a/pkg/shared/idlehandler/source_idlehandler.go +++ b/pkg/shared/idlehandler/source_idlehandler.go @@ -105,7 +105,7 @@ func (iw *SourceIdleHandler) PublishSourceIdleWatermark(partitions []int32) { // in this case, we can publish the idle watermark as the last published idle watermark + the increment by value. nextIdleWM = iw.lastPublishedIdleWm.Add(iw.config.IdleSource.GetIncrementBy()) } else { - // if its not -1, then we can publish the idle watermark as the computed watermark + the increment by value. + // if it's not -1, then we can publish the idle watermark as the computed watermark + the increment by value. nextIdleWM = computedWm.Add(iw.config.IdleSource.GetIncrementBy()) } diff --git a/pkg/sources/forward/data_forward.go b/pkg/sources/forward/data_forward.go index facbff8e2b..7da6195339 100644 --- a/pkg/sources/forward/data_forward.go +++ b/pkg/sources/forward/data_forward.go @@ -250,7 +250,7 @@ func (df *DataForward) forwardAChunk(ctx context.Context) error { for toVertexName, toVertexBuffers := range df.toBuffers { for index := range toVertexBuffers { // publish idle watermark to all the source partitions owned by this reader. - // it is 1:1 for many (HTTP, tickgen, etc.) but for e.g., for Kafka it is 1:N and the list of partitions in the N could keep changing. + // it is 1:1 for many (HTTP, tick-gen, etc.) but for e.g., for Kafka it is 1:N and the list of partitions in the N could keep changing. for _, sp := range df.reader.Partitions(df.ctx) { if vertexPublishers, ok := df.toVertexWMPublishers[toVertexName]; ok { var publisher, ok = vertexPublishers[sp] diff --git a/rust/Cargo.lock b/rust/Cargo.lock index ccdd13a3c7..229dba817e 100644 --- a/rust/Cargo.lock +++ b/rust/Cargo.lock @@ -1028,6 +1028,41 @@ dependencies = [ "syn 2.0.90", ] +[[package]] +name = "darling" +version = "0.20.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6f63b86c8a8826a49b8c21f08a2d07338eec8d900540f8630dc76284be802989" +dependencies = [ + "darling_core", + "darling_macro", +] + +[[package]] +name = "darling_core" +version = "0.20.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "95133861a8032aaea082871032f5815eb9e98cef03fa916ab4500513994df9e5" +dependencies = [ + "fnv", + "ident_case", + "proc-macro2", + "quote", + "strsim", + "syn 2.0.90", +] + +[[package]] +name = "darling_macro" +version = "0.20.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d336a2a514f6ccccaa3e09b02d41d35330c07ddf03a62165fcec10bb561c7806" +dependencies = [ + "darling_core", + "quote", + "syn 2.0.90", +] + [[package]] name = "data-encoding" version = "2.6.0" @@ -1055,6 +1090,37 @@ dependencies = [ "serde", ] +[[package]] +name = "derive_builder" +version = "0.20.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "507dfb09ea8b7fa618fcf76e953f4f5e192547945816d5358edffe39f6f94947" +dependencies = [ + "derive_builder_macro", +] + +[[package]] +name = "derive_builder_core" +version = "0.20.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2d5bcf7b024d6835cfb3d473887cd966994907effbe9227e8c8219824d06c4e8" +dependencies = [ + "darling", + "proc-macro2", + "quote", + "syn 2.0.90", +] + +[[package]] +name = "derive_builder_macro" +version = "0.20.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ab63b0e2bf4d5928aff72e83a7dace85d7bba5fe12dcc3c5a572d78caffd3f3c" +dependencies = [ + "derive_builder_core", + "syn 2.0.90", +] + [[package]] name = "diff" version = "0.1.13" @@ -1791,6 +1857,12 @@ dependencies = [ "syn 2.0.90", ] +[[package]] +name = "ident_case" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b9e0384b61958566e926dc50660321d12159025e767c18e043daf26b70104c39" + [[package]] name = "idna" version = "1.0.3" @@ -2285,6 +2357,7 @@ dependencies = [ "base64 0.22.1", "bytes", "chrono", + "derive_builder", "futures", "http 1.2.0", "hyper-util", @@ -3635,6 +3708,12 @@ version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a8f112729512f8e442d81f95a8a7ddf2b7c6b8a1a6f509a95864142b30cab2d3" +[[package]] +name = "strsim" +version = "0.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7da8b5736845d9f2fcb837ea5d9e2628564b3b043a70948a3f0b778838c5fb4f" + [[package]] name = "subtle" version = "2.6.1" diff --git a/rust/extns/numaflow-pulsar/src/source.rs b/rust/extns/numaflow-pulsar/src/source.rs index 587e4bb85f..508714b9f8 100644 --- a/rust/extns/numaflow-pulsar/src/source.rs +++ b/rust/extns/numaflow-pulsar/src/source.rs @@ -238,6 +238,7 @@ pub struct PulsarSource { /// timeout for each batch read request timeout: Duration, actor_tx: mpsc::Sender, + vertex_replica: u16, } impl PulsarSource { @@ -245,6 +246,7 @@ impl PulsarSource { config: PulsarSourceConfig, batch_size: usize, timeout: Duration, + vertex_replica: u16, ) -> Result { let (tx, rx) = mpsc::channel(10); ConsumerReaderActor::start(config, rx).await?; @@ -252,6 +254,7 @@ impl PulsarSource { actor_tx: tx, batch_size, timeout, + vertex_replica, }) } } @@ -292,7 +295,7 @@ impl PulsarSource { None } - pub fn partitions(&self) -> Vec { - unimplemented!() + pub fn partitions_vec(&self) -> Vec { + vec![self.vertex_replica] } } diff --git a/rust/extns/numaflow-sqs/src/source.rs b/rust/extns/numaflow-sqs/src/source.rs index f0f05a63ab..af9a20fda4 100644 --- a/rust/extns/numaflow-sqs/src/source.rs +++ b/rust/extns/numaflow-sqs/src/source.rs @@ -29,7 +29,7 @@ pub const SQS_DEFAULT_REGION: &str = "us-west-2"; /// Implements serde::Deserialize to support loading from configuration files. /// TODO: add support for all sqs configs and different ways to authenticate #[derive(serde::Deserialize, Clone, PartialEq)] -pub struct SqsSourceConfig { +pub struct SQSSourceConfig { pub region: String, pub queue_name: String, } @@ -40,9 +40,9 @@ pub struct SqsSourceConfig { /// - Ensure thread-safe access to the SQS client /// - Manage connection state and retries /// - Handle concurrent requests without locks -enum SqsActorMessage { +enum SQSActorMessage { Receive { - respond_to: oneshot::Sender>>, + respond_to: oneshot::Sender>>, count: i32, timeout_at: Instant, }, @@ -62,7 +62,7 @@ enum SqsActorMessage { /// - Includes full message metadata for debugging /// - Maintains original SQS attributes and headers #[derive(Debug)] -pub struct SqsMessage { +pub struct SQSMessage { pub key: String, pub payload: Bytes, pub offset: String, @@ -77,13 +77,13 @@ pub struct SqsMessage { /// - Single SQS client instance /// - Message channel for handling concurrent requests struct SqsActor { - handler_rx: mpsc::Receiver, + handler_rx: mpsc::Receiver, client: Client, queue_url: String, } impl SqsActor { - fn new(handler_rx: mpsc::Receiver, client: Client, queue_url: String) -> Self { + fn new(handler_rx: mpsc::Receiver, client: Client, queue_url: String) -> Self { Self { handler_rx, client, @@ -91,7 +91,7 @@ impl SqsActor { } } - async fn create_sqs_client(config: Option) -> Client { + async fn create_sqs_client(config: Option) -> Client { let region = match config { Some(config) => config.region.clone(), None => SQS_DEFAULT_REGION.to_string(), @@ -124,9 +124,9 @@ impl SqsActor { } } - async fn handle_message(&mut self, msg: SqsActorMessage) { + async fn handle_message(&mut self, msg: SQSActorMessage) { match msg { - SqsActorMessage::Receive { + SQSActorMessage::Receive { respond_to, count, timeout_at, @@ -136,7 +136,7 @@ impl SqsActor { .send(messages) .expect("failed to send response from SqsActorMessage::Receive"); } - SqsActorMessage::Delete { + SQSActorMessage::Delete { respond_to, offsets, } => { @@ -145,7 +145,7 @@ impl SqsActor { .send(status) .expect("failed to send response from SqsActorMessage::Delete"); } - SqsActorMessage::GetPending { respond_to } => { + SQSActorMessage::GetPending { respond_to } => { let status = self.get_pending_messages().await; respond_to .send(status) @@ -160,7 +160,7 @@ impl SqsActor { /// - Respects timeout for long polling /// - Processes message attributes and system metadata /// - Returns messages in a normalized format - async fn get_messages(&mut self, count: i32, timeout_at: Instant) -> Result> { + async fn get_messages(&mut self, count: i32, timeout_at: Instant) -> Result> { let remaining_time = timeout_at - Instant::now(); // default to one second if remaining time is less than one second @@ -234,7 +234,7 @@ impl SqsActor { }) .unwrap_or_default(); - SqsMessage { + SQSMessage { key, payload, offset, @@ -334,11 +334,11 @@ impl SqsActor { /// - Clean abstraction of SQS complexity /// - Efficient message processing #[derive(Clone)] -pub struct SqsSource { +pub struct SQSSource { batch_size: usize, /// timeout for each batch read request timeout: Duration, - actor_tx: mpsc::Sender, + actor_tx: mpsc::Sender, } /// Builder for creating an `SqsSource`. @@ -347,7 +347,7 @@ pub struct SqsSource { /// such as region, queue name, batch size, timeout, and an optional SQS client. #[derive(Clone)] pub struct SqsSourceBuilder { - config: SqsSourceConfig, + config: SQSSourceConfig, batch_size: usize, timeout: Duration, client: Option, @@ -355,7 +355,7 @@ pub struct SqsSourceBuilder { impl Default for SqsSourceBuilder { fn default() -> Self { - Self::new(SqsSourceConfig { + Self::new(SQSSourceConfig { region: SQS_DEFAULT_REGION.to_string(), queue_name: "".to_string(), }) @@ -363,7 +363,7 @@ impl Default for SqsSourceBuilder { } impl SqsSourceBuilder { - pub fn new(config: SqsSourceConfig) -> Self { + pub fn new(config: SQSSourceConfig) -> Self { Self { config, batch_size: 1, @@ -371,7 +371,7 @@ impl SqsSourceBuilder { client: None, } } - pub fn config(mut self, config: SqsSourceConfig) -> Self { + pub fn config(mut self, config: SQSSourceConfig) -> Self { self.config = config; self } @@ -400,7 +400,7 @@ impl SqsSourceBuilder { /// # Returns /// - `Ok(SqsSource)` if the source is successfully built. /// - `Err(Error)` if there is an error during the initialization process. - pub async fn build(self) -> Result { + pub async fn build(self) -> Result { let sqs_client = match self.client { Some(client) => client, None => SqsActor::create_sqs_client(Some(self.config.clone())).await, @@ -427,7 +427,7 @@ impl SqsSourceBuilder { actor.run().await; }); - Ok(SqsSource { + Ok(SQSSource { batch_size: self.batch_size, timeout: self.timeout, actor_tx: handler_tx, @@ -435,14 +435,14 @@ impl SqsSourceBuilder { } } -impl SqsSource { +impl SQSSource { // read messages from SQS, corresponding sqs sdk method is receive_message - pub async fn read_messages(&self) -> Result> { + pub async fn read_messages(&self) -> Result> { tracing::debug!("Reading messages from SQS"); let start = Instant::now(); let (tx, rx) = oneshot::channel(); - let msg = SqsActorMessage::Receive { + let msg = SQSActorMessage::Receive { respond_to: tx, count: self.batch_size as i32, timeout_at: start + self.timeout, @@ -464,7 +464,7 @@ impl SqsSource { pub async fn ack_offsets(&self, offsets: Vec) -> Result<()> { tracing::debug!(offsets = ?offsets, "Acknowledging offsets"); let (tx, rx) = oneshot::channel(); - let msg = SqsActorMessage::Delete { + let msg = SQSActorMessage::Delete { offsets, respond_to: tx, }; @@ -477,7 +477,7 @@ impl SqsSource { // with the attribute name ApproximateNumberOfMessages pub async fn pending_count(&self) -> Option { let (tx, rx) = oneshot::channel(); - let msg = SqsActorMessage::GetPending { respond_to: tx }; + let msg = SQSActorMessage::GetPending { respond_to: tx }; let _ = self.actor_tx.send(msg).await; let actor_result = rx.await.map_err(Error::ActorTaskTerminated); @@ -523,7 +523,7 @@ mod tests { let sqs_mock_client = Client::from_conf(get_test_config_with_interceptor(sqs_operation_mocks)); - let source = SqsSourceBuilder::new(SqsSourceConfig { + let source = SqsSourceBuilder::new(SQSSourceConfig { region: SQS_DEFAULT_REGION.to_string(), queue_name: "test-q".to_string(), }) @@ -563,7 +563,7 @@ mod tests { let sqs_mock_client = Client::from_conf(get_test_config_with_interceptor(sqs_operation_mocks)); - let source = SqsSourceBuilder::new(SqsSourceConfig { + let source = SqsSourceBuilder::new(SQSSourceConfig { region: SQS_DEFAULT_REGION.to_string(), queue_name: "test-q".to_string(), }) @@ -593,7 +593,7 @@ mod tests { let sqs_mock_client = Client::from_conf(get_test_config_with_interceptor(sqs_operation_mocks)); - let source = SqsSourceBuilder::new(SqsSourceConfig { + let source = SqsSourceBuilder::new(SQSSourceConfig { region: SQS_DEFAULT_REGION.to_string(), queue_name: "test-q".to_string(), }) @@ -618,7 +618,7 @@ mod tests { let sqs_mock_client = Client::from_conf(get_test_config_with_interceptor(sqs_operation_mocks)); - let source = SqsSourceBuilder::new(SqsSourceConfig { + let source = SqsSourceBuilder::new(SQSSourceConfig { region: SQS_DEFAULT_REGION.to_string(), queue_name: "test-q".to_string(), }) @@ -633,7 +633,7 @@ mod tests { #[tokio::test] #[should_panic(expected = "not implemented")] async fn test_partitions_unimplemented() { - let source = SqsSource { + let source = SQSSource { batch_size: 1, timeout: Duration::from_secs(0), actor_tx: mpsc::channel(1).0, diff --git a/rust/numaflow-core/Cargo.toml b/rust/numaflow-core/Cargo.toml index 46c544e84b..c431d528b7 100644 --- a/rust/numaflow-core/Cargo.toml +++ b/rust/numaflow-core/Cargo.toml @@ -49,6 +49,7 @@ futures = "0.3.30" pin-project = "1.1.5" rand = "0.8.5" async-nats = "0.38.0" +derive_builder = "0.20.2" [dev-dependencies] tempfile = "3.11.0" diff --git a/rust/numaflow-core/src/config/pipeline.rs b/rust/numaflow-core/src/config/pipeline.rs index 4ceb527642..d68b97eea6 100644 --- a/rust/numaflow-core/src/config/pipeline.rs +++ b/rust/numaflow-core/src/config/pipeline.rs @@ -4,7 +4,7 @@ use std::time::Duration; use base64::prelude::BASE64_STANDARD; use base64::Engine; -use numaflow_models::models::{ForwardConditions, Vertex}; +use numaflow_models::models::{ForwardConditions, Vertex, Watermark}; use serde_json::from_slice; use tracing::info; @@ -18,9 +18,9 @@ use crate::config::get_vertex_replica; use crate::config::pipeline::isb::{BufferReaderConfig, BufferWriterConfig, Stream}; use crate::config::pipeline::map::MapMode; use crate::config::pipeline::map::MapVtxConfig; -use crate::config::pipeline::watermark::SourceWatermarkConfig; use crate::config::pipeline::watermark::WatermarkConfig; use crate::config::pipeline::watermark::{BucketConfig, EdgeWatermarkConfig}; +use crate::config::pipeline::watermark::{IdleConfig, SourceWatermarkConfig}; use crate::error::Error; use crate::Result; @@ -52,7 +52,7 @@ pub(crate) struct PipelineConfig { pub(crate) js_client_config: isb::jetstream::ClientConfig, // TODO: make it enum, since we can have different ISB implementations pub(crate) from_vertex_config: Vec, pub(crate) to_vertex_config: Vec, - pub(crate) vertex_config: VertexType, + pub(crate) vertex_type_config: VertexType, pub(crate) metrics_config: MetricsConfig, pub(crate) watermark_config: Option, pub(crate) callback_config: Option, @@ -75,7 +75,7 @@ impl Default for PipelineConfig { js_client_config: isb::jetstream::ClientConfig::default(), from_vertex_config: vec![], to_vertex_config: vec![], - vertex_config: VertexType::Source(SourceVtxConfig { + vertex_type_config: VertexType::Source(SourceVtxConfig { source_config: Default::default(), transformer_config: None, }), @@ -195,14 +195,14 @@ impl std::fmt::Display for VertexType { #[derive(Debug, Clone, PartialEq)] pub(crate) struct FromVertexConfig { - pub(crate) name: String, + pub(crate) name: &'static str, pub(crate) reader_config: BufferReaderConfig, pub(crate) partitions: u16, } #[derive(Debug, Clone, PartialEq)] pub(crate) struct ToVertexConfig { - pub(crate) name: String, + pub(crate) name: &'static str, pub(crate) partitions: u16, pub(crate) writer_config: BufferWriterConfig, pub(crate) conditions: Option>, @@ -347,7 +347,7 @@ impl PipelineConfig { .collect(); from_vertex_config.push(FromVertexConfig { - name: edge.from.clone(), + name: Box::leak(edge.from.clone().into_boxed_str()), reader_config: BufferReaderConfig { streams, ..Default::default() @@ -373,7 +373,7 @@ impl PipelineConfig { let default_writer_config = BufferWriterConfig::default(); to_vertex_config.push(ToVertexConfig { - name: edge.to.clone(), + name: Box::leak(edge.to.clone().into_boxed_str()), partitions: partition_count, writer_config: BufferWriterConfig { streams, @@ -402,9 +402,11 @@ impl PipelineConfig { let watermark_config = if vertex_obj .spec .watermark + .clone() .map_or(true, |w| w.disabled.unwrap_or(true)) { Self::create_watermark_config( + vertex_obj.spec.watermark.clone(), &namespace, &pipeline_name, &vertex_name, @@ -454,7 +456,7 @@ impl PipelineConfig { js_client_config, from_vertex_config, to_vertex_config, - vertex_config: vertex, + vertex_type_config: vertex, metrics_config: MetricsConfig::with_lookback_window_in_secs(look_back_window), watermark_config, callback_config, @@ -462,6 +464,7 @@ impl PipelineConfig { } fn create_watermark_config( + watermark_spec: Option>, namespace: &str, pipeline_name: &str, vertex_name: &str, @@ -469,8 +472,23 @@ impl PipelineConfig { from_vertex_config: &[FromVertexConfig], to_vertex_config: &[ToVertexConfig], ) -> Option { + let max_delay = watermark_spec + .as_ref() + .and_then(|w| w.max_delay.map(|x| Duration::from(x).as_millis() as u64)) + .unwrap_or(0); + + let idle_config = watermark_spec + .as_ref() + .and_then(|w| w.idle_source.as_ref()) + .map(|idle| IdleConfig { + increment_by: idle.increment_by.map(Duration::from).unwrap_or_default(), + step_interval: idle.step_interval.map(Duration::from).unwrap_or_default(), + threshold: idle.threshold.map(Duration::from).unwrap_or_default(), + }); + match vertex { VertexType::Source(_) => Some(WatermarkConfig::Source(SourceWatermarkConfig { + max_delay: Duration::from_millis(max_delay), source_bucket_config: BucketConfig { vertex: Box::leak(vertex_name.to_string().into_boxed_str()), partitions: 1, // source will have only one partition @@ -489,7 +507,7 @@ impl PipelineConfig { to_vertex_bucket_config: to_vertex_config .iter() .map(|to| BucketConfig { - vertex: Box::leak(to.name.clone().into_boxed_str()), + vertex: to.name, partitions: to.partitions, ot_bucket: Box::leak( format!( @@ -507,13 +525,14 @@ impl PipelineConfig { ), }) .collect(), + idle_config, })), VertexType::Sink(_) | VertexType::Map(_) => { Some(WatermarkConfig::Edge(EdgeWatermarkConfig { from_vertex_config: from_vertex_config .iter() .map(|from| BucketConfig { - vertex: Box::leak(from.name.clone().into_boxed_str()), + vertex: from.name, partitions: from.partitions, ot_bucket: Box::leak( format!( @@ -534,7 +553,7 @@ impl PipelineConfig { to_vertex_config: to_vertex_config .iter() .map(|to| BucketConfig { - vertex: Box::leak(to.name.clone().into_boxed_str()), + vertex: to.name, partitions: to.partitions, ot_bucket: Box::leak( format!( @@ -580,7 +599,7 @@ mod tests { js_client_config: isb::jetstream::ClientConfig::default(), from_vertex_config: vec![], to_vertex_config: vec![], - vertex_config: VertexType::Source(SourceVtxConfig { + vertex_type_config: VertexType::Source(SourceVtxConfig { source_config: Default::default(), transformer_config: None, }), @@ -631,7 +650,7 @@ mod tests { password: None, }, from_vertex_config: vec![FromVertexConfig { - name: "in".to_string(), + name: "in", reader_config: BufferReaderConfig { streams: vec![Stream::new("default-simple-pipeline-out-0", "out", 0)], wip_ack_interval: Duration::from_secs(1), @@ -639,7 +658,7 @@ mod tests { partitions: 1, }], to_vertex_config: vec![], - vertex_config: VertexType::Sink(SinkVtxConfig { + vertex_type_config: VertexType::Sink(SinkVtxConfig { sink_config: SinkConfig { sink_type: SinkType::Blackhole(BlackholeConfig {}), retry_config: None, @@ -688,7 +707,7 @@ mod tests { }, from_vertex_config: vec![], to_vertex_config: vec![ToVertexConfig { - name: "out".to_string(), + name: "out", partitions: 1, writer_config: BufferWriterConfig { streams: vec![Stream::new("default-simple-pipeline-out-0", "out", 0)], @@ -698,7 +717,7 @@ mod tests { }, conditions: None, }], - vertex_config: VertexType::Source(SourceVtxConfig { + vertex_type_config: VertexType::Source(SourceVtxConfig { source_config: SourceConfig { read_ahead: false, source_type: SourceType::Generator(GeneratorConfig { @@ -715,6 +734,7 @@ mod tests { }), metrics_config: Default::default(), watermark_config: Some(WatermarkConfig::Source(SourceWatermarkConfig { + max_delay: Default::default(), source_bucket_config: BucketConfig { vertex: "in", partitions: 1, @@ -727,6 +747,7 @@ mod tests { ot_bucket: "default-simple-pipeline-in-out_OT", hb_bucket: "default-simple-pipeline-in-out_PROCESSORS", }], + idle_config: None, })), ..Default::default() }; @@ -756,7 +777,7 @@ mod tests { }, from_vertex_config: vec![], to_vertex_config: vec![ToVertexConfig { - name: "out".to_string(), + name: "out", partitions: 1, writer_config: BufferWriterConfig { streams: vec![Stream::new("default-simple-pipeline-out-0", "out", 0)], @@ -766,7 +787,7 @@ mod tests { }, conditions: None, }], - vertex_config: VertexType::Source(SourceVtxConfig { + vertex_type_config: VertexType::Source(SourceVtxConfig { source_config: SourceConfig { read_ahead: false, source_type: SourceType::Pulsar(PulsarSourceConfig { @@ -782,6 +803,7 @@ mod tests { }), metrics_config: Default::default(), watermark_config: Some(WatermarkConfig::Source(SourceWatermarkConfig { + max_delay: Default::default(), source_bucket_config: BucketConfig { vertex: "in", partitions: 1, @@ -794,6 +816,7 @@ mod tests { ot_bucket: "default-simple-pipeline-in-out_OT", hb_bucket: "default-simple-pipeline-in-out_PROCESSORS", }], + idle_config: None, })), ..Default::default() }; @@ -893,7 +916,7 @@ mod tests { password: None, }, from_vertex_config: vec![FromVertexConfig { - name: "in".to_string(), + name: "in", reader_config: BufferReaderConfig { streams: vec![Stream::new("default-simple-pipeline-map-0", "map", 0)], wip_ack_interval: Duration::from_secs(1), @@ -901,7 +924,7 @@ mod tests { partitions: 1, }], to_vertex_config: vec![], - vertex_config: VertexType::Map(MapVtxConfig { + vertex_type_config: VertexType::Map(MapVtxConfig { concurrency: 500, map_type: MapType::UserDefined(UserDefinedConfig { grpc_max_message_size: DEFAULT_GRPC_MAX_MESSAGE_SIZE, diff --git a/rust/numaflow-core/src/config/pipeline/isb.rs b/rust/numaflow-core/src/config/pipeline/isb.rs index f258a6d5b3..a1d28514f2 100644 --- a/rust/numaflow-core/src/config/pipeline/isb.rs +++ b/rust/numaflow-core/src/config/pipeline/isb.rs @@ -37,6 +37,16 @@ pub(crate) struct Stream { pub(crate) partition: u16, } +impl Default for Stream { + fn default() -> Self { + Stream { + name: "", + vertex: "", + partition: DEFAULT_PARTITION_IDX, + } + } +} + impl Stream { pub(crate) fn new(name: &'static str, vertex: &'static str, partition: u16) -> Self { Stream { diff --git a/rust/numaflow-core/src/config/pipeline/watermark.rs b/rust/numaflow-core/src/config/pipeline/watermark.rs index dbfb7cf109..b560cd6c04 100644 --- a/rust/numaflow-core/src/config/pipeline/watermark.rs +++ b/rust/numaflow-core/src/config/pipeline/watermark.rs @@ -1,3 +1,5 @@ +use std::time::Duration; + /// Watermark config for different types of Vertex. #[derive(Debug, Clone, PartialEq)] pub(crate) enum WatermarkConfig { @@ -9,8 +11,29 @@ pub(crate) enum WatermarkConfig { /// Watermark starts at Source. #[derive(Debug, Clone, PartialEq)] pub(crate) struct SourceWatermarkConfig { + pub(crate) max_delay: Duration, pub(crate) source_bucket_config: BucketConfig, pub(crate) to_vertex_bucket_config: Vec, + pub(crate) idle_config: Option, +} + +/// Idle configuration for detecting idleness when there is no data +/// from source and publish the Watermark. +#[derive(Debug, Clone, PartialEq)] +pub(crate) struct IdleConfig { + pub(crate) increment_by: Duration, + pub(crate) step_interval: Duration, + pub(crate) threshold: Duration, +} + +impl Default for IdleConfig { + fn default() -> Self { + IdleConfig { + increment_by: Duration::from_millis(0), + step_interval: Duration::from_millis(0), + threshold: Duration::from_millis(0), + } + } } /// Watermark movements are captured via a Key/Value bucket. diff --git a/rust/numaflow-core/src/mapper/map.rs b/rust/numaflow-core/src/mapper/map.rs index d8682b0620..55e73814af 100644 --- a/rust/numaflow-core/src/mapper/map.rs +++ b/rust/numaflow-core/src/mapper/map.rs @@ -560,6 +560,7 @@ mod tests { .await?; let message = Message { + typ: Default::default(), keys: Arc::from(vec!["first".into()]), tags: None, value: "hello".into(), @@ -654,6 +655,7 @@ mod tests { for i in 0..5 { let message = Message { + typ: Default::default(), keys: Arc::from(vec![format!("key_{}", i)]), tags: None, value: format!("value_{}", i).into(), @@ -745,6 +747,7 @@ mod tests { let input_stream = ReceiverStream::new(input_rx); let message = Message { + typ: Default::default(), keys: Arc::from(vec!["first".into()]), tags: None, value: "hello".into(), @@ -841,6 +844,7 @@ mod tests { let messages = vec![ Message { + typ: Default::default(), keys: Arc::from(vec!["first".into()]), tags: None, value: "hello".into(), @@ -856,6 +860,7 @@ mod tests { metadata: None, }, Message { + typ: Default::default(), keys: Arc::from(vec!["second".into()]), tags: None, value: "world".into(), @@ -955,6 +960,7 @@ mod tests { let messages = vec![ Message { + typ: Default::default(), keys: Arc::from(vec!["first".into()]), tags: None, value: "hello".into(), @@ -970,6 +976,7 @@ mod tests { metadata: None, }, Message { + typ: Default::default(), keys: Arc::from(vec!["second".into()]), tags: None, value: "world".into(), @@ -1069,6 +1076,7 @@ mod tests { .await?; let message = Message { + typ: Default::default(), keys: Arc::from(vec!["first".into()]), tags: None, value: "test,map,stream".into(), @@ -1167,6 +1175,7 @@ mod tests { .await?; let message = Message { + typ: Default::default(), keys: Arc::from(vec!["first".into()]), tags: None, value: "panic".into(), diff --git a/rust/numaflow-core/src/mapper/map/user_defined.rs b/rust/numaflow-core/src/mapper/map/user_defined.rs index b8d71bd5e2..d55d2e2347 100644 --- a/rust/numaflow-core/src/mapper/map/user_defined.rs +++ b/rust/numaflow-core/src/mapper/map/user_defined.rs @@ -250,6 +250,7 @@ async fn process_response(sender_map: &ResponseSenderMap, resp: MapResponse) { let mut response_messages = vec![]; for (i, result) in resp.results.into_iter().enumerate() { let message = Message { + typ: Default::default(), id: MessageID { vertex_name: get_vertex_name().to_string().into(), index: i as i32, @@ -378,6 +379,7 @@ impl UserDefinedStreamMap { for (i, result) in resp.results.into_iter().enumerate() { let message = Message { + typ: Default::default(), id: MessageID { vertex_name: get_vertex_name().to_string().into(), index: i as i32, @@ -486,6 +488,7 @@ mod tests { .await?; let message = crate::message::Message { + typ: Default::default(), keys: Arc::from(vec!["first".into()]), tags: None, value: "hello".into(), @@ -575,6 +578,7 @@ mod tests { let messages = vec![ crate::message::Message { + typ: Default::default(), keys: Arc::from(vec!["first".into()]), tags: None, value: "hello".into(), @@ -590,6 +594,7 @@ mod tests { metadata: None, }, crate::message::Message { + typ: Default::default(), keys: Arc::from(vec!["second".into()]), tags: None, value: "world".into(), @@ -688,6 +693,7 @@ mod tests { .await?; let message = crate::message::Message { + typ: Default::default(), keys: Arc::from(vec!["first".into()]), tags: None, value: "test,map,stream".into(), diff --git a/rust/numaflow-core/src/message.rs b/rust/numaflow-core/src/message.rs index 8ebd90788f..6a82761986 100644 --- a/rust/numaflow-core/src/message.rs +++ b/rust/numaflow-core/src/message.rs @@ -13,10 +13,12 @@ use crate::Error; const DROP: &str = "U+005C__DROP__"; -/// A message that is sent from the source to the sink. -/// It is cheap to clone. +/// The message that is passed from the source to the sink. +/// NOTE: It is cheap to clone. #[derive(Debug, Clone)] pub(crate) struct Message { + /// Type of the message that flows through the ISB. + pub(crate) typ: MessageType, /// keys of the message pub(crate) keys: Arc<[String]>, /// tags of the message @@ -39,6 +41,45 @@ pub(crate) struct Message { pub(crate) metadata: Option, } +/// Type of the [Message]. +#[derive(Debug, Clone, Default)] +pub(crate) enum MessageType { + /// the payload is Data + #[default] + Data, + /// the payload is a control message. + #[allow(clippy::upper_case_acronyms)] + WMB, +} + +impl fmt::Display for MessageType { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + MessageType::Data => write!(f, "Data"), + MessageType::WMB => write!(f, "WMB"), + } + } +} + +// proto enum is an i32 type and WMB is defined as enum in the proto. +impl From for MessageType { + fn from(kind: i32) -> Self { + match kind { + 0 => MessageType::Data, + _ => MessageType::WMB, + } + } +} + +impl From for i32 { + fn from(kind: MessageType) -> Self { + match kind { + MessageType::Data => 0, + MessageType::WMB => 1, + } + } +} + impl Default for Message { fn default() -> Self { Self { @@ -51,6 +92,7 @@ impl Default for Message { id: Default::default(), headers: HashMap::new(), metadata: None, + typ: Default::default(), } } } @@ -210,7 +252,7 @@ impl TryFrom for BytesMut { event_time: Some(prost_timestamp_from_utc(message.event_time)), is_late: false, // Set this according to your logic }), - kind: numaflow_pb::objects::isb::MessageKind::Data as i32, + kind: message.typ.into(), id: Some(message.id.into()), keys: message.keys.to_vec(), headers: message.headers, @@ -262,6 +304,7 @@ mod tests { #[test] fn test_message_to_vec_u8() { let message = Message { + typ: Default::default(), keys: Arc::from(vec!["key1".to_string()]), tags: None, value: vec![1, 2, 3].into(), diff --git a/rust/numaflow-core/src/metrics.rs b/rust/numaflow-core/src/metrics.rs index 46bf45fd68..d23a9fa0a5 100644 --- a/rust/numaflow-core/src/metrics.rs +++ b/rust/numaflow-core/src/metrics.rs @@ -49,7 +49,7 @@ const PENDING_PERIOD_LABEL: &str = "period"; const PIPELINE_NAME_LABEL: &str = "pipeline"; const PIPELINE_REPLICA_LABEL: &str = "replica"; -const PIPELINE_PARTITION_NAME_LABEL: &str = "partition_name"; +pub(crate) const PIPELINE_PARTITION_NAME_LABEL: &str = "partition_name"; const PIPELINE_VERTEX_LABEL: &str = "vertex"; const PIPELINE_VERTEX_TYPE_LABEL: &str = "vertex_type"; @@ -528,12 +528,9 @@ pub(crate) fn mvtx_forward_metric_labels() -> &'static Vec<(String, String)> { static PIPELINE_READ_METRICS_LABELS: OnceLock> = OnceLock::new(); -pub(crate) fn pipeline_forward_metric_labels( - vertex_type: &str, - partition_name: Option<&str>, -) -> &'static Vec<(String, String)> { +pub(crate) fn pipeline_forward_metric_labels(vertex_type: &str) -> &'static Vec<(String, String)> { PIPELINE_READ_METRICS_LABELS.get_or_init(|| { - let mut labels = vec![ + vec![ ( PIPELINE_NAME_LABEL.to_string(), get_pipeline_name().to_string(), @@ -550,16 +547,7 @@ pub(crate) fn pipeline_forward_metric_labels( PIPELINE_VERTEX_LABEL.to_string(), get_vertex_name().to_string(), ), - ]; - - if let Some(partition) = partition_name { - labels.push(( - PIPELINE_PARTITION_NAME_LABEL.to_string(), - partition.to_string(), - )); - } - - labels + ] }) } @@ -965,9 +953,10 @@ async fn expose_pending_metrics( .get_or_create(&metric_labels) .set(pending); } else { - let mut metric_labels = - pipeline_forward_metric_labels(name, Some(name)).clone(); + let mut metric_labels = pipeline_forward_metric_labels(name).clone(); metric_labels.push((PENDING_PERIOD_LABEL.to_string(), label.to_string())); + metric_labels + .push((PIPELINE_PARTITION_NAME_LABEL.to_string(), name.to_string())); pipeline_metrics() .pending .get_or_create(&metric_labels) @@ -1308,18 +1297,24 @@ mod tests { { for (i, (label, _)) in lookback_seconds_map.iter().enumerate() { - let mut metric_labels = - pipeline_forward_metric_labels("stream1", Some("stream1")).clone(); + let mut metric_labels = pipeline_forward_metric_labels("stream1").clone(); metric_labels.push((PENDING_PERIOD_LABEL.to_string(), label.to_string())); + metric_labels.push(( + PIPELINE_PARTITION_NAME_LABEL.to_string(), + "stream1".to_string(), + )); let guage = pipeline_metrics() .pending .get_or_create(&metric_labels) .get(); stored_values_stream_one[i] = guage; - let mut metric_labels = - pipeline_forward_metric_labels("stream2", Some("stream2")).clone(); + let mut metric_labels = pipeline_forward_metric_labels("stream2").clone(); metric_labels.push((PENDING_PERIOD_LABEL.to_string(), label.to_string())); + metric_labels.push(( + PIPELINE_PARTITION_NAME_LABEL.to_string(), + "stream2".to_string(), + )); let guage = pipeline_metrics() .pending .get_or_create(&metric_labels) diff --git a/rust/numaflow-core/src/pipeline.rs b/rust/numaflow-core/src/pipeline.rs index e787ae1df5..9029775e15 100644 --- a/rust/numaflow-core/src/pipeline.rs +++ b/rust/numaflow-core/src/pipeline.rs @@ -37,7 +37,7 @@ pub(crate) async fn start_forwarder( ) -> Result<()> { let js_context = create_js_context(config.js_client_config.clone()).await?; - match &config.vertex_config { + match &config.vertex_type_config { pipeline::VertexType::Source(source) => { info!("Starting source forwarder"); @@ -45,7 +45,15 @@ pub(crate) async fn start_forwarder( let source_watermark_handle = match &config.watermark_config { Some(wm_config) => { if let WatermarkConfig::Source(source_config) = wm_config { - Some(SourceWatermarkHandle::new(js_context.clone(), source_config).await?) + Some( + SourceWatermarkHandle::new( + config.read_timeout, + js_context.clone(), + &config.to_vertex_config, + source_config, + ) + .await?, + ) } else { None } @@ -73,8 +81,10 @@ pub(crate) async fn start_forwarder( ISBWatermarkHandle::new( config.vertex_name, config.replica, + config.read_timeout, js_context.clone(), edge_config, + &config.to_vertex_config, ) .await?, ) @@ -105,8 +115,10 @@ pub(crate) async fn start_forwarder( ISBWatermarkHandle::new( config.vertex_name, config.replica, + config.read_timeout, js_context.clone(), edge_config, + &config.to_vertex_config, ) .await?, ) @@ -221,7 +233,9 @@ async fn start_map_forwarder( let tracker_handle = TrackerHandle::new(watermark_handle.clone(), serving_callback_handler.clone()); + info!("Creating buffer reader for stream {:?}", stream); let buffer_reader = create_buffer_reader( + config.vertex_type_config.to_string(), stream, reader_config.clone(), js_context.clone(), @@ -324,6 +338,7 @@ async fn start_sink_forwarder( TrackerHandle::new(watermark_handle.clone(), serving_callback_handler.clone()); let buffer_reader = create_buffer_reader( + config.vertex_type_config.to_string(), stream, reader_config.clone(), js_context.clone(), @@ -411,6 +426,7 @@ async fn create_buffer_writer( } async fn create_buffer_reader( + vertex_type: String, stream: Stream, reader_config: BufferReaderConfig, js_context: Context, @@ -419,6 +435,7 @@ async fn create_buffer_reader( watermark_handle: Option, ) -> Result { JetStreamReader::new( + vertex_type, stream, js_context, reader_config, @@ -542,7 +559,7 @@ mod tests { }, from_vertex_config: vec![], to_vertex_config: vec![ToVertexConfig { - name: "out".to_string(), + name: "out", partitions: 5, writer_config: BufferWriterConfig { streams: streams.clone(), @@ -552,7 +569,7 @@ mod tests { }, conditions: None, }], - vertex_config: VertexType::Source(SourceVtxConfig { + vertex_type_config: VertexType::Source(SourceVtxConfig { source_config: SourceConfig { read_ahead: false, source_type: SourceType::Generator(GeneratorConfig { @@ -655,6 +672,7 @@ mod tests { use crate::message::{Message, MessageID, Offset, StringOffset}; let message = Message { + typ: Default::default(), keys: Arc::from(vec!["key1".to_string()]), tags: None, value: vec![1, 2, 3].into(), @@ -708,14 +726,14 @@ mod tests { }, to_vertex_config: vec![], from_vertex_config: vec![FromVertexConfig { - name: "in".to_string(), + name: "in", reader_config: BufferReaderConfig { streams: streams.clone(), wip_ack_interval: Duration::from_secs(1), }, partitions: 0, }], - vertex_config: VertexType::Sink(SinkVtxConfig { + vertex_type_config: VertexType::Sink(SinkVtxConfig { sink_config: SinkConfig { sink_type: SinkType::Blackhole(BlackholeConfig::default()), retry_config: None, @@ -849,6 +867,7 @@ mod tests { use crate::message::{Message, MessageID, Offset, StringOffset}; let message = Message { + typ: Default::default(), keys: Arc::from(vec!["key1".to_string()]), tags: None, value: vec![1, 2, 3].into(), @@ -931,7 +950,7 @@ mod tests { password: None, }, to_vertex_config: vec![ToVertexConfig { - name: "map-out".to_string(), + name: "map-out", partitions: 5, writer_config: BufferWriterConfig { streams: output_streams.clone(), @@ -942,14 +961,14 @@ mod tests { conditions: None, }], from_vertex_config: vec![FromVertexConfig { - name: "map-in".to_string(), + name: "map-in", reader_config: BufferReaderConfig { streams: input_streams.clone(), wip_ack_interval: Duration::from_secs(1), }, partitions: 0, }], - vertex_config: VertexType::Map(MapVtxConfig { + vertex_type_config: VertexType::Map(MapVtxConfig { concurrency: 10, map_type: MapType::UserDefined(UserDefinedConfig { grpc_max_message_size: 4 * 1024 * 1024, diff --git a/rust/numaflow-core/src/pipeline/forwarder/source_forwarder.rs b/rust/numaflow-core/src/pipeline/forwarder/source_forwarder.rs index c7128482dd..0c590e4940 100644 --- a/rust/numaflow-core/src/pipeline/forwarder/source_forwarder.rs +++ b/rust/numaflow-core/src/pipeline/forwarder/source_forwarder.rs @@ -268,7 +268,7 @@ mod tests { ..Default::default() }, conditions: None, - name: "test-vertex".to_string(), + name: "test-vertex", }], context.clone(), 100, diff --git a/rust/numaflow-core/src/pipeline/isb/jetstream/reader.rs b/rust/numaflow-core/src/pipeline/isb/jetstream/reader.rs index ffbe3fc8f2..036e0741b6 100644 --- a/rust/numaflow-core/src/pipeline/isb/jetstream/reader.rs +++ b/rust/numaflow-core/src/pipeline/isb/jetstream/reader.rs @@ -17,14 +17,14 @@ use tracing::{error, info}; use crate::config::get_vertex_name; use crate::config::pipeline::isb::{BufferReaderConfig, Stream}; use crate::error::Error; -use crate::message::{IntOffset, Message, MessageID, Metadata, Offset, ReadAck}; +use crate::message::{IntOffset, Message, MessageID, MessageType, Metadata, Offset, ReadAck}; use crate::metrics::{ pipeline_forward_metric_labels, pipeline_isb_metric_labels, pipeline_metrics, }; use crate::shared::grpc::utc_from_timestamp; use crate::tracker::TrackerHandle; use crate::watermark::isb::ISBWatermarkHandle; -use crate::Result; +use crate::{metrics, Result}; /// The JetStreamReader is a handle to the background actor that continuously fetches messages from JetStream. /// It can be used to cancel the background task and stop reading from JetStream. @@ -38,10 +38,12 @@ pub(crate) struct JetStreamReader { tracker_handle: TrackerHandle, batch_size: usize, watermark_handle: Option, + vertex_type: String, } /// JSWrappedMessage is a wrapper around the JetStream message that includes the /// partition index and the vertex name. +#[derive(Debug)] struct JSWrappedMessage { partition_idx: u16, message: async_nats::jetstream::Message, @@ -78,6 +80,7 @@ impl TryFrom for Message { )); Ok(Message { + typ: header.kind.into(), keys: Arc::from(header.keys.into_boxed_slice()), tags: None, value: body.payload.into(), @@ -102,6 +105,7 @@ impl TryFrom for Message { impl JetStreamReader { pub(crate) async fn new( + vertex_type: String, stream: Stream, js_ctx: Context, config: BufferReaderConfig, @@ -130,6 +134,7 @@ impl JetStreamReader { config.wip_ack_interval = wip_ack_interval; Ok(Self { + vertex_type, stream, config: config.clone(), consumer, @@ -158,9 +163,14 @@ impl JetStreamReader { let tracker_handle = self.tracker_handle.clone(); let cancel_token = cancel_token.clone(); let watermark_handle = self.watermark_handle.clone(); + let vertex_type = self.vertex_type.clone(); async move { - let labels = pipeline_forward_metric_labels("Sink", Some(stream.name)); + let mut labels = pipeline_forward_metric_labels(&vertex_type).clone(); + labels.push(( + metrics::PIPELINE_PARTITION_NAME_LABEL.to_string(), + stream.name.to_string(), + )); let mut message_stream = consumer.messages().await.map_err(|e| { Error::ISB(format!( @@ -196,16 +206,18 @@ impl JetStreamReader { vertex_name: get_vertex_name().to_string(), }; - let mut message: Message = match js_message.try_into() { - Ok(message) => message, - Err(e) => { - error!( - ?e, ?stream, ?jetstream_message, - "Failed to parse message payload received from Jetstream", - ); - continue; - } - }; + let mut message: Message = js_message.try_into().map_err(|e| { + Error::ISB(format!("Failed to convert JetStream message to Message: {:?}", e)) + })?; + + // we can ignore the wmb messages + if let MessageType::WMB = message.typ { + // ack the message and continue + jetstream_message.ack().await.map_err(|e| { + Error::ISB(format!("Failed to ack the wmb message: {:?}", e)) + })?; + continue; + } if let Some(watermark_handle) = watermark_handle.as_ref() { let watermark = watermark_handle.fetch_watermark(message.offset.clone()).await?; @@ -235,7 +247,7 @@ impl JetStreamReader { pipeline_metrics() .forwarder .read_total - .get_or_create(labels) + .get_or_create(&labels) .inc(); } } @@ -263,7 +275,7 @@ impl JetStreamReader { let ack_result = msg.ack_with(AckKind::Progress).await; if let Err(e) = ack_result { // We expect that the ack in the next iteration will be successful. - // If its some unrecoverable Jetstream error, the fetching messages in the JestreamReader implementation should also fail and cause the system to shut down. + // If its some unrecoverable Jetstream error, the fetching messages in the JetstreamReader implementation should also fail and cause the system to shut down. error!(?e, "Failed to send InProgress Ack to Jetstream for message"); } }; @@ -385,6 +397,7 @@ mod tests { wip_ack_interval: Duration::from_millis(5), }; let js_reader = JetStreamReader::new( + "Map".to_string(), stream.clone(), context.clone(), buf_reader_config, @@ -403,6 +416,7 @@ mod tests { for i in 0..10 { let message = Message { + typ: Default::default(), keys: Arc::from(vec![format!("key_{}", i)]), tags: None, value: format!("message {}", i).as_bytes().to_vec().into(), @@ -483,6 +497,7 @@ mod tests { wip_ack_interval: Duration::from_millis(5), }; let js_reader = JetStreamReader::new( + "Map".to_string(), js_stream.clone(), context.clone(), buf_reader_config, @@ -503,6 +518,7 @@ mod tests { // write 5 messages for i in 0..5 { let message = Message { + typ: Default::default(), keys: Arc::from(vec![format!("key_{}", i)]), tags: None, value: format!("message {}", i).as_bytes().to_vec().into(), diff --git a/rust/numaflow-core/src/pipeline/isb/jetstream/writer.rs b/rust/numaflow-core/src/pipeline/isb/jetstream/writer.rs index 0521d25a32..6067ec3565 100644 --- a/rust/numaflow-core/src/pipeline/isb/jetstream/writer.rs +++ b/rust/numaflow-core/src/pipeline/isb/jetstream/writer.rs @@ -9,7 +9,7 @@ use async_nats::jetstream::context::PublishAckFuture; use async_nats::jetstream::publish::PublishAck; use async_nats::jetstream::stream::RetentionPolicy::Limits; use async_nats::jetstream::Context; -use bytes::{Bytes, BytesMut}; +use bytes::BytesMut; use tokio::sync::Semaphore; use tokio::task::JoinHandle; use tokio::time::{sleep, Instant}; @@ -59,12 +59,12 @@ impl JetstreamWriter { cancel_token: CancellationToken, watermark_handle: Option, ) -> Self { - let streams = config + let to_vertex_streams = config .iter() .flat_map(|c| c.writer_config.streams.clone()) .collect::>(); - let is_full = streams + let is_full = to_vertex_streams .iter() .map(|stream| (stream.name, Arc::new(AtomicBool::new(false)))) .collect::>(); @@ -224,6 +224,7 @@ impl JetstreamWriter { .get(partition as usize) .expect("stream should be present") .clone(); + if let Some(paf) = this .write( stream.clone(), @@ -232,7 +233,7 @@ impl JetstreamWriter { ) .await { - pafs.push((stream.clone(), paf)); + pafs.push((stream, paf)); } } @@ -274,8 +275,7 @@ impl JetstreamWriter { message: Message, on_full: BufferFullStrategy, ) -> Option { - let mut counter = 500u16; - + let mut log_counter = 500u16; let offset = message.offset.clone(); let payload: BytesMut = message .try_into() @@ -291,11 +291,11 @@ impl JetstreamWriter { { Some(true) => { // FIXME: add metrics - if counter >= 500 { + if log_counter >= 500 { warn!(stream=?stream, "stream is full (throttled logging)"); - counter = 0; + log_counter = 0; } - counter += 1; + log_counter += 1; match on_full { BufferFullStrategy::DiscardLatest => { // delete the entry from tracker @@ -310,23 +310,19 @@ impl JetstreamWriter { } Some(false) => match self .js_ctx - .publish(stream.name, Bytes::from(payload.clone())) + .publish(stream.name, payload.clone().freeze()) .await { - Ok(paf) => { - break paf; - } - Err(e) => { - error!(?e, "publishing failed, retrying"); - } + Ok(paf) => break paf, + Err(e) => error!(?e, "publishing failed, retrying"), }, - None => { - error!("Stream {} not found in is_full map", stream); - } + None => error!("Stream {} not found in is_full map", stream), } + // short-circuit out in failure mode if shutdown has been initiated if self.cancel_token.is_cancelled() { error!("Shutdown signal received, exiting write loop"); + return None; } // sleep to avoid busy looping @@ -350,10 +346,7 @@ impl JetstreamWriter { .await .map_err(|_e| Error::ISB("Failed to acquire semaphore permit".to_string()))?; - let js_ctx = self.js_ctx.clone(); - let cancel_token = self.cancel_token.clone(); - let tracker_handle = self.tracker_handle.clone(); - let watermark_handle = self.watermark_handle.clone(); + let this = self.clone(); tokio::spawn(async move { let _permit = permit; @@ -361,100 +354,56 @@ impl JetstreamWriter { // resolve the pafs for (stream, paf) in pafs { - match paf.await { + let ack = match paf.await { + Ok(ack) => Ok(ack), + Err(e) => { + error!( + ?e, stream = ?stream, + "Failed to resolve the future trying blocking write", + ); + this.blocking_write(stream.clone(), message.clone()).await + } + }; + + match ack { Ok(ack) => { if ack.duplicate { warn!( - "Duplicate message detected for stream {}, ignoring {:?}", - stream, ack + stream = ?stream, + ack = ?ack, + "Duplicate message detected" ); } offsets.push(( stream.clone(), Offset::Int(IntOffset::new(ack.sequence as i64, stream.partition)), )); - tracker_handle - .delete(message.offset.clone()) - .await - .expect("Failed to delete offset from tracker"); } Err(e) => { - error!( - ?e, - "Failed to resolve the future for stream {}, trying blocking write", - stream - ); - match JetstreamWriter::blocking_write( - stream.clone(), - message.value.clone(), - js_ctx.clone(), - cancel_token.clone(), - ) - .await - { - Ok(ack) => { - if ack.duplicate { - warn!( - "Duplicate message detected for stream {}, ignoring {:?}", - stream, ack - ); - } - offsets.push(( - stream.clone(), - Offset::Int(IntOffset::new( - ack.sequence as i64, - stream.partition, - )), - )); - tracker_handle - .delete(message.offset.clone()) - .await - .expect("Failed to delete offset from tracker"); - } - Err(e) => { - error!(?e, "Blocking write failed for stream {}", stream); - // Since we failed to write to the stream, we need to send a NAK to the reader - tracker_handle - .discard(message.offset.clone()) - .await - .expect("Failed to discard offset from the tracker"); - return; - } - } + error!(?e, stream = ?stream, "Blocking write failed"); + // Since we failed to write to the stream, we need to send a NAK to the reader + this.tracker_handle + .discard(message.offset.clone()) + .await + .expect("Failed to discard offset from the tracker"); + return; } } } // now the pafs have resolved, lets use the offsets to send watermark for (stream, offset) in offsets { - if let Some(watermark_handle) = watermark_handle.as_ref() { - match watermark_handle { - WatermarkHandle::ISB(handle) => { - handle - .publish_watermark(stream, offset) - .await - .map_err(|e| { - Error::ISB(format!("Failed to update watermark: {:?}", e)) - }) - .expect("Failed to publish watermark"); - } - WatermarkHandle::Source(handle) => { - let input_partition = match &message.offset { - Offset::Int(offset) => offset.partition_idx, - Offset::String(offset) => offset.partition_idx, - }; - handle - .publish_source_isb_watermark(stream, offset, input_partition) - .await - .map_err(|e| { - Error::ISB(format!("Failed to update watermark: {:?}", e)) - }) - .expect("Failed to publish watermark"); - } - } + if let Some(watermark_handle) = this.watermark_handle.as_ref() { + JetstreamWriter::publish_watermark(watermark_handle, stream, offset, &message) + .await; } } + this.tracker_handle + .delete(message.offset.clone()) + .await + .expect("Failed to delete offset from tracker"); + pipeline_metrics() .isb .paf_resolution_time @@ -464,26 +413,28 @@ impl JetstreamWriter { Ok(()) } - /// Writes the message to the JetStream ISB and returns the PublishAck. It will do /// infinite retries until the message gets published successfully. If it returns /// an error it means it is fatal non-retryable error. - async fn blocking_write( + pub(crate) async fn blocking_write( + &self, stream: Stream, - payload: Bytes, - js_ctx: Context, - cln_token: CancellationToken, + message: Message, ) -> Result { let start_time = Instant::now(); - info!("Blocking write for stream {}", stream); + let payload: BytesMut = message + .try_into() + .expect("message serialization should not fail"); + loop { - match js_ctx.publish(stream.name, payload.clone()).await { + match self + .js_ctx + .publish(stream.name, payload.clone().freeze()) + .await + { Ok(paf) => match paf.await { Ok(ack) => { if ack.duplicate { - // should we return an error here? Because duplicate messages are not fatal - // But it can mess up the watermark progression because the offset will be - // same as the previous message offset warn!(?ack, "Duplicate message detected, ignoring"); } debug!( @@ -502,11 +453,39 @@ impl JetstreamWriter { sleep(Duration::from_millis(DEFAULT_RETRY_INTERVAL_MILLIS)).await; } } - if cln_token.is_cancelled() { + + if self.cancel_token.is_cancelled() { return Err(Error::ISB("Shutdown signal received".to_string())); } } } + + /// publishes the watermark for the given stream and offset + async fn publish_watermark( + watermark_handle: &WatermarkHandle, + stream: Stream, + offset: Offset, + message: &Message, + ) { + match watermark_handle { + WatermarkHandle::ISB(handle) => { + handle + .publish_watermark(stream, offset) + .await + .map_err(|e| Error::ISB(format!("Failed to update watermark: {:?}", e))) + .expect("Failed to publish watermark"); + } + WatermarkHandle::Source(handle) => { + let input_partition = match &message.offset { + Offset::Int(offset) => offset.partition_idx, + Offset::String(offset) => offset.partition_idx, + }; + handle + .publish_source_isb_watermark(stream, offset, input_partition) + .await; + } + } + } } #[cfg(test)] @@ -517,7 +496,7 @@ mod tests { use async_nats::jetstream; use async_nats::jetstream::consumer::{Config, Consumer}; use async_nats::jetstream::{consumer, stream}; - use bytes::BytesMut; + use bytes::Bytes; use chrono::Utc; use numaflow_models::models::ForwardConditions; use numaflow_models::models::TagConditions; @@ -562,7 +541,7 @@ mod tests { let writer = JetstreamWriter::new( vec![ToVertexConfig { - name: "test-vertex".to_string(), + name: "test-vertex", partitions: 1, writer_config: BufferWriterConfig { streams: vec![stream.clone()], @@ -578,6 +557,7 @@ mod tests { ); let message = Message { + typ: Default::default(), keys: Arc::from(vec!["key_0".to_string()]), tags: None, value: "message 0".as_bytes().to_vec().into(), @@ -639,6 +619,7 @@ mod tests { .unwrap(); let message = Message { + typ: Default::default(), keys: Arc::from(vec!["key_0".to_string()]), tags: None, value: "message 0".as_bytes().to_vec().into(), @@ -654,14 +635,24 @@ mod tests { metadata: None, }; - let message_bytes: BytesMut = message.try_into().unwrap(); - let result = JetstreamWriter::blocking_write( - stream.clone(), - message_bytes.into(), + let writer = JetstreamWriter::new( + vec![ToVertexConfig { + name: "test-vertex", + partitions: 1, + writer_config: BufferWriterConfig { + streams: vec![stream.clone()], + ..Default::default() + }, + conditions: None, + }], context.clone(), + 100, + TrackerHandle::new(None, None), cln_token.clone(), - ) - .await; + None, + ); + + let result = writer.blocking_write(stream.clone(), message).await; assert!(result.is_ok()); let publish_ack = result.unwrap(); @@ -708,7 +699,7 @@ mod tests { let writer = JetstreamWriter::new( vec![ToVertexConfig { - name: "test-vertex".to_string(), + name: "test-vertex", partitions: 1, writer_config: BufferWriterConfig { streams: vec![stream.clone()], @@ -727,6 +718,7 @@ mod tests { // Publish 10 messages successfully for i in 0..10 { let message = Message { + typ: Default::default(), keys: Arc::from(vec![format!("key_{}", i)]), tags: None, value: format!("message {}", i).as_bytes().to_vec().into(), @@ -754,6 +746,7 @@ mod tests { // Attempt to publish a message which has a payload size greater than the max_message_size // so that it fails and sync write will be attempted and it will be blocked let message = Message { + typ: Default::default(), keys: Arc::from(vec!["key_11".to_string()]), tags: None, value: vec![0; 1025].into(), @@ -914,7 +907,7 @@ mod tests { let cancel_token = CancellationToken::new(); let writer = JetstreamWriter::new( vec![ToVertexConfig { - name: "test-vertex".to_string(), + name: "test-vertex", partitions: 1, writer_config: BufferWriterConfig { streams: vec![stream.clone()], @@ -1006,7 +999,7 @@ mod tests { let writer = JetstreamWriter::new( vec![ToVertexConfig { - name: "test-vertex".to_string(), + name: "test-vertex", partitions: 1, writer_config: BufferWriterConfig { streams: vec![stream.clone()], @@ -1027,6 +1020,7 @@ mod tests { // Publish 500 messages for i in 0..500 { let message = Message { + typ: Default::default(), keys: Arc::from(vec![format!("key_{}", i)]), tags: None, value: format!("message {}", i).as_bytes().to_vec().into(), @@ -1096,7 +1090,7 @@ mod tests { let cancel_token = CancellationToken::new(); let writer = JetstreamWriter::new( vec![ToVertexConfig { - name: "test-vertex".to_string(), + name: "test-vertex", partitions: 1, writer_config: BufferWriterConfig { streams: vec![stream.clone()], @@ -1116,6 +1110,7 @@ mod tests { // Publish 100 messages successfully for i in 0..100 { let message = Message { + typ: Default::default(), keys: Arc::from(vec![format!("key_{}", i)]), tags: None, value: format!("message {}", i).as_bytes().to_vec().into(), @@ -1142,6 +1137,7 @@ mod tests { // Attempt to publish the 101st message, which should get stuck in the retry loop // because the max message size is set to 1024 let message = Message { + typ: Default::default(), keys: Arc::from(vec!["key_101".to_string()]), tags: None, value: vec![0; 1025].into(), @@ -1208,7 +1204,7 @@ mod tests { let writer = JetstreamWriter::new( vec![ ToVertexConfig { - name: "vertex1".to_string(), + name: "vertex1", partitions: 2, writer_config: BufferWriterConfig { streams: vertex1_streams.clone(), @@ -1220,7 +1216,7 @@ mod tests { }))), }, ToVertexConfig { - name: "vertex2".to_string(), + name: "vertex2", partitions: 2, writer_config: BufferWriterConfig { streams: vertex2_streams.clone(), @@ -1232,7 +1228,7 @@ mod tests { }))), }, ToVertexConfig { - name: "vertex3".to_string(), + name: "vertex3", partitions: 2, writer_config: BufferWriterConfig { streams: vertex3_streams.clone(), @@ -1255,6 +1251,7 @@ mod tests { let mut ack_rxs = vec![]; for i in 0..10 { let message = Message { + typ: Default::default(), keys: Arc::from(vec![format!("key_{}", i)]), tags: Some(Arc::from(vec!["tag1".to_string(), "tag2".to_string()])), value: format!("message {}", i).as_bytes().to_vec().into(), diff --git a/rust/numaflow-core/src/shared/create_components.rs b/rust/numaflow-core/src/shared/create_components.rs index aca93d864d..ecffe43fef 100644 --- a/rust/numaflow-core/src/shared/create_components.rs +++ b/rust/numaflow-core/src/shared/create_components.rs @@ -327,13 +327,19 @@ pub async fn create_source( tracker_handle, source_config.read_ahead, transformer, - None, + watermark_handle, ), Some(source_grpc_client), )) } SourceType::Pulsar(pulsar_config) => { - let pulsar = new_pulsar_source(pulsar_config.clone(), batch_size, read_timeout).await?; + let pulsar = new_pulsar_source( + pulsar_config.clone(), + batch_size, + read_timeout, + *get_vertex_replica(), + ) + .await?; Ok(( Source::new( batch_size, @@ -341,7 +347,7 @@ pub async fn create_source( tracker_handle, source_config.read_ahead, transformer, - None, + watermark_handle, ), None, )) @@ -361,7 +367,7 @@ pub async fn create_source( tracker_handle, source_config.read_ahead, transformer, - None, + watermark_handle, ), None, )) diff --git a/rust/numaflow-core/src/sink.rs b/rust/numaflow-core/src/sink.rs index 8d4fba9d26..c360281764 100644 --- a/rust/numaflow-core/src/sink.rs +++ b/rust/numaflow-core/src/sink.rs @@ -330,12 +330,12 @@ impl SinkWriter { pipeline_metrics() .forwarder .write_time - .get_or_create(pipeline_forward_metric_labels("Sink", None)) + .get_or_create(pipeline_forward_metric_labels("Sink")) .observe(sink_start.elapsed().as_micros() as f64); pipeline_metrics() .forwarder .dropped_total - .get_or_create(pipeline_forward_metric_labels("Sink", None)) + .get_or_create(pipeline_forward_metric_labels("Sink")) .inc_by((total_msgs - total_valid_msgs) as u64); } @@ -452,7 +452,7 @@ impl SinkWriter { pipeline_metrics() .forwarder .write_total - .get_or_create(pipeline_forward_metric_labels("Sink", None)) + .get_or_create(pipeline_forward_metric_labels("Sink")) .inc_by(total_msgs as u64); } @@ -765,6 +765,7 @@ mod tests { let messages: Vec = (0..5) .map(|i| Message { + typ: Default::default(), keys: Arc::from(vec![format!("key_{}", i)]), tags: None, value: format!("message {}", i).as_bytes().to_vec().into(), @@ -802,6 +803,7 @@ mod tests { let messages: Vec = (0..10) .map(|i| Message { + typ: Default::default(), keys: Arc::from(vec![format!("key_{}", i)]), tags: None, value: format!("message {}", i).as_bytes().to_vec().into(), @@ -879,6 +881,7 @@ mod tests { let messages: Vec = (0..10) .map(|i| Message { + typ: Default::default(), keys: Arc::from(vec!["error".to_string()]), tags: None, value: format!("message {}", i).as_bytes().to_vec().into(), @@ -965,6 +968,7 @@ mod tests { let messages: Vec = (0..20) .map(|i| Message { + typ: Default::default(), keys: Arc::from(vec!["fallback".to_string()]), tags: None, value: format!("message {}", i).as_bytes().to_vec().into(), @@ -1008,6 +1012,7 @@ mod tests { #[test] fn test_message_to_sink_request() { let message = Message { + typ: Default::default(), keys: Arc::from(vec!["key1".to_string()]), tags: None, value: vec![1, 2, 3].into(), diff --git a/rust/numaflow-core/src/sink/blackhole.rs b/rust/numaflow-core/src/sink/blackhole.rs index 1cfb994f06..8add971a96 100644 --- a/rust/numaflow-core/src/sink/blackhole.rs +++ b/rust/numaflow-core/src/sink/blackhole.rs @@ -33,6 +33,7 @@ mod tests { let mut sink = BlackholeSink; let messages = vec![ Message { + typ: Default::default(), keys: Arc::from(vec![]), tags: None, value: b"Hello, World!".to_vec().into(), @@ -48,6 +49,7 @@ mod tests { metadata: None, }, Message { + typ: Default::default(), keys: Arc::from(vec![]), tags: None, value: b"Hello, World!".to_vec().into(), diff --git a/rust/numaflow-core/src/sink/log.rs b/rust/numaflow-core/src/sink/log.rs index f71ca7f077..1c95cd1b9e 100644 --- a/rust/numaflow-core/src/sink/log.rs +++ b/rust/numaflow-core/src/sink/log.rs @@ -46,6 +46,7 @@ mod tests { let mut sink = LogSink; let messages = vec![ Message { + typ: Default::default(), keys: Arc::from(vec![]), tags: None, value: b"Hello, World!".to_vec().into(), @@ -61,6 +62,7 @@ mod tests { metadata: None, }, Message { + typ: Default::default(), keys: Arc::from(vec![]), tags: None, value: b"Hello, World!".to_vec().into(), diff --git a/rust/numaflow-core/src/sink/user_defined.rs b/rust/numaflow-core/src/sink/user_defined.rs index b6246c2b0f..7eb0384dcb 100644 --- a/rust/numaflow-core/src/sink/user_defined.rs +++ b/rust/numaflow-core/src/sink/user_defined.rs @@ -201,6 +201,7 @@ mod tests { let messages = vec![ Message { + typ: Default::default(), keys: Arc::from(vec![]), tags: None, value: b"Hello, World!".to_vec().into(), @@ -216,6 +217,7 @@ mod tests { metadata: None, }, Message { + typ: Default::default(), keys: Arc::from(vec![]), tags: None, value: b"Hello, World!".to_vec().into(), diff --git a/rust/numaflow-core/src/source.rs b/rust/numaflow-core/src/source.rs index 33f9def62c..f7e1cc1fc9 100644 --- a/rust/numaflow-core/src/source.rs +++ b/rust/numaflow-core/src/source.rs @@ -7,7 +7,7 @@ use std::sync::Arc; use numaflow_pulsar::source::PulsarSource; -use numaflow_sqs::source::SqsSource; +use numaflow_sqs::source::SQSSource; use tokio::sync::OwnedSemaphorePermit; use tokio::sync::Semaphore; use tokio::sync::{mpsc, oneshot}; @@ -15,7 +15,7 @@ use tokio::task::JoinHandle; use tokio::time::Instant; use tokio_stream::wrappers::ReceiverStream; use tokio_util::sync::CancellationToken; -use tracing::{error, info, warn}; +use tracing::{error, info}; use crate::config::{get_vertex_name, is_mono_vertex}; use crate::error::{Error, Result}; @@ -27,6 +27,7 @@ use crate::metrics::{ use crate::tracker::TrackerHandle; use crate::{ message::{Message, Offset}, + metrics, reader::LagReader, }; @@ -61,9 +62,8 @@ pub(crate) trait SourceReader { async fn read(&mut self) -> Result>; - #[allow(dead_code)] /// number of partitions processed by this source. - fn partitions(&self) -> Vec; + async fn partitions(&mut self) -> Result>; } /// Set of Ack related items that has to be implemented to become a Source. @@ -84,7 +84,9 @@ pub(crate) enum SourceType { generator::GeneratorLagReader, ), Pulsar(PulsarSource), - Sqs(SqsSource), + #[allow(clippy::upper_case_acronyms)] + #[allow(dead_code)] // TODO(SQS): remove it when integrated with controller + SQS(SQSSource), Serving(ServingSource), } @@ -103,6 +105,9 @@ enum ActorMessage { Pending { respond_to: oneshot::Sender>>, }, + Partitions { + respond_to: oneshot::Sender>>, + }, } struct SourceActor { @@ -148,6 +153,10 @@ where let pending = self.lag_reader.pending().await; let _ = respond_to.send(pending); } + ActorMessage::Partitions { respond_to } => { + let partitions = self.reader.partitions().await; + let _ = respond_to.send(partitions); + } } } @@ -206,7 +215,7 @@ impl Source { actor.run().await; }); } - SourceType::Sqs(sqs_source) => { + SourceType::SQS(sqs_source) => { tokio::spawn(async move { let actor = SourceActor::new( receiver, @@ -274,6 +283,18 @@ impl Source { .map_err(|e| Error::ActorPatternRecv(e.to_string()))? } + /// get the source partitions from which the source is reading from. + async fn partitions(source_handle: mpsc::Sender) -> Result> { + let (sender, receiver) = oneshot::channel(); + let msg = ActorMessage::Partitions { respond_to: sender }; + // Ignore send errors. If send fails, so does the recv.await below. There's no reason + // to check for the same failure twice. + let _ = source_handle.send(msg).await; + receiver + .await + .map_err(|e| Error::ActorPatternRecv(e.to_string()))? + } + /// Starts streaming messages from the source. It returns a stream of messages and /// a handle to the spawned task. pub(crate) fn streaming_read( @@ -288,7 +309,12 @@ impl Source { let mut transformer = self.transformer.clone(); let mut watermark_handle = self.watermark_handle.clone(); - let pipeline_labels = pipeline_forward_metric_labels("Source", Some(get_vertex_name())); + let mut pipeline_labels = pipeline_forward_metric_labels("Source").clone(); + pipeline_labels.push(( + metrics::PIPELINE_PARTITION_NAME_LABEL.to_string(), + get_vertex_name().to_string(), + )); + let mvtx_labels = mvtx_forward_metric_labels(); info!(?batch_size, "Started streaming source with batch size"); @@ -322,11 +348,18 @@ impl Source { }; let msgs_len = messages.len(); + Self::send_read_metrics(&pipeline_labels, mvtx_labels, read_start_time, msgs_len); - Self::send_read_metrics(pipeline_labels, mvtx_labels, read_start_time, msgs_len); - + // attempt to publish idle watermark since we are not able to read any message from + // the source. if msgs_len == 0 { - continue; + if let Some(watermark_handle) = watermark_handle.as_mut() { + watermark_handle + .publish_source_idle_watermark( + Self::partitions(source_handle.clone()).await?, + ) + .await; + } } let mut ack_batch = Vec::with_capacity(msgs_len); @@ -406,10 +439,7 @@ impl Source { let start = Instant::now(); if !offsets_to_ack.is_empty() { Self::ack(source_handle, offsets_to_ack).await?; - } else { - warn!("no messages to ack, perhaps all are to be `nack'ed`"); } - Self::send_ack_metrics(e2e_start_time, n, start); Ok(()) @@ -624,6 +654,9 @@ mod tests { let pending = source.pending().await.unwrap(); assert_eq!(pending, Some(0)); + let partitions = Source::partitions(source.sender.clone()).await.unwrap(); + assert_eq!(partitions, vec![1, 2]); + cln_token.cancel(); let _ = handle.await.unwrap(); drop(source); diff --git a/rust/numaflow-core/src/source/generator.rs b/rust/numaflow-core/src/source/generator.rs index 97cdc8d20b..b03370f21e 100644 --- a/rust/numaflow-core/src/source/generator.rs +++ b/rust/numaflow-core/src/source/generator.rs @@ -1,6 +1,7 @@ use tokio_stream::StreamExt; use crate::config::components::source::GeneratorConfig; +use crate::config::get_vertex_replica; use crate::message::{Message, Offset}; use crate::reader; use crate::source; @@ -164,6 +165,7 @@ mod stream_generator { } Message { + typ: Default::default(), keys: Arc::from(self.next_key_to_be_fetched()), tags: None, value: data.into(), @@ -350,8 +352,8 @@ impl source::SourceReader for GeneratorRead { Ok(messages) } - fn partitions(&self) -> Vec { - unimplemented!() + async fn partitions(&mut self) -> crate::error::Result> { + Ok(vec![*get_vertex_replica()]) } } diff --git a/rust/numaflow-core/src/source/pulsar.rs b/rust/numaflow-core/src/source/pulsar.rs index 0c32598a17..d14bad5aa3 100644 --- a/rust/numaflow-core/src/source/pulsar.rs +++ b/rust/numaflow-core/src/source/pulsar.rs @@ -15,6 +15,7 @@ impl TryFrom for Message { let offset = Offset::Int(IntOffset::new(message.offset as i64, *get_vertex_replica())); Ok(Message { + typ: Default::default(), keys: Arc::from(vec![message.key]), tags: None, value: message.payload, @@ -52,8 +53,9 @@ pub(crate) async fn new_pulsar_source( cfg: PulsarSourceConfig, batch_size: usize, timeout: Duration, + vertex_replica: u16, ) -> crate::Result { - Ok(PulsarSource::new(cfg, batch_size, timeout).await?) + Ok(PulsarSource::new(cfg, batch_size, timeout, vertex_replica).await?) } impl source::SourceReader for PulsarSource { @@ -69,8 +71,8 @@ impl source::SourceReader for PulsarSource { .collect() } - fn partitions(&self) -> Vec { - Self::partitions(self) + async fn partitions(&mut self) -> crate::error::Result> { + Ok(self.partitions_vec()) } } @@ -115,7 +117,7 @@ mod tests { max_unack: 100, auth: None, }; - let mut pulsar = new_pulsar_source(cfg, 10, Duration::from_millis(200)).await?; + let mut pulsar = new_pulsar_source(cfg, 10, Duration::from_millis(200), 0).await?; assert_eq!(pulsar.name(), "Pulsar"); // Read should return before the timeout diff --git a/rust/numaflow-core/src/source/serving.rs b/rust/numaflow-core/src/source/serving.rs index 3c1e8bc77a..dfb73e2b18 100644 --- a/rust/numaflow-core/src/source/serving.rs +++ b/rust/numaflow-core/src/source/serving.rs @@ -16,6 +16,7 @@ impl TryFrom for Message { Ok(Message { // we do not support keys from HTTP client + typ: Default::default(), keys: Arc::from(vec![]), tags: None, value: message.value, @@ -54,8 +55,8 @@ impl super::SourceReader for ServingSource { .collect() } - fn partitions(&self) -> Vec { - vec![*get_vertex_replica()] + async fn partitions(&mut self) -> Result> { + Ok(vec![*get_vertex_replica()]) } } @@ -148,7 +149,7 @@ mod tests { ..Default::default() }; let settings = Arc::new(settings); - // Setup the CryptoProvider (controls core cryptography used by rustls) for the process + // Set up the CryptoProvider (controls core cryptography used by rustls) for the process // ServingSource starts an Axum HTTPS server in the background. Rustls is used to generate // self-signed certs when starting the server. let _ = rustls::crypto::aws_lc_rs::default_provider().install_default(); diff --git a/rust/numaflow-core/src/source/sqs.rs b/rust/numaflow-core/src/source/sqs.rs index 3de406ed3d..47fbbd27fa 100644 --- a/rust/numaflow-core/src/source/sqs.rs +++ b/rust/numaflow-core/src/source/sqs.rs @@ -1,20 +1,21 @@ use std::sync::Arc; use std::time::Duration; -use numaflow_sqs::source::{SqsMessage, SqsSource, SqsSourceBuilder, SqsSourceConfig}; +use numaflow_sqs::source::{SQSMessage, SQSSource, SQSSourceConfig, SqsSourceBuilder}; use crate::config::{get_vertex_name, get_vertex_replica}; use crate::error::Error; use crate::message::{Message, MessageID, Offset, StringOffset}; use crate::source; -impl TryFrom for Message { +impl TryFrom for Message { type Error = Error; - fn try_from(message: SqsMessage) -> crate::Result { + fn try_from(message: SQSMessage) -> crate::Result { let offset = Offset::String(StringOffset::new(message.offset, *get_vertex_replica())); Ok(Message { + typ: Default::default(), keys: Arc::from(vec![message.key]), tags: None, value: message.payload, @@ -44,11 +45,12 @@ impl From for Error { } } +#[allow(dead_code)] // TODO(SQS): remove it when integrated with controller pub(crate) async fn new_sqs_source( - cfg: SqsSourceConfig, + cfg: SQSSourceConfig, batch_size: usize, timeout: Duration, -) -> crate::Result { +) -> crate::Result { Ok(SqsSourceBuilder::new(cfg) .batch_size(batch_size) .timeout(timeout) @@ -56,7 +58,7 @@ pub(crate) async fn new_sqs_source( .await?) } -impl source::SourceReader for SqsSource { +impl source::SourceReader for SQSSource { fn name(&self) -> &'static str { "Sqs" } @@ -70,12 +72,12 @@ impl source::SourceReader for SqsSource { } // if source doesn't support partitions, we should return the vec![vertex_replica] - fn partitions(&self) -> Vec { - vec![*get_vertex_replica()] + async fn partitions(&mut self) -> crate::Result> { + Ok(vec![*get_vertex_replica()]) } } -impl source::SourceAcker for SqsSource { +impl source::SourceAcker for SQSSource { async fn ack(&mut self, offsets: Vec) -> crate::error::Result<()> { let mut sqs_offsets = Vec::with_capacity(offsets.len()); for offset in offsets { @@ -90,7 +92,7 @@ impl source::SourceAcker for SqsSource { } } -impl source::LagReader for SqsSource { +impl source::LagReader for SQSSource { async fn pending(&mut self) -> crate::error::Result> { Ok(self.pending_count().await) } @@ -120,7 +122,7 @@ pub mod tests { let mut headers = HashMap::new(); headers.insert("foo".to_string(), "bar".to_string()); - let sqs_message = SqsMessage { + let sqs_message = SQSMessage { key: "key".to_string(), payload: Bytes::from("value".to_string()), offset: "offset".to_string(), @@ -162,7 +164,7 @@ pub mod tests { let sqs_client = aws_sdk_sqs::Client::from_conf(get_test_config_with_interceptor(sqs_operation_mocks)); - let sqs_source = SqsSourceBuilder::new(SqsSourceConfig { + let sqs_source = SqsSourceBuilder::new(SQSSourceConfig { region: SQS_DEFAULT_REGION.to_string(), queue_name: "test-q".to_string(), }) @@ -178,7 +180,7 @@ pub mod tests { let tracker_handle = TrackerHandle::new(None, None); let source = Source::new( 1, - SourceType::Sqs(sqs_source), + SourceType::SQS(sqs_source), tracker_handle.clone(), true, None, diff --git a/rust/numaflow-core/src/source/user_defined.rs b/rust/numaflow-core/src/source/user_defined.rs index 374a9ead66..e6e1ac90be 100644 --- a/rust/numaflow-core/src/source/user_defined.rs +++ b/rust/numaflow-core/src/source/user_defined.rs @@ -26,6 +26,7 @@ pub(crate) struct UserDefinedSourceRead { resp_stream: Streaming, num_records: usize, timeout: Duration, + source_client: SourceClient, } /// User-Defined Source to operative on custom sources. @@ -54,17 +55,18 @@ pub(crate) async fn new_source( impl UserDefinedSourceRead { async fn new( - mut client: SourceClient, + client: SourceClient, batch_size: usize, timeout: Duration, ) -> Result { - let (read_tx, resp_stream) = Self::create_reader(batch_size, &mut client).await?; + let (read_tx, resp_stream) = Self::create_reader(batch_size, &mut client.clone()).await?; Ok(Self { read_tx, resp_stream, num_records: batch_size, timeout, + source_client: client, }) } @@ -118,6 +120,7 @@ impl TryFrom for Message { }; Ok(Message { + typ: Default::default(), keys: Arc::from(result.keys), tags: None, value: result.payload.into(), @@ -189,8 +192,18 @@ impl SourceReader for UserDefinedSourceRead { Ok(messages) } - fn partitions(&self) -> Vec { - unimplemented!() + async fn partitions(&mut self) -> Result> { + let partitions = self + .source_client + .partitions_fn(Request::new(())) + .await + .map_err(|e| Error::Source(e.to_string()))? + .into_inner() + .result + .expect("partitions not found") + .partitions; + + Ok(partitions.iter().map(|p| *p as u16).collect()) } } @@ -398,6 +411,9 @@ mod tests { let pending = lag_reader.pending().await.unwrap(); assert_eq!(pending, Some(0)); + let partitions = src_read.partitions().await.unwrap(); + assert_eq!(partitions, vec![2]); + // we need to drop the client, because if there are any in-flight requests // server fails to shut down. https://github.com/numaproj/numaflow-rs/issues/85 drop(src_read); diff --git a/rust/numaflow-core/src/tracker.rs b/rust/numaflow-core/src/tracker.rs index 67672123df..286b91ae61 100644 --- a/rust/numaflow-core/src/tracker.rs +++ b/rust/numaflow-core/src/tracker.rs @@ -358,7 +358,7 @@ impl TrackerHandle { callback_handler: Option, ) -> Self { let enable_callbacks = callback_handler.is_some(); - let (sender, receiver) = mpsc::channel(100); + let (sender, receiver) = mpsc::channel(1000); let tracker = Tracker::new(receiver, watermark_handle, callback_handler); tokio::spawn(tracker.run()); Self { @@ -491,6 +491,7 @@ mod tests { #[test] fn test_message_to_callback_info_conversion() { let mut message = Message { + typ: Default::default(), keys: Arc::from([]), tags: None, value: Bytes::from_static(b"test"), @@ -537,6 +538,7 @@ mod tests { let (ack_send, ack_recv) = oneshot::channel(); let message = Message { + typ: Default::default(), keys: Arc::from([]), tags: None, value: Bytes::from_static(b"test"), @@ -577,6 +579,7 @@ mod tests { let handle = TrackerHandle::new(None, None); let (ack_send, ack_recv) = oneshot::channel(); let message = Message { + typ: Default::default(), keys: Arc::from([]), tags: None, value: Bytes::from_static(b"test"), @@ -622,6 +625,7 @@ mod tests { let (ack_send, ack_recv) = oneshot::channel(); let message = Message { + typ: Default::default(), keys: Arc::from([]), tags: None, value: Bytes::from_static(b"test"), @@ -656,6 +660,7 @@ mod tests { let (ack_send, ack_recv) = oneshot::channel(); let message = Message { + typ: Default::default(), keys: Arc::from([]), tags: None, value: Bytes::from_static(b"test"), @@ -755,6 +760,7 @@ mod tests { let offset = Offset::String(StringOffset::new("offset1".to_string(), 0)); let message = Message { + typ: Default::default(), keys: Arc::from([]), tags: None, value: Bytes::from_static(b"test"), diff --git a/rust/numaflow-core/src/transformer.rs b/rust/numaflow-core/src/transformer.rs index f49a473819..c7429cc068 100644 --- a/rust/numaflow-core/src/transformer.rs +++ b/rust/numaflow-core/src/transformer.rs @@ -226,6 +226,7 @@ mod tests { let transformer = Transformer::new(500, 10, client, tracker_handle.clone()).await?; let message = Message { + typ: Default::default(), keys: Arc::from(vec!["first".into()]), tags: None, value: "hello".into(), @@ -292,6 +293,7 @@ mod tests { let mut messages = vec![]; for i in 0..5 { let message = Message { + typ: Default::default(), keys: Arc::from(vec![format!("key_{}", i)]), tags: None, value: format!("value_{}", i).into(), @@ -367,6 +369,7 @@ mod tests { let transformer = Transformer::new(500, 10, client, tracker_handle.clone()).await?; let message = Message { + typ: Default::default(), keys: Arc::from(vec!["first".into()]), tags: None, value: "hello".into(), diff --git a/rust/numaflow-core/src/transformer/user_defined.rs b/rust/numaflow-core/src/transformer/user_defined.rs index e2bdb558b8..5d5e2e812f 100644 --- a/rust/numaflow-core/src/transformer/user_defined.rs +++ b/rust/numaflow-core/src/transformer/user_defined.rs @@ -129,6 +129,7 @@ impl UserDefinedTransformer { let mut response_messages = vec![]; for (i, result) in resp.results.into_iter().enumerate() { let message = Message { + typ: Default::default(), id: MessageID { vertex_name: get_vertex_name().to_string().into(), index: i as i32, @@ -234,6 +235,7 @@ mod tests { .await?; let message = Message { + typ: Default::default(), keys: Arc::from(vec!["first".into()]), tags: None, value: "hello".into(), @@ -277,6 +279,7 @@ mod tests { #[test] fn test_message_to_source_transform_request() { let message = Message { + typ: Default::default(), keys: Arc::from(vec!["key1".to_string()]), tags: None, value: vec![1, 2, 3].into(), diff --git a/rust/numaflow-core/src/watermark.rs b/rust/numaflow-core/src/watermark.rs index a469428436..f43152970a 100644 --- a/rust/numaflow-core/src/watermark.rs +++ b/rust/numaflow-core/src/watermark.rs @@ -37,6 +37,9 @@ mod processor; /// the watermark starts at source, so we will have to do a publishing followed by a fetch and publish. pub(crate) mod source; +/// Publishes idle watermarks for the source and ISB when they are idling. +mod idle; + /// Stores WMB related data. mod wmb; diff --git a/rust/numaflow-core/src/watermark/idle.rs b/rust/numaflow-core/src/watermark/idle.rs new file mode 100644 index 0000000000..5ede6a7b19 --- /dev/null +++ b/rust/numaflow-core/src/watermark/idle.rs @@ -0,0 +1,18 @@ +//! Idling happens during following scenarios +//! +//! > Read == 0 +//! +//! If no messages are being read from the source or ISB, we should be able to detect +//! and publish the idle watermark. +//! +//! > Branch Idling +//! +//! With conditional forwarding, some streams(branches) will get less frequent writes +//! than others. In such cases, we should be able to detect and publish the idle +//! watermark to those branches. + +/// Idle detection and handling for ISB. +pub(super) mod isb; + +/// Idle detection and handling for Source. +pub(super) mod source; diff --git a/rust/numaflow-core/src/watermark/idle/isb.rs b/rust/numaflow-core/src/watermark/idle/isb.rs new file mode 100644 index 0000000000..a3c8daa636 --- /dev/null +++ b/rust/numaflow-core/src/watermark/idle/isb.rs @@ -0,0 +1,347 @@ +//! ISB Idle Manager resolves the following conundrums: +//! +//! > How to decide if the ISB ([Stream]) is idling? +//! +//! If we have not published any WM for that given [Stream] for X duration, then it is considered +//! idling. +//! +//! > When to publish the idle watermark? +//! +//! Once the X duration has passed, an idle WM will be published. +//! +//! > What to publish as the idle watermark? +//! +//! Fetch the `min(wm(Head Idle Offset), wm(smallest offset of inflight messages))` and publish as +//! idle. + +use std::collections::HashMap; +use std::sync::{Arc, RwLock}; +use std::time::Duration; + +use bytes::BytesMut; +use chrono::{DateTime, Utc}; + +use crate::config::pipeline::isb::Stream; +use crate::config::pipeline::ToVertexConfig; + +/// State of each partition in the ISB. It has the information required to identify whether the +/// partition is idling or not. +#[derive(Clone)] +struct IdleState { + stream: Stream, + last_wm_published_time: DateTime, + /// This offset's WM will keep increasing as long as the [Stream] is idling. + wmb_msg_offset: Option, +} + +impl Default for IdleState { + fn default() -> Self { + IdleState { + stream: Stream::default(), + last_wm_published_time: Utc::now(), + wmb_msg_offset: None, + } + } +} + +/// ISBIdleDetector detects the idle partitions in the ISB. It keeps track of the last published watermark +/// state to detect the idle partitions, it also keeps track of the ctrl message offset that should +/// be used for publishing the idle watermark. +#[derive(Clone)] +pub(crate) struct ISBIdleDetector { + /// last published wm state per [Stream]. + last_published_wm_state: Arc>>>, + js_context: async_nats::jetstream::Context, + /// X duration we wait before we start publishing idle WM. + idle_timeout: Duration, +} + +impl ISBIdleDetector { + /// Creates a new ISBIdleManager. + pub(crate) async fn new( + idle_timeout: Duration, + to_vertex_configs: &[ToVertexConfig], + js_context: async_nats::jetstream::Context, + ) -> Self { + let mut last_published_wm = HashMap::new(); + + // for each vertex, we need per stream (branch) idle state + for config in to_vertex_configs { + let idle_states = config + .writer_config + .streams + .iter() + .map(|stream| { + let stream = stream.clone(); + IdleState { + stream, + ..Default::default() + } + }) + .collect(); + + last_published_wm.insert(config.name, idle_states); + } + + ISBIdleDetector { + idle_timeout, + last_published_wm_state: Arc::new(RwLock::new(last_published_wm)), + js_context, + } + } + + /// resets the stream's idle metadata by updating the last published time and resets the ctrl + /// message offset. It implicitly marks that stream as active. + pub(crate) async fn reset_idle(&mut self, stream: &Stream) { + let mut write_guard = self + .last_published_wm_state + .write() + .expect("Failed to get write lock"); + + let last_published_wm = write_guard + .get_mut(stream.vertex) + .unwrap_or_else(|| panic!("Invalid vertex: {}", stream.vertex)); + + last_published_wm[stream.partition as usize].last_wm_published_time = Utc::now(); + // setting None for wmb-offset means it is active + last_published_wm[stream.partition as usize].wmb_msg_offset = None; + } + + /// fetches the offset to be used for publishing the idle watermark. Only a WMB can be used + /// to send idle watermark, hence if not WMB's are published, we publish an WMB and return its + /// offset, or return the current "active" WMB's offset. + pub(crate) async fn fetch_idle_offset(&self, stream: &Stream) -> crate::error::Result { + let idle_state = { + let read_guard = self + .last_published_wm_state + .read() + .expect("Failed to get read lock"); + let last_published_wm = read_guard.get(stream.vertex).expect("Invalid vertex"); + last_published_wm[stream.partition as usize].clone() + }; + + if let Some(offset) = idle_state.wmb_msg_offset { + return Ok(offset); + } + + let ctrl_msg_bytes: BytesMut = crate::message::Message { + typ: crate::message::MessageType::WMB, + ..Default::default() + } + .try_into()?; + + let offset = self + .js_context + .publish(idle_state.stream.name, ctrl_msg_bytes.freeze()) + .await + .map_err(|e| crate::error::Error::Watermark(e.to_string()))? + .await + .map_err(|e| crate::error::Error::Watermark(e.to_string()))? + .sequence; + + Ok(offset as i64) + } + + /// updates the idle stream's metadata, by setting the ctrl message offset and updates the last published time. + pub(crate) async fn update_idle_metadata(&mut self, stream: &Stream, offset: i64) { + let mut write_guard = self + .last_published_wm_state + .write() + .expect("Failed to get write lock"); + let last_published_wm = write_guard.get_mut(stream.vertex).expect("Invalid vertex"); + // setting an offset for wmb-offset means it is idle and we will do inplace incr of WM for that offset. + last_published_wm[stream.partition as usize].wmb_msg_offset = Some(offset); + last_published_wm[stream.partition as usize].last_wm_published_time = Utc::now(); + } + + /// fetches the idle streams, we consider a stream as idle if the last published + /// time is greater than the idle timeout. + pub(crate) async fn fetch_idle_streams(&self) -> Vec { + let read_guard = self + .last_published_wm_state + .read() + .expect("Failed to get read lock"); + + read_guard + .iter() + .flat_map(|(_, partitions)| { + partitions + .iter() + .filter(|partition| { + Utc::now().timestamp_millis() + - partition.last_wm_published_time.timestamp_millis() + > self.idle_timeout.as_millis() as i64 + }) + .map(move |partition| partition.stream.clone()) + }) + .collect() + } + + /// fetch all the partitions for the vertices. + pub(crate) async fn fetch_all_streams(&self) -> Vec { + let read_guard = self + .last_published_wm_state + .read() + .expect("Failed to get read lock"); + + read_guard + .iter() + .flat_map(|(_, partitions)| partitions.iter().map(|partition| partition.stream.clone())) + .collect() + } +} + +#[cfg(test)] +mod tests { + use std::time::Duration; + + use async_nats::jetstream; + use async_nats::jetstream::stream; + use tokio::time::sleep; + + use super::*; + use crate::config::pipeline::isb::BufferWriterConfig; + use crate::config::pipeline::isb::Stream; + use crate::config::pipeline::ToVertexConfig; + + #[tokio::test] + async fn test_mark_active() { + let client = async_nats::connect("localhost:4222").await.unwrap(); + let js_context = jetstream::new(client); + + let stream = Stream::new("test_stream", "test_vertex", 0); + let to_vertex_config = ToVertexConfig { + name: "test_vertex", + partitions: 1, + writer_config: BufferWriterConfig { + streams: vec![stream.clone()], + ..Default::default() + }, + conditions: None, + }; + + let mut manager = + ISBIdleDetector::new(Duration::from_millis(100), &[to_vertex_config], js_context).await; + + manager.reset_idle(&stream).await; + + let read_guard = manager + .last_published_wm_state + .read() + .expect("Failed to get read lock"); + let idle_state = &read_guard["test_vertex"][0]; + assert_eq!(idle_state.stream, stream); + assert!(idle_state.wmb_msg_offset.is_none()); + } + + #[tokio::test] + async fn test_fetch_idle_offset() { + let client = async_nats::connect("localhost:4222").await.unwrap(); + let js_context = jetstream::new(client); + let stream = Stream::new("test_stream", "test_vertex", 0); + + // Delete stream if it exists + let _ = js_context.delete_stream(stream.name).await; + let _stream = js_context + .get_or_create_stream(stream::Config { + name: stream.name.to_string(), + subjects: vec![stream.name.to_string()], + ..Default::default() + }) + .await + .unwrap(); + + let to_vertex_config = ToVertexConfig { + name: "test_vertex", + partitions: 1, + writer_config: BufferWriterConfig { + streams: vec![stream.clone()], + ..Default::default() + }, + conditions: None, + }; + + let manager = + ISBIdleDetector::new(Duration::from_millis(100), &[to_vertex_config], js_context).await; + + let offset = manager + .fetch_idle_offset(&stream) + .await + .expect("Failed to fetch idle offset"); + assert!(offset > 0); + } + + #[tokio::test] + async fn test_mark_idle() { + let client = async_nats::connect("localhost:4222").await.unwrap(); + let js_context = jetstream::new(client); + + let stream = Stream::new("test_stream", "test_vertex", 0); + // Delete stream if it exists + let _ = js_context.delete_stream(stream.name).await; + let _stream = js_context + .get_or_create_stream(stream::Config { + name: stream.name.to_string(), + subjects: vec![stream.name.to_string()], + ..Default::default() + }) + .await + .unwrap(); + + let to_vertex_config = ToVertexConfig { + name: "test_vertex", + partitions: 1, + writer_config: BufferWriterConfig { + streams: vec![stream.clone()], + ..Default::default() + }, + conditions: None, + }; + + let mut manager = + ISBIdleDetector::new(Duration::from_millis(100), &[to_vertex_config], js_context).await; + + let offset = manager + .fetch_idle_offset(&stream) + .await + .expect("Failed to fetch idle offset"); + manager.update_idle_metadata(&stream, offset).await; + + let read_guard = manager + .last_published_wm_state + .read() + .expect("Failed to get read lock"); + let idle_state = &read_guard["test_vertex"][0]; + assert_eq!(idle_state.wmb_msg_offset, Some(offset)); + } + + #[tokio::test] + async fn test_fetch_idle_streams() { + let client = async_nats::connect("localhost:4222").await.unwrap(); + let js_context = jetstream::new(client); + + let stream = Stream::new("test_stream", "test_vertex", 0); + let to_vertex_config = ToVertexConfig { + name: "test_vertex", + partitions: 1, + writer_config: BufferWriterConfig { + streams: vec![stream.clone()], + ..Default::default() + }, + conditions: None, + }; + + let mut manager = + ISBIdleDetector::new(Duration::from_millis(10), &[to_vertex_config], js_context).await; + + // Mark the stream as active first + manager.reset_idle(&stream).await; + + // Simulate idle timeout + sleep(Duration::from_millis(20)).await; + + let idle_streams = manager.fetch_idle_streams().await; + assert_eq!(idle_streams.len(), 1); + assert_eq!(idle_streams[0], stream); + } +} diff --git a/rust/numaflow-core/src/watermark/idle/source.rs b/rust/numaflow-core/src/watermark/idle/source.rs new file mode 100644 index 0000000000..1405b67066 --- /dev/null +++ b/rust/numaflow-core/src/watermark/idle/source.rs @@ -0,0 +1,148 @@ +//! Source Idle Manager resolves the following conundrums: +//! +//! > How to decide if the source is idling? +//! +//! If the source is not reading any messages for threshold (provided by the user) +//! time, then it is considered idling. +//! +//! > When to publish the idle watermark? +//! +//! If the source is idling and the step interval has passed (also provided by the user). +//! +//! > What to publish as the idle watermark? +//! +//! The current watermark + increment_by (provided by the user). We will ensure that the +//! increment will never cross `(time.now() - max_delay)`. + +use chrono::{DateTime, Utc}; +use tracing::warn; + +use crate::config::pipeline::watermark::IdleConfig; + +/// Responsible for detecting the idle state of the source and publishing idle watermarks. +pub(crate) struct SourceIdleDetector { + config: IdleConfig, + /// last_idle_wm_published_time is for comparing with the step interval + last_idle_wm_published_time: DateTime, + updated_ts: DateTime, +} + +impl SourceIdleDetector { + /// Creates a new instance of SourceIdleManager. + pub fn new(config: IdleConfig) -> Self { + let default_time = DateTime::from_timestamp_millis(-1).expect("Invalid timestamp"); + SourceIdleDetector { + config, + updated_ts: Utc::now(), + last_idle_wm_published_time: default_time, + } + } + + /// Returns true if the source has been idling and the step interval has passed. + pub(crate) fn is_source_idling(&self) -> bool { + self.is_source_idling_internal() && self.has_step_interval_passed() + } + + /// Checks if the source is idling by comparing the last updated timestamp with the threshold. + fn is_source_idling_internal(&self) -> bool { + Utc::now().timestamp_millis() - self.updated_ts.timestamp_millis() + >= self.config.threshold.as_millis() as i64 + } + + /// Verifies if the step interval has passed. + fn has_step_interval_passed(&self) -> bool { + self.last_idle_wm_published_time.timestamp_millis() == -1 + || Utc::now().timestamp_millis() - self.last_idle_wm_published_time.timestamp_millis() + > self.config.step_interval.as_millis() as i64 + } + + /// Resets the updated_ts to the current time. + pub(crate) fn reset(&mut self) { + self.updated_ts = Utc::now(); + self.last_idle_wm_published_time = + DateTime::from_timestamp_millis(-1).expect("Invalid timestamp"); + } + + /// Updates and gets the idle watermark to be published. + pub(crate) fn update_and_fetch_idle_wm(&mut self, computed_wm: i64) -> i64 { + let increment_by = self.config.increment_by.as_millis() as i64; + // check if the computed watermark is -1 + // last computed watermark can be -1, when the pod is restarted or when the processor entity is not created yet. + if computed_wm == -1 { + return -1; + } + + let mut idle_wm = computed_wm + increment_by; + // do not assign future timestamps for WM. + // this could happen if step interval and increment-by are set aggressively + let now = Utc::now().timestamp_millis(); + if idle_wm > now { + warn!(?idle_wm, "idle config is aggressive (reduce step/increment-by), wm > now(), resetting to now()"); + idle_wm = now; + } + + self.last_idle_wm_published_time = Utc::now(); + idle_wm + } +} + +#[cfg(test)] +mod tests { + use std::time::Duration; + + use super::*; + use crate::config::pipeline::watermark::IdleConfig; + + #[test] + fn test_is_source_idling() { + let config = IdleConfig { + threshold: Duration::from_millis(100), + step_interval: Duration::from_millis(50), + increment_by: Duration::from_millis(10), + }; + let mut manager = SourceIdleDetector::new(config); + + // Initially, the source should not be idling + assert!(!manager.is_source_idling()); + + // Simulate the source idling by advancing the updated timestamp + manager.updated_ts = Utc::now() - chrono::Duration::milliseconds(200); + assert!(manager.is_source_idling()); + } + + #[test] + fn test_reset() { + let config = IdleConfig { + threshold: Duration::from_millis(100), + step_interval: Duration::from_millis(50), + increment_by: Duration::from_millis(10), + }; + let mut manager = SourceIdleDetector::new(config); + + // Simulate the source idling by advancing the updated timestamp + manager.updated_ts = Utc::now() - chrono::Duration::milliseconds(200); + assert!(manager.is_source_idling()); + + // Reset the manager and check if the source is no longer idling + manager.reset(); + assert!(!manager.is_source_idling()); + } + + #[test] + fn test_update_and_fetch_idle_wm() { + let config = IdleConfig { + threshold: Duration::from_millis(100), + step_interval: Duration::from_millis(50), + increment_by: Duration::from_millis(10), + }; + let mut manager = SourceIdleDetector::new(config); + + // Update and fetch idle watermark with computed_wm = -1 + let idle_wm = manager.update_and_fetch_idle_wm(-1); + assert_eq!(idle_wm, -1); + + // Update and fetch idle watermark with a valid computed_wm + let idle_wm = manager.update_and_fetch_idle_wm(1000); + assert_eq!(idle_wm, 1010); + } +} diff --git a/rust/numaflow-core/src/watermark/isb.rs b/rust/numaflow-core/src/watermark/isb.rs index 6521a42129..b1581d9d6e 100644 --- a/rust/numaflow-core/src/watermark/isb.rs +++ b/rust/numaflow-core/src/watermark/isb.rs @@ -6,6 +6,8 @@ //! increasing. Fetch and publish will be two different flows, but we will have natural ordering //! because we use actor model. Since we do streaming within the vertex we have to track the //! messages so that even if any messages get stuck we consider them while publishing watermarks. +//! Starts a background task to publish idle watermarks for the downstream idle partitions, idle +//! partitions are those partitions where we have not published any watermark for a certain time. //! //! //! **Fetch Flow** @@ -17,17 +19,19 @@ //! ```text //! (Write to ISB) -------> (Publish Watermark) ------> (Remove tracked Offset) //! ``` - use std::cmp::Ordering; use std::collections::{BTreeSet, HashMap}; +use std::time::Duration; use tokio::sync::mpsc::Receiver; use tracing::error; use crate::config::pipeline::isb::Stream; use crate::config::pipeline::watermark::EdgeWatermarkConfig; +use crate::config::pipeline::ToVertexConfig; use crate::error::{Error, Result}; use crate::message::{IntOffset, Offset}; +use crate::watermark::idle::isb::ISBIdleDetector; use crate::watermark::isb::wm_fetcher::ISBWatermarkFetcher; use crate::watermark::isb::wm_publisher::ISBWatermarkPublisher; use crate::watermark::processor::manager::ProcessorManager; @@ -51,10 +55,11 @@ enum ISBWaterMarkActorMessage { offset: IntOffset, watermark: Watermark, }, + CheckAndPublishIdleWatermark, } /// Tuple of offset and watermark. We will use this to track the inflight messages. -#[derive(Eq, PartialEq)] +#[derive(Eq, PartialEq, Debug)] struct OffsetWatermark { /// offset can be -1 if watermark cannot be derived. offset: i64, @@ -87,14 +92,20 @@ struct ISBWatermarkActor { /// though insertion and deletion are O(1). We do almost same amount insertion, deletion and /// getting the lowest watermark so BTreeSet is the best choice. offset_set: HashMap>, + idle_manager: ISBIdleDetector, } impl ISBWatermarkActor { - fn new(fetcher: ISBWatermarkFetcher, publisher: ISBWatermarkPublisher) -> Self { + fn new( + fetcher: ISBWatermarkFetcher, + publisher: ISBWatermarkPublisher, + idle_manager: ISBIdleDetector, + ) -> Self { Self { fetcher, publisher, offset_set: HashMap::new(), + idle_manager, } } @@ -114,8 +125,7 @@ impl ISBWatermarkActor { ISBWaterMarkActorMessage::FetchWatermark { offset, oneshot_tx } => { let watermark = self .fetcher - .fetch_watermark(offset.offset, offset.partition_idx) - .await?; + .fetch_watermark(offset.offset, offset.partition_idx); oneshot_tx .send(Ok(watermark)) @@ -130,8 +140,10 @@ impl ISBWatermarkActor { .unwrap_or(Watermark::from_timestamp_millis(-1).unwrap()); self.publisher - .publish_watermark(stream, offset.offset, min_wm.timestamp_millis()) - .await?; + .publish_watermark(&stream, offset.offset, min_wm.timestamp_millis(), false) + .await; + + self.idle_manager.reset_idle(&stream).await; } // removes the offset from the tracked offsets @@ -143,6 +155,42 @@ impl ISBWatermarkActor { ISBWaterMarkActorMessage::InsertOffset { offset, watermark } => { self.insert_offset(offset.partition_idx, offset.offset, watermark); } + + // check for idleness and publish idle watermark for those downstream idle partitions + ISBWaterMarkActorMessage::CheckAndPublishIdleWatermark => { + // if there are any inflight messages, consider the lowest watermark among them + let mut min_wm = self + .get_lowest_watermark() + .unwrap_or(Watermark::from_timestamp_millis(-1).expect("failed to parse time")); + + // if there are no inflight messages, use the head idle watermark + if min_wm.timestamp_millis() == -1 { + min_wm = self.fetcher.fetch_head_idle_watermark(); + } + + // we are not able to compute WM, pod restarts, etc. + if min_wm.timestamp_millis() == -1 { + return Ok(()); + } + + // identify the streams that are idle. + let idle_streams = self.idle_manager.fetch_idle_streams().await; + + // publish the idle watermark for the idle partitions + for stream in idle_streams.iter() { + let offset = match self.idle_manager.fetch_idle_offset(stream).await { + Ok(offset) => offset, + Err(e) => { + error!("failed to fetch idle offset: {:?}", e); + continue; + } + }; + self.publisher + .publish_watermark(stream, offset, min_wm.timestamp_millis(), true) + .await; + self.idle_manager.update_idle_metadata(stream, offset).await; + } + } } Ok(()) @@ -183,12 +231,14 @@ pub(crate) struct ISBWatermarkHandle { } impl ISBWatermarkHandle { - /// new creates a new [ISBWatermarkHandle]. + /// new creates a new [ISBWatermarkHandle]. We also start a background task to detect WM idleness. pub(crate) async fn new( vertex_name: &'static str, vertex_replica: u16, + idle_timeout: Duration, js_context: async_nats::jetstream::Context, config: &EdgeWatermarkConfig, + to_vertex_configs: &[ToVertexConfig], ) -> Result { let (sender, receiver) = tokio::sync::mpsc::channel(100); @@ -210,9 +260,31 @@ impl ISBWatermarkHandle { ) .await?; - let actor = ISBWatermarkActor::new(fetcher, publisher); + let idle_manager = + ISBIdleDetector::new(idle_timeout, to_vertex_configs, js_context.clone()).await; + + let actor = ISBWatermarkActor::new(fetcher, publisher, idle_manager); tokio::spawn(async move { actor.run(receiver).await }); - Ok(Self { sender }) + + let isb_watermark_handle = Self { sender }; + + // start a task to keep publishing idle watermarks every idle_timeout + tokio::spawn({ + let isb_watermark_handle = isb_watermark_handle.clone(); + let mut interval_ticker = tokio::time::interval(idle_timeout); + async move { + loop { + // TODO(idle): add cancellation token. + interval_ticker.tick().await; + isb_watermark_handle + .publish_idle_watermark() + .await + .expect("failed to publish idle watermark"); + } + } + }); + + Ok(isb_watermark_handle) } /// Fetches the watermark for the given offset. @@ -279,6 +351,14 @@ impl ISBWatermarkHandle { Err(Error::Watermark("invalid offset type".to_string())) } } + + pub(crate) async fn publish_idle_watermark(&self) -> Result<()> { + self.sender + .send(ISBWaterMarkActorMessage::CheckAndPublishIdleWatermark) + .await + .map_err(|_| Error::Watermark("failed to send message".to_string()))?; + Ok(()) + } } #[cfg(test)] @@ -288,7 +368,7 @@ mod tests { use tokio::time::sleep; use super::*; - use crate::config::pipeline::isb::Stream; + use crate::config::pipeline::isb::{BufferWriterConfig, Stream}; use crate::config::pipeline::watermark::BucketConfig; use crate::message::IntOffset; use crate::watermark::wmb::WMB; @@ -362,9 +442,24 @@ mod tests { to_vertex_config: vec![to_bucket_config.clone()], }; - let handle = ISBWatermarkHandle::new(vertex_name, 0, js_context.clone(), &edge_config) - .await - .expect("Failed to create ISBWatermarkHandle"); + let handle = ISBWatermarkHandle::new( + vertex_name, + 0, + Duration::from_millis(100), + js_context.clone(), + &edge_config, + &vec![ToVertexConfig { + name: "to_vertex", + partitions: 0, + writer_config: BufferWriterConfig { + streams: vec![Stream::new("test_stream", "to_vertex", 0)], + ..Default::default() + }, + conditions: None, + }], + ) + .await + .expect("Failed to create ISBWatermarkHandle"); handle .insert_offset( @@ -392,7 +487,7 @@ mod tests { .publish_watermark( Stream { name: "test_stream", - vertex: "from_vertex", + vertex: "to_vertex", partition: 0, }, Offset::Int(IntOffset { @@ -420,7 +515,7 @@ mod tests { assert_eq!(wmb.watermark, 100); wmb_found = true; } - sleep(std::time::Duration::from_millis(10)).await; + sleep(Duration::from_millis(10)).await; } if !wmb_found { @@ -440,7 +535,7 @@ mod tests { .publish_watermark( Stream { name: "test_stream", - vertex: "from_vertex", + vertex: "to_vertex", partition: 0, }, Offset::Int(IntOffset { @@ -463,7 +558,7 @@ mod tests { assert_eq!(wmb.watermark, 200); wmb_found = true; } - sleep(std::time::Duration::from_millis(10)).await; + sleep(Duration::from_millis(10)).await; } if !wmb_found { @@ -531,9 +626,24 @@ mod tests { to_vertex_config: vec![from_bucket_config.clone()], }; - let handle = ISBWatermarkHandle::new(vertex_name, 0, js_context.clone(), &edge_config) - .await - .expect("Failed to create ISBWatermarkHandle"); + let handle = ISBWatermarkHandle::new( + vertex_name, + 0, + Duration::from_millis(100), + js_context.clone(), + &edge_config, + &vec![ToVertexConfig { + name: "from_vertex", + partitions: 0, + writer_config: BufferWriterConfig { + streams: vec![Stream::new("test_stream", "from_vertex", 0)], + ..Default::default() + }, + conditions: None, + }], + ) + .await + .expect("Failed to create ISBWatermarkHandle"); let mut fetched_watermark = -1; // publish watermark and try fetching to see if something is getting published @@ -578,7 +688,7 @@ mod tests { fetched_watermark = watermark.timestamp_millis(); break; } - sleep(std::time::Duration::from_millis(10)).await; + sleep(Duration::from_millis(10)).await; handle .remove_offset(offset.clone()) .await @@ -597,4 +707,152 @@ mod tests { .await .unwrap(); } + + #[cfg(feature = "nats-tests")] + #[tokio::test] + async fn test_publish_idle_watermark() { + let client = async_nats::connect("localhost:4222").await.unwrap(); + let js_context = jetstream::new(client); + + let ot_bucket_name = "test_publish_idle_watermark_OT"; + let hb_bucket_name = "test_publish_idle_watermark_PROCESSORS"; + let to_ot_bucket_name = "test_publish_idle_watermark_to_OT"; + let to_hb_bucket_name = "test_publish_idle_watermark_to_PROCESSORS"; + + let vertex_name = "test-vertex"; + + let from_bucket_config = BucketConfig { + vertex: "from_vertex", + partitions: 1, + ot_bucket: ot_bucket_name, + hb_bucket: hb_bucket_name, + }; + + let to_bucket_config = BucketConfig { + vertex: "to_vertex", + partitions: 1, + ot_bucket: to_ot_bucket_name, + hb_bucket: to_hb_bucket_name, + }; + + // create key value stores + js_context + .create_key_value(Config { + bucket: ot_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + + js_context + .create_key_value(Config { + bucket: hb_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + + js_context + .create_key_value(Config { + bucket: to_ot_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + + js_context + .create_key_value(Config { + bucket: to_hb_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + + let edge_config = EdgeWatermarkConfig { + from_vertex_config: vec![from_bucket_config.clone()], + to_vertex_config: vec![to_bucket_config.clone()], + }; + + let handle = ISBWatermarkHandle::new( + vertex_name, + 0, + Duration::from_millis(10), // Set idle timeout to a very short duration + js_context.clone(), + &edge_config, + &vec![ToVertexConfig { + name: "to_vertex", + partitions: 0, + writer_config: BufferWriterConfig { + streams: vec![Stream::new("test_stream", "to_vertex", 0)], + ..Default::default() + }, + conditions: None, + }], + ) + .await + .expect("Failed to create ISBWatermarkHandle"); + + // Insert multiple offsets + for i in 1..=3 { + handle + .insert_offset( + Offset::Int(IntOffset { + offset: i, + partition_idx: 0, + }), + Some(Watermark::from_timestamp_millis(i * 100).unwrap()), + ) + .await + .expect("Failed to insert offset"); + } + + // Wait for the idle timeout to trigger + tokio::time::sleep(Duration::from_millis(20)).await; + + // Check if the idle watermark is published + let ot_bucket = js_context + .get_key_value(to_ot_bucket_name) + .await + .expect("Failed to get ot bucket"); + + let mut wmb_found = false; + for _ in 0..10 { + if let Some(wmb) = ot_bucket + .get("test-vertex-0") + .await + .expect("Failed to get wmb") + { + let wmb: WMB = wmb.try_into().unwrap(); + if wmb.idle { + wmb_found = true; + break; + } + } + tokio::time::sleep(Duration::from_millis(10)).await; + } + + assert!(wmb_found, "Idle watermark not found"); + + // delete the stores + js_context + .delete_key_value(ot_bucket_name.to_string()) + .await + .unwrap(); + js_context + .delete_key_value(hb_bucket_name.to_string()) + .await + .unwrap(); + js_context + .delete_key_value(to_ot_bucket_name.to_string()) + .await + .unwrap(); + js_context + .delete_key_value(to_hb_bucket_name.to_string()) + .await + .unwrap(); + } } diff --git a/rust/numaflow-core/src/watermark/isb/wm_fetcher.rs b/rust/numaflow-core/src/watermark/isb/wm_fetcher.rs index eea2e5ded7..42d8b93908 100644 --- a/rust/numaflow-core/src/watermark/isb/wm_fetcher.rs +++ b/rust/numaflow-core/src/watermark/isb/wm_fetcher.rs @@ -9,7 +9,7 @@ use std::collections::HashMap; use crate::config::pipeline::watermark::BucketConfig; -use crate::error::{Error, Result}; +use crate::error::Result; use crate::watermark::processor::manager::ProcessorManager; use crate::watermark::wmb::Watermark; @@ -44,11 +44,7 @@ impl ISBWatermarkFetcher { } /// Fetches the watermark for the given offset and partition. - pub(crate) async fn fetch_watermark( - &mut self, - offset: i64, - partition_idx: u16, - ) -> Result { + pub(crate) fn fetch_watermark(&mut self, offset: i64, partition_idx: u16) -> Watermark { // Iterate over all the processor managers and get the smallest watermark. (join case) for (edge, processor_manager) in self.processor_managers.iter() { let mut epoch = i64::MAX; @@ -107,62 +103,55 @@ impl ISBWatermarkFetcher { /// Fetches the latest idle WMB with the smallest watermark for the given partition /// Only returns one if all Publishers are idle and if it's the smallest one of any partitions - #[allow(dead_code)] - pub(crate) async fn fetch_head_idle_watermark(&mut self, partition_idx: u16) -> Result { + pub(crate) fn fetch_head_idle_watermark(&mut self) -> Watermark { let mut min_wm = i64::MAX; - for (edge, processor_manager) in self.processor_managers.iter() { + + for (edge, processor_manager) in &self.processor_managers { let mut epoch = i64::MAX; - for processor in processor_manager + + let processors = processor_manager .processors .read() - .expect("failed to acquire lock") - .values() - { - // if the processor is not active, skip - if !processor.is_active() { - continue; - } + .expect("failed to acquire lock"); - // retrieve the head watermark of the partition, we only care about the head watermark - // because by looking at the head wmb we will know whether the processor is idle or not - let head_wmb = processor - .timelines - .get(partition_idx as usize) - .ok_or(Error::Watermark("Partition not found".to_string()))? - .get_head_wmb(); - - if let Some(wmb) = head_wmb { - // if the processor is not idle, return early - if !wmb.idle { - return Ok(-1); - } - // consider the smallest watermark among all the partitions - if wmb.watermark < epoch { - epoch = wmb.watermark; + let active_processors = processors + .values() + .filter(|processor| processor.is_active()); + + for processor in active_processors { + for timeline in &processor.timelines { + if let Some(head_wmb) = timeline.get_head_wmb() { + // if the processor is not idle, return early + if !head_wmb.idle { + return Watermark::from_timestamp_millis(-1) + .expect("failed to parse time"); + } + // consider the smallest watermark among all the partitions + epoch = epoch.min(head_wmb.watermark); } } } - if epoch < min_wm { - min_wm = epoch; - } - - if epoch != i64::MAX { - // update the last processed watermark for this particular edge and the partition - self.last_processed_wm.get_mut(edge).expect("invalid edge") - [partition_idx as usize] = epoch; + if epoch < i64::MAX { + min_wm = min_wm.min(epoch); + // update the last processed watermark for this particular edge and all the partitions + self.last_processed_wm + .get_mut(edge) + .unwrap_or_else(|| panic!("invalid vertex {}", edge)) + .iter_mut() + .for_each(|partition| *partition = epoch); } } if min_wm == i64::MAX { - return Ok(-1); + min_wm = -1; } - Ok(min_wm) + Watermark::from_timestamp_millis(min_wm).expect("failed to parse time") } /// returns the smallest last processed watermark among all the partitions - fn get_watermark(&self) -> Result { + fn get_watermark(&self) -> Watermark { let mut min_wm = i64::MAX; for wm in self.last_processed_wm.values() { for &w in wm { @@ -173,9 +162,9 @@ impl ISBWatermarkFetcher { } if min_wm == i64::MAX { - return Ok(Watermark::from_timestamp_millis(-1).expect("failed to parse time")); + return Watermark::from_timestamp_millis(-1).expect("failed to parse time"); } - Ok(Watermark::from_timestamp_millis(min_wm).expect("failed to parse time")) + Watermark::from_timestamp_millis(min_wm).expect("failed to parse time") } } @@ -183,9 +172,9 @@ impl ISBWatermarkFetcher { mod tests { use std::collections::HashMap; use std::sync::Arc; + use std::sync::RwLock; use bytes::Bytes; - use std::sync::RwLock; use super::*; use crate::watermark::processor::manager::{Processor, Status}; @@ -248,7 +237,7 @@ mod tests { .unwrap(); // Invoke fetch_watermark and verify the result - let watermark = fetcher.fetch_watermark(2, 0).await.unwrap(); + let watermark = fetcher.fetch_watermark(2, 0); assert_eq!(watermark.timestamp_millis(), 100); } @@ -386,7 +375,7 @@ mod tests { .unwrap(); // Invoke fetch_watermark and verify the result - let watermark = fetcher.fetch_watermark(12, 0).await.unwrap(); + let watermark = fetcher.fetch_watermark(12, 0); assert_eq!(watermark.timestamp_millis(), 150); } @@ -618,11 +607,11 @@ mod tests { // Invoke fetch_watermark and verify the result for partition 0, first fetch will be -1 because we have not fetched for other // partition (we consider min across the last fetched watermark) - let watermark_p0 = fetcher.fetch_watermark(12, 0).await.unwrap(); + let watermark_p0 = fetcher.fetch_watermark(12, 0); assert_eq!(watermark_p0.timestamp_millis(), -1); // Invoke fetch_watermark and verify the result for partition 1 (we consider min across the last fetch wm for all partitions) - let watermark_p1 = fetcher.fetch_watermark(32, 1).await.unwrap(); + let watermark_p1 = fetcher.fetch_watermark(32, 1); assert_eq!(watermark_p1.timestamp_millis(), 150); } @@ -845,11 +834,538 @@ mod tests { .unwrap(); // Invoke fetch_watermark and verify the result for partition 0 - let watermark_p0 = fetcher.fetch_watermark(12, 0).await.unwrap(); + let watermark_p0 = fetcher.fetch_watermark(12, 0); assert_eq!(watermark_p0.timestamp_millis(), -1); // Invoke fetch_watermark and verify the result for partition 1 - let watermark_p1 = fetcher.fetch_watermark(32, 1).await.unwrap(); + let watermark_p1 = fetcher.fetch_watermark(32, 1); assert_eq!(watermark_p1.timestamp_millis(), 150); } + + #[tokio::test] + async fn test_fetch_head_idle_watermark_single_edge_single_processor_single_partition() { + // Create a ProcessorManager with a single Processor and a single OffsetTimeline + let processor_name = Bytes::from("processor1"); + let mut processor = Processor::new(processor_name.clone(), Status::Active, 1); + let timeline = OffsetTimeline::new(10); + + // Populate the OffsetTimeline with sorted WMB entries + let wmb1 = WMB { + watermark: 100, + offset: 1, + idle: true, + partition: 0, + }; + let wmb2 = WMB { + watermark: 200, + offset: 2, + idle: true, + partition: 0, + }; + + timeline.put(wmb1); + timeline.put(wmb2); + + processor.timelines[0] = timeline; + + let mut processors = HashMap::new(); + processors.insert(processor_name.clone(), processor); + + let processor_manager = ProcessorManager { + processors: Arc::new(RwLock::new(processors)), + handles: vec![], + }; + + let mut processor_managers = HashMap::new(); + processor_managers.insert("from_vtx", processor_manager); + + let bucket_config = BucketConfig { + vertex: "from_vtx", + ot_bucket: "ot_bucket", + hb_bucket: "hb_bucket", + partitions: 1, + }; + + let mut fetcher = ISBWatermarkFetcher::new(processor_managers, &[bucket_config]) + .await + .unwrap(); + + // Invoke fetch_head_idle_watermark and verify the result + let watermark = fetcher.fetch_head_idle_watermark(); + assert_eq!(watermark.timestamp_millis(), 200); + } + + #[tokio::test] + async fn test_fetch_head_idle_watermark_single_edge_multi_processor_single_partition() { + // Create ProcessorManager with multiple Processors and different OffsetTimelines + let processor_name1 = Bytes::from("processor1"); + let processor_name2 = Bytes::from("processor2"); + + let mut processor1 = Processor::new(processor_name1.clone(), Status::Active, 1); + let mut processor2 = Processor::new(processor_name2.clone(), Status::Active, 1); + + let timeline1 = OffsetTimeline::new(10); + let timeline2 = OffsetTimeline::new(10); + + // Populate the OffsetTimelines with sorted WMB entries + let wmbs1 = vec![ + WMB { + watermark: 100, + offset: 5, + idle: true, + partition: 0, + }, + WMB { + watermark: 150, + offset: 10, + idle: true, + partition: 0, + }, + ]; + let wmbs2 = vec![ + WMB { + watermark: 110, + offset: 3, + idle: true, + partition: 0, + }, + WMB { + watermark: 160, + offset: 8, + idle: true, + partition: 0, + }, + ]; + + for wmb in wmbs1 { + timeline1.put(wmb); + } + for wmb in wmbs2 { + timeline2.put(wmb); + } + + processor1.timelines[0] = timeline1; + processor2.timelines[0] = timeline2; + + let mut processors = HashMap::new(); + processors.insert(processor_name1.clone(), processor1); + processors.insert(processor_name2.clone(), processor2); + + let processor_manager = ProcessorManager { + processors: Arc::new(RwLock::new(processors)), + handles: vec![], + }; + + let mut processor_managers = HashMap::new(); + processor_managers.insert("from_vtx", processor_manager); + + let bucket_config = BucketConfig { + vertex: "from_vtx", + ot_bucket: "ot_bucket", + hb_bucket: "hb_bucket", + partitions: 1, + }; + + let mut fetcher = ISBWatermarkFetcher::new(processor_managers, &[bucket_config]) + .await + .unwrap(); + + // Invoke fetch_head_idle_watermark and verify the result + let watermark = fetcher.fetch_head_idle_watermark(); + assert_eq!(watermark.timestamp_millis(), 150); + } + + #[tokio::test] + async fn test_fetch_head_idle_watermark_single_edge_multi_processor_multi_partition() { + // Create ProcessorManager with multiple Processors and different OffsetTimelines + let processor_name1 = Bytes::from("processor1"); + let processor_name2 = Bytes::from("processor2"); + + let mut processor1 = Processor::new(processor_name1.clone(), Status::Active, 2); + let mut processor2 = Processor::new(processor_name2.clone(), Status::Active, 2); + + let timeline1_p0 = OffsetTimeline::new(10); + let timeline1_p1 = OffsetTimeline::new(10); + let timeline2_p0 = OffsetTimeline::new(10); + let timeline2_p1 = OffsetTimeline::new(10); + + // Populate the OffsetTimelines with sorted WMB entries + let wmbs1_p0 = vec![ + WMB { + watermark: 100, + offset: 6, + idle: true, + partition: 0, + }, + WMB { + watermark: 150, + offset: 10, + idle: true, + partition: 0, + }, + ]; + let wmbs1_p1 = vec![ + WMB { + watermark: 110, + offset: 25, + idle: true, + partition: 1, + }, + WMB { + watermark: 160, + offset: 30, + idle: true, + partition: 1, + }, + ]; + let wmbs2_p0 = vec![ + WMB { + watermark: 120, + offset: 3, + idle: true, + partition: 0, + }, + WMB { + watermark: 170, + offset: 8, + idle: true, + partition: 0, + }, + ]; + let wmbs2_p1 = vec![ + WMB { + watermark: 130, + offset: 23, + idle: true, + partition: 1, + }, + WMB { + watermark: 180, + offset: 28, + idle: true, + partition: 1, + }, + ]; + + for wmb in wmbs1_p0 { + timeline1_p0.put(wmb); + } + for wmb in wmbs1_p1 { + timeline1_p1.put(wmb); + } + for wmb in wmbs2_p0 { + timeline2_p0.put(wmb); + } + for wmb in wmbs2_p1 { + timeline2_p1.put(wmb); + } + + processor1.timelines[0] = timeline1_p0; + processor1.timelines[1] = timeline1_p1; + processor2.timelines[0] = timeline2_p0; + processor2.timelines[1] = timeline2_p1; + + let mut processors = HashMap::new(); + processors.insert(processor_name1.clone(), processor1); + processors.insert(processor_name2.clone(), processor2); + + let processor_manager = ProcessorManager { + processors: Arc::new(RwLock::new(processors)), + handles: vec![], + }; + + let mut processor_managers = HashMap::new(); + processor_managers.insert("from_vtx", processor_manager); + + let bucket_config = BucketConfig { + vertex: "from_vtx", + ot_bucket: "ot_bucket", + hb_bucket: "hb_bucket", + partitions: 2, + }; + + let mut fetcher = ISBWatermarkFetcher::new(processor_managers, &[bucket_config]) + .await + .unwrap(); + + // Invoke fetch_head_idle_watermark and verify the result + let watermark = fetcher.fetch_head_idle_watermark(); + assert_eq!(watermark.timestamp_millis(), 150); + } + + #[tokio::test] + async fn test_fetch_head_idle_watermark_two_edges_multi_processor_multi_partition() { + // Create ProcessorManagers with multiple Processors and different OffsetTimelines for edge1 + let processor_name1_edge1 = Bytes::from("processor1_edge1"); + let processor_name2_edge1 = Bytes::from("processor2_edge1"); + + let mut processor1_edge1 = Processor::new(processor_name1_edge1.clone(), Status::Active, 2); + let mut processor2_edge1 = Processor::new(processor_name2_edge1.clone(), Status::Active, 2); + + let timeline1_p0_edge1 = OffsetTimeline::new(10); + let timeline1_p1_edge1 = OffsetTimeline::new(10); + let timeline2_p0_edge1 = OffsetTimeline::new(10); + let timeline2_p1_edge1 = OffsetTimeline::new(10); + + // Populate the OffsetTimelines with sorted WMB entries + let wmbs1_p0_edge1 = vec![ + WMB { + watermark: 100, + offset: 6, + idle: true, + partition: 0, + }, + WMB { + watermark: 150, + offset: 10, + idle: true, + partition: 0, + }, + ]; + let wmbs1_p1_edge1 = vec![ + WMB { + watermark: 110, + offset: 25, + idle: true, + partition: 1, + }, + WMB { + watermark: 160, + offset: 30, + idle: true, + partition: 1, + }, + ]; + let wmbs2_p0_edge1 = vec![ + WMB { + watermark: 120, + offset: 3, + idle: true, + partition: 0, + }, + WMB { + watermark: 170, + offset: 8, + idle: true, + partition: 0, + }, + ]; + let wmbs2_p1_edge1 = vec![ + WMB { + watermark: 130, + offset: 23, + idle: true, + partition: 1, + }, + WMB { + watermark: 180, + offset: 28, + idle: true, + partition: 1, + }, + ]; + + for wmb in wmbs1_p0_edge1 { + timeline1_p0_edge1.put(wmb); + } + for wmb in wmbs1_p1_edge1 { + timeline1_p1_edge1.put(wmb); + } + for wmb in wmbs2_p0_edge1 { + timeline2_p0_edge1.put(wmb); + } + for wmb in wmbs2_p1_edge1 { + timeline2_p1_edge1.put(wmb); + } + + processor1_edge1.timelines[0] = timeline1_p0_edge1; + processor1_edge1.timelines[1] = timeline1_p1_edge1; + processor2_edge1.timelines[0] = timeline2_p0_edge1; + processor2_edge1.timelines[1] = timeline2_p1_edge1; + + let mut processors_edge1 = HashMap::new(); + processors_edge1.insert(processor_name1_edge1.clone(), processor1_edge1); + processors_edge1.insert(processor_name2_edge1.clone(), processor2_edge1); + + let processor_manager_edge1 = ProcessorManager { + processors: Arc::new(RwLock::new(processors_edge1)), + handles: vec![], + }; + + // Create ProcessorManagers with multiple Processors and different OffsetTimelines for edge2 + let processor_name1_edge2 = Bytes::from("processor1_edge2"); + let processor_name2_edge2 = Bytes::from("processor2_edge2"); + + let mut processor1_edge2 = Processor::new(processor_name1_edge2.clone(), Status::Active, 2); + let mut processor2_edge2 = Processor::new(processor_name2_edge2.clone(), Status::Active, 2); + + let timeline1_p0_edge2 = OffsetTimeline::new(10); + let timeline1_p1_edge2 = OffsetTimeline::new(10); + let timeline2_p0_edge2 = OffsetTimeline::new(10); + let timeline2_p1_edge2 = OffsetTimeline::new(10); + + // Populate the OffsetTimelines with sorted WMB entries + let wmbs1_p0_edge2 = vec![ + WMB { + watermark: 140, + offset: 2, + idle: true, + partition: 0, + }, + WMB { + watermark: 190, + offset: 7, + idle: true, + partition: 0, + }, + ]; + let wmbs1_p1_edge2 = vec![ + WMB { + watermark: 150, + offset: 22, + idle: true, + partition: 1, + }, + WMB { + watermark: 200, + offset: 27, + idle: true, + partition: 1, + }, + ]; + let wmbs2_p0_edge2 = vec![ + WMB { + watermark: 160, + offset: 4, + idle: true, + partition: 0, + }, + WMB { + watermark: 210, + offset: 9, + idle: true, + partition: 0, + }, + ]; + let wmbs2_p1_edge2 = vec![ + WMB { + watermark: 170, + offset: 24, + idle: true, + partition: 1, + }, + WMB { + watermark: 220, + offset: 29, + idle: true, + partition: 1, + }, + ]; + + for wmb in wmbs1_p0_edge2 { + timeline1_p0_edge2.put(wmb); + } + for wmb in wmbs1_p1_edge2 { + timeline1_p1_edge2.put(wmb); + } + for wmb in wmbs2_p0_edge2 { + timeline2_p0_edge2.put(wmb); + } + for wmb in wmbs2_p1_edge2 { + timeline2_p1_edge2.put(wmb); + } + + processor1_edge2.timelines[0] = timeline1_p0_edge2; + processor1_edge2.timelines[1] = timeline1_p1_edge2; + processor2_edge2.timelines[0] = timeline2_p0_edge2; + processor2_edge2.timelines[1] = timeline2_p1_edge2; + + let mut processors_edge2 = HashMap::new(); + processors_edge2.insert(processor_name1_edge2.clone(), processor1_edge2); + processors_edge2.insert(processor_name2_edge2.clone(), processor2_edge2); + + let processor_manager_edge2 = ProcessorManager { + processors: Arc::new(RwLock::new(processors_edge2)), + handles: vec![], + }; + + let mut processor_managers = HashMap::new(); + processor_managers.insert("edge1", processor_manager_edge1); + processor_managers.insert("edge2", processor_manager_edge2); + + let bucket_config1 = BucketConfig { + vertex: "edge1", + ot_bucket: "ot_bucket1", + hb_bucket: "hb_bucket1", + partitions: 2, + }; + let bucket_config2 = BucketConfig { + vertex: "edge2", + ot_bucket: "ot_bucket2", + hb_bucket: "hb_bucket2", + partitions: 2, + }; + + let mut fetcher = + ISBWatermarkFetcher::new(processor_managers, &[bucket_config1, bucket_config2]) + .await + .unwrap(); + + // Invoke fetch_head_idle_watermark and verify the result + let watermark = fetcher.fetch_head_idle_watermark(); + assert_eq!(watermark.timestamp_millis(), 150); + } + + #[tokio::test] + async fn test_fetch_head_idle_watermark_not_idle() { + // Create a ProcessorManager with a single Processor and a single OffsetTimeline + let processor_name = Bytes::from("processor1"); + let mut processor = Processor::new(processor_name.clone(), Status::Active, 1); + let timeline = OffsetTimeline::new(10); + + // Populate the OffsetTimeline with sorted WMB entries + let wmb1 = WMB { + watermark: 100, + offset: 1, + idle: true, + partition: 0, + }; + let wmb2 = WMB { + watermark: 200, + offset: 2, + idle: false, // Not idle + partition: 0, + }; + + timeline.put(wmb1); + timeline.put(wmb2); + + processor.timelines[0] = timeline; + + let mut processors = HashMap::new(); + processors.insert(processor_name.clone(), processor); + + let processor_manager = ProcessorManager { + processors: Arc::new(RwLock::new(processors)), + handles: vec![], + }; + + let mut processor_managers = HashMap::new(); + processor_managers.insert("from_vtx", processor_manager); + + let bucket_config = BucketConfig { + vertex: "from_vtx", + ot_bucket: "ot_bucket", + hb_bucket: "hb_bucket", + partitions: 1, + }; + + let mut fetcher = ISBWatermarkFetcher::new(processor_managers, &[bucket_config]) + .await + .unwrap(); + + // Invoke fetch_head_idle_watermark and verify the result + let watermark = fetcher.fetch_head_idle_watermark(); + assert_eq!(watermark.timestamp_millis(), -1); + } } diff --git a/rust/numaflow-core/src/watermark/isb/wm_publisher.rs b/rust/numaflow-core/src/watermark/isb/wm_publisher.rs index 6b5d3e85e5..4d1fe3d92b 100644 --- a/rust/numaflow-core/src/watermark/isb/wm_publisher.rs +++ b/rust/numaflow-core/src/watermark/isb/wm_publisher.rs @@ -9,7 +9,7 @@ use std::time::{Duration, SystemTime}; use bytes::BytesMut; use prost::Message; -use tracing::info; +use tracing::{debug, error, info}; use crate::config::pipeline::isb::Stream; use crate::config::pipeline::watermark::BucketConfig; @@ -27,6 +27,15 @@ struct LastPublishedState { watermark: i64, } +impl Default for LastPublishedState { + fn default() -> Self { + LastPublishedState { + offset: -1, + watermark: -1, + } + } +} + /// ISBWatermarkPublisher is the watermark publisher for the outgoing edges. pub(crate) struct ISBWatermarkPublisher { /// name of the processor(node) that is publishing the watermark. @@ -73,13 +82,7 @@ impl ISBWatermarkPublisher { hb_buckets.push(hb_bucket); last_published_wm.insert( config.vertex, - vec![ - LastPublishedState { - offset: -1, - watermark: -1, - }; - config.partitions as usize - ], + vec![LastPublishedState::default(); config.partitions as usize], ); } @@ -117,13 +120,14 @@ impl ISBWatermarkPublisher { .expect("Failed to encode heartbeat"); for hb_bucket in hb_buckets.iter() { - info!(heartbeat = ?heartbeat.heartbeat, processor = ?processor_name, + debug!(heartbeat = ?heartbeat.heartbeat, processor = ?processor_name, "Publishing heartbeat", ); hb_bucket .put(processor_name.clone(), bytes.clone().freeze()) .await - .expect("Failed to publish heartbeat"); + .map_err(|e| error!(?e, "Failed to write heartbeat to hb bucket")) + .ok(); } } } @@ -131,43 +135,44 @@ impl ISBWatermarkPublisher { /// publish_watermark publishes the watermark for the given offset and the stream. pub(crate) async fn publish_watermark( &mut self, - stream: Stream, + stream: &Stream, offset: i64, watermark: i64, - ) -> Result<()> { - let last_published_wm_state = match self.last_published_wm.get_mut(stream.vertex) { - Some(wm) => wm, - None => return Err(Error::Watermark("Invalid vertex".to_string())), - }; + idle: bool, + ) { + let last_published_wm_state = self + .last_published_wm + .get_mut(stream.vertex) + .expect("Invalid vertex, no last published watermark state found"); // we can avoid publishing the watermark if it is <= the last published watermark (optimization) let last_state = &last_published_wm_state[stream.partition as usize]; - if offset <= last_state.offset || watermark <= last_state.watermark { - return Ok(()); + if offset < last_state.offset || watermark <= last_state.watermark { + return; } - let ot_bucket = self.ot_buckets.get(stream.vertex).ok_or(Error::Watermark( - "Invalid vertex, no ot bucket found".to_string(), - ))?; + let ot_bucket = self.ot_buckets.get(stream.vertex).expect("Invalid vertex"); let wmb_bytes: BytesMut = WMB { - idle: false, + idle, offset, watermark, partition: stream.partition, } .try_into() - .map_err(|e| Error::Watermark(format!("{}", e)))?; + .expect("Failed to convert WMB to bytes"); + + // ot writes can fail when isb is not healthy, we can ignore since subsequent writes will + // go through ot_bucket .put(self.processor_name.clone(), wmb_bytes.freeze()) .await - .map_err(|e| Error::Watermark(e.to_string()))?; + .map_err(|e| error!("Failed to write wmb to ot bucket: {}", e)) + .ok(); // update the last published watermark state last_published_wm_state[stream.partition as usize] = LastPublishedState { offset, watermark }; - - Ok(()) } } @@ -237,9 +242,8 @@ mod tests { // Publish watermark for partition 0 publisher - .publish_watermark(stream_partition_0.clone(), 1, 100) - .await - .expect("Failed to publish watermark"); + .publish_watermark(&stream_partition_0, 1, 100, false) + .await; let ot_bucket = js_context .get_key_value("isb_publisher_one_edge_OT") @@ -258,9 +262,8 @@ mod tests { // Try publishing a smaller watermark for the same partition, it should not be published publisher - .publish_watermark(stream_partition_0.clone(), 0, 50) - .await - .expect("Failed to publish watermark"); + .publish_watermark(&stream_partition_0, 0, 50, false) + .await; let wmb = ot_bucket .get("processor1") @@ -274,9 +277,8 @@ mod tests { // Publish a smaller watermark for a different partition, it should be published publisher - .publish_watermark(stream_partition_1.clone(), 0, 50) - .await - .expect("Failed to publish watermark"); + .publish_watermark(&stream_partition_1, 0, 50, false) + .await; let wmb = ot_bucket .get("processor1") @@ -381,15 +383,9 @@ mod tests { partition: 0, }; - publisher - .publish_watermark(stream1.clone(), 1, 100) - .await - .expect("Failed to publish watermark"); + publisher.publish_watermark(&stream1, 1, 100, false).await; - publisher - .publish_watermark(stream2.clone(), 1, 200) - .await - .expect("Failed to publish watermark"); + publisher.publish_watermark(&stream2, 1, 200, false).await; let ot_bucket_v1 = js_context .get_key_value(ot_bucket_name_v1) @@ -439,4 +435,82 @@ mod tests { .await .unwrap(); } + + #[cfg(feature = "nats-tests")] + #[tokio::test] + async fn test_isb_publisher_idle_flag() { + let client = async_nats::connect("localhost:4222").await.unwrap(); + let js_context = jetstream::new(client); + + let ot_bucket_name = "isb_publisher_idle_flag_OT"; + let hb_bucket_name = "isb_publisher_idle_flag_PROCESSORS"; + + let bucket_configs = vec![BucketConfig { + vertex: "v1", + partitions: 1, + ot_bucket: ot_bucket_name, + hb_bucket: hb_bucket_name, + }]; + + // create key value stores + js_context + .create_key_value(Config { + bucket: ot_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + js_context + .create_key_value(Config { + bucket: hb_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + + let mut publisher = ISBWatermarkPublisher::new( + "processor1".to_string(), + js_context.clone(), + &bucket_configs, + ) + .await + .expect("Failed to create publisher"); + + let stream = Stream { + name: "v1-0", + vertex: "v1", + partition: 0, + }; + + // Publish watermark with idle flag set to true + publisher.publish_watermark(&stream, 1, 100, true).await; + + let ot_bucket = js_context + .get_key_value(ot_bucket_name) + .await + .expect("Failed to get ot bucket"); + + let wmb = ot_bucket + .get("processor1") + .await + .expect("Failed to get wmb"); + assert!(wmb.is_some()); + + let wmb: WMB = wmb.unwrap().try_into().unwrap(); + assert_eq!(wmb.offset, 1); + assert_eq!(wmb.watermark, 100); + assert!(wmb.idle); + + // delete the stores + js_context + .delete_key_value(hb_bucket_name.to_string()) + .await + .unwrap(); + js_context + .delete_key_value(ot_bucket_name.to_string()) + .await + .unwrap(); + } } diff --git a/rust/numaflow-core/src/watermark/processor/manager.rs b/rust/numaflow-core/src/watermark/processor/manager.rs index 33f1a9ea0e..094416c7b0 100644 --- a/rust/numaflow-core/src/watermark/processor/manager.rs +++ b/rust/numaflow-core/src/watermark/processor/manager.rs @@ -5,6 +5,7 @@ use std::collections::HashMap; use std::fmt::{Debug, Formatter}; use std::sync::Arc; +use std::sync::RwLock; use std::time::Duration; use std::time::SystemTime; @@ -12,9 +13,8 @@ use async_nats::jetstream::kv::Watch; use backoff::retry::Retry; use backoff::strategy::fixed; use bytes::Bytes; -use futures::StreamExt; +use futures::{StreamExt, TryStreamExt}; use prost::Message as ProtoMessage; -use std::sync::RwLock; use tracing::{debug, error, info, warn}; use crate::config::pipeline::watermark::BucketConfig; @@ -112,7 +112,8 @@ impl Drop for ProcessorManager { } impl ProcessorManager { - /// Creates a new ProcessorManager. + /// Creates a new ProcessorManager. It prepopulates the processor-map with previous data + /// fetched from the OT and HB buckets. pub(crate) async fn new( js_context: async_nats::jetstream::Context, bucket_config: &BucketConfig, @@ -137,8 +138,12 @@ impl ProcessorManager { )) })?; - let processors = Arc::new(RwLock::new(HashMap::new())); - let heartbeats = Arc::new(RwLock::new(HashMap::new())); + // fetch old data + let (processors_map, heartbeats_map) = + Self::prepopulate_processors(&hb_bucket, &ot_bucket, bucket_config).await; + // point to populated data + let processors = Arc::new(RwLock::new(processors_map)); + let heartbeats = Arc::new(RwLock::new(heartbeats_map)); // start the ot watcher, to listen to the OT bucket and update the timelines let ot_handle = tokio::spawn(Self::start_ot_watcher(ot_bucket, Arc::clone(&processors))); @@ -166,6 +171,80 @@ impl ProcessorManager { }) } + /// Prepopulate processors and timelines from the hb and ot buckets. + async fn prepopulate_processors( + hb_bucket: &async_nats::jetstream::kv::Store, + ot_bucket: &async_nats::jetstream::kv::Store, + bucket_config: &BucketConfig, + ) -> (HashMap, HashMap) { + let mut processors = HashMap::new(); + let mut heartbeats = HashMap::new(); + + // Get all existing keys from the hb bucket and create processors + let hb_keys = match hb_bucket.keys().await { + Ok(keys) => keys + .try_collect::>() + .await + .unwrap_or_else(|_| Vec::new()), + Err(e) => { + error!(?e, "Failed to get keys from hb bucket"); + Vec::new() + } + }; + + for key in hb_keys { + let processor_name = Bytes::from(key.clone()); + let processor = Processor::new( + processor_name.clone(), + Status::Active, + bucket_config.partitions as usize, + ); + processors.insert(processor_name.clone(), processor); + + let Ok(Some(value)) = hb_bucket.get(&key).await else { + continue; + }; + + let hb = numaflow_pb::objects::watermark::Heartbeat::decode(value) + .expect("Failed to decode heartbeat") + .heartbeat; + heartbeats.insert(processor_name, hb); + } + + // Get all existing entries from the ot bucket and store them in the timeline + let ot_keys = match ot_bucket.keys().await { + Ok(keys) => keys + .try_collect::>() + .await + .unwrap_or_else(|_| Vec::new()), + Err(e) => { + warn!(?e, "Failed to get keys from ot bucket"); + Vec::new() + } + }; + + for ot_key in ot_keys { + let processor_name = Bytes::from(ot_key.clone()); + let processor = match processors.get_mut(&processor_name) { + Some(processor) => processor, + None => { + warn!(?processor_name, "Processor not found, skipping"); + continue; + } + }; + + let Ok(Some(ot_value)) = ot_bucket.get(&ot_key).await else { + continue; + }; + let wmb: WMB = ot_value.try_into().expect("Failed to decode WMB"); + + let timeline = &mut processor.timelines[wmb.partition as usize]; + timeline.put(wmb); + } + + (processors, heartbeats) + } + /// Starts refreshing the processors status based on the last heartbeat, if the last heartbeat /// is more than 10 times the refreshing rate, the processor is marked as deleted async fn start_refreshing_processors( @@ -227,7 +306,7 @@ impl ProcessorManager { let processor_name = Bytes::from(kv.key); let wmb: WMB = kv.value.try_into().expect("Failed to decode WMB"); - info!(wmb = ?wmb, processor = ?processor_name, "Received wmb from watcher"); + debug!(wmb = ?wmb, processor = ?processor_name, "Received wmb from watcher"); if let Some(processor) = processors .write() .expect("failed to acquire lock") @@ -284,7 +363,7 @@ impl ProcessorManager { .expect("failed to acquire lock") .insert(processor_name.clone(), hb); - info!(hb = ?hb, processor = ?processor_name, "Received heartbeat from watcher"); + debug!(hb = ?hb, processor = ?processor_name, "Received heartbeat from watcher"); // if the processor is not in the processors map, add it // or if processor status is not active, set it to active let mut processors = processors.write().expect("failed to acquire lock"); diff --git a/rust/numaflow-core/src/watermark/processor/timeline.rs b/rust/numaflow-core/src/watermark/processor/timeline.rs index ff31738372..47d074e5b2 100644 --- a/rust/numaflow-core/src/watermark/processor/timeline.rs +++ b/rust/numaflow-core/src/watermark/processor/timeline.rs @@ -6,9 +6,9 @@ use std::cmp::Ordering; use std::collections::VecDeque; use std::fmt; use std::sync::Arc; - use std::sync::RwLock; -use tracing::{debug, error, info}; + +use tracing::{debug, error}; use crate::watermark::wmb::WMB; @@ -65,10 +65,13 @@ impl OffsetTimeline { error!("The new input offset should never be smaller than the existing offset"); } (Ordering::Less, _) => { - error!("Watermark should not regress"); + error!( + "Watermark should not regress, current: {:?}, new: {:?}", + element_node, node + ); } (Ordering::Greater, Ordering::Equal) => { - info!(?node, "Idle Watermark detected"); + debug!(?node, "Idle Watermark detected"); element_node.watermark = node.watermark; } } diff --git a/rust/numaflow-core/src/watermark/source.rs b/rust/numaflow-core/src/watermark/source.rs index 2bd1064400..760ff78bba 100644 --- a/rust/numaflow-core/src/watermark/source.rs +++ b/rust/numaflow-core/src/watermark/source.rs @@ -16,14 +16,18 @@ //! [ISB]: https://numaflow.numaproj.io/core-concepts/inter-step-buffer/ use std::collections::HashMap; +use std::time::Duration; use tokio::sync::mpsc::Receiver; use tracing::error; use crate::config::pipeline::isb::Stream; use crate::config::pipeline::watermark::SourceWatermarkConfig; +use crate::config::pipeline::ToVertexConfig; use crate::error::{Error, Result}; use crate::message::{IntOffset, Message, Offset}; +use crate::watermark::idle::isb::ISBIdleDetector; +use crate::watermark::idle::source::SourceIdleDetector; use crate::watermark::processor::manager::ProcessorManager; use crate::watermark::source::source_wm_fetcher::SourceWatermarkFetcher; use crate::watermark::source::source_wm_publisher::SourceWatermarkPublisher; @@ -35,6 +39,7 @@ pub(crate) mod source_wm_fetcher; pub(crate) mod source_wm_publisher; /// Messages that can be sent to the SourceWatermarkActor +#[allow(clippy::enum_variant_names)] enum SourceActorMessage { PublishSourceWatermark { map: HashMap, @@ -44,18 +49,37 @@ enum SourceActorMessage { stream: Stream, input_partition: u16, }, + CheckAndPublishSourceIdleWatermark { + /// partitions which are/were reading messages. + source_partitions: Vec, + }, + CheckAndPublishISBIdleWatermark, } /// SourceWatermarkActor comprises SourcePublisher and SourceFetcher. struct SourceWatermarkActor { publisher: SourceWatermarkPublisher, fetcher: SourceWatermarkFetcher, + isb_idle_manager: ISBIdleDetector, + source_idle_manager: Option, + active_input_partitions: HashMap, } impl SourceWatermarkActor { /// Creates a new SourceWatermarkActor. - fn new(publisher: SourceWatermarkPublisher, fetcher: SourceWatermarkFetcher) -> Self { - Self { publisher, fetcher } + fn new( + publisher: SourceWatermarkPublisher, + fetcher: SourceWatermarkFetcher, + isb_idle_manager: ISBIdleDetector, + source_idle_manager: Option, + ) -> Self { + Self { + publisher, + fetcher, + isb_idle_manager, + source_idle_manager, + active_input_partitions: HashMap::new(), + } } /// Runs the SourceWatermarkActor @@ -70,27 +94,140 @@ impl SourceWatermarkActor { /// Handles the SourceActorMessage. async fn handle_message(&mut self, message: SourceActorMessage) -> Result<()> { match message { + // publish the watermark for the messages read from the source to the source bucket + // we consider the min event time that we have seen in the batch for each partition SourceActorMessage::PublishSourceWatermark { map } => { + if map.is_empty() { + return Ok(()); + } for (partition, event_time) in map { self.publisher - .publish_source_watermark(partition, event_time) + .publish_source_watermark(partition, event_time, false) .await; + // cache the active input partitions, we need it for publishing isb idle watermark + self.active_input_partitions.insert(partition, true); + } + if let Some(source_idle_manager) = &mut self.source_idle_manager { + source_idle_manager.reset(); } } + + // publish the watermark for the messages written to ISB SourceActorMessage::PublishISBWatermark { offset, stream, input_partition, } => { - let watermark = self.fetcher.fetch_source_watermark()?; + let watermark = self.fetcher.fetch_source_watermark(); self.publisher .publish_isb_watermark( input_partition, - stream, + &stream, offset.offset, watermark.timestamp_millis(), + false, ) .await; + // mark the vertex and partition as active since we published the watermark + self.isb_idle_manager.reset_idle(&stream).await; + } + + // publish the idle watermark for the given partitions (when source is not reading any + // messages) + SourceActorMessage::CheckAndPublishSourceIdleWatermark { source_partitions } => { + let Some(source_idle_manager) = &mut self.source_idle_manager else { + return Ok(()); + }; + + // check if idling is enabled on source and all conditions are met + if !source_idle_manager.is_source_idling() { + return Ok(()); + } + + // compute the watermark that should be published. + let compute_wm = self.fetcher.fetch_source_watermark(); + // using the computed watermark, fetch the idle watermark. + let idle_wm = + source_idle_manager.update_and_fetch_idle_wm(compute_wm.timestamp_millis()); + + // publish the idle watermark for the given partitions + for partition in source_partitions.iter() { + self.publisher + .publish_source_watermark(*partition, idle_wm, true) + .await; + } + + // since isb will also be idling since we are not reading any data + // we need to propagate idle watermarks to ISB + let compute_wm = self.fetcher.fetch_source_watermark(); + if compute_wm.timestamp_millis() == -1 { + return Ok(()); + } + + // all the isb partitions will be idling because the source is idling, fetch the idle offset + // for each vertex and partition and publish the idle watermark + let vertex_streams = self.isb_idle_manager.fetch_all_streams().await; + for stream in vertex_streams.iter() { + let offset = self.isb_idle_manager.fetch_idle_offset(stream).await?; + for idle_partition in source_partitions.iter() { + self.publisher + .publish_isb_watermark( + *idle_partition, + stream, + offset, + compute_wm.timestamp_millis(), + true, + ) + .await; + } + + // mark the vertex and partition as idle, since we published the idle watermark + self.isb_idle_manager + .update_idle_metadata(stream, offset) + .await; + } + } + + // publish the idle watermark for the ISB partitions, this will be invoked by the periodic + // task which keeps check every idle_timeout duration to see if there are any idle partitions + // branch idling case(cf) + SourceActorMessage::CheckAndPublishISBIdleWatermark => { + // if source is idling, we can avoid publishing the idle watermark since we publish + // the idle watermark for all the downstream partitions in the source idling control flow + if let Some(source_idle_manager) = &self.source_idle_manager { + if source_idle_manager.is_source_idling() { + return Ok(()); + } + } + + // fetch the source watermark, identify the idle partitions and publish the idle watermark + let compute_wm = self.fetcher.fetch_source_watermark(); + if compute_wm.timestamp_millis() == -1 { + return Ok(()); + } + + // we should only publish to active input partitions, because we consider input-partitions as + // the processing entity while publishing watermark inside source + let idle_streams = self.isb_idle_manager.fetch_idle_streams().await; + for stream in idle_streams.iter() { + let offset = self.isb_idle_manager.fetch_idle_offset(stream).await?; + for partition in self.active_input_partitions.keys() { + self.publisher + .publish_isb_watermark( + *partition, + stream, + offset, + compute_wm.timestamp_millis(), + true, + ) + .await; + } + self.isb_idle_manager + .update_idle_metadata(stream, offset) + .await; + } + // clear the cache since we published the idle watermarks + self.active_input_partitions.clear(); } } @@ -108,7 +245,9 @@ pub(crate) struct SourceWatermarkHandle { impl SourceWatermarkHandle { /// Creates a new SourceWatermarkHandle. pub(crate) async fn new( + idle_timeout: Duration, js_context: async_nats::jetstream::Context, + to_vertex_configs: &[ToVertexConfig], config: &SourceWatermarkConfig, ) -> Result { let (sender, receiver) = tokio::sync::mpsc::channel(100); @@ -118,15 +257,40 @@ impl SourceWatermarkHandle { let fetcher = SourceWatermarkFetcher::new(processor_manager); let publisher = SourceWatermarkPublisher::new( js_context.clone(), + config.max_delay, config.source_bucket_config.clone(), config.to_vertex_bucket_config.clone(), ) .await .map_err(|e| Error::Watermark(e.to_string()))?; - let actor = SourceWatermarkActor::new(publisher, fetcher); + let source_idle_manager = config + .idle_config + .as_ref() + .map(|idle_config| SourceIdleDetector::new(idle_config.clone())); + + let isb_idle_manager = + ISBIdleDetector::new(idle_timeout, to_vertex_configs, js_context.clone()).await; + + let actor = + SourceWatermarkActor::new(publisher, fetcher, isb_idle_manager, source_idle_manager); tokio::spawn(async move { actor.run(receiver).await }); - Ok(Self { sender }) + + let source_watermark_handle = Self { sender }; + + // start a task to keep publishing idle watermarks every 100ms + tokio::spawn({ + let source_watermark_handle = source_watermark_handle.clone(); + let mut interval_ticker = tokio::time::interval(idle_timeout); + async move { + loop { + interval_ticker.tick().await; + source_watermark_handle.publish_isb_idle_watermark().await; + } + } + }); + + Ok(source_watermark_handle) } /// Generates and Publishes the source watermark for the given messages. @@ -167,7 +331,7 @@ impl SourceWatermarkHandle { stream: Stream, offset: Offset, input_partition: u16, - ) -> Result<()> { + ) { // the fetching happens in the handler if let Offset::Int(offset) = offset { self.sender @@ -177,24 +341,43 @@ impl SourceWatermarkHandle { input_partition, }) .await - .map_err(|_| Error::Watermark("failed to send message".to_string()))?; - - Ok(()) - } else { - Err(Error::Watermark("invalid offset type".to_string())) + .expect("failed to send message"); } } + + pub(crate) async fn publish_source_idle_watermark(&self, partitions: Vec) { + self.sender + .send(SourceActorMessage::CheckAndPublishSourceIdleWatermark { + source_partitions: partitions, + }) + .await + .map_err(|_| Error::Watermark("failed to send message".to_string())) + .expect("failed to send message"); + } + + pub(crate) async fn publish_isb_idle_watermark(&self) { + self.sender + .send(SourceActorMessage::CheckAndPublishISBIdleWatermark) + .await + .map_err(|_| Error::Watermark("failed to send message".to_string())) + .expect("failed to send message"); + } } #[cfg(test)] mod tests { use async_nats::jetstream; use async_nats::jetstream::kv::Config; - use chrono::DateTime; + use async_nats::jetstream::stream; + use bytes::BytesMut; + use chrono::{DateTime, Utc}; + use numaflow_pb::objects::watermark::Heartbeat; + use prost::Message as _; + use tokio::time::sleep; use super::*; - use crate::config::pipeline::isb::Stream; - use crate::config::pipeline::watermark::BucketConfig; + use crate::config::pipeline::isb::BufferWriterConfig; + use crate::config::pipeline::watermark::{BucketConfig, IdleConfig}; use crate::message::{IntOffset, Message}; use crate::watermark::wmb::WMB; @@ -208,6 +391,7 @@ mod tests { let hb_bucket_name = "test_publish_source_watermark_PROCESSORS"; let source_config = SourceWatermarkConfig { + max_delay: Default::default(), source_bucket_config: BucketConfig { vertex: "source_vertex", partitions: 1, // partitions is always one for source @@ -215,6 +399,7 @@ mod tests { hb_bucket: hb_bucket_name, }, to_vertex_bucket_config: vec![], + idle_config: None, }; // create key value stores @@ -236,9 +421,14 @@ mod tests { .await .unwrap(); - let handle = SourceWatermarkHandle::new(js_context.clone(), &source_config) - .await - .expect("Failed to create source watermark handle"); + let handle = SourceWatermarkHandle::new( + Duration::from_millis(100), + js_context.clone(), + Default::default(), + &source_config, + ) + .await + .expect("Failed to create source watermark handle"); let messages = vec![ Message { @@ -275,7 +465,7 @@ mod tests { let mut wmb_found = false; for _ in 0..10 { let wmb = ot_bucket - .get("source_vertex-0") + .get("source-source_vertex-0") .await .expect("Failed to get wmb"); if wmb.is_some() { @@ -284,7 +474,7 @@ mod tests { wmb_found = true; break; } else { - tokio::time::sleep(tokio::time::Duration::from_millis(100)).await; + tokio::time::sleep(Duration::from_millis(100)).await; } } @@ -315,6 +505,7 @@ mod tests { let edge_hb_bucket_name = "test_publish_source_edge_watermark_edge_PROCESSORS"; let source_config = SourceWatermarkConfig { + max_delay: Default::default(), source_bucket_config: BucketConfig { vertex: "source_vertex", partitions: 2, @@ -327,6 +518,7 @@ mod tests { ot_bucket: edge_ot_bucket_name, hb_bucket: edge_hb_bucket_name, }], + idle_config: None, }; // create key value stores for source @@ -365,9 +557,26 @@ mod tests { .await .unwrap(); - let handle = SourceWatermarkHandle::new(js_context.clone(), &source_config) - .await - .expect("Failed to create source watermark handle"); + let handle = SourceWatermarkHandle::new( + Duration::from_millis(100), + js_context.clone(), + &vec![ToVertexConfig { + name: "edge_vertex", + writer_config: BufferWriterConfig { + streams: vec![Stream { + name: "edge_stream", + vertex: "edge_vertex", + partition: 0, + }], + ..Default::default() + }, + conditions: None, + partitions: 1, + }], + &source_config, + ) + .await + .expect("Failed to create source watermark handle"); let ot_bucket = js_context .get_key_value(edge_ot_bucket_name) @@ -413,12 +622,11 @@ mod tests { }); handle .publish_source_isb_watermark(stream.clone(), offset, 0) - .await - .expect("Failed to publish edge watermark"); + .await; // check if the watermark is published let wmb = ot_bucket - .get("source_vertex-edge_vertex-0") + .get("source_vertex-0") .await .expect("Failed to get wmb"); if wmb.is_some() { @@ -427,7 +635,7 @@ mod tests { wmb_found = true; break; } else { - tokio::time::sleep(tokio::time::Duration::from_millis(100)).await; + tokio::time::sleep(Duration::from_millis(100)).await; } } @@ -453,4 +661,393 @@ mod tests { .await .unwrap(); } + + #[cfg(feature = "nats-tests")] + #[tokio::test] + async fn test_invoke_publish_source_idle_watermark() { + let client = async_nats::connect("localhost:4222").await.unwrap(); + let js_context = jetstream::new(client); + + let ot_bucket_name = "test_invoke_publish_source_idle_watermark_OT"; + let hb_bucket_name = "test_invoke_publish_source_idle_watermark_PROCESSORS"; + let to_vertex_ot_bucket_name = "test_invoke_publish_source_idle_watermark_TO_VERTEX_OT"; + let to_vertex_hb_bucket_name = + "test_invoke_publish_source_idle_watermark_TO_VERTEX_PROCESSORS"; + + let to_vertex_configs = vec![ToVertexConfig { + name: "edge_vertex", + writer_config: BufferWriterConfig { + streams: vec![Stream { + name: "edge_stream", + vertex: "edge_vertex", + partition: 0, + }], + ..Default::default() + }, + conditions: None, + partitions: 1, + }]; + + // create to vertex stream since we will be writing ctrl message to it + js_context + .get_or_create_stream(stream::Config { + name: "edge_stream".to_string(), + subjects: vec!["edge_stream".to_string()], + max_message_size: 1024, + ..Default::default() + }) + .await + .unwrap(); + + let source_bucket_config = BucketConfig { + vertex: "v1", + partitions: 1, + ot_bucket: ot_bucket_name, + hb_bucket: hb_bucket_name, + }; + + let to_vertex_bucket_config = BucketConfig { + vertex: "edge_vertex", + partitions: 1, + ot_bucket: to_vertex_ot_bucket_name, + hb_bucket: to_vertex_hb_bucket_name, + }; + + // create key value stores + js_context + .create_key_value(Config { + bucket: ot_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + js_context + .create_key_value(Config { + bucket: hb_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + js_context + .create_key_value(Config { + bucket: to_vertex_ot_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + js_context + .create_key_value(Config { + bucket: to_vertex_hb_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + + let source_idle_config = IdleConfig { + threshold: Duration::from_millis(10), + step_interval: Duration::from_millis(5), + increment_by: Duration::from_millis(1), + }; + + let handle = SourceWatermarkHandle::new( + Duration::from_millis(10), + js_context.clone(), + &to_vertex_configs, + &SourceWatermarkConfig { + max_delay: Default::default(), + source_bucket_config, + to_vertex_bucket_config: vec![to_vertex_bucket_config], + idle_config: Some(source_idle_config), + }, + ) + .await + .expect("Failed to create SourceWatermarkHandle"); + + // get ot and hb buckets for source and publish some wmb and heartbeats + let ot_bucket = js_context + .get_key_value(ot_bucket_name) + .await + .expect("Failed to get ot bucket"); + let hb_bucket = js_context + .get_key_value(hb_bucket_name) + .await + .expect("Failed to get hb bucket"); + + for i in 1..11 { + let wmb: BytesMut = WMB { + watermark: 1000 * i, + offset: i, + idle: false, + partition: 0, + } + .try_into() + .unwrap(); + ot_bucket + .put("source-v1-0", wmb.freeze()) + .await + .expect("Failed to put wmb"); + + let heartbeat = Heartbeat { + heartbeat: Utc::now().timestamp_millis(), + }; + let mut bytes = BytesMut::new(); + heartbeat + .encode(&mut bytes) + .expect("Failed to encode heartbeat"); + + hb_bucket + .put("source-v1-0", bytes.freeze()) + .await + .expect("Failed to put hb"); + sleep(Duration::from_millis(3)).await; + } + + // sleep so that the idle condition is met + tokio::time::sleep(Duration::from_millis(20)).await; + + // Invoke publish_source_idle_watermark + handle.publish_source_idle_watermark(vec![0]).await; + + // Check if the idle watermark is published + let ot_bucket = js_context + .get_key_value(to_vertex_ot_bucket_name) + .await + .expect("Failed to get ot bucket"); + + let mut wmb_found = false; + for _ in 0..10 { + if let Some(wmb) = ot_bucket.get("v1-0").await.expect("Failed to get wmb") { + let wmb: WMB = wmb.try_into().unwrap(); + // idle watermark should be published + if wmb.idle { + wmb_found = true; + break; + } + } + tokio::time::sleep(Duration::from_millis(10)).await; + } + + assert!(wmb_found, "Idle watermark not found"); + + // delete the stores + js_context + .delete_key_value(ot_bucket_name.to_string()) + .await + .unwrap(); + js_context + .delete_key_value(hb_bucket_name.to_string()) + .await + .unwrap(); + js_context + .delete_key_value(to_vertex_ot_bucket_name.to_string()) + .await + .unwrap(); + js_context + .delete_key_value(to_vertex_hb_bucket_name.to_string()) + .await + .unwrap(); + } + + #[cfg(feature = "nats-tests")] + #[tokio::test] + async fn test_publish_source_isb_idle_watermark() { + let client = async_nats::connect("localhost:4222").await.unwrap(); + let js_context = jetstream::new(client); + + let ot_bucket_name = "test_publish_source_isb_idle_watermark_OT"; + let hb_bucket_name = "test_publish_source_isb_idle_watermark_PROCESSORS"; + let to_vertex_ot_bucket_name = "test_publish_source_isb_idle_watermark_TO_VERTEX_OT"; + let to_vertex_hb_bucket_name = + "test_publish_source_isb_idle_watermark_TO_VERTEX_PROCESSORS"; + + let to_vertex_configs = vec![ToVertexConfig { + name: "edge_vertex", + writer_config: BufferWriterConfig { + streams: vec![Stream { + name: "edge_stream", + vertex: "edge_vertex", + partition: 0, + }], + ..Default::default() + }, + conditions: None, + partitions: 1, + }]; + + // create to vertex stream since we will be writing ctrl message to it + js_context + .get_or_create_stream(stream::Config { + name: "edge_stream".to_string(), + subjects: vec!["edge_stream".to_string()], + max_message_size: 1024, + ..Default::default() + }) + .await + .unwrap(); + + let source_bucket_config = BucketConfig { + vertex: "v1", + partitions: 1, + ot_bucket: ot_bucket_name, + hb_bucket: hb_bucket_name, + }; + + let to_vertex_bucket_config = BucketConfig { + vertex: "edge_vertex", + partitions: 1, + ot_bucket: to_vertex_ot_bucket_name, + hb_bucket: to_vertex_hb_bucket_name, + }; + + // create key value stores + js_context + .create_key_value(Config { + bucket: ot_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + js_context + .create_key_value(Config { + bucket: hb_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + js_context + .create_key_value(Config { + bucket: to_vertex_ot_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + js_context + .create_key_value(Config { + bucket: to_vertex_hb_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + + let source_idle_config = IdleConfig { + threshold: Duration::from_millis(2000), // set higher value so that the source won't be idling + step_interval: Duration::from_millis(5), + increment_by: Duration::from_millis(1), + }; + + let handle = SourceWatermarkHandle::new( + Duration::from_millis(5), + js_context.clone(), + &to_vertex_configs, + &SourceWatermarkConfig { + max_delay: Default::default(), + source_bucket_config, + to_vertex_bucket_config: vec![to_vertex_bucket_config], + idle_config: Some(source_idle_config), + }, + ) + .await + .expect("Failed to create SourceWatermarkHandle"); + + let messages = vec![Message { + offset: Offset::Int(IntOffset { + offset: 1, + partition_idx: 0, + }), + event_time: DateTime::from_timestamp_millis(100).unwrap(), + ..Default::default() + }]; + + // generate some watermarks to make partition active + handle + .generate_and_publish_source_watermark(&messages) + .await + .expect("Failed to publish source watermark"); + + // get ot and hb buckets for source and publish some wmb and heartbeats + let ot_bucket = js_context + .get_key_value(ot_bucket_name) + .await + .expect("Failed to get ot bucket"); + let hb_bucket = js_context + .get_key_value(hb_bucket_name) + .await + .expect("Failed to get hb bucket"); + + for i in 1..10 { + let wmb: BytesMut = WMB { + watermark: 1000 * i, + offset: i, + idle: false, + partition: 0, + } + .try_into() + .unwrap(); + ot_bucket + .put("source-v1-0", wmb.freeze()) + .await + .expect("Failed to put wmb"); + + let heartbeat = Heartbeat { + heartbeat: Utc::now().timestamp_millis(), + }; + let mut bytes = BytesMut::new(); + heartbeat + .encode(&mut bytes) + .expect("Failed to encode heartbeat"); + + hb_bucket + .put("source-v1-0", bytes.freeze()) + .await + .expect("Failed to put hb"); + sleep(Duration::from_millis(3)).await; + } + + // Check if the idle watermark is published + let ot_bucket = js_context + .get_key_value(to_vertex_ot_bucket_name) + .await + .expect("Failed to get ot bucket"); + + let mut wmb_found = false; + for _ in 0..10 { + if let Some(wmb) = ot_bucket.get("v1-0").await.expect("Failed to get wmb") { + let wmb: WMB = wmb.try_into().expect("Failed to convert to WMB"); + // idle watermark should be published + if wmb.idle { + wmb_found = true; + break; + } + } + tokio::time::sleep(Duration::from_millis(10)).await; + } + + assert!(wmb_found, "Idle watermark not found"); + + // delete the stores + js_context + .delete_key_value(ot_bucket_name.to_string()) + .await + .unwrap(); + js_context + .delete_key_value(hb_bucket_name.to_string()) + .await + .unwrap(); + js_context + .delete_key_value(to_vertex_ot_bucket_name.to_string()) + .await + .unwrap(); + js_context + .delete_key_value(to_vertex_hb_bucket_name.to_string()) + .await + .unwrap(); + } } diff --git a/rust/numaflow-core/src/watermark/source/source_wm_fetcher.rs b/rust/numaflow-core/src/watermark/source/source_wm_fetcher.rs index 67ad7716bc..2c194b904f 100644 --- a/rust/numaflow-core/src/watermark/source/source_wm_fetcher.rs +++ b/rust/numaflow-core/src/watermark/source/source_wm_fetcher.rs @@ -3,7 +3,6 @@ //! timelines to determine the watermark. We don't care about offsets here since the watermark starts //! at source, we only consider the head watermark and consider the minimum watermark of all the active //! processors. -use crate::error::Result; use crate::watermark::processor::manager::ProcessorManager; use crate::watermark::wmb::Watermark; @@ -20,7 +19,7 @@ impl SourceWatermarkFetcher { /// Fetches the watermark for the source, which is the minimum watermark of all the active /// processors. - pub(crate) fn fetch_source_watermark(&mut self) -> Result { + pub(crate) fn fetch_source_watermark(&mut self) -> Watermark { let mut min_wm = i64::MAX; for (_, processor) in self @@ -47,7 +46,7 @@ impl SourceWatermarkFetcher { min_wm = -1; } - Ok(Watermark::from_timestamp_millis(min_wm).expect("Invalid watermark")) + Watermark::from_timestamp_millis(min_wm).expect("Failed to parse watermark") } } @@ -55,9 +54,9 @@ impl SourceWatermarkFetcher { mod tests { use std::collections::HashMap; use std::sync::Arc; + use std::sync::RwLock; use bytes::Bytes; - use std::sync::RwLock; use super::*; use crate::watermark::processor::manager::{Processor, Status}; @@ -108,7 +107,7 @@ mod tests { let mut fetcher = SourceWatermarkFetcher::new(processor_manager); // Invoke fetch_watermark and verify the result - let watermark = fetcher.fetch_source_watermark().unwrap(); + let watermark = fetcher.fetch_source_watermark(); assert_eq!(watermark.timestamp_millis(), 300); } @@ -187,7 +186,7 @@ mod tests { let mut fetcher = SourceWatermarkFetcher::new(processor_manager); // Invoke fetch_watermark and verify the result - let watermark = fetcher.fetch_source_watermark().unwrap(); + let watermark = fetcher.fetch_source_watermark(); assert_eq!(watermark.timestamp_millis(), 323); } } diff --git a/rust/numaflow-core/src/watermark/source/source_wm_publisher.rs b/rust/numaflow-core/src/watermark/source/source_wm_publisher.rs index f20d2e3fb9..5a629fcb43 100644 --- a/rust/numaflow-core/src/watermark/source/source_wm_publisher.rs +++ b/rust/numaflow-core/src/watermark/source/source_wm_publisher.rs @@ -3,6 +3,7 @@ //! the watermark to the ISB. Unlike other vertices we don't use pod as the processing entity for publishing //! watermark we use the partition(watermark originates here). use std::collections::HashMap; +use std::time::Duration; use chrono::Utc; use tracing::info; @@ -15,6 +16,7 @@ use crate::watermark::isb::wm_publisher::ISBWatermarkPublisher; /// SourcePublisher is the watermark publisher for the source vertex. pub(crate) struct SourceWatermarkPublisher { js_context: async_nats::jetstream::Context, + max_delay: Duration, source_config: BucketConfig, to_vertex_configs: Vec, publishers: HashMap, @@ -24,11 +26,13 @@ impl SourceWatermarkPublisher { /// Creates a new [SourceWatermarkPublisher]. pub(crate) async fn new( js_context: async_nats::jetstream::Context, + max_delay: Duration, source_config: BucketConfig, to_vertex_configs: Vec, ) -> error::Result { Ok(SourceWatermarkPublisher { js_context, + max_delay, source_config, to_vertex_configs, publishers: HashMap::new(), @@ -37,11 +41,16 @@ impl SourceWatermarkPublisher { /// Publishes the source watermark for the input partition. It internally uses edge publisher /// with processor set to the input partition and source OT. - pub(crate) async fn publish_source_watermark(&mut self, partition: u16, watermark: i64) { + pub(crate) async fn publish_source_watermark( + &mut self, + partition: u16, + watermark: i64, + idle: bool, + ) { // for source, we do partition-based watermark publishing rather than pod-based, hence // the processing entity is the partition itself. We create a publisher for each partition // and publish the watermark to it. - let processor_name = format!("{}-{}", self.source_config.vertex, partition); + let processor_name = format!("source-{}-{}", self.source_config.vertex, partition); // create a publisher if not exists if !self.publishers.contains_key(&processor_name) { let publisher = ISBWatermarkPublisher::new( @@ -61,16 +70,16 @@ impl SourceWatermarkPublisher { .get_mut(&processor_name) .expect("Publisher not found") .publish_watermark( - Stream { + &Stream { name: "source", vertex: self.source_config.vertex, partition, }, Utc::now().timestamp_micros(), // we don't care about the offsets - watermark, + watermark - self.max_delay.as_millis() as i64, // consider the max delay configured by the user while publishing source watermark + idle, ) - .await - .expect("Failed to publish watermark"); + .await; } /// Publishes the ISB watermark for the input partition. It internally uses ISB publisher with @@ -78,14 +87,12 @@ impl SourceWatermarkPublisher { pub(crate) async fn publish_isb_watermark( &mut self, input_partition: u16, - stream: Stream, + stream: &Stream, offset: i64, watermark: i64, + idle: bool, ) { - let processor_name = format!( - "{}-{}-{}", - self.source_config.vertex, stream.vertex, input_partition - ); + let processor_name = format!("{}-{}", self.source_config.vertex, input_partition); // In source, since we do partition-based watermark publishing rather than pod-based, we // create a publisher for each partition and publish the watermark to it. if !self.publishers.contains_key(&processor_name) { @@ -105,14 +112,15 @@ impl SourceWatermarkPublisher { self.publishers .get_mut(&processor_name) .expect("Publisher not found") - .publish_watermark(stream, offset, watermark) - .await - .expect("Failed to publish watermark"); + .publish_watermark(stream, offset, watermark, idle) + .await; } } #[cfg(test)] mod tests { + use std::time::Duration; + use async_nats::jetstream; use async_nats::jetstream::kv::Config; @@ -155,13 +163,19 @@ mod tests { .await .unwrap(); - let mut source_publisher = - SourceWatermarkPublisher::new(js_context.clone(), source_config.clone(), vec![]) - .await - .expect("Failed to create source publisher"); + let mut source_publisher = SourceWatermarkPublisher::new( + js_context.clone(), + Duration::from_secs(0), + source_config.clone(), + vec![], + ) + .await + .expect("Failed to create source publisher"); // Publish source watermark for partition 0 - source_publisher.publish_source_watermark(0, 100).await; + source_publisher + .publish_source_watermark(0, 100, false) + .await; let ot_bucket = js_context .get_key_value(ot_bucket_name) @@ -169,7 +183,7 @@ mod tests { .expect("Failed to get ot bucket"); let wmb = ot_bucket - .get("source_vertex-0") + .get("source-source_vertex-0") .await .expect("Failed to get wmb"); assert!(wmb.is_some()); @@ -251,6 +265,7 @@ mod tests { let mut source_publisher = SourceWatermarkPublisher::new( js_context.clone(), + Duration::from_secs(0), source_config.clone(), vec![edge_config.clone()], ) @@ -265,7 +280,7 @@ mod tests { // Publish edge watermark for partition 0 source_publisher - .publish_isb_watermark(0, stream.clone(), 1, 200) + .publish_isb_watermark(0, &stream, 1, 200, false) .await; let ot_bucket = js_context @@ -274,7 +289,197 @@ mod tests { .expect("Failed to get ot bucket"); let wmb = ot_bucket - .get("source_vertex-edge_vertex-0") + .get("source_vertex-0") + .await + .expect("Failed to get wmb"); + assert!(wmb.is_some()); + + let wmb: WMB = wmb.unwrap().try_into().unwrap(); + assert_eq!(wmb.offset, 1); + assert_eq!(wmb.watermark, 200); + + // delete the stores + js_context + .delete_key_value(source_hb_bucket_name.to_string()) + .await + .unwrap(); + js_context + .delete_key_value(source_ot_bucket_name.to_string()) + .await + .unwrap(); + js_context + .delete_key_value(edge_hb_bucket_name.to_string()) + .await + .unwrap(); + js_context + .delete_key_value(edge_ot_bucket_name.to_string()) + .await + .unwrap(); + } + + #[cfg(feature = "nats-tests")] + #[tokio::test] + async fn test_publish_source_watermark_idle() { + let client = async_nats::connect("localhost:4222").await.unwrap(); + let js_context = jetstream::new(client); + + let ot_bucket_name = "source_watermark_idle_OT"; + let hb_bucket_name = "source_watermark_idle_PROCESSORS"; + + let source_config = BucketConfig { + vertex: "source_vertex", + partitions: 2, + ot_bucket: ot_bucket_name, + hb_bucket: hb_bucket_name, + }; + + // create key value stores + js_context + .create_key_value(Config { + bucket: ot_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + + js_context + .create_key_value(Config { + bucket: hb_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + + let mut source_publisher = SourceWatermarkPublisher::new( + js_context.clone(), + Duration::from_secs(0), + source_config.clone(), + vec![], + ) + .await + .expect("Failed to create source publisher"); + + // Publish source watermark for partition 0 with idle flag set to true + source_publisher + .publish_source_watermark(0, 100, true) + .await; + + let ot_bucket = js_context + .get_key_value(ot_bucket_name) + .await + .expect("Failed to get ot bucket"); + + let wmb = ot_bucket + .get("source-source_vertex-0") + .await + .expect("Failed to get wmb"); + assert!(wmb.is_some()); + + let wmb: WMB = wmb.unwrap().try_into().unwrap(); + assert_eq!(wmb.watermark, 100); + assert!(wmb.idle); + + // delete the stores + js_context + .delete_key_value(hb_bucket_name.to_string()) + .await + .unwrap(); + js_context + .delete_key_value(ot_bucket_name.to_string()) + .await + .unwrap(); + } + + #[cfg(feature = "nats-tests")] + #[tokio::test] + async fn test_publish_edge_watermark_idle() { + let client = async_nats::connect("localhost:4222").await.unwrap(); + let js_context = jetstream::new(client); + + let source_ot_bucket_name = "source_edge_watermark_idle_source_OT"; + let source_hb_bucket_name = "source_edge_watermark_idle_source_PROCESSORS"; + let edge_ot_bucket_name = "source_edge_watermark_idle_edge_OT"; + let edge_hb_bucket_name = "source_edge_watermark_idle_edge_PROCESSORS"; + + let source_config = BucketConfig { + vertex: "source_vertex", + partitions: 2, + ot_bucket: source_ot_bucket_name, + hb_bucket: source_hb_bucket_name, + }; + + let edge_config = BucketConfig { + vertex: "edge_vertex", + partitions: 2, + ot_bucket: edge_ot_bucket_name, + hb_bucket: edge_hb_bucket_name, + }; + + // create key value stores for source + js_context + .create_key_value(Config { + bucket: source_ot_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + js_context + .create_key_value(Config { + bucket: source_hb_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + + // create key value stores for edge + js_context + .create_key_value(Config { + bucket: edge_ot_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + js_context + .create_key_value(Config { + bucket: edge_hb_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + + let mut source_publisher = SourceWatermarkPublisher::new( + js_context.clone(), + Duration::from_secs(0), + source_config.clone(), + vec![edge_config.clone()], + ) + .await + .expect("Failed to create source publisher"); + + let stream = Stream { + name: "edge_stream", + vertex: "edge_vertex", + partition: 0, + }; + + // Publish edge watermark for partition 0 with idle flag set to true + source_publisher + .publish_isb_watermark(0, &stream, 1, 200, true) + .await; + + let ot_bucket = js_context + .get_key_value(edge_ot_bucket_name) + .await + .expect("Failed to get ot bucket"); + + let wmb = ot_bucket + .get("source_vertex-0") .await .expect("Failed to get wmb"); assert!(wmb.is_some()); @@ -282,6 +487,7 @@ mod tests { let wmb: WMB = wmb.unwrap().try_into().unwrap(); assert_eq!(wmb.offset, 1); assert_eq!(wmb.watermark, 200); + assert!(wmb.idle); // delete the stores js_context diff --git a/rust/serving/src/config.rs b/rust/serving/src/config.rs index 9445bd2bd1..6b94cb28b5 100644 --- a/rust/serving/src/config.rs +++ b/rust/serving/src/config.rs @@ -1,16 +1,14 @@ -use std::collections::HashMap; -use std::fmt::Debug; - +use crate::{ + pipeline::PipelineDCG, + Error::{self, ParseConfig}, +}; use base64::prelude::BASE64_STANDARD; use base64::Engine; use numaflow_models::models::Vertex; use rcgen::{generate_simple_self_signed, Certificate, CertifiedKey, KeyPair}; use serde::{Deserialize, Serialize}; - -use crate::{ - pipeline::PipelineDCG, - Error::{self, ParseConfig}, -}; +use std::collections::HashMap; +use std::fmt::Debug; const ENV_NUMAFLOW_SERVING_HOST_IP: &str = "NUMAFLOW_SERVING_HOST_IP"; const ENV_NUMAFLOW_SERVING_APP_PORT: &str = "NUMAFLOW_SERVING_APP_LISTEN_PORT";