From 07e83d6e0c26a5c71162871c1770d91a6ea86888 Mon Sep 17 00:00:00 2001 From: Anthony Dodd Date: Wed, 13 Oct 2021 17:40:00 -0700 Subject: [PATCH 01/10] Add additional tests for various components --- hadron-stream/Cargo.lock | 1 + hadron-stream/Cargo.toml | 3 + hadron-stream/src/config.rs | 24 ++++++ hadron-stream/src/grpc/mod.rs | 1 + hadron-stream/src/grpc/stream_ext.rs | 15 ++++ hadron-stream/src/stream/mod.rs | 4 + hadron-stream/src/stream/mod_test.rs | 95 ++++++++++++++++++++++ hadron-stream/src/stream/publisher.rs | 26 +++--- hadron-stream/src/stream/publisher_test.rs | 82 +++++++++++++++++++ hadron-stream/src/stream/subscriber.rs | 8 +- 10 files changed, 242 insertions(+), 17 deletions(-) create mode 100644 hadron-stream/src/grpc/stream_ext.rs create mode 100644 hadron-stream/src/stream/mod_test.rs create mode 100644 hadron-stream/src/stream/publisher_test.rs diff --git a/hadron-stream/Cargo.lock b/hadron-stream/Cargo.lock index 92413f6..f394a9a 100644 --- a/hadron-stream/Cargo.lock +++ b/hadron-stream/Cargo.lock @@ -579,6 +579,7 @@ dependencies = [ "rand", "serde", "sled", + "tempfile", "thiserror", "tokio", "tokio-stream", diff --git a/hadron-stream/Cargo.toml b/hadron-stream/Cargo.toml index e858aaa..aa7c411 100644 --- a/hadron-stream/Cargo.toml +++ b/hadron-stream/Cargo.toml @@ -40,3 +40,6 @@ uuid = { version="0.8", default-features=false, features=["serde", "v4"] } [build-dependencies] anyhow = "1" tonic-build = "0.5" + +[dev-dependencies] +tempfile = "3.2.0" diff --git a/hadron-stream/src/config.rs b/hadron-stream/src/config.rs index 990bec7..946275d 100644 --- a/hadron-stream/src/config.rs +++ b/hadron-stream/src/config.rs @@ -1,5 +1,7 @@ //! Runtime configuration. +use std::sync::Arc; + use anyhow::{Context, Result}; use serde::Deserialize; @@ -49,4 +51,26 @@ impl Config { .context("invalid pod name, expected offset suffix at the end of the name")?; Ok(config) } + + /// Build an instance for use in tests. + #[cfg(test)] + pub fn new_test() -> Result<(Arc, tempfile::TempDir)> { + let tmpdir = tempfile::tempdir_in("/tmp").context("error creating tmp dir in /tmp")?; + Ok(( + Arc::new(Self { + rust_log: "".into(), + client_port: 7000, + server_port: 7000, + + namespace: "default".into(), + stream: "testing".into(), + statefulset: "testing".into(), + pod_name: "testing-0".into(), + partition: 0, + + storage_data_path: tmpdir.path().to_string_lossy().to_string(), + }), + tmpdir, + )) + } } diff --git a/hadron-stream/src/grpc/mod.rs b/hadron-stream/src/grpc/mod.rs index ba3f330..3bb0bdb 100644 --- a/hadron-stream/src/grpc/mod.rs +++ b/hadron-stream/src/grpc/mod.rs @@ -1,4 +1,5 @@ mod stream; +mod stream_ext; pub type StreamSubscribeRequestAction = stream::stream_subscribe_request::Action; pub type StreamSubscribeSetupStartingPoint = stream::stream_subscribe_setup::StartingPoint; diff --git a/hadron-stream/src/grpc/stream_ext.rs b/hadron-stream/src/grpc/stream_ext.rs new file mode 100644 index 0000000..52d2ee8 --- /dev/null +++ b/hadron-stream/src/grpc/stream_ext.rs @@ -0,0 +1,15 @@ +use crate::grpc::stream::Event; + +impl Event { + #[cfg(test)] + pub fn new_test(id: T, source: &str, r#type: &str) -> Self { + Self { + id: id.to_string(), + source: source.into(), + specversion: "1.0".into(), + r#type: r#type.into(), + optattrs: Default::default(), + data: Default::default(), + } + } +} diff --git a/hadron-stream/src/stream/mod.rs b/hadron-stream/src/stream/mod.rs index 4e18b0a..99eba26 100644 --- a/hadron-stream/src/stream/mod.rs +++ b/hadron-stream/src/stream/mod.rs @@ -22,7 +22,11 @@ //! Replication is limited to 2. When there is only 1 partition, there is no replication. When //! there are only 2 partitions, only a single replica will be possible. +#[cfg(test)] +mod mod_test; mod publisher; +#[cfg(test)] +mod publisher_test; mod subscriber; use std::collections::HashMap; diff --git a/hadron-stream/src/stream/mod_test.rs b/hadron-stream/src/stream/mod_test.rs new file mode 100644 index 0000000..a56406a --- /dev/null +++ b/hadron-stream/src/stream/mod_test.rs @@ -0,0 +1,95 @@ +use anyhow::{Context, Result}; +use rand::prelude::*; + +use crate::config::Config; +use crate::database::Database; +use crate::grpc::Event; +use crate::models::stream::Subscription; +use crate::utils; + +#[tokio::test] +async fn recover_stream_state_empty_state() -> Result<()> { + let (config, _tmpdir) = Config::new_test()?; + let db = Database::new(config.clone()).await?; + let stream_tree = db.get_stream_tree().await?; + let metadata_tree = db.get_stream_tree_metadata().await?; + + let (offset, subscriptions) = super::recover_stream_state(&stream_tree, &metadata_tree).await?; + + assert!(offset == 0, "expected offset to be 0 got {}", offset); + assert!(subscriptions.is_empty(), "expected subscriptions len to be 0 got {}", offset); + + Ok(()) +} + +#[tokio::test] +async fn recover_stream_state_with_previous_state() -> Result<()> { + let (config, _tmpdir) = Config::new_test()?; + let db = Database::new(config.clone()).await?; + let stream_tree = db.get_stream_tree().await?; + let metadata_tree = db.get_stream_tree_metadata().await?; + + let expected_offset = setup_stream_data(&stream_tree).await?; + + let (offset, subs) = super::recover_stream_state(&stream_tree, &metadata_tree).await?; + + assert!(offset == expected_offset, "expected offset to be {} got {}", expected_offset, offset); + assert!(subs.is_empty(), "expected subscriptions len to be 0 got {}", offset); + + Ok(()) +} + +#[tokio::test] +async fn recover_stream_state_with_previous_state_and_subs() -> Result<()> { + let (config, _tmpdir) = Config::new_test()?; + let db = Database::new(config.clone()).await?; + let stream_tree = db.get_stream_tree().await?; + let metadata_tree = db.get_stream_tree_metadata().await?; + + let expected_offset = setup_stream_data(&stream_tree).await?; + let expected_subs = setup_subs_data(&metadata_tree).await?; + + let (offset, mut subs) = super::recover_stream_state(&stream_tree, &metadata_tree).await?; + subs.sort_by(|a, b| a.1.cmp(&b.1)); + + assert!(offset == expected_offset, "expected offset to be {} got {}", expected_offset, offset); + assert_eq!(subs, expected_subs, "expected subscriptions to match {:?}\n{:?}", subs, expected_subs); + + Ok(()) +} + +/// Setup some stream data in the given DB tree returning the last written offset. +async fn setup_stream_data(db: &sled::Tree) -> Result { + use rand::prelude::*; + + let mut batch = sled::Batch::default(); + let mut last_offset = 0; + for offset in 0..rand::thread_rng().gen_range(50..100) { + let event = Event::new_test(offset, "test", "empty"); + let event_bytes = utils::encode_model(&event)?; + batch.insert(&utils::encode_u64(offset), event_bytes.as_slice()); + last_offset = offset; + } + db.apply_batch(batch) + .context("error applying batch to write test data to stream")?; + Ok(last_offset) +} + +/// Setup some subscriptions data in the given DB tree returning the set of created subs. +async fn setup_subs_data(db: &sled::Tree) -> Result> { + let mut batch = sled::Batch::default(); + let mut subs = vec![]; + for offset in 0..rand::thread_rng().gen_range(50..100) { + let sub = Subscription { group_name: offset.to_string(), max_batch_size: 50 }; + let sub_encoded = utils::encode_model(&sub)?; + let sub_model_key = format!("/subscribers/{}", &sub.group_name); + let sub_offset_key = format!("/subscriber_offsets/{}", &sub.group_name); + batch.insert(sub_model_key.as_bytes(), sub_encoded.as_slice()); + batch.insert(sub_offset_key.as_bytes(), &utils::encode_u64(offset)); + subs.push((sub, offset)); + } + db.apply_batch(batch) + .context("error applying batch to write test data to stream")?; + subs.sort_by(|a, b| a.1.cmp(&b.1)); + Ok(subs) +} diff --git a/hadron-stream/src/stream/publisher.rs b/hadron-stream/src/stream/publisher.rs index f5e0609..c694174 100644 --- a/hadron-stream/src/stream/publisher.rs +++ b/hadron-stream/src/stream/publisher.rs @@ -1,5 +1,5 @@ use anyhow::{bail, Context, Result}; -use tokio::sync::oneshot; +use tokio::sync::{oneshot, watch}; use crate::error::{AppError, AppErrorExt, RpcResult, ShutdownError, ERR_DB_FLUSH}; use crate::grpc::{StreamPublishRequest, StreamPublishResponse}; @@ -11,7 +11,7 @@ impl StreamCtl { pub(super) async fn handle_publisher_request(&mut self, tx: oneshot::Sender>, data: StreamPublishRequest) { // Publish the new data frame. let _write_ack = data.ack; - let offset = match self.publish_data_frame(data).await { + let offset = match Self::publish_data_frame(&self.tree, &mut self.current_offset, &self.offset_signal, data).await { Ok(offset) => offset, Err(err) => { tracing::error!(error = ?err, "error while publishing data to stream"); @@ -33,8 +33,10 @@ impl StreamCtl { } /// Publish a frame of data to the target stream, returning the offset of the last entry written. - #[tracing::instrument(level = "trace", skip(self, req))] - async fn publish_data_frame(&mut self, req: StreamPublishRequest) -> Result { + #[tracing::instrument(level = "trace", skip(tree, current_offset, offset_signal, req))] + pub(super) async fn publish_data_frame( + tree: &sled::Tree, current_offset: &mut u64, offset_signal: &watch::Sender, req: StreamPublishRequest, + ) -> Result { tracing::debug!("writing data to stream"); if req.batch.is_empty() { bail!(AppError::InvalidInput("entries batch was empty, no-op".into())); @@ -43,26 +45,24 @@ impl StreamCtl { // Assign an offset to each entry in the payload and write as a batch. let mut batch = sled::Batch::default(); for new_event in req.batch { - self.current_offset += 1; + *current_offset += 1; let entry = utils::encode_model(&new_event).context("error encoding stream event record for storage")?; - batch.insert(&utils::encode_u64(self.current_offset), entry.as_slice()); + batch.insert(&utils::encode_u64(*current_offset), entry.as_slice()); } - self.tree - .apply_batch(batch) + tree.apply_batch(batch) .context("error applying write batch") .map_err(ShutdownError::from)?; // Fsync if requested. if req.fsync { - self.tree - .flush_async() + tree.flush_async() .await .context(ERR_DB_FLUSH) .map_err(ShutdownError::from)?; } - tracing::debug!(self.current_offset, "finished writing data to stream"); - let _ = self.offset_signal.send(self.current_offset); - Ok(self.current_offset) + tracing::debug!(current_offset, "finished writing data to stream"); + let _ = offset_signal.send(*current_offset); + Ok(*current_offset) } } diff --git a/hadron-stream/src/stream/publisher_test.rs b/hadron-stream/src/stream/publisher_test.rs new file mode 100644 index 0000000..ecb7736 --- /dev/null +++ b/hadron-stream/src/stream/publisher_test.rs @@ -0,0 +1,82 @@ +use anyhow::{Context, Result}; +use rand::prelude::*; +use tokio::sync::watch; + +use crate::config::Config; +use crate::database::Database; +use crate::error::AppError; +use crate::grpc::{Event, StreamPublishRequest}; +use crate::models::stream::Subscription; +use crate::utils; + +#[tokio::test] +async fn publish_data_frame_err_with_empty_batch() -> Result<()> { + let (config, _tmpdir) = Config::new_test()?; + let db = Database::new(config.clone()).await?; + let stream_tree = db.get_stream_tree().await?; + let mut current_offset = 0u64; + let (tx, rx) = watch::channel(current_offset); + let req = StreamPublishRequest { batch: vec![], fsync: true, ack: 0 }; + + let res = super::StreamCtl::publish_data_frame(&stream_tree, &mut current_offset, &tx, req).await; + + assert!(res.is_err(), "expected an error to be returned"); + let err = res.unwrap_err(); + let app_err = err.downcast::().context("unexpected error type")?; + assert!( + matches!(app_err, AppError::InvalidInput(val) if val == "entries batch was empty, no-op"), + "unexpected error returned", + ); + + Ok(()) +} + +#[tokio::test] +async fn publish_data_frame() -> Result<()> { + let (config, _tmpdir) = Config::new_test()?; + let db = Database::new(config.clone()).await?; + let stream_tree = db.get_stream_tree().await?; + let mut current_offset = 0u64; + let (tx, rx) = watch::channel(current_offset); + + let mut req = StreamPublishRequest { batch: vec![], fsync: true, ack: 0 }; + let (mut expected_events, expected_offset) = + (1u64..rand::thread_rng().gen_range(50u64..100u64)).fold((vec![], 0u64), |(mut events, _), offset| { + let event = Event::new_test(offset, "test", "empty"); + req.batch.push(event.clone()); + events.push(event); + (events, offset) + }); + expected_events.sort_by(|a, b| a.id.cmp(&b.id)); + + let last_offset = super::StreamCtl::publish_data_frame(&stream_tree, &mut current_offset, &tx, req).await?; + + let last_watcher_offset = *rx.borrow(); + assert!( + last_offset == expected_offset, + "expected offset to be {} got {}", + expected_offset, + last_offset + ); + assert!( + last_watcher_offset == expected_offset, + "expected watcher offset to be {} got {}", + expected_offset, + last_watcher_offset + ); + + let mut events = vec![]; + for kv_res in stream_tree.iter() { + let (_, val) = kv_res.context("error reading data from stream in test")?; + let val: Event = utils::decode_model(&val)?; + events.push(val); + } + events.sort_by(|a, b| a.id.cmp(&b.id)); + assert_eq!( + events, expected_events, + "unexpected data on stream\nexpected: {:?}\ngot: {:?}", + expected_events, events, + ); + + Ok(()) +} diff --git a/hadron-stream/src/stream/subscriber.rs b/hadron-stream/src/stream/subscriber.rs index 4dda770..f03a311 100644 --- a/hadron-stream/src/stream/subscriber.rs +++ b/hadron-stream/src/stream/subscriber.rs @@ -296,12 +296,12 @@ impl StreamSubCtl { .subscription .encode(&mut buf) .context("error encoding subscription record")?; - let stream_model_key = format!("{}{}", PREFIX_STREAM_SUBS, sub.group_name); - let stream_offset_key = format!("{}{}", PREFIX_STREAM_SUB_OFFSETS, sub.group_name); + let sub_model_key = format!("{}{}", PREFIX_STREAM_SUBS, sub.group_name); + let sub_offset_key = format!("{}{}", PREFIX_STREAM_SUB_OFFSETS, sub.group_name); let mut batch = sled::Batch::default(); - batch.insert(stream_model_key.as_bytes(), buf.freeze().as_ref()); - batch.insert(stream_offset_key.as_bytes(), &utils::encode_u64(offset)); + batch.insert(sub_model_key.as_bytes(), buf.freeze().as_ref()); + batch.insert(sub_offset_key.as_bytes(), &utils::encode_u64(offset)); self.tree_metadata .apply_batch(batch) .context("error writing subscription record and offset to disk") From a3f68014876752c68809426f11af2c609aa746c8 Mon Sep 17 00:00:00 2001 From: Anthony Dodd Date: Thu, 21 Oct 2021 21:24:31 -0500 Subject: [PATCH 02/10] Secondary index on Streams for recording timestamps of offsets This is a building block for implementing time-based compaction of streams. When batches are published, a timestamp is recorded in a secondary index pointing to the last offset of the batch. Time-based compaction will then be able to truncate data based on the offsets recorded in the secondary index. --- hadron-stream/Cargo.lock | 1 + hadron-stream/Cargo.toml | 1 + hadron-stream/src/config.rs | 6 +-- hadron-stream/src/database.rs | 11 ----- hadron-stream/src/pipeline/mod.rs | 10 ++-- hadron-stream/src/stream/mod.rs | 47 ++++++++++-------- hadron-stream/src/stream/mod_test.rs | 33 ++++++++----- hadron-stream/src/stream/publisher.rs | 10 ++-- hadron-stream/src/stream/publisher_test.rs | 51 +++++++++++++++++++- hadron-stream/src/stream/subscriber.rs | 50 ++++++++++++-------- hadron-stream/src/utils.rs | 55 +++++++++++++++++----- 11 files changed, 186 insertions(+), 89 deletions(-) diff --git a/hadron-stream/Cargo.lock b/hadron-stream/Cargo.lock index f394a9a..0903d4f 100644 --- a/hadron-stream/Cargo.lock +++ b/hadron-stream/Cargo.lock @@ -567,6 +567,7 @@ dependencies = [ "arc-swap", "base64 0.13.0", "bytes", + "chrono", "envy", "futures", "hadron-core", diff --git a/hadron-stream/Cargo.toml b/hadron-stream/Cargo.toml index aa7c411..dc4a207 100644 --- a/hadron-stream/Cargo.toml +++ b/hadron-stream/Cargo.toml @@ -14,6 +14,7 @@ anyhow = "1" arc-swap = "1" base64 = "0.13" bytes = "1" +chrono = "0.4" envy = "0.4" futures = "0.3" hadron-core = { path = "../hadron-core" } diff --git a/hadron-stream/src/config.rs b/hadron-stream/src/config.rs index 946275d..a4d7c33 100644 --- a/hadron-stream/src/config.rs +++ b/hadron-stream/src/config.rs @@ -1,7 +1,5 @@ //! Runtime configuration. -use std::sync::Arc; - use anyhow::{Context, Result}; use serde::Deserialize; @@ -54,10 +52,10 @@ impl Config { /// Build an instance for use in tests. #[cfg(test)] - pub fn new_test() -> Result<(Arc, tempfile::TempDir)> { + pub fn new_test() -> Result<(std::sync::Arc, tempfile::TempDir)> { let tmpdir = tempfile::tempdir_in("/tmp").context("error creating tmp dir in /tmp")?; Ok(( - Arc::new(Self { + std::sync::Arc::new(Self { rust_log: "".into(), client_port: 7000, server_port: 7000, diff --git a/hadron-stream/src/database.rs b/hadron-stream/src/database.rs index 1e65654..9299d6f 100644 --- a/hadron-stream/src/database.rs +++ b/hadron-stream/src/database.rs @@ -15,8 +15,6 @@ pub type Tree = sled::Tree; pub const DEFAULT_DATA_PATH: &str = "/usr/local/hadron/db"; /// The DB tree prefix used for the partition stream. const TREE_STREAM: &str = "stream"; -/// The DB tree prefix used for the partition stream's metadata. -const TREE_STREAM_METADATA: &str = "stream_metadata"; /// The DB tree prefix used for pipelines. const TREE_PIPELINE_PREFIX: &str = "pipelines"; /// The DB tree prefix used for pipeline metadata. @@ -80,15 +78,6 @@ impl Database { Ok(tree) } - /// Get a handle to the DB tree for a stream partition's metadata. - pub async fn get_stream_tree_metadata(&self) -> ShutdownResult { - let (db, ivname) = (self.inner.db.clone(), IVec::from(TREE_STREAM_METADATA)); - let tree = Self::spawn_blocking(move || -> Result { Ok(db.open_tree(ivname)?) }) - .await - .and_then(|res| res.map_err(|err| ShutdownError(anyhow!("could not open DB tree {} {}", TREE_STREAM_METADATA, err))))?; - Ok(tree) - } - /// Get a handle to the DB tree for a pipeline partition. pub async fn get_pipeline_tree(&self, name: &str) -> ShutdownResult { let name = format!("{}/{}", TREE_PIPELINE_PREFIX, name); diff --git a/hadron-stream/src/pipeline/mod.rs b/hadron-stream/src/pipeline/mod.rs index 939e84c..7841f1e 100644 --- a/hadron-stream/src/pipeline/mod.rs +++ b/hadron-stream/src/pipeline/mod.rs @@ -34,11 +34,11 @@ const DEFAULT_MAX_PARALLEL: u32 = 50; const KEY_LAST_OFFSET_PROCESSED: &str = "/meta/last_offset_processed"; /// A metadata key prefix used for tracking active pipeline instances. /// -/// Active instances are keyed as `/a/{offset}` where `{offset}` is the offset +/// Active instances are keyed as `a{offset}` where `{offset}` is the offset /// of the event from the source stream. The value is the offset. -const PREFIX_META_ACTIVE_INSTANCES: &[u8; 3] = b"/a/"; +const PREFIX_META_ACTIVE_INSTANCES: &[u8; 1] = b"a"; /// The key prefix under which pipeline stage outputs are stored. -const PREFIX_PIPELINE_STAGE_OUTPUTS: &[u8; 3] = b"/s/"; +const PREFIX_PIPELINE_STAGE_OUTPUTS: &[u8; 1] = b"o"; /// The liveness stream type used by a pipeline controller. type PipelineLivenessStream = LivenessStream, PipelineSubscribeRequest>; @@ -596,7 +596,7 @@ impl PipelineCtl { } // Construct pipeline instance & add to batch. - metadata_batch.insert(&utils::encode_3_byte_prefix(PREFIX_META_ACTIVE_INSTANCES, offset), &key); + metadata_batch.insert(&utils::encode_byte_prefix(PREFIX_META_ACTIVE_INSTANCES, offset), &key); let inst = ActivePipelineInstance { root_event, root_event_offset: offset, @@ -677,7 +677,7 @@ async fn recover_pipeline_state( // Iterate over all outputs currently recorded for this pipeline instance. // See `try_record_delivery_response`, these are keyed as `/s/{offset}/{stage_name}`. let mut outputs = HashMap::new(); - for iter_res in pipeline_tree.scan_prefix(&utils::encode_3_byte_prefix(PREFIX_PIPELINE_STAGE_OUTPUTS, offset)) { + for iter_res in pipeline_tree.scan_prefix(&utils::encode_byte_prefix(PREFIX_PIPELINE_STAGE_OUTPUTS, offset)) { let (key, val) = iter_res.context(ERR_ITER_FAILURE)?; let key = std::str::from_utf8(&key).context("data corruption: all keys should be valid utf8")?; let stage = key.split('/').last().unwrap_or(""); diff --git a/hadron-stream/src/stream/mod.rs b/hadron-stream/src/stream/mod.rs index 99eba26..b3e0985 100644 --- a/hadron-stream/src/stream/mod.rs +++ b/hadron-stream/src/stream/mod.rs @@ -49,10 +49,25 @@ use crate::models::stream::Subscription; use crate::stream::subscriber::StreamSubCtlMsg; use crate::utils; +/* +TODO: +- [x] simplify PREFIX_STREAM_SUBS & PREFIX_STREAM_SUB_OFFSETS with minimal `{byte}` prefixes + and use IVec::from to build IVec keys. +- store the stream's last written offset under the KEY_STREAM_LAST_WRITTEN_OFFSET key. +- store all stream entries using a `{byte}` prefix. +- [x] collapse the metadata stream into the standard stream tree. +*/ + +/// The key prefix used for storing stream events. +const PREFIX_STREAM_EVENT: &[u8; 1] = b"e"; +/// The key prefix used for storing stream event timestamps. +const PREFIX_STREAM_TS: &[u8; 1] = b"t"; /// The database key prefix used for storing stream subscriber data. -const PREFIX_STREAM_SUBS: &str = "/subscribers/"; -/// The database key prefix used for storing stream subscriber data. -const PREFIX_STREAM_SUB_OFFSETS: &str = "/subscriber_offsets/"; +const PREFIX_STREAM_SUBS: &[u8; 1] = b"s"; +/// The database key prefix used for storing stream subscriber offsets. +const PREFIX_STREAM_SUB_OFFSETS: &[u8; 1] = b"o"; +/// The key used to store the last written offset for the stream. +const KEY_STREAM_LAST_WRITTEN_OFFSET: &[u8; 1] = b"l"; const ERR_DECODING_STREAM_META_GROUP_NAME: &str = "error decoding stream meta group name from storage"; @@ -64,8 +79,6 @@ pub struct StreamCtl { _db: Database, /// This stream's database tree. tree: Tree, - /// This stream's database tree for metadata storage. - _tree_metadata: Tree, /// The stream partition of this controller. partition: u32, @@ -97,8 +110,7 @@ impl StreamCtl { // Recover stream state. let partition = config.partition; let tree = db.get_stream_tree().await?; - let tree_metadata = db.get_stream_tree_metadata().await?; - let (current_offset, subs) = recover_stream_state(&tree, &tree_metadata).await?; + let (current_offset, subs) = recover_stream_state(tree.clone()).await?; // Spawn the subscriber controller. let (offset_signal, offset_signal_rx) = watch::channel(current_offset); @@ -107,7 +119,6 @@ impl StreamCtl { config.clone(), db.clone(), tree.clone(), - tree_metadata.clone(), partition, shutdown_tx.clone(), subs_tx.clone(), @@ -123,7 +134,6 @@ impl StreamCtl { config, _db: db, tree, - _tree_metadata: tree_metadata, partition, requests: ReceiverStream::new(requests), subs_tx, @@ -189,25 +199,24 @@ impl StreamCtl { } /// Recover this stream's last recorded state. -async fn recover_stream_state(stream_tree: &Tree, metadata_tree: &Tree) -> Result<(u64, Vec<(Subscription, u64)>)> { - let (stream_tree, metadata_tree) = (stream_tree.clone(), metadata_tree.clone()); +async fn recover_stream_state(tree: Tree) -> Result<(u64, Vec<(Subscription, u64)>)> { let val = Database::spawn_blocking(move || -> Result<(u64, Vec<(Subscription, u64)>)> { // Fetch next offset info. - let kv_opt = stream_tree - .last() + let offset_opt = tree + .get(KEY_STREAM_LAST_WRITTEN_OFFSET) .context("error fetching next offset key during recovery")?; - let last_written_offset = kv_opt - .map(|(key, _val)| utils::decode_u64(&key).context("error decoding offset value from storage")) + let last_written_offset = offset_opt + .map(|val| utils::decode_u64(&val).context("error decoding offset value from storage")) .transpose()? .unwrap_or(0); // Fetch all stream subscriber info. let mut subs = HashMap::new(); - for entry_res in metadata_tree.scan_prefix(PREFIX_STREAM_SUBS) { + for entry_res in tree.scan_prefix(PREFIX_STREAM_SUBS) { let (key, val) = entry_res.context(ERR_ITER_FAILURE)?; let group_name = std::str::from_utf8(key.as_ref()) .context(ERR_DECODING_STREAM_META_GROUP_NAME)? - .strip_prefix(PREFIX_STREAM_SUBS) + .strip_prefix(unsafe { std::str::from_utf8_unchecked(PREFIX_STREAM_SUBS) }) .unwrap_or("") .to_string(); let sub = Subscription::decode(val.as_ref()).context("error decoding subscriber record from storage")?; @@ -215,11 +224,11 @@ async fn recover_stream_state(stream_tree: &Tree, metadata_tree: &Tree) -> Resul } // Fetch all stream subscriber offsets. - for entry_res in metadata_tree.scan_prefix(PREFIX_STREAM_SUB_OFFSETS) { + for entry_res in tree.scan_prefix(PREFIX_STREAM_SUB_OFFSETS) { let (key, entry) = entry_res.context(ERR_ITER_FAILURE)?; let group_name = std::str::from_utf8(key.as_ref()) .context(ERR_DECODING_STREAM_META_GROUP_NAME)? - .strip_prefix(PREFIX_STREAM_SUB_OFFSETS) + .strip_prefix(unsafe { std::str::from_utf8_unchecked(PREFIX_STREAM_SUB_OFFSETS) }) .unwrap_or(""); let offset = utils::decode_u64(entry.as_ref()).context("error decoding stream offset from storage")?; if let Some((_sub, offset_val)) = subs.get_mut(group_name) { diff --git a/hadron-stream/src/stream/mod_test.rs b/hadron-stream/src/stream/mod_test.rs index a56406a..3759b57 100644 --- a/hadron-stream/src/stream/mod_test.rs +++ b/hadron-stream/src/stream/mod_test.rs @@ -5,6 +5,7 @@ use crate::config::Config; use crate::database::Database; use crate::grpc::Event; use crate::models::stream::Subscription; +use crate::stream::{KEY_STREAM_LAST_WRITTEN_OFFSET, PREFIX_STREAM_EVENT, PREFIX_STREAM_SUBS, PREFIX_STREAM_SUB_OFFSETS}; use crate::utils; #[tokio::test] @@ -12,9 +13,8 @@ async fn recover_stream_state_empty_state() -> Result<()> { let (config, _tmpdir) = Config::new_test()?; let db = Database::new(config.clone()).await?; let stream_tree = db.get_stream_tree().await?; - let metadata_tree = db.get_stream_tree_metadata().await?; - let (offset, subscriptions) = super::recover_stream_state(&stream_tree, &metadata_tree).await?; + let (offset, subscriptions) = super::recover_stream_state(stream_tree).await?; assert!(offset == 0, "expected offset to be 0 got {}", offset); assert!(subscriptions.is_empty(), "expected subscriptions len to be 0 got {}", offset); @@ -27,11 +27,10 @@ async fn recover_stream_state_with_previous_state() -> Result<()> { let (config, _tmpdir) = Config::new_test()?; let db = Database::new(config.clone()).await?; let stream_tree = db.get_stream_tree().await?; - let metadata_tree = db.get_stream_tree_metadata().await?; let expected_offset = setup_stream_data(&stream_tree).await?; - let (offset, subs) = super::recover_stream_state(&stream_tree, &metadata_tree).await?; + let (offset, subs) = super::recover_stream_state(stream_tree).await?; assert!(offset == expected_offset, "expected offset to be {} got {}", expected_offset, offset); assert!(subs.is_empty(), "expected subscriptions len to be 0 got {}", offset); @@ -44,12 +43,11 @@ async fn recover_stream_state_with_previous_state_and_subs() -> Result<()> { let (config, _tmpdir) = Config::new_test()?; let db = Database::new(config.clone()).await?; let stream_tree = db.get_stream_tree().await?; - let metadata_tree = db.get_stream_tree_metadata().await?; let expected_offset = setup_stream_data(&stream_tree).await?; - let expected_subs = setup_subs_data(&metadata_tree).await?; + let expected_subs = setup_subs_data(&stream_tree).await?; - let (offset, mut subs) = super::recover_stream_state(&stream_tree, &metadata_tree).await?; + let (offset, mut subs) = super::recover_stream_state(stream_tree).await?; subs.sort_by(|a, b| a.1.cmp(&b.1)); assert!(offset == expected_offset, "expected offset to be {} got {}", expected_offset, offset); @@ -67,9 +65,10 @@ async fn setup_stream_data(db: &sled::Tree) -> Result { for offset in 0..rand::thread_rng().gen_range(50..100) { let event = Event::new_test(offset, "test", "empty"); let event_bytes = utils::encode_model(&event)?; - batch.insert(&utils::encode_u64(offset), event_bytes.as_slice()); + batch.insert(&utils::encode_byte_prefix(PREFIX_STREAM_EVENT, offset), event_bytes.as_slice()); last_offset = offset; } + batch.insert(KEY_STREAM_LAST_WRITTEN_OFFSET, &utils::encode_u64(last_offset)); db.apply_batch(batch) .context("error applying batch to write test data to stream")?; Ok(last_offset) @@ -82,10 +81,20 @@ async fn setup_subs_data(db: &sled::Tree) -> Result> { for offset in 0..rand::thread_rng().gen_range(50..100) { let sub = Subscription { group_name: offset.to_string(), max_batch_size: 50 }; let sub_encoded = utils::encode_model(&sub)?; - let sub_model_key = format!("/subscribers/{}", &sub.group_name); - let sub_offset_key = format!("/subscriber_offsets/{}", &sub.group_name); - batch.insert(sub_model_key.as_bytes(), sub_encoded.as_slice()); - batch.insert(sub_offset_key.as_bytes(), &utils::encode_u64(offset)); + let sub_model_key = utils::ivec_from_iter( + PREFIX_STREAM_SUBS + .iter() + .copied() + .chain(sub.group_name.as_bytes().iter().copied()), + ); + let sub_offset_key = utils::ivec_from_iter( + PREFIX_STREAM_SUB_OFFSETS + .iter() + .copied() + .chain(sub.group_name.as_bytes().iter().copied()), + ); + batch.insert(sub_model_key, sub_encoded.as_slice()); + batch.insert(sub_offset_key, &utils::encode_u64(offset)); subs.push((sub, offset)); } db.apply_batch(batch) diff --git a/hadron-stream/src/stream/publisher.rs b/hadron-stream/src/stream/publisher.rs index c694174..e4657ee 100644 --- a/hadron-stream/src/stream/publisher.rs +++ b/hadron-stream/src/stream/publisher.rs @@ -3,7 +3,7 @@ use tokio::sync::{oneshot, watch}; use crate::error::{AppError, AppErrorExt, RpcResult, ShutdownError, ERR_DB_FLUSH}; use crate::grpc::{StreamPublishRequest, StreamPublishResponse}; -use crate::stream::StreamCtl; +use crate::stream::{StreamCtl, KEY_STREAM_LAST_WRITTEN_OFFSET, PREFIX_STREAM_EVENT, PREFIX_STREAM_TS}; use crate::utils; impl StreamCtl { @@ -42,13 +42,17 @@ impl StreamCtl { bail!(AppError::InvalidInput("entries batch was empty, no-op".into())); } - // Assign an offset to each entry in the payload and write as a batch. + // Assign an offset to each event in the batch, and record a timestamp in a secondary + // index for the last offset in the batch. + let ts = chrono::Utc::now().timestamp_millis(); let mut batch = sled::Batch::default(); for new_event in req.batch { *current_offset += 1; let entry = utils::encode_model(&new_event).context("error encoding stream event record for storage")?; - batch.insert(&utils::encode_u64(*current_offset), entry.as_slice()); + batch.insert(&utils::encode_byte_prefix(PREFIX_STREAM_EVENT, *current_offset), entry.as_slice()); } + batch.insert(&utils::encode_byte_prefix_i64(PREFIX_STREAM_TS, ts), &utils::encode_u64(*current_offset)); + batch.insert(KEY_STREAM_LAST_WRITTEN_OFFSET, &utils::encode_u64(*current_offset)); tree.apply_batch(batch) .context("error applying write batch") .map_err(ShutdownError::from)?; diff --git a/hadron-stream/src/stream/publisher_test.rs b/hadron-stream/src/stream/publisher_test.rs index ecb7736..f6940ca 100644 --- a/hadron-stream/src/stream/publisher_test.rs +++ b/hadron-stream/src/stream/publisher_test.rs @@ -6,9 +6,11 @@ use crate::config::Config; use crate::database::Database; use crate::error::AppError; use crate::grpc::{Event, StreamPublishRequest}; -use crate::models::stream::Subscription; +use crate::stream::{KEY_STREAM_LAST_WRITTEN_OFFSET, PREFIX_STREAM_EVENT}; use crate::utils; +use super::PREFIX_STREAM_TS; + #[tokio::test] async fn publish_data_frame_err_with_empty_batch() -> Result<()> { let (config, _tmpdir) = Config::new_test()?; @@ -20,6 +22,12 @@ async fn publish_data_frame_err_with_empty_batch() -> Result<()> { let res = super::StreamCtl::publish_data_frame(&stream_tree, &mut current_offset, &tx, req).await; + let last_watcher_offset = *rx.borrow(); + assert_eq!( + last_watcher_offset, current_offset, + "expected watcher offset to be {} got {}", + current_offset, last_watcher_offset, + ); assert!(res.is_err(), "expected an error to be returned"); let err = res.unwrap_err(); let app_err = err.downcast::().context("unexpected error type")?; @@ -38,6 +46,7 @@ async fn publish_data_frame() -> Result<()> { let stream_tree = db.get_stream_tree().await?; let mut current_offset = 0u64; let (tx, rx) = watch::channel(current_offset); + let expected_ts_min = chrono::Utc::now().timestamp_millis() - 5; let mut req = StreamPublishRequest { batch: vec![], fsync: true, ack: 0 }; let (mut expected_events, expected_offset) = @@ -51,6 +60,7 @@ async fn publish_data_frame() -> Result<()> { let last_offset = super::StreamCtl::publish_data_frame(&stream_tree, &mut current_offset, &tx, req).await?; + // Check emitted info on last offset. let last_watcher_offset = *rx.borrow(); assert!( last_offset == expected_offset, @@ -65,8 +75,9 @@ async fn publish_data_frame() -> Result<()> { last_watcher_offset ); + // Check all written events. let mut events = vec![]; - for kv_res in stream_tree.iter() { + for kv_res in stream_tree.scan_prefix(PREFIX_STREAM_EVENT) { let (_, val) = kv_res.context("error reading data from stream in test")?; let val: Event = utils::decode_model(&val)?; events.push(val); @@ -78,5 +89,41 @@ async fn publish_data_frame() -> Result<()> { expected_events, events, ); + // Check storage for the last offset key. + let db_offset_ivec = stream_tree + .get(KEY_STREAM_LAST_WRITTEN_OFFSET) + .context("error fetching last written offset from storage")? + .context("no value found for last written offset")?; + let db_offset = utils::decode_u64(&db_offset_ivec)?; + assert_eq!( + db_offset, expected_offset, + "expected db last written offset to be {} got {}", + expected_offset, db_offset + ); + + // Check for secondary timestamp index. + let ts_idx = stream_tree + .scan_prefix(PREFIX_STREAM_TS) + .try_fold(vec![], |mut acc, kv_res| -> Result> { + let (key, val) = kv_res.context("error scanning stream timestamp index")?; + let ts = utils::decode_i64(&key[1..])?; + let offset = utils::decode_u64(&val)?; + acc.push((ts, offset)); + Ok(acc) + }) + .context("error reading stream timestamp index")?; + assert_eq!(ts_idx.len(), 1, "expected one timestamp index entry, got {}", ts_idx.len()); + assert_eq!( + ts_idx[0].1, expected_offset, + "expected timestamp index entry to point to offset {} got {}", + expected_offset, ts_idx[0].1 + ); + assert!( + ts_idx[0].0 > expected_ts_min, + "expected stream index entry timestamp {} to be > {}", + expected_ts_min, + ts_idx[0].0 + ); + Ok(()) } diff --git a/hadron-stream/src/stream/subscriber.rs b/hadron-stream/src/stream/subscriber.rs index f03a311..7544238 100644 --- a/hadron-stream/src/stream/subscriber.rs +++ b/hadron-stream/src/stream/subscriber.rs @@ -43,8 +43,6 @@ pub struct StreamSubCtl { _db: Database, /// This stream's database tree for storing stream records. tree: Tree, - /// This stream's database tree for metadata storage. - tree_metadata: Tree, /// The stream partition of this controller. partition: u32, @@ -76,16 +74,14 @@ pub struct StreamSubCtl { impl StreamSubCtl { /// Create a new instance. pub fn new( - config: Arc, db: Database, tree: Tree, tree_metadata: Tree, partition: u32, shutdown_tx: broadcast::Sender<()>, - events_tx: mpsc::Sender, events_rx: mpsc::Receiver, stream_offset: watch::Receiver, - subs: Vec<(Subscription, u64)>, current_offset: u64, + config: Arc, db: Database, tree: Tree, partition: u32, shutdown_tx: broadcast::Sender<()>, events_tx: mpsc::Sender, + events_rx: mpsc::Receiver, stream_offset: watch::Receiver, subs: Vec<(Subscription, u64)>, current_offset: u64, ) -> Self { let subs = SubscriberInfo::new(subs); Self { config, _db: db, tree, - tree_metadata, partition, subs, current_offset, @@ -296,17 +292,28 @@ impl StreamSubCtl { .subscription .encode(&mut buf) .context("error encoding subscription record")?; - let sub_model_key = format!("{}{}", PREFIX_STREAM_SUBS, sub.group_name); - let sub_offset_key = format!("{}{}", PREFIX_STREAM_SUB_OFFSETS, sub.group_name); + + let sub_model_key = utils::ivec_from_iter( + PREFIX_STREAM_SUBS + .iter() + .copied() + .chain(sub.group_name.as_bytes().iter().copied()), + ); + let sub_offset_key = utils::ivec_from_iter( + PREFIX_STREAM_SUB_OFFSETS + .iter() + .copied() + .chain(sub.group_name.as_bytes().iter().copied()), + ); let mut batch = sled::Batch::default(); - batch.insert(sub_model_key.as_bytes(), buf.freeze().as_ref()); - batch.insert(sub_offset_key.as_bytes(), &utils::encode_u64(offset)); - self.tree_metadata + batch.insert(sub_model_key, buf.freeze().as_ref()); + batch.insert(sub_offset_key, &utils::encode_u64(offset)); + self.tree .apply_batch(batch) .context("error writing subscription record and offset to disk") .map_err(ShutdownError::from)?; - self.tree_metadata + self.tree .flush_async() .await .context(ERR_DB_FLUSH) @@ -415,7 +422,7 @@ impl StreamSubCtl { _ => Err("unexpected or malformed response returned from subscriber, expected ack or nack".into()), }; if group.durable { - Self::try_record_delivery_response(record_res, group.group_name.clone(), self.tree_metadata.clone()) + Self::try_record_delivery_response(record_res, group.group_name.clone(), self.tree.clone()) .await .context("error while recording subscriber delivery response")?; } @@ -423,8 +430,8 @@ impl StreamSubCtl { } /// Record the ack/nack response from a subscriber delivery. - #[tracing::instrument(level = "trace", skip(res, group_name, tree_metadata))] - async fn try_record_delivery_response(res: std::result::Result, group_name: Arc, tree_metadata: Tree) -> ShutdownResult<()> { + #[tracing::instrument(level = "trace", skip(res, group_name, tree))] + async fn try_record_delivery_response(res: std::result::Result, group_name: Arc, tree: Tree) -> ShutdownResult<()> { let offset = match res { Ok(offset) => offset, Err(_err) => { @@ -432,13 +439,16 @@ impl StreamSubCtl { return Ok(()); } }; - let key = format!("{}{}", PREFIX_STREAM_SUB_OFFSETS, &*group_name); - tree_metadata - .insert(key.as_bytes(), &utils::encode_u64(offset)) + let key = utils::ivec_from_iter( + PREFIX_STREAM_SUB_OFFSETS + .iter() + .copied() + .chain(group_name.as_bytes().iter().copied()), + ); + tree.insert(key, &utils::encode_u64(offset)) .context("error updating subscription offsets on disk") .map_err(ShutdownError::from)?; - tree_metadata - .flush_async() + tree.flush_async() .await .context(ERR_DB_FLUSH) .map_err(ShutdownError::from)?; diff --git a/hadron-stream/src/utils.rs b/hadron-stream/src/utils.rs index a4eaf5e..1f088a1 100644 --- a/hadron-stream/src/utils.rs +++ b/hadron-stream/src/utils.rs @@ -1,34 +1,58 @@ #![allow(dead_code)] +use std::iter::FromIterator; + use anyhow::{bail, Context, Result}; use prost::Message; +use sled::IVec; /// The CloudEvents spec version currently being used. pub const CLOUD_EVENTS_SPEC_VERSION: &str = "1.0"; -/// Encode a stream entry key using the given prefix & offset. -pub fn encode_3_byte_prefix(prefix: &[u8; 3], offset: u64) -> [u8; 12] { - let mut key = [0u8; 12]; - match prefix { - [b0, b1, b2] => { - key[0] = *b0; - key[1] = *b1; - key[2] = *b2; +/// Encode a byte + u64 prefix key. +/// +/// This allows for efficient BTree prefix storage without the overhead of allocating additional +/// vectors, strings or other sorts of buffers. +/// +/// NOTE: if any data in a tree is encoded with a prefix, then all data in that stream will need +/// to be encoded with a well-defined prefix as well in order to avoid unintended collisions +/// and or data corruption. +pub fn encode_byte_prefix(prefix: &[u8; 1], offset: u64) -> [u8; 9] { + let mut key = [0u8; 9]; + key[0] = prefix[0]; + match encode_u64(offset) { + [b1, b2, b3, b4, b5, b6, b7, b8] => { + key[1] = b1; + key[2] = b2; + key[3] = b3; + key[4] = b4; + key[5] = b5; + key[6] = b6; + key[7] = b7; + key[8] = b8; } } - match encode_u64(offset) { - [b3, b4, b5, b6, b7, b8, b9, b10] => { + key +} + +/// Encode a byte + i64 prefix key. +/// +/// See `encode_byte_prefix` for more details. +pub fn encode_byte_prefix_i64(prefix: &[u8; 1], offset: i64) -> [u8; 9] { + let mut key = [0u8; 9]; + key[0] = prefix[0]; + match encode_i64(offset) { + [b1, b2, b3, b4, b5, b6, b7, b8] => { + key[1] = b1; + key[2] = b2; key[3] = b3; key[4] = b4; key[5] = b5; key[6] = b6; key[7] = b7; key[8] = b8; - key[9] = b9; - key[10] = b10; } } - key[11] = b'/'; key } @@ -69,3 +93,8 @@ pub fn encode_model(model: &M) -> Result> { pub fn decode_model(data: &[u8]) -> Result { M::decode(data).context("error decoding object from storage") } + +/// Encode the given bytes iterator as an IVec. +pub fn ivec_from_iter>(data: T) -> IVec { + IVec::from_iter(data) +} From e035cc1d35f9b46642c2065a50b7bec08d3d785e Mon Sep 17 00:00:00 2001 From: Anthony Dodd Date: Sun, 24 Oct 2021 10:05:50 -0500 Subject: [PATCH 03/10] Update scheduler, pipeline indexing refactor & tests This commit updates the scheduler to ensure that a Stream's generated StatefulSet targets its corresponding headless service correctly. The pipelines module has been refactored for more optimal storage pattern, which has removed the need for a parallel metadata tree for the pipeline. The utils module has been thoroughly tested which now guards against regressions in our data storage and indexing strategy. This also demonstrates the expected behavior of lexicographically ordered range scans and prefix scans. --- hadron-operator/src/k8s/scheduler.rs | 7 ++- hadron-stream/src/database.rs | 12 ---- hadron-stream/src/main.rs | 2 + hadron-stream/src/pipeline/mod.rs | 36 +++++------ hadron-stream/src/stream/mod.rs | 21 ++++--- hadron-stream/src/utils_test.rs | 92 ++++++++++++++++++++++++++++ 6 files changed, 129 insertions(+), 41 deletions(-) create mode 100644 hadron-stream/src/utils_test.rs diff --git a/hadron-operator/src/k8s/scheduler.rs b/hadron-operator/src/k8s/scheduler.rs index cc5bb4e..5fe36f1 100644 --- a/hadron-operator/src/k8s/scheduler.rs +++ b/hadron-operator/src/k8s/scheduler.rs @@ -379,7 +379,7 @@ impl Controller { match self.streams.get(name.as_ref()) { None => (), Some(stream) => { - let mut new_sts = self.build_stream_statefulset(&stream); + let mut new_sts = self.build_stream_statefulset(stream); new_sts = match self.create_statefulset(new_sts).await { Ok(new_sts) => new_sts, Err(err) => { @@ -486,6 +486,8 @@ impl Controller { let selector = spec.selector.get_or_insert_with(Default::default); set_cannonical_labels(selector); selector.insert(LABEL_HADRON_RS_STREAM.into(), stream.name().into()); + spec.cluster_ip = Some("None".into()); + spec.type_ = Some("ClusterIP".into()); spec.ports = Some(vec![ ServicePort { name: Some("client-port".into()), @@ -608,6 +610,7 @@ impl Controller { type_: Some("RollingUpdate".into()), rolling_update: None, }); + spec.service_name = stream.name().into(); spec.replicas = Some(stream.spec.partitions as i32); spec.selector = LabelSelector { match_labels: Some(labels.clone()), @@ -826,7 +829,7 @@ impl Controller { } let api: Api = Api::namespaced(self.client.clone(), &self.config.namespace); let params = kube::api::PostParams::default(); - timeout(API_TIMEOUT, api.create(¶ms, &secret)) + timeout(API_TIMEOUT, api.create(¶ms, secret)) .await .context("timeout while creating secret")? .context("error creating secret") diff --git a/hadron-stream/src/database.rs b/hadron-stream/src/database.rs index 9299d6f..8b0ea60 100644 --- a/hadron-stream/src/database.rs +++ b/hadron-stream/src/database.rs @@ -17,8 +17,6 @@ pub const DEFAULT_DATA_PATH: &str = "/usr/local/hadron/db"; const TREE_STREAM: &str = "stream"; /// The DB tree prefix used for pipelines. const TREE_PIPELINE_PREFIX: &str = "pipelines"; -/// The DB tree prefix used for pipeline metadata. -const TREE_PIPELINE_METADATA: &str = "pipelines_metadata"; /// The default path to use for data storage. pub fn default_data_path() -> String { @@ -87,14 +85,4 @@ impl Database { .and_then(|res| res.map_err(|err| ShutdownError(anyhow!("could not open DB tree {} {}", &name, err))))?; Ok(tree) } - - /// Get a handle to the DB tree for a pipeline partition's metadata. - pub async fn get_pipeline_tree_metadata(&self, name: &str) -> ShutdownResult { - let name = format!("{}/{}", TREE_PIPELINE_METADATA, name); - let (db, ivname) = (self.inner.db.clone(), IVec::from(name.as_str())); - let tree = Self::spawn_blocking(move || -> Result { Ok(db.open_tree(ivname)?) }) - .await - .and_then(|res| res.map_err(|err| ShutdownError(anyhow!("could not open DB tree {} {}", &name, err))))?; - Ok(tree) - } } diff --git a/hadron-stream/src/main.rs b/hadron-stream/src/main.rs index 346c1bf..244d45b 100644 --- a/hadron-stream/src/main.rs +++ b/hadron-stream/src/main.rs @@ -11,6 +11,8 @@ mod pipeline; mod server; mod stream; mod utils; +#[cfg(test)] +mod utils_test; mod watchers; use std::io::Write; diff --git a/hadron-stream/src/pipeline/mod.rs b/hadron-stream/src/pipeline/mod.rs index 7841f1e..2a64278 100644 --- a/hadron-stream/src/pipeline/mod.rs +++ b/hadron-stream/src/pipeline/mod.rs @@ -31,7 +31,7 @@ const DEFAULT_MAX_PARALLEL: u32 = 50; /// /// NOTE: this does not necessarily indicate that the pipeline for this key has actually been /// executed, but only that it has been prepared for execution. -const KEY_LAST_OFFSET_PROCESSED: &str = "/meta/last_offset_processed"; +const KEY_LAST_OFFSET_PROCESSED: &[u8; 1] = b"l"; /// A metadata key prefix used for tracking active pipeline instances. /// /// Active instances are keyed as `a{offset}` where `{offset}` is the offset @@ -53,8 +53,6 @@ pub struct PipelineCtl { _db: Database, /// The database tree for storing this pipeline's instance records. tree: Tree, - /// The database tree for this pipeline's metadata storage. - tree_metadata: Tree, /// The database tree of this pipeline's source stream; which is only ever used for reading. tree_stream: Tree, /// The data model of the pipeline with which this controller is associated. @@ -97,17 +95,15 @@ impl PipelineCtl { shutdown_tx: broadcast::Sender<()>, events_tx: mpsc::Sender, events_rx: mpsc::Receiver, ) -> Result { let tree = db.get_pipeline_tree(pipeline.name()).await?; - let tree_metadata = db.get_pipeline_tree_metadata(pipeline.name()).await?; let stream_tree = db.get_stream_tree().await?; let stream_offset = *stream_signal.borrow(); let (last_offset_processed, active_pipelines) = - recover_pipeline_state(&tree, &tree_metadata, &stream_tree, pipeline.clone(), stream_offset).await?; + recover_pipeline_state(tree.clone(), stream_tree.clone(), pipeline.clone(), stream_offset).await?; Ok(Self { config, _db: db, tree, - tree_metadata, tree_stream: stream_tree, pipeline, partition, @@ -564,7 +560,7 @@ impl PipelineCtl { self.is_fetching_stream_data = true; tokio::spawn(Self::try_fetch_stream_data( self.tree_stream.clone(), - self.tree_metadata.clone(), + self.tree.clone(), self.last_offset_processed, self.pipeline.clone(), self.max_parallel(), @@ -572,16 +568,19 @@ impl PipelineCtl { )); } - #[tracing::instrument(level = "trace", skip(tree_stream, tree_metadata, last_offset_processed, pipeline, max_parallel, tx))] + #[tracing::instrument(level = "trace", skip(tree_stream, tree_pipeline, last_offset_processed, pipeline, max_parallel, tx))] async fn try_fetch_stream_data( - tree_stream: Tree, tree_metadata: Tree, last_offset_processed: u64, pipeline: Arc, max_parallel: u32, + tree_stream: Tree, tree_pipeline: Tree, last_offset_processed: u64, pipeline: Arc, max_parallel: u32, tx: mpsc::Sender, ) { tracing::debug!("fetching stream data for pipeline"); let data_res = Database::spawn_blocking(move || -> Result { // Iterate over the records of the stream up to the maximum parallel allowed. - let (mut metadata_batch, mut new_instances) = (sled::Batch::default(), Vec::with_capacity(max_parallel as usize)); + let (mut pipeline_batch, mut new_instances) = (sled::Batch::default(), Vec::with_capacity(max_parallel as usize)); let (start, mut last_processed, mut count) = (&utils::encode_u64(last_offset_processed + 1), last_offset_processed, 0); + // TODO: need to iterate the tree correctly only scanning events. + // TODO: ensure the pipeline tree is used correctly below with prefixes. + // TODO: store the root event in the pipeline as well, to avoid issues with compaction. let stream = tree_stream.range::<_, std::ops::RangeFrom<&[u8]>>(start..); for event_res in stream { // Decode the records offset. @@ -596,7 +595,7 @@ impl PipelineCtl { } // Construct pipeline instance & add to batch. - metadata_batch.insert(&utils::encode_byte_prefix(PREFIX_META_ACTIVE_INSTANCES, offset), &key); + pipeline_batch.insert(&utils::encode_byte_prefix(PREFIX_META_ACTIVE_INSTANCES, offset), &key); let inst = ActivePipelineInstance { root_event, root_event_offset: offset, @@ -613,9 +612,9 @@ impl PipelineCtl { } // Apply the batch of changes. - metadata_batch.insert(KEY_LAST_OFFSET_PROCESSED, &utils::encode_u64(last_processed)); - let _res = tree_metadata - .apply_batch(metadata_batch) + pipeline_batch.insert(KEY_LAST_OFFSET_PROCESSED, &utils::encode_u64(last_processed)); + let _res = tree_pipeline + .apply_batch(pipeline_batch) .context("error applying metadata batch while fetching stream data for pipeline")?; Ok(FetchStreamRecords { @@ -641,12 +640,11 @@ impl PipelineCtl { /// - The output of each stage of a pipeline instance is recorded under `/s/{instance}/{stage}` where /// `{instance}` is the input stream record's offset and `{stage}` is the name of the pipeline stage. async fn recover_pipeline_state( - pipeline_tree: &Tree, metadata_tree: &Tree, stream_tree: &Tree, pipeline: Arc, stream_latest_offset: u64, + pipeline_tree: Tree, stream_tree: Tree, pipeline: Arc, stream_latest_offset: u64, ) -> Result<(u64, BTreeMap)> { - let (pipeline_tree, metadata_tree, stream_tree) = (pipeline_tree.clone(), metadata_tree.clone(), stream_tree.clone()); let val = Database::spawn_blocking(move || -> Result<(u64, BTreeMap)> { // Fetch last source stream offset to have been processed by this pipeline. - let last_offset = metadata_tree + let last_offset = pipeline_tree .get(KEY_LAST_OFFSET_PROCESSED) .context("error fetching pipeline last offset processed key")? .map(|val| utils::decode_u64(&val)) @@ -660,13 +658,15 @@ async fn recover_pipeline_state( }); // Fetch active instances. - let active_instances = metadata_tree.scan_prefix(PREFIX_META_ACTIVE_INSTANCES).values().try_fold( + let active_instances = pipeline_tree.scan_prefix(PREFIX_META_ACTIVE_INSTANCES).values().try_fold( BTreeMap::new(), |mut acc, val| -> Result> { let offset_ivec = val.context(ERR_ITER_FAILURE)?; let offset = utils::decode_u64(&offset_ivec).context("error decoding active pipeline offset")?; // Fetch the stream event which triggered this pipeline instance. + // TODO: refactor this. The root event needs to be stored in the pipeline tree as well so that + // compaction does not cause complications. let root_event = stream_tree .get(&offset_ivec) .context("error fetching pipeline instance root event")? diff --git a/hadron-stream/src/stream/mod.rs b/hadron-stream/src/stream/mod.rs index b3e0985..80076a1 100644 --- a/hadron-stream/src/stream/mod.rs +++ b/hadron-stream/src/stream/mod.rs @@ -49,22 +49,25 @@ use crate::models::stream::Subscription; use crate::stream::subscriber::StreamSubCtlMsg; use crate::utils; -/* -TODO: -- [x] simplify PREFIX_STREAM_SUBS & PREFIX_STREAM_SUB_OFFSETS with minimal `{byte}` prefixes - and use IVec::from to build IVec keys. -- store the stream's last written offset under the KEY_STREAM_LAST_WRITTEN_OFFSET key. -- store all stream entries using a `{byte}` prefix. -- [x] collapse the metadata stream into the standard stream tree. -*/ - /// The key prefix used for storing stream events. +/// +/// NOTE: in order to preserve lexicographical ordering of keys, it is important to always use +/// the `utils::encode_byte_prefix*` methods. const PREFIX_STREAM_EVENT: &[u8; 1] = b"e"; /// The key prefix used for storing stream event timestamps. +/// +/// NOTE: in order to preserve lexicographical ordering of keys, it is important to always use +/// the `utils::encode_byte_prefix*` methods. const PREFIX_STREAM_TS: &[u8; 1] = b"t"; /// The database key prefix used for storing stream subscriber data. +/// +/// NOTE: in order to preserve lexicographical ordering of keys, it is important to always use +/// the `utils::encode_byte_prefix*` methods. const PREFIX_STREAM_SUBS: &[u8; 1] = b"s"; /// The database key prefix used for storing stream subscriber offsets. +/// +/// NOTE: in order to preserve lexicographical ordering of keys, it is important to always use +/// the `utils::encode_byte_prefix*` methods. const PREFIX_STREAM_SUB_OFFSETS: &[u8; 1] = b"o"; /// The key used to store the last written offset for the stream. const KEY_STREAM_LAST_WRITTEN_OFFSET: &[u8; 1] = b"l"; diff --git a/hadron-stream/src/utils_test.rs b/hadron-stream/src/utils_test.rs new file mode 100644 index 0000000..c1614fe --- /dev/null +++ b/hadron-stream/src/utils_test.rs @@ -0,0 +1,92 @@ +use anyhow::{Context, Result}; + +use crate::config::Config; +use crate::database::Database; +use crate::utils; + +const ERR_MSG_ITER: &str = "error iterating scanned data"; +const NUM_ENTRIES: u64 = 1_001; +const PREFIX_A: &[u8; 1] = b"a"; +/// We use this in tests as it is middle in lexicographical sort order. +const PREFIX_B: &[u8; 1] = b"b"; +const PREFIX_C: &[u8; 1] = b"c"; + +#[tokio::test] +async fn test_exhaustive_scan_prefix_and_range_behavior() -> Result<()> { + let (config, _tmpdir) = Config::new_test()?; + let db = Database::new(config.clone()).await?; + let tree = db.get_stream_tree().await?; + + // Load data distributed across three key prefixes which are used to assert correctness of + // range scans and prefix scans, which depend upon the correctness of key encoding. + load_data(&tree)?; + + // Assert that prefix scan finds the correct amount of data. + let mut count = 0; + for kv_res in tree.scan_prefix(PREFIX_B) { + let (key, val) = kv_res.context(ERR_MSG_ITER)?; + if key[0] != PREFIX_B[0] { + println!("bad key prefix: got {}; expected: {};", key[0], PREFIX_B[0]); + } else { + count += 1; + } + let _key = utils::decode_u64(&key[1..])?; + let _val = utils::decode_u64(&val)?; + } + assert_eq!(count, NUM_ENTRIES, "expected scan_prefix to find {} entries, got {}", NUM_ENTRIES, count); + + // Assert that range scans preserve sort order based on our key prefix strategy. + let (start, stop, mut count, mut current_offset) = (PREFIX_B, PREFIX_C, 0, 0u64); + for kv_res in tree.range::<_, std::ops::Range<&[u8]>>(start..stop) { + let (key, val) = kv_res.context(ERR_MSG_ITER)?; + if key[0] != PREFIX_B[0] { + println!("bad key prefix: got {}; expected: {};", key[0], &PREFIX_B[0]); + } else { + count += 1; + } + let key = utils::decode_u64(&key[1..])?; + let val = utils::decode_u64(&val)?; + assert_eq!( + key, current_offset, + "db.range with prefix iterated out of order, expected key {} got {}", + current_offset, key + ); + assert_eq!( + val, current_offset, + "db.range with prefix iterated out of order, expected val {} got {}", + current_offset, val + ); + current_offset += 1; + } + assert_eq!(count, NUM_ENTRIES, "expected range to find {} entries, got {}", NUM_ENTRIES, count); + + Ok(()) +} + +#[test] +fn test_ivec_and_encoding_compat() { + let i0 = utils::ivec_from_iter(PREFIX_B.iter().copied().chain(u64::MIN.to_be_bytes().iter().copied())); + let i1 = utils::ivec_from_iter(PREFIX_B.iter().copied().chain(1u64.to_be_bytes().iter().copied())); + let i2 = utils::ivec_from_iter(PREFIX_B.iter().copied().chain(10u64.to_be_bytes().iter().copied())); + + let e0 = utils::encode_byte_prefix(PREFIX_B, u64::MIN); + let e1 = utils::encode_byte_prefix(PREFIX_B, 1u64); + let e2 = utils::encode_byte_prefix(PREFIX_B, 10u64); + + assert_eq!(&i0, &e0, "ivec slice i0 is different from encoded slice:\n{:?}\n{:?}", &i0, &e0); + assert_eq!(&i1, &e1, "ivec slice i1 is different from encoded slice:\n{:?}\n{:?}", &i1, &e1); + assert_eq!(&i2, &e2, "ivec slice i2 is different from encoded slice:\n{:?}\n{:?}", &i2, &e2); +} + +fn load_data(db: &sled::Tree) -> Result<()> { + for prefix in [PREFIX_A, PREFIX_B, PREFIX_C] { + let mut batch = sled::Batch::default(); + for offset in 0..NUM_ENTRIES { + let key = utils::encode_byte_prefix(prefix, offset); + batch.insert(&key, &utils::encode_u64(offset)); + } + db.apply_batch(batch).context("error inserting data")?; + } + db.flush().context("error flusing data")?; + Ok(()) +} From 54da45ac87ca11b7ed9e3a1915eec68b0767613d Mon Sep 17 00:00:00 2001 From: Anthony Dodd Date: Sun, 24 Oct 2021 16:48:59 -0500 Subject: [PATCH 04/10] Pipeline storage has been updated Pipeline data indexing strategy has been updated to use a single tree and a more efficient indexing strategy based upon lexicographical ordering of encoded keys. --- guide/src/overview/quick-start.md | 4 -- hadron-stream/src/pipeline/mod.rs | 76 ++++++++++++++----------------- hadron-stream/src/stream/mod.rs | 10 ++-- hadron-stream/src/utils_test.rs | 21 +++++++++ 4 files changed, 60 insertions(+), 51 deletions(-) diff --git a/guide/src/overview/quick-start.md b/guide/src/overview/quick-start.md index 0d81b94..13b4efb 100644 --- a/guide/src/overview/quick-start.md +++ b/guide/src/overview/quick-start.md @@ -43,8 +43,6 @@ spec: pvcVolumeSize: "5Gi" ``` - - See the [Stream Reference](../reference/streams.md) for more details on the spec fields listed above, as well as other config options available for Streams. Now apply the file to your Kubernetes cluster as shown below using the `kubectl` CLI (part of the Kubernetes distribution). ```sh @@ -93,8 +91,6 @@ kubectl run hadron-cli --rm -it \ --image ghcr.io/hadron-project/hadron/hadron-cli:latest ``` - - Here we are running a temporary pod which will be removed from the Kubernetes cluster when disconnected. Once the pod session is started, you should see the help text of the CLI displayed, and then you should have access to the shell prompt. From here, we can execute CLI commands to interact with our new Stream. Let's publish a simple event: diff --git a/hadron-stream/src/pipeline/mod.rs b/hadron-stream/src/pipeline/mod.rs index 2a64278..f57e7ab 100644 --- a/hadron-stream/src/pipeline/mod.rs +++ b/hadron-stream/src/pipeline/mod.rs @@ -22,6 +22,7 @@ use crate::database::Database; use crate::error::{RpcResult, ShutdownError, ShutdownResult, ERR_DB_FLUSH, ERR_ITER_FAILURE}; use crate::futures::LivenessStream; use crate::grpc::{Event, PipelineSubscribeRequest, PipelineSubscribeRequestAction, PipelineSubscribeResponse}; +use crate::stream::PREFIX_STREAM_EVENT; use crate::utils; use hadron_core::crd::{Pipeline, PipelineStartPointLocation, RequiredMetadata}; @@ -32,11 +33,11 @@ const DEFAULT_MAX_PARALLEL: u32 = 50; /// NOTE: this does not necessarily indicate that the pipeline for this key has actually been /// executed, but only that it has been prepared for execution. const KEY_LAST_OFFSET_PROCESSED: &[u8; 1] = b"l"; -/// A metadata key prefix used for tracking active pipeline instances. +/// A key prefix used for tracking active pipeline instances. /// /// Active instances are keyed as `a{offset}` where `{offset}` is the offset -/// of the event from the source stream. The value is the offset. -const PREFIX_META_ACTIVE_INSTANCES: &[u8; 1] = b"a"; +/// of the event from the source stream. The value is the corresponding root event. +const PREFIX_ACTIVE_INSTANCES: &[u8; 1] = b"a"; /// The key prefix under which pipeline stage outputs are stored. const PREFIX_PIPELINE_STAGE_OUTPUTS: &[u8; 1] = b"o"; @@ -97,8 +98,7 @@ impl PipelineCtl { let tree = db.get_pipeline_tree(pipeline.name()).await?; let stream_tree = db.get_stream_tree().await?; let stream_offset = *stream_signal.borrow(); - let (last_offset_processed, active_pipelines) = - recover_pipeline_state(tree.clone(), stream_tree.clone(), pipeline.clone(), stream_offset).await?; + let (last_offset_processed, active_pipelines) = recover_pipeline_state(tree.clone(), pipeline.clone(), stream_offset).await?; Ok(Self { config, @@ -534,15 +534,16 @@ impl PipelineCtl { return Ok(()); } }; - let key = format!( - "{}{}/{}", - unsafe { std::str::from_utf8_unchecked(PREFIX_PIPELINE_STAGE_OUTPUTS) }, - offset, - &*stage_name, + let key = utils::ivec_from_iter( + PREFIX_PIPELINE_STAGE_OUTPUTS + .iter() + .copied() + .chain(utils::encode_u64(offset)) + .chain(stage_name.as_bytes().iter().copied()), ); let event_bytes = utils::encode_model(&event)?; pipeline_tree - .insert(key.as_bytes(), event_bytes.as_slice()) + .insert(key, event_bytes.as_slice()) .context("error recording pipeline stage output on disk") .map_err(ShutdownError::from)?; pipeline_tree @@ -577,15 +578,16 @@ impl PipelineCtl { let data_res = Database::spawn_blocking(move || -> Result { // Iterate over the records of the stream up to the maximum parallel allowed. let (mut pipeline_batch, mut new_instances) = (sled::Batch::default(), Vec::with_capacity(max_parallel as usize)); - let (start, mut last_processed, mut count) = (&utils::encode_u64(last_offset_processed + 1), last_offset_processed, 0); - // TODO: need to iterate the tree correctly only scanning events. - // TODO: ensure the pipeline tree is used correctly below with prefixes. - // TODO: store the root event in the pipeline as well, to avoid issues with compaction. - let stream = tree_stream.range::<_, std::ops::RangeFrom<&[u8]>>(start..); - for event_res in stream { + let (mut last_processed, mut count) = (last_offset_processed, 0); + let (start, stop) = ( + &utils::encode_byte_prefix(PREFIX_STREAM_EVENT, last_offset_processed + 1), + &utils::encode_byte_prefix(PREFIX_STREAM_EVENT, u64::MAX), + ); + let kv_iter = tree_stream.range::<_, std::ops::RangeInclusive<&[u8]>>(start..=stop); + for event_res in kv_iter { // Decode the records offset. let (key, root_event_bytes) = event_res.context(ERR_ITER_FAILURE)?; - let offset = utils::decode_u64(key.as_ref())?; + let offset = utils::decode_u64(&key[1..])?; last_processed = offset; let root_event: Event = utils::decode_model(root_event_bytes.as_ref()).context("error decoding event from storage")?; @@ -595,7 +597,7 @@ impl PipelineCtl { } // Construct pipeline instance & add to batch. - pipeline_batch.insert(&utils::encode_byte_prefix(PREFIX_META_ACTIVE_INSTANCES, offset), &key); + pipeline_batch.insert(&utils::encode_byte_prefix(PREFIX_ACTIVE_INSTANCES, offset), &root_event_bytes); let inst = ActivePipelineInstance { root_event, root_event_offset: offset, @@ -616,6 +618,7 @@ impl PipelineCtl { let _res = tree_pipeline .apply_batch(pipeline_batch) .context("error applying metadata batch while fetching stream data for pipeline")?; + let _res = tree_pipeline.flush().context(ERR_DB_FLUSH)?; Ok(FetchStreamRecords { last_offset_processed: last_processed, @@ -635,12 +638,12 @@ impl PipelineCtl { /// The pipeline tree records pipeline instances/executions based on the input stream's /// offset, which provides easy "exactly once" consumption of the input stream. In the pipeline /// tree: -/// - The key for a pipeline instance will be roughly `/i/{instance}/`, where `{instance}` is the -/// input stream record's offset. The value stored here is top-level metadata of the pipeline instance. -/// - The output of each stage of a pipeline instance is recorded under `/s/{instance}/{stage}` where -/// `{instance}` is the input stream record's offset and `{stage}` is the name of the pipeline stage. +/// - The key for a pipeline instance will be roughly `a{offset}/`, where `{offset}` is the +/// source stream record's offset. The value stored here is a copy of the root event from the stream. +/// - The output of each stage of a pipeline instance is recorded under `o{instance}{stage}` where +/// `{instance}` is the source stream record's offset and `{stage}` is the name of the pipeline stage. async fn recover_pipeline_state( - pipeline_tree: Tree, stream_tree: Tree, pipeline: Arc, stream_latest_offset: u64, + pipeline_tree: Tree, pipeline: Arc, stream_latest_offset: u64, ) -> Result<(u64, BTreeMap)> { let val = Database::spawn_blocking(move || -> Result<(u64, BTreeMap)> { // Fetch last source stream offset to have been processed by this pipeline. @@ -658,33 +661,22 @@ async fn recover_pipeline_state( }); // Fetch active instances. - let active_instances = pipeline_tree.scan_prefix(PREFIX_META_ACTIVE_INSTANCES).values().try_fold( + let active_instances = pipeline_tree.scan_prefix(PREFIX_ACTIVE_INSTANCES).try_fold( BTreeMap::new(), - |mut acc, val| -> Result> { - let offset_ivec = val.context(ERR_ITER_FAILURE)?; - let offset = utils::decode_u64(&offset_ivec).context("error decoding active pipeline offset")?; - - // Fetch the stream event which triggered this pipeline instance. - // TODO: refactor this. The root event needs to be stored in the pipeline tree as well so that - // compaction does not cause complications. - let root_event = stream_tree - .get(&offset_ivec) - .context("error fetching pipeline instance root event")? - .map(|data| -> Result { utils::decode_model::(data.as_ref()).context("error decoding event from storage") }) - .transpose()? - .context("source event of pipeline instance not found")?; + |mut acc, kv_res| -> Result> { + let (key, val) = kv_res.context(ERR_ITER_FAILURE)?; + let offset = utils::decode_u64(&key[1..]).context("error decoding active pipeline offset")?; + let root_event: Event = utils::decode_model(&val).context("error decoding event from storage")?; // Iterate over all outputs currently recorded for this pipeline instance. - // See `try_record_delivery_response`, these are keyed as `/s/{offset}/{stage_name}`. + // See `try_record_delivery_response`, these are keyed as `o{offset}{stage}`. let mut outputs = HashMap::new(); for iter_res in pipeline_tree.scan_prefix(&utils::encode_byte_prefix(PREFIX_PIPELINE_STAGE_OUTPUTS, offset)) { let (key, val) = iter_res.context(ERR_ITER_FAILURE)?; - let key = std::str::from_utf8(&key).context("data corruption: all keys should be valid utf8")?; - let stage = key.split('/').last().unwrap_or(""); + let stage = std::str::from_utf8(&key[9..]).context("data corruption: stage name for pipeline stage output is not valid utf8")?; let output = Event::decode(val.as_ref()).context("error decoding pipeline stage output")?; outputs.insert(stage.into(), output); } - let inst = ActivePipelineInstance { root_event, root_event_offset: offset, diff --git a/hadron-stream/src/stream/mod.rs b/hadron-stream/src/stream/mod.rs index 80076a1..e622bbf 100644 --- a/hadron-stream/src/stream/mod.rs +++ b/hadron-stream/src/stream/mod.rs @@ -53,24 +53,24 @@ use crate::utils; /// /// NOTE: in order to preserve lexicographical ordering of keys, it is important to always use /// the `utils::encode_byte_prefix*` methods. -const PREFIX_STREAM_EVENT: &[u8; 1] = b"e"; +pub const PREFIX_STREAM_EVENT: &[u8; 1] = b"e"; /// The key prefix used for storing stream event timestamps. /// /// NOTE: in order to preserve lexicographical ordering of keys, it is important to always use /// the `utils::encode_byte_prefix*` methods. -const PREFIX_STREAM_TS: &[u8; 1] = b"t"; +pub const PREFIX_STREAM_TS: &[u8; 1] = b"t"; /// The database key prefix used for storing stream subscriber data. /// /// NOTE: in order to preserve lexicographical ordering of keys, it is important to always use /// the `utils::encode_byte_prefix*` methods. -const PREFIX_STREAM_SUBS: &[u8; 1] = b"s"; +pub const PREFIX_STREAM_SUBS: &[u8; 1] = b"s"; /// The database key prefix used for storing stream subscriber offsets. /// /// NOTE: in order to preserve lexicographical ordering of keys, it is important to always use /// the `utils::encode_byte_prefix*` methods. -const PREFIX_STREAM_SUB_OFFSETS: &[u8; 1] = b"o"; +pub const PREFIX_STREAM_SUB_OFFSETS: &[u8; 1] = b"o"; /// The key used to store the last written offset for the stream. -const KEY_STREAM_LAST_WRITTEN_OFFSET: &[u8; 1] = b"l"; +pub const KEY_STREAM_LAST_WRITTEN_OFFSET: &[u8; 1] = b"l"; const ERR_DECODING_STREAM_META_GROUP_NAME: &str = "error decoding stream meta group name from storage"; diff --git a/hadron-stream/src/utils_test.rs b/hadron-stream/src/utils_test.rs index c1614fe..16ca8a4 100644 --- a/hadron-stream/src/utils_test.rs +++ b/hadron-stream/src/utils_test.rs @@ -68,6 +68,13 @@ fn test_ivec_and_encoding_compat() { let i0 = utils::ivec_from_iter(PREFIX_B.iter().copied().chain(u64::MIN.to_be_bytes().iter().copied())); let i1 = utils::ivec_from_iter(PREFIX_B.iter().copied().chain(1u64.to_be_bytes().iter().copied())); let i2 = utils::ivec_from_iter(PREFIX_B.iter().copied().chain(10u64.to_be_bytes().iter().copied())); + let i3 = utils::ivec_from_iter( + PREFIX_B + .iter() + .copied() + .chain(utils::encode_u64(10)) + .chain("stage_name".as_bytes().iter().copied()), + ); let e0 = utils::encode_byte_prefix(PREFIX_B, u64::MIN); let e1 = utils::encode_byte_prefix(PREFIX_B, 1u64); @@ -76,6 +83,20 @@ fn test_ivec_and_encoding_compat() { assert_eq!(&i0, &e0, "ivec slice i0 is different from encoded slice:\n{:?}\n{:?}", &i0, &e0); assert_eq!(&i1, &e1, "ivec slice i1 is different from encoded slice:\n{:?}\n{:?}", &i1, &e1); assert_eq!(&i2, &e2, "ivec slice i2 is different from encoded slice:\n{:?}\n{:?}", &i2, &e2); + assert_eq!( + &i3[..9], + &e2, + "the first 9 bytes of i3 do not match the byte encoded prefix:\n{:?}\n{:?}", + &i3[..9], + &e2 + ); + assert_eq!( + &i3[9..], + b"stage_name", + "the last 10 bytes of i3 do not match the expected stage name:\n{:?}\n{:?}", + &i3[9..], + b"stage_name" + ); } fn load_data(db: &sled::Tree) -> Result<()> { From 456911a167d606d4ed9cd4fb7389debdecd52766 Mon Sep 17 00:00:00 2001 From: Anthony Dodd Date: Tue, 26 Oct 2021 23:23:01 -0500 Subject: [PATCH 05/10] stream: update docker rustc version & pipelines refactor Updated to the latest rustc 1.56.0. Refactored Pipelines to simplify the delivery pass mechanism. Fixed a bug where active pipeline instances restored from disk were not being properly pruned if they were already complete. As part of the compaction story, finished pipelines will be removed from disk. Finished pipelines are now being removed. --- .../Dockerfile | 2 +- .../Dockerfile | 2 +- hadron-core/src/auth/token.rs | 4 +- hadron-operator/Dockerfile | 2 +- hadron-stream/Dockerfile | 2 +- hadron-stream/src/pipeline/mod.rs | 224 ++++++++++++------ 6 files changed, 158 insertions(+), 78 deletions(-) diff --git a/examples/pipeline-transactional-processing/Dockerfile b/examples/pipeline-transactional-processing/Dockerfile index 4138593..875a139 100644 --- a/examples/pipeline-transactional-processing/Dockerfile +++ b/examples/pipeline-transactional-processing/Dockerfile @@ -1,7 +1,7 @@ ## Dockerfile for pipeline-transactional-processing (txp) demo app. # syntax=docker/dockerfile:1.3 -ARG RUST_TAG=1.55.0 +ARG RUST_TAG=1.56.0 ARG ALPINE_TAG=3.14 FROM rust:${RUST_TAG}-alpine${ALPINE_TAG} as builder diff --git a/examples/stream-transactional-processing/Dockerfile b/examples/stream-transactional-processing/Dockerfile index 1f03016..5bee7cb 100644 --- a/examples/stream-transactional-processing/Dockerfile +++ b/examples/stream-transactional-processing/Dockerfile @@ -1,7 +1,7 @@ ## Dockerfile for stream-transactional-processing (txp) demo app. # syntax=docker/dockerfile:1.3 -ARG RUST_TAG=1.55.0 +ARG RUST_TAG=1.56.0 ARG ALPINE_TAG=3.14 FROM rust:${RUST_TAG}-alpine${ALPINE_TAG} as builder diff --git a/hadron-core/src/auth/token.rs b/hadron-core/src/auth/token.rs index 03be6b3..5302b44 100644 --- a/hadron-core/src/auth/token.rs +++ b/hadron-core/src/auth/token.rs @@ -88,7 +88,7 @@ impl TokenClaims { /// Encode this claims body as a JWT. pub fn encode(&self, key: &EncodingKey) -> jsonwebtoken::errors::Result { let header = Header::new(Algorithm::HS512); - jsonwebtoken::encode(&header, &self, &key) + jsonwebtoken::encode(&header, &self, key) } /// Decode the given string as a JWT with a `TokenClaims` body. @@ -99,7 +99,7 @@ impl TokenClaims { validate_nbf: false, ..Default::default() }; - jsonwebtoken::decode(token.as_ref(), &key, &validation).map(|body| body.claims) + jsonwebtoken::decode(token.as_ref(), key, &validation).map(|body| body.claims) } /// Decode the given string as a JWT with a `TokenClaims` body. diff --git a/hadron-operator/Dockerfile b/hadron-operator/Dockerfile index f943512..dc3f232 100644 --- a/hadron-operator/Dockerfile +++ b/hadron-operator/Dockerfile @@ -3,7 +3,7 @@ ## NOTE WELL: this Dockerfile assumes a context from the root of this repo. # syntax=docker/dockerfile:1.3 -ARG RUST_TAG=1.55.0 +ARG RUST_TAG=1.56.0 ARG ALPINE_TAG=3.14 FROM rust:${RUST_TAG}-alpine${ALPINE_TAG} as builder diff --git a/hadron-stream/Dockerfile b/hadron-stream/Dockerfile index bc11112..3390fa5 100644 --- a/hadron-stream/Dockerfile +++ b/hadron-stream/Dockerfile @@ -3,7 +3,7 @@ ## NOTE WELL: this Dockerfile assumes a context from the root of this repo. # syntax=docker/dockerfile:1.3 -ARG RUST_TAG=1.55.0 +ARG RUST_TAG=1.56.0 ARG ALPINE_TAG=3.14 FROM rust:${RUST_TAG}-alpine${ALPINE_TAG} as builder diff --git a/hadron-stream/src/pipeline/mod.rs b/hadron-stream/src/pipeline/mod.rs index f57e7ab..9ac4956 100644 --- a/hadron-stream/src/pipeline/mod.rs +++ b/hadron-stream/src/pipeline/mod.rs @@ -24,7 +24,7 @@ use crate::futures::LivenessStream; use crate::grpc::{Event, PipelineSubscribeRequest, PipelineSubscribeRequestAction, PipelineSubscribeResponse}; use crate::stream::PREFIX_STREAM_EVENT; use crate::utils; -use hadron_core::crd::{Pipeline, PipelineStartPointLocation, RequiredMetadata}; +use hadron_core::crd::{Pipeline, PipelineStage, PipelineStartPointLocation, RequiredMetadata}; const DEFAULT_MAX_PARALLEL: u32 = 50; /// A pipeline metadata key used to track the last offset of the source stream to have been @@ -39,6 +39,8 @@ const KEY_LAST_OFFSET_PROCESSED: &[u8; 1] = b"l"; /// of the event from the source stream. The value is the corresponding root event. const PREFIX_ACTIVE_INSTANCES: &[u8; 1] = b"a"; /// The key prefix under which pipeline stage outputs are stored. +/// +/// Outputs are keyed as `o{offset}{stage_name}`. const PREFIX_PIPELINE_STAGE_OUTPUTS: &[u8; 1] = b"o"; /// The liveness stream type used by a pipeline controller. @@ -128,12 +130,33 @@ impl PipelineCtl { async fn run(mut self) -> Result<()> { tracing::debug!( + last_offset_processed = self.last_offset_processed, "pipeline controller {}/{}/{} has started", self.config.stream, self.partition, self.pipeline.name() ); + // Check for active pipelines which need to be removed. + let (events_tx, pipeline) = (self.events_tx.clone(), &self.pipeline); + self.active_pipelines.retain(|offset, inst| { + if pipeline + .spec + .stages + .iter() + .all(|stage| inst.outputs.contains_key(&stage.name)) + { + tracing::debug!(offset, "pruning old finished pipeline instance"); + let (events_tx, offset) = (events_tx.clone(), *offset); + tokio::spawn(async move { + let _res = events_tx.send(PipelineCtlMsg::PipelineInstanceComplete(offset)).await; + }); + false + } else { + true + } + }); + loop { if self.descheduled { break; @@ -148,6 +171,7 @@ impl PipelineCtl { // Begin shutdown routine. tracing::debug!( + last_offset_processed = self.last_offset_processed, "pipeline controller {}/{}/{} has shutdown", self.config.stream, self.partition, @@ -168,10 +192,10 @@ impl PipelineCtl { /// Handle any dead channels. #[tracing::instrument(level = "trace", skip(self, dead_chan))] async fn handle_dead_subscriber(&mut self, dead_chan: (Arc, Uuid, ClientChannel)) { - let (group_name, id, _chan) = (dead_chan.0, dead_chan.1, dead_chan.2); - tracing::debug!(?id, group_name = ?&*group_name, "dropping pipeline subscriber channel"); + let (stage_name, id, _chan) = (dead_chan.0, dead_chan.1, dead_chan.2); + tracing::debug!(?id, group_name = ?&*stage_name, "dropping pipeline stage subscriber channel"); self.liveness_checks.remove(&id); - let mut group = match self.stage_subs.remove(&*group_name) { + let mut group = match self.stage_subs.remove(&*stage_name) { Some(group) => group, None => return, }; @@ -212,6 +236,7 @@ impl PipelineCtl { PipelineCtlMsg::DeliveryResponse(res) => self.handle_delivery_response(res).await, PipelineCtlMsg::PipelineUpdated(pipeline) => self.handle_pipeline_updated(pipeline), PipelineCtlMsg::PipelineDeleted(pipeline) => self.handle_pipeline_deleted(pipeline), + PipelineCtlMsg::PipelineInstanceComplete(offset) => self.handle_pipeline_instance_complete(offset).await, } } @@ -288,7 +313,7 @@ impl PipelineCtl { } /// Handle a request which has been sent to this controller. - #[tracing::instrument(level = "trace", skip(self, tx, rx, stage_name))] + #[tracing::instrument(level = "trace", skip(self, tx, rx))] async fn handle_request(&mut self, (tx, rx): ClientChannel, stage_name: String) { tracing::debug!("request received on pipeline controller"); // Validate contents of setup request. @@ -324,6 +349,42 @@ impl PipelineCtl { self.execute_delivery_pass().await; } + /// Handle an event indicating that the pipeline instance at the given + /// offset is ready to be deleted. + #[tracing::instrument(level = "trace", skip(self))] + async fn handle_pipeline_instance_complete(&mut self, offset: u64) { + // Build up delete op. + let (tree, mut batch) = (self.tree.clone(), sled::Batch::default()); + batch.remove(&utils::encode_byte_prefix(PREFIX_ACTIVE_INSTANCES, offset)); + for stage_name in self.pipeline.spec.stages.iter().map(|stage| stage.name.as_str()) { + let key = utils::ivec_from_iter( + PREFIX_PIPELINE_STAGE_OUTPUTS + .iter() + .copied() + .chain(utils::encode_u64(offset)) + .chain(stage_name.as_bytes().iter().copied()), + ); + batch.remove(key); + } + + // Apply batch. + let res = Database::spawn_blocking(move || -> Result<()> { + tree.apply_batch(batch) + .context("error applying batch delete on finished pipeline")?; + tree.flush().context(ERR_DB_FLUSH)?; + Ok(()) + }) + .await + .map_err(ShutdownError::from) + .and_then(|res| res.map_err(ShutdownError::from)); + + // Shutdown if needed. + if let Err(err) = res { + tracing::error!(error = ?err, "error deleting finished pipeline data, shutting down"); + let _ = self.shutdown_tx.send(()); + } + } + /// Execute a loop over all active pipeline stage subscription groups, delivering data if possible. /// /// ### Notes @@ -363,73 +424,8 @@ impl PipelineCtl { None => continue, // Just skip. }; - // Randomly select one of the available subscriptions for the stage. - let chan_key_opt = sub_group - .active_channels - .iter() - .filter(|(_, val)| matches!(val, SubChannelState::MonitoringLiveness)) - .choose(&mut rand::thread_rng()) - .map(|(key, _val)| *key); - let chan_id = match chan_key_opt { - Some(chan_key) => chan_key, - None => continue, // This would only mean that all channels are busy. - }; - let _old_state = match sub_group.active_channels.remove(&chan_id) { - Some(chan_data) => chan_data, - None => continue, // Unreachable. - }; - let mut chan = match self.liveness_checks.remove(&chan_id) { - Some(chan_opt) => match chan_opt.chan { - Some(chan) => chan, - None => continue, - }, - None => { - tracing::error!(?chan_id, "pipeline subscription channel was not properly held in liveness stream"); - continue; - } - }; - - // We are ready to delivery some data to the target channel. Accumulate delivery - // payload with all needed inputs. - let mut payload = PipelineSubscribeResponse { - stage: stage.name.clone(), - root_event: Some(inst.root_event.clone()), - inputs: Default::default(), - }; - stage.dependencies.iter().for_each(|dep| match inst.outputs.get(dep) { - Some(input) => { - payload.inputs.insert(dep.clone(), input.clone()); - } - None => tracing::error!("failed to accumulate stage dependencies even though all were accounted for"), - }); - - // Payload is ready, send it. - let _res = chan - .0 - .send(Ok(payload)) - .await - .context("error sending pipeline delivery payload")?; - - // Spawn off a task to await the response from the client. - inst.active_deliveries.insert(sub_group.stage_name.clone(), chan_id); - sub_group.active_channels.insert(chan_id, SubChannelState::OutForDelivery); - let (tx, group_name, id, offset) = (self.events_tx.clone(), sub_group.stage_name.clone(), chan_id, inst.root_event_offset); - tokio::spawn(async move { - // TODO: add optional timeouts here based on pipeline config. - let output = chan - .1 - .next() - .await - .map(|res| res.map_err(anyhow::Error::from).map(|data| (chan, data))); - let _ = tx - .send(PipelineCtlMsg::DeliveryResponse(DeliveryResponse { - id, - offset, - stage_name: group_name, - output, - })) - .await; - }); + // Create a delivery payload and send it to a randomly selected subscriber. + Self::spawn_payload_delivery(stage, inst, sub_group, &mut self.liveness_checks, self.events_tx.clone()).await?; } } @@ -450,6 +446,81 @@ impl PipelineCtl { Ok(()) } + /// Deliver a payload to a stage consumer & spawn a task to await its response. + #[tracing::instrument(level = "debug", skip(stage, inst, sub_group, liveness_checks, events_tx))] + async fn spawn_payload_delivery( + stage: &PipelineStage, inst: &mut ActivePipelineInstance, sub_group: &mut SubscriptionGroup, + liveness_checks: &mut StreamMap, events_tx: mpsc::Sender, + ) -> Result<()> { + // Randomly select one of the available subscriptions for the stage. + let chan_key_opt = sub_group + .active_channels + .iter() + .filter(|(_, val)| matches!(val, SubChannelState::MonitoringLiveness)) + .choose(&mut rand::thread_rng()) + .map(|(key, _val)| *key); + let chan_id = match chan_key_opt { + Some(chan_key) => chan_key, + None => return Ok(()), // This would only mean that all channels are busy. + }; + let _old_state = match sub_group.active_channels.remove(&chan_id) { + Some(chan_data) => chan_data, + None => return Ok(()), // Unreachable. + }; + let mut chan = match liveness_checks.remove(&chan_id) { + Some(chan_opt) => match chan_opt.chan { + Some(chan) => chan, + None => return Ok(()), + }, + None => { + tracing::error!(?chan_id, "pipeline subscription channel was not properly held in liveness stream"); + return Ok(()); + } + }; + + // Accumulate delivery payload with all needed inputs. + let mut payload = PipelineSubscribeResponse { + stage: stage.name.clone(), + root_event: Some(inst.root_event.clone()), + inputs: Default::default(), + }; + stage.dependencies.iter().for_each(|dep| match inst.outputs.get(dep) { + Some(input) => { + payload.inputs.insert(dep.clone(), input.clone()); + } + None => tracing::error!("bug: failed to accumulate stage dependencies even though all were accounted for"), + }); + + // Payload is ready, send it. + let _res = chan + .0 + .send(Ok(payload)) + .await + .context("error sending pipeline delivery payload")?; + + // Spawn off a task to await the response from the client. + inst.active_deliveries.insert(sub_group.stage_name.clone(), chan_id); + sub_group.active_channels.insert(chan_id, SubChannelState::OutForDelivery); + let (tx, group_name, id, offset) = (events_tx, sub_group.stage_name.clone(), chan_id, inst.root_event_offset); + tokio::spawn(async move { + // FUTURE: add optional timeouts here based on pipeline config. + let output = chan + .1 + .next() + .await + .map(|res| res.map_err(anyhow::Error::from).map(|data| (chan, data))); + let _ = tx + .send(PipelineCtlMsg::DeliveryResponse(DeliveryResponse { + id, + offset, + stage_name: group_name, + output, + })) + .await; + }); + Ok(()) + } + /// Handle a pipeline stage delivery response. #[tracing::instrument(level = "trace", skip(self, res))] async fn try_handle_delivery_response(&mut self, res: DeliveryResponse) -> Result<()> { @@ -517,6 +588,11 @@ impl PipelineCtl { .all(|stage| inst.outputs.contains_key(&stage.name)) { self.active_pipelines.remove(&offset); + tracing::debug!(offset, "pipeline workflow finished"); + let events_tx = self.events_tx.clone(); + tokio::spawn(async move { + let _res = events_tx.send(PipelineCtlMsg::PipelineInstanceComplete(offset)).await; + }); } Ok(()) @@ -642,6 +718,7 @@ impl PipelineCtl { /// source stream record's offset. The value stored here is a copy of the root event from the stream. /// - The output of each stage of a pipeline instance is recorded under `o{instance}{stage}` where /// `{instance}` is the source stream record's offset and `{stage}` is the name of the pipeline stage. +#[tracing::instrument(level = "debug", skip(pipeline_tree, pipeline, stream_latest_offset))] async fn recover_pipeline_state( pipeline_tree: Tree, pipeline: Arc, stream_latest_offset: u64, ) -> Result<(u64, BTreeMap)> { @@ -691,6 +768,7 @@ async fn recover_pipeline_state( Ok((last_offset, active_instances)) }) .await??; + tracing::debug!(last_offset = val.0, active_instances = val.1.len(), "recovered pipeline state"); Ok(val) } @@ -714,6 +792,8 @@ pub enum PipelineCtlMsg { PipelineUpdated(Arc), /// An update indicating that this pipeline has been deleted. PipelineDeleted(Arc), + /// The pipeline instance at the given offset is complete and can be deleted. + PipelineInstanceComplete(u64), } /// A result from fetching stream records for pipeline creation. From ca035e0f733d67a526af92c1c665a76e36e05368 Mon Sep 17 00:00:00 2001 From: Anthony Dodd Date: Thu, 28 Oct 2021 08:17:16 -0500 Subject: [PATCH 06/10] stream: add stream retention policy Tracking of stream earliest timestamp records is being tracked. This provides the foundation for our stream compaction/truncation system. Updated tests to assert the proper functionality of the earliest timestamp tracking pattern. Fixed a few small docs items in the pipeline-txp example. Update pipeline spawning routine to ensure that only pipelines which are part of the parent stream are spawned. --- Justfile | 4 +- charts/hadron-operator/crds/stream.yaml | 30 +++++++++-- .../src/handlers/cleanup.rs | 4 +- .../src/handlers/deploy_service.rs | 4 +- .../src/handlers/notify_user.rs | 4 +- .../src/handlers/setup_billing.rs | 4 +- .../src/handlers/setup_monitoring.rs | 4 +- hadron-cli/src/cmd/stream/sub.rs | 3 +- hadron-client/src/client/pipeline.rs | 2 +- hadron-client/src/client/subscriber.rs | 2 +- hadron-core/src/crd/stream.rs | 43 +++++++++++++-- hadron-stream/src/stream/mod.rs | 53 +++++++++++++++---- hadron-stream/src/stream/mod_test.rs | 50 ++++++++++++----- hadron-stream/src/stream/publisher.rs | 18 ++++++- hadron-stream/src/stream/publisher_test.rs | 28 ++++++++-- hadron-stream/src/watchers/pipelines.rs | 9 ++++ 16 files changed, 207 insertions(+), 55 deletions(-) diff --git a/Justfile b/Justfile index 502ba77..7f04386 100644 --- a/Justfile +++ b/Justfile @@ -20,8 +20,8 @@ kindLoadCli tag="latest": kind load docker-image --name hadron ghcr.io/hadron-project/hadron/hadron-cli:{{tag}} # Run the Hadron CLI within the kind cluster. -runCli tag="latest" pullPolicy="IfNotPresent": - kubectl --context="kind-hadron" run hadron-cli --rm -it \ +runCli num="0" tag="latest" pullPolicy="IfNotPresent": + kubectl --context="kind-hadron" run hadron-cli-{{num}} --rm -it \ --env HADRON_TOKEN=$(kubectl get secret hadron-full-access -o=jsonpath='{.data.token}' | base64 --decode) \ --env HADRON_URL="http://events.default.svc.cluster.local:7000" \ --image ghcr.io/hadron-project/hadron/hadron-cli:{{tag}} --image-pull-policy={{pullPolicy}} diff --git a/charts/hadron-operator/crds/stream.yaml b/charts/hadron-operator/crds/stream.yaml index f52a742..6234218 100644 --- a/charts/hadron-operator/crds/stream.yaml +++ b/charts/hadron-operator/crds/stream.yaml @@ -21,13 +21,16 @@ spec: - jsonPath: ".spec.debug" name: Debug type: boolean - - jsonPath: ".spec.pvc_volume_size" + - jsonPath: ".spec.retentionPolicy" + name: Retention Policy + type: string + - jsonPath: ".spec.pvcVolumeSize" name: PVC Volume Size type: string - - jsonPath: ".spec.pvc_access_modes" + - jsonPath: ".spec.pvcAccessModes" name: PVC Access Modes type: string - - jsonPath: ".spec.pvc_storage_class" + - jsonPath: ".spec.pvcStorageClass" name: PVC Storage Class type: string name: v1beta1 @@ -63,6 +66,27 @@ spec: pvcVolumeSize: description: "The volume size to use for the Stream's backing StatefulSet PVCs." type: string + retentionPolicy: + default: + retentionSeconds: 604800 + strategy: time + description: The retention policy to use for data on the Stream. + properties: + retentionSeconds: + description: "For `Time` retention policy, this specifies the amount of time to retain data on the Stream in seconds." + format: uint64 + minimum: 0.0 + nullable: true + type: integer + strategy: + description: Retention policy to use. + enum: + - retain + - time + type: string + required: + - strategy + type: object required: - image - partitions diff --git a/examples/pipeline-transactional-processing/src/handlers/cleanup.rs b/examples/pipeline-transactional-processing/src/handlers/cleanup.rs index 71a387f..2d40b86 100644 --- a/examples/pipeline-transactional-processing/src/handlers/cleanup.rs +++ b/examples/pipeline-transactional-processing/src/handlers/cleanup.rs @@ -43,9 +43,7 @@ impl hadron::PipelineHandler for CleanupHandler { return Ok(event); } - // Time for BUSINESS LOGIC! Do whatever our microservice needs to do here. - // This is the `deploy-service` handler, so we should probably deploy something. Within - // this transaction, we can update state to track our progress within this task. + // Time for BUSINESS LOGIC! Do whatever our Pipeline stage handler needs to do here. // Once business logic has finished, we need to generate our output event, write it to our // in_table_pipelines along with `(id, source, stage)` to guard against duplicate processing. diff --git a/examples/pipeline-transactional-processing/src/handlers/deploy_service.rs b/examples/pipeline-transactional-processing/src/handlers/deploy_service.rs index 9fdd177..04c6921 100644 --- a/examples/pipeline-transactional-processing/src/handlers/deploy_service.rs +++ b/examples/pipeline-transactional-processing/src/handlers/deploy_service.rs @@ -43,9 +43,7 @@ impl hadron::PipelineHandler for DeployServiceHandler { return Ok(event); } - // Time for BUSINESS LOGIC! Do whatever our microservice needs to do here. - // This is the `deploy-service` handler, so we should probably deploy something. Within - // this transaction, we can update state to track our progress within this task. + // Time for BUSINESS LOGIC! Do whatever our Pipeline stage handler needs to do here. // Once business logic has finished, we need to generate our output event, write it to our // in_table_pipelines along with `(id, source, stage)` to guard against duplicate processing. diff --git a/examples/pipeline-transactional-processing/src/handlers/notify_user.rs b/examples/pipeline-transactional-processing/src/handlers/notify_user.rs index 4bbfe99..0c40be7 100644 --- a/examples/pipeline-transactional-processing/src/handlers/notify_user.rs +++ b/examples/pipeline-transactional-processing/src/handlers/notify_user.rs @@ -43,9 +43,7 @@ impl hadron::PipelineHandler for NotifyUserHandler { return Ok(event); } - // Time for BUSINESS LOGIC! Do whatever our microservice needs to do here. - // This is the `deploy-service` handler, so we should probably deploy something. Within - // this transaction, we can update state to track our progress within this task. + // Time for BUSINESS LOGIC! Do whatever our Pipeline stage handler needs to do here. // Once business logic has finished, we need to generate our output event, write it to our // in_table_pipelines along with `(id, source, stage)` to guard against duplicate processing. diff --git a/examples/pipeline-transactional-processing/src/handlers/setup_billing.rs b/examples/pipeline-transactional-processing/src/handlers/setup_billing.rs index d922821..37c1627 100644 --- a/examples/pipeline-transactional-processing/src/handlers/setup_billing.rs +++ b/examples/pipeline-transactional-processing/src/handlers/setup_billing.rs @@ -43,9 +43,7 @@ impl hadron::PipelineHandler for SetupBillingHandler { return Ok(event); } - // Time for BUSINESS LOGIC! Do whatever our microservice needs to do here. - // This is the `deploy-service` handler, so we should probably deploy something. Within - // this transaction, we can update state to track our progress within this task. + // Time for BUSINESS LOGIC! Do whatever our Pipeline stage handler needs to do here. // Once business logic has finished, we need to generate our output event, write it to our // in_table_pipelines along with `(id, source, stage)` to guard against duplicate processing. diff --git a/examples/pipeline-transactional-processing/src/handlers/setup_monitoring.rs b/examples/pipeline-transactional-processing/src/handlers/setup_monitoring.rs index acb7c95..5f5f09d 100644 --- a/examples/pipeline-transactional-processing/src/handlers/setup_monitoring.rs +++ b/examples/pipeline-transactional-processing/src/handlers/setup_monitoring.rs @@ -43,9 +43,7 @@ impl hadron::PipelineHandler for SetupMonitoringHandler { return Ok(event); } - // Time for BUSINESS LOGIC! Do whatever our microservice needs to do here. - // This is the `deploy-service` handler, so we should probably deploy something. Within - // this transaction, we can update state to track our progress within this task. + // Time for BUSINESS LOGIC! Do whatever our Pipeline stage handler needs to do here. // Once business logic has finished, we need to generate our output event, write it to our // in_table_pipelines along with `(id, source, stage)` to guard against duplicate processing. diff --git a/hadron-cli/src/cmd/stream/sub.rs b/hadron-cli/src/cmd/stream/sub.rs index fde8131..872b54c 100644 --- a/hadron-cli/src/cmd/stream/sub.rs +++ b/hadron-cli/src/cmd/stream/sub.rs @@ -26,6 +26,7 @@ pub struct Sub { start_beginning: bool, /// Start from the latest offset of the stream, default. #[structopt(long, group = "start_point")] + #[allow(dead_code)] start_latest: bool, /// Start from the given offset, defaults to latest. #[structopt(long, group = "start_point")] @@ -43,8 +44,6 @@ impl Sub { SubscriptionStartingPoint::Offset(offset) } else if self.start_beginning { SubscriptionStartingPoint::Beginning - } else if self.start_latest { - SubscriptionStartingPoint::Latest } else { SubscriptionStartingPoint::Latest }, diff --git a/hadron-client/src/client/pipeline.rs b/hadron-client/src/client/pipeline.rs index d9aa13f..c998dea 100644 --- a/hadron-client/src/client/pipeline.rs +++ b/hadron-client/src/client/pipeline.rs @@ -121,7 +121,7 @@ impl PipelineSubscriptionTask { async fn build_subscriptions(&mut self) { let conns = self.client.inner.conns.load(); for (partition, chan) in conns.iter() { - if self.active_subs.contains(&partition) { + if self.active_subs.contains(partition) { continue; } diff --git a/hadron-client/src/client/subscriber.rs b/hadron-client/src/client/subscriber.rs index 383957d..562f6e0 100644 --- a/hadron-client/src/client/subscriber.rs +++ b/hadron-client/src/client/subscriber.rs @@ -134,7 +134,7 @@ impl SubscriptionTask { self.config.max_batch_size }; for (partition, chan) in conns.iter() { - if self.active_subs.contains(&partition) { + if self.active_subs.contains(partition) { continue; } diff --git a/hadron-core/src/crd/stream.rs b/hadron-core/src/crd/stream.rs index eac2ef1..5afefff 100644 --- a/hadron-core/src/crd/stream.rs +++ b/hadron-core/src/crd/stream.rs @@ -22,9 +22,10 @@ pub type Stream = StreamCRD; // Mostly to resolve a Rust Analyzer issue. shortname = "stream", printcolumn = r#"{"name":"Partitions","type":"number","jsonPath":".spec.partitions"}"#, printcolumn = r#"{"name":"Debug","type":"boolean","jsonPath":".spec.debug"}"#, - printcolumn = r#"{"name":"PVC Volume Size","type":"string","jsonPath":".spec.pvc_volume_size"}"#, - printcolumn = r#"{"name":"PVC Access Modes","type":"string","jsonPath":".spec.pvc_access_modes"}"#, - printcolumn = r#"{"name":"PVC Storage Class","type":"string","jsonPath":".spec.pvc_storage_class"}"# + printcolumn = r#"{"name":"Retention Policy","type":"string","jsonPath":".spec.retentionPolicy"}"#, + printcolumn = r#"{"name":"PVC Volume Size","type":"string","jsonPath":".spec.pvcVolumeSize"}"#, + printcolumn = r#"{"name":"PVC Access Modes","type":"string","jsonPath":".spec.pvcAccessModes"}"#, + printcolumn = r#"{"name":"PVC Storage Class","type":"string","jsonPath":".spec.pvcStorageClass"}"# )] #[serde(rename_all = "camelCase")] pub struct StreamSpec { @@ -37,6 +38,9 @@ pub struct StreamSpec { /// Enable debug mode for the Stream's StatefulSet pods. #[serde(default)] pub debug: bool, + /// The retention policy to use for data on the Stream. + #[serde(default)] + pub retention_policy: StreamRetentionSpec, /// Force an exact image to be used for the backing StatefulSet. /// @@ -57,3 +61,36 @@ pub struct StreamSpec { /// CRD status object. #[derive(Clone, Debug, Default, Deserialize, Serialize, PartialEq, JsonSchema)] pub struct StreamStatus {} + +/// The data retention spec for the data on the Stream. +/// +/// Defaults to `Time` based retention of 7 days. +#[derive(Clone, Debug, Deserialize, Serialize, PartialEq, JsonSchema)] +#[serde(rename_all = "camelCase")] +pub struct StreamRetentionSpec { + /// Retention policy to use. + pub strategy: StreamRetentionPolicy, + /// For `Time` retention policy, this specifies the amount of time + /// to retain data on the Stream in seconds. + #[serde(default)] + pub retention_seconds: Option, +} + +impl Default for StreamRetentionSpec { + fn default() -> Self { + Self { + strategy: StreamRetentionPolicy::Time, + retention_seconds: Some(604_800), // 7 days. + } + } +} + +/// The retention policy to use for data on the Stream. +#[derive(Clone, Debug, Deserialize, Serialize, PartialEq, JsonSchema)] +#[serde(rename_all = "lowercase")] +pub enum StreamRetentionPolicy { + /// Retain data on the Stream indefinitely. + Retain, + /// Retain data on the Stream based on secondary timestamp index. + Time, +} diff --git a/hadron-stream/src/stream/mod.rs b/hadron-stream/src/stream/mod.rs index e622bbf..4abaf09 100644 --- a/hadron-stream/src/stream/mod.rs +++ b/hadron-stream/src/stream/mod.rs @@ -54,7 +54,7 @@ use crate::utils; /// NOTE: in order to preserve lexicographical ordering of keys, it is important to always use /// the `utils::encode_byte_prefix*` methods. pub const PREFIX_STREAM_EVENT: &[u8; 1] = b"e"; -/// The key prefix used for storing stream event timestamps. +/// The key prefix used for storing stream event timestamps, always stored as i64 milliseconds. /// /// NOTE: in order to preserve lexicographical ordering of keys, it is important to always use /// the `utils::encode_byte_prefix*` methods. @@ -103,6 +103,11 @@ pub struct StreamCtl { /// The last written offset of the stream. current_offset: u64, + /// The earliest timestamp recorded in the stream. + /// + /// A `None` value indicates that no timestamp record exists on disk, and it will be populated + /// when the next batch is written to the stream. + earliest_timestamp: Option<(i64, u64)>, } impl StreamCtl { @@ -113,10 +118,10 @@ impl StreamCtl { // Recover stream state. let partition = config.partition; let tree = db.get_stream_tree().await?; - let (current_offset, subs) = recover_stream_state(tree.clone()).await?; + let recovery_data = recover_stream_state(tree.clone()).await?; // Spawn the subscriber controller. - let (offset_signal, offset_signal_rx) = watch::channel(current_offset); + let (offset_signal, offset_signal_rx) = watch::channel(recovery_data.last_written_offset); let (subs_tx, subs_rx) = mpsc::channel(100); let sub_ctl = subscriber::StreamSubCtl::new( config.clone(), @@ -127,8 +132,8 @@ impl StreamCtl { subs_tx.clone(), subs_rx, offset_signal_rx.clone(), - subs, - current_offset, + recovery_data.subscriptions, + recovery_data.last_written_offset, ) .spawn(); @@ -145,7 +150,8 @@ impl StreamCtl { _shutdown_tx: shutdown_tx, descheduled: false, sub_ctl, - current_offset, + current_offset: recovery_data.last_written_offset, + earliest_timestamp: recovery_data.first_timestamp_opt, }, offset_signal_rx, )) @@ -202,8 +208,8 @@ impl StreamCtl { } /// Recover this stream's last recorded state. -async fn recover_stream_state(tree: Tree) -> Result<(u64, Vec<(Subscription, u64)>)> { - let val = Database::spawn_blocking(move || -> Result<(u64, Vec<(Subscription, u64)>)> { +async fn recover_stream_state(tree: Tree) -> Result { + let val = Database::spawn_blocking(move || -> Result { // Fetch next offset info. let offset_opt = tree .get(KEY_STREAM_LAST_WRITTEN_OFFSET) @@ -213,6 +219,19 @@ async fn recover_stream_state(tree: Tree) -> Result<(u64, Vec<(Subscription, u64 .transpose()? .unwrap_or(0); + // Fetch first timestamp record. + let first_timestamp_opt = tree + .scan_prefix(PREFIX_STREAM_TS) + .next() + .transpose() + .context("error fetching first timestamp record")? + .map(|(key, val)| -> Result<(i64, u64)> { + let timestamp = utils::decode_i64(&key[1..])?; + let offset = utils::decode_u64(&val)?; + Ok((timestamp, offset)) + }) + .transpose()?; + // Fetch all stream subscriber info. let mut subs = HashMap::new(); for entry_res in tree.scan_prefix(PREFIX_STREAM_SUBS) { @@ -239,13 +258,27 @@ async fn recover_stream_state(tree: Tree) -> Result<(u64, Vec<(Subscription, u64 } } - let subs: Vec<_> = subs.into_iter().map(|(_, val)| val).collect(); - Ok((last_written_offset, subs)) + let subscriptions: Vec<_> = subs.into_iter().map(|(_, val)| val).collect(); + Ok(StreamRecoveryState { + last_written_offset, + subscriptions, + first_timestamp_opt, + }) }) .await??; Ok(val) } +/// A representation of a Stream's state recovered from disk on startup. +struct StreamRecoveryState { + /// The last offset to have been written to disk. + last_written_offset: u64, + /// All stream subscriber info. + subscriptions: Vec<(Subscription, u64)>, + /// The first timestamp record found, if any. + first_timestamp_opt: Option<(i64, u64)>, +} + /// A message bound for a stream controller. pub enum StreamCtlMsg { /// A client request to setup a publisher channel. diff --git a/hadron-stream/src/stream/mod_test.rs b/hadron-stream/src/stream/mod_test.rs index 3759b57..0119a1b 100644 --- a/hadron-stream/src/stream/mod_test.rs +++ b/hadron-stream/src/stream/mod_test.rs @@ -14,10 +14,18 @@ async fn recover_stream_state_empty_state() -> Result<()> { let db = Database::new(config.clone()).await?; let stream_tree = db.get_stream_tree().await?; - let (offset, subscriptions) = super::recover_stream_state(stream_tree).await?; - - assert!(offset == 0, "expected offset to be 0 got {}", offset); - assert!(subscriptions.is_empty(), "expected subscriptions len to be 0 got {}", offset); + let output = super::recover_stream_state(stream_tree).await?; + + assert!( + output.last_written_offset == 0, + "expected offset to be 0 got {}", + output.last_written_offset + ); + assert!( + output.subscriptions.is_empty(), + "expected subscriptions len to be 0 got {}", + output.subscriptions.len() + ); Ok(()) } @@ -30,10 +38,19 @@ async fn recover_stream_state_with_previous_state() -> Result<()> { let expected_offset = setup_stream_data(&stream_tree).await?; - let (offset, subs) = super::recover_stream_state(stream_tree).await?; + let output = super::recover_stream_state(stream_tree).await?; - assert!(offset == expected_offset, "expected offset to be {} got {}", expected_offset, offset); - assert!(subs.is_empty(), "expected subscriptions len to be 0 got {}", offset); + assert!( + output.last_written_offset == expected_offset, + "expected offset to be {} got {}", + expected_offset, + output.last_written_offset + ); + assert!( + output.subscriptions.is_empty(), + "expected subscriptions len to be 0 got {}", + output.subscriptions.len() + ); Ok(()) } @@ -47,11 +64,20 @@ async fn recover_stream_state_with_previous_state_and_subs() -> Result<()> { let expected_offset = setup_stream_data(&stream_tree).await?; let expected_subs = setup_subs_data(&stream_tree).await?; - let (offset, mut subs) = super::recover_stream_state(stream_tree).await?; - subs.sort_by(|a, b| a.1.cmp(&b.1)); - - assert!(offset == expected_offset, "expected offset to be {} got {}", expected_offset, offset); - assert_eq!(subs, expected_subs, "expected subscriptions to match {:?}\n{:?}", subs, expected_subs); + let mut output = super::recover_stream_state(stream_tree).await?; + output.subscriptions.sort_by(|a, b| a.1.cmp(&b.1)); + + assert!( + output.last_written_offset == expected_offset, + "expected offset to be {} got {}", + expected_offset, + output.last_written_offset + ); + assert_eq!( + output.subscriptions, expected_subs, + "expected subscriptions to match {:?}\n{:?}", + output.subscriptions, expected_subs + ); Ok(()) } diff --git a/hadron-stream/src/stream/publisher.rs b/hadron-stream/src/stream/publisher.rs index e4657ee..270543e 100644 --- a/hadron-stream/src/stream/publisher.rs +++ b/hadron-stream/src/stream/publisher.rs @@ -11,7 +11,15 @@ impl StreamCtl { pub(super) async fn handle_publisher_request(&mut self, tx: oneshot::Sender>, data: StreamPublishRequest) { // Publish the new data frame. let _write_ack = data.ack; - let offset = match Self::publish_data_frame(&self.tree, &mut self.current_offset, &self.offset_signal, data).await { + let offset = match Self::publish_data_frame( + &self.tree, + &mut self.current_offset, + &mut self.earliest_timestamp, + &self.offset_signal, + data, + ) + .await + { Ok(offset) => offset, Err(err) => { tracing::error!(error = ?err, "error while publishing data to stream"); @@ -35,7 +43,8 @@ impl StreamCtl { /// Publish a frame of data to the target stream, returning the offset of the last entry written. #[tracing::instrument(level = "trace", skip(tree, current_offset, offset_signal, req))] pub(super) async fn publish_data_frame( - tree: &sled::Tree, current_offset: &mut u64, offset_signal: &watch::Sender, req: StreamPublishRequest, + tree: &sled::Tree, current_offset: &mut u64, earliest_timestamp: &mut Option<(i64, u64)>, offset_signal: &watch::Sender, + req: StreamPublishRequest, ) -> Result { tracing::debug!("writing data to stream"); if req.batch.is_empty() { @@ -65,6 +74,11 @@ impl StreamCtl { .map_err(ShutdownError::from)?; } + // If the earliest recorded timestamp is `None`, then update its value. + if earliest_timestamp.is_none() { + *earliest_timestamp = Some((ts, *current_offset)); + } + tracing::debug!(current_offset, "finished writing data to stream"); let _ = offset_signal.send(*current_offset); Ok(*current_offset) diff --git a/hadron-stream/src/stream/publisher_test.rs b/hadron-stream/src/stream/publisher_test.rs index f6940ca..422583f 100644 --- a/hadron-stream/src/stream/publisher_test.rs +++ b/hadron-stream/src/stream/publisher_test.rs @@ -16,11 +16,11 @@ async fn publish_data_frame_err_with_empty_batch() -> Result<()> { let (config, _tmpdir) = Config::new_test()?; let db = Database::new(config.clone()).await?; let stream_tree = db.get_stream_tree().await?; - let mut current_offset = 0u64; + let (mut current_offset, mut earliest_timestamp) = (0u64, None); let (tx, rx) = watch::channel(current_offset); let req = StreamPublishRequest { batch: vec![], fsync: true, ack: 0 }; - let res = super::StreamCtl::publish_data_frame(&stream_tree, &mut current_offset, &tx, req).await; + let res = super::StreamCtl::publish_data_frame(&stream_tree, &mut current_offset, &mut earliest_timestamp, &tx, req).await; let last_watcher_offset = *rx.borrow(); assert_eq!( @@ -35,6 +35,11 @@ async fn publish_data_frame_err_with_empty_batch() -> Result<()> { matches!(app_err, AppError::InvalidInput(val) if val == "entries batch was empty, no-op"), "unexpected error returned", ); + assert!( + earliest_timestamp.is_none(), + "expected earliest_timestamp to remain `None`, got {:?}", + earliest_timestamp, + ); Ok(()) } @@ -44,7 +49,7 @@ async fn publish_data_frame() -> Result<()> { let (config, _tmpdir) = Config::new_test()?; let db = Database::new(config.clone()).await?; let stream_tree = db.get_stream_tree().await?; - let mut current_offset = 0u64; + let (mut current_offset, mut earliest_timestamp) = (0u64, None); let (tx, rx) = watch::channel(current_offset); let expected_ts_min = chrono::Utc::now().timestamp_millis() - 5; @@ -58,7 +63,7 @@ async fn publish_data_frame() -> Result<()> { }); expected_events.sort_by(|a, b| a.id.cmp(&b.id)); - let last_offset = super::StreamCtl::publish_data_frame(&stream_tree, &mut current_offset, &tx, req).await?; + let last_offset = super::StreamCtl::publish_data_frame(&stream_tree, &mut current_offset, &mut earliest_timestamp, &tx, req).await?; // Check emitted info on last offset. let last_watcher_offset = *rx.borrow(); @@ -124,6 +129,21 @@ async fn publish_data_frame() -> Result<()> { expected_ts_min, ts_idx[0].0 ); + assert!(earliest_timestamp.is_some(), "expected earliest_timestamp to be updated"); + assert_eq!( + earliest_timestamp.map(|val| val.0), + Some(ts_idx[0].0), + "expected earliest_timestamp to be {:?}, got {:?}", + ts_idx[0].0, + earliest_timestamp.map(|val| val.0) + ); + assert_eq!( + earliest_timestamp.map(|val| val.1), + Some(ts_idx[0].1), + "expected earliest_timestamp to be {:?}, got {:?}", + ts_idx[0].1, + earliest_timestamp.map(|val| val.1) + ); Ok(()) } diff --git a/hadron-stream/src/watchers/pipelines.rs b/hadron-stream/src/watchers/pipelines.rs index 85a8d5b..358d878 100644 --- a/hadron-stream/src/watchers/pipelines.rs +++ b/hadron-stream/src/watchers/pipelines.rs @@ -135,6 +135,10 @@ impl PipelineWatcher { let orig = self.pipelines.load(); let mut new_pipelines = HashMap::new(); for pipeline in pipelines { + // Only process Pipelines for this Stream. + if pipeline.spec.source_stream != self.config.stream { + continue; + } let name = match &pipeline.metadata.name { Some(name) => name, None => continue, @@ -186,10 +190,15 @@ impl PipelineWatcher { /// Handle a pipeline applied/updated event. #[tracing::instrument(level = "debug", skip(self, pipeline))] async fn handle_pipeline_applied(&mut self, pipeline: Pipeline) { + // Only process Pipelines for this Stream. + if pipeline.spec.source_stream != self.config.stream { + return; + } let name = match &pipeline.metadata.name { Some(name) => name, None => return, }; + tracing::debug!(%name, "adding new Pipeline CR"); let orig = self.pipelines.load_full(); From ad08d59a6ca60d19f5814a1bd4566942b9d3556b Mon Sep 17 00:00:00 2001 From: Anthony Dodd Date: Thu, 4 Nov 2021 21:27:10 -0500 Subject: [PATCH 07/10] Implement compaction Compaction routine is now well-tested. Woot woot! Operator has been updated to pass along retention policy config to stream. Updated deps across all components. closes #99 --- .../Cargo.lock | 157 +++++------ .../Cargo.toml | 6 +- .../Dockerfile | 2 +- .../Cargo.lock | 157 +++++------ .../Cargo.toml | 6 +- .../Dockerfile | 2 +- hadron-cli/Cargo.lock | 165 ++++-------- hadron-cli/Cargo.toml | 6 +- hadron-cli/Dockerfile | 2 +- hadron-client/Cargo.lock | 145 ++++++---- hadron-client/Cargo.toml | 12 +- hadron-client/src/client/mod.rs | 10 +- hadron-core/Cargo.lock | 151 ++++++----- hadron-core/Cargo.toml | 12 +- hadron-core/src/crd/mod.rs | 4 +- hadron-core/src/crd/stream.rs | 22 +- hadron-operator/Cargo.lock | 251 ++++++++---------- hadron-operator/Cargo.toml | 17 +- hadron-operator/Dockerfile | 2 +- hadron-operator/src/grpc/operator.rs | 2 +- hadron-operator/src/k8s/coordination.rs | 2 +- hadron-operator/src/k8s/data.rs | 2 +- hadron-operator/src/k8s/mod.rs | 2 +- hadron-operator/src/k8s/scheduler.rs | 15 ++ hadron-operator/src/server/webhook.rs | 2 +- hadron-stream/Cargo.lock | 243 ++++++++--------- hadron-stream/Cargo.toml | 17 +- hadron-stream/Dockerfile | 2 +- hadron-stream/src/app.rs | 2 +- hadron-stream/src/config.rs | 7 + hadron-stream/src/config_test.rs | 173 ++++++++++++ hadron-stream/src/grpc/stream.rs | 20 +- hadron-stream/src/main.rs | 2 + hadron-stream/src/stream/mod.rs | 164 ++++++++++-- hadron-stream/src/stream/mod_test.rs | 150 ++++++++++- hadron-stream/src/stream/subscriber.rs | 11 - hadron-stream/src/watchers/pipelines.rs | 6 +- hadron-stream/src/watchers/stream.rs | 2 +- hadron-stream/src/watchers/tokens.rs | 2 +- 39 files changed, 1133 insertions(+), 822 deletions(-) create mode 100644 hadron-stream/src/config_test.rs diff --git a/examples/pipeline-transactional-processing/Cargo.lock b/examples/pipeline-transactional-processing/Cargo.lock index 46c155f..8440800 100644 --- a/examples/pipeline-transactional-processing/Cargo.lock +++ b/examples/pipeline-transactional-processing/Cargo.lock @@ -4,9 +4,9 @@ version = 3 [[package]] name = "ahash" -version = "0.7.4" +version = "0.7.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "43bb833f0bf979d8475d38fbf09ed3b8a55e1885fe93ad3f93239fc6a4f17b98" +checksum = "fcb51a0695d8f838b1ee009b3fbf66bda078cd64590202a864a8f3e8c4315c47" dependencies = [ "getrandom", "once_cell", @@ -24,9 +24,9 @@ dependencies = [ [[package]] name = "anyhow" -version = "1.0.44" +version = "1.0.45" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "61604a8f862e1d5c3229fdd78f8b02c68dcf73a4c4b05fd636d12240aaa242c1" +checksum = "ee10e43ae4a853c0a3591d4e2ada1719e553be18199d9da9d4a83f5927c2f5c7" [[package]] name = "arc-swap" @@ -110,9 +110,9 @@ dependencies = [ [[package]] name = "bumpalo" -version = "3.7.1" +version = "3.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d9df67f7bf9ef8498769f994239c45613ef0c5899415fb58e9add412d2c1a538" +checksum = "8f1e260c3a9040a7c19a12468758f4c16f31a81a1fe087482be9570ec864bb6c" [[package]] name = "byteorder" @@ -128,9 +128,9 @@ checksum = "c4872d67bab6358e59559027aa3b9157c53d9358c51423c17554809a8858e0f8" [[package]] name = "cc" -version = "1.0.70" +version = "1.0.71" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d26a6ce4b6a484fa3edb70f7efa6fc430fd2b87285fe8b84304fd0936faa0dc0" +checksum = "79c2681d6594606957bbb8631c4b90a7fcaaa72cdb714743a437b156d6a7eedd" [[package]] name = "cfg-if" @@ -138,18 +138,6 @@ version = "1.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" -[[package]] -name = "chrono" -version = "0.4.19" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "670ad68c9088c2a963aaa298cb369688cf3f9465ce5e2d4ca10e6e0098a1ce73" -dependencies = [ - "libc", - "num-integer", - "num-traits", - "winapi", -] - [[package]] name = "const_fn" version = "0.4.8" @@ -167,9 +155,9 @@ dependencies = [ [[package]] name = "crc" -version = "2.0.0" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "10c2722795460108a7872e1cd933a85d6ec38abc4baecad51028f702da28889f" +checksum = "49fc9a695bca7f35f5f4c15cddc84415f66a74ea78eef08e90c5024f2b540e23" dependencies = [ "crc-catalog", ] @@ -414,9 +402,9 @@ dependencies = [ [[package]] name = "h2" -version = "0.3.6" +version = "0.3.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c06815895acec637cd6ed6e9662c935b866d20a106f8361892893a7d9234964" +checksum = "7fd819562fcebdac5afc5c113c3ec36f902840b70fd4fc458799c8ce4607ae55" dependencies = [ "bytes", "fnv", @@ -518,9 +506,9 @@ dependencies = [ [[package]] name = "http-body" -version = "0.4.3" +version = "0.4.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "399c583b2979440c60be0821a6199eca73bc3c8dcd9d070d75ac726e2c6186e5" +checksum = "1ff4f84919677303da5f147645dbea6b1881f368d03ac84e1dc09031ebd7b2c6" dependencies = [ "bytes", "http", @@ -541,9 +529,9 @@ checksum = "6456b8a6c8f33fee7d958fcd1b60d55b11940a79e63ae87013e6d22e26034440" [[package]] name = "hyper" -version = "0.14.13" +version = "0.14.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "15d1cfb9e4f68655fa04c01f59edb405b6074a0f7118ea881e5026e4a1cd8593" +checksum = "2b91bb1f221b6ea1f1e4371216b70f40748774c2fb5971b450c07773fb92d26b" dependencies = [ "bytes", "futures-channel", @@ -598,9 +586,9 @@ dependencies = [ [[package]] name = "instant" -version = "0.1.11" +version = "0.1.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "716d3d89f35ac6a34fd0eed635395f4c3b76fa889338a4632e5231a8684216bd" +checksum = "7a5bbe824c507c5da5956355e86a746d82e0e1464f65d862cc5e71da70e94b2c" dependencies = [ "cfg-if", ] @@ -637,9 +625,9 @@ checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" [[package]] name = "libc" -version = "0.2.103" +version = "0.2.106" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dd8f7255a17a627354f321ef0055d63b898c6fb27eff628af4d1b66b7331edf6" +checksum = "a60553f9a9e039a333b4e9b20573b9e9b9c0bb3a11e201ccc48ef4283456d673" [[package]] name = "lock_api" @@ -661,9 +649,9 @@ dependencies = [ [[package]] name = "matchers" -version = "0.0.1" +version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f099785f7595cc4b4553a174ce30dd7589ef93391ff414dbb67f62392b9e0ce1" +checksum = "8263075bb86c5a1b1427b5ae862e8889656f126e9f77c484496e8b47cf5c5558" dependencies = [ "regex-automata", ] @@ -693,15 +681,15 @@ checksum = "308cc39be01b73d0d18f82a0e7b2a3df85245f84af96fdddc5d202d27e47b86a" [[package]] name = "minimal-lexical" -version = "0.1.4" +version = "0.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c64630dcdd71f1a64c435f54885086a0de5d6a12d104d69b165fb7d5286d677" +checksum = "68354c5c6bd36d73ff3feceb05efa59b6acb7626617f4962be322a825e61f79a" [[package]] name = "mio" -version = "0.7.13" +version = "0.7.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8c2bdb6314ec10835cd3293dd268473a835c02b7b352e788be788b3c6ca6bb16" +checksum = "8067b404fe97c70829f082dec8bcf4f71225d7eaea1d8645349cb76fa06205cc" dependencies = [ "libc", "log", @@ -721,9 +709,9 @@ dependencies = [ [[package]] name = "nom" -version = "7.0.0" +version = "7.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7ffd9d26838a953b4af82cbeb9f1592c6798916983959be223a7124e992742c1" +checksum = "1b1d11e1ef389c76fe5b81bcaf2ea32cf88b62bc494e19f493d0b30e7a930109" dependencies = [ "memchr", "minimal-lexical", @@ -739,16 +727,6 @@ dependencies = [ "winapi", ] -[[package]] -name = "num-integer" -version = "0.1.44" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2cc698a63b549a70bc047073d2949cce27cd1c7b0a4a862d08a8031bc2801db" -dependencies = [ - "autocfg", - "num-traits", -] - [[package]] name = "num-traits" version = "0.2.14" @@ -857,9 +835,9 @@ dependencies = [ [[package]] name = "ppv-lite86" -version = "0.2.10" +version = "0.2.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ac74c624d6b2d21f425f752262f42188365d7b8ff1aff74c82e45136510a4857" +checksum = "ed0cfbc8191465bed66e1718596ee0b0b35d5ee1f41c5df2189d0fe8bde535ba" [[package]] name = "proc-macro-hack" @@ -875,9 +853,9 @@ checksum = "bc881b2c22681370c6a780e47af9840ef841837bc98118431d4e1868bd0c1086" [[package]] name = "proc-macro2" -version = "1.0.29" +version = "1.0.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b9f5105d4fdaab20335ca9565e106a5d9b82b6219b5ba735731124ac6711d23d" +checksum = "ba508cc11742c0dc5c1659771673afbab7a0efab23aa17e854cbab0837ed0b43" dependencies = [ "unicode-xid", ] @@ -1143,9 +1121,9 @@ dependencies = [ [[package]] name = "sharded-slab" -version = "0.1.3" +version = "0.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "740223c51853f3145fe7c90360d2d4232f2b62e3449489c207eccde818979982" +checksum = "900fba806f70c630b0a382d0d825e17a0f19fcd059a2ade1ff237bcddf446b31" dependencies = [ "lazy_static", ] @@ -1161,9 +1139,9 @@ dependencies = [ [[package]] name = "slab" -version = "0.4.4" +version = "0.4.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c307a32c1c5c437f38c7fd45d753050587732ba8628319fbdf12a7e289ccc590" +checksum = "9def91fd1e018fe007022791f865d0ccc9b3a0d5001e01aabb8b40e46000afb5" [[package]] name = "smallvec" @@ -1370,9 +1348,9 @@ checksum = "6bdef32e8150c2a081110b42772ffe7d7c9032b606bc226c8260fd97e0976601" [[package]] name = "syn" -version = "1.0.78" +version = "1.0.81" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4eac2e6c19f5c3abc0c229bea31ff0b9b091c7b14990e8924b92902a303a0c0" +checksum = "f2afee18b8beb5a596ecb4a2dce128c719b4ba399d34126b9e4396e3f9860966" dependencies = [ "proc-macro2", "quote", @@ -1381,18 +1359,18 @@ dependencies = [ [[package]] name = "thiserror" -version = "1.0.29" +version = "1.0.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "602eca064b2d83369e2b2f34b09c70b605402801927c65c11071ac911d299b88" +checksum = "854babe52e4df1653706b98fcfc05843010039b406875930a70e4d9644e5c417" dependencies = [ "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "1.0.29" +version = "1.0.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bad553cc2c78e8de258400763a647e80e6d1b31ee237275d756f6836d204494c" +checksum = "aa32fd3f627f367fe16f893e2597ae3c05020f8bba2666a4e6ea73d377e5714b" dependencies = [ "proc-macro2", "quote", @@ -1463,9 +1441,9 @@ checksum = "cda74da7e1a664f795bb1f8a87ec406fb89a02522cf6e50620d016add6dbbf5c" [[package]] name = "tokio" -version = "1.12.0" +version = "1.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c2c2416fdedca8443ae44b4527de1ea633af61d8f7169ffa6e72c5b53d24efcc" +checksum = "588b2d10a336da58d877567cd8fb8a14b463e2104910f8132cd054b4b96e29ee" dependencies = [ "autocfg", "bytes", @@ -1492,9 +1470,9 @@ dependencies = [ [[package]] name = "tokio-macros" -version = "1.4.1" +version = "1.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "154794c8f499c2619acd19e839294703e9e32e7630ef5f46ea80d4ef0fbee5eb" +checksum = "114383b041aa6212c579467afa0075fbbdd0718de036100bc0ba7961d8cb9095" dependencies = [ "proc-macro2", "quote", @@ -1514,9 +1492,9 @@ dependencies = [ [[package]] name = "tokio-stream" -version = "0.1.7" +version = "0.1.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b2f3f698253f03119ac0102beaa64f67a67e08074d03a22d18784104543727f" +checksum = "50145484efff8818b5ccd256697f36863f587da82cf8b409c53adf1e840798e3" dependencies = [ "futures-core", "pin-project-lite", @@ -1526,9 +1504,9 @@ dependencies = [ [[package]] name = "tokio-util" -version = "0.6.8" +version = "0.6.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08d3725d3efa29485e87311c5b699de63cde14b00ed4d256b8318aa30ca452cd" +checksum = "9e99e1983e5d376cd8eb4b66604d2e99e79f5bd988c3055891dcd8c9e2604cc0" dependencies = [ "bytes", "futures-core", @@ -1572,14 +1550,15 @@ dependencies = [ [[package]] name = "tower" -version = "0.4.8" +version = "0.4.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f60422bc7fefa2f3ec70359b8ff1caff59d785877eb70595904605bcc412470f" +checksum = "c00e500fff5fa1131c866b246041a6bf96da9c965f8fe4128cb1421f23e93c00" dependencies = [ "futures-core", "futures-util", "indexmap", "pin-project", + "pin-project-lite", "rand", "slab", "tokio", @@ -1604,9 +1583,9 @@ checksum = "360dfd1d6d30e05fda32ace2c8c70e9c0a9da713275777f5a4dbb8a1893930c6" [[package]] name = "tracing" -version = "0.1.28" +version = "0.1.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "84f96e095c0c82419687c20ddf5cb3eadb61f4e1405923c9dc8e53a1adacbda8" +checksum = "375a639232caf30edfc78e8d89b2d4c375515393e7af7e16f01cd96917fb2105" dependencies = [ "cfg-if", "log", @@ -1617,9 +1596,9 @@ dependencies = [ [[package]] name = "tracing-attributes" -version = "0.1.16" +version = "0.1.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "98863d0dd09fa59a1b79c6750ad80dbda6b75f4e71c437a6a1a8cb91a8bcbd77" +checksum = "f4f480b8f81512e825f337ad51e94c1eb5d3bbdf2b363dcd01e2b19a9ffe3f8e" dependencies = [ "proc-macro2", "quote", @@ -1656,36 +1635,22 @@ dependencies = [ "tracing-core", ] -[[package]] -name = "tracing-serde" -version = "0.1.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fb65ea441fbb84f9f6748fd496cf7f63ec9af5bca94dd86456978d055e8eb28b" -dependencies = [ - "serde", - "tracing-core", -] - [[package]] name = "tracing-subscriber" -version = "0.2.24" +version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fdd0568dbfe3baf7048b7908d2b32bca0d81cd56bec6d2a8f894b01d74f86be3" +checksum = "80a4ddde70311d8da398062ecf6fc2c309337de6b0f77d6c27aff8d53f6fca52" dependencies = [ "ansi_term", - "chrono", "lazy_static", "matchers", "regex", - "serde", - "serde_json", "sharded-slab", "smallvec", "thread_local", "tracing", "tracing-core", "tracing-log", - "tracing-serde", ] [[package]] @@ -1702,9 +1667,9 @@ checksum = "b63708a265f51345575b27fe43f9500ad611579e764c79edbc2037b1121959ec" [[package]] name = "unicode-bidi" -version = "0.3.6" +version = "0.3.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "246f4c42e67e7a4e3c6106ff716a5d067d4132a642840b242e357e468a2a0085" +checksum = "1a01404663e3db436ed2746d9fefef640d868edae3cceb81c3b8d5732fda678f" [[package]] name = "unicode-normalization" diff --git a/examples/pipeline-transactional-processing/Cargo.toml b/examples/pipeline-transactional-processing/Cargo.toml index 5f62f8d..3c21e0b 100644 --- a/examples/pipeline-transactional-processing/Cargo.toml +++ b/examples/pipeline-transactional-processing/Cargo.toml @@ -1,11 +1,11 @@ [package] name = "pipeline-txp" version = "0.1.0" -edition = "2018" +edition = "2021" authors = ["Anthony Dodd "] [dependencies] -anyhow = "1.0.44" +anyhow = "1" hadron = { package = "hadron-client", version = "0.1.0-beta.1" } sqlx = { version = "0.5", features = [ "runtime-tokio-rustls", "postgres", "offline", "time" ] } tokio = { version = "1", default-features = false, features = [ @@ -13,4 +13,4 @@ tokio = { version = "1", default-features = false, features = [ "rt", "macros", "sync", "signal", ]} tracing = { version = "0.1", features = ["attributes"] } -tracing-subscriber = "0.2" +tracing-subscriber = { version = "0.3", features = ["env-filter", "registry"] } diff --git a/examples/pipeline-transactional-processing/Dockerfile b/examples/pipeline-transactional-processing/Dockerfile index 875a139..e0e60c1 100644 --- a/examples/pipeline-transactional-processing/Dockerfile +++ b/examples/pipeline-transactional-processing/Dockerfile @@ -1,7 +1,7 @@ ## Dockerfile for pipeline-transactional-processing (txp) demo app. # syntax=docker/dockerfile:1.3 -ARG RUST_TAG=1.56.0 +ARG RUST_TAG=1.56.1 ARG ALPINE_TAG=3.14 FROM rust:${RUST_TAG}-alpine${ALPINE_TAG} as builder diff --git a/examples/stream-transactional-processing/Cargo.lock b/examples/stream-transactional-processing/Cargo.lock index dbddb39..b8f7b72 100644 --- a/examples/stream-transactional-processing/Cargo.lock +++ b/examples/stream-transactional-processing/Cargo.lock @@ -4,9 +4,9 @@ version = 3 [[package]] name = "ahash" -version = "0.7.4" +version = "0.7.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "43bb833f0bf979d8475d38fbf09ed3b8a55e1885fe93ad3f93239fc6a4f17b98" +checksum = "fcb51a0695d8f838b1ee009b3fbf66bda078cd64590202a864a8f3e8c4315c47" dependencies = [ "getrandom", "once_cell", @@ -24,9 +24,9 @@ dependencies = [ [[package]] name = "anyhow" -version = "1.0.44" +version = "1.0.45" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "61604a8f862e1d5c3229fdd78f8b02c68dcf73a4c4b05fd636d12240aaa242c1" +checksum = "ee10e43ae4a853c0a3591d4e2ada1719e553be18199d9da9d4a83f5927c2f5c7" [[package]] name = "arc-swap" @@ -104,9 +104,9 @@ dependencies = [ [[package]] name = "bumpalo" -version = "3.7.1" +version = "3.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d9df67f7bf9ef8498769f994239c45613ef0c5899415fb58e9add412d2c1a538" +checksum = "8f1e260c3a9040a7c19a12468758f4c16f31a81a1fe087482be9570ec864bb6c" [[package]] name = "byteorder" @@ -122,9 +122,9 @@ checksum = "c4872d67bab6358e59559027aa3b9157c53d9358c51423c17554809a8858e0f8" [[package]] name = "cc" -version = "1.0.70" +version = "1.0.71" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d26a6ce4b6a484fa3edb70f7efa6fc430fd2b87285fe8b84304fd0936faa0dc0" +checksum = "79c2681d6594606957bbb8631c4b90a7fcaaa72cdb714743a437b156d6a7eedd" [[package]] name = "cfg-if" @@ -132,18 +132,6 @@ version = "1.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" -[[package]] -name = "chrono" -version = "0.4.19" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "670ad68c9088c2a963aaa298cb369688cf3f9465ce5e2d4ca10e6e0098a1ce73" -dependencies = [ - "libc", - "num-integer", - "num-traits", - "winapi", -] - [[package]] name = "cpufeatures" version = "0.2.1" @@ -155,9 +143,9 @@ dependencies = [ [[package]] name = "crc" -version = "2.0.0" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "10c2722795460108a7872e1cd933a85d6ec38abc4baecad51028f702da28889f" +checksum = "49fc9a695bca7f35f5f4c15cddc84415f66a74ea78eef08e90c5024f2b540e23" dependencies = [ "crc-catalog", ] @@ -396,9 +384,9 @@ dependencies = [ [[package]] name = "h2" -version = "0.3.6" +version = "0.3.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c06815895acec637cd6ed6e9662c935b866d20a106f8361892893a7d9234964" +checksum = "7fd819562fcebdac5afc5c113c3ec36f902840b70fd4fc458799c8ce4607ae55" dependencies = [ "bytes", "fnv", @@ -500,9 +488,9 @@ dependencies = [ [[package]] name = "http-body" -version = "0.4.3" +version = "0.4.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "399c583b2979440c60be0821a6199eca73bc3c8dcd9d070d75ac726e2c6186e5" +checksum = "1ff4f84919677303da5f147645dbea6b1881f368d03ac84e1dc09031ebd7b2c6" dependencies = [ "bytes", "http", @@ -523,9 +511,9 @@ checksum = "6456b8a6c8f33fee7d958fcd1b60d55b11940a79e63ae87013e6d22e26034440" [[package]] name = "hyper" -version = "0.14.13" +version = "0.14.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "15d1cfb9e4f68655fa04c01f59edb405b6074a0f7118ea881e5026e4a1cd8593" +checksum = "2b91bb1f221b6ea1f1e4371216b70f40748774c2fb5971b450c07773fb92d26b" dependencies = [ "bytes", "futures-channel", @@ -580,9 +568,9 @@ dependencies = [ [[package]] name = "instant" -version = "0.1.11" +version = "0.1.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "716d3d89f35ac6a34fd0eed635395f4c3b76fa889338a4632e5231a8684216bd" +checksum = "7a5bbe824c507c5da5956355e86a746d82e0e1464f65d862cc5e71da70e94b2c" dependencies = [ "cfg-if", ] @@ -619,9 +607,9 @@ checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" [[package]] name = "libc" -version = "0.2.103" +version = "0.2.106" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dd8f7255a17a627354f321ef0055d63b898c6fb27eff628af4d1b66b7331edf6" +checksum = "a60553f9a9e039a333b4e9b20573b9e9b9c0bb3a11e201ccc48ef4283456d673" [[package]] name = "lock_api" @@ -643,9 +631,9 @@ dependencies = [ [[package]] name = "matchers" -version = "0.0.1" +version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f099785f7595cc4b4553a174ce30dd7589ef93391ff414dbb67f62392b9e0ce1" +checksum = "8263075bb86c5a1b1427b5ae862e8889656f126e9f77c484496e8b47cf5c5558" dependencies = [ "regex-automata", ] @@ -675,15 +663,15 @@ checksum = "308cc39be01b73d0d18f82a0e7b2a3df85245f84af96fdddc5d202d27e47b86a" [[package]] name = "minimal-lexical" -version = "0.1.4" +version = "0.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c64630dcdd71f1a64c435f54885086a0de5d6a12d104d69b165fb7d5286d677" +checksum = "68354c5c6bd36d73ff3feceb05efa59b6acb7626617f4962be322a825e61f79a" [[package]] name = "mio" -version = "0.7.13" +version = "0.7.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8c2bdb6314ec10835cd3293dd268473a835c02b7b352e788be788b3c6ca6bb16" +checksum = "8067b404fe97c70829f082dec8bcf4f71225d7eaea1d8645349cb76fa06205cc" dependencies = [ "libc", "log", @@ -703,9 +691,9 @@ dependencies = [ [[package]] name = "nom" -version = "7.0.0" +version = "7.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7ffd9d26838a953b4af82cbeb9f1592c6798916983959be223a7124e992742c1" +checksum = "1b1d11e1ef389c76fe5b81bcaf2ea32cf88b62bc494e19f493d0b30e7a930109" dependencies = [ "memchr", "minimal-lexical", @@ -721,16 +709,6 @@ dependencies = [ "winapi", ] -[[package]] -name = "num-integer" -version = "0.1.44" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2cc698a63b549a70bc047073d2949cce27cd1c7b0a4a862d08a8031bc2801db" -dependencies = [ - "autocfg", - "num-traits", -] - [[package]] name = "num-traits" version = "0.2.14" @@ -827,9 +805,9 @@ checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" [[package]] name = "ppv-lite86" -version = "0.2.10" +version = "0.2.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ac74c624d6b2d21f425f752262f42188365d7b8ff1aff74c82e45136510a4857" +checksum = "ed0cfbc8191465bed66e1718596ee0b0b35d5ee1f41c5df2189d0fe8bde535ba" [[package]] name = "proc-macro-hack" @@ -845,9 +823,9 @@ checksum = "bc881b2c22681370c6a780e47af9840ef841837bc98118431d4e1868bd0c1086" [[package]] name = "proc-macro2" -version = "1.0.29" +version = "1.0.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b9f5105d4fdaab20335ca9565e106a5d9b82b6219b5ba735731124ac6711d23d" +checksum = "ba508cc11742c0dc5c1659771673afbab7a0efab23aa17e854cbab0837ed0b43" dependencies = [ "unicode-xid", ] @@ -1083,9 +1061,9 @@ dependencies = [ [[package]] name = "sharded-slab" -version = "0.1.3" +version = "0.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "740223c51853f3145fe7c90360d2d4232f2b62e3449489c207eccde818979982" +checksum = "900fba806f70c630b0a382d0d825e17a0f19fcd059a2ade1ff237bcddf446b31" dependencies = [ "lazy_static", ] @@ -1101,9 +1079,9 @@ dependencies = [ [[package]] name = "slab" -version = "0.4.4" +version = "0.4.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c307a32c1c5c437f38c7fd45d753050587732ba8628319fbdf12a7e289ccc590" +checksum = "9def91fd1e018fe007022791f865d0ccc9b3a0d5001e01aabb8b40e46000afb5" [[package]] name = "smallvec" @@ -1263,9 +1241,9 @@ checksum = "6bdef32e8150c2a081110b42772ffe7d7c9032b606bc226c8260fd97e0976601" [[package]] name = "syn" -version = "1.0.78" +version = "1.0.81" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4eac2e6c19f5c3abc0c229bea31ff0b9b091c7b14990e8924b92902a303a0c0" +checksum = "f2afee18b8beb5a596ecb4a2dce128c719b4ba399d34126b9e4396e3f9860966" dependencies = [ "proc-macro2", "quote", @@ -1274,18 +1252,18 @@ dependencies = [ [[package]] name = "thiserror" -version = "1.0.29" +version = "1.0.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "602eca064b2d83369e2b2f34b09c70b605402801927c65c11071ac911d299b88" +checksum = "854babe52e4df1653706b98fcfc05843010039b406875930a70e4d9644e5c417" dependencies = [ "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "1.0.29" +version = "1.0.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bad553cc2c78e8de258400763a647e80e6d1b31ee237275d756f6836d204494c" +checksum = "aa32fd3f627f367fe16f893e2597ae3c05020f8bba2666a4e6ea73d377e5714b" dependencies = [ "proc-macro2", "quote", @@ -1318,9 +1296,9 @@ checksum = "cda74da7e1a664f795bb1f8a87ec406fb89a02522cf6e50620d016add6dbbf5c" [[package]] name = "tokio" -version = "1.12.0" +version = "1.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c2c2416fdedca8443ae44b4527de1ea633af61d8f7169ffa6e72c5b53d24efcc" +checksum = "588b2d10a336da58d877567cd8fb8a14b463e2104910f8132cd054b4b96e29ee" dependencies = [ "autocfg", "bytes", @@ -1347,9 +1325,9 @@ dependencies = [ [[package]] name = "tokio-macros" -version = "1.4.1" +version = "1.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "154794c8f499c2619acd19e839294703e9e32e7630ef5f46ea80d4ef0fbee5eb" +checksum = "114383b041aa6212c579467afa0075fbbdd0718de036100bc0ba7961d8cb9095" dependencies = [ "proc-macro2", "quote", @@ -1369,9 +1347,9 @@ dependencies = [ [[package]] name = "tokio-stream" -version = "0.1.7" +version = "0.1.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b2f3f698253f03119ac0102beaa64f67a67e08074d03a22d18784104543727f" +checksum = "50145484efff8818b5ccd256697f36863f587da82cf8b409c53adf1e840798e3" dependencies = [ "futures-core", "pin-project-lite", @@ -1381,9 +1359,9 @@ dependencies = [ [[package]] name = "tokio-util" -version = "0.6.8" +version = "0.6.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08d3725d3efa29485e87311c5b699de63cde14b00ed4d256b8318aa30ca452cd" +checksum = "9e99e1983e5d376cd8eb4b66604d2e99e79f5bd988c3055891dcd8c9e2604cc0" dependencies = [ "bytes", "futures-core", @@ -1427,14 +1405,15 @@ dependencies = [ [[package]] name = "tower" -version = "0.4.8" +version = "0.4.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f60422bc7fefa2f3ec70359b8ff1caff59d785877eb70595904605bcc412470f" +checksum = "c00e500fff5fa1131c866b246041a6bf96da9c965f8fe4128cb1421f23e93c00" dependencies = [ "futures-core", "futures-util", "indexmap", "pin-project", + "pin-project-lite", "rand", "slab", "tokio", @@ -1459,9 +1438,9 @@ checksum = "360dfd1d6d30e05fda32ace2c8c70e9c0a9da713275777f5a4dbb8a1893930c6" [[package]] name = "tracing" -version = "0.1.28" +version = "0.1.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "84f96e095c0c82419687c20ddf5cb3eadb61f4e1405923c9dc8e53a1adacbda8" +checksum = "375a639232caf30edfc78e8d89b2d4c375515393e7af7e16f01cd96917fb2105" dependencies = [ "cfg-if", "log", @@ -1472,9 +1451,9 @@ dependencies = [ [[package]] name = "tracing-attributes" -version = "0.1.16" +version = "0.1.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "98863d0dd09fa59a1b79c6750ad80dbda6b75f4e71c437a6a1a8cb91a8bcbd77" +checksum = "f4f480b8f81512e825f337ad51e94c1eb5d3bbdf2b363dcd01e2b19a9ffe3f8e" dependencies = [ "proc-macro2", "quote", @@ -1511,36 +1490,22 @@ dependencies = [ "tracing-core", ] -[[package]] -name = "tracing-serde" -version = "0.1.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fb65ea441fbb84f9f6748fd496cf7f63ec9af5bca94dd86456978d055e8eb28b" -dependencies = [ - "serde", - "tracing-core", -] - [[package]] name = "tracing-subscriber" -version = "0.2.24" +version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fdd0568dbfe3baf7048b7908d2b32bca0d81cd56bec6d2a8f894b01d74f86be3" +checksum = "80a4ddde70311d8da398062ecf6fc2c309337de6b0f77d6c27aff8d53f6fca52" dependencies = [ "ansi_term", - "chrono", "lazy_static", "matchers", "regex", - "serde", - "serde_json", "sharded-slab", "smallvec", "thread_local", "tracing", "tracing-core", "tracing-log", - "tracing-serde", ] [[package]] @@ -1557,9 +1522,9 @@ checksum = "b63708a265f51345575b27fe43f9500ad611579e764c79edbc2037b1121959ec" [[package]] name = "unicode-bidi" -version = "0.3.6" +version = "0.3.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "246f4c42e67e7a4e3c6106ff716a5d067d4132a642840b242e357e468a2a0085" +checksum = "1a01404663e3db436ed2746d9fefef640d868edae3cceb81c3b8d5732fda678f" [[package]] name = "unicode-normalization" diff --git a/examples/stream-transactional-processing/Cargo.toml b/examples/stream-transactional-processing/Cargo.toml index 25d41f6..eb17b31 100644 --- a/examples/stream-transactional-processing/Cargo.toml +++ b/examples/stream-transactional-processing/Cargo.toml @@ -1,11 +1,11 @@ [package] name = "stream-txp" version = "0.1.0" -edition = "2018" +edition = "2021" authors = ["Anthony Dodd "] [dependencies] -anyhow = "1.0.44" +anyhow = "1" hadron = { package = "hadron-client", version = "0.1.0-beta.1" } sqlx = { version = "0.5", features = [ "runtime-tokio-rustls", "postgres", "offline" ] } tokio = { version = "1", default-features = false, features = [ @@ -13,4 +13,4 @@ tokio = { version = "1", default-features = false, features = [ "rt", "macros", "sync", "signal", ]} tracing = { version = "0.1", features = ["attributes"] } -tracing-subscriber = "0.2" +tracing-subscriber = { version = "0.3", features = ["env-filter", "registry"] } diff --git a/examples/stream-transactional-processing/Dockerfile b/examples/stream-transactional-processing/Dockerfile index 5bee7cb..6fed233 100644 --- a/examples/stream-transactional-processing/Dockerfile +++ b/examples/stream-transactional-processing/Dockerfile @@ -1,7 +1,7 @@ ## Dockerfile for stream-transactional-processing (txp) demo app. # syntax=docker/dockerfile:1.3 -ARG RUST_TAG=1.56.0 +ARG RUST_TAG=1.56.1 ARG ALPINE_TAG=3.14 FROM rust:${RUST_TAG}-alpine${ALPINE_TAG} as builder diff --git a/hadron-cli/Cargo.lock b/hadron-cli/Cargo.lock index 0274000..4f5b2e3 100644 --- a/hadron-cli/Cargo.lock +++ b/hadron-cli/Cargo.lock @@ -31,9 +31,9 @@ dependencies = [ [[package]] name = "anyhow" -version = "1.0.44" +version = "1.0.45" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "61604a8f862e1d5c3229fdd78f8b02c68dcf73a4c4b05fd636d12240aaa242c1" +checksum = "ee10e43ae4a853c0a3591d4e2ada1719e553be18199d9da9d4a83f5927c2f5c7" [[package]] name = "arc-swap" @@ -104,9 +104,9 @@ checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" [[package]] name = "bumpalo" -version = "3.7.1" +version = "3.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d9df67f7bf9ef8498769f994239c45613ef0c5899415fb58e9add412d2c1a538" +checksum = "8f1e260c3a9040a7c19a12468758f4c16f31a81a1fe087482be9570ec864bb6c" [[package]] name = "bytes" @@ -116,9 +116,9 @@ checksum = "c4872d67bab6358e59559027aa3b9157c53d9358c51423c17554809a8858e0f8" [[package]] name = "cc" -version = "1.0.70" +version = "1.0.71" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d26a6ce4b6a484fa3edb70f7efa6fc430fd2b87285fe8b84304fd0936faa0dc0" +checksum = "79c2681d6594606957bbb8631c4b90a7fcaaa72cdb714743a437b156d6a7eedd" [[package]] name = "cfg-if" @@ -126,18 +126,6 @@ version = "1.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" -[[package]] -name = "chrono" -version = "0.4.19" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "670ad68c9088c2a963aaa298cb369688cf3f9465ce5e2d4ca10e6e0098a1ce73" -dependencies = [ - "libc", - "num-integer", - "num-traits", - "winapi", -] - [[package]] name = "clap" version = "2.33.3" @@ -272,9 +260,9 @@ dependencies = [ [[package]] name = "h2" -version = "0.3.6" +version = "0.3.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c06815895acec637cd6ed6e9662c935b866d20a106f8361892893a7d9234964" +checksum = "7fd819562fcebdac5afc5c113c3ec36f902840b70fd4fc458799c8ce4607ae55" dependencies = [ "bytes", "fnv", @@ -367,9 +355,9 @@ dependencies = [ [[package]] name = "http-body" -version = "0.4.3" +version = "0.4.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "399c583b2979440c60be0821a6199eca73bc3c8dcd9d070d75ac726e2c6186e5" +checksum = "1ff4f84919677303da5f147645dbea6b1881f368d03ac84e1dc09031ebd7b2c6" dependencies = [ "bytes", "http", @@ -390,9 +378,9 @@ checksum = "6456b8a6c8f33fee7d958fcd1b60d55b11940a79e63ae87013e6d22e26034440" [[package]] name = "hyper" -version = "0.14.13" +version = "0.14.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "15d1cfb9e4f68655fa04c01f59edb405b6074a0f7118ea881e5026e4a1cd8593" +checksum = "2b91bb1f221b6ea1f1e4371216b70f40748774c2fb5971b450c07773fb92d26b" dependencies = [ "bytes", "futures-channel", @@ -466,9 +454,9 @@ checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" [[package]] name = "libc" -version = "0.2.103" +version = "0.2.106" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dd8f7255a17a627354f321ef0055d63b898c6fb27eff628af4d1b66b7331edf6" +checksum = "a60553f9a9e039a333b4e9b20573b9e9b9c0bb3a11e201ccc48ef4283456d673" [[package]] name = "log" @@ -481,9 +469,9 @@ dependencies = [ [[package]] name = "matchers" -version = "0.0.1" +version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f099785f7595cc4b4553a174ce30dd7589ef93391ff414dbb67f62392b9e0ce1" +checksum = "8263075bb86c5a1b1427b5ae862e8889656f126e9f77c484496e8b47cf5c5558" dependencies = [ "regex-automata", ] @@ -496,9 +484,9 @@ checksum = "308cc39be01b73d0d18f82a0e7b2a3df85245f84af96fdddc5d202d27e47b86a" [[package]] name = "mio" -version = "0.7.13" +version = "0.7.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8c2bdb6314ec10835cd3293dd268473a835c02b7b352e788be788b3c6ca6bb16" +checksum = "8067b404fe97c70829f082dec8bcf4f71225d7eaea1d8645349cb76fa06205cc" dependencies = [ "libc", "log", @@ -525,25 +513,6 @@ dependencies = [ "winapi", ] -[[package]] -name = "num-integer" -version = "0.1.44" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2cc698a63b549a70bc047073d2949cce27cd1c7b0a4a862d08a8031bc2801db" -dependencies = [ - "autocfg", - "num-traits", -] - -[[package]] -name = "num-traits" -version = "0.2.14" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9a64b1ec5cda2586e284722486d802acf1f7dbdc623e2bfc57e65ca1cd099290" -dependencies = [ - "autocfg", -] - [[package]] name = "once_cell" version = "1.8.0" @@ -590,9 +559,9 @@ checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" [[package]] name = "ppv-lite86" -version = "0.2.10" +version = "0.2.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ac74c624d6b2d21f425f752262f42188365d7b8ff1aff74c82e45136510a4857" +checksum = "ed0cfbc8191465bed66e1718596ee0b0b35d5ee1f41c5df2189d0fe8bde535ba" [[package]] name = "proc-macro-error" @@ -632,9 +601,9 @@ checksum = "bc881b2c22681370c6a780e47af9840ef841837bc98118431d4e1868bd0c1086" [[package]] name = "proc-macro2" -version = "1.0.29" +version = "1.0.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b9f5105d4fdaab20335ca9565e106a5d9b82b6219b5ba735731124ac6711d23d" +checksum = "ba508cc11742c0dc5c1659771673afbab7a0efab23aa17e854cbab0837ed0b43" dependencies = [ "unicode-xid", ] @@ -765,12 +734,6 @@ dependencies = [ "webpki", ] -[[package]] -name = "ryu" -version = "1.0.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "71d301d4193d031abdd79ff7e3dd721168a9572ef3fe51a1517aba235bd8f86e" - [[package]] name = "sct" version = "0.6.1" @@ -787,28 +750,11 @@ version = "4.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1c107b6f4780854c8b126e228ea8869f4d7b71260f962fefb57b996b8959ba6b" -[[package]] -name = "serde" -version = "1.0.130" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f12d06de37cf59146fbdecab66aa99f9fe4f78722e3607577a5375d66bd0c913" - -[[package]] -name = "serde_json" -version = "1.0.68" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0f690853975602e1bfe1ccbf50504d67174e3bcf340f23b5ea9992e0587a52d8" -dependencies = [ - "itoa", - "ryu", - "serde", -] - [[package]] name = "sharded-slab" -version = "0.1.3" +version = "0.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "740223c51853f3145fe7c90360d2d4232f2b62e3449489c207eccde818979982" +checksum = "900fba806f70c630b0a382d0d825e17a0f19fcd059a2ade1ff237bcddf446b31" dependencies = [ "lazy_static", ] @@ -824,9 +770,9 @@ dependencies = [ [[package]] name = "slab" -version = "0.4.4" +version = "0.4.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c307a32c1c5c437f38c7fd45d753050587732ba8628319fbdf12a7e289ccc590" +checksum = "9def91fd1e018fe007022791f865d0ccc9b3a0d5001e01aabb8b40e46000afb5" [[package]] name = "smallvec" @@ -858,9 +804,9 @@ checksum = "8ea5119cdb4c55b55d432abb513a0429384878c15dde60cc77b1c99de1a95a6a" [[package]] name = "structopt" -version = "0.3.23" +version = "0.3.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bf9d950ef167e25e0bdb073cf1d68e9ad2795ac826f2f3f59647817cf23c0bfa" +checksum = "40b9788f4202aa75c240ecc9c15c65185e6a39ccdeb0fd5d008b98825464c87c" dependencies = [ "clap", "lazy_static", @@ -869,9 +815,9 @@ dependencies = [ [[package]] name = "structopt-derive" -version = "0.4.16" +version = "0.4.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "134d838a2c9943ac3125cf6df165eda53493451b719f3255b2a26b85f772d0ba" +checksum = "dcb5ae327f9cc13b68763b5749770cb9e048a99bd9dfdfa58d0cf05d5f64afe0" dependencies = [ "heck", "proc-macro-error", @@ -882,9 +828,9 @@ dependencies = [ [[package]] name = "syn" -version = "1.0.78" +version = "1.0.81" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4eac2e6c19f5c3abc0c229bea31ff0b9b091c7b14990e8924b92902a303a0c0" +checksum = "f2afee18b8beb5a596ecb4a2dce128c719b4ba399d34126b9e4396e3f9860966" dependencies = [ "proc-macro2", "quote", @@ -911,9 +857,9 @@ dependencies = [ [[package]] name = "tokio" -version = "1.12.0" +version = "1.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c2c2416fdedca8443ae44b4527de1ea633af61d8f7169ffa6e72c5b53d24efcc" +checksum = "588b2d10a336da58d877567cd8fb8a14b463e2104910f8132cd054b4b96e29ee" dependencies = [ "autocfg", "bytes", @@ -939,9 +885,9 @@ dependencies = [ [[package]] name = "tokio-macros" -version = "1.4.1" +version = "1.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "154794c8f499c2619acd19e839294703e9e32e7630ef5f46ea80d4ef0fbee5eb" +checksum = "114383b041aa6212c579467afa0075fbbdd0718de036100bc0ba7961d8cb9095" dependencies = [ "proc-macro2", "quote", @@ -961,9 +907,9 @@ dependencies = [ [[package]] name = "tokio-stream" -version = "0.1.7" +version = "0.1.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b2f3f698253f03119ac0102beaa64f67a67e08074d03a22d18784104543727f" +checksum = "50145484efff8818b5ccd256697f36863f587da82cf8b409c53adf1e840798e3" dependencies = [ "futures-core", "pin-project-lite", @@ -973,9 +919,9 @@ dependencies = [ [[package]] name = "tokio-util" -version = "0.6.8" +version = "0.6.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08d3725d3efa29485e87311c5b699de63cde14b00ed4d256b8318aa30ca452cd" +checksum = "9e99e1983e5d376cd8eb4b66604d2e99e79f5bd988c3055891dcd8c9e2604cc0" dependencies = [ "bytes", "futures-core", @@ -1019,14 +965,15 @@ dependencies = [ [[package]] name = "tower" -version = "0.4.8" +version = "0.4.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f60422bc7fefa2f3ec70359b8ff1caff59d785877eb70595904605bcc412470f" +checksum = "c00e500fff5fa1131c866b246041a6bf96da9c965f8fe4128cb1421f23e93c00" dependencies = [ "futures-core", "futures-util", "indexmap", "pin-project", + "pin-project-lite", "rand", "slab", "tokio", @@ -1051,9 +998,9 @@ checksum = "360dfd1d6d30e05fda32ace2c8c70e9c0a9da713275777f5a4dbb8a1893930c6" [[package]] name = "tracing" -version = "0.1.28" +version = "0.1.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "84f96e095c0c82419687c20ddf5cb3eadb61f4e1405923c9dc8e53a1adacbda8" +checksum = "375a639232caf30edfc78e8d89b2d4c375515393e7af7e16f01cd96917fb2105" dependencies = [ "cfg-if", "log", @@ -1064,9 +1011,9 @@ dependencies = [ [[package]] name = "tracing-attributes" -version = "0.1.16" +version = "0.1.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "98863d0dd09fa59a1b79c6750ad80dbda6b75f4e71c437a6a1a8cb91a8bcbd77" +checksum = "f4f480b8f81512e825f337ad51e94c1eb5d3bbdf2b363dcd01e2b19a9ffe3f8e" dependencies = [ "proc-macro2", "quote", @@ -1103,36 +1050,22 @@ dependencies = [ "tracing-core", ] -[[package]] -name = "tracing-serde" -version = "0.1.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fb65ea441fbb84f9f6748fd496cf7f63ec9af5bca94dd86456978d055e8eb28b" -dependencies = [ - "serde", - "tracing-core", -] - [[package]] name = "tracing-subscriber" -version = "0.2.24" +version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fdd0568dbfe3baf7048b7908d2b32bca0d81cd56bec6d2a8f894b01d74f86be3" +checksum = "80a4ddde70311d8da398062ecf6fc2c309337de6b0f77d6c27aff8d53f6fca52" dependencies = [ "ansi_term 0.12.1", - "chrono", "lazy_static", "matchers", "regex", - "serde", - "serde_json", "sharded-slab", "smallvec", "thread_local", "tracing", "tracing-core", "tracing-log", - "tracing-serde", ] [[package]] diff --git a/hadron-cli/Cargo.toml b/hadron-cli/Cargo.toml index 8f92ebf..3fc2954 100644 --- a/hadron-cli/Cargo.toml +++ b/hadron-cli/Cargo.toml @@ -1,13 +1,13 @@ [package] authors = ["Anthony Dodd "] -edition = "2018" +edition = "2021" name = "hadron-cli" version = "0.1.0-beta.0" description = "The Hadron CLI." [dependencies] anyhow = "1" -base64 = "0.13.0" +base64 = "0.13" futures = "0.3" hadron = { package = "hadron-client", version = "0.1.0-beta.0" } lazy_static = "1" @@ -20,5 +20,5 @@ tokio = { version = "1", default-features = false, features = [ ]} tokio-stream = { version = "0.1", default-features = false, features = ["time", "net", "signal", "sync"] } tracing = { version = "0.1", features = ["attributes"] } -tracing-subscriber = "0.2" +tracing-subscriber = { version = "0.3", features = ["env-filter", "registry"] } uuid = { version = "0.8", default-features = false, features = ["v4"] } diff --git a/hadron-cli/Dockerfile b/hadron-cli/Dockerfile index a88e185..8d4da17 100644 --- a/hadron-cli/Dockerfile +++ b/hadron-cli/Dockerfile @@ -3,7 +3,7 @@ ## NOTE WELL: this Dockerfile assumes a context from the root of this repo. # syntax=docker/dockerfile:1.3 -ARG RUST_TAG=1.55.0 +ARG RUST_TAG=1.56.1 ARG ALPINE_TAG=3.14 FROM rust:${RUST_TAG}-alpine${ALPINE_TAG} as builder diff --git a/hadron-client/Cargo.lock b/hadron-client/Cargo.lock index 85a4047..e1cadd5 100644 --- a/hadron-client/Cargo.lock +++ b/hadron-client/Cargo.lock @@ -2,11 +2,20 @@ # It is not intended for manual editing. version = 3 +[[package]] +name = "aho-corasick" +version = "0.7.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e37cfd5e7657ada45f742d6e99ca5788580b5c529dc78faf11ece6dc702656f" +dependencies = [ + "memchr", +] + [[package]] name = "anyhow" -version = "1.0.44" +version = "1.0.45" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "61604a8f862e1d5c3229fdd78f8b02c68dcf73a4c4b05fd636d12240aaa242c1" +checksum = "ee10e43ae4a853c0a3591d4e2ada1719e553be18199d9da9d4a83f5927c2f5c7" [[package]] name = "arc-swap" @@ -66,9 +75,9 @@ checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" [[package]] name = "bumpalo" -version = "3.7.1" +version = "3.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d9df67f7bf9ef8498769f994239c45613ef0c5899415fb58e9add412d2c1a538" +checksum = "8f1e260c3a9040a7c19a12468758f4c16f31a81a1fe087482be9570ec864bb6c" [[package]] name = "bytes" @@ -78,9 +87,9 @@ checksum = "c4872d67bab6358e59559027aa3b9157c53d9358c51423c17554809a8858e0f8" [[package]] name = "cc" -version = "1.0.70" +version = "1.0.71" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d26a6ce4b6a484fa3edb70f7efa6fc430fd2b87285fe8b84304fd0936faa0dc0" +checksum = "79c2681d6594606957bbb8631c4b90a7fcaaa72cdb714743a437b156d6a7eedd" [[package]] name = "cfg-if" @@ -96,9 +105,9 @@ checksum = "e78d4f1cc4ae33bbfc157ed5d5a5ef3bc29227303d595861deb238fcec4e9457" [[package]] name = "fixedbitset" -version = "0.2.0" +version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "37ab347416e802de484e4d03c7316c48f1ecb56574dfd4a46a80f173ce1de04d" +checksum = "398ea4fabe40b9b0d885340a2a991a44c8a645624075ad966d21f88688e2b69e" [[package]] name = "fnv" @@ -213,9 +222,9 @@ dependencies = [ [[package]] name = "h2" -version = "0.3.4" +version = "0.3.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d7f3675cfef6a30c8031cf9e6493ebdc3bb3272a3fea3923c4210d1830e6a472" +checksum = "7fd819562fcebdac5afc5c113c3ec36f902840b70fd4fc458799c8ce4607ae55" dependencies = [ "bytes", "fnv", @@ -279,9 +288,9 @@ dependencies = [ [[package]] name = "http-body" -version = "0.4.3" +version = "0.4.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "399c583b2979440c60be0821a6199eca73bc3c8dcd9d070d75ac726e2c6186e5" +checksum = "1ff4f84919677303da5f147645dbea6b1881f368d03ac84e1dc09031ebd7b2c6" dependencies = [ "bytes", "http", @@ -302,9 +311,9 @@ checksum = "6456b8a6c8f33fee7d958fcd1b60d55b11940a79e63ae87013e6d22e26034440" [[package]] name = "hyper" -version = "0.14.13" +version = "0.14.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "15d1cfb9e4f68655fa04c01f59edb405b6074a0f7118ea881e5026e4a1cd8593" +checksum = "2b91bb1f221b6ea1f1e4371216b70f40748774c2fb5971b450c07773fb92d26b" dependencies = [ "bytes", "futures-channel", @@ -378,9 +387,9 @@ checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" [[package]] name = "libc" -version = "0.2.103" +version = "0.2.106" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dd8f7255a17a627354f321ef0055d63b898c6fb27eff628af4d1b66b7331edf6" +checksum = "a60553f9a9e039a333b4e9b20573b9e9b9c0bb3a11e201ccc48ef4283456d673" [[package]] name = "log" @@ -399,9 +408,9 @@ checksum = "308cc39be01b73d0d18f82a0e7b2a3df85245f84af96fdddc5d202d27e47b86a" [[package]] name = "mio" -version = "0.7.13" +version = "0.7.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8c2bdb6314ec10835cd3293dd268473a835c02b7b352e788be788b3c6ca6bb16" +checksum = "8067b404fe97c70829f082dec8bcf4f71225d7eaea1d8645349cb76fa06205cc" dependencies = [ "libc", "log", @@ -448,9 +457,9 @@ checksum = "d4fd5641d01c8f18a23da7b6fe29298ff4b55afcccdf78973b24cf3175fee32e" [[package]] name = "petgraph" -version = "0.5.1" +version = "0.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "467d164a6de56270bd7c4d070df81d07beace25012d5103ced4e9ff08d6afdb7" +checksum = "4a13a2fa9d0b63e5f22328828741e523766fff0ee9e779316902290dff3f824f" dependencies = [ "fixedbitset", "indexmap", @@ -490,9 +499,9 @@ checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" [[package]] name = "ppv-lite86" -version = "0.2.10" +version = "0.2.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ac74c624d6b2d21f425f752262f42188365d7b8ff1aff74c82e45136510a4857" +checksum = "ed0cfbc8191465bed66e1718596ee0b0b35d5ee1f41c5df2189d0fe8bde535ba" [[package]] name = "proc-macro-hack" @@ -508,18 +517,18 @@ checksum = "bc881b2c22681370c6a780e47af9840ef841837bc98118431d4e1868bd0c1086" [[package]] name = "proc-macro2" -version = "1.0.29" +version = "1.0.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b9f5105d4fdaab20335ca9565e106a5d9b82b6219b5ba735731124ac6711d23d" +checksum = "ba508cc11742c0dc5c1659771673afbab7a0efab23aa17e854cbab0837ed0b43" dependencies = [ "unicode-xid", ] [[package]] name = "prost" -version = "0.8.0" +version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "de5e2533f59d08fcf364fd374ebda0692a70bd6d7e66ef97f306f45c6c5d8020" +checksum = "444879275cb4fd84958b1a1d5420d15e6fcf7c235fe47f053c9c2a80aceb6001" dependencies = [ "bytes", "prost-derive", @@ -527,27 +536,29 @@ dependencies = [ [[package]] name = "prost-build" -version = "0.8.0" +version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "355f634b43cdd80724ee7848f95770e7e70eefa6dcf14fea676216573b8fd603" +checksum = "62941722fb675d463659e49c4f3fe1fe792ff24fe5bbaa9c08cd3b98a1c354f5" dependencies = [ "bytes", "heck", "itertools", + "lazy_static", "log", "multimap", "petgraph", "prost", "prost-types", + "regex", "tempfile", "which", ] [[package]] name = "prost-derive" -version = "0.8.0" +version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "600d2f334aa05acb02a755e217ef1ab6dea4d51b58b7846588b747edec04efba" +checksum = "f9cc1a3263e07e0bf68e96268f37665207b49560d98739662cdfaae215c720fe" dependencies = [ "anyhow", "itertools", @@ -558,9 +569,9 @@ dependencies = [ [[package]] name = "prost-types" -version = "0.8.0" +version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "603bbd6394701d13f3f25aada59c7de9d35a6a5887cfc156181234a44002771b" +checksum = "534b7a0e836e3c482d2693070f982e39e7611da9695d4d1f5a4b186b51faef0a" dependencies = [ "bytes", "prost", @@ -568,9 +579,9 @@ dependencies = [ [[package]] name = "quote" -version = "1.0.9" +version = "1.0.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c3d0b9745dc2debf507c8422de05d7226cc1f0644216dfdfead988f9b1ab32a7" +checksum = "38bc8cc6a5f2e3655e0899c1b848643b2562f853f114bfec7be120678e3ace05" dependencies = [ "proc-macro2", ] @@ -624,6 +635,23 @@ dependencies = [ "bitflags", ] +[[package]] +name = "regex" +version = "1.5.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d07a8629359eb56f1e2fb1652bb04212c072a87ba68546a04065d525673ac461" +dependencies = [ + "aho-corasick", + "memchr", + "regex-syntax", +] + +[[package]] +name = "regex-syntax" +version = "0.6.25" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f497285884f3fcff424ffc933e56d7cbca511def0c9831a7f9b5f6153e3cc89b" + [[package]] name = "remove_dir_all" version = "0.5.3" @@ -688,9 +716,9 @@ dependencies = [ [[package]] name = "slab" -version = "0.4.4" +version = "0.4.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c307a32c1c5c437f38c7fd45d753050587732ba8628319fbdf12a7e289ccc590" +checksum = "9def91fd1e018fe007022791f865d0ccc9b3a0d5001e01aabb8b40e46000afb5" [[package]] name = "socket2" @@ -710,9 +738,9 @@ checksum = "6e63cff320ae2c57904679ba7cb63280a3dc4613885beafb148ee7bf9aa9042d" [[package]] name = "syn" -version = "1.0.77" +version = "1.0.81" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5239bc68e0fef57495900cfea4e8dc75596d9a319d7e16b1e0a440d24e6fe0a0" +checksum = "f2afee18b8beb5a596ecb4a2dce128c719b4ba399d34126b9e4396e3f9860966" dependencies = [ "proc-macro2", "quote", @@ -735,9 +763,9 @@ dependencies = [ [[package]] name = "tokio" -version = "1.12.0" +version = "1.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c2c2416fdedca8443ae44b4527de1ea633af61d8f7169ffa6e72c5b53d24efcc" +checksum = "588b2d10a336da58d877567cd8fb8a14b463e2104910f8132cd054b4b96e29ee" dependencies = [ "autocfg", "bytes", @@ -763,9 +791,9 @@ dependencies = [ [[package]] name = "tokio-macros" -version = "1.3.0" +version = "1.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "54473be61f4ebe4efd09cec9bd5d16fa51d70ea0192213d754d2d500457db110" +checksum = "114383b041aa6212c579467afa0075fbbdd0718de036100bc0ba7961d8cb9095" dependencies = [ "proc-macro2", "quote", @@ -785,9 +813,9 @@ dependencies = [ [[package]] name = "tokio-stream" -version = "0.1.7" +version = "0.1.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b2f3f698253f03119ac0102beaa64f67a67e08074d03a22d18784104543727f" +checksum = "50145484efff8818b5ccd256697f36863f587da82cf8b409c53adf1e840798e3" dependencies = [ "futures-core", "pin-project-lite", @@ -797,9 +825,9 @@ dependencies = [ [[package]] name = "tokio-util" -version = "0.6.8" +version = "0.6.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08d3725d3efa29485e87311c5b699de63cde14b00ed4d256b8318aa30ca452cd" +checksum = "9e99e1983e5d376cd8eb4b66604d2e99e79f5bd988c3055891dcd8c9e2604cc0" dependencies = [ "bytes", "futures-core", @@ -811,9 +839,9 @@ dependencies = [ [[package]] name = "tonic" -version = "0.5.2" +version = "0.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "796c5e1cd49905e65dd8e700d4cb1dffcbfdb4fc9d017de08c1a537afd83627c" +checksum = "24203b79cf2d68909da91178db3026e77054effba0c5d93deb870d3ca7b35afa" dependencies = [ "async-stream", "async-trait", @@ -843,9 +871,9 @@ dependencies = [ [[package]] name = "tonic-build" -version = "0.5.2" +version = "0.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "12b52d07035516c2b74337d2ac7746075e7dcae7643816c1b12c5ff8a7484c08" +checksum = "88358bb1dcfeb62dcce85c63006cafb964b7be481d522b7e09589d4d1e718d2a" dependencies = [ "proc-macro2", "prost-build", @@ -855,14 +883,15 @@ dependencies = [ [[package]] name = "tower" -version = "0.4.8" +version = "0.4.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f60422bc7fefa2f3ec70359b8ff1caff59d785877eb70595904605bcc412470f" +checksum = "c00e500fff5fa1131c866b246041a6bf96da9c965f8fe4128cb1421f23e93c00" dependencies = [ "futures-core", "futures-util", "indexmap", "pin-project", + "pin-project-lite", "rand", "slab", "tokio", @@ -887,9 +916,9 @@ checksum = "360dfd1d6d30e05fda32ace2c8c70e9c0a9da713275777f5a4dbb8a1893930c6" [[package]] name = "tracing" -version = "0.1.28" +version = "0.1.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "84f96e095c0c82419687c20ddf5cb3eadb61f4e1405923c9dc8e53a1adacbda8" +checksum = "375a639232caf30edfc78e8d89b2d4c375515393e7af7e16f01cd96917fb2105" dependencies = [ "cfg-if", "log", @@ -900,9 +929,9 @@ dependencies = [ [[package]] name = "tracing-attributes" -version = "0.1.16" +version = "0.1.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "98863d0dd09fa59a1b79c6750ad80dbda6b75f4e71c437a6a1a8cb91a8bcbd77" +checksum = "f4f480b8f81512e825f337ad51e94c1eb5d3bbdf2b363dcd01e2b19a9ffe3f8e" dependencies = [ "proc-macro2", "quote", @@ -911,9 +940,9 @@ dependencies = [ [[package]] name = "tracing-core" -version = "0.1.20" +version = "0.1.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "46125608c26121c81b0c6d693eab5a420e416da7e43c426d2e8f7df8da8a3acf" +checksum = "1f4ed65637b8390770814083d20756f87bfa2c21bf2f110babdc5438351746e4" dependencies = [ "lazy_static", ] diff --git a/hadron-client/Cargo.toml b/hadron-client/Cargo.toml index 549dd1e..5c57825 100644 --- a/hadron-client/Cargo.toml +++ b/hadron-client/Cargo.toml @@ -4,7 +4,7 @@ version = "0.1.0-beta.1" description = "The Hadron Rust client library." authors = ["Anthony Dodd "] categories = ["api-bindings", "concurrency", "database"] -edition = "2018" +edition = "2021" keywords = ["hadron", "events", "CloudEvents", "stream-processing", "workflows"] license = "MIT/Apache-2.0" readme = "README.md" @@ -12,23 +12,23 @@ repository = "https://github.com/hadron-project/hadron" [dependencies] anyhow = "1" -arc-swap = "1.3.1" +arc-swap = "1.3" async-trait = "0.1" base64 = "0.13" futures = "0.3" http = "0.2" -prost = "0.8" +prost = "0.9" rand = "0.8" -seahash = "4.1.0" +seahash = "4.1" tokio = { version = "1", default-features = false, features = [ # See https://docs.rs/tokio/latest/tokio/#feature-flags "rt", "io-util", "net", "time", "macros", "sync", "signal", ]} tokio-stream = { version = "0.1", default-features = false, features = ["time", "net", "signal", "sync"] } -tonic = { version = "0.5", default-features = false, features = ["transport", "codegen", "prost", "tls"] } +tonic = { version = "0.6", default-features = false, features = ["transport", "codegen", "prost", "tls"] } tower = { version = "0.4", default-features = false, features = ["discover"] } tracing = { version = "0.1", features = ["attributes"] } [dev-dependencies] anyhow = "1" -tonic-build = "0.5" +tonic-build = "0.6" diff --git a/hadron-client/src/client/mod.rs b/hadron-client/src/client/mod.rs index 4794364..4ff2970 100644 --- a/hadron-client/src/client/mod.rs +++ b/hadron-client/src/client/mod.rs @@ -167,15 +167,7 @@ async fn diff_and_update_endpoints( continue; } }; - let chan = match endpoint.connect_lazy() { - Ok(chan) => chan, - Err(err) => { - // NOTE: as of tonic@0.5, this actually can not fail, but returns - // a result for API future-proofing. - tracing::error!(error = ?err, "error building connection to endpoint"); - continue; - } - }; + let chan = endpoint.connect_lazy(); let conn = StreamControllerClient::new(chan); changes.push(Change::Insert(partition.partition, conn.clone())); tracing::debug!(endpoint = new_endpoint_str, "new endpoint connection established"); diff --git a/hadron-core/Cargo.lock b/hadron-core/Cargo.lock index 3f92ec9..e4f1a41 100644 --- a/hadron-core/Cargo.lock +++ b/hadron-core/Cargo.lock @@ -13,9 +13,9 @@ dependencies = [ [[package]] name = "anyhow" -version = "1.0.44" +version = "1.0.45" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "61604a8f862e1d5c3229fdd78f8b02c68dcf73a4c4b05fd636d12240aaa242c1" +checksum = "ee10e43ae4a853c0a3591d4e2ada1719e553be18199d9da9d4a83f5927c2f5c7" [[package]] name = "async-stream" @@ -69,9 +69,9 @@ checksum = "904dfeac50f3cdaba28fc6f57fdcddb75f49ed61346676a78c4ffe55877802fd" [[package]] name = "bumpalo" -version = "3.7.1" +version = "3.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d9df67f7bf9ef8498769f994239c45613ef0c5899415fb58e9add412d2c1a538" +checksum = "8f1e260c3a9040a7c19a12468758f4c16f31a81a1fe087482be9570ec864bb6c" [[package]] name = "bytes" @@ -81,9 +81,9 @@ checksum = "c4872d67bab6358e59559027aa3b9157c53d9358c51423c17554809a8858e0f8" [[package]] name = "cc" -version = "1.0.70" +version = "1.0.71" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d26a6ce4b6a484fa3edb70f7efa6fc430fd2b87285fe8b84304fd0936faa0dc0" +checksum = "79c2681d6594606957bbb8631c4b90a7fcaaa72cdb714743a437b156d6a7eedd" [[package]] name = "cfg-if" @@ -107,9 +107,9 @@ dependencies = [ [[package]] name = "darling" -version = "0.12.4" +version = "0.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f2c43f534ea4b0b049015d00269734195e6d3f0f6635cb692251aca6f9f8b3c" +checksum = "757c0ded2af11d8e739c4daea1ac623dd1624b06c844cf3f5a39f1bdbd99bb12" dependencies = [ "darling_core", "darling_macro", @@ -117,9 +117,9 @@ dependencies = [ [[package]] name = "darling_core" -version = "0.12.4" +version = "0.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8e91455b86830a1c21799d94524df0845183fa55bafd9aa137b01c7d1065fa36" +checksum = "2c34d8efb62d0c2d7f60ece80f75e5c63c1588ba68032740494b0b9a996466e3" dependencies = [ "fnv", "ident_case", @@ -131,9 +131,9 @@ dependencies = [ [[package]] name = "darling_macro" -version = "0.12.4" +version = "0.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "29b5acf0dea37a7f66f7b25d2c5e93fd46f8f6968b1a5d7a3e02e97768afc95a" +checksum = "ade7bff147130fe5e6d39f089c6bd49ec0250f35d70b2eebf72afdfc919f15cc" dependencies = [ "darling_core", "quote", @@ -233,9 +233,9 @@ dependencies = [ [[package]] name = "h2" -version = "0.3.6" +version = "0.3.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c06815895acec637cd6ed6e9662c935b866d20a106f8361892893a7d9234964" +checksum = "7fd819562fcebdac5afc5c113c3ec36f902840b70fd4fc458799c8ce4607ae55" dependencies = [ "bytes", "fnv", @@ -290,9 +290,9 @@ dependencies = [ [[package]] name = "http-body" -version = "0.4.3" +version = "0.4.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "399c583b2979440c60be0821a6199eca73bc3c8dcd9d070d75ac726e2c6186e5" +checksum = "1ff4f84919677303da5f147645dbea6b1881f368d03ac84e1dc09031ebd7b2c6" dependencies = [ "bytes", "http", @@ -313,9 +313,9 @@ checksum = "6456b8a6c8f33fee7d958fcd1b60d55b11940a79e63ae87013e6d22e26034440" [[package]] name = "hyper" -version = "0.14.13" +version = "0.14.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "15d1cfb9e4f68655fa04c01f59edb405b6074a0f7118ea881e5026e4a1cd8593" +checksum = "2b91bb1f221b6ea1f1e4371216b70f40748774c2fb5971b450c07773fb92d26b" dependencies = [ "bytes", "futures-channel", @@ -425,9 +425,9 @@ dependencies = [ [[package]] name = "k8s-openapi" -version = "0.13.0" +version = "0.13.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "748acc444200aa3528dc131a8048e131a9e75a611a52d152e276e99199313d1a" +checksum = "4f8de9873b904e74b3533f77493731ee26742418077503683db44e1b3c54aa5c" dependencies = [ "base64 0.13.0", "bytes", @@ -442,25 +442,22 @@ dependencies = [ [[package]] name = "kube" -version = "0.59.0" +version = "0.63.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2bfa22c305a6d817b57a7afcd2e6ee23c80c6c93933edb02f210fdf73f837cc" +checksum = "75e877325e5540a3041b519bd7ee27a858691f9f816cf533d652cbb33cbfea45" dependencies = [ - "http", "k8s-openapi", "kube-core", "kube-derive", - "serde", - "serde_json", - "thiserror", ] [[package]] name = "kube-core" -version = "0.59.0" +version = "0.63.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c33d2272d8e530938bafc6cf4ac76f2a6f6c9ca684defcfab6c357913a43bcc" +checksum = "a91e572d244436fbc0d0b5a4829d96b9d623e08eb6b5d1e80418c1fab10b162a" dependencies = [ + "chrono", "form_urlencoded", "http", "json-patch", @@ -473,9 +470,9 @@ dependencies = [ [[package]] name = "kube-derive" -version = "0.59.0" +version = "0.63.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "53dc9fa719dd21d1a4c155cf8936f618a687f3590885e09d9261727cd5dc56a5" +checksum = "2034f57f3db36978ef366f45f1e263e623d9a6a8fcc6a6b1ef8879a213e1d2c4" dependencies = [ "darling", "proc-macro2", @@ -492,9 +489,9 @@ checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" [[package]] name = "libc" -version = "0.2.103" +version = "0.2.106" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dd8f7255a17a627354f321ef0055d63b898c6fb27eff628af4d1b66b7331edf6" +checksum = "a60553f9a9e039a333b4e9b20573b9e9b9c0bb3a11e201ccc48ef4283456d673" [[package]] name = "linked-hash-map" @@ -525,9 +522,9 @@ checksum = "308cc39be01b73d0d18f82a0e7b2a3df85245f84af96fdddc5d202d27e47b86a" [[package]] name = "mio" -version = "0.7.13" +version = "0.7.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8c2bdb6314ec10835cd3293dd268473a835c02b7b352e788be788b3c6ca6bb16" +checksum = "8067b404fe97c70829f082dec8bcf4f71225d7eaea1d8645349cb76fa06205cc" dependencies = [ "libc", "log", @@ -660,24 +657,24 @@ checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" [[package]] name = "ppv-lite86" -version = "0.2.10" +version = "0.2.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ac74c624d6b2d21f425f752262f42188365d7b8ff1aff74c82e45136510a4857" +checksum = "ed0cfbc8191465bed66e1718596ee0b0b35d5ee1f41c5df2189d0fe8bde535ba" [[package]] name = "proc-macro2" -version = "1.0.29" +version = "1.0.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b9f5105d4fdaab20335ca9565e106a5d9b82b6219b5ba735731124ac6711d23d" +checksum = "ba508cc11742c0dc5c1659771673afbab7a0efab23aa17e854cbab0837ed0b43" dependencies = [ "unicode-xid", ] [[package]] name = "prost" -version = "0.8.0" +version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "de5e2533f59d08fcf364fd374ebda0692a70bd6d7e66ef97f306f45c6c5d8020" +checksum = "444879275cb4fd84958b1a1d5420d15e6fcf7c235fe47f053c9c2a80aceb6001" dependencies = [ "bytes", "prost-derive", @@ -685,9 +682,9 @@ dependencies = [ [[package]] name = "prost-derive" -version = "0.8.0" +version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "600d2f334aa05acb02a755e217ef1ab6dea4d51b58b7846588b747edec04efba" +checksum = "f9cc1a3263e07e0bf68e96268f37665207b49560d98739662cdfaae215c720fe" dependencies = [ "anyhow", "itertools", @@ -698,9 +695,9 @@ dependencies = [ [[package]] name = "quote" -version = "1.0.9" +version = "1.0.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c3d0b9745dc2debf507c8422de05d7226cc1f0644216dfdfead988f9b1ab32a7" +checksum = "38bc8cc6a5f2e3655e0899c1b848643b2562f853f114bfec7be120678e3ace05" dependencies = [ "proc-macro2", ] @@ -884,9 +881,9 @@ dependencies = [ [[package]] name = "slab" -version = "0.4.4" +version = "0.4.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c307a32c1c5c437f38c7fd45d753050587732ba8628319fbdf12a7e289ccc590" +checksum = "9def91fd1e018fe007022791f865d0ccc9b3a0d5001e01aabb8b40e46000afb5" [[package]] name = "socket2" @@ -912,9 +909,9 @@ checksum = "73473c0e59e6d5812c5dfe2a064a6444949f089e20eec9a2e5506596494e4623" [[package]] name = "syn" -version = "1.0.77" +version = "1.0.81" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5239bc68e0fef57495900cfea4e8dc75596d9a319d7e16b1e0a440d24e6fe0a0" +checksum = "f2afee18b8beb5a596ecb4a2dce128c719b4ba399d34126b9e4396e3f9860966" dependencies = [ "proc-macro2", "quote", @@ -923,18 +920,18 @@ dependencies = [ [[package]] name = "thiserror" -version = "1.0.29" +version = "1.0.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "602eca064b2d83369e2b2f34b09c70b605402801927c65c11071ac911d299b88" +checksum = "854babe52e4df1653706b98fcfc05843010039b406875930a70e4d9644e5c417" dependencies = [ "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "1.0.29" +version = "1.0.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bad553cc2c78e8de258400763a647e80e6d1b31ee237275d756f6836d204494c" +checksum = "aa32fd3f627f367fe16f893e2597ae3c05020f8bba2666a4e6ea73d377e5714b" dependencies = [ "proc-macro2", "quote", @@ -943,11 +940,12 @@ dependencies = [ [[package]] name = "time" -version = "0.1.43" +version = "0.1.44" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ca8a50ef2360fbd1eeb0ecd46795a87a19024eb4b53c5dc916ca1fd95fe62438" +checksum = "6db9e6914ab8b1ae1c260a4ae7a49b6c5611b40328a735b21862567685e73255" dependencies = [ "libc", + "wasi", "winapi", ] @@ -968,9 +966,9 @@ checksum = "cda74da7e1a664f795bb1f8a87ec406fb89a02522cf6e50620d016add6dbbf5c" [[package]] name = "tokio" -version = "1.12.0" +version = "1.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c2c2416fdedca8443ae44b4527de1ea633af61d8f7169ffa6e72c5b53d24efcc" +checksum = "588b2d10a336da58d877567cd8fb8a14b463e2104910f8132cd054b4b96e29ee" dependencies = [ "autocfg", "bytes", @@ -994,9 +992,9 @@ dependencies = [ [[package]] name = "tokio-macros" -version = "1.4.1" +version = "1.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "154794c8f499c2619acd19e839294703e9e32e7630ef5f46ea80d4ef0fbee5eb" +checksum = "114383b041aa6212c579467afa0075fbbdd0718de036100bc0ba7961d8cb9095" dependencies = [ "proc-macro2", "quote", @@ -1005,9 +1003,9 @@ dependencies = [ [[package]] name = "tokio-stream" -version = "0.1.7" +version = "0.1.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b2f3f698253f03119ac0102beaa64f67a67e08074d03a22d18784104543727f" +checksum = "50145484efff8818b5ccd256697f36863f587da82cf8b409c53adf1e840798e3" dependencies = [ "futures-core", "pin-project-lite", @@ -1016,9 +1014,9 @@ dependencies = [ [[package]] name = "tokio-util" -version = "0.6.8" +version = "0.6.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08d3725d3efa29485e87311c5b699de63cde14b00ed4d256b8318aa30ca452cd" +checksum = "9e99e1983e5d376cd8eb4b66604d2e99e79f5bd988c3055891dcd8c9e2604cc0" dependencies = [ "bytes", "futures-core", @@ -1030,9 +1028,9 @@ dependencies = [ [[package]] name = "tonic" -version = "0.5.2" +version = "0.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "796c5e1cd49905e65dd8e700d4cb1dffcbfdb4fc9d017de08c1a537afd83627c" +checksum = "24203b79cf2d68909da91178db3026e77054effba0c5d93deb870d3ca7b35afa" dependencies = [ "async-stream", "async-trait", @@ -1061,14 +1059,15 @@ dependencies = [ [[package]] name = "tower" -version = "0.4.8" +version = "0.4.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f60422bc7fefa2f3ec70359b8ff1caff59d785877eb70595904605bcc412470f" +checksum = "c00e500fff5fa1131c866b246041a6bf96da9c965f8fe4128cb1421f23e93c00" dependencies = [ "futures-core", "futures-util", "indexmap", "pin-project", + "pin-project-lite", "rand", "slab", "tokio", @@ -1093,9 +1092,9 @@ checksum = "360dfd1d6d30e05fda32ace2c8c70e9c0a9da713275777f5a4dbb8a1893930c6" [[package]] name = "tracing" -version = "0.1.28" +version = "0.1.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "84f96e095c0c82419687c20ddf5cb3eadb61f4e1405923c9dc8e53a1adacbda8" +checksum = "375a639232caf30edfc78e8d89b2d4c375515393e7af7e16f01cd96917fb2105" dependencies = [ "cfg-if", "log", @@ -1106,9 +1105,9 @@ dependencies = [ [[package]] name = "tracing-attributes" -version = "0.1.16" +version = "0.1.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "98863d0dd09fa59a1b79c6750ad80dbda6b75f4e71c437a6a1a8cb91a8bcbd77" +checksum = "f4f480b8f81512e825f337ad51e94c1eb5d3bbdf2b363dcd01e2b19a9ffe3f8e" dependencies = [ "proc-macro2", "quote", @@ -1117,9 +1116,9 @@ dependencies = [ [[package]] name = "tracing-core" -version = "0.1.20" +version = "0.1.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "46125608c26121c81b0c6d693eab5a420e416da7e43c426d2e8f7df8da8a3acf" +checksum = "1f4ed65637b8390770814083d20756f87bfa2c21bf2f110babdc5438351746e4" dependencies = [ "lazy_static", ] @@ -1151,9 +1150,9 @@ checksum = "59547bce71d9c38b83d9c0e92b6066c4253371f15005def0c30d9657f50c7642" [[package]] name = "unicode-bidi" -version = "0.3.6" +version = "0.3.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "246f4c42e67e7a4e3c6106ff716a5d067d4132a642840b242e357e468a2a0085" +checksum = "1a01404663e3db436ed2746d9fefef640d868edae3cceb81c3b8d5732fda678f" [[package]] name = "unicode-normalization" @@ -1210,9 +1209,9 @@ dependencies = [ [[package]] name = "wasi" -version = "0.10.2+wasi-snapshot-preview1" +version = "0.10.0+wasi-snapshot-preview1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fd6fbd9a79829dd1ad0cc20627bf1ed606756a7f77edff7b66b7064f9cb327c6" +checksum = "1a143597ca7c7793eff794def352d41792a93c481eb1042423ff7ff72ba2c31f" [[package]] name = "wasm-bindgen" diff --git a/hadron-core/Cargo.toml b/hadron-core/Cargo.toml index 3157bc4..f46d846 100644 --- a/hadron-core/Cargo.toml +++ b/hadron-core/Cargo.toml @@ -7,15 +7,15 @@ edition = "2018" anyhow = "1" base64 = "0.13" jsonwebtoken = "7" -k8s-openapi = { version = "0.13", default-features = false, features = ["api", "v1_18"] } -kube = { version = "0.59", default-features = false, features = ["derive", "jsonpatch"] } -lazy_static = "1.4.0" -petgraph = "0.6.0" -regex = "1.5.4" +k8s-openapi = { version = "0.13", default-features = false, features = ["api", "v1_22"] } +kube = { version = "0.63", default-features = false, features = ["derive", "jsonpatch"] } +lazy_static = "1.4" +petgraph = "0.6" +regex = "1.5" schemars = "0.8" serde = { version="1", features = ["derive"] } serde_json = "1" serde_yaml = "0.8" thiserror = "1" -tonic = "0.5" +tonic = "0.6" uuid = { version="0.8", default-features=false, features=["serde", "v4"] } diff --git a/hadron-core/src/crd/mod.rs b/hadron-core/src/crd/mod.rs index 0838be9..4b0fdca 100644 --- a/hadron-core/src/crd/mod.rs +++ b/hadron-core/src/crd/mod.rs @@ -11,8 +11,8 @@ mod token; use kube::Resource; -pub use pipeline::{Pipeline, PipelineSpec, PipelineStage, PipelineStatus, PipelineStartPointLocation, PipelineStartPoint}; -pub use stream::{Stream, StreamSpec, StreamStatus}; +pub use pipeline::{Pipeline, PipelineSpec, PipelineStage, PipelineStartPoint, PipelineStartPointLocation, PipelineStatus}; +pub use stream::{Stream, StreamRetentionPolicy, StreamRetentionSpec, StreamSpec, StreamStatus}; pub use token::{PubSubAccess, Token, TokenSpec, TokenStatus}; /// A convenience trait built around the fact that all implementors diff --git a/hadron-core/src/crd/stream.rs b/hadron-core/src/crd/stream.rs index 5afefff..6788b0f 100644 --- a/hadron-core/src/crd/stream.rs +++ b/hadron-core/src/crd/stream.rs @@ -76,11 +76,18 @@ pub struct StreamRetentionSpec { pub retention_seconds: Option, } +impl StreamRetentionSpec { + /// The default retention seconds value, which is 7 days. + pub fn retention_seconds_default() -> u64 { + 604_800 // 7 days. + } +} + impl Default for StreamRetentionSpec { fn default() -> Self { Self { strategy: StreamRetentionPolicy::Time, - retention_seconds: Some(604_800), // 7 days. + retention_seconds: Some(StreamRetentionSpec::retention_seconds_default()), } } } @@ -94,3 +101,16 @@ pub enum StreamRetentionPolicy { /// Retain data on the Stream based on secondary timestamp index. Time, } + +impl std::fmt::Display for StreamRetentionPolicy { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!( + f, + "{}", + match self { + Self::Retain => "retain", + Self::Time => "time", + } + ) + } +} diff --git a/hadron-operator/Cargo.lock b/hadron-operator/Cargo.lock index 26e4e5a..fbbd00a 100644 --- a/hadron-operator/Cargo.lock +++ b/hadron-operator/Cargo.lock @@ -28,9 +28,9 @@ dependencies = [ [[package]] name = "anyhow" -version = "1.0.44" +version = "1.0.45" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "61604a8f862e1d5c3229fdd78f8b02c68dcf73a4c4b05fd636d12240aaa242c1" +checksum = "ee10e43ae4a853c0a3591d4e2ada1719e553be18199d9da9d4a83f5927c2f5c7" [[package]] name = "async-compression" @@ -85,9 +85,9 @@ checksum = "cdb031dd78e28731d87d56cc8ffef4a8f36ca26c38fe2de700543e627f8a464a" [[package]] name = "axum" -version = "0.2.5" +version = "0.2.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b6519a24c07bab4effe38e226c447faef56869f99aa66aa92502aba7ad47b168" +checksum = "8f08f95a202e827209db1491047aa57c18c8adb4c5efcfcfd4a2da4838ee3a72" dependencies = [ "async-trait", "bitflags", @@ -139,9 +139,9 @@ dependencies = [ [[package]] name = "bumpalo" -version = "3.7.1" +version = "3.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d9df67f7bf9ef8498769f994239c45613ef0c5899415fb58e9add412d2c1a538" +checksum = "8f1e260c3a9040a7c19a12468758f4c16f31a81a1fe087482be9570ec864bb6c" [[package]] name = "byteorder" @@ -157,9 +157,9 @@ checksum = "c4872d67bab6358e59559027aa3b9157c53d9358c51423c17554809a8858e0f8" [[package]] name = "cc" -version = "1.0.70" +version = "1.0.71" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d26a6ce4b6a484fa3edb70f7efa6fc430fd2b87285fe8b84304fd0936faa0dc0" +checksum = "79c2681d6594606957bbb8631c4b90a7fcaaa72cdb714743a437b156d6a7eedd" [[package]] name = "cfg-if" @@ -183,9 +183,9 @@ dependencies = [ [[package]] name = "core-foundation" -version = "0.9.1" +version = "0.9.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0a89e2ae426ea83155dccf10c0fa6b1463ef6d5fcb44cee0b224a408fa640a62" +checksum = "6888e10551bb93e424d8df1d07f1a8b4fceb0001a3a4b048bfc47554946f47b3" dependencies = [ "core-foundation-sys", "libc", @@ -193,9 +193,9 @@ dependencies = [ [[package]] name = "core-foundation-sys" -version = "0.8.2" +version = "0.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ea221b5284a47e40033bf9b66f35f984ec0ea2931eb03505246cd27a963f981b" +checksum = "5827cebf4670468b8772dd191856768aedcb1b0278a04f989f7766351917b9dc" [[package]] name = "cpufeatures" @@ -226,9 +226,9 @@ dependencies = [ [[package]] name = "darling" -version = "0.12.4" +version = "0.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f2c43f534ea4b0b049015d00269734195e6d3f0f6635cb692251aca6f9f8b3c" +checksum = "757c0ded2af11d8e739c4daea1ac623dd1624b06c844cf3f5a39f1bdbd99bb12" dependencies = [ "darling_core", "darling_macro", @@ -236,9 +236,9 @@ dependencies = [ [[package]] name = "darling_core" -version = "0.12.4" +version = "0.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8e91455b86830a1c21799d94524df0845183fa55bafd9aa137b01c7d1065fa36" +checksum = "2c34d8efb62d0c2d7f60ece80f75e5c63c1588ba68032740494b0b9a996466e3" dependencies = [ "fnv", "ident_case", @@ -250,9 +250,9 @@ dependencies = [ [[package]] name = "darling_macro" -version = "0.12.4" +version = "0.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "29b5acf0dea37a7f66f7b25d2c5e93fd46f8f6968b1a5d7a3e02e97768afc95a" +checksum = "ade7bff147130fe5e6d39f089c6bd49ec0250f35d70b2eebf72afdfc919f15cc" dependencies = [ "darling_core", "quote", @@ -343,12 +343,6 @@ dependencies = [ "serde", ] -[[package]] -name = "fixedbitset" -version = "0.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "37ab347416e802de484e4d03c7316c48f1ecb56574dfd4a46a80f173ce1de04d" - [[package]] name = "fixedbitset" version = "0.4.0" @@ -500,9 +494,9 @@ dependencies = [ [[package]] name = "h2" -version = "0.3.6" +version = "0.3.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c06815895acec637cd6ed6e9662c935b866d20a106f8361892893a7d9234964" +checksum = "7fd819562fcebdac5afc5c113c3ec36f902840b70fd4fc458799c8ce4607ae55" dependencies = [ "bytes", "fnv", @@ -527,7 +521,7 @@ dependencies = [ "k8s-openapi", "kube", "lazy_static", - "petgraph 0.6.0", + "petgraph", "regex", "schemars", "serde", @@ -554,7 +548,6 @@ dependencies = [ "jsonwebtoken", "k8s-openapi", "kube", - "kube-runtime", "maplit", "prost", "ring", @@ -606,9 +599,9 @@ dependencies = [ [[package]] name = "http-body" -version = "0.4.3" +version = "0.4.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "399c583b2979440c60be0821a6199eca73bc3c8dcd9d070d75ac726e2c6186e5" +checksum = "1ff4f84919677303da5f147645dbea6b1881f368d03ac84e1dc09031ebd7b2c6" dependencies = [ "bytes", "http", @@ -629,9 +622,9 @@ checksum = "6456b8a6c8f33fee7d958fcd1b60d55b11940a79e63ae87013e6d22e26034440" [[package]] name = "hyper" -version = "0.14.13" +version = "0.14.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "15d1cfb9e4f68655fa04c01f59edb405b6074a0f7118ea881e5026e4a1cd8593" +checksum = "2b91bb1f221b6ea1f1e4371216b70f40748774c2fb5971b450c07773fb92d26b" dependencies = [ "bytes", "futures-channel", @@ -707,15 +700,6 @@ dependencies = [ "hashbrown", ] -[[package]] -name = "input_buffer" -version = "0.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f97967975f448f1a7ddb12b0bc41069d09ed6a1c161a92687e057325db35d413" -dependencies = [ - "bytes", -] - [[package]] name = "itertools" version = "0.10.1" @@ -778,9 +762,9 @@ dependencies = [ [[package]] name = "k8s-openapi" -version = "0.13.0" +version = "0.13.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "748acc444200aa3528dc131a8048e131a9e75a611a52d152e276e99199313d1a" +checksum = "4f8de9873b904e74b3533f77493731ee26742418077503683db44e1b3c54aa5c" dependencies = [ "base64 0.13.0", "bytes", @@ -795,9 +779,22 @@ dependencies = [ [[package]] name = "kube" -version = "0.59.0" +version = "0.63.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "75e877325e5540a3041b519bd7ee27a858691f9f816cf533d652cbb33cbfea45" +dependencies = [ + "k8s-openapi", + "kube-client", + "kube-core", + "kube-derive", + "kube-runtime", +] + +[[package]] +name = "kube-client" +version = "0.63.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2bfa22c305a6d817b57a7afcd2e6ee23c80c6c93933edb02f210fdf73f837cc" +checksum = "bb8e1a36f17c63e263ba0ffa2c0658de315c75decad983d83aaeafeda578cc78" dependencies = [ "base64 0.13.0", "bytes", @@ -813,7 +810,6 @@ dependencies = [ "jsonpath_lib", "k8s-openapi", "kube-core", - "kube-derive", "pem", "pin-project 1.0.8", "rand", @@ -834,10 +830,11 @@ dependencies = [ [[package]] name = "kube-core" -version = "0.59.0" +version = "0.63.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c33d2272d8e530938bafc6cf4ac76f2a6f6c9ca684defcfab6c357913a43bcc" +checksum = "a91e572d244436fbc0d0b5a4829d96b9d623e08eb6b5d1e80418c1fab10b162a" dependencies = [ + "chrono", "form_urlencoded", "http", "json-patch", @@ -850,9 +847,9 @@ dependencies = [ [[package]] name = "kube-derive" -version = "0.59.0" +version = "0.63.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "53dc9fa719dd21d1a4c155cf8936f618a687f3590885e09d9261727cd5dc56a5" +checksum = "2034f57f3db36978ef366f45f1e263e623d9a6a8fcc6a6b1ef8879a213e1d2c4" dependencies = [ "darling", "proc-macro2", @@ -863,16 +860,16 @@ dependencies = [ [[package]] name = "kube-runtime" -version = "0.59.0" +version = "0.63.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cb40d5730a3ac47b7153c7ad0494a66881bbdf0c17ead478b714dd82c9dba259" +checksum = "6018cf8410f9d460be3a3ac35deef63b71c860c368016d7bf6871994343728b4" dependencies = [ "dashmap", "derivative", "futures", "json-patch", "k8s-openapi", - "kube", + "kube-client", "pin-project 1.0.8", "serde", "serde_json", @@ -891,9 +888,9 @@ checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" [[package]] name = "libc" -version = "0.2.103" +version = "0.2.106" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dd8f7255a17a627354f321ef0055d63b898c6fb27eff628af4d1b66b7331edf6" +checksum = "a60553f9a9e039a333b4e9b20573b9e9b9c0bb3a11e201ccc48ef4283456d673" [[package]] name = "linked-hash-map" @@ -918,9 +915,9 @@ checksum = "3e2e65a1a2e43cfcb47a895c4c8b10d1f4a61097f9f254f183aee60cad9c651d" [[package]] name = "matchers" -version = "0.0.1" +version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f099785f7595cc4b4553a174ce30dd7589ef93391ff414dbb67f62392b9e0ce1" +checksum = "8263075bb86c5a1b1427b5ae862e8889656f126e9f77c484496e8b47cf5c5558" dependencies = [ "regex-automata", ] @@ -949,9 +946,9 @@ dependencies = [ [[package]] name = "mio" -version = "0.7.13" +version = "0.7.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8c2bdb6314ec10835cd3293dd268473a835c02b7b352e788be788b3c6ca6bb16" +checksum = "8067b404fe97c70829f082dec8bcf4f71225d7eaea1d8645349cb76fa06205cc" dependencies = [ "libc", "log", @@ -1068,23 +1065,13 @@ version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d4fd5641d01c8f18a23da7b6fe29298ff4b55afcccdf78973b24cf3175fee32e" -[[package]] -name = "petgraph" -version = "0.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "467d164a6de56270bd7c4d070df81d07beace25012d5103ced4e9ff08d6afdb7" -dependencies = [ - "fixedbitset 0.2.0", - "indexmap", -] - [[package]] name = "petgraph" version = "0.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4a13a2fa9d0b63e5f22328828741e523766fff0ee9e779316902290dff3f824f" dependencies = [ - "fixedbitset 0.4.0", + "fixedbitset", "indexmap", ] @@ -1142,9 +1129,9 @@ checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" [[package]] name = "ppv-lite86" -version = "0.2.10" +version = "0.2.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ac74c624d6b2d21f425f752262f42188365d7b8ff1aff74c82e45136510a4857" +checksum = "ed0cfbc8191465bed66e1718596ee0b0b35d5ee1f41c5df2189d0fe8bde535ba" [[package]] name = "proc-macro-hack" @@ -1160,18 +1147,18 @@ checksum = "bc881b2c22681370c6a780e47af9840ef841837bc98118431d4e1868bd0c1086" [[package]] name = "proc-macro2" -version = "1.0.29" +version = "1.0.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b9f5105d4fdaab20335ca9565e106a5d9b82b6219b5ba735731124ac6711d23d" +checksum = "ba508cc11742c0dc5c1659771673afbab7a0efab23aa17e854cbab0837ed0b43" dependencies = [ "unicode-xid", ] [[package]] name = "prost" -version = "0.8.0" +version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "de5e2533f59d08fcf364fd374ebda0692a70bd6d7e66ef97f306f45c6c5d8020" +checksum = "444879275cb4fd84958b1a1d5420d15e6fcf7c235fe47f053c9c2a80aceb6001" dependencies = [ "bytes", "prost-derive", @@ -1179,27 +1166,29 @@ dependencies = [ [[package]] name = "prost-build" -version = "0.8.0" +version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "355f634b43cdd80724ee7848f95770e7e70eefa6dcf14fea676216573b8fd603" +checksum = "62941722fb675d463659e49c4f3fe1fe792ff24fe5bbaa9c08cd3b98a1c354f5" dependencies = [ "bytes", "heck", "itertools", + "lazy_static", "log", "multimap", - "petgraph 0.5.1", + "petgraph", "prost", "prost-types", + "regex", "tempfile", "which", ] [[package]] name = "prost-derive" -version = "0.8.0" +version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "600d2f334aa05acb02a755e217ef1ab6dea4d51b58b7846588b747edec04efba" +checksum = "f9cc1a3263e07e0bf68e96268f37665207b49560d98739662cdfaae215c720fe" dependencies = [ "anyhow", "itertools", @@ -1210,9 +1199,9 @@ dependencies = [ [[package]] name = "prost-types" -version = "0.8.0" +version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "603bbd6394701d13f3f25aada59c7de9d35a6a5887cfc156181234a44002771b" +checksum = "534b7a0e836e3c482d2693070f982e39e7611da9695d4d1f5a4b186b51faef0a" dependencies = [ "bytes", "prost", @@ -1220,9 +1209,9 @@ dependencies = [ [[package]] name = "quote" -version = "1.0.9" +version = "1.0.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c3d0b9745dc2debf507c8422de05d7226cc1f0644216dfdfead988f9b1ab32a7" +checksum = "38bc8cc6a5f2e3655e0899c1b848643b2562f853f114bfec7be120678e3ace05" dependencies = [ "proc-macro2", ] @@ -1535,9 +1524,9 @@ dependencies = [ [[package]] name = "sharded-slab" -version = "0.1.3" +version = "0.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "740223c51853f3145fe7c90360d2d4232f2b62e3449489c207eccde818979982" +checksum = "900fba806f70c630b0a382d0d825e17a0f19fcd059a2ade1ff237bcddf446b31" dependencies = [ "lazy_static", ] @@ -1564,9 +1553,9 @@ dependencies = [ [[package]] name = "slab" -version = "0.4.4" +version = "0.4.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c307a32c1c5c437f38c7fd45d753050587732ba8628319fbdf12a7e289ccc590" +checksum = "9def91fd1e018fe007022791f865d0ccc9b3a0d5001e01aabb8b40e46000afb5" [[package]] name = "smallvec" @@ -1621,9 +1610,9 @@ checksum = "73473c0e59e6d5812c5dfe2a064a6444949f089e20eec9a2e5506596494e4623" [[package]] name = "syn" -version = "1.0.77" +version = "1.0.81" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5239bc68e0fef57495900cfea4e8dc75596d9a319d7e16b1e0a440d24e6fe0a0" +checksum = "f2afee18b8beb5a596ecb4a2dce128c719b4ba399d34126b9e4396e3f9860966" dependencies = [ "proc-macro2", "quote", @@ -1652,18 +1641,18 @@ dependencies = [ [[package]] name = "thiserror" -version = "1.0.29" +version = "1.0.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "602eca064b2d83369e2b2f34b09c70b605402801927c65c11071ac911d299b88" +checksum = "854babe52e4df1653706b98fcfc05843010039b406875930a70e4d9644e5c417" dependencies = [ "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "1.0.29" +version = "1.0.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bad553cc2c78e8de258400763a647e80e6d1b31ee237275d756f6836d204494c" +checksum = "aa32fd3f627f367fe16f893e2597ae3c05020f8bba2666a4e6ea73d377e5714b" dependencies = [ "proc-macro2", "quote", @@ -1707,9 +1696,9 @@ checksum = "cda74da7e1a664f795bb1f8a87ec406fb89a02522cf6e50620d016add6dbbf5c" [[package]] name = "tokio" -version = "1.12.0" +version = "1.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c2c2416fdedca8443ae44b4527de1ea633af61d8f7169ffa6e72c5b53d24efcc" +checksum = "588b2d10a336da58d877567cd8fb8a14b463e2104910f8132cd054b4b96e29ee" dependencies = [ "autocfg", "bytes", @@ -1736,9 +1725,9 @@ dependencies = [ [[package]] name = "tokio-macros" -version = "1.4.1" +version = "1.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "154794c8f499c2619acd19e839294703e9e32e7630ef5f46ea80d4ef0fbee5eb" +checksum = "114383b041aa6212c579467afa0075fbbdd0718de036100bc0ba7961d8cb9095" dependencies = [ "proc-macro2", "quote", @@ -1758,9 +1747,9 @@ dependencies = [ [[package]] name = "tokio-stream" -version = "0.1.7" +version = "0.1.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b2f3f698253f03119ac0102beaa64f67a67e08074d03a22d18784104543727f" +checksum = "50145484efff8818b5ccd256697f36863f587da82cf8b409c53adf1e840798e3" dependencies = [ "futures-core", "pin-project-lite", @@ -1770,9 +1759,9 @@ dependencies = [ [[package]] name = "tokio-tungstenite" -version = "0.14.0" +version = "0.15.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1e96bb520beab540ab664bd5a9cfeaa1fcd846fa68c830b42e2c8963071251d2" +checksum = "511de3f85caf1c98983545490c3d09685fa8eb634e57eec22bb4db271f46cbd8" dependencies = [ "futures-util", "log", @@ -1783,9 +1772,9 @@ dependencies = [ [[package]] name = "tokio-util" -version = "0.6.8" +version = "0.6.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08d3725d3efa29485e87311c5b699de63cde14b00ed4d256b8318aa30ca452cd" +checksum = "9e99e1983e5d376cd8eb4b66604d2e99e79f5bd988c3055891dcd8c9e2604cc0" dependencies = [ "bytes", "futures-core", @@ -1798,9 +1787,9 @@ dependencies = [ [[package]] name = "tonic" -version = "0.5.2" +version = "0.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "796c5e1cd49905e65dd8e700d4cb1dffcbfdb4fc9d017de08c1a537afd83627c" +checksum = "24203b79cf2d68909da91178db3026e77054effba0c5d93deb870d3ca7b35afa" dependencies = [ "async-stream", "async-trait", @@ -1829,9 +1818,9 @@ dependencies = [ [[package]] name = "tonic-build" -version = "0.5.2" +version = "0.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "12b52d07035516c2b74337d2ac7746075e7dcae7643816c1b12c5ff8a7484c08" +checksum = "88358bb1dcfeb62dcce85c63006cafb964b7be481d522b7e09589d4d1e718d2a" dependencies = [ "proc-macro2", "prost-build", @@ -1841,14 +1830,15 @@ dependencies = [ [[package]] name = "tower" -version = "0.4.8" +version = "0.4.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f60422bc7fefa2f3ec70359b8ff1caff59d785877eb70595904605bcc412470f" +checksum = "c00e500fff5fa1131c866b246041a6bf96da9c965f8fe4128cb1421f23e93c00" dependencies = [ "futures-core", "futures-util", "indexmap", "pin-project 1.0.8", + "pin-project-lite", "rand", "slab", "tokio", @@ -1894,9 +1884,9 @@ checksum = "360dfd1d6d30e05fda32ace2c8c70e9c0a9da713275777f5a4dbb8a1893930c6" [[package]] name = "tracing" -version = "0.1.28" +version = "0.1.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "84f96e095c0c82419687c20ddf5cb3eadb61f4e1405923c9dc8e53a1adacbda8" +checksum = "375a639232caf30edfc78e8d89b2d4c375515393e7af7e16f01cd96917fb2105" dependencies = [ "cfg-if", "log", @@ -1907,9 +1897,9 @@ dependencies = [ [[package]] name = "tracing-attributes" -version = "0.1.16" +version = "0.1.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "98863d0dd09fa59a1b79c6750ad80dbda6b75f4e71c437a6a1a8cb91a8bcbd77" +checksum = "f4f480b8f81512e825f337ad51e94c1eb5d3bbdf2b363dcd01e2b19a9ffe3f8e" dependencies = [ "proc-macro2", "quote", @@ -1918,9 +1908,9 @@ dependencies = [ [[package]] name = "tracing-core" -version = "0.1.20" +version = "0.1.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "46125608c26121c81b0c6d693eab5a420e416da7e43c426d2e8f7df8da8a3acf" +checksum = "1f4ed65637b8390770814083d20756f87bfa2c21bf2f110babdc5438351746e4" dependencies = [ "lazy_static", ] @@ -1946,36 +1936,22 @@ dependencies = [ "tracing-core", ] -[[package]] -name = "tracing-serde" -version = "0.1.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fb65ea441fbb84f9f6748fd496cf7f63ec9af5bca94dd86456978d055e8eb28b" -dependencies = [ - "serde", - "tracing-core", -] - [[package]] name = "tracing-subscriber" -version = "0.2.24" +version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fdd0568dbfe3baf7048b7908d2b32bca0d81cd56bec6d2a8f894b01d74f86be3" +checksum = "80a4ddde70311d8da398062ecf6fc2c309337de6b0f77d6c27aff8d53f6fca52" dependencies = [ "ansi_term", - "chrono", "lazy_static", "matchers", "regex", - "serde", - "serde_json", "sharded-slab", "smallvec", "thread_local", "tracing", "tracing-core", "tracing-log", - "tracing-serde", ] [[package]] @@ -1995,16 +1971,15 @@ checksum = "59547bce71d9c38b83d9c0e92b6066c4253371f15005def0c30d9657f50c7642" [[package]] name = "tungstenite" -version = "0.13.0" +version = "0.14.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5fe8dada8c1a3aeca77d6b51a4f1314e0f4b8e438b7b1b71e3ddaca8080e4093" +checksum = "a0b2d8558abd2e276b0a8df5c05a2ec762609344191e5fd23e292c910e9165b5" dependencies = [ "base64 0.13.0", "byteorder", "bytes", "http", "httparse", - "input_buffer", "log", "rand", "sha-1", @@ -2021,9 +1996,9 @@ checksum = "b63708a265f51345575b27fe43f9500ad611579e764c79edbc2037b1121959ec" [[package]] name = "unicode-bidi" -version = "0.3.6" +version = "0.3.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "246f4c42e67e7a4e3c6106ff716a5d067d4132a642840b242e357e468a2a0085" +checksum = "1a01404663e3db436ed2746d9fefef640d868edae3cceb81c3b8d5732fda678f" [[package]] name = "unicode-normalization" diff --git a/hadron-operator/Cargo.toml b/hadron-operator/Cargo.toml index d2d5231..5d15db8 100644 --- a/hadron-operator/Cargo.toml +++ b/hadron-operator/Cargo.toml @@ -1,6 +1,6 @@ [package] authors = ["Anthony Dodd "] -edition = "2018" +edition = "2021" name = "hadron-operator" version = "0.1.0-beta.0" description = "The Hadron Kubernetes operator." @@ -20,24 +20,23 @@ hadron-core = { path = "../hadron-core" } http = "0.2" hyper = { version = "0.14", default-features = false, features = ["http1", "http2", "server", "runtime"] } jsonwebtoken = "7.2" -k8s-openapi = { version = "0.13", default-features = false, features = ["api", "v1_18"] } -kube = { version = "0.59", default-features = false, features = ["admission", "derive", "gzip", "jsonpatch", "rustls-tls", "ws"] } -kube-runtime = { version = "0.59", default-features = false } +k8s-openapi = { version = "0.13", default-features = false, features = ["api", "v1_22"] } +kube = { version = "0.63", default-features = false, features = ["admission", "client", "derive", "gzip", "jsonpatch", "runtime", "rustls-tls", "ws"] } maplit = "1" -prost = "0.8" +prost = "0.9" ring = "0.16" serde = { version="1", features = ["derive"] } tokio = { version = "1", default-features = false, features = [ # See https://docs.rs/tokio/latest/tokio/#feature-flags "rt-multi-thread", "io-util", "net", "time", "macros", "sync", "signal", "fs", ]} -tokio-rustls = "0.22.0" +tokio-rustls = "0.22" tokio-stream = { version = "0.1", default-features = false, features = ["time", "net", "signal", "sync"] } -tonic = "0.5" +tonic = "0.6" tower-http = "0.1.1" tracing = { version = "0.1", features = ["attributes", "release_max_level_trace"] } -tracing-subscriber = "0.2" +tracing-subscriber = { version = "0.3", features = ["env-filter", "registry"] } [build-dependencies] anyhow = "1" -tonic-build = "0.5" +tonic-build = "0.6" diff --git a/hadron-operator/Dockerfile b/hadron-operator/Dockerfile index dc3f232..0d290ff 100644 --- a/hadron-operator/Dockerfile +++ b/hadron-operator/Dockerfile @@ -3,7 +3,7 @@ ## NOTE WELL: this Dockerfile assumes a context from the root of this repo. # syntax=docker/dockerfile:1.3 -ARG RUST_TAG=1.56.0 +ARG RUST_TAG=1.56.1 ARG ALPINE_TAG=3.14 FROM rust:${RUST_TAG}-alpine${ALPINE_TAG} as builder diff --git a/hadron-operator/src/grpc/operator.rs b/hadron-operator/src/grpc/operator.rs index 0ccdc1d..4123e34 100644 --- a/hadron-operator/src/grpc/operator.rs +++ b/hadron-operator/src/grpc/operator.rs @@ -33,7 +33,7 @@ pub mod operator_server { impl tonic::codegen::Service> for OperatorServer where T: Operator, - B: Body + Send + Sync + 'static, + B: Body + Send + 'static, B::Error: Into + Send + 'static, { type Response = http::Response; diff --git a/hadron-operator/src/k8s/coordination.rs b/hadron-operator/src/k8s/coordination.rs index 8c5f8de..69e0b09 100644 --- a/hadron-operator/src/k8s/coordination.rs +++ b/hadron-operator/src/k8s/coordination.rs @@ -15,7 +15,7 @@ use k8s_openapi::api::coordination::v1::Lease; use k8s_openapi::apimachinery::pkg::apis::meta::v1::MicroTime; use kube::api::{Api, ListParams, Patch, PatchParams}; use kube::client::Client; -use kube_runtime::{ +use kube::runtime::{ watcher, watcher::{Event, Result as WatcherResult}, }; diff --git a/hadron-operator/src/k8s/data.rs b/hadron-operator/src/k8s/data.rs index bc1a3e0..942e0d1 100644 --- a/hadron-operator/src/k8s/data.rs +++ b/hadron-operator/src/k8s/data.rs @@ -15,8 +15,8 @@ use std::time::Duration; use k8s_openapi::api::apps::v1::StatefulSet; use k8s_openapi::api::core::v1::{Secret, Service}; +use kube::runtime::watcher::Event; use kube::Resource; -use kube_runtime::watcher::Event; use crate::k8s::scheduler::SchedulerTask; use crate::k8s::{Controller, EventResult}; diff --git a/hadron-operator/src/k8s/mod.rs b/hadron-operator/src/k8s/mod.rs index 82b6ebc..d7eba62 100644 --- a/hadron-operator/src/k8s/mod.rs +++ b/hadron-operator/src/k8s/mod.rs @@ -25,7 +25,7 @@ use k8s_openapi::api::core::v1::{Secret, Service}; use k8s_openapi::apimachinery::pkg::apis::meta::v1::{MicroTime, ObjectMeta}; use kube::api::{Api, ListParams}; use kube::client::Client; -use kube_runtime::watcher::{watcher, Error as WatcherError, Event}; +use kube::runtime::watcher::{watcher, Error as WatcherError, Event}; use maplit::btreemap; use tokio::sync::{broadcast, mpsc}; use tokio::task::JoinHandle; diff --git a/hadron-operator/src/k8s/scheduler.rs b/hadron-operator/src/k8s/scheduler.rs index 5fe36f1..1985285 100644 --- a/hadron-operator/src/k8s/scheduler.rs +++ b/hadron-operator/src/k8s/scheduler.rs @@ -700,6 +700,21 @@ impl Controller { value: Some(STREAM_DATA_PATH.into()), ..Default::default() }, + EnvVar { + name: "RETENTION_POLICY_STRATEGY".into(), + value: Some(stream.spec.retention_policy.strategy.to_string()), + ..Default::default() + }, + EnvVar { + name: "RETENTION_POLICY_RETENTION_SECONDS".into(), + value: stream + .spec + .retention_policy + .retention_seconds + .as_ref() + .map(|val| format!("{}", val)), + ..Default::default() + }, ]), volume_mounts: Some(vec![VolumeMount { name: "data".into(), diff --git a/hadron-operator/src/server/webhook.rs b/hadron-operator/src/server/webhook.rs index b316a0c..c2b63a4 100644 --- a/hadron-operator/src/server/webhook.rs +++ b/hadron-operator/src/server/webhook.rs @@ -6,8 +6,8 @@ use axum::handler::{get, post, Handler}; use axum::http::StatusCode; use axum::{extract, Router}; use hyper::server::conn::Http; -use kube::api::admission::{AdmissionResponse, AdmissionReview, Operation}; use kube::api::DynamicObject; +use kube::core::admission::{AdmissionResponse, AdmissionReview, Operation}; use tokio::net::TcpListener; use tokio::sync::broadcast; use tokio::task::JoinHandle; diff --git a/hadron-stream/Cargo.lock b/hadron-stream/Cargo.lock index 0903d4f..e6af7ec 100644 --- a/hadron-stream/Cargo.lock +++ b/hadron-stream/Cargo.lock @@ -28,9 +28,9 @@ dependencies = [ [[package]] name = "anyhow" -version = "1.0.44" +version = "1.0.45" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "61604a8f862e1d5c3229fdd78f8b02c68dcf73a4c4b05fd636d12240aaa242c1" +checksum = "ee10e43ae4a853c0a3591d4e2ada1719e553be18199d9da9d4a83f5927c2f5c7" [[package]] name = "arc-swap" @@ -118,9 +118,9 @@ dependencies = [ [[package]] name = "bumpalo" -version = "3.7.1" +version = "3.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d9df67f7bf9ef8498769f994239c45613ef0c5899415fb58e9add412d2c1a538" +checksum = "8f1e260c3a9040a7c19a12468758f4c16f31a81a1fe087482be9570ec864bb6c" [[package]] name = "byteorder" @@ -136,9 +136,9 @@ checksum = "c4872d67bab6358e59559027aa3b9157c53d9358c51423c17554809a8858e0f8" [[package]] name = "cc" -version = "1.0.70" +version = "1.0.71" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d26a6ce4b6a484fa3edb70f7efa6fc430fd2b87285fe8b84304fd0936faa0dc0" +checksum = "79c2681d6594606957bbb8631c4b90a7fcaaa72cdb714743a437b156d6a7eedd" [[package]] name = "cfg-if" @@ -162,9 +162,9 @@ dependencies = [ [[package]] name = "core-foundation" -version = "0.9.1" +version = "0.9.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0a89e2ae426ea83155dccf10c0fa6b1463ef6d5fcb44cee0b224a408fa640a62" +checksum = "6888e10551bb93e424d8df1d07f1a8b4fceb0001a3a4b048bfc47554946f47b3" dependencies = [ "core-foundation-sys", "libc", @@ -172,9 +172,9 @@ dependencies = [ [[package]] name = "core-foundation-sys" -version = "0.8.2" +version = "0.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ea221b5284a47e40033bf9b66f35f984ec0ea2931eb03505246cd27a963f981b" +checksum = "5827cebf4670468b8772dd191856768aedcb1b0278a04f989f7766351917b9dc" [[package]] name = "cpufeatures" @@ -228,9 +228,9 @@ dependencies = [ [[package]] name = "darling" -version = "0.12.4" +version = "0.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f2c43f534ea4b0b049015d00269734195e6d3f0f6635cb692251aca6f9f8b3c" +checksum = "757c0ded2af11d8e739c4daea1ac623dd1624b06c844cf3f5a39f1bdbd99bb12" dependencies = [ "darling_core", "darling_macro", @@ -238,9 +238,9 @@ dependencies = [ [[package]] name = "darling_core" -version = "0.12.4" +version = "0.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8e91455b86830a1c21799d94524df0845183fa55bafd9aa137b01c7d1065fa36" +checksum = "2c34d8efb62d0c2d7f60ece80f75e5c63c1588ba68032740494b0b9a996466e3" dependencies = [ "fnv", "ident_case", @@ -252,9 +252,9 @@ dependencies = [ [[package]] name = "darling_macro" -version = "0.12.4" +version = "0.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "29b5acf0dea37a7f66f7b25d2c5e93fd46f8f6968b1a5d7a3e02e97768afc95a" +checksum = "ade7bff147130fe5e6d39f089c6bd49ec0250f35d70b2eebf72afdfc919f15cc" dependencies = [ "darling_core", "quote", @@ -345,12 +345,6 @@ dependencies = [ "serde", ] -[[package]] -name = "fixedbitset" -version = "0.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "37ab347416e802de484e4d03c7316c48f1ecb56574dfd4a46a80f173ce1de04d" - [[package]] name = "fixedbitset" version = "0.4.0" @@ -521,9 +515,9 @@ dependencies = [ [[package]] name = "h2" -version = "0.3.6" +version = "0.3.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c06815895acec637cd6ed6e9662c935b866d20a106f8361892893a7d9234964" +checksum = "7fd819562fcebdac5afc5c113c3ec36f902840b70fd4fc458799c8ce4607ae55" dependencies = [ "bytes", "fnv", @@ -548,7 +542,7 @@ dependencies = [ "k8s-openapi", "kube", "lazy_static", - "petgraph 0.6.0", + "petgraph", "regex", "schemars", "serde", @@ -575,7 +569,6 @@ dependencies = [ "jsonwebtoken", "k8s-openapi", "kube", - "kube-runtime", "prost", "rand", "serde", @@ -628,9 +621,9 @@ dependencies = [ [[package]] name = "http-body" -version = "0.4.3" +version = "0.4.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "399c583b2979440c60be0821a6199eca73bc3c8dcd9d070d75ac726e2c6186e5" +checksum = "1ff4f84919677303da5f147645dbea6b1881f368d03ac84e1dc09031ebd7b2c6" dependencies = [ "bytes", "http", @@ -651,9 +644,9 @@ checksum = "6456b8a6c8f33fee7d958fcd1b60d55b11940a79e63ae87013e6d22e26034440" [[package]] name = "hyper" -version = "0.14.13" +version = "0.14.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "15d1cfb9e4f68655fa04c01f59edb405b6074a0f7118ea881e5026e4a1cd8593" +checksum = "2b91bb1f221b6ea1f1e4371216b70f40748774c2fb5971b450c07773fb92d26b" dependencies = [ "bytes", "futures-channel", @@ -729,20 +722,11 @@ dependencies = [ "hashbrown", ] -[[package]] -name = "input_buffer" -version = "0.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f97967975f448f1a7ddb12b0bc41069d09ed6a1c161a92687e057325db35d413" -dependencies = [ - "bytes", -] - [[package]] name = "instant" -version = "0.1.11" +version = "0.1.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "716d3d89f35ac6a34fd0eed635395f4c3b76fa889338a4632e5231a8684216bd" +checksum = "7a5bbe824c507c5da5956355e86a746d82e0e1464f65d862cc5e71da70e94b2c" dependencies = [ "cfg-if", ] @@ -809,9 +793,9 @@ dependencies = [ [[package]] name = "k8s-openapi" -version = "0.13.0" +version = "0.13.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "748acc444200aa3528dc131a8048e131a9e75a611a52d152e276e99199313d1a" +checksum = "4f8de9873b904e74b3533f77493731ee26742418077503683db44e1b3c54aa5c" dependencies = [ "base64 0.13.0", "bytes", @@ -826,9 +810,22 @@ dependencies = [ [[package]] name = "kube" -version = "0.59.0" +version = "0.63.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "75e877325e5540a3041b519bd7ee27a858691f9f816cf533d652cbb33cbfea45" +dependencies = [ + "k8s-openapi", + "kube-client", + "kube-core", + "kube-derive", + "kube-runtime", +] + +[[package]] +name = "kube-client" +version = "0.63.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2bfa22c305a6d817b57a7afcd2e6ee23c80c6c93933edb02f210fdf73f837cc" +checksum = "bb8e1a36f17c63e263ba0ffa2c0658de315c75decad983d83aaeafeda578cc78" dependencies = [ "base64 0.13.0", "bytes", @@ -844,7 +841,6 @@ dependencies = [ "jsonpath_lib", "k8s-openapi", "kube-core", - "kube-derive", "pem", "pin-project 1.0.8", "rand", @@ -865,10 +861,11 @@ dependencies = [ [[package]] name = "kube-core" -version = "0.59.0" +version = "0.63.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c33d2272d8e530938bafc6cf4ac76f2a6f6c9ca684defcfab6c357913a43bcc" +checksum = "a91e572d244436fbc0d0b5a4829d96b9d623e08eb6b5d1e80418c1fab10b162a" dependencies = [ + "chrono", "form_urlencoded", "http", "json-patch", @@ -881,9 +878,9 @@ dependencies = [ [[package]] name = "kube-derive" -version = "0.59.0" +version = "0.63.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "53dc9fa719dd21d1a4c155cf8936f618a687f3590885e09d9261727cd5dc56a5" +checksum = "2034f57f3db36978ef366f45f1e263e623d9a6a8fcc6a6b1ef8879a213e1d2c4" dependencies = [ "darling", "proc-macro2", @@ -894,16 +891,16 @@ dependencies = [ [[package]] name = "kube-runtime" -version = "0.59.0" +version = "0.63.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cb40d5730a3ac47b7153c7ad0494a66881bbdf0c17ead478b714dd82c9dba259" +checksum = "6018cf8410f9d460be3a3ac35deef63b71c860c368016d7bf6871994343728b4" dependencies = [ "dashmap", "derivative", "futures", "json-patch", "k8s-openapi", - "kube", + "kube-client", "pin-project 1.0.8", "serde", "serde_json", @@ -922,9 +919,9 @@ checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" [[package]] name = "libc" -version = "0.2.103" +version = "0.2.106" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dd8f7255a17a627354f321ef0055d63b898c6fb27eff628af4d1b66b7331edf6" +checksum = "a60553f9a9e039a333b4e9b20573b9e9b9c0bb3a11e201ccc48ef4283456d673" [[package]] name = "linked-hash-map" @@ -952,9 +949,9 @@ dependencies = [ [[package]] name = "matchers" -version = "0.0.1" +version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f099785f7595cc4b4553a174ce30dd7589ef93391ff414dbb67f62392b9e0ce1" +checksum = "8263075bb86c5a1b1427b5ae862e8889656f126e9f77c484496e8b47cf5c5558" dependencies = [ "regex-automata", ] @@ -992,9 +989,9 @@ dependencies = [ [[package]] name = "mio" -version = "0.7.13" +version = "0.7.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8c2bdb6314ec10835cd3293dd268473a835c02b7b352e788be788b3c6ca6bb16" +checksum = "8067b404fe97c70829f082dec8bcf4f71225d7eaea1d8645349cb76fa06205cc" dependencies = [ "libc", "log", @@ -1136,23 +1133,13 @@ version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d4fd5641d01c8f18a23da7b6fe29298ff4b55afcccdf78973b24cf3175fee32e" -[[package]] -name = "petgraph" -version = "0.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "467d164a6de56270bd7c4d070df81d07beace25012d5103ced4e9ff08d6afdb7" -dependencies = [ - "fixedbitset 0.2.0", - "indexmap", -] - [[package]] name = "petgraph" version = "0.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4a13a2fa9d0b63e5f22328828741e523766fff0ee9e779316902290dff3f824f" dependencies = [ - "fixedbitset 0.4.0", + "fixedbitset", "indexmap", ] @@ -1210,9 +1197,9 @@ checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" [[package]] name = "ppv-lite86" -version = "0.2.10" +version = "0.2.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ac74c624d6b2d21f425f752262f42188365d7b8ff1aff74c82e45136510a4857" +checksum = "ed0cfbc8191465bed66e1718596ee0b0b35d5ee1f41c5df2189d0fe8bde535ba" [[package]] name = "proc-macro-hack" @@ -1228,18 +1215,18 @@ checksum = "bc881b2c22681370c6a780e47af9840ef841837bc98118431d4e1868bd0c1086" [[package]] name = "proc-macro2" -version = "1.0.29" +version = "1.0.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b9f5105d4fdaab20335ca9565e106a5d9b82b6219b5ba735731124ac6711d23d" +checksum = "ba508cc11742c0dc5c1659771673afbab7a0efab23aa17e854cbab0837ed0b43" dependencies = [ "unicode-xid", ] [[package]] name = "prost" -version = "0.8.0" +version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "de5e2533f59d08fcf364fd374ebda0692a70bd6d7e66ef97f306f45c6c5d8020" +checksum = "444879275cb4fd84958b1a1d5420d15e6fcf7c235fe47f053c9c2a80aceb6001" dependencies = [ "bytes", "prost-derive", @@ -1247,27 +1234,29 @@ dependencies = [ [[package]] name = "prost-build" -version = "0.8.0" +version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "355f634b43cdd80724ee7848f95770e7e70eefa6dcf14fea676216573b8fd603" +checksum = "62941722fb675d463659e49c4f3fe1fe792ff24fe5bbaa9c08cd3b98a1c354f5" dependencies = [ "bytes", "heck", "itertools", + "lazy_static", "log", "multimap", - "petgraph 0.5.1", + "petgraph", "prost", "prost-types", + "regex", "tempfile", "which", ] [[package]] name = "prost-derive" -version = "0.8.0" +version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "600d2f334aa05acb02a755e217ef1ab6dea4d51b58b7846588b747edec04efba" +checksum = "f9cc1a3263e07e0bf68e96268f37665207b49560d98739662cdfaae215c720fe" dependencies = [ "anyhow", "itertools", @@ -1278,9 +1267,9 @@ dependencies = [ [[package]] name = "prost-types" -version = "0.8.0" +version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "603bbd6394701d13f3f25aada59c7de9d35a6a5887cfc156181234a44002771b" +checksum = "534b7a0e836e3c482d2693070f982e39e7611da9695d4d1f5a4b186b51faef0a" dependencies = [ "bytes", "prost", @@ -1597,9 +1586,9 @@ dependencies = [ [[package]] name = "sharded-slab" -version = "0.1.3" +version = "0.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "740223c51853f3145fe7c90360d2d4232f2b62e3449489c207eccde818979982" +checksum = "900fba806f70c630b0a382d0d825e17a0f19fcd059a2ade1ff237bcddf446b31" dependencies = [ "lazy_static", ] @@ -1626,9 +1615,9 @@ dependencies = [ [[package]] name = "slab" -version = "0.4.4" +version = "0.4.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c307a32c1c5c437f38c7fd45d753050587732ba8628319fbdf12a7e289ccc590" +checksum = "9def91fd1e018fe007022791f865d0ccc9b3a0d5001e01aabb8b40e46000afb5" [[package]] name = "sled" @@ -1699,9 +1688,9 @@ checksum = "73473c0e59e6d5812c5dfe2a064a6444949f089e20eec9a2e5506596494e4623" [[package]] name = "syn" -version = "1.0.78" +version = "1.0.81" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4eac2e6c19f5c3abc0c229bea31ff0b9b091c7b14990e8924b92902a303a0c0" +checksum = "f2afee18b8beb5a596ecb4a2dce128c719b4ba399d34126b9e4396e3f9860966" dependencies = [ "proc-macro2", "quote", @@ -1724,18 +1713,18 @@ dependencies = [ [[package]] name = "thiserror" -version = "1.0.29" +version = "1.0.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "602eca064b2d83369e2b2f34b09c70b605402801927c65c11071ac911d299b88" +checksum = "854babe52e4df1653706b98fcfc05843010039b406875930a70e4d9644e5c417" dependencies = [ "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "1.0.29" +version = "1.0.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bad553cc2c78e8de258400763a647e80e6d1b31ee237275d756f6836d204494c" +checksum = "aa32fd3f627f367fe16f893e2597ae3c05020f8bba2666a4e6ea73d377e5714b" dependencies = [ "proc-macro2", "quote", @@ -1779,9 +1768,9 @@ checksum = "cda74da7e1a664f795bb1f8a87ec406fb89a02522cf6e50620d016add6dbbf5c" [[package]] name = "tokio" -version = "1.12.0" +version = "1.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c2c2416fdedca8443ae44b4527de1ea633af61d8f7169ffa6e72c5b53d24efcc" +checksum = "588b2d10a336da58d877567cd8fb8a14b463e2104910f8132cd054b4b96e29ee" dependencies = [ "autocfg", "bytes", @@ -1808,9 +1797,9 @@ dependencies = [ [[package]] name = "tokio-macros" -version = "1.4.1" +version = "1.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "154794c8f499c2619acd19e839294703e9e32e7630ef5f46ea80d4ef0fbee5eb" +checksum = "114383b041aa6212c579467afa0075fbbdd0718de036100bc0ba7961d8cb9095" dependencies = [ "proc-macro2", "quote", @@ -1830,9 +1819,9 @@ dependencies = [ [[package]] name = "tokio-stream" -version = "0.1.7" +version = "0.1.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b2f3f698253f03119ac0102beaa64f67a67e08074d03a22d18784104543727f" +checksum = "50145484efff8818b5ccd256697f36863f587da82cf8b409c53adf1e840798e3" dependencies = [ "futures-core", "pin-project-lite", @@ -1842,9 +1831,9 @@ dependencies = [ [[package]] name = "tokio-tungstenite" -version = "0.14.0" +version = "0.15.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1e96bb520beab540ab664bd5a9cfeaa1fcd846fa68c830b42e2c8963071251d2" +checksum = "511de3f85caf1c98983545490c3d09685fa8eb634e57eec22bb4db271f46cbd8" dependencies = [ "futures-util", "log", @@ -1855,9 +1844,9 @@ dependencies = [ [[package]] name = "tokio-util" -version = "0.6.8" +version = "0.6.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08d3725d3efa29485e87311c5b699de63cde14b00ed4d256b8318aa30ca452cd" +checksum = "9e99e1983e5d376cd8eb4b66604d2e99e79f5bd988c3055891dcd8c9e2604cc0" dependencies = [ "bytes", "futures-core", @@ -1870,9 +1859,9 @@ dependencies = [ [[package]] name = "tonic" -version = "0.5.2" +version = "0.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "796c5e1cd49905e65dd8e700d4cb1dffcbfdb4fc9d017de08c1a537afd83627c" +checksum = "24203b79cf2d68909da91178db3026e77054effba0c5d93deb870d3ca7b35afa" dependencies = [ "async-stream", "async-trait", @@ -1901,9 +1890,9 @@ dependencies = [ [[package]] name = "tonic-build" -version = "0.5.2" +version = "0.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "12b52d07035516c2b74337d2ac7746075e7dcae7643816c1b12c5ff8a7484c08" +checksum = "88358bb1dcfeb62dcce85c63006cafb964b7be481d522b7e09589d4d1e718d2a" dependencies = [ "proc-macro2", "prost-build", @@ -1913,14 +1902,15 @@ dependencies = [ [[package]] name = "tower" -version = "0.4.8" +version = "0.4.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f60422bc7fefa2f3ec70359b8ff1caff59d785877eb70595904605bcc412470f" +checksum = "c00e500fff5fa1131c866b246041a6bf96da9c965f8fe4128cb1421f23e93c00" dependencies = [ "futures-core", "futures-util", "indexmap", "pin-project 1.0.8", + "pin-project-lite", "rand", "slab", "tokio", @@ -1966,9 +1956,9 @@ checksum = "360dfd1d6d30e05fda32ace2c8c70e9c0a9da713275777f5a4dbb8a1893930c6" [[package]] name = "tracing" -version = "0.1.28" +version = "0.1.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "84f96e095c0c82419687c20ddf5cb3eadb61f4e1405923c9dc8e53a1adacbda8" +checksum = "375a639232caf30edfc78e8d89b2d4c375515393e7af7e16f01cd96917fb2105" dependencies = [ "cfg-if", "log", @@ -1979,9 +1969,9 @@ dependencies = [ [[package]] name = "tracing-attributes" -version = "0.1.16" +version = "0.1.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "98863d0dd09fa59a1b79c6750ad80dbda6b75f4e71c437a6a1a8cb91a8bcbd77" +checksum = "f4f480b8f81512e825f337ad51e94c1eb5d3bbdf2b363dcd01e2b19a9ffe3f8e" dependencies = [ "proc-macro2", "quote", @@ -2018,36 +2008,22 @@ dependencies = [ "tracing-core", ] -[[package]] -name = "tracing-serde" -version = "0.1.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fb65ea441fbb84f9f6748fd496cf7f63ec9af5bca94dd86456978d055e8eb28b" -dependencies = [ - "serde", - "tracing-core", -] - [[package]] name = "tracing-subscriber" -version = "0.2.24" +version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fdd0568dbfe3baf7048b7908d2b32bca0d81cd56bec6d2a8f894b01d74f86be3" +checksum = "80a4ddde70311d8da398062ecf6fc2c309337de6b0f77d6c27aff8d53f6fca52" dependencies = [ "ansi_term", - "chrono", "lazy_static", "matchers", "regex", - "serde", - "serde_json", "sharded-slab", "smallvec", "thread_local", "tracing", "tracing-core", "tracing-log", - "tracing-serde", ] [[package]] @@ -2067,16 +2043,15 @@ checksum = "59547bce71d9c38b83d9c0e92b6066c4253371f15005def0c30d9657f50c7642" [[package]] name = "tungstenite" -version = "0.13.0" +version = "0.14.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5fe8dada8c1a3aeca77d6b51a4f1314e0f4b8e438b7b1b71e3ddaca8080e4093" +checksum = "a0b2d8558abd2e276b0a8df5c05a2ec762609344191e5fd23e292c910e9165b5" dependencies = [ "base64 0.13.0", "byteorder", "bytes", "http", "httparse", - "input_buffer", "log", "rand", "sha-1", @@ -2093,9 +2068,9 @@ checksum = "b63708a265f51345575b27fe43f9500ad611579e764c79edbc2037b1121959ec" [[package]] name = "unicode-bidi" -version = "0.3.6" +version = "0.3.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "246f4c42e67e7a4e3c6106ff716a5d067d4132a642840b242e357e468a2a0085" +checksum = "1a01404663e3db436ed2746d9fefef640d868edae3cceb81c3b8d5732fda678f" [[package]] name = "unicode-normalization" diff --git a/hadron-stream/Cargo.toml b/hadron-stream/Cargo.toml index dc4a207..278569c 100644 --- a/hadron-stream/Cargo.toml +++ b/hadron-stream/Cargo.toml @@ -1,6 +1,6 @@ [package] authors = ["Anthony Dodd "] -edition = "2018" +edition = "2021" name = "hadron-stream" version = "0.1.0-beta.0" description = "The Hadron stream controller." @@ -20,10 +20,9 @@ futures = "0.3" hadron-core = { path = "../hadron-core" } http = "0.2" jsonwebtoken = "7.2" -k8s-openapi = { version = "0.13", default-features = false, features = ["api", "v1_18"] } -kube = { version = "0.59", default-features = false, features = ["admission", "derive", "gzip", "jsonpatch", "rustls-tls", "ws"] } -kube-runtime = { version = "0.59", default-features = false } -prost = "0.8" +k8s-openapi = { version = "0.13", default-features = false, features = ["api", "v1_22"] } +kube = { version = "0.63", default-features = false, features = ["admission", "client", "derive", "gzip", "jsonpatch", "runtime", "rustls-tls", "ws"] } +prost = "0.9" rand = "0.8" serde = { version="1", features = ["derive"] } sled = "0.34" @@ -33,14 +32,14 @@ tokio = { version = "1", default-features = false, features = [ "rt-multi-thread", "io-util", "net", "time", "macros", "sync", "signal", "fs", ]} tokio-stream = { version = "0.1", default-features = false, features = ["time", "net", "signal", "sync"] } -tonic = "0.5" +tonic = "0.6" tracing = { version = "0.1", features = ["attributes", "release_max_level_trace"] } -tracing-subscriber = "0.2" +tracing-subscriber = { version = "0.3", features = ["env-filter", "registry"] } uuid = { version="0.8", default-features=false, features=["serde", "v4"] } [build-dependencies] anyhow = "1" -tonic-build = "0.5" +tonic-build = "0.6" [dev-dependencies] -tempfile = "3.2.0" +tempfile = "3" diff --git a/hadron-stream/Dockerfile b/hadron-stream/Dockerfile index 3390fa5..94bd9bc 100644 --- a/hadron-stream/Dockerfile +++ b/hadron-stream/Dockerfile @@ -3,7 +3,7 @@ ## NOTE WELL: this Dockerfile assumes a context from the root of this repo. # syntax=docker/dockerfile:1.3 -ARG RUST_TAG=1.56.0 +ARG RUST_TAG=1.56.1 ARG ALPINE_TAG=3.14 FROM rust:${RUST_TAG}-alpine${ALPINE_TAG} as builder diff --git a/hadron-stream/src/app.rs b/hadron-stream/src/app.rs index f1c4579..569fb57 100644 --- a/hadron-stream/src/app.rs +++ b/hadron-stream/src/app.rs @@ -64,7 +64,7 @@ impl App { let tokens_handle = tokens.spawn(); let (stream_tx, stream_rx) = mpsc::channel(1000); - let (stream_ctl, stream_offset_signal) = StreamCtl::new(config.clone(), db.clone(), shutdown_tx.clone(), stream_rx) + let (stream_ctl, stream_offset_signal) = StreamCtl::new(config.clone(), db.clone(), shutdown_tx.clone(), stream_tx.clone(), stream_rx) .await .context("error spawning stream controller")?; let stream_handle = stream_ctl.spawn(); diff --git a/hadron-stream/src/config.rs b/hadron-stream/src/config.rs index a4d7c33..dc65b39 100644 --- a/hadron-stream/src/config.rs +++ b/hadron-stream/src/config.rs @@ -3,6 +3,8 @@ use anyhow::{Context, Result}; use serde::Deserialize; +use hadron_core::crd::StreamRetentionSpec; + /// Runtime configuration data. #[derive(Clone, Debug, Deserialize)] pub struct Config { @@ -30,6 +32,10 @@ pub struct Config { /// The path to the database on disk. #[serde(default = "crate::database::default_data_path")] pub storage_data_path: String, + + /// The retention policy to use for data on the Stream. + #[serde(default)] + pub retention_policy: StreamRetentionSpec, } impl Config { @@ -67,6 +73,7 @@ impl Config { partition: 0, storage_data_path: tmpdir.path().to_string_lossy().to_string(), + retention_policy: StreamRetentionSpec::default(), }), tmpdir, )) diff --git a/hadron-stream/src/config_test.rs b/hadron-stream/src/config_test.rs new file mode 100644 index 0000000..9603e7e --- /dev/null +++ b/hadron-stream/src/config_test.rs @@ -0,0 +1,173 @@ +use anyhow::Result; + +use super::*; +use hadron_core::crd::StreamRetentionPolicy; + +#[test] +fn config_deserializes_from_full_env() -> Result<()> { + let config: Config = envy::from_iter(vec![ + ("RUST_LOG".into(), "error".into()), + ("CLIENT_PORT".into(), "7000".into()), + ("SERVER_PORT".into(), "7001".into()), + ("NAMESPACE".into(), "default".into()), + ("STREAM".into(), "events".into()), + ("STATEFULSET".into(), "events".into()), + ("POD_NAME".into(), "events-0".into()), + ("STORAGE_DATA_PATH".into(), "/usr/local/hadron-stream/data".into()), + ("RETENTION_POLICY_STRATEGY".into(), "time".into()), + ("RETENTION_POLICY_RETENTION_SECONDS".into(), "604800".into()), + ])?; + + assert!( + config.rust_log == "error", + "unexpected value parsed for RUST_LOG, got {}, expected {}", + config.rust_log, + "" + ); + assert!( + config.client_port == 7000, + "unexpected value parsed for CLIENT_PORT, got {}, expected {}", + config.client_port, + "7000" + ); + assert!( + config.server_port == 7001, + "unexpected value parsed for SERVER_PORT, got {}, expected {}", + config.server_port, + "7001" + ); + assert!( + config.namespace == "default", + "unexpected value parsed for NAMESPACE, got {}, expected {}", + config.namespace, + "default" + ); + assert!( + config.stream == "events", + "unexpected value parsed for STREAM, got {}, expected {}", + config.stream, + "events" + ); + assert!( + config.statefulset == "events", + "unexpected value parsed for STATEFULSET, got {}, expected {}", + config.statefulset, + "events" + ); + assert!( + config.pod_name == "events-0", + "unexpected value parsed for POD_NAME, got {}, expected {}", + config.pod_name, + "events-0" + ); + assert!( + config.partition == 0, + "unexpected value derived for partition, got {}, expected {}", + config.partition, + 0 + ); + assert!( + config.storage_data_path == "/usr/local/hadron-stream/data", + "unexpected value parsed for STORAGE_DATA_PATH, got {}, expected {}", + config.storage_data_path, + "/usr/local/hadron-stream/data" + ); + assert!( + config.retention_policy.strategy == StreamRetentionPolicy::Time, + "unexpected value parsed for RETENTION_POLICY_STRATEGY, got {}, expected {}", + config.retention_policy.strategy, + StreamRetentionPolicy::Time + ); + assert!( + config.retention_policy.retention_seconds == Some(604800), + "unexpected value parsed for RETENTION_POLICY_RETENTION_SECONDS, got {:?}, expected {:?}", + config.retention_policy.retention_seconds, + Some(604800) + ); + + Ok(()) +} + +#[test] +fn config_deserializes_from_sparse_env() -> Result<()> { + let config: Config = envy::from_iter(vec![ + ("RUST_LOG".into(), "error".into()), + ("CLIENT_PORT".into(), "7000".into()), + ("SERVER_PORT".into(), "7001".into()), + ("NAMESPACE".into(), "default".into()), + ("STREAM".into(), "events".into()), + ("STATEFULSET".into(), "events".into()), + ("POD_NAME".into(), "events-0".into()), + ("STORAGE_DATA_PATH".into(), "/usr/local/hadron-stream/data".into()), + ("RETENTION_POLICY_STRATEGY".into(), "time".into()), + ])?; + + assert!( + config.rust_log == "error", + "unexpected value parsed for RUST_LOG, got {}, expected {}", + config.rust_log, + "" + ); + assert!( + config.client_port == 7000, + "unexpected value parsed for CLIENT_PORT, got {}, expected {}", + config.client_port, + "7000" + ); + assert!( + config.server_port == 7001, + "unexpected value parsed for SERVER_PORT, got {}, expected {}", + config.server_port, + "7001" + ); + assert!( + config.namespace == "default", + "unexpected value parsed for NAMESPACE, got {}, expected {}", + config.namespace, + "default" + ); + assert!( + config.stream == "events", + "unexpected value parsed for STREAM, got {}, expected {}", + config.stream, + "events" + ); + assert!( + config.statefulset == "events", + "unexpected value parsed for STATEFULSET, got {}, expected {}", + config.statefulset, + "events" + ); + assert!( + config.pod_name == "events-0", + "unexpected value parsed for POD_NAME, got {}, expected {}", + config.pod_name, + "events-0" + ); + assert!( + config.partition == 0, + "unexpected value derived for partition, got {}, expected {}", + config.partition, + 0 + ); + assert!( + config.storage_data_path == "/usr/local/hadron-stream/data", + "unexpected value parsed for STORAGE_DATA_PATH, got {}, expected {}", + config.storage_data_path, + "/usr/local/hadron-stream/data" + ); + assert!( + config.retention_policy.strategy == StreamRetentionPolicy::Time, + "unexpected value parsed for RETENTION_POLICY_STRATEGY, got {}, expected {}", + config.retention_policy.strategy, + StreamRetentionPolicy::Time + ); + assert!( + config.retention_policy.retention_seconds == Some(604800), + "unexpected value parsed for RETENTION_POLICY_RETENTION_SECONDS, got {:?}, expected {:?}", + config.retention_policy.retention_seconds, + Some(604800) + ); + + Ok(()) +} diff --git a/hadron-stream/src/grpc/stream.rs b/hadron-stream/src/grpc/stream.rs index ad37bd0..5642db6 100644 --- a/hadron-stream/src/grpc/stream.rs +++ b/hadron-stream/src/grpc/stream.rs @@ -9,28 +9,28 @@ pub struct Empty {} pub struct Event { /// The application defined ID of this event. /// - /// See [`id`](https://github.com/cloudevents/spec/blob/v1.0.1/spec.md#id). + /// See \[`id`\](). #[prost(string, tag = "1")] pub id: ::prost::alloc::string::String, /// The application defined source of this event. /// - /// See [`source`](https://github.com/cloudevents/spec/blob/v1.0.1/spec.md#source-1). + /// See \[`source`\](). #[prost(string, tag = "2")] pub source: ::prost::alloc::string::String, /// The CloudEvents specification version which the event uses. /// - /// See [`specversion`](https://github.com/cloudevents/spec/blob/v1.0.1/spec.md#specversion). + /// See \[`specversion`\](). #[prost(string, tag = "3")] pub specversion: ::prost::alloc::string::String, /// The type identifier of this event. /// - /// See [`type`](https://github.com/cloudevents/spec/blob/v1.0.1/spec.md#type). + /// See \[`type`\](). #[prost(string, tag = "4")] pub r#type: ::prost::alloc::string::String, /// Any additional optional attributes or extension attributes of this event. /// - /// See [`optional attributes`](https://github.com/cloudevents/spec/blob/v1.0.1/spec.md#optional-attributes) - /// and [`extension context attributes`](https://github.com/cloudevents/spec/blob/v1.0.1/spec.md#extension-context-attributes). + /// See [`optional attributes`]() + /// and [`extension context attributes`](). #[prost(map = "string, string", tag = "5")] pub optattrs: ::std::collections::HashMap<::prost::alloc::string::String, ::prost::alloc::string::String>, /// The data payload of this event. @@ -228,7 +228,7 @@ pub mod stream_controller_server { #[async_trait] pub trait StreamController: Send + Sync + 'static { #[doc = "Server streaming response type for the Metadata method."] - type MetadataStream: futures_core::Stream> + Send + Sync + 'static; + type MetadataStream: futures_core::Stream> + Send + 'static; #[doc = " Open a metadata stream."] async fn metadata(&self, request: tonic::Request) -> Result, tonic::Status>; #[doc = " Open a stream publisher channel."] @@ -236,13 +236,13 @@ pub mod stream_controller_server { &self, request: tonic::Request, ) -> Result, tonic::Status>; #[doc = "Server streaming response type for the StreamSubscribe method."] - type StreamSubscribeStream: futures_core::Stream> + Send + Sync + 'static; + type StreamSubscribeStream: futures_core::Stream> + Send + 'static; #[doc = " Open a stream subscriber channel."] async fn stream_subscribe( &self, request: tonic::Request>, ) -> Result, tonic::Status>; #[doc = "Server streaming response type for the PipelineSubscribe method."] - type PipelineSubscribeStream: futures_core::Stream> + Send + Sync + 'static; + type PipelineSubscribeStream: futures_core::Stream> + Send + 'static; #[doc = " Open a pipeline subscriber channel."] async fn pipeline_subscribe( &self, request: tonic::Request>, @@ -276,7 +276,7 @@ pub mod stream_controller_server { impl tonic::codegen::Service> for StreamControllerServer where T: StreamController, - B: Body + Send + Sync + 'static, + B: Body + Send + 'static, B::Error: Into + Send + 'static, { type Response = http::Response; diff --git a/hadron-stream/src/main.rs b/hadron-stream/src/main.rs index 244d45b..cfbb611 100644 --- a/hadron-stream/src/main.rs +++ b/hadron-stream/src/main.rs @@ -2,6 +2,8 @@ mod app; mod config; +#[cfg(test)] +mod config_test; mod database; mod error; mod futures; diff --git a/hadron-stream/src/stream/mod.rs b/hadron-stream/src/stream/mod.rs index 4abaf09..1733895 100644 --- a/hadron-stream/src/stream/mod.rs +++ b/hadron-stream/src/stream/mod.rs @@ -43,11 +43,12 @@ use tonic::Streaming; use crate::config::Config; use crate::database::Database; -use crate::error::{RpcResult, ERR_ITER_FAILURE}; +use crate::error::{RpcResult, ERR_DB_FLUSH, ERR_ITER_FAILURE}; use crate::grpc::{StreamPublishRequest, StreamPublishResponse, StreamSubscribeRequest, StreamSubscribeResponse, StreamSubscribeSetup}; use crate::models::stream::Subscription; use crate::stream::subscriber::StreamSubCtlMsg; use crate::utils; +use hadron_core::crd::{StreamRetentionPolicy, StreamRetentionSpec}; /// The key prefix used for storing stream events. /// @@ -72,6 +73,7 @@ pub const PREFIX_STREAM_SUB_OFFSETS: &[u8; 1] = b"o"; /// The key used to store the last written offset for the stream. pub const KEY_STREAM_LAST_WRITTEN_OFFSET: &[u8; 1] = b"l"; +const COMPACTION_INTERVAL: std::time::Duration = std::time::Duration::from_secs(60 * 30); const ERR_DECODING_STREAM_META_GROUP_NAME: &str = "error decoding stream meta group name from storage"; /// A controller encapsulating all logic for interacting with a stream. @@ -86,18 +88,18 @@ pub struct StreamCtl { partition: u32, /// A channel of inbound client requests. - requests: ReceiverStream, + requests_tx: mpsc::Sender, + /// A channel of inbound client requests. + requests_rx: ReceiverStream, /// A channel of requests for stream subscription. subs_tx: mpsc::Sender, /// A channel used for communicating the stream's last written offset value. offset_signal: watch::Sender, /// A channel used for triggering graceful shutdown. - _shutdown_tx: broadcast::Sender<()>, + shutdown_tx: broadcast::Sender<()>, /// A channel used for triggering graceful shutdown. shutdown_rx: BroadcastStream<()>, - /// A bool indicating that this controller has been descheduled and needs to shutdown. - descheduled: bool, /// A handle to this controller's spawned subscription controller. sub_ctl: JoinHandle>, @@ -108,12 +110,15 @@ pub struct StreamCtl { /// A `None` value indicates that no timestamp record exists on disk, and it will be populated /// when the next batch is written to the stream. earliest_timestamp: Option<(i64, u64)>, + /// A bool indicating if the stream is currently undergoing a compaction routine. + is_compacting: bool, } impl StreamCtl { /// Create a new instance. pub async fn new( - config: Arc, db: Database, shutdown_tx: broadcast::Sender<()>, requests: mpsc::Receiver, + config: Arc, db: Database, shutdown_tx: broadcast::Sender<()>, requests_tx: mpsc::Sender, + requests_rx: mpsc::Receiver, ) -> Result<(Self, watch::Receiver)> { // Recover stream state. let partition = config.partition; @@ -143,15 +148,16 @@ impl StreamCtl { _db: db, tree, partition, - requests: ReceiverStream::new(requests), + requests_tx, + requests_rx: ReceiverStream::new(requests_rx), subs_tx, offset_signal, shutdown_rx: BroadcastStream::new(shutdown_tx.subscribe()), - _shutdown_tx: shutdown_tx, - descheduled: false, + shutdown_tx, sub_ctl, current_offset: recovery_data.last_written_offset, earliest_timestamp: recovery_data.first_timestamp_opt, + is_compacting: false, }, offset_signal_rx, )) @@ -164,12 +170,15 @@ impl StreamCtl { async fn run(mut self) -> Result<()> { tracing::debug!("stream controller {}/{} has started", self.config.stream, self.partition); + let compaction_timer = tokio::time::sleep(COMPACTION_INTERVAL); + tokio::pin!(compaction_timer); loop { - if self.descheduled { - break; - } tokio::select! { - msg_opt = self.requests.next() => self.handle_ctl_msg(msg_opt).await, + msg_opt = self.requests_rx.next() => self.handle_ctl_msg(msg_opt).await, + _ = &mut compaction_timer => { + compaction_timer.set(tokio::time::sleep(COMPACTION_INTERVAL)); + self.spawn_compaction_task(); + } _ = self.shutdown_rx.next() => break, } } @@ -188,14 +197,14 @@ impl StreamCtl { let msg = match msg_opt { Some(msg) => msg, None => { - let _ = self.subs_tx.send(StreamSubCtlMsg::Shutdown).await; - self.descheduled = true; + let _res = self.shutdown_tx.send(()); return; } }; match msg { StreamCtlMsg::RequestPublish { tx, request } => self.handle_publisher_request(tx, request).await, StreamCtlMsg::RequestSubscribe { tx, rx, setup } => self.handle_request_subscribe(tx, rx, setup).await, + StreamCtlMsg::CompactionFinished { earliest_timestamp } => self.handle_compaction_finished(earliest_timestamp).await, } } @@ -205,6 +214,126 @@ impl StreamCtl { ) { let _ = self.subs_tx.send(StreamSubCtlMsg::Request { tx, rx, setup }).await; } + + /// Begin a compaction routine, if possible. + #[tracing::instrument(level = "trace", skip(self))] + fn spawn_compaction_task(&mut self) { + // FUTURE: instrument the compaction routine so that we can generate metrics over durations. + if self.is_compacting { + return; + } + self.is_compacting = true; + let (config, tree, ts, stream_tx, shutdown_tx) = ( + self.config.clone(), + self.tree.clone(), + self.earliest_timestamp, + self.requests_tx.clone(), + self.shutdown_tx.clone(), + ); + let _handle = tokio::spawn(async move { + match compact_stream(config, tree, ts).await { + Ok(earliest_timestamp) => { + let _res = stream_tx + .send(StreamCtlMsg::CompactionFinished { earliest_timestamp }) + .await; + } + Err(err) => { + tracing::error!(error = ?err, "error during compaction routine, shutting down"); + let _res = shutdown_tx.send(()); + } + } + }); + } + + /// Handle a compaction finalization. + #[tracing::instrument(level = "trace", skip(self))] + async fn handle_compaction_finished(&mut self, earliest_timestamp: Option<(i64, u64)>) { + self.is_compacting = false; + self.earliest_timestamp = earliest_timestamp; + } +} + +/// Execute a compaction routine on the given DB tree. +/// +/// If compaction is not configured for the stream, then this routine will immediately finish and +/// will return the given `earliest_timestamp` as its output. +/// +/// This routine currently supports time based compaction, and its algorithm is as follows: +/// - Extract the configured TTL of the policy. +/// - Calculate an invalidation threshold based on a delta of now and the configured TTL. +/// - Search the secondary time index for all records which fall behind the threshold. +/// - Delete any records which fall behind the threshold. +/// - Return the next earliest timestamp record as output. +/// +/// **NOTE: any error returned from this routine will cause a shutdown to be issued.** +#[tracing::instrument(level = "trace", skip(config, tree, earliest_timestamp))] +async fn compact_stream(config: Arc, tree: Tree, earliest_timestamp: Option<(i64, u64)>) -> Result> { + tracing::debug!("compaction routine is starting"); + + // Extract the configured retention policy. + let ttl = match &config.retention_policy.strategy { + StreamRetentionPolicy::Retain => return Ok(earliest_timestamp), + StreamRetentionPolicy::Time => { + let ttl = config + .retention_policy + .retention_seconds + .unwrap_or_else(StreamRetentionSpec::retention_seconds_default); + chrono::Duration::seconds(i64::try_from(ttl).unwrap_or(i64::MAX)) + } + }; + let threshold = (chrono::Utc::now() - ttl).timestamp_millis(); + + // Search for all timestamps falling behind the calculated threshold. + let earliest_timestamp_opt = Database::spawn_blocking(move || -> Result> { + // Scan over timestamp index entries, adding to batch for deletion. + let mut batch = sled::Batch::default(); + let ts_start: &[u8] = PREFIX_STREAM_TS; + let ts_threshold: &[u8] = &utils::encode_byte_prefix_i64(PREFIX_STREAM_TS, threshold); + let mut last_ts_offset_opt = None; // Last event u64 offset targeted based on timestamp index. + for kv_res in tree.range(ts_start..=ts_threshold) { + let (key, val) = kv_res.context(ERR_ITER_FAILURE)?; + batch.remove(key); + last_ts_offset_opt = Some(utils::decode_u64(&val).context("error decoding timestamp index offset, data corrupted")?); + } + let last_ts_offset = match last_ts_offset_opt { + Some(last_ts_offset) => last_ts_offset, + None => return Ok(None), + }; + + // Scan over stream event entries, adding to batch for deletion, stopping at the offset + // of the last timestamp index offset of the compaction threshold. + let event_start: &[u8] = PREFIX_STREAM_EVENT; + let event_stop: &[u8] = &utils::encode_byte_prefix(PREFIX_STREAM_EVENT, last_ts_offset); + for key_res in tree.range(event_start..=event_stop).keys() { + let key = key_res.context(ERR_ITER_FAILURE)?; + batch.remove(key); + } + + // Find the next timestamp index which takes the place of the earliest timestamp. + let ts_stop: &[u8] = &utils::encode_byte_prefix_i64(PREFIX_STREAM_TS, i64::MAX); + let next_earliest_timestamp = tree + .range(ts_threshold..=ts_stop) + .next() + .transpose() + .context(ERR_ITER_FAILURE)? + .map(|(key, val)| -> Result<(i64, u64)> { + let ts = utils::decode_i64(&key[1..]).context("error decoding timestamp index key, data corrupted")?; + let offset = utils::decode_u64(&val).context("error decoding timestamp index value, data corrupted")?; + Ok((ts, offset)) + }) + .transpose()?; + + // Apply the batch. + tracing::debug!("compacting timestamp and event records up through offset {}", last_ts_offset); + tree.apply_batch(batch) + .context("error applying compaction batch to stream tree")?; + tree.flush().context(ERR_DB_FLUSH)?; + Ok(next_earliest_timestamp) + }) + .await + .context("error from compaction routine") + .and_then(|res| res)?; + Ok(earliest_timestamp_opt) } /// Recover this stream's last recorded state. @@ -292,4 +421,9 @@ pub enum StreamCtlMsg { rx: Streaming, setup: StreamSubscribeSetup, }, + /// A compaction routine has finished. + CompactionFinished { + /// The new earliest timestamp of data on disk. + earliest_timestamp: Option<(i64, u64)>, + }, } diff --git a/hadron-stream/src/stream/mod_test.rs b/hadron-stream/src/stream/mod_test.rs index 0119a1b..af0f06d 100644 --- a/hadron-stream/src/stream/mod_test.rs +++ b/hadron-stream/src/stream/mod_test.rs @@ -1,3 +1,5 @@ +use std::sync::Arc; + use anyhow::{Context, Result}; use rand::prelude::*; @@ -5,8 +7,9 @@ use crate::config::Config; use crate::database::Database; use crate::grpc::Event; use crate::models::stream::Subscription; -use crate::stream::{KEY_STREAM_LAST_WRITTEN_OFFSET, PREFIX_STREAM_EVENT, PREFIX_STREAM_SUBS, PREFIX_STREAM_SUB_OFFSETS}; +use crate::stream::{KEY_STREAM_LAST_WRITTEN_OFFSET, PREFIX_STREAM_EVENT, PREFIX_STREAM_SUBS, PREFIX_STREAM_SUB_OFFSETS, PREFIX_STREAM_TS}; use crate::utils; +use hadron_core::crd::StreamRetentionPolicy; #[tokio::test] async fn recover_stream_state_empty_state() -> Result<()> { @@ -36,7 +39,7 @@ async fn recover_stream_state_with_previous_state() -> Result<()> { let db = Database::new(config.clone()).await?; let stream_tree = db.get_stream_tree().await?; - let expected_offset = setup_stream_data(&stream_tree).await?; + let expected_offset = setup_stream_data(&stream_tree).await?.1; let output = super::recover_stream_state(stream_tree).await?; @@ -51,6 +54,10 @@ async fn recover_stream_state_with_previous_state() -> Result<()> { "expected subscriptions len to be 0 got {}", output.subscriptions.len() ); + assert!( + output.first_timestamp_opt.is_some(), + "expected first_timestamp_opt to be populated, got None" + ); Ok(()) } @@ -61,7 +68,7 @@ async fn recover_stream_state_with_previous_state_and_subs() -> Result<()> { let db = Database::new(config.clone()).await?; let stream_tree = db.get_stream_tree().await?; - let expected_offset = setup_stream_data(&stream_tree).await?; + let expected_offset = setup_stream_data(&stream_tree).await?.1; let expected_subs = setup_subs_data(&stream_tree).await?; let mut output = super::recover_stream_state(stream_tree).await?; @@ -82,12 +89,140 @@ async fn recover_stream_state_with_previous_state_and_subs() -> Result<()> { Ok(()) } -/// Setup some stream data in the given DB tree returning the last written offset. -async fn setup_stream_data(db: &sled::Tree) -> Result { - use rand::prelude::*; +#[tokio::test] +async fn compact_stream_noop_with_empty_tree() -> Result<()> { + let (config, _tmpdir) = Config::new_test()?; + let db = Database::new(config.clone()).await?; + let stream_tree = db.get_stream_tree().await?; + + let earliest_timestamp_opt = super::compact_stream(config, stream_tree.clone(), None) + .await + .context("unexpected error from compaction")?; + + assert!( + earliest_timestamp_opt.is_none(), + "expected compaction to return None, got {:?}", + earliest_timestamp_opt + ); + let count = stream_tree.scan_prefix(PREFIX_STREAM_EVENT).count(); + assert_eq!( + count, 0, + "mismatch number of stream events after compaction, got {} expected {}", + count, 0 + ); + + Ok(()) +} + +#[tokio::test] +async fn compact_stream_noop_retention_policy_retain() -> Result<()> { + let (mut config, _tmpdir) = Config::new_test()?; + Arc::make_mut(&mut config).retention_policy.strategy = StreamRetentionPolicy::Retain; + let db = Database::new(config.clone()).await?; + let stream_tree = db.get_stream_tree().await?; + let last_offset = setup_stream_data(&stream_tree).await?.1; + + let earliest_timestamp_opt = super::compact_stream(config, stream_tree.clone(), None) + .await + .context("unexpected error from compaction")?; + + assert!( + earliest_timestamp_opt.is_none(), + "expected compaction to return None, got {:?}", + earliest_timestamp_opt + ); + let count = stream_tree.scan_prefix(PREFIX_STREAM_EVENT).count(); + assert_eq!( + count, + last_offset as usize + 1, + "mismatch number of stream events after compaction, got {} expected {}", + count, + last_offset + 1 + ); + + Ok(()) +} + +#[tokio::test] +async fn compact_stream_deletes_all_data() -> Result<()> { + let (mut config, _tmpdir) = Config::new_test()?; + Arc::make_mut(&mut config).retention_policy.retention_seconds = Some(0); // Should ensure that all data is removed. + let db = Database::new(config.clone()).await?; + let stream_tree = db.get_stream_tree().await?; + let last_offset = setup_stream_data(&stream_tree).await?.1; + + let earliest_timestamp_opt = super::compact_stream(config, stream_tree.clone(), None) + .await + .context("unexpected error from compaction")?; + + assert!( + last_offset > 50, + "expected at least offset 50 from setup_stream_data, got {}", + last_offset + ); + assert!( + earliest_timestamp_opt.is_none(), + "expected compaction to return None, got {:?}", + earliest_timestamp_opt + ); + let count = stream_tree.scan_prefix(PREFIX_STREAM_EVENT).count(); + assert_eq!( + count, 0, + "mismatch number of stream events after compaction, got {} expected {}", + count, 0 + ); + + Ok(()) +} +#[tokio::test] +async fn compact_stream_deletes_only_old_data() -> Result<()> { + let (config, _tmpdir) = Config::new_test()?; + let db = Database::new(config.clone()).await?; + let stream_tree = db.get_stream_tree().await?; + let (last_ts, last_offset) = setup_stream_data(&stream_tree).await?; + let ts_two_weeks_ago = (chrono::Utc::now() - chrono::Duration::weeks(2)).timestamp_millis(); + let old_offset = last_offset / 2; + stream_tree + .insert( + &utils::encode_byte_prefix_i64(PREFIX_STREAM_TS, ts_two_weeks_ago), + &utils::encode_u64(old_offset), + ) + .context("error inserting fake timestamp index record for compaction test")?; + + let earliest_timestamp = super::compact_stream(config, stream_tree.clone(), None) + .await + .context("unexpected error from compaction")? + .context("expected next earliest timestamp record to be returned")?; + + assert_eq!( + earliest_timestamp.0, last_ts, + "expected earliest timestamp to be {}, got {}", + earliest_timestamp.0, last_ts + ); + assert_eq!( + earliest_timestamp.1, last_offset, + "expected earliest timestamp offset to be {}, got {}", + earliest_timestamp.1, last_offset + ); + let event_count = stream_tree.scan_prefix(PREFIX_STREAM_EVENT).count(); + let expected_count = (old_offset..last_offset).count(); // `..` because old_offset is deleted. + assert_eq!( + event_count, expected_count, + "mismatch number of stream events after compaction, got {} expected {}", + event_count, expected_count + ); + let ts_count = stream_tree.scan_prefix(PREFIX_STREAM_TS).count(); + assert_eq!(ts_count, 1, "expected 1 timestamp index entry, got {}", ts_count); + + Ok(()) +} + +/// Setup some stream data in the given DB tree returning the last written offset. +async fn setup_stream_data(db: &sled::Tree) -> Result<(i64, u64)> { let mut batch = sled::Batch::default(); let mut last_offset = 0; + let ts = chrono::Utc::now().timestamp_millis(); for offset in 0..rand::thread_rng().gen_range(50..100) { let event = Event::new_test(offset, "test", "empty"); let event_bytes = utils::encode_model(&event)?; @@ -95,9 +230,10 @@ async fn setup_stream_data(db: &sled::Tree) -> Result { last_offset = offset; } batch.insert(KEY_STREAM_LAST_WRITTEN_OFFSET, &utils::encode_u64(last_offset)); + batch.insert(&utils::encode_byte_prefix_i64(PREFIX_STREAM_TS, ts), &utils::encode_u64(last_offset)); db.apply_batch(batch) .context("error applying batch to write test data to stream")?; - Ok(last_offset) + Ok((ts, last_offset)) } /// Setup some subscriptions data in the given DB tree returning the set of created subs. diff --git a/hadron-stream/src/stream/subscriber.rs b/hadron-stream/src/stream/subscriber.rs index 7544238..c57535e 100644 --- a/hadron-stream/src/stream/subscriber.rs +++ b/hadron-stream/src/stream/subscriber.rs @@ -64,8 +64,6 @@ pub struct StreamSubCtl { shutdown_tx: broadcast::Sender<()>, /// A channel used for triggering graceful shutdown. shutdown_rx: BroadcastStream<()>, - /// A bool indicating that this controller has been descheduled and needs to shutdown. - descheduled: bool, /// A general purpose reusable bytes buffer, safe for concurrent use. buf: BytesMut, @@ -91,7 +89,6 @@ impl StreamSubCtl { liveness_checks: StreamMap::new(), shutdown_rx: BroadcastStream::new(shutdown_tx.subscribe()), shutdown_tx, - descheduled: false, buf: BytesMut::with_capacity(5000), } } @@ -104,9 +101,6 @@ impl StreamSubCtl { tracing::debug!("stream subscriber controller {}/{} has started", self.config.stream, self.partition,); loop { - if self.descheduled { - break; - } tokio::select! { Some(msg) = self.events_rx.next() => self.handle_msg(msg).await, Some(offset) = self.stream_offset.next() => self.handle_offset_update(offset).await, @@ -127,9 +121,6 @@ impl StreamSubCtl { StreamSubCtlMsg::Request { tx, rx, setup } => self.handle_request(tx, rx, setup).await, StreamSubCtlMsg::FetchStreamRecords(res) => self.handle_fetch_stream_records_result(res).await, StreamSubCtlMsg::DeliveryResponse(res) => self.handle_delivery_response(res).await, - StreamSubCtlMsg::Shutdown => { - self.descheduled = true; - } } } @@ -580,8 +571,6 @@ pub enum StreamSubCtlMsg { FetchStreamRecords(ShutdownResult), /// A response from a subscriber following a delivery of data for processing. DeliveryResponse(DeliveryResponse), - /// The parent controller is shutting down, so this controller needs to do the same. - Shutdown, } pub struct FetchStreamRecords { diff --git a/hadron-stream/src/watchers/pipelines.rs b/hadron-stream/src/watchers/pipelines.rs index 358d878..6721040 100644 --- a/hadron-stream/src/watchers/pipelines.rs +++ b/hadron-stream/src/watchers/pipelines.rs @@ -6,7 +6,7 @@ use arc_swap::ArcSwap; use futures::stream::StreamExt; use kube::api::{Api, ListParams}; use kube::client::Client; -use kube_runtime::watcher::{watcher, Error as WatcherError, Event}; +use kube::runtime::watcher::{watcher, Error as WatcherError, Event}; use tokio::sync::{broadcast, mpsc, watch}; use tokio::task::JoinHandle; use tokio_stream::wrappers::BroadcastStream; @@ -97,6 +97,10 @@ impl PipelineWatcher { match event { Event::Applied(pipeline) => self.handle_pipeline_applied(pipeline).await, Event::Deleted(pipeline) => { + // Only process Pipelines for this Stream. + if pipeline.spec.source_stream != self.config.stream { + return; + } let pipeline = Arc::new(pipeline); let name = match &pipeline.metadata.name { Some(name) => name, diff --git a/hadron-stream/src/watchers/stream.rs b/hadron-stream/src/watchers/stream.rs index f42c8ad..e01a8a1 100644 --- a/hadron-stream/src/watchers/stream.rs +++ b/hadron-stream/src/watchers/stream.rs @@ -4,7 +4,7 @@ use anyhow::Result; use futures::stream::StreamExt; use kube::api::{Api, ListParams}; use kube::client::Client; -use kube_runtime::watcher::{watcher, Error as WatcherError, Event}; +use kube::runtime::watcher::{watcher, Error as WatcherError, Event}; use tokio::sync::{broadcast, watch}; use tokio::task::JoinHandle; use tokio_stream::wrappers::BroadcastStream; diff --git a/hadron-stream/src/watchers/tokens.rs b/hadron-stream/src/watchers/tokens.rs index 94b9644..e307383 100644 --- a/hadron-stream/src/watchers/tokens.rs +++ b/hadron-stream/src/watchers/tokens.rs @@ -8,7 +8,7 @@ use jsonwebtoken::DecodingKey; use k8s_openapi::api::core::v1::Secret; use kube::api::{Api, ListParams}; use kube::client::Client; -use kube_runtime::watcher::{watcher, Error as WatcherError, Event}; +use kube::runtime::watcher::{watcher, Error as WatcherError, Event}; use tokio::sync::broadcast; use tokio::task::JoinHandle; use tokio_stream::wrappers::BroadcastStream; From 00bfb0a5aa50443416b601c97e7a5d3802584c55 Mon Sep 17 00:00:00 2001 From: Anthony Dodd Date: Mon, 8 Nov 2021 08:08:18 -0600 Subject: [PATCH 08/10] Compaction is now using disk-backed checkpoints This ensures that spurious liveness profiles do not interrupt compaction from ever being run. Replaced chrono w/ time crate. --- hadron-core/Cargo.lock | 12 +++++- hadron-core/Cargo.toml | 1 + hadron-operator/Cargo.lock | 13 ++++++- hadron-operator/Cargo.toml | 1 + hadron-stream/Cargo.lock | 14 ++++++- hadron-stream/Cargo.toml | 2 +- hadron-stream/src/main.rs | 2 + hadron-stream/src/stream/mod.rs | 45 ++++++++++++++++++++-- hadron-stream/src/stream/mod_test.rs | 32 ++++++++++++--- hadron-stream/src/stream/publisher.rs | 2 +- hadron-stream/src/stream/publisher_test.rs | 2 +- 11 files changed, 110 insertions(+), 16 deletions(-) diff --git a/hadron-core/Cargo.lock b/hadron-core/Cargo.lock index e4f1a41..be558d9 100644 --- a/hadron-core/Cargo.lock +++ b/hadron-core/Cargo.lock @@ -101,7 +101,7 @@ dependencies = [ "num-integer", "num-traits", "serde", - "time", + "time 0.1.44", "winapi", ] @@ -267,6 +267,7 @@ dependencies = [ "serde_json", "serde_yaml", "thiserror", + "time 0.3.4", "tonic", "uuid", ] @@ -949,6 +950,15 @@ dependencies = [ "winapi", ] +[[package]] +name = "time" +version = "0.3.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "99beeb0daeac2bd1e86ac2c21caddecb244b39a093594da1a661ec2060c7aedd" +dependencies = [ + "libc", +] + [[package]] name = "tinyvec" version = "1.5.0" diff --git a/hadron-core/Cargo.toml b/hadron-core/Cargo.toml index f46d846..1e39106 100644 --- a/hadron-core/Cargo.toml +++ b/hadron-core/Cargo.toml @@ -17,5 +17,6 @@ serde = { version="1", features = ["derive"] } serde_json = "1" serde_yaml = "0.8" thiserror = "1" +time = "0.3" tonic = "0.6" uuid = { version="0.8", default-features=false, features=["serde", "v4"] } diff --git a/hadron-operator/Cargo.lock b/hadron-operator/Cargo.lock index fbbd00a..f1d77d5 100644 --- a/hadron-operator/Cargo.lock +++ b/hadron-operator/Cargo.lock @@ -177,7 +177,7 @@ dependencies = [ "num-integer", "num-traits", "serde", - "time", + "time 0.1.44", "winapi", ] @@ -528,6 +528,7 @@ dependencies = [ "serde_json", "serde_yaml", "thiserror", + "time 0.3.4", "tonic", "uuid", ] @@ -552,6 +553,7 @@ dependencies = [ "prost", "ring", "serde", + "time 0.3.4", "tokio", "tokio-rustls", "tokio-stream", @@ -1679,6 +1681,15 @@ dependencies = [ "winapi", ] +[[package]] +name = "time" +version = "0.3.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "99beeb0daeac2bd1e86ac2c21caddecb244b39a093594da1a661ec2060c7aedd" +dependencies = [ + "libc", +] + [[package]] name = "tinyvec" version = "1.5.0" diff --git a/hadron-operator/Cargo.toml b/hadron-operator/Cargo.toml index 5d15db8..0fa3020 100644 --- a/hadron-operator/Cargo.toml +++ b/hadron-operator/Cargo.toml @@ -26,6 +26,7 @@ maplit = "1" prost = "0.9" ring = "0.16" serde = { version="1", features = ["derive"] } +time = "0.3.4" tokio = { version = "1", default-features = false, features = [ # See https://docs.rs/tokio/latest/tokio/#feature-flags "rt-multi-thread", "io-util", "net", "time", "macros", "sync", "signal", "fs", diff --git a/hadron-stream/Cargo.lock b/hadron-stream/Cargo.lock index e6af7ec..8546bdc 100644 --- a/hadron-stream/Cargo.lock +++ b/hadron-stream/Cargo.lock @@ -156,7 +156,7 @@ dependencies = [ "num-integer", "num-traits", "serde", - "time", + "time 0.1.44", "winapi", ] @@ -549,6 +549,7 @@ dependencies = [ "serde_json", "serde_yaml", "thiserror", + "time 0.3.4", "tonic", "uuid", ] @@ -561,7 +562,6 @@ dependencies = [ "arc-swap", "base64 0.13.0", "bytes", - "chrono", "envy", "futures", "hadron-core", @@ -575,6 +575,7 @@ dependencies = [ "sled", "tempfile", "thiserror", + "time 0.3.4", "tokio", "tokio-stream", "tonic", @@ -1751,6 +1752,15 @@ dependencies = [ "winapi", ] +[[package]] +name = "time" +version = "0.3.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "99beeb0daeac2bd1e86ac2c21caddecb244b39a093594da1a661ec2060c7aedd" +dependencies = [ + "libc", +] + [[package]] name = "tinyvec" version = "1.5.0" diff --git a/hadron-stream/Cargo.toml b/hadron-stream/Cargo.toml index 278569c..7fa355d 100644 --- a/hadron-stream/Cargo.toml +++ b/hadron-stream/Cargo.toml @@ -14,7 +14,6 @@ anyhow = "1" arc-swap = "1" base64 = "0.13" bytes = "1" -chrono = "0.4" envy = "0.4" futures = "0.3" hadron-core = { path = "../hadron-core" } @@ -27,6 +26,7 @@ rand = "0.8" serde = { version="1", features = ["derive"] } sled = "0.34" thiserror = "1" +time = "0.3.4" tokio = { version = "1", default-features = false, features = [ # See https://docs.rs/tokio/latest/tokio/#feature-flags "rt-multi-thread", "io-util", "net", "time", "macros", "sync", "signal", "fs", diff --git a/hadron-stream/src/main.rs b/hadron-stream/src/main.rs index cfbb611..158051e 100644 --- a/hadron-stream/src/main.rs +++ b/hadron-stream/src/main.rs @@ -51,6 +51,8 @@ async fn main() -> Result<()> { pod_name = %cfg.pod_name, partition = %cfg.partition, storage_data_path = %cfg.storage_data_path, + retention_policy_strategy = ?cfg.retention_policy.strategy, + retention_policy_seconds = ?cfg.retention_policy.retention_seconds, "starting Hadron Stream controller", ); if let Err(err) = App::new(cfg).await?.spawn().await { diff --git a/hadron-stream/src/stream/mod.rs b/hadron-stream/src/stream/mod.rs index 1733895..801423a 100644 --- a/hadron-stream/src/stream/mod.rs +++ b/hadron-stream/src/stream/mod.rs @@ -55,7 +55,7 @@ use hadron_core::crd::{StreamRetentionPolicy, StreamRetentionSpec}; /// NOTE: in order to preserve lexicographical ordering of keys, it is important to always use /// the `utils::encode_byte_prefix*` methods. pub const PREFIX_STREAM_EVENT: &[u8; 1] = b"e"; -/// The key prefix used for storing stream event timestamps, always stored as i64 milliseconds. +/// The key prefix used for storing stream event timestamps, always stored as i64 seconds timestamp. /// /// NOTE: in order to preserve lexicographical ordering of keys, it is important to always use /// the `utils::encode_byte_prefix*` methods. @@ -72,6 +72,8 @@ pub const PREFIX_STREAM_SUBS: &[u8; 1] = b"s"; pub const PREFIX_STREAM_SUB_OFFSETS: &[u8; 1] = b"o"; /// The key used to store the last written offset for the stream. pub const KEY_STREAM_LAST_WRITTEN_OFFSET: &[u8; 1] = b"l"; +/// The key used to store the seconds timestamp of the last compaction event. +pub const KEY_STREAM_LAST_COMPACTION: &[u8; 1] = b"c"; const COMPACTION_INTERVAL: std::time::Duration = std::time::Duration::from_secs(60 * 30); const ERR_DECODING_STREAM_META_GROUP_NAME: &str = "error decoding stream meta group name from storage"; @@ -112,6 +114,8 @@ pub struct StreamCtl { earliest_timestamp: Option<(i64, u64)>, /// A bool indicating if the stream is currently undergoing a compaction routine. is_compacting: bool, + /// The timestamp of the last compaction event. + last_compaction_event: Option, } impl StreamCtl { @@ -158,6 +162,7 @@ impl StreamCtl { current_offset: recovery_data.last_written_offset, earliest_timestamp: recovery_data.first_timestamp_opt, is_compacting: false, + last_compaction_event: recovery_data.last_compaction_opt, }, offset_signal_rx, )) @@ -170,8 +175,11 @@ impl StreamCtl { async fn run(mut self) -> Result<()> { tracing::debug!("stream controller {}/{} has started", self.config.stream, self.partition); - let compaction_timer = tokio::time::sleep(COMPACTION_INTERVAL); + // Calculate the delay to be used for the initial compaction event. + let delay = calculate_initial_compaction_delay(self.last_compaction_event); + let compaction_timer = tokio::time::sleep(delay); tokio::pin!(compaction_timer); + loop { tokio::select! { msg_opt = self.requests_rx.next() => self.handle_ctl_msg(msg_opt).await, @@ -253,6 +261,21 @@ impl StreamCtl { } } +/// Calculate the initial compaction delay based on the given last compaction timestamp. +fn calculate_initial_compaction_delay(last_compaction_timestamp: Option) -> std::time::Duration { + let now = time::OffsetDateTime::now_utc().unix_timestamp(); + match last_compaction_timestamp { + Some(last_timestamp) => { + let delay_seconds = time::Duration::seconds(now.saturating_sub(last_timestamp)); + std::time::Duration::try_from(delay_seconds).unwrap_or_else(|err| { + tracing::error!(error = ?err, "error converting last compaction timestamp into std duration"); + COMPACTION_INTERVAL + }) + } + None => COMPACTION_INTERVAL, + } +} + /// Execute a compaction routine on the given DB tree. /// /// If compaction is not configured for the stream, then this routine will immediately finish and @@ -278,10 +301,11 @@ async fn compact_stream(config: Arc, tree: Tree, earliest_timestamp: Opt .retention_policy .retention_seconds .unwrap_or_else(StreamRetentionSpec::retention_seconds_default); - chrono::Duration::seconds(i64::try_from(ttl).unwrap_or(i64::MAX)) + time::Duration::seconds(i64::try_from(ttl).unwrap_or(i64::MAX)) } }; - let threshold = (chrono::Utc::now() - ttl).timestamp_millis(); + let now = time::OffsetDateTime::now_utc(); + let threshold = (now - ttl).unix_timestamp(); // Search for all timestamps falling behind the calculated threshold. let earliest_timestamp_opt = Database::spawn_blocking(move || -> Result> { @@ -323,6 +347,9 @@ async fn compact_stream(config: Arc, tree: Tree, earliest_timestamp: Opt }) .transpose()?; + // Record the timestamp of this compaction event. + batch.insert(KEY_STREAM_LAST_COMPACTION, &utils::encode_i64(now.unix_timestamp())); + // Apply the batch. tracing::debug!("compacting timestamp and event records up through offset {}", last_ts_offset); tree.apply_batch(batch) @@ -361,6 +388,13 @@ async fn recover_stream_state(tree: Tree) -> Result { }) .transpose()?; + // Fetch timestamp of last compaction event. + let last_compaction_opt = tree + .get(KEY_STREAM_LAST_COMPACTION) + .context("error fetching last compaction key during recovery")? + .map(|val| utils::decode_i64(&val).context("error decoding last compaction event timestamp, data corrupted")) + .transpose()?; + // Fetch all stream subscriber info. let mut subs = HashMap::new(); for entry_res in tree.scan_prefix(PREFIX_STREAM_SUBS) { @@ -392,6 +426,7 @@ async fn recover_stream_state(tree: Tree) -> Result { last_written_offset, subscriptions, first_timestamp_opt, + last_compaction_opt, }) }) .await??; @@ -406,6 +441,8 @@ struct StreamRecoveryState { subscriptions: Vec<(Subscription, u64)>, /// The first timestamp record found, if any. first_timestamp_opt: Option<(i64, u64)>, + /// The timestamp of the last compaction event. + last_compaction_opt: Option, } /// A message bound for a stream controller. diff --git a/hadron-stream/src/stream/mod_test.rs b/hadron-stream/src/stream/mod_test.rs index af0f06d..2bf5529 100644 --- a/hadron-stream/src/stream/mod_test.rs +++ b/hadron-stream/src/stream/mod_test.rs @@ -149,15 +149,25 @@ async fn compact_stream_deletes_all_data() -> Result<()> { Arc::make_mut(&mut config).retention_policy.retention_seconds = Some(0); // Should ensure that all data is removed. let db = Database::new(config.clone()).await?; let stream_tree = db.get_stream_tree().await?; - let last_offset = setup_stream_data(&stream_tree).await?.1; + let (last_ts, last_offset) = setup_stream_data(&stream_tree).await?; + let ts_two_weeks_ago = (time::OffsetDateTime::now_utc() - time::Duration::weeks(2)).unix_timestamp(); + stream_tree + .insert( + &utils::encode_byte_prefix_i64(PREFIX_STREAM_TS, ts_two_weeks_ago), + &utils::encode_u64(last_offset), + ) + .context("error inserting older timestamp index record for compaction test")?; + stream_tree + .remove(&utils::encode_byte_prefix_i64(PREFIX_STREAM_TS, last_ts)) + .context("error removing original timestamp record for compaction test setup")?; let earliest_timestamp_opt = super::compact_stream(config, stream_tree.clone(), None) .await .context("unexpected error from compaction")?; assert!( - last_offset > 50, - "expected at least offset 50 from setup_stream_data, got {}", + last_offset >= 49, + "expected at least offset 49 from setup_stream_data, got {}", last_offset ); assert!( @@ -181,7 +191,7 @@ async fn compact_stream_deletes_only_old_data() -> Result<()> { let db = Database::new(config.clone()).await?; let stream_tree = db.get_stream_tree().await?; let (last_ts, last_offset) = setup_stream_data(&stream_tree).await?; - let ts_two_weeks_ago = (chrono::Utc::now() - chrono::Duration::weeks(2)).timestamp_millis(); + let ts_two_weeks_ago = (time::OffsetDateTime::now_utc() - time::Duration::weeks(2)).unix_timestamp(); let old_offset = last_offset / 2; stream_tree .insert( @@ -218,11 +228,23 @@ async fn compact_stream_deletes_only_old_data() -> Result<()> { Ok(()) } +#[test] +fn calculate_initial_compaction_delay_returns_delta_under_30_min() { + let expected_seconds = time::OffsetDateTime::now_utc().unix_timestamp() - (60 * 25); + let expected_output = time::Duration::seconds(expected_seconds); + let output = super::calculate_initial_compaction_delay(Some(60 * 25)); + assert_eq!( + output, expected_output, + "unexpected duration returned, expected {:?}, got {:?}", + expected_output, output, + ); +} + /// Setup some stream data in the given DB tree returning the last written offset. async fn setup_stream_data(db: &sled::Tree) -> Result<(i64, u64)> { let mut batch = sled::Batch::default(); let mut last_offset = 0; - let ts = chrono::Utc::now().timestamp_millis(); + let ts = time::OffsetDateTime::now_utc().unix_timestamp(); for offset in 0..rand::thread_rng().gen_range(50..100) { let event = Event::new_test(offset, "test", "empty"); let event_bytes = utils::encode_model(&event)?; diff --git a/hadron-stream/src/stream/publisher.rs b/hadron-stream/src/stream/publisher.rs index 270543e..d159201 100644 --- a/hadron-stream/src/stream/publisher.rs +++ b/hadron-stream/src/stream/publisher.rs @@ -53,7 +53,7 @@ impl StreamCtl { // Assign an offset to each event in the batch, and record a timestamp in a secondary // index for the last offset in the batch. - let ts = chrono::Utc::now().timestamp_millis(); + let ts = time::OffsetDateTime::now_utc().unix_timestamp(); let mut batch = sled::Batch::default(); for new_event in req.batch { *current_offset += 1; diff --git a/hadron-stream/src/stream/publisher_test.rs b/hadron-stream/src/stream/publisher_test.rs index 422583f..f984da5 100644 --- a/hadron-stream/src/stream/publisher_test.rs +++ b/hadron-stream/src/stream/publisher_test.rs @@ -51,7 +51,7 @@ async fn publish_data_frame() -> Result<()> { let stream_tree = db.get_stream_tree().await?; let (mut current_offset, mut earliest_timestamp) = (0u64, None); let (tx, rx) = watch::channel(current_offset); - let expected_ts_min = chrono::Utc::now().timestamp_millis() - 5; + let expected_ts_min = time::OffsetDateTime::now_utc().unix_timestamp() - 5; let mut req = StreamPublishRequest { batch: vec![], fsync: true, ack: 0 }; let (mut expected_events, expected_offset) = From 75ea29867f962ff403f9e76a85f1def322f654e5 Mon Sep 17 00:00:00 2001 From: Anthony Dodd Date: Mon, 8 Nov 2021 08:58:35 -0600 Subject: [PATCH 09/10] Add tests for stream subscription functionality Added tests for stream::subscriber::spawn_group_fetch. Added tests for stream::subscriber::try_record_delivery_response. Added tests for stream::subscriber::ensure_subscriber_record. --- hadron-cli/Cargo.lock | 4 +- hadron-operator/Cargo.lock | 8 +- hadron-stream/Cargo.lock | 9 +- hadron-stream/Cargo.toml | 1 - hadron-stream/src/fixtures.rs | 54 +++ hadron-stream/src/grpc/stream.rs | 24 +- hadron-stream/src/main.rs | 2 + hadron-stream/src/stream/mod.rs | 33 +- hadron-stream/src/stream/mod_test.rs | 67 +--- hadron-stream/src/stream/subscriber.rs | 354 ++++++++------------ hadron-stream/src/stream/subscriber_test.rs | 286 ++++++++++++++++ rustfmt.toml | 24 +- 12 files changed, 527 insertions(+), 339 deletions(-) create mode 100644 hadron-stream/src/fixtures.rs create mode 100644 hadron-stream/src/stream/subscriber_test.rs diff --git a/hadron-cli/Cargo.lock b/hadron-cli/Cargo.lock index 4f5b2e3..666ddd2 100644 --- a/hadron-cli/Cargo.lock +++ b/hadron-cli/Cargo.lock @@ -454,9 +454,9 @@ checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" [[package]] name = "libc" -version = "0.2.106" +version = "0.2.107" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a60553f9a9e039a333b4e9b20573b9e9b9c0bb3a11e201ccc48ef4283456d673" +checksum = "fbe5e23404da5b4f555ef85ebed98fb4083e55a00c317800bc2a50ede9f3d219" [[package]] name = "log" diff --git a/hadron-operator/Cargo.lock b/hadron-operator/Cargo.lock index f1d77d5..36651cf 100644 --- a/hadron-operator/Cargo.lock +++ b/hadron-operator/Cargo.lock @@ -890,9 +890,9 @@ checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" [[package]] name = "libc" -version = "0.2.106" +version = "0.2.107" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a60553f9a9e039a333b4e9b20573b9e9b9c0bb3a11e201ccc48ef4283456d673" +checksum = "fbe5e23404da5b4f555ef85ebed98fb4083e55a00c317800bc2a50ede9f3d219" [[package]] name = "linked-hash-map" @@ -1477,9 +1477,9 @@ dependencies = [ [[package]] name = "serde_json" -version = "1.0.68" +version = "1.0.69" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0f690853975602e1bfe1ccbf50504d67174e3bcf340f23b5ea9992e0587a52d8" +checksum = "e466864e431129c7e0d3476b92f20458e5879919a0596c6472738d9fa2d342f8" dependencies = [ "indexmap", "itoa", diff --git a/hadron-stream/Cargo.lock b/hadron-stream/Cargo.lock index 8546bdc..ef0a0c1 100644 --- a/hadron-stream/Cargo.lock +++ b/hadron-stream/Cargo.lock @@ -561,7 +561,6 @@ dependencies = [ "anyhow", "arc-swap", "base64 0.13.0", - "bytes", "envy", "futures", "hadron-core", @@ -920,9 +919,9 @@ checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" [[package]] name = "libc" -version = "0.2.106" +version = "0.2.107" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a60553f9a9e039a333b4e9b20573b9e9b9c0bb3a11e201ccc48ef4283456d673" +checksum = "fbe5e23404da5b4f555ef85ebed98fb4083e55a00c317800bc2a50ede9f3d219" [[package]] name = "linked-hash-map" @@ -1550,9 +1549,9 @@ dependencies = [ [[package]] name = "serde_json" -version = "1.0.68" +version = "1.0.69" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0f690853975602e1bfe1ccbf50504d67174e3bcf340f23b5ea9992e0587a52d8" +checksum = "e466864e431129c7e0d3476b92f20458e5879919a0596c6472738d9fa2d342f8" dependencies = [ "indexmap", "itoa", diff --git a/hadron-stream/Cargo.toml b/hadron-stream/Cargo.toml index 7fa355d..e6bf042 100644 --- a/hadron-stream/Cargo.toml +++ b/hadron-stream/Cargo.toml @@ -13,7 +13,6 @@ codegen-units = 1 anyhow = "1" arc-swap = "1" base64 = "0.13" -bytes = "1" envy = "0.4" futures = "0.3" hadron-core = { path = "../hadron-core" } diff --git a/hadron-stream/src/fixtures.rs b/hadron-stream/src/fixtures.rs new file mode 100644 index 0000000..4ffb672 --- /dev/null +++ b/hadron-stream/src/fixtures.rs @@ -0,0 +1,54 @@ +use anyhow::{Context, Result}; +use rand::prelude::*; + +use crate::grpc::Event; +use crate::models::stream::Subscription; +use crate::stream::{KEY_STREAM_LAST_WRITTEN_OFFSET, PREFIX_STREAM_EVENT, PREFIX_STREAM_SUBS, PREFIX_STREAM_SUB_OFFSETS, PREFIX_STREAM_TS}; +use crate::utils; + +/// Setup some stream data in the given DB tree returning the last written offset. +pub async fn setup_stream_data(db: &sled::Tree) -> Result<(i64, u64)> { + let mut batch = sled::Batch::default(); + let mut last_offset = 0; + let ts = time::OffsetDateTime::now_utc().unix_timestamp(); + for offset in 0..rand::thread_rng().gen_range(50..100) { + let event = Event::new_test(offset, "test", "empty"); + let event_bytes = utils::encode_model(&event)?; + batch.insert(&utils::encode_byte_prefix(PREFIX_STREAM_EVENT, offset), event_bytes.as_slice()); + last_offset = offset; + } + batch.insert(KEY_STREAM_LAST_WRITTEN_OFFSET, &utils::encode_u64(last_offset)); + batch.insert(&utils::encode_byte_prefix_i64(PREFIX_STREAM_TS, ts), &utils::encode_u64(last_offset)); + db.apply_batch(batch) + .context("error applying batch to write test data to stream")?; + Ok((ts, last_offset)) +} + +/// Setup some subscriptions data in the given DB tree returning the set of created subs. +pub async fn setup_subs_data(db: &sled::Tree) -> Result> { + let mut batch = sled::Batch::default(); + let mut subs = vec![]; + for offset in 0..rand::thread_rng().gen_range(50..100) { + let sub = Subscription { group_name: offset.to_string(), max_batch_size: 50 }; + let sub_encoded = utils::encode_model(&sub)?; + let sub_model_key = utils::ivec_from_iter( + PREFIX_STREAM_SUBS + .iter() + .copied() + .chain(sub.group_name.as_bytes().iter().copied()), + ); + let sub_offset_key = utils::ivec_from_iter( + PREFIX_STREAM_SUB_OFFSETS + .iter() + .copied() + .chain(sub.group_name.as_bytes().iter().copied()), + ); + batch.insert(sub_model_key, sub_encoded.as_slice()); + batch.insert(sub_offset_key, &utils::encode_u64(offset)); + subs.push((sub, offset)); + } + db.apply_batch(batch) + .context("error applying batch to write test data to stream")?; + subs.sort_by(|a, b| a.1.cmp(&b.1)); + Ok(subs) +} diff --git a/hadron-stream/src/grpc/stream.rs b/hadron-stream/src/grpc/stream.rs index 5642db6..c2a42cd 100644 --- a/hadron-stream/src/grpc/stream.rs +++ b/hadron-stream/src/grpc/stream.rs @@ -232,21 +232,15 @@ pub mod stream_controller_server { #[doc = " Open a metadata stream."] async fn metadata(&self, request: tonic::Request) -> Result, tonic::Status>; #[doc = " Open a stream publisher channel."] - async fn stream_publish( - &self, request: tonic::Request, - ) -> Result, tonic::Status>; + async fn stream_publish(&self, request: tonic::Request) -> Result, tonic::Status>; #[doc = "Server streaming response type for the StreamSubscribe method."] type StreamSubscribeStream: futures_core::Stream> + Send + 'static; #[doc = " Open a stream subscriber channel."] - async fn stream_subscribe( - &self, request: tonic::Request>, - ) -> Result, tonic::Status>; + async fn stream_subscribe(&self, request: tonic::Request>) -> Result, tonic::Status>; #[doc = "Server streaming response type for the PipelineSubscribe method."] type PipelineSubscribeStream: futures_core::Stream> + Send + 'static; #[doc = " Open a pipeline subscriber channel."] - async fn pipeline_subscribe( - &self, request: tonic::Request>, - ) -> Result, tonic::Status>; + async fn pipeline_subscribe(&self, request: tonic::Request>) -> Result, tonic::Status>; } #[doc = " The Hadron stream controller interface."] #[derive(Debug)] @@ -308,8 +302,7 @@ pub mod stream_controller_server { let inner = inner.0; let method = MetadataSvc(inner); let codec = tonic::codec::ProstCodec::default(); - let mut grpc = - tonic::server::Grpc::new(codec).apply_compression_config(accept_compression_encodings, send_compression_encodings); + let mut grpc = tonic::server::Grpc::new(codec).apply_compression_config(accept_compression_encodings, send_compression_encodings); let res = grpc.server_streaming(method, req).await; Ok(res) }; @@ -334,8 +327,7 @@ pub mod stream_controller_server { let inner = inner.0; let method = StreamPublishSvc(inner); let codec = tonic::codec::ProstCodec::default(); - let mut grpc = - tonic::server::Grpc::new(codec).apply_compression_config(accept_compression_encodings, send_compression_encodings); + let mut grpc = tonic::server::Grpc::new(codec).apply_compression_config(accept_compression_encodings, send_compression_encodings); let res = grpc.unary(method, req).await; Ok(res) }; @@ -361,8 +353,7 @@ pub mod stream_controller_server { let inner = inner.0; let method = StreamSubscribeSvc(inner); let codec = tonic::codec::ProstCodec::default(); - let mut grpc = - tonic::server::Grpc::new(codec).apply_compression_config(accept_compression_encodings, send_compression_encodings); + let mut grpc = tonic::server::Grpc::new(codec).apply_compression_config(accept_compression_encodings, send_compression_encodings); let res = grpc.streaming(method, req).await; Ok(res) }; @@ -388,8 +379,7 @@ pub mod stream_controller_server { let inner = inner.0; let method = PipelineSubscribeSvc(inner); let codec = tonic::codec::ProstCodec::default(); - let mut grpc = - tonic::server::Grpc::new(codec).apply_compression_config(accept_compression_encodings, send_compression_encodings); + let mut grpc = tonic::server::Grpc::new(codec).apply_compression_config(accept_compression_encodings, send_compression_encodings); let res = grpc.streaming(method, req).await; Ok(res) }; diff --git a/hadron-stream/src/main.rs b/hadron-stream/src/main.rs index 158051e..8e96913 100644 --- a/hadron-stream/src/main.rs +++ b/hadron-stream/src/main.rs @@ -6,6 +6,8 @@ mod config; mod config_test; mod database; mod error; +#[cfg(test)] +mod fixtures; mod futures; mod grpc; mod models; diff --git a/hadron-stream/src/stream/mod.rs b/hadron-stream/src/stream/mod.rs index 801423a..e201777 100644 --- a/hadron-stream/src/stream/mod.rs +++ b/hadron-stream/src/stream/mod.rs @@ -28,6 +28,8 @@ mod publisher; #[cfg(test)] mod publisher_test; mod subscriber; +#[cfg(test)] +mod subscriber_test; use std::collections::HashMap; use std::sync::Arc; @@ -121,8 +123,7 @@ pub struct StreamCtl { impl StreamCtl { /// Create a new instance. pub async fn new( - config: Arc, db: Database, shutdown_tx: broadcast::Sender<()>, requests_tx: mpsc::Sender, - requests_rx: mpsc::Receiver, + config: Arc, db: Database, shutdown_tx: broadcast::Sender<()>, requests_tx: mpsc::Sender, requests_rx: mpsc::Receiver, ) -> Result<(Self, watch::Receiver)> { // Recover stream state. let partition = config.partition; @@ -217,9 +218,7 @@ impl StreamCtl { } /// Handle a request to setup a subscriber channel. - async fn handle_request_subscribe( - &mut self, tx: mpsc::Sender>, rx: Streaming, setup: StreamSubscribeSetup, - ) { + async fn handle_request_subscribe(&mut self, tx: mpsc::Sender>, rx: Streaming, setup: StreamSubscribeSetup) { let _ = self.subs_tx.send(StreamSubCtlMsg::Request { tx, rx, setup }).await; } @@ -231,19 +230,11 @@ impl StreamCtl { return; } self.is_compacting = true; - let (config, tree, ts, stream_tx, shutdown_tx) = ( - self.config.clone(), - self.tree.clone(), - self.earliest_timestamp, - self.requests_tx.clone(), - self.shutdown_tx.clone(), - ); + let (config, tree, ts, stream_tx, shutdown_tx) = (self.config.clone(), self.tree.clone(), self.earliest_timestamp, self.requests_tx.clone(), self.shutdown_tx.clone()); let _handle = tokio::spawn(async move { match compact_stream(config, tree, ts).await { Ok(earliest_timestamp) => { - let _res = stream_tx - .send(StreamCtlMsg::CompactionFinished { earliest_timestamp }) - .await; + let _res = stream_tx.send(StreamCtlMsg::CompactionFinished { earliest_timestamp }).await; } Err(err) => { tracing::error!(error = ?err, "error during compaction routine, shutting down"); @@ -297,10 +288,7 @@ async fn compact_stream(config: Arc, tree: Tree, earliest_timestamp: Opt let ttl = match &config.retention_policy.strategy { StreamRetentionPolicy::Retain => return Ok(earliest_timestamp), StreamRetentionPolicy::Time => { - let ttl = config - .retention_policy - .retention_seconds - .unwrap_or_else(StreamRetentionSpec::retention_seconds_default); + let ttl = config.retention_policy.retention_seconds.unwrap_or_else(StreamRetentionSpec::retention_seconds_default); time::Duration::seconds(i64::try_from(ttl).unwrap_or(i64::MAX)) } }; @@ -352,8 +340,7 @@ async fn compact_stream(config: Arc, tree: Tree, earliest_timestamp: Opt // Apply the batch. tracing::debug!("compacting timestamp and event records up through offset {}", last_ts_offset); - tree.apply_batch(batch) - .context("error applying compaction batch to stream tree")?; + tree.apply_batch(batch).context("error applying compaction batch to stream tree")?; tree.flush().context(ERR_DB_FLUSH)?; Ok(next_earliest_timestamp) }) @@ -367,9 +354,7 @@ async fn compact_stream(config: Arc, tree: Tree, earliest_timestamp: Opt async fn recover_stream_state(tree: Tree) -> Result { let val = Database::spawn_blocking(move || -> Result { // Fetch next offset info. - let offset_opt = tree - .get(KEY_STREAM_LAST_WRITTEN_OFFSET) - .context("error fetching next offset key during recovery")?; + let offset_opt = tree.get(KEY_STREAM_LAST_WRITTEN_OFFSET).context("error fetching next offset key during recovery")?; let last_written_offset = offset_opt .map(|val| utils::decode_u64(&val).context("error decoding offset value from storage")) .transpose()? diff --git a/hadron-stream/src/stream/mod_test.rs b/hadron-stream/src/stream/mod_test.rs index 2bf5529..b2a264a 100644 --- a/hadron-stream/src/stream/mod_test.rs +++ b/hadron-stream/src/stream/mod_test.rs @@ -1,13 +1,11 @@ use std::sync::Arc; use anyhow::{Context, Result}; -use rand::prelude::*; use crate::config::Config; use crate::database::Database; -use crate::grpc::Event; -use crate::models::stream::Subscription; -use crate::stream::{KEY_STREAM_LAST_WRITTEN_OFFSET, PREFIX_STREAM_EVENT, PREFIX_STREAM_SUBS, PREFIX_STREAM_SUB_OFFSETS, PREFIX_STREAM_TS}; +use crate::fixtures; +use crate::stream::{PREFIX_STREAM_EVENT, PREFIX_STREAM_TS}; use crate::utils; use hadron_core::crd::StreamRetentionPolicy; @@ -39,7 +37,7 @@ async fn recover_stream_state_with_previous_state() -> Result<()> { let db = Database::new(config.clone()).await?; let stream_tree = db.get_stream_tree().await?; - let expected_offset = setup_stream_data(&stream_tree).await?.1; + let expected_offset = fixtures::setup_stream_data(&stream_tree).await?.1; let output = super::recover_stream_state(stream_tree).await?; @@ -68,8 +66,8 @@ async fn recover_stream_state_with_previous_state_and_subs() -> Result<()> { let db = Database::new(config.clone()).await?; let stream_tree = db.get_stream_tree().await?; - let expected_offset = setup_stream_data(&stream_tree).await?.1; - let expected_subs = setup_subs_data(&stream_tree).await?; + let expected_offset = fixtures::setup_stream_data(&stream_tree).await?.1; + let expected_subs = fixtures::setup_subs_data(&stream_tree).await?; let mut output = super::recover_stream_state(stream_tree).await?; output.subscriptions.sort_by(|a, b| a.1.cmp(&b.1)); @@ -120,7 +118,7 @@ async fn compact_stream_noop_retention_policy_retain() -> Result<()> { Arc::make_mut(&mut config).retention_policy.strategy = StreamRetentionPolicy::Retain; let db = Database::new(config.clone()).await?; let stream_tree = db.get_stream_tree().await?; - let last_offset = setup_stream_data(&stream_tree).await?.1; + let last_offset = fixtures::setup_stream_data(&stream_tree).await?.1; let earliest_timestamp_opt = super::compact_stream(config, stream_tree.clone(), None) .await @@ -149,7 +147,7 @@ async fn compact_stream_deletes_all_data() -> Result<()> { Arc::make_mut(&mut config).retention_policy.retention_seconds = Some(0); // Should ensure that all data is removed. let db = Database::new(config.clone()).await?; let stream_tree = db.get_stream_tree().await?; - let (last_ts, last_offset) = setup_stream_data(&stream_tree).await?; + let (last_ts, last_offset) = fixtures::setup_stream_data(&stream_tree).await?; let ts_two_weeks_ago = (time::OffsetDateTime::now_utc() - time::Duration::weeks(2)).unix_timestamp(); stream_tree .insert( @@ -167,7 +165,7 @@ async fn compact_stream_deletes_all_data() -> Result<()> { assert!( last_offset >= 49, - "expected at least offset 49 from setup_stream_data, got {}", + "expected at least offset 49 from fixtures::setup_stream_data, got {}", last_offset ); assert!( @@ -190,7 +188,7 @@ async fn compact_stream_deletes_only_old_data() -> Result<()> { let (config, _tmpdir) = Config::new_test()?; let db = Database::new(config.clone()).await?; let stream_tree = db.get_stream_tree().await?; - let (last_ts, last_offset) = setup_stream_data(&stream_tree).await?; + let (last_ts, last_offset) = fixtures::setup_stream_data(&stream_tree).await?; let ts_two_weeks_ago = (time::OffsetDateTime::now_utc() - time::Duration::weeks(2)).unix_timestamp(); let old_offset = last_offset / 2; stream_tree @@ -239,50 +237,3 @@ fn calculate_initial_compaction_delay_returns_delta_under_30_min() { expected_output, output, ); } - -/// Setup some stream data in the given DB tree returning the last written offset. -async fn setup_stream_data(db: &sled::Tree) -> Result<(i64, u64)> { - let mut batch = sled::Batch::default(); - let mut last_offset = 0; - let ts = time::OffsetDateTime::now_utc().unix_timestamp(); - for offset in 0..rand::thread_rng().gen_range(50..100) { - let event = Event::new_test(offset, "test", "empty"); - let event_bytes = utils::encode_model(&event)?; - batch.insert(&utils::encode_byte_prefix(PREFIX_STREAM_EVENT, offset), event_bytes.as_slice()); - last_offset = offset; - } - batch.insert(KEY_STREAM_LAST_WRITTEN_OFFSET, &utils::encode_u64(last_offset)); - batch.insert(&utils::encode_byte_prefix_i64(PREFIX_STREAM_TS, ts), &utils::encode_u64(last_offset)); - db.apply_batch(batch) - .context("error applying batch to write test data to stream")?; - Ok((ts, last_offset)) -} - -/// Setup some subscriptions data in the given DB tree returning the set of created subs. -async fn setup_subs_data(db: &sled::Tree) -> Result> { - let mut batch = sled::Batch::default(); - let mut subs = vec![]; - for offset in 0..rand::thread_rng().gen_range(50..100) { - let sub = Subscription { group_name: offset.to_string(), max_batch_size: 50 }; - let sub_encoded = utils::encode_model(&sub)?; - let sub_model_key = utils::ivec_from_iter( - PREFIX_STREAM_SUBS - .iter() - .copied() - .chain(sub.group_name.as_bytes().iter().copied()), - ); - let sub_offset_key = utils::ivec_from_iter( - PREFIX_STREAM_SUB_OFFSETS - .iter() - .copied() - .chain(sub.group_name.as_bytes().iter().copied()), - ); - batch.insert(sub_model_key, sub_encoded.as_slice()); - batch.insert(sub_offset_key, &utils::encode_u64(offset)); - subs.push((sub, offset)); - } - db.apply_batch(batch) - .context("error applying batch to write test data to stream")?; - subs.sort_by(|a, b| a.1.cmp(&b.1)); - Ok(subs) -} diff --git a/hadron-stream/src/stream/subscriber.rs b/hadron-stream/src/stream/subscriber.rs index c57535e..7aa85ca 100644 --- a/hadron-stream/src/stream/subscriber.rs +++ b/hadron-stream/src/stream/subscriber.rs @@ -2,7 +2,6 @@ use std::collections::HashMap; use std::sync::Arc; use anyhow::{Context, Result}; -use bytes::BytesMut; use futures::stream::StreamExt; use prost::Message; use rand::seq::IteratorRandom; @@ -20,13 +19,13 @@ use crate::config::Config; use crate::database::Database; use crate::error::{AppError, AppErrorExt, RpcResult, ShutdownError, ShutdownResult, ERR_DB_FLUSH, ERR_ITER_FAILURE}; use crate::futures::LivenessStream; -use crate::grpc::{ - Event, StreamSubscribeRequest, StreamSubscribeRequestAction, StreamSubscribeResponse, StreamSubscribeSetup, StreamSubscribeSetupStartingPoint, -}; +use crate::grpc::{Event, StreamSubscribeRequest, StreamSubscribeRequestAction, StreamSubscribeResponse, StreamSubscribeSetup, StreamSubscribeSetupStartingPoint}; use crate::models::stream::Subscription; use crate::stream::{PREFIX_STREAM_SUBS, PREFIX_STREAM_SUB_OFFSETS}; use crate::utils; +use super::PREFIX_STREAM_EVENT; + /// The default max batch size for subscription groups. const DEFAULT_MAX_BATCH_SIZE: u32 = 1; @@ -64,16 +63,13 @@ pub struct StreamSubCtl { shutdown_tx: broadcast::Sender<()>, /// A channel used for triggering graceful shutdown. shutdown_rx: BroadcastStream<()>, - - /// A general purpose reusable bytes buffer, safe for concurrent use. - buf: BytesMut, } impl StreamSubCtl { /// Create a new instance. pub fn new( - config: Arc, db: Database, tree: Tree, partition: u32, shutdown_tx: broadcast::Sender<()>, events_tx: mpsc::Sender, - events_rx: mpsc::Receiver, stream_offset: watch::Receiver, subs: Vec<(Subscription, u64)>, current_offset: u64, + config: Arc, db: Database, tree: Tree, partition: u32, shutdown_tx: broadcast::Sender<()>, events_tx: mpsc::Sender, events_rx: mpsc::Receiver, + stream_offset: watch::Receiver, subs: Vec<(Subscription, u64)>, current_offset: u64, ) -> Self { let subs = SubscriberInfo::new(subs); Self { @@ -89,7 +85,6 @@ impl StreamSubCtl { liveness_checks: StreamMap::new(), shutdown_rx: BroadcastStream::new(shutdown_tx.subscribe()), shutdown_tx, - buf: BytesMut::with_capacity(5000), } } @@ -142,32 +137,24 @@ impl StreamSubCtl { None => return, }; group.active_channels.remove(&id); - self.subs - .groups - .retain(|_, group| group.durable || !group.active_channels.is_empty()); + self.subs.groups.retain(|_, group| group.durable || !group.active_channels.is_empty()); } /// Handle a request which has been sent to this controller. #[tracing::instrument(level = "trace", skip(self, tx, rx, setup))] - async fn handle_request( - &mut self, tx: mpsc::Sender>, rx: Streaming, setup: StreamSubscribeSetup, - ) { + async fn handle_request(&mut self, tx: mpsc::Sender>, rx: Streaming, setup: StreamSubscribeSetup) { // Validate contents of setup request. if setup.group_name.is_empty() { - let _res = tx - .send(Err(Status::invalid_argument("subscriber group name may not be an empty string"))) - .await; + let _res = tx.send(Err(Status::invalid_argument("subscriber group name may not be an empty string"))).await; return; } if setup.max_batch_size == 0 { - let _res = tx - .send(Err(Status::invalid_argument("subscriber batch size must be greater than 0"))) - .await; + let _res = tx.send(Err(Status::invalid_argument("subscriber batch size must be greater than 0"))).await; return; } // Ensure the subscription is properly recorded. - let group = match self.ensure_subscriber_record(&setup).await { + let group = match ensure_subscriber_record(&self.tree, &setup, &mut self.subs, self.current_offset).await { Ok(group) => group, Err(err) => { if err.downcast_ref::().is_some() { @@ -225,7 +212,10 @@ impl StreamSubCtl { return; } }; - let msg = StreamSubscribeResponse { last_included_offset, batch: fetched_data.data }; + let msg = StreamSubscribeResponse { + last_included_offset, + batch: fetched_data.data, + }; group.delivery_cache = SubGroupDataCache::NeedsDelivery(Arc::new(msg)); // Attempt to deliver the data. @@ -246,74 +236,6 @@ impl StreamSubCtl { self.execute_delivery_pass().await; } - /// Ensure a subscription record exists if it does not already exist in the index. - #[tracing::instrument(level = "trace", skip(self, sub))] - async fn ensure_subscriber_record(&mut self, sub: &StreamSubscribeSetup) -> Result<&mut SubscriptionGroup> { - // Get a handle to the group subscriber data, creating one if not present. - let already_exists = self.subs.groups.contains_key(&sub.group_name); - let offset = match &sub.starting_point { - Some(StreamSubscribeSetupStartingPoint::Beginning(_empty)) => 0, - Some(StreamSubscribeSetupStartingPoint::Latest(_empty)) => self.current_offset, - Some(StreamSubscribeSetupStartingPoint::Offset(offset)) => { - let offset = if *offset == 0 { 0 } else { offset - 1 }; - if offset > self.current_offset { - self.current_offset - } else { - offset - } - } - None => self.current_offset, - }; - let entry = self.subs.groups.entry(sub.group_name.clone()) - // Ensure the subscription model exists. - .or_insert_with(|| { - let durable = sub.durable; - let max_batch_size = if sub.max_batch_size == 0 { DEFAULT_MAX_BATCH_SIZE } else { sub.max_batch_size }; - let sub = Subscription { - group_name: sub.group_name.clone(), - max_batch_size, - }; - SubscriptionGroup::new(sub, offset, durable) - }); - - // If the subscription is durable & did not already exist, then write the subscription model to disk. - if sub.durable && !already_exists { - let mut buf = self.buf.split(); - entry - .subscription - .encode(&mut buf) - .context("error encoding subscription record")?; - - let sub_model_key = utils::ivec_from_iter( - PREFIX_STREAM_SUBS - .iter() - .copied() - .chain(sub.group_name.as_bytes().iter().copied()), - ); - let sub_offset_key = utils::ivec_from_iter( - PREFIX_STREAM_SUB_OFFSETS - .iter() - .copied() - .chain(sub.group_name.as_bytes().iter().copied()), - ); - - let mut batch = sled::Batch::default(); - batch.insert(sub_model_key, buf.freeze().as_ref()); - batch.insert(sub_offset_key, &utils::encode_u64(offset)); - self.tree - .apply_batch(batch) - .context("error writing subscription record and offset to disk") - .map_err(ShutdownError::from)?; - self.tree - .flush_async() - .await - .context(ERR_DB_FLUSH) - .map_err(ShutdownError::from)?; - } - - Ok(entry) - } - /// Execute a loop over all active subscription groups, delivering data if possible. /// /// ### Notes @@ -353,13 +275,7 @@ impl StreamSubCtl { // This group is ready to have some data fetched, so spawn a task to do so. group.is_fetching_data = true; - Self::spawn_group_fetch( - group.group_name.clone(), - group.offset + 1, - group.subscription.max_batch_size, - self.tree.clone(), - self.events_tx.clone(), - ); + spawn_group_fetch(group.group_name.clone(), group.offset + 1, group.subscription.max_batch_size, self.tree.clone(), self.events_tx.clone()); } } @@ -367,28 +283,19 @@ impl StreamSubCtl { async fn try_handle_delivery_response(&mut self, delivery_res: DeliveryResponse) -> Result<()> { // Get a mutable handle to subscription group to which this response applies. let (chan_id, group_name) = (&delivery_res.id, &*delivery_res.group_name); - let group = self - .subs - .groups - .get_mut(group_name) - .context("response from subscription delivery dropped as group no longer exists")?; + let group = self.subs.groups.get_mut(group_name).context("response from subscription delivery dropped as group no longer exists")?; // Unpack response body. let last_offset = delivery_res.orig_data.last_included_offset; group.delivery_cache = SubGroupDataCache::NeedsDelivery(delivery_res.orig_data); - let res = delivery_res - .output - .context("subscriber channel closed while awaiting delivery response") - .map_err(|err| { - let _ = group.active_channels.remove(chan_id); - err - })?; - let (client_chan, body) = res - .context("error returned while awaiting subscriber delivery response") - .map_err(|err| { - let _ = group.active_channels.remove(chan_id); - err - })?; + let res = delivery_res.output.context("subscriber channel closed while awaiting delivery response").map_err(|err| { + let _ = group.active_channels.remove(chan_id); + err + })?; + let (client_chan, body) = res.context("error returned while awaiting subscriber delivery response").map_err(|err| { + let _ = group.active_channels.remove(chan_id); + err + })?; if let Some(chan_wrapper) = group.active_channels.get_mut(chan_id) { *chan_wrapper = SubChannelState::MonitoringLiveness; self.liveness_checks.insert( @@ -413,44 +320,16 @@ impl StreamSubCtl { _ => Err("unexpected or malformed response returned from subscriber, expected ack or nack".into()), }; if group.durable { - Self::try_record_delivery_response(record_res, group.group_name.clone(), self.tree.clone()) + try_record_delivery_response(record_res, group.group_name.clone(), self.tree.clone()) .await .context("error while recording subscriber delivery response")?; } Ok(()) } - /// Record the ack/nack response from a subscriber delivery. - #[tracing::instrument(level = "trace", skip(res, group_name, tree))] - async fn try_record_delivery_response(res: std::result::Result, group_name: Arc, tree: Tree) -> ShutdownResult<()> { - let offset = match res { - Ok(offset) => offset, - Err(_err) => { - // TODO[telemetry]: in the future, we will record this for observability system. - return Ok(()); - } - }; - let key = utils::ivec_from_iter( - PREFIX_STREAM_SUB_OFFSETS - .iter() - .copied() - .chain(group_name.as_bytes().iter().copied()), - ); - tree.insert(key, &utils::encode_u64(offset)) - .context("error updating subscription offsets on disk") - .map_err(ShutdownError::from)?; - tree.flush_async() - .await - .context(ERR_DB_FLUSH) - .map_err(ShutdownError::from)?; - Ok(()) - } - /// Attempt to deliver a payload of data to the target group. #[tracing::instrument(level = "trace", skip(group, liveness_stream, tx))] - async fn try_deliver_data_to_sub( - group: &mut SubscriptionGroup, liveness_stream: &mut StreamMap, tx: mpsc::Sender, - ) { + async fn try_deliver_data_to_sub(group: &mut SubscriptionGroup, liveness_stream: &mut StreamMap, tx: mpsc::Sender) { // Get a handle to the group's cached data, else there is nothing to do here. let data = match &group.delivery_cache { SubGroupDataCache::NeedsDelivery(data) => data.clone(), @@ -508,58 +387,118 @@ impl StreamSubCtl { let (tx, group_name, id, orig_data) = (tx.clone(), group.group_name.clone(), chan_id, data); tokio::spawn(async move { // TODO: add optional timeouts here based on subscription group config. - let output = chan - .1 - .next() - .await - .map(|res| res.map_err(anyhow::Error::from).map(|data| (chan, data))); - let _ = tx - .send(StreamSubCtlMsg::DeliveryResponse(DeliveryResponse { id, group_name, output, orig_data })) - .await; + let output = chan.1.next().await.map(|res| res.map_err(anyhow::Error::from).map(|data| (chan, data))); + let _ = tx.send(StreamSubCtlMsg::DeliveryResponse(DeliveryResponse { id, group_name, output, orig_data })).await; }); return; } } +} - /// Spawn a data fetch operation to pull data from the stream for a subscription group. - #[tracing::instrument(level = "trace", skip(group_name, max_batch_size, tree, tx))] - fn spawn_group_fetch(group_name: Arc, next_offset: u64, max_batch_size: u32, tree: Tree, tx: mpsc::Sender) { - tokio::spawn(async move { - // Spawn a blocking read of the stream. - let res = Database::spawn_blocking(move || -> Result { - let start = utils::encode_u64(next_offset); - let stop = utils::encode_u64(next_offset + max_batch_size as u64); - let mut data = Vec::with_capacity(max_batch_size as usize); - let mut last_included_offset = None; - for iter_res in tree.range(start..stop) { - let (key, val) = iter_res.context(ERR_ITER_FAILURE).map_err(ShutdownError::from)?; - let offset = utils::decode_u64(&key) - .context("error decoding event offset") - .map_err(ShutdownError::from)?; - let event: Event = utils::decode_model(val.as_ref()) - .context("error decoding event from storage") - .map_err(ShutdownError::from)?; - data.push(event); - last_included_offset = Some(offset); - } - Ok(FetchStreamRecords { group_name, data, last_included_offset }) - }) - .await - .and_then(|res| res.map_err(ShutdownError::from)); - - match res { - Ok(fetched_data) => { - let _ = tx.send(StreamSubCtlMsg::FetchStreamRecords(Ok(fetched_data))).await; - } - Err(err) => { - let _ = tx.send(StreamSubCtlMsg::FetchStreamRecords(Err(err))).await; - } +/// Ensure a subscription record exists if it does not already exist in the index. +#[tracing::instrument(level = "trace", skip(tree, sub, subs, current_offset))] +pub(super) async fn ensure_subscriber_record<'a>(tree: &Tree, sub: &StreamSubscribeSetup, subs: &'a mut SubscriberInfo, current_offset: u64) -> Result<&'a mut SubscriptionGroup> { + // Get a handle to the group subscriber data, creating one if not present. + let already_exists = subs.groups.contains_key(&sub.group_name); + let offset = match &sub.starting_point { + Some(StreamSubscribeSetupStartingPoint::Beginning(_empty)) => 0, + Some(StreamSubscribeSetupStartingPoint::Latest(_empty)) => current_offset, + Some(StreamSubscribeSetupStartingPoint::Offset(offset)) => { + let offset = if *offset == 0 { 0 } else { offset - 1 }; + if offset > current_offset { + current_offset + } else { + offset } + } + None => current_offset, + }; + let entry = subs.groups.entry(sub.group_name.clone()) + // Ensure the subscription model exists. + .or_insert_with(|| { + let durable = sub.durable; + let max_batch_size = if sub.max_batch_size == 0 { DEFAULT_MAX_BATCH_SIZE } else { sub.max_batch_size }; + let sub = Subscription { + group_name: sub.group_name.clone(), + max_batch_size, + }; + SubscriptionGroup::new(sub, offset, durable) }); + + // If the subscription is durable & did not already exist, then write the subscription model to disk. + if sub.durable && !already_exists { + let model = entry.subscription.encode_to_vec(); + + let sub_model_key = utils::ivec_from_iter(PREFIX_STREAM_SUBS.iter().copied().chain(sub.group_name.as_bytes().iter().copied())); + let sub_offset_key = utils::ivec_from_iter(PREFIX_STREAM_SUB_OFFSETS.iter().copied().chain(sub.group_name.as_bytes().iter().copied())); + + let mut batch = sled::Batch::default(); + batch.insert(sub_model_key, model.as_slice()); + batch.insert(sub_offset_key, &utils::encode_u64(offset)); + tree.apply_batch(batch).context("error writing subscription record and offset to disk").map_err(ShutdownError::from)?; + tree.flush_async().await.context(ERR_DB_FLUSH).map_err(ShutdownError::from)?; } + + Ok(entry) +} + +/// Spawn a data fetch operation to pull data from the stream for a subscription group. +#[tracing::instrument(level = "trace", skip(group_name, max_batch_size, tree, tx))] +pub(super) fn spawn_group_fetch(group_name: Arc, next_offset: u64, max_batch_size: u32, tree: Tree, tx: mpsc::Sender) { + tokio::spawn(async move { + // Spawn a blocking read of the stream. + let res = Database::spawn_blocking(move || -> Result { + let start = utils::encode_byte_prefix(PREFIX_STREAM_EVENT, next_offset); + let stop = utils::encode_byte_prefix(PREFIX_STREAM_EVENT, next_offset + max_batch_size as u64); + let mut data = Vec::with_capacity(max_batch_size as usize); + let mut last_included_offset = None; + for iter_res in tree.range(start..stop) { + let (key, val) = iter_res.context(ERR_ITER_FAILURE).map_err(ShutdownError::from)?; + let offset = utils::decode_u64(&key[1..]).context("error decoding event offset").map_err(ShutdownError::from)?; + let event: Event = utils::decode_model(val.as_ref()).context("error decoding event from storage").map_err(ShutdownError::from)?; + data.push(event); + last_included_offset = Some(offset); + } + Ok(FetchStreamRecords { + group_name, + data, + last_included_offset, + }) + }) + .await + .and_then(|res| res.map_err(ShutdownError::from)); + + match res { + Ok(fetched_data) => { + let _ = tx.send(StreamSubCtlMsg::FetchStreamRecords(Ok(fetched_data))).await; + } + Err(err) => { + let _ = tx.send(StreamSubCtlMsg::FetchStreamRecords(Err(err))).await; + } + } + }); +} + +/// Record the ack/nack response from a subscriber delivery. +#[tracing::instrument(level = "trace", skip(res, group_name, tree))] +pub(super) async fn try_record_delivery_response(res: std::result::Result, group_name: Arc, tree: Tree) -> ShutdownResult<()> { + let offset = match res { + Ok(offset) => offset, + Err(_err) => { + // TODO[telemetry]: in the future, we will record this for observability system. + return Ok(()); + } + }; + let key = utils::ivec_from_iter(PREFIX_STREAM_SUB_OFFSETS.iter().copied().chain(group_name.as_bytes().iter().copied())); + tree.insert(key, &utils::encode_u64(offset)) + .context("error updating subscription offsets on disk") + .map_err(ShutdownError::from)?; + tree.flush_async().await.context(ERR_DB_FLUSH).map_err(ShutdownError::from)?; + Ok(()) } /// A message bound for a stream subscription controller. +#[derive(Debug)] pub enum StreamSubCtlMsg { /// A client request being routed to the controller. Request { @@ -573,28 +512,31 @@ pub enum StreamSubCtlMsg { DeliveryResponse(DeliveryResponse), } +#[derive(Debug)] pub struct FetchStreamRecords { - group_name: Arc, - data: Vec, - last_included_offset: Option, + pub group_name: Arc, + pub data: Vec, + pub last_included_offset: Option, } +#[derive(Debug)] pub struct DeliveryResponse { /// The ID of the subscription channel. - id: Uuid, + pub id: Uuid, /// The name of the subscription group. - group_name: Arc, + pub group_name: Arc, /// The output of awaiting for the subscriber's response. - output: Option>, + pub output: Option>, /// The original data delivered. - orig_data: Arc, + pub orig_data: Arc, } ////////////////////////////////////////////////////////////////////////////// ////////////////////////////////////////////////////////////////////////////// /// Data on all subscriptions along with their active subscriber channels. -struct SubscriberInfo { +#[derive(Default)] +pub(super) struct SubscriberInfo { /// A mapping of all subscriptions by group name. pub groups: HashMap, } @@ -614,22 +556,22 @@ impl SubscriberInfo { } /// Data on a subscription group. -struct SubscriptionGroup { +pub(super) struct SubscriptionGroup { /// An Arc'd copy of the group's name for easy sharing across threads /// without the need for additional allocations. - group_name: Arc, + pub group_name: Arc, /// The data model of this subscription. - subscription: Subscription, + pub subscription: Subscription, /// A bool indicating if this is a durable group or not. - durable: bool, + pub durable: bool, /// The last offset to have been processed by this subscription. - offset: u64, + pub offset: u64, /// A mapping of all active subscribers of this group. - active_channels: HashMap, + pub active_channels: HashMap, /// The possible states of this group's data delivery cache. - delivery_cache: SubGroupDataCache, + pub delivery_cache: SubGroupDataCache, /// A bool indicating if data is currently being fetched for this group. - is_fetching_data: bool, + pub is_fetching_data: bool, } impl SubscriptionGroup { @@ -648,7 +590,7 @@ impl SubscriptionGroup { } /// A type wrapping an H2 data channel which be unavailable while out deliverying data. -enum SubChannelState { +pub(super) enum SubChannelState { /// The channel is currently out as it is being used to deliver data. OutForDelivery, /// The channel is currently held in a stream monitoring its liveness. @@ -656,7 +598,7 @@ enum SubChannelState { } /// The possible states of a subscription group's data delivery cache. -enum SubGroupDataCache { +pub(super) enum SubGroupDataCache { /// No data is currently cached. None, /// Data is cached and needs to be delivered. diff --git a/hadron-stream/src/stream/subscriber_test.rs b/hadron-stream/src/stream/subscriber_test.rs new file mode 100644 index 0000000..6e47ceb --- /dev/null +++ b/hadron-stream/src/stream/subscriber_test.rs @@ -0,0 +1,286 @@ +use std::sync::Arc; + +use anyhow::{Context, Result}; +use tokio::sync::mpsc; + +use crate::config::Config; +use crate::database::Database; +use crate::fixtures; +use crate::grpc::{StreamSubscribeSetup, StreamSubscribeSetupStartingPoint}; +use crate::models::stream::Subscription; +use crate::stream::subscriber::{self, StreamSubCtlMsg, SubGroupDataCache, SubscriberInfo, SubscriptionGroup}; +use crate::stream::{PREFIX_STREAM_SUBS, PREFIX_STREAM_SUB_OFFSETS}; +use crate::utils; + +#[tokio::test] +async fn spawn_group_fetch_ok() -> Result<()> { + let (config, _tmpdir) = Config::new_test()?; + let db = Database::new(config.clone()).await?; + let stream_tree = db.get_stream_tree().await?; + let group_name = Arc::new(String::from("test-group")); + let (_last_ts, _last_offset) = fixtures::setup_stream_data(&stream_tree).await?; + let ((tx, mut rx), timeout) = (mpsc::channel(1), std::time::Duration::from_secs(5)); + + subscriber::spawn_group_fetch(group_name.clone(), 0, 20, stream_tree.clone(), tx.clone()); + let output_msg = tokio::time::timeout(timeout, rx.recv()) + .await + .context("timeout from group fetch")? + .context("error from spawn group fetch")?; + let output = match output_msg { + StreamSubCtlMsg::FetchStreamRecords(data) => data?, + res => anyhow::bail!("unexpected response type from spawn_group_fetch, expected FetchStreamRecords, got {:?}", res), + }; + + assert_eq!(&output.group_name, &group_name, "mismatch group names, expected {}, got {}", &group_name, &output.group_name); + assert_eq!( + output.last_included_offset, + Some(19), + "mismatch last_included_offset, expected {:?}, got {:?}", + output.last_included_offset, + Some(19), + ); + assert_eq!(output.data.len(), 20, "mismatch payload len, expected {}, got {}", 20, output.data.len()); + + Ok(()) +} + +#[tokio::test] +async fn spawn_group_fetch_empty_response_when_no_new_data() -> Result<()> { + let (config, _tmpdir) = Config::new_test()?; + let db = Database::new(config.clone()).await?; + let stream_tree = db.get_stream_tree().await?; + let group_name = Arc::new(String::from("test-group")); + let (_last_ts, last_offset) = fixtures::setup_stream_data(&stream_tree).await?; + let ((tx, mut rx), timeout) = (mpsc::channel(1), std::time::Duration::from_secs(5)); + + subscriber::spawn_group_fetch(group_name.clone(), last_offset + 1, 20, stream_tree.clone(), tx.clone()); + let output_msg = tokio::time::timeout(timeout, rx.recv()) + .await + .context("timeout from group fetch")? + .context("error from spawn group fetch")?; + let output = match output_msg { + StreamSubCtlMsg::FetchStreamRecords(data) => data?, + res => anyhow::bail!("unexpected response type from spawn_group_fetch, expected FetchStreamRecords, got {:?}", res), + }; + + assert_eq!(&output.group_name, &group_name, "mismatch group names, expected {}, got {}", &group_name, &output.group_name); + assert!( + output.last_included_offset.is_none(), + "mismatch last_included_offset, expected None, got {:?}", + output.last_included_offset + ); + assert!(output.data.is_empty(), "mismatch payload len, expected data to be empty, got len {}", output.data.len()); + + Ok(()) +} + +#[tokio::test] +async fn try_record_delivery_response_ok_no_update_with_error_response() -> Result<()> { + let (config, _tmpdir) = Config::new_test()?; + let db = Database::new(config.clone()).await?; + let stream_tree = db.get_stream_tree().await?; + let group_name = Arc::new(String::from("test-group")); + let err_msg = String::from("test error message"); + + let _output = subscriber::try_record_delivery_response(Err(err_msg), group_name.clone(), stream_tree.clone()) + .await + .context("error from try_record_delivery_response")?; + + let key = utils::ivec_from_iter(PREFIX_STREAM_SUB_OFFSETS.iter().copied().chain(group_name.as_bytes().iter().copied())); + let opt_sub_offset = stream_tree.get(key).context("error fetching stream sub offset")?; + assert!(opt_sub_offset.is_none(), "expected stream subscription offset to be None, got {:?}", opt_sub_offset); + + Ok(()) +} + +#[tokio::test] +async fn try_record_delivery_response_ok_with_expected_stream_record() -> Result<()> { + let (config, _tmpdir) = Config::new_test()?; + let db = Database::new(config.clone()).await?; + let stream_tree = db.get_stream_tree().await?; + let group_name = Arc::new(String::from("test-group")); + let last_offset_processed = 100; + + let _output = subscriber::try_record_delivery_response(Ok(last_offset_processed), group_name.clone(), stream_tree.clone()) + .await + .context("error from try_record_delivery_response")?; + + let key = utils::ivec_from_iter(PREFIX_STREAM_SUB_OFFSETS.iter().copied().chain(group_name.as_bytes().iter().copied())); + let sub_offset_ivec = stream_tree + .get(key) + .context("error fetching stream sub offset")? + .context("expected stream subscription offset to be recorded, got None")?; + let sub_offset = utils::decode_u64(&sub_offset_ivec)?; + assert_eq!( + sub_offset, last_offset_processed, + "expected stream subscription offset to be {}, got {}", + last_offset_processed, sub_offset + ); + + Ok(()) +} + +#[tokio::test] +async fn ensure_subscriber_record_ok_new_record_offset_beginning() -> Result<()> { + let (config, _tmpdir) = Config::new_test()?; + let db = Database::new(config.clone()).await?; + let stream_tree = db.get_stream_tree().await?; + let (group_name, current_offset, max_batch_size) = (String::from("test-group"), 100, 500); + let setup = StreamSubscribeSetup { + group_name: group_name.clone(), + durable: true, + max_batch_size, + starting_point: Some(StreamSubscribeSetupStartingPoint::Beginning(Default::default())), + }; + let mut subs = SubscriberInfo::default(); + + let output = subscriber::ensure_subscriber_record(&stream_tree, &setup, &mut subs, current_offset) + .await + .context("error ensuring subscriber record")?; + + assert_eq!(output.group_name.as_str(), group_name.as_str(), "expected group name {}, got {}", group_name, output.group_name); + assert!(output.durable, "expected subscription group to be durable, got false"); + assert_eq!(output.offset, 0, "expected subscription group offset to be 0, got {}", output.offset); + assert!(output.active_channels.is_empty(), "expected subscription active channels to be initially empty"); + assert!(matches!(output.delivery_cache, SubGroupDataCache::None), "expected subscription active channels to be initially empty"); + assert!(!output.is_fetching_data, "expected subscription to not be fetching data initially"); + + let sub_model_key = utils::ivec_from_iter(PREFIX_STREAM_SUBS.iter().copied().chain(group_name.as_bytes().iter().copied())); + let sub_model_ivec = stream_tree + .get(sub_model_key) + .context("error fetching stream subscription")? + .context("expected subscription to be recorded")?; + let sub_model: Subscription = utils::decode_model(&sub_model_ivec).context("error decoding sub_model_ivec")?; + assert_eq!(sub_model.group_name, group_name, "expected group name {}, got {}", group_name, sub_model.group_name); + assert_eq!(sub_model.max_batch_size, max_batch_size, "expected group name {}, got {}", max_batch_size, sub_model.max_batch_size); + assert_eq!( + output.subscription, sub_model, + "expected subscription models to be identical, expected {:?}, got {:?}", + sub_model, output.subscription + ); + + let sub_offset_key = utils::ivec_from_iter(PREFIX_STREAM_SUB_OFFSETS.iter().copied().chain(group_name.as_bytes().iter().copied())); + let sub_offset_ivec = stream_tree + .get(sub_offset_key) + .context("error fetching stream subscription offset")? + .context("expected subscription offset to be recorded")?; + let sub_offset = utils::decode_u64(&sub_offset_ivec).context("error decoding sub_offset_ivec")?; + assert_eq!(sub_offset, 0, "expected new sub offset to be 0, got {}", sub_offset); + + Ok(()) +} + +#[tokio::test] +async fn ensure_subscriber_record_ok_noop_if_already_exists() -> Result<()> { + let (config, _tmpdir) = Config::new_test()?; + let db = Database::new(config.clone()).await?; + let stream_tree = db.get_stream_tree().await?; + let (group_name, current_offset, max_batch_size) = (String::from("test-group"), 100, 500); + let setup = StreamSubscribeSetup { + group_name: group_name.clone(), + durable: true, + max_batch_size, + starting_point: Some(StreamSubscribeSetupStartingPoint::Beginning(Default::default())), + }; + let mut subs = SubscriberInfo::default(); + subs.groups.insert( + group_name.clone(), + SubscriptionGroup { + group_name: Arc::new(String::from("something-else-test")), + subscription: Default::default(), + durable: true, + offset: 0, + active_channels: Default::default(), + delivery_cache: SubGroupDataCache::None, + is_fetching_data: false, + }, + ); + + let output = subscriber::ensure_subscriber_record(&stream_tree, &setup, &mut subs, current_offset) + .await + .context("error ensuring subscriber record")?; + + assert_ne!(output.group_name.as_str(), group_name.as_str(), "expected group names to be different to assert no-op behavior"); + + let sub_model_key = utils::ivec_from_iter(PREFIX_STREAM_SUBS.iter().copied().chain(group_name.as_bytes().iter().copied())); + let sub_model_opt = stream_tree.get(sub_model_key).context("error fetching stream subscription")?; + assert!(sub_model_opt.is_none(), "expected sub_model_opt to be None"); + + let sub_offset_key = utils::ivec_from_iter(PREFIX_STREAM_SUB_OFFSETS.iter().copied().chain(group_name.as_bytes().iter().copied())); + let sub_offset_opt = stream_tree.get(sub_offset_key).context("error fetching stream subscription offset")?; + assert!(sub_offset_opt.is_none(), "expected sub_offset_opt to be None"); + + Ok(()) +} + +#[tokio::test] +async fn ensure_subscriber_record_ok_start_point_latest() -> Result<()> { + let (config, _tmpdir) = Config::new_test()?; + let db = Database::new(config.clone()).await?; + let stream_tree = db.get_stream_tree().await?; + let (group_name, current_offset, max_batch_size) = (String::from("test-group"), 100, 500); + let setup = StreamSubscribeSetup { + group_name: group_name.clone(), + durable: false, + max_batch_size, + starting_point: Some(StreamSubscribeSetupStartingPoint::Latest(Default::default())), + }; + let mut subs = SubscriberInfo::default(); + + let output = subscriber::ensure_subscriber_record(&stream_tree, &setup, &mut subs, current_offset) + .await + .context("error ensuring subscriber record")?; + + assert_eq!(output.group_name.as_str(), group_name.as_str(), "expected group name {}, got {}", group_name, output.group_name); + assert!(!output.durable, "expected subscription group to be durable, got false"); + assert_eq!(output.offset, current_offset, "expected subscription group offset to be {}, got {}", current_offset, output.offset); + assert!(output.active_channels.is_empty(), "expected subscription active channels to be initially empty"); + assert!(matches!(output.delivery_cache, SubGroupDataCache::None), "expected subscription active channels to be initially empty"); + assert!(!output.is_fetching_data, "expected subscription to not be fetching data initially"); + + let sub_model_key = utils::ivec_from_iter(PREFIX_STREAM_SUBS.iter().copied().chain(group_name.as_bytes().iter().copied())); + let sub_model_opt = stream_tree.get(sub_model_key).context("error fetching stream subscription")?; + assert!(sub_model_opt.is_none(), "expected sub_model_opt to be None"); + + let sub_offset_key = utils::ivec_from_iter(PREFIX_STREAM_SUB_OFFSETS.iter().copied().chain(group_name.as_bytes().iter().copied())); + let sub_offset_opt = stream_tree.get(sub_offset_key).context("error fetching stream subscription offset")?; + assert!(sub_offset_opt.is_none(), "expected sub_offset_opt to be None"); + + Ok(()) +} + +#[tokio::test] +async fn ensure_subscriber_record_ok_start_point_offset() -> Result<()> { + let (config, _tmpdir) = Config::new_test()?; + let db = Database::new(config.clone()).await?; + let stream_tree = db.get_stream_tree().await?; + let (group_name, current_offset, max_batch_size, start_offset) = (String::from("test-group"), 100, 500, 50); + let setup = StreamSubscribeSetup { + group_name: group_name.clone(), + durable: false, + max_batch_size, + starting_point: Some(StreamSubscribeSetupStartingPoint::Offset(start_offset)), + }; + let mut subs = SubscriberInfo::default(); + + let output = subscriber::ensure_subscriber_record(&stream_tree, &setup, &mut subs, current_offset) + .await + .context("error ensuring subscriber record")?; + + assert_eq!(output.group_name.as_str(), group_name.as_str(), "expected group name {}, got {}", group_name, output.group_name); + assert!(!output.durable, "expected subscription group to be durable, got false"); + assert_eq!(output.offset, start_offset - 1, "expected subscription group offset to be {}, got {}", start_offset - 1, output.offset); + assert!(output.active_channels.is_empty(), "expected subscription active channels to be initially empty"); + assert!(matches!(output.delivery_cache, SubGroupDataCache::None), "expected subscription active channels to be initially empty"); + assert!(!output.is_fetching_data, "expected subscription to not be fetching data initially"); + + let sub_model_key = utils::ivec_from_iter(PREFIX_STREAM_SUBS.iter().copied().chain(group_name.as_bytes().iter().copied())); + let sub_model_opt = stream_tree.get(sub_model_key).context("error fetching stream subscription")?; + assert!(sub_model_opt.is_none(), "expected sub_model_opt to be None"); + + let sub_offset_key = utils::ivec_from_iter(PREFIX_STREAM_SUB_OFFSETS.iter().copied().chain(group_name.as_bytes().iter().copied())); + let sub_offset_opt = stream_tree.get(sub_offset_key).context("error fetching stream subscription offset")?; + assert!(sub_offset_opt.is_none(), "expected sub_offset_opt to be None"); + + Ok(()) +} diff --git a/rustfmt.toml b/rustfmt.toml index a2e931b..5801e69 100644 --- a/rustfmt.toml +++ b/rustfmt.toml @@ -1,27 +1,7 @@ -unstable_features = true -edition = "2018" +edition = "2021" comment_width = 100 fn_args_layout = "Compressed" -max_width = 150 +max_width = 200 use_small_heuristics = "Default" use_try_shorthand = true - -# pre-unstable -chain_width = 75 -single_line_if_else_max_width = 75 -space_around_attr_eq = false -struct_lit_width = 50 - -# unstable -condense_wildcard_suffixes = true -format_code_in_doc_comments = true -format_strings = true -match_block_trailing_comma = false -normalize_comments = true -normalize_doc_attributes = true -reorder_impl_items = true -struct_lit_single_line = true -trailing_comma = "Vertical" -use_field_init_shorthand = true -wrap_comments = true From f431ddc736a5aba807c8399fb058979bae1fa319 Mon Sep 17 00:00:00 2001 From: Anthony Dodd Date: Tue, 9 Nov 2021 21:12:20 -0600 Subject: [PATCH 10/10] Configure base CI --- .github/workflows/_rust-ci-base.yaml | 105 +++++++++++++++++ .github/workflows/ci-cli.yaml | 16 +++ .github/workflows/ci-client.yaml | 16 +++ .github/workflows/ci-core.yaml | 16 +++ .github/workflows/ci-examples.yaml | 22 ++++ .github/workflows/ci-guide.yaml | 22 ++++ .github/workflows/ci-operator.yaml | 16 +++ .github/workflows/ci-stream.yaml | 16 +++ .github/workflows/ci.yaml | 15 --- .../{pages.yaml => release-guide.yaml} | 0 .github/workflows/release.yaml | 12 -- hadron-stream/src/app.rs | 42 ++----- hadron-stream/src/config_test.rs | 98 +++------------- hadron-stream/src/database.rs | 8 +- hadron-stream/src/fixtures.rs | 25 ++-- hadron-stream/src/pipeline/mod.rs | 111 ++++++------------ hadron-stream/src/server.rs | 64 ++++------ hadron-stream/src/stream/mod_test.rs | 99 +++------------- hadron-stream/src/stream/publisher.rs | 22 +--- hadron-stream/src/stream/publisher_test.rs | 63 +++------- hadron-stream/src/utils_test.rs | 28 +---- hadron-stream/src/watchers/pipelines.rs | 55 +++++---- hadron-stream/src/watchers/stream.rs | 10 +- hadron-stream/src/watchers/tokens.rs | 4 +- rustfmt.toml | 1 - 25 files changed, 392 insertions(+), 494 deletions(-) create mode 100644 .github/workflows/_rust-ci-base.yaml create mode 100644 .github/workflows/ci-cli.yaml create mode 100644 .github/workflows/ci-client.yaml create mode 100644 .github/workflows/ci-core.yaml create mode 100644 .github/workflows/ci-examples.yaml create mode 100644 .github/workflows/ci-guide.yaml create mode 100644 .github/workflows/ci-operator.yaml create mode 100644 .github/workflows/ci-stream.yaml delete mode 100644 .github/workflows/ci.yaml rename .github/workflows/{pages.yaml => release-guide.yaml} (100%) delete mode 100644 .github/workflows/release.yaml diff --git a/.github/workflows/_rust-ci-base.yaml b/.github/workflows/_rust-ci-base.yaml new file mode 100644 index 0000000..70d08e2 --- /dev/null +++ b/.github/workflows/_rust-ci-base.yaml @@ -0,0 +1,105 @@ +name: CI Base +on: + workflow_call: + inputs: + project: + type: string + required: true + description: 'The Rust project to target' + dir: + type: string + required: false + default: '.' + description: 'The directory to start from' + +jobs: + lint: + name: lint + runs-on: ubuntu-latest + steps: + - name: Setup | Checkout + uses: actions/checkout@v2 + + - uses: actions/cache@v2 + with: + path: | + ~/.cargo/registry + ~/.cargo/git + ${{ inputs.dir }}/${{ inputs.project }}/target + key: cargo-${{ hashFiles(format('{0}/{1}/Cargo.lock', inputs.dir, inputs.project)) }} + restore-keys: | + ${{ inputs.project }}-cargo- + + - name: Setup | Toolchain + uses: actions-rs/toolchain@v1 + with: + toolchain: stable + default: true + components: clippy,rustfmt + + - name: Build | Clippy + uses: actions-rs/clippy-check@v1 + with: + token: ${{ secrets.GITHUB_TOKEN }} + args: --manifest-path ${{ inputs.dir }}/${{ inputs.project }}/Cargo.toml --all-targets -- -D warnings + + - name: Build | Rustfmt + run: cargo fmt --manifest-path ${{ inputs.dir }}/${{ inputs.project }}/Cargo.toml -- --check + + check: + name: check + runs-on: ubuntu-latest + steps: + - name: Setup | Checkout + uses: actions/checkout@v2 + + - name: Setup | Cache Cargo + uses: actions/cache@v2 + with: + path: | + ~/.cargo/registry + ~/.cargo/git + ${{ inputs.dir }}/${{ inputs.project }}/target + key: cargo-${{ hashFiles(format('{0}/{1}/Cargo.lock', inputs.dir, inputs.project)) }} + restore-keys: | + ${{ inputs.project }}-cargo- + + - name: Setup | Rust + uses: actions-rs/toolchain@v1 + with: + toolchain: stable + profile: minimal + + - name: Build | Check + run: cargo check --all --manifest-path ${{ inputs.dir }}/${{ inputs.project }}/Cargo.toml + + test: + name: test + needs: check # Ensure check is run first. + runs-on: ubuntu-latest + steps: + - name: Setup | Checkout + uses: actions/checkout@v2 + + - name: Setup | Cache Cargo + uses: actions/cache@v2 + with: + path: | + ~/.cargo/registry + ~/.cargo/git + ${{ inputs.dir }}/${{ inputs.project }}/target + key: cargo-${{ hashFiles(format('{0}/{1}/Cargo.lock', inputs.dir, inputs.project)) }} + restore-keys: | + ${{ inputs.project }}-cargo- + + - name: Setup | Rust + uses: actions-rs/toolchain@v1 + with: + toolchain: stable + profile: minimal + + - name: Build | Test + run: cargo test --manifest-path ${{ inputs.dir }}/${{ inputs.project }}/Cargo.toml + + - name: Build | Debug + run: cargo build --manifest-path ${{ inputs.dir }}/${{ inputs.project }}/Cargo.toml diff --git a/.github/workflows/ci-cli.yaml b/.github/workflows/ci-cli.yaml new file mode 100644 index 0000000..ccdbbcb --- /dev/null +++ b/.github/workflows/ci-cli.yaml @@ -0,0 +1,16 @@ +name: CI CLI +on: + push: + branches: + - "*" + paths: + - 'hadron-cli/**' + - "!**.md" + tags-ignore: + - "*" + +jobs: + ci: + uses: hadron-project/hadron/.github/workflows/_rust-ci-base.yaml@99-compaction + with: + project: hadron-cli diff --git a/.github/workflows/ci-client.yaml b/.github/workflows/ci-client.yaml new file mode 100644 index 0000000..771831d --- /dev/null +++ b/.github/workflows/ci-client.yaml @@ -0,0 +1,16 @@ +name: CI Client +on: + push: + branches: + - "*" + paths: + - 'hadron-client/**' + - "!**.md" + tags-ignore: + - "*" + +jobs: + ci: + uses: hadron-project/hadron/.github/workflows/_rust-ci-base.yaml@99-compaction + with: + project: hadron-client diff --git a/.github/workflows/ci-core.yaml b/.github/workflows/ci-core.yaml new file mode 100644 index 0000000..1ed1a57 --- /dev/null +++ b/.github/workflows/ci-core.yaml @@ -0,0 +1,16 @@ +name: CI Core +on: + push: + branches: + - "*" + paths: + - 'hadron-core/**' + - "!**.md" + tags-ignore: + - "*" + +jobs: + ci: + uses: hadron-project/hadron/.github/workflows/_rust-ci-base.yaml@99-compaction + with: + project: hadron-core diff --git a/.github/workflows/ci-examples.yaml b/.github/workflows/ci-examples.yaml new file mode 100644 index 0000000..1d9d75f --- /dev/null +++ b/.github/workflows/ci-examples.yaml @@ -0,0 +1,22 @@ +name: CI Core +on: + push: + branches: + - "*" + paths: + - 'examples/**' + - "!**.md" + tags-ignore: + - "*" + +jobs: + pipeline-txp: + uses: hadron-project/hadron/.github/workflows/_rust-ci-base.yaml@99-compaction + with: + project: pipeline-transactional-processing + dir: examples + pipeline-txp: + uses: hadron-project/hadron/.github/workflows/_rust-ci-base.yaml@99-compaction + with: + project: stream-transactional-processing + dir: examples diff --git a/.github/workflows/ci-guide.yaml b/.github/workflows/ci-guide.yaml new file mode 100644 index 0000000..338f199 --- /dev/null +++ b/.github/workflows/ci-guide.yaml @@ -0,0 +1,22 @@ +name: CI Guide +on: + push: + branches: + - * + paths: + - guide/** + +jobs: + deploy: + runs-on: ubuntu-latest + steps: + - name: Setup | Checkout + uses: actions/checkout@v2 + + - name: Setup mdBook + uses: peaceiris/actions-mdbook@v1 + with: + mdbook-version: '0.4.6' + + - name: Build + run: mdbook build guide diff --git a/.github/workflows/ci-operator.yaml b/.github/workflows/ci-operator.yaml new file mode 100644 index 0000000..64da8c2 --- /dev/null +++ b/.github/workflows/ci-operator.yaml @@ -0,0 +1,16 @@ +name: CI Operator +on: + push: + branches: + - "*" + paths: + - 'hadron-operator/**' + - "!**.md" + tags-ignore: + - "*" + +jobs: + ci: + uses: hadron-project/hadron/.github/workflows/_rust-ci-base.yaml@99-compaction + with: + project: hadron-operator diff --git a/.github/workflows/ci-stream.yaml b/.github/workflows/ci-stream.yaml new file mode 100644 index 0000000..8718faf --- /dev/null +++ b/.github/workflows/ci-stream.yaml @@ -0,0 +1,16 @@ +name: CI Stream +on: + push: + branches: + - "*" + paths: + - 'hadron-stream/**' + - "!**.md" + tags-ignore: + - "*" + +jobs: + ci: + uses: hadron-project/hadron/.github/workflows/_rust-ci-base.yaml@99-compaction + with: + project: hadron-stream diff --git a/.github/workflows/ci.yaml b/.github/workflows/ci.yaml deleted file mode 100644 index 8942936..0000000 --- a/.github/workflows/ci.yaml +++ /dev/null @@ -1,15 +0,0 @@ -name: CI -on: - push: - paths-ignore: - - "**.md" - pull_request: - paths-ignore: - - "**.md" - -jobs: - build: - runs-on: ubuntu-latest - steps: - - name: - run: echo 'WIP' diff --git a/.github/workflows/pages.yaml b/.github/workflows/release-guide.yaml similarity index 100% rename from .github/workflows/pages.yaml rename to .github/workflows/release-guide.yaml diff --git a/.github/workflows/release.yaml b/.github/workflows/release.yaml deleted file mode 100644 index a38bef6..0000000 --- a/.github/workflows/release.yaml +++ /dev/null @@ -1,12 +0,0 @@ -name: Release -on: - push: - tags: - - "v*" - -jobs: - build: - runs-on: ubuntu-latest - steps: - - name: - run: echo 'WIP' diff --git a/hadron-stream/src/app.rs b/hadron-stream/src/app.rs index 569fb57..257fb14 100644 --- a/hadron-stream/src/app.rs +++ b/hadron-stream/src/app.rs @@ -55,9 +55,7 @@ impl App { let db = Database::new(config.clone()).await.context("error opening database")?; // Initialize K8s client. - let client = kube::Client::try_default() - .await - .context("error initializing K8s client")?; + let client = kube::Client::try_default().await.context("error initializing K8s client")?; // Spawn various core tasks. let (tokens, tokens_map, secrets_map) = TokensWatcher::new(client.clone(), config.clone(), shutdown_tx.subscribe()); @@ -109,14 +107,8 @@ impl App { async fn run(mut self) -> Result<()> { let mut signals = StreamMap::new(); - signals.insert( - "sigterm", - SignalStream::new(signal(SignalKind::terminate()).context("error building signal stream")?), - ); - signals.insert( - "sigint", - SignalStream::new(signal(SignalKind::interrupt()).context("error building signal stream")?), - ); + signals.insert("sigterm", SignalStream::new(signal(SignalKind::terminate()).context("error building signal stream")?)); + signals.insert("sigint", SignalStream::new(signal(SignalKind::interrupt()).context("error building signal stream")?)); loop { tokio::select! { @@ -131,36 +123,16 @@ impl App { // Begin shutdown routine. tracing::debug!("Hadron is shutting down"); - if let Err(err) = self - .tokens_handle - .await - .context("error joining token watcher handle") - .and_then(|res| res) - { + if let Err(err) = self.tokens_handle.await.context("error joining token watcher handle").and_then(|res| res) { tracing::error!(error = ?err, "error shutting down tokens watcher"); } - if let Err(err) = self - .pipelines_handle - .await - .context("error joining pipelines watcher handle") - .and_then(|res| res) - { + if let Err(err) = self.pipelines_handle.await.context("error joining pipelines watcher handle").and_then(|res| res) { tracing::error!(error = ?err, "error shutting down pipelines watcher"); } - if let Err(err) = self - .stream_handle - .await - .context("error joining stream controller handle") - .and_then(|res| res) - { + if let Err(err) = self.stream_handle.await.context("error joining stream controller handle").and_then(|res| res) { tracing::error!(error = ?err, "error shutting down stream controller"); } - if let Err(err) = self - .stream_watcher_handle - .await - .context("error joining stream CR watcher handle") - .and_then(|res| res) - { + if let Err(err) = self.stream_watcher_handle.await.context("error joining stream CR watcher handle").and_then(|res| res) { tracing::error!(error = ?err, "error shutting down stream CR watcher"); } if let Err(err) = self.client_server.await { diff --git a/hadron-stream/src/config_test.rs b/hadron-stream/src/config_test.rs index 9603e7e..9b46d1e 100644 --- a/hadron-stream/src/config_test.rs +++ b/hadron-stream/src/config_test.rs @@ -18,54 +18,19 @@ fn config_deserializes_from_full_env() -> Result<()> { ("RETENTION_POLICY_RETENTION_SECONDS".into(), "604800".into()), ])?; - assert!( - config.rust_log == "error", - "unexpected value parsed for RUST_LOG, got {}, expected {}", - config.rust_log, - "" - ); - assert!( - config.client_port == 7000, - "unexpected value parsed for CLIENT_PORT, got {}, expected {}", - config.client_port, - "7000" - ); - assert!( - config.server_port == 7001, - "unexpected value parsed for SERVER_PORT, got {}, expected {}", - config.server_port, - "7001" - ); - assert!( - config.namespace == "default", - "unexpected value parsed for NAMESPACE, got {}, expected {}", - config.namespace, - "default" - ); - assert!( - config.stream == "events", - "unexpected value parsed for STREAM, got {}, expected {}", - config.stream, - "events" - ); + assert!(config.rust_log == "error", "unexpected value parsed for RUST_LOG, got {}, expected {}", config.rust_log, ""); + assert!(config.client_port == 7000, "unexpected value parsed for CLIENT_PORT, got {}, expected {}", config.client_port, "7000"); + assert!(config.server_port == 7001, "unexpected value parsed for SERVER_PORT, got {}, expected {}", config.server_port, "7001"); + assert!(config.namespace == "default", "unexpected value parsed for NAMESPACE, got {}, expected {}", config.namespace, "default"); + assert!(config.stream == "events", "unexpected value parsed for STREAM, got {}, expected {}", config.stream, "events"); assert!( config.statefulset == "events", "unexpected value parsed for STATEFULSET, got {}, expected {}", config.statefulset, "events" ); - assert!( - config.pod_name == "events-0", - "unexpected value parsed for POD_NAME, got {}, expected {}", - config.pod_name, - "events-0" - ); - assert!( - config.partition == 0, - "unexpected value derived for partition, got {}, expected {}", - config.partition, - 0 - ); + assert!(config.pod_name == "events-0", "unexpected value parsed for POD_NAME, got {}, expected {}", config.pod_name, "events-0"); + assert!(config.partition == 0, "unexpected value derived for partition, got {}, expected {}", config.partition, 0); assert!( config.storage_data_path == "/usr/local/hadron-stream/data", "unexpected value parsed for STORAGE_DATA_PATH, got {}, expected {}", @@ -102,54 +67,19 @@ fn config_deserializes_from_sparse_env() -> Result<()> { ("RETENTION_POLICY_STRATEGY".into(), "time".into()), ])?; - assert!( - config.rust_log == "error", - "unexpected value parsed for RUST_LOG, got {}, expected {}", - config.rust_log, - "" - ); - assert!( - config.client_port == 7000, - "unexpected value parsed for CLIENT_PORT, got {}, expected {}", - config.client_port, - "7000" - ); - assert!( - config.server_port == 7001, - "unexpected value parsed for SERVER_PORT, got {}, expected {}", - config.server_port, - "7001" - ); - assert!( - config.namespace == "default", - "unexpected value parsed for NAMESPACE, got {}, expected {}", - config.namespace, - "default" - ); - assert!( - config.stream == "events", - "unexpected value parsed for STREAM, got {}, expected {}", - config.stream, - "events" - ); + assert!(config.rust_log == "error", "unexpected value parsed for RUST_LOG, got {}, expected {}", config.rust_log, ""); + assert!(config.client_port == 7000, "unexpected value parsed for CLIENT_PORT, got {}, expected {}", config.client_port, "7000"); + assert!(config.server_port == 7001, "unexpected value parsed for SERVER_PORT, got {}, expected {}", config.server_port, "7001"); + assert!(config.namespace == "default", "unexpected value parsed for NAMESPACE, got {}, expected {}", config.namespace, "default"); + assert!(config.stream == "events", "unexpected value parsed for STREAM, got {}, expected {}", config.stream, "events"); assert!( config.statefulset == "events", "unexpected value parsed for STATEFULSET, got {}, expected {}", config.statefulset, "events" ); - assert!( - config.pod_name == "events-0", - "unexpected value parsed for POD_NAME, got {}, expected {}", - config.pod_name, - "events-0" - ); - assert!( - config.partition == 0, - "unexpected value derived for partition, got {}, expected {}", - config.partition, - 0 - ); + assert!(config.pod_name == "events-0", "unexpected value parsed for POD_NAME, got {}, expected {}", config.pod_name, "events-0"); + assert!(config.partition == 0, "unexpected value derived for partition, got {}, expected {}", config.partition, 0); assert!( config.storage_data_path == "/usr/local/hadron-stream/data", "unexpected value parsed for STORAGE_DATA_PATH, got {}, expected {}", diff --git a/hadron-stream/src/database.rs b/hadron-stream/src/database.rs index 8b0ea60..a83e5a7 100644 --- a/hadron-stream/src/database.rs +++ b/hadron-stream/src/database.rs @@ -42,9 +42,7 @@ impl Database { pub async fn new(config: Arc) -> Result { // Determine the database path, and ensure it exists. let dbpath = PathBuf::from(&config.storage_data_path).join(config.pod_name.as_str()); - tokio::fs::create_dir_all(&dbpath) - .await - .context("error creating dir for hadron core database")?; + tokio::fs::create_dir_all(&dbpath).await.context("error creating dir for hadron core database")?; Self::spawn_blocking(move || -> Result { let db = SledConfig::new().path(dbpath).mode(sled::Mode::HighThroughput).open()?; @@ -62,9 +60,7 @@ impl Database { F: FnOnce() -> R + Send + 'static, R: Send + 'static, { - tokio::task::spawn_blocking(f) - .await - .map_err(|err| ShutdownError::from(anyhow::Error::from(err))) + tokio::task::spawn_blocking(f).await.map_err(|err| ShutdownError::from(anyhow::Error::from(err))) } /// Get a handle to the DB tree for a stream partition. diff --git a/hadron-stream/src/fixtures.rs b/hadron-stream/src/fixtures.rs index 4ffb672..dabd4fe 100644 --- a/hadron-stream/src/fixtures.rs +++ b/hadron-stream/src/fixtures.rs @@ -19,8 +19,7 @@ pub async fn setup_stream_data(db: &sled::Tree) -> Result<(i64, u64)> { } batch.insert(KEY_STREAM_LAST_WRITTEN_OFFSET, &utils::encode_u64(last_offset)); batch.insert(&utils::encode_byte_prefix_i64(PREFIX_STREAM_TS, ts), &utils::encode_u64(last_offset)); - db.apply_batch(batch) - .context("error applying batch to write test data to stream")?; + db.apply_batch(batch).context("error applying batch to write test data to stream")?; Ok((ts, last_offset)) } @@ -29,26 +28,18 @@ pub async fn setup_subs_data(db: &sled::Tree) -> Result let mut batch = sled::Batch::default(); let mut subs = vec![]; for offset in 0..rand::thread_rng().gen_range(50..100) { - let sub = Subscription { group_name: offset.to_string(), max_batch_size: 50 }; + let sub = Subscription { + group_name: offset.to_string(), + max_batch_size: 50, + }; let sub_encoded = utils::encode_model(&sub)?; - let sub_model_key = utils::ivec_from_iter( - PREFIX_STREAM_SUBS - .iter() - .copied() - .chain(sub.group_name.as_bytes().iter().copied()), - ); - let sub_offset_key = utils::ivec_from_iter( - PREFIX_STREAM_SUB_OFFSETS - .iter() - .copied() - .chain(sub.group_name.as_bytes().iter().copied()), - ); + let sub_model_key = utils::ivec_from_iter(PREFIX_STREAM_SUBS.iter().copied().chain(sub.group_name.as_bytes().iter().copied())); + let sub_offset_key = utils::ivec_from_iter(PREFIX_STREAM_SUB_OFFSETS.iter().copied().chain(sub.group_name.as_bytes().iter().copied())); batch.insert(sub_model_key, sub_encoded.as_slice()); batch.insert(sub_offset_key, &utils::encode_u64(offset)); subs.push((sub, offset)); } - db.apply_batch(batch) - .context("error applying batch to write test data to stream")?; + db.apply_batch(batch).context("error applying batch to write test data to stream")?; subs.sort_by(|a, b| a.1.cmp(&b.1)); Ok(subs) } diff --git a/hadron-stream/src/pipeline/mod.rs b/hadron-stream/src/pipeline/mod.rs index 9ac4956..a8221e7 100644 --- a/hadron-stream/src/pipeline/mod.rs +++ b/hadron-stream/src/pipeline/mod.rs @@ -94,8 +94,8 @@ pub struct PipelineCtl { impl PipelineCtl { /// Create a new instance. pub async fn new( - config: Arc, db: Database, pipeline: Arc, partition: u32, stream_signal: watch::Receiver, - shutdown_tx: broadcast::Sender<()>, events_tx: mpsc::Sender, events_rx: mpsc::Receiver, + config: Arc, db: Database, pipeline: Arc, partition: u32, stream_signal: watch::Receiver, shutdown_tx: broadcast::Sender<()>, events_tx: mpsc::Sender, + events_rx: mpsc::Receiver, ) -> Result { let tree = db.get_pipeline_tree(pipeline.name()).await?; let stream_tree = db.get_stream_tree().await?; @@ -140,12 +140,7 @@ impl PipelineCtl { // Check for active pipelines which need to be removed. let (events_tx, pipeline) = (self.events_tx.clone(), &self.pipeline); self.active_pipelines.retain(|offset, inst| { - if pipeline - .spec - .stages - .iter() - .all(|stage| inst.outputs.contains_key(&stage.name)) - { + if pipeline.spec.stages.iter().all(|stage| inst.outputs.contains_key(&stage.name)) { tracing::debug!(offset, "pruning old finished pipeline instance"); let (events_tx, offset) = (events_tx.clone(), *offset); tokio::spawn(async move { @@ -262,8 +257,7 @@ impl PipelineCtl { #[tracing::instrument(level = "trace", skip(self, new))] fn handle_pipeline_updated(&mut self, new: Arc) { self.pipeline = new.clone(); - self.stage_subs - .retain(|group, _| new.spec.stages.iter().any(|stage| stage.name.as_str() == group.as_str())); + self.stage_subs.retain(|group, _| new.spec.stages.iter().any(|stage| stage.name.as_str() == group.as_str())); } #[tracing::instrument(level = "trace", skip(self, new))] @@ -287,11 +281,7 @@ impl PipelineCtl { }; self.last_offset_processed = data.last_offset_processed; tracing::debug!(data.last_offset_processed, "response from pipeline data fetch"); - self.active_pipelines.extend( - data.new_pipeline_instances - .into_iter() - .map(|inst| (inst.root_event_offset, inst)), - ); + self.active_pipelines.extend(data.new_pipeline_instances.into_iter().map(|inst| (inst.root_event_offset, inst))); // Drive another delivery pass. self.execute_delivery_pass().await; @@ -330,10 +320,7 @@ impl PipelineCtl { // Add the new stage subscriber to its corresponding group. let stage_name = Arc::new(stage_name.clone()); - let group = self - .stage_subs - .entry(stage_name.clone()) - .or_insert_with(|| SubscriptionGroup::new(stage_name.clone())); + let group = self.stage_subs.entry(stage_name.clone()).or_insert_with(|| SubscriptionGroup::new(stage_name.clone())); // Roll a new ID for the channel & add it to the group's active channels. let id = Uuid::new_v4(); @@ -369,8 +356,7 @@ impl PipelineCtl { // Apply batch. let res = Database::spawn_blocking(move || -> Result<()> { - tree.apply_batch(batch) - .context("error applying batch delete on finished pipeline")?; + tree.apply_batch(batch).context("error applying batch delete on finished pipeline")?; tree.flush().context(ERR_DB_FLUSH)?; Ok(()) }) @@ -431,10 +417,7 @@ impl PipelineCtl { // If number of active instances is < the pipeline's max parallel settings, then fetch // more data from the source stream to find matching records for pipeline instantiation. - if self.active_pipelines.len() < self.max_parallel() as usize - && self.stream_offset > self.last_offset_processed - && !self.is_fetching_stream_data - { + if self.active_pipelines.len() < self.max_parallel() as usize && self.stream_offset > self.last_offset_processed && !self.is_fetching_stream_data { tracing::info!( active_pipelines = self.active_pipelines.len(), stream_offset = self.stream_offset, @@ -449,8 +432,8 @@ impl PipelineCtl { /// Deliver a payload to a stage consumer & spawn a task to await its response. #[tracing::instrument(level = "debug", skip(stage, inst, sub_group, liveness_checks, events_tx))] async fn spawn_payload_delivery( - stage: &PipelineStage, inst: &mut ActivePipelineInstance, sub_group: &mut SubscriptionGroup, - liveness_checks: &mut StreamMap, events_tx: mpsc::Sender, + stage: &PipelineStage, inst: &mut ActivePipelineInstance, sub_group: &mut SubscriptionGroup, liveness_checks: &mut StreamMap, + events_tx: mpsc::Sender, ) -> Result<()> { // Randomly select one of the available subscriptions for the stage. let chan_key_opt = sub_group @@ -492,11 +475,7 @@ impl PipelineCtl { }); // Payload is ready, send it. - let _res = chan - .0 - .send(Ok(payload)) - .await - .context("error sending pipeline delivery payload")?; + let _res = chan.0.send(Ok(payload)).await.context("error sending pipeline delivery payload")?; // Spawn off a task to await the response from the client. inst.active_deliveries.insert(sub_group.stage_name.clone(), chan_id); @@ -504,11 +483,7 @@ impl PipelineCtl { let (tx, group_name, id, offset) = (events_tx, sub_group.stage_name.clone(), chan_id, inst.root_event_offset); tokio::spawn(async move { // FUTURE: add optional timeouts here based on pipeline config. - let output = chan - .1 - .next() - .await - .map(|res| res.map_err(anyhow::Error::from).map(|data| (chan, data))); + let output = chan.1.next().await.map(|res| res.map_err(anyhow::Error::from).map(|data| (chan, data))); let _ = tx .send(PipelineCtlMsg::DeliveryResponse(DeliveryResponse { id, @@ -539,19 +514,14 @@ impl PipelineCtl { .context("response from pipeline subscription delivery dropped as stage subscription group no longer exists")?; // Unpack response body. - let res = res - .output - .context("subscriber channel closed while awaiting delivery response") - .map_err(|err| { - let _ = group.active_channels.remove(chan_id); - err - })?; - let (client_chan, body) = res - .context("error returned while awaiting subscriber delivery response") - .map_err(|err| { - let _ = group.active_channels.remove(chan_id); - err - })?; + let res = res.output.context("subscriber channel closed while awaiting delivery response").map_err(|err| { + let _ = group.active_channels.remove(chan_id); + err + })?; + let (client_chan, body) = res.context("error returned while awaiting subscriber delivery response").map_err(|err| { + let _ = group.active_channels.remove(chan_id); + err + })?; if let Some(chan_wrapper) = group.active_channels.get_mut(chan_id) { *chan_wrapper = SubChannelState::MonitoringLiveness; self.liveness_checks.insert( @@ -580,13 +550,7 @@ impl PipelineCtl { // Finally, if this was the last outstanding stage of the pipeline instance, then remove // it from the active instances set. - if self - .pipeline - .spec - .stages - .iter() - .all(|stage| inst.outputs.contains_key(&stage.name)) - { + if self.pipeline.spec.stages.iter().all(|stage| inst.outputs.contains_key(&stage.name)) { self.active_pipelines.remove(&offset); tracing::debug!(offset, "pipeline workflow finished"); let events_tx = self.events_tx.clone(); @@ -600,9 +564,7 @@ impl PipelineCtl { /// Record the ack/nack response from a subscriber delivery. #[tracing::instrument(level = "trace", skip(res, offset, stage_name, pipeline_tree))] - async fn try_record_delivery_response( - res: std::result::Result, offset: u64, stage_name: Arc, pipeline_tree: Tree, - ) -> ShutdownResult<()> { + async fn try_record_delivery_response(res: std::result::Result, offset: u64, stage_name: Arc, pipeline_tree: Tree) -> ShutdownResult<()> { let event = match res { Ok(event) => event, Err(_err) => { @@ -622,11 +584,7 @@ impl PipelineCtl { .insert(key, event_bytes.as_slice()) .context("error recording pipeline stage output on disk") .map_err(ShutdownError::from)?; - pipeline_tree - .flush_async() - .await - .context(ERR_DB_FLUSH) - .map_err(ShutdownError::from)?; + pipeline_tree.flush_async().await.context(ERR_DB_FLUSH).map_err(ShutdownError::from)?; Ok(()) } @@ -646,10 +604,7 @@ impl PipelineCtl { } #[tracing::instrument(level = "trace", skip(tree_stream, tree_pipeline, last_offset_processed, pipeline, max_parallel, tx))] - async fn try_fetch_stream_data( - tree_stream: Tree, tree_pipeline: Tree, last_offset_processed: u64, pipeline: Arc, max_parallel: u32, - tx: mpsc::Sender, - ) { + async fn try_fetch_stream_data(tree_stream: Tree, tree_pipeline: Tree, last_offset_processed: u64, pipeline: Arc, max_parallel: u32, tx: mpsc::Sender) { tracing::debug!("fetching stream data for pipeline"); let data_res = Database::spawn_blocking(move || -> Result { // Iterate over the records of the stream up to the maximum parallel allowed. @@ -719,9 +674,7 @@ impl PipelineCtl { /// - The output of each stage of a pipeline instance is recorded under `o{instance}{stage}` where /// `{instance}` is the source stream record's offset and `{stage}` is the name of the pipeline stage. #[tracing::instrument(level = "debug", skip(pipeline_tree, pipeline, stream_latest_offset))] -async fn recover_pipeline_state( - pipeline_tree: Tree, pipeline: Arc, stream_latest_offset: u64, -) -> Result<(u64, BTreeMap)> { +async fn recover_pipeline_state(pipeline_tree: Tree, pipeline: Arc, stream_latest_offset: u64) -> Result<(u64, BTreeMap)> { let val = Database::spawn_blocking(move || -> Result<(u64, BTreeMap)> { // Fetch last source stream offset to have been processed by this pipeline. let last_offset = pipeline_tree @@ -738,9 +691,9 @@ async fn recover_pipeline_state( }); // Fetch active instances. - let active_instances = pipeline_tree.scan_prefix(PREFIX_ACTIVE_INSTANCES).try_fold( - BTreeMap::new(), - |mut acc, kv_res| -> Result> { + let active_instances = pipeline_tree + .scan_prefix(PREFIX_ACTIVE_INSTANCES) + .try_fold(BTreeMap::new(), |mut acc, kv_res| -> Result> { let (key, val) = kv_res.context(ERR_ITER_FAILURE)?; let offset = utils::decode_u64(&key[1..]).context("error decoding active pipeline offset")?; let root_event: Event = utils::decode_model(&val).context("error decoding event from storage")?; @@ -762,8 +715,7 @@ async fn recover_pipeline_state( }; acc.insert(offset, inst); Ok(acc) - }, - )?; + })?; Ok((last_offset, active_instances)) }) @@ -828,7 +780,10 @@ struct SubscriptionGroup { impl SubscriptionGroup { /// Create a new instance. pub fn new(stage_name: Arc) -> Self { - Self { stage_name, active_channels: Default::default() } + Self { + stage_name, + active_channels: Default::default(), + } } } diff --git a/hadron-stream/src/server.rs b/hadron-stream/src/server.rs index 6090a47..407cf9d 100644 --- a/hadron-stream/src/server.rs +++ b/hadron-stream/src/server.rs @@ -42,8 +42,7 @@ pub struct AppServer { impl AppServer { /// Create a new instance. pub fn new( - config: Arc, pipelines: PipelinesMap, tokens: TokensMap, secrets: SecretsMap, metadata_rx: StreamMetadataRx, - shutdown: broadcast::Sender<()>, stream_tx: mpsc::Sender, + config: Arc, pipelines: PipelinesMap, tokens: TokensMap, secrets: SecretsMap, metadata_rx: StreamMetadataRx, shutdown: broadcast::Sender<()>, stream_tx: mpsc::Sender, ) -> Self { Self { config, @@ -61,11 +60,9 @@ impl AppServer { let addr = CLIENT_ADDR.parse().context("failed to parse listener address")?; let (shutdown, mut shutdown_rx) = (self.shutdown.clone(), self.shutdown.subscribe()); let service = grpc::StreamControllerServer::new(self); - let fut = Server::builder() - .add_service(service) - .serve_with_shutdown(addr, async move { - let _res = shutdown_rx.recv().await; - }); + let fut = Server::builder().add_service(service).serve_with_shutdown(addr, async move { + let _res = shutdown_rx.recv().await; + }); Ok(tokio::spawn(async move { if let Err(err) = fut.await { tracing::error!(error = ?err, "error from client gRPC server"); @@ -77,11 +74,7 @@ impl AppServer { /// Extract the given request's auth token, else fail. fn must_get_token(&self, req: &Request) -> Result { // Extract the authorization header. - let header_val = req - .metadata() - .get("authorization") - .cloned() - .ok_or(AppError::Unauthorized)?; + let header_val = req.metadata().get("authorization").cloned().ok_or(AppError::Unauthorized)?; auth::UnverifiedTokenCredentials::from_auth_header(header_val) } @@ -102,11 +95,7 @@ impl AppServer { #[allow(dead_code)] fn must_get_user(&self, req: &Request) -> Result { // Extract the authorization header. - let header_val = req - .metadata() - .get("authorization") - .cloned() - .ok_or(AppError::Unauthorized)?; + let header_val = req.metadata().get("authorization").cloned().ok_or(AppError::Unauthorized)?; auth::UserCredentials::from_auth_header(header_val) } } @@ -159,9 +148,7 @@ impl grpc::StreamController for AppServer { .send(StreamCtlMsg::RequestPublish { tx, request: request.into_inner() }) .await .map_err(|_err| AppError::grpc(anyhow!("error communicating with stream controller")))?; - let res = rx - .await - .map_err(|_err| AppError::grpc(anyhow!("error awaiting response from stream controller")))??; + let res = rx.await.map_err(|_err| AppError::grpc(anyhow!("error awaiting response from stream controller")))??; Ok(Response::new(res)) } @@ -181,46 +168,35 @@ impl grpc::StreamController for AppServer { .ok_or_else(|| Status::invalid_argument("no action variant received in request"))?; let setup: grpc::StreamSubscribeSetup = match req_action { grpc::StreamSubscribeRequestAction::Setup(setup) => setup, - _ => { - return Err(Status::invalid_argument( - "invalid action variant received in request, expected `setup` variant", - )) - } + _ => return Err(Status::invalid_argument("invalid action variant received in request, expected `setup` variant")), }; let (res_tx, res_rx) = mpsc::channel(10); self.stream_tx - .send(StreamCtlMsg::RequestSubscribe { tx: res_tx, rx: request_stream, setup }) + .send(StreamCtlMsg::RequestSubscribe { + tx: res_tx, + rx: request_stream, + setup, + }) .await .map_err(|_err| AppError::grpc(anyhow!("error communicating with stream controller")))?; Ok(Response::new(ReceiverStream::new(res_rx))) } /// Open a pipeline subscriber channel. - async fn pipeline_subscribe( - &self, request: Request>, - ) -> RpcResult> { + async fn pipeline_subscribe(&self, request: Request>) -> RpcResult> { let creds = self.must_get_token(&request).map_err(AppError::grpc)?; let (claims, _creds) = self.must_get_token_claims(creds).map_err(AppError::grpc)?; claims.check_stream_sub_auth(&self.config.stream).map_err(AppError::grpc)?; // Await initial setup payload and use it to find the target controller. let mut request_stream = request.into_inner(); - let setup_request = request_stream - .message() - .await? - .ok_or_else(|| Status::invalid_argument("no subscription setup request received"))?; + let setup_request = request_stream.message().await?.ok_or_else(|| Status::invalid_argument("no subscription setup request received"))?; let pipeline = &setup_request.pipeline; - let req_action = setup_request - .action - .ok_or_else(|| Status::invalid_argument("no action variant received in request"))?; + let req_action = setup_request.action.ok_or_else(|| Status::invalid_argument("no action variant received in request"))?; let stage_name = match req_action { grpc::PipelineSubscribeRequestAction::StageName(stage) => stage, - _ => { - return Err(Status::invalid_argument( - "invalid action variant received in request, expected `StageName` variant", - )) - } + _ => return Err(Status::invalid_argument("invalid action variant received in request, expected `StageName` variant")), }; // Find the target controller & forward the request. @@ -233,7 +209,11 @@ impl grpc::StreamController for AppServer { let (res_tx, res_rx) = mpsc::channel(10); pipeline_handle .tx - .send(PipelineCtlMsg::Request { tx: res_tx, rx: request_stream, stage_name }) + .send(PipelineCtlMsg::Request { + tx: res_tx, + rx: request_stream, + stage_name, + }) .await .map_err(|_err| AppError::grpc(anyhow!("error communicating with pipeline controller")))?; diff --git a/hadron-stream/src/stream/mod_test.rs b/hadron-stream/src/stream/mod_test.rs index b2a264a..a04fb50 100644 --- a/hadron-stream/src/stream/mod_test.rs +++ b/hadron-stream/src/stream/mod_test.rs @@ -17,16 +17,8 @@ async fn recover_stream_state_empty_state() -> Result<()> { let output = super::recover_stream_state(stream_tree).await?; - assert!( - output.last_written_offset == 0, - "expected offset to be 0 got {}", - output.last_written_offset - ); - assert!( - output.subscriptions.is_empty(), - "expected subscriptions len to be 0 got {}", - output.subscriptions.len() - ); + assert!(output.last_written_offset == 0, "expected offset to be 0 got {}", output.last_written_offset); + assert!(output.subscriptions.is_empty(), "expected subscriptions len to be 0 got {}", output.subscriptions.len()); Ok(()) } @@ -47,15 +39,8 @@ async fn recover_stream_state_with_previous_state() -> Result<()> { expected_offset, output.last_written_offset ); - assert!( - output.subscriptions.is_empty(), - "expected subscriptions len to be 0 got {}", - output.subscriptions.len() - ); - assert!( - output.first_timestamp_opt.is_some(), - "expected first_timestamp_opt to be populated, got None" - ); + assert!(output.subscriptions.is_empty(), "expected subscriptions len to be 0 got {}", output.subscriptions.len()); + assert!(output.first_timestamp_opt.is_some(), "expected first_timestamp_opt to be populated, got None"); Ok(()) } @@ -78,11 +63,7 @@ async fn recover_stream_state_with_previous_state_and_subs() -> Result<()> { expected_offset, output.last_written_offset ); - assert_eq!( - output.subscriptions, expected_subs, - "expected subscriptions to match {:?}\n{:?}", - output.subscriptions, expected_subs - ); + assert_eq!(output.subscriptions, expected_subs, "expected subscriptions to match {:?}\n{:?}", output.subscriptions, expected_subs); Ok(()) } @@ -93,21 +74,11 @@ async fn compact_stream_noop_with_empty_tree() -> Result<()> { let db = Database::new(config.clone()).await?; let stream_tree = db.get_stream_tree().await?; - let earliest_timestamp_opt = super::compact_stream(config, stream_tree.clone(), None) - .await - .context("unexpected error from compaction")?; + let earliest_timestamp_opt = super::compact_stream(config, stream_tree.clone(), None).await.context("unexpected error from compaction")?; - assert!( - earliest_timestamp_opt.is_none(), - "expected compaction to return None, got {:?}", - earliest_timestamp_opt - ); + assert!(earliest_timestamp_opt.is_none(), "expected compaction to return None, got {:?}", earliest_timestamp_opt); let count = stream_tree.scan_prefix(PREFIX_STREAM_EVENT).count(); - assert_eq!( - count, 0, - "mismatch number of stream events after compaction, got {} expected {}", - count, 0 - ); + assert_eq!(count, 0, "mismatch number of stream events after compaction, got {} expected {}", count, 0); Ok(()) } @@ -120,15 +91,9 @@ async fn compact_stream_noop_retention_policy_retain() -> Result<()> { let stream_tree = db.get_stream_tree().await?; let last_offset = fixtures::setup_stream_data(&stream_tree).await?.1; - let earliest_timestamp_opt = super::compact_stream(config, stream_tree.clone(), None) - .await - .context("unexpected error from compaction")?; + let earliest_timestamp_opt = super::compact_stream(config, stream_tree.clone(), None).await.context("unexpected error from compaction")?; - assert!( - earliest_timestamp_opt.is_none(), - "expected compaction to return None, got {:?}", - earliest_timestamp_opt - ); + assert!(earliest_timestamp_opt.is_none(), "expected compaction to return None, got {:?}", earliest_timestamp_opt); let count = stream_tree.scan_prefix(PREFIX_STREAM_EVENT).count(); assert_eq!( count, @@ -150,35 +115,18 @@ async fn compact_stream_deletes_all_data() -> Result<()> { let (last_ts, last_offset) = fixtures::setup_stream_data(&stream_tree).await?; let ts_two_weeks_ago = (time::OffsetDateTime::now_utc() - time::Duration::weeks(2)).unix_timestamp(); stream_tree - .insert( - &utils::encode_byte_prefix_i64(PREFIX_STREAM_TS, ts_two_weeks_ago), - &utils::encode_u64(last_offset), - ) + .insert(&utils::encode_byte_prefix_i64(PREFIX_STREAM_TS, ts_two_weeks_ago), &utils::encode_u64(last_offset)) .context("error inserting older timestamp index record for compaction test")?; stream_tree .remove(&utils::encode_byte_prefix_i64(PREFIX_STREAM_TS, last_ts)) .context("error removing original timestamp record for compaction test setup")?; - let earliest_timestamp_opt = super::compact_stream(config, stream_tree.clone(), None) - .await - .context("unexpected error from compaction")?; + let earliest_timestamp_opt = super::compact_stream(config, stream_tree.clone(), None).await.context("unexpected error from compaction")?; - assert!( - last_offset >= 49, - "expected at least offset 49 from fixtures::setup_stream_data, got {}", - last_offset - ); - assert!( - earliest_timestamp_opt.is_none(), - "expected compaction to return None, got {:?}", - earliest_timestamp_opt - ); + assert!(last_offset >= 49, "expected at least offset 49 from fixtures::setup_stream_data, got {}", last_offset); + assert!(earliest_timestamp_opt.is_none(), "expected compaction to return None, got {:?}", earliest_timestamp_opt); let count = stream_tree.scan_prefix(PREFIX_STREAM_EVENT).count(); - assert_eq!( - count, 0, - "mismatch number of stream events after compaction, got {} expected {}", - count, 0 - ); + assert_eq!(count, 0, "mismatch number of stream events after compaction, got {} expected {}", count, 0); Ok(()) } @@ -192,10 +140,7 @@ async fn compact_stream_deletes_only_old_data() -> Result<()> { let ts_two_weeks_ago = (time::OffsetDateTime::now_utc() - time::Duration::weeks(2)).unix_timestamp(); let old_offset = last_offset / 2; stream_tree - .insert( - &utils::encode_byte_prefix_i64(PREFIX_STREAM_TS, ts_two_weeks_ago), - &utils::encode_u64(old_offset), - ) + .insert(&utils::encode_byte_prefix_i64(PREFIX_STREAM_TS, ts_two_weeks_ago), &utils::encode_u64(old_offset)) .context("error inserting fake timestamp index record for compaction test")?; let earliest_timestamp = super::compact_stream(config, stream_tree.clone(), None) @@ -203,11 +148,7 @@ async fn compact_stream_deletes_only_old_data() -> Result<()> { .context("unexpected error from compaction")? .context("expected next earliest timestamp record to be returned")?; - assert_eq!( - earliest_timestamp.0, last_ts, - "expected earliest timestamp to be {}, got {}", - earliest_timestamp.0, last_ts - ); + assert_eq!(earliest_timestamp.0, last_ts, "expected earliest timestamp to be {}, got {}", earliest_timestamp.0, last_ts); assert_eq!( earliest_timestamp.1, last_offset, "expected earliest timestamp offset to be {}, got {}", @@ -231,9 +172,5 @@ fn calculate_initial_compaction_delay_returns_delta_under_30_min() { let expected_seconds = time::OffsetDateTime::now_utc().unix_timestamp() - (60 * 25); let expected_output = time::Duration::seconds(expected_seconds); let output = super::calculate_initial_compaction_delay(Some(60 * 25)); - assert_eq!( - output, expected_output, - "unexpected duration returned, expected {:?}, got {:?}", - expected_output, output, - ); + assert_eq!(output, expected_output, "unexpected duration returned, expected {:?}, got {:?}", expected_output, output,); } diff --git a/hadron-stream/src/stream/publisher.rs b/hadron-stream/src/stream/publisher.rs index d159201..4986a24 100644 --- a/hadron-stream/src/stream/publisher.rs +++ b/hadron-stream/src/stream/publisher.rs @@ -11,15 +11,7 @@ impl StreamCtl { pub(super) async fn handle_publisher_request(&mut self, tx: oneshot::Sender>, data: StreamPublishRequest) { // Publish the new data frame. let _write_ack = data.ack; - let offset = match Self::publish_data_frame( - &self.tree, - &mut self.current_offset, - &mut self.earliest_timestamp, - &self.offset_signal, - data, - ) - .await - { + let offset = match Self::publish_data_frame(&self.tree, &mut self.current_offset, &mut self.earliest_timestamp, &self.offset_signal, data).await { Ok(offset) => offset, Err(err) => { tracing::error!(error = ?err, "error while publishing data to stream"); @@ -43,8 +35,7 @@ impl StreamCtl { /// Publish a frame of data to the target stream, returning the offset of the last entry written. #[tracing::instrument(level = "trace", skip(tree, current_offset, offset_signal, req))] pub(super) async fn publish_data_frame( - tree: &sled::Tree, current_offset: &mut u64, earliest_timestamp: &mut Option<(i64, u64)>, offset_signal: &watch::Sender, - req: StreamPublishRequest, + tree: &sled::Tree, current_offset: &mut u64, earliest_timestamp: &mut Option<(i64, u64)>, offset_signal: &watch::Sender, req: StreamPublishRequest, ) -> Result { tracing::debug!("writing data to stream"); if req.batch.is_empty() { @@ -62,16 +53,11 @@ impl StreamCtl { } batch.insert(&utils::encode_byte_prefix_i64(PREFIX_STREAM_TS, ts), &utils::encode_u64(*current_offset)); batch.insert(KEY_STREAM_LAST_WRITTEN_OFFSET, &utils::encode_u64(*current_offset)); - tree.apply_batch(batch) - .context("error applying write batch") - .map_err(ShutdownError::from)?; + tree.apply_batch(batch).context("error applying write batch").map_err(ShutdownError::from)?; // Fsync if requested. if req.fsync { - tree.flush_async() - .await - .context(ERR_DB_FLUSH) - .map_err(ShutdownError::from)?; + tree.flush_async().await.context(ERR_DB_FLUSH).map_err(ShutdownError::from)?; } // If the earliest recorded timestamp is `None`, then update its value. diff --git a/hadron-stream/src/stream/publisher_test.rs b/hadron-stream/src/stream/publisher_test.rs index f984da5..529e8bd 100644 --- a/hadron-stream/src/stream/publisher_test.rs +++ b/hadron-stream/src/stream/publisher_test.rs @@ -23,23 +23,12 @@ async fn publish_data_frame_err_with_empty_batch() -> Result<()> { let res = super::StreamCtl::publish_data_frame(&stream_tree, &mut current_offset, &mut earliest_timestamp, &tx, req).await; let last_watcher_offset = *rx.borrow(); - assert_eq!( - last_watcher_offset, current_offset, - "expected watcher offset to be {} got {}", - current_offset, last_watcher_offset, - ); + assert_eq!(last_watcher_offset, current_offset, "expected watcher offset to be {} got {}", current_offset, last_watcher_offset,); assert!(res.is_err(), "expected an error to be returned"); let err = res.unwrap_err(); let app_err = err.downcast::().context("unexpected error type")?; - assert!( - matches!(app_err, AppError::InvalidInput(val) if val == "entries batch was empty, no-op"), - "unexpected error returned", - ); - assert!( - earliest_timestamp.is_none(), - "expected earliest_timestamp to remain `None`, got {:?}", - earliest_timestamp, - ); + assert!(matches!(app_err, AppError::InvalidInput(val) if val == "entries batch was empty, no-op"), "unexpected error returned",); + assert!(earliest_timestamp.is_none(), "expected earliest_timestamp to remain `None`, got {:?}", earliest_timestamp,); Ok(()) } @@ -54,31 +43,20 @@ async fn publish_data_frame() -> Result<()> { let expected_ts_min = time::OffsetDateTime::now_utc().unix_timestamp() - 5; let mut req = StreamPublishRequest { batch: vec![], fsync: true, ack: 0 }; - let (mut expected_events, expected_offset) = - (1u64..rand::thread_rng().gen_range(50u64..100u64)).fold((vec![], 0u64), |(mut events, _), offset| { - let event = Event::new_test(offset, "test", "empty"); - req.batch.push(event.clone()); - events.push(event); - (events, offset) - }); + let (mut expected_events, expected_offset) = (1u64..rand::thread_rng().gen_range(50u64..100u64)).fold((vec![], 0u64), |(mut events, _), offset| { + let event = Event::new_test(offset, "test", "empty"); + req.batch.push(event.clone()); + events.push(event); + (events, offset) + }); expected_events.sort_by(|a, b| a.id.cmp(&b.id)); let last_offset = super::StreamCtl::publish_data_frame(&stream_tree, &mut current_offset, &mut earliest_timestamp, &tx, req).await?; // Check emitted info on last offset. let last_watcher_offset = *rx.borrow(); - assert!( - last_offset == expected_offset, - "expected offset to be {} got {}", - expected_offset, - last_offset - ); - assert!( - last_watcher_offset == expected_offset, - "expected watcher offset to be {} got {}", - expected_offset, - last_watcher_offset - ); + assert!(last_offset == expected_offset, "expected offset to be {} got {}", expected_offset, last_offset); + assert!(last_watcher_offset == expected_offset, "expected watcher offset to be {} got {}", expected_offset, last_watcher_offset); // Check all written events. let mut events = vec![]; @@ -88,11 +66,7 @@ async fn publish_data_frame() -> Result<()> { events.push(val); } events.sort_by(|a, b| a.id.cmp(&b.id)); - assert_eq!( - events, expected_events, - "unexpected data on stream\nexpected: {:?}\ngot: {:?}", - expected_events, events, - ); + assert_eq!(events, expected_events, "unexpected data on stream\nexpected: {:?}\ngot: {:?}", expected_events, events,); // Check storage for the last offset key. let db_offset_ivec = stream_tree @@ -100,11 +74,7 @@ async fn publish_data_frame() -> Result<()> { .context("error fetching last written offset from storage")? .context("no value found for last written offset")?; let db_offset = utils::decode_u64(&db_offset_ivec)?; - assert_eq!( - db_offset, expected_offset, - "expected db last written offset to be {} got {}", - expected_offset, db_offset - ); + assert_eq!(db_offset, expected_offset, "expected db last written offset to be {} got {}", expected_offset, db_offset); // Check for secondary timestamp index. let ts_idx = stream_tree @@ -123,12 +93,7 @@ async fn publish_data_frame() -> Result<()> { "expected timestamp index entry to point to offset {} got {}", expected_offset, ts_idx[0].1 ); - assert!( - ts_idx[0].0 > expected_ts_min, - "expected stream index entry timestamp {} to be > {}", - expected_ts_min, - ts_idx[0].0 - ); + assert!(ts_idx[0].0 > expected_ts_min, "expected stream index entry timestamp {} to be > {}", expected_ts_min, ts_idx[0].0); assert!(earliest_timestamp.is_some(), "expected earliest_timestamp to be updated"); assert_eq!( earliest_timestamp.map(|val| val.0), diff --git a/hadron-stream/src/utils_test.rs b/hadron-stream/src/utils_test.rs index 16ca8a4..372ee39 100644 --- a/hadron-stream/src/utils_test.rs +++ b/hadron-stream/src/utils_test.rs @@ -46,16 +46,8 @@ async fn test_exhaustive_scan_prefix_and_range_behavior() -> Result<()> { } let key = utils::decode_u64(&key[1..])?; let val = utils::decode_u64(&val)?; - assert_eq!( - key, current_offset, - "db.range with prefix iterated out of order, expected key {} got {}", - current_offset, key - ); - assert_eq!( - val, current_offset, - "db.range with prefix iterated out of order, expected val {} got {}", - current_offset, val - ); + assert_eq!(key, current_offset, "db.range with prefix iterated out of order, expected key {} got {}", current_offset, key); + assert_eq!(val, current_offset, "db.range with prefix iterated out of order, expected val {} got {}", current_offset, val); current_offset += 1; } assert_eq!(count, NUM_ENTRIES, "expected range to find {} entries, got {}", NUM_ENTRIES, count); @@ -68,13 +60,7 @@ fn test_ivec_and_encoding_compat() { let i0 = utils::ivec_from_iter(PREFIX_B.iter().copied().chain(u64::MIN.to_be_bytes().iter().copied())); let i1 = utils::ivec_from_iter(PREFIX_B.iter().copied().chain(1u64.to_be_bytes().iter().copied())); let i2 = utils::ivec_from_iter(PREFIX_B.iter().copied().chain(10u64.to_be_bytes().iter().copied())); - let i3 = utils::ivec_from_iter( - PREFIX_B - .iter() - .copied() - .chain(utils::encode_u64(10)) - .chain("stage_name".as_bytes().iter().copied()), - ); + let i3 = utils::ivec_from_iter(PREFIX_B.iter().copied().chain(utils::encode_u64(10)).chain("stage_name".as_bytes().iter().copied())); let e0 = utils::encode_byte_prefix(PREFIX_B, u64::MIN); let e1 = utils::encode_byte_prefix(PREFIX_B, 1u64); @@ -83,13 +69,7 @@ fn test_ivec_and_encoding_compat() { assert_eq!(&i0, &e0, "ivec slice i0 is different from encoded slice:\n{:?}\n{:?}", &i0, &e0); assert_eq!(&i1, &e1, "ivec slice i1 is different from encoded slice:\n{:?}\n{:?}", &i1, &e1); assert_eq!(&i2, &e2, "ivec slice i2 is different from encoded slice:\n{:?}\n{:?}", &i2, &e2); - assert_eq!( - &i3[..9], - &e2, - "the first 9 bytes of i3 do not match the byte encoded prefix:\n{:?}\n{:?}", - &i3[..9], - &e2 - ); + assert_eq!(&i3[..9], &e2, "the first 9 bytes of i3 do not match the byte encoded prefix:\n{:?}\n{:?}", &i3[..9], &e2); assert_eq!( &i3[9..], b"stage_name", diff --git a/hadron-stream/src/watchers/pipelines.rs b/hadron-stream/src/watchers/pipelines.rs index 6721040..aec7e0f 100644 --- a/hadron-stream/src/watchers/pipelines.rs +++ b/hadron-stream/src/watchers/pipelines.rs @@ -44,9 +44,7 @@ pub struct PipelineWatcher { impl PipelineWatcher { /// Create a new instance. - pub fn new( - client: Client, config: Arc, db: Database, stream_signal: watch::Receiver, shutdown_tx: broadcast::Sender<()>, - ) -> (Self, PipelinesMap) { + pub fn new(client: Client, config: Arc, db: Database, stream_signal: watch::Receiver, shutdown_tx: broadcast::Sender<()>) -> (Self, PipelinesMap) { let pipelines: PipelinesMap = Default::default(); ( Self { @@ -120,11 +118,7 @@ impl PipelineWatcher { self.pipelines.store(Arc::new(updated)); let _res = old.tx.send(PipelineCtlMsg::PipelineDeleted(pipeline.clone())).await; if let Some(handle) = self.pipeline_handles.remove(name) { - if let Err(err) = handle - .await - .context("error joining pipeline controller handle") - .and_then(|res| res) - { + if let Err(err) = handle.await.context("error joining pipeline controller handle").and_then(|res| res) { tracing::error!(error = ?err, "error shutting down pipeline controller"); } } @@ -151,35 +145,37 @@ impl PipelineWatcher { // Pipeline already exists, so just re-index & pass along the updated model. Some((k, v)) => { let pipeline = Arc::new(pipeline); - new_pipelines.insert(k.clone(), PipelineHandle { pipeline: pipeline.clone(), tx: v.tx.clone() }); + new_pipelines.insert( + k.clone(), + PipelineHandle { + pipeline: pipeline.clone(), + tx: v.tx.clone(), + }, + ); let _res = v.tx.send(PipelineCtlMsg::PipelineUpdated(pipeline)).await; } None => { let (pipeline_name, pipeline) = (Arc::new(name.to_string()), Arc::new(pipeline)); - let events_tx = match self - .spawn_pipeline_controller(pipeline_name.clone(), pipeline.clone()) - .await - { + let events_tx = match self.spawn_pipeline_controller(pipeline_name.clone(), pipeline.clone()).await { Some(events_tx) => events_tx, None => continue, }; - new_pipelines.insert(pipeline_name.clone(), PipelineHandle { pipeline: pipeline.clone(), tx: events_tx }); + new_pipelines.insert( + pipeline_name.clone(), + PipelineHandle { + pipeline: pipeline.clone(), + tx: events_tx, + }, + ); } }; } // For any old pipelines which are about to be dropped, we need to stop them first. for (old_name, old_val) in orig.iter().filter(|(k, _v)| !new_pipelines.contains_key(k.as_ref())) { - let _res = old_val - .tx - .send(PipelineCtlMsg::PipelineDeleted(old_val.pipeline.clone())) - .await; + let _res = old_val.tx.send(PipelineCtlMsg::PipelineDeleted(old_val.pipeline.clone())).await; if let Some(handle) = self.pipeline_handles.remove(old_name) { - if let Err(err) = handle - .await - .context("error joining pipeline controller handle") - .and_then(|res| res) - { + if let Err(err) = handle.await.context("error joining pipeline controller handle").and_then(|res| res) { tracing::error!(error = ?err, "error shutting down pipeline controller"); } } @@ -215,15 +211,18 @@ impl PipelineWatcher { // Else, spawn new pipeline. let mut updated = orig.as_ref().clone(); let (pipeline_name, pipeline) = (Arc::new(name.to_string()), Arc::new(pipeline)); - let events_tx = match self - .spawn_pipeline_controller(pipeline_name.clone(), pipeline.clone()) - .await - { + let events_tx = match self.spawn_pipeline_controller(pipeline_name.clone(), pipeline.clone()).await { Some(events_tx) => events_tx, None => return, }; - updated.insert(pipeline_name.clone(), PipelineHandle { pipeline: pipeline.clone(), tx: events_tx }); + updated.insert( + pipeline_name.clone(), + PipelineHandle { + pipeline: pipeline.clone(), + tx: events_tx, + }, + ); self.pipelines.store(Arc::new(updated)); } diff --git a/hadron-stream/src/watchers/stream.rs b/hadron-stream/src/watchers/stream.rs index e01a8a1..0e48de5 100644 --- a/hadron-stream/src/watchers/stream.rs +++ b/hadron-stream/src/watchers/stream.rs @@ -38,7 +38,15 @@ impl StreamWatcher { pub fn new(client: Client, config: Arc, shutdown: broadcast::Receiver<()>) -> (Self, StreamMetadataRx) { let shutdown = BroadcastStream::new(shutdown); let (tx, rx) = watch::channel(vec![]); - (Self { client, config, shutdown, metadata: tx }, rx) + ( + Self { + client, + config, + shutdown, + metadata: tx, + }, + rx, + ) } pub fn spawn(self) -> JoinHandle> { diff --git a/hadron-stream/src/watchers/tokens.rs b/hadron-stream/src/watchers/tokens.rs index e307383..8354979 100644 --- a/hadron-stream/src/watchers/tokens.rs +++ b/hadron-stream/src/watchers/tokens.rs @@ -221,9 +221,7 @@ impl SecretWithDecodingKey { /// Extract the decoding key from the given Secret. fn extract_decoding_key_from_secret(secret: &Secret) -> Result> { let data = secret.data.as_ref().context("no data found in secret")?; - let hmac_key = data - .get(hadron_core::auth::SECRET_HMAC_KEY) - .context("no secret data found for HMAC key")?; + let hmac_key = data.get(hadron_core::auth::SECRET_HMAC_KEY).context("no secret data found for HMAC key")?; Ok(DecodingKey::from_secret(&hmac_key.0).into_static()) } diff --git a/rustfmt.toml b/rustfmt.toml index 5801e69..e74c0b0 100644 --- a/rustfmt.toml +++ b/rustfmt.toml @@ -1,6 +1,5 @@ edition = "2021" -comment_width = 100 fn_args_layout = "Compressed" max_width = 200 use_small_heuristics = "Default"