From e2f9a31f95750755bc8bd7c385d3da06b8f29355 Mon Sep 17 00:00:00 2001 From: Vigith Maurice Date: Mon, 5 Aug 2024 15:21:58 -0700 Subject: [PATCH 01/15] feat: sink source Signed-off-by: Vigith Maurice --- serving/Cargo.toml | 2 +- serving/source-sink/Cargo.toml | 33 ++ serving/source-sink/build.rs | 13 + serving/source-sink/proto/sink.proto | 57 +++ serving/source-sink/proto/source.proto | 153 ++++++++ .../source-sink/proto/sourcetransform.proto | 47 +++ serving/source-sink/src/error.rs | 30 ++ serving/source-sink/src/forwarder.rs | 342 ++++++++++++++++++ serving/source-sink/src/lib.rs | 156 ++++++++ serving/source-sink/src/main.rs | 23 ++ serving/source-sink/src/message.rs | 27 ++ serving/source-sink/src/metrics.rs | 77 ++++ serving/source-sink/src/shared.rs | 38 ++ serving/source-sink/src/sink.rs | 155 ++++++++ serving/source-sink/src/source.rs | 226 ++++++++++++ serving/source-sink/src/transformer.rs | 133 +++++++ 16 files changed, 1511 insertions(+), 1 deletion(-) create mode 100644 serving/source-sink/Cargo.toml create mode 100644 serving/source-sink/build.rs create mode 100644 serving/source-sink/proto/sink.proto create mode 100644 serving/source-sink/proto/source.proto create mode 100644 serving/source-sink/proto/sourcetransform.proto create mode 100644 serving/source-sink/src/error.rs create mode 100644 serving/source-sink/src/forwarder.rs create mode 100644 serving/source-sink/src/lib.rs create mode 100644 serving/source-sink/src/main.rs create mode 100644 serving/source-sink/src/message.rs create mode 100644 serving/source-sink/src/metrics.rs create mode 100644 serving/source-sink/src/shared.rs create mode 100644 serving/source-sink/src/sink.rs create mode 100644 serving/source-sink/src/source.rs create mode 100644 serving/source-sink/src/transformer.rs diff --git a/serving/Cargo.toml b/serving/Cargo.toml index 6353aac742..3fc70caa7d 100644 --- a/serving/Cargo.toml +++ b/serving/Cargo.toml @@ -1,4 +1,4 @@ -workspace = { members = ["backoff", "extras/upstreams", "numaflow-models", "servesink"] } +workspace = { members = ["backoff", "extras/upstreams", "numaflow-models", "servesink", "source-sink"] } [package] name = "serve" version = "0.1.0" diff --git a/serving/source-sink/Cargo.toml b/serving/source-sink/Cargo.toml new file mode 100644 index 0000000000..886364cdf1 --- /dev/null +++ b/serving/source-sink/Cargo.toml @@ -0,0 +1,33 @@ +[package] +name = "sourcer-sinker" +version = "0.1.0" +edition = "2021" + +[dependencies] +axum = "0.7.5" +tonic = "0.12.0" +bytes = "1.7.1" +thiserror = "1.0.63" +tokio = { version = "1.39.2", features = ["full"] } +tracing = "0.1.40" +tokio-util = "0.7.11" +tokio-stream = "0.1.15" +prost = "0.13.1" +prost-types = "0.13.1" +chrono = "0.4.31" +base64 = "0.22.1" +metrics = { version = "0.23.0", default-features = false } +metrics-exporter-prometheus = { version = "0.15.3", default-features = false } +log = "0.4.22" +tracing-subscriber = "0.3.18" +hyper-util = "0.1.6" +tower = "0.4.13" +uuid = { version = "1.10.0", features = ["v4"] } + +[dev-dependencies] +tower = "0.4.13" +tempfile = "3.11.0" +numaflow = { git = "https://github.com/numaproj/numaflow-rs.git", branch="main" } + +[build-dependencies] +tonic-build = "0.12.1" diff --git a/serving/source-sink/build.rs b/serving/source-sink/build.rs new file mode 100644 index 0000000000..fc30e6b678 --- /dev/null +++ b/serving/source-sink/build.rs @@ -0,0 +1,13 @@ +fn main() { + tonic_build::configure() + .build_server(true) + .compile( + &[ + "proto/source.proto", + "proto/sourcetransform.proto", + "proto/sink.proto", + ], + &["proto"], + ) + .unwrap_or_else(|e| panic!("failed to compile the proto, {:?}", e)) +} diff --git a/serving/source-sink/proto/sink.proto b/serving/source-sink/proto/sink.proto new file mode 100644 index 0000000000..c413ea863b --- /dev/null +++ b/serving/source-sink/proto/sink.proto @@ -0,0 +1,57 @@ +syntax = "proto3"; + +import "google/protobuf/empty.proto"; +import "google/protobuf/timestamp.proto"; + +package sink.v1; + +service Sink { + // SinkFn writes the request to a user defined sink. + rpc SinkFn(stream SinkRequest) returns (SinkResponse); + + // IsReady is the heartbeat endpoint for gRPC. + rpc IsReady(google.protobuf.Empty) returns (ReadyResponse); +} + +/** + * SinkRequest represents a request element. + */ +message SinkRequest { + repeated string keys = 1; + bytes value = 2; + google.protobuf.Timestamp event_time = 3; + google.protobuf.Timestamp watermark = 4; + string id = 5; + map headers = 6; +} + +/** + * ReadyResponse is the health check result. + */ +message ReadyResponse { + bool ready = 1; +} + +/** + * SinkResponse is the individual response of each message written to the sink. + */ +message SinkResponse { + message Result { + // id is the ID of the message, can be used to uniquely identify the message. + string id = 1; + // status denotes the status of persisting to sink. It can be SUCCESS, FAILURE, or FALLBACK. + Status status = 2; + // err_msg is the error message, set it if success is set to false. + string err_msg = 3; + } + repeated Result results = 1; +} + +/* + * Status is the status of the response. + */ +enum Status { + SUCCESS = 0; + FAILURE = 1; + FALLBACK = 2; +} \ No newline at end of file diff --git a/serving/source-sink/proto/source.proto b/serving/source-sink/proto/source.proto new file mode 100644 index 0000000000..131cc36d30 --- /dev/null +++ b/serving/source-sink/proto/source.proto @@ -0,0 +1,153 @@ +syntax = "proto3"; + +import "google/protobuf/timestamp.proto"; +import "google/protobuf/empty.proto"; + +package source.v1; + +service Source { + // Read returns a stream of datum responses. + // The size of the returned ReadResponse is less than or equal to the num_records specified in ReadRequest. + // If the request timeout is reached on server side, the returned ReadResponse will contain all the datum that have been read (which could be an empty list). + rpc ReadFn(ReadRequest) returns (stream ReadResponse); + + // AckFn acknowledges a list of datum offsets. + // When AckFn is called, it implicitly indicates that the datum stream has been processed by the source vertex. + // The caller (numa) expects the AckFn to be successful, and it does not expect any errors. + // If there are some irrecoverable errors when the callee (UDSource) is processing the AckFn request, + // then it is best to crash because there are no other retry mechanisms possible. + rpc AckFn(AckRequest) returns (AckResponse); + + // PendingFn returns the number of pending records at the user defined source. + rpc PendingFn(google.protobuf.Empty) returns (PendingResponse); + + // PartitionsFn returns the list of partitions for the user defined source. + rpc PartitionsFn(google.protobuf.Empty) returns (PartitionsResponse); + + // IsReady is the heartbeat endpoint for user defined source gRPC. + rpc IsReady(google.protobuf.Empty) returns (ReadyResponse); +} + +/* + * ReadRequest is the request for reading datum stream from user defined source. + */ +message ReadRequest { + message Request { + // Required field indicating the number of records to read. + uint64 num_records = 1; + // Required field indicating the request timeout in milliseconds. + // uint32 can represent 2^32 milliseconds, which is about 49 days. + // We don't use uint64 because time.Duration takes int64 as nano seconds. Using uint64 for milli will cause overflow. + uint32 timeout_in_ms = 2; + } + // Required field indicating the request. + Request request = 1; +} + +/* + * ReadResponse is the response for reading datum stream from user defined source. + */ +message ReadResponse { + message Result { + // Required field holding the payload of the datum. + bytes payload = 1; + // Required field indicating the offset information of the datum. + Offset offset = 2; + // Required field representing the time associated with each datum. It is used for watermarking. + google.protobuf.Timestamp event_time = 3; + // Optional list of keys associated with the datum. + // Key is the "key" attribute in (key,value) as in the map-reduce paradigm. + // We add this optional field to support the use case where the user defined source can provide keys for the datum. + // e.g. Kafka and Redis Stream message usually include information about the keys. + repeated string keys = 4; + // Optional list of headers associated with the datum. + // Headers are the metadata associated with the datum. + // e.g. Kafka and Redis Stream message usually include information about the headers. + map headers = 5; + } + // Required field holding the result. + Result result = 1; +} + +/* + * AckRequest is the request for acknowledging datum. + * It takes a list of offsets to be acknowledged. + */ +message AckRequest { + message Request { + // Required field holding a list of offsets to be acknowledged. + // The offsets must be strictly corresponding to the previously read batch, + // meaning the offsets must be in the same order as the datum responses in the ReadResponse. + // By enforcing ordering, we can save deserialization effort on the server side, assuming the server keeps a local copy of the raw/un-serialized offsets. + repeated Offset offsets = 1; + } + // Required field holding the request. The list will be ordered and will have the same order as the original Read response. + Request request = 1; +} + +/* + * AckResponse is the response for acknowledging datum. It contains one empty field confirming + * the batch of offsets that have been successfully acknowledged. The contract between client and server + * is that the server will only return the AckResponse if the ack request is successful. + * If the server hangs during the ack request, the client can decide to timeout and error out the data forwarder. + * The reason why we define such contract is that we always expect the server to be able to process the ack request. + * Client is expected to send the AckRequest to the server with offsets that are strictly + * corresponding to the previously read batch. If the client sends the AckRequest with offsets that are not, + * it is considered as a client error and the server will not return the AckResponse. + */ +message AckResponse { + message Result { + // Required field indicating the ack request is successful. + google.protobuf.Empty success = 1; + } + // Required field holding the result. + Result result = 1; +} + +/* + * ReadyResponse is the health check result for user defined source. + */ +message ReadyResponse { + // Required field holding the health check result. + bool ready = 1; +} + +/* + * PendingResponse is the response for the pending request. + */ +message PendingResponse { + message Result { + // Required field holding the number of pending records at the user defined source. + // A negative count indicates that the pending information is not available. + int64 count = 1; + } + // Required field holding the result. + Result result = 1; +} + +/* + * PartitionsResponse is the response for the partitions request. + */ +message PartitionsResponse { + message Result { + // Required field holding the list of partitions. + repeated int32 partitions = 1; + } + // Required field holding the result. + Result result = 1; +} + +/* + * Offset is the offset of the datum. + */ +message Offset { + // offset is the offset of the datum. This field is required. + // We define Offset as a byte array because different input data sources can have different representations for Offset. + // The only way to generalize it is to define it as a byte array, + // Such that we can let the UDSource to de-serialize the offset using its own interpretation logics. + bytes offset = 1; + // Optional partition_id indicates which partition of the source the datum belongs to. + // It is useful for sources that have multiple partitions. e.g. Kafka. + // If the partition_id is not specified, it is assumed that the source has a single partition. + int32 partition_id = 2; +} \ No newline at end of file diff --git a/serving/source-sink/proto/sourcetransform.proto b/serving/source-sink/proto/sourcetransform.proto new file mode 100644 index 0000000000..18e045c323 --- /dev/null +++ b/serving/source-sink/proto/sourcetransform.proto @@ -0,0 +1,47 @@ +syntax = "proto3"; + +import "google/protobuf/timestamp.proto"; +import "google/protobuf/empty.proto"; + +package sourcetransformer.v1; + +service SourceTransform { + // SourceTransformFn applies a function to each request element. + // In addition to map function, SourceTransformFn also supports assigning a new event time to response. + // SourceTransformFn can be used only at source vertex by source data transformer. + rpc SourceTransformFn(SourceTransformRequest) returns (SourceTransformResponse); + + // IsReady is the heartbeat endpoint for gRPC. + rpc IsReady(google.protobuf.Empty) returns (ReadyResponse); +} + +/** + * SourceTransformerRequest represents a request element. + */ +message SourceTransformRequest { + repeated string keys = 1; + bytes value = 2; + google.protobuf.Timestamp event_time = 3; + google.protobuf.Timestamp watermark = 4; + map headers = 5; +} + +/** + * SourceTransformerResponse represents a response element. + */ +message SourceTransformResponse { + message Result { + repeated string keys = 1; + bytes value = 2; + google.protobuf.Timestamp event_time = 3; + repeated string tags = 4; + } + repeated Result results = 1; +} + +/** + * ReadyResponse is the health check result. + */ +message ReadyResponse { + bool ready = 1; +} \ No newline at end of file diff --git a/serving/source-sink/src/error.rs b/serving/source-sink/src/error.rs new file mode 100644 index 0000000000..f3e2e3b7d9 --- /dev/null +++ b/serving/source-sink/src/error.rs @@ -0,0 +1,30 @@ +use thiserror::Error; + +pub type Result = std::result::Result; + +#[derive(Error, Debug, Clone)] +pub enum Error { + #[error("Metrics Error - {0}")] + MetricsError(String), + + #[error("Source Error - {0}")] + SourceError(String), + + #[error("Sink Error - {0}")] + SinkError(String), + + #[error("Transformer Error - {0}")] + TransformerError(String), + + #[error("Forwarder Error - {0}")] + ForwarderError(String), + + #[error("Connection Error - {0}")] + ConnectionError(String), +} + +impl From for Error { + fn from(status: tonic::Status) -> Self { + Error::SourceError(status.to_string()) + } +} diff --git a/serving/source-sink/src/forwarder.rs b/serving/source-sink/src/forwarder.rs new file mode 100644 index 0000000000..8db6c41af3 --- /dev/null +++ b/serving/source-sink/src/forwarder.rs @@ -0,0 +1,342 @@ +use tokio::sync::oneshot; +use tokio::task::JoinSet; +use tracing::info; + +use crate::error::{Error, Result}; +use crate::sink::SinkClient; +use crate::source::SourceClient; +use crate::transformer::TransformerClient; + +pub(crate) struct Forwarder { + source_client: SourceClient, + sink_client: SinkClient, + transformer_client: Option, + timeout_in_ms: u32, + batch_size: u64, + shutdown_rx: oneshot::Receiver<()>, +} + +impl Forwarder { + pub(crate) async fn new( + source_client: SourceClient, + sink_client: SinkClient, + transformer_client: Option, + timeout_in_ms: u32, + batch_size: u64, + shutdown_rx: oneshot::Receiver<()>, + ) -> Result { + Ok(Self { + source_client, + sink_client, + transformer_client, + timeout_in_ms, + batch_size, + shutdown_rx, + }) + } + + pub(crate) async fn run(&mut self) -> Result<()> { + loop { + tokio::select! { + _ = &mut self.shutdown_rx => { + info!("Shutdown signal received, stopping forwarder..."); + break; + } + result = self.source_client.read_fn(self.batch_size, self.timeout_in_ms) => { + // Read messages from the source + let messages = result?; + + // Extract offsets from the messages + let offsets = messages.iter().map(|message| message.offset.clone()).collect(); + + // Apply transformation if transformer is present + let transformed_messages = if let Some(transformer_client) = &self.transformer_client { + let mut jh = JoinSet::new(); + for message in messages { + let mut transformer_client = transformer_client.clone(); + jh.spawn(async move { transformer_client.transform_fn(message).await }); + } + + let mut results = Vec::new(); + while let Some(task) = jh.join_next().await { + let result = task.map_err(|e| Error::TransformerError(format!("{:?}", e)))?; + let result = result?; + results.extend(result); + } + results + } else { + messages + }; + + // Write messages to the sink + self.sink_client.sink_fn(transformed_messages).await?; + + // Acknowledge the messages + self.source_client.ack_fn(offsets).await?; + } + } + } + Ok(()) + } +} + +#[cfg(test)] +mod tests { + use std::collections::HashSet; + + use chrono::Utc; + use numaflow::source::{Message, Offset, SourceReadRequest}; + use numaflow::{sink, source, sourcetransform}; + use tokio::sync::mpsc::Sender; + + use crate::forwarder::Forwarder; + use crate::sink::SinkClient; + use crate::source::SourceClient; + use crate::transformer::TransformerClient; + + struct SimpleSource { + yet_to_be_acked: std::sync::RwLock>, + } + + impl SimpleSource { + fn new() -> Self { + Self { + yet_to_be_acked: std::sync::RwLock::new(HashSet::new()), + } + } + } + + #[tonic::async_trait] + impl source::Sourcer for SimpleSource { + async fn read(&self, request: SourceReadRequest, transmitter: Sender) { + let event_time = Utc::now(); + let mut message_offsets = Vec::with_capacity(request.count); + for i in 0..2 { + let offset = format!("{}-{}", event_time.timestamp_nanos_opt().unwrap(), i); + transmitter + .send(Message { + value: "test-message".as_bytes().to_vec(), + event_time, + offset: Offset { + offset: offset.clone().into_bytes(), + partition_id: 0, + }, + keys: vec!["test-key".to_string()], + headers: Default::default(), + }) + .await + .unwrap(); + message_offsets.push(offset) + } + self.yet_to_be_acked + .write() + .unwrap() + .extend(message_offsets) + } + + async fn ack(&self, offsets: Vec) { + for offset in offsets { + self.yet_to_be_acked + .write() + .unwrap() + .remove(&String::from_utf8(offset.offset).unwrap()); + } + } + + async fn pending(&self) -> usize { + self.yet_to_be_acked.read().unwrap().len() + } + + async fn partitions(&self) -> Option> { + Some(vec![0]) + } + } + + struct SimpleTransformer; + #[tonic::async_trait] + impl sourcetransform::SourceTransformer for SimpleTransformer { + async fn transform( + &self, + input: sourcetransform::SourceTransformRequest, + ) -> Vec { + let keys = input + .keys + .iter() + .map(|k| k.clone() + "-transformed") + .collect(); + let message = sourcetransform::Message::new(input.value, Utc::now()) + .keys(keys) + .tags(vec![]); + vec![message] + } + } + + struct InMemorySink { + sender: Sender, + } + + impl InMemorySink { + fn new(sender: Sender) -> Self { + Self { sender } + } + } + + #[tonic::async_trait] + impl sink::Sinker for InMemorySink { + async fn sink( + &self, + mut input: tokio::sync::mpsc::Receiver, + ) -> Vec { + let mut responses: Vec = Vec::new(); + while let Some(datum) = input.recv().await { + let response = match std::str::from_utf8(&datum.value) { + Ok(_) => { + self.sender + .send(Message { + value: datum.value.clone(), + event_time: datum.event_time, + offset: Offset { + offset: "test-offset".to_string().into_bytes(), + partition_id: 0, + }, + keys: datum.keys.clone(), + headers: Default::default(), + }) + .await + .unwrap(); + sink::Response::ok(datum.id) + } + Err(e) => { + sink::Response::failure(datum.id, format!("Invalid UTF-8 sequence: {}", e)) + } + }; + responses.push(response); + } + responses + } + } + + #[tokio::test] + async fn test_forwarder_source_sink() { + // Create channels for communication + let (sink_tx, mut sink_rx) = tokio::sync::mpsc::channel(10); + + // Start the source server + let (source_shutdown_tx, source_shutdown_rx) = tokio::sync::oneshot::channel(); + let tmp_dir = tempfile::TempDir::new().unwrap(); + let source_sock_file = tmp_dir.path().join("source.sock"); + + let source_socket = source_sock_file.clone(); + let source_server_handle = tokio::spawn(async move { + let server_info_file = tmp_dir.path().join("source-server-info"); + source::Server::new(SimpleSource::new()) + .with_socket_file(source_socket) + .with_server_info_file(server_info_file) + .start_with_shutdown(source_shutdown_rx) + .await + .unwrap(); + }); + + // Start the sink server + let (sink_shutdown_tx, sink_shutdown_rx) = tokio::sync::oneshot::channel(); + let sink_tmp_dir = tempfile::TempDir::new().unwrap(); + let sink_sock_file = sink_tmp_dir.path().join("sink.sock"); + + let sink_socket = sink_sock_file.clone(); + let sink_server_handle = tokio::spawn(async move { + let server_info_file = sink_tmp_dir.path().join("sink-server-info"); + sink::Server::new(InMemorySink::new(sink_tx)) + .with_socket_file(sink_socket) + .with_server_info_file(server_info_file) + .start_with_shutdown(sink_shutdown_rx) + .await + .unwrap(); + }); + + // Start the transformer server + let (transformer_shutdown_tx, transformer_shutdown_rx) = tokio::sync::oneshot::channel(); + let tmp_dir = tempfile::TempDir::new().unwrap(); + let transformer_sock_file = tmp_dir.path().join("transformer.sock"); + + let transformer_socket = transformer_sock_file.clone(); + let transformer_server_handle = tokio::spawn(async move { + let server_info_file = tmp_dir.path().join("transformer-server-info"); + sourcetransform::Server::new(SimpleTransformer) + .with_socket_file(transformer_socket) + .with_server_info_file(server_info_file) + .start_with_shutdown(transformer_shutdown_rx) + .await + .unwrap(); + }); + + // Wait for the servers to start + tokio::time::sleep(tokio::time::Duration::from_millis(100)).await; + + let (forwarder_shutdown_tx, forwarder_shutdown_rx) = tokio::sync::oneshot::channel(); + + let source_client = SourceClient::connect(source_sock_file) + .await + .expect("failed to connect to source server"); + + let sink_client = SinkClient::connect(sink_sock_file) + .await + .expect("failed to connect to sink server"); + + let transformer_client = TransformerClient::connect(transformer_sock_file) + .await + .expect("failed to connect to transformer server"); + + let mut forwarder = Forwarder::new( + source_client, + sink_client, + Some(transformer_client), + 1000, + 10, + forwarder_shutdown_rx, + ) + .await + .expect("failed to create forwarder"); + + let forwarder_handle = tokio::spawn(async move { + forwarder.run().await.unwrap(); + }); + + // Receive messages from the sink + let received_message = sink_rx.recv().await.unwrap(); + assert_eq!(received_message.value, "test-message".as_bytes()); + assert_eq!( + received_message.keys, + vec!["test-key-transformed".to_string()] + ); + + // stop the forwarder + forwarder_shutdown_tx + .send(()) + .expect("failed to send shutdown signal"); + forwarder_handle + .await + .expect("failed to join forwarder task"); + + // stop the servers + source_shutdown_tx + .send(()) + .expect("failed to send shutdown signal"); + source_server_handle + .await + .expect("failed to join source server task"); + + transformer_shutdown_tx + .send(()) + .expect("failed to send shutdown signal"); + transformer_server_handle + .await + .expect("failed to join transformer server task"); + + sink_shutdown_tx + .send(()) + .expect("failed to send shutdown signal"); + sink_server_handle + .await + .expect("failed to join sink server task"); + } +} diff --git a/serving/source-sink/src/lib.rs b/serving/source-sink/src/lib.rs new file mode 100644 index 0000000000..0662434de1 --- /dev/null +++ b/serving/source-sink/src/lib.rs @@ -0,0 +1,156 @@ +use std::env; + +use tokio::signal; +use tokio::sync::oneshot; +use tokio::task::JoinHandle; +use tracing::info; + +use crate::error::Error; +use crate::forwarder::Forwarder; +use crate::sink::SinkClient; +use crate::source::SourceClient; +use crate::transformer::TransformerClient; + +///! SourcerSinker orchestrates data movement from the Source to the Sink via the optional SourceTransformer. +///! The infamous forward-a-chunk executes the following in an infinite loop: +///! - Read X messages from the source +///! - Invokes the SourceTransformer concurrently +///! - Calls the Sinker to write the batch to the Sink +///! - Send Acknowledgement back to the Source + +/// TODO +/// - [ ] integrate with main +/// - [ ] add metrics and metrics-server +/// - [ ] integrate with trace! +/// - [ ] add code comment +/// - [ ] error handling using anyhow +/// - [ ] unit testing >= 85% +/// - [ ] local integration testing +pub use self::error::Result; + +pub mod error; + +pub mod metrics; + +pub(crate) mod source; + +pub(crate) mod sink; + +pub(crate) mod transformer; + +pub(crate) mod forwarder; + +pub(crate) mod message; + +pub(crate) mod shared; + +const SOURCE_SOCKET: &str = "/var/run/numaflow/source.sock"; +const SINK_SOCKET: &str = "/var/run/numaflow/sink.sock"; +const TRANSFORMER_SOCKET: &str = "/var/run/numaflow/sourcetransform.sock"; +const TIMEOUT_IN_MS: u32 = 1000; +const BATCH_SIZE: u64 = 500; + +pub async fn run_forwarder() -> Result<()> { + let mut source_client = SourceClient::connect(SOURCE_SOCKET.into()).await?; + let mut sink_client = SinkClient::connect(SINK_SOCKET.into()).await?; + let mut transformer_client = if env::var("NUMAFLOW_TRANSFORMER").is_ok() { + Some(TransformerClient::connect(TRANSFORMER_SOCKET.into()).await?) + } else { + None + }; + + let (shutdown_tx, shutdown_rx) = oneshot::channel(); + + // readiness check for all the servers + wait_until_ready( + &mut source_client, + &mut sink_client, + &mut transformer_client, + ) + .await?; + + let mut forwarder = Forwarder::new( + source_client, + sink_client, + transformer_client, + TIMEOUT_IN_MS, + BATCH_SIZE, + shutdown_rx, + ) + .await?; + + let forwarder_handle: JoinHandle> = tokio::spawn(async move { + forwarder.run().await?; + Ok(()) + }); + + let shutdown_handle: JoinHandle> = tokio::spawn(async move { + shutdown_signal().await; + shutdown_tx + .send(()) + .map_err(|_| Error::ForwarderError("Failed to send shutdown signal".to_string()))?; + Ok(()) + }); + + let _ = tokio::try_join!(forwarder_handle, shutdown_handle) + .map_err(|e| Error::ForwarderError(format!("{:?}", e)))?; + + info!("Forwarder stopped gracefully"); + Ok(()) +} + +async fn wait_until_ready( + source_client: &mut SourceClient, + sink_client: &mut SinkClient, + transformer_client: &mut Option, +) -> Result<()> { + loop { + let source_ready = source_client.is_ready().await.is_ok(); + if !source_ready { + info!("UDSource is not ready, waiting..."); + } + + let sink_ready = sink_client.is_ready().await.is_ok(); + if !sink_ready { + info!("UDSink is not ready, waiting..."); + } + + let transformer_ready = if let Some(client) = transformer_client { + let ready = client.is_ready().await.is_ok(); + if !ready { + info!("UDTransformer is not ready, waiting..."); + } + ready + } else { + true + }; + + if source_ready && sink_ready && transformer_ready { + break; + } + + tokio::time::sleep(tokio::time::Duration::from_secs(1)).await; + } + + Ok(()) +} + +async fn shutdown_signal() { + let ctrl_c = async { + signal::ctrl_c() + .await + .expect("failed to install Ctrl+C handler"); + }; + + let terminate = async { + signal::unix::signal(signal::unix::SignalKind::terminate()) + .expect("failed to install signal handler") + .recv() + .await; + }; + + tokio::select! { + _ = ctrl_c => {}, + _ = terminate => {}, + } +} diff --git a/serving/source-sink/src/main.rs b/serving/source-sink/src/main.rs new file mode 100644 index 0000000000..9fc093bb53 --- /dev/null +++ b/serving/source-sink/src/main.rs @@ -0,0 +1,23 @@ +use std::net::SocketAddr; + +use tracing::error; + +use sourcer_sinker::{metrics::start_metrics_server, run_forwarder}; + +#[tokio::main] +async fn main() { + tracing_subscriber::fmt::init(); + + // Start the metrics server + let metrics_addr: SocketAddr = "0.0.0.0:9090".parse().expect("Invalid address"); + tokio::spawn(async move { + if let Err(e) = start_metrics_server(metrics_addr).await { + error!("Metrics server error: {:?}", e); + } + }); + + // Run the forwarder + if let Err(e) = run_forwarder().await { + error!("Application error: {:?}", e); + } +} diff --git a/serving/source-sink/src/message.rs b/serving/source-sink/src/message.rs new file mode 100644 index 0000000000..bacc2e8b85 --- /dev/null +++ b/serving/source-sink/src/message.rs @@ -0,0 +1,27 @@ +use std::collections::HashMap; + +use chrono::{DateTime, Utc}; + +/// A message that is sent from the source to the sink. +#[derive(Debug, Clone)] +pub(crate) struct Message { + /// keys of the message + pub(crate) keys: Vec, + /// actual payload of the message + pub(crate) value: Vec, + /// offset of the message + pub(crate) offset: Offset, + /// event time of the message + pub(crate) event_time: DateTime, + /// headers of the message + pub(crate) headers: HashMap, +} + +/// Offset of the message which will be used to acknowledge the message. +#[derive(Debug, Clone)] +pub(crate) struct Offset { + /// unique identifier of the message + pub(crate) offset: String, + /// partition id of the message + pub(crate) partition_id: i32, +} diff --git a/serving/source-sink/src/metrics.rs b/serving/source-sink/src/metrics.rs new file mode 100644 index 0000000000..f56daf4b43 --- /dev/null +++ b/serving/source-sink/src/metrics.rs @@ -0,0 +1,77 @@ +use std::future::ready; + +use axum::{routing::get, Router}; +use metrics_exporter_prometheus::{Matcher, PrometheusBuilder, PrometheusHandle}; +use tokio::net::{TcpListener, ToSocketAddrs}; +use tracing::debug; + +use crate::error::Error; + +/// Collect and emit prometheus metrics. + +/// Metrics router and server +pub async fn start_metrics_server(addr: A) -> crate::Result<()> +where + A: ToSocketAddrs + std::fmt::Debug, +{ + // setup_metrics_recorder should only be invoked once + let recorder_handle = setup_metrics_recorder()?; + let metrics_app = Router::new().route("/metrics", get(move || ready(recorder_handle.render()))); + + let listener = TcpListener::bind(&addr) + .await + .map_err(|e| Error::MetricsError(format!("Creating listener on {:?}: {}", addr, e)))?; + + debug!("metrics server started at addr: {:?}", addr); + + axum::serve(listener, metrics_app) + .await + .map_err(|e| Error::MetricsError(format!("Starting web server for metrics: {}", e)))?; + Ok(()) +} + +/// setup the Prometheus metrics recorder. +fn setup_metrics_recorder() -> crate::Result { + // 1 micro-sec < t < 1000 seconds + let log_to_power_of_sqrt2_bins: [f64; 62] = (0..62) + .map(|i| 2_f64.sqrt().powf(i as f64)) + .collect::>() + .try_into() + .unwrap(); + + let prometheus_handle = PrometheusBuilder::new() + .set_buckets_for_metric( + Matcher::Full("fac_total_duration_micros".to_string()), // fac == forward-a-chunk + &log_to_power_of_sqrt2_bins, + ) + .map_err(|e| Error::MetricsError(format!("Prometheus install_recorder: {}", e)))? + .install_recorder() + .map_err(|e| Error::MetricsError(format!("Prometheus install_recorder: {}", e)))?; + + Ok(prometheus_handle) +} + +#[cfg(test)] +mod tests { + use std::net::SocketAddr; + use std::time::Duration; + + use tokio::time::sleep; + + use super::*; + + #[tokio::test] + async fn test_start_metrics_server() { + let addr = SocketAddr::from(([127, 0, 0, 1], 0)); + let server = tokio::spawn(async move { + let result = start_metrics_server(addr).await; + assert!(result.is_ok()) + }); + + // Give the server a little bit of time to start + sleep(Duration::from_millis(100)).await; + + // Stop the server + server.abort(); + } +} diff --git a/serving/source-sink/src/shared.rs b/serving/source-sink/src/shared.rs new file mode 100644 index 0000000000..2c63244647 --- /dev/null +++ b/serving/source-sink/src/shared.rs @@ -0,0 +1,38 @@ +use std::path::PathBuf; + +use chrono::{DateTime, TimeZone, Timelike, Utc}; +use prost_types::Timestamp; +use tokio::net::UnixStream; +use tonic::transport::{Channel, Endpoint, Uri}; +use tower::service_fn; + +use crate::error::Error; + +pub(crate) fn utc_from_timestamp(t: Option) -> DateTime { + t.map_or(Utc.timestamp_nanos(-1), |t| { + DateTime::from_timestamp(t.seconds, t.nanos as u32).unwrap_or(Utc.timestamp_nanos(-1)) + }) +} + +pub(crate) fn prost_timestamp_from_utc(t: DateTime) -> Option { + Some(Timestamp { + seconds: t.timestamp(), + nanos: t.nanosecond() as i32, + }) +} + +pub(crate) async fn connect_with_uds(uds_path: PathBuf) -> Result { + let channel = Endpoint::try_from("http://[::]:50051") + .map_err(|e| Error::ConnectionError(format!("Failed to create endpoint: {:?}", e)))? + .connect_with_connector(service_fn(move |_: Uri| { + let uds_socket = uds_path.clone(); + async move { + Ok::<_, std::io::Error>(hyper_util::rt::TokioIo::new( + UnixStream::connect(uds_socket).await?, + )) + } + })) + .await + .map_err(|e| Error::ConnectionError(format!("Failed to connect: {:?}", e)))?; + Ok(channel) +} diff --git a/serving/source-sink/src/sink.rs b/serving/source-sink/src/sink.rs new file mode 100644 index 0000000000..5a0094df5b --- /dev/null +++ b/serving/source-sink/src/sink.rs @@ -0,0 +1,155 @@ +use std::path::PathBuf; + +use tonic::transport::Channel; +use tonic::Request; + +use crate::error::Result; +use crate::message::Message; +use crate::shared::{connect_with_uds, prost_timestamp_from_utc}; + +pub mod proto { + tonic::include_proto!("sink.v1"); +} + +pub struct SinkClient { + client: proto::sink_client::SinkClient, +} + +impl SinkClient { + pub(crate) async fn connect(uds_path: PathBuf) -> Result { + let channel = connect_with_uds(uds_path).await?; + let client = proto::sink_client::SinkClient::new(channel); + Ok(Self { client }) + } + + pub(crate) async fn sink_fn(&mut self, messages: Vec) -> Result { + let requests: Vec = messages + .into_iter() + .map(|message| proto::SinkRequest { + keys: message.keys, + value: message.value, + event_time: prost_timestamp_from_utc(message.event_time), + watermark: None, + id: format!("{}-{}", message.offset.partition_id, message.offset.offset), + headers: message.headers, + }) + .collect(); + + let (tx, rx) = tokio::sync::mpsc::channel(1); + + tokio::spawn(async move { + for request in requests { + if tx.send(request).await.is_err() { + break; + } + } + }); + + let response = self + .client + .sink_fn(tokio_stream::wrappers::ReceiverStream::new(rx)) + .await? + .into_inner(); + Ok(response) + } + + pub(crate) async fn is_ready(&mut self) -> Result { + let request = Request::new(()); + let response = self.client.is_ready(request).await?.into_inner(); + Ok(response) + } +} + +#[cfg(test)] +mod tests { + use chrono::offset::Utc; + use log::info; + use numaflow::sink; + + use crate::message::Offset; + + use super::*; + + struct Logger; + #[tonic::async_trait] + impl sink::Sinker for Logger { + async fn sink( + &self, + mut input: tokio::sync::mpsc::Receiver, + ) -> Vec { + let mut responses: Vec = Vec::new(); + while let Some(datum) = input.recv().await { + let response = match std::str::from_utf8(&datum.value) { + Ok(v) => { + info!("{}", v); + sink::Response::ok(datum.id) + } + Err(e) => { + sink::Response::failure(datum.id, format!("Invalid UTF-8 sequence: {}", e)) + } + }; + responses.push(response); + } + responses + } + } + #[tokio::test] + async fn sink_operations() { + // start the server + let (shutdown_tx, shutdown_rx) = tokio::sync::oneshot::channel(); + let tmp_dir = tempfile::TempDir::new().unwrap(); + let sock_file = tmp_dir.path().join("sink.sock"); + + let server_socket = sock_file.clone(); + let server_handle = tokio::spawn(async move { + let server_info_file = tmp_dir.path().join("sink-server-info"); + sink::Server::new(Logger) + .with_socket_file(server_socket) + .with_server_info_file(server_info_file) + .start_with_shutdown(shutdown_rx) + .await + .unwrap(); + }); + + // wait for the server to start + tokio::time::sleep(tokio::time::Duration::from_millis(100)).await; + + let mut sink_client = SinkClient::connect(sock_file) + .await + .expect("failed to connect to sink server"); + + let messages = vec![ + Message { + keys: vec![], + value: b"Hello, World!".to_vec(), + offset: Offset { + offset: "1".to_string(), + partition_id: 0, + }, + event_time: Utc::now(), + headers: Default::default(), + }, + Message { + keys: vec![], + value: b"Hello, World!".to_vec(), + offset: Offset { + offset: "2".to_string(), + partition_id: 0, + }, + event_time: Utc::now(), + headers: Default::default(), + }, + ]; + + let ready_response = sink_client.is_ready().await.unwrap(); + assert_eq!(ready_response.ready, true); + + let response = sink_client.sink_fn(messages).await.unwrap(); + assert_eq!(response.results.len(), 2); + + shutdown_tx + .send(()) + .expect("failed to send shutdown signal"); + server_handle.await.expect("failed to join server task"); + } +} diff --git a/serving/source-sink/src/source.rs b/serving/source-sink/src/source.rs new file mode 100644 index 0000000000..dfdbe6b7e3 --- /dev/null +++ b/serving/source-sink/src/source.rs @@ -0,0 +1,226 @@ +use std::path::PathBuf; + +use base64::prelude::BASE64_STANDARD; +use base64::Engine; +use tokio_stream::StreamExt; +use tonic::transport::Channel; +use tonic::Request; + +use crate::error::{Error, Result}; +use crate::message::{Message, Offset}; +use crate::shared::{connect_with_uds, utc_from_timestamp}; + +pub mod proto { + tonic::include_proto!("source.v1"); +} + +#[derive(Debug, Clone)] +pub(crate) struct SourceClient { + client: proto::source_client::SourceClient, +} + +impl SourceClient { + pub(crate) async fn connect(uds_path: PathBuf) -> Result { + let channel = connect_with_uds(uds_path).await?; + let client = proto::source_client::SourceClient::new(channel); + Ok(Self { client }) + } + + pub(crate) async fn read_fn( + &mut self, + num_records: u64, + timeout_in_ms: u32, + ) -> Result> { + let request = Request::new(proto::ReadRequest { + request: Some(proto::read_request::Request { + num_records, + timeout_in_ms, + }), + }); + + let mut stream = self.client.read_fn(request).await?.into_inner(); + let mut messages = Vec::new(); + + while let Some(response) = stream.next().await { + let result = response? + .result + .ok_or_else(|| Error::SourceError("Offset not found".to_string()))?; + + let source_offset = match result.offset { + Some(o) => Offset { + offset: BASE64_STANDARD.encode(o.offset), + partition_id: o.partition_id, + }, + None => return Err(Error::SourceError("Offset not found".to_string())), + }; + + let message = Message { + keys: result.keys, + value: result.payload, + offset: source_offset, + event_time: utc_from_timestamp(result.event_time), + headers: result.headers, + }; + messages.push(message); + } + + Ok(messages) + } + + pub(crate) async fn ack_fn(&mut self, offsets: Vec) -> Result { + let offsets = offsets + .into_iter() + .map(|offset| proto::Offset { + offset: BASE64_STANDARD.decode(offset.offset).unwrap(), // we only control the encoding, so this should never fail + partition_id: offset.partition_id, + }) + .collect(); + + let request = Request::new(proto::AckRequest { + request: Some(proto::ack_request::Request { offsets }), + }); + + let response = self.client.ack_fn(request).await?.into_inner(); + Ok(response) + } + + #[allow(dead_code)] + pub(crate) async fn pending_fn(&mut self) -> Result { + let request = Request::new(()); + let response = self.client.pending_fn(request).await?.into_inner(); + Ok(response) + } + + #[allow(dead_code)] + pub(crate) async fn partitions_fn(&mut self) -> Result { + let request = Request::new(()); + let response = self.client.partitions_fn(request).await?.into_inner(); + Ok(response) + } + + pub(crate) async fn is_ready(&mut self) -> Result { + let request = Request::new(()); + let response = self.client.is_ready(request).await?.into_inner(); + Ok(response) + } +} + +#[cfg(test)] +mod tests { + use std::collections::HashSet; + use std::error::Error; + + use chrono::Utc; + use numaflow::source; + use numaflow::source::{Message, Offset, SourceReadRequest}; + use tokio::sync::mpsc::Sender; + + use crate::source::SourceClient; + + struct SimpleSource { + num: usize, + yet_to_ack: std::sync::RwLock>, + } + + impl SimpleSource { + fn new(num: usize) -> Self { + Self { + num, + yet_to_ack: std::sync::RwLock::new(HashSet::new()), + } + } + } + + #[tonic::async_trait] + impl source::Sourcer for SimpleSource { + async fn read(&self, request: SourceReadRequest, transmitter: Sender) { + let event_time = Utc::now(); + let mut message_offsets = Vec::with_capacity(request.count); + for i in 0..request.count { + let offset = format!("{}-{}", event_time.timestamp_nanos_opt().unwrap(), i); + transmitter + .send(Message { + value: self.num.to_le_bytes().to_vec(), + event_time, + offset: Offset { + offset: offset.clone().into_bytes(), + partition_id: 0, + }, + keys: vec![], + headers: Default::default(), + }) + .await + .unwrap(); + message_offsets.push(offset) + } + self.yet_to_ack.write().unwrap().extend(message_offsets) + } + + async fn ack(&self, offsets: Vec) { + for offset in offsets { + self.yet_to_ack + .write() + .unwrap() + .remove(&String::from_utf8(offset.offset).unwrap()); + } + } + + async fn pending(&self) -> usize { + self.yet_to_ack.read().unwrap().len() + } + + async fn partitions(&self) -> Option> { + Some(vec![2]) + } + } + + #[tokio::test] + async fn source_operations() -> Result<(), Box> { + // start the server + let (shutdown_tx, shutdown_rx) = tokio::sync::oneshot::channel(); + let tmp_dir = tempfile::TempDir::new().unwrap(); + let sock_file = tmp_dir.path().join("source.sock"); + + let server_socket = sock_file.clone(); + let server_handle = tokio::spawn(async move { + let server_info_file = tmp_dir.path().join("source-server-info"); + source::Server::new(SimpleSource::new(10)) + .with_socket_file(server_socket) + .with_server_info_file(server_info_file) + .start_with_shutdown(shutdown_rx) + .await + .unwrap(); + }); + + // wait for the server to start + tokio::time::sleep(tokio::time::Duration::from_millis(100)).await; + + let mut source_client = SourceClient::connect(sock_file) + .await + .expect("failed to connect to source server"); + + let response = source_client.is_ready().await.unwrap(); + assert_eq!(response.ready, true); + + let messages = source_client.read_fn(5, 1000).await.unwrap(); + assert_eq!(messages.len(), 5); + + let response = source_client + .ack_fn(messages.iter().map(|m| m.offset.clone()).collect()) + .await + .unwrap(); + assert!(response.result.unwrap().success.is_some()); + + let pending = source_client.pending_fn().await.unwrap(); + assert_eq!(pending.result.unwrap().count, 0); + + let partitions = source_client.partitions_fn().await.unwrap(); + assert_eq!(partitions.result.unwrap().partitions, vec![2]); + + shutdown_tx + .send(()) + .expect("failed to send shutdown signal"); + server_handle.await.expect("failed to join server task"); + Ok(()) + } +} diff --git a/serving/source-sink/src/transformer.rs b/serving/source-sink/src/transformer.rs new file mode 100644 index 0000000000..bc0bbc46ae --- /dev/null +++ b/serving/source-sink/src/transformer.rs @@ -0,0 +1,133 @@ +use std::path::PathBuf; + +use tonic::transport::Channel; +use tonic::Request; + +use crate::error::Result; +use crate::message::Message; +use crate::shared::{connect_with_uds, prost_timestamp_from_utc, utc_from_timestamp}; +use crate::transformer::proto::SourceTransformRequest; + +pub mod proto { + tonic::include_proto!("sourcetransformer.v1"); +} + +#[derive(Clone)] +pub struct TransformerClient { + client: proto::source_transform_client::SourceTransformClient, +} + +impl TransformerClient { + pub(crate) async fn connect(uds_path: PathBuf) -> Result { + let channel = connect_with_uds(uds_path).await?; + let client = proto::source_transform_client::SourceTransformClient::new(channel); + Ok(Self { client }) + } + + pub(crate) async fn transform_fn(&mut self, message: Message) -> Result> { + // fields which will not be changed + let offset = message.offset.clone(); + let headers = message.headers.clone(); + + let request = SourceTransformRequest { + keys: message.keys, + value: message.value, + event_time: prost_timestamp_from_utc(message.event_time), + watermark: None, + headers: message.headers, + }; + + let response = self.client.source_transform_fn(request).await?.into_inner(); + + let mut messages = Vec::new(); + for result in response.results { + let message = Message { + keys: result.keys, + value: result.value, + offset: offset.clone(), + event_time: utc_from_timestamp(result.event_time), + headers: headers.clone(), + }; + messages.push(message); + } + + Ok(messages) + } + + pub(crate) async fn is_ready(&mut self) -> Result { + let request = Request::new(()); + let response = self.client.is_ready(request).await?.into_inner(); + Ok(response) + } +} + +#[cfg(test)] +mod tests { + use std::error::Error; + + use numaflow::sourcetransform; + use tempfile::TempDir; + + use crate::transformer::TransformerClient; + + struct NowCat; + + #[tonic::async_trait] + impl sourcetransform::SourceTransformer for NowCat { + async fn transform( + &self, + input: sourcetransform::SourceTransformRequest, + ) -> Vec { + let message = sourcetransform::Message::new(input.value, chrono::offset::Utc::now()) + .keys(input.keys) + .tags(vec![]); + vec![message] + } + } + + #[tokio::test] + async fn transformer_operations() -> Result<(), Box> { + let (shutdown_tx, shutdown_rx) = tokio::sync::oneshot::channel(); + let tmp_dir = TempDir::new()?; + let sock_file = tmp_dir.path().join("sourcetransform.sock"); + + let server_socket = sock_file.clone(); + let handle = tokio::spawn(async move { + let server_info_file = tmp_dir.path().join("sourcetransformer-server-info"); + sourcetransform::Server::new(NowCat) + .with_socket_file(server_socket) + .with_server_info_file(server_info_file) + .start_with_shutdown(shutdown_rx) + .await + .expect("server failed"); + }); + + // wait for the server to start + tokio::time::sleep(tokio::time::Duration::from_millis(100)).await; + + let mut client = TransformerClient::connect(sock_file).await?; + + let message = crate::message::Message { + keys: vec!["first".into(), "second".into()], + value: "hello".into(), + offset: crate::message::Offset { + partition_id: 0, + offset: "0".into(), + }, + event_time: chrono::Utc::now(), + headers: Default::default(), + }; + + let resp = client.is_ready().await?; + assert_eq!(resp.ready, true); + + let resp = client.transform_fn(message).await?; + assert_eq!(resp.len(), 1); + + shutdown_tx + .send(()) + .expect("failed to send shutdown signal"); + handle.await.expect("failed to join server task"); + Ok(()) + } +} From 17609da67a816b58adea871573d30919bb599ee2 Mon Sep 17 00:00:00 2001 From: Vigith Maurice Date: Mon, 5 Aug 2024 21:35:25 -0700 Subject: [PATCH 02/15] chore: some changes Signed-off-by: Vigith Maurice --- serving/source-sink/src/forwarder.rs | 2 + serving/source-sink/src/lib.rs | 92 +++++++++++++++++++++++++- serving/source-sink/src/main.rs | 2 +- serving/source-sink/src/sink.rs | 2 + serving/source-sink/src/source.rs | 7 +- serving/source-sink/src/transformer.rs | 1 + 6 files changed, 99 insertions(+), 7 deletions(-) diff --git a/serving/source-sink/src/forwarder.rs b/serving/source-sink/src/forwarder.rs index 8db6c41af3..31b2d41e14 100644 --- a/serving/source-sink/src/forwarder.rs +++ b/serving/source-sink/src/forwarder.rs @@ -7,6 +7,8 @@ use crate::sink::SinkClient; use crate::source::SourceClient; use crate::transformer::TransformerClient; +/// Forwarder is responsible for reading messages from the source, applying transformation if +/// transformer is present, and writing the messages to the sinK and acknowledging the messages. pub(crate) struct Forwarder { source_client: SourceClient, sink_client: SinkClient, diff --git a/serving/source-sink/src/lib.rs b/serving/source-sink/src/lib.rs index 0662434de1..7d788f279c 100644 --- a/serving/source-sink/src/lib.rs +++ b/serving/source-sink/src/lib.rs @@ -50,7 +50,7 @@ const TRANSFORMER_SOCKET: &str = "/var/run/numaflow/sourcetransform.sock"; const TIMEOUT_IN_MS: u32 = 1000; const BATCH_SIZE: u64 = 500; -pub async fn run_forwarder() -> Result<()> { +pub async fn run_forwarder(custom_shutdown_rx: Option>) -> Result<()> { let mut source_client = SourceClient::connect(SOURCE_SOCKET.into()).await?; let mut sink_client = SinkClient::connect(SINK_SOCKET.into()).await?; let mut transformer_client = if env::var("NUMAFLOW_TRANSFORMER").is_ok() { @@ -85,7 +85,7 @@ pub async fn run_forwarder() -> Result<()> { }); let shutdown_handle: JoinHandle> = tokio::spawn(async move { - shutdown_signal().await; + shutdown_signal(custom_shutdown_rx).await; shutdown_tx .send(()) .map_err(|_| Error::ForwarderError("Failed to send shutdown signal".to_string()))?; @@ -135,7 +135,7 @@ async fn wait_until_ready( Ok(()) } -async fn shutdown_signal() { +async fn shutdown_signal(shutdown_rx: Option>) { let ctrl_c = async { signal::ctrl_c() .await @@ -149,8 +149,94 @@ async fn shutdown_signal() { .await; }; + let custom_shutdown = async { + if let Some(rx) = shutdown_rx { + rx.await.ok(); + } + }; + tokio::select! { _ = ctrl_c => {}, _ = terminate => {}, + _ = custom_shutdown => {}, } } + +// #[cfg(test)] +// mod tests { +// use numaflow::{sink, source}; +// use numaflow::source::{Message, Offset, SourceReadRequest}; +// use tokio::sync::mpsc::Sender; +// +// struct SimpleSource; +// #[tonic::async_trait] +// impl source::Sourcer for SimpleSource { +// async fn read(&self, _: SourceReadRequest, _: Sender) { +// } +// +// async fn ack(&self, _: Vec) { +// } +// +// async fn pending(&self) -> usize { +// 0 +// } +// +// async fn partitions(&self) -> Option> { +// None +// } +// } +// +// struct SimpleSink; +// +// #[tonic::async_trait] +// impl sink::Sinker for SimpleSink { +// async fn sink(&self, input: tokio::sync::mpsc::Receiver) -> Vec { +// vec![] +// } +// } +// #[tokio::test] +// async fn run_forwarder() { +// let (src_shutdown_tx, src_shutdown_rx) = tokio::sync::oneshot::channel(); +// let tmp_dir = tempfile::TempDir::new().unwrap(); +// let sock_file = tmp_dir.path().join("source.sock"); +// +// let server_socket = sock_file.clone(); +// let src_server_handle = tokio::spawn(async move { +// let server_info_file = tmp_dir.path().join("source-server-info"); +// source::Server::new(SimpleSource) +// .with_socket_file(server_socket) +// .with_server_info_file(server_info_file) +// .start_with_shutdown(src_shutdown_rx) +// .await +// .unwrap(); +// }); +// +// let (sink_shutdown_tx, sink_shutdown_rx) = tokio::sync::oneshot::channel(); +// let tmp_dir = tempfile::TempDir::new().unwrap(); +// let sock_file = tmp_dir.path().join("sink.sock"); +// +// let server_socket = sock_file.clone(); +// let sink_server_handle = tokio::spawn(async move { +// let server_info_file = tmp_dir.path().join("sink-server-info"); +// sink::Server::new(SimpleSink) +// .with_socket_file(server_socket) +// .with_server_info_file(server_info_file) +// .start_with_shutdown(sink_shutdown_rx) +// .await +// .unwrap(); +// }); +// +// let (shutdown_tx, shutdown_rx) = tokio::sync::oneshot::channel(); +// +// let result = super::run_forwarder(Some(shutdown_rx)).await; +// println!("{:?}", result); +// assert!(result.is_ok()); +// +// // stop the source and sink servers +// src_shutdown_tx.send(()).unwrap(); +// sink_shutdown_tx.send(()).unwrap(); +// +// src_server_handle.await.unwrap(); +// sink_server_handle.await.unwrap(); +// } +// } \ No newline at end of file diff --git a/serving/source-sink/src/main.rs b/serving/source-sink/src/main.rs index 9fc093bb53..557c01c472 100644 --- a/serving/source-sink/src/main.rs +++ b/serving/source-sink/src/main.rs @@ -17,7 +17,7 @@ async fn main() { }); // Run the forwarder - if let Err(e) = run_forwarder().await { + if let Err(e) = run_forwarder(None).await { error!("Application error: {:?}", e); } } diff --git a/serving/source-sink/src/sink.rs b/serving/source-sink/src/sink.rs index 5a0094df5b..160fad2449 100644 --- a/serving/source-sink/src/sink.rs +++ b/serving/source-sink/src/sink.rs @@ -11,6 +11,7 @@ pub mod proto { tonic::include_proto!("sink.v1"); } +/// SinkClient is a client to interact with the sink server. pub struct SinkClient { client: proto::sink_client::SinkClient, } @@ -45,6 +46,7 @@ impl SinkClient { } }); + // TODO: retry for response with failure status let response = self .client .sink_fn(tokio_stream::wrappers::ReceiverStream::new(rx)) diff --git a/serving/source-sink/src/source.rs b/serving/source-sink/src/source.rs index dfdbe6b7e3..18395c486b 100644 --- a/serving/source-sink/src/source.rs +++ b/serving/source-sink/src/source.rs @@ -14,6 +14,7 @@ pub mod proto { tonic::include_proto!("source.v1"); } +/// SourceClient is a client to interact with the source server. #[derive(Debug, Clone)] pub(crate) struct SourceClient { client: proto::source_client::SourceClient, @@ -92,10 +93,10 @@ impl SourceClient { } #[allow(dead_code)] - pub(crate) async fn partitions_fn(&mut self) -> Result { + pub(crate) async fn partitions_fn(&mut self) -> Result> { let request = Request::new(()); let response = self.client.partitions_fn(request).await?.into_inner(); - Ok(response) + Ok(response.result.map_or(vec![], |r| r.partitions)) } pub(crate) async fn is_ready(&mut self) -> Result { @@ -215,7 +216,7 @@ mod tests { assert_eq!(pending.result.unwrap().count, 0); let partitions = source_client.partitions_fn().await.unwrap(); - assert_eq!(partitions.result.unwrap().partitions, vec![2]); + assert_eq!(partitions, vec![2]); shutdown_tx .send(()) diff --git a/serving/source-sink/src/transformer.rs b/serving/source-sink/src/transformer.rs index bc0bbc46ae..57a47681de 100644 --- a/serving/source-sink/src/transformer.rs +++ b/serving/source-sink/src/transformer.rs @@ -12,6 +12,7 @@ pub mod proto { tonic::include_proto!("sourcetransformer.v1"); } +/// TransformerClient is a client to interact with the transformer server. #[derive(Clone)] pub struct TransformerClient { client: proto::source_transform_client::SourceTransformClient, From d097d31b01f5733baff61adac9e4997a7bc916df Mon Sep 17 00:00:00 2001 From: Yashash H L Date: Tue, 6 Aug 2024 19:08:06 +0530 Subject: [PATCH 03/15] introducing config and wait for server info Signed-off-by: Yashash H L --- .../numaflow-models/src/apis/configuration.rs | 6 +- serving/numaflow-models/src/apis/mod.rs | 17 +- serving/numaflow-models/src/lib.rs | 2 +- .../src/models/abstract_pod_template.rs | 11 +- .../src/models/abstract_sink.rs | 7 +- .../src/models/abstract_vertex.rs | 22 +- .../src/models/authorization.rs | 11 +- .../numaflow-models/src/models/basic_auth.rs | 6 +- .../numaflow-models/src/models/blackhole.rs | 12 +- .../src/models/buffer_service_config.rs | 7 +- .../src/models/combined_edge.rs | 23 +- .../numaflow-models/src/models/container.rs | 6 +- .../src/models/container_builder.rs | 17 +- .../src/models/container_template.rs | 6 +- .../src/models/daemon_template.rs | 17 +- serving/numaflow-models/src/models/edge.rs | 7 +- .../src/models/fixed_window.rs | 6 +- .../src/models/forward_conditions.rs | 7 +- .../numaflow-models/src/models/function.rs | 7 +- .../src/models/generator_source.rs | 7 +- .../src/models/get_container_req.rs | 16 +- .../src/models/get_daemon_deployment_req.rs | 15 +- .../models/get_jet_stream_service_spec_req.rs | 15 +- .../get_jet_stream_stateful_set_spec_req.rs | 25 +- .../src/models/get_redis_service_spec_req.rs | 13 +- .../models/get_redis_stateful_set_spec_req.rs | 26 +- .../models/get_side_input_deployment_req.rs | 15 +- .../src/models/get_vertex_pod_spec_req.rs | 18 +- .../numaflow-models/src/models/group_by.rs | 6 +- serving/numaflow-models/src/models/gssapi.rs | 17 +- .../numaflow-models/src/models/http_source.rs | 7 +- .../numaflow-models/src/models/idle_source.rs | 7 +- .../src/models/inter_step_buffer_service.rs | 7 +- .../models/inter_step_buffer_service_list.rs | 11 +- .../models/inter_step_buffer_service_spec.rs | 7 +- .../inter_step_buffer_service_status.rs | 7 +- .../src/models/jet_stream_buffer_service.rs | 22 +- .../src/models/jet_stream_config.rs | 7 +- .../src/models/jet_stream_source.rs | 7 +- .../src/models/job_template.rs | 17 +- .../numaflow-models/src/models/kafka_sink.rs | 7 +- .../src/models/kafka_source.rs | 7 +- .../numaflow-models/src/models/lifecycle.rs | 17 +- serving/numaflow-models/src/models/log.rs | 13 +- .../numaflow-models/src/models/metadata.rs | 7 +- .../src/models/native_redis.rs | 32 ++- .../numaflow-models/src/models/nats_auth.rs | 6 +- .../numaflow-models/src/models/nats_source.rs | 7 +- .../numaflow-models/src/models/no_store.rs | 12 +- .../numaflow-models/src/models/pbq_storage.rs | 11 +- .../src/models/persistence_strategy.rs | 6 +- .../numaflow-models/src/models/pipeline.rs | 7 +- .../src/models/pipeline_limits.rs | 7 +- .../src/models/pipeline_list.rs | 7 +- .../src/models/pipeline_spec.rs | 12 +- .../src/models/pipeline_status.rs | 7 +- .../src/models/redis_buffer_service.rs | 7 +- .../src/models/redis_config.rs | 7 +- .../src/models/redis_settings.rs | 7 +- serving/numaflow-models/src/models/sasl.rs | 7 +- .../numaflow-models/src/models/sasl_plain.rs | 12 +- serving/numaflow-models/src/models/scale.rs | 31 ++- .../src/models/serving_source.rs | 6 +- .../src/models/serving_store.rs | 11 +- .../src/models/session_window.rs | 10 +- .../numaflow-models/src/models/side_input.rs | 12 +- .../src/models/side_input_trigger.rs | 7 +- .../models/side_inputs_manager_template.rs | 17 +- serving/numaflow-models/src/models/sink.rs | 7 +- .../src/models/sliding_window.rs | 6 +- serving/numaflow-models/src/models/source.rs | 7 +- serving/numaflow-models/src/models/status.rs | 10 +- .../src/models/tag_conditions.rs | 7 +- .../numaflow-models/src/models/templates.rs | 7 +- serving/numaflow-models/src/models/tls.rs | 7 +- .../numaflow-models/src/models/transformer.rs | 7 +- serving/numaflow-models/src/models/ud_sink.rs | 7 +- .../numaflow-models/src/models/ud_source.rs | 7 +- .../src/models/ud_transformer.rs | 7 +- serving/numaflow-models/src/models/udf.rs | 7 +- serving/numaflow-models/src/models/vertex.rs | 7 +- .../src/models/vertex_instance.rs | 6 +- .../src/models/vertex_limits.rs | 7 +- .../numaflow-models/src/models/vertex_list.rs | 7 +- .../numaflow-models/src/models/vertex_spec.rs | 27 +- .../src/models/vertex_status.rs | 7 +- .../src/models/vertex_template.rs | 17 +- .../numaflow-models/src/models/watermark.rs | 7 +- serving/numaflow-models/src/models/window.rs | 6 +- serving/source-sink/src/forwarder.rs | 42 ++- serving/source-sink/src/lib.rs | 259 +++++++++++------- serving/source-sink/src/main.rs | 14 +- serving/source-sink/src/sink.rs | 46 +++- serving/source-sink/src/source.rs | 48 +++- serving/source-sink/src/transformer.rs | 45 ++- 95 files changed, 670 insertions(+), 729 deletions(-) diff --git a/serving/numaflow-models/src/apis/configuration.rs b/serving/numaflow-models/src/apis/configuration.rs index 5c65cc5721..da73e26e0b 100644 --- a/serving/numaflow-models/src/apis/configuration.rs +++ b/serving/numaflow-models/src/apis/configuration.rs @@ -4,12 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - #[derive(Debug, Clone)] pub struct Configuration { pub base_path: String, @@ -30,7 +28,6 @@ pub struct ApiKey { pub key: String, } - impl Configuration { pub fn new() -> Configuration { Configuration::default() @@ -47,7 +44,6 @@ impl Default for Configuration { oauth_access_token: None, bearer_access_token: None, api_key: None, - } } } diff --git a/serving/numaflow-models/src/apis/mod.rs b/serving/numaflow-models/src/apis/mod.rs index dccbc940fb..5fb0cf38c7 100644 --- a/serving/numaflow-models/src/apis/mod.rs +++ b/serving/numaflow-models/src/apis/mod.rs @@ -16,7 +16,7 @@ pub enum Error { ResponseError(ResponseContent), } -impl fmt::Display for Error { +impl fmt::Display for Error { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { let (module, e) = match self { Error::Reqwest(e) => ("reqwest", e.to_string()), @@ -28,7 +28,7 @@ impl fmt::Display for Error { } } -impl error::Error for Error { +impl error::Error for Error { fn source(&self) -> Option<&(dyn error::Error + 'static)> { Some(match self { Error::Reqwest(e) => e, @@ -39,19 +39,19 @@ impl error::Error for Error { } } -impl From for Error { +impl From for Error { fn from(e: reqwest::Error) -> Self { Error::Reqwest(e) } } -impl From for Error { +impl From for Error { fn from(e: serde_json::Error) -> Self { Error::Serde(e) } } -impl From for Error { +impl From for Error { fn from(e: std::io::Error) -> Self { Error::Io(e) } @@ -78,8 +78,10 @@ pub fn parse_deep_object(prefix: &str, value: &serde_json::Value) -> Vec<(String value, )); } - }, - serde_json::Value::String(s) => params.push((format!("{}[{}]", prefix, key), s.clone())), + } + serde_json::Value::String(s) => { + params.push((format!("{}[{}]", prefix, key), s.clone())) + } _ => params.push((format!("{}[{}]", prefix, key), value.to_string())), } } @@ -90,5 +92,4 @@ pub fn parse_deep_object(prefix: &str, value: &serde_json::Value) -> Vec<(String unimplemented!("Only objects are supported with style=deepObject") } - pub mod configuration; diff --git a/serving/numaflow-models/src/lib.rs b/serving/numaflow-models/src/lib.rs index c1dd666f79..fc22e4e4b9 100644 --- a/serving/numaflow-models/src/lib.rs +++ b/serving/numaflow-models/src/lib.rs @@ -1,10 +1,10 @@ #[macro_use] extern crate serde_derive; +extern crate reqwest; extern crate serde; extern crate serde_json; extern crate url; -extern crate reqwest; pub mod apis; pub mod models; diff --git a/serving/numaflow-models/src/models/abstract_pod_template.rs b/serving/numaflow-models/src/models/abstract_pod_template.rs index fd19a7b425..6de4149ab1 100644 --- a/serving/numaflow-models/src/models/abstract_pod_template.rs +++ b/serving/numaflow-models/src/models/abstract_pod_template.rs @@ -4,20 +4,21 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// AbstractPodTemplate : AbstractPodTemplate provides a template for pod customization in vertices, daemon deployments and so on. - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct AbstractPodTemplate { #[serde(rename = "affinity", skip_serializing_if = "Option::is_none")] pub affinity: Option, /// AutomountServiceAccountToken indicates whether a service account token should be automatically mounted. - #[serde(rename = "automountServiceAccountToken", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "automountServiceAccountToken", + skip_serializing_if = "Option::is_none" + )] pub automount_service_account_token: Option, #[serde(rename = "dnsConfig", skip_serializing_if = "Option::is_none")] pub dns_config: Option, @@ -71,5 +72,3 @@ impl AbstractPodTemplate { } } } - - diff --git a/serving/numaflow-models/src/models/abstract_sink.rs b/serving/numaflow-models/src/models/abstract_sink.rs index 48bf8c8276..ab8f2ba058 100644 --- a/serving/numaflow-models/src/models/abstract_sink.rs +++ b/serving/numaflow-models/src/models/abstract_sink.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct AbstractSink { #[serde(rename = "blackhole", skip_serializing_if = "Option::is_none")] @@ -33,5 +30,3 @@ impl AbstractSink { } } } - - diff --git a/serving/numaflow-models/src/models/abstract_vertex.rs b/serving/numaflow-models/src/models/abstract_vertex.rs index 1ee864662e..62fc1240ce 100644 --- a/serving/numaflow-models/src/models/abstract_vertex.rs +++ b/serving/numaflow-models/src/models/abstract_vertex.rs @@ -4,19 +4,19 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct AbstractVertex { #[serde(rename = "affinity", skip_serializing_if = "Option::is_none")] pub affinity: Option, /// AutomountServiceAccountToken indicates whether a service account token should be automatically mounted. - #[serde(rename = "automountServiceAccountToken", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "automountServiceAccountToken", + skip_serializing_if = "Option::is_none" + )] pub automount_service_account_token: Option, #[serde(rename = "containerTemplate", skip_serializing_if = "Option::is_none")] pub container_template: Option>, @@ -28,7 +28,10 @@ pub struct AbstractVertex { /// ImagePullSecrets is an optional list of references to secrets in the same namespace to use for pulling any of the images used by this PodSpec. If specified, these secrets will be passed to individual puller implementations for them to use. For example, in the case of docker, only DockerConfig type secrets are honored. More info: https://kubernetes.io/docs/concepts/containers/images#specifying-imagepullsecrets-on-a-pod #[serde(rename = "imagePullSecrets", skip_serializing_if = "Option::is_none")] pub image_pull_secrets: Option>, - #[serde(rename = "initContainerTemplate", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "initContainerTemplate", + skip_serializing_if = "Option::is_none" + )] pub init_container_template: Option>, /// List of customized init containers belonging to the pod. More info: https://kubernetes.io/docs/concepts/workloads/pods/init-containers/ #[serde(rename = "initContainers", skip_serializing_if = "Option::is_none")] @@ -64,7 +67,10 @@ pub struct AbstractVertex { /// Names of the side inputs used in this vertex. #[serde(rename = "sideInputs", skip_serializing_if = "Option::is_none")] pub side_inputs: Option>, - #[serde(rename = "sideInputsContainerTemplate", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "sideInputsContainerTemplate", + skip_serializing_if = "Option::is_none" + )] pub side_inputs_container_template: Option>, /// List of customized sidecar containers belonging to the pod. #[serde(rename = "sidecars", skip_serializing_if = "Option::is_none")] @@ -115,5 +121,3 @@ impl AbstractVertex { } } } - - diff --git a/serving/numaflow-models/src/models/authorization.rs b/serving/numaflow-models/src/models/authorization.rs index 6589dc82f4..f1242768d2 100644 --- a/serving/numaflow-models/src/models/authorization.rs +++ b/serving/numaflow-models/src/models/authorization.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Authorization { #[serde(rename = "token", skip_serializing_if = "Option::is_none")] @@ -19,10 +16,6 @@ pub struct Authorization { impl Authorization { pub fn new() -> Authorization { - Authorization { - token: None, - } + Authorization { token: None } } } - - diff --git a/serving/numaflow-models/src/models/basic_auth.rs b/serving/numaflow-models/src/models/basic_auth.rs index e7a4e7d2c7..1b5eea25d8 100644 --- a/serving/numaflow-models/src/models/basic_auth.rs +++ b/serving/numaflow-models/src/models/basic_auth.rs @@ -4,14 +4,12 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// BasicAuth : BasicAuth represents the basic authentication approach which contains a user name and a password. - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct BasicAuth { #[serde(rename = "password", skip_serializing_if = "Option::is_none")] @@ -29,5 +27,3 @@ impl BasicAuth { } } } - - diff --git a/serving/numaflow-models/src/models/blackhole.rs b/serving/numaflow-models/src/models/blackhole.rs index f84586c042..145871aa16 100644 --- a/serving/numaflow-models/src/models/blackhole.rs +++ b/serving/numaflow-models/src/models/blackhole.rs @@ -4,24 +4,18 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// Blackhole : Blackhole is a sink to emulate /dev/null - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] -pub struct Blackhole { -} +pub struct Blackhole {} impl Blackhole { /// Blackhole is a sink to emulate /dev/null pub fn new() -> Blackhole { - Blackhole { - } + Blackhole {} } } - - diff --git a/serving/numaflow-models/src/models/buffer_service_config.rs b/serving/numaflow-models/src/models/buffer_service_config.rs index bad5eec796..24e621c1ea 100644 --- a/serving/numaflow-models/src/models/buffer_service_config.rs +++ b/serving/numaflow-models/src/models/buffer_service_config.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct BufferServiceConfig { #[serde(rename = "jetstream", skip_serializing_if = "Option::is_none")] @@ -27,5 +24,3 @@ impl BufferServiceConfig { } } } - - diff --git a/serving/numaflow-models/src/models/combined_edge.rs b/serving/numaflow-models/src/models/combined_edge.rs index b7fdf05446..94d478727c 100644 --- a/serving/numaflow-models/src/models/combined_edge.rs +++ b/serving/numaflow-models/src/models/combined_edge.rs @@ -4,14 +4,12 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// CombinedEdge : CombinedEdge is a combination of Edge and some other properties such as vertex type, partitions, limits. It's used to decorate the fromEdges and toEdges of the generated Vertex objects, so that in the vertex pod, it knows the properties of the connected vertices, for example, how many partitioned buffers I should write to, what is the write buffer length, etc. - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct CombinedEdge { #[serde(rename = "conditions", skip_serializing_if = "Option::is_none")] @@ -21,7 +19,10 @@ pub struct CombinedEdge { #[serde(rename = "fromVertexLimits", skip_serializing_if = "Option::is_none")] pub from_vertex_limits: Option>, /// The number of partitions of the from vertex, if not provided, the default value is set to \"1\". - #[serde(rename = "fromVertexPartitionCount", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "fromVertexPartitionCount", + skip_serializing_if = "Option::is_none" + )] pub from_vertex_partition_count: Option, /// From vertex type. #[serde(rename = "fromVertexType")] @@ -34,7 +35,10 @@ pub struct CombinedEdge { #[serde(rename = "toVertexLimits", skip_serializing_if = "Option::is_none")] pub to_vertex_limits: Option>, /// The number of partitions of the to vertex, if not provided, the default value is set to \"1\". - #[serde(rename = "toVertexPartitionCount", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "toVertexPartitionCount", + skip_serializing_if = "Option::is_none" + )] pub to_vertex_partition_count: Option, /// To vertex type. #[serde(rename = "toVertexType")] @@ -43,7 +47,12 @@ pub struct CombinedEdge { impl CombinedEdge { /// CombinedEdge is a combination of Edge and some other properties such as vertex type, partitions, limits. It's used to decorate the fromEdges and toEdges of the generated Vertex objects, so that in the vertex pod, it knows the properties of the connected vertices, for example, how many partitioned buffers I should write to, what is the write buffer length, etc. - pub fn new(from: String, from_vertex_type: String, to: String, to_vertex_type: String) -> CombinedEdge { + pub fn new( + from: String, + from_vertex_type: String, + to: String, + to_vertex_type: String, + ) -> CombinedEdge { CombinedEdge { conditions: None, from, @@ -58,5 +67,3 @@ impl CombinedEdge { } } } - - diff --git a/serving/numaflow-models/src/models/container.rs b/serving/numaflow-models/src/models/container.rs index 22d8643134..4d3d2730a0 100644 --- a/serving/numaflow-models/src/models/container.rs +++ b/serving/numaflow-models/src/models/container.rs @@ -4,14 +4,12 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// Container : Container is used to define the container properties for user-defined functions, sinks, etc. - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Container { #[serde(rename = "args", skip_serializing_if = "Option::is_none")] @@ -50,5 +48,3 @@ impl Container { } } } - - diff --git a/serving/numaflow-models/src/models/container_builder.rs b/serving/numaflow-models/src/models/container_builder.rs index 47a84c6fe7..35898065d5 100644 --- a/serving/numaflow-models/src/models/container_builder.rs +++ b/serving/numaflow-models/src/models/container_builder.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct ContainerBuilder { /// Arguments to the entrypoint. The container image's CMD is used if this is not provided. Variable references $(VAR_NAME) are expanded using the container's environment. If a variable cannot be resolved, the reference in the input string will be unchanged. Double $$ are reduced to a single $, which allows for escaping the $(VAR_NAME) syntax: i.e. \"$$(VAR_NAME)\" will produce the string literal \"$(VAR_NAME)\". Escaped references will never be expanded, regardless of whether the variable exists or not. Cannot be updated. More info: https://kubernetes.io/docs/tasks/inject-data-application/define-command-argument-container/#running-a-command-in-a-shell @@ -62,10 +59,16 @@ pub struct ContainerBuilder { #[serde(rename = "stdinOnce", skip_serializing_if = "Option::is_none")] pub stdin_once: Option, /// Optional: Path at which the file to which the container's termination message will be written is mounted into the container's filesystem. Message written is intended to be brief final status, such as an assertion failure message. Will be truncated by the node if greater than 4096 bytes. The total message length across all containers will be limited to 12kb. Defaults to /dev/termination-log. Cannot be updated. - #[serde(rename = "terminationMessagePath", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "terminationMessagePath", + skip_serializing_if = "Option::is_none" + )] pub termination_message_path: Option, /// Indicate how the termination message should be populated. File will use the contents of terminationMessagePath to populate the container status message on both success and failure. FallbackToLogsOnError will use the last chunk of container log output if the termination message file is empty and the container exited with an error. The log output is limited to 2048 bytes or 80 lines, whichever is smaller. Defaults to File. Cannot be updated. - #[serde(rename = "terminationMessagePolicy", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "terminationMessagePolicy", + skip_serializing_if = "Option::is_none" + )] pub termination_message_policy: Option, /// Whether this container should allocate a TTY for itself, also requires 'stdin' to be true. Default is false. #[serde(rename = "tty", skip_serializing_if = "Option::is_none")] @@ -111,5 +114,3 @@ impl ContainerBuilder { } } } - - diff --git a/serving/numaflow-models/src/models/container_template.rs b/serving/numaflow-models/src/models/container_template.rs index 737fbf7701..4c85b7a7f6 100644 --- a/serving/numaflow-models/src/models/container_template.rs +++ b/serving/numaflow-models/src/models/container_template.rs @@ -4,14 +4,12 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// ContainerTemplate : ContainerTemplate defines customized spec for a container - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct ContainerTemplate { #[serde(rename = "env", skip_serializing_if = "Option::is_none")] @@ -38,5 +36,3 @@ impl ContainerTemplate { } } } - - diff --git a/serving/numaflow-models/src/models/daemon_template.rs b/serving/numaflow-models/src/models/daemon_template.rs index 82c8e4ecc5..97290e6fac 100644 --- a/serving/numaflow-models/src/models/daemon_template.rs +++ b/serving/numaflow-models/src/models/daemon_template.rs @@ -4,19 +4,19 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct DaemonTemplate { #[serde(rename = "affinity", skip_serializing_if = "Option::is_none")] pub affinity: Option, /// AutomountServiceAccountToken indicates whether a service account token should be automatically mounted. - #[serde(rename = "automountServiceAccountToken", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "automountServiceAccountToken", + skip_serializing_if = "Option::is_none" + )] pub automount_service_account_token: Option, #[serde(rename = "containerTemplate", skip_serializing_if = "Option::is_none")] pub container_template: Option>, @@ -28,7 +28,10 @@ pub struct DaemonTemplate { /// ImagePullSecrets is an optional list of references to secrets in the same namespace to use for pulling any of the images used by this PodSpec. If specified, these secrets will be passed to individual puller implementations for them to use. For example, in the case of docker, only DockerConfig type secrets are honored. More info: https://kubernetes.io/docs/concepts/containers/images#specifying-imagepullsecrets-on-a-pod #[serde(rename = "imagePullSecrets", skip_serializing_if = "Option::is_none")] pub image_pull_secrets: Option>, - #[serde(rename = "initContainerTemplate", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "initContainerTemplate", + skip_serializing_if = "Option::is_none" + )] pub init_container_template: Option>, #[serde(rename = "metadata", skip_serializing_if = "Option::is_none")] pub metadata: Option>, @@ -79,5 +82,3 @@ impl DaemonTemplate { } } } - - diff --git a/serving/numaflow-models/src/models/edge.rs b/serving/numaflow-models/src/models/edge.rs index 33afab8014..95a3b9742b 100644 --- a/serving/numaflow-models/src/models/edge.rs +++ b/serving/numaflow-models/src/models/edge.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Edge { #[serde(rename = "conditions", skip_serializing_if = "Option::is_none")] @@ -34,5 +31,3 @@ impl Edge { } } } - - diff --git a/serving/numaflow-models/src/models/fixed_window.rs b/serving/numaflow-models/src/models/fixed_window.rs index e62eee6d1a..76a3a9dc1c 100644 --- a/serving/numaflow-models/src/models/fixed_window.rs +++ b/serving/numaflow-models/src/models/fixed_window.rs @@ -4,14 +4,12 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// FixedWindow : FixedWindow describes a fixed window - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct FixedWindow { #[serde(rename = "length", skip_serializing_if = "Option::is_none")] @@ -30,5 +28,3 @@ impl FixedWindow { } } } - - diff --git a/serving/numaflow-models/src/models/forward_conditions.rs b/serving/numaflow-models/src/models/forward_conditions.rs index c75cb03634..ea99358902 100644 --- a/serving/numaflow-models/src/models/forward_conditions.rs +++ b/serving/numaflow-models/src/models/forward_conditions.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct ForwardConditions { #[serde(rename = "tags")] @@ -24,5 +21,3 @@ impl ForwardConditions { } } } - - diff --git a/serving/numaflow-models/src/models/function.rs b/serving/numaflow-models/src/models/function.rs index 147fd444fe..8081c827fe 100644 --- a/serving/numaflow-models/src/models/function.rs +++ b/serving/numaflow-models/src/models/function.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Function { #[serde(rename = "args", skip_serializing_if = "Option::is_none")] @@ -30,5 +27,3 @@ impl Function { } } } - - diff --git a/serving/numaflow-models/src/models/generator_source.rs b/serving/numaflow-models/src/models/generator_source.rs index d22bb877da..268652d0e3 100644 --- a/serving/numaflow-models/src/models/generator_source.rs +++ b/serving/numaflow-models/src/models/generator_source.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct GeneratorSource { #[serde(rename = "duration", skip_serializing_if = "Option::is_none")] @@ -46,5 +43,3 @@ impl GeneratorSource { } } } - - diff --git a/serving/numaflow-models/src/models/get_container_req.rs b/serving/numaflow-models/src/models/get_container_req.rs index 5b2695f98f..2d97d73f1b 100644 --- a/serving/numaflow-models/src/models/get_container_req.rs +++ b/serving/numaflow-models/src/models/get_container_req.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct GetContainerReq { #[serde(rename = "env")] @@ -28,7 +25,14 @@ pub struct GetContainerReq { } impl GetContainerReq { - pub fn new(env: Vec, image: String, image_pull_policy: String, isb_svc_type: String, resources: k8s_openapi::api::core::v1::ResourceRequirements, volume_mounts: Vec) -> GetContainerReq { + pub fn new( + env: Vec, + image: String, + image_pull_policy: String, + isb_svc_type: String, + resources: k8s_openapi::api::core::v1::ResourceRequirements, + volume_mounts: Vec, + ) -> GetContainerReq { GetContainerReq { env, image, @@ -39,5 +43,3 @@ impl GetContainerReq { } } } - - diff --git a/serving/numaflow-models/src/models/get_daemon_deployment_req.rs b/serving/numaflow-models/src/models/get_daemon_deployment_req.rs index 4f8a7cc3b9..2c20b6eba2 100644 --- a/serving/numaflow-models/src/models/get_daemon_deployment_req.rs +++ b/serving/numaflow-models/src/models/get_daemon_deployment_req.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct GetDaemonDeploymentReq { #[serde(rename = "DefaultResources")] @@ -26,7 +23,13 @@ pub struct GetDaemonDeploymentReq { } impl GetDaemonDeploymentReq { - pub fn new(default_resources: k8s_openapi::api::core::v1::ResourceRequirements, env: Vec, isb_svc_type: String, image: String, pull_policy: String) -> GetDaemonDeploymentReq { + pub fn new( + default_resources: k8s_openapi::api::core::v1::ResourceRequirements, + env: Vec, + isb_svc_type: String, + image: String, + pull_policy: String, + ) -> GetDaemonDeploymentReq { GetDaemonDeploymentReq { default_resources, env, @@ -36,5 +39,3 @@ impl GetDaemonDeploymentReq { } } } - - diff --git a/serving/numaflow-models/src/models/get_jet_stream_service_spec_req.rs b/serving/numaflow-models/src/models/get_jet_stream_service_spec_req.rs index 9c7891afd4..e6168e7d3e 100644 --- a/serving/numaflow-models/src/models/get_jet_stream_service_spec_req.rs +++ b/serving/numaflow-models/src/models/get_jet_stream_service_spec_req.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct GetJetStreamServiceSpecReq { #[serde(rename = "ClientPort")] @@ -26,7 +23,13 @@ pub struct GetJetStreamServiceSpecReq { } impl GetJetStreamServiceSpecReq { - pub fn new(client_port: i32, cluster_port: i32, labels: ::std::collections::HashMap, metrics_port: i32, monitor_port: i32) -> GetJetStreamServiceSpecReq { + pub fn new( + client_port: i32, + cluster_port: i32, + labels: ::std::collections::HashMap, + metrics_port: i32, + monitor_port: i32, + ) -> GetJetStreamServiceSpecReq { GetJetStreamServiceSpecReq { client_port, cluster_port, @@ -36,5 +39,3 @@ impl GetJetStreamServiceSpecReq { } } } - - diff --git a/serving/numaflow-models/src/models/get_jet_stream_stateful_set_spec_req.rs b/serving/numaflow-models/src/models/get_jet_stream_stateful_set_spec_req.rs index 894ae495fd..ffa11c5bbe 100644 --- a/serving/numaflow-models/src/models/get_jet_stream_stateful_set_spec_req.rs +++ b/serving/numaflow-models/src/models/get_jet_stream_stateful_set_spec_req.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct GetJetStreamStatefulSetSpecReq { #[serde(rename = "ClientPort")] @@ -46,7 +43,23 @@ pub struct GetJetStreamStatefulSetSpecReq { } impl GetJetStreamStatefulSetSpecReq { - pub fn new(client_port: i32, cluster_port: i32, config_map_name: String, config_reloader_image: String, default_resources: k8s_openapi::api::core::v1::ResourceRequirements, labels: ::std::collections::HashMap, metrics_exporter_image: String, metrics_port: i32, monitor_port: i32, nats_image: String, pvc_name_if_needed: String, server_auth_secret_name: String, server_encryption_secret_name: String, service_name: String, start_command: String) -> GetJetStreamStatefulSetSpecReq { + pub fn new( + client_port: i32, + cluster_port: i32, + config_map_name: String, + config_reloader_image: String, + default_resources: k8s_openapi::api::core::v1::ResourceRequirements, + labels: ::std::collections::HashMap, + metrics_exporter_image: String, + metrics_port: i32, + monitor_port: i32, + nats_image: String, + pvc_name_if_needed: String, + server_auth_secret_name: String, + server_encryption_secret_name: String, + service_name: String, + start_command: String, + ) -> GetJetStreamStatefulSetSpecReq { GetJetStreamStatefulSetSpecReq { client_port, cluster_port, @@ -66,5 +79,3 @@ impl GetJetStreamStatefulSetSpecReq { } } } - - diff --git a/serving/numaflow-models/src/models/get_redis_service_spec_req.rs b/serving/numaflow-models/src/models/get_redis_service_spec_req.rs index 651a67d08e..df584ea71a 100644 --- a/serving/numaflow-models/src/models/get_redis_service_spec_req.rs +++ b/serving/numaflow-models/src/models/get_redis_service_spec_req.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct GetRedisServiceSpecReq { #[serde(rename = "Labels")] @@ -22,7 +19,11 @@ pub struct GetRedisServiceSpecReq { } impl GetRedisServiceSpecReq { - pub fn new(labels: ::std::collections::HashMap, redis_container_port: i32, sentinel_container_port: i32) -> GetRedisServiceSpecReq { + pub fn new( + labels: ::std::collections::HashMap, + redis_container_port: i32, + sentinel_container_port: i32, + ) -> GetRedisServiceSpecReq { GetRedisServiceSpecReq { labels, redis_container_port, @@ -30,5 +31,3 @@ impl GetRedisServiceSpecReq { } } } - - diff --git a/serving/numaflow-models/src/models/get_redis_stateful_set_spec_req.rs b/serving/numaflow-models/src/models/get_redis_stateful_set_spec_req.rs index 736ea82fa4..81cc11b7c9 100644 --- a/serving/numaflow-models/src/models/get_redis_stateful_set_spec_req.rs +++ b/serving/numaflow-models/src/models/get_redis_stateful_set_spec_req.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct GetRedisStatefulSetSpecReq { #[serde(rename = "ConfConfigMapName")] @@ -48,7 +45,24 @@ pub struct GetRedisStatefulSetSpecReq { } impl GetRedisStatefulSetSpecReq { - pub fn new(conf_config_map_name: String, credential_secret_name: String, default_resources: k8s_openapi::api::core::v1::ResourceRequirements, health_config_map_name: String, init_container_image: String, labels: ::std::collections::HashMap, metrics_exporter_image: String, pvc_name_if_needed: String, redis_container_port: i32, redis_image: String, redis_metrics_container_port: i32, scripts_config_map_name: String, sentinel_container_port: i32, sentinel_image: String, service_name: String, tls_enabled: bool) -> GetRedisStatefulSetSpecReq { + pub fn new( + conf_config_map_name: String, + credential_secret_name: String, + default_resources: k8s_openapi::api::core::v1::ResourceRequirements, + health_config_map_name: String, + init_container_image: String, + labels: ::std::collections::HashMap, + metrics_exporter_image: String, + pvc_name_if_needed: String, + redis_container_port: i32, + redis_image: String, + redis_metrics_container_port: i32, + scripts_config_map_name: String, + sentinel_container_port: i32, + sentinel_image: String, + service_name: String, + tls_enabled: bool, + ) -> GetRedisStatefulSetSpecReq { GetRedisStatefulSetSpecReq { conf_config_map_name, credential_secret_name, @@ -69,5 +83,3 @@ impl GetRedisStatefulSetSpecReq { } } } - - diff --git a/serving/numaflow-models/src/models/get_side_input_deployment_req.rs b/serving/numaflow-models/src/models/get_side_input_deployment_req.rs index a2d389d107..f548abcf4d 100644 --- a/serving/numaflow-models/src/models/get_side_input_deployment_req.rs +++ b/serving/numaflow-models/src/models/get_side_input_deployment_req.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct GetSideInputDeploymentReq { #[serde(rename = "DefaultResources")] @@ -26,7 +23,13 @@ pub struct GetSideInputDeploymentReq { } impl GetSideInputDeploymentReq { - pub fn new(default_resources: k8s_openapi::api::core::v1::ResourceRequirements, env: Vec, isb_svc_type: String, image: String, pull_policy: String) -> GetSideInputDeploymentReq { + pub fn new( + default_resources: k8s_openapi::api::core::v1::ResourceRequirements, + env: Vec, + isb_svc_type: String, + image: String, + pull_policy: String, + ) -> GetSideInputDeploymentReq { GetSideInputDeploymentReq { default_resources, env, @@ -36,5 +39,3 @@ impl GetSideInputDeploymentReq { } } } - - diff --git a/serving/numaflow-models/src/models/get_vertex_pod_spec_req.rs b/serving/numaflow-models/src/models/get_vertex_pod_spec_req.rs index 26f6185385..bd976cf1f9 100644 --- a/serving/numaflow-models/src/models/get_vertex_pod_spec_req.rs +++ b/serving/numaflow-models/src/models/get_vertex_pod_spec_req.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct GetVertexPodSpecReq { #[serde(rename = "DefaultResources")] @@ -32,7 +29,16 @@ pub struct GetVertexPodSpecReq { } impl GetVertexPodSpecReq { - pub fn new(default_resources: k8s_openapi::api::core::v1::ResourceRequirements, env: Vec, isb_svc_type: String, image: String, pipeline_spec: crate::models::PipelineSpec, pull_policy: String, serving_source_stream_name: String, side_inputs_store_name: String) -> GetVertexPodSpecReq { + pub fn new( + default_resources: k8s_openapi::api::core::v1::ResourceRequirements, + env: Vec, + isb_svc_type: String, + image: String, + pipeline_spec: crate::models::PipelineSpec, + pull_policy: String, + serving_source_stream_name: String, + side_inputs_store_name: String, + ) -> GetVertexPodSpecReq { GetVertexPodSpecReq { default_resources, env, @@ -45,5 +51,3 @@ impl GetVertexPodSpecReq { } } } - - diff --git a/serving/numaflow-models/src/models/group_by.rs b/serving/numaflow-models/src/models/group_by.rs index 605b3bc61a..915e8a5bca 100644 --- a/serving/numaflow-models/src/models/group_by.rs +++ b/serving/numaflow-models/src/models/group_by.rs @@ -4,14 +4,12 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// GroupBy : GroupBy indicates it is a reducer UDF - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct GroupBy { #[serde(rename = "allowedLateness", skip_serializing_if = "Option::is_none")] @@ -35,5 +33,3 @@ impl GroupBy { } } } - - diff --git a/serving/numaflow-models/src/models/gssapi.rs b/serving/numaflow-models/src/models/gssapi.rs index c5fe33aad6..d040f83cb7 100644 --- a/serving/numaflow-models/src/models/gssapi.rs +++ b/serving/numaflow-models/src/models/gssapi.rs @@ -4,20 +4,21 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// Gssapi : GSSAPI represents a SASL GSSAPI config - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Gssapi { /// valid inputs - KRB5_USER_AUTH, KRB5_KEYTAB_AUTH Possible enum values: - `\"KRB5_KEYTAB_AUTH\"` represents the password method KRB5KeytabAuth = \"KRB5_KEYTAB_AUTH\" = 2 - `\"KRB5_USER_AUTH\"` represents the password method KRB5UserAuth = \"KRB5_USER_AUTH\" = 1 #[serde(rename = "authType")] pub auth_type: AuthType, - #[serde(rename = "kerberosConfigSecret", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "kerberosConfigSecret", + skip_serializing_if = "Option::is_none" + )] pub kerberos_config_secret: Option, #[serde(rename = "keytabSecret", skip_serializing_if = "Option::is_none")] pub keytab_secret: Option, @@ -33,7 +34,12 @@ pub struct Gssapi { impl Gssapi { /// GSSAPI represents a SASL GSSAPI config - pub fn new(auth_type: AuthType, realm: String, service_name: String, username_secret: k8s_openapi::api::core::v1::SecretKeySelector) -> Gssapi { + pub fn new( + auth_type: AuthType, + realm: String, + service_name: String, + username_secret: k8s_openapi::api::core::v1::SecretKeySelector, + ) -> Gssapi { Gssapi { auth_type, kerberos_config_secret: None, @@ -60,4 +66,3 @@ impl Default for AuthType { Self::KeytabAuth } } - diff --git a/serving/numaflow-models/src/models/http_source.rs b/serving/numaflow-models/src/models/http_source.rs index 8a5b7c7c97..e5178c86e3 100644 --- a/serving/numaflow-models/src/models/http_source.rs +++ b/serving/numaflow-models/src/models/http_source.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct HttpSource { #[serde(rename = "auth", skip_serializing_if = "Option::is_none")] @@ -28,5 +25,3 @@ impl HttpSource { } } } - - diff --git a/serving/numaflow-models/src/models/idle_source.rs b/serving/numaflow-models/src/models/idle_source.rs index 3d593c5375..cbe629ae9f 100644 --- a/serving/numaflow-models/src/models/idle_source.rs +++ b/serving/numaflow-models/src/models/idle_source.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct IdleSource { #[serde(rename = "incrementBy", skip_serializing_if = "Option::is_none")] @@ -30,5 +27,3 @@ impl IdleSource { } } } - - diff --git a/serving/numaflow-models/src/models/inter_step_buffer_service.rs b/serving/numaflow-models/src/models/inter_step_buffer_service.rs index fc6154b756..8e6a9e20f4 100644 --- a/serving/numaflow-models/src/models/inter_step_buffer_service.rs +++ b/serving/numaflow-models/src/models/inter_step_buffer_service.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct InterStepBufferService { /// APIVersion defines the versioned schema of this representation of an object. Servers should convert recognized schemas to the latest internal value, and may reject unrecognized values. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#resources @@ -38,5 +35,3 @@ impl InterStepBufferService { } } } - - diff --git a/serving/numaflow-models/src/models/inter_step_buffer_service_list.rs b/serving/numaflow-models/src/models/inter_step_buffer_service_list.rs index e933727ce5..4ef7a04c49 100644 --- a/serving/numaflow-models/src/models/inter_step_buffer_service_list.rs +++ b/serving/numaflow-models/src/models/inter_step_buffer_service_list.rs @@ -4,14 +4,12 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// InterStepBufferServiceList : InterStepBufferServiceList is the list of InterStepBufferService resources - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct InterStepBufferServiceList { /// APIVersion defines the versioned schema of this representation of an object. Servers should convert recognized schemas to the latest internal value, and may reject unrecognized values. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#resources @@ -28,7 +26,10 @@ pub struct InterStepBufferServiceList { impl InterStepBufferServiceList { /// InterStepBufferServiceList is the list of InterStepBufferService resources - pub fn new(items: Vec, metadata: k8s_openapi::apimachinery::pkg::apis::meta::v1::ListMeta) -> InterStepBufferServiceList { + pub fn new( + items: Vec, + metadata: k8s_openapi::apimachinery::pkg::apis::meta::v1::ListMeta, + ) -> InterStepBufferServiceList { InterStepBufferServiceList { api_version: None, items, @@ -37,5 +38,3 @@ impl InterStepBufferServiceList { } } } - - diff --git a/serving/numaflow-models/src/models/inter_step_buffer_service_spec.rs b/serving/numaflow-models/src/models/inter_step_buffer_service_spec.rs index 79be40fa45..2de93da314 100644 --- a/serving/numaflow-models/src/models/inter_step_buffer_service_spec.rs +++ b/serving/numaflow-models/src/models/inter_step_buffer_service_spec.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct InterStepBufferServiceSpec { #[serde(rename = "jetstream", skip_serializing_if = "Option::is_none")] @@ -27,5 +24,3 @@ impl InterStepBufferServiceSpec { } } } - - diff --git a/serving/numaflow-models/src/models/inter_step_buffer_service_status.rs b/serving/numaflow-models/src/models/inter_step_buffer_service_status.rs index 6cf0a96745..43c7cd16cf 100644 --- a/serving/numaflow-models/src/models/inter_step_buffer_service_status.rs +++ b/serving/numaflow-models/src/models/inter_step_buffer_service_status.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct InterStepBufferServiceStatus { /// Conditions are the latest available observations of a resource's current state. @@ -40,5 +37,3 @@ impl InterStepBufferServiceStatus { } } } - - diff --git a/serving/numaflow-models/src/models/jet_stream_buffer_service.rs b/serving/numaflow-models/src/models/jet_stream_buffer_service.rs index 9441f6d156..de5392a051 100644 --- a/serving/numaflow-models/src/models/jet_stream_buffer_service.rs +++ b/serving/numaflow-models/src/models/jet_stream_buffer_service.rs @@ -4,19 +4,19 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct JetStreamBufferService { #[serde(rename = "affinity", skip_serializing_if = "Option::is_none")] pub affinity: Option, /// AutomountServiceAccountToken indicates whether a service account token should be automatically mounted. - #[serde(rename = "automountServiceAccountToken", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "automountServiceAccountToken", + skip_serializing_if = "Option::is_none" + )] pub automount_service_account_token: Option, /// Optional configuration for the streams, consumers and buckets to be created in this JetStream service, if specified, it will be merged with the default configuration in numaflow-controller-config. It accepts a YAML format configuration, it may include 4 sections, \"stream\", \"consumer\", \"otBucket\" and \"procBucket\". Available fields under \"stream\" include \"retention\" (e.g. interest, limits, workerQueue), \"maxMsgs\", \"maxAge\" (e.g. 72h), \"replicas\" (1, 3, 5), \"duplicates\" (e.g. 5m). Available fields under \"consumer\" include \"ackWait\" (e.g. 60s) Available fields under \"otBucket\" include \"maxValueSize\", \"history\", \"ttl\" (e.g. 72h), \"maxBytes\", \"replicas\" (1, 3, 5). Available fields under \"procBucket\" include \"maxValueSize\", \"history\", \"ttl\" (e.g. 72h), \"maxBytes\", \"replicas\" (1, 3, 5). #[serde(rename = "bufferConfig", skip_serializing_if = "Option::is_none")] @@ -36,7 +36,10 @@ pub struct JetStreamBufferService { pub image_pull_secrets: Option>, #[serde(rename = "metadata", skip_serializing_if = "Option::is_none")] pub metadata: Option>, - #[serde(rename = "metricsContainerTemplate", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "metricsContainerTemplate", + skip_serializing_if = "Option::is_none" + )] pub metrics_container_template: Option>, /// NodeSelector is a selector which must be true for the pod to fit on a node. Selector which must match a node's labels for the pod to be scheduled on that node. More info: https://kubernetes.io/docs/concepts/configuration/assign-pod-node/ #[serde(rename = "nodeSelector", skip_serializing_if = "Option::is_none")] @@ -49,7 +52,10 @@ pub struct JetStreamBufferService { /// If specified, indicates the Redis pod's priority. \"system-node-critical\" and \"system-cluster-critical\" are two special keywords which indicate the highest priorities with the former being the highest priority. Any other name must be defined by creating a PriorityClass object with that name. If not specified, the pod priority will be default or zero if there is no default. More info: https://kubernetes.io/docs/concepts/configuration/pod-priority-preemption/ #[serde(rename = "priorityClassName", skip_serializing_if = "Option::is_none")] pub priority_class_name: Option, - #[serde(rename = "reloaderContainerTemplate", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "reloaderContainerTemplate", + skip_serializing_if = "Option::is_none" + )] pub reloader_container_template: Option>, /// JetStream StatefulSet size #[serde(rename = "replicas", skip_serializing_if = "Option::is_none")] @@ -109,5 +115,3 @@ impl JetStreamBufferService { } } } - - diff --git a/serving/numaflow-models/src/models/jet_stream_config.rs b/serving/numaflow-models/src/models/jet_stream_config.rs index bc4c648c1a..d62ceab186 100644 --- a/serving/numaflow-models/src/models/jet_stream_config.rs +++ b/serving/numaflow-models/src/models/jet_stream_config.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct JetStreamConfig { #[serde(rename = "auth", skip_serializing_if = "Option::is_none")] @@ -35,5 +32,3 @@ impl JetStreamConfig { } } } - - diff --git a/serving/numaflow-models/src/models/jet_stream_source.rs b/serving/numaflow-models/src/models/jet_stream_source.rs index 6da52b583c..c1040259e3 100644 --- a/serving/numaflow-models/src/models/jet_stream_source.rs +++ b/serving/numaflow-models/src/models/jet_stream_source.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct JetStreamSource { #[serde(rename = "auth", skip_serializing_if = "Option::is_none")] @@ -35,5 +32,3 @@ impl JetStreamSource { } } } - - diff --git a/serving/numaflow-models/src/models/job_template.rs b/serving/numaflow-models/src/models/job_template.rs index dcae7d906a..9ffe8e162e 100644 --- a/serving/numaflow-models/src/models/job_template.rs +++ b/serving/numaflow-models/src/models/job_template.rs @@ -4,19 +4,19 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct JobTemplate { #[serde(rename = "affinity", skip_serializing_if = "Option::is_none")] pub affinity: Option, /// AutomountServiceAccountToken indicates whether a service account token should be automatically mounted. - #[serde(rename = "automountServiceAccountToken", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "automountServiceAccountToken", + skip_serializing_if = "Option::is_none" + )] pub automount_service_account_token: Option, /// Specifies the number of retries before marking this job failed. More info: https://kubernetes.io/docs/concepts/workloads/controllers/job/#pod-backoff-failure-policy Numaflow defaults to 20 #[serde(rename = "backoffLimit", skip_serializing_if = "Option::is_none")] @@ -54,7 +54,10 @@ pub struct JobTemplate { #[serde(rename = "tolerations", skip_serializing_if = "Option::is_none")] pub tolerations: Option>, /// ttlSecondsAfterFinished limits the lifetime of a Job that has finished execution (either Complete or Failed). If this field is set, ttlSecondsAfterFinished after the Job finishes, it is eligible to be automatically deleted. When the Job is being deleted, its lifecycle guarantees (e.g. finalizers) will be honored. If this field is unset, the Job won't be automatically deleted. If this field is set to zero, the Job becomes eligible to be deleted immediately after it finishes. Numaflow defaults to 30 - #[serde(rename = "ttlSecondsAfterFinished", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "ttlSecondsAfterFinished", + skip_serializing_if = "Option::is_none" + )] pub ttl_seconds_after_finished: Option, } @@ -80,5 +83,3 @@ impl JobTemplate { } } } - - diff --git a/serving/numaflow-models/src/models/kafka_sink.rs b/serving/numaflow-models/src/models/kafka_sink.rs index 3b2c730fa5..99dabd7a4e 100644 --- a/serving/numaflow-models/src/models/kafka_sink.rs +++ b/serving/numaflow-models/src/models/kafka_sink.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct KafkaSink { #[serde(rename = "brokers", skip_serializing_if = "Option::is_none")] @@ -36,5 +33,3 @@ impl KafkaSink { } } } - - diff --git a/serving/numaflow-models/src/models/kafka_source.rs b/serving/numaflow-models/src/models/kafka_source.rs index 078eeee69a..b0771d8e33 100644 --- a/serving/numaflow-models/src/models/kafka_source.rs +++ b/serving/numaflow-models/src/models/kafka_source.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct KafkaSource { #[serde(rename = "brokers", skip_serializing_if = "Option::is_none")] @@ -39,5 +36,3 @@ impl KafkaSource { } } } - - diff --git a/serving/numaflow-models/src/models/lifecycle.rs b/serving/numaflow-models/src/models/lifecycle.rs index 3c1a478171..75065cf816 100644 --- a/serving/numaflow-models/src/models/lifecycle.rs +++ b/serving/numaflow-models/src/models/lifecycle.rs @@ -4,23 +4,26 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Lifecycle { /// DeleteGracePeriodSeconds used to delete pipeline gracefully - #[serde(rename = "deleteGracePeriodSeconds", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "deleteGracePeriodSeconds", + skip_serializing_if = "Option::is_none" + )] pub delete_grace_period_seconds: Option, /// DesiredPhase used to bring the pipeline from current phase to desired phase #[serde(rename = "desiredPhase", skip_serializing_if = "Option::is_none")] pub desired_phase: Option, /// PauseGracePeriodSeconds used to pause pipeline gracefully - #[serde(rename = "pauseGracePeriodSeconds", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "pauseGracePeriodSeconds", + skip_serializing_if = "Option::is_none" + )] pub pause_grace_period_seconds: Option, } @@ -33,5 +36,3 @@ impl Lifecycle { } } } - - diff --git a/serving/numaflow-models/src/models/log.rs b/serving/numaflow-models/src/models/log.rs index 141bb39e94..c1452a70ff 100644 --- a/serving/numaflow-models/src/models/log.rs +++ b/serving/numaflow-models/src/models/log.rs @@ -4,22 +4,15 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] -pub struct Log { -} +pub struct Log {} impl Log { pub fn new() -> Log { - Log { - } + Log {} } } - - diff --git a/serving/numaflow-models/src/models/metadata.rs b/serving/numaflow-models/src/models/metadata.rs index 13c95358f6..56c214a01e 100644 --- a/serving/numaflow-models/src/models/metadata.rs +++ b/serving/numaflow-models/src/models/metadata.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Metadata { #[serde(rename = "annotations", skip_serializing_if = "Option::is_none")] @@ -27,5 +24,3 @@ impl Metadata { } } } - - diff --git a/serving/numaflow-models/src/models/native_redis.rs b/serving/numaflow-models/src/models/native_redis.rs index c538f13b2a..a22fb37436 100644 --- a/serving/numaflow-models/src/models/native_redis.rs +++ b/serving/numaflow-models/src/models/native_redis.rs @@ -4,19 +4,19 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct NativeRedis { #[serde(rename = "affinity", skip_serializing_if = "Option::is_none")] pub affinity: Option, /// AutomountServiceAccountToken indicates whether a service account token should be automatically mounted. - #[serde(rename = "automountServiceAccountToken", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "automountServiceAccountToken", + skip_serializing_if = "Option::is_none" + )] pub automount_service_account_token: Option, #[serde(rename = "dnsConfig", skip_serializing_if = "Option::is_none")] pub dns_config: Option, @@ -26,11 +26,17 @@ pub struct NativeRedis { /// ImagePullSecrets is an optional list of references to secrets in the same namespace to use for pulling any of the images used by this PodSpec. If specified, these secrets will be passed to individual puller implementations for them to use. For example, in the case of docker, only DockerConfig type secrets are honored. More info: https://kubernetes.io/docs/concepts/containers/images#specifying-imagepullsecrets-on-a-pod #[serde(rename = "imagePullSecrets", skip_serializing_if = "Option::is_none")] pub image_pull_secrets: Option>, - #[serde(rename = "initContainerTemplate", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "initContainerTemplate", + skip_serializing_if = "Option::is_none" + )] pub init_container_template: Option>, #[serde(rename = "metadata", skip_serializing_if = "Option::is_none")] pub metadata: Option>, - #[serde(rename = "metricsContainerTemplate", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "metricsContainerTemplate", + skip_serializing_if = "Option::is_none" + )] pub metrics_container_template: Option>, /// NodeSelector is a selector which must be true for the pod to fit on a node. Selector which must match a node's labels for the pod to be scheduled on that node. More info: https://kubernetes.io/docs/concepts/configuration/assign-pod-node/ #[serde(rename = "nodeSelector", skip_serializing_if = "Option::is_none")] @@ -43,7 +49,10 @@ pub struct NativeRedis { /// If specified, indicates the Redis pod's priority. \"system-node-critical\" and \"system-cluster-critical\" are two special keywords which indicate the highest priorities with the former being the highest priority. Any other name must be defined by creating a PriorityClass object with that name. If not specified, the pod priority will be default or zero if there is no default. More info: https://kubernetes.io/docs/concepts/configuration/pod-priority-preemption/ #[serde(rename = "priorityClassName", skip_serializing_if = "Option::is_none")] pub priority_class_name: Option, - #[serde(rename = "redisContainerTemplate", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "redisContainerTemplate", + skip_serializing_if = "Option::is_none" + )] pub redis_container_template: Option>, /// Redis StatefulSet size #[serde(rename = "replicas", skip_serializing_if = "Option::is_none")] @@ -53,7 +62,10 @@ pub struct NativeRedis { pub runtime_class_name: Option, #[serde(rename = "securityContext", skip_serializing_if = "Option::is_none")] pub security_context: Option, - #[serde(rename = "sentinelContainerTemplate", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "sentinelContainerTemplate", + skip_serializing_if = "Option::is_none" + )] pub sentinel_container_template: Option>, /// ServiceAccountName applied to the pod #[serde(rename = "serviceAccountName", skip_serializing_if = "Option::is_none")] @@ -95,5 +107,3 @@ impl NativeRedis { } } } - - diff --git a/serving/numaflow-models/src/models/nats_auth.rs b/serving/numaflow-models/src/models/nats_auth.rs index 8c5c977109..7b085650f9 100644 --- a/serving/numaflow-models/src/models/nats_auth.rs +++ b/serving/numaflow-models/src/models/nats_auth.rs @@ -4,14 +4,12 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// NatsAuth : NatsAuth defines how to authenticate the nats access - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct NatsAuth { #[serde(rename = "basic", skip_serializing_if = "Option::is_none")] @@ -32,5 +30,3 @@ impl NatsAuth { } } } - - diff --git a/serving/numaflow-models/src/models/nats_source.rs b/serving/numaflow-models/src/models/nats_source.rs index d60f175c23..666fcbb884 100644 --- a/serving/numaflow-models/src/models/nats_source.rs +++ b/serving/numaflow-models/src/models/nats_source.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct NatsSource { #[serde(rename = "auth", skip_serializing_if = "Option::is_none")] @@ -39,5 +36,3 @@ impl NatsSource { } } } - - diff --git a/serving/numaflow-models/src/models/no_store.rs b/serving/numaflow-models/src/models/no_store.rs index 187e93d96b..f91247af2f 100644 --- a/serving/numaflow-models/src/models/no_store.rs +++ b/serving/numaflow-models/src/models/no_store.rs @@ -4,24 +4,18 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// NoStore : NoStore means there will be no persistence storage and there will be data loss during pod restarts. Use this option only if you do not care about correctness (e.g., approx statistics pipeline like sampling rate, etc.). - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] -pub struct NoStore { -} +pub struct NoStore {} impl NoStore { /// NoStore means there will be no persistence storage and there will be data loss during pod restarts. Use this option only if you do not care about correctness (e.g., approx statistics pipeline like sampling rate, etc.). pub fn new() -> NoStore { - NoStore { - } + NoStore {} } } - - diff --git a/serving/numaflow-models/src/models/pbq_storage.rs b/serving/numaflow-models/src/models/pbq_storage.rs index ea9cb9514e..c01ffe9eba 100644 --- a/serving/numaflow-models/src/models/pbq_storage.rs +++ b/serving/numaflow-models/src/models/pbq_storage.rs @@ -4,21 +4,22 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// PbqStorage : PBQStorage defines the persistence configuration for a vertex. - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct PbqStorage { #[serde(rename = "emptyDir", skip_serializing_if = "Option::is_none")] pub empty_dir: Option, #[serde(rename = "no_store", skip_serializing_if = "Option::is_none")] pub no_store: Option>, - #[serde(rename = "persistentVolumeClaim", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "persistentVolumeClaim", + skip_serializing_if = "Option::is_none" + )] pub persistent_volume_claim: Option>, } @@ -32,5 +33,3 @@ impl PbqStorage { } } } - - diff --git a/serving/numaflow-models/src/models/persistence_strategy.rs b/serving/numaflow-models/src/models/persistence_strategy.rs index 49351a146d..3541e83b91 100644 --- a/serving/numaflow-models/src/models/persistence_strategy.rs +++ b/serving/numaflow-models/src/models/persistence_strategy.rs @@ -4,14 +4,12 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// PersistenceStrategy : PersistenceStrategy defines the strategy of persistence - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct PersistenceStrategy { /// Available access modes such as ReadWriteOnce, ReadWriteMany https://kubernetes.io/docs/concepts/storage/persistent-volumes/#access-modes @@ -34,5 +32,3 @@ impl PersistenceStrategy { } } } - - diff --git a/serving/numaflow-models/src/models/pipeline.rs b/serving/numaflow-models/src/models/pipeline.rs index e6d7054b36..52db105f24 100644 --- a/serving/numaflow-models/src/models/pipeline.rs +++ b/serving/numaflow-models/src/models/pipeline.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Pipeline { /// APIVersion defines the versioned schema of this representation of an object. Servers should convert recognized schemas to the latest internal value, and may reject unrecognized values. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#resources @@ -38,5 +35,3 @@ impl Pipeline { } } } - - diff --git a/serving/numaflow-models/src/models/pipeline_limits.rs b/serving/numaflow-models/src/models/pipeline_limits.rs index 3e3a2e58f1..c4a158e6ae 100644 --- a/serving/numaflow-models/src/models/pipeline_limits.rs +++ b/serving/numaflow-models/src/models/pipeline_limits.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct PipelineLimits { /// BufferMaxLength is used to define the max length of a buffer. Only applies to UDF and Source vertices as only they do buffer write. It can be overridden by the settings in vertex limits. @@ -36,5 +33,3 @@ impl PipelineLimits { } } } - - diff --git a/serving/numaflow-models/src/models/pipeline_list.rs b/serving/numaflow-models/src/models/pipeline_list.rs index a159018e5e..7fc027c1a7 100644 --- a/serving/numaflow-models/src/models/pipeline_list.rs +++ b/serving/numaflow-models/src/models/pipeline_list.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct PipelineList { /// APIVersion defines the versioned schema of this representation of an object. Servers should convert recognized schemas to the latest internal value, and may reject unrecognized values. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#resources @@ -35,5 +32,3 @@ impl PipelineList { } } } - - diff --git a/serving/numaflow-models/src/models/pipeline_spec.rs b/serving/numaflow-models/src/models/pipeline_spec.rs index 66c29acf08..c33cdb1f80 100644 --- a/serving/numaflow-models/src/models/pipeline_spec.rs +++ b/serving/numaflow-models/src/models/pipeline_spec.rs @@ -4,19 +4,19 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct PipelineSpec { /// Edges define the relationships between vertices #[serde(rename = "edges", skip_serializing_if = "Option::is_none")] pub edges: Option>, - #[serde(rename = "interStepBufferServiceName", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "interStepBufferServiceName", + skip_serializing_if = "Option::is_none" + )] pub inter_step_buffer_service_name: Option, #[serde(rename = "lifecycle", skip_serializing_if = "Option::is_none")] pub lifecycle: Option>, @@ -47,5 +47,3 @@ impl PipelineSpec { } } } - - diff --git a/serving/numaflow-models/src/models/pipeline_status.rs b/serving/numaflow-models/src/models/pipeline_status.rs index 64e8934fab..8756844a7c 100644 --- a/serving/numaflow-models/src/models/pipeline_status.rs +++ b/serving/numaflow-models/src/models/pipeline_status.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct PipelineStatus { /// Conditions are the latest available observations of a resource's current state. @@ -55,5 +52,3 @@ impl PipelineStatus { } } } - - diff --git a/serving/numaflow-models/src/models/redis_buffer_service.rs b/serving/numaflow-models/src/models/redis_buffer_service.rs index 10994749ff..5dc527b391 100644 --- a/serving/numaflow-models/src/models/redis_buffer_service.rs +++ b/serving/numaflow-models/src/models/redis_buffer_service.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct RedisBufferService { #[serde(rename = "external", skip_serializing_if = "Option::is_none")] @@ -27,5 +24,3 @@ impl RedisBufferService { } } } - - diff --git a/serving/numaflow-models/src/models/redis_config.rs b/serving/numaflow-models/src/models/redis_config.rs index 9c6c5180e6..cf563ab9ec 100644 --- a/serving/numaflow-models/src/models/redis_config.rs +++ b/serving/numaflow-models/src/models/redis_config.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct RedisConfig { /// Only required when Sentinel is used @@ -43,5 +40,3 @@ impl RedisConfig { } } } - - diff --git a/serving/numaflow-models/src/models/redis_settings.rs b/serving/numaflow-models/src/models/redis_settings.rs index 3b8e813a28..f5b4b4c2b7 100644 --- a/serving/numaflow-models/src/models/redis_settings.rs +++ b/serving/numaflow-models/src/models/redis_settings.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct RedisSettings { /// Special settings for Redis master node, will override the global settings from controller config @@ -37,5 +34,3 @@ impl RedisSettings { } } } - - diff --git a/serving/numaflow-models/src/models/sasl.rs b/serving/numaflow-models/src/models/sasl.rs index 163eb8a1b0..2f56e8cce0 100644 --- a/serving/numaflow-models/src/models/sasl.rs +++ b/serving/numaflow-models/src/models/sasl.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Sasl { #[serde(rename = "gssapi", skip_serializing_if = "Option::is_none")] @@ -37,5 +34,3 @@ impl Sasl { } } } - - diff --git a/serving/numaflow-models/src/models/sasl_plain.rs b/serving/numaflow-models/src/models/sasl_plain.rs index 085581f975..7719720dd3 100644 --- a/serving/numaflow-models/src/models/sasl_plain.rs +++ b/serving/numaflow-models/src/models/sasl_plain.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct SaslPlain { #[serde(rename = "handshake")] @@ -22,7 +19,10 @@ pub struct SaslPlain { } impl SaslPlain { - pub fn new(handshake: bool, user_secret: k8s_openapi::api::core::v1::SecretKeySelector) -> SaslPlain { + pub fn new( + handshake: bool, + user_secret: k8s_openapi::api::core::v1::SecretKeySelector, + ) -> SaslPlain { SaslPlain { handshake, password_secret: None, @@ -30,5 +30,3 @@ impl SaslPlain { } } } - - diff --git a/serving/numaflow-models/src/models/scale.rs b/serving/numaflow-models/src/models/scale.rs index 7510676ce1..03673cca73 100644 --- a/serving/numaflow-models/src/models/scale.rs +++ b/serving/numaflow-models/src/models/scale.rs @@ -4,14 +4,12 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// Scale : Scale defines the parameters for autoscaling. - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Scale { /// Deprecated: Use scaleUpCooldownSeconds and scaleDownCooldownSeconds instead. Cooldown seconds after a scaling operation before another one. @@ -33,19 +31,34 @@ pub struct Scale { #[serde(rename = "replicasPerScale", skip_serializing_if = "Option::is_none")] pub replicas_per_scale: Option, /// ScaleDownCooldownSeconds defines the cooldown seconds after a scaling operation, before a follow-up scaling down. It defaults to the CooldownSeconds if not set. - #[serde(rename = "scaleDownCooldownSeconds", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "scaleDownCooldownSeconds", + skip_serializing_if = "Option::is_none" + )] pub scale_down_cooldown_seconds: Option, /// ScaleUpCooldownSeconds defines the cooldown seconds after a scaling operation, before a follow-up scaling up. It defaults to the CooldownSeconds if not set. - #[serde(rename = "scaleUpCooldownSeconds", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "scaleUpCooldownSeconds", + skip_serializing_if = "Option::is_none" + )] pub scale_up_cooldown_seconds: Option, /// TargetBufferAvailability is used to define the target percentage of the buffer availability. A valid and meaningful value should be less than the BufferUsageLimit defined in the Edge spec (or Pipeline spec), for example, 50. It only applies to UDF and Sink vertices because only they have buffers to read. - #[serde(rename = "targetBufferAvailability", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "targetBufferAvailability", + skip_serializing_if = "Option::is_none" + )] pub target_buffer_availability: Option, /// TargetProcessingSeconds is used to tune the aggressiveness of autoscaling for source vertices, it measures how fast you want the vertex to process all the pending messages. Typically increasing the value, which leads to lower processing rate, thus less replicas. It's only effective for source vertices. - #[serde(rename = "targetProcessingSeconds", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "targetProcessingSeconds", + skip_serializing_if = "Option::is_none" + )] pub target_processing_seconds: Option, /// After scaling down the source vertex to 0, sleep how many seconds before scaling the source vertex back up to peek. - #[serde(rename = "zeroReplicaSleepSeconds", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "zeroReplicaSleepSeconds", + skip_serializing_if = "Option::is_none" + )] pub zero_replica_sleep_seconds: Option, } @@ -67,5 +80,3 @@ impl Scale { } } } - - diff --git a/serving/numaflow-models/src/models/serving_source.rs b/serving/numaflow-models/src/models/serving_source.rs index e1e6c444cf..25703a7031 100644 --- a/serving/numaflow-models/src/models/serving_source.rs +++ b/serving/numaflow-models/src/models/serving_source.rs @@ -4,14 +4,12 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// ServingSource : ServingSource is the HTTP endpoint for Numaflow. - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct ServingSource { #[serde(rename = "auth", skip_serializing_if = "Option::is_none")] @@ -37,5 +35,3 @@ impl ServingSource { } } } - - diff --git a/serving/numaflow-models/src/models/serving_store.rs b/serving/numaflow-models/src/models/serving_store.rs index 44998b5e73..7a3086aea9 100644 --- a/serving/numaflow-models/src/models/serving_store.rs +++ b/serving/numaflow-models/src/models/serving_store.rs @@ -4,14 +4,12 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// ServingStore : ServingStore to track and store data and metadata for tracking and serving. - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct ServingStore { #[serde(rename = "ttl", skip_serializing_if = "Option::is_none")] @@ -24,11 +22,6 @@ pub struct ServingStore { impl ServingStore { /// ServingStore to track and store data and metadata for tracking and serving. pub fn new(url: String) -> ServingStore { - ServingStore { - ttl: None, - url, - } + ServingStore { ttl: None, url } } } - - diff --git a/serving/numaflow-models/src/models/session_window.rs b/serving/numaflow-models/src/models/session_window.rs index 67f1f440bc..551f164cde 100644 --- a/serving/numaflow-models/src/models/session_window.rs +++ b/serving/numaflow-models/src/models/session_window.rs @@ -4,14 +4,12 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// SessionWindow : SessionWindow describes a session window - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct SessionWindow { #[serde(rename = "timeout", skip_serializing_if = "Option::is_none")] @@ -21,10 +19,6 @@ pub struct SessionWindow { impl SessionWindow { /// SessionWindow describes a session window pub fn new() -> SessionWindow { - SessionWindow { - timeout: None, - } + SessionWindow { timeout: None } } } - - diff --git a/serving/numaflow-models/src/models/side_input.rs b/serving/numaflow-models/src/models/side_input.rs index 9cb2040749..275af684af 100644 --- a/serving/numaflow-models/src/models/side_input.rs +++ b/serving/numaflow-models/src/models/side_input.rs @@ -4,14 +4,12 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// SideInput : SideInput defines information of a Side Input - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct SideInput { #[serde(rename = "container")] @@ -26,7 +24,11 @@ pub struct SideInput { impl SideInput { /// SideInput defines information of a Side Input - pub fn new(container: crate::models::Container, name: String, trigger: crate::models::SideInputTrigger) -> SideInput { + pub fn new( + container: crate::models::Container, + name: String, + trigger: crate::models::SideInputTrigger, + ) -> SideInput { SideInput { container: Box::new(container), name, @@ -35,5 +37,3 @@ impl SideInput { } } } - - diff --git a/serving/numaflow-models/src/models/side_input_trigger.rs b/serving/numaflow-models/src/models/side_input_trigger.rs index f92cff7cd9..497f5461b3 100644 --- a/serving/numaflow-models/src/models/side_input_trigger.rs +++ b/serving/numaflow-models/src/models/side_input_trigger.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct SideInputTrigger { /// The schedule to trigger the retrievement of the side input data. It supports cron format, for example, \"0 30 * * * *\". Or interval based format, such as \"@hourly\", \"@every 1h30m\", etc. @@ -28,5 +25,3 @@ impl SideInputTrigger { } } } - - diff --git a/serving/numaflow-models/src/models/side_inputs_manager_template.rs b/serving/numaflow-models/src/models/side_inputs_manager_template.rs index d9a8e6a4fc..82f7afe5d1 100644 --- a/serving/numaflow-models/src/models/side_inputs_manager_template.rs +++ b/serving/numaflow-models/src/models/side_inputs_manager_template.rs @@ -4,19 +4,19 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct SideInputsManagerTemplate { #[serde(rename = "affinity", skip_serializing_if = "Option::is_none")] pub affinity: Option, /// AutomountServiceAccountToken indicates whether a service account token should be automatically mounted. - #[serde(rename = "automountServiceAccountToken", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "automountServiceAccountToken", + skip_serializing_if = "Option::is_none" + )] pub automount_service_account_token: Option, #[serde(rename = "containerTemplate", skip_serializing_if = "Option::is_none")] pub container_template: Option>, @@ -28,7 +28,10 @@ pub struct SideInputsManagerTemplate { /// ImagePullSecrets is an optional list of references to secrets in the same namespace to use for pulling any of the images used by this PodSpec. If specified, these secrets will be passed to individual puller implementations for them to use. For example, in the case of docker, only DockerConfig type secrets are honored. More info: https://kubernetes.io/docs/concepts/containers/images#specifying-imagepullsecrets-on-a-pod #[serde(rename = "imagePullSecrets", skip_serializing_if = "Option::is_none")] pub image_pull_secrets: Option>, - #[serde(rename = "initContainerTemplate", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "initContainerTemplate", + skip_serializing_if = "Option::is_none" + )] pub init_container_template: Option>, #[serde(rename = "metadata", skip_serializing_if = "Option::is_none")] pub metadata: Option>, @@ -75,5 +78,3 @@ impl SideInputsManagerTemplate { } } } - - diff --git a/serving/numaflow-models/src/models/sink.rs b/serving/numaflow-models/src/models/sink.rs index ebed76c690..6cd57bb2ec 100644 --- a/serving/numaflow-models/src/models/sink.rs +++ b/serving/numaflow-models/src/models/sink.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Sink { #[serde(rename = "blackhole", skip_serializing_if = "Option::is_none")] @@ -36,5 +33,3 @@ impl Sink { } } } - - diff --git a/serving/numaflow-models/src/models/sliding_window.rs b/serving/numaflow-models/src/models/sliding_window.rs index 10530f04b3..4d2f9a06c7 100644 --- a/serving/numaflow-models/src/models/sliding_window.rs +++ b/serving/numaflow-models/src/models/sliding_window.rs @@ -4,14 +4,12 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// SlidingWindow : SlidingWindow describes a sliding window - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct SlidingWindow { #[serde(rename = "length", skip_serializing_if = "Option::is_none")] @@ -33,5 +31,3 @@ impl SlidingWindow { } } } - - diff --git a/serving/numaflow-models/src/models/source.rs b/serving/numaflow-models/src/models/source.rs index fadfe1a402..b7331b4942 100644 --- a/serving/numaflow-models/src/models/source.rs +++ b/serving/numaflow-models/src/models/source.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Source { #[serde(rename = "generator", skip_serializing_if = "Option::is_none")] @@ -45,5 +42,3 @@ impl Source { } } } - - diff --git a/serving/numaflow-models/src/models/status.rs b/serving/numaflow-models/src/models/status.rs index accf9e49d2..cbfa91bf3d 100644 --- a/serving/numaflow-models/src/models/status.rs +++ b/serving/numaflow-models/src/models/status.rs @@ -4,14 +4,12 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// Status : Status is a common structure which can be used for Status field. - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Status { /// Conditions are the latest available observations of a resource's current state. @@ -22,10 +20,6 @@ pub struct Status { impl Status { /// Status is a common structure which can be used for Status field. pub fn new() -> Status { - Status { - conditions: None, - } + Status { conditions: None } } } - - diff --git a/serving/numaflow-models/src/models/tag_conditions.rs b/serving/numaflow-models/src/models/tag_conditions.rs index 42b6b94e0a..60d4b3ca58 100644 --- a/serving/numaflow-models/src/models/tag_conditions.rs +++ b/serving/numaflow-models/src/models/tag_conditions.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct TagConditions { /// Operator specifies the type of operation that should be used for conditional forwarding value could be \"and\", \"or\", \"not\" @@ -29,5 +26,3 @@ impl TagConditions { } } } - - diff --git a/serving/numaflow-models/src/models/templates.rs b/serving/numaflow-models/src/models/templates.rs index 382019d5b5..98876b49a8 100644 --- a/serving/numaflow-models/src/models/templates.rs +++ b/serving/numaflow-models/src/models/templates.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Templates { #[serde(rename = "daemon", skip_serializing_if = "Option::is_none")] @@ -33,5 +30,3 @@ impl Templates { } } } - - diff --git a/serving/numaflow-models/src/models/tls.rs b/serving/numaflow-models/src/models/tls.rs index b140b68531..b0c896e8ea 100644 --- a/serving/numaflow-models/src/models/tls.rs +++ b/serving/numaflow-models/src/models/tls.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Tls { #[serde(rename = "caCertSecret", skip_serializing_if = "Option::is_none")] @@ -33,5 +30,3 @@ impl Tls { } } } - - diff --git a/serving/numaflow-models/src/models/transformer.rs b/serving/numaflow-models/src/models/transformer.rs index 5540a6b6f9..a17255f010 100644 --- a/serving/numaflow-models/src/models/transformer.rs +++ b/serving/numaflow-models/src/models/transformer.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Transformer { #[serde(rename = "args", skip_serializing_if = "Option::is_none")] @@ -30,5 +27,3 @@ impl Transformer { } } } - - diff --git a/serving/numaflow-models/src/models/ud_sink.rs b/serving/numaflow-models/src/models/ud_sink.rs index f39a053de5..f247400f86 100644 --- a/serving/numaflow-models/src/models/ud_sink.rs +++ b/serving/numaflow-models/src/models/ud_sink.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct UdSink { #[serde(rename = "container")] @@ -24,5 +21,3 @@ impl UdSink { } } } - - diff --git a/serving/numaflow-models/src/models/ud_source.rs b/serving/numaflow-models/src/models/ud_source.rs index 2242908feb..65986169a6 100644 --- a/serving/numaflow-models/src/models/ud_source.rs +++ b/serving/numaflow-models/src/models/ud_source.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct UdSource { #[serde(rename = "container")] @@ -24,5 +21,3 @@ impl UdSource { } } } - - diff --git a/serving/numaflow-models/src/models/ud_transformer.rs b/serving/numaflow-models/src/models/ud_transformer.rs index d397305f2b..ff698fab80 100644 --- a/serving/numaflow-models/src/models/ud_transformer.rs +++ b/serving/numaflow-models/src/models/ud_transformer.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct UdTransformer { #[serde(rename = "builtin", skip_serializing_if = "Option::is_none")] @@ -27,5 +24,3 @@ impl UdTransformer { } } } - - diff --git a/serving/numaflow-models/src/models/udf.rs b/serving/numaflow-models/src/models/udf.rs index cfeae12666..cd1f6b520f 100644 --- a/serving/numaflow-models/src/models/udf.rs +++ b/serving/numaflow-models/src/models/udf.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Udf { #[serde(rename = "builtin", skip_serializing_if = "Option::is_none")] @@ -30,5 +27,3 @@ impl Udf { } } } - - diff --git a/serving/numaflow-models/src/models/vertex.rs b/serving/numaflow-models/src/models/vertex.rs index d23f3023c5..1f9cc0e617 100644 --- a/serving/numaflow-models/src/models/vertex.rs +++ b/serving/numaflow-models/src/models/vertex.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Vertex { /// APIVersion defines the versioned schema of this representation of an object. Servers should convert recognized schemas to the latest internal value, and may reject unrecognized values. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#resources @@ -38,5 +35,3 @@ impl Vertex { } } } - - diff --git a/serving/numaflow-models/src/models/vertex_instance.rs b/serving/numaflow-models/src/models/vertex_instance.rs index 6c3298c345..e7ba664e60 100644 --- a/serving/numaflow-models/src/models/vertex_instance.rs +++ b/serving/numaflow-models/src/models/vertex_instance.rs @@ -4,14 +4,12 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// VertexInstance : VertexInstance is a wrapper of a vertex instance, which contains the vertex spec and the instance information such as hostname and replica index. - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct VertexInstance { #[serde(rename = "hostname", skip_serializing_if = "Option::is_none")] @@ -32,5 +30,3 @@ impl VertexInstance { } } } - - diff --git a/serving/numaflow-models/src/models/vertex_limits.rs b/serving/numaflow-models/src/models/vertex_limits.rs index 713c0faa41..6f201e8de8 100644 --- a/serving/numaflow-models/src/models/vertex_limits.rs +++ b/serving/numaflow-models/src/models/vertex_limits.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct VertexLimits { /// BufferMaxLength is used to define the max length of a buffer. It overrides the settings from pipeline limits. @@ -36,5 +33,3 @@ impl VertexLimits { } } } - - diff --git a/serving/numaflow-models/src/models/vertex_list.rs b/serving/numaflow-models/src/models/vertex_list.rs index a350a3b983..68967d8aa5 100644 --- a/serving/numaflow-models/src/models/vertex_list.rs +++ b/serving/numaflow-models/src/models/vertex_list.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct VertexList { /// APIVersion defines the versioned schema of this representation of an object. Servers should convert recognized schemas to the latest internal value, and may reject unrecognized values. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#resources @@ -35,5 +32,3 @@ impl VertexList { } } } - - diff --git a/serving/numaflow-models/src/models/vertex_spec.rs b/serving/numaflow-models/src/models/vertex_spec.rs index f96eb218ed..a3fafd4882 100644 --- a/serving/numaflow-models/src/models/vertex_spec.rs +++ b/serving/numaflow-models/src/models/vertex_spec.rs @@ -4,19 +4,19 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct VertexSpec { #[serde(rename = "affinity", skip_serializing_if = "Option::is_none")] pub affinity: Option, /// AutomountServiceAccountToken indicates whether a service account token should be automatically mounted. - #[serde(rename = "automountServiceAccountToken", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "automountServiceAccountToken", + skip_serializing_if = "Option::is_none" + )] pub automount_service_account_token: Option, #[serde(rename = "containerTemplate", skip_serializing_if = "Option::is_none")] pub container_template: Option>, @@ -30,12 +30,18 @@ pub struct VertexSpec { /// ImagePullSecrets is an optional list of references to secrets in the same namespace to use for pulling any of the images used by this PodSpec. If specified, these secrets will be passed to individual puller implementations for them to use. For example, in the case of docker, only DockerConfig type secrets are honored. More info: https://kubernetes.io/docs/concepts/containers/images#specifying-imagepullsecrets-on-a-pod #[serde(rename = "imagePullSecrets", skip_serializing_if = "Option::is_none")] pub image_pull_secrets: Option>, - #[serde(rename = "initContainerTemplate", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "initContainerTemplate", + skip_serializing_if = "Option::is_none" + )] pub init_container_template: Option>, /// List of customized init containers belonging to the pod. More info: https://kubernetes.io/docs/concepts/workloads/pods/init-containers/ #[serde(rename = "initContainers", skip_serializing_if = "Option::is_none")] pub init_containers: Option>, - #[serde(rename = "interStepBufferServiceName", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "interStepBufferServiceName", + skip_serializing_if = "Option::is_none" + )] pub inter_step_buffer_service_name: Option, #[serde(rename = "limits", skip_serializing_if = "Option::is_none")] pub limits: Option>, @@ -72,7 +78,10 @@ pub struct VertexSpec { /// Names of the side inputs used in this vertex. #[serde(rename = "sideInputs", skip_serializing_if = "Option::is_none")] pub side_inputs: Option>, - #[serde(rename = "sideInputsContainerTemplate", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "sideInputsContainerTemplate", + skip_serializing_if = "Option::is_none" + )] pub side_inputs_container_template: Option>, /// List of customized sidecar containers belonging to the pod. #[serde(rename = "sidecars", skip_serializing_if = "Option::is_none")] @@ -133,5 +142,3 @@ impl VertexSpec { } } } - - diff --git a/serving/numaflow-models/src/models/vertex_status.rs b/serving/numaflow-models/src/models/vertex_status.rs index 3e0377a795..6bcea1f106 100644 --- a/serving/numaflow-models/src/models/vertex_status.rs +++ b/serving/numaflow-models/src/models/vertex_status.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct VertexStatus { /// Conditions are the latest available observations of a resource's current state. @@ -46,5 +43,3 @@ impl VertexStatus { } } } - - diff --git a/serving/numaflow-models/src/models/vertex_template.rs b/serving/numaflow-models/src/models/vertex_template.rs index 6bd04e9cad..34aae02b0e 100644 --- a/serving/numaflow-models/src/models/vertex_template.rs +++ b/serving/numaflow-models/src/models/vertex_template.rs @@ -4,19 +4,19 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct VertexTemplate { #[serde(rename = "affinity", skip_serializing_if = "Option::is_none")] pub affinity: Option, /// AutomountServiceAccountToken indicates whether a service account token should be automatically mounted. - #[serde(rename = "automountServiceAccountToken", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "automountServiceAccountToken", + skip_serializing_if = "Option::is_none" + )] pub automount_service_account_token: Option, #[serde(rename = "containerTemplate", skip_serializing_if = "Option::is_none")] pub container_template: Option>, @@ -28,7 +28,10 @@ pub struct VertexTemplate { /// ImagePullSecrets is an optional list of references to secrets in the same namespace to use for pulling any of the images used by this PodSpec. If specified, these secrets will be passed to individual puller implementations for them to use. For example, in the case of docker, only DockerConfig type secrets are honored. More info: https://kubernetes.io/docs/concepts/containers/images#specifying-imagepullsecrets-on-a-pod #[serde(rename = "imagePullSecrets", skip_serializing_if = "Option::is_none")] pub image_pull_secrets: Option>, - #[serde(rename = "initContainerTemplate", skip_serializing_if = "Option::is_none")] + #[serde( + rename = "initContainerTemplate", + skip_serializing_if = "Option::is_none" + )] pub init_container_template: Option>, #[serde(rename = "metadata", skip_serializing_if = "Option::is_none")] pub metadata: Option>, @@ -75,5 +78,3 @@ impl VertexTemplate { } } } - - diff --git a/serving/numaflow-models/src/models/watermark.rs b/serving/numaflow-models/src/models/watermark.rs index e32de79335..b54bf2a74b 100644 --- a/serving/numaflow-models/src/models/watermark.rs +++ b/serving/numaflow-models/src/models/watermark.rs @@ -4,13 +4,10 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ - - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Watermark { /// Disabled toggles the watermark propagation, defaults to false. @@ -31,5 +28,3 @@ impl Watermark { } } } - - diff --git a/serving/numaflow-models/src/models/window.rs b/serving/numaflow-models/src/models/window.rs index 3f72f7be34..28d4d42243 100644 --- a/serving/numaflow-models/src/models/window.rs +++ b/serving/numaflow-models/src/models/window.rs @@ -4,14 +4,12 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// Window : Window describes windowing strategy - - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Window { #[serde(rename = "fixed", skip_serializing_if = "Option::is_none")] @@ -32,5 +30,3 @@ impl Window { } } } - - diff --git a/serving/source-sink/src/forwarder.rs b/serving/source-sink/src/forwarder.rs index 31b2d41e14..3d814549f4 100644 --- a/serving/source-sink/src/forwarder.rs +++ b/serving/source-sink/src/forwarder.rs @@ -92,9 +92,9 @@ mod tests { use tokio::sync::mpsc::Sender; use crate::forwarder::Forwarder; - use crate::sink::SinkClient; - use crate::source::SourceClient; - use crate::transformer::TransformerClient; + use crate::sink::{SinkClient, SinkConfig}; + use crate::source::{SourceClient, SourceConfig}; + use crate::transformer::{TransformerClient, TransformerConfig}; struct SimpleSource { yet_to_be_acked: std::sync::RwLock>, @@ -227,64 +227,82 @@ mod tests { let (source_shutdown_tx, source_shutdown_rx) = tokio::sync::oneshot::channel(); let tmp_dir = tempfile::TempDir::new().unwrap(); let source_sock_file = tmp_dir.path().join("source.sock"); + let server_info_file = tmp_dir.path().join("source-server-info"); + let server_info = server_info_file.clone(); let source_socket = source_sock_file.clone(); let source_server_handle = tokio::spawn(async move { - let server_info_file = tmp_dir.path().join("source-server-info"); source::Server::new(SimpleSource::new()) .with_socket_file(source_socket) - .with_server_info_file(server_info_file) + .with_server_info_file(server_info) .start_with_shutdown(source_shutdown_rx) .await .unwrap(); }); + let source_config = SourceConfig { + socket_path: source_sock_file.to_str().unwrap().to_string(), + server_info_file: server_info_file.to_str().unwrap().to_string(), + max_message_size: 4 * 1024 * 1024, + }; // Start the sink server let (sink_shutdown_tx, sink_shutdown_rx) = tokio::sync::oneshot::channel(); let sink_tmp_dir = tempfile::TempDir::new().unwrap(); let sink_sock_file = sink_tmp_dir.path().join("sink.sock"); + let server_info_file = sink_tmp_dir.path().join("sink-server-info"); + let server_info = server_info_file.clone(); let sink_socket = sink_sock_file.clone(); let sink_server_handle = tokio::spawn(async move { - let server_info_file = sink_tmp_dir.path().join("sink-server-info"); sink::Server::new(InMemorySink::new(sink_tx)) .with_socket_file(sink_socket) - .with_server_info_file(server_info_file) + .with_server_info_file(server_info) .start_with_shutdown(sink_shutdown_rx) .await .unwrap(); }); + let sink_config = SinkConfig { + socket_path: sink_sock_file.to_str().unwrap().to_string(), + server_info_file: server_info_file.to_str().unwrap().to_string(), + max_message_size: 4 * 1024 * 1024, + }; // Start the transformer server let (transformer_shutdown_tx, transformer_shutdown_rx) = tokio::sync::oneshot::channel(); let tmp_dir = tempfile::TempDir::new().unwrap(); let transformer_sock_file = tmp_dir.path().join("transformer.sock"); + let server_info_file = tmp_dir.path().join("transformer-server-info"); + let server_info = server_info_file.clone(); let transformer_socket = transformer_sock_file.clone(); let transformer_server_handle = tokio::spawn(async move { - let server_info_file = tmp_dir.path().join("transformer-server-info"); sourcetransform::Server::new(SimpleTransformer) .with_socket_file(transformer_socket) - .with_server_info_file(server_info_file) + .with_server_info_file(server_info) .start_with_shutdown(transformer_shutdown_rx) .await .unwrap(); }); + let transformer_config = TransformerConfig { + socket_path: transformer_sock_file.to_str().unwrap().to_string(), + server_info_file: server_info_file.to_str().unwrap().to_string(), + max_message_size: 4 * 1024 * 1024, + }; // Wait for the servers to start tokio::time::sleep(tokio::time::Duration::from_millis(100)).await; let (forwarder_shutdown_tx, forwarder_shutdown_rx) = tokio::sync::oneshot::channel(); - let source_client = SourceClient::connect(source_sock_file) + let source_client = SourceClient::connect(source_config) .await .expect("failed to connect to source server"); - let sink_client = SinkClient::connect(sink_sock_file) + let sink_client = SinkClient::connect(sink_config) .await .expect("failed to connect to sink server"); - let transformer_client = TransformerClient::connect(transformer_sock_file) + let transformer_client = TransformerClient::connect(transformer_config) .await .expect("failed to connect to transformer server"); diff --git a/serving/source-sink/src/lib.rs b/serving/source-sink/src/lib.rs index 7d788f279c..6ecdc0052d 100644 --- a/serving/source-sink/src/lib.rs +++ b/serving/source-sink/src/lib.rs @@ -1,22 +1,23 @@ -use std::env; - +use std::fs; +use std::time::Duration; use tokio::signal; use tokio::sync::oneshot; use tokio::task::JoinHandle; +use tokio::time::sleep; use tracing::info; use crate::error::Error; use crate::forwarder::Forwarder; -use crate::sink::SinkClient; -use crate::source::SourceClient; -use crate::transformer::TransformerClient; +use crate::sink::{SinkClient, SinkConfig}; +use crate::source::{SourceClient, SourceConfig}; +use crate::transformer::{TransformerClient, TransformerConfig}; -///! SourcerSinker orchestrates data movement from the Source to the Sink via the optional SourceTransformer. -///! The infamous forward-a-chunk executes the following in an infinite loop: -///! - Read X messages from the source -///! - Invokes the SourceTransformer concurrently -///! - Calls the Sinker to write the batch to the Sink -///! - Send Acknowledgement back to the Source +/// SourcerSinker orchestrates data movement from the Source to the Sink via the optional SourceTransformer. +/// The infamous forward-a-chunk executes the following in an infinite loop: +/// - Read X messages from the source +/// - Invokes the SourceTransformer concurrently +/// - Calls the Sinker to write the batch to the Sink +/// - Send Acknowledgement back to the Source /// TODO /// - [ ] integrate with main @@ -32,36 +33,43 @@ pub mod error; pub mod metrics; -pub(crate) mod source; +pub mod source; -pub(crate) mod sink; +pub mod sink; -pub(crate) mod transformer; +pub mod transformer; -pub(crate) mod forwarder; +pub mod forwarder; -pub(crate) mod message; +pub mod message; pub(crate) mod shared; -const SOURCE_SOCKET: &str = "/var/run/numaflow/source.sock"; -const SINK_SOCKET: &str = "/var/run/numaflow/sink.sock"; -const TRANSFORMER_SOCKET: &str = "/var/run/numaflow/sourcetransform.sock"; const TIMEOUT_IN_MS: u32 = 1000; const BATCH_SIZE: u64 = 500; -pub async fn run_forwarder(custom_shutdown_rx: Option>) -> Result<()> { - let mut source_client = SourceClient::connect(SOURCE_SOCKET.into()).await?; - let mut sink_client = SinkClient::connect(SINK_SOCKET.into()).await?; - let mut transformer_client = if env::var("NUMAFLOW_TRANSFORMER").is_ok() { - Some(TransformerClient::connect(TRANSFORMER_SOCKET.into()).await?) +pub async fn run_forwarder( + custom_shutdown_rx: Option>, + source_config: SourceConfig, + sink_config: SinkConfig, + transformer_config: Option, +) -> Result<()> { + wait_for_server_info(&source_config.server_info_file).await?; + let mut source_client = SourceClient::connect(source_config).await?; + + wait_for_server_info(&sink_config.server_info_file).await?; + let mut sink_client = SinkClient::connect(sink_config).await?; + + let mut transformer_client = if let Some(config) = transformer_config { + wait_for_server_info(&config.server_info_file).await?; + Some(TransformerClient::connect(config).await?) } else { None }; let (shutdown_tx, shutdown_rx) = oneshot::channel(); - // readiness check for all the servers + // readiness check for all the ud containers wait_until_ready( &mut source_client, &mut sink_client, @@ -99,6 +107,18 @@ pub async fn run_forwarder(custom_shutdown_rx: Option>) -> Ok(()) } +async fn wait_for_server_info(file_path: &str) -> Result<()> { + loop { + if let Ok(metadata) = fs::metadata(file_path) { + if metadata.len() > 0 { + return Ok(()); + } + } + info!("Server info file {} is not ready, waiting...", file_path); + sleep(Duration::from_secs(1)).await; + } +} + async fn wait_until_ready( source_client: &mut SourceClient, sink_client: &mut SinkClient, @@ -129,7 +149,7 @@ async fn wait_until_ready( break; } - tokio::time::sleep(tokio::time::Duration::from_secs(1)).await; + sleep(Duration::from_secs(1)).await; } Ok(()) @@ -162,81 +182,112 @@ async fn shutdown_signal(shutdown_rx: Option>) { } } -// #[cfg(test)] -// mod tests { -// use numaflow::{sink, source}; -// use numaflow::source::{Message, Offset, SourceReadRequest}; -// use tokio::sync::mpsc::Sender; -// -// struct SimpleSource; -// #[tonic::async_trait] -// impl source::Sourcer for SimpleSource { -// async fn read(&self, _: SourceReadRequest, _: Sender) { -// } -// -// async fn ack(&self, _: Vec) { -// } -// -// async fn pending(&self) -> usize { -// 0 -// } -// -// async fn partitions(&self) -> Option> { -// None -// } -// } -// -// struct SimpleSink; -// -// #[tonic::async_trait] -// impl sink::Sinker for SimpleSink { -// async fn sink(&self, input: tokio::sync::mpsc::Receiver) -> Vec { -// vec![] -// } -// } -// #[tokio::test] -// async fn run_forwarder() { -// let (src_shutdown_tx, src_shutdown_rx) = tokio::sync::oneshot::channel(); -// let tmp_dir = tempfile::TempDir::new().unwrap(); -// let sock_file = tmp_dir.path().join("source.sock"); -// -// let server_socket = sock_file.clone(); -// let src_server_handle = tokio::spawn(async move { -// let server_info_file = tmp_dir.path().join("source-server-info"); -// source::Server::new(SimpleSource) -// .with_socket_file(server_socket) -// .with_server_info_file(server_info_file) -// .start_with_shutdown(src_shutdown_rx) -// .await -// .unwrap(); -// }); -// -// let (sink_shutdown_tx, sink_shutdown_rx) = tokio::sync::oneshot::channel(); -// let tmp_dir = tempfile::TempDir::new().unwrap(); -// let sock_file = tmp_dir.path().join("sink.sock"); -// -// let server_socket = sock_file.clone(); -// let sink_server_handle = tokio::spawn(async move { -// let server_info_file = tmp_dir.path().join("sink-server-info"); -// sink::Server::new(SimpleSink) -// .with_socket_file(server_socket) -// .with_server_info_file(server_info_file) -// .start_with_shutdown(sink_shutdown_rx) -// .await -// .unwrap(); -// }); -// -// let (shutdown_tx, shutdown_rx) = tokio::sync::oneshot::channel(); -// -// let result = super::run_forwarder(Some(shutdown_rx)).await; -// println!("{:?}", result); -// assert!(result.is_ok()); -// -// // stop the source and sink servers -// src_shutdown_tx.send(()).unwrap(); -// sink_shutdown_tx.send(()).unwrap(); -// -// src_server_handle.await.unwrap(); -// sink_server_handle.await.unwrap(); -// } -// } \ No newline at end of file +#[cfg(test)] +mod tests { + use crate::sink::SinkConfig; + use crate::source::SourceConfig; + use numaflow::source::{Message, Offset, SourceReadRequest}; + use numaflow::{sink, source}; + use std::env; + use tokio::sync::mpsc::Sender; + + struct SimpleSource; + #[tonic::async_trait] + impl source::Sourcer for SimpleSource { + async fn read(&self, _: SourceReadRequest, _: Sender) {} + + async fn ack(&self, _: Vec) {} + + async fn pending(&self) -> usize { + 0 + } + + async fn partitions(&self) -> Option> { + None + } + } + + struct SimpleSink; + + #[tonic::async_trait] + impl sink::Sinker for SimpleSink { + async fn sink( + &self, + _input: tokio::sync::mpsc::Receiver, + ) -> Vec { + vec![] + } + } + #[tokio::test] + async fn run_forwarder() { + let (src_shutdown_tx, src_shutdown_rx) = tokio::sync::oneshot::channel(); + let tmp_dir = tempfile::TempDir::new().unwrap(); + let src_sock_file = tmp_dir.path().join("source.sock"); + let src_info_file = tmp_dir.path().join("source-server-info"); + + let server_info = src_info_file.clone(); + let server_socket = src_sock_file.clone(); + let src_server_handle = tokio::spawn(async move { + source::Server::new(SimpleSource) + .with_socket_file(server_socket) + .with_server_info_file(server_info) + .start_with_shutdown(src_shutdown_rx) + .await + .unwrap(); + }); + let source_config = SourceConfig { + socket_path: src_sock_file.to_str().unwrap().to_string(), + server_info_file: src_info_file.to_str().unwrap().to_string(), + max_message_size: 100, + }; + + let (sink_shutdown_tx, sink_shutdown_rx) = tokio::sync::oneshot::channel(); + let tmp_dir = tempfile::TempDir::new().unwrap(); + let sink_sock_file = tmp_dir.path().join("sink.sock"); + let sink_server_info = tmp_dir.path().join("sink-server-info"); + + let server_socket = sink_sock_file.clone(); + let server_info = sink_server_info.clone(); + let sink_server_handle = tokio::spawn(async move { + sink::Server::new(SimpleSink) + .with_socket_file(server_socket) + .with_server_info_file(server_info) + .start_with_shutdown(sink_shutdown_rx) + .await + .unwrap(); + }); + let sink_config = SinkConfig { + socket_path: sink_sock_file.to_str().unwrap().to_string(), + server_info_file: sink_server_info.to_str().unwrap().to_string(), + max_message_size: 100, + }; + + // wait for the servers to start + tokio::time::sleep(tokio::time::Duration::from_millis(100)).await; + + env::set_var("SOURCE_SOCKET", src_sock_file.to_str().unwrap()); + env::set_var("SINK_SOCKET", sink_sock_file.to_str().unwrap()); + + let (shutdown_tx, shutdown_rx) = tokio::sync::oneshot::channel(); + + let forwarder_handle = tokio::spawn(async move { + let result = + super::run_forwarder(Some(shutdown_rx), source_config, sink_config, None).await; + assert!(result.is_ok()); + }); + + // wait for the forwarder to start + tokio::time::sleep(tokio::time::Duration::from_millis(100)).await; + + // stop the forwarder + shutdown_tx.send(()).unwrap(); + forwarder_handle.await.unwrap(); + + // stop the source and sink servers + src_shutdown_tx.send(()).unwrap(); + sink_shutdown_tx.send(()).unwrap(); + + src_server_handle.await.unwrap(); + sink_server_handle.await.unwrap(); + } +} diff --git a/serving/source-sink/src/main.rs b/serving/source-sink/src/main.rs index 557c01c472..9b4e21452f 100644 --- a/serving/source-sink/src/main.rs +++ b/serving/source-sink/src/main.rs @@ -1,7 +1,11 @@ +use std::env; use std::net::SocketAddr; use tracing::error; +use sourcer_sinker::sink::SinkConfig; +use sourcer_sinker::source::SourceConfig; +use sourcer_sinker::transformer::TransformerConfig; use sourcer_sinker::{metrics::start_metrics_server, run_forwarder}; #[tokio::main] @@ -16,8 +20,16 @@ async fn main() { } }); + let source_config = SourceConfig::default(); + let sink_config = SinkConfig::default(); + let transformer_config = if env::var("NUMAFLOW_TRANSFORMER").is_ok() { + Some(TransformerConfig::default()) + } else { + None + }; + // Run the forwarder - if let Err(e) = run_forwarder(None).await { + if let Err(e) = run_forwarder(None, source_config, sink_config, transformer_config).await { error!("Application error: {:?}", e); } } diff --git a/serving/source-sink/src/sink.rs b/serving/source-sink/src/sink.rs index 160fad2449..228a294c96 100644 --- a/serving/source-sink/src/sink.rs +++ b/serving/source-sink/src/sink.rs @@ -1,5 +1,3 @@ -use std::path::PathBuf; - use tonic::transport::Channel; use tonic::Request; @@ -11,15 +9,38 @@ pub mod proto { tonic::include_proto!("sink.v1"); } +const SINK_SOCKET: &str = "/var/run/numaflow/sink.sock"; +const SINK_SERVER_INFO_FILE: &str = "/var/run/numaflow/sink-server-info"; + +/// SinkConfig is the configuration for the sink server. +#[derive(Debug, Clone)] +pub struct SinkConfig { + pub socket_path: String, + pub server_info_file: String, + pub max_message_size: usize, +} + +impl Default for SinkConfig { + fn default() -> Self { + SinkConfig { + socket_path: SINK_SOCKET.to_string(), + server_info_file: SINK_SERVER_INFO_FILE.to_string(), + max_message_size: 64 * 1024 * 1024, // 64 MB + } + } +} + /// SinkClient is a client to interact with the sink server. pub struct SinkClient { client: proto::sink_client::SinkClient, } impl SinkClient { - pub(crate) async fn connect(uds_path: PathBuf) -> Result { - let channel = connect_with_uds(uds_path).await?; - let client = proto::sink_client::SinkClient::new(channel); + pub(crate) async fn connect(config: SinkConfig) -> Result { + let channel = connect_with_uds(config.socket_path.into()).await?; + let client = proto::sink_client::SinkClient::new(channel) + .max_decoding_message_size(config.max_message_size) + .max_encoding_message_size(config.max_message_size); Ok(Self { client }) } @@ -101,13 +122,14 @@ mod tests { let (shutdown_tx, shutdown_rx) = tokio::sync::oneshot::channel(); let tmp_dir = tempfile::TempDir::new().unwrap(); let sock_file = tmp_dir.path().join("sink.sock"); + let server_info_file = tmp_dir.path().join("sink-server-info"); + let server_info = server_info_file.clone(); let server_socket = sock_file.clone(); let server_handle = tokio::spawn(async move { - let server_info_file = tmp_dir.path().join("sink-server-info"); sink::Server::new(Logger) .with_socket_file(server_socket) - .with_server_info_file(server_info_file) + .with_server_info_file(server_info) .start_with_shutdown(shutdown_rx) .await .unwrap(); @@ -116,9 +138,13 @@ mod tests { // wait for the server to start tokio::time::sleep(tokio::time::Duration::from_millis(100)).await; - let mut sink_client = SinkClient::connect(sock_file) - .await - .expect("failed to connect to sink server"); + let mut sink_client = SinkClient::connect(SinkConfig { + socket_path: sock_file.to_str().unwrap().to_string(), + server_info_file: server_info_file.to_str().unwrap().to_string(), + max_message_size: 4 * 1024 * 1024, + }) + .await + .expect("failed to connect to sink server"); let messages = vec![ Message { diff --git a/serving/source-sink/src/source.rs b/serving/source-sink/src/source.rs index 18395c486b..389318d49f 100644 --- a/serving/source-sink/src/source.rs +++ b/serving/source-sink/src/source.rs @@ -1,5 +1,3 @@ -use std::path::PathBuf; - use base64::prelude::BASE64_STANDARD; use base64::Engine; use tokio_stream::StreamExt; @@ -14,6 +12,27 @@ pub mod proto { tonic::include_proto!("source.v1"); } +const SOURCE_SOCKET: &str = "/var/run/numaflow/source.sock"; +const SOURCE_SERVER_INFO_FILE: &str = "/var/run/numaflow/source-server-info"; + +/// SourceConfig is the configuration for the source server. +#[derive(Debug, Clone)] +pub struct SourceConfig { + pub socket_path: String, + pub server_info_file: String, + pub max_message_size: usize, +} + +impl Default for SourceConfig { + fn default() -> Self { + SourceConfig { + socket_path: SOURCE_SOCKET.to_string(), + server_info_file: SOURCE_SERVER_INFO_FILE.to_string(), + max_message_size: 64 * 1024 * 1024, // 64 MB + } + } +} + /// SourceClient is a client to interact with the source server. #[derive(Debug, Clone)] pub(crate) struct SourceClient { @@ -21,9 +40,11 @@ pub(crate) struct SourceClient { } impl SourceClient { - pub(crate) async fn connect(uds_path: PathBuf) -> Result { - let channel = connect_with_uds(uds_path).await?; - let client = proto::source_client::SourceClient::new(channel); + pub(crate) async fn connect(config: SourceConfig) -> Result { + let channel = connect_with_uds(config.socket_path.into()).await?; + let client = proto::source_client::SourceClient::new(channel) + .max_encoding_message_size(config.max_message_size) + .max_decoding_message_size(config.max_message_size); Ok(Self { client }) } @@ -116,7 +137,7 @@ mod tests { use numaflow::source::{Message, Offset, SourceReadRequest}; use tokio::sync::mpsc::Sender; - use crate::source::SourceClient; + use crate::source::{SourceClient, SourceConfig}; struct SimpleSource { num: usize, @@ -181,13 +202,14 @@ mod tests { let (shutdown_tx, shutdown_rx) = tokio::sync::oneshot::channel(); let tmp_dir = tempfile::TempDir::new().unwrap(); let sock_file = tmp_dir.path().join("source.sock"); + let server_info_file = tmp_dir.path().join("source-server-info"); + let server_info = server_info_file.clone(); let server_socket = sock_file.clone(); let server_handle = tokio::spawn(async move { - let server_info_file = tmp_dir.path().join("source-server-info"); source::Server::new(SimpleSource::new(10)) .with_socket_file(server_socket) - .with_server_info_file(server_info_file) + .with_server_info_file(server_info) .start_with_shutdown(shutdown_rx) .await .unwrap(); @@ -196,9 +218,13 @@ mod tests { // wait for the server to start tokio::time::sleep(tokio::time::Duration::from_millis(100)).await; - let mut source_client = SourceClient::connect(sock_file) - .await - .expect("failed to connect to source server"); + let mut source_client = SourceClient::connect(SourceConfig { + socket_path: sock_file.to_str().unwrap().to_string(), + server_info_file: server_info_file.to_str().unwrap().to_string(), + max_message_size: 4 * 1024 * 1024, + }) + .await + .expect("failed to connect to source server"); let response = source_client.is_ready().await.unwrap(); assert_eq!(response.ready, true); diff --git a/serving/source-sink/src/transformer.rs b/serving/source-sink/src/transformer.rs index 57a47681de..a2747383f4 100644 --- a/serving/source-sink/src/transformer.rs +++ b/serving/source-sink/src/transformer.rs @@ -1,5 +1,3 @@ -use std::path::PathBuf; - use tonic::transport::Channel; use tonic::Request; @@ -12,6 +10,27 @@ pub mod proto { tonic::include_proto!("sourcetransformer.v1"); } +const TRANSFORMER_SOCKET: &str = "/var/run/numaflow/sourcetransform.sock"; +const TRANSFORMER_SERVER_INFO_FILE: &str = "/var/run/numaflow/transformer-server-info"; + +/// TransformerConfig is the configuration for the transformer server. +#[derive(Debug, Clone)] +pub struct TransformerConfig { + pub socket_path: String, + pub server_info_file: String, + pub max_message_size: usize, +} + +impl Default for TransformerConfig { + fn default() -> Self { + TransformerConfig { + socket_path: TRANSFORMER_SOCKET.to_string(), + server_info_file: TRANSFORMER_SERVER_INFO_FILE.to_string(), + max_message_size: 64 * 1024 * 1024, // 64 MB + } + } +} + /// TransformerClient is a client to interact with the transformer server. #[derive(Clone)] pub struct TransformerClient { @@ -19,9 +38,11 @@ pub struct TransformerClient { } impl TransformerClient { - pub(crate) async fn connect(uds_path: PathBuf) -> Result { - let channel = connect_with_uds(uds_path).await?; - let client = proto::source_transform_client::SourceTransformClient::new(channel); + pub(crate) async fn connect(config: TransformerConfig) -> Result { + let channel = connect_with_uds(config.socket_path.into()).await?; + let client = proto::source_transform_client::SourceTransformClient::new(channel) + .max_decoding_message_size(config.max_message_size) + .max_encoding_message_size(config.max_message_size); Ok(Self { client }) } @@ -69,7 +90,7 @@ mod tests { use numaflow::sourcetransform; use tempfile::TempDir; - use crate::transformer::TransformerClient; + use crate::transformer::{TransformerClient, TransformerConfig}; struct NowCat; @@ -91,13 +112,14 @@ mod tests { let (shutdown_tx, shutdown_rx) = tokio::sync::oneshot::channel(); let tmp_dir = TempDir::new()?; let sock_file = tmp_dir.path().join("sourcetransform.sock"); + let server_info_file = tmp_dir.path().join("sourcetransformer-server-info"); + let server_info = server_info_file.clone(); let server_socket = sock_file.clone(); let handle = tokio::spawn(async move { - let server_info_file = tmp_dir.path().join("sourcetransformer-server-info"); sourcetransform::Server::new(NowCat) .with_socket_file(server_socket) - .with_server_info_file(server_info_file) + .with_server_info_file(server_info) .start_with_shutdown(shutdown_rx) .await .expect("server failed"); @@ -106,7 +128,12 @@ mod tests { // wait for the server to start tokio::time::sleep(tokio::time::Duration::from_millis(100)).await; - let mut client = TransformerClient::connect(sock_file).await?; + let mut client = TransformerClient::connect(TransformerConfig { + socket_path: sock_file.to_str().unwrap().to_string(), + server_info_file: server_info_file.to_str().unwrap().to_string(), + max_message_size: 4 * 1024 * 1024, + }) + .await?; let message = crate::message::Message { keys: vec!["first".into(), "second".into()], From a66bfed7fef1b270209d84bd1c709203d7222bdd Mon Sep 17 00:00:00 2001 From: Yashash H L Date: Tue, 6 Aug 2024 19:36:06 +0530 Subject: [PATCH 04/15] revert fmt Signed-off-by: Yashash H L --- .../numaflow-models/src/apis/configuration.rs | 6 +++- serving/numaflow-models/src/apis/mod.rs | 17 +++++----- serving/numaflow-models/src/lib.rs | 2 +- .../src/models/abstract_pod_template.rs | 11 ++++--- .../src/models/abstract_sink.rs | 7 +++- .../src/models/abstract_vertex.rs | 22 ++++++------- .../src/models/authorization.rs | 11 +++++-- .../numaflow-models/src/models/basic_auth.rs | 6 +++- .../numaflow-models/src/models/blackhole.rs | 12 +++++-- .../src/models/buffer_service_config.rs | 7 +++- .../src/models/combined_edge.rs | 23 +++++-------- .../numaflow-models/src/models/container.rs | 6 +++- .../src/models/container_builder.rs | 17 +++++----- .../src/models/container_template.rs | 6 +++- .../src/models/daemon_template.rs | 17 +++++----- serving/numaflow-models/src/models/edge.rs | 7 +++- .../src/models/fixed_window.rs | 6 +++- .../src/models/forward_conditions.rs | 7 +++- .../numaflow-models/src/models/function.rs | 7 +++- .../src/models/generator_source.rs | 7 +++- .../src/models/get_container_req.rs | 16 ++++------ .../src/models/get_daemon_deployment_req.rs | 15 ++++----- .../models/get_jet_stream_service_spec_req.rs | 15 ++++----- .../get_jet_stream_stateful_set_spec_req.rs | 25 ++++----------- .../src/models/get_redis_service_spec_req.rs | 13 ++++---- .../models/get_redis_stateful_set_spec_req.rs | 26 ++++----------- .../models/get_side_input_deployment_req.rs | 15 ++++----- .../src/models/get_vertex_pod_spec_req.rs | 18 ++++------- .../numaflow-models/src/models/group_by.rs | 6 +++- serving/numaflow-models/src/models/gssapi.rs | 17 ++++------ .../numaflow-models/src/models/http_source.rs | 7 +++- .../numaflow-models/src/models/idle_source.rs | 7 +++- .../src/models/inter_step_buffer_service.rs | 7 +++- .../models/inter_step_buffer_service_list.rs | 11 ++++--- .../models/inter_step_buffer_service_spec.rs | 7 +++- .../inter_step_buffer_service_status.rs | 7 +++- .../src/models/jet_stream_buffer_service.rs | 22 ++++++------- .../src/models/jet_stream_config.rs | 7 +++- .../src/models/jet_stream_source.rs | 7 +++- .../src/models/job_template.rs | 17 +++++----- .../numaflow-models/src/models/kafka_sink.rs | 7 +++- .../src/models/kafka_source.rs | 7 +++- .../numaflow-models/src/models/lifecycle.rs | 17 +++++----- serving/numaflow-models/src/models/log.rs | 13 ++++++-- .../numaflow-models/src/models/metadata.rs | 7 +++- .../src/models/native_redis.rs | 32 +++++++------------ .../numaflow-models/src/models/nats_auth.rs | 6 +++- .../numaflow-models/src/models/nats_source.rs | 7 +++- .../numaflow-models/src/models/no_store.rs | 12 +++++-- .../numaflow-models/src/models/pbq_storage.rs | 11 ++++--- .../src/models/persistence_strategy.rs | 6 +++- .../numaflow-models/src/models/pipeline.rs | 7 +++- .../src/models/pipeline_limits.rs | 7 +++- .../src/models/pipeline_list.rs | 7 +++- .../src/models/pipeline_spec.rs | 12 ++++--- .../src/models/pipeline_status.rs | 7 +++- .../src/models/redis_buffer_service.rs | 7 +++- .../src/models/redis_config.rs | 7 +++- .../src/models/redis_settings.rs | 7 +++- serving/numaflow-models/src/models/sasl.rs | 7 +++- .../numaflow-models/src/models/sasl_plain.rs | 12 ++++--- serving/numaflow-models/src/models/scale.rs | 31 ++++++------------ .../src/models/serving_source.rs | 6 +++- .../src/models/serving_store.rs | 11 +++++-- .../src/models/session_window.rs | 10 ++++-- .../numaflow-models/src/models/side_input.rs | 12 +++---- .../src/models/side_input_trigger.rs | 7 +++- .../models/side_inputs_manager_template.rs | 17 +++++----- serving/numaflow-models/src/models/sink.rs | 7 +++- .../src/models/sliding_window.rs | 6 +++- serving/numaflow-models/src/models/source.rs | 7 +++- serving/numaflow-models/src/models/status.rs | 10 ++++-- .../src/models/tag_conditions.rs | 7 +++- .../numaflow-models/src/models/templates.rs | 7 +++- serving/numaflow-models/src/models/tls.rs | 7 +++- .../numaflow-models/src/models/transformer.rs | 7 +++- serving/numaflow-models/src/models/ud_sink.rs | 7 +++- .../numaflow-models/src/models/ud_source.rs | 7 +++- .../src/models/ud_transformer.rs | 7 +++- serving/numaflow-models/src/models/udf.rs | 7 +++- serving/numaflow-models/src/models/vertex.rs | 7 +++- .../src/models/vertex_instance.rs | 6 +++- .../src/models/vertex_limits.rs | 7 +++- .../numaflow-models/src/models/vertex_list.rs | 7 +++- .../numaflow-models/src/models/vertex_spec.rs | 27 ++++++---------- .../src/models/vertex_status.rs | 7 +++- .../src/models/vertex_template.rs | 17 +++++----- .../numaflow-models/src/models/watermark.rs | 7 +++- serving/numaflow-models/src/models/window.rs | 6 +++- 89 files changed, 582 insertions(+), 363 deletions(-) diff --git a/serving/numaflow-models/src/apis/configuration.rs b/serving/numaflow-models/src/apis/configuration.rs index da73e26e0b..5c65cc5721 100644 --- a/serving/numaflow-models/src/apis/configuration.rs +++ b/serving/numaflow-models/src/apis/configuration.rs @@ -4,10 +4,12 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + #[derive(Debug, Clone)] pub struct Configuration { pub base_path: String, @@ -28,6 +30,7 @@ pub struct ApiKey { pub key: String, } + impl Configuration { pub fn new() -> Configuration { Configuration::default() @@ -44,6 +47,7 @@ impl Default for Configuration { oauth_access_token: None, bearer_access_token: None, api_key: None, + } } } diff --git a/serving/numaflow-models/src/apis/mod.rs b/serving/numaflow-models/src/apis/mod.rs index 5fb0cf38c7..dccbc940fb 100644 --- a/serving/numaflow-models/src/apis/mod.rs +++ b/serving/numaflow-models/src/apis/mod.rs @@ -16,7 +16,7 @@ pub enum Error { ResponseError(ResponseContent), } -impl fmt::Display for Error { +impl fmt::Display for Error { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { let (module, e) = match self { Error::Reqwest(e) => ("reqwest", e.to_string()), @@ -28,7 +28,7 @@ impl fmt::Display for Error { } } -impl error::Error for Error { +impl error::Error for Error { fn source(&self) -> Option<&(dyn error::Error + 'static)> { Some(match self { Error::Reqwest(e) => e, @@ -39,19 +39,19 @@ impl error::Error for Error { } } -impl From for Error { +impl From for Error { fn from(e: reqwest::Error) -> Self { Error::Reqwest(e) } } -impl From for Error { +impl From for Error { fn from(e: serde_json::Error) -> Self { Error::Serde(e) } } -impl From for Error { +impl From for Error { fn from(e: std::io::Error) -> Self { Error::Io(e) } @@ -78,10 +78,8 @@ pub fn parse_deep_object(prefix: &str, value: &serde_json::Value) -> Vec<(String value, )); } - } - serde_json::Value::String(s) => { - params.push((format!("{}[{}]", prefix, key), s.clone())) - } + }, + serde_json::Value::String(s) => params.push((format!("{}[{}]", prefix, key), s.clone())), _ => params.push((format!("{}[{}]", prefix, key), value.to_string())), } } @@ -92,4 +90,5 @@ pub fn parse_deep_object(prefix: &str, value: &serde_json::Value) -> Vec<(String unimplemented!("Only objects are supported with style=deepObject") } + pub mod configuration; diff --git a/serving/numaflow-models/src/lib.rs b/serving/numaflow-models/src/lib.rs index fc22e4e4b9..c1dd666f79 100644 --- a/serving/numaflow-models/src/lib.rs +++ b/serving/numaflow-models/src/lib.rs @@ -1,10 +1,10 @@ #[macro_use] extern crate serde_derive; -extern crate reqwest; extern crate serde; extern crate serde_json; extern crate url; +extern crate reqwest; pub mod apis; pub mod models; diff --git a/serving/numaflow-models/src/models/abstract_pod_template.rs b/serving/numaflow-models/src/models/abstract_pod_template.rs index 6de4149ab1..fd19a7b425 100644 --- a/serving/numaflow-models/src/models/abstract_pod_template.rs +++ b/serving/numaflow-models/src/models/abstract_pod_template.rs @@ -4,21 +4,20 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// AbstractPodTemplate : AbstractPodTemplate provides a template for pod customization in vertices, daemon deployments and so on. + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct AbstractPodTemplate { #[serde(rename = "affinity", skip_serializing_if = "Option::is_none")] pub affinity: Option, /// AutomountServiceAccountToken indicates whether a service account token should be automatically mounted. - #[serde( - rename = "automountServiceAccountToken", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "automountServiceAccountToken", skip_serializing_if = "Option::is_none")] pub automount_service_account_token: Option, #[serde(rename = "dnsConfig", skip_serializing_if = "Option::is_none")] pub dns_config: Option, @@ -72,3 +71,5 @@ impl AbstractPodTemplate { } } } + + diff --git a/serving/numaflow-models/src/models/abstract_sink.rs b/serving/numaflow-models/src/models/abstract_sink.rs index ab8f2ba058..48bf8c8276 100644 --- a/serving/numaflow-models/src/models/abstract_sink.rs +++ b/serving/numaflow-models/src/models/abstract_sink.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct AbstractSink { #[serde(rename = "blackhole", skip_serializing_if = "Option::is_none")] @@ -30,3 +33,5 @@ impl AbstractSink { } } } + + diff --git a/serving/numaflow-models/src/models/abstract_vertex.rs b/serving/numaflow-models/src/models/abstract_vertex.rs index 62fc1240ce..1ee864662e 100644 --- a/serving/numaflow-models/src/models/abstract_vertex.rs +++ b/serving/numaflow-models/src/models/abstract_vertex.rs @@ -4,19 +4,19 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct AbstractVertex { #[serde(rename = "affinity", skip_serializing_if = "Option::is_none")] pub affinity: Option, /// AutomountServiceAccountToken indicates whether a service account token should be automatically mounted. - #[serde( - rename = "automountServiceAccountToken", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "automountServiceAccountToken", skip_serializing_if = "Option::is_none")] pub automount_service_account_token: Option, #[serde(rename = "containerTemplate", skip_serializing_if = "Option::is_none")] pub container_template: Option>, @@ -28,10 +28,7 @@ pub struct AbstractVertex { /// ImagePullSecrets is an optional list of references to secrets in the same namespace to use for pulling any of the images used by this PodSpec. If specified, these secrets will be passed to individual puller implementations for them to use. For example, in the case of docker, only DockerConfig type secrets are honored. More info: https://kubernetes.io/docs/concepts/containers/images#specifying-imagepullsecrets-on-a-pod #[serde(rename = "imagePullSecrets", skip_serializing_if = "Option::is_none")] pub image_pull_secrets: Option>, - #[serde( - rename = "initContainerTemplate", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "initContainerTemplate", skip_serializing_if = "Option::is_none")] pub init_container_template: Option>, /// List of customized init containers belonging to the pod. More info: https://kubernetes.io/docs/concepts/workloads/pods/init-containers/ #[serde(rename = "initContainers", skip_serializing_if = "Option::is_none")] @@ -67,10 +64,7 @@ pub struct AbstractVertex { /// Names of the side inputs used in this vertex. #[serde(rename = "sideInputs", skip_serializing_if = "Option::is_none")] pub side_inputs: Option>, - #[serde( - rename = "sideInputsContainerTemplate", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "sideInputsContainerTemplate", skip_serializing_if = "Option::is_none")] pub side_inputs_container_template: Option>, /// List of customized sidecar containers belonging to the pod. #[serde(rename = "sidecars", skip_serializing_if = "Option::is_none")] @@ -121,3 +115,5 @@ impl AbstractVertex { } } } + + diff --git a/serving/numaflow-models/src/models/authorization.rs b/serving/numaflow-models/src/models/authorization.rs index f1242768d2..6589dc82f4 100644 --- a/serving/numaflow-models/src/models/authorization.rs +++ b/serving/numaflow-models/src/models/authorization.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Authorization { #[serde(rename = "token", skip_serializing_if = "Option::is_none")] @@ -16,6 +19,10 @@ pub struct Authorization { impl Authorization { pub fn new() -> Authorization { - Authorization { token: None } + Authorization { + token: None, + } } } + + diff --git a/serving/numaflow-models/src/models/basic_auth.rs b/serving/numaflow-models/src/models/basic_auth.rs index 1b5eea25d8..e7a4e7d2c7 100644 --- a/serving/numaflow-models/src/models/basic_auth.rs +++ b/serving/numaflow-models/src/models/basic_auth.rs @@ -4,12 +4,14 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// BasicAuth : BasicAuth represents the basic authentication approach which contains a user name and a password. + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct BasicAuth { #[serde(rename = "password", skip_serializing_if = "Option::is_none")] @@ -27,3 +29,5 @@ impl BasicAuth { } } } + + diff --git a/serving/numaflow-models/src/models/blackhole.rs b/serving/numaflow-models/src/models/blackhole.rs index 145871aa16..f84586c042 100644 --- a/serving/numaflow-models/src/models/blackhole.rs +++ b/serving/numaflow-models/src/models/blackhole.rs @@ -4,18 +4,24 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// Blackhole : Blackhole is a sink to emulate /dev/null + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] -pub struct Blackhole {} +pub struct Blackhole { +} impl Blackhole { /// Blackhole is a sink to emulate /dev/null pub fn new() -> Blackhole { - Blackhole {} + Blackhole { + } } } + + diff --git a/serving/numaflow-models/src/models/buffer_service_config.rs b/serving/numaflow-models/src/models/buffer_service_config.rs index 24e621c1ea..bad5eec796 100644 --- a/serving/numaflow-models/src/models/buffer_service_config.rs +++ b/serving/numaflow-models/src/models/buffer_service_config.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct BufferServiceConfig { #[serde(rename = "jetstream", skip_serializing_if = "Option::is_none")] @@ -24,3 +27,5 @@ impl BufferServiceConfig { } } } + + diff --git a/serving/numaflow-models/src/models/combined_edge.rs b/serving/numaflow-models/src/models/combined_edge.rs index 94d478727c..b7fdf05446 100644 --- a/serving/numaflow-models/src/models/combined_edge.rs +++ b/serving/numaflow-models/src/models/combined_edge.rs @@ -4,12 +4,14 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// CombinedEdge : CombinedEdge is a combination of Edge and some other properties such as vertex type, partitions, limits. It's used to decorate the fromEdges and toEdges of the generated Vertex objects, so that in the vertex pod, it knows the properties of the connected vertices, for example, how many partitioned buffers I should write to, what is the write buffer length, etc. + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct CombinedEdge { #[serde(rename = "conditions", skip_serializing_if = "Option::is_none")] @@ -19,10 +21,7 @@ pub struct CombinedEdge { #[serde(rename = "fromVertexLimits", skip_serializing_if = "Option::is_none")] pub from_vertex_limits: Option>, /// The number of partitions of the from vertex, if not provided, the default value is set to \"1\". - #[serde( - rename = "fromVertexPartitionCount", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "fromVertexPartitionCount", skip_serializing_if = "Option::is_none")] pub from_vertex_partition_count: Option, /// From vertex type. #[serde(rename = "fromVertexType")] @@ -35,10 +34,7 @@ pub struct CombinedEdge { #[serde(rename = "toVertexLimits", skip_serializing_if = "Option::is_none")] pub to_vertex_limits: Option>, /// The number of partitions of the to vertex, if not provided, the default value is set to \"1\". - #[serde( - rename = "toVertexPartitionCount", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "toVertexPartitionCount", skip_serializing_if = "Option::is_none")] pub to_vertex_partition_count: Option, /// To vertex type. #[serde(rename = "toVertexType")] @@ -47,12 +43,7 @@ pub struct CombinedEdge { impl CombinedEdge { /// CombinedEdge is a combination of Edge and some other properties such as vertex type, partitions, limits. It's used to decorate the fromEdges and toEdges of the generated Vertex objects, so that in the vertex pod, it knows the properties of the connected vertices, for example, how many partitioned buffers I should write to, what is the write buffer length, etc. - pub fn new( - from: String, - from_vertex_type: String, - to: String, - to_vertex_type: String, - ) -> CombinedEdge { + pub fn new(from: String, from_vertex_type: String, to: String, to_vertex_type: String) -> CombinedEdge { CombinedEdge { conditions: None, from, @@ -67,3 +58,5 @@ impl CombinedEdge { } } } + + diff --git a/serving/numaflow-models/src/models/container.rs b/serving/numaflow-models/src/models/container.rs index 4d3d2730a0..22d8643134 100644 --- a/serving/numaflow-models/src/models/container.rs +++ b/serving/numaflow-models/src/models/container.rs @@ -4,12 +4,14 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// Container : Container is used to define the container properties for user-defined functions, sinks, etc. + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Container { #[serde(rename = "args", skip_serializing_if = "Option::is_none")] @@ -48,3 +50,5 @@ impl Container { } } } + + diff --git a/serving/numaflow-models/src/models/container_builder.rs b/serving/numaflow-models/src/models/container_builder.rs index 35898065d5..47a84c6fe7 100644 --- a/serving/numaflow-models/src/models/container_builder.rs +++ b/serving/numaflow-models/src/models/container_builder.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct ContainerBuilder { /// Arguments to the entrypoint. The container image's CMD is used if this is not provided. Variable references $(VAR_NAME) are expanded using the container's environment. If a variable cannot be resolved, the reference in the input string will be unchanged. Double $$ are reduced to a single $, which allows for escaping the $(VAR_NAME) syntax: i.e. \"$$(VAR_NAME)\" will produce the string literal \"$(VAR_NAME)\". Escaped references will never be expanded, regardless of whether the variable exists or not. Cannot be updated. More info: https://kubernetes.io/docs/tasks/inject-data-application/define-command-argument-container/#running-a-command-in-a-shell @@ -59,16 +62,10 @@ pub struct ContainerBuilder { #[serde(rename = "stdinOnce", skip_serializing_if = "Option::is_none")] pub stdin_once: Option, /// Optional: Path at which the file to which the container's termination message will be written is mounted into the container's filesystem. Message written is intended to be brief final status, such as an assertion failure message. Will be truncated by the node if greater than 4096 bytes. The total message length across all containers will be limited to 12kb. Defaults to /dev/termination-log. Cannot be updated. - #[serde( - rename = "terminationMessagePath", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "terminationMessagePath", skip_serializing_if = "Option::is_none")] pub termination_message_path: Option, /// Indicate how the termination message should be populated. File will use the contents of terminationMessagePath to populate the container status message on both success and failure. FallbackToLogsOnError will use the last chunk of container log output if the termination message file is empty and the container exited with an error. The log output is limited to 2048 bytes or 80 lines, whichever is smaller. Defaults to File. Cannot be updated. - #[serde( - rename = "terminationMessagePolicy", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "terminationMessagePolicy", skip_serializing_if = "Option::is_none")] pub termination_message_policy: Option, /// Whether this container should allocate a TTY for itself, also requires 'stdin' to be true. Default is false. #[serde(rename = "tty", skip_serializing_if = "Option::is_none")] @@ -114,3 +111,5 @@ impl ContainerBuilder { } } } + + diff --git a/serving/numaflow-models/src/models/container_template.rs b/serving/numaflow-models/src/models/container_template.rs index 4c85b7a7f6..737fbf7701 100644 --- a/serving/numaflow-models/src/models/container_template.rs +++ b/serving/numaflow-models/src/models/container_template.rs @@ -4,12 +4,14 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// ContainerTemplate : ContainerTemplate defines customized spec for a container + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct ContainerTemplate { #[serde(rename = "env", skip_serializing_if = "Option::is_none")] @@ -36,3 +38,5 @@ impl ContainerTemplate { } } } + + diff --git a/serving/numaflow-models/src/models/daemon_template.rs b/serving/numaflow-models/src/models/daemon_template.rs index 97290e6fac..82c8e4ecc5 100644 --- a/serving/numaflow-models/src/models/daemon_template.rs +++ b/serving/numaflow-models/src/models/daemon_template.rs @@ -4,19 +4,19 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct DaemonTemplate { #[serde(rename = "affinity", skip_serializing_if = "Option::is_none")] pub affinity: Option, /// AutomountServiceAccountToken indicates whether a service account token should be automatically mounted. - #[serde( - rename = "automountServiceAccountToken", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "automountServiceAccountToken", skip_serializing_if = "Option::is_none")] pub automount_service_account_token: Option, #[serde(rename = "containerTemplate", skip_serializing_if = "Option::is_none")] pub container_template: Option>, @@ -28,10 +28,7 @@ pub struct DaemonTemplate { /// ImagePullSecrets is an optional list of references to secrets in the same namespace to use for pulling any of the images used by this PodSpec. If specified, these secrets will be passed to individual puller implementations for them to use. For example, in the case of docker, only DockerConfig type secrets are honored. More info: https://kubernetes.io/docs/concepts/containers/images#specifying-imagepullsecrets-on-a-pod #[serde(rename = "imagePullSecrets", skip_serializing_if = "Option::is_none")] pub image_pull_secrets: Option>, - #[serde( - rename = "initContainerTemplate", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "initContainerTemplate", skip_serializing_if = "Option::is_none")] pub init_container_template: Option>, #[serde(rename = "metadata", skip_serializing_if = "Option::is_none")] pub metadata: Option>, @@ -82,3 +79,5 @@ impl DaemonTemplate { } } } + + diff --git a/serving/numaflow-models/src/models/edge.rs b/serving/numaflow-models/src/models/edge.rs index 95a3b9742b..33afab8014 100644 --- a/serving/numaflow-models/src/models/edge.rs +++ b/serving/numaflow-models/src/models/edge.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Edge { #[serde(rename = "conditions", skip_serializing_if = "Option::is_none")] @@ -31,3 +34,5 @@ impl Edge { } } } + + diff --git a/serving/numaflow-models/src/models/fixed_window.rs b/serving/numaflow-models/src/models/fixed_window.rs index 76a3a9dc1c..e62eee6d1a 100644 --- a/serving/numaflow-models/src/models/fixed_window.rs +++ b/serving/numaflow-models/src/models/fixed_window.rs @@ -4,12 +4,14 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// FixedWindow : FixedWindow describes a fixed window + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct FixedWindow { #[serde(rename = "length", skip_serializing_if = "Option::is_none")] @@ -28,3 +30,5 @@ impl FixedWindow { } } } + + diff --git a/serving/numaflow-models/src/models/forward_conditions.rs b/serving/numaflow-models/src/models/forward_conditions.rs index ea99358902..c75cb03634 100644 --- a/serving/numaflow-models/src/models/forward_conditions.rs +++ b/serving/numaflow-models/src/models/forward_conditions.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct ForwardConditions { #[serde(rename = "tags")] @@ -21,3 +24,5 @@ impl ForwardConditions { } } } + + diff --git a/serving/numaflow-models/src/models/function.rs b/serving/numaflow-models/src/models/function.rs index 8081c827fe..147fd444fe 100644 --- a/serving/numaflow-models/src/models/function.rs +++ b/serving/numaflow-models/src/models/function.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Function { #[serde(rename = "args", skip_serializing_if = "Option::is_none")] @@ -27,3 +30,5 @@ impl Function { } } } + + diff --git a/serving/numaflow-models/src/models/generator_source.rs b/serving/numaflow-models/src/models/generator_source.rs index 268652d0e3..d22bb877da 100644 --- a/serving/numaflow-models/src/models/generator_source.rs +++ b/serving/numaflow-models/src/models/generator_source.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct GeneratorSource { #[serde(rename = "duration", skip_serializing_if = "Option::is_none")] @@ -43,3 +46,5 @@ impl GeneratorSource { } } } + + diff --git a/serving/numaflow-models/src/models/get_container_req.rs b/serving/numaflow-models/src/models/get_container_req.rs index 2d97d73f1b..5b2695f98f 100644 --- a/serving/numaflow-models/src/models/get_container_req.rs +++ b/serving/numaflow-models/src/models/get_container_req.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct GetContainerReq { #[serde(rename = "env")] @@ -25,14 +28,7 @@ pub struct GetContainerReq { } impl GetContainerReq { - pub fn new( - env: Vec, - image: String, - image_pull_policy: String, - isb_svc_type: String, - resources: k8s_openapi::api::core::v1::ResourceRequirements, - volume_mounts: Vec, - ) -> GetContainerReq { + pub fn new(env: Vec, image: String, image_pull_policy: String, isb_svc_type: String, resources: k8s_openapi::api::core::v1::ResourceRequirements, volume_mounts: Vec) -> GetContainerReq { GetContainerReq { env, image, @@ -43,3 +39,5 @@ impl GetContainerReq { } } } + + diff --git a/serving/numaflow-models/src/models/get_daemon_deployment_req.rs b/serving/numaflow-models/src/models/get_daemon_deployment_req.rs index 2c20b6eba2..4f8a7cc3b9 100644 --- a/serving/numaflow-models/src/models/get_daemon_deployment_req.rs +++ b/serving/numaflow-models/src/models/get_daemon_deployment_req.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct GetDaemonDeploymentReq { #[serde(rename = "DefaultResources")] @@ -23,13 +26,7 @@ pub struct GetDaemonDeploymentReq { } impl GetDaemonDeploymentReq { - pub fn new( - default_resources: k8s_openapi::api::core::v1::ResourceRequirements, - env: Vec, - isb_svc_type: String, - image: String, - pull_policy: String, - ) -> GetDaemonDeploymentReq { + pub fn new(default_resources: k8s_openapi::api::core::v1::ResourceRequirements, env: Vec, isb_svc_type: String, image: String, pull_policy: String) -> GetDaemonDeploymentReq { GetDaemonDeploymentReq { default_resources, env, @@ -39,3 +36,5 @@ impl GetDaemonDeploymentReq { } } } + + diff --git a/serving/numaflow-models/src/models/get_jet_stream_service_spec_req.rs b/serving/numaflow-models/src/models/get_jet_stream_service_spec_req.rs index e6168e7d3e..9c7891afd4 100644 --- a/serving/numaflow-models/src/models/get_jet_stream_service_spec_req.rs +++ b/serving/numaflow-models/src/models/get_jet_stream_service_spec_req.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct GetJetStreamServiceSpecReq { #[serde(rename = "ClientPort")] @@ -23,13 +26,7 @@ pub struct GetJetStreamServiceSpecReq { } impl GetJetStreamServiceSpecReq { - pub fn new( - client_port: i32, - cluster_port: i32, - labels: ::std::collections::HashMap, - metrics_port: i32, - monitor_port: i32, - ) -> GetJetStreamServiceSpecReq { + pub fn new(client_port: i32, cluster_port: i32, labels: ::std::collections::HashMap, metrics_port: i32, monitor_port: i32) -> GetJetStreamServiceSpecReq { GetJetStreamServiceSpecReq { client_port, cluster_port, @@ -39,3 +36,5 @@ impl GetJetStreamServiceSpecReq { } } } + + diff --git a/serving/numaflow-models/src/models/get_jet_stream_stateful_set_spec_req.rs b/serving/numaflow-models/src/models/get_jet_stream_stateful_set_spec_req.rs index ffa11c5bbe..894ae495fd 100644 --- a/serving/numaflow-models/src/models/get_jet_stream_stateful_set_spec_req.rs +++ b/serving/numaflow-models/src/models/get_jet_stream_stateful_set_spec_req.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct GetJetStreamStatefulSetSpecReq { #[serde(rename = "ClientPort")] @@ -43,23 +46,7 @@ pub struct GetJetStreamStatefulSetSpecReq { } impl GetJetStreamStatefulSetSpecReq { - pub fn new( - client_port: i32, - cluster_port: i32, - config_map_name: String, - config_reloader_image: String, - default_resources: k8s_openapi::api::core::v1::ResourceRequirements, - labels: ::std::collections::HashMap, - metrics_exporter_image: String, - metrics_port: i32, - monitor_port: i32, - nats_image: String, - pvc_name_if_needed: String, - server_auth_secret_name: String, - server_encryption_secret_name: String, - service_name: String, - start_command: String, - ) -> GetJetStreamStatefulSetSpecReq { + pub fn new(client_port: i32, cluster_port: i32, config_map_name: String, config_reloader_image: String, default_resources: k8s_openapi::api::core::v1::ResourceRequirements, labels: ::std::collections::HashMap, metrics_exporter_image: String, metrics_port: i32, monitor_port: i32, nats_image: String, pvc_name_if_needed: String, server_auth_secret_name: String, server_encryption_secret_name: String, service_name: String, start_command: String) -> GetJetStreamStatefulSetSpecReq { GetJetStreamStatefulSetSpecReq { client_port, cluster_port, @@ -79,3 +66,5 @@ impl GetJetStreamStatefulSetSpecReq { } } } + + diff --git a/serving/numaflow-models/src/models/get_redis_service_spec_req.rs b/serving/numaflow-models/src/models/get_redis_service_spec_req.rs index df584ea71a..651a67d08e 100644 --- a/serving/numaflow-models/src/models/get_redis_service_spec_req.rs +++ b/serving/numaflow-models/src/models/get_redis_service_spec_req.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct GetRedisServiceSpecReq { #[serde(rename = "Labels")] @@ -19,11 +22,7 @@ pub struct GetRedisServiceSpecReq { } impl GetRedisServiceSpecReq { - pub fn new( - labels: ::std::collections::HashMap, - redis_container_port: i32, - sentinel_container_port: i32, - ) -> GetRedisServiceSpecReq { + pub fn new(labels: ::std::collections::HashMap, redis_container_port: i32, sentinel_container_port: i32) -> GetRedisServiceSpecReq { GetRedisServiceSpecReq { labels, redis_container_port, @@ -31,3 +30,5 @@ impl GetRedisServiceSpecReq { } } } + + diff --git a/serving/numaflow-models/src/models/get_redis_stateful_set_spec_req.rs b/serving/numaflow-models/src/models/get_redis_stateful_set_spec_req.rs index 81cc11b7c9..736ea82fa4 100644 --- a/serving/numaflow-models/src/models/get_redis_stateful_set_spec_req.rs +++ b/serving/numaflow-models/src/models/get_redis_stateful_set_spec_req.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct GetRedisStatefulSetSpecReq { #[serde(rename = "ConfConfigMapName")] @@ -45,24 +48,7 @@ pub struct GetRedisStatefulSetSpecReq { } impl GetRedisStatefulSetSpecReq { - pub fn new( - conf_config_map_name: String, - credential_secret_name: String, - default_resources: k8s_openapi::api::core::v1::ResourceRequirements, - health_config_map_name: String, - init_container_image: String, - labels: ::std::collections::HashMap, - metrics_exporter_image: String, - pvc_name_if_needed: String, - redis_container_port: i32, - redis_image: String, - redis_metrics_container_port: i32, - scripts_config_map_name: String, - sentinel_container_port: i32, - sentinel_image: String, - service_name: String, - tls_enabled: bool, - ) -> GetRedisStatefulSetSpecReq { + pub fn new(conf_config_map_name: String, credential_secret_name: String, default_resources: k8s_openapi::api::core::v1::ResourceRequirements, health_config_map_name: String, init_container_image: String, labels: ::std::collections::HashMap, metrics_exporter_image: String, pvc_name_if_needed: String, redis_container_port: i32, redis_image: String, redis_metrics_container_port: i32, scripts_config_map_name: String, sentinel_container_port: i32, sentinel_image: String, service_name: String, tls_enabled: bool) -> GetRedisStatefulSetSpecReq { GetRedisStatefulSetSpecReq { conf_config_map_name, credential_secret_name, @@ -83,3 +69,5 @@ impl GetRedisStatefulSetSpecReq { } } } + + diff --git a/serving/numaflow-models/src/models/get_side_input_deployment_req.rs b/serving/numaflow-models/src/models/get_side_input_deployment_req.rs index f548abcf4d..a2d389d107 100644 --- a/serving/numaflow-models/src/models/get_side_input_deployment_req.rs +++ b/serving/numaflow-models/src/models/get_side_input_deployment_req.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct GetSideInputDeploymentReq { #[serde(rename = "DefaultResources")] @@ -23,13 +26,7 @@ pub struct GetSideInputDeploymentReq { } impl GetSideInputDeploymentReq { - pub fn new( - default_resources: k8s_openapi::api::core::v1::ResourceRequirements, - env: Vec, - isb_svc_type: String, - image: String, - pull_policy: String, - ) -> GetSideInputDeploymentReq { + pub fn new(default_resources: k8s_openapi::api::core::v1::ResourceRequirements, env: Vec, isb_svc_type: String, image: String, pull_policy: String) -> GetSideInputDeploymentReq { GetSideInputDeploymentReq { default_resources, env, @@ -39,3 +36,5 @@ impl GetSideInputDeploymentReq { } } } + + diff --git a/serving/numaflow-models/src/models/get_vertex_pod_spec_req.rs b/serving/numaflow-models/src/models/get_vertex_pod_spec_req.rs index bd976cf1f9..26f6185385 100644 --- a/serving/numaflow-models/src/models/get_vertex_pod_spec_req.rs +++ b/serving/numaflow-models/src/models/get_vertex_pod_spec_req.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct GetVertexPodSpecReq { #[serde(rename = "DefaultResources")] @@ -29,16 +32,7 @@ pub struct GetVertexPodSpecReq { } impl GetVertexPodSpecReq { - pub fn new( - default_resources: k8s_openapi::api::core::v1::ResourceRequirements, - env: Vec, - isb_svc_type: String, - image: String, - pipeline_spec: crate::models::PipelineSpec, - pull_policy: String, - serving_source_stream_name: String, - side_inputs_store_name: String, - ) -> GetVertexPodSpecReq { + pub fn new(default_resources: k8s_openapi::api::core::v1::ResourceRequirements, env: Vec, isb_svc_type: String, image: String, pipeline_spec: crate::models::PipelineSpec, pull_policy: String, serving_source_stream_name: String, side_inputs_store_name: String) -> GetVertexPodSpecReq { GetVertexPodSpecReq { default_resources, env, @@ -51,3 +45,5 @@ impl GetVertexPodSpecReq { } } } + + diff --git a/serving/numaflow-models/src/models/group_by.rs b/serving/numaflow-models/src/models/group_by.rs index 915e8a5bca..605b3bc61a 100644 --- a/serving/numaflow-models/src/models/group_by.rs +++ b/serving/numaflow-models/src/models/group_by.rs @@ -4,12 +4,14 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// GroupBy : GroupBy indicates it is a reducer UDF + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct GroupBy { #[serde(rename = "allowedLateness", skip_serializing_if = "Option::is_none")] @@ -33,3 +35,5 @@ impl GroupBy { } } } + + diff --git a/serving/numaflow-models/src/models/gssapi.rs b/serving/numaflow-models/src/models/gssapi.rs index d040f83cb7..c5fe33aad6 100644 --- a/serving/numaflow-models/src/models/gssapi.rs +++ b/serving/numaflow-models/src/models/gssapi.rs @@ -4,21 +4,20 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// Gssapi : GSSAPI represents a SASL GSSAPI config + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Gssapi { /// valid inputs - KRB5_USER_AUTH, KRB5_KEYTAB_AUTH Possible enum values: - `\"KRB5_KEYTAB_AUTH\"` represents the password method KRB5KeytabAuth = \"KRB5_KEYTAB_AUTH\" = 2 - `\"KRB5_USER_AUTH\"` represents the password method KRB5UserAuth = \"KRB5_USER_AUTH\" = 1 #[serde(rename = "authType")] pub auth_type: AuthType, - #[serde( - rename = "kerberosConfigSecret", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "kerberosConfigSecret", skip_serializing_if = "Option::is_none")] pub kerberos_config_secret: Option, #[serde(rename = "keytabSecret", skip_serializing_if = "Option::is_none")] pub keytab_secret: Option, @@ -34,12 +33,7 @@ pub struct Gssapi { impl Gssapi { /// GSSAPI represents a SASL GSSAPI config - pub fn new( - auth_type: AuthType, - realm: String, - service_name: String, - username_secret: k8s_openapi::api::core::v1::SecretKeySelector, - ) -> Gssapi { + pub fn new(auth_type: AuthType, realm: String, service_name: String, username_secret: k8s_openapi::api::core::v1::SecretKeySelector) -> Gssapi { Gssapi { auth_type, kerberos_config_secret: None, @@ -66,3 +60,4 @@ impl Default for AuthType { Self::KeytabAuth } } + diff --git a/serving/numaflow-models/src/models/http_source.rs b/serving/numaflow-models/src/models/http_source.rs index e5178c86e3..8a5b7c7c97 100644 --- a/serving/numaflow-models/src/models/http_source.rs +++ b/serving/numaflow-models/src/models/http_source.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct HttpSource { #[serde(rename = "auth", skip_serializing_if = "Option::is_none")] @@ -25,3 +28,5 @@ impl HttpSource { } } } + + diff --git a/serving/numaflow-models/src/models/idle_source.rs b/serving/numaflow-models/src/models/idle_source.rs index cbe629ae9f..3d593c5375 100644 --- a/serving/numaflow-models/src/models/idle_source.rs +++ b/serving/numaflow-models/src/models/idle_source.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct IdleSource { #[serde(rename = "incrementBy", skip_serializing_if = "Option::is_none")] @@ -27,3 +30,5 @@ impl IdleSource { } } } + + diff --git a/serving/numaflow-models/src/models/inter_step_buffer_service.rs b/serving/numaflow-models/src/models/inter_step_buffer_service.rs index 8e6a9e20f4..fc6154b756 100644 --- a/serving/numaflow-models/src/models/inter_step_buffer_service.rs +++ b/serving/numaflow-models/src/models/inter_step_buffer_service.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct InterStepBufferService { /// APIVersion defines the versioned schema of this representation of an object. Servers should convert recognized schemas to the latest internal value, and may reject unrecognized values. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#resources @@ -35,3 +38,5 @@ impl InterStepBufferService { } } } + + diff --git a/serving/numaflow-models/src/models/inter_step_buffer_service_list.rs b/serving/numaflow-models/src/models/inter_step_buffer_service_list.rs index 4ef7a04c49..e933727ce5 100644 --- a/serving/numaflow-models/src/models/inter_step_buffer_service_list.rs +++ b/serving/numaflow-models/src/models/inter_step_buffer_service_list.rs @@ -4,12 +4,14 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// InterStepBufferServiceList : InterStepBufferServiceList is the list of InterStepBufferService resources + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct InterStepBufferServiceList { /// APIVersion defines the versioned schema of this representation of an object. Servers should convert recognized schemas to the latest internal value, and may reject unrecognized values. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#resources @@ -26,10 +28,7 @@ pub struct InterStepBufferServiceList { impl InterStepBufferServiceList { /// InterStepBufferServiceList is the list of InterStepBufferService resources - pub fn new( - items: Vec, - metadata: k8s_openapi::apimachinery::pkg::apis::meta::v1::ListMeta, - ) -> InterStepBufferServiceList { + pub fn new(items: Vec, metadata: k8s_openapi::apimachinery::pkg::apis::meta::v1::ListMeta) -> InterStepBufferServiceList { InterStepBufferServiceList { api_version: None, items, @@ -38,3 +37,5 @@ impl InterStepBufferServiceList { } } } + + diff --git a/serving/numaflow-models/src/models/inter_step_buffer_service_spec.rs b/serving/numaflow-models/src/models/inter_step_buffer_service_spec.rs index 2de93da314..79be40fa45 100644 --- a/serving/numaflow-models/src/models/inter_step_buffer_service_spec.rs +++ b/serving/numaflow-models/src/models/inter_step_buffer_service_spec.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct InterStepBufferServiceSpec { #[serde(rename = "jetstream", skip_serializing_if = "Option::is_none")] @@ -24,3 +27,5 @@ impl InterStepBufferServiceSpec { } } } + + diff --git a/serving/numaflow-models/src/models/inter_step_buffer_service_status.rs b/serving/numaflow-models/src/models/inter_step_buffer_service_status.rs index 43c7cd16cf..6cf0a96745 100644 --- a/serving/numaflow-models/src/models/inter_step_buffer_service_status.rs +++ b/serving/numaflow-models/src/models/inter_step_buffer_service_status.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct InterStepBufferServiceStatus { /// Conditions are the latest available observations of a resource's current state. @@ -37,3 +40,5 @@ impl InterStepBufferServiceStatus { } } } + + diff --git a/serving/numaflow-models/src/models/jet_stream_buffer_service.rs b/serving/numaflow-models/src/models/jet_stream_buffer_service.rs index de5392a051..9441f6d156 100644 --- a/serving/numaflow-models/src/models/jet_stream_buffer_service.rs +++ b/serving/numaflow-models/src/models/jet_stream_buffer_service.rs @@ -4,19 +4,19 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct JetStreamBufferService { #[serde(rename = "affinity", skip_serializing_if = "Option::is_none")] pub affinity: Option, /// AutomountServiceAccountToken indicates whether a service account token should be automatically mounted. - #[serde( - rename = "automountServiceAccountToken", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "automountServiceAccountToken", skip_serializing_if = "Option::is_none")] pub automount_service_account_token: Option, /// Optional configuration for the streams, consumers and buckets to be created in this JetStream service, if specified, it will be merged with the default configuration in numaflow-controller-config. It accepts a YAML format configuration, it may include 4 sections, \"stream\", \"consumer\", \"otBucket\" and \"procBucket\". Available fields under \"stream\" include \"retention\" (e.g. interest, limits, workerQueue), \"maxMsgs\", \"maxAge\" (e.g. 72h), \"replicas\" (1, 3, 5), \"duplicates\" (e.g. 5m). Available fields under \"consumer\" include \"ackWait\" (e.g. 60s) Available fields under \"otBucket\" include \"maxValueSize\", \"history\", \"ttl\" (e.g. 72h), \"maxBytes\", \"replicas\" (1, 3, 5). Available fields under \"procBucket\" include \"maxValueSize\", \"history\", \"ttl\" (e.g. 72h), \"maxBytes\", \"replicas\" (1, 3, 5). #[serde(rename = "bufferConfig", skip_serializing_if = "Option::is_none")] @@ -36,10 +36,7 @@ pub struct JetStreamBufferService { pub image_pull_secrets: Option>, #[serde(rename = "metadata", skip_serializing_if = "Option::is_none")] pub metadata: Option>, - #[serde( - rename = "metricsContainerTemplate", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "metricsContainerTemplate", skip_serializing_if = "Option::is_none")] pub metrics_container_template: Option>, /// NodeSelector is a selector which must be true for the pod to fit on a node. Selector which must match a node's labels for the pod to be scheduled on that node. More info: https://kubernetes.io/docs/concepts/configuration/assign-pod-node/ #[serde(rename = "nodeSelector", skip_serializing_if = "Option::is_none")] @@ -52,10 +49,7 @@ pub struct JetStreamBufferService { /// If specified, indicates the Redis pod's priority. \"system-node-critical\" and \"system-cluster-critical\" are two special keywords which indicate the highest priorities with the former being the highest priority. Any other name must be defined by creating a PriorityClass object with that name. If not specified, the pod priority will be default or zero if there is no default. More info: https://kubernetes.io/docs/concepts/configuration/pod-priority-preemption/ #[serde(rename = "priorityClassName", skip_serializing_if = "Option::is_none")] pub priority_class_name: Option, - #[serde( - rename = "reloaderContainerTemplate", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "reloaderContainerTemplate", skip_serializing_if = "Option::is_none")] pub reloader_container_template: Option>, /// JetStream StatefulSet size #[serde(rename = "replicas", skip_serializing_if = "Option::is_none")] @@ -115,3 +109,5 @@ impl JetStreamBufferService { } } } + + diff --git a/serving/numaflow-models/src/models/jet_stream_config.rs b/serving/numaflow-models/src/models/jet_stream_config.rs index d62ceab186..bc4c648c1a 100644 --- a/serving/numaflow-models/src/models/jet_stream_config.rs +++ b/serving/numaflow-models/src/models/jet_stream_config.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct JetStreamConfig { #[serde(rename = "auth", skip_serializing_if = "Option::is_none")] @@ -32,3 +35,5 @@ impl JetStreamConfig { } } } + + diff --git a/serving/numaflow-models/src/models/jet_stream_source.rs b/serving/numaflow-models/src/models/jet_stream_source.rs index c1040259e3..6da52b583c 100644 --- a/serving/numaflow-models/src/models/jet_stream_source.rs +++ b/serving/numaflow-models/src/models/jet_stream_source.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct JetStreamSource { #[serde(rename = "auth", skip_serializing_if = "Option::is_none")] @@ -32,3 +35,5 @@ impl JetStreamSource { } } } + + diff --git a/serving/numaflow-models/src/models/job_template.rs b/serving/numaflow-models/src/models/job_template.rs index 9ffe8e162e..dcae7d906a 100644 --- a/serving/numaflow-models/src/models/job_template.rs +++ b/serving/numaflow-models/src/models/job_template.rs @@ -4,19 +4,19 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct JobTemplate { #[serde(rename = "affinity", skip_serializing_if = "Option::is_none")] pub affinity: Option, /// AutomountServiceAccountToken indicates whether a service account token should be automatically mounted. - #[serde( - rename = "automountServiceAccountToken", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "automountServiceAccountToken", skip_serializing_if = "Option::is_none")] pub automount_service_account_token: Option, /// Specifies the number of retries before marking this job failed. More info: https://kubernetes.io/docs/concepts/workloads/controllers/job/#pod-backoff-failure-policy Numaflow defaults to 20 #[serde(rename = "backoffLimit", skip_serializing_if = "Option::is_none")] @@ -54,10 +54,7 @@ pub struct JobTemplate { #[serde(rename = "tolerations", skip_serializing_if = "Option::is_none")] pub tolerations: Option>, /// ttlSecondsAfterFinished limits the lifetime of a Job that has finished execution (either Complete or Failed). If this field is set, ttlSecondsAfterFinished after the Job finishes, it is eligible to be automatically deleted. When the Job is being deleted, its lifecycle guarantees (e.g. finalizers) will be honored. If this field is unset, the Job won't be automatically deleted. If this field is set to zero, the Job becomes eligible to be deleted immediately after it finishes. Numaflow defaults to 30 - #[serde( - rename = "ttlSecondsAfterFinished", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "ttlSecondsAfterFinished", skip_serializing_if = "Option::is_none")] pub ttl_seconds_after_finished: Option, } @@ -83,3 +80,5 @@ impl JobTemplate { } } } + + diff --git a/serving/numaflow-models/src/models/kafka_sink.rs b/serving/numaflow-models/src/models/kafka_sink.rs index 99dabd7a4e..3b2c730fa5 100644 --- a/serving/numaflow-models/src/models/kafka_sink.rs +++ b/serving/numaflow-models/src/models/kafka_sink.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct KafkaSink { #[serde(rename = "brokers", skip_serializing_if = "Option::is_none")] @@ -33,3 +36,5 @@ impl KafkaSink { } } } + + diff --git a/serving/numaflow-models/src/models/kafka_source.rs b/serving/numaflow-models/src/models/kafka_source.rs index b0771d8e33..078eeee69a 100644 --- a/serving/numaflow-models/src/models/kafka_source.rs +++ b/serving/numaflow-models/src/models/kafka_source.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct KafkaSource { #[serde(rename = "brokers", skip_serializing_if = "Option::is_none")] @@ -36,3 +39,5 @@ impl KafkaSource { } } } + + diff --git a/serving/numaflow-models/src/models/lifecycle.rs b/serving/numaflow-models/src/models/lifecycle.rs index 75065cf816..3c1a478171 100644 --- a/serving/numaflow-models/src/models/lifecycle.rs +++ b/serving/numaflow-models/src/models/lifecycle.rs @@ -4,26 +4,23 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Lifecycle { /// DeleteGracePeriodSeconds used to delete pipeline gracefully - #[serde( - rename = "deleteGracePeriodSeconds", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "deleteGracePeriodSeconds", skip_serializing_if = "Option::is_none")] pub delete_grace_period_seconds: Option, /// DesiredPhase used to bring the pipeline from current phase to desired phase #[serde(rename = "desiredPhase", skip_serializing_if = "Option::is_none")] pub desired_phase: Option, /// PauseGracePeriodSeconds used to pause pipeline gracefully - #[serde( - rename = "pauseGracePeriodSeconds", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "pauseGracePeriodSeconds", skip_serializing_if = "Option::is_none")] pub pause_grace_period_seconds: Option, } @@ -36,3 +33,5 @@ impl Lifecycle { } } } + + diff --git a/serving/numaflow-models/src/models/log.rs b/serving/numaflow-models/src/models/log.rs index c1452a70ff..141bb39e94 100644 --- a/serving/numaflow-models/src/models/log.rs +++ b/serving/numaflow-models/src/models/log.rs @@ -4,15 +4,22 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] -pub struct Log {} +pub struct Log { +} impl Log { pub fn new() -> Log { - Log {} + Log { + } } } + + diff --git a/serving/numaflow-models/src/models/metadata.rs b/serving/numaflow-models/src/models/metadata.rs index 56c214a01e..13c95358f6 100644 --- a/serving/numaflow-models/src/models/metadata.rs +++ b/serving/numaflow-models/src/models/metadata.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Metadata { #[serde(rename = "annotations", skip_serializing_if = "Option::is_none")] @@ -24,3 +27,5 @@ impl Metadata { } } } + + diff --git a/serving/numaflow-models/src/models/native_redis.rs b/serving/numaflow-models/src/models/native_redis.rs index a22fb37436..c538f13b2a 100644 --- a/serving/numaflow-models/src/models/native_redis.rs +++ b/serving/numaflow-models/src/models/native_redis.rs @@ -4,19 +4,19 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct NativeRedis { #[serde(rename = "affinity", skip_serializing_if = "Option::is_none")] pub affinity: Option, /// AutomountServiceAccountToken indicates whether a service account token should be automatically mounted. - #[serde( - rename = "automountServiceAccountToken", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "automountServiceAccountToken", skip_serializing_if = "Option::is_none")] pub automount_service_account_token: Option, #[serde(rename = "dnsConfig", skip_serializing_if = "Option::is_none")] pub dns_config: Option, @@ -26,17 +26,11 @@ pub struct NativeRedis { /// ImagePullSecrets is an optional list of references to secrets in the same namespace to use for pulling any of the images used by this PodSpec. If specified, these secrets will be passed to individual puller implementations for them to use. For example, in the case of docker, only DockerConfig type secrets are honored. More info: https://kubernetes.io/docs/concepts/containers/images#specifying-imagepullsecrets-on-a-pod #[serde(rename = "imagePullSecrets", skip_serializing_if = "Option::is_none")] pub image_pull_secrets: Option>, - #[serde( - rename = "initContainerTemplate", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "initContainerTemplate", skip_serializing_if = "Option::is_none")] pub init_container_template: Option>, #[serde(rename = "metadata", skip_serializing_if = "Option::is_none")] pub metadata: Option>, - #[serde( - rename = "metricsContainerTemplate", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "metricsContainerTemplate", skip_serializing_if = "Option::is_none")] pub metrics_container_template: Option>, /// NodeSelector is a selector which must be true for the pod to fit on a node. Selector which must match a node's labels for the pod to be scheduled on that node. More info: https://kubernetes.io/docs/concepts/configuration/assign-pod-node/ #[serde(rename = "nodeSelector", skip_serializing_if = "Option::is_none")] @@ -49,10 +43,7 @@ pub struct NativeRedis { /// If specified, indicates the Redis pod's priority. \"system-node-critical\" and \"system-cluster-critical\" are two special keywords which indicate the highest priorities with the former being the highest priority. Any other name must be defined by creating a PriorityClass object with that name. If not specified, the pod priority will be default or zero if there is no default. More info: https://kubernetes.io/docs/concepts/configuration/pod-priority-preemption/ #[serde(rename = "priorityClassName", skip_serializing_if = "Option::is_none")] pub priority_class_name: Option, - #[serde( - rename = "redisContainerTemplate", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "redisContainerTemplate", skip_serializing_if = "Option::is_none")] pub redis_container_template: Option>, /// Redis StatefulSet size #[serde(rename = "replicas", skip_serializing_if = "Option::is_none")] @@ -62,10 +53,7 @@ pub struct NativeRedis { pub runtime_class_name: Option, #[serde(rename = "securityContext", skip_serializing_if = "Option::is_none")] pub security_context: Option, - #[serde( - rename = "sentinelContainerTemplate", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "sentinelContainerTemplate", skip_serializing_if = "Option::is_none")] pub sentinel_container_template: Option>, /// ServiceAccountName applied to the pod #[serde(rename = "serviceAccountName", skip_serializing_if = "Option::is_none")] @@ -107,3 +95,5 @@ impl NativeRedis { } } } + + diff --git a/serving/numaflow-models/src/models/nats_auth.rs b/serving/numaflow-models/src/models/nats_auth.rs index 7b085650f9..8c5c977109 100644 --- a/serving/numaflow-models/src/models/nats_auth.rs +++ b/serving/numaflow-models/src/models/nats_auth.rs @@ -4,12 +4,14 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// NatsAuth : NatsAuth defines how to authenticate the nats access + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct NatsAuth { #[serde(rename = "basic", skip_serializing_if = "Option::is_none")] @@ -30,3 +32,5 @@ impl NatsAuth { } } } + + diff --git a/serving/numaflow-models/src/models/nats_source.rs b/serving/numaflow-models/src/models/nats_source.rs index 666fcbb884..d60f175c23 100644 --- a/serving/numaflow-models/src/models/nats_source.rs +++ b/serving/numaflow-models/src/models/nats_source.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct NatsSource { #[serde(rename = "auth", skip_serializing_if = "Option::is_none")] @@ -36,3 +39,5 @@ impl NatsSource { } } } + + diff --git a/serving/numaflow-models/src/models/no_store.rs b/serving/numaflow-models/src/models/no_store.rs index f91247af2f..187e93d96b 100644 --- a/serving/numaflow-models/src/models/no_store.rs +++ b/serving/numaflow-models/src/models/no_store.rs @@ -4,18 +4,24 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// NoStore : NoStore means there will be no persistence storage and there will be data loss during pod restarts. Use this option only if you do not care about correctness (e.g., approx statistics pipeline like sampling rate, etc.). + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] -pub struct NoStore {} +pub struct NoStore { +} impl NoStore { /// NoStore means there will be no persistence storage and there will be data loss during pod restarts. Use this option only if you do not care about correctness (e.g., approx statistics pipeline like sampling rate, etc.). pub fn new() -> NoStore { - NoStore {} + NoStore { + } } } + + diff --git a/serving/numaflow-models/src/models/pbq_storage.rs b/serving/numaflow-models/src/models/pbq_storage.rs index c01ffe9eba..ea9cb9514e 100644 --- a/serving/numaflow-models/src/models/pbq_storage.rs +++ b/serving/numaflow-models/src/models/pbq_storage.rs @@ -4,22 +4,21 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// PbqStorage : PBQStorage defines the persistence configuration for a vertex. + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct PbqStorage { #[serde(rename = "emptyDir", skip_serializing_if = "Option::is_none")] pub empty_dir: Option, #[serde(rename = "no_store", skip_serializing_if = "Option::is_none")] pub no_store: Option>, - #[serde( - rename = "persistentVolumeClaim", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "persistentVolumeClaim", skip_serializing_if = "Option::is_none")] pub persistent_volume_claim: Option>, } @@ -33,3 +32,5 @@ impl PbqStorage { } } } + + diff --git a/serving/numaflow-models/src/models/persistence_strategy.rs b/serving/numaflow-models/src/models/persistence_strategy.rs index 3541e83b91..49351a146d 100644 --- a/serving/numaflow-models/src/models/persistence_strategy.rs +++ b/serving/numaflow-models/src/models/persistence_strategy.rs @@ -4,12 +4,14 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// PersistenceStrategy : PersistenceStrategy defines the strategy of persistence + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct PersistenceStrategy { /// Available access modes such as ReadWriteOnce, ReadWriteMany https://kubernetes.io/docs/concepts/storage/persistent-volumes/#access-modes @@ -32,3 +34,5 @@ impl PersistenceStrategy { } } } + + diff --git a/serving/numaflow-models/src/models/pipeline.rs b/serving/numaflow-models/src/models/pipeline.rs index 52db105f24..e6d7054b36 100644 --- a/serving/numaflow-models/src/models/pipeline.rs +++ b/serving/numaflow-models/src/models/pipeline.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Pipeline { /// APIVersion defines the versioned schema of this representation of an object. Servers should convert recognized schemas to the latest internal value, and may reject unrecognized values. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#resources @@ -35,3 +38,5 @@ impl Pipeline { } } } + + diff --git a/serving/numaflow-models/src/models/pipeline_limits.rs b/serving/numaflow-models/src/models/pipeline_limits.rs index c4a158e6ae..3e3a2e58f1 100644 --- a/serving/numaflow-models/src/models/pipeline_limits.rs +++ b/serving/numaflow-models/src/models/pipeline_limits.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct PipelineLimits { /// BufferMaxLength is used to define the max length of a buffer. Only applies to UDF and Source vertices as only they do buffer write. It can be overridden by the settings in vertex limits. @@ -33,3 +36,5 @@ impl PipelineLimits { } } } + + diff --git a/serving/numaflow-models/src/models/pipeline_list.rs b/serving/numaflow-models/src/models/pipeline_list.rs index 7fc027c1a7..a159018e5e 100644 --- a/serving/numaflow-models/src/models/pipeline_list.rs +++ b/serving/numaflow-models/src/models/pipeline_list.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct PipelineList { /// APIVersion defines the versioned schema of this representation of an object. Servers should convert recognized schemas to the latest internal value, and may reject unrecognized values. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#resources @@ -32,3 +35,5 @@ impl PipelineList { } } } + + diff --git a/serving/numaflow-models/src/models/pipeline_spec.rs b/serving/numaflow-models/src/models/pipeline_spec.rs index c33cdb1f80..66c29acf08 100644 --- a/serving/numaflow-models/src/models/pipeline_spec.rs +++ b/serving/numaflow-models/src/models/pipeline_spec.rs @@ -4,19 +4,19 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct PipelineSpec { /// Edges define the relationships between vertices #[serde(rename = "edges", skip_serializing_if = "Option::is_none")] pub edges: Option>, - #[serde( - rename = "interStepBufferServiceName", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "interStepBufferServiceName", skip_serializing_if = "Option::is_none")] pub inter_step_buffer_service_name: Option, #[serde(rename = "lifecycle", skip_serializing_if = "Option::is_none")] pub lifecycle: Option>, @@ -47,3 +47,5 @@ impl PipelineSpec { } } } + + diff --git a/serving/numaflow-models/src/models/pipeline_status.rs b/serving/numaflow-models/src/models/pipeline_status.rs index 8756844a7c..64e8934fab 100644 --- a/serving/numaflow-models/src/models/pipeline_status.rs +++ b/serving/numaflow-models/src/models/pipeline_status.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct PipelineStatus { /// Conditions are the latest available observations of a resource's current state. @@ -52,3 +55,5 @@ impl PipelineStatus { } } } + + diff --git a/serving/numaflow-models/src/models/redis_buffer_service.rs b/serving/numaflow-models/src/models/redis_buffer_service.rs index 5dc527b391..10994749ff 100644 --- a/serving/numaflow-models/src/models/redis_buffer_service.rs +++ b/serving/numaflow-models/src/models/redis_buffer_service.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct RedisBufferService { #[serde(rename = "external", skip_serializing_if = "Option::is_none")] @@ -24,3 +27,5 @@ impl RedisBufferService { } } } + + diff --git a/serving/numaflow-models/src/models/redis_config.rs b/serving/numaflow-models/src/models/redis_config.rs index cf563ab9ec..9c6c5180e6 100644 --- a/serving/numaflow-models/src/models/redis_config.rs +++ b/serving/numaflow-models/src/models/redis_config.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct RedisConfig { /// Only required when Sentinel is used @@ -40,3 +43,5 @@ impl RedisConfig { } } } + + diff --git a/serving/numaflow-models/src/models/redis_settings.rs b/serving/numaflow-models/src/models/redis_settings.rs index f5b4b4c2b7..3b8e813a28 100644 --- a/serving/numaflow-models/src/models/redis_settings.rs +++ b/serving/numaflow-models/src/models/redis_settings.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct RedisSettings { /// Special settings for Redis master node, will override the global settings from controller config @@ -34,3 +37,5 @@ impl RedisSettings { } } } + + diff --git a/serving/numaflow-models/src/models/sasl.rs b/serving/numaflow-models/src/models/sasl.rs index 2f56e8cce0..163eb8a1b0 100644 --- a/serving/numaflow-models/src/models/sasl.rs +++ b/serving/numaflow-models/src/models/sasl.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Sasl { #[serde(rename = "gssapi", skip_serializing_if = "Option::is_none")] @@ -34,3 +37,5 @@ impl Sasl { } } } + + diff --git a/serving/numaflow-models/src/models/sasl_plain.rs b/serving/numaflow-models/src/models/sasl_plain.rs index 7719720dd3..085581f975 100644 --- a/serving/numaflow-models/src/models/sasl_plain.rs +++ b/serving/numaflow-models/src/models/sasl_plain.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct SaslPlain { #[serde(rename = "handshake")] @@ -19,10 +22,7 @@ pub struct SaslPlain { } impl SaslPlain { - pub fn new( - handshake: bool, - user_secret: k8s_openapi::api::core::v1::SecretKeySelector, - ) -> SaslPlain { + pub fn new(handshake: bool, user_secret: k8s_openapi::api::core::v1::SecretKeySelector) -> SaslPlain { SaslPlain { handshake, password_secret: None, @@ -30,3 +30,5 @@ impl SaslPlain { } } } + + diff --git a/serving/numaflow-models/src/models/scale.rs b/serving/numaflow-models/src/models/scale.rs index 03673cca73..7510676ce1 100644 --- a/serving/numaflow-models/src/models/scale.rs +++ b/serving/numaflow-models/src/models/scale.rs @@ -4,12 +4,14 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// Scale : Scale defines the parameters for autoscaling. + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Scale { /// Deprecated: Use scaleUpCooldownSeconds and scaleDownCooldownSeconds instead. Cooldown seconds after a scaling operation before another one. @@ -31,34 +33,19 @@ pub struct Scale { #[serde(rename = "replicasPerScale", skip_serializing_if = "Option::is_none")] pub replicas_per_scale: Option, /// ScaleDownCooldownSeconds defines the cooldown seconds after a scaling operation, before a follow-up scaling down. It defaults to the CooldownSeconds if not set. - #[serde( - rename = "scaleDownCooldownSeconds", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "scaleDownCooldownSeconds", skip_serializing_if = "Option::is_none")] pub scale_down_cooldown_seconds: Option, /// ScaleUpCooldownSeconds defines the cooldown seconds after a scaling operation, before a follow-up scaling up. It defaults to the CooldownSeconds if not set. - #[serde( - rename = "scaleUpCooldownSeconds", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "scaleUpCooldownSeconds", skip_serializing_if = "Option::is_none")] pub scale_up_cooldown_seconds: Option, /// TargetBufferAvailability is used to define the target percentage of the buffer availability. A valid and meaningful value should be less than the BufferUsageLimit defined in the Edge spec (or Pipeline spec), for example, 50. It only applies to UDF and Sink vertices because only they have buffers to read. - #[serde( - rename = "targetBufferAvailability", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "targetBufferAvailability", skip_serializing_if = "Option::is_none")] pub target_buffer_availability: Option, /// TargetProcessingSeconds is used to tune the aggressiveness of autoscaling for source vertices, it measures how fast you want the vertex to process all the pending messages. Typically increasing the value, which leads to lower processing rate, thus less replicas. It's only effective for source vertices. - #[serde( - rename = "targetProcessingSeconds", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "targetProcessingSeconds", skip_serializing_if = "Option::is_none")] pub target_processing_seconds: Option, /// After scaling down the source vertex to 0, sleep how many seconds before scaling the source vertex back up to peek. - #[serde( - rename = "zeroReplicaSleepSeconds", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "zeroReplicaSleepSeconds", skip_serializing_if = "Option::is_none")] pub zero_replica_sleep_seconds: Option, } @@ -80,3 +67,5 @@ impl Scale { } } } + + diff --git a/serving/numaflow-models/src/models/serving_source.rs b/serving/numaflow-models/src/models/serving_source.rs index 25703a7031..e1e6c444cf 100644 --- a/serving/numaflow-models/src/models/serving_source.rs +++ b/serving/numaflow-models/src/models/serving_source.rs @@ -4,12 +4,14 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// ServingSource : ServingSource is the HTTP endpoint for Numaflow. + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct ServingSource { #[serde(rename = "auth", skip_serializing_if = "Option::is_none")] @@ -35,3 +37,5 @@ impl ServingSource { } } } + + diff --git a/serving/numaflow-models/src/models/serving_store.rs b/serving/numaflow-models/src/models/serving_store.rs index 7a3086aea9..44998b5e73 100644 --- a/serving/numaflow-models/src/models/serving_store.rs +++ b/serving/numaflow-models/src/models/serving_store.rs @@ -4,12 +4,14 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// ServingStore : ServingStore to track and store data and metadata for tracking and serving. + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct ServingStore { #[serde(rename = "ttl", skip_serializing_if = "Option::is_none")] @@ -22,6 +24,11 @@ pub struct ServingStore { impl ServingStore { /// ServingStore to track and store data and metadata for tracking and serving. pub fn new(url: String) -> ServingStore { - ServingStore { ttl: None, url } + ServingStore { + ttl: None, + url, + } } } + + diff --git a/serving/numaflow-models/src/models/session_window.rs b/serving/numaflow-models/src/models/session_window.rs index 551f164cde..67f1f440bc 100644 --- a/serving/numaflow-models/src/models/session_window.rs +++ b/serving/numaflow-models/src/models/session_window.rs @@ -4,12 +4,14 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// SessionWindow : SessionWindow describes a session window + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct SessionWindow { #[serde(rename = "timeout", skip_serializing_if = "Option::is_none")] @@ -19,6 +21,10 @@ pub struct SessionWindow { impl SessionWindow { /// SessionWindow describes a session window pub fn new() -> SessionWindow { - SessionWindow { timeout: None } + SessionWindow { + timeout: None, + } } } + + diff --git a/serving/numaflow-models/src/models/side_input.rs b/serving/numaflow-models/src/models/side_input.rs index 275af684af..9cb2040749 100644 --- a/serving/numaflow-models/src/models/side_input.rs +++ b/serving/numaflow-models/src/models/side_input.rs @@ -4,12 +4,14 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// SideInput : SideInput defines information of a Side Input + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct SideInput { #[serde(rename = "container")] @@ -24,11 +26,7 @@ pub struct SideInput { impl SideInput { /// SideInput defines information of a Side Input - pub fn new( - container: crate::models::Container, - name: String, - trigger: crate::models::SideInputTrigger, - ) -> SideInput { + pub fn new(container: crate::models::Container, name: String, trigger: crate::models::SideInputTrigger) -> SideInput { SideInput { container: Box::new(container), name, @@ -37,3 +35,5 @@ impl SideInput { } } } + + diff --git a/serving/numaflow-models/src/models/side_input_trigger.rs b/serving/numaflow-models/src/models/side_input_trigger.rs index 497f5461b3..f92cff7cd9 100644 --- a/serving/numaflow-models/src/models/side_input_trigger.rs +++ b/serving/numaflow-models/src/models/side_input_trigger.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct SideInputTrigger { /// The schedule to trigger the retrievement of the side input data. It supports cron format, for example, \"0 30 * * * *\". Or interval based format, such as \"@hourly\", \"@every 1h30m\", etc. @@ -25,3 +28,5 @@ impl SideInputTrigger { } } } + + diff --git a/serving/numaflow-models/src/models/side_inputs_manager_template.rs b/serving/numaflow-models/src/models/side_inputs_manager_template.rs index 82f7afe5d1..d9a8e6a4fc 100644 --- a/serving/numaflow-models/src/models/side_inputs_manager_template.rs +++ b/serving/numaflow-models/src/models/side_inputs_manager_template.rs @@ -4,19 +4,19 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct SideInputsManagerTemplate { #[serde(rename = "affinity", skip_serializing_if = "Option::is_none")] pub affinity: Option, /// AutomountServiceAccountToken indicates whether a service account token should be automatically mounted. - #[serde( - rename = "automountServiceAccountToken", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "automountServiceAccountToken", skip_serializing_if = "Option::is_none")] pub automount_service_account_token: Option, #[serde(rename = "containerTemplate", skip_serializing_if = "Option::is_none")] pub container_template: Option>, @@ -28,10 +28,7 @@ pub struct SideInputsManagerTemplate { /// ImagePullSecrets is an optional list of references to secrets in the same namespace to use for pulling any of the images used by this PodSpec. If specified, these secrets will be passed to individual puller implementations for them to use. For example, in the case of docker, only DockerConfig type secrets are honored. More info: https://kubernetes.io/docs/concepts/containers/images#specifying-imagepullsecrets-on-a-pod #[serde(rename = "imagePullSecrets", skip_serializing_if = "Option::is_none")] pub image_pull_secrets: Option>, - #[serde( - rename = "initContainerTemplate", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "initContainerTemplate", skip_serializing_if = "Option::is_none")] pub init_container_template: Option>, #[serde(rename = "metadata", skip_serializing_if = "Option::is_none")] pub metadata: Option>, @@ -78,3 +75,5 @@ impl SideInputsManagerTemplate { } } } + + diff --git a/serving/numaflow-models/src/models/sink.rs b/serving/numaflow-models/src/models/sink.rs index 6cd57bb2ec..ebed76c690 100644 --- a/serving/numaflow-models/src/models/sink.rs +++ b/serving/numaflow-models/src/models/sink.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Sink { #[serde(rename = "blackhole", skip_serializing_if = "Option::is_none")] @@ -33,3 +36,5 @@ impl Sink { } } } + + diff --git a/serving/numaflow-models/src/models/sliding_window.rs b/serving/numaflow-models/src/models/sliding_window.rs index 4d2f9a06c7..10530f04b3 100644 --- a/serving/numaflow-models/src/models/sliding_window.rs +++ b/serving/numaflow-models/src/models/sliding_window.rs @@ -4,12 +4,14 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// SlidingWindow : SlidingWindow describes a sliding window + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct SlidingWindow { #[serde(rename = "length", skip_serializing_if = "Option::is_none")] @@ -31,3 +33,5 @@ impl SlidingWindow { } } } + + diff --git a/serving/numaflow-models/src/models/source.rs b/serving/numaflow-models/src/models/source.rs index b7331b4942..fadfe1a402 100644 --- a/serving/numaflow-models/src/models/source.rs +++ b/serving/numaflow-models/src/models/source.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Source { #[serde(rename = "generator", skip_serializing_if = "Option::is_none")] @@ -42,3 +45,5 @@ impl Source { } } } + + diff --git a/serving/numaflow-models/src/models/status.rs b/serving/numaflow-models/src/models/status.rs index cbfa91bf3d..accf9e49d2 100644 --- a/serving/numaflow-models/src/models/status.rs +++ b/serving/numaflow-models/src/models/status.rs @@ -4,12 +4,14 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// Status : Status is a common structure which can be used for Status field. + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Status { /// Conditions are the latest available observations of a resource's current state. @@ -20,6 +22,10 @@ pub struct Status { impl Status { /// Status is a common structure which can be used for Status field. pub fn new() -> Status { - Status { conditions: None } + Status { + conditions: None, + } } } + + diff --git a/serving/numaflow-models/src/models/tag_conditions.rs b/serving/numaflow-models/src/models/tag_conditions.rs index 60d4b3ca58..42b6b94e0a 100644 --- a/serving/numaflow-models/src/models/tag_conditions.rs +++ b/serving/numaflow-models/src/models/tag_conditions.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct TagConditions { /// Operator specifies the type of operation that should be used for conditional forwarding value could be \"and\", \"or\", \"not\" @@ -26,3 +29,5 @@ impl TagConditions { } } } + + diff --git a/serving/numaflow-models/src/models/templates.rs b/serving/numaflow-models/src/models/templates.rs index 98876b49a8..382019d5b5 100644 --- a/serving/numaflow-models/src/models/templates.rs +++ b/serving/numaflow-models/src/models/templates.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Templates { #[serde(rename = "daemon", skip_serializing_if = "Option::is_none")] @@ -30,3 +33,5 @@ impl Templates { } } } + + diff --git a/serving/numaflow-models/src/models/tls.rs b/serving/numaflow-models/src/models/tls.rs index b0c896e8ea..b140b68531 100644 --- a/serving/numaflow-models/src/models/tls.rs +++ b/serving/numaflow-models/src/models/tls.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Tls { #[serde(rename = "caCertSecret", skip_serializing_if = "Option::is_none")] @@ -30,3 +33,5 @@ impl Tls { } } } + + diff --git a/serving/numaflow-models/src/models/transformer.rs b/serving/numaflow-models/src/models/transformer.rs index a17255f010..5540a6b6f9 100644 --- a/serving/numaflow-models/src/models/transformer.rs +++ b/serving/numaflow-models/src/models/transformer.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Transformer { #[serde(rename = "args", skip_serializing_if = "Option::is_none")] @@ -27,3 +30,5 @@ impl Transformer { } } } + + diff --git a/serving/numaflow-models/src/models/ud_sink.rs b/serving/numaflow-models/src/models/ud_sink.rs index f247400f86..f39a053de5 100644 --- a/serving/numaflow-models/src/models/ud_sink.rs +++ b/serving/numaflow-models/src/models/ud_sink.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct UdSink { #[serde(rename = "container")] @@ -21,3 +24,5 @@ impl UdSink { } } } + + diff --git a/serving/numaflow-models/src/models/ud_source.rs b/serving/numaflow-models/src/models/ud_source.rs index 65986169a6..2242908feb 100644 --- a/serving/numaflow-models/src/models/ud_source.rs +++ b/serving/numaflow-models/src/models/ud_source.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct UdSource { #[serde(rename = "container")] @@ -21,3 +24,5 @@ impl UdSource { } } } + + diff --git a/serving/numaflow-models/src/models/ud_transformer.rs b/serving/numaflow-models/src/models/ud_transformer.rs index ff698fab80..d397305f2b 100644 --- a/serving/numaflow-models/src/models/ud_transformer.rs +++ b/serving/numaflow-models/src/models/ud_transformer.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct UdTransformer { #[serde(rename = "builtin", skip_serializing_if = "Option::is_none")] @@ -24,3 +27,5 @@ impl UdTransformer { } } } + + diff --git a/serving/numaflow-models/src/models/udf.rs b/serving/numaflow-models/src/models/udf.rs index cd1f6b520f..cfeae12666 100644 --- a/serving/numaflow-models/src/models/udf.rs +++ b/serving/numaflow-models/src/models/udf.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Udf { #[serde(rename = "builtin", skip_serializing_if = "Option::is_none")] @@ -27,3 +30,5 @@ impl Udf { } } } + + diff --git a/serving/numaflow-models/src/models/vertex.rs b/serving/numaflow-models/src/models/vertex.rs index 1f9cc0e617..d23f3023c5 100644 --- a/serving/numaflow-models/src/models/vertex.rs +++ b/serving/numaflow-models/src/models/vertex.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Vertex { /// APIVersion defines the versioned schema of this representation of an object. Servers should convert recognized schemas to the latest internal value, and may reject unrecognized values. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#resources @@ -35,3 +38,5 @@ impl Vertex { } } } + + diff --git a/serving/numaflow-models/src/models/vertex_instance.rs b/serving/numaflow-models/src/models/vertex_instance.rs index e7ba664e60..6c3298c345 100644 --- a/serving/numaflow-models/src/models/vertex_instance.rs +++ b/serving/numaflow-models/src/models/vertex_instance.rs @@ -4,12 +4,14 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// VertexInstance : VertexInstance is a wrapper of a vertex instance, which contains the vertex spec and the instance information such as hostname and replica index. + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct VertexInstance { #[serde(rename = "hostname", skip_serializing_if = "Option::is_none")] @@ -30,3 +32,5 @@ impl VertexInstance { } } } + + diff --git a/serving/numaflow-models/src/models/vertex_limits.rs b/serving/numaflow-models/src/models/vertex_limits.rs index 6f201e8de8..713c0faa41 100644 --- a/serving/numaflow-models/src/models/vertex_limits.rs +++ b/serving/numaflow-models/src/models/vertex_limits.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct VertexLimits { /// BufferMaxLength is used to define the max length of a buffer. It overrides the settings from pipeline limits. @@ -33,3 +36,5 @@ impl VertexLimits { } } } + + diff --git a/serving/numaflow-models/src/models/vertex_list.rs b/serving/numaflow-models/src/models/vertex_list.rs index 68967d8aa5..a350a3b983 100644 --- a/serving/numaflow-models/src/models/vertex_list.rs +++ b/serving/numaflow-models/src/models/vertex_list.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct VertexList { /// APIVersion defines the versioned schema of this representation of an object. Servers should convert recognized schemas to the latest internal value, and may reject unrecognized values. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#resources @@ -32,3 +35,5 @@ impl VertexList { } } } + + diff --git a/serving/numaflow-models/src/models/vertex_spec.rs b/serving/numaflow-models/src/models/vertex_spec.rs index a3fafd4882..f96eb218ed 100644 --- a/serving/numaflow-models/src/models/vertex_spec.rs +++ b/serving/numaflow-models/src/models/vertex_spec.rs @@ -4,19 +4,19 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct VertexSpec { #[serde(rename = "affinity", skip_serializing_if = "Option::is_none")] pub affinity: Option, /// AutomountServiceAccountToken indicates whether a service account token should be automatically mounted. - #[serde( - rename = "automountServiceAccountToken", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "automountServiceAccountToken", skip_serializing_if = "Option::is_none")] pub automount_service_account_token: Option, #[serde(rename = "containerTemplate", skip_serializing_if = "Option::is_none")] pub container_template: Option>, @@ -30,18 +30,12 @@ pub struct VertexSpec { /// ImagePullSecrets is an optional list of references to secrets in the same namespace to use for pulling any of the images used by this PodSpec. If specified, these secrets will be passed to individual puller implementations for them to use. For example, in the case of docker, only DockerConfig type secrets are honored. More info: https://kubernetes.io/docs/concepts/containers/images#specifying-imagepullsecrets-on-a-pod #[serde(rename = "imagePullSecrets", skip_serializing_if = "Option::is_none")] pub image_pull_secrets: Option>, - #[serde( - rename = "initContainerTemplate", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "initContainerTemplate", skip_serializing_if = "Option::is_none")] pub init_container_template: Option>, /// List of customized init containers belonging to the pod. More info: https://kubernetes.io/docs/concepts/workloads/pods/init-containers/ #[serde(rename = "initContainers", skip_serializing_if = "Option::is_none")] pub init_containers: Option>, - #[serde( - rename = "interStepBufferServiceName", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "interStepBufferServiceName", skip_serializing_if = "Option::is_none")] pub inter_step_buffer_service_name: Option, #[serde(rename = "limits", skip_serializing_if = "Option::is_none")] pub limits: Option>, @@ -78,10 +72,7 @@ pub struct VertexSpec { /// Names of the side inputs used in this vertex. #[serde(rename = "sideInputs", skip_serializing_if = "Option::is_none")] pub side_inputs: Option>, - #[serde( - rename = "sideInputsContainerTemplate", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "sideInputsContainerTemplate", skip_serializing_if = "Option::is_none")] pub side_inputs_container_template: Option>, /// List of customized sidecar containers belonging to the pod. #[serde(rename = "sidecars", skip_serializing_if = "Option::is_none")] @@ -142,3 +133,5 @@ impl VertexSpec { } } } + + diff --git a/serving/numaflow-models/src/models/vertex_status.rs b/serving/numaflow-models/src/models/vertex_status.rs index 6bcea1f106..3e0377a795 100644 --- a/serving/numaflow-models/src/models/vertex_status.rs +++ b/serving/numaflow-models/src/models/vertex_status.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct VertexStatus { /// Conditions are the latest available observations of a resource's current state. @@ -43,3 +46,5 @@ impl VertexStatus { } } } + + diff --git a/serving/numaflow-models/src/models/vertex_template.rs b/serving/numaflow-models/src/models/vertex_template.rs index 34aae02b0e..6bd04e9cad 100644 --- a/serving/numaflow-models/src/models/vertex_template.rs +++ b/serving/numaflow-models/src/models/vertex_template.rs @@ -4,19 +4,19 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct VertexTemplate { #[serde(rename = "affinity", skip_serializing_if = "Option::is_none")] pub affinity: Option, /// AutomountServiceAccountToken indicates whether a service account token should be automatically mounted. - #[serde( - rename = "automountServiceAccountToken", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "automountServiceAccountToken", skip_serializing_if = "Option::is_none")] pub automount_service_account_token: Option, #[serde(rename = "containerTemplate", skip_serializing_if = "Option::is_none")] pub container_template: Option>, @@ -28,10 +28,7 @@ pub struct VertexTemplate { /// ImagePullSecrets is an optional list of references to secrets in the same namespace to use for pulling any of the images used by this PodSpec. If specified, these secrets will be passed to individual puller implementations for them to use. For example, in the case of docker, only DockerConfig type secrets are honored. More info: https://kubernetes.io/docs/concepts/containers/images#specifying-imagepullsecrets-on-a-pod #[serde(rename = "imagePullSecrets", skip_serializing_if = "Option::is_none")] pub image_pull_secrets: Option>, - #[serde( - rename = "initContainerTemplate", - skip_serializing_if = "Option::is_none" - )] + #[serde(rename = "initContainerTemplate", skip_serializing_if = "Option::is_none")] pub init_container_template: Option>, #[serde(rename = "metadata", skip_serializing_if = "Option::is_none")] pub metadata: Option>, @@ -78,3 +75,5 @@ impl VertexTemplate { } } } + + diff --git a/serving/numaflow-models/src/models/watermark.rs b/serving/numaflow-models/src/models/watermark.rs index b54bf2a74b..e32de79335 100644 --- a/serving/numaflow-models/src/models/watermark.rs +++ b/serving/numaflow-models/src/models/watermark.rs @@ -4,10 +4,13 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ + + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Watermark { /// Disabled toggles the watermark propagation, defaults to false. @@ -28,3 +31,5 @@ impl Watermark { } } } + + diff --git a/serving/numaflow-models/src/models/window.rs b/serving/numaflow-models/src/models/window.rs index 28d4d42243..3f72f7be34 100644 --- a/serving/numaflow-models/src/models/window.rs +++ b/serving/numaflow-models/src/models/window.rs @@ -4,12 +4,14 @@ * No description provided (generated by Openapi Generator https://github.com/openapitools/openapi-generator) * * The version of the OpenAPI document: latest - * + * * Generated by: https://openapi-generator.tech */ /// Window : Window describes windowing strategy + + #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct Window { #[serde(rename = "fixed", skip_serializing_if = "Option::is_none")] @@ -30,3 +32,5 @@ impl Window { } } } + + From 8121ab0fa48c2999841ef9defabbc41efff4a12e Mon Sep 17 00:00:00 2001 From: Vigith Maurice Date: Tue, 6 Aug 2024 13:46:11 -0700 Subject: [PATCH 05/15] chore: refactor to use From trait + TODOs Signed-off-by: Vigith Maurice --- serving/source-sink/Cargo.toml | 2 +- serving/source-sink/src/error.rs | 5 ++- serving/source-sink/src/forwarder.rs | 13 +++++- serving/source-sink/src/lib.rs | 34 +++++++-------- serving/source-sink/src/main.rs | 3 +- serving/source-sink/src/message.rs | 57 ++++++++++++++++++++++++++ serving/source-sink/src/sink.rs | 15 ++----- serving/source-sink/src/source.rs | 36 ++++++---------- serving/source-sink/src/transformer.rs | 18 ++++---- 9 files changed, 116 insertions(+), 67 deletions(-) diff --git a/serving/source-sink/Cargo.toml b/serving/source-sink/Cargo.toml index 886364cdf1..0ca88308bd 100644 --- a/serving/source-sink/Cargo.toml +++ b/serving/source-sink/Cargo.toml @@ -19,7 +19,7 @@ base64 = "0.22.1" metrics = { version = "0.23.0", default-features = false } metrics-exporter-prometheus = { version = "0.15.3", default-features = false } log = "0.4.22" -tracing-subscriber = "0.3.18" +tracing-subscriber = { version = "0.3.18", features = ["env-filter"] } hyper-util = "0.1.6" tower = "0.4.13" uuid = { version = "1.10.0", features = ["v4"] } diff --git a/serving/source-sink/src/error.rs b/serving/source-sink/src/error.rs index f3e2e3b7d9..84aec64bcc 100644 --- a/serving/source-sink/src/error.rs +++ b/serving/source-sink/src/error.rs @@ -21,10 +21,13 @@ pub enum Error { #[error("Connection Error - {0}")] ConnectionError(String), + + #[error("gRPC Error - {0}")] + GRPCError(String), } impl From for Error { fn from(status: tonic::Status) -> Self { - Error::SourceError(status.to_string()) + Error::GRPCError(status.to_string()) } } diff --git a/serving/source-sink/src/forwarder.rs b/serving/source-sink/src/forwarder.rs index 3d814549f4..d5716ae8df 100644 --- a/serving/source-sink/src/forwarder.rs +++ b/serving/source-sink/src/forwarder.rs @@ -8,7 +8,8 @@ use crate::source::SourceClient; use crate::transformer::TransformerClient; /// Forwarder is responsible for reading messages from the source, applying transformation if -/// transformer is present, and writing the messages to the sinK and acknowledging the messages. +/// transformer is present, writing the messages to the sink, and the acknowledging the messages +/// back to the source. pub(crate) struct Forwarder { source_client: SourceClient, sink_client: SinkClient, @@ -37,8 +38,12 @@ impl Forwarder { }) } + /// run starts the forward-a-chunk loop and exits only after a chunk has been forwarded and ack'ed. + /// this means that, in the happy path scenario a block is always completely processed. + /// this function will return on any error and will cause end up in a non-0 exit code. pub(crate) async fn run(&mut self) -> Result<()> { loop { + // two arms, either shutdown or forward-a-chunk tokio::select! { _ = &mut self.shutdown_rx => { info!("Shutdown signal received, stopping forwarder..."); @@ -71,9 +76,15 @@ impl Forwarder { }; // Write messages to the sink + // TODO: should we retry writing? what if the error is transient? + // we could rely on gRPC retries and say that any error that is bubbled up is worthy of non-0 exit. + // we need to confirm this via FMEA tests. self.sink_client.sink_fn(transformed_messages).await?; // Acknowledge the messages + // TODO: should we retry acking? what if the error is transient? + // we could rely on gRPC retries and say that any error that is bubbled up is worthy of non-0 exit. + // we need to confirm this via FMEA tests. self.source_client.ack_fn(offsets).await?; } } diff --git a/serving/source-sink/src/lib.rs b/serving/source-sink/src/lib.rs index 6ecdc0052d..d377aaa243 100644 --- a/serving/source-sink/src/lib.rs +++ b/serving/source-sink/src/lib.rs @@ -6,29 +6,20 @@ use tokio::task::JoinHandle; use tokio::time::sleep; use tracing::info; -use crate::error::Error; use crate::forwarder::Forwarder; use crate::sink::{SinkClient, SinkConfig}; use crate::source::{SourceClient, SourceConfig}; use crate::transformer::{TransformerClient, TransformerConfig}; +pub(crate) use self::error::Result; +pub(crate) use crate::error::Error; + /// SourcerSinker orchestrates data movement from the Source to the Sink via the optional SourceTransformer. -/// The infamous forward-a-chunk executes the following in an infinite loop: +/// The forward-a-chunk executes the following in an infinite loop till a shutdown signal is received: /// - Read X messages from the source /// - Invokes the SourceTransformer concurrently /// - Calls the Sinker to write the batch to the Sink /// - Send Acknowledgement back to the Source - -/// TODO -/// - [ ] integrate with main -/// - [ ] add metrics and metrics-server -/// - [ ] integrate with trace! -/// - [ ] add code comment -/// - [ ] error handling using anyhow -/// - [ ] unit testing >= 85% -/// - [ ] local integration testing -pub use self::error::Result; - pub mod error; pub mod metrics; @@ -48,11 +39,13 @@ pub(crate) mod shared; const TIMEOUT_IN_MS: u32 = 1000; const BATCH_SIZE: u64 = 500; +/// forwards a chunk of data from the source to the sink via an optional transformer. +/// It takes an optional custom_shutdown_rx for shutting down the forwarder, useful for testing. pub async fn run_forwarder( - custom_shutdown_rx: Option>, source_config: SourceConfig, sink_config: SinkConfig, transformer_config: Option, + custom_shutdown_rx: Option>, ) -> Result<()> { wait_for_server_info(&source_config.server_info_file).await?; let mut source_client = SourceClient::connect(source_config).await?; @@ -77,6 +70,7 @@ pub async fn run_forwarder( ) .await?; + // TODO: use builder pattern of options like TIMEOUT, BATCH_SIZE, etc? let mut forwarder = Forwarder::new( source_client, sink_client, @@ -263,20 +257,26 @@ mod tests { }; // wait for the servers to start + // FIXME: we need to have a better way, this is flaky tokio::time::sleep(tokio::time::Duration::from_millis(100)).await; - env::set_var("SOURCE_SOCKET", src_sock_file.to_str().unwrap()); - env::set_var("SINK_SOCKET", sink_sock_file.to_str().unwrap()); + unsafe { + env::set_var("SOURCE_SOCKET", src_sock_file.to_str().unwrap()); + } + unsafe { + env::set_var("SINK_SOCKET", sink_sock_file.to_str().unwrap()); + } let (shutdown_tx, shutdown_rx) = tokio::sync::oneshot::channel(); let forwarder_handle = tokio::spawn(async move { let result = - super::run_forwarder(Some(shutdown_rx), source_config, sink_config, None).await; + super::run_forwarder(source_config, sink_config, None, Some(shutdown_rx)).await; assert!(result.is_ok()); }); // wait for the forwarder to start + // FIXME: we need to have a better way, this is flaky tokio::time::sleep(tokio::time::Duration::from_millis(100)).await; // stop the forwarder diff --git a/serving/source-sink/src/main.rs b/serving/source-sink/src/main.rs index 9b4e21452f..47a86ebf44 100644 --- a/serving/source-sink/src/main.rs +++ b/serving/source-sink/src/main.rs @@ -13,6 +13,7 @@ async fn main() { tracing_subscriber::fmt::init(); // Start the metrics server + // TODO: make the port configurable. let metrics_addr: SocketAddr = "0.0.0.0:9090".parse().expect("Invalid address"); tokio::spawn(async move { if let Err(e) = start_metrics_server(metrics_addr).await { @@ -29,7 +30,7 @@ async fn main() { }; // Run the forwarder - if let Err(e) = run_forwarder(None, source_config, sink_config, transformer_config).await { + if let Err(e) = run_forwarder(source_config, sink_config, transformer_config, None).await { error!("Application error: {:?}", e); } } diff --git a/serving/source-sink/src/message.rs b/serving/source-sink/src/message.rs index bacc2e8b85..09caf0b195 100644 --- a/serving/source-sink/src/message.rs +++ b/serving/source-sink/src/message.rs @@ -1,5 +1,12 @@ use std::collections::HashMap; +use crate::error::Error; +use crate::shared::{prost_timestamp_from_utc, utc_from_timestamp}; +use crate::sink::proto; +use crate::source::proto::read_response; +use crate::transformer::proto::SourceTransformRequest; +use base64::engine::general_purpose::STANDARD as BASE64_STANDARD; +use base64::Engine; use chrono::{DateTime, Utc}; /// A message that is sent from the source to the sink. @@ -25,3 +32,53 @@ pub(crate) struct Offset { /// partition id of the message pub(crate) partition_id: i32, } + +/// Convert the [`Message`] to [`SourceTransformRequest`] +impl From for SourceTransformRequest { + fn from(message: Message) -> Self { + Self { + keys: message.keys, + value: message.value, + event_time: prost_timestamp_from_utc(message.event_time), + watermark: None, + headers: message.headers, + } + } +} + +/// Convert [`read_response::Result`] to [`Message`] +impl TryFrom for Message { + type Error = crate::Error; + + fn try_from(result: read_response::Result) -> std::result::Result { + let source_offset = match result.offset { + Some(o) => Offset { + offset: BASE64_STANDARD.encode(o.offset), + partition_id: o.partition_id, + }, + None => return Err(Error::SourceError("Offset not found".to_string())), + }; + + Ok(Message { + keys: result.keys, + value: result.payload, + offset: source_offset, + event_time: utc_from_timestamp(result.event_time), + headers: result.headers, + }) + } +} + +/// Convert [`Message`] to [`proto::SinkRequest`] +impl From for proto::SinkRequest { + fn from(message: Message) -> Self { + Self { + keys: message.keys, + value: message.value, + event_time: prost_timestamp_from_utc(message.event_time), + watermark: None, + id: format!("{}-{}", message.offset.partition_id, message.offset.offset), + headers: message.headers, + } + } +} diff --git a/serving/source-sink/src/sink.rs b/serving/source-sink/src/sink.rs index 228a294c96..4aee6576bb 100644 --- a/serving/source-sink/src/sink.rs +++ b/serving/source-sink/src/sink.rs @@ -3,7 +3,7 @@ use tonic::Request; use crate::error::Result; use crate::message::Message; -use crate::shared::{connect_with_uds, prost_timestamp_from_utc}; +use crate::shared::connect_with_uds; pub mod proto { tonic::include_proto!("sink.v1"); @@ -45,17 +45,8 @@ impl SinkClient { } pub(crate) async fn sink_fn(&mut self, messages: Vec) -> Result { - let requests: Vec = messages - .into_iter() - .map(|message| proto::SinkRequest { - keys: message.keys, - value: message.value, - event_time: prost_timestamp_from_utc(message.event_time), - watermark: None, - id: format!("{}-{}", message.offset.partition_id, message.offset.offset), - headers: message.headers, - }) - .collect(); + let requests: Vec = + messages.into_iter().map(|message| message.into()).collect(); let (tx, rx) = tokio::sync::mpsc::channel(1); diff --git a/serving/source-sink/src/source.rs b/serving/source-sink/src/source.rs index 389318d49f..8a5c39ecd4 100644 --- a/serving/source-sink/src/source.rs +++ b/serving/source-sink/src/source.rs @@ -6,8 +6,7 @@ use tonic::Request; use crate::error::{Error, Result}; use crate::message::{Message, Offset}; -use crate::shared::{connect_with_uds, utc_from_timestamp}; - +use crate::shared::connect_with_uds; pub mod proto { tonic::include_proto!("source.v1"); } @@ -61,29 +60,14 @@ impl SourceClient { }); let mut stream = self.client.read_fn(request).await?.into_inner(); - let mut messages = Vec::new(); + let mut messages = Vec::with_capacity(num_records as usize); while let Some(response) = stream.next().await { let result = response? .result - .ok_or_else(|| Error::SourceError("Offset not found".to_string()))?; - - let source_offset = match result.offset { - Some(o) => Offset { - offset: BASE64_STANDARD.encode(o.offset), - partition_id: o.partition_id, - }, - None => return Err(Error::SourceError("Offset not found".to_string())), - }; - - let message = Message { - keys: result.keys, - value: result.payload, - offset: source_offset, - event_time: utc_from_timestamp(result.event_time), - headers: result.headers, - }; - messages.push(message); + .ok_or_else(|| Error::SourceError("Empty message".to_string()))?; + + messages.push(result.try_into()?); } Ok(messages) @@ -93,7 +77,9 @@ impl SourceClient { let offsets = offsets .into_iter() .map(|offset| proto::Offset { - offset: BASE64_STANDARD.decode(offset.offset).unwrap(), // we only control the encoding, so this should never fail + offset: BASE64_STANDARD + .decode(offset.offset) + .expect("we control the encoding, so this should never fail"), partition_id: offset.partition_id, }) .collect(); @@ -102,11 +88,11 @@ impl SourceClient { request: Some(proto::ack_request::Request { offsets }), }); - let response = self.client.ack_fn(request).await?.into_inner(); - Ok(response) + Ok(self.client.ack_fn(request).await?.into_inner()) } #[allow(dead_code)] + // TODO: remove dead_code pub(crate) async fn pending_fn(&mut self) -> Result { let request = Request::new(()); let response = self.client.pending_fn(request).await?.into_inner(); @@ -114,6 +100,7 @@ impl SourceClient { } #[allow(dead_code)] + // TODO: remove dead_code pub(crate) async fn partitions_fn(&mut self) -> Result> { let request = Request::new(()); let response = self.client.partitions_fn(request).await?.into_inner(); @@ -216,6 +203,7 @@ mod tests { }); // wait for the server to start + // TODO: flaky tokio::time::sleep(tokio::time::Duration::from_millis(100)).await; let mut source_client = SourceClient::connect(SourceConfig { diff --git a/serving/source-sink/src/transformer.rs b/serving/source-sink/src/transformer.rs index a2747383f4..487744f854 100644 --- a/serving/source-sink/src/transformer.rs +++ b/serving/source-sink/src/transformer.rs @@ -3,7 +3,7 @@ use tonic::Request; use crate::error::Result; use crate::message::Message; -use crate::shared::{connect_with_uds, prost_timestamp_from_utc, utc_from_timestamp}; +use crate::shared::{connect_with_uds, utc_from_timestamp}; use crate::transformer::proto::SourceTransformRequest; pub mod proto { @@ -51,15 +51,13 @@ impl TransformerClient { let offset = message.offset.clone(); let headers = message.headers.clone(); - let request = SourceTransformRequest { - keys: message.keys, - value: message.value, - event_time: prost_timestamp_from_utc(message.event_time), - watermark: None, - headers: message.headers, - }; - - let response = self.client.source_transform_fn(request).await?.into_inner(); + // TODO: is this complex? the reason to do this is, tomorrow when we have the normal + // Pipeline CRD, we can require the Into trait. + let response = self + .client + .source_transform_fn(>::into(message)) + .await? + .into_inner(); let mut messages = Vec::new(); for result in response.results { From 4f9e9467b93fe2fd600871b9f19e1beb16a4f5bf Mon Sep 17 00:00:00 2001 From: Vigith Maurice Date: Tue, 6 Aug 2024 14:38:44 -0700 Subject: [PATCH 06/15] chore: update lock file Signed-off-by: Vigith Maurice --- serving/Cargo.lock | 28 ++++++++++++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/serving/Cargo.lock b/serving/Cargo.lock index e27de24107..2694e0656c 100644 --- a/serving/Cargo.lock +++ b/serving/Cargo.lock @@ -2535,6 +2535,34 @@ dependencies = [ "windows-sys 0.52.0", ] +[[package]] +name = "sourcer-sinker" +version = "0.1.0" +dependencies = [ + "axum", + "base64 0.22.1", + "bytes", + "chrono", + "hyper-util", + "log", + "metrics", + "metrics-exporter-prometheus", + "numaflow", + "prost", + "prost-types", + "tempfile", + "thiserror", + "tokio", + "tokio-stream", + "tokio-util", + "tonic", + "tonic-build", + "tower", + "tracing", + "tracing-subscriber", + "uuid", +] + [[package]] name = "spin" version = "0.9.8" From 81f1ba0f066a149492a74ed1a9d74525dcdd557e Mon Sep 17 00:00:00 2001 From: Vigith Maurice Date: Tue, 6 Aug 2024 14:59:18 -0700 Subject: [PATCH 07/15] fix: add source-sink to Dockerfile Signed-off-by: Vigith Maurice --- Dockerfile | 3 +++ 1 file changed, 3 insertions(+) diff --git a/Dockerfile b/Dockerfile index 4ea993836f..bbf3fa3663 100644 --- a/Dockerfile +++ b/Dockerfile @@ -32,6 +32,9 @@ COPY ./serving/backoff/Cargo.toml ./backoff/ RUN cargo new numaflow-models COPY ./serving/numaflow-models/Cargo.toml ./numaflow-models/ +RUN cargo new source-sink +COPY ./serving/source-sink/Cargo.toml ./source-sink/ + # Copy all Cargo.toml and Cargo.lock files for caching dependencies COPY ./serving/Cargo.toml ./serving/Cargo.lock ./ From 612c0580a2f1e58f0f91abc9c0ad1dd0000bf54e Mon Sep 17 00:00:00 2001 From: Yashash H L Date: Wed, 7 Aug 2024 22:35:06 +0530 Subject: [PATCH 08/15] fix paths Signed-off-by: Yashash H L --- serving/source-sink/Dockerfile | 19 +++++++++++++++++++ serving/source-sink/src/forwarder.rs | 15 ++++++++++++++- serving/source-sink/src/lib.rs | 7 +++++++ serving/source-sink/src/sink.rs | 2 +- serving/source-sink/src/source.rs | 2 +- serving/source-sink/src/transformer.rs | 2 +- 6 files changed, 43 insertions(+), 4 deletions(-) create mode 100644 serving/source-sink/Dockerfile diff --git a/serving/source-sink/Dockerfile b/serving/source-sink/Dockerfile new file mode 100644 index 0000000000..9727b2eb25 --- /dev/null +++ b/serving/source-sink/Dockerfile @@ -0,0 +1,19 @@ +FROM rust:1.76-bookworm AS build + +RUN apt-get update +RUN apt-get install protobuf-compiler -y + +WORKDIR /source-sink +COPY ./ ./ + +# build for release +RUN cargo build --release + +# our final base +FROM debian:bookworm AS simple-source + +# copy the build artifact from the build stage +COPY --from=build /source-sink/target/release/source-sink . + +# set the startup command to run your binary +CMD ["./source-sink"] \ No newline at end of file diff --git a/serving/source-sink/src/forwarder.rs b/serving/source-sink/src/forwarder.rs index d5716ae8df..1e7df3a21c 100644 --- a/serving/source-sink/src/forwarder.rs +++ b/serving/source-sink/src/forwarder.rs @@ -1,3 +1,4 @@ +use chrono::Utc; use tokio::sync::oneshot; use tokio::task::JoinSet; use tracing::info; @@ -42,6 +43,8 @@ impl Forwarder { /// this means that, in the happy path scenario a block is always completely processed. /// this function will return on any error and will cause end up in a non-0 exit code. pub(crate) async fn run(&mut self) -> Result<()> { + let mut messages_count: u64 = 0; + let mut last_forwarded_at = std::time::Instant::now(); loop { // two arms, either shutdown or forward-a-chunk tokio::select! { @@ -52,7 +55,7 @@ impl Forwarder { result = self.source_client.read_fn(self.batch_size, self.timeout_in_ms) => { // Read messages from the source let messages = result?; - + messages_count += messages.len() as u64; // Extract offsets from the messages let offsets = messages.iter().map(|message| message.offset.clone()).collect(); @@ -88,6 +91,16 @@ impl Forwarder { self.source_client.ack_fn(offsets).await?; } } + // if the last forward was more than 1 second ago, forward a chunk print the number of messages forwarded + if last_forwarded_at.elapsed().as_millis() >= 1000 { + info!( + "Forwarded {} messages at time {}", + messages_count, + Utc::now() + ); + messages_count = 0; + last_forwarded_at = std::time::Instant::now(); + } } Ok(()) } diff --git a/serving/source-sink/src/lib.rs b/serving/source-sink/src/lib.rs index d377aaa243..ae471eca53 100644 --- a/serving/source-sink/src/lib.rs +++ b/serving/source-sink/src/lib.rs @@ -154,6 +154,7 @@ async fn shutdown_signal(shutdown_rx: Option>) { signal::ctrl_c() .await .expect("failed to install Ctrl+C handler"); + info!("Received Ctrl+C signal"); }; let terminate = async { @@ -161,12 +162,18 @@ async fn shutdown_signal(shutdown_rx: Option>) { .expect("failed to install signal handler") .recv() .await; + info!("Received terminate signal"); }; let custom_shutdown = async { if let Some(rx) = shutdown_rx { rx.await.ok(); + } else { + // Create a watch channel that never sends + let (_tx, mut rx) = tokio::sync::watch::channel(()); + rx.changed().await.ok(); } + info!("Received custom shutdown signal"); }; tokio::select! { diff --git a/serving/source-sink/src/sink.rs b/serving/source-sink/src/sink.rs index 4aee6576bb..e2801873df 100644 --- a/serving/source-sink/src/sink.rs +++ b/serving/source-sink/src/sink.rs @@ -10,7 +10,7 @@ pub mod proto { } const SINK_SOCKET: &str = "/var/run/numaflow/sink.sock"; -const SINK_SERVER_INFO_FILE: &str = "/var/run/numaflow/sink-server-info"; +const SINK_SERVER_INFO_FILE: &str = "/var/run/numaflow/sinker-server-info"; /// SinkConfig is the configuration for the sink server. #[derive(Debug, Clone)] diff --git a/serving/source-sink/src/source.rs b/serving/source-sink/src/source.rs index 8a5c39ecd4..a1e23bfa99 100644 --- a/serving/source-sink/src/source.rs +++ b/serving/source-sink/src/source.rs @@ -12,7 +12,7 @@ pub mod proto { } const SOURCE_SOCKET: &str = "/var/run/numaflow/source.sock"; -const SOURCE_SERVER_INFO_FILE: &str = "/var/run/numaflow/source-server-info"; +const SOURCE_SERVER_INFO_FILE: &str = "/var/run/numaflow/sourcer-server-info"; /// SourceConfig is the configuration for the source server. #[derive(Debug, Clone)] diff --git a/serving/source-sink/src/transformer.rs b/serving/source-sink/src/transformer.rs index 487744f854..2bbca45bce 100644 --- a/serving/source-sink/src/transformer.rs +++ b/serving/source-sink/src/transformer.rs @@ -11,7 +11,7 @@ pub mod proto { } const TRANSFORMER_SOCKET: &str = "/var/run/numaflow/sourcetransform.sock"; -const TRANSFORMER_SERVER_INFO_FILE: &str = "/var/run/numaflow/transformer-server-info"; +const TRANSFORMER_SERVER_INFO_FILE: &str = "/var/run/numaflow/sourcetransformer-server-info"; /// TransformerConfig is the configuration for the transformer server. #[derive(Debug, Clone)] From 13dd0f98e0b16cdfefd47b2589194c14e91e8a28 Mon Sep 17 00:00:00 2001 From: Vigith Maurice Date: Wed, 7 Aug 2024 18:17:54 -0700 Subject: [PATCH 09/15] fix: binary name Signed-off-by: Vigith Maurice --- serving/source-sink/Dockerfile | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/serving/source-sink/Dockerfile b/serving/source-sink/Dockerfile index 9727b2eb25..9ff7dc019e 100644 --- a/serving/source-sink/Dockerfile +++ b/serving/source-sink/Dockerfile @@ -13,7 +13,7 @@ RUN cargo build --release FROM debian:bookworm AS simple-source # copy the build artifact from the build stage -COPY --from=build /source-sink/target/release/source-sink . +COPY --from=build /source-sink/target/release/sourcer-sinker . # set the startup command to run your binary -CMD ["./source-sink"] \ No newline at end of file +CMD ["./source-sink"] From 6445a93be8152e511a2bb1b7906727fb61bb161b Mon Sep 17 00:00:00 2001 From: Sidhant Kohli Date: Wed, 7 Aug 2024 18:32:50 -0700 Subject: [PATCH 10/15] chore: init metrics (#1909) Signed-off-by: Sidhant Kohli --- serving/source-sink/src/forwarder.rs | 7 ++++++- serving/source-sink/src/lib.rs | 1 - serving/source-sink/src/main.rs | 10 +++++++++- serving/source-sink/src/metrics.rs | 7 +++++-- 4 files changed, 20 insertions(+), 5 deletions(-) diff --git a/serving/source-sink/src/forwarder.rs b/serving/source-sink/src/forwarder.rs index 1e7df3a21c..9cfd6f91b3 100644 --- a/serving/source-sink/src/forwarder.rs +++ b/serving/source-sink/src/forwarder.rs @@ -1,7 +1,8 @@ use chrono::Utc; +use metrics::counter; use tokio::sync::oneshot; use tokio::task::JoinSet; -use tracing::info; +use tracing::{info, trace}; use crate::error::{Error, Result}; use crate::sink::SinkClient; @@ -56,6 +57,8 @@ impl Forwarder { // Read messages from the source let messages = result?; messages_count += messages.len() as u64; + // INCOMING_REQUESTS.inc_by(messages.len() as i64); + counter!("data_read_total", "vertex" => "vertex1", "pipeline" => "pipeline1", "vertex_type" => "type1", "vertex_replica_index" => "index1", "partition_name" => "partition1").increment(messages_count); // Extract offsets from the messages let offsets = messages.iter().map(|message| message.offset.clone()).collect(); @@ -89,8 +92,10 @@ impl Forwarder { // we could rely on gRPC retries and say that any error that is bubbled up is worthy of non-0 exit. // we need to confirm this via FMEA tests. self.source_client.ack_fn(offsets).await?; + trace!("Forwarded {} messages", messages_count); } } + // TODO: print slow forwarder (make the time configurable) // if the last forward was more than 1 second ago, forward a chunk print the number of messages forwarded if last_forwarded_at.elapsed().as_millis() >= 1000 { info!( diff --git a/serving/source-sink/src/lib.rs b/serving/source-sink/src/lib.rs index ae471eca53..b8f4911384 100644 --- a/serving/source-sink/src/lib.rs +++ b/serving/source-sink/src/lib.rs @@ -33,7 +33,6 @@ pub mod transformer; pub mod forwarder; pub mod message; - pub(crate) mod shared; const TIMEOUT_IN_MS: u32 = 1000; diff --git a/serving/source-sink/src/main.rs b/serving/source-sink/src/main.rs index 47a86ebf44..08c760bea2 100644 --- a/serving/source-sink/src/main.rs +++ b/serving/source-sink/src/main.rs @@ -10,17 +10,25 @@ use sourcer_sinker::{metrics::start_metrics_server, run_forwarder}; #[tokio::main] async fn main() { + // Initialize the logger tracing_subscriber::fmt::init(); - // Start the metrics server + // Start the metrics server, which server the prometheus metrics. // TODO: make the port configurable. let metrics_addr: SocketAddr = "0.0.0.0:9090".parse().expect("Invalid address"); + + // Start the metrics server in a separate background async spawn, + // This should be running throughout the lifetime of the application, hence the handle is not + // joined. tokio::spawn(async move { if let Err(e) = start_metrics_server(metrics_addr).await { error!("Metrics server error: {:?}", e); } }); + // Initialize the source, sink and transformer configurations + // We are using the default configurations for now. + // TODO: Make these configurations configurable or we see them not changing? let source_config = SourceConfig::default(); let sink_config = SinkConfig::default(); let transformer_config = if env::var("NUMAFLOW_TRANSFORMER").is_ok() { diff --git a/serving/source-sink/src/metrics.rs b/serving/source-sink/src/metrics.rs index f56daf4b43..c077fc3ba0 100644 --- a/serving/source-sink/src/metrics.rs +++ b/serving/source-sink/src/metrics.rs @@ -1,14 +1,13 @@ use std::future::ready; use axum::{routing::get, Router}; +use metrics::describe_counter; use metrics_exporter_prometheus::{Matcher, PrometheusBuilder, PrometheusHandle}; use tokio::net::{TcpListener, ToSocketAddrs}; use tracing::debug; use crate::error::Error; - /// Collect and emit prometheus metrics. - /// Metrics router and server pub async fn start_metrics_server(addr: A) -> crate::Result<()> where @@ -48,6 +47,10 @@ fn setup_metrics_recorder() -> crate::Result { .install_recorder() .map_err(|e| Error::MetricsError(format!("Prometheus install_recorder: {}", e)))?; + // Define the metrics + describe_counter!("data_read_total", "Total number of Data Messages Read"); + describe_counter!("read_bytes_total", "Total number of bytes read"); + Ok(prometheus_handle) } From bd68cd92f581e7e95d919385a5cd20a21237f92c Mon Sep 17 00:00:00 2001 From: Yashash H L Date: Thu, 8 Aug 2024 21:24:24 +0530 Subject: [PATCH 11/15] metrics Signed-off-by: Yashash H L --- serving/source-sink/src/forwarder.rs | 64 ++++++++-- serving/source-sink/src/lib.rs | 36 ++++-- serving/source-sink/src/main.rs | 24 +++- serving/source-sink/src/message.rs | 2 +- serving/source-sink/src/metrics.rs | 170 +++++++++++++++++++++++++-- serving/source-sink/src/source.rs | 16 ++- 6 files changed, 273 insertions(+), 39 deletions(-) diff --git a/serving/source-sink/src/forwarder.rs b/serving/source-sink/src/forwarder.rs index 9cfd6f91b3..609bc461a4 100644 --- a/serving/source-sink/src/forwarder.rs +++ b/serving/source-sink/src/forwarder.rs @@ -1,13 +1,20 @@ +use crate::error::{Error, Result}; +use crate::metrics::{ + FORWARDER_ACK_LATENCY, FORWARDER_ACK_TOTAL, FORWARDER_LATENCY, FORWARDER_READ_BYTES_TOTAL, + FORWARDER_READ_LATENCY, FORWARDER_READ_TOTAL, FORWARDER_TRANSFORMER_LATENCY, + FORWARDER_WRITE_LATENCY, FORWARDER_WRITE_TOTAL, PARTITION_LABEL, PIPELINE_LABEL, REPLICA_LABEL, + VERTEX_LABEL, VERTEX_TYPE_LABEL, +}; +use crate::sink::SinkClient; +use crate::source::SourceClient; +use crate::transformer::TransformerClient; use chrono::Utc; -use metrics::counter; +use metrics::{counter, histogram}; use tokio::sync::oneshot; use tokio::task::JoinSet; use tracing::{info, trace}; -use crate::error::{Error, Result}; -use crate::sink::SinkClient; -use crate::source::SourceClient; -use crate::transformer::TransformerClient; +const SOURCER_SINKER_VERTEX_TYPE: &str = "sourcer-sinker"; /// Forwarder is responsible for reading messages from the source, applying transformation if /// transformer is present, writing the messages to the sink, and the acknowledging the messages @@ -19,10 +26,15 @@ pub(crate) struct Forwarder { timeout_in_ms: u32, batch_size: u64, shutdown_rx: oneshot::Receiver<()>, + common_labels: Vec<(String, String)>, } impl Forwarder { + #[allow(clippy::too_many_arguments)] // we will use one vertex object to pass all the arguments pub(crate) async fn new( + vertex_name: String, + pipeline_name: String, + replica: u32, source_client: SourceClient, sink_client: SinkClient, transformer_client: Option, @@ -30,6 +42,17 @@ impl Forwarder { batch_size: u64, shutdown_rx: oneshot::Receiver<()>, ) -> Result { + let common_labels = vec![ + (VERTEX_LABEL.to_string(), vertex_name), + (PIPELINE_LABEL.to_string(), pipeline_name), + ( + VERTEX_TYPE_LABEL.to_string(), + SOURCER_SINKER_VERTEX_TYPE.to_string(), + ), + (REPLICA_LABEL.to_string(), replica.to_string()), + (PARTITION_LABEL.to_string(), "0".to_string()), + ]; + Ok(Self { source_client, sink_client, @@ -37,6 +60,7 @@ impl Forwarder { timeout_in_ms, batch_size, shutdown_rx, + common_labels, }) } @@ -47,6 +71,7 @@ impl Forwarder { let mut messages_count: u64 = 0; let mut last_forwarded_at = std::time::Instant::now(); loop { + let start_time = std::time::Instant::now(); // two arms, either shutdown or forward-a-chunk tokio::select! { _ = &mut self.shutdown_rx => { @@ -56,14 +81,21 @@ impl Forwarder { result = self.source_client.read_fn(self.batch_size, self.timeout_in_ms) => { // Read messages from the source let messages = result?; + info!("Read batch size: {}", messages.len()); + + histogram!(FORWARDER_READ_LATENCY, &self.common_labels) + .record(start_time.elapsed().as_millis() as f64); + messages_count += messages.len() as u64; - // INCOMING_REQUESTS.inc_by(messages.len() as i64); - counter!("data_read_total", "vertex" => "vertex1", "pipeline" => "pipeline1", "vertex_type" => "type1", "vertex_replica_index" => "index1", "partition_name" => "partition1").increment(messages_count); + let bytes_count = messages.iter().map(|msg| msg.value.len() as u64).sum::(); + counter!(FORWARDER_READ_TOTAL, &self.common_labels).increment(messages_count); + counter!(FORWARDER_READ_BYTES_TOTAL, &self.common_labels).increment(bytes_count); + // Extract offsets from the messages let offsets = messages.iter().map(|message| message.offset.clone()).collect(); - // Apply transformation if transformer is present let transformed_messages = if let Some(transformer_client) = &self.transformer_client { + let start_time = std::time::Instant::now(); let mut jh = JoinSet::new(); for message in messages { let mut transformer_client = transformer_client.clone(); @@ -76,26 +108,35 @@ impl Forwarder { let result = result?; results.extend(result); } + histogram!(FORWARDER_TRANSFORMER_LATENCY, &self.common_labels) + .record(start_time.elapsed().as_millis() as f64); results } else { messages }; + let sink_write_start_time = std::time::Instant::now(); // Write messages to the sink // TODO: should we retry writing? what if the error is transient? // we could rely on gRPC retries and say that any error that is bubbled up is worthy of non-0 exit. // we need to confirm this via FMEA tests. self.sink_client.sink_fn(transformed_messages).await?; + histogram!(FORWARDER_WRITE_LATENCY, &self.common_labels) + .record(sink_write_start_time.elapsed().as_millis() as f64); + counter!(FORWARDER_WRITE_TOTAL, &self.common_labels).increment(messages_count); // Acknowledge the messages // TODO: should we retry acking? what if the error is transient? // we could rely on gRPC retries and say that any error that is bubbled up is worthy of non-0 exit. // we need to confirm this via FMEA tests. + let ack_start_time = std::time::Instant::now(); self.source_client.ack_fn(offsets).await?; + histogram!(FORWARDER_ACK_LATENCY, &self.common_labels) + .record(ack_start_time.elapsed().as_millis() as f64); + counter!(FORWARDER_ACK_TOTAL, &self.common_labels).increment(messages_count); trace!("Forwarded {} messages", messages_count); } } - // TODO: print slow forwarder (make the time configurable) // if the last forward was more than 1 second ago, forward a chunk print the number of messages forwarded if last_forwarded_at.elapsed().as_millis() >= 1000 { info!( @@ -106,6 +147,8 @@ impl Forwarder { messages_count = 0; last_forwarded_at = std::time::Instant::now(); } + histogram!(FORWARDER_LATENCY, &self.common_labels) + .record(start_time.elapsed().as_millis() as f64); } Ok(()) } @@ -336,6 +379,9 @@ mod tests { .expect("failed to connect to transformer server"); let mut forwarder = Forwarder::new( + "test-vertex".to_string(), + "test-pipeline".to_string(), + 0, source_client, sink_client, Some(transformer_client), diff --git a/serving/source-sink/src/lib.rs b/serving/source-sink/src/lib.rs index b8f4911384..3f2cef7fd8 100644 --- a/serving/source-sink/src/lib.rs +++ b/serving/source-sink/src/lib.rs @@ -1,5 +1,5 @@ -use std::fs; use std::time::Duration; +use std::{env, fs}; use tokio::signal; use tokio::sync::oneshot; use tokio::task::JoinHandle; @@ -35,8 +35,8 @@ pub mod forwarder; pub mod message; pub(crate) mod shared; -const TIMEOUT_IN_MS: u32 = 1000; -const BATCH_SIZE: u64 = 500; +const TIMEOUT_IN_MS: &str = "1000"; +const BATCH_SIZE: &str = "500"; /// forwards a chunk of data from the source to the sink via an optional transformer. /// It takes an optional custom_shutdown_rx for shutting down the forwarder, useful for testing. @@ -46,6 +46,11 @@ pub async fn run_forwarder( transformer_config: Option, custom_shutdown_rx: Option>, ) -> Result<()> { + // TODO: get this from vertex object, controller will have to pass this + let vertex_name = env::var("VERTEX_NAME").unwrap_or_else(|_| "vertex".to_string()); + let pipeline_name = env::var("PIPELINE_NAME").unwrap_or_else(|_| "pipeline".to_string()); + let replica = 0; + wait_for_server_info(&source_config.server_info_file).await?; let mut source_client = SourceClient::connect(source_config).await?; @@ -69,13 +74,26 @@ pub async fn run_forwarder( ) .await?; + // TODO get these from the vertex object + let timeout_in_ms: u32 = env::var("TIMEOUT_IN_MS") + .unwrap_or_else(|_| TIMEOUT_IN_MS.to_string()) + .parse() + .expect("Invalid TIMEOUT_IN_MS"); + let batch_size: u64 = env::var("BATCH_SIZE") + .unwrap_or_else(|_| BATCH_SIZE.to_string()) + .parse() + .expect("Invalid BATCH_SIZE"); + // TODO: use builder pattern of options like TIMEOUT, BATCH_SIZE, etc? let mut forwarder = Forwarder::new( + vertex_name, + pipeline_name, + replica, source_client, sink_client, transformer_client, - TIMEOUT_IN_MS, - BATCH_SIZE, + timeout_in_ms, + batch_size, shutdown_rx, ) .await?; @@ -266,12 +284,8 @@ mod tests { // FIXME: we need to have a better way, this is flaky tokio::time::sleep(tokio::time::Duration::from_millis(100)).await; - unsafe { - env::set_var("SOURCE_SOCKET", src_sock_file.to_str().unwrap()); - } - unsafe { - env::set_var("SINK_SOCKET", sink_sock_file.to_str().unwrap()); - } + env::set_var("SOURCE_SOCKET", src_sock_file.to_str().unwrap()); + env::set_var("SINK_SOCKET", sink_sock_file.to_str().unwrap()); let (shutdown_tx, shutdown_rx) = tokio::sync::oneshot::channel(); diff --git a/serving/source-sink/src/main.rs b/serving/source-sink/src/main.rs index 08c760bea2..4762626aba 100644 --- a/serving/source-sink/src/main.rs +++ b/serving/source-sink/src/main.rs @@ -1,18 +1,29 @@ -use std::env; -use std::net::SocketAddr; - -use tracing::error; - +use log::info; use sourcer_sinker::sink::SinkConfig; use sourcer_sinker::source::SourceConfig; use sourcer_sinker::transformer::TransformerConfig; use sourcer_sinker::{metrics::start_metrics_server, run_forwarder}; +use std::env; +use std::net::SocketAddr; +use tracing::error; +use tracing::level_filters::LevelFilter; +use tracing_subscriber::EnvFilter; #[tokio::main] async fn main() { + let log_level = env::var("NUMAFLOW_DEBUG").unwrap_or_else(|_| "info".to_string()); + // Initialize the logger - tracing_subscriber::fmt::init(); + tracing_subscriber::fmt() + .with_env_filter( + EnvFilter::builder() + .with_default_directive(LevelFilter::INFO.into()) + .parse_lossy(log_level), + ) + .with_target(false) + .init(); + info!("Starting the forwarder"); // Start the metrics server, which server the prometheus metrics. // TODO: make the port configurable. let metrics_addr: SocketAddr = "0.0.0.0:9090".parse().expect("Invalid address"); @@ -31,6 +42,7 @@ async fn main() { // TODO: Make these configurations configurable or we see them not changing? let source_config = SourceConfig::default(); let sink_config = SinkConfig::default(); + // TODO: We should decide transformer is enabled based on the mono vertex spec let transformer_config = if env::var("NUMAFLOW_TRANSFORMER").is_ok() { Some(TransformerConfig::default()) } else { diff --git a/serving/source-sink/src/message.rs b/serving/source-sink/src/message.rs index 09caf0b195..f59cfbfa05 100644 --- a/serving/source-sink/src/message.rs +++ b/serving/source-sink/src/message.rs @@ -50,7 +50,7 @@ impl From for SourceTransformRequest { impl TryFrom for Message { type Error = crate::Error; - fn try_from(result: read_response::Result) -> std::result::Result { + fn try_from(result: read_response::Result) -> Result { let source_offset = match result.offset { Some(o) => Offset { offset: BASE64_STANDARD.encode(o.offset), diff --git a/serving/source-sink/src/metrics.rs b/serving/source-sink/src/metrics.rs index c077fc3ba0..745c9251b2 100644 --- a/serving/source-sink/src/metrics.rs +++ b/serving/source-sink/src/metrics.rs @@ -1,12 +1,36 @@ -use std::future::ready; - use axum::{routing::get, Router}; -use metrics::describe_counter; +use log::info; +use metrics::{describe_counter, describe_histogram}; use metrics_exporter_prometheus::{Matcher, PrometheusBuilder, PrometheusHandle}; +use std::future::ready; +use std::time::Duration; use tokio::net::{TcpListener, ToSocketAddrs}; -use tracing::debug; +use tokio::sync::mpsc; +use tokio::time; +use tracing::{debug, error}; use crate::error::Error; +use crate::source::SourceClient; + +// Define the labels for the metrics +pub const VERTEX_LABEL: &str = "vertex"; +pub const PIPELINE_LABEL: &str = "pipeline"; +pub const REPLICA_LABEL: &str = "replica"; +pub const PARTITION_LABEL: &str = "partition_name"; +pub const VERTEX_TYPE_LABEL: &str = "vertex_type"; + +// Define the metrics +pub const FORWARDER_READ_TOTAL: &str = "forwarder_read_total"; +pub const FORWARDER_READ_LATENCY: &str = "forwarder_read_time"; +pub const FORWARDER_READ_BYTES_TOTAL: &str = "forwarder_read_bytes_total"; +pub const FORWARDER_TRANSFORMER_LATENCY: &str = "forwarder_transformer_time"; + +pub const FORWARDER_LATENCY: &str = "forwarder_forward_chunk_processing_time"; +pub const FORWARDER_ACK_TOTAL: &str = "forwarder_ack_total"; +pub const FORWARDER_ACK_LATENCY: &str = "forwarder_ack_time"; +pub const FORWARDER_WRITE_TOTAL: &str = "forwarder_write_total"; +pub const FORWARDER_WRITE_LATENCY: &str = "forwarder_write_time"; + /// Collect and emit prometheus metrics. /// Metrics router and server pub async fn start_metrics_server(addr: A) -> crate::Result<()> @@ -47,13 +71,143 @@ fn setup_metrics_recorder() -> crate::Result { .install_recorder() .map_err(|e| Error::MetricsError(format!("Prometheus install_recorder: {}", e)))?; - // Define the metrics - describe_counter!("data_read_total", "Total number of Data Messages Read"); - describe_counter!("read_bytes_total", "Total number of bytes read"); - + // Define forwarder metrics + describe_counter!( + FORWARDER_READ_TOTAL, + "Total number of Data Messages Read in the forwarder" + ); + describe_counter!( + FORWARDER_READ_BYTES_TOTAL, + "Total number of bytes read in the forwarder" + ); + describe_histogram!( + FORWARDER_READ_LATENCY, + "Latency of one read iteration in the forwarder" + ); + describe_histogram!( + FORWARDER_TRANSFORMER_LATENCY, + "Latency of one transform iteration in the forwarder" + ); + describe_histogram!( + FORWARDER_LATENCY, + "Latency of one forward a chunk iteration in the forwarder" + ); + describe_counter!( + FORWARDER_ACK_TOTAL, + "Total number of acknowledgments by the forwarder" + ); + describe_histogram!( + FORWARDER_ACK_LATENCY, + "Latency of one ack iteration in the forwarder" + ); + describe_counter!( + FORWARDER_WRITE_TOTAL, + "Total number of Data Messages written by the forwarder" + ); + describe_histogram!( + FORWARDER_WRITE_LATENCY, + "Latency of one write iteration in the forwarder" + ); Ok(prometheus_handle) } +// LagReader is responsible for reading pending information from +// the source client and exposing it as metrics. +#[allow(dead_code)] +pub(crate) struct LagReader { + source_client: SourceClient, + pending_stats: Vec, +} + +#[allow(dead_code)] +impl LagReader { + pub(crate) fn new(source_client: SourceClient) -> Self { + Self { + source_client, + pending_stats: Vec::new(), + } + } + + pub(crate) async fn buildup_pending_info( + &mut self, + mut shutdown_rx: mpsc::Receiver<()>, + lag_checking_interval: Duration, + ) { + let mut ticker = time::interval(lag_checking_interval); + loop { + tokio::select! { + _ = shutdown_rx.recv() => { + return; + } + _ = ticker.tick() => { + match self.source_client.pending_fn().await { + Ok(pending) => { + if pending != -1 { + let ts = TimestampedPending { pending, timestamp: std::time::Instant::now() }; + self.pending_stats.push(ts); + } + } + Err(err) => { + error!("Failed to get pending messages: {:?}", err); + } + } + } + } + } + } + + pub(crate) async fn expose_pending_metrics( + &self, + mut shutdown_rx: mpsc::Receiver<()>, + refresh_interval: Duration, + ) { + let mut ticker = time::interval(refresh_interval); + let lookback_seconds_map = vec![("1m", 60), ("5m", 300), ("15m", 900)]; + loop { + tokio::select! { + _ = shutdown_rx.recv() => { + return; + } + _ = ticker.tick() => { + for (label, seconds) in &lookback_seconds_map { + let pending = self.calculate_pending(*seconds).await; + if pending != -1 { + info!("Pending messages ({}): {}", label, pending); + } + } + } + } + } + } + + async fn calculate_pending(&self, seconds: i64) -> i64 { + let mut result = -1; + let mut total = 0; + let mut num = 0; + let now = std::time::Instant::now(); + + for item in self.pending_stats.iter().rev() { + if now.duration_since(item.timestamp).as_secs() < seconds as u64 { + total += item.pending; + num += 1; + } else { + break; + } + } + + if num > 0 { + result = total / num; + } + + result + } +} + +struct TimestampedPending { + pending: i64, + timestamp: std::time::Instant, +} + #[cfg(test)] mod tests { use std::net::SocketAddr; diff --git a/serving/source-sink/src/source.rs b/serving/source-sink/src/source.rs index a1e23bfa99..3c164bb5e2 100644 --- a/serving/source-sink/src/source.rs +++ b/serving/source-sink/src/source.rs @@ -7,6 +7,7 @@ use tonic::Request; use crate::error::{Error, Result}; use crate::message::{Message, Offset}; use crate::shared::connect_with_uds; + pub mod proto { tonic::include_proto!("source.v1"); } @@ -44,6 +45,7 @@ impl SourceClient { let client = proto::source_client::SourceClient::new(channel) .max_encoding_message_size(config.max_message_size) .max_decoding_message_size(config.max_message_size); + Ok(Self { client }) } @@ -93,9 +95,15 @@ impl SourceClient { #[allow(dead_code)] // TODO: remove dead_code - pub(crate) async fn pending_fn(&mut self) -> Result { + pub(crate) async fn pending_fn(&mut self) -> Result { let request = Request::new(()); - let response = self.client.pending_fn(request).await?.into_inner(); + let response = self + .client + .pending_fn(request) + .await? + .into_inner() + .result + .map_or(0, |r| r.count); Ok(response) } @@ -215,7 +223,7 @@ mod tests { .expect("failed to connect to source server"); let response = source_client.is_ready().await.unwrap(); - assert_eq!(response.ready, true); + assert!(response.ready); let messages = source_client.read_fn(5, 1000).await.unwrap(); assert_eq!(messages.len(), 5); @@ -227,7 +235,7 @@ mod tests { assert!(response.result.unwrap().success.is_some()); let pending = source_client.pending_fn().await.unwrap(); - assert_eq!(pending.result.unwrap().count, 0); + assert_eq!(pending, 0); let partitions = source_client.partitions_fn().await.unwrap(); assert_eq!(partitions, vec![2]); From 19fede0fc9bef8136560d43420a5c0f067f473cf Mon Sep 17 00:00:00 2001 From: Yashash H L Date: Fri, 9 Aug 2024 09:04:11 +0530 Subject: [PATCH 12/15] lag reader Signed-off-by: Yashash H L --- serving/source-sink/src/forwarder.rs | 47 +++--- serving/source-sink/src/lib.rs | 16 +- serving/source-sink/src/main.rs | 12 +- serving/source-sink/src/message.rs | 7 +- serving/source-sink/src/metrics.rs | 238 ++++++++++++++++----------- 5 files changed, 185 insertions(+), 135 deletions(-) diff --git a/serving/source-sink/src/forwarder.rs b/serving/source-sink/src/forwarder.rs index 609bc461a4..2e0731c38c 100644 --- a/serving/source-sink/src/forwarder.rs +++ b/serving/source-sink/src/forwarder.rs @@ -1,23 +1,22 @@ +use chrono::Utc; +use metrics::counter; +use tokio::sync::oneshot; +use tokio::task::JoinSet; +use tracing::{info, trace}; + use crate::error::{Error, Result}; use crate::metrics::{ - FORWARDER_ACK_LATENCY, FORWARDER_ACK_TOTAL, FORWARDER_LATENCY, FORWARDER_READ_BYTES_TOTAL, - FORWARDER_READ_LATENCY, FORWARDER_READ_TOTAL, FORWARDER_TRANSFORMER_LATENCY, - FORWARDER_WRITE_LATENCY, FORWARDER_WRITE_TOTAL, PARTITION_LABEL, PIPELINE_LABEL, REPLICA_LABEL, - VERTEX_LABEL, VERTEX_TYPE_LABEL, + FORWARDER_ACK_TOTAL, FORWARDER_READ_BYTES_TOTAL, FORWARDER_READ_TOTAL, FORWARDER_WRITE_TOTAL, + PARTITION_LABEL, PIPELINE_LABEL, REPLICA_LABEL, VERTEX_LABEL, VERTEX_TYPE_LABEL, }; use crate::sink::SinkClient; use crate::source::SourceClient; use crate::transformer::TransformerClient; -use chrono::Utc; -use metrics::{counter, histogram}; -use tokio::sync::oneshot; -use tokio::task::JoinSet; -use tracing::{info, trace}; const SOURCER_SINKER_VERTEX_TYPE: &str = "sourcer-sinker"; /// Forwarder is responsible for reading messages from the source, applying transformation if -/// transformer is present, writing the messages to the sink, and the acknowledging the messages +/// transformer is present, writing the messages to the sink, and then acknowledging the messages /// back to the source. pub(crate) struct Forwarder { source_client: SourceClient, @@ -66,12 +65,13 @@ impl Forwarder { /// run starts the forward-a-chunk loop and exits only after a chunk has been forwarded and ack'ed. /// this means that, in the happy path scenario a block is always completely processed. - /// this function will return on any error and will cause end up in a non-0 exit code. + /// this function will return on any error and will cause end up in a non-0 exit code. pub(crate) async fn run(&mut self) -> Result<()> { let mut messages_count: u64 = 0; let mut last_forwarded_at = std::time::Instant::now(); loop { - let start_time = std::time::Instant::now(); + // TODO: emit latency metrics, metrics-rs histograms has memory leak issues. + let start_time = tokio::time::Instant::now(); // two arms, either shutdown or forward-a-chunk tokio::select! { _ = &mut self.shutdown_rx => { @@ -81,10 +81,7 @@ impl Forwarder { result = self.source_client.read_fn(self.batch_size, self.timeout_in_ms) => { // Read messages from the source let messages = result?; - info!("Read batch size: {}", messages.len()); - - histogram!(FORWARDER_READ_LATENCY, &self.common_labels) - .record(start_time.elapsed().as_millis() as f64); + info!("Read batch size: {} and latency - {}ms", messages.len(), start_time.elapsed().as_millis()); messages_count += messages.len() as u64; let bytes_count = messages.iter().map(|msg| msg.value.len() as u64).sum::(); @@ -95,7 +92,7 @@ impl Forwarder { let offsets = messages.iter().map(|message| message.offset.clone()).collect(); // Apply transformation if transformer is present let transformed_messages = if let Some(transformer_client) = &self.transformer_client { - let start_time = std::time::Instant::now(); + let start_time = tokio::time::Instant::now(); let mut jh = JoinSet::new(); for message in messages { let mut transformer_client = transformer_client.clone(); @@ -108,31 +105,29 @@ impl Forwarder { let result = result?; results.extend(result); } - histogram!(FORWARDER_TRANSFORMER_LATENCY, &self.common_labels) - .record(start_time.elapsed().as_millis() as f64); + info!("Transformed latency - {}ms", start_time.elapsed().as_millis()); results } else { messages }; - let sink_write_start_time = std::time::Instant::now(); // Write messages to the sink // TODO: should we retry writing? what if the error is transient? // we could rely on gRPC retries and say that any error that is bubbled up is worthy of non-0 exit. // we need to confirm this via FMEA tests. + let start_time = tokio::time::Instant::now(); self.sink_client.sink_fn(transformed_messages).await?; - histogram!(FORWARDER_WRITE_LATENCY, &self.common_labels) - .record(sink_write_start_time.elapsed().as_millis() as f64); + info!("Sink latency - {}ms", start_time.elapsed().as_millis()); counter!(FORWARDER_WRITE_TOTAL, &self.common_labels).increment(messages_count); // Acknowledge the messages // TODO: should we retry acking? what if the error is transient? // we could rely on gRPC retries and say that any error that is bubbled up is worthy of non-0 exit. // we need to confirm this via FMEA tests. - let ack_start_time = std::time::Instant::now(); + let start_time = tokio::time::Instant::now(); self.source_client.ack_fn(offsets).await?; - histogram!(FORWARDER_ACK_LATENCY, &self.common_labels) - .record(ack_start_time.elapsed().as_millis() as f64); + info!("Ack latency - {}ms", start_time.elapsed().as_millis()); + counter!(FORWARDER_ACK_TOTAL, &self.common_labels).increment(messages_count); trace!("Forwarded {} messages", messages_count); } @@ -147,8 +142,6 @@ impl Forwarder { messages_count = 0; last_forwarded_at = std::time::Instant::now(); } - histogram!(FORWARDER_LATENCY, &self.common_labels) - .record(start_time.elapsed().as_millis() as f64); } Ok(()) } diff --git a/serving/source-sink/src/lib.rs b/serving/source-sink/src/lib.rs index 3f2cef7fd8..a638ecdb8c 100644 --- a/serving/source-sink/src/lib.rs +++ b/serving/source-sink/src/lib.rs @@ -1,18 +1,19 @@ use std::time::Duration; use std::{env, fs}; + use tokio::signal; use tokio::sync::oneshot; use tokio::task::JoinHandle; use tokio::time::sleep; use tracing::info; +pub(crate) use crate::error::Error; use crate::forwarder::Forwarder; use crate::sink::{SinkClient, SinkConfig}; use crate::source::{SourceClient, SourceConfig}; use crate::transformer::{TransformerClient, TransformerConfig}; pub(crate) use self::error::Result; -pub(crate) use crate::error::Error; /// SourcerSinker orchestrates data movement from the Source to the Sink via the optional SourceTransformer. /// The forward-a-chunk executes the following in an infinite loop till a shutdown signal is received: @@ -54,6 +55,10 @@ pub async fn run_forwarder( wait_for_server_info(&source_config.server_info_file).await?; let mut source_client = SourceClient::connect(source_config).await?; + // start the lag reader to publish lag metrics + let mut lag_reader = metrics::LagReader::new(source_client.clone(), None, None); + lag_reader.start().await; + wait_for_server_info(&sink_config.server_info_file).await?; let mut sink_client = SinkClient::connect(sink_config).await?; @@ -114,6 +119,7 @@ pub async fn run_forwarder( let _ = tokio::try_join!(forwarder_handle, shutdown_handle) .map_err(|e| Error::ForwarderError(format!("{:?}", e)))?; + lag_reader.shutdown().await; info!("Forwarder stopped gracefully"); Ok(()) } @@ -202,13 +208,15 @@ async fn shutdown_signal(shutdown_rx: Option>) { #[cfg(test)] mod tests { - use crate::sink::SinkConfig; - use crate::source::SourceConfig; + use std::env; + use numaflow::source::{Message, Offset, SourceReadRequest}; use numaflow::{sink, source}; - use std::env; use tokio::sync::mpsc::Sender; + use crate::sink::SinkConfig; + use crate::source::SourceConfig; + struct SimpleSource; #[tonic::async_trait] impl source::Sourcer for SimpleSource { diff --git a/serving/source-sink/src/main.rs b/serving/source-sink/src/main.rs index 4762626aba..45de6513db 100644 --- a/serving/source-sink/src/main.rs +++ b/serving/source-sink/src/main.rs @@ -1,14 +1,16 @@ -use log::info; -use sourcer_sinker::sink::SinkConfig; -use sourcer_sinker::source::SourceConfig; -use sourcer_sinker::transformer::TransformerConfig; -use sourcer_sinker::{metrics::start_metrics_server, run_forwarder}; use std::env; use std::net::SocketAddr; + +use log::info; use tracing::error; use tracing::level_filters::LevelFilter; use tracing_subscriber::EnvFilter; +use sourcer_sinker::sink::SinkConfig; +use sourcer_sinker::source::SourceConfig; +use sourcer_sinker::transformer::TransformerConfig; +use sourcer_sinker::{metrics::start_metrics_server, run_forwarder}; + #[tokio::main] async fn main() { let log_level = env::var("NUMAFLOW_DEBUG").unwrap_or_else(|_| "info".to_string()); diff --git a/serving/source-sink/src/message.rs b/serving/source-sink/src/message.rs index f59cfbfa05..1ca69e9878 100644 --- a/serving/source-sink/src/message.rs +++ b/serving/source-sink/src/message.rs @@ -1,13 +1,14 @@ use std::collections::HashMap; +use base64::engine::general_purpose::STANDARD as BASE64_STANDARD; +use base64::Engine; +use chrono::{DateTime, Utc}; + use crate::error::Error; use crate::shared::{prost_timestamp_from_utc, utc_from_timestamp}; use crate::sink::proto; use crate::source::proto::read_response; use crate::transformer::proto::SourceTransformRequest; -use base64::engine::general_purpose::STANDARD as BASE64_STANDARD; -use base64::Engine; -use chrono::{DateTime, Utc}; /// A message that is sent from the source to the sink. #[derive(Debug, Clone)] diff --git a/serving/source-sink/src/metrics.rs b/serving/source-sink/src/metrics.rs index 745c9251b2..ad6aa5e308 100644 --- a/serving/source-sink/src/metrics.rs +++ b/serving/source-sink/src/metrics.rs @@ -1,12 +1,16 @@ +use std::future::ready; +use std::sync::Arc; +use std::time::Duration; + use axum::{routing::get, Router}; use log::info; -use metrics::{describe_counter, describe_histogram}; +use metrics::describe_counter; use metrics_exporter_prometheus::{Matcher, PrometheusBuilder, PrometheusHandle}; -use std::future::ready; -use std::time::Duration; use tokio::net::{TcpListener, ToSocketAddrs}; -use tokio::sync::mpsc; +use tokio::sync::Mutex; +use tokio::task::JoinHandle; use tokio::time; +use tokio_util::sync::CancellationToken; use tracing::{debug, error}; use crate::error::Error; @@ -21,15 +25,10 @@ pub const VERTEX_TYPE_LABEL: &str = "vertex_type"; // Define the metrics pub const FORWARDER_READ_TOTAL: &str = "forwarder_read_total"; -pub const FORWARDER_READ_LATENCY: &str = "forwarder_read_time"; pub const FORWARDER_READ_BYTES_TOTAL: &str = "forwarder_read_bytes_total"; -pub const FORWARDER_TRANSFORMER_LATENCY: &str = "forwarder_transformer_time"; -pub const FORWARDER_LATENCY: &str = "forwarder_forward_chunk_processing_time"; pub const FORWARDER_ACK_TOTAL: &str = "forwarder_ack_total"; -pub const FORWARDER_ACK_LATENCY: &str = "forwarder_ack_time"; pub const FORWARDER_WRITE_TOTAL: &str = "forwarder_write_total"; -pub const FORWARDER_WRITE_LATENCY: &str = "forwarder_write_time"; /// Collect and emit prometheus metrics. /// Metrics router and server @@ -80,134 +79,181 @@ fn setup_metrics_recorder() -> crate::Result { FORWARDER_READ_BYTES_TOTAL, "Total number of bytes read in the forwarder" ); - describe_histogram!( - FORWARDER_READ_LATENCY, - "Latency of one read iteration in the forwarder" - ); - describe_histogram!( - FORWARDER_TRANSFORMER_LATENCY, - "Latency of one transform iteration in the forwarder" - ); - describe_histogram!( - FORWARDER_LATENCY, - "Latency of one forward a chunk iteration in the forwarder" - ); describe_counter!( FORWARDER_ACK_TOTAL, "Total number of acknowledgments by the forwarder" ); - describe_histogram!( - FORWARDER_ACK_LATENCY, - "Latency of one ack iteration in the forwarder" - ); describe_counter!( FORWARDER_WRITE_TOTAL, "Total number of Data Messages written by the forwarder" ); - describe_histogram!( - FORWARDER_WRITE_LATENCY, - "Latency of one write iteration in the forwarder" - ); Ok(prometheus_handle) } -// LagReader is responsible for reading pending information from -// the source client and exposing it as metrics. -#[allow(dead_code)] +const MAX_PENDING_STATS: usize = 1800; + +// Pending info with timestamp +struct TimestampedPending { + pending: i64, + timestamp: std::time::Instant, +} + +/// `LagReader` is responsible for periodically checking the lag of the source client +/// and exposing the metrics. It maintains a list of pending stats and ensures that +/// only the most recent entries are kept. pub(crate) struct LagReader { source_client: SourceClient, - pending_stats: Vec, + lag_checking_interval: Duration, + refresh_interval: Duration, + cancellation_token: CancellationToken, + buildup_handle: Option>, + expose_handle: Option>, + pending_stats: Arc>>, } -#[allow(dead_code)] impl LagReader { - pub(crate) fn new(source_client: SourceClient) -> Self { + /// Creates a new `LagReader` instance. + pub(crate) fn new( + source_client: SourceClient, + lag_checking_interval: Option, + refresh_interval: Option, + ) -> Self { Self { source_client, - pending_stats: Vec::new(), + lag_checking_interval: lag_checking_interval.unwrap_or_else(|| Duration::from_secs(3)), + refresh_interval: refresh_interval.unwrap_or_else(|| Duration::from_secs(5)), + cancellation_token: CancellationToken::new(), + buildup_handle: None, + expose_handle: None, + pending_stats: Arc::new(Mutex::new(Vec::new())), } } - pub(crate) async fn buildup_pending_info( - &mut self, - mut shutdown_rx: mpsc::Receiver<()>, - lag_checking_interval: Duration, - ) { - let mut ticker = time::interval(lag_checking_interval); - loop { - tokio::select! { - _ = shutdown_rx.recv() => { - return; - } - _ = ticker.tick() => { - match self.source_client.pending_fn().await { - Ok(pending) => { - if pending != -1 { - let ts = TimestampedPending { pending, timestamp: std::time::Instant::now() }; - self.pending_stats.push(ts); + /// Starts the lag reader by spawning tasks to build up pending info and expose pending metrics. + /// + /// This method spawns two asynchronous tasks: + /// - One to periodically check the lag and update the pending stats. + /// - Another to periodically expose the pending metrics. + pub async fn start(&mut self) { + let token = self.cancellation_token.clone(); + let source_client = self.source_client.clone(); + let lag_checking_interval = self.lag_checking_interval; + let refresh_interval = self.refresh_interval; + let pending_stats = self.pending_stats.clone(); + + self.buildup_handle = Some(tokio::spawn(async move { + buildup_pending_info(source_client, token, lag_checking_interval, pending_stats).await; + })); + + let token = self.cancellation_token.clone(); + let pending_stats = self.pending_stats.clone(); + self.expose_handle = Some(tokio::spawn(async move { + expose_pending_metrics(token, refresh_interval, pending_stats).await; + })); + } + + /// Shuts down the lag reader by cancelling the tasks and waiting for them to complete. + pub(crate) async fn shutdown(self) { + self.cancellation_token.cancel(); + if let Some(handle) = self.buildup_handle { + let _ = handle.await; + } + if let Some(handle) = self.expose_handle { + let _ = handle.await; + } + } +} + +// Periodically checks the pending messages from the source client and updates the pending stats. +async fn buildup_pending_info( + mut source_client: SourceClient, + cancellation_token: CancellationToken, + lag_checking_interval: Duration, + pending_stats: Arc>>, +) { + let mut ticker = time::interval(lag_checking_interval); + loop { + tokio::select! { + _ = cancellation_token.cancelled() => { + return; + } + _ = ticker.tick() => { + match source_client.pending_fn().await { + Ok(pending) => { + if pending != -1 { + let mut stats = pending_stats.lock().await; + stats.push(TimestampedPending { + pending, + timestamp: std::time::Instant::now(), + }); + let n = stats.len(); + // Ensure only the most recent MAX_PENDING_STATS entries are kept + if n > MAX_PENDING_STATS { + stats.drain(0..(n - MAX_PENDING_STATS)); } } - Err(err) => { - error!("Failed to get pending messages: {:?}", err); - } + } + Err(err) => { + error!("Failed to get pending messages: {:?}", err); } } } } } +} - pub(crate) async fn expose_pending_metrics( - &self, - mut shutdown_rx: mpsc::Receiver<()>, - refresh_interval: Duration, - ) { - let mut ticker = time::interval(refresh_interval); - let lookback_seconds_map = vec![("1m", 60), ("5m", 300), ("15m", 900)]; - loop { - tokio::select! { - _ = shutdown_rx.recv() => { - return; - } - _ = ticker.tick() => { - for (label, seconds) in &lookback_seconds_map { - let pending = self.calculate_pending(*seconds).await; - if pending != -1 { - info!("Pending messages ({}): {}", label, pending); - } +// Periodically exposes the pending metrics by calculating the average pending messages over different intervals. +async fn expose_pending_metrics( + cancellation_token: CancellationToken, + refresh_interval: Duration, + pending_stats: Arc>>, +) { + let mut ticker = time::interval(refresh_interval); + let lookback_seconds_map = vec![("1m", 60), ("5m", 300), ("15m", 900)]; + loop { + tokio::select! { + _ = cancellation_token.cancelled() => { + return; + } + _ = ticker.tick() => { + for (label, seconds) in &lookback_seconds_map { + let pending = calculate_pending(*seconds, &pending_stats).await; + if pending != -1 { + // TODO: emit it as a metric + info!("Pending messages ({}): {}", label, pending); } } } } } +} - async fn calculate_pending(&self, seconds: i64) -> i64 { - let mut result = -1; - let mut total = 0; - let mut num = 0; - let now = std::time::Instant::now(); - - for item in self.pending_stats.iter().rev() { - if now.duration_since(item.timestamp).as_secs() < seconds as u64 { - total += item.pending; - num += 1; - } else { - break; - } - } +// Calculate the average pending messages over the last `seconds` seconds. +async fn calculate_pending( + seconds: i64, + pending_stats: &Arc>>, +) -> i64 { + let mut result = -1; + let mut total = 0; + let mut num = 0; + let now = std::time::Instant::now(); - if num > 0 { - result = total / num; + let stats = pending_stats.lock().await; + for item in stats.iter().rev() { + if now.duration_since(item.timestamp).as_secs() < seconds as u64 { + total += item.pending; + num += 1; + } else { + break; } + } - result + if num > 0 { + result = total / num; } -} -struct TimestampedPending { - pending: i64, - timestamp: std::time::Instant, + result } - #[cfg(test)] mod tests { use std::net::SocketAddr; From 50670d280c3a64ee4d9cc831ad95ce5adced4c5d Mon Sep 17 00:00:00 2001 From: Yashash H L Date: Fri, 9 Aug 2024 11:31:04 +0530 Subject: [PATCH 13/15] setup tls Signed-off-by: Yashash H L --- serving/source-sink/Cargo.toml | 3 +++ serving/source-sink/src/main.rs | 2 +- serving/source-sink/src/metrics.rs | 43 ++++++++++++++++++++++-------- 3 files changed, 36 insertions(+), 12 deletions(-) diff --git a/serving/source-sink/Cargo.toml b/serving/source-sink/Cargo.toml index 0ca88308bd..bc3481381e 100644 --- a/serving/source-sink/Cargo.toml +++ b/serving/source-sink/Cargo.toml @@ -5,6 +5,7 @@ edition = "2021" [dependencies] axum = "0.7.5" +axum-server = { version = "0.7.1", features = ["tls-rustls"]} tonic = "0.12.0" bytes = "1.7.1" thiserror = "1.0.63" @@ -23,6 +24,8 @@ tracing-subscriber = { version = "0.3.18", features = ["env-filter"] } hyper-util = "0.1.6" tower = "0.4.13" uuid = { version = "1.10.0", features = ["v4"] } +once_cell = "1.19.0" +rcgen = "0.13.1" [dev-dependencies] tower = "0.4.13" diff --git a/serving/source-sink/src/main.rs b/serving/source-sink/src/main.rs index 45de6513db..d251185cf8 100644 --- a/serving/source-sink/src/main.rs +++ b/serving/source-sink/src/main.rs @@ -28,7 +28,7 @@ async fn main() { info!("Starting the forwarder"); // Start the metrics server, which server the prometheus metrics. // TODO: make the port configurable. - let metrics_addr: SocketAddr = "0.0.0.0:9090".parse().expect("Invalid address"); + let metrics_addr: SocketAddr = "0.0.0.0:2469".parse().expect("Invalid address"); // Start the metrics server in a separate background async spawn, // This should be running throughout the lifetime of the application, hence the handle is not diff --git a/serving/source-sink/src/metrics.rs b/serving/source-sink/src/metrics.rs index ad6aa5e308..c9571a183e 100644 --- a/serving/source-sink/src/metrics.rs +++ b/serving/source-sink/src/metrics.rs @@ -1,17 +1,21 @@ use std::future::ready; +use std::net::SocketAddr; use std::sync::Arc; use std::time::Duration; +use axum::http::StatusCode; +use axum::response::IntoResponse; use axum::{routing::get, Router}; +use axum_server::tls_rustls::RustlsConfig; use log::info; use metrics::describe_counter; use metrics_exporter_prometheus::{Matcher, PrometheusBuilder, PrometheusHandle}; -use tokio::net::{TcpListener, ToSocketAddrs}; +use rcgen::{CertifiedKey, generate_simple_self_signed}; use tokio::sync::Mutex; use tokio::task::JoinHandle; use tokio::time; use tokio_util::sync::CancellationToken; -use tracing::{debug, error}; +use tracing::error; use crate::error::Error; use crate::source::SourceClient; @@ -32,26 +36,43 @@ pub const FORWARDER_WRITE_TOTAL: &str = "forwarder_write_total"; /// Collect and emit prometheus metrics. /// Metrics router and server -pub async fn start_metrics_server(addr: A) -> crate::Result<()> +pub async fn start_metrics_server(addr: SocketAddr) -> crate::Result<()> where - A: ToSocketAddrs + std::fmt::Debug, { - // setup_metrics_recorder should only be invoked once - let recorder_handle = setup_metrics_recorder()?; - let metrics_app = Router::new().route("/metrics", get(move || ready(recorder_handle.render()))); + // Generate a self-signed certificate + let CertifiedKey { cert, key_pair } = generate_simple_self_signed(vec!["localhost".into()]).map_err(|e| { + Error::MetricsError(format!("Generating self-signed certificate: {}", e)) + })?; + - let listener = TcpListener::bind(&addr) + let tls_config = RustlsConfig::from_pem(cert.pem().into(), key_pair.serialize_pem().into()) .await - .map_err(|e| Error::MetricsError(format!("Creating listener on {:?}: {}", addr, e)))?; + .map_err(|e| Error::MetricsError(format!("Creating tlsConfig from pem: {}", e)))?; - debug!("metrics server started at addr: {:?}", addr); + // setup_metrics_recorder should only be invoked once + let recorder_handle = setup_metrics_recorder()?; + + let metrics_app = Router::new() + .route("/metrics", get(move || ready(recorder_handle.render()))) + .route("/livez", get(livez)) + .route("/readyz", get(readyz)); - axum::serve(listener, metrics_app) + axum_server::bind_rustls(addr, tls_config) + .serve(metrics_app.into_make_service()) .await .map_err(|e| Error::MetricsError(format!("Starting web server for metrics: {}", e)))?; + Ok(()) } +async fn livez() -> impl IntoResponse { + StatusCode::NO_CONTENT +} + +async fn readyz() -> impl IntoResponse { + StatusCode::NO_CONTENT +} + /// setup the Prometheus metrics recorder. fn setup_metrics_recorder() -> crate::Result { // 1 micro-sec < t < 1000 seconds From c50b4332c699f0beece5441f1053263a033df537 Mon Sep 17 00:00:00 2001 From: Yashash H L Date: Fri, 9 Aug 2024 20:26:00 +0530 Subject: [PATCH 14/15] integrate with controller Signed-off-by: Yashash H L --- Dockerfile | 17 +- Makefile | 2 +- examples/21-simple-mono-vertex.yaml | 9 +- .../numaflow/v1alpha1/mono_vertex_types.go | 7 +- pkg/apis/numaflow/v1alpha1/vertex_types.go | 1 + pkg/metrics/metrics_server.go | 1 - pkg/udf/map_udf.go | 5 - serving/Cargo.lock | 3 + serving/Dockerfile | 20 ++- serving/source-sink/Cargo.toml | 4 +- serving/source-sink/Dockerfile | 4 +- serving/source-sink/src/config.rs | 153 ++++++++++++++++++ serving/source-sink/src/error.rs | 3 + serving/source-sink/src/forwarder.rs | 45 ++---- serving/source-sink/src/lib.rs | 53 +++--- serving/source-sink/src/main.rs | 39 +++-- serving/source-sink/src/metrics.rs | 71 +++++--- 17 files changed, 309 insertions(+), 128 deletions(-) create mode 100644 serving/source-sink/src/config.rs diff --git a/Dockerfile b/Dockerfile index bbf3fa3663..350a9fe4e0 100644 --- a/Dockerfile +++ b/Dockerfile @@ -17,6 +17,9 @@ FROM rust:1.79-bookworm as extension-base RUN curl -L --proto '=https' --tlsv1.2 -sSf https://raw.githubusercontent.com/cargo-bins/cargo-binstall/main/install-from-binstall-release.sh | bash +RUN apt-get update +RUN apt-get install protobuf-compiler -y + RUN cargo new serve # Create a new empty shell project WORKDIR /serve @@ -47,21 +50,29 @@ COPY ./serving/servesink/src ./servesink/src COPY ./serving/extras/upstreams/src ./extras/upstreams/src COPY ./serving/backoff/src ./backoff/src COPY ./serving/numaflow-models/src ./numaflow-models/src +COPY ./serving/source-sink/src ./source-sink/src +COPY ./serving/source-sink/build.rs ./source-sink/build.rs +COPY ./serving/source-sink/proto ./source-sink/proto # Build the real binaries -RUN touch src/main.rs servesink/main.rs extras/upstreams/main.rs numaflow-models/main.rs && \ - cargo build --release +RUN touch src/main.rs servesink/src/main.rs numaflow-models/src/main.rs source-sink/src/main.rs && \ + cargo build --workspace --all --release #################################################################################################### # numaflow #################################################################################################### ARG BASE_IMAGE -FROM ${BASE_IMAGE} as numaflow +FROM debian:bookworm as numaflow + +# Install necessary libraries +RUN apt-get update && apt-get install -y libssl3 COPY --from=base /bin/numaflow /bin/numaflow COPY ui/build /ui/build COPY --from=extension-base /serve/target/release/serve /bin/serve +COPY --from=extension-base /serve/target/release/sourcer-sinker /bin/sourcer-sinker + COPY ./serving/config config ENTRYPOINT [ "/bin/numaflow" ] diff --git a/Makefile b/Makefile index 4e1ee98fa4..1c11b01583 100644 --- a/Makefile +++ b/Makefile @@ -6,7 +6,7 @@ DIST_DIR=${CURRENT_DIR}/dist BINARY_NAME:=numaflow DOCKERFILE:=Dockerfile DEV_BASE_IMAGE:=debian:bookworm -RELEASE_BASE_IMAGE:=gcr.io/distroless/cc-debian12 +RELEASE_BASE_IMAGE:=debian:bookworm BUILD_DATE=$(shell date -u +'%Y-%m-%dT%H:%M:%SZ') GIT_COMMIT=$(shell git rev-parse HEAD) diff --git a/examples/21-simple-mono-vertex.yaml b/examples/21-simple-mono-vertex.yaml index b1f7dbd1fd..f3bc307ff1 100644 --- a/examples/21-simple-mono-vertex.yaml +++ b/examples/21-simple-mono-vertex.yaml @@ -3,11 +3,16 @@ kind: MonoVertex metadata: name: simple-mono-vertex spec: + limits: + readBatchSize: 1000 source: udsource: container: - image: quay.io/numaio/numaflow-java/source-simple-source:stable + image: docker.intuit.com/personal/yhl01/simple-source:stable + transformer: + container: + image: quay.io/yhl25/mapt-event-time-filter:stable sink: udsink: container: - image: quay.io/numaio/numaflow-java/simple-sink:stable + image: docker.intuit.com/personal/yhl01/blackhole-sink:stable diff --git a/pkg/apis/numaflow/v1alpha1/mono_vertex_types.go b/pkg/apis/numaflow/v1alpha1/mono_vertex_types.go index 359056c14c..6243e8a0bc 100644 --- a/pkg/apis/numaflow/v1alpha1/mono_vertex_types.go +++ b/pkg/apis/numaflow/v1alpha1/mono_vertex_types.go @@ -319,7 +319,7 @@ func (mv MonoVertex) GetPodSpec(req GetMonoVertexPodSpecReq) (*corev1.PodSpec, e HTTPGet: &corev1.HTTPGetAction{ Path: "/readyz", Port: intstr.FromInt32(MonoVertexMetricsPort), - Scheme: corev1.URISchemeHTTPS, + Scheme: corev1.URISchemeHTTP, }, }, InitialDelaySeconds: 3, @@ -331,7 +331,7 @@ func (mv MonoVertex) GetPodSpec(req GetMonoVertexPodSpecReq) (*corev1.PodSpec, e HTTPGet: &corev1.HTTPGetAction{ Path: "/livez", Port: intstr.FromInt32(MonoVertexMetricsPort), - Scheme: corev1.URISchemeHTTPS, + Scheme: corev1.URISchemeHTTP, }, }, InitialDelaySeconds: 20, @@ -403,7 +403,8 @@ func (mvspec MonoVertexSpec) DeepCopyWithoutReplicas() MonoVertexSpec { func (mvspec MonoVertexSpec) buildContainers(req getContainerReq) []corev1.Container { mainContainer := containerBuilder{}. - init(req).command("/bin/serve").build() // TODO: command + init(req).command(MonoVertexBinary).build() + containers := []corev1.Container{mainContainer} if mvspec.Source.UDSource != nil { // Only support UDSource for now. containers = append(containers, mvspec.Source.getUDSourceContainer(req)) diff --git a/pkg/apis/numaflow/v1alpha1/vertex_types.go b/pkg/apis/numaflow/v1alpha1/vertex_types.go index c93af36a98..71cd9c7171 100644 --- a/pkg/apis/numaflow/v1alpha1/vertex_types.go +++ b/pkg/apis/numaflow/v1alpha1/vertex_types.go @@ -51,6 +51,7 @@ const ( ) const ServingBinary = "/bin/serve" +const MonoVertexBinary = "/bin/sourcer-sinker" // +genclient // +kubebuilder:object:root=true diff --git a/pkg/metrics/metrics_server.go b/pkg/metrics/metrics_server.go index bb7626aa78..7cafaa9070 100644 --- a/pkg/metrics/metrics_server.go +++ b/pkg/metrics/metrics_server.go @@ -148,7 +148,6 @@ func NewMetricsServer(vertex *dfv1.Vertex, opts ...Option) *metricsServer { if m.lagReaders != nil { for partitionName := range m.lagReaders { m.partitionPendingInfo[partitionName] = sharedqueue.New[timestampedPending](1800) - } } return m diff --git a/pkg/udf/map_udf.go b/pkg/udf/map_udf.go index 3e33d5f8b9..44cb1b5aa7 100644 --- a/pkg/udf/map_udf.go +++ b/pkg/udf/map_udf.go @@ -219,11 +219,6 @@ func (u *MapUDFProcessor) Start(ctx context.Context) error { opts = append(opts, forward.WithUDFUnaryMap(mapHandler)) } - // We can have the vertex running only of the map modes - if enableMapUdfStream && enableBatchMapUdf { - return fmt.Errorf("vertex cannot have both map stream and batch map modes enabled") - } - for index, bufferPartition := range fromBuffer { // Populate shuffle function map shuffleFuncMap := make(map[string]*shuffle.Shuffle) diff --git a/serving/Cargo.lock b/serving/Cargo.lock index 2694e0656c..b5a667f660 100644 --- a/serving/Cargo.lock +++ b/serving/Cargo.lock @@ -2548,8 +2548,11 @@ dependencies = [ "metrics", "metrics-exporter-prometheus", "numaflow", + "numaflow-models", + "once_cell", "prost", "prost-types", + "serde_json", "tempfile", "thiserror", "tokio", diff --git a/serving/Dockerfile b/serving/Dockerfile index 863b999a81..697cfd6f27 100644 --- a/serving/Dockerfile +++ b/serving/Dockerfile @@ -4,6 +4,9 @@ FROM rust:1.79-bookworm as builder RUN curl -L --proto '=https' --tlsv1.2 -sSf https://raw.githubusercontent.com/cargo-bins/cargo-binstall/main/install-from-binstall-release.sh | bash +RUN apt-get update +RUN apt-get install protobuf-compiler -y + RUN cargo new serve # Create a new empty shell project WORKDIR /serve @@ -19,6 +22,9 @@ COPY ./backoff/Cargo.toml ./backoff/Cargo.toml RUN cargo new numaflow-models COPY ./numaflow-models/Cargo.toml ./numaflow-models/ +RUN cargo new source-sink +COPY ./source-sink/Cargo.toml ./source-sink/Cargo.toml + # Copy all Cargo.toml and Cargo.lock files for caching dependencies COPY ./Cargo.toml ./Cargo.lock ./ @@ -31,16 +37,20 @@ COPY ./servesink/src ./servesink/src COPY ./extras/upstreams/src ./extras/upstreams/src COPY ./backoff/src ./backoff/src COPY ./numaflow-models/src ./numaflow-models/src +COPY ./source-sink/src ./source-sink/src +COPY ./source-sink/build.rs ./source-sink/build.rs +COPY ./source-sink/proto ./source-sink/proto # Build the real binaries -RUN touch src/main.rs servesink/main.rs extras/upstreams/main.rs numaflow-models/main.rs && \ - cargo build --release +RUN touch src/main.rs servesink/src/main.rs numaflow-models/src/main.rs source-sink/src/main.rs && \ + cargo build --workspace --all --release # Use a lightweight image for the runtime -FROM gcr.io/distroless/cc-debian12 as numaflow-ext +FROM debian:bookworm as numaflow-ext -COPY --from=builder /serve/target/release/serve . -COPY ./config config +RUN apt-get update && apt-get install -y libssl3 +COPY --from=builder /serve/target/release/ . +COPY ./config config ENTRYPOINT ["./serve"] \ No newline at end of file diff --git a/serving/source-sink/Cargo.toml b/serving/source-sink/Cargo.toml index bc3481381e..b048755f35 100644 --- a/serving/source-sink/Cargo.toml +++ b/serving/source-sink/Cargo.toml @@ -5,7 +5,6 @@ edition = "2021" [dependencies] axum = "0.7.5" -axum-server = { version = "0.7.1", features = ["tls-rustls"]} tonic = "0.12.0" bytes = "1.7.1" thiserror = "1.0.63" @@ -25,7 +24,8 @@ hyper-util = "0.1.6" tower = "0.4.13" uuid = { version = "1.10.0", features = ["v4"] } once_cell = "1.19.0" -rcgen = "0.13.1" +serde_json = "1.0.122" +numaflow-models = { path = "../numaflow-models"} [dev-dependencies] tower = "0.4.13" diff --git a/serving/source-sink/Dockerfile b/serving/source-sink/Dockerfile index 9ff7dc019e..4ed8bb62f7 100644 --- a/serving/source-sink/Dockerfile +++ b/serving/source-sink/Dockerfile @@ -13,7 +13,7 @@ RUN cargo build --release FROM debian:bookworm AS simple-source # copy the build artifact from the build stage -COPY --from=build /source-sink/target/release/sourcer-sinker . +COPY --from=build /source-sink/target/release/source-sink /bin/serve # set the startup command to run your binary -CMD ["./source-sink"] +CMD ["/bin/serve"] diff --git a/serving/source-sink/src/config.rs b/serving/source-sink/src/config.rs new file mode 100644 index 0000000000..9ac27a3413 --- /dev/null +++ b/serving/source-sink/src/config.rs @@ -0,0 +1,153 @@ +use crate::error::Error; +use base64::prelude::BASE64_STANDARD; +use base64::Engine; +use log::LevelFilter; +use numaflow_models::models::MonoVertex; +use std::env; +use std::sync::OnceLock; + +const ENV_MONO_VERTEX_OBJ: &str = "NUMAFLOW_MONO_VERTEX_OBJECT"; +const ENV_GRPC_MAX_MESSAGE_SIZE: &str = "NUMAFLOW_GRPC_MAX_MESSAGE_SIZE"; +const ENV_POD_REPLICA: &str = "NUMAFLOW_REPLICA"; +const DEFAULT_GRPC_MAX_MESSAGE_SIZE: usize = 64 * 1024 * 1024; // 64 MB +const DEFAULT_METRICS_PORT: u16 = 2469; +const ENV_LOG_LEVEL: &str = "NUMAFLOW_DEBUG"; +const DEFAULT_LAG_CHECK_INTERVAL_IN_SECS: u16 = 5; +const DEFAULT_LAG_REFRESH_INTERVAL_IN_SECS: u16 = 3; +const DEFAULT_BATCH_SIZE: u64 = 500; +const DEFAULT_TIMEOUT_IN_MS: u32 = 1000; + +pub fn config() -> &'static Settings { + static CONF: OnceLock = OnceLock::new(); + CONF.get_or_init(|| match Settings::load() { + Ok(v) => v, + Err(e) => { + panic!("Failed to load configuration: {:?}", e); + } + }) +} + +pub struct Settings { + pub mono_vertex_name: String, + pub replica: u32, + pub batch_size: u64, + pub timeout_in_ms: u32, + pub metrics_server_listen_port: u16, + pub log_level: String, + pub grpc_max_message_size: usize, + pub is_transformer_enabled: bool, + pub lag_check_interval_in_secs: u16, + pub lag_refresh_interval_in_secs: u16, +} + +impl Default for Settings { + fn default() -> Self { + Self { + mono_vertex_name: "default".to_string(), + replica: 0, + batch_size: DEFAULT_BATCH_SIZE, + timeout_in_ms: DEFAULT_TIMEOUT_IN_MS, + metrics_server_listen_port: DEFAULT_METRICS_PORT, + log_level: "info".to_string(), + grpc_max_message_size: DEFAULT_GRPC_MAX_MESSAGE_SIZE, + is_transformer_enabled: false, + lag_check_interval_in_secs: DEFAULT_LAG_CHECK_INTERVAL_IN_SECS, + lag_refresh_interval_in_secs: DEFAULT_LAG_REFRESH_INTERVAL_IN_SECS, + } + } +} + +impl Settings { + fn load() -> Result { + let mut settings = Settings::default(); + if let Ok(mono_vertex_spec) = env::var(ENV_MONO_VERTEX_OBJ) { + // decode the spec it will be base64 encoded + let mono_vertex_spec = BASE64_STANDARD + .decode(mono_vertex_spec.as_bytes()) + .map_err(|e| { + Error::ConfigError(format!("Failed to decode mono vertex spec: {:?}", e)) + })?; + + let mono_vertex_obj: MonoVertex = + serde_json::from_slice(&mono_vertex_spec).map_err(|e| { + Error::ConfigError(format!("Failed to parse mono vertex spec: {:?}", e)) + })?; + + settings.batch_size = mono_vertex_obj + .spec + .limits + .clone() + .unwrap() + .read_batch_size + .map(|x| x as u64) + .unwrap_or(DEFAULT_BATCH_SIZE); + + settings.timeout_in_ms = mono_vertex_obj + .spec + .limits + .clone() + .unwrap() + .read_timeout + .map(|x| std::time::Duration::from(x).as_millis() as u32) + .unwrap_or(DEFAULT_TIMEOUT_IN_MS); + + settings.mono_vertex_name = mono_vertex_obj + .metadata + .and_then(|metadata| metadata.name) + .ok_or_else(|| Error::ConfigError("Mono vertex name not found".to_string()))?; + + settings.is_transformer_enabled = mono_vertex_obj + .spec + .source + .ok_or(Error::ConfigError("Source not found".to_string()))? + .transformer + .is_some(); + } + + settings.log_level = + env::var(ENV_LOG_LEVEL).unwrap_or_else(|_| LevelFilter::Info.to_string()); + + settings.grpc_max_message_size = env::var(ENV_GRPC_MAX_MESSAGE_SIZE) + .unwrap_or_else(|_| DEFAULT_GRPC_MAX_MESSAGE_SIZE.to_string()) + .parse() + .map_err(|e| { + Error::ConfigError(format!("Failed to parse grpc max message size: {:?}", e)) + })?; + + settings.replica = env::var(ENV_POD_REPLICA) + .unwrap_or_else(|_| "0".to_string()) + .parse() + .map_err(|e| Error::ConfigError(format!("Failed to parse pod replica: {:?}", e)))?; + + Ok(settings) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use std::env; + + #[test] + fn test_settings_load() { + // Set up environment variables + env::set_var(ENV_MONO_VERTEX_OBJ, "eyJtZXRhZGF0YSI6eyJuYW1lIjoic2ltcGxlLW1vbm8tdmVydGV4IiwibmFtZXNwYWNlIjoiZGVmYXVsdCIsImNyZWF0aW9uVGltZXN0YW1wIjpudWxsfSwic3BlYyI6eyJyZXBsaWNhcyI6MCwic291cmNlIjp7InRyYW5zZm9ybWVyIjp7ImNvbnRhaW5lciI6eyJpbWFnZSI6InF1YXkuaW8vbnVtYWlvL251bWFmbG93LXJzL21hcHQtZXZlbnQtdGltZS1maWx0ZXI6c3RhYmxlIiwicmVzb3VyY2VzIjp7fX0sImJ1aWx0aW4iOm51bGx9LCJ1ZHNvdXJjZSI6eyJjb250YWluZXIiOnsiaW1hZ2UiOiJkb2NrZXIuaW50dWl0LmNvbS9wZXJzb25hbC95aGwwMS9zaW1wbGUtc291cmNlOnN0YWJsZSIsInJlc291cmNlcyI6e319fX0sInNpbmsiOnsidWRzaW5rIjp7ImNvbnRhaW5lciI6eyJpbWFnZSI6ImRvY2tlci5pbnR1aXQuY29tL3BlcnNvbmFsL3lobDAxL2JsYWNraG9sZS1zaW5rOnN0YWJsZSIsInJlc291cmNlcyI6e319fX0sImxpbWl0cyI6eyJyZWFkQmF0Y2hTaXplIjo1MDAsInJlYWRUaW1lb3V0IjoiMXMifSwic2NhbGUiOnt9fSwic3RhdHVzIjp7InJlcGxpY2FzIjowLCJsYXN0VXBkYXRlZCI6bnVsbCwibGFzdFNjYWxlZEF0IjpudWxsfX0="); + env::set_var(ENV_LOG_LEVEL, "debug"); + env::set_var(ENV_GRPC_MAX_MESSAGE_SIZE, "128000000"); + + // Load settings + let settings = Settings::load().unwrap(); + + // Verify settings + assert_eq!(settings.mono_vertex_name, "simple-mono-vertex"); + assert_eq!(settings.batch_size, 500); + assert_eq!(settings.timeout_in_ms, 1000); + assert_eq!(settings.log_level, "debug"); + assert_eq!(settings.grpc_max_message_size, 128000000); + + // Clean up environment variables + env::remove_var(ENV_MONO_VERTEX_OBJ); + env::remove_var(ENV_LOG_LEVEL); + env::remove_var(ENV_GRPC_MAX_MESSAGE_SIZE); + } +} diff --git a/serving/source-sink/src/error.rs b/serving/source-sink/src/error.rs index 84aec64bcc..76ae1ce590 100644 --- a/serving/source-sink/src/error.rs +++ b/serving/source-sink/src/error.rs @@ -24,6 +24,9 @@ pub enum Error { #[error("gRPC Error - {0}")] GRPCError(String), + + #[error("Config Error - {0}")] + ConfigError(String), } impl From for Error { diff --git a/serving/source-sink/src/forwarder.rs b/serving/source-sink/src/forwarder.rs index 2e0731c38c..cd39038a7c 100644 --- a/serving/source-sink/src/forwarder.rs +++ b/serving/source-sink/src/forwarder.rs @@ -1,19 +1,19 @@ -use chrono::Utc; -use metrics::counter; -use tokio::sync::oneshot; -use tokio::task::JoinSet; -use tracing::{info, trace}; - +use crate::config::config; use crate::error::{Error, Result}; use crate::metrics::{ FORWARDER_ACK_TOTAL, FORWARDER_READ_BYTES_TOTAL, FORWARDER_READ_TOTAL, FORWARDER_WRITE_TOTAL, - PARTITION_LABEL, PIPELINE_LABEL, REPLICA_LABEL, VERTEX_LABEL, VERTEX_TYPE_LABEL, + MONO_VERTEX_NAME, PARTITION_LABEL, REPLICA_LABEL, VERTEX_TYPE_LABEL, }; use crate::sink::SinkClient; use crate::source::SourceClient; use crate::transformer::TransformerClient; +use chrono::Utc; +use metrics::counter; +use tokio::sync::oneshot; +use tokio::task::JoinSet; +use tracing::{info, trace}; -const SOURCER_SINKER_VERTEX_TYPE: &str = "sourcer-sinker"; +const MONO_VERTEX_TYPE: &str = "mono_vertex"; /// Forwarder is responsible for reading messages from the source, applying transformation if /// transformer is present, writing the messages to the sink, and then acknowledging the messages @@ -22,33 +22,25 @@ pub(crate) struct Forwarder { source_client: SourceClient, sink_client: SinkClient, transformer_client: Option, - timeout_in_ms: u32, - batch_size: u64, shutdown_rx: oneshot::Receiver<()>, common_labels: Vec<(String, String)>, } impl Forwarder { - #[allow(clippy::too_many_arguments)] // we will use one vertex object to pass all the arguments + #[allow(clippy::too_many_arguments)] pub(crate) async fn new( - vertex_name: String, - pipeline_name: String, - replica: u32, source_client: SourceClient, sink_client: SinkClient, transformer_client: Option, - timeout_in_ms: u32, - batch_size: u64, shutdown_rx: oneshot::Receiver<()>, ) -> Result { let common_labels = vec![ - (VERTEX_LABEL.to_string(), vertex_name), - (PIPELINE_LABEL.to_string(), pipeline_name), ( - VERTEX_TYPE_LABEL.to_string(), - SOURCER_SINKER_VERTEX_TYPE.to_string(), + MONO_VERTEX_NAME.to_string(), + config().mono_vertex_name.clone(), ), - (REPLICA_LABEL.to_string(), replica.to_string()), + (VERTEX_TYPE_LABEL.to_string(), MONO_VERTEX_TYPE.to_string()), + (REPLICA_LABEL.to_string(), config().replica.to_string()), (PARTITION_LABEL.to_string(), "0".to_string()), ]; @@ -56,8 +48,6 @@ impl Forwarder { source_client, sink_client, transformer_client, - timeout_in_ms, - batch_size, shutdown_rx, common_labels, }) @@ -78,7 +68,7 @@ impl Forwarder { info!("Shutdown signal received, stopping forwarder..."); break; } - result = self.source_client.read_fn(self.batch_size, self.timeout_in_ms) => { + result = self.source_client.read_fn(config().batch_size, config().timeout_in_ms) => { // Read messages from the source let messages = result?; info!("Read batch size: {} and latency - {}ms", messages.len(), start_time.elapsed().as_millis()); @@ -105,7 +95,7 @@ impl Forwarder { let result = result?; results.extend(result); } - info!("Transformed latency - {}ms", start_time.elapsed().as_millis()); + info!("Transformer latency - {}ms", start_time.elapsed().as_millis()); results } else { messages @@ -372,14 +362,9 @@ mod tests { .expect("failed to connect to transformer server"); let mut forwarder = Forwarder::new( - "test-vertex".to_string(), - "test-pipeline".to_string(), - 0, source_client, sink_client, Some(transformer_client), - 1000, - 10, forwarder_shutdown_rx, ) .await diff --git a/serving/source-sink/src/lib.rs b/serving/source-sink/src/lib.rs index a638ecdb8c..2099bc63b9 100644 --- a/serving/source-sink/src/lib.rs +++ b/serving/source-sink/src/lib.rs @@ -1,11 +1,11 @@ +use std::fs; use std::time::Duration; -use std::{env, fs}; use tokio::signal; use tokio::sync::oneshot; use tokio::task::JoinHandle; use tokio::time::sleep; -use tracing::info; +use tracing::{error, info}; pub(crate) use crate::error::Error; use crate::forwarder::Forwarder; @@ -33,12 +33,11 @@ pub mod transformer; pub mod forwarder; +pub mod config; + pub mod message; pub(crate) mod shared; -const TIMEOUT_IN_MS: &str = "1000"; -const BATCH_SIZE: &str = "500"; - /// forwards a chunk of data from the source to the sink via an optional transformer. /// It takes an optional custom_shutdown_rx for shutting down the forwarder, useful for testing. pub async fn run_forwarder( @@ -47,11 +46,6 @@ pub async fn run_forwarder( transformer_config: Option, custom_shutdown_rx: Option>, ) -> Result<()> { - // TODO: get this from vertex object, controller will have to pass this - let vertex_name = env::var("VERTEX_NAME").unwrap_or_else(|_| "vertex".to_string()); - let pipeline_name = env::var("PIPELINE_NAME").unwrap_or_else(|_| "pipeline".to_string()); - let replica = 0; - wait_for_server_info(&source_config.server_info_file).await?; let mut source_client = SourceClient::connect(source_config).await?; @@ -79,29 +73,9 @@ pub async fn run_forwarder( ) .await?; - // TODO get these from the vertex object - let timeout_in_ms: u32 = env::var("TIMEOUT_IN_MS") - .unwrap_or_else(|_| TIMEOUT_IN_MS.to_string()) - .parse() - .expect("Invalid TIMEOUT_IN_MS"); - let batch_size: u64 = env::var("BATCH_SIZE") - .unwrap_or_else(|_| BATCH_SIZE.to_string()) - .parse() - .expect("Invalid BATCH_SIZE"); - // TODO: use builder pattern of options like TIMEOUT, BATCH_SIZE, etc? - let mut forwarder = Forwarder::new( - vertex_name, - pipeline_name, - replica, - source_client, - sink_client, - transformer_client, - timeout_in_ms, - batch_size, - shutdown_rx, - ) - .await?; + let mut forwarder = + Forwarder::new(source_client, sink_client, transformer_client, shutdown_rx).await?; let forwarder_handle: JoinHandle> = tokio::spawn(async move { forwarder.run().await?; @@ -116,8 +90,19 @@ pub async fn run_forwarder( Ok(()) }); - let _ = tokio::try_join!(forwarder_handle, shutdown_handle) - .map_err(|e| Error::ForwarderError(format!("{:?}", e)))?; + forwarder_handle + .await + .unwrap_or_else(|e| { + error!("Forwarder task panicked: {:?}", e); + Err(Error::ForwarderError("Forwarder task panicked".to_string())) + }) + .unwrap_or_else(|e| { + error!("Forwarder failed: {:?}", e); + }); + + if !shutdown_handle.is_finished() { + shutdown_handle.abort(); + } lag_reader.shutdown().await; info!("Forwarder stopped gracefully"); diff --git a/serving/source-sink/src/main.rs b/serving/source-sink/src/main.rs index d251185cf8..30d3591ff3 100644 --- a/serving/source-sink/src/main.rs +++ b/serving/source-sink/src/main.rs @@ -1,20 +1,18 @@ +use log::Level::Info; +use sourcer_sinker::config::config; +use sourcer_sinker::sink::SinkConfig; +use sourcer_sinker::source::SourceConfig; +use sourcer_sinker::transformer::TransformerConfig; +use sourcer_sinker::{metrics::start_metrics_http_server, run_forwarder}; use std::env; use std::net::SocketAddr; - -use log::info; use tracing::error; use tracing::level_filters::LevelFilter; use tracing_subscriber::EnvFilter; -use sourcer_sinker::sink::SinkConfig; -use sourcer_sinker::source::SourceConfig; -use sourcer_sinker::transformer::TransformerConfig; -use sourcer_sinker::{metrics::start_metrics_server, run_forwarder}; - #[tokio::main] async fn main() { - let log_level = env::var("NUMAFLOW_DEBUG").unwrap_or_else(|_| "info".to_string()); - + let log_level = env::var("NUMAFLOW_DEBUG").unwrap_or_else(|_| Info.to_string()); // Initialize the logger tracing_subscriber::fmt() .with_env_filter( @@ -25,7 +23,6 @@ async fn main() { .with_target(false) .init(); - info!("Starting the forwarder"); // Start the metrics server, which server the prometheus metrics. // TODO: make the port configurable. let metrics_addr: SocketAddr = "0.0.0.0:2469".parse().expect("Invalid address"); @@ -34,19 +31,27 @@ async fn main() { // This should be running throughout the lifetime of the application, hence the handle is not // joined. tokio::spawn(async move { - if let Err(e) = start_metrics_server(metrics_addr).await { + if let Err(e) = start_metrics_http_server(metrics_addr).await { error!("Metrics server error: {:?}", e); } }); // Initialize the source, sink and transformer configurations // We are using the default configurations for now. - // TODO: Make these configurations configurable or we see them not changing? - let source_config = SourceConfig::default(); - let sink_config = SinkConfig::default(); - // TODO: We should decide transformer is enabled based on the mono vertex spec - let transformer_config = if env::var("NUMAFLOW_TRANSFORMER").is_ok() { - Some(TransformerConfig::default()) + let source_config = SourceConfig { + max_message_size: config().grpc_max_message_size, + ..Default::default() + }; + + let sink_config = SinkConfig { + max_message_size: config().grpc_max_message_size, + ..Default::default() + }; + let transformer_config = if config().is_transformer_enabled { + Some(TransformerConfig { + max_message_size: config().grpc_max_message_size, + ..Default::default() + }) } else { None }; diff --git a/serving/source-sink/src/metrics.rs b/serving/source-sink/src/metrics.rs index c9571a183e..c2a4eab6aa 100644 --- a/serving/source-sink/src/metrics.rs +++ b/serving/source-sink/src/metrics.rs @@ -1,28 +1,25 @@ use std::future::ready; -use std::net::SocketAddr; use std::sync::Arc; use std::time::Duration; use axum::http::StatusCode; use axum::response::IntoResponse; use axum::{routing::get, Router}; -use axum_server::tls_rustls::RustlsConfig; use log::info; use metrics::describe_counter; use metrics_exporter_prometheus::{Matcher, PrometheusBuilder, PrometheusHandle}; -use rcgen::{CertifiedKey, generate_simple_self_signed}; +use tokio::net::{TcpListener, ToSocketAddrs}; use tokio::sync::Mutex; use tokio::task::JoinHandle; use tokio::time; use tokio_util::sync::CancellationToken; -use tracing::error; +use tracing::{debug, error}; use crate::error::Error; use crate::source::SourceClient; // Define the labels for the metrics -pub const VERTEX_LABEL: &str = "vertex"; -pub const PIPELINE_LABEL: &str = "pipeline"; +pub const MONO_VERTEX_NAME: &str = "vertex"; pub const REPLICA_LABEL: &str = "replica"; pub const PARTITION_LABEL: &str = "partition_name"; pub const VERTEX_TYPE_LABEL: &str = "vertex_type"; @@ -36,35 +33,59 @@ pub const FORWARDER_WRITE_TOTAL: &str = "forwarder_write_total"; /// Collect and emit prometheus metrics. /// Metrics router and server -pub async fn start_metrics_server(addr: SocketAddr) -> crate::Result<()> +pub async fn start_metrics_http_server(addr: A) -> crate::Result<()> where + A: ToSocketAddrs + std::fmt::Debug, { - // Generate a self-signed certificate - let CertifiedKey { cert, key_pair } = generate_simple_self_signed(vec!["localhost".into()]).map_err(|e| { - Error::MetricsError(format!("Generating self-signed certificate: {}", e)) - })?; - - - let tls_config = RustlsConfig::from_pem(cert.pem().into(), key_pair.serialize_pem().into()) - .await - .map_err(|e| Error::MetricsError(format!("Creating tlsConfig from pem: {}", e)))?; - // setup_metrics_recorder should only be invoked once let recorder_handle = setup_metrics_recorder()?; - + let metrics_app = Router::new() .route("/metrics", get(move || ready(recorder_handle.render()))) .route("/livez", get(livez)) - .route("/readyz", get(readyz)); + .route("/readyz", get(readyz)) + .route("/sidecar/livez", get(sidecar_livez)); - axum_server::bind_rustls(addr, tls_config) - .serve(metrics_app.into_make_service()) + let listener = TcpListener::bind(&addr) .await - .map_err(|e| Error::MetricsError(format!("Starting web server for metrics: {}", e)))?; + .map_err(|e| Error::MetricsError(format!("Creating listener on {:?}: {}", addr, e)))?; + + debug!("metrics server started at addr: {:?}", addr); + axum::serve(listener, metrics_app) + .await + .map_err(|e| Error::MetricsError(format!("Starting web server for metrics: {}", e)))?; Ok(()) } +// FIXME: facing some issues with the tls_rustls feature +// pub async fn start_metrics_https_server(addr: SocketAddr) -> crate::Result<()> +// where +// { +// // Generate a self-signed certificate +// let CertifiedKey { cert, key_pair } = generate_simple_self_signed(vec!["localhost".into()]) +// .map_err(|e| Error::MetricsError(format!("Generating self-signed certificate: {}", e)))?; +// +// let tls_config = RustlsConfig::from_pem(cert.pem().into(), key_pair.serialize_pem().into()) +// .await +// .map_err(|e| Error::MetricsError(format!("Creating tlsConfig from pem: {}", e)))?; +// +// // setup_metrics_recorder should only be invoked once +// let recorder_handle = setup_metrics_recorder()?; +// +// let metrics_app = Router::new() +// .route("/metrics", get(move || ready(recorder_handle.render()))) +// .route("/livez", get(livez)) +// .route("/readyz", get(readyz)); +// +// axum_server::bind_rustls(addr, tls_config) +// .serve(metrics_app.into_make_service()) +// .await +// .map_err(|e| Error::MetricsError(format!("Starting web server for metrics: {}", e)))?; +// +// Ok(()) +// } + async fn livez() -> impl IntoResponse { StatusCode::NO_CONTENT } @@ -73,6 +94,10 @@ async fn readyz() -> impl IntoResponse { StatusCode::NO_CONTENT } +async fn sidecar_livez() -> impl IntoResponse { + StatusCode::NO_CONTENT +} + /// setup the Prometheus metrics recorder. fn setup_metrics_recorder() -> crate::Result { // 1 micro-sec < t < 1000 seconds @@ -288,7 +313,7 @@ mod tests { async fn test_start_metrics_server() { let addr = SocketAddr::from(([127, 0, 0, 1], 0)); let server = tokio::spawn(async move { - let result = start_metrics_server(addr).await; + let result = start_metrics_http_server(addr).await; assert!(result.is_ok()) }); From c279908ddc4fcc8345481d471d36c2698df701ec Mon Sep 17 00:00:00 2001 From: Yashash H L Date: Fri, 9 Aug 2024 22:30:03 +0530 Subject: [PATCH 15/15] working version Signed-off-by: Yashash H L --- examples/21-simple-mono-vertex.yaml | 9 +- .../numaflow/v1alpha1/mono_vertex_types.go | 4 +- serving/Cargo.lock | 240 +++++++++++++++++- serving/source-sink/Cargo.toml | 3 + serving/source-sink/src/main.rs | 5 +- serving/source-sink/src/metrics.rs | 62 +++-- 6 files changed, 282 insertions(+), 41 deletions(-) diff --git a/examples/21-simple-mono-vertex.yaml b/examples/21-simple-mono-vertex.yaml index f3bc307ff1..be625c41d2 100644 --- a/examples/21-simple-mono-vertex.yaml +++ b/examples/21-simple-mono-vertex.yaml @@ -3,16 +3,11 @@ kind: MonoVertex metadata: name: simple-mono-vertex spec: - limits: - readBatchSize: 1000 source: udsource: container: - image: docker.intuit.com/personal/yhl01/simple-source:stable - transformer: - container: - image: quay.io/yhl25/mapt-event-time-filter:stable + image: quay.io/numaio/numaflow-java/source-simple-source:stable sink: udsink: container: - image: docker.intuit.com/personal/yhl01/blackhole-sink:stable + image: quay.io/numaio/numaflow-java/simple-sink:stable \ No newline at end of file diff --git a/pkg/apis/numaflow/v1alpha1/mono_vertex_types.go b/pkg/apis/numaflow/v1alpha1/mono_vertex_types.go index 6243e8a0bc..7dd39fd2e8 100644 --- a/pkg/apis/numaflow/v1alpha1/mono_vertex_types.go +++ b/pkg/apis/numaflow/v1alpha1/mono_vertex_types.go @@ -319,7 +319,7 @@ func (mv MonoVertex) GetPodSpec(req GetMonoVertexPodSpecReq) (*corev1.PodSpec, e HTTPGet: &corev1.HTTPGetAction{ Path: "/readyz", Port: intstr.FromInt32(MonoVertexMetricsPort), - Scheme: corev1.URISchemeHTTP, + Scheme: corev1.URISchemeHTTPS, }, }, InitialDelaySeconds: 3, @@ -331,7 +331,7 @@ func (mv MonoVertex) GetPodSpec(req GetMonoVertexPodSpecReq) (*corev1.PodSpec, e HTTPGet: &corev1.HTTPGetAction{ Path: "/livez", Port: intstr.FromInt32(MonoVertexMetricsPort), - Scheme: corev1.URISchemeHTTP, + Scheme: corev1.URISchemeHTTPS, }, }, InitialDelaySeconds: 20, diff --git a/serving/Cargo.lock b/serving/Cargo.lock index b5a667f660..0e70179df8 100644 --- a/serving/Cargo.lock +++ b/serving/Cargo.lock @@ -143,6 +143,33 @@ version = "1.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0c4b4d0bd25bd0b74681c0ad21497610ce1b7c91b1022cd21c80c6fbdd9476b0" +[[package]] +name = "aws-lc-rs" +version = "1.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4ae74d9bd0a7530e8afd1770739ad34b36838829d6ad61818f9230f683f5ad77" +dependencies = [ + "aws-lc-sys", + "mirai-annotations", + "paste", + "zeroize", +] + +[[package]] +name = "aws-lc-sys" +version = "0.20.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0f0e249228c6ad2d240c2dc94b714d711629d52bad946075d8e9b2f5391f0703" +dependencies = [ + "bindgen", + "cc", + "cmake", + "dunce", + "fs_extra", + "libc", + "paste", +] + [[package]] name = "axum" version = "0.7.5" @@ -210,6 +237,30 @@ dependencies = [ "syn", ] +[[package]] +name = "axum-server" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "56bac90848f6a9393ac03c63c640925c4b7c8ca21654de40d53f55964667c7d8" +dependencies = [ + "arc-swap", + "bytes", + "futures-util", + "http 1.1.0", + "http-body 1.0.1", + "http-body-util", + "hyper 1.4.1", + "hyper-util", + "pin-project-lite", + "rustls", + "rustls-pemfile 2.1.3", + "rustls-pki-types", + "tokio", + "tokio-rustls", + "tower", + "tower-service", +] + [[package]] name = "backoff" version = "0.1.0" @@ -251,6 +302,29 @@ version = "1.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8c3c1a368f70d6cf7302d78f8f7093da241fb8e8807c05cc9e51a125895a6d5b" +[[package]] +name = "bindgen" +version = "0.69.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a00dc851838a2120612785d195287475a3ac45514741da670b735818822129a0" +dependencies = [ + "bitflags 2.6.0", + "cexpr", + "clang-sys", + "itertools 0.12.1", + "lazy_static", + "lazycell", + "log", + "prettyplease", + "proc-macro2", + "quote", + "regex", + "rustc-hash", + "shlex", + "syn", + "which", +] + [[package]] name = "bitflags" version = "1.3.2" @@ -301,6 +375,19 @@ name = "cc" version = "1.1.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "26a5c3fd7bfa1ce3897a3a3501d362b2d87b7f2583ebcb4a949ec25911025cbc" +dependencies = [ + "jobserver", + "libc", +] + +[[package]] +name = "cexpr" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6fac387a98bb7c37292057cffc56d62ecb629900026402633ae9160df93a8766" +dependencies = [ + "nom", +] [[package]] name = "cfg-if" @@ -323,6 +410,26 @@ dependencies = [ "windows-targets 0.52.6", ] +[[package]] +name = "clang-sys" +version = "1.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0b023947811758c97c59bf9d1c188fd619ad4718dcaa767947df1cadb14f39f4" +dependencies = [ + "glob", + "libc", + "libloading", +] + +[[package]] +name = "cmake" +version = "0.1.50" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a31c789563b815f77f4250caee12365734369f942439b7defd71e18a48197130" +dependencies = [ + "cc", +] + [[package]] name = "combine" version = "4.6.7" @@ -520,6 +627,12 @@ dependencies = [ "const-random", ] +[[package]] +name = "dunce" +version = "1.0.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "92773504d58c093f6de2459af4af33faa518c13451eb8f2b5698ed3d36e7c813" + [[package]] name = "ed25519" version = "2.2.3" @@ -621,6 +734,12 @@ dependencies = [ "percent-encoding", ] +[[package]] +name = "fs_extra" +version = "1.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "42703706b716c37f96a77aea830392ad231f44c9e9a67872fa5548707e11b11c" + [[package]] name = "futures" version = "0.3.30" @@ -737,6 +856,12 @@ version = "0.29.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "40ecd4077b5ae9fd2e9e169b102c6c330d0605168eb0e8bf79952b256dbefffd" +[[package]] +name = "glob" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d2fabcfbdc87f4758337ca535fb41a6d701b65693ce38287d856d1674551ec9b" + [[package]] name = "h2" version = "0.3.26" @@ -1120,6 +1245,15 @@ version = "2.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8f518f335dce6725a761382244631d86cf0ccb2863413590b31338feb467f9c3" +[[package]] +name = "itertools" +version = "0.12.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ba291022dbbd398a455acf126c1e341954079855bc60dfdda641363bd6922569" +dependencies = [ + "either", +] + [[package]] name = "itertools" version = "0.13.0" @@ -1135,6 +1269,15 @@ version = "1.0.11" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "49f1f14873335454500d59611f1cf4a4b0f786f9ac11f4312a78e4cf2566695b" +[[package]] +name = "jobserver" +version = "0.1.32" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "48d1dbcbbeb6a7fec7e059840aa538bd62aaccf972c7346c4d9d2059312853d0" +dependencies = [ + "libc", +] + [[package]] name = "js-sys" version = "0.3.69" @@ -1253,12 +1396,28 @@ version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bbd2bcb4c963f2ddae06a2efc7e9f3591312473c50c6685e1f298068316e66fe" +[[package]] +name = "lazycell" +version = "1.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "830d08ce1d1d941e6b30645f1a0eb5643013d835ce3779a5fc208261dbe10f55" + [[package]] name = "libc" version = "0.2.155" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "97b3888a4aecf77e811145cadf6eef5901f4782c53886191b2f693f24761847c" +[[package]] +name = "libloading" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4979f22fdb869068da03c9f7528f8297c6fd2606bc3a4affe42e6a823fdb8da4" +dependencies = [ + "cfg-if", + "windows-targets 0.52.6", +] + [[package]] name = "linked-hash-map" version = "0.5.6" @@ -1390,6 +1549,12 @@ dependencies = [ "windows-sys 0.52.0", ] +[[package]] +name = "mirai-annotations" +version = "1.12.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c9be0862c1b3f26a88803c4a49de6889c10e608b3ee9344e6ef5b45fb37ad3d1" + [[package]] name = "multimap" version = "0.10.0" @@ -1644,6 +1809,12 @@ dependencies = [ "windows-targets 0.52.6", ] +[[package]] +name = "paste" +version = "1.0.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "57c0d7b74b563b49d38dae00a0c37d4d6de9b432382b2892f0574ddcae73fd0a" + [[package]] name = "pathdiff" version = "0.2.1" @@ -1836,7 +2007,7 @@ checksum = "5bb182580f71dd070f88d01ce3de9f4da5021db7115d2e1c3605a754153b77c1" dependencies = [ "bytes", "heck 0.5.0", - "itertools", + "itertools 0.13.0", "log", "multimap", "once_cell", @@ -1856,7 +2027,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "18bec9b0adc4eba778b33684b7ba3e7137789434769ee3ce3930463ef904cfca" dependencies = [ "anyhow", - "itertools", + "itertools 0.13.0", "proc-macro2", "quote", "syn", @@ -1934,6 +2105,19 @@ dependencies = [ "bitflags 2.6.0", ] +[[package]] +name = "rcgen" +version = "0.13.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "54077e1872c46788540de1ea3d7f4ccb1983d12f9aa909b234468676c1a36779" +dependencies = [ + "pem", + "ring", + "rustls-pki-types", + "time", + "yasna", +] + [[package]] name = "redis" version = "0.26.1" @@ -2139,6 +2323,12 @@ version = "0.1.24" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "719b953e2095829ee67db738b3bfa9fa368c94900df327b3f07fe6e794d2fe1f" +[[package]] +name = "rustc-hash" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "08d43f7aa6b08d49f382cde6a7982047c3426db949b1424bc4b7ec9ae12c6ce2" + [[package]] name = "rustc_version" version = "0.4.0" @@ -2167,6 +2357,7 @@ version = "0.23.12" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c58f8c84392efc0a126acce10fa59ff7b3d2ac06ab451a33f2741989b806b044" dependencies = [ + "aws-lc-rs", "log", "once_cell", "ring", @@ -2220,6 +2411,7 @@ version = "0.102.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8e6b52d4fda176fd835fdc55a835d4a89b8499cad995885a21149d5ad62f852e" dependencies = [ + "aws-lc-rs", "ring", "rustls-pki-types", "untrusted", @@ -2473,6 +2665,12 @@ dependencies = [ "lazy_static", ] +[[package]] +name = "shlex" +version = "1.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0fda2ff0d084019ba4d7c6f371c95d8fd75ce3524c3cb8fb653a3023f6323e64" + [[package]] name = "signal-hook-registry" version = "1.4.2" @@ -2540,6 +2738,7 @@ name = "sourcer-sinker" version = "0.1.0" dependencies = [ "axum", + "axum-server", "base64 0.22.1", "bytes", "chrono", @@ -2552,6 +2751,8 @@ dependencies = [ "once_cell", "prost", "prost-types", + "rcgen", + "rustls", "serde_json", "tempfile", "thiserror", @@ -3240,6 +3441,18 @@ dependencies = [ "wasm-bindgen", ] +[[package]] +name = "which" +version = "4.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "87ba24419a2078cd2b0f2ede2691b6c66d8e47836da3b6db8265ebad47afbfc7" +dependencies = [ + "either", + "home", + "once_cell", + "rustix", +] + [[package]] name = "winapi" version = "0.3.9" @@ -3448,6 +3661,15 @@ dependencies = [ "linked-hash-map", ] +[[package]] +name = "yasna" +version = "0.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e17bb3549cc1321ae1296b9cdc2698e2b6cb1992adfa19a8c72e5b7a738f44cd" +dependencies = [ + "time", +] + [[package]] name = "zerocopy" version = "0.7.35" @@ -3474,3 +3696,17 @@ name = "zeroize" version = "1.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ced3678a2879b30306d323f4542626697a464a97c0a07c9aebf7ebca65cd4dde" +dependencies = [ + "zeroize_derive", +] + +[[package]] +name = "zeroize_derive" +version = "1.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ce36e65b0d2999d2aafac989fb249189a141aee1f53c612c1f37d72631959f69" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] diff --git a/serving/source-sink/Cargo.toml b/serving/source-sink/Cargo.toml index b048755f35..9db2bbff35 100644 --- a/serving/source-sink/Cargo.toml +++ b/serving/source-sink/Cargo.toml @@ -5,6 +5,7 @@ edition = "2021" [dependencies] axum = "0.7.5" +axum-server = { version = "0.7.1", features = ["tls-rustls"] } tonic = "0.12.0" bytes = "1.7.1" thiserror = "1.0.63" @@ -26,6 +27,8 @@ uuid = { version = "1.10.0", features = ["v4"] } once_cell = "1.19.0" serde_json = "1.0.122" numaflow-models = { path = "../numaflow-models"} +rcgen = "0.13.1" +rustls = { version = "0.23.12", features = ["aws_lc_rs"] } [dev-dependencies] tower = "0.4.13" diff --git a/serving/source-sink/src/main.rs b/serving/source-sink/src/main.rs index 30d3591ff3..aa1d8c0605 100644 --- a/serving/source-sink/src/main.rs +++ b/serving/source-sink/src/main.rs @@ -3,12 +3,13 @@ use sourcer_sinker::config::config; use sourcer_sinker::sink::SinkConfig; use sourcer_sinker::source::SourceConfig; use sourcer_sinker::transformer::TransformerConfig; -use sourcer_sinker::{metrics::start_metrics_http_server, run_forwarder}; +use sourcer_sinker::run_forwarder; use std::env; use std::net::SocketAddr; use tracing::error; use tracing::level_filters::LevelFilter; use tracing_subscriber::EnvFilter; +use sourcer_sinker::metrics::start_metrics_https_server; #[tokio::main] async fn main() { @@ -31,7 +32,7 @@ async fn main() { // This should be running throughout the lifetime of the application, hence the handle is not // joined. tokio::spawn(async move { - if let Err(e) = start_metrics_http_server(metrics_addr).await { + if let Err(e) = start_metrics_https_server(metrics_addr).await { error!("Metrics server error: {:?}", e); } }); diff --git a/serving/source-sink/src/metrics.rs b/serving/source-sink/src/metrics.rs index c2a4eab6aa..d257609c76 100644 --- a/serving/source-sink/src/metrics.rs +++ b/serving/source-sink/src/metrics.rs @@ -1,13 +1,16 @@ use std::future::ready; +use std::net::SocketAddr; use std::sync::Arc; use std::time::Duration; use axum::http::StatusCode; use axum::response::IntoResponse; use axum::{routing::get, Router}; +use axum_server::tls_rustls::RustlsConfig; use log::info; use metrics::describe_counter; use metrics_exporter_prometheus::{Matcher, PrometheusBuilder, PrometheusHandle}; +use rcgen::{CertifiedKey, generate_simple_self_signed}; use tokio::net::{TcpListener, ToSocketAddrs}; use tokio::sync::Mutex; use tokio::task::JoinHandle; @@ -44,7 +47,7 @@ where .route("/metrics", get(move || ready(recorder_handle.render()))) .route("/livez", get(livez)) .route("/readyz", get(readyz)) - .route("/sidecar/livez", get(sidecar_livez)); + .route("/sidecar-livez", get(sidecar_livez)); let listener = TcpListener::bind(&addr) .await @@ -58,33 +61,36 @@ where Ok(()) } -// FIXME: facing some issues with the tls_rustls feature -// pub async fn start_metrics_https_server(addr: SocketAddr) -> crate::Result<()> -// where -// { -// // Generate a self-signed certificate -// let CertifiedKey { cert, key_pair } = generate_simple_self_signed(vec!["localhost".into()]) -// .map_err(|e| Error::MetricsError(format!("Generating self-signed certificate: {}", e)))?; -// -// let tls_config = RustlsConfig::from_pem(cert.pem().into(), key_pair.serialize_pem().into()) -// .await -// .map_err(|e| Error::MetricsError(format!("Creating tlsConfig from pem: {}", e)))?; -// -// // setup_metrics_recorder should only be invoked once -// let recorder_handle = setup_metrics_recorder()?; -// -// let metrics_app = Router::new() -// .route("/metrics", get(move || ready(recorder_handle.render()))) -// .route("/livez", get(livez)) -// .route("/readyz", get(readyz)); -// -// axum_server::bind_rustls(addr, tls_config) -// .serve(metrics_app.into_make_service()) -// .await -// .map_err(|e| Error::MetricsError(format!("Starting web server for metrics: {}", e)))?; -// -// Ok(()) -// } +pub async fn start_metrics_https_server(addr: SocketAddr) -> crate::Result<()> +where +{ + let _ = rustls::crypto::aws_lc_rs::default_provider().install_default(); + + // Generate a self-signed certificate + let CertifiedKey { cert, key_pair } = generate_simple_self_signed(vec!["localhost".into()]) + .map_err(|e| Error::MetricsError(format!("Generating self-signed certificate: {}", e)))?; + + + let tls_config = RustlsConfig::from_pem(cert.pem().into(), key_pair.serialize_pem().into()) + .await + .map_err(|e| Error::MetricsError(format!("Creating tlsConfig from pem: {}", e)))?; + + // setup_metrics_recorder should only be invoked once + let recorder_handle = setup_metrics_recorder()?; + + let metrics_app = Router::new() + .route("/metrics", get(move || ready(recorder_handle.render()))) + .route("/livez", get(livez)) + .route("/readyz", get(readyz)) + .route("/sidecar-livez", get(sidecar_livez)); + + axum_server::bind_rustls(addr, tls_config) + .serve(metrics_app.into_make_service()) + .await + .map_err(|e| Error::MetricsError(format!("Starting web server for metrics: {}", e)))?; + + Ok(()) +} async fn livez() -> impl IntoResponse { StatusCode::NO_CONTENT