From de3e0c02ca7404b9cf26349857477ee64d02f0c2 Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Thu, 4 Jul 2024 13:16:12 +0200 Subject: [PATCH 01/60] feat(gelf): Initial implementation of chunked gelf --- Cargo.toml | 3 +- lib/codecs/Cargo.toml | 1 + .../src/decoding/framing/chunked_gelf.rs | 424 ++++++++++++++++++ lib/codecs/src/decoding/framing/mod.rs | 2 + lib/codecs/src/decoding/mod.rs | 27 ++ lib/codecs/src/encoding/format/native_json.rs | 9 +- lib/codecs/src/encoding/mod.rs | 2 + src/components/validation/resources/mod.rs | 2 + src/sources/socket/mod.rs | 39 +- src/sources/socket/udp.rs | 10 +- 10 files changed, 490 insertions(+), 29 deletions(-) create mode 100644 lib/codecs/src/decoding/framing/chunked_gelf.rs diff --git a/Cargo.toml b/Cargo.toml index 80bc84af79df9..8978aada45eb8 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -139,6 +139,7 @@ serde_json = { version = "1.0.117", default-features = false, features = ["raw_v serde = { version = "1.0.203", default-features = false, features = ["alloc", "derive", "rc"] } toml = { version = "0.8.14", default-features = false, features = ["display", "parse"] } vrl = { version = "0.16.0", features = ["arbitrary", "cli", "test", "test_framework"] } +tokio = { version = "1.38.0", default-features = false, features = ["full"] } [dependencies] pin-project.workspace = true @@ -160,7 +161,7 @@ loki-logproto = { path = "lib/loki-logproto", optional = true } async-stream = { version = "0.3.5", default-features = false } async-trait = { version = "0.1.80", default-features = false } futures = { version = "0.3.30", default-features = false, features = ["compat", "io-compat"], package = "futures" } -tokio = { version = "1.38.0", default-features = false, features = ["full"] } +tokio.workspace = true tokio-openssl = { version = "0.6.4", default-features = false } tokio-stream = { version = "0.1.15", default-features = false, features = ["net", "sync", "time"] } tokio-util = { version = "0.7", default-features = false, features = ["io", "time"] } diff --git a/lib/codecs/Cargo.toml b/lib/codecs/Cargo.toml index 6f1d9ee7e39a1..014a7b691d866 100644 --- a/lib/codecs/Cargo.toml +++ b/lib/codecs/Cargo.toml @@ -29,6 +29,7 @@ smallvec = { version = "1", default-features = false, features = ["union"] } snafu = { version = "0.7.5", default-features = false, features = ["futures"] } syslog_loose = { version = "0.21", default-features = false, optional = true } tokio-util = { version = "0.7", default-features = false, features = ["codec"] } +tokio.workspace = true tracing = { version = "0.1", default-features = false } vrl.workspace = true vector-common = { path = "../vector-common", default-features = false } diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs new file mode 100644 index 0000000000000..3bc18f0cdeb85 --- /dev/null +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -0,0 +1,424 @@ +use super::BoxedFramingError; +use bytes::{Buf, Bytes, BytesMut}; +use derivative::Derivative; +use std::collections::HashMap; +use std::sync::{Arc, Mutex}; +use std::time::Duration; +use tokio; +use tokio_util::codec::Decoder; +use tracing::warn; +use vector_config::configurable_component; + +const GELF_MAGIC: [u8; 2] = [0x1e, 0x0f]; +const MAX_TOTAL_CHUNKS: u8 = 128; +const DEFAULT_CHUNKS: [Bytes; MAX_TOTAL_CHUNKS as usize] = + [const { Bytes::new() }; MAX_TOTAL_CHUNKS as usize]; +const DEFAULT_TIMEOUT_MILLIS: u64 = 5000; + +/// Config used to build a `ChunkedGelfDecoderConfig`. +#[configurable_component] +#[derive(Debug, Clone, Default, PartialEq, Eq)] +pub struct ChunkedGelfDecoderConfig { + /// Options for the chunked gelf decoder. + #[serde(default, skip_serializing_if = "vector_core::serde::is_default")] + pub chunked_gelf: ChunkedGelfDecoderOptions, +} + +impl ChunkedGelfDecoderConfig { + /// Creates a new `BytesDecoderConfig`. + pub fn new() -> Self { + Default::default() + } + + /// Build the `ByteDecoder` from this configuration. + pub fn build(&self) -> ChunkedGelfDecoder { + ChunkedGelfDecoder::new(self.chunked_gelf.timeout_millis) + } +} + +const fn default_timeout_millis() -> u64 { + DEFAULT_TIMEOUT_MILLIS +} + +/// Options for building a `ChunkedGelfDecoder`. +#[configurable_component] +#[derive(Clone, Debug, Derivative, PartialEq, Eq)] +pub struct ChunkedGelfDecoderOptions { + /// The timeout in milliseconds for a message to be fully received. + #[serde( + default = "default_timeout_millis", + skip_serializing_if = "vector_core::serde::is_default" + )] + pub timeout_millis: u64, +} + +impl Default for ChunkedGelfDecoderOptions { + fn default() -> Self { + Self { + timeout_millis: default_timeout_millis(), + } + } +} + +/// A decoder for handling GELF messages that are chunked. +// TODO: manual implement clone, it is not okay to clone the Arc as it is, we should create a new decoder +#[derive(Debug, Clone)] +pub struct ChunkedGelfDecoder { + /// TODO + state: Arc>>, + timeout: Duration, +} + +#[derive(Debug, Clone)] +pub struct MessageState { + total_chunks: u8, + chunks: [Bytes; MAX_TOTAL_CHUNKS as usize], + chunks_bitmap: u128, +} + +impl MessageState { + pub fn new(total_chunks: u8) -> Self { + Self { + total_chunks, + chunks: DEFAULT_CHUNKS, + chunks_bitmap: 0, + } + } +} + +impl ChunkedGelfDecoder { + /// Creates a new `ChunkedGelfDecoder`. + pub fn new(timeout_millis: u64) -> Self { + Self { + state: Arc::new(Mutex::new(HashMap::new())), + timeout: Duration::from_millis(timeout_millis), + } + } + + /// TODO: document this + pub fn decode_chunk( + &mut self, + src: &mut bytes::BytesMut, + ) -> Result, BoxedFramingError> { + // TODO: handle malformed and do not panic + let message_id = src.get_u64(); + let sequence_number = src.get_u8(); + let total_chunks = src.get_u8(); + + // TODO: Warn and do not panic + assert!(total_chunks <= MAX_TOTAL_CHUNKS); + assert!(sequence_number < total_chunks); + + // TODO: handle this unwrap + let mut state_lock = self.state.lock().unwrap(); + let message_state = state_lock.entry(message_id).or_insert_with(|| { + // TODO: we need tokio due to the sleep function. We need to spawn a task that will clear the message state after a certain time + // otherwise we will have a memory leak + // let timeout = self.timeout.clone(); + let state = Arc::clone(&self.state); + let timeout = self.timeout.clone(); + tokio::spawn(async move { + tokio::time::sleep(timeout).await; + let mut state_lock = state.lock().unwrap(); + if let Some(_) = state_lock.remove(&message_id) { + warn!("Message with id {message_id} was not fully received within the timeout window of {}ms. Discarding it.",timeout.as_millis()); + } + }); + MessageState::new(total_chunks) + }); + + if message_state.total_chunks != total_chunks { + // TODO: improve logging + warn!("Received a chunk with a different total_chunks than the original. Ignoring it."); + src.clear(); + return Ok(None); + } + + let chunk_bitmap_id = 1 << sequence_number; + if message_state.chunks_bitmap & chunk_bitmap_id != 0 { + // TOOD: improve logging + warn!("Received a duplicate chunk. Ignoring it."); + src.clear(); + return Ok(None); + } + + let chunk = src.split().freeze(); + message_state.chunks[sequence_number as usize] = chunk; + message_state.chunks_bitmap |= chunk_bitmap_id; + + if message_state.chunks_bitmap.count_ones() == message_state.total_chunks as u32 { + let chunks = &message_state.chunks[0..message_state.total_chunks as usize]; + let mut message = BytesMut::new(); + for chunk in chunks { + message.extend_from_slice(chunk); + } + state_lock.remove(&message_id); + Ok(Some(message.freeze())) + } else { + Ok(None) + } + } +} + +impl Default for ChunkedGelfDecoder { + fn default() -> Self { + Self::new(DEFAULT_TIMEOUT_MILLIS) + } +} + +impl Decoder for ChunkedGelfDecoder { + type Item = Bytes; + + type Error = BoxedFramingError; + + fn decode(&mut self, src: &mut bytes::BytesMut) -> Result, Self::Error> { + if src.is_empty() { + return Ok(None); + } + + let magic = src.get(0..2); + if magic.is_some_and(|magic| magic == GELF_MAGIC) { + src.advance(2); + self.decode_chunk(src) + } else { + // The gelf message is not chunked + let frame = src.split(); + return Ok(Some(frame.freeze())); + } + } + + // TODO: implement decode_eof +} + +#[cfg(test)] +mod tests { + use super::*; + use bytes::{BufMut, BytesMut}; + use rstest::{fixture, rstest}; + + fn create_chunk( + message_id: u64, + sequence_number: u8, + total_chunks: u8, + payload: &str, + ) -> Bytes { + let mut chunk = BytesMut::new(); + chunk.put_slice(&GELF_MAGIC); + chunk.put_u64(message_id); + chunk.put_u8(sequence_number); + chunk.put_u8(total_chunks); + chunk.extend_from_slice(payload.as_bytes()); + chunk.freeze() + } + + #[fixture] + fn unchunked_message() -> (Bytes, String) { + let payload = "foo"; + (Bytes::from(payload), payload.to_string()) + } + + // TODO: add a malformed chunk message + + #[fixture] + fn two_chunks_message() -> ([Bytes; 2], String) { + let message_id = 1u64; + let total_chunks = 2u8; + + let first_sequence_number = 0u8; + let first_payload = "foo"; + let first_chunk = create_chunk( + message_id, + first_sequence_number, + total_chunks, + first_payload, + ); + + let second_sequence_number = 1u8; + let second_payload = "bar"; + let second_chunk = create_chunk( + message_id, + second_sequence_number, + total_chunks, + second_payload, + ); + + ( + [first_chunk, second_chunk], + format!("{first_payload}{second_payload}"), + ) + } + + #[fixture] + fn three_chunks_message() -> ([Bytes; 3], String) { + let message_id = 2u64; + let total_chunks = 3u8; + + let first_sequence_number = 0u8; + let first_payload = "foo"; + let first_chunk = create_chunk( + message_id, + first_sequence_number, + total_chunks, + first_payload, + ); + + let second_sequence_number = 1u8; + let second_payload = "bar"; + let second_chunk = create_chunk( + message_id, + second_sequence_number, + total_chunks, + second_payload, + ); + + let third_sequence_number = 2u8; + let third_payload = "baz"; + let third_chunk = create_chunk( + message_id, + third_sequence_number, + total_chunks, + third_payload, + ); + + ( + [first_chunk, second_chunk, third_chunk], + format!("{first_payload}{second_payload}{third_payload}"), + ) + } + + #[rstest] + #[tokio::test] + async fn decode_chunked(two_chunks_message: ([Bytes; 2], String)) { + let mut src = BytesMut::new(); + let (chunks, expected_message) = two_chunks_message; + let mut decoder = ChunkedGelfDecoder::default(); + + src.extend_from_slice(&chunks[0]); + let frame = decoder.decode(&mut src).unwrap(); + assert!(frame.is_none()); + + src.extend_from_slice(&chunks[1]); + let frame = decoder.decode(&mut src).unwrap(); + + assert_eq!(frame, Some(Bytes::from(expected_message))); + } + + #[rstest] + #[tokio::test] + async fn decode_unchunked(unchunked_message: (Bytes, String)) { + let mut src = BytesMut::new(); + let (message, expected_message) = unchunked_message; + let mut decoder = ChunkedGelfDecoder::default(); + + src.extend_from_slice(&message); + let frame = decoder.decode(&mut src).unwrap(); + assert_eq!(frame, Some(Bytes::from(expected_message))); + } + + #[rstest] + #[tokio::test] + async fn decode_unordered_chunks(two_chunks_message: ([Bytes; 2], String)) { + let mut src = BytesMut::new(); + let (chunks, expected_message) = two_chunks_message; + let mut decoder = ChunkedGelfDecoder::default(); + + src.extend_from_slice(&chunks[1]); + let frame = decoder.decode(&mut src).unwrap(); + assert!(frame.is_none()); + + src.extend_from_slice(&chunks[0]); + let frame = decoder.decode(&mut src).unwrap(); + + assert_eq!(frame, Some(Bytes::from(expected_message))); + } + + #[rstest] + #[tokio::test] + async fn decode_unordered_messages( + two_chunks_message: ([Bytes; 2], String), + three_chunks_message: ([Bytes; 3], String), + ) { + let mut src = BytesMut::new(); + let (two_chunks, two_chunks_expected) = two_chunks_message; + let (three_chunks, three_chunks_expected) = three_chunks_message; + let mut decoder = ChunkedGelfDecoder::default(); + + src.extend_from_slice(&three_chunks[2]); + let frame = decoder.decode(&mut src).unwrap(); + assert!(frame.is_none()); + + src.extend_from_slice(&two_chunks[0]); + let frame = decoder.decode(&mut src).unwrap(); + assert!(frame.is_none()); + + src.extend_from_slice(&three_chunks[0]); + let frame = decoder.decode(&mut src).unwrap(); + assert!(frame.is_none()); + + src.extend_from_slice(&two_chunks[1]); + let frame = decoder.decode(&mut src).unwrap(); + assert_eq!(frame, Some(Bytes::from(two_chunks_expected))); + + src.extend_from_slice(&three_chunks[1]); + let frame = decoder.decode(&mut src).unwrap(); + assert_eq!(frame, Some(Bytes::from(three_chunks_expected))); + } + + #[rstest] + #[tokio::test] + async fn decode_mixed_chunked_and_unchunked_messages( + unchunked_message: (Bytes, String), + two_chunks_message: ([Bytes; 2], String), + ) { + let mut src = BytesMut::new(); + let (unchunked_message, expected_unchunked_message) = unchunked_message; + let (chunks, expected_chunked_message) = two_chunks_message; + let mut decoder = ChunkedGelfDecoder::default(); + + src.extend_from_slice(&chunks[1]); + let frame = decoder.decode(&mut src).unwrap(); + assert!(frame.is_none()); + + src.extend_from_slice(&unchunked_message); + let frame = decoder.decode(&mut src).unwrap(); + assert_eq!(frame, Some(Bytes::from(expected_unchunked_message))); + + src.extend_from_slice(&chunks[0]); + let frame = decoder.decode(&mut src).unwrap(); + assert_eq!(frame, Some(Bytes::from(expected_chunked_message))); + } + + #[rstest] + #[tokio::test(start_paused = true)] + async fn decode_timeout(two_chunks_message: ([Bytes; 2], String)) { + let timeout = 300; + let mut src = BytesMut::new(); + let (chunks, _) = two_chunks_message; + let mut decoder = ChunkedGelfDecoder::new(timeout); + + src.extend_from_slice(&chunks[0]); + let frame = decoder.decode(&mut src).unwrap(); + assert!(frame.is_none()); + assert!(!decoder.state.lock().unwrap().is_empty()); + + // The message state should be cleared after a certain time + tokio::time::sleep(Duration::from_millis(timeout + 1)).await; + assert!(decoder.state.lock().unwrap().is_empty()); + + src.extend_from_slice(&chunks[1]); + let frame = decoder.decode(&mut src).unwrap(); + assert!(frame.is_none()); + + tokio::time::sleep(Duration::from_millis(timeout + 1)).await; + assert!(decoder.state.lock().unwrap().is_empty()); + } + + #[tokio::test] + async fn decode_empty_input() { + let mut src = BytesMut::new(); + let mut decoder = ChunkedGelfDecoder::default(); + + let frame = decoder.decode(&mut src).unwrap(); + assert!(frame.is_none()); + } +} diff --git a/lib/codecs/src/decoding/framing/mod.rs b/lib/codecs/src/decoding/framing/mod.rs index f991b969575a6..e998f02abf525 100644 --- a/lib/codecs/src/decoding/framing/mod.rs +++ b/lib/codecs/src/decoding/framing/mod.rs @@ -5,6 +5,7 @@ mod bytes; mod character_delimited; +mod chunked_gelf; mod length_delimited; mod newline_delimited; mod octet_counting; @@ -15,6 +16,7 @@ use ::bytes::Bytes; pub use character_delimited::{ CharacterDelimitedDecoder, CharacterDelimitedDecoderConfig, CharacterDelimitedDecoderOptions, }; +pub use chunked_gelf::{ChunkedGelfDecoder, ChunkedGelfDecoderConfig}; use dyn_clone::DynClone; pub use length_delimited::{LengthDelimitedDecoder, LengthDelimitedDecoderConfig}; pub use newline_delimited::{ diff --git a/lib/codecs/src/decoding/mod.rs b/lib/codecs/src/decoding/mod.rs index 2db0dfcfcb327..09f00606af826 100644 --- a/lib/codecs/src/decoding/mod.rs +++ b/lib/codecs/src/decoding/mod.rs @@ -24,6 +24,7 @@ pub use framing::{ NewlineDelimitedDecoderConfig, NewlineDelimitedDecoderOptions, OctetCountingDecoder, OctetCountingDecoderConfig, OctetCountingDecoderOptions, }; +use framing::{ChunkedGelfDecoder, ChunkedGelfDecoderConfig}; use smallvec::SmallVec; use std::fmt::Debug; use vector_config::configurable_component; @@ -98,6 +99,11 @@ pub enum FramingConfig { /// /// [octet_counting]: https://tools.ietf.org/html/rfc6587#section-3.4.1 OctetCounting(OctetCountingDecoderConfig), + + /// Byte frames which are chunked GELF messages. + /// + /// [chunked_gelf]: https://go2docs.graylog.org/current/getting_in_log_data/gelf.html + ChunkedGelf(ChunkedGelfDecoderConfig), } impl From for FramingConfig { @@ -129,6 +135,11 @@ impl From for FramingConfig { Self::OctetCounting(config) } } +impl From for FramingConfig { + fn from(config: ChunkedGelfDecoderConfig) -> Self { + Self::ChunkedGelf(config) + } +} impl FramingConfig { /// Build the `Framer` from this configuration. @@ -139,6 +150,7 @@ impl FramingConfig { FramingConfig::LengthDelimited(config) => Framer::LengthDelimited(config.build()), FramingConfig::NewlineDelimited(config) => Framer::NewlineDelimited(config.build()), FramingConfig::OctetCounting(config) => Framer::OctetCounting(config.build()), + FramingConfig::ChunkedGelf(config) => Framer::ChunkedGelf(config.build()), } } } @@ -158,6 +170,8 @@ pub enum Framer { OctetCounting(OctetCountingDecoder), /// Uses an opaque `Framer` implementation for framing. Boxed(BoxedFramer), + /// Uses a `ChunkedGelfDecoder` for framing. + ChunkedGelf(ChunkedGelfDecoder), } impl tokio_util::codec::Decoder for Framer { @@ -172,6 +186,7 @@ impl tokio_util::codec::Decoder for Framer { Framer::NewlineDelimited(framer) => framer.decode(src), Framer::OctetCounting(framer) => framer.decode(src), Framer::Boxed(framer) => framer.decode(src), + Framer::ChunkedGelf(framer) => framer.decode(src), } } @@ -183,6 +198,7 @@ impl tokio_util::codec::Decoder for Framer { Framer::NewlineDelimited(framer) => framer.decode_eof(src), Framer::OctetCounting(framer) => framer.decode_eof(src), Framer::Boxed(framer) => framer.decode_eof(src), + Framer::ChunkedGelf(framer) => framer.decode_eof(src), } } } @@ -338,6 +354,8 @@ impl DeserializerConfig { | DeserializerConfig::NativeJson(_) => { FramingConfig::NewlineDelimited(Default::default()) } + // TODO: the default framing of the gelf codec should be chunked gelf? + // or only with the udp input? Why is the newline delimited used for gelf? gelf is not newline delimited DeserializerConfig::Protobuf(_) => FramingConfig::Bytes, #[cfg(feature = "syslog")] DeserializerConfig::Syslog(_) => FramingConfig::NewlineDelimited(Default::default()), @@ -345,6 +363,15 @@ impl DeserializerConfig { } } + /// Returns an appropriate default framing config for the given deserializer with message based inputs. + /// This is only relevant for the GELF codec (or any chunked codec) with datagram inputs, such as udp sockest or unix datagram sockets. + pub fn default_message_based_framing(&self) -> FramingConfig { + match self { + DeserializerConfig::Gelf(_) => FramingConfig::ChunkedGelf(Default::default()), + _ => FramingConfig::Bytes, + } + } + /// Return the type of event build by this deserializer. pub fn output_type(&self) -> DataType { match self { diff --git a/lib/codecs/src/encoding/format/native_json.rs b/lib/codecs/src/encoding/format/native_json.rs index 90464bfcc0631..7e4543e2063c6 100644 --- a/lib/codecs/src/encoding/format/native_json.rs +++ b/lib/codecs/src/encoding/format/native_json.rs @@ -98,7 +98,12 @@ mod tests { serializer .encode(histogram_event.clone(), &mut bytes) .unwrap(); - let json = serializer.to_json_value(histogram_event).unwrap(); - assert_eq!(bytes.freeze(), serde_json::to_string(&json).unwrap()); + // TODO: remove this comment. Add a PR comment stating that converting the histogram_event to a serde_json::Value + // Changed the order of the keys to alphabetical order and the string comparison failed due to the Event struct + // not serializing the keys in the same order as the serde_json::Value. + assert_eq!( + bytes.freeze(), + serde_json::to_string(&histogram_event).unwrap() + ); } } diff --git a/lib/codecs/src/encoding/mod.rs b/lib/codecs/src/encoding/mod.rs index 0d766f73e4d17..c19c8123d847d 100644 --- a/lib/codecs/src/encoding/mod.rs +++ b/lib/codecs/src/encoding/mod.rs @@ -550,3 +550,5 @@ impl tokio_util::codec::Encoder for Serializer { } } } + +// TODO: add tests here for GELF chunked framing and parsing diff --git a/src/components/validation/resources/mod.rs b/src/components/validation/resources/mod.rs index feb90716a899d..7fa8bbac88e84 100644 --- a/src/components/validation/resources/mod.rs +++ b/src/components/validation/resources/mod.rs @@ -196,6 +196,8 @@ fn decoder_framing_to_encoding_framer(framing: &decoding::FramingConfig) -> enco // TODO: There's no equivalent octet counting framer for encoding... although // there's no particular reason that would make it hard to write. decoding::FramingConfig::OctetCounting(_) => todo!(), + // TODO: chunked gelf is not supported yet in encoding + decoding::FramingConfig::ChunkedGelf(_) => todo!(), }; framing_config.build() diff --git a/src/sources/socket/mod.rs b/src/sources/socket/mod.rs index bcd32859d0c20..404e7bbb4253a 100644 --- a/src/sources/socket/mod.rs +++ b/src/sources/socket/mod.rs @@ -10,7 +10,6 @@ use vector_lib::lookup::{lookup_v2::OptionalValuePath, owned_value_path}; use vrl::value::{kind::Collection, Kind}; #[cfg(unix)] -use crate::serde::default_framing_message_based; use crate::{ codecs::DecodingConfig, config::{GenerateConfig, Resource, SourceConfig, SourceContext, SourceOutput}, @@ -152,12 +151,12 @@ impl SourceConfig for SocketConfig { } Mode::Udp(config) => { let log_namespace = cx.log_namespace(config.log_namespace); - let decoder = DecodingConfig::new( - config.framing().clone(), - config.decoding().clone(), - log_namespace, - ) - .build()?; + let decoding = config.decoding().clone(); + let framing = config + .framing() + .clone() + .unwrap_or_else(|| decoding.default_message_based_framing()); + let decoder = DecodingConfig::new(framing, decoding, log_namespace).build()?; Ok(udp::udp( config, decoder, @@ -168,16 +167,14 @@ impl SourceConfig for SocketConfig { } #[cfg(unix)] Mode::UnixDatagram(config) => { + // TODO: test for unix datagram with chunked gelf let log_namespace = cx.log_namespace(config.log_namespace); - let decoder = DecodingConfig::new( - config - .framing - .clone() - .unwrap_or_else(default_framing_message_based), - config.decoding.clone(), - log_namespace, - ) - .build()?; + let decoding = config.decoding.clone(); + let framing = config + .framing + .clone() + .unwrap_or_else(|| decoding.default_message_based_framing()); + let decoder = DecodingConfig::new(framing, decoding, log_namespace).build()?; unix::unix_datagram(config, decoder, cx.shutdown, cx.out, log_namespace) } @@ -1059,10 +1056,12 @@ mod test { let address = next_addr(); let mut config = UdpConfig::from_address(address.into()); config.max_length = 10; - config.framing = CharacterDelimitedDecoderConfig { - character_delimited: CharacterDelimitedDecoderOptions::new(b',', None), - } - .into(); + config.framing = Some( + CharacterDelimitedDecoderConfig { + character_delimited: CharacterDelimitedDecoderOptions::new(b',', None), + } + .into(), + ); let address = init_udp_with_config(tx, config).await; send_lines_udp( diff --git a/src/sources/socket/udp.rs b/src/sources/socket/udp.rs index 8b80fff217211..b4acddd38b70c 100644 --- a/src/sources/socket/udp.rs +++ b/src/sources/socket/udp.rs @@ -23,7 +23,7 @@ use crate::{ SocketBindError, SocketEventsReceived, SocketMode, SocketReceiveError, StreamClosedError, }, net, - serde::{default_decoding, default_framing_message_based}, + serde::default_decoding, shutdown::ShutdownSignal, sources::{ socket::SocketConfig, @@ -75,8 +75,7 @@ pub struct UdpConfig { receive_buffer_bytes: Option, #[configurable(derived)] - #[serde(default = "default_framing_message_based")] - pub(super) framing: FramingConfig, + pub(super) framing: Option, #[configurable(derived)] #[serde(default = "default_decoding")] @@ -109,7 +108,7 @@ impl UdpConfig { &self.port_key } - pub(super) const fn framing(&self) -> &FramingConfig { + pub(super) const fn framing(&self) -> &Option { &self.framing } @@ -128,7 +127,7 @@ impl UdpConfig { host_key: default_host_key(), port_key: default_port_key(), receive_buffer_bytes: None, - framing: default_framing_message_based(), + framing: None, decoding: default_decoding(), log_namespace: None, } @@ -173,7 +172,6 @@ pub(super) fn udp( let bytes_received = register!(BytesReceived::from(Protocol::UDP)); info!(message = "Listening.", address = %config.address); - // We add 1 to the max_length in order to determine if the received data has been truncated. let mut buf = BytesMut::with_capacity(max_length + 1); loop { From c6e32048dafcbb18ca425c99765fc51dfca53ff8 Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Thu, 4 Jul 2024 18:40:32 +0200 Subject: [PATCH 02/60] feat(gelf): improve error handling when decoding --- Cargo.lock | 1 + Cargo.toml | 1 + lib/codecs/Cargo.toml | 1 + .../src/decoding/framing/chunked_gelf.rs | 173 ++++++++++++++---- lib/dnsmsg-parser/Cargo.toml | 2 +- 5 files changed, 139 insertions(+), 39 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 648b956c23475..bdfd62e17b45c 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2110,6 +2110,7 @@ dependencies = [ "smallvec", "snafu 0.7.5", "syslog_loose", + "thiserror", "tokio", "tokio-util", "tracing 0.1.40", diff --git a/Cargo.toml b/Cargo.toml index 8978aada45eb8..eaaa4f687069a 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -140,6 +140,7 @@ serde = { version = "1.0.203", default-features = false, features = ["alloc", "d toml = { version = "0.8.14", default-features = false, features = ["display", "parse"] } vrl = { version = "0.16.0", features = ["arbitrary", "cli", "test", "test_framework"] } tokio = { version = "1.38.0", default-features = false, features = ["full"] } +thiserror = "1.0" [dependencies] pin-project.workspace = true diff --git a/lib/codecs/Cargo.toml b/lib/codecs/Cargo.toml index 014a7b691d866..78718b5f71aa1 100644 --- a/lib/codecs/Cargo.toml +++ b/lib/codecs/Cargo.toml @@ -37,6 +37,7 @@ vector-config = { path = "../vector-config", default-features = false } vector-config-common = { path = "../vector-config-common", default-features = false } vector-config-macros = { path = "../vector-config-macros", default-features = false } vector-core = { path = "../vector-core", default-features = false, features = ["vrl"] } +thiserror.workspace = true [dev-dependencies] futures = { version = "0.3", default-features = false } diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index 3bc18f0cdeb85..94d1022e2e9d9 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -1,12 +1,16 @@ -use super::BoxedFramingError; +use crate::StreamDecodingError; + +use super::{BoxedFramingError, FramingError}; use bytes::{Buf, Bytes, BytesMut}; use derivative::Derivative; use std::collections::HashMap; use std::sync::{Arc, Mutex}; use std::time::Duration; +use thiserror::Error; use tokio; +use tokio::task::JoinHandle; use tokio_util::codec::Decoder; -use tracing::warn; +use tracing::{info, warn}; use vector_config::configurable_component; const GELF_MAGIC: [u8; 2] = [0x1e, 0x0f]; @@ -60,28 +64,88 @@ impl Default for ChunkedGelfDecoderOptions { } } -/// A decoder for handling GELF messages that are chunked. -// TODO: manual implement clone, it is not okay to clone the Arc as it is, we should create a new decoder -#[derive(Debug, Clone)] -pub struct ChunkedGelfDecoder { - /// TODO - state: Arc>>, - timeout: Duration, -} - -#[derive(Debug, Clone)] +#[derive(Debug)] pub struct MessageState { total_chunks: u8, chunks: [Bytes; MAX_TOTAL_CHUNKS as usize], chunks_bitmap: u128, + timeout_task: JoinHandle<()>, } impl MessageState { - pub fn new(total_chunks: u8) -> Self { + pub fn new(total_chunks: u8, timeout_task: JoinHandle<()>) -> Self { Self { total_chunks, chunks: DEFAULT_CHUNKS, chunks_bitmap: 0, + timeout_task, + } + } + + pub fn is_chunk_present(&self, sequence_number: u8) -> bool { + let chunk_bitmap_id = 1 << sequence_number; + self.chunks_bitmap & chunk_bitmap_id != 0 + } + + pub fn add_chunk(&mut self, sequence_number: u8, chunk: Bytes) { + let chunk_bitmap_id = 1 << sequence_number; + self.chunks[sequence_number as usize] = chunk; + self.chunks_bitmap |= chunk_bitmap_id; + } + + pub fn is_complete(&self) -> bool { + self.chunks_bitmap.count_ones() == self.total_chunks as u32 + } + + pub fn retrieve_message(&mut self) -> Option { + if self.is_complete() { + self.timeout_task.abort(); + let chunks = &self.chunks[0..self.total_chunks as usize]; + let mut message = BytesMut::new(); + for chunk in chunks { + message.extend_from_slice(chunk); + } + Some(message.freeze()) + } else { + None + } + } +} + +#[derive(Debug, Error)] +pub enum ChunkedGelfDecoderError { + #[error("Poisoned lock")] + PoisonedLock, +} + +impl From for BoxedFramingError { + fn from(error: ChunkedGelfDecoderError) -> Self { + Box::new(error) + } +} +impl StreamDecodingError for ChunkedGelfDecoderError { + fn can_continue(&self) -> bool { + match self { + ChunkedGelfDecoderError::PoisonedLock => false, + } + } +} + +impl FramingError for ChunkedGelfDecoderError {} + +/// A decoder for handling GELF messages that are chunked. +#[derive(Debug)] +pub struct ChunkedGelfDecoder { + /// TODO + state: Arc>>, + timeout: Duration, +} + +impl Clone for ChunkedGelfDecoder { + fn clone(&self) -> Self { + Self { + state: Arc::new(Mutex::new(HashMap::new())), + timeout: self.timeout, } } } @@ -99,61 +163,94 @@ impl ChunkedGelfDecoder { pub fn decode_chunk( &mut self, src: &mut bytes::BytesMut, - ) -> Result, BoxedFramingError> { - // TODO: handle malformed and do not panic + ) -> Result, ChunkedGelfDecoderError> { + // We need 10 bits to read the message id, sequence number and total chunks + if src.remaining() < 10 { + let src_display = format!("{src:?}"); + warn!(message = "Received malformed chunk headers (message ID, sequence number and total chunks) with less than 10 bytes. Ignoring it.", + src = src_display, + remaining = src.remaining(), + internal_log_rate_limit = true); + src.clear(); + return Ok(None); + } let message_id = src.get_u64(); let sequence_number = src.get_u8(); let total_chunks = src.get_u8(); - // TODO: Warn and do not panic - assert!(total_chunks <= MAX_TOTAL_CHUNKS); - assert!(sequence_number < total_chunks); + if total_chunks == 0 || total_chunks > MAX_TOTAL_CHUNKS { + warn!( + message = "Received a chunk with an invalid total chunks value. Ignoring it.", + message_id = message_id, + sequence_number = sequence_number, + total_chunks = total_chunks, + internal_log_rate_limit = true + ); + src.clear(); + return Ok(None); + } + + if sequence_number >= total_chunks { + warn!( + message = "Received a chunk with a sequence number greater than total chunks. Ignoring it.", + message_id = message_id, + sequence_number = sequence_number, + total_chunks = total_chunks, + internal_log_rate_limit = true + ); + src.clear(); + return Ok(None); + } // TODO: handle this unwrap - let mut state_lock = self.state.lock().unwrap(); + let Ok(mut state_lock) = self.state.lock() else { + return Err(ChunkedGelfDecoderError::PoisonedLock); + }; + let message_state = state_lock.entry(message_id).or_insert_with(|| { // TODO: we need tokio due to the sleep function. We need to spawn a task that will clear the message state after a certain time // otherwise we will have a memory leak - // let timeout = self.timeout.clone(); let state = Arc::clone(&self.state); let timeout = self.timeout.clone(); - tokio::spawn(async move { + let timeout_handle = tokio::spawn(async move { tokio::time::sleep(timeout).await; let mut state_lock = state.lock().unwrap(); if let Some(_) = state_lock.remove(&message_id) { - warn!("Message with id {message_id} was not fully received within the timeout window of {}ms. Discarding it.",timeout.as_millis()); + let message = format!("Message with id {message_id} was not fully received within the timeout window of {}ms. Discarding it.", timeout.as_millis()); + warn!(message = message, internal_log_rate_limit = true); } }); - MessageState::new(total_chunks) + MessageState::new(total_chunks, timeout_handle) }); if message_state.total_chunks != total_chunks { - // TODO: improve logging - warn!("Received a chunk with a different total_chunks than the original. Ignoring it."); + warn!(message_id = "Received a chunk with a different total_chunks than the original. Ignoring it.", + original_total_chunks = message_state.total_chunks, + received_total_chunks = total_chunks, + internal_log_rate_limit = true, + message_id = message_id); src.clear(); return Ok(None); } - let chunk_bitmap_id = 1 << sequence_number; - if message_state.chunks_bitmap & chunk_bitmap_id != 0 { + if message_state.is_chunk_present(sequence_number) { // TOOD: improve logging - warn!("Received a duplicate chunk. Ignoring it."); + info!( + message = "Received a duplicate chunk. Ignoring it.", + sequence_number = sequence_number, + message_id = message_id, + internal_log_rate_limit = true + ); src.clear(); return Ok(None); } let chunk = src.split().freeze(); - message_state.chunks[sequence_number as usize] = chunk; - message_state.chunks_bitmap |= chunk_bitmap_id; + message_state.add_chunk(sequence_number, chunk); - if message_state.chunks_bitmap.count_ones() == message_state.total_chunks as u32 { - let chunks = &message_state.chunks[0..message_state.total_chunks as usize]; - let mut message = BytesMut::new(); - for chunk in chunks { - message.extend_from_slice(chunk); - } + if let Some(message) = message_state.retrieve_message() { state_lock.remove(&message_id); - Ok(Some(message.freeze())) + Ok(Some(message)) } else { Ok(None) } @@ -179,7 +276,7 @@ impl Decoder for ChunkedGelfDecoder { let magic = src.get(0..2); if magic.is_some_and(|magic| magic == GELF_MAGIC) { src.advance(2); - self.decode_chunk(src) + Ok(self.decode_chunk(src)?) } else { // The gelf message is not chunked let frame = src.split(); diff --git a/lib/dnsmsg-parser/Cargo.toml b/lib/dnsmsg-parser/Cargo.toml index 4e29a718e69b0..5aeb6fbacb1e3 100644 --- a/lib/dnsmsg-parser/Cargo.toml +++ b/lib/dnsmsg-parser/Cargo.toml @@ -7,8 +7,8 @@ publish = false license = "MIT" [dependencies] +thiserror.workspace = true data-encoding = "2.6" -thiserror = "1.0" hickory-proto = { version = "0.24", features = ["dnssec"] } [dev-dependencies] From 5a1b49fabd83c55419d195297066c038634af22e Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Thu, 4 Jul 2024 18:42:59 +0200 Subject: [PATCH 03/60] feat(gelf): improve error handling when decoding --- lib/codecs/src/decoding/framing/chunked_gelf.rs | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index 94d1022e2e9d9..515fbe114b42c 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -216,8 +216,16 @@ impl ChunkedGelfDecoder { tokio::time::sleep(timeout).await; let mut state_lock = state.lock().unwrap(); if let Some(_) = state_lock.remove(&message_id) { - let message = format!("Message with id {message_id} was not fully received within the timeout window of {}ms. Discarding it.", timeout.as_millis()); - warn!(message = message, internal_log_rate_limit = true); + let message = format!( + "Message was not fully received within the timeout window. Discarding it." + ); + // TODO: log the variables in the message or use structured logging? + warn!( + message = message, + message_id = message_id, + timeout = timeout.as_millis(), + internal_log_rate_limit = true + ); } }); MessageState::new(total_chunks, timeout_handle) @@ -234,7 +242,6 @@ impl ChunkedGelfDecoder { } if message_state.is_chunk_present(sequence_number) { - // TOOD: improve logging info!( message = "Received a duplicate chunk. Ignoring it.", sequence_number = sequence_number, From b85235ec38c823fa5ae3373c2319e94c68819b38 Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Fri, 5 Jul 2024 11:09:07 +0200 Subject: [PATCH 04/60] test(gelf): Add more tests for chunked gelf --- Cargo.lock | 23 ++ lib/codecs/Cargo.toml | 1 + .../src/decoding/framing/chunked_gelf.rs | 198 ++++++++++++++++-- 3 files changed, 200 insertions(+), 22 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index bdfd62e17b45c..f408f51ae3890 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2114,6 +2114,7 @@ dependencies = [ "tokio", "tokio-util", "tracing 0.1.40", + "tracing-test", "uuid", "vector-common", "vector-config", @@ -9984,6 +9985,7 @@ dependencies = [ "serde", "serde_json", "sharded-slab", + "smallvec", "thread_local", "tracing 0.1.40", "tracing-core 0.1.32", @@ -9991,6 +9993,27 @@ dependencies = [ "tracing-serde", ] +[[package]] +name = "tracing-test" +version = "0.2.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "557b891436fe0d5e0e363427fc7f217abf9ccd510d5136549847bdcbcd011d68" +dependencies = [ + "tracing-core 0.1.32", + "tracing-subscriber", + "tracing-test-macro", +] + +[[package]] +name = "tracing-test-macro" +version = "0.2.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "04659ddb06c87d233c566112c1c9c5b9e98256d9af50ec3bc9c8327f873a7568" +dependencies = [ + "quote 1.0.36", + "syn 2.0.66", +] + [[package]] name = "tracing-tower" version = "0.1.0" diff --git a/lib/codecs/Cargo.toml b/lib/codecs/Cargo.toml index 78718b5f71aa1..baf3565d2d7d0 100644 --- a/lib/codecs/Cargo.toml +++ b/lib/codecs/Cargo.toml @@ -47,6 +47,7 @@ similar-asserts = "1.5.0" vector-core = { path = "../vector-core", default-features = false, features = ["vrl", "test"] } uuid = { version = "1", default-features = false, features = ["serde", "v4"] } rstest = "0.21.0" +tracing-test = "0.2.5" vrl.workspace = true [features] diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index 515fbe114b42c..5a1face30aa48 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -18,6 +18,15 @@ const MAX_TOTAL_CHUNKS: u8 = 128; const DEFAULT_CHUNKS: [Bytes; MAX_TOTAL_CHUNKS as usize] = [const { Bytes::new() }; MAX_TOTAL_CHUNKS as usize]; const DEFAULT_TIMEOUT_MILLIS: u64 = 5000; +const DEFAULT_PENDING_MESSAGES_LIMIT: usize = 1000; + +const fn default_timeout_millis() -> u64 { + DEFAULT_TIMEOUT_MILLIS +} + +const fn default_pending_messages_limit() -> usize { + DEFAULT_PENDING_MESSAGES_LIMIT +} /// Config used to build a `ChunkedGelfDecoderConfig`. #[configurable_component] @@ -36,30 +45,39 @@ impl ChunkedGelfDecoderConfig { /// Build the `ByteDecoder` from this configuration. pub fn build(&self) -> ChunkedGelfDecoder { - ChunkedGelfDecoder::new(self.chunked_gelf.timeout_millis) + ChunkedGelfDecoder::new( + self.chunked_gelf.timeout_millis, + self.chunked_gelf.pending_messages_limit, + ) } } -const fn default_timeout_millis() -> u64 { - DEFAULT_TIMEOUT_MILLIS -} - /// Options for building a `ChunkedGelfDecoder`. #[configurable_component] #[derive(Clone, Debug, Derivative, PartialEq, Eq)] pub struct ChunkedGelfDecoderOptions { - /// The timeout in milliseconds for a message to be fully received. + /// The timeout in milliseconds for a message to be fully received. If the timeout is reached, the + /// decoder will drop all the received chunks of the message and start over. #[serde( default = "default_timeout_millis", skip_serializing_if = "vector_core::serde::is_default" )] pub timeout_millis: u64, + + /// The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + /// dropping chunks of new messages. + #[serde( + default = "default_pending_messages_limit", + skip_serializing_if = "vector_core::serde::is_default" + )] + pub pending_messages_limit: usize, } impl Default for ChunkedGelfDecoderOptions { fn default() -> Self { Self { timeout_millis: default_timeout_millis(), + pending_messages_limit: default_pending_messages_limit(), } } } @@ -139,6 +157,7 @@ pub struct ChunkedGelfDecoder { /// TODO state: Arc>>, timeout: Duration, + pending_messages_limit: usize, } impl Clone for ChunkedGelfDecoder { @@ -146,16 +165,18 @@ impl Clone for ChunkedGelfDecoder { Self { state: Arc::new(Mutex::new(HashMap::new())), timeout: self.timeout, + pending_messages_limit: self.pending_messages_limit, } } } impl ChunkedGelfDecoder { /// Creates a new `ChunkedGelfDecoder`. - pub fn new(timeout_millis: u64) -> Self { + pub fn new(timeout_millis: u64, pending_messages_limit: usize) -> Self { Self { state: Arc::new(Mutex::new(HashMap::new())), timeout: Duration::from_millis(timeout_millis), + pending_messages_limit, } } @@ -171,7 +192,6 @@ impl ChunkedGelfDecoder { src = src_display, remaining = src.remaining(), internal_log_rate_limit = true); - src.clear(); return Ok(None); } let message_id = src.get_u64(); @@ -186,7 +206,6 @@ impl ChunkedGelfDecoder { total_chunks = total_chunks, internal_log_rate_limit = true ); - src.clear(); return Ok(None); } @@ -198,15 +217,25 @@ impl ChunkedGelfDecoder { total_chunks = total_chunks, internal_log_rate_limit = true ); - src.clear(); return Ok(None); } - // TODO: handle this unwrap let Ok(mut state_lock) = self.state.lock() else { return Err(ChunkedGelfDecoderError::PoisonedLock); }; + if state_lock.len() >= self.pending_messages_limit { + warn!( + message = "Received a chunk but reached the pending messages limit. Ignoring it.", + message_id = message_id, + sequence_number = sequence_number, + total_chunks = total_chunks, + pending_messages_limit = self.pending_messages_limit, + internal_log_rate_limit = true + ); + return Ok(None); + } + let message_state = state_lock.entry(message_id).or_insert_with(|| { // TODO: we need tokio due to the sleep function. We need to spawn a task that will clear the message state after a certain time // otherwise we will have a memory leak @@ -216,12 +245,9 @@ impl ChunkedGelfDecoder { tokio::time::sleep(timeout).await; let mut state_lock = state.lock().unwrap(); if let Some(_) = state_lock.remove(&message_id) { - let message = format!( - "Message was not fully received within the timeout window. Discarding it." - ); // TODO: log the variables in the message or use structured logging? warn!( - message = message, + message = "Message was not fully received within the timeout window. Discarding it.", message_id = message_id, timeout = timeout.as_millis(), internal_log_rate_limit = true @@ -232,12 +258,12 @@ impl ChunkedGelfDecoder { }); if message_state.total_chunks != total_chunks { - warn!(message_id = "Received a chunk with a different total_chunks than the original. Ignoring it.", + warn!(message_id = "Received a chunk with a different total chunks than the original. Ignoring it.", original_total_chunks = message_state.total_chunks, received_total_chunks = total_chunks, internal_log_rate_limit = true, - message_id = message_id); - src.clear(); + message_id = message_id + ); return Ok(None); } @@ -248,7 +274,6 @@ impl ChunkedGelfDecoder { message_id = message_id, internal_log_rate_limit = true ); - src.clear(); return Ok(None); } @@ -266,7 +291,7 @@ impl ChunkedGelfDecoder { impl Default for ChunkedGelfDecoder { fn default() -> Self { - Self::new(DEFAULT_TIMEOUT_MILLIS) + Self::new(DEFAULT_TIMEOUT_MILLIS, DEFAULT_PENDING_MESSAGES_LIMIT) } } @@ -283,7 +308,9 @@ impl Decoder for ChunkedGelfDecoder { let magic = src.get(0..2); if magic.is_some_and(|magic| magic == GELF_MAGIC) { src.advance(2); - Ok(self.decode_chunk(src)?) + let frame = self.decode_chunk(src)?; + src.clear(); + return Ok(frame); } else { // The gelf message is not chunked let frame = src.split(); @@ -299,6 +326,7 @@ mod tests { use super::*; use bytes::{BufMut, BytesMut}; use rstest::{fixture, rstest}; + use tracing_test::traced_test; fn create_chunk( message_id: u64, @@ -494,11 +522,12 @@ mod tests { #[rstest] #[tokio::test(start_paused = true)] + #[traced_test] async fn decode_timeout(two_chunks_message: ([Bytes; 2], String)) { let timeout = 300; let mut src = BytesMut::new(); let (chunks, _) = two_chunks_message; - let mut decoder = ChunkedGelfDecoder::new(timeout); + let mut decoder = ChunkedGelfDecoder::new(timeout, DEFAULT_PENDING_MESSAGES_LIMIT); src.extend_from_slice(&chunks[0]); let frame = decoder.decode(&mut src).unwrap(); @@ -508,6 +537,9 @@ mod tests { // The message state should be cleared after a certain time tokio::time::sleep(Duration::from_millis(timeout + 1)).await; assert!(decoder.state.lock().unwrap().is_empty()); + assert!(logs_contain( + "Message was not fully received within the timeout window. Discarding it." + )); src.extend_from_slice(&chunks[1]); let frame = decoder.decode(&mut src).unwrap(); @@ -525,4 +557,126 @@ mod tests { let frame = decoder.decode(&mut src).unwrap(); assert!(frame.is_none()); } + + #[tokio::test] + #[traced_test] + async fn decode_chunk_with_malformed_header() { + let malformed_chunk = [0x12, 0x23]; + let mut src = BytesMut::new(); + let mut decoder = ChunkedGelfDecoder::default(); + + src.extend_from_slice(&GELF_MAGIC); + src.extend_from_slice(&malformed_chunk); + let frame = decoder.decode(&mut src).unwrap(); + assert!(frame.is_none()); + assert!(logs_contain("Received malformed chunk headers (message ID, sequence number and total chunks) with less than 10 bytes. Ignoring it.")); + } + + #[tokio::test] + #[traced_test] + async fn decode_chunk_with_invalid_total_chunks() { + let message_id = 1u64; + let sequence_number = 1u8; + let invalid_total_chunks = MAX_TOTAL_CHUNKS + 1; + let payload = "foo"; + let chunk = create_chunk(message_id, sequence_number, invalid_total_chunks, payload); + let mut src = BytesMut::new(); + let mut decoder = ChunkedGelfDecoder::default(); + + src.extend_from_slice(&chunk); + let frame = decoder.decode(&mut src).unwrap(); + assert!(frame.is_none()); + assert!(logs_contain( + "Received a chunk with an invalid total chunks value. Ignoring it." + )); + } + + #[tokio::test] + #[traced_test] + async fn decode_chunk_with_invalid_sequence_number() { + let message_id = 1u64; + let total_chunks = 2u8; + let invalid_sequence_number = total_chunks + 1; + let payload = "foo"; + let chunk = create_chunk(message_id, invalid_sequence_number, total_chunks, payload); + let mut src = BytesMut::new(); + let mut decoder = ChunkedGelfDecoder::default(); + + src.extend_from_slice(&chunk); + let frame = decoder.decode(&mut src).unwrap(); + assert!(frame.is_none()); + assert!(logs_contain( + "Received a chunk with a sequence number greater than total chunks. Ignoring it." + )); + } + + #[rstest] + #[tokio::test] + #[traced_test] + async fn decode_when_reached_pending_messages_limit( + two_chunks_message: ([Bytes; 2], String), + three_chunks_message: ([Bytes; 3], String), + ) { + let pending_messages_limit = 1; + let mut src = BytesMut::new(); + let (two_chunks, _) = two_chunks_message; + let (three_chunks, _) = three_chunks_message; + let mut decoder = ChunkedGelfDecoder::new(DEFAULT_TIMEOUT_MILLIS, pending_messages_limit); + + src.extend_from_slice(&two_chunks[0]); + let frame = decoder.decode(&mut src).unwrap(); + assert!(frame.is_none()); + assert!(decoder.state.lock().unwrap().len() == 1); + + src.extend_from_slice(&three_chunks[0]); + let frame = decoder.decode(&mut src).unwrap(); + assert!(frame.is_none()); + assert!(decoder.state.lock().unwrap().len() == 1); + assert!(logs_contain( + "Received a chunk but reached the pending messages limit. Ignoring it." + )); + } + + #[rstest] + #[tokio::test] + #[traced_test] + async fn decode_chunk_with_different_total_chunks() { + let message_id = 1u64; + let sequence_number = 0u8; + let total_chunks = 2u8; + let payload = "foo"; + let first_chunk = create_chunk(message_id, sequence_number, total_chunks, payload); + let second_chunk = create_chunk(message_id, sequence_number + 1, total_chunks + 1, payload); + let mut src = BytesMut::new(); + let mut decoder = ChunkedGelfDecoder::default(); + + src.extend_from_slice(&first_chunk); + let frame = decoder.decode(&mut src).unwrap(); + assert!(frame.is_none()); + + src.extend_from_slice(&second_chunk); + let frame = decoder.decode(&mut src).unwrap(); + assert!(frame.is_none()); + assert!(logs_contain( + "Received a chunk with a different total chunks than the original. Ignoring it." + )); + } + + #[rstest] + #[tokio::test] + #[traced_test] + async fn decode_when_duplicated_chunk(two_chunks_message: ([Bytes; 2], String)) { + let mut src = BytesMut::new(); + let (chunks, _) = two_chunks_message; + let mut decoder = ChunkedGelfDecoder::default(); + + src.extend_from_slice(&chunks[0]); + let frame = decoder.decode(&mut src).unwrap(); + assert!(frame.is_none()); + + src.extend_from_slice(&chunks[0]); + let frame = decoder.decode(&mut src).unwrap(); + assert!(frame.is_none()); + assert!(logs_contain("Received a duplicate chunk. Ignoring it.")); + } } From baccabac41142bab7115b5c9c634327adcebbce5 Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Fri, 5 Jul 2024 12:35:35 +0200 Subject: [PATCH 05/60] feat(gelf): derive clone for chunked gelf decoder --- lib/codecs/src/decoding/framing/chunked_gelf.rs | 15 +++------------ 1 file changed, 3 insertions(+), 12 deletions(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index 5a1face30aa48..cdb904479093e 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -151,8 +151,8 @@ impl StreamDecodingError for ChunkedGelfDecoderError { impl FramingError for ChunkedGelfDecoderError {} -/// A decoder for handling GELF messages that are chunked. -#[derive(Debug)] +/// A decoder for handling GELF messages that may be chunked. +#[derive(Debug, Clone)] pub struct ChunkedGelfDecoder { /// TODO state: Arc>>, @@ -160,16 +160,6 @@ pub struct ChunkedGelfDecoder { pending_messages_limit: usize, } -impl Clone for ChunkedGelfDecoder { - fn clone(&self) -> Self { - Self { - state: Arc::new(Mutex::new(HashMap::new())), - timeout: self.timeout, - pending_messages_limit: self.pending_messages_limit, - } - } -} - impl ChunkedGelfDecoder { /// Creates a new `ChunkedGelfDecoder`. pub fn new(timeout_millis: u64, pending_messages_limit: usize) -> Self { @@ -268,6 +258,7 @@ impl ChunkedGelfDecoder { } if message_state.is_chunk_present(sequence_number) { + // TODO: add a PR comment asking for info or warn info!( message = "Received a duplicate chunk. Ignoring it.", sequence_number = sequence_number, From 148e09be5333f665e51444a569603123d3f319e2 Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Fri, 5 Jul 2024 12:38:17 +0200 Subject: [PATCH 06/60] docs(gelf): Improve doc comments --- lib/codecs/src/decoding/mod.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/lib/codecs/src/decoding/mod.rs b/lib/codecs/src/decoding/mod.rs index 09f00606af826..f9f16505fc27d 100644 --- a/lib/codecs/src/decoding/mod.rs +++ b/lib/codecs/src/decoding/mod.rs @@ -364,7 +364,6 @@ impl DeserializerConfig { } /// Returns an appropriate default framing config for the given deserializer with message based inputs. - /// This is only relevant for the GELF codec (or any chunked codec) with datagram inputs, such as udp sockest or unix datagram sockets. pub fn default_message_based_framing(&self) -> FramingConfig { match self { DeserializerConfig::Gelf(_) => FramingConfig::ChunkedGelf(Default::default()), From 530f2434c7b52b4b9c86957479f5bb322b4b7de1 Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Fri, 5 Jul 2024 12:48:21 +0200 Subject: [PATCH 07/60] feat(gelf): improve mutex poisioning handling --- .../src/decoding/framing/chunked_gelf.rs | 31 +++---------------- 1 file changed, 4 insertions(+), 27 deletions(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index cdb904479093e..91f4c019bd0d2 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -130,27 +130,6 @@ impl MessageState { } } -#[derive(Debug, Error)] -pub enum ChunkedGelfDecoderError { - #[error("Poisoned lock")] - PoisonedLock, -} - -impl From for BoxedFramingError { - fn from(error: ChunkedGelfDecoderError) -> Self { - Box::new(error) - } -} -impl StreamDecodingError for ChunkedGelfDecoderError { - fn can_continue(&self) -> bool { - match self { - ChunkedGelfDecoderError::PoisonedLock => false, - } - } -} - -impl FramingError for ChunkedGelfDecoderError {} - /// A decoder for handling GELF messages that may be chunked. #[derive(Debug, Clone)] pub struct ChunkedGelfDecoder { @@ -174,7 +153,7 @@ impl ChunkedGelfDecoder { pub fn decode_chunk( &mut self, src: &mut bytes::BytesMut, - ) -> Result, ChunkedGelfDecoderError> { + ) -> Result, BoxedFramingError> { // We need 10 bits to read the message id, sequence number and total chunks if src.remaining() < 10 { let src_display = format!("{src:?}"); @@ -210,9 +189,7 @@ impl ChunkedGelfDecoder { return Ok(None); } - let Ok(mut state_lock) = self.state.lock() else { - return Err(ChunkedGelfDecoderError::PoisonedLock); - }; + let mut state_lock = self.state.lock().unwrap(); if state_lock.len() >= self.pending_messages_limit { warn!( @@ -299,9 +276,9 @@ impl Decoder for ChunkedGelfDecoder { let magic = src.get(0..2); if magic.is_some_and(|magic| magic == GELF_MAGIC) { src.advance(2); - let frame = self.decode_chunk(src)?; + let frame = self.decode_chunk(src); src.clear(); - return Ok(frame); + return frame; } else { // The gelf message is not chunked let frame = src.split(); From 2bc10a725629c93ec492a34dab9147d5f8e00991 Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Fri, 5 Jul 2024 12:59:20 +0200 Subject: [PATCH 08/60] feat(gelf): improve mutex poisioning handling --- Cargo.lock | 1 - Cargo.toml | 1363 ++++++++++++----- lib/codecs/Cargo.toml | 23 +- .../src/decoding/framing/chunked_gelf.rs | 5 +- lib/dnsmsg-parser/Cargo.toml | 2 +- 5 files changed, 993 insertions(+), 401 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index f408f51ae3890..b66d694b44110 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2110,7 +2110,6 @@ dependencies = [ "smallvec", "snafu 0.7.5", "syslog_loose", - "thiserror", "tokio", "tokio-util", "tracing 0.1.40", diff --git a/Cargo.toml b/Cargo.toml index eaaa4f687069a..e2d50a4a37129 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -9,7 +9,7 @@ license = "MPL-2.0" readme = "README.md" publish = false default-run = "vector" -autobenches = false # our benchmarks are not runnable on their own either way +autobenches = false # our benchmarks are not runnable on their own either way # Minimum supported rust version # See docs/DEVELOPING.md for policy rust-version = "1.78" @@ -56,14 +56,46 @@ section = "admin" maintainer-scripts = "distribution/debian/scripts/" conf-files = ["/etc/vector/vector.yaml", "/etc/default/vector"] assets = [ - ["target/release/vector", "/usr/bin/", "755"], - ["config/vector.yaml", "/etc/vector/vector.yaml", "644"], - ["config/examples/*", "/etc/vector/examples/", "644"], - ["distribution/systemd/vector.service", "/lib/systemd/system/vector.service", "644"], - ["distribution/systemd/vector.default", "/etc/default/vector", "600"], - ["licenses/*", "/usr/share/vector/licenses/", "644"], - ["NOTICE", "/usr/share/vector/NOTICE", "644"], - ["LICENSE-3rdparty.csv", "/usr/share/vector/LICENSE-3rdparty.csv", "644"], + [ + "target/release/vector", + "/usr/bin/", + "755", + ], + [ + "config/vector.yaml", + "/etc/vector/vector.yaml", + "644", + ], + [ + "config/examples/*", + "/etc/vector/examples/", + "644", + ], + [ + "distribution/systemd/vector.service", + "/lib/systemd/system/vector.service", + "644", + ], + [ + "distribution/systemd/vector.default", + "/etc/default/vector", + "600", + ], + [ + "licenses/*", + "/usr/share/vector/licenses/", + "644", + ], + [ + "NOTICE", + "/usr/share/vector/NOTICE", + "644", + ], + [ + "LICENSE-3rdparty.csv", + "/usr/share/vector/LICENSE-3rdparty.csv", + "644", + ], ] license-file = ["target/debian-license.txt"] extended-description-file = "target/debian-extended-description.txt" @@ -97,50 +129,79 @@ depends = "" [workspace] members = [ - ".", - "lib/codecs", - "lib/dnsmsg-parser", - "lib/docs-renderer", - "lib/enrichment", - "lib/fakedata", - "lib/file-source", - "lib/k8s-e2e-tests", - "lib/k8s-test-framework", - "lib/loki-logproto", - "lib/portpicker", - "lib/prometheus-parser", - "lib/opentelemetry-proto", - "lib/tracing-limit", - "lib/vector-api-client", - "lib/vector-buffers", - "lib/vector-common", - "lib/vector-config", - "lib/vector-config-common", - "lib/vector-config-macros", - "lib/vector-core", - "lib/vector-lib", - "lib/vector-lookup", - "lib/vector-stream", - "lib/vector-vrl/cli", - "lib/vector-vrl/functions", - "lib/vector-vrl/tests", - "lib/vector-vrl/web-playground", - "vdev", + ".", + "lib/codecs", + "lib/dnsmsg-parser", + "lib/docs-renderer", + "lib/enrichment", + "lib/fakedata", + "lib/file-source", + "lib/k8s-e2e-tests", + "lib/k8s-test-framework", + "lib/loki-logproto", + "lib/portpicker", + "lib/prometheus-parser", + "lib/opentelemetry-proto", + "lib/tracing-limit", + "lib/vector-api-client", + "lib/vector-buffers", + "lib/vector-common", + "lib/vector-config", + "lib/vector-config-common", + "lib/vector-config-macros", + "lib/vector-core", + "lib/vector-lib", + "lib/vector-lookup", + "lib/vector-stream", + "lib/vector-vrl/cli", + "lib/vector-vrl/functions", + "lib/vector-vrl/tests", + "lib/vector-vrl/web-playground", + "vdev", ] [workspace.dependencies] -chrono = { version = "0.4.37", default-features = false, features = ["clock", "serde"] } -clap = { version = "4.5.7", default-features = false, features = ["derive", "error-context", "env", "help", "std", "string", "usage", "wrap_help"] } -indexmap = { version = "2.2.6", default-features = false, features = ["serde", "std"] } +chrono = { version = "0.4.37", default-features = false, features = [ + "clock", + "serde", +] } +clap = { version = "4.5.7", default-features = false, features = [ + "derive", + "error-context", + "env", + "help", + "std", + "string", + "usage", + "wrap_help", +] } +indexmap = { version = "2.2.6", default-features = false, features = [ + "serde", + "std", +] } pin-project = { version = "1.1.5", default-features = false } proptest = { version = "1.4" } proptest-derive = { version = "0.4.0" } -serde_json = { version = "1.0.117", default-features = false, features = ["raw_value", "std"] } -serde = { version = "1.0.203", default-features = false, features = ["alloc", "derive", "rc"] } -toml = { version = "0.8.14", default-features = false, features = ["display", "parse"] } -vrl = { version = "0.16.0", features = ["arbitrary", "cli", "test", "test_framework"] } +serde_json = { version = "1.0.117", default-features = false, features = [ + "raw_value", + "std", +] } +serde = { version = "1.0.203", default-features = false, features = [ + "alloc", + "derive", + "rc", +] } +toml = { version = "0.8.14", default-features = false, features = [ + "display", + "parse", +] } +vrl = { version = "0.16.0", features = [ + "arbitrary", + "cli", + "test", + "test_framework", +] } tokio = { version = "1.38.0", default-features = false, features = ["full"] } -thiserror = "1.0" [dependencies] pin-project.workspace = true @@ -154,25 +215,46 @@ dnsmsg-parser = { path = "lib/dnsmsg-parser", optional = true } fakedata = { path = "lib/fakedata", optional = true } portpicker = { path = "lib/portpicker" } tracing-limit = { path = "lib/tracing-limit" } -vector-lib = { path = "lib/vector-lib", default-features = false, features = ["vrl"] } +vector-lib = { path = "lib/vector-lib", default-features = false, features = [ + "vrl", +] } vector-vrl-functions = { path = "lib/vector-vrl/functions" } loki-logproto = { path = "lib/loki-logproto", optional = true } # Tokio / Futures async-stream = { version = "0.3.5", default-features = false } async-trait = { version = "0.1.80", default-features = false } -futures = { version = "0.3.30", default-features = false, features = ["compat", "io-compat"], package = "futures" } +futures = { version = "0.3.30", default-features = false, features = [ + "compat", + "io-compat", +], package = "futures" } tokio.workspace = true tokio-openssl = { version = "0.6.4", default-features = false } -tokio-stream = { version = "0.1.15", default-features = false, features = ["net", "sync", "time"] } -tokio-util = { version = "0.7", default-features = false, features = ["io", "time"] } +tokio-stream = { version = "0.1.15", default-features = false, features = [ + "net", + "sync", + "time", +] } +tokio-util = { version = "0.7", default-features = false, features = [ + "io", + "time", +] } console-subscriber = { version = "0.3.0", default-features = false, optional = true } # Tracing tracing = { version = "0.1.34", default-features = false } tracing-core = { version = "0.1.26", default-features = false } -tracing-futures = { version = "0.2.5", default-features = false, features = ["futures-03"] } -tracing-subscriber = { version = "0.3.18", default-features = false, features = ["ansi", "env-filter", "fmt", "json", "registry", "tracing-log"] } +tracing-futures = { version = "0.2.5", default-features = false, features = [ + "futures-03", +] } +tracing-subscriber = { version = "0.3.18", default-features = false, features = [ + "ansi", + "env-filter", + "fmt", + "json", + "registry", + "tracing-log", +] } tracing-tower = { git = "https://github.com/tokio-rs/tracing", default-features = false, rev = "e0642d949891546a3bb7e47080365ee7274f05cd" } # Metrics @@ -180,52 +262,113 @@ metrics = "0.21.1" metrics-tracing-context = { version = "0.14.0", default-features = false } # AWS - Official SDK -aws-sdk-s3 = { version = "1.4.0", default-features = false, features = ["behavior-version-latest"], optional = true } -aws-sdk-sqs = { version = "1.3.0", default-features = false, features = ["behavior-version-latest"], optional = true } -aws-sdk-sns = { version = "1.3.0", default-features = false, features = ["behavior-version-latest"], optional = true } -aws-sdk-cloudwatch = { version = "1.3.0", default-features = false, features = ["behavior-version-latest"], optional = true } -aws-sdk-cloudwatchlogs = { version = "1.3.0", default-features = false, features = ["behavior-version-latest"], optional = true } -aws-sdk-elasticsearch = { version = "1.3.0", default-features = false, features = ["behavior-version-latest"], optional = true } -aws-sdk-firehose = { version = "1.3.0", default-features = false, features = ["behavior-version-latest"], optional = true } -aws-sdk-kinesis = { version = "1.3.0", default-features = false, features = ["behavior-version-latest"], optional = true } -aws-sdk-secretsmanager = { version = "1.3.0", default-features = false, features = ["behavior-version-latest"], optional = true } +aws-sdk-s3 = { version = "1.4.0", default-features = false, features = [ + "behavior-version-latest", +], optional = true } +aws-sdk-sqs = { version = "1.3.0", default-features = false, features = [ + "behavior-version-latest", +], optional = true } +aws-sdk-sns = { version = "1.3.0", default-features = false, features = [ + "behavior-version-latest", +], optional = true } +aws-sdk-cloudwatch = { version = "1.3.0", default-features = false, features = [ + "behavior-version-latest", +], optional = true } +aws-sdk-cloudwatchlogs = { version = "1.3.0", default-features = false, features = [ + "behavior-version-latest", +], optional = true } +aws-sdk-elasticsearch = { version = "1.3.0", default-features = false, features = [ + "behavior-version-latest", +], optional = true } +aws-sdk-firehose = { version = "1.3.0", default-features = false, features = [ + "behavior-version-latest", +], optional = true } +aws-sdk-kinesis = { version = "1.3.0", default-features = false, features = [ + "behavior-version-latest", +], optional = true } +aws-sdk-secretsmanager = { version = "1.3.0", default-features = false, features = [ + "behavior-version-latest", +], optional = true } # The sts crate is needed despite not being referred to anywhere in the code because we need to set the # `behavior-version-latest` feature. Without this we get a runtime panic when `auth.assume_role` authentication # is configured. -aws-sdk-sts = { version = "1.3.1", default-features = false, features = ["behavior-version-latest"], optional = true } +aws-sdk-sts = { version = "1.3.1", default-features = false, features = [ + "behavior-version-latest", +], optional = true } aws-types = { version = "1.3.2", default-features = false, optional = true } -aws-sigv4 = { version = "1.2.2", default-features = false, features = ["sign-http"], optional = true } -aws-config = { version = "1.0.1", default-features = false, features = ["behavior-version-latest", "credentials-process"], optional = true } -aws-credential-types = { version = "1.2.0", default-features = false, features = ["hardcoded-credentials"], optional = true } -aws-smithy-http = { version = "0.60", default-features = false, features = ["event-stream"], optional = true } +aws-sigv4 = { version = "1.2.2", default-features = false, features = [ + "sign-http", +], optional = true } +aws-config = { version = "1.0.1", default-features = false, features = [ + "behavior-version-latest", + "credentials-process", +], optional = true } +aws-credential-types = { version = "1.2.0", default-features = false, features = [ + "hardcoded-credentials", +], optional = true } +aws-smithy-http = { version = "0.60", default-features = false, features = [ + "event-stream", +], optional = true } aws-smithy-types = { version = "1.2.0", default-features = false, optional = true } aws-smithy-runtime-api = { version = "1.7.0", default-features = false, optional = true } -aws-smithy-runtime = { version = "1.6.0", default-features = false, features = ["client", "connector-hyper-0-14-x", "rt-tokio"], optional = true } -aws-smithy-async = { version = "1.2.1", default-features = false, features = ["rt-tokio"], optional = true } +aws-smithy-runtime = { version = "1.6.0", default-features = false, features = [ + "client", + "connector-hyper-0-14-x", + "rt-tokio", +], optional = true } +aws-smithy-async = { version = "1.2.1", default-features = false, features = [ + "rt-tokio", +], optional = true } # Azure -azure_core = { version = "0.17", default-features = false, features = ["enable_reqwest"], optional = true } -azure_identity = { version = "0.17", default-features = false, features = ["enable_reqwest"], optional = true } +azure_core = { version = "0.17", default-features = false, features = [ + "enable_reqwest", +], optional = true } +azure_identity = { version = "0.17", default-features = false, features = [ + "enable_reqwest", +], optional = true } azure_storage = { version = "0.17", default-features = false, optional = true } azure_storage_blobs = { version = "0.17", default-features = false, optional = true } # OpenDAL -opendal = {version = "0.45", default-features = false, features = ["native-tls", "services-webhdfs"], optional = true} +opendal = { version = "0.45", default-features = false, features = [ + "native-tls", + "services-webhdfs", +], optional = true } # Tower -tower = { version = "0.4.13", default-features = false, features = ["buffer", "limit", "retry", "timeout", "util", "balance", "discover"] } -tower-http = { version = "0.4.4", default-features = false, features = ["compression-full", "decompression-gzip", "trace"]} +tower = { version = "0.4.13", default-features = false, features = [ + "buffer", + "limit", + "retry", + "timeout", + "util", + "balance", + "discover", +] } +tower-http = { version = "0.4.4", default-features = false, features = [ + "compression-full", + "decompression-gzip", + "trace", +] } # Serde serde.workspace = true serde-toml-merge = { version = "0.3.8", default-features = false } -serde_bytes = { version = "0.11.14", default-features = false, features = ["std"], optional = true } +serde_bytes = { version = "0.11.14", default-features = false, features = [ + "std", +], optional = true } serde_json.workspace = true -serde_with = { version = "3.8.1", default-features = false, features = ["macros", "std"] } +serde_with = { version = "3.8.1", default-features = false, features = [ + "macros", + "std", +] } serde_yaml = { version = "0.9.34", default-features = false } # Messagepack rmp-serde = { version = "1.3.0", default-features = false, optional = true } -rmpv = { version = "1.3.0", default-features = false, features = ["with-serde"], optional = true } +rmpv = { version = "1.3.0", default-features = false, features = [ + "with-serde", +], optional = true } # Prost / Protocol Buffers prost = { version = "0.12", default-features = false, features = ["std"] } @@ -237,17 +380,31 @@ goauth = { version = "0.14.0", optional = true } smpl_jwt = { version = "0.8.0", default-features = false, optional = true } # AMQP -lapin = { version = "2.3.4", default-features = false, features = ["native-tls"], optional = true } +lapin = { version = "2.3.4", default-features = false, features = [ + "native-tls", +], optional = true } # API -async-graphql = { version = "7.0.6", default-features = false, optional = true, features = ["chrono", "playground"] } +async-graphql = { version = "7.0.6", default-features = false, optional = true, features = [ + "chrono", + "playground", +] } async-graphql-warp = { version = "7.0.6", default-features = false, optional = true } # API client -crossterm = { version = "0.27.0", default-features = false, features = ["event-stream", "windows"], optional = true } -num-format = { version = "0.4.4", default-features = false, features = ["with-num-bigint"], optional = true } -number_prefix = { version = "0.4.0", default-features = false, features = ["std"], optional = true } -ratatui = { version = "0.26.3", optional = true, default-features = false, features = ["crossterm"] } +crossterm = { version = "0.27.0", default-features = false, features = [ + "event-stream", + "windows", +], optional = true } +num-format = { version = "0.4.4", default-features = false, features = [ + "with-num-bigint", +], optional = true } +number_prefix = { version = "0.4.0", default-features = false, features = [ + "std", +], optional = true } +ratatui = { version = "0.26.3", optional = true, default-features = false, features = [ + "crossterm", +] } # Opentelemetry @@ -258,12 +415,19 @@ greptimedb-client = { git = "https://github.com/GreptimeTeam/greptimedb-ingester # External libs arc-swap = { version = "1.7", default-features = false, optional = true } -async-compression = { version = "0.4.11", default-features = false, features = ["tokio", "gzip", "zstd"], optional = true } +async-compression = { version = "0.4.11", default-features = false, features = [ + "tokio", + "gzip", + "zstd", +], optional = true } apache-avro = { version = "0.16.0", default-features = false, optional = true } axum = { version = "0.6.20", default-features = false } base64 = { version = "0.22.1", default-features = false, optional = true } -bloomy = { version = "1.2.0", default-features = false, optional = true } -bollard = { version = "0.16.1", default-features = false, features = ["ssl", "chrono"], optional = true } +bloomy = { version = "1.2.0", default-features = false, optional = true } +bollard = { version = "0.16.1", default-features = false, features = [ + "ssl", + "chrono", +], optional = true } bytes = { version = "1.6.0", default-features = false, features = ["serde"] } bytesize = { version = "1.3.0", default-features = false } chrono.workspace = true @@ -271,77 +435,163 @@ chrono-tz = { version = "0.9.0", default-features = false } cidr-utils = { version = "0.6.1", default-features = false } colored = { version = "2.1.0", default-features = false } csv = { version = "1.3", default-features = false } -databend-client ={ version = "0.18.3", default-features = false, features = ["rustls"], optional = true } +databend-client = { version = "0.18.3", default-features = false, features = [ + "rustls", +], optional = true } derivative = { version = "2.2.0", default-features = false } dirs-next = { version = "2.0.0", default-features = false, optional = true } dyn-clone = { version = "1.0.17", default-features = false } -encoding_rs = { version = "0.8.34", default-features = false, features = ["serde"] } +encoding_rs = { version = "0.8.34", default-features = false, features = [ + "serde", +] } enum_dispatch = { version = "0.3.13", default-features = false } exitcode = { version = "1.1.2", default-features = false } -flate2 = { version = "1.0.30", default-features = false, features = ["default"] } +flate2 = { version = "1.0.30", default-features = false, features = [ + "default", +] } futures-util = { version = "0.3.29", default-features = false } glob = { version = "0.3.1", default-features = false } -governor = { version = "0.6.3", default-features = false, features = ["dashmap", "jitter", "std"], optional = true } +governor = { version = "0.6.3", default-features = false, features = [ + "dashmap", + "jitter", + "std", +], optional = true } grok = { version = "2.0.0", default-features = false, optional = true } h2 = { version = "0.4.5", default-features = false, optional = true } hash_hasher = { version = "2.0.0", default-features = false } -hashbrown = { version = "0.14.5", default-features = false, optional = true, features = ["ahash"] } +hashbrown = { version = "0.14.5", default-features = false, optional = true, features = [ + "ahash", +] } headers = { version = "0.3.9", default-features = false } hostname = { version = "0.4.0", default-features = false } http = { version = "0.2.9", default-features = false } http-serde = "1.1.3" http-body = { version = "0.4.5", default-features = false } -hyper = { version = "0.14.28", default-features = false, features = ["client", "runtime", "http1", "http2", "server", "stream"] } +hyper = { version = "0.14.28", default-features = false, features = [ + "client", + "runtime", + "http1", + "http2", + "server", + "stream", +] } hyper-openssl = { version = "0.9.2", default-features = false } -hyper-proxy = { version = "0.9.1", default-features = false, features = ["openssl-tls"] } +hyper-proxy = { version = "0.9.1", default-features = false, features = [ + "openssl-tls", +] } indexmap.workspace = true -infer = { version = "0.16.0", default-features = false, optional = true} +infer = { version = "0.16.0", default-features = false, optional = true } indoc = { version = "2.0.5", default-features = false } inventory = { version = "0.3.15", default-features = false } -ipnet = { version = "2", default-features = false, optional = true, features = ["serde", "std"] } -itertools = { version = "0.13.0", default-features = false, optional = false, features = ["use_alloc"] } -k8s-openapi = { version = "0.18.0", default-features = false, features = ["api", "v1_26"], optional = true } -kube = { version = "0.82.0", default-features = false, features = ["client", "openssl-tls", "runtime"], optional = true } +ipnet = { version = "2", default-features = false, optional = true, features = [ + "serde", + "std", +] } +itertools = { version = "0.13.0", default-features = false, optional = false, features = [ + "use_alloc", +] } +k8s-openapi = { version = "0.18.0", default-features = false, features = [ + "api", + "v1_26", +], optional = true } +kube = { version = "0.82.0", default-features = false, features = [ + "client", + "openssl-tls", + "runtime", +], optional = true } listenfd = { version = "1.0.1", default-features = false, optional = true } logfmt = { version = "0.0.2", default-features = false, optional = true } lru = { version = "0.12.3", default-features = false, optional = true } maxminddb = { version = "0.24.0", default-features = false, optional = true } md-5 = { version = "0.10", default-features = false, optional = true } -mongodb = { version = "2.8.2", default-features = false, features = ["tokio-runtime"], optional = true } +mongodb = { version = "2.8.2", default-features = false, features = [ + "tokio-runtime", +], optional = true } async-nats = { version = "0.33.0", default-features = false, optional = true } nkeys = { version = "0.4.1", default-features = false, optional = true } nom = { version = "7.1.3", default-features = false, optional = true } -notify = { version = "6.1.1", default-features = false, features = ["macos_fsevent"] } +notify = { version = "6.1.1", default-features = false, features = [ + "macos_fsevent", +] } once_cell = { version = "1.19", default-features = false } -openssl = { version = "0.10.64", default-features = false, features = ["vendored"] } +openssl = { version = "0.10.64", default-features = false, features = [ + "vendored", +] } openssl-probe = { version = "0.1.5", default-features = false } ordered-float = { version = "4.2.0", default-features = false } paste = "1.0.15" percent-encoding = { version = "2.3.1", default-features = false } -postgres-openssl = { version = "0.5.0", default-features = false, features = ["runtime"], optional = true } -pulsar = { version = "6.3.0", default-features = false, features = ["tokio-runtime", "auth-oauth2", "flate2", "lz4", "snap", "zstd"], optional = true } +postgres-openssl = { version = "0.5.0", default-features = false, features = [ + "runtime", +], optional = true } +pulsar = { version = "6.3.0", default-features = false, features = [ + "tokio-runtime", + "auth-oauth2", + "flate2", + "lz4", + "snap", + "zstd", +], optional = true } rand = { version = "0.8.5", default-features = false, features = ["small_rng"] } rand_distr = { version = "0.4.3", default-features = false } -rdkafka = { version = "0.35.0", default-features = false, features = ["tokio", "libz", "ssl", "zstd"], optional = true } -redis = { version = "0.24.0", default-features = false, features = ["connection-manager", "tokio-comp", "tokio-native-tls-comp"], optional = true } -regex = { version = "1.10.5", default-features = false, features = ["std", "perf"] } -roaring = { version = "0.10.5", default-features = false, features = ["std"], optional = true } -rumqttc = { version = "0.24.0", default-features = false, features = ["use-rustls"], optional = true } +rdkafka = { version = "0.35.0", default-features = false, features = [ + "tokio", + "libz", + "ssl", + "zstd", +], optional = true } +redis = { version = "0.24.0", default-features = false, features = [ + "connection-manager", + "tokio-comp", + "tokio-native-tls-comp", +], optional = true } +regex = { version = "1.10.5", default-features = false, features = [ + "std", + "perf", +] } +roaring = { version = "0.10.5", default-features = false, features = [ + "std", +], optional = true } +rumqttc = { version = "0.24.0", default-features = false, features = [ + "use-rustls", +], optional = true } seahash = { version = "4.1.0", default-features = false } -semver = { version = "1.0.23", default-features = false, features = ["serde", "std"], optional = true } -smallvec = { version = "1", default-features = false, features = ["union", "serde"] } +semver = { version = "1.0.23", default-features = false, features = [ + "serde", + "std", +], optional = true } +smallvec = { version = "1", default-features = false, features = [ + "union", + "serde", +] } snafu = { version = "0.7.5", default-features = false, features = ["futures"] } snap = { version = "1.1.1", default-features = false } socket2 = { version = "0.5.7", default-features = false } stream-cancel = { version = "0.8.2", default-features = false } strip-ansi-escapes = { version = "0.2.0", default-features = false } syslog = { version = "6.1.1", default-features = false, optional = true } -tikv-jemallocator = { version = "0.5.4", default-features = false, features = ["unprefixed_malloc_on_supported_platforms"], optional = true } -tokio-postgres = { version = "0.7.10", default-features = false, features = ["runtime", "with-chrono-0_4"], optional = true } -tokio-tungstenite = {version = "0.20.1", default-features = false, features = ["connect"], optional = true} +tikv-jemallocator = { version = "0.5.4", default-features = false, features = [ + "unprefixed_malloc_on_supported_platforms", +], optional = true } +tokio-postgres = { version = "0.7.10", default-features = false, features = [ + "runtime", + "with-chrono-0_4", +], optional = true } +tokio-tungstenite = { version = "0.20.1", default-features = false, features = [ + "connect", +], optional = true } toml.workspace = true -tonic = { version = "0.10", optional = true, default-features = false, features = ["transport", "codegen", "prost", "tls", "tls-roots", "gzip"] } -hickory-proto = { version = "0.24.1", default-features = false, features = ["dnssec"], optional = true } +tonic = { version = "0.10", optional = true, default-features = false, features = [ + "transport", + "codegen", + "prost", + "tls", + "tls-roots", + "gzip", +] } +hickory-proto = { version = "0.24.1", default-features = false, features = [ + "dnssec", +], optional = true } typetag = { version = "0.2.16", default-features = false } url = { version = "2.5.1", default-features = false, features = ["serde"] } uuid = { version = "1", default-features = false, features = ["serde", "v4"] } @@ -351,46 +601,76 @@ arr_macro = { version = "0.2.1" } # depending on fork for bumped nix dependency # https://github.com/heim-rs/heim/pull/360 -heim = { git = "https://github.com/vectordotdev/heim.git", branch = "update-nix", default-features = false, features = ["disk"] } +heim = { git = "https://github.com/vectordotdev/heim.git", branch = "update-nix", default-features = false, features = [ + "disk", +] } # make sure to update the external docs when the Lua version changes -mlua = { version = "0.9.9", default-features = false, features = ["lua54", "send", "vendored", "macros"], optional = true } +mlua = { version = "0.9.9", default-features = false, features = [ + "lua54", + "send", + "vendored", + "macros", +], optional = true } [target.'cfg(windows)'.dependencies] windows-service = "0.7.0" [target.'cfg(unix)'.dependencies] -nix = { version = "0.26.2", default-features = false, features = ["socket", "signal"] } +nix = { version = "0.26.2", default-features = false, features = [ + "socket", + "signal", +] } [build-dependencies] prost-build = { version = "0.12", default-features = false, optional = true } -tonic-build = { version = "0.10", default-features = false, features = ["transport", "prost"], optional = true } +tonic-build = { version = "0.10", default-features = false, features = [ + "transport", + "prost", +], optional = true } # update 'openssl_version' in website/config.toml whenever version changes -openssl-src = { version = "300", default-features = false, features = ["force-engine", "legacy"] } +openssl-src = { version = "300", default-features = false, features = [ + "force-engine", + "legacy", +] } [dev-dependencies] approx = "0.5.1" assert_cmd = { version = "2.0.14", default-features = false } -aws-smithy-runtime = { version = "1.6.0", default-features = false, features = ["tls-rustls"] } -azure_core = { version = "0.17", default-features = false, features = ["enable_reqwest", "azurite_workaround"] } -azure_identity = { version = "0.17", default-features = false, features = ["enable_reqwest"] } -azure_storage_blobs = { version = "0.17", default-features = false, features = ["azurite_workaround"] } +aws-smithy-runtime = { version = "1.6.0", default-features = false, features = [ + "tls-rustls", +] } +azure_core = { version = "0.17", default-features = false, features = [ + "enable_reqwest", + "azurite_workaround", +] } +azure_identity = { version = "0.17", default-features = false, features = [ + "enable_reqwest", +] } +azure_storage_blobs = { version = "0.17", default-features = false, features = [ + "azurite_workaround", +] } azure_storage = { version = "0.17", default-features = false } base64 = "0.22.1" criterion = { version = "0.5.1", features = ["html_reports", "async_tokio"] } -itertools = { version = "0.13.0", default-features = false, features = ["use_alloc"] } +itertools = { version = "0.13.0", default-features = false, features = [ + "use_alloc", +] } libc = "0.2.155" similar-asserts = "1.5.0" proptest.workspace = true quickcheck = "1.0.3" reqwest = { version = "0.11", features = ["json"] } -rstest = {version = "0.21.0"} +rstest = { version = "0.21.0" } tempfile = "3.10.1" test-generator = "0.3.1" tokio = { version = "1.38.0", features = ["test-util"] } tokio-test = "0.4.4" tower-test = "0.4.0" -vector-lib = { path = "lib/vector-lib", default-features = false, features = ["vrl", "test"] } +vector-lib = { path = "lib/vector-lib", default-features = false, features = [ + "vrl", + "test", +] } vrl.workspace = true wiremock = "0.5.22" @@ -406,17 +686,90 @@ ntapi = { git = "https://github.com/MSxDOS/ntapi.git", rev = "24fc1e47677fc9f6e3 [features] # Default features for *-unknown-linux-gnu and *-apple-darwin -default = ["api", "api-client", "enrichment-tables", "sinks", "sources", "sources-dnstap", "transforms", "unix", "rdkafka?/gssapi-vendored", "secrets"] +default = [ + "api", + "api-client", + "enrichment-tables", + "sinks", + "sources", + "sources-dnstap", + "transforms", + "unix", + "rdkafka?/gssapi-vendored", + "secrets", +] # Default features for `cargo docs`. The same as `default` but without `rdkafka?/gssapi-vendored` which would require installing libsasl in our doc build environment. -docs = ["api", "api-client", "enrichment-tables", "sinks", "sources", "sources-dnstap", "transforms", "unix", "secrets"] +docs = [ + "api", + "api-client", + "enrichment-tables", + "sinks", + "sources", + "sources-dnstap", + "transforms", + "unix", + "secrets", +] # Default features for *-unknown-linux-* which make use of `cmake` for dependencies -default-cmake = ["api", "api-client", "enrichment-tables", "rdkafka?/cmake_build", "sinks", "sources", "sources-dnstap", "transforms", "unix", "rdkafka?/gssapi-vendored", "secrets"] +default-cmake = [ + "api", + "api-client", + "enrichment-tables", + "rdkafka?/cmake_build", + "sinks", + "sources", + "sources-dnstap", + "transforms", + "unix", + "rdkafka?/gssapi-vendored", + "secrets", +] # Default features for *-pc-windows-msvc # TODO: Enable SASL https://github.com/vectordotdev/vector/pull/3081#issuecomment-659298042 -default-msvc = ["api", "api-client", "enrichment-tables", "rdkafka?/cmake_build", "sinks", "sources", "transforms", "secrets"] -default-musl = ["api", "api-client", "enrichment-tables", "rdkafka?/cmake_build", "sinks", "sources", "sources-dnstap", "transforms", "unix", "rdkafka?/gssapi-vendored", "secrets"] -default-no-api-client = ["api", "enrichment-tables", "sinks", "sources", "sources-dnstap", "transforms", "unix", "rdkafka?/gssapi-vendored", "secrets"] -default-no-vrl-cli = ["api", "sinks", "sources", "sources-dnstap", "transforms", "unix", "rdkafka?/gssapi-vendored", "secrets"] +default-msvc = [ + "api", + "api-client", + "enrichment-tables", + "rdkafka?/cmake_build", + "sinks", + "sources", + "transforms", + "secrets", +] +default-musl = [ + "api", + "api-client", + "enrichment-tables", + "rdkafka?/cmake_build", + "sinks", + "sources", + "sources-dnstap", + "transforms", + "unix", + "rdkafka?/gssapi-vendored", + "secrets", +] +default-no-api-client = [ + "api", + "enrichment-tables", + "sinks", + "sources", + "sources-dnstap", + "transforms", + "unix", + "rdkafka?/gssapi-vendored", + "secrets", +] +default-no-vrl-cli = [ + "api", + "sinks", + "sources", + "sources-dnstap", + "transforms", + "unix", + "rdkafka?/gssapi-vendored", + "secrets", +] tokio-console = ["dep:console-subscriber", "tokio/tracing"] # Enables the binary secret-backend-example @@ -428,18 +781,127 @@ all-metrics = ["sinks-metrics", "sources-metrics", "transforms-metrics"] # Target specific release features. # The `make` tasks will select this according to the appropriate triple. # Use this section to turn off or on specific features for specific triples. -target-aarch64-unknown-linux-gnu = ["api", "api-client", "enrichment-tables", "rdkafka?/cmake_build", "sinks", "sources", "sources-dnstap", "transforms", "unix", "secrets"] -target-aarch64-unknown-linux-musl = ["api", "api-client", "enrichment-tables", "rdkafka?/cmake_build", "sinks", "sources", "sources-dnstap", "transforms", "unix", "secrets"] -target-armv7-unknown-linux-gnueabihf = ["api", "api-client", "enrichment-tables", "rdkafka?/cmake_build", "sinks", "sources", "sources-dnstap", "transforms", "unix", "secrets"] -target-armv7-unknown-linux-musleabihf = ["api", "api-client", "rdkafka?/cmake_build", "enrichment-tables", "sinks", "sources", "sources-dnstap", "transforms", "secrets"] -target-arm-unknown-linux-gnueabi = ["api", "api-client", "enrichment-tables", "rdkafka?/cmake_build", "sinks", "sources", "sources-dnstap", "transforms", "unix", "secrets"] -target-arm-unknown-linux-musleabi = ["api", "api-client", "rdkafka?/cmake_build", "enrichment-tables", "sinks", "sources", "sources-dnstap", "transforms", "secrets"] -target-x86_64-unknown-linux-gnu = ["api", "api-client", "rdkafka?/cmake_build", "enrichment-tables", "sinks", "sources", "sources-dnstap", "transforms", "unix", "rdkafka?/gssapi-vendored", "secrets"] -target-x86_64-unknown-linux-musl = ["api", "api-client", "rdkafka?/cmake_build", "enrichment-tables", "sinks", "sources", "sources-dnstap", "transforms", "unix", "secrets"] +target-aarch64-unknown-linux-gnu = [ + "api", + "api-client", + "enrichment-tables", + "rdkafka?/cmake_build", + "sinks", + "sources", + "sources-dnstap", + "transforms", + "unix", + "secrets", +] +target-aarch64-unknown-linux-musl = [ + "api", + "api-client", + "enrichment-tables", + "rdkafka?/cmake_build", + "sinks", + "sources", + "sources-dnstap", + "transforms", + "unix", + "secrets", +] +target-armv7-unknown-linux-gnueabihf = [ + "api", + "api-client", + "enrichment-tables", + "rdkafka?/cmake_build", + "sinks", + "sources", + "sources-dnstap", + "transforms", + "unix", + "secrets", +] +target-armv7-unknown-linux-musleabihf = [ + "api", + "api-client", + "rdkafka?/cmake_build", + "enrichment-tables", + "sinks", + "sources", + "sources-dnstap", + "transforms", + "secrets", +] +target-arm-unknown-linux-gnueabi = [ + "api", + "api-client", + "enrichment-tables", + "rdkafka?/cmake_build", + "sinks", + "sources", + "sources-dnstap", + "transforms", + "unix", + "secrets", +] +target-arm-unknown-linux-musleabi = [ + "api", + "api-client", + "rdkafka?/cmake_build", + "enrichment-tables", + "sinks", + "sources", + "sources-dnstap", + "transforms", + "secrets", +] +target-x86_64-unknown-linux-gnu = [ + "api", + "api-client", + "rdkafka?/cmake_build", + "enrichment-tables", + "sinks", + "sources", + "sources-dnstap", + "transforms", + "unix", + "rdkafka?/gssapi-vendored", + "secrets", +] +target-x86_64-unknown-linux-musl = [ + "api", + "api-client", + "rdkafka?/cmake_build", + "enrichment-tables", + "sinks", + "sources", + "sources-dnstap", + "transforms", + "unix", + "secrets", +] # Does not currently build -target-powerpc64le-unknown-linux-gnu = ["api", "api-client", "enrichment-tables", "rdkafka?/cmake_build", "sinks", "sources", "sources-dnstap", "transforms", "unix", "secrets"] +target-powerpc64le-unknown-linux-gnu = [ + "api", + "api-client", + "enrichment-tables", + "rdkafka?/cmake_build", + "sinks", + "sources", + "sources-dnstap", + "transforms", + "unix", + "secrets", +] # Currently doesn't build due to lack of support for 64-bit atomics -target-powerpc-unknown-linux-gnu = ["api", "api-client", "enrichment-tables", "rdkafka?/cmake_build", "sinks", "sources", "sources-dnstap", "transforms", "unix", "secrets"] +target-powerpc-unknown-linux-gnu = [ + "api", + "api-client", + "enrichment-tables", + "rdkafka?/cmake_build", + "sinks", + "sources", + "sources-dnstap", + "transforms", + "unix", + "secrets", +] # Enables features that work only on systems providing `cfg(unix)` unix = ["tikv-jemallocator", "allocation-tracing"] @@ -453,33 +915,33 @@ docker = ["dep:bollard", "dep:dirs-next"] # API api = [ - "dep:async-graphql", - "dep:async-graphql-warp", - "dep:base64", - "vector-lib/api", + "dep:async-graphql", + "dep:async-graphql-warp", + "dep:base64", + "vector-lib/api", ] # API client api-client = [ - "dep:crossterm", - "dep:num-format", - "dep:number_prefix", - "dep:ratatui", - "vector-lib/api", - "vector-lib/api-client", + "dep:crossterm", + "dep:num-format", + "dep:number_prefix", + "dep:ratatui", + "vector-lib/api", + "vector-lib/api-client", ] aws-core = [ - "aws-config", - "dep:aws-credential-types", - "dep:aws-sigv4", - "dep:aws-types", - "dep:aws-smithy-async", - "dep:aws-smithy-http", - "dep:aws-smithy-types", - "dep:aws-smithy-runtime", - "dep:aws-smithy-runtime-api", - "dep:aws-sdk-sts", + "aws-config", + "dep:aws-credential-types", + "dep:aws-sigv4", + "dep:aws-types", + "dep:aws-smithy-async", + "dep:aws-smithy-http", + "dep:aws-smithy-types", + "dep:aws-smithy-runtime", + "dep:aws-smithy-runtime-api", + "dep:aws-sdk-sts", ] # Anything that requires Protocol Buffers. @@ -503,100 +965,172 @@ secrets-aws-secrets-manager = ["aws-core", "dep:aws-sdk-secretsmanager"] # Sources sources = ["sources-logs", "sources-metrics"] sources-logs = [ - "sources-amqp", - "sources-aws_kinesis_firehose", - "sources-aws_s3", - "sources-aws_sqs", - "sources-datadog_agent", - "sources-demo_logs", - "sources-docker_logs", - "sources-exec", - "sources-file", - "sources-fluent", - "sources-gcp_pubsub", - "sources-heroku_logs", - "sources-http_server", - "sources-http_client", - "sources-internal_logs", - "sources-journald", - "sources-kafka", - "sources-kubernetes_logs", - "sources-logstash", - "sources-nats", - "sources-opentelemetry", - "sources-pulsar", - "sources-file-descriptor", - "sources-redis", - "sources-socket", - "sources-splunk_hec", - "sources-stdin", - "sources-syslog", - "sources-vector", + "sources-amqp", + "sources-aws_kinesis_firehose", + "sources-aws_s3", + "sources-aws_sqs", + "sources-datadog_agent", + "sources-demo_logs", + "sources-docker_logs", + "sources-exec", + "sources-file", + "sources-fluent", + "sources-gcp_pubsub", + "sources-heroku_logs", + "sources-http_server", + "sources-http_client", + "sources-internal_logs", + "sources-journald", + "sources-kafka", + "sources-kubernetes_logs", + "sources-logstash", + "sources-nats", + "sources-opentelemetry", + "sources-pulsar", + "sources-file-descriptor", + "sources-redis", + "sources-socket", + "sources-splunk_hec", + "sources-stdin", + "sources-syslog", + "sources-vector", ] sources-metrics = [ - "sources-apache_metrics", - "sources-aws_ecs_metrics", - "sources-eventstoredb_metrics", - "sources-host_metrics", - "sources-internal_metrics", - "sources-mongodb_metrics", - "sources-nginx_metrics", - "sources-postgresql_metrics", - "sources-prometheus", - "sources-statsd", - "sources-vector", + "sources-apache_metrics", + "sources-aws_ecs_metrics", + "sources-eventstoredb_metrics", + "sources-host_metrics", + "sources-internal_metrics", + "sources-mongodb_metrics", + "sources-nginx_metrics", + "sources-postgresql_metrics", + "sources-prometheus", + "sources-statsd", + "sources-vector", ] sources-amqp = ["lapin"] sources-apache_metrics = ["sources-utils-http-client"] sources-aws_ecs_metrics = ["sources-utils-http-client"] sources-aws_kinesis_firehose = ["dep:base64", "dep:infer"] -sources-aws_s3 = ["aws-core", "dep:aws-sdk-sqs", "dep:aws-sdk-s3", "dep:semver", "dep:async-compression", "sources-aws_sqs", "tokio-util/io"] +sources-aws_s3 = [ + "aws-core", + "dep:aws-sdk-sqs", + "dep:aws-sdk-s3", + "dep:semver", + "dep:async-compression", + "sources-aws_sqs", + "tokio-util/io", +] sources-aws_sqs = ["aws-core", "dep:aws-sdk-sqs"] sources-datadog_agent = ["sources-utils-http-error", "protobuf-build"] sources-demo_logs = ["dep:fakedata"] -sources-dnstap = ["sources-utils-net-tcp", "dep:base64", "dep:hickory-proto", "dep:dnsmsg-parser", "protobuf-build"] +sources-dnstap = [ + "sources-utils-net-tcp", + "dep:base64", + "dep:hickory-proto", + "dep:dnsmsg-parser", + "protobuf-build", +] sources-docker_logs = ["docker"] sources-eventstoredb_metrics = [] sources-exec = [] sources-file = ["vector-lib/file-source"] sources-file-descriptor = ["tokio-util/io"] -sources-fluent = ["dep:base64", "sources-utils-net-tcp", "tokio-util/net", "dep:rmpv", "dep:rmp-serde", "dep:serde_bytes"] -sources-gcp_pubsub = ["gcp", "dep:h2", "dep:prost-types", "protobuf-build", "dep:tonic"] -sources-heroku_logs = ["sources-utils-http", "sources-utils-http-query", "sources-http_server"] -sources-host_metrics = ["heim/cpu", "heim/host", "heim/memory", "heim/net"] +sources-fluent = [ + "dep:base64", + "sources-utils-net-tcp", + "tokio-util/net", + "dep:rmpv", + "dep:rmp-serde", + "dep:serde_bytes", +] +sources-gcp_pubsub = [ + "gcp", + "dep:h2", + "dep:prost-types", + "protobuf-build", + "dep:tonic", +] +sources-heroku_logs = [ + "sources-utils-http", + "sources-utils-http-query", + "sources-http_server", +] +sources-host_metrics = ["heim/cpu", "heim/host", "heim/memory", "heim/net"] sources-http_client = ["sources-utils-http-client"] sources-http_server = ["sources-utils-http", "sources-utils-http-query"] sources-internal_logs = [] sources-internal_metrics = [] sources-journald = [] sources-kafka = ["dep:rdkafka"] -sources-kubernetes_logs = ["vector-lib/file-source", "kubernetes", "transforms-reduce"] +sources-kubernetes_logs = [ + "vector-lib/file-source", + "kubernetes", + "transforms-reduce", +] sources-logstash = ["sources-utils-net-tcp", "tokio-util/net"] sources-mongodb_metrics = ["dep:mongodb"] sources-nats = ["dep:async-nats", "dep:nkeys"] sources-nginx_metrics = ["dep:nom"] -sources-opentelemetry = ["dep:hex", "vector-lib/opentelemetry", "dep:prost-types", "sources-http_server", "sources-utils-http", "sources-vector"] +sources-opentelemetry = [ + "dep:hex", + "vector-lib/opentelemetry", + "dep:prost-types", + "sources-http_server", + "sources-utils-http", + "sources-vector", +] sources-postgresql_metrics = ["dep:postgres-openssl", "dep:tokio-postgres"] -sources-prometheus = ["sources-prometheus-scrape", "sources-prometheus-remote-write", "sources-prometheus-pushgateway"] -sources-prometheus-scrape = ["sinks-prometheus", "sources-utils-http-client", "vector-lib/prometheus"] -sources-prometheus-remote-write = ["sinks-prometheus", "sources-utils-http", "vector-lib/prometheus"] -sources-prometheus-pushgateway = ["sinks-prometheus", "sources-utils-http", "vector-lib/prometheus"] +sources-prometheus = [ + "sources-prometheus-scrape", + "sources-prometheus-remote-write", + "sources-prometheus-pushgateway", +] +sources-prometheus-scrape = [ + "sinks-prometheus", + "sources-utils-http-client", + "vector-lib/prometheus", +] +sources-prometheus-remote-write = [ + "sinks-prometheus", + "sources-utils-http", + "vector-lib/prometheus", +] +sources-prometheus-pushgateway = [ + "sinks-prometheus", + "sources-utils-http", + "vector-lib/prometheus", +] sources-pulsar = ["dep:apache-avro", "dep:pulsar"] -sources-redis= ["dep:redis"] +sources-redis = ["dep:redis"] sources-socket = ["sources-utils-net", "tokio-util/net"] sources-splunk_hec = ["dep:roaring"] sources-statsd = ["sources-utils-net", "tokio-util/net"] sources-stdin = ["tokio-util/io"] sources-syslog = ["codecs-syslog", "sources-utils-net", "tokio-util/net"] -sources-utils-http = ["sources-utils-http-auth", "sources-utils-http-encoding", "sources-utils-http-error", "sources-utils-http-prelude"] +sources-utils-http = [ + "sources-utils-http-auth", + "sources-utils-http-encoding", + "sources-utils-http-error", + "sources-utils-http-prelude", +] sources-utils-http-auth = ["sources-utils-http-error"] sources-utils-http-encoding = ["sources-utils-http-error"] sources-utils-http-error = [] -sources-utils-http-prelude = ["sources-utils-http", "sources-utils-http-auth", "sources-utils-http-encoding", "sources-utils-http-error"] +sources-utils-http-prelude = [ + "sources-utils-http", + "sources-utils-http-auth", + "sources-utils-http-encoding", + "sources-utils-http-error", +] sources-utils-http-query = [] sources-utils-http-client = ["sources-utils-http", "sources-http_server"] -sources-utils-net = ["sources-utils-net-tcp", "sources-utils-net-udp", "sources-utils-net-unix"] +sources-utils-net = [ + "sources-utils-net-tcp", + "sources-utils-net-udp", + "sources-utils-net-unix", +] sources-utils-net-tcp = ["listenfd", "dep:ipnet"] sources-utils-net-udp = ["listenfd"] sources-utils-net-unix = [] @@ -606,29 +1140,29 @@ sources-vector = ["dep:tonic", "protobuf-build"] # Transforms transforms = ["transforms-logs", "transforms-metrics"] transforms-logs = [ - "transforms-aws_ec2_metadata", - "transforms-dedupe", - "transforms-filter", - "transforms-log_to_metric", - "transforms-lua", - "transforms-metric_to_log", - "transforms-pipelines", - "transforms-reduce", - "transforms-remap", - "transforms-route", - "transforms-sample", - "transforms-throttle", + "transforms-aws_ec2_metadata", + "transforms-dedupe", + "transforms-filter", + "transforms-log_to_metric", + "transforms-lua", + "transforms-metric_to_log", + "transforms-pipelines", + "transforms-reduce", + "transforms-remap", + "transforms-route", + "transforms-sample", + "transforms-throttle", ] transforms-metrics = [ - "transforms-aggregate", - "transforms-filter", - "transforms-log_to_metric", - "transforms-lua", - "transforms-metric_to_log", - "transforms-pipelines", - "transforms-remap", - "transforms-tag_cardinality_limit", - "transforms-throttle", + "transforms-aggregate", + "transforms-filter", + "transforms-log_to_metric", + "transforms-lua", + "transforms-metric_to_log", + "transforms-pipelines", + "transforms-remap", + "transforms-tag_cardinality_limit", + "transforms-throttle", ] transforms-aggregate = [] @@ -654,64 +1188,64 @@ transforms-impl-reduce = [] # Sinks sinks = ["sinks-logs", "sinks-metrics"] sinks-logs = [ - "sinks-amqp", - "sinks-appsignal", - "sinks-aws_cloudwatch_logs", - "sinks-aws_kinesis_firehose", - "sinks-aws_kinesis_streams", - "sinks-aws_s3", - "sinks-aws_sqs", - "sinks-aws_sns", - "sinks-axiom", - "sinks-azure_blob", - "sinks-azure_monitor_logs", - "sinks-blackhole", - "sinks-chronicle", - "sinks-clickhouse", - "sinks-console", - "sinks-databend", - "sinks-datadog_events", - "sinks-datadog_logs", - "sinks-datadog_traces", - "sinks-elasticsearch", - "sinks-file", - "sinks-gcp", - "sinks-honeycomb", - "sinks-http", - "sinks-humio", - "sinks-influxdb", - "sinks-kafka", - "sinks-mezmo", - "sinks-loki", - "sinks-mqtt", - "sinks-nats", - "sinks-new_relic_logs", - "sinks-new_relic", - "sinks-papertrail", - "sinks-pulsar", - "sinks-redis", - "sinks-sematext", - "sinks-socket", - "sinks-splunk_hec", - "sinks-vector", - "sinks-webhdfs", - "sinks-websocket", + "sinks-amqp", + "sinks-appsignal", + "sinks-aws_cloudwatch_logs", + "sinks-aws_kinesis_firehose", + "sinks-aws_kinesis_streams", + "sinks-aws_s3", + "sinks-aws_sqs", + "sinks-aws_sns", + "sinks-axiom", + "sinks-azure_blob", + "sinks-azure_monitor_logs", + "sinks-blackhole", + "sinks-chronicle", + "sinks-clickhouse", + "sinks-console", + "sinks-databend", + "sinks-datadog_events", + "sinks-datadog_logs", + "sinks-datadog_traces", + "sinks-elasticsearch", + "sinks-file", + "sinks-gcp", + "sinks-honeycomb", + "sinks-http", + "sinks-humio", + "sinks-influxdb", + "sinks-kafka", + "sinks-mezmo", + "sinks-loki", + "sinks-mqtt", + "sinks-nats", + "sinks-new_relic_logs", + "sinks-new_relic", + "sinks-papertrail", + "sinks-pulsar", + "sinks-redis", + "sinks-sematext", + "sinks-socket", + "sinks-splunk_hec", + "sinks-vector", + "sinks-webhdfs", + "sinks-websocket", ] sinks-metrics = [ - "sinks-appsignal", - "sinks-aws_cloudwatch_metrics", - "sinks-blackhole", - "sinks-console", - "sinks-datadog_metrics", - "sinks-greptimedb", - "sinks-humio", - "sinks-influxdb", - "sinks-kafka", - "sinks-prometheus", - "sinks-sematext", - "sinks-statsd", - "sinks-vector", - "sinks-splunk_hec" + "sinks-appsignal", + "sinks-aws_cloudwatch_metrics", + "sinks-blackhole", + "sinks-console", + "sinks-datadog_metrics", + "sinks-greptimedb", + "sinks-humio", + "sinks-influxdb", + "sinks-kafka", + "sinks-prometheus", + "sinks-sematext", + "sinks-statsd", + "sinks-vector", + "sinks-splunk_hec", ] sinks-amqp = ["lapin"] @@ -724,7 +1258,12 @@ sinks-aws_s3 = ["dep:base64", "dep:md-5", "aws-core", "dep:aws-sdk-s3"] sinks-aws_sqs = ["aws-core", "dep:aws-sdk-sqs"] sinks-aws_sns = ["aws-core", "dep:aws-sdk-sns"] sinks-axiom = ["sinks-elasticsearch"] -sinks-azure_blob = ["dep:azure_core", "dep:azure_identity", "dep:azure_storage", "dep:azure_storage_blobs"] +sinks-azure_blob = [ + "dep:azure_core", + "dep:azure_identity", + "dep:azure_storage", + "dep:azure_storage_blobs", +] sinks-azure_monitor_logs = [] sinks-blackhole = [] sinks-chronicle = [] @@ -734,11 +1273,16 @@ sinks-databend = ["dep:databend-client"] sinks-datadog_events = [] sinks-datadog_logs = [] sinks-datadog_metrics = ["protobuf-build", "dep:prost-reflect"] -sinks-datadog_traces = ["protobuf-build", "dep:rmpv", "dep:rmp-serde", "dep:serde_bytes"] +sinks-datadog_traces = [ + "protobuf-build", + "dep:rmpv", + "dep:rmp-serde", + "dep:serde_bytes", +] sinks-elasticsearch = ["transforms-metric_to_log"] sinks-file = ["dep:async-compression"] sinks-gcp = ["sinks-gcp-chronicle", "dep:base64", "gcp"] -sinks-gcp-chronicle = ["gcp"] +sinks-gcp-chronicle = ["gcp"] sinks-greptimedb = ["dep:greptimedb-client"] sinks-honeycomb = [] sinks-http = [] @@ -769,69 +1313,71 @@ nightly = [] # Integration testing-related features all-integration-tests = [ - "amqp-integration-tests", - "appsignal-integration-tests", - "aws-integration-tests", - "axiom-integration-tests", - "azure-integration-tests", - "chronicle-integration-tests", - "clickhouse-integration-tests", - "databend-integration-tests", - "datadog-agent-integration-tests", - "datadog-logs-integration-tests", - "datadog-metrics-integration-tests", - "datadog-traces-integration-tests", - "docker-logs-integration-tests", - "es-integration-tests", - "eventstoredb_metrics-integration-tests", - "fluent-integration-tests", - "gcp-cloud-storage-integration-tests", - "gcp-integration-tests", - "gcp-pubsub-integration-tests", - "greptimedb-integration-tests", - "http-client-integration-tests", - "humio-integration-tests", - "influxdb-integration-tests", - "kafka-integration-tests", - "logstash-integration-tests", - "loki-integration-tests", - "mongodb_metrics-integration-tests", - "nats-integration-tests", - "nginx-integration-tests", - "opentelemetry-integration-tests", - "postgresql_metrics-integration-tests", - "prometheus-integration-tests", - "pulsar-integration-tests", - "redis-integration-tests", - "splunk-integration-tests", - "dnstap-integration-tests", - "webhdfs-integration-tests", + "amqp-integration-tests", + "appsignal-integration-tests", + "aws-integration-tests", + "axiom-integration-tests", + "azure-integration-tests", + "chronicle-integration-tests", + "clickhouse-integration-tests", + "databend-integration-tests", + "datadog-agent-integration-tests", + "datadog-logs-integration-tests", + "datadog-metrics-integration-tests", + "datadog-traces-integration-tests", + "docker-logs-integration-tests", + "es-integration-tests", + "eventstoredb_metrics-integration-tests", + "fluent-integration-tests", + "gcp-cloud-storage-integration-tests", + "gcp-integration-tests", + "gcp-pubsub-integration-tests", + "greptimedb-integration-tests", + "http-client-integration-tests", + "humio-integration-tests", + "influxdb-integration-tests", + "kafka-integration-tests", + "logstash-integration-tests", + "loki-integration-tests", + "mongodb_metrics-integration-tests", + "nats-integration-tests", + "nginx-integration-tests", + "opentelemetry-integration-tests", + "postgresql_metrics-integration-tests", + "prometheus-integration-tests", + "pulsar-integration-tests", + "redis-integration-tests", + "splunk-integration-tests", + "dnstap-integration-tests", + "webhdfs-integration-tests", ] amqp-integration-tests = ["sources-amqp", "sinks-amqp"] appsignal-integration-tests = ["sinks-appsignal"] aws-integration-tests = [ - "aws-cloudwatch-logs-integration-tests", - "aws-cloudwatch-metrics-integration-tests", - "aws-ec2-metadata-integration-tests", - "aws-ecs-metrics-integration-tests", - "aws-kinesis-firehose-integration-tests", - "aws-kinesis-streams-integration-tests", - "aws-s3-integration-tests", - "aws-sqs-integration-tests", - "aws-sns-integration-tests", + "aws-cloudwatch-logs-integration-tests", + "aws-cloudwatch-metrics-integration-tests", + "aws-ec2-metadata-integration-tests", + "aws-ecs-metrics-integration-tests", + "aws-kinesis-firehose-integration-tests", + "aws-kinesis-streams-integration-tests", + "aws-s3-integration-tests", + "aws-sqs-integration-tests", + "aws-sns-integration-tests", ] -azure-integration-tests = [ - "azure-blob-integration-tests" -] +azure-integration-tests = ["azure-blob-integration-tests"] aws-cloudwatch-logs-integration-tests = ["sinks-aws_cloudwatch_logs"] aws-cloudwatch-metrics-integration-tests = ["sinks-aws_cloudwatch_metrics"] aws-ec2-metadata-integration-tests = ["transforms-aws_ec2_metadata"] aws-ecs-metrics-integration-tests = ["sources-aws_ecs_metrics"] -aws-kinesis-firehose-integration-tests = ["sinks-aws_kinesis_firehose", "dep:aws-sdk-elasticsearch", "sinks-elasticsearch"] +aws-kinesis-firehose-integration-tests = [ + "sinks-aws_kinesis_firehose", + "dep:aws-sdk-elasticsearch", + "sinks-elasticsearch", +] aws-kinesis-streams-integration-tests = ["sinks-aws_kinesis_streams"] aws-s3-integration-tests = ["sinks-aws_s3", "sources-aws_s3"] aws-sqs-integration-tests = ["sinks-aws_sqs"] @@ -844,7 +1390,11 @@ databend-integration-tests = ["sinks-databend"] datadog-agent-integration-tests = ["sources-datadog_agent"] datadog-logs-integration-tests = ["sinks-datadog_logs"] datadog-metrics-integration-tests = ["sinks-datadog_metrics"] -datadog-traces-integration-tests = ["sources-datadog_agent", "sinks-datadog_traces", "axum/tokio"] +datadog-traces-integration-tests = [ + "sources-datadog_agent", + "sinks-datadog_traces", + "axum/tokio", +] docker-logs-integration-tests = ["sources-docker_logs", "unix"] es-integration-tests = ["sinks-elasticsearch", "aws-core"] eventstoredb_metrics-integration-tests = ["sources-eventstoredb_metrics"] @@ -865,78 +1415,113 @@ nats-integration-tests = ["sinks-nats", "sources-nats"] nginx-integration-tests = ["sources-nginx_metrics"] opentelemetry-integration-tests = ["sources-opentelemetry"] postgresql_metrics-integration-tests = ["sources-postgresql_metrics"] -prometheus-integration-tests = ["sinks-prometheus", "sources-prometheus", "sinks-influxdb"] +prometheus-integration-tests = [ + "sinks-prometheus", + "sources-prometheus", + "sinks-influxdb", +] pulsar-integration-tests = ["sinks-pulsar", "sources-pulsar"] redis-integration-tests = ["sinks-redis", "sources-redis"] splunk-integration-tests = ["sinks-splunk_hec"] dnstap-integration-tests = ["sources-dnstap", "dep:bollard"] webhdfs-integration-tests = ["sinks-webhdfs"] disable-resolv-conf = [] -shutdown-tests = ["api", "sinks-blackhole", "sinks-console", "sinks-prometheus", "sources", "transforms-lua", "transforms-remap", "unix"] -cli-tests = ["sinks-blackhole", "sinks-socket", "sources-demo_logs", "sources-file"] +shutdown-tests = [ + "api", + "sinks-blackhole", + "sinks-console", + "sinks-prometheus", + "sources", + "transforms-lua", + "transforms-remap", + "unix", +] +cli-tests = [ + "sinks-blackhole", + "sinks-socket", + "sources-demo_logs", + "sources-file", +] test-utils = [] # End-to-End testing-related features -all-e2e-tests = [ - "e2e-tests-datadog" -] +all-e2e-tests = ["e2e-tests-datadog"] e2e-tests-datadog = [ - "sources-datadog_agent", - "sinks-datadog_logs", - "sinks-datadog_metrics" + "sources-datadog_agent", + "sinks-datadog_logs", + "sinks-datadog_metrics", ] vector-api-tests = [ - "sources-demo_logs", - "transforms-log_to_metric", - "transforms-remap", - "sinks-blackhole" + "sources-demo_logs", + "transforms-log_to_metric", + "transforms-remap", + "sinks-blackhole", ] vector-unit-test-tests = [ - "sources-demo_logs", - "transforms-remap", - "transforms-route", - "transforms-filter", - "transforms-reduce", - "sinks-console" + "sources-demo_logs", + "transforms-remap", + "transforms-route", + "transforms-filter", + "transforms-reduce", + "sinks-console", ] -component-validation-runner = ["dep:tonic", "sources-internal_logs", "sources-internal_metrics", "sources-vector", "sinks-vector"] +component-validation-runner = [ + "dep:tonic", + "sources-internal_logs", + "sources-internal_metrics", + "sources-vector", + "sinks-vector", +] # For now, only include components that implement ValidatableComponent. # In the future, this can change to simply reference the targets `sources`, `transforms`, `sinks` component-validation-tests = [ - "component-validation-runner", - "sources-http_client", - "sources-http_server", - "sinks-http", - "sinks-splunk_hec", - "sources-splunk_hec", - "sinks-datadog_logs", - "sources-datadog_agent", + "component-validation-runner", + "sources-http_client", + "sources-http_server", + "sinks-http", + "sinks-splunk_hec", + "sources-splunk_hec", + "sinks-datadog_logs", + "sources-datadog_agent", ] # Grouping together features for benchmarks. We exclude the API client due to it causing the build process to run out # of memory when those additional dependencies are built in CI. benches = [ - "sinks-file", - "sinks-http", - "sinks-socket", - "sources-file", - "sources-socket", - "sources-syslog", - "transforms-lua", - "transforms-sample", + "sinks-file", + "sinks-http", + "sinks-socket", + "sources-file", + "sources-socket", + "sources-syslog", + "transforms-lua", + "transforms-sample", ] dnstap-benches = ["sources-dnstap"] -language-benches = ["sinks-socket", "sources-socket", "transforms-lua", "transforms-remap"] +language-benches = [ + "sinks-socket", + "sources-socket", + "transforms-lua", + "transforms-remap", +] # Separate benching process for metrics due to the nature of the bootstrap procedures. statistic-benches = [] remap-benches = ["transforms-remap"] -transform-benches = ["transforms-filter", "transforms-dedupe", "transforms-reduce", "transforms-route"] +transform-benches = [ + "transforms-filter", + "transforms-dedupe", + "transforms-reduce", + "transforms-route", +] codecs-benches = [] loki-benches = ["sinks-loki"] -enrichment-tables-benches = ["enrichment-tables-geoip", "enrichment-tables-mmdb"] +enrichment-tables-benches = [ + "enrichment-tables-geoip", + "enrichment-tables-mmdb", +] proptest = ["dep:proptest", "dep:proptest-derive"] [[bench]] diff --git a/lib/codecs/Cargo.toml b/lib/codecs/Cargo.toml index baf3565d2d7d0..19c45f73bf146 100644 --- a/lib/codecs/Cargo.toml +++ b/lib/codecs/Cargo.toml @@ -16,13 +16,20 @@ chrono.workspace = true csv-core = { version = "0.1.10", default-features = false } derivative = { version = "2", default-features = false } dyn-clone = { version = "1", default-features = false } -lookup = { package = "vector-lookup", path = "../vector-lookup", default-features = false, features = ["test"] } +lookup = { package = "vector-lookup", path = "../vector-lookup", default-features = false, features = [ + "test", +] } memchr = { version = "2", default-features = false } once_cell = { version = "1.19", default-features = false } ordered-float = { version = "4.2.0", default-features = false } prost = { version = "0.12.6", default-features = false, features = ["std"] } -prost-reflect = { version = "0.13", default-features = false, features = ["serde"] } -regex = { version = "1.10.5", default-features = false, features = ["std", "perf"] } +prost-reflect = { version = "0.13", default-features = false, features = [ + "serde", +] } +regex = { version = "1.10.5", default-features = false, features = [ + "std", + "perf", +] } serde.workspace = true serde_json.workspace = true smallvec = { version = "1", default-features = false, features = ["union"] } @@ -36,15 +43,19 @@ vector-common = { path = "../vector-common", default-features = false } vector-config = { path = "../vector-config", default-features = false } vector-config-common = { path = "../vector-config-common", default-features = false } vector-config-macros = { path = "../vector-config-macros", default-features = false } -vector-core = { path = "../vector-core", default-features = false, features = ["vrl"] } -thiserror.workspace = true +vector-core = { path = "../vector-core", default-features = false, features = [ + "vrl", +] } [dev-dependencies] futures = { version = "0.3", default-features = false } indoc = { version = "2", default-features = false } tokio = { version = "1", features = ["test-util"] } similar-asserts = "1.5.0" -vector-core = { path = "../vector-core", default-features = false, features = ["vrl", "test"] } +vector-core = { path = "../vector-core", default-features = false, features = [ + "vrl", + "test", +] } uuid = { version = "1", default-features = false, features = ["serde", "v4"] } rstest = "0.21.0" tracing-test = "0.2.5" diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index 91f4c019bd0d2..5c7d35c3bddc2 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -1,12 +1,9 @@ -use crate::StreamDecodingError; - -use super::{BoxedFramingError, FramingError}; +use super::BoxedFramingError; use bytes::{Buf, Bytes, BytesMut}; use derivative::Derivative; use std::collections::HashMap; use std::sync::{Arc, Mutex}; use std::time::Duration; -use thiserror::Error; use tokio; use tokio::task::JoinHandle; use tokio_util::codec::Decoder; diff --git a/lib/dnsmsg-parser/Cargo.toml b/lib/dnsmsg-parser/Cargo.toml index 5aeb6fbacb1e3..15a88adf4eecb 100644 --- a/lib/dnsmsg-parser/Cargo.toml +++ b/lib/dnsmsg-parser/Cargo.toml @@ -7,7 +7,7 @@ publish = false license = "MIT" [dependencies] -thiserror.workspace = true +thiserror = "1.0" data-encoding = "2.6" hickory-proto = { version = "0.24", features = ["dnssec"] } From 94b1cf6933d3de8ff228f91f44aba35bbcdf51b4 Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Fri, 5 Jul 2024 13:52:25 +0200 Subject: [PATCH 09/60] style: fix Cargo.toml stlye --- Cargo.toml | 1364 +++++------------ lib/codecs/Cargo.toml | 22 +- .../src/decoding/framing/chunked_gelf.rs | 2 - lib/dnsmsg-parser/Cargo.toml | 2 +- 4 files changed, 395 insertions(+), 995 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index e2d50a4a37129..d9aeab991a3f2 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -9,7 +9,7 @@ license = "MPL-2.0" readme = "README.md" publish = false default-run = "vector" -autobenches = false # our benchmarks are not runnable on their own either way +autobenches = false # our benchmarks are not runnable on their own either way # Minimum supported rust version # See docs/DEVELOPING.md for policy rust-version = "1.78" @@ -56,46 +56,14 @@ section = "admin" maintainer-scripts = "distribution/debian/scripts/" conf-files = ["/etc/vector/vector.yaml", "/etc/default/vector"] assets = [ - [ - "target/release/vector", - "/usr/bin/", - "755", - ], - [ - "config/vector.yaml", - "/etc/vector/vector.yaml", - "644", - ], - [ - "config/examples/*", - "/etc/vector/examples/", - "644", - ], - [ - "distribution/systemd/vector.service", - "/lib/systemd/system/vector.service", - "644", - ], - [ - "distribution/systemd/vector.default", - "/etc/default/vector", - "600", - ], - [ - "licenses/*", - "/usr/share/vector/licenses/", - "644", - ], - [ - "NOTICE", - "/usr/share/vector/NOTICE", - "644", - ], - [ - "LICENSE-3rdparty.csv", - "/usr/share/vector/LICENSE-3rdparty.csv", - "644", - ], + ["target/release/vector", "/usr/bin/", "755"], + ["config/vector.yaml", "/etc/vector/vector.yaml", "644"], + ["config/examples/*", "/etc/vector/examples/", "644"], + ["distribution/systemd/vector.service", "/lib/systemd/system/vector.service", "644"], + ["distribution/systemd/vector.default", "/etc/default/vector", "600"], + ["licenses/*", "/usr/share/vector/licenses/", "644"], + ["NOTICE", "/usr/share/vector/NOTICE", "644"], + ["LICENSE-3rdparty.csv", "/usr/share/vector/LICENSE-3rdparty.csv", "644"], ] license-file = ["target/debian-license.txt"] extended-description-file = "target/debian-extended-description.txt" @@ -129,78 +97,48 @@ depends = "" [workspace] members = [ - ".", - "lib/codecs", - "lib/dnsmsg-parser", - "lib/docs-renderer", - "lib/enrichment", - "lib/fakedata", - "lib/file-source", - "lib/k8s-e2e-tests", - "lib/k8s-test-framework", - "lib/loki-logproto", - "lib/portpicker", - "lib/prometheus-parser", - "lib/opentelemetry-proto", - "lib/tracing-limit", - "lib/vector-api-client", - "lib/vector-buffers", - "lib/vector-common", - "lib/vector-config", - "lib/vector-config-common", - "lib/vector-config-macros", - "lib/vector-core", - "lib/vector-lib", - "lib/vector-lookup", - "lib/vector-stream", - "lib/vector-vrl/cli", - "lib/vector-vrl/functions", - "lib/vector-vrl/tests", - "lib/vector-vrl/web-playground", - "vdev", + ".", + "lib/codecs", + "lib/dnsmsg-parser", + "lib/docs-renderer", + "lib/enrichment", + "lib/fakedata", + "lib/file-source", + "lib/k8s-e2e-tests", + "lib/k8s-test-framework", + "lib/loki-logproto", + "lib/portpicker", + "lib/prometheus-parser", + "lib/opentelemetry-proto", + "lib/tracing-limit", + "lib/vector-api-client", + "lib/vector-buffers", + "lib/vector-common", + "lib/vector-config", + "lib/vector-config-common", + "lib/vector-config-macros", + "lib/vector-core", + "lib/vector-lib", + "lib/vector-lookup", + "lib/vector-stream", + "lib/vector-vrl/cli", + "lib/vector-vrl/functions", + "lib/vector-vrl/tests", + "lib/vector-vrl/web-playground", + "vdev", ] [workspace.dependencies] -chrono = { version = "0.4.37", default-features = false, features = [ - "clock", - "serde", -] } -clap = { version = "4.5.7", default-features = false, features = [ - "derive", - "error-context", - "env", - "help", - "std", - "string", - "usage", - "wrap_help", -] } -indexmap = { version = "2.2.6", default-features = false, features = [ - "serde", - "std", -] } +chrono = { version = "0.4.37", default-features = false, features = ["clock", "serde"] } +clap = { version = "4.5.7", default-features = false, features = ["derive", "error-context", "env", "help", "std", "string", "usage", "wrap_help"] } +indexmap = { version = "2.2.6", default-features = false, features = ["serde", "std"] } pin-project = { version = "1.1.5", default-features = false } proptest = { version = "1.4" } proptest-derive = { version = "0.4.0" } -serde_json = { version = "1.0.117", default-features = false, features = [ - "raw_value", - "std", -] } -serde = { version = "1.0.203", default-features = false, features = [ - "alloc", - "derive", - "rc", -] } -toml = { version = "0.8.14", default-features = false, features = [ - "display", - "parse", -] } -vrl = { version = "0.16.0", features = [ - "arbitrary", - "cli", - "test", - "test_framework", -] } +serde_json = { version = "1.0.117", default-features = false, features = ["raw_value", "std"] } +serde = { version = "1.0.203", default-features = false, features = ["alloc", "derive", "rc"] } +toml = { version = "0.8.14", default-features = false, features = ["display", "parse"] } +vrl = { version = "0.16.0", features = ["arbitrary", "cli", "test", "test_framework"] } tokio = { version = "1.38.0", default-features = false, features = ["full"] } [dependencies] @@ -215,46 +153,25 @@ dnsmsg-parser = { path = "lib/dnsmsg-parser", optional = true } fakedata = { path = "lib/fakedata", optional = true } portpicker = { path = "lib/portpicker" } tracing-limit = { path = "lib/tracing-limit" } -vector-lib = { path = "lib/vector-lib", default-features = false, features = [ - "vrl", -] } +vector-lib = { path = "lib/vector-lib", default-features = false, features = ["vrl"] } vector-vrl-functions = { path = "lib/vector-vrl/functions" } loki-logproto = { path = "lib/loki-logproto", optional = true } # Tokio / Futures async-stream = { version = "0.3.5", default-features = false } async-trait = { version = "0.1.80", default-features = false } -futures = { version = "0.3.30", default-features = false, features = [ - "compat", - "io-compat", -], package = "futures" } +futures = { version = "0.3.30", default-features = false, features = ["compat", "io-compat"], package = "futures" } tokio.workspace = true tokio-openssl = { version = "0.6.4", default-features = false } -tokio-stream = { version = "0.1.15", default-features = false, features = [ - "net", - "sync", - "time", -] } -tokio-util = { version = "0.7", default-features = false, features = [ - "io", - "time", -] } +tokio-stream = { version = "0.1.15", default-features = false, features = ["net", "sync", "time"] } +tokio-util = { version = "0.7", default-features = false, features = ["io", "time"] } console-subscriber = { version = "0.3.0", default-features = false, optional = true } # Tracing tracing = { version = "0.1.34", default-features = false } tracing-core = { version = "0.1.26", default-features = false } -tracing-futures = { version = "0.2.5", default-features = false, features = [ - "futures-03", -] } -tracing-subscriber = { version = "0.3.18", default-features = false, features = [ - "ansi", - "env-filter", - "fmt", - "json", - "registry", - "tracing-log", -] } +tracing-futures = { version = "0.2.5", default-features = false, features = ["futures-03"] } +tracing-subscriber = { version = "0.3.18", default-features = false, features = ["ansi", "env-filter", "fmt", "json", "registry", "tracing-log"] } tracing-tower = { git = "https://github.com/tokio-rs/tracing", default-features = false, rev = "e0642d949891546a3bb7e47080365ee7274f05cd" } # Metrics @@ -262,113 +179,52 @@ metrics = "0.21.1" metrics-tracing-context = { version = "0.14.0", default-features = false } # AWS - Official SDK -aws-sdk-s3 = { version = "1.4.0", default-features = false, features = [ - "behavior-version-latest", -], optional = true } -aws-sdk-sqs = { version = "1.3.0", default-features = false, features = [ - "behavior-version-latest", -], optional = true } -aws-sdk-sns = { version = "1.3.0", default-features = false, features = [ - "behavior-version-latest", -], optional = true } -aws-sdk-cloudwatch = { version = "1.3.0", default-features = false, features = [ - "behavior-version-latest", -], optional = true } -aws-sdk-cloudwatchlogs = { version = "1.3.0", default-features = false, features = [ - "behavior-version-latest", -], optional = true } -aws-sdk-elasticsearch = { version = "1.3.0", default-features = false, features = [ - "behavior-version-latest", -], optional = true } -aws-sdk-firehose = { version = "1.3.0", default-features = false, features = [ - "behavior-version-latest", -], optional = true } -aws-sdk-kinesis = { version = "1.3.0", default-features = false, features = [ - "behavior-version-latest", -], optional = true } -aws-sdk-secretsmanager = { version = "1.3.0", default-features = false, features = [ - "behavior-version-latest", -], optional = true } +aws-sdk-s3 = { version = "1.4.0", default-features = false, features = ["behavior-version-latest"], optional = true } +aws-sdk-sqs = { version = "1.3.0", default-features = false, features = ["behavior-version-latest"], optional = true } +aws-sdk-sns = { version = "1.3.0", default-features = false, features = ["behavior-version-latest"], optional = true } +aws-sdk-cloudwatch = { version = "1.3.0", default-features = false, features = ["behavior-version-latest"], optional = true } +aws-sdk-cloudwatchlogs = { version = "1.3.0", default-features = false, features = ["behavior-version-latest"], optional = true } +aws-sdk-elasticsearch = { version = "1.3.0", default-features = false, features = ["behavior-version-latest"], optional = true } +aws-sdk-firehose = { version = "1.3.0", default-features = false, features = ["behavior-version-latest"], optional = true } +aws-sdk-kinesis = { version = "1.3.0", default-features = false, features = ["behavior-version-latest"], optional = true } +aws-sdk-secretsmanager = { version = "1.3.0", default-features = false, features = ["behavior-version-latest"], optional = true } # The sts crate is needed despite not being referred to anywhere in the code because we need to set the # `behavior-version-latest` feature. Without this we get a runtime panic when `auth.assume_role` authentication # is configured. -aws-sdk-sts = { version = "1.3.1", default-features = false, features = [ - "behavior-version-latest", -], optional = true } +aws-sdk-sts = { version = "1.3.1", default-features = false, features = ["behavior-version-latest"], optional = true } aws-types = { version = "1.3.2", default-features = false, optional = true } -aws-sigv4 = { version = "1.2.2", default-features = false, features = [ - "sign-http", -], optional = true } -aws-config = { version = "1.0.1", default-features = false, features = [ - "behavior-version-latest", - "credentials-process", -], optional = true } -aws-credential-types = { version = "1.2.0", default-features = false, features = [ - "hardcoded-credentials", -], optional = true } -aws-smithy-http = { version = "0.60", default-features = false, features = [ - "event-stream", -], optional = true } +aws-sigv4 = { version = "1.2.2", default-features = false, features = ["sign-http"], optional = true } +aws-config = { version = "1.0.1", default-features = false, features = ["behavior-version-latest", "credentials-process"], optional = true } +aws-credential-types = { version = "1.2.0", default-features = false, features = ["hardcoded-credentials"], optional = true } +aws-smithy-http = { version = "0.60", default-features = false, features = ["event-stream"], optional = true } aws-smithy-types = { version = "1.2.0", default-features = false, optional = true } aws-smithy-runtime-api = { version = "1.7.0", default-features = false, optional = true } -aws-smithy-runtime = { version = "1.6.0", default-features = false, features = [ - "client", - "connector-hyper-0-14-x", - "rt-tokio", -], optional = true } -aws-smithy-async = { version = "1.2.1", default-features = false, features = [ - "rt-tokio", -], optional = true } +aws-smithy-runtime = { version = "1.6.0", default-features = false, features = ["client", "connector-hyper-0-14-x", "rt-tokio"], optional = true } +aws-smithy-async = { version = "1.2.1", default-features = false, features = ["rt-tokio"], optional = true } # Azure -azure_core = { version = "0.17", default-features = false, features = [ - "enable_reqwest", -], optional = true } -azure_identity = { version = "0.17", default-features = false, features = [ - "enable_reqwest", -], optional = true } +azure_core = { version = "0.17", default-features = false, features = ["enable_reqwest"], optional = true } +azure_identity = { version = "0.17", default-features = false, features = ["enable_reqwest"], optional = true } azure_storage = { version = "0.17", default-features = false, optional = true } azure_storage_blobs = { version = "0.17", default-features = false, optional = true } # OpenDAL -opendal = { version = "0.45", default-features = false, features = [ - "native-tls", - "services-webhdfs", -], optional = true } +opendal = {version = "0.45", default-features = false, features = ["native-tls", "services-webhdfs"], optional = true} # Tower -tower = { version = "0.4.13", default-features = false, features = [ - "buffer", - "limit", - "retry", - "timeout", - "util", - "balance", - "discover", -] } -tower-http = { version = "0.4.4", default-features = false, features = [ - "compression-full", - "decompression-gzip", - "trace", -] } +tower = { version = "0.4.13", default-features = false, features = ["buffer", "limit", "retry", "timeout", "util", "balance", "discover"] } +tower-http = { version = "0.4.4", default-features = false, features = ["compression-full", "decompression-gzip", "trace"]} # Serde serde.workspace = true serde-toml-merge = { version = "0.3.8", default-features = false } -serde_bytes = { version = "0.11.14", default-features = false, features = [ - "std", -], optional = true } +serde_bytes = { version = "0.11.14", default-features = false, features = ["std"], optional = true } serde_json.workspace = true -serde_with = { version = "3.8.1", default-features = false, features = [ - "macros", - "std", -] } +serde_with = { version = "3.8.1", default-features = false, features = ["macros", "std"] } serde_yaml = { version = "0.9.34", default-features = false } # Messagepack rmp-serde = { version = "1.3.0", default-features = false, optional = true } -rmpv = { version = "1.3.0", default-features = false, features = [ - "with-serde", -], optional = true } +rmpv = { version = "1.3.0", default-features = false, features = ["with-serde"], optional = true } # Prost / Protocol Buffers prost = { version = "0.12", default-features = false, features = ["std"] } @@ -380,31 +236,17 @@ goauth = { version = "0.14.0", optional = true } smpl_jwt = { version = "0.8.0", default-features = false, optional = true } # AMQP -lapin = { version = "2.3.4", default-features = false, features = [ - "native-tls", -], optional = true } +lapin = { version = "2.3.4", default-features = false, features = ["native-tls"], optional = true } # API -async-graphql = { version = "7.0.6", default-features = false, optional = true, features = [ - "chrono", - "playground", -] } +async-graphql = { version = "7.0.6", default-features = false, optional = true, features = ["chrono", "playground"] } async-graphql-warp = { version = "7.0.6", default-features = false, optional = true } # API client -crossterm = { version = "0.27.0", default-features = false, features = [ - "event-stream", - "windows", -], optional = true } -num-format = { version = "0.4.4", default-features = false, features = [ - "with-num-bigint", -], optional = true } -number_prefix = { version = "0.4.0", default-features = false, features = [ - "std", -], optional = true } -ratatui = { version = "0.26.3", optional = true, default-features = false, features = [ - "crossterm", -] } +crossterm = { version = "0.27.0", default-features = false, features = ["event-stream", "windows"], optional = true } +num-format = { version = "0.4.4", default-features = false, features = ["with-num-bigint"], optional = true } +number_prefix = { version = "0.4.0", default-features = false, features = ["std"], optional = true } +ratatui = { version = "0.26.3", optional = true, default-features = false, features = ["crossterm"] } # Opentelemetry @@ -415,19 +257,12 @@ greptimedb-client = { git = "https://github.com/GreptimeTeam/greptimedb-ingester # External libs arc-swap = { version = "1.7", default-features = false, optional = true } -async-compression = { version = "0.4.11", default-features = false, features = [ - "tokio", - "gzip", - "zstd", -], optional = true } +async-compression = { version = "0.4.11", default-features = false, features = ["tokio", "gzip", "zstd"], optional = true } apache-avro = { version = "0.16.0", default-features = false, optional = true } axum = { version = "0.6.20", default-features = false } base64 = { version = "0.22.1", default-features = false, optional = true } -bloomy = { version = "1.2.0", default-features = false, optional = true } -bollard = { version = "0.16.1", default-features = false, features = [ - "ssl", - "chrono", -], optional = true } +bloomy = { version = "1.2.0", default-features = false, optional = true } +bollard = { version = "0.16.1", default-features = false, features = ["ssl", "chrono"], optional = true } bytes = { version = "1.6.0", default-features = false, features = ["serde"] } bytesize = { version = "1.3.0", default-features = false } chrono.workspace = true @@ -435,163 +270,77 @@ chrono-tz = { version = "0.9.0", default-features = false } cidr-utils = { version = "0.6.1", default-features = false } colored = { version = "2.1.0", default-features = false } csv = { version = "1.3", default-features = false } -databend-client = { version = "0.18.3", default-features = false, features = [ - "rustls", -], optional = true } +databend-client ={ version = "0.18.3", default-features = false, features = ["rustls"], optional = true } derivative = { version = "2.2.0", default-features = false } dirs-next = { version = "2.0.0", default-features = false, optional = true } dyn-clone = { version = "1.0.17", default-features = false } -encoding_rs = { version = "0.8.34", default-features = false, features = [ - "serde", -] } +encoding_rs = { version = "0.8.34", default-features = false, features = ["serde"] } enum_dispatch = { version = "0.3.13", default-features = false } exitcode = { version = "1.1.2", default-features = false } -flate2 = { version = "1.0.30", default-features = false, features = [ - "default", -] } +flate2 = { version = "1.0.30", default-features = false, features = ["default"] } futures-util = { version = "0.3.29", default-features = false } glob = { version = "0.3.1", default-features = false } -governor = { version = "0.6.3", default-features = false, features = [ - "dashmap", - "jitter", - "std", -], optional = true } +governor = { version = "0.6.3", default-features = false, features = ["dashmap", "jitter", "std"], optional = true } grok = { version = "2.0.0", default-features = false, optional = true } h2 = { version = "0.4.5", default-features = false, optional = true } hash_hasher = { version = "2.0.0", default-features = false } -hashbrown = { version = "0.14.5", default-features = false, optional = true, features = [ - "ahash", -] } +hashbrown = { version = "0.14.5", default-features = false, optional = true, features = ["ahash"] } headers = { version = "0.3.9", default-features = false } hostname = { version = "0.4.0", default-features = false } http = { version = "0.2.9", default-features = false } http-serde = "1.1.3" http-body = { version = "0.4.5", default-features = false } -hyper = { version = "0.14.28", default-features = false, features = [ - "client", - "runtime", - "http1", - "http2", - "server", - "stream", -] } +hyper = { version = "0.14.28", default-features = false, features = ["client", "runtime", "http1", "http2", "server", "stream"] } hyper-openssl = { version = "0.9.2", default-features = false } -hyper-proxy = { version = "0.9.1", default-features = false, features = [ - "openssl-tls", -] } +hyper-proxy = { version = "0.9.1", default-features = false, features = ["openssl-tls"] } indexmap.workspace = true -infer = { version = "0.16.0", default-features = false, optional = true } +infer = { version = "0.16.0", default-features = false, optional = true} indoc = { version = "2.0.5", default-features = false } inventory = { version = "0.3.15", default-features = false } -ipnet = { version = "2", default-features = false, optional = true, features = [ - "serde", - "std", -] } -itertools = { version = "0.13.0", default-features = false, optional = false, features = [ - "use_alloc", -] } -k8s-openapi = { version = "0.18.0", default-features = false, features = [ - "api", - "v1_26", -], optional = true } -kube = { version = "0.82.0", default-features = false, features = [ - "client", - "openssl-tls", - "runtime", -], optional = true } +ipnet = { version = "2", default-features = false, optional = true, features = ["serde", "std"] } +itertools = { version = "0.13.0", default-features = false, optional = false, features = ["use_alloc"] } +k8s-openapi = { version = "0.18.0", default-features = false, features = ["api", "v1_26"], optional = true } +kube = { version = "0.82.0", default-features = false, features = ["client", "openssl-tls", "runtime"], optional = true } listenfd = { version = "1.0.1", default-features = false, optional = true } logfmt = { version = "0.0.2", default-features = false, optional = true } lru = { version = "0.12.3", default-features = false, optional = true } maxminddb = { version = "0.24.0", default-features = false, optional = true } md-5 = { version = "0.10", default-features = false, optional = true } -mongodb = { version = "2.8.2", default-features = false, features = [ - "tokio-runtime", -], optional = true } +mongodb = { version = "2.8.2", default-features = false, features = ["tokio-runtime"], optional = true } async-nats = { version = "0.33.0", default-features = false, optional = true } nkeys = { version = "0.4.1", default-features = false, optional = true } nom = { version = "7.1.3", default-features = false, optional = true } -notify = { version = "6.1.1", default-features = false, features = [ - "macos_fsevent", -] } +notify = { version = "6.1.1", default-features = false, features = ["macos_fsevent"] } once_cell = { version = "1.19", default-features = false } -openssl = { version = "0.10.64", default-features = false, features = [ - "vendored", -] } +openssl = { version = "0.10.64", default-features = false, features = ["vendored"] } openssl-probe = { version = "0.1.5", default-features = false } ordered-float = { version = "4.2.0", default-features = false } paste = "1.0.15" percent-encoding = { version = "2.3.1", default-features = false } -postgres-openssl = { version = "0.5.0", default-features = false, features = [ - "runtime", -], optional = true } -pulsar = { version = "6.3.0", default-features = false, features = [ - "tokio-runtime", - "auth-oauth2", - "flate2", - "lz4", - "snap", - "zstd", -], optional = true } +postgres-openssl = { version = "0.5.0", default-features = false, features = ["runtime"], optional = true } +pulsar = { version = "6.3.0", default-features = false, features = ["tokio-runtime", "auth-oauth2", "flate2", "lz4", "snap", "zstd"], optional = true } rand = { version = "0.8.5", default-features = false, features = ["small_rng"] } rand_distr = { version = "0.4.3", default-features = false } -rdkafka = { version = "0.35.0", default-features = false, features = [ - "tokio", - "libz", - "ssl", - "zstd", -], optional = true } -redis = { version = "0.24.0", default-features = false, features = [ - "connection-manager", - "tokio-comp", - "tokio-native-tls-comp", -], optional = true } -regex = { version = "1.10.5", default-features = false, features = [ - "std", - "perf", -] } -roaring = { version = "0.10.5", default-features = false, features = [ - "std", -], optional = true } -rumqttc = { version = "0.24.0", default-features = false, features = [ - "use-rustls", -], optional = true } +rdkafka = { version = "0.35.0", default-features = false, features = ["tokio", "libz", "ssl", "zstd"], optional = true } +redis = { version = "0.24.0", default-features = false, features = ["connection-manager", "tokio-comp", "tokio-native-tls-comp"], optional = true } +regex = { version = "1.10.5", default-features = false, features = ["std", "perf"] } +roaring = { version = "0.10.5", default-features = false, features = ["std"], optional = true } +rumqttc = { version = "0.24.0", default-features = false, features = ["use-rustls"], optional = true } seahash = { version = "4.1.0", default-features = false } -semver = { version = "1.0.23", default-features = false, features = [ - "serde", - "std", -], optional = true } -smallvec = { version = "1", default-features = false, features = [ - "union", - "serde", -] } +semver = { version = "1.0.23", default-features = false, features = ["serde", "std"], optional = true } +smallvec = { version = "1", default-features = false, features = ["union", "serde"] } snafu = { version = "0.7.5", default-features = false, features = ["futures"] } snap = { version = "1.1.1", default-features = false } socket2 = { version = "0.5.7", default-features = false } stream-cancel = { version = "0.8.2", default-features = false } strip-ansi-escapes = { version = "0.2.0", default-features = false } syslog = { version = "6.1.1", default-features = false, optional = true } -tikv-jemallocator = { version = "0.5.4", default-features = false, features = [ - "unprefixed_malloc_on_supported_platforms", -], optional = true } -tokio-postgres = { version = "0.7.10", default-features = false, features = [ - "runtime", - "with-chrono-0_4", -], optional = true } -tokio-tungstenite = { version = "0.20.1", default-features = false, features = [ - "connect", -], optional = true } +tikv-jemallocator = { version = "0.5.4", default-features = false, features = ["unprefixed_malloc_on_supported_platforms"], optional = true } +tokio-postgres = { version = "0.7.10", default-features = false, features = ["runtime", "with-chrono-0_4"], optional = true } +tokio-tungstenite = {version = "0.20.1", default-features = false, features = ["connect"], optional = true} toml.workspace = true -tonic = { version = "0.10", optional = true, default-features = false, features = [ - "transport", - "codegen", - "prost", - "tls", - "tls-roots", - "gzip", -] } -hickory-proto = { version = "0.24.1", default-features = false, features = [ - "dnssec", -], optional = true } +tonic = { version = "0.10", optional = true, default-features = false, features = ["transport", "codegen", "prost", "tls", "tls-roots", "gzip"] } +hickory-proto = { version = "0.24.1", default-features = false, features = ["dnssec"], optional = true } typetag = { version = "0.2.16", default-features = false } url = { version = "2.5.1", default-features = false, features = ["serde"] } uuid = { version = "1", default-features = false, features = ["serde", "v4"] } @@ -601,76 +350,46 @@ arr_macro = { version = "0.2.1" } # depending on fork for bumped nix dependency # https://github.com/heim-rs/heim/pull/360 -heim = { git = "https://github.com/vectordotdev/heim.git", branch = "update-nix", default-features = false, features = [ - "disk", -] } +heim = { git = "https://github.com/vectordotdev/heim.git", branch = "update-nix", default-features = false, features = ["disk"] } # make sure to update the external docs when the Lua version changes -mlua = { version = "0.9.9", default-features = false, features = [ - "lua54", - "send", - "vendored", - "macros", -], optional = true } +mlua = { version = "0.9.9", default-features = false, features = ["lua54", "send", "vendored", "macros"], optional = true } [target.'cfg(windows)'.dependencies] windows-service = "0.7.0" [target.'cfg(unix)'.dependencies] -nix = { version = "0.26.2", default-features = false, features = [ - "socket", - "signal", -] } +nix = { version = "0.26.2", default-features = false, features = ["socket", "signal"] } [build-dependencies] prost-build = { version = "0.12", default-features = false, optional = true } -tonic-build = { version = "0.10", default-features = false, features = [ - "transport", - "prost", -], optional = true } +tonic-build = { version = "0.10", default-features = false, features = ["transport", "prost"], optional = true } # update 'openssl_version' in website/config.toml whenever version changes -openssl-src = { version = "300", default-features = false, features = [ - "force-engine", - "legacy", -] } +openssl-src = { version = "300", default-features = false, features = ["force-engine", "legacy"] } [dev-dependencies] approx = "0.5.1" assert_cmd = { version = "2.0.14", default-features = false } -aws-smithy-runtime = { version = "1.6.0", default-features = false, features = [ - "tls-rustls", -] } -azure_core = { version = "0.17", default-features = false, features = [ - "enable_reqwest", - "azurite_workaround", -] } -azure_identity = { version = "0.17", default-features = false, features = [ - "enable_reqwest", -] } -azure_storage_blobs = { version = "0.17", default-features = false, features = [ - "azurite_workaround", -] } +aws-smithy-runtime = { version = "1.6.0", default-features = false, features = ["tls-rustls"] } +azure_core = { version = "0.17", default-features = false, features = ["enable_reqwest", "azurite_workaround"] } +azure_identity = { version = "0.17", default-features = false, features = ["enable_reqwest"] } +azure_storage_blobs = { version = "0.17", default-features = false, features = ["azurite_workaround"] } azure_storage = { version = "0.17", default-features = false } base64 = "0.22.1" criterion = { version = "0.5.1", features = ["html_reports", "async_tokio"] } -itertools = { version = "0.13.0", default-features = false, features = [ - "use_alloc", -] } +itertools = { version = "0.13.0", default-features = false, features = ["use_alloc"] } libc = "0.2.155" similar-asserts = "1.5.0" proptest.workspace = true quickcheck = "1.0.3" reqwest = { version = "0.11", features = ["json"] } -rstest = { version = "0.21.0" } +rstest = {version = "0.21.0"} tempfile = "3.10.1" test-generator = "0.3.1" tokio = { version = "1.38.0", features = ["test-util"] } tokio-test = "0.4.4" tower-test = "0.4.0" -vector-lib = { path = "lib/vector-lib", default-features = false, features = [ - "vrl", - "test", -] } +vector-lib = { path = "lib/vector-lib", default-features = false, features = ["vrl", "test"] } vrl.workspace = true wiremock = "0.5.22" @@ -686,90 +405,17 @@ ntapi = { git = "https://github.com/MSxDOS/ntapi.git", rev = "24fc1e47677fc9f6e3 [features] # Default features for *-unknown-linux-gnu and *-apple-darwin -default = [ - "api", - "api-client", - "enrichment-tables", - "sinks", - "sources", - "sources-dnstap", - "transforms", - "unix", - "rdkafka?/gssapi-vendored", - "secrets", -] +default = ["api", "api-client", "enrichment-tables", "sinks", "sources", "sources-dnstap", "transforms", "unix", "rdkafka?/gssapi-vendored", "secrets"] # Default features for `cargo docs`. The same as `default` but without `rdkafka?/gssapi-vendored` which would require installing libsasl in our doc build environment. -docs = [ - "api", - "api-client", - "enrichment-tables", - "sinks", - "sources", - "sources-dnstap", - "transforms", - "unix", - "secrets", -] +docs = ["api", "api-client", "enrichment-tables", "sinks", "sources", "sources-dnstap", "transforms", "unix", "secrets"] # Default features for *-unknown-linux-* which make use of `cmake` for dependencies -default-cmake = [ - "api", - "api-client", - "enrichment-tables", - "rdkafka?/cmake_build", - "sinks", - "sources", - "sources-dnstap", - "transforms", - "unix", - "rdkafka?/gssapi-vendored", - "secrets", -] +default-cmake = ["api", "api-client", "enrichment-tables", "rdkafka?/cmake_build", "sinks", "sources", "sources-dnstap", "transforms", "unix", "rdkafka?/gssapi-vendored", "secrets"] # Default features for *-pc-windows-msvc # TODO: Enable SASL https://github.com/vectordotdev/vector/pull/3081#issuecomment-659298042 -default-msvc = [ - "api", - "api-client", - "enrichment-tables", - "rdkafka?/cmake_build", - "sinks", - "sources", - "transforms", - "secrets", -] -default-musl = [ - "api", - "api-client", - "enrichment-tables", - "rdkafka?/cmake_build", - "sinks", - "sources", - "sources-dnstap", - "transforms", - "unix", - "rdkafka?/gssapi-vendored", - "secrets", -] -default-no-api-client = [ - "api", - "enrichment-tables", - "sinks", - "sources", - "sources-dnstap", - "transforms", - "unix", - "rdkafka?/gssapi-vendored", - "secrets", -] -default-no-vrl-cli = [ - "api", - "sinks", - "sources", - "sources-dnstap", - "transforms", - "unix", - "rdkafka?/gssapi-vendored", - "secrets", -] +default-msvc = ["api", "api-client", "enrichment-tables", "rdkafka?/cmake_build", "sinks", "sources", "transforms", "secrets"] +default-musl = ["api", "api-client", "enrichment-tables", "rdkafka?/cmake_build", "sinks", "sources", "sources-dnstap", "transforms", "unix", "rdkafka?/gssapi-vendored", "secrets"] +default-no-api-client = ["api", "enrichment-tables", "sinks", "sources", "sources-dnstap", "transforms", "unix", "rdkafka?/gssapi-vendored", "secrets"] +default-no-vrl-cli = ["api", "sinks", "sources", "sources-dnstap", "transforms", "unix", "rdkafka?/gssapi-vendored", "secrets"] tokio-console = ["dep:console-subscriber", "tokio/tracing"] # Enables the binary secret-backend-example @@ -781,127 +427,18 @@ all-metrics = ["sinks-metrics", "sources-metrics", "transforms-metrics"] # Target specific release features. # The `make` tasks will select this according to the appropriate triple. # Use this section to turn off or on specific features for specific triples. -target-aarch64-unknown-linux-gnu = [ - "api", - "api-client", - "enrichment-tables", - "rdkafka?/cmake_build", - "sinks", - "sources", - "sources-dnstap", - "transforms", - "unix", - "secrets", -] -target-aarch64-unknown-linux-musl = [ - "api", - "api-client", - "enrichment-tables", - "rdkafka?/cmake_build", - "sinks", - "sources", - "sources-dnstap", - "transforms", - "unix", - "secrets", -] -target-armv7-unknown-linux-gnueabihf = [ - "api", - "api-client", - "enrichment-tables", - "rdkafka?/cmake_build", - "sinks", - "sources", - "sources-dnstap", - "transforms", - "unix", - "secrets", -] -target-armv7-unknown-linux-musleabihf = [ - "api", - "api-client", - "rdkafka?/cmake_build", - "enrichment-tables", - "sinks", - "sources", - "sources-dnstap", - "transforms", - "secrets", -] -target-arm-unknown-linux-gnueabi = [ - "api", - "api-client", - "enrichment-tables", - "rdkafka?/cmake_build", - "sinks", - "sources", - "sources-dnstap", - "transforms", - "unix", - "secrets", -] -target-arm-unknown-linux-musleabi = [ - "api", - "api-client", - "rdkafka?/cmake_build", - "enrichment-tables", - "sinks", - "sources", - "sources-dnstap", - "transforms", - "secrets", -] -target-x86_64-unknown-linux-gnu = [ - "api", - "api-client", - "rdkafka?/cmake_build", - "enrichment-tables", - "sinks", - "sources", - "sources-dnstap", - "transforms", - "unix", - "rdkafka?/gssapi-vendored", - "secrets", -] -target-x86_64-unknown-linux-musl = [ - "api", - "api-client", - "rdkafka?/cmake_build", - "enrichment-tables", - "sinks", - "sources", - "sources-dnstap", - "transforms", - "unix", - "secrets", -] +target-aarch64-unknown-linux-gnu = ["api", "api-client", "enrichment-tables", "rdkafka?/cmake_build", "sinks", "sources", "sources-dnstap", "transforms", "unix", "secrets"] +target-aarch64-unknown-linux-musl = ["api", "api-client", "enrichment-tables", "rdkafka?/cmake_build", "sinks", "sources", "sources-dnstap", "transforms", "unix", "secrets"] +target-armv7-unknown-linux-gnueabihf = ["api", "api-client", "enrichment-tables", "rdkafka?/cmake_build", "sinks", "sources", "sources-dnstap", "transforms", "unix", "secrets"] +target-armv7-unknown-linux-musleabihf = ["api", "api-client", "rdkafka?/cmake_build", "enrichment-tables", "sinks", "sources", "sources-dnstap", "transforms", "secrets"] +target-arm-unknown-linux-gnueabi = ["api", "api-client", "enrichment-tables", "rdkafka?/cmake_build", "sinks", "sources", "sources-dnstap", "transforms", "unix", "secrets"] +target-arm-unknown-linux-musleabi = ["api", "api-client", "rdkafka?/cmake_build", "enrichment-tables", "sinks", "sources", "sources-dnstap", "transforms", "secrets"] +target-x86_64-unknown-linux-gnu = ["api", "api-client", "rdkafka?/cmake_build", "enrichment-tables", "sinks", "sources", "sources-dnstap", "transforms", "unix", "rdkafka?/gssapi-vendored", "secrets"] +target-x86_64-unknown-linux-musl = ["api", "api-client", "rdkafka?/cmake_build", "enrichment-tables", "sinks", "sources", "sources-dnstap", "transforms", "unix", "secrets"] # Does not currently build -target-powerpc64le-unknown-linux-gnu = [ - "api", - "api-client", - "enrichment-tables", - "rdkafka?/cmake_build", - "sinks", - "sources", - "sources-dnstap", - "transforms", - "unix", - "secrets", -] +target-powerpc64le-unknown-linux-gnu = ["api", "api-client", "enrichment-tables", "rdkafka?/cmake_build", "sinks", "sources", "sources-dnstap", "transforms", "unix", "secrets"] # Currently doesn't build due to lack of support for 64-bit atomics -target-powerpc-unknown-linux-gnu = [ - "api", - "api-client", - "enrichment-tables", - "rdkafka?/cmake_build", - "sinks", - "sources", - "sources-dnstap", - "transforms", - "unix", - "secrets", -] +target-powerpc-unknown-linux-gnu = ["api", "api-client", "enrichment-tables", "rdkafka?/cmake_build", "sinks", "sources", "sources-dnstap", "transforms", "unix", "secrets"] # Enables features that work only on systems providing `cfg(unix)` unix = ["tikv-jemallocator", "allocation-tracing"] @@ -915,33 +452,33 @@ docker = ["dep:bollard", "dep:dirs-next"] # API api = [ - "dep:async-graphql", - "dep:async-graphql-warp", - "dep:base64", - "vector-lib/api", + "dep:async-graphql", + "dep:async-graphql-warp", + "dep:base64", + "vector-lib/api", ] # API client api-client = [ - "dep:crossterm", - "dep:num-format", - "dep:number_prefix", - "dep:ratatui", - "vector-lib/api", - "vector-lib/api-client", + "dep:crossterm", + "dep:num-format", + "dep:number_prefix", + "dep:ratatui", + "vector-lib/api", + "vector-lib/api-client", ] aws-core = [ - "aws-config", - "dep:aws-credential-types", - "dep:aws-sigv4", - "dep:aws-types", - "dep:aws-smithy-async", - "dep:aws-smithy-http", - "dep:aws-smithy-types", - "dep:aws-smithy-runtime", - "dep:aws-smithy-runtime-api", - "dep:aws-sdk-sts", + "aws-config", + "dep:aws-credential-types", + "dep:aws-sigv4", + "dep:aws-types", + "dep:aws-smithy-async", + "dep:aws-smithy-http", + "dep:aws-smithy-types", + "dep:aws-smithy-runtime", + "dep:aws-smithy-runtime-api", + "dep:aws-sdk-sts", ] # Anything that requires Protocol Buffers. @@ -965,172 +502,100 @@ secrets-aws-secrets-manager = ["aws-core", "dep:aws-sdk-secretsmanager"] # Sources sources = ["sources-logs", "sources-metrics"] sources-logs = [ - "sources-amqp", - "sources-aws_kinesis_firehose", - "sources-aws_s3", - "sources-aws_sqs", - "sources-datadog_agent", - "sources-demo_logs", - "sources-docker_logs", - "sources-exec", - "sources-file", - "sources-fluent", - "sources-gcp_pubsub", - "sources-heroku_logs", - "sources-http_server", - "sources-http_client", - "sources-internal_logs", - "sources-journald", - "sources-kafka", - "sources-kubernetes_logs", - "sources-logstash", - "sources-nats", - "sources-opentelemetry", - "sources-pulsar", - "sources-file-descriptor", - "sources-redis", - "sources-socket", - "sources-splunk_hec", - "sources-stdin", - "sources-syslog", - "sources-vector", + "sources-amqp", + "sources-aws_kinesis_firehose", + "sources-aws_s3", + "sources-aws_sqs", + "sources-datadog_agent", + "sources-demo_logs", + "sources-docker_logs", + "sources-exec", + "sources-file", + "sources-fluent", + "sources-gcp_pubsub", + "sources-heroku_logs", + "sources-http_server", + "sources-http_client", + "sources-internal_logs", + "sources-journald", + "sources-kafka", + "sources-kubernetes_logs", + "sources-logstash", + "sources-nats", + "sources-opentelemetry", + "sources-pulsar", + "sources-file-descriptor", + "sources-redis", + "sources-socket", + "sources-splunk_hec", + "sources-stdin", + "sources-syslog", + "sources-vector", ] sources-metrics = [ - "sources-apache_metrics", - "sources-aws_ecs_metrics", - "sources-eventstoredb_metrics", - "sources-host_metrics", - "sources-internal_metrics", - "sources-mongodb_metrics", - "sources-nginx_metrics", - "sources-postgresql_metrics", - "sources-prometheus", - "sources-statsd", - "sources-vector", + "sources-apache_metrics", + "sources-aws_ecs_metrics", + "sources-eventstoredb_metrics", + "sources-host_metrics", + "sources-internal_metrics", + "sources-mongodb_metrics", + "sources-nginx_metrics", + "sources-postgresql_metrics", + "sources-prometheus", + "sources-statsd", + "sources-vector", ] sources-amqp = ["lapin"] sources-apache_metrics = ["sources-utils-http-client"] sources-aws_ecs_metrics = ["sources-utils-http-client"] sources-aws_kinesis_firehose = ["dep:base64", "dep:infer"] -sources-aws_s3 = [ - "aws-core", - "dep:aws-sdk-sqs", - "dep:aws-sdk-s3", - "dep:semver", - "dep:async-compression", - "sources-aws_sqs", - "tokio-util/io", -] +sources-aws_s3 = ["aws-core", "dep:aws-sdk-sqs", "dep:aws-sdk-s3", "dep:semver", "dep:async-compression", "sources-aws_sqs", "tokio-util/io"] sources-aws_sqs = ["aws-core", "dep:aws-sdk-sqs"] sources-datadog_agent = ["sources-utils-http-error", "protobuf-build"] sources-demo_logs = ["dep:fakedata"] -sources-dnstap = [ - "sources-utils-net-tcp", - "dep:base64", - "dep:hickory-proto", - "dep:dnsmsg-parser", - "protobuf-build", -] +sources-dnstap = ["sources-utils-net-tcp", "dep:base64", "dep:hickory-proto", "dep:dnsmsg-parser", "protobuf-build"] sources-docker_logs = ["docker"] sources-eventstoredb_metrics = [] sources-exec = [] sources-file = ["vector-lib/file-source"] sources-file-descriptor = ["tokio-util/io"] -sources-fluent = [ - "dep:base64", - "sources-utils-net-tcp", - "tokio-util/net", - "dep:rmpv", - "dep:rmp-serde", - "dep:serde_bytes", -] -sources-gcp_pubsub = [ - "gcp", - "dep:h2", - "dep:prost-types", - "protobuf-build", - "dep:tonic", -] -sources-heroku_logs = [ - "sources-utils-http", - "sources-utils-http-query", - "sources-http_server", -] -sources-host_metrics = ["heim/cpu", "heim/host", "heim/memory", "heim/net"] +sources-fluent = ["dep:base64", "sources-utils-net-tcp", "tokio-util/net", "dep:rmpv", "dep:rmp-serde", "dep:serde_bytes"] +sources-gcp_pubsub = ["gcp", "dep:h2", "dep:prost-types", "protobuf-build", "dep:tonic"] +sources-heroku_logs = ["sources-utils-http", "sources-utils-http-query", "sources-http_server"] +sources-host_metrics = ["heim/cpu", "heim/host", "heim/memory", "heim/net"] sources-http_client = ["sources-utils-http-client"] sources-http_server = ["sources-utils-http", "sources-utils-http-query"] sources-internal_logs = [] sources-internal_metrics = [] sources-journald = [] sources-kafka = ["dep:rdkafka"] -sources-kubernetes_logs = [ - "vector-lib/file-source", - "kubernetes", - "transforms-reduce", -] +sources-kubernetes_logs = ["vector-lib/file-source", "kubernetes", "transforms-reduce"] sources-logstash = ["sources-utils-net-tcp", "tokio-util/net"] sources-mongodb_metrics = ["dep:mongodb"] sources-nats = ["dep:async-nats", "dep:nkeys"] sources-nginx_metrics = ["dep:nom"] -sources-opentelemetry = [ - "dep:hex", - "vector-lib/opentelemetry", - "dep:prost-types", - "sources-http_server", - "sources-utils-http", - "sources-vector", -] +sources-opentelemetry = ["dep:hex", "vector-lib/opentelemetry", "dep:prost-types", "sources-http_server", "sources-utils-http", "sources-vector"] sources-postgresql_metrics = ["dep:postgres-openssl", "dep:tokio-postgres"] -sources-prometheus = [ - "sources-prometheus-scrape", - "sources-prometheus-remote-write", - "sources-prometheus-pushgateway", -] -sources-prometheus-scrape = [ - "sinks-prometheus", - "sources-utils-http-client", - "vector-lib/prometheus", -] -sources-prometheus-remote-write = [ - "sinks-prometheus", - "sources-utils-http", - "vector-lib/prometheus", -] -sources-prometheus-pushgateway = [ - "sinks-prometheus", - "sources-utils-http", - "vector-lib/prometheus", -] +sources-prometheus = ["sources-prometheus-scrape", "sources-prometheus-remote-write", "sources-prometheus-pushgateway"] +sources-prometheus-scrape = ["sinks-prometheus", "sources-utils-http-client", "vector-lib/prometheus"] +sources-prometheus-remote-write = ["sinks-prometheus", "sources-utils-http", "vector-lib/prometheus"] +sources-prometheus-pushgateway = ["sinks-prometheus", "sources-utils-http", "vector-lib/prometheus"] sources-pulsar = ["dep:apache-avro", "dep:pulsar"] -sources-redis = ["dep:redis"] +sources-redis= ["dep:redis"] sources-socket = ["sources-utils-net", "tokio-util/net"] sources-splunk_hec = ["dep:roaring"] sources-statsd = ["sources-utils-net", "tokio-util/net"] sources-stdin = ["tokio-util/io"] sources-syslog = ["codecs-syslog", "sources-utils-net", "tokio-util/net"] -sources-utils-http = [ - "sources-utils-http-auth", - "sources-utils-http-encoding", - "sources-utils-http-error", - "sources-utils-http-prelude", -] +sources-utils-http = ["sources-utils-http-auth", "sources-utils-http-encoding", "sources-utils-http-error", "sources-utils-http-prelude"] sources-utils-http-auth = ["sources-utils-http-error"] sources-utils-http-encoding = ["sources-utils-http-error"] sources-utils-http-error = [] -sources-utils-http-prelude = [ - "sources-utils-http", - "sources-utils-http-auth", - "sources-utils-http-encoding", - "sources-utils-http-error", -] +sources-utils-http-prelude = ["sources-utils-http", "sources-utils-http-auth", "sources-utils-http-encoding", "sources-utils-http-error"] sources-utils-http-query = [] sources-utils-http-client = ["sources-utils-http", "sources-http_server"] -sources-utils-net = [ - "sources-utils-net-tcp", - "sources-utils-net-udp", - "sources-utils-net-unix", -] +sources-utils-net = ["sources-utils-net-tcp", "sources-utils-net-udp", "sources-utils-net-unix"] sources-utils-net-tcp = ["listenfd", "dep:ipnet"] sources-utils-net-udp = ["listenfd"] sources-utils-net-unix = [] @@ -1140,29 +605,29 @@ sources-vector = ["dep:tonic", "protobuf-build"] # Transforms transforms = ["transforms-logs", "transforms-metrics"] transforms-logs = [ - "transforms-aws_ec2_metadata", - "transforms-dedupe", - "transforms-filter", - "transforms-log_to_metric", - "transforms-lua", - "transforms-metric_to_log", - "transforms-pipelines", - "transforms-reduce", - "transforms-remap", - "transforms-route", - "transforms-sample", - "transforms-throttle", + "transforms-aws_ec2_metadata", + "transforms-dedupe", + "transforms-filter", + "transforms-log_to_metric", + "transforms-lua", + "transforms-metric_to_log", + "transforms-pipelines", + "transforms-reduce", + "transforms-remap", + "transforms-route", + "transforms-sample", + "transforms-throttle", ] transforms-metrics = [ - "transforms-aggregate", - "transforms-filter", - "transforms-log_to_metric", - "transforms-lua", - "transforms-metric_to_log", - "transforms-pipelines", - "transforms-remap", - "transforms-tag_cardinality_limit", - "transforms-throttle", + "transforms-aggregate", + "transforms-filter", + "transforms-log_to_metric", + "transforms-lua", + "transforms-metric_to_log", + "transforms-pipelines", + "transforms-remap", + "transforms-tag_cardinality_limit", + "transforms-throttle", ] transforms-aggregate = [] @@ -1188,64 +653,64 @@ transforms-impl-reduce = [] # Sinks sinks = ["sinks-logs", "sinks-metrics"] sinks-logs = [ - "sinks-amqp", - "sinks-appsignal", - "sinks-aws_cloudwatch_logs", - "sinks-aws_kinesis_firehose", - "sinks-aws_kinesis_streams", - "sinks-aws_s3", - "sinks-aws_sqs", - "sinks-aws_sns", - "sinks-axiom", - "sinks-azure_blob", - "sinks-azure_monitor_logs", - "sinks-blackhole", - "sinks-chronicle", - "sinks-clickhouse", - "sinks-console", - "sinks-databend", - "sinks-datadog_events", - "sinks-datadog_logs", - "sinks-datadog_traces", - "sinks-elasticsearch", - "sinks-file", - "sinks-gcp", - "sinks-honeycomb", - "sinks-http", - "sinks-humio", - "sinks-influxdb", - "sinks-kafka", - "sinks-mezmo", - "sinks-loki", - "sinks-mqtt", - "sinks-nats", - "sinks-new_relic_logs", - "sinks-new_relic", - "sinks-papertrail", - "sinks-pulsar", - "sinks-redis", - "sinks-sematext", - "sinks-socket", - "sinks-splunk_hec", - "sinks-vector", - "sinks-webhdfs", - "sinks-websocket", + "sinks-amqp", + "sinks-appsignal", + "sinks-aws_cloudwatch_logs", + "sinks-aws_kinesis_firehose", + "sinks-aws_kinesis_streams", + "sinks-aws_s3", + "sinks-aws_sqs", + "sinks-aws_sns", + "sinks-axiom", + "sinks-azure_blob", + "sinks-azure_monitor_logs", + "sinks-blackhole", + "sinks-chronicle", + "sinks-clickhouse", + "sinks-console", + "sinks-databend", + "sinks-datadog_events", + "sinks-datadog_logs", + "sinks-datadog_traces", + "sinks-elasticsearch", + "sinks-file", + "sinks-gcp", + "sinks-honeycomb", + "sinks-http", + "sinks-humio", + "sinks-influxdb", + "sinks-kafka", + "sinks-mezmo", + "sinks-loki", + "sinks-mqtt", + "sinks-nats", + "sinks-new_relic_logs", + "sinks-new_relic", + "sinks-papertrail", + "sinks-pulsar", + "sinks-redis", + "sinks-sematext", + "sinks-socket", + "sinks-splunk_hec", + "sinks-vector", + "sinks-webhdfs", + "sinks-websocket", ] sinks-metrics = [ - "sinks-appsignal", - "sinks-aws_cloudwatch_metrics", - "sinks-blackhole", - "sinks-console", - "sinks-datadog_metrics", - "sinks-greptimedb", - "sinks-humio", - "sinks-influxdb", - "sinks-kafka", - "sinks-prometheus", - "sinks-sematext", - "sinks-statsd", - "sinks-vector", - "sinks-splunk_hec", + "sinks-appsignal", + "sinks-aws_cloudwatch_metrics", + "sinks-blackhole", + "sinks-console", + "sinks-datadog_metrics", + "sinks-greptimedb", + "sinks-humio", + "sinks-influxdb", + "sinks-kafka", + "sinks-prometheus", + "sinks-sematext", + "sinks-statsd", + "sinks-vector", + "sinks-splunk_hec" ] sinks-amqp = ["lapin"] @@ -1258,12 +723,7 @@ sinks-aws_s3 = ["dep:base64", "dep:md-5", "aws-core", "dep:aws-sdk-s3"] sinks-aws_sqs = ["aws-core", "dep:aws-sdk-sqs"] sinks-aws_sns = ["aws-core", "dep:aws-sdk-sns"] sinks-axiom = ["sinks-elasticsearch"] -sinks-azure_blob = [ - "dep:azure_core", - "dep:azure_identity", - "dep:azure_storage", - "dep:azure_storage_blobs", -] +sinks-azure_blob = ["dep:azure_core", "dep:azure_identity", "dep:azure_storage", "dep:azure_storage_blobs"] sinks-azure_monitor_logs = [] sinks-blackhole = [] sinks-chronicle = [] @@ -1273,16 +733,11 @@ sinks-databend = ["dep:databend-client"] sinks-datadog_events = [] sinks-datadog_logs = [] sinks-datadog_metrics = ["protobuf-build", "dep:prost-reflect"] -sinks-datadog_traces = [ - "protobuf-build", - "dep:rmpv", - "dep:rmp-serde", - "dep:serde_bytes", -] +sinks-datadog_traces = ["protobuf-build", "dep:rmpv", "dep:rmp-serde", "dep:serde_bytes"] sinks-elasticsearch = ["transforms-metric_to_log"] sinks-file = ["dep:async-compression"] sinks-gcp = ["sinks-gcp-chronicle", "dep:base64", "gcp"] -sinks-gcp-chronicle = ["gcp"] +sinks-gcp-chronicle = ["gcp"] sinks-greptimedb = ["dep:greptimedb-client"] sinks-honeycomb = [] sinks-http = [] @@ -1313,71 +768,69 @@ nightly = [] # Integration testing-related features all-integration-tests = [ - "amqp-integration-tests", - "appsignal-integration-tests", - "aws-integration-tests", - "axiom-integration-tests", - "azure-integration-tests", - "chronicle-integration-tests", - "clickhouse-integration-tests", - "databend-integration-tests", - "datadog-agent-integration-tests", - "datadog-logs-integration-tests", - "datadog-metrics-integration-tests", - "datadog-traces-integration-tests", - "docker-logs-integration-tests", - "es-integration-tests", - "eventstoredb_metrics-integration-tests", - "fluent-integration-tests", - "gcp-cloud-storage-integration-tests", - "gcp-integration-tests", - "gcp-pubsub-integration-tests", - "greptimedb-integration-tests", - "http-client-integration-tests", - "humio-integration-tests", - "influxdb-integration-tests", - "kafka-integration-tests", - "logstash-integration-tests", - "loki-integration-tests", - "mongodb_metrics-integration-tests", - "nats-integration-tests", - "nginx-integration-tests", - "opentelemetry-integration-tests", - "postgresql_metrics-integration-tests", - "prometheus-integration-tests", - "pulsar-integration-tests", - "redis-integration-tests", - "splunk-integration-tests", - "dnstap-integration-tests", - "webhdfs-integration-tests", + "amqp-integration-tests", + "appsignal-integration-tests", + "aws-integration-tests", + "axiom-integration-tests", + "azure-integration-tests", + "chronicle-integration-tests", + "clickhouse-integration-tests", + "databend-integration-tests", + "datadog-agent-integration-tests", + "datadog-logs-integration-tests", + "datadog-metrics-integration-tests", + "datadog-traces-integration-tests", + "docker-logs-integration-tests", + "es-integration-tests", + "eventstoredb_metrics-integration-tests", + "fluent-integration-tests", + "gcp-cloud-storage-integration-tests", + "gcp-integration-tests", + "gcp-pubsub-integration-tests", + "greptimedb-integration-tests", + "http-client-integration-tests", + "humio-integration-tests", + "influxdb-integration-tests", + "kafka-integration-tests", + "logstash-integration-tests", + "loki-integration-tests", + "mongodb_metrics-integration-tests", + "nats-integration-tests", + "nginx-integration-tests", + "opentelemetry-integration-tests", + "postgresql_metrics-integration-tests", + "prometheus-integration-tests", + "pulsar-integration-tests", + "redis-integration-tests", + "splunk-integration-tests", + "dnstap-integration-tests", + "webhdfs-integration-tests", ] amqp-integration-tests = ["sources-amqp", "sinks-amqp"] appsignal-integration-tests = ["sinks-appsignal"] aws-integration-tests = [ - "aws-cloudwatch-logs-integration-tests", - "aws-cloudwatch-metrics-integration-tests", - "aws-ec2-metadata-integration-tests", - "aws-ecs-metrics-integration-tests", - "aws-kinesis-firehose-integration-tests", - "aws-kinesis-streams-integration-tests", - "aws-s3-integration-tests", - "aws-sqs-integration-tests", - "aws-sns-integration-tests", + "aws-cloudwatch-logs-integration-tests", + "aws-cloudwatch-metrics-integration-tests", + "aws-ec2-metadata-integration-tests", + "aws-ecs-metrics-integration-tests", + "aws-kinesis-firehose-integration-tests", + "aws-kinesis-streams-integration-tests", + "aws-s3-integration-tests", + "aws-sqs-integration-tests", + "aws-sns-integration-tests", ] -azure-integration-tests = ["azure-blob-integration-tests"] +azure-integration-tests = [ + "azure-blob-integration-tests" +] aws-cloudwatch-logs-integration-tests = ["sinks-aws_cloudwatch_logs"] aws-cloudwatch-metrics-integration-tests = ["sinks-aws_cloudwatch_metrics"] aws-ec2-metadata-integration-tests = ["transforms-aws_ec2_metadata"] aws-ecs-metrics-integration-tests = ["sources-aws_ecs_metrics"] -aws-kinesis-firehose-integration-tests = [ - "sinks-aws_kinesis_firehose", - "dep:aws-sdk-elasticsearch", - "sinks-elasticsearch", -] +aws-kinesis-firehose-integration-tests = ["sinks-aws_kinesis_firehose", "dep:aws-sdk-elasticsearch", "sinks-elasticsearch"] aws-kinesis-streams-integration-tests = ["sinks-aws_kinesis_streams"] aws-s3-integration-tests = ["sinks-aws_s3", "sources-aws_s3"] aws-sqs-integration-tests = ["sinks-aws_sqs"] @@ -1390,11 +843,7 @@ databend-integration-tests = ["sinks-databend"] datadog-agent-integration-tests = ["sources-datadog_agent"] datadog-logs-integration-tests = ["sinks-datadog_logs"] datadog-metrics-integration-tests = ["sinks-datadog_metrics"] -datadog-traces-integration-tests = [ - "sources-datadog_agent", - "sinks-datadog_traces", - "axum/tokio", -] +datadog-traces-integration-tests = ["sources-datadog_agent", "sinks-datadog_traces", "axum/tokio"] docker-logs-integration-tests = ["sources-docker_logs", "unix"] es-integration-tests = ["sinks-elasticsearch", "aws-core"] eventstoredb_metrics-integration-tests = ["sources-eventstoredb_metrics"] @@ -1415,113 +864,78 @@ nats-integration-tests = ["sinks-nats", "sources-nats"] nginx-integration-tests = ["sources-nginx_metrics"] opentelemetry-integration-tests = ["sources-opentelemetry"] postgresql_metrics-integration-tests = ["sources-postgresql_metrics"] -prometheus-integration-tests = [ - "sinks-prometheus", - "sources-prometheus", - "sinks-influxdb", -] +prometheus-integration-tests = ["sinks-prometheus", "sources-prometheus", "sinks-influxdb"] pulsar-integration-tests = ["sinks-pulsar", "sources-pulsar"] redis-integration-tests = ["sinks-redis", "sources-redis"] splunk-integration-tests = ["sinks-splunk_hec"] dnstap-integration-tests = ["sources-dnstap", "dep:bollard"] webhdfs-integration-tests = ["sinks-webhdfs"] disable-resolv-conf = [] -shutdown-tests = [ - "api", - "sinks-blackhole", - "sinks-console", - "sinks-prometheus", - "sources", - "transforms-lua", - "transforms-remap", - "unix", -] -cli-tests = [ - "sinks-blackhole", - "sinks-socket", - "sources-demo_logs", - "sources-file", -] +shutdown-tests = ["api", "sinks-blackhole", "sinks-console", "sinks-prometheus", "sources", "transforms-lua", "transforms-remap", "unix"] +cli-tests = ["sinks-blackhole", "sinks-socket", "sources-demo_logs", "sources-file"] test-utils = [] # End-to-End testing-related features -all-e2e-tests = ["e2e-tests-datadog"] +all-e2e-tests = [ + "e2e-tests-datadog" +] e2e-tests-datadog = [ - "sources-datadog_agent", - "sinks-datadog_logs", - "sinks-datadog_metrics", + "sources-datadog_agent", + "sinks-datadog_logs", + "sinks-datadog_metrics" ] vector-api-tests = [ - "sources-demo_logs", - "transforms-log_to_metric", - "transforms-remap", - "sinks-blackhole", + "sources-demo_logs", + "transforms-log_to_metric", + "transforms-remap", + "sinks-blackhole" ] vector-unit-test-tests = [ - "sources-demo_logs", - "transforms-remap", - "transforms-route", - "transforms-filter", - "transforms-reduce", - "sinks-console", + "sources-demo_logs", + "transforms-remap", + "transforms-route", + "transforms-filter", + "transforms-reduce", + "sinks-console" ] -component-validation-runner = [ - "dep:tonic", - "sources-internal_logs", - "sources-internal_metrics", - "sources-vector", - "sinks-vector", -] +component-validation-runner = ["dep:tonic", "sources-internal_logs", "sources-internal_metrics", "sources-vector", "sinks-vector"] # For now, only include components that implement ValidatableComponent. # In the future, this can change to simply reference the targets `sources`, `transforms`, `sinks` component-validation-tests = [ - "component-validation-runner", - "sources-http_client", - "sources-http_server", - "sinks-http", - "sinks-splunk_hec", - "sources-splunk_hec", - "sinks-datadog_logs", - "sources-datadog_agent", + "component-validation-runner", + "sources-http_client", + "sources-http_server", + "sinks-http", + "sinks-splunk_hec", + "sources-splunk_hec", + "sinks-datadog_logs", + "sources-datadog_agent", ] # Grouping together features for benchmarks. We exclude the API client due to it causing the build process to run out # of memory when those additional dependencies are built in CI. benches = [ - "sinks-file", - "sinks-http", - "sinks-socket", - "sources-file", - "sources-socket", - "sources-syslog", - "transforms-lua", - "transforms-sample", + "sinks-file", + "sinks-http", + "sinks-socket", + "sources-file", + "sources-socket", + "sources-syslog", + "transforms-lua", + "transforms-sample", ] dnstap-benches = ["sources-dnstap"] -language-benches = [ - "sinks-socket", - "sources-socket", - "transforms-lua", - "transforms-remap", -] +language-benches = ["sinks-socket", "sources-socket", "transforms-lua", "transforms-remap"] # Separate benching process for metrics due to the nature of the bootstrap procedures. statistic-benches = [] remap-benches = ["transforms-remap"] -transform-benches = [ - "transforms-filter", - "transforms-dedupe", - "transforms-reduce", - "transforms-route", -] +transform-benches = ["transforms-filter", "transforms-dedupe", "transforms-reduce", "transforms-route"] codecs-benches = [] loki-benches = ["sinks-loki"] -enrichment-tables-benches = [ - "enrichment-tables-geoip", - "enrichment-tables-mmdb", -] +enrichment-tables-benches = ["enrichment-tables-geoip", "enrichment-tables-mmdb"] proptest = ["dep:proptest", "dep:proptest-derive"] [[bench]] @@ -1571,4 +985,4 @@ required-features = ["transform-benches"] name = "codecs" path = "benches/codecs/main.rs" harness = false -required-features = ["codecs-benches"] +required-features = ["codecs-benches"] \ No newline at end of file diff --git a/lib/codecs/Cargo.toml b/lib/codecs/Cargo.toml index 19c45f73bf146..bbdfb7769c24c 100644 --- a/lib/codecs/Cargo.toml +++ b/lib/codecs/Cargo.toml @@ -16,20 +16,13 @@ chrono.workspace = true csv-core = { version = "0.1.10", default-features = false } derivative = { version = "2", default-features = false } dyn-clone = { version = "1", default-features = false } -lookup = { package = "vector-lookup", path = "../vector-lookup", default-features = false, features = [ - "test", -] } +lookup = { package = "vector-lookup", path = "../vector-lookup", default-features = false, features = ["test"] } memchr = { version = "2", default-features = false } once_cell = { version = "1.19", default-features = false } ordered-float = { version = "4.2.0", default-features = false } prost = { version = "0.12.6", default-features = false, features = ["std"] } -prost-reflect = { version = "0.13", default-features = false, features = [ - "serde", -] } -regex = { version = "1.10.5", default-features = false, features = [ - "std", - "perf", -] } +prost-reflect = { version = "0.13", default-features = false, features = ["serde"] } +regex = { version = "1.10.5", default-features = false, features = ["std", "perf"] } serde.workspace = true serde_json.workspace = true smallvec = { version = "1", default-features = false, features = ["union"] } @@ -43,19 +36,14 @@ vector-common = { path = "../vector-common", default-features = false } vector-config = { path = "../vector-config", default-features = false } vector-config-common = { path = "../vector-config-common", default-features = false } vector-config-macros = { path = "../vector-config-macros", default-features = false } -vector-core = { path = "../vector-core", default-features = false, features = [ - "vrl", -] } +vector-core = { path = "../vector-core", default-features = false, features = ["vrl"] } [dev-dependencies] futures = { version = "0.3", default-features = false } indoc = { version = "2", default-features = false } tokio = { version = "1", features = ["test-util"] } similar-asserts = "1.5.0" -vector-core = { path = "../vector-core", default-features = false, features = [ - "vrl", - "test", -] } +vector-core = { path = "../vector-core", default-features = false, features = ["vrl", "test"] } uuid = { version = "1", default-features = false, features = ["serde", "v4"] } rstest = "0.21.0" tracing-test = "0.2.5" diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index 5c7d35c3bddc2..2c28b0eb46f97 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -282,8 +282,6 @@ impl Decoder for ChunkedGelfDecoder { return Ok(Some(frame.freeze())); } } - - // TODO: implement decode_eof } #[cfg(test)] diff --git a/lib/dnsmsg-parser/Cargo.toml b/lib/dnsmsg-parser/Cargo.toml index 15a88adf4eecb..4e29a718e69b0 100644 --- a/lib/dnsmsg-parser/Cargo.toml +++ b/lib/dnsmsg-parser/Cargo.toml @@ -7,8 +7,8 @@ publish = false license = "MIT" [dependencies] -thiserror = "1.0" data-encoding = "2.6" +thiserror = "1.0" hickory-proto = { version = "0.24", features = ["dnssec"] } [dev-dependencies] From d405e6c4af781f6025f07dce42da5417224512eb Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Fri, 5 Jul 2024 13:53:09 +0200 Subject: [PATCH 10/60] style: fix Cargo.toml stlye --- Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Cargo.toml b/Cargo.toml index d9aeab991a3f2..8978aada45eb8 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -985,4 +985,4 @@ required-features = ["transform-benches"] name = "codecs" path = "benches/codecs/main.rs" harness = false -required-features = ["codecs-benches"] \ No newline at end of file +required-features = ["codecs-benches"] From 9b692642d1fe20b071bce8a1352fd8becd494c4f Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Fri, 5 Jul 2024 13:54:26 +0200 Subject: [PATCH 11/60] chore: remove comment --- lib/codecs/src/decoding/mod.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/lib/codecs/src/decoding/mod.rs b/lib/codecs/src/decoding/mod.rs index f9f16505fc27d..0eaef4929142c 100644 --- a/lib/codecs/src/decoding/mod.rs +++ b/lib/codecs/src/decoding/mod.rs @@ -354,8 +354,6 @@ impl DeserializerConfig { | DeserializerConfig::NativeJson(_) => { FramingConfig::NewlineDelimited(Default::default()) } - // TODO: the default framing of the gelf codec should be chunked gelf? - // or only with the udp input? Why is the newline delimited used for gelf? gelf is not newline delimited DeserializerConfig::Protobuf(_) => FramingConfig::Bytes, #[cfg(feature = "syslog")] DeserializerConfig::Syslog(_) => FramingConfig::NewlineDelimited(Default::default()), From bfc332e0d5b5a0c898db502e4d1f2672b066d84a Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Sat, 6 Jul 2024 12:14:30 +0200 Subject: [PATCH 12/60] fix(gelf): Fix partial read of buffer --- .../src/decoding/framing/chunked_gelf.rs | 121 +++++++++++------- src/sources/socket/mod.rs | 2 + src/sources/socket/udp.rs | 2 - 3 files changed, 76 insertions(+), 49 deletions(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index 2c28b0eb46f97..b8ba71c4fdb25 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -1,3 +1,5 @@ +use crate::BytesDecoder; + use super::BoxedFramingError; use bytes::{Buf, Bytes, BytesMut}; use derivative::Derivative; @@ -130,7 +132,10 @@ impl MessageState { /// A decoder for handling GELF messages that may be chunked. #[derive(Debug, Clone)] pub struct ChunkedGelfDecoder { - /// TODO + // We have to use this decoder to read all the bytes from the buffer first and don't let tokio + // read it buffered, as tokio FramedRead will not always call the decode method with the + // whole message. (see https://docs.rs/tokio-util/latest/src/tokio_util/codec/framed_impl.rs.html#26) + inner_decoder: BytesDecoder, state: Arc>>, timeout: Duration, pending_messages_limit: usize, @@ -140,6 +145,7 @@ impl ChunkedGelfDecoder { /// Creates a new `ChunkedGelfDecoder`. pub fn new(timeout_millis: u64, pending_messages_limit: usize) -> Self { Self { + inner_decoder: BytesDecoder::new(), state: Arc::new(Mutex::new(HashMap::new())), timeout: Duration::from_millis(timeout_millis), pending_messages_limit, @@ -147,22 +153,19 @@ impl ChunkedGelfDecoder { } /// TODO: document this - pub fn decode_chunk( - &mut self, - src: &mut bytes::BytesMut, - ) -> Result, BoxedFramingError> { + pub fn decode_chunk(&mut self, mut chunk: Bytes) -> Result, BoxedFramingError> { // We need 10 bits to read the message id, sequence number and total chunks - if src.remaining() < 10 { - let src_display = format!("{src:?}"); + if chunk.remaining() < 10 { + let src_display = format!("{chunk:?}"); warn!(message = "Received malformed chunk headers (message ID, sequence number and total chunks) with less than 10 bytes. Ignoring it.", src = src_display, - remaining = src.remaining(), + remaining = chunk.remaining(), internal_log_rate_limit = true); return Ok(None); } - let message_id = src.get_u64(); - let sequence_number = src.get_u8(); - let total_chunks = src.get_u8(); + let message_id = chunk.get_u64(); + let sequence_number = chunk.get_u8(); + let total_chunks = chunk.get_u8(); if total_chunks == 0 || total_chunks > MAX_TOTAL_CHUNKS { warn!( @@ -210,6 +213,7 @@ impl ChunkedGelfDecoder { let mut state_lock = state.lock().unwrap(); if let Some(_) = state_lock.remove(&message_id) { // TODO: log the variables in the message or use structured logging? + // TODO: record metrics here? it would be insteresting to know how many messages are being discarded warn!( message = "Message was not fully received within the timeout window. Discarding it.", message_id = message_id, @@ -242,7 +246,6 @@ impl ChunkedGelfDecoder { return Ok(None); } - let chunk = src.split().freeze(); message_state.add_chunk(sequence_number, chunk); if let Some(message) = message_state.retrieve_message() { @@ -252,6 +255,18 @@ impl ChunkedGelfDecoder { Ok(None) } } + + /// TODO: document this + pub fn decode_message(&mut self, mut src: Bytes) -> Result, BoxedFramingError> { + let magic = src.get(0..2); + if magic.is_some_and(|magic| magic == GELF_MAGIC) { + src.advance(2); + let frame = self.decode_chunk(src); + return frame; + } else { + return Ok(Some(src)); + } + } } impl Default for ChunkedGelfDecoder { @@ -266,21 +281,28 @@ impl Decoder for ChunkedGelfDecoder { type Error = BoxedFramingError; fn decode(&mut self, src: &mut bytes::BytesMut) -> Result, Self::Error> { + // TODO: add a PR comment here stating that this will never call the decode_message since + // the bytes decoder will always return a Ok(None) in this method, but leaving this + // here for consistency. Would be better to add a unreachable/panic here if the inner decoder returns + // the Some variant? if src.is_empty() { return Ok(None); } - let magic = src.get(0..2); - if magic.is_some_and(|magic| magic == GELF_MAGIC) { - src.advance(2); - let frame = self.decode_chunk(src); - src.clear(); - return frame; - } else { - // The gelf message is not chunked - let frame = src.split(); - return Ok(Some(frame.freeze())); + self.inner_decoder + .decode(src)? + .and_then(|frame| self.decode_message(frame).transpose()) + .transpose() + } + fn decode_eof(&mut self, buf: &mut BytesMut) -> Result, Self::Error> { + if buf.is_empty() { + return Ok(None); } + + self.inner_decoder + .decode_eof(buf)? + .and_then(|frame| self.decode_message(frame).transpose()) + .transpose() } } @@ -291,6 +313,7 @@ mod tests { use rstest::{fixture, rstest}; use tracing_test::traced_test; + // TODO: return BytesMut instead of Bytes fn create_chunk( message_id: u64, sequence_number: u8, @@ -381,6 +404,8 @@ mod tests { ) } + // TODO: refactor those tests so we use the FramedRead from tokio as the + // bytes decoder #[rstest] #[tokio::test] async fn decode_chunked(two_chunks_message: ([Bytes; 2], String)) { @@ -388,12 +413,14 @@ mod tests { let (chunks, expected_message) = two_chunks_message; let mut decoder = ChunkedGelfDecoder::default(); + // TODO: replace extend_from_slice to just call the decoder + // with the chunk[0] src.extend_from_slice(&chunks[0]); - let frame = decoder.decode(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut src).unwrap(); assert!(frame.is_none()); src.extend_from_slice(&chunks[1]); - let frame = decoder.decode(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut src).unwrap(); assert_eq!(frame, Some(Bytes::from(expected_message))); } @@ -406,7 +433,7 @@ mod tests { let mut decoder = ChunkedGelfDecoder::default(); src.extend_from_slice(&message); - let frame = decoder.decode(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut src).unwrap(); assert_eq!(frame, Some(Bytes::from(expected_message))); } @@ -418,11 +445,11 @@ mod tests { let mut decoder = ChunkedGelfDecoder::default(); src.extend_from_slice(&chunks[1]); - let frame = decoder.decode(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut src).unwrap(); assert!(frame.is_none()); src.extend_from_slice(&chunks[0]); - let frame = decoder.decode(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut src).unwrap(); assert_eq!(frame, Some(Bytes::from(expected_message))); } @@ -439,23 +466,23 @@ mod tests { let mut decoder = ChunkedGelfDecoder::default(); src.extend_from_slice(&three_chunks[2]); - let frame = decoder.decode(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut src).unwrap(); assert!(frame.is_none()); src.extend_from_slice(&two_chunks[0]); - let frame = decoder.decode(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut src).unwrap(); assert!(frame.is_none()); src.extend_from_slice(&three_chunks[0]); - let frame = decoder.decode(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut src).unwrap(); assert!(frame.is_none()); src.extend_from_slice(&two_chunks[1]); - let frame = decoder.decode(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut src).unwrap(); assert_eq!(frame, Some(Bytes::from(two_chunks_expected))); src.extend_from_slice(&three_chunks[1]); - let frame = decoder.decode(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut src).unwrap(); assert_eq!(frame, Some(Bytes::from(three_chunks_expected))); } @@ -471,15 +498,15 @@ mod tests { let mut decoder = ChunkedGelfDecoder::default(); src.extend_from_slice(&chunks[1]); - let frame = decoder.decode(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut src).unwrap(); assert!(frame.is_none()); src.extend_from_slice(&unchunked_message); - let frame = decoder.decode(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut src).unwrap(); assert_eq!(frame, Some(Bytes::from(expected_unchunked_message))); src.extend_from_slice(&chunks[0]); - let frame = decoder.decode(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut src).unwrap(); assert_eq!(frame, Some(Bytes::from(expected_chunked_message))); } @@ -493,7 +520,7 @@ mod tests { let mut decoder = ChunkedGelfDecoder::new(timeout, DEFAULT_PENDING_MESSAGES_LIMIT); src.extend_from_slice(&chunks[0]); - let frame = decoder.decode(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut src).unwrap(); assert!(frame.is_none()); assert!(!decoder.state.lock().unwrap().is_empty()); @@ -505,7 +532,7 @@ mod tests { )); src.extend_from_slice(&chunks[1]); - let frame = decoder.decode(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut src).unwrap(); assert!(frame.is_none()); tokio::time::sleep(Duration::from_millis(timeout + 1)).await; @@ -517,7 +544,7 @@ mod tests { let mut src = BytesMut::new(); let mut decoder = ChunkedGelfDecoder::default(); - let frame = decoder.decode(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut src).unwrap(); assert!(frame.is_none()); } @@ -530,7 +557,7 @@ mod tests { src.extend_from_slice(&GELF_MAGIC); src.extend_from_slice(&malformed_chunk); - let frame = decoder.decode(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut src).unwrap(); assert!(frame.is_none()); assert!(logs_contain("Received malformed chunk headers (message ID, sequence number and total chunks) with less than 10 bytes. Ignoring it.")); } @@ -547,7 +574,7 @@ mod tests { let mut decoder = ChunkedGelfDecoder::default(); src.extend_from_slice(&chunk); - let frame = decoder.decode(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut src).unwrap(); assert!(frame.is_none()); assert!(logs_contain( "Received a chunk with an invalid total chunks value. Ignoring it." @@ -566,7 +593,7 @@ mod tests { let mut decoder = ChunkedGelfDecoder::default(); src.extend_from_slice(&chunk); - let frame = decoder.decode(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut src).unwrap(); assert!(frame.is_none()); assert!(logs_contain( "Received a chunk with a sequence number greater than total chunks. Ignoring it." @@ -587,12 +614,12 @@ mod tests { let mut decoder = ChunkedGelfDecoder::new(DEFAULT_TIMEOUT_MILLIS, pending_messages_limit); src.extend_from_slice(&two_chunks[0]); - let frame = decoder.decode(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut src).unwrap(); assert!(frame.is_none()); assert!(decoder.state.lock().unwrap().len() == 1); src.extend_from_slice(&three_chunks[0]); - let frame = decoder.decode(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut src).unwrap(); assert!(frame.is_none()); assert!(decoder.state.lock().unwrap().len() == 1); assert!(logs_contain( @@ -614,11 +641,11 @@ mod tests { let mut decoder = ChunkedGelfDecoder::default(); src.extend_from_slice(&first_chunk); - let frame = decoder.decode(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut src).unwrap(); assert!(frame.is_none()); src.extend_from_slice(&second_chunk); - let frame = decoder.decode(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut src).unwrap(); assert!(frame.is_none()); assert!(logs_contain( "Received a chunk with a different total chunks than the original. Ignoring it." @@ -634,11 +661,11 @@ mod tests { let mut decoder = ChunkedGelfDecoder::default(); src.extend_from_slice(&chunks[0]); - let frame = decoder.decode(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut src).unwrap(); assert!(frame.is_none()); src.extend_from_slice(&chunks[0]); - let frame = decoder.decode(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut src).unwrap(); assert!(frame.is_none()); assert!(logs_contain("Received a duplicate chunk. Ignoring it.")); } diff --git a/src/sources/socket/mod.rs b/src/sources/socket/mod.rs index 404e7bbb4253a..d435d1e19bd0d 100644 --- a/src/sources/socket/mod.rs +++ b/src/sources/socket/mod.rs @@ -150,6 +150,8 @@ impl SourceConfig for SocketConfig { ) } Mode::Udp(config) => { + // TODO: add tests for udp with chunked gelf, use a big payload so we ensure + // that the decode_eof method gets called due to tokio buffering let log_namespace = cx.log_namespace(config.log_namespace); let decoding = config.decoding().clone(); let framing = config diff --git a/src/sources/socket/udp.rs b/src/sources/socket/udp.rs index b4acddd38b70c..80ed93da1468d 100644 --- a/src/sources/socket/udp.rs +++ b/src/sources/socket/udp.rs @@ -202,10 +202,8 @@ pub(super) fn udp( }; bytes_received.emit(ByteSize(byte_size)); - let payload = buf.split_to(byte_size); let truncated = byte_size == max_length + 1; - let mut stream = FramedRead::new(payload.as_ref(), decoder.clone()).peekable(); while let Some(result) = stream.next().await { From 0bc3376da4d580df6449d29203661ec9cfb00909 Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Tue, 9 Jul 2024 10:50:21 +0200 Subject: [PATCH 13/60] test(gelf): Refactor some tests --- .../src/decoding/framing/chunked_gelf.rs | 177 +++++++----------- 1 file changed, 71 insertions(+), 106 deletions(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index b8ba71c4fdb25..9b1642f1ea610 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -64,7 +64,7 @@ pub struct ChunkedGelfDecoderOptions { pub timeout_millis: u64, /// The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start - /// dropping chunks of new messages. + /// dropping chunks of new messages. This limit ensures the memory usage of the decoder is bounded. #[serde( default = "default_pending_messages_limit", skip_serializing_if = "vector_core::serde::is_default" @@ -134,7 +134,9 @@ impl MessageState { pub struct ChunkedGelfDecoder { // We have to use this decoder to read all the bytes from the buffer first and don't let tokio // read it buffered, as tokio FramedRead will not always call the decode method with the - // whole message. (see https://docs.rs/tokio-util/latest/src/tokio_util/codec/framed_impl.rs.html#26) + // whole message. (see https://docs.rs/tokio-util/latest/src/tokio_util/codec/framed_impl.rs.html#26). + // This limitation is due to the fact that the GELF format does not specify the length of the + // message, so we have to read all the bytes from the message (datagram) inner_decoder: BytesDecoder, state: Arc>>, timeout: Duration, @@ -204,8 +206,8 @@ impl ChunkedGelfDecoder { } let message_state = state_lock.entry(message_id).or_insert_with(|| { - // TODO: we need tokio due to the sleep function. We need to spawn a task that will clear the message state after a certain time - // otherwise we will have a memory leak + // We need to spawn a task that will clear the message state after a certain time + // otherwise we will have a memory leak due to messages that never complete let state = Arc::clone(&self.state); let timeout = self.timeout.clone(); let timeout_handle = tokio::spawn(async move { @@ -281,14 +283,14 @@ impl Decoder for ChunkedGelfDecoder { type Error = BoxedFramingError; fn decode(&mut self, src: &mut bytes::BytesMut) -> Result, Self::Error> { - // TODO: add a PR comment here stating that this will never call the decode_message since - // the bytes decoder will always return a Ok(None) in this method, but leaving this - // here for consistency. Would be better to add a unreachable/panic here if the inner decoder returns - // the Some variant? if src.is_empty() { return Ok(None); } + // TODO: add a PR comment here stating that this will never call the decode_message since + // the bytes decoder will always return a Ok(None) in this method, but leaving this + // here for consistency. Would be better to add a unreachable/panic here if the inner decoder returns + // the Some variant? self.inner_decoder .decode(src)? .and_then(|frame| self.decode_message(frame).transpose()) @@ -313,32 +315,31 @@ mod tests { use rstest::{fixture, rstest}; use tracing_test::traced_test; - // TODO: return BytesMut instead of Bytes fn create_chunk( message_id: u64, sequence_number: u8, total_chunks: u8, payload: &str, - ) -> Bytes { + ) -> BytesMut { let mut chunk = BytesMut::new(); chunk.put_slice(&GELF_MAGIC); chunk.put_u64(message_id); chunk.put_u8(sequence_number); chunk.put_u8(total_chunks); chunk.extend_from_slice(payload.as_bytes()); - chunk.freeze() + chunk } #[fixture] - fn unchunked_message() -> (Bytes, String) { + fn unchunked_message() -> (BytesMut, String) { let payload = "foo"; - (Bytes::from(payload), payload.to_string()) + (BytesMut::from(payload), payload.to_string()) } // TODO: add a malformed chunk message #[fixture] - fn two_chunks_message() -> ([Bytes; 2], String) { + fn two_chunks_message() -> ([BytesMut; 2], String) { let message_id = 1u64; let total_chunks = 2u8; @@ -367,7 +368,7 @@ mod tests { } #[fixture] - fn three_chunks_message() -> ([Bytes; 3], String) { + fn three_chunks_message() -> ([BytesMut; 3], String) { let message_id = 2u64; let total_chunks = 3u8; @@ -404,123 +405,97 @@ mod tests { ) } - // TODO: refactor those tests so we use the FramedRead from tokio as the - // bytes decoder #[rstest] #[tokio::test] - async fn decode_chunked(two_chunks_message: ([Bytes; 2], String)) { - let mut src = BytesMut::new(); - let (chunks, expected_message) = two_chunks_message; + async fn decode_chunked(two_chunks_message: ([BytesMut; 2], String)) { + let (mut chunks, expected_message) = two_chunks_message; let mut decoder = ChunkedGelfDecoder::default(); - // TODO: replace extend_from_slice to just call the decoder - // with the chunk[0] - src.extend_from_slice(&chunks[0]); - let frame = decoder.decode_eof(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut chunks[0]).unwrap(); assert!(frame.is_none()); - src.extend_from_slice(&chunks[1]); - let frame = decoder.decode_eof(&mut src).unwrap(); - + let frame = decoder.decode_eof(&mut chunks[1]).unwrap(); assert_eq!(frame, Some(Bytes::from(expected_message))); } #[rstest] #[tokio::test] - async fn decode_unchunked(unchunked_message: (Bytes, String)) { - let mut src = BytesMut::new(); - let (message, expected_message) = unchunked_message; + async fn decode_unchunked(unchunked_message: (BytesMut, String)) { + let (mut message, expected_message) = unchunked_message; let mut decoder = ChunkedGelfDecoder::default(); - src.extend_from_slice(&message); - let frame = decoder.decode_eof(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut message).unwrap(); assert_eq!(frame, Some(Bytes::from(expected_message))); } #[rstest] #[tokio::test] - async fn decode_unordered_chunks(two_chunks_message: ([Bytes; 2], String)) { - let mut src = BytesMut::new(); - let (chunks, expected_message) = two_chunks_message; + async fn decode_unordered_chunks(two_chunks_message: ([BytesMut; 2], String)) { + let (mut chunks, expected_message) = two_chunks_message; let mut decoder = ChunkedGelfDecoder::default(); - src.extend_from_slice(&chunks[1]); - let frame = decoder.decode_eof(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut chunks[1]).unwrap(); assert!(frame.is_none()); - src.extend_from_slice(&chunks[0]); - let frame = decoder.decode_eof(&mut src).unwrap(); - + let frame = decoder.decode_eof(&mut chunks[0]).unwrap(); assert_eq!(frame, Some(Bytes::from(expected_message))); } #[rstest] #[tokio::test] async fn decode_unordered_messages( - two_chunks_message: ([Bytes; 2], String), - three_chunks_message: ([Bytes; 3], String), + two_chunks_message: ([BytesMut; 2], String), + three_chunks_message: ([BytesMut; 3], String), ) { - let mut src = BytesMut::new(); - let (two_chunks, two_chunks_expected) = two_chunks_message; - let (three_chunks, three_chunks_expected) = three_chunks_message; + let (mut two_chunks, two_chunks_expected) = two_chunks_message; + let (mut three_chunks, three_chunks_expected) = three_chunks_message; let mut decoder = ChunkedGelfDecoder::default(); - src.extend_from_slice(&three_chunks[2]); - let frame = decoder.decode_eof(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut three_chunks[2]).unwrap(); assert!(frame.is_none()); - src.extend_from_slice(&two_chunks[0]); - let frame = decoder.decode_eof(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut two_chunks[0]).unwrap(); assert!(frame.is_none()); - src.extend_from_slice(&three_chunks[0]); - let frame = decoder.decode_eof(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut three_chunks[0]).unwrap(); assert!(frame.is_none()); - src.extend_from_slice(&two_chunks[1]); - let frame = decoder.decode_eof(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut two_chunks[1]).unwrap(); assert_eq!(frame, Some(Bytes::from(two_chunks_expected))); - src.extend_from_slice(&three_chunks[1]); - let frame = decoder.decode_eof(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut three_chunks[1]).unwrap(); assert_eq!(frame, Some(Bytes::from(three_chunks_expected))); } #[rstest] #[tokio::test] async fn decode_mixed_chunked_and_unchunked_messages( - unchunked_message: (Bytes, String), - two_chunks_message: ([Bytes; 2], String), + unchunked_message: (BytesMut, String), + two_chunks_message: ([BytesMut; 2], String), ) { - let mut src = BytesMut::new(); - let (unchunked_message, expected_unchunked_message) = unchunked_message; - let (chunks, expected_chunked_message) = two_chunks_message; + let (mut unchunked_message, expected_unchunked_message) = unchunked_message; + let (mut chunks, expected_chunked_message) = two_chunks_message; let mut decoder = ChunkedGelfDecoder::default(); - src.extend_from_slice(&chunks[1]); - let frame = decoder.decode_eof(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut chunks[1]).unwrap(); assert!(frame.is_none()); - src.extend_from_slice(&unchunked_message); - let frame = decoder.decode_eof(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut unchunked_message).unwrap(); assert_eq!(frame, Some(Bytes::from(expected_unchunked_message))); - src.extend_from_slice(&chunks[0]); - let frame = decoder.decode_eof(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut chunks[0]).unwrap(); assert_eq!(frame, Some(Bytes::from(expected_chunked_message))); } #[rstest] #[tokio::test(start_paused = true)] #[traced_test] - async fn decode_timeout(two_chunks_message: ([Bytes; 2], String)) { + async fn decode_timeout(two_chunks_message: ([BytesMut; 2], String)) { let timeout = 300; - let mut src = BytesMut::new(); - let (chunks, _) = two_chunks_message; + let (mut chunks, _) = two_chunks_message; let mut decoder = ChunkedGelfDecoder::new(timeout, DEFAULT_PENDING_MESSAGES_LIMIT); - src.extend_from_slice(&chunks[0]); - let frame = decoder.decode_eof(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut chunks[0]).unwrap(); assert!(frame.is_none()); assert!(!decoder.state.lock().unwrap().is_empty()); @@ -531,12 +506,14 @@ mod tests { "Message was not fully received within the timeout window. Discarding it." )); - src.extend_from_slice(&chunks[1]); - let frame = decoder.decode_eof(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut chunks[1]).unwrap(); assert!(frame.is_none()); tokio::time::sleep(Duration::from_millis(timeout + 1)).await; assert!(decoder.state.lock().unwrap().is_empty()); + assert!(logs_contain( + "Message was not fully received within the timeout window. Discarding it." + )); } #[tokio::test] @@ -553,10 +530,10 @@ mod tests { async fn decode_chunk_with_malformed_header() { let malformed_chunk = [0x12, 0x23]; let mut src = BytesMut::new(); - let mut decoder = ChunkedGelfDecoder::default(); - src.extend_from_slice(&GELF_MAGIC); src.extend_from_slice(&malformed_chunk); + let mut decoder = ChunkedGelfDecoder::default(); + let frame = decoder.decode_eof(&mut src).unwrap(); assert!(frame.is_none()); assert!(logs_contain("Received malformed chunk headers (message ID, sequence number and total chunks) with less than 10 bytes. Ignoring it.")); @@ -569,12 +546,10 @@ mod tests { let sequence_number = 1u8; let invalid_total_chunks = MAX_TOTAL_CHUNKS + 1; let payload = "foo"; - let chunk = create_chunk(message_id, sequence_number, invalid_total_chunks, payload); - let mut src = BytesMut::new(); + let mut chunk = create_chunk(message_id, sequence_number, invalid_total_chunks, payload); let mut decoder = ChunkedGelfDecoder::default(); - src.extend_from_slice(&chunk); - let frame = decoder.decode_eof(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut chunk).unwrap(); assert!(frame.is_none()); assert!(logs_contain( "Received a chunk with an invalid total chunks value. Ignoring it." @@ -588,12 +563,10 @@ mod tests { let total_chunks = 2u8; let invalid_sequence_number = total_chunks + 1; let payload = "foo"; - let chunk = create_chunk(message_id, invalid_sequence_number, total_chunks, payload); - let mut src = BytesMut::new(); + let mut chunk = create_chunk(message_id, invalid_sequence_number, total_chunks, payload); let mut decoder = ChunkedGelfDecoder::default(); - src.extend_from_slice(&chunk); - let frame = decoder.decode_eof(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut chunk).unwrap(); assert!(frame.is_none()); assert!(logs_contain( "Received a chunk with a sequence number greater than total chunks. Ignoring it." @@ -604,22 +577,19 @@ mod tests { #[tokio::test] #[traced_test] async fn decode_when_reached_pending_messages_limit( - two_chunks_message: ([Bytes; 2], String), - three_chunks_message: ([Bytes; 3], String), + two_chunks_message: ([BytesMut; 2], String), + three_chunks_message: ([BytesMut; 3], String), ) { let pending_messages_limit = 1; - let mut src = BytesMut::new(); - let (two_chunks, _) = two_chunks_message; - let (three_chunks, _) = three_chunks_message; + let (mut two_chunks, _) = two_chunks_message; + let (mut three_chunks, _) = three_chunks_message; let mut decoder = ChunkedGelfDecoder::new(DEFAULT_TIMEOUT_MILLIS, pending_messages_limit); - src.extend_from_slice(&two_chunks[0]); - let frame = decoder.decode_eof(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut two_chunks[0]).unwrap(); assert!(frame.is_none()); assert!(decoder.state.lock().unwrap().len() == 1); - src.extend_from_slice(&three_chunks[0]); - let frame = decoder.decode_eof(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut three_chunks[0]).unwrap(); assert!(frame.is_none()); assert!(decoder.state.lock().unwrap().len() == 1); assert!(logs_contain( @@ -635,17 +605,15 @@ mod tests { let sequence_number = 0u8; let total_chunks = 2u8; let payload = "foo"; - let first_chunk = create_chunk(message_id, sequence_number, total_chunks, payload); - let second_chunk = create_chunk(message_id, sequence_number + 1, total_chunks + 1, payload); - let mut src = BytesMut::new(); + let mut first_chunk = create_chunk(message_id, sequence_number, total_chunks, payload); + let mut second_chunk = + create_chunk(message_id, sequence_number + 1, total_chunks + 1, payload); let mut decoder = ChunkedGelfDecoder::default(); - src.extend_from_slice(&first_chunk); - let frame = decoder.decode_eof(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut first_chunk).unwrap(); assert!(frame.is_none()); - src.extend_from_slice(&second_chunk); - let frame = decoder.decode_eof(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut second_chunk).unwrap(); assert!(frame.is_none()); assert!(logs_contain( "Received a chunk with a different total chunks than the original. Ignoring it." @@ -655,17 +623,14 @@ mod tests { #[rstest] #[tokio::test] #[traced_test] - async fn decode_when_duplicated_chunk(two_chunks_message: ([Bytes; 2], String)) { - let mut src = BytesMut::new(); - let (chunks, _) = two_chunks_message; + async fn decode_when_duplicated_chunk(two_chunks_message: ([BytesMut; 2], String)) { + let (mut chunks, _) = two_chunks_message; let mut decoder = ChunkedGelfDecoder::default(); - src.extend_from_slice(&chunks[0]); - let frame = decoder.decode_eof(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut chunks[0].clone()).unwrap(); assert!(frame.is_none()); - src.extend_from_slice(&chunks[0]); - let frame = decoder.decode_eof(&mut src).unwrap(); + let frame = decoder.decode_eof(&mut chunks[0]).unwrap(); assert!(frame.is_none()); assert!(logs_contain("Received a duplicate chunk. Ignoring it.")); } From 7f633b0a4011cc404a730b30f37d7ec2ca8804d6 Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Tue, 9 Jul 2024 11:57:28 +0200 Subject: [PATCH 14/60] style: fix log style --- lib/codecs/src/decoding/framing/chunked_gelf.rs | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index 9b1642f1ea610..cd298bf8fa8e8 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -13,7 +13,7 @@ use tracing::{info, warn}; use vector_config::configurable_component; const GELF_MAGIC: [u8; 2] = [0x1e, 0x0f]; -const MAX_TOTAL_CHUNKS: u8 = 128; +pub const MAX_TOTAL_CHUNKS: u8 = 128; const DEFAULT_CHUNKS: [Bytes; MAX_TOTAL_CHUNKS as usize] = [const { Bytes::new() }; MAX_TOTAL_CHUNKS as usize]; const DEFAULT_TIMEOUT_MILLIS: u64 = 5000; @@ -162,7 +162,8 @@ impl ChunkedGelfDecoder { warn!(message = "Received malformed chunk headers (message ID, sequence number and total chunks) with less than 10 bytes. Ignoring it.", src = src_display, remaining = chunk.remaining(), - internal_log_rate_limit = true); + internal_log_rate_limit = true + ); return Ok(None); } let message_id = chunk.get_u64(); @@ -229,10 +230,10 @@ impl ChunkedGelfDecoder { if message_state.total_chunks != total_chunks { warn!(message_id = "Received a chunk with a different total chunks than the original. Ignoring it.", + message_id = message_id original_total_chunks = message_state.total_chunks, received_total_chunks = total_chunks, internal_log_rate_limit = true, - message_id = message_id ); return Ok(None); } From 103b93323dfae0ac0935b948c24bde18cf8365df Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Tue, 9 Jul 2024 11:58:00 +0200 Subject: [PATCH 15/60] style: fix log style --- lib/codecs/src/decoding/framing/chunked_gelf.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index cd298bf8fa8e8..9862653e56ebe 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -230,10 +230,10 @@ impl ChunkedGelfDecoder { if message_state.total_chunks != total_chunks { warn!(message_id = "Received a chunk with a different total chunks than the original. Ignoring it.", - message_id = message_id + message_id = message_id, original_total_chunks = message_state.total_chunks, received_total_chunks = total_chunks, - internal_log_rate_limit = true, + internal_log_rate_limit = true ); return Ok(None); } From 6d4f8c76fc299638617bfef7b7f2ccfed71f3b88 Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Wed, 10 Jul 2024 08:49:58 +0200 Subject: [PATCH 16/60] test(socket): Add chunked gelf tests for socket source --- .../src/decoding/framing/chunked_gelf.rs | 12 +-- lib/codecs/src/decoding/framing/mod.rs | 2 +- lib/codecs/src/decoding/mod.rs | 10 +- src/sources/socket/mod.rs | 97 ++++++++++++++++++- src/sources/socket/udp.rs | 2 +- 5 files changed, 106 insertions(+), 17 deletions(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index 9862653e56ebe..1818e46211b6e 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -13,9 +13,9 @@ use tracing::{info, warn}; use vector_config::configurable_component; const GELF_MAGIC: [u8; 2] = [0x1e, 0x0f]; -pub const MAX_TOTAL_CHUNKS: u8 = 128; -const DEFAULT_CHUNKS: [Bytes; MAX_TOTAL_CHUNKS as usize] = - [const { Bytes::new() }; MAX_TOTAL_CHUNKS as usize]; +const GELF_MAX_TOTAL_CHUNKS: u8 = 128; +const DEFAULT_CHUNKS: [Bytes; GELF_MAX_TOTAL_CHUNKS as usize] = + [const { Bytes::new() }; GELF_MAX_TOTAL_CHUNKS as usize]; const DEFAULT_TIMEOUT_MILLIS: u64 = 5000; const DEFAULT_PENDING_MESSAGES_LIMIT: usize = 1000; @@ -84,7 +84,7 @@ impl Default for ChunkedGelfDecoderOptions { #[derive(Debug)] pub struct MessageState { total_chunks: u8, - chunks: [Bytes; MAX_TOTAL_CHUNKS as usize], + chunks: [Bytes; GELF_MAX_TOTAL_CHUNKS as usize], chunks_bitmap: u128, timeout_task: JoinHandle<()>, } @@ -170,7 +170,7 @@ impl ChunkedGelfDecoder { let sequence_number = chunk.get_u8(); let total_chunks = chunk.get_u8(); - if total_chunks == 0 || total_chunks > MAX_TOTAL_CHUNKS { + if total_chunks == 0 || total_chunks > GELF_MAX_TOTAL_CHUNKS { warn!( message = "Received a chunk with an invalid total chunks value. Ignoring it.", message_id = message_id, @@ -545,7 +545,7 @@ mod tests { async fn decode_chunk_with_invalid_total_chunks() { let message_id = 1u64; let sequence_number = 1u8; - let invalid_total_chunks = MAX_TOTAL_CHUNKS + 1; + let invalid_total_chunks = GELF_MAX_TOTAL_CHUNKS + 1; let payload = "foo"; let mut chunk = create_chunk(message_id, sequence_number, invalid_total_chunks, payload); let mut decoder = ChunkedGelfDecoder::default(); diff --git a/lib/codecs/src/decoding/framing/mod.rs b/lib/codecs/src/decoding/framing/mod.rs index e998f02abf525..49b615ed152b3 100644 --- a/lib/codecs/src/decoding/framing/mod.rs +++ b/lib/codecs/src/decoding/framing/mod.rs @@ -16,7 +16,7 @@ use ::bytes::Bytes; pub use character_delimited::{ CharacterDelimitedDecoder, CharacterDelimitedDecoderConfig, CharacterDelimitedDecoderOptions, }; -pub use chunked_gelf::{ChunkedGelfDecoder, ChunkedGelfDecoderConfig}; +pub use chunked_gelf::{ChunkedGelfDecoder, ChunkedGelfDecoderConfig, ChunkedGelfDecoderOptions}; use dyn_clone::DynClone; pub use length_delimited::{LengthDelimitedDecoder, LengthDelimitedDecoderConfig}; pub use newline_delimited::{ diff --git a/lib/codecs/src/decoding/mod.rs b/lib/codecs/src/decoding/mod.rs index 0eaef4929142c..9152b803a3bfa 100644 --- a/lib/codecs/src/decoding/mod.rs +++ b/lib/codecs/src/decoding/mod.rs @@ -19,12 +19,12 @@ pub use format::{ pub use format::{SyslogDeserializer, SyslogDeserializerConfig, SyslogDeserializerOptions}; pub use framing::{ BoxedFramer, BoxedFramingError, BytesDecoder, BytesDecoderConfig, CharacterDelimitedDecoder, - CharacterDelimitedDecoderConfig, CharacterDelimitedDecoderOptions, FramingError, - LengthDelimitedDecoder, LengthDelimitedDecoderConfig, NewlineDelimitedDecoder, - NewlineDelimitedDecoderConfig, NewlineDelimitedDecoderOptions, OctetCountingDecoder, - OctetCountingDecoderConfig, OctetCountingDecoderOptions, + CharacterDelimitedDecoderConfig, CharacterDelimitedDecoderOptions, ChunkedGelfDecoder, + ChunkedGelfDecoderConfig, ChunkedGelfDecoderOptions, FramingError, LengthDelimitedDecoder, + LengthDelimitedDecoderConfig, NewlineDelimitedDecoder, NewlineDelimitedDecoderConfig, + NewlineDelimitedDecoderOptions, OctetCountingDecoder, OctetCountingDecoderConfig, + OctetCountingDecoderOptions, }; -use framing::{ChunkedGelfDecoder, ChunkedGelfDecoderConfig}; use smallvec::SmallVec; use std::fmt::Debug; use vector_config::configurable_component; diff --git a/src/sources/socket/mod.rs b/src/sources/socket/mod.rs index d435d1e19bd0d..32d70902b2b00 100644 --- a/src/sources/socket/mod.rs +++ b/src/sources/socket/mod.rs @@ -333,17 +333,19 @@ mod test { use bytes::{BufMut, Bytes, BytesMut}; use futures::{stream, StreamExt}; + use rand::{rngs::SmallRng, seq::SliceRandom, SeedableRng}; + use serde_json::json; use tokio::io::AsyncReadExt; use tokio::net::TcpStream; use tokio::{ task::JoinHandle, time::{timeout, Duration, Instant}, }; - use vector_lib::codecs::NewlineDelimitedDecoderConfig; #[cfg(unix)] use vector_lib::codecs::{ decoding::CharacterDelimitedDecoderOptions, CharacterDelimitedDecoderConfig, }; + use vector_lib::codecs::{GelfDeserializerConfig, NewlineDelimitedDecoderConfig}; use vector_lib::event::EventContainer; use vector_lib::lookup::{lookup_v2::OptionalValuePath, owned_value_path, path}; use vrl::value::ObjectMap; @@ -854,20 +856,24 @@ mod test { //////// UDP TESTS //////// fn send_lines_udp(addr: SocketAddr, lines: impl IntoIterator) -> SocketAddr { + send_packets_udp(addr, lines.into_iter().map(|line| line.into())) + } + + fn send_packets_udp(addr: SocketAddr, packets: impl IntoIterator) -> SocketAddr { let bind = next_addr(); let socket = UdpSocket::bind(bind) .map_err(|error| panic!("{:}", error)) .ok() .unwrap(); - for line in lines { + for packet in packets { assert_eq!( socket - .send_to(line.as_bytes(), addr) + .send_to(&packet, addr) .map_err(|error| panic!("{:}", error)) .ok() .unwrap(), - line.as_bytes().len() + packet.len() ); // Space things out slightly to try to avoid dropped packets thread::sleep(Duration::from_millis(1)); @@ -880,6 +886,53 @@ mod test { bind } + fn get_gelf_payload(message: &str) -> String { + serde_json::to_string(&json!({ + "version": "1.1", + "host": "example.org", + "short_message": message, + "timestamp": 1234567890.123, + "level": 6, + "_foo": "bar", + })) + .unwrap() + } + + fn create_gelf_chunk( + message_id: u64, + sequence_number: u8, + total_chunks: u8, + payload: &[u8], + ) -> Bytes { + const GELF_MAGIC: [u8; 2] = [0x1e, 0x0f]; + let mut chunk = BytesMut::new(); + chunk.put_slice(&GELF_MAGIC); + chunk.put_u64(message_id); + chunk.put_u8(sequence_number); + chunk.put_u8(total_chunks); + chunk.put(payload); + chunk.freeze() + } + + fn get_gelf_chunks(short_message: &str, max_size: usize, rng: &mut SmallRng) -> Vec { + let message_id = rand::random(); + let payload = get_gelf_payload(short_message); + let payload_chunks = payload.as_bytes().chunks(max_size).collect::>(); + let total_chunks = payload_chunks.len(); + assert!(total_chunks <= 128, "too many gelf chunks"); + + let mut chunks = payload_chunks + .into_iter() + .enumerate() + .map(|(i, payload_chunk)| { + create_gelf_chunk(message_id, i as u8, total_chunks as u8, payload_chunk) + }) + .collect::>(); + // Shuffle the chunks to simulate out-of-order delivery + chunks.shuffle(rng); + chunks + } + async fn init_udp_with_shutdown( sender: SourceSender, source_id: &ComponentKey, @@ -1084,6 +1137,42 @@ mod test { .await; } + // TODO: maybe this should be in an integration test, such as `src/sources/redis/mod.rs` and `scripts/integration/redis` + // and so? There currently are no integration tests for the socket source + #[tokio::test] + async fn udp_decodes_chunked_gelf_messages() { + assert_source_compliance(&SOCKET_PUSH_SOURCE_TAGS, async { + let (tx, rx) = SourceSender::new_test(); + let address = next_addr(); + let mut config = UdpConfig::from_address(address.into()); + config.decoding = GelfDeserializerConfig::default().into(); + let address = init_udp_with_config(tx, config).await; + let seed = 42; + let mut rng = SmallRng::seed_from_u64(seed); + let max_size = 300; + let big_message = "This is a very large message".repeat(1000); + let another_big_message = "This is another very large message".repeat(1000); + let mut chunks = get_gelf_chunks(big_message.as_str(), max_size, &mut rng); + let mut another_chunks = + get_gelf_chunks(another_big_message.as_str(), max_size, &mut rng); + chunks.append(&mut another_chunks); + chunks.shuffle(&mut rng); + + send_packets_udp(address, chunks); + + let events = collect_n(rx, 2).await; + assert_eq!( + events[0].as_log()[log_schema().message_key().unwrap().to_string()], + big_message.into() + ); + assert_eq!( + events[1].as_log()[log_schema().message_key().unwrap().to_string()], + another_big_message.into() + ); + }) + .await; + } + #[tokio::test] async fn udp_it_includes_host() { assert_source_compliance(&SOCKET_PUSH_SOURCE_TAGS, async { diff --git a/src/sources/socket/udp.rs b/src/sources/socket/udp.rs index 80ed93da1468d..da5b97c75f7bf 100644 --- a/src/sources/socket/udp.rs +++ b/src/sources/socket/udp.rs @@ -79,7 +79,7 @@ pub struct UdpConfig { #[configurable(derived)] #[serde(default = "default_decoding")] - decoding: DeserializerConfig, + pub(super) decoding: DeserializerConfig, /// The namespace to use for logs. This overrides the global setting. #[serde(default)] From 2fda018ceed712c3b2796f13903719d07abc390f Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Wed, 10 Jul 2024 10:21:07 +0200 Subject: [PATCH 17/60] docs(gelf): Improve doc comments --- .../src/decoding/framing/chunked_gelf.rs | 67 +++++++++++-------- 1 file changed, 39 insertions(+), 28 deletions(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index 1818e46211b6e..2dcc92b931855 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -17,6 +17,7 @@ const GELF_MAX_TOTAL_CHUNKS: u8 = 128; const DEFAULT_CHUNKS: [Bytes; GELF_MAX_TOTAL_CHUNKS as usize] = [const { Bytes::new() }; GELF_MAX_TOTAL_CHUNKS as usize]; const DEFAULT_TIMEOUT_MILLIS: u64 = 5000; +// TODO: ask what would be an appropriate default value for this const DEFAULT_PENDING_MESSAGES_LIMIT: usize = 1000; const fn default_timeout_millis() -> u64 { @@ -27,7 +28,7 @@ const fn default_pending_messages_limit() -> usize { DEFAULT_PENDING_MESSAGES_LIMIT } -/// Config used to build a `ChunkedGelfDecoderConfig`. +/// Config used to build a `ChunkedGelfDecoder`. #[configurable_component] #[derive(Debug, Clone, Default, PartialEq, Eq)] pub struct ChunkedGelfDecoderConfig { @@ -37,12 +38,7 @@ pub struct ChunkedGelfDecoderConfig { } impl ChunkedGelfDecoderConfig { - /// Creates a new `BytesDecoderConfig`. - pub fn new() -> Self { - Default::default() - } - - /// Build the `ByteDecoder` from this configuration. + /// Build the `ChunkedGelfDecoder` from this configuration. pub fn build(&self) -> ChunkedGelfDecoder { ChunkedGelfDecoder::new( self.chunked_gelf.timeout_millis, @@ -56,7 +52,8 @@ impl ChunkedGelfDecoderConfig { #[derive(Clone, Debug, Derivative, PartialEq, Eq)] pub struct ChunkedGelfDecoderOptions { /// The timeout in milliseconds for a message to be fully received. If the timeout is reached, the - /// decoder will drop all the received chunks of the message and start over. + /// decoder will drop all the received chunks of the uncomplete message and start over. + /// The default value is 5 seconds. #[serde( default = "default_timeout_millis", skip_serializing_if = "vector_core::serde::is_default" @@ -64,7 +61,8 @@ pub struct ChunkedGelfDecoderOptions { pub timeout_millis: u64, /// The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start - /// dropping chunks of new messages. This limit ensures the memory usage of the decoder is bounded. + /// dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + /// The default value is 1000. #[serde( default = "default_pending_messages_limit", skip_serializing_if = "vector_core::serde::is_default" @@ -82,7 +80,7 @@ impl Default for ChunkedGelfDecoderOptions { } #[derive(Debug)] -pub struct MessageState { +struct MessageState { total_chunks: u8, chunks: [Bytes; GELF_MAX_TOTAL_CHUNKS as usize], chunks_bitmap: u128, @@ -129,7 +127,7 @@ impl MessageState { } } -/// A decoder for handling GELF messages that may be chunked. +/// A codec for handling GELF messages that may be chunked. #[derive(Debug, Clone)] pub struct ChunkedGelfDecoder { // We have to use this decoder to read all the bytes from the buffer first and don't let tokio @@ -137,7 +135,7 @@ pub struct ChunkedGelfDecoder { // whole message. (see https://docs.rs/tokio-util/latest/src/tokio_util/codec/framed_impl.rs.html#26). // This limitation is due to the fact that the GELF format does not specify the length of the // message, so we have to read all the bytes from the message (datagram) - inner_decoder: BytesDecoder, + bytes_decoder: BytesDecoder, state: Arc>>, timeout: Duration, pending_messages_limit: usize, @@ -147,15 +145,28 @@ impl ChunkedGelfDecoder { /// Creates a new `ChunkedGelfDecoder`. pub fn new(timeout_millis: u64, pending_messages_limit: usize) -> Self { Self { - inner_decoder: BytesDecoder::new(), + bytes_decoder: BytesDecoder::new(), state: Arc::new(Mutex::new(HashMap::new())), timeout: Duration::from_millis(timeout_millis), pending_messages_limit, } } - /// TODO: document this + /// Decode a GELF chunk pub fn decode_chunk(&mut self, mut chunk: Bytes) -> Result, BoxedFramingError> { + // Encoding scheme: + // + // +------------+-----------------+--------------+----------------------+ + // | Message id | Sequence number | Total chunks | Chunk payload | + // +------------+-----------------+--------------+----------------------+ + // | 64 bits | 8 bits | 8 bits | remaining bits | + // +------------+-----------------+--------------+----------------------+ + // + // As this codec is oriented for UDP, the chunks (datagrams) are not guaranteed to be received in order, + // nor to be received at all. So, we have to store the chunks in a buffer (state field) until we receive + // all the chunks of a message. When we receive all the chunks of a message, we can concatenate them + // and return the complete payload. + // We need 10 bits to read the message id, sequence number and total chunks if chunk.remaining() < 10 { let src_display = format!("{chunk:?}"); @@ -199,7 +210,6 @@ impl ChunkedGelfDecoder { message = "Received a chunk but reached the pending messages limit. Ignoring it.", message_id = message_id, sequence_number = sequence_number, - total_chunks = total_chunks, pending_messages_limit = self.pending_messages_limit, internal_log_rate_limit = true ); @@ -215,10 +225,11 @@ impl ChunkedGelfDecoder { tokio::time::sleep(timeout).await; let mut state_lock = state.lock().unwrap(); if let Some(_) = state_lock.remove(&message_id) { - // TODO: log the variables in the message or use structured logging? // TODO: record metrics here? it would be insteresting to know how many messages are being discarded + let message = format!("Message was not fully received within the timeout window of {}ms. Discarding it.", + timeout.as_millis()); warn!( - message = "Message was not fully received within the timeout window. Discarding it.", + message = message, message_id = message_id, timeout = timeout.as_millis(), internal_log_rate_limit = true @@ -242,8 +253,8 @@ impl ChunkedGelfDecoder { // TODO: add a PR comment asking for info or warn info!( message = "Received a duplicate chunk. Ignoring it.", - sequence_number = sequence_number, message_id = message_id, + sequence_number = sequence_number, internal_log_rate_limit = true ); return Ok(None); @@ -259,7 +270,9 @@ impl ChunkedGelfDecoder { } } - /// TODO: document this + /// Decode a GELF message that may be chunked or not. The source bytes are expected to be + /// datagram-based (or message-based), so it must not contain multiple GELF messages + /// delimited by '\0', such as it would be in a stream-based protocol. pub fn decode_message(&mut self, mut src: Bytes) -> Result, BoxedFramingError> { let magic = src.get(0..2); if magic.is_some_and(|magic| magic == GELF_MAGIC) { @@ -292,7 +305,7 @@ impl Decoder for ChunkedGelfDecoder { // the bytes decoder will always return a Ok(None) in this method, but leaving this // here for consistency. Would be better to add a unreachable/panic here if the inner decoder returns // the Some variant? - self.inner_decoder + self.bytes_decoder .decode(src)? .and_then(|frame| self.decode_message(frame).transpose()) .transpose() @@ -302,7 +315,7 @@ impl Decoder for ChunkedGelfDecoder { return Ok(None); } - self.inner_decoder + self.bytes_decoder .decode_eof(buf)? .and_then(|frame| self.decode_message(frame).transpose()) .transpose() @@ -337,8 +350,6 @@ mod tests { (BytesMut::from(payload), payload.to_string()) } - // TODO: add a malformed chunk message - #[fixture] fn two_chunks_message() -> ([BytesMut; 2], String) { let message_id = 1u64; @@ -492,16 +503,15 @@ mod tests { #[tokio::test(start_paused = true)] #[traced_test] async fn decode_timeout(two_chunks_message: ([BytesMut; 2], String)) { - let timeout = 300; let (mut chunks, _) = two_chunks_message; - let mut decoder = ChunkedGelfDecoder::new(timeout, DEFAULT_PENDING_MESSAGES_LIMIT); + let mut decoder = ChunkedGelfDecoder::default(); let frame = decoder.decode_eof(&mut chunks[0]).unwrap(); assert!(frame.is_none()); assert!(!decoder.state.lock().unwrap().is_empty()); // The message state should be cleared after a certain time - tokio::time::sleep(Duration::from_millis(timeout + 1)).await; + tokio::time::sleep(Duration::from_millis(DEFAULT_TIMEOUT_MILLIS + 1)).await; assert!(decoder.state.lock().unwrap().is_empty()); assert!(logs_contain( "Message was not fully received within the timeout window. Discarding it." @@ -510,7 +520,7 @@ mod tests { let frame = decoder.decode_eof(&mut chunks[1]).unwrap(); assert!(frame.is_none()); - tokio::time::sleep(Duration::from_millis(timeout + 1)).await; + tokio::time::sleep(Duration::from_millis(DEFAULT_TIMEOUT_MILLIS + 1)).await; assert!(decoder.state.lock().unwrap().is_empty()); assert!(logs_contain( "Message was not fully received within the timeout window. Discarding it." @@ -529,9 +539,10 @@ mod tests { #[tokio::test] #[traced_test] async fn decode_chunk_with_malformed_header() { - let malformed_chunk = [0x12, 0x23]; let mut src = BytesMut::new(); src.extend_from_slice(&GELF_MAGIC); + // Malformed chunk header with less than 10 bytes + let malformed_chunk = [0x12, 0x34]; src.extend_from_slice(&malformed_chunk); let mut decoder = ChunkedGelfDecoder::default(); From 0862c105d8f1a870a23f7980c321be823de93033 Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Thu, 11 Jul 2024 10:06:17 +0200 Subject: [PATCH 18/60] docs(gelf): Improve doc comments --- lib/codecs/src/decoding/framing/chunked_gelf.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index 2dcc92b931855..0fe31bf67f70f 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -167,7 +167,7 @@ impl ChunkedGelfDecoder { // all the chunks of a message. When we receive all the chunks of a message, we can concatenate them // and return the complete payload. - // We need 10 bits to read the message id, sequence number and total chunks + // We need 10 bytes to read the message id, sequence number and total chunks if chunk.remaining() < 10 { let src_display = format!("{chunk:?}"); warn!(message = "Received malformed chunk headers (message ID, sequence number and total chunks) with less than 10 bytes. Ignoring it.", From a5b112182a374671360b2b401088f558fc91f94c Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Thu, 11 Jul 2024 17:59:20 +0200 Subject: [PATCH 19/60] test(socket): Add chunked gelf tests for unix socket source --- .../src/decoding/framing/chunked_gelf.rs | 2 +- src/sources/socket/mod.rs | 175 ++++++++++++------ 2 files changed, 122 insertions(+), 55 deletions(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index 0fe31bf67f70f..a50da09038de7 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -523,7 +523,7 @@ mod tests { tokio::time::sleep(Duration::from_millis(DEFAULT_TIMEOUT_MILLIS + 1)).await; assert!(decoder.state.lock().unwrap().is_empty()); assert!(logs_contain( - "Message was not fully received within the timeout window. Discarding it." + "Message was not fully received within the timeout window of 5000ms. Discarding it" )); } diff --git a/src/sources/socket/mod.rs b/src/sources/socket/mod.rs index 32d70902b2b00..85c9e719c0785 100644 --- a/src/sources/socket/mod.rs +++ b/src/sources/socket/mod.rs @@ -150,8 +150,6 @@ impl SourceConfig for SocketConfig { ) } Mode::Udp(config) => { - // TODO: add tests for udp with chunked gelf, use a big payload so we ensure - // that the decode_eof method gets called due to tokio buffering let log_namespace = cx.log_namespace(config.log_namespace); let decoding = config.decoding().clone(); let framing = config @@ -169,7 +167,6 @@ impl SourceConfig for SocketConfig { } #[cfg(unix)] Mode::UnixDatagram(config) => { - // TODO: test for unix datagram with chunked gelf let log_namespace = cx.log_namespace(config.log_namespace); let decoding = config.decoding.clone(); let framing = config @@ -384,6 +381,53 @@ mod test { SourceSender, }; + fn get_gelf_payload(message: &str) -> String { + serde_json::to_string(&json!({ + "version": "1.1", + "host": "example.org", + "short_message": message, + "timestamp": 1234567890.123, + "level": 6, + "_foo": "bar", + })) + .unwrap() + } + + fn create_gelf_chunk( + message_id: u64, + sequence_number: u8, + total_chunks: u8, + payload: &[u8], + ) -> Bytes { + const GELF_MAGIC: [u8; 2] = [0x1e, 0x0f]; + let mut chunk = BytesMut::new(); + chunk.put_slice(&GELF_MAGIC); + chunk.put_u64(message_id); + chunk.put_u8(sequence_number); + chunk.put_u8(total_chunks); + chunk.put(payload); + chunk.freeze() + } + + fn get_gelf_chunks(short_message: &str, max_size: usize, rng: &mut SmallRng) -> Vec { + let message_id = rand::random(); + let payload = get_gelf_payload(short_message); + let payload_chunks = payload.as_bytes().chunks(max_size).collect::>(); + let total_chunks = payload_chunks.len(); + assert!(total_chunks <= 128, "too many gelf chunks"); + + let mut chunks = payload_chunks + .into_iter() + .enumerate() + .map(|(i, payload_chunk)| { + create_gelf_chunk(message_id, i as u8, total_chunks as u8, payload_chunk) + }) + .collect::>(); + // Shuffle the chunks to simulate out-of-order delivery + chunks.shuffle(rng); + chunks + } + #[test] fn generate_config() { crate::test_util::test_generate_config::(); @@ -886,53 +930,6 @@ mod test { bind } - fn get_gelf_payload(message: &str) -> String { - serde_json::to_string(&json!({ - "version": "1.1", - "host": "example.org", - "short_message": message, - "timestamp": 1234567890.123, - "level": 6, - "_foo": "bar", - })) - .unwrap() - } - - fn create_gelf_chunk( - message_id: u64, - sequence_number: u8, - total_chunks: u8, - payload: &[u8], - ) -> Bytes { - const GELF_MAGIC: [u8; 2] = [0x1e, 0x0f]; - let mut chunk = BytesMut::new(); - chunk.put_slice(&GELF_MAGIC); - chunk.put_u64(message_id); - chunk.put_u8(sequence_number); - chunk.put_u8(total_chunks); - chunk.put(payload); - chunk.freeze() - } - - fn get_gelf_chunks(short_message: &str, max_size: usize, rng: &mut SmallRng) -> Vec { - let message_id = rand::random(); - let payload = get_gelf_payload(short_message); - let payload_chunks = payload.as_bytes().chunks(max_size).collect::>(); - let total_chunks = payload_chunks.len(); - assert!(total_chunks <= 128, "too many gelf chunks"); - - let mut chunks = payload_chunks - .into_iter() - .enumerate() - .map(|(i, payload_chunk)| { - create_gelf_chunk(message_id, i as u8, total_chunks as u8, payload_chunk) - }) - .collect::>(); - // Shuffle the chunks to simulate out-of-order delivery - chunks.shuffle(rng); - chunks - } - async fn init_udp_with_shutdown( sender: SourceSender, source_id: &ComponentKey, @@ -1310,11 +1307,35 @@ mod test { } ////////////// UNIX TEST LIBS ////////////// + #[cfg(unix)] async fn init_unix(sender: SourceSender, stream: bool, use_vector_namespace: bool) -> PathBuf { - let in_path = tempfile::tempdir().unwrap().into_path().join("unix_test"); + init_unix_inner(sender, stream, use_vector_namespace, None).await + } + + #[cfg(unix)] + async fn init_unix_with_config( + sender: SourceSender, + stream: bool, + use_vector_namespace: bool, + config: UnixConfig, + ) -> PathBuf { + init_unix_inner(sender, stream, use_vector_namespace, Some(config)).await + } + + #[cfg(unix)] + async fn init_unix_inner( + sender: SourceSender, + stream: bool, + use_vector_namespace: bool, + config: Option, + ) -> PathBuf { + let mut config = config.unwrap_or_else(|| { + UnixConfig::new(tempfile::tempdir().unwrap().into_path().join("unix_test")) + }); + + let in_path = config.path.clone(); - let mut config = UnixConfig::new(in_path.clone()); if use_vector_namespace { config.log_namespace = Some(true); } @@ -1324,6 +1345,7 @@ mod test { } else { Mode::UnixDatagram(config) }; + let server = SocketConfig { mode } .build(SourceContext::new_test(sender, None)) .await @@ -1413,11 +1435,17 @@ mod test { ////////////// UNIX DATAGRAM TESTS ////////////// #[cfg(unix)] async fn send_lines_unix_datagram(path: PathBuf, lines: &[&str]) { + let packets = lines.into_iter().map(|line| Bytes::from(line.to_string())); + send_packets_unix_datagram(path, packets).await; + } + + #[cfg(unix)] + async fn send_packets_unix_datagram(path: PathBuf, packets: impl IntoIterator) { let socket = UnixDatagram::unbound().unwrap(); socket.connect(path).unwrap(); - for line in lines { - socket.send(line.as_bytes()).await.unwrap(); + for packet in packets { + socket.send(&packet).await.unwrap(); } socket.shutdown(std::net::Shutdown::Both).unwrap(); } @@ -1489,6 +1517,45 @@ mod test { assert_eq!(dgram, bytes); } + // TODO: maybe this should be in an integration test, such as `src/sources/redis/mod.rs` and `scripts/integration/redis` + // and so? There currently are no integration tests for the socket source + #[cfg(unix)] + #[tokio::test] + async fn unix_datagram_chunked_gelf_messages() { + assert_source_compliance(&SOCKET_PUSH_SOURCE_TAGS, async { + let (tx, rx) = SourceSender::new_test(); + let in_path = tempfile::tempdir().unwrap().into_path().join("unix_test"); + let mut config = UnixConfig::new(in_path.clone()); + config.decoding = GelfDeserializerConfig::default().into(); + let path = init_unix_with_config(tx, false, false, config).await; + let seed = 42; + let mut rng = SmallRng::seed_from_u64(seed); + // TODO: add a PR comment here stating that in MACOS at least, with bigger messages, the test fails + // with the `No buffer space available` error + let max_size = 20; + let big_message = "This is a very large message".repeat(5); + let another_big_message = "This is another very large message".repeat(5); + let mut chunks = get_gelf_chunks(big_message.as_str(), max_size, &mut rng); + let mut another_chunks = + get_gelf_chunks(another_big_message.as_str(), max_size, &mut rng); + chunks.append(&mut another_chunks); + chunks.shuffle(&mut rng); + + send_packets_unix_datagram(path, chunks).await; + + let events = collect_n(rx, 2).await; + assert_eq!( + events[1].as_log()[log_schema().message_key().unwrap().to_string()], + big_message.into() + ); + assert_eq!( + events[0].as_log()[log_schema().message_key().unwrap().to_string()], + another_big_message.into() + ); + }) + .await; + } + #[cfg(unix)] #[tokio::test] async fn unix_datagram_message_with_vector_namespace() { From a5757c6542a1c0b82da52f9721988b3722662f07 Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Mon, 15 Jul 2024 10:42:28 +0200 Subject: [PATCH 20/60] chore: remove comments --- lib/codecs/src/encoding/mod.rs | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/lib/codecs/src/encoding/mod.rs b/lib/codecs/src/encoding/mod.rs index c19c8123d847d..a9f51366eb538 100644 --- a/lib/codecs/src/encoding/mod.rs +++ b/lib/codecs/src/encoding/mod.rs @@ -549,6 +549,4 @@ impl tokio_util::codec::Encoder for Serializer { Serializer::Text(serializer) => serializer.encode(event, buffer), } } -} - -// TODO: add tests here for GELF chunked framing and parsing +} \ No newline at end of file From 33bda3657eb3a9ac592e44d90f0256770783e6db Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Mon, 15 Jul 2024 10:59:45 +0200 Subject: [PATCH 21/60] docs: add changelog --- changelog.d/20769-chunked_gelf_decoding.feature.md | 5 +++++ lib/codecs/src/decoding/framing/chunked_gelf.rs | 3 ++- 2 files changed, 7 insertions(+), 1 deletion(-) create mode 100644 changelog.d/20769-chunked_gelf_decoding.feature.md diff --git a/changelog.d/20769-chunked_gelf_decoding.feature.md b/changelog.d/20769-chunked_gelf_decoding.feature.md new file mode 100644 index 0000000000000..2dc8e4cc15547 --- /dev/null +++ b/changelog.d/20769-chunked_gelf_decoding.feature.md @@ -0,0 +1,5 @@ +Allows for chunked gelf decoding in message-based sources, such as UDP sockets or unix datagram sockets. Implementation is based on [Graylog's documentation](https://go2docs.graylog.org/5-0/getting_in_log_data/gelf.html#GELFviaUDP). + +This framing method can be configured via the `framing.method = "chunked_gelf"` option in the source configuration. + +authors: jorgehermo9 diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index a50da09038de7..86e45cbd7c96d 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -127,7 +127,8 @@ impl MessageState { } } -/// A codec for handling GELF messages that may be chunked. +/// A codec for handling GELF messages that may be chunked. The implementation is based on [Graylog's GELF documentation](https://go2docs.graylog.org/5-0/getting_in_log_data/gelf.html#GELFviaUDP) +/// and [Graylog's go-gelf library](https://github.com/Graylog2/go-gelf/blob/v1/gelf/reader.go). #[derive(Debug, Clone)] pub struct ChunkedGelfDecoder { // We have to use this decoder to read all the bytes from the buffer first and don't let tokio From 7cff6a798f63c46b28da4c96d39d47728b24087b Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Mon, 15 Jul 2024 12:38:46 +0200 Subject: [PATCH 22/60] chore: Remove comments --- lib/codecs/src/decoding/framing/chunked_gelf.rs | 7 ++----- lib/codecs/src/decoding/mod.rs | 1 + lib/codecs/src/encoding/format/native_json.rs | 3 --- lib/codecs/src/encoding/mod.rs | 2 +- src/sources/socket/mod.rs | 4 ---- 5 files changed, 4 insertions(+), 13 deletions(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index 86e45cbd7c96d..fa42989fc0a0f 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -17,7 +17,6 @@ const GELF_MAX_TOTAL_CHUNKS: u8 = 128; const DEFAULT_CHUNKS: [Bytes; GELF_MAX_TOTAL_CHUNKS as usize] = [const { Bytes::new() }; GELF_MAX_TOTAL_CHUNKS as usize]; const DEFAULT_TIMEOUT_MILLIS: u64 = 5000; -// TODO: ask what would be an appropriate default value for this const DEFAULT_PENDING_MESSAGES_LIMIT: usize = 1000; const fn default_timeout_millis() -> u64 { @@ -53,7 +52,7 @@ impl ChunkedGelfDecoderConfig { pub struct ChunkedGelfDecoderOptions { /// The timeout in milliseconds for a message to be fully received. If the timeout is reached, the /// decoder will drop all the received chunks of the uncomplete message and start over. - /// The default value is 5 seconds. + /// The default value is 5 seconds. #[serde( default = "default_timeout_millis", skip_serializing_if = "vector_core::serde::is_default" @@ -88,7 +87,7 @@ struct MessageState { } impl MessageState { - pub fn new(total_chunks: u8, timeout_task: JoinHandle<()>) -> Self { + pub const fn new(total_chunks: u8, timeout_task: JoinHandle<()>) -> Self { Self { total_chunks, chunks: DEFAULT_CHUNKS, @@ -226,7 +225,6 @@ impl ChunkedGelfDecoder { tokio::time::sleep(timeout).await; let mut state_lock = state.lock().unwrap(); if let Some(_) = state_lock.remove(&message_id) { - // TODO: record metrics here? it would be insteresting to know how many messages are being discarded let message = format!("Message was not fully received within the timeout window of {}ms. Discarding it.", timeout.as_millis()); warn!( @@ -251,7 +249,6 @@ impl ChunkedGelfDecoder { } if message_state.is_chunk_present(sequence_number) { - // TODO: add a PR comment asking for info or warn info!( message = "Received a duplicate chunk. Ignoring it.", message_id = message_id, diff --git a/lib/codecs/src/decoding/mod.rs b/lib/codecs/src/decoding/mod.rs index 9152b803a3bfa..4fcec583a0f53 100644 --- a/lib/codecs/src/decoding/mod.rs +++ b/lib/codecs/src/decoding/mod.rs @@ -135,6 +135,7 @@ impl From for FramingConfig { Self::OctetCounting(config) } } + impl From for FramingConfig { fn from(config: ChunkedGelfDecoderConfig) -> Self { Self::ChunkedGelf(config) diff --git a/lib/codecs/src/encoding/format/native_json.rs b/lib/codecs/src/encoding/format/native_json.rs index 7e4543e2063c6..60d43725d8017 100644 --- a/lib/codecs/src/encoding/format/native_json.rs +++ b/lib/codecs/src/encoding/format/native_json.rs @@ -98,9 +98,6 @@ mod tests { serializer .encode(histogram_event.clone(), &mut bytes) .unwrap(); - // TODO: remove this comment. Add a PR comment stating that converting the histogram_event to a serde_json::Value - // Changed the order of the keys to alphabetical order and the string comparison failed due to the Event struct - // not serializing the keys in the same order as the serde_json::Value. assert_eq!( bytes.freeze(), serde_json::to_string(&histogram_event).unwrap() diff --git a/lib/codecs/src/encoding/mod.rs b/lib/codecs/src/encoding/mod.rs index a9f51366eb538..0d766f73e4d17 100644 --- a/lib/codecs/src/encoding/mod.rs +++ b/lib/codecs/src/encoding/mod.rs @@ -549,4 +549,4 @@ impl tokio_util::codec::Encoder for Serializer { Serializer::Text(serializer) => serializer.encode(event, buffer), } } -} \ No newline at end of file +} diff --git a/src/sources/socket/mod.rs b/src/sources/socket/mod.rs index 85c9e719c0785..f4118046a4680 100644 --- a/src/sources/socket/mod.rs +++ b/src/sources/socket/mod.rs @@ -1134,8 +1134,6 @@ mod test { .await; } - // TODO: maybe this should be in an integration test, such as `src/sources/redis/mod.rs` and `scripts/integration/redis` - // and so? There currently are no integration tests for the socket source #[tokio::test] async fn udp_decodes_chunked_gelf_messages() { assert_source_compliance(&SOCKET_PUSH_SOURCE_TAGS, async { @@ -1517,8 +1515,6 @@ mod test { assert_eq!(dgram, bytes); } - // TODO: maybe this should be in an integration test, such as `src/sources/redis/mod.rs` and `scripts/integration/redis` - // and so? There currently are no integration tests for the socket source #[cfg(unix)] #[tokio::test] async fn unix_datagram_chunked_gelf_messages() { From 119e1223da158597ab332e01b4f8ecac5245796d Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Mon, 15 Jul 2024 12:40:02 +0200 Subject: [PATCH 23/60] chore: Remove comments --- src/sources/socket/mod.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/sources/socket/mod.rs b/src/sources/socket/mod.rs index f4118046a4680..c06fb141440fe 100644 --- a/src/sources/socket/mod.rs +++ b/src/sources/socket/mod.rs @@ -1526,8 +1526,6 @@ mod test { let path = init_unix_with_config(tx, false, false, config).await; let seed = 42; let mut rng = SmallRng::seed_from_u64(seed); - // TODO: add a PR comment here stating that in MACOS at least, with bigger messages, the test fails - // with the `No buffer space available` error let max_size = 20; let big_message = "This is a very large message".repeat(5); let another_big_message = "This is another very large message".repeat(5); From 1f2751b219b287e26fae78c90ecd9c2c775e768c Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Tue, 16 Jul 2024 13:25:54 +0200 Subject: [PATCH 24/60] docs(framing): Add chunked gelf framing documentation --- .../components/sources/base/amqp.cue | 34 +++++++++++++++++-- .../sources/base/aws_kinesis_firehose.cue | 34 +++++++++++++++++-- .../components/sources/base/aws_s3.cue | 34 +++++++++++++++++-- .../components/sources/base/aws_sqs.cue | 34 +++++++++++++++++-- .../components/sources/base/datadog_agent.cue | 34 +++++++++++++++++-- .../components/sources/base/demo_logs.cue | 34 +++++++++++++++++-- .../components/sources/base/exec.cue | 34 +++++++++++++++++-- .../sources/base/file_descriptor.cue | 34 +++++++++++++++++-- .../components/sources/base/gcp_pubsub.cue | 34 +++++++++++++++++-- .../components/sources/base/heroku_logs.cue | 34 +++++++++++++++++-- .../components/sources/base/http.cue | 34 +++++++++++++++++-- .../components/sources/base/http_client.cue | 34 +++++++++++++++++-- .../components/sources/base/http_server.cue | 34 +++++++++++++++++-- .../components/sources/base/kafka.cue | 34 +++++++++++++++++-- .../components/sources/base/nats.cue | 34 +++++++++++++++++-- .../components/sources/base/pulsar.cue | 34 +++++++++++++++++-- .../components/sources/base/redis.cue | 34 +++++++++++++++++-- .../components/sources/base/socket.cue | 34 +++++++++++++++++-- .../components/sources/base/stdin.cue | 34 +++++++++++++++++-- 19 files changed, 608 insertions(+), 38 deletions(-) diff --git a/website/cue/reference/components/sources/base/amqp.cue b/website/cue/reference/components/sources/base/amqp.cue index 667b607d88044..ca655a51068c9 100644 --- a/website/cue/reference/components/sources/base/amqp.cue +++ b/website/cue/reference/components/sources/base/amqp.cue @@ -309,6 +309,31 @@ base: components: sources: amqp: configuration: { } } } + chunked_gelf: { + description: "Options for the chunked gelf decoder." + relevant_when: "method = \"chunked_gelf\"" + required: false + type: object: options: { + pending_messages_limit: { + description: """ + The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + The default value is 1000. + """ + required: false + type: uint: default: 1000 + } + timeout_millis: { + description: """ + The timeout in milliseconds for a message to be fully received. If the timeout is reached, the + decoder will drop all the received chunks of the uncomplete message and start over. + The default value is 5 seconds. + """ + required: false + type: uint: default: 5000 + } + } + } length_delimited: { description: "Options for the length delimited decoder." relevant_when: "method = \"length_delimited\"" @@ -344,8 +369,13 @@ base: components: sources: amqp: configuration: { enum: { bytes: "Byte frames are passed through as-is according to the underlying I/O boundaries (for example, split between messages or stream segments)." character_delimited: "Byte frames which are delimited by a chosen character." - length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." - newline_delimited: "Byte frames which are delimited by a newline character." + chunked_gelf: """ + Byte frames which are chunked GELF messages. + + [chunked_gelf]: https://go2docs.graylog.org/current/getting_in_log_data/gelf.html + """ + length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." + newline_delimited: "Byte frames which are delimited by a newline character." octet_counting: """ Byte frames according to the [octet counting][octet_counting] format. diff --git a/website/cue/reference/components/sources/base/aws_kinesis_firehose.cue b/website/cue/reference/components/sources/base/aws_kinesis_firehose.cue index 12418c29fc811..eff1f5f71c40c 100644 --- a/website/cue/reference/components/sources/base/aws_kinesis_firehose.cue +++ b/website/cue/reference/components/sources/base/aws_kinesis_firehose.cue @@ -307,6 +307,31 @@ base: components: sources: aws_kinesis_firehose: configuration: { } } } + chunked_gelf: { + description: "Options for the chunked gelf decoder." + relevant_when: "method = \"chunked_gelf\"" + required: false + type: object: options: { + pending_messages_limit: { + description: """ + The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + The default value is 1000. + """ + required: false + type: uint: default: 1000 + } + timeout_millis: { + description: """ + The timeout in milliseconds for a message to be fully received. If the timeout is reached, the + decoder will drop all the received chunks of the uncomplete message and start over. + The default value is 5 seconds. + """ + required: false + type: uint: default: 5000 + } + } + } length_delimited: { description: "Options for the length delimited decoder." relevant_when: "method = \"length_delimited\"" @@ -342,8 +367,13 @@ base: components: sources: aws_kinesis_firehose: configuration: { enum: { bytes: "Byte frames are passed through as-is according to the underlying I/O boundaries (for example, split between messages or stream segments)." character_delimited: "Byte frames which are delimited by a chosen character." - length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." - newline_delimited: "Byte frames which are delimited by a newline character." + chunked_gelf: """ + Byte frames which are chunked GELF messages. + + [chunked_gelf]: https://go2docs.graylog.org/current/getting_in_log_data/gelf.html + """ + length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." + newline_delimited: "Byte frames which are delimited by a newline character." octet_counting: """ Byte frames according to the [octet counting][octet_counting] format. diff --git a/website/cue/reference/components/sources/base/aws_s3.cue b/website/cue/reference/components/sources/base/aws_s3.cue index 8827d8b883771..5012e5099e0eb 100644 --- a/website/cue/reference/components/sources/base/aws_s3.cue +++ b/website/cue/reference/components/sources/base/aws_s3.cue @@ -407,6 +407,31 @@ base: components: sources: aws_s3: configuration: { } } } + chunked_gelf: { + description: "Options for the chunked gelf decoder." + relevant_when: "method = \"chunked_gelf\"" + required: false + type: object: options: { + pending_messages_limit: { + description: """ + The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + The default value is 1000. + """ + required: false + type: uint: default: 1000 + } + timeout_millis: { + description: """ + The timeout in milliseconds for a message to be fully received. If the timeout is reached, the + decoder will drop all the received chunks of the uncomplete message and start over. + The default value is 5 seconds. + """ + required: false + type: uint: default: 5000 + } + } + } length_delimited: { description: "Options for the length delimited decoder." relevant_when: "method = \"length_delimited\"" @@ -442,8 +467,13 @@ base: components: sources: aws_s3: configuration: { enum: { bytes: "Byte frames are passed through as-is according to the underlying I/O boundaries (for example, split between messages or stream segments)." character_delimited: "Byte frames which are delimited by a chosen character." - length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." - newline_delimited: "Byte frames which are delimited by a newline character." + chunked_gelf: """ + Byte frames which are chunked GELF messages. + + [chunked_gelf]: https://go2docs.graylog.org/current/getting_in_log_data/gelf.html + """ + length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." + newline_delimited: "Byte frames which are delimited by a newline character." octet_counting: """ Byte frames according to the [octet counting][octet_counting] format. diff --git a/website/cue/reference/components/sources/base/aws_sqs.cue b/website/cue/reference/components/sources/base/aws_sqs.cue index 7573e19e67068..d7792abd9cc3a 100644 --- a/website/cue/reference/components/sources/base/aws_sqs.cue +++ b/website/cue/reference/components/sources/base/aws_sqs.cue @@ -411,6 +411,31 @@ base: components: sources: aws_sqs: configuration: { } } } + chunked_gelf: { + description: "Options for the chunked gelf decoder." + relevant_when: "method = \"chunked_gelf\"" + required: false + type: object: options: { + pending_messages_limit: { + description: """ + The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + The default value is 1000. + """ + required: false + type: uint: default: 1000 + } + timeout_millis: { + description: """ + The timeout in milliseconds for a message to be fully received. If the timeout is reached, the + decoder will drop all the received chunks of the uncomplete message and start over. + The default value is 5 seconds. + """ + required: false + type: uint: default: 5000 + } + } + } length_delimited: { description: "Options for the length delimited decoder." relevant_when: "method = \"length_delimited\"" @@ -446,8 +471,13 @@ base: components: sources: aws_sqs: configuration: { enum: { bytes: "Byte frames are passed through as-is according to the underlying I/O boundaries (for example, split between messages or stream segments)." character_delimited: "Byte frames which are delimited by a chosen character." - length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." - newline_delimited: "Byte frames which are delimited by a newline character." + chunked_gelf: """ + Byte frames which are chunked GELF messages. + + [chunked_gelf]: https://go2docs.graylog.org/current/getting_in_log_data/gelf.html + """ + length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." + newline_delimited: "Byte frames which are delimited by a newline character." octet_counting: """ Byte frames according to the [octet counting][octet_counting] format. diff --git a/website/cue/reference/components/sources/base/datadog_agent.cue b/website/cue/reference/components/sources/base/datadog_agent.cue index ba8c42768b138..e154d3d82b632 100644 --- a/website/cue/reference/components/sources/base/datadog_agent.cue +++ b/website/cue/reference/components/sources/base/datadog_agent.cue @@ -304,6 +304,31 @@ base: components: sources: datadog_agent: configuration: { } } } + chunked_gelf: { + description: "Options for the chunked gelf decoder." + relevant_when: "method = \"chunked_gelf\"" + required: false + type: object: options: { + pending_messages_limit: { + description: """ + The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + The default value is 1000. + """ + required: false + type: uint: default: 1000 + } + timeout_millis: { + description: """ + The timeout in milliseconds for a message to be fully received. If the timeout is reached, the + decoder will drop all the received chunks of the uncomplete message and start over. + The default value is 5 seconds. + """ + required: false + type: uint: default: 5000 + } + } + } length_delimited: { description: "Options for the length delimited decoder." relevant_when: "method = \"length_delimited\"" @@ -339,8 +364,13 @@ base: components: sources: datadog_agent: configuration: { enum: { bytes: "Byte frames are passed through as-is according to the underlying I/O boundaries (for example, split between messages or stream segments)." character_delimited: "Byte frames which are delimited by a chosen character." - length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." - newline_delimited: "Byte frames which are delimited by a newline character." + chunked_gelf: """ + Byte frames which are chunked GELF messages. + + [chunked_gelf]: https://go2docs.graylog.org/current/getting_in_log_data/gelf.html + """ + length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." + newline_delimited: "Byte frames which are delimited by a newline character." octet_counting: """ Byte frames according to the [octet counting][octet_counting] format. diff --git a/website/cue/reference/components/sources/base/demo_logs.cue b/website/cue/reference/components/sources/base/demo_logs.cue index 3d2a892210d8c..bc7b938508221 100644 --- a/website/cue/reference/components/sources/base/demo_logs.cue +++ b/website/cue/reference/components/sources/base/demo_logs.cue @@ -300,6 +300,31 @@ base: components: sources: demo_logs: configuration: { } } } + chunked_gelf: { + description: "Options for the chunked gelf decoder." + relevant_when: "method = \"chunked_gelf\"" + required: false + type: object: options: { + pending_messages_limit: { + description: """ + The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + The default value is 1000. + """ + required: false + type: uint: default: 1000 + } + timeout_millis: { + description: """ + The timeout in milliseconds for a message to be fully received. If the timeout is reached, the + decoder will drop all the received chunks of the uncomplete message and start over. + The default value is 5 seconds. + """ + required: false + type: uint: default: 5000 + } + } + } length_delimited: { description: "Options for the length delimited decoder." relevant_when: "method = \"length_delimited\"" @@ -335,8 +360,13 @@ base: components: sources: demo_logs: configuration: { enum: { bytes: "Byte frames are passed through as-is according to the underlying I/O boundaries (for example, split between messages or stream segments)." character_delimited: "Byte frames which are delimited by a chosen character." - length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." - newline_delimited: "Byte frames which are delimited by a newline character." + chunked_gelf: """ + Byte frames which are chunked GELF messages. + + [chunked_gelf]: https://go2docs.graylog.org/current/getting_in_log_data/gelf.html + """ + length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." + newline_delimited: "Byte frames which are delimited by a newline character." octet_counting: """ Byte frames according to the [octet counting][octet_counting] format. diff --git a/website/cue/reference/components/sources/base/exec.cue b/website/cue/reference/components/sources/base/exec.cue index 0369f085bb761..ed59eb864edf8 100644 --- a/website/cue/reference/components/sources/base/exec.cue +++ b/website/cue/reference/components/sources/base/exec.cue @@ -288,6 +288,31 @@ base: components: sources: exec: configuration: { } } } + chunked_gelf: { + description: "Options for the chunked gelf decoder." + relevant_when: "method = \"chunked_gelf\"" + required: false + type: object: options: { + pending_messages_limit: { + description: """ + The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + The default value is 1000. + """ + required: false + type: uint: default: 1000 + } + timeout_millis: { + description: """ + The timeout in milliseconds for a message to be fully received. If the timeout is reached, the + decoder will drop all the received chunks of the uncomplete message and start over. + The default value is 5 seconds. + """ + required: false + type: uint: default: 5000 + } + } + } length_delimited: { description: "Options for the length delimited decoder." relevant_when: "method = \"length_delimited\"" @@ -321,8 +346,13 @@ base: components: sources: exec: configuration: { type: string: enum: { bytes: "Byte frames are passed through as-is according to the underlying I/O boundaries (for example, split between messages or stream segments)." character_delimited: "Byte frames which are delimited by a chosen character." - length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." - newline_delimited: "Byte frames which are delimited by a newline character." + chunked_gelf: """ + Byte frames which are chunked GELF messages. + + [chunked_gelf]: https://go2docs.graylog.org/current/getting_in_log_data/gelf.html + """ + length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." + newline_delimited: "Byte frames which are delimited by a newline character." octet_counting: """ Byte frames according to the [octet counting][octet_counting] format. diff --git a/website/cue/reference/components/sources/base/file_descriptor.cue b/website/cue/reference/components/sources/base/file_descriptor.cue index 63387d882f290..0ed97d8c9c61c 100644 --- a/website/cue/reference/components/sources/base/file_descriptor.cue +++ b/website/cue/reference/components/sources/base/file_descriptor.cue @@ -266,6 +266,31 @@ base: components: sources: file_descriptor: configuration: { } } } + chunked_gelf: { + description: "Options for the chunked gelf decoder." + relevant_when: "method = \"chunked_gelf\"" + required: false + type: object: options: { + pending_messages_limit: { + description: """ + The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + The default value is 1000. + """ + required: false + type: uint: default: 1000 + } + timeout_millis: { + description: """ + The timeout in milliseconds for a message to be fully received. If the timeout is reached, the + decoder will drop all the received chunks of the uncomplete message and start over. + The default value is 5 seconds. + """ + required: false + type: uint: default: 5000 + } + } + } length_delimited: { description: "Options for the length delimited decoder." relevant_when: "method = \"length_delimited\"" @@ -299,8 +324,13 @@ base: components: sources: file_descriptor: configuration: { type: string: enum: { bytes: "Byte frames are passed through as-is according to the underlying I/O boundaries (for example, split between messages or stream segments)." character_delimited: "Byte frames which are delimited by a chosen character." - length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." - newline_delimited: "Byte frames which are delimited by a newline character." + chunked_gelf: """ + Byte frames which are chunked GELF messages. + + [chunked_gelf]: https://go2docs.graylog.org/current/getting_in_log_data/gelf.html + """ + length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." + newline_delimited: "Byte frames which are delimited by a newline character." octet_counting: """ Byte frames according to the [octet counting][octet_counting] format. diff --git a/website/cue/reference/components/sources/base/gcp_pubsub.cue b/website/cue/reference/components/sources/base/gcp_pubsub.cue index cd655c4c3dd0b..248057af5e2e7 100644 --- a/website/cue/reference/components/sources/base/gcp_pubsub.cue +++ b/website/cue/reference/components/sources/base/gcp_pubsub.cue @@ -343,6 +343,31 @@ base: components: sources: gcp_pubsub: configuration: { } } } + chunked_gelf: { + description: "Options for the chunked gelf decoder." + relevant_when: "method = \"chunked_gelf\"" + required: false + type: object: options: { + pending_messages_limit: { + description: """ + The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + The default value is 1000. + """ + required: false + type: uint: default: 1000 + } + timeout_millis: { + description: """ + The timeout in milliseconds for a message to be fully received. If the timeout is reached, the + decoder will drop all the received chunks of the uncomplete message and start over. + The default value is 5 seconds. + """ + required: false + type: uint: default: 5000 + } + } + } length_delimited: { description: "Options for the length delimited decoder." relevant_when: "method = \"length_delimited\"" @@ -378,8 +403,13 @@ base: components: sources: gcp_pubsub: configuration: { enum: { bytes: "Byte frames are passed through as-is according to the underlying I/O boundaries (for example, split between messages or stream segments)." character_delimited: "Byte frames which are delimited by a chosen character." - length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." - newline_delimited: "Byte frames which are delimited by a newline character." + chunked_gelf: """ + Byte frames which are chunked GELF messages. + + [chunked_gelf]: https://go2docs.graylog.org/current/getting_in_log_data/gelf.html + """ + length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." + newline_delimited: "Byte frames which are delimited by a newline character." octet_counting: """ Byte frames according to the [octet counting][octet_counting] format. diff --git a/website/cue/reference/components/sources/base/heroku_logs.cue b/website/cue/reference/components/sources/base/heroku_logs.cue index 33f6be7194f6a..54b9e5eea48b6 100644 --- a/website/cue/reference/components/sources/base/heroku_logs.cue +++ b/website/cue/reference/components/sources/base/heroku_logs.cue @@ -301,6 +301,31 @@ base: components: sources: heroku_logs: configuration: { } } } + chunked_gelf: { + description: "Options for the chunked gelf decoder." + relevant_when: "method = \"chunked_gelf\"" + required: false + type: object: options: { + pending_messages_limit: { + description: """ + The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + The default value is 1000. + """ + required: false + type: uint: default: 1000 + } + timeout_millis: { + description: """ + The timeout in milliseconds for a message to be fully received. If the timeout is reached, the + decoder will drop all the received chunks of the uncomplete message and start over. + The default value is 5 seconds. + """ + required: false + type: uint: default: 5000 + } + } + } length_delimited: { description: "Options for the length delimited decoder." relevant_when: "method = \"length_delimited\"" @@ -336,8 +361,13 @@ base: components: sources: heroku_logs: configuration: { enum: { bytes: "Byte frames are passed through as-is according to the underlying I/O boundaries (for example, split between messages or stream segments)." character_delimited: "Byte frames which are delimited by a chosen character." - length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." - newline_delimited: "Byte frames which are delimited by a newline character." + chunked_gelf: """ + Byte frames which are chunked GELF messages. + + [chunked_gelf]: https://go2docs.graylog.org/current/getting_in_log_data/gelf.html + """ + length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." + newline_delimited: "Byte frames which are delimited by a newline character." octet_counting: """ Byte frames according to the [octet counting][octet_counting] format. diff --git a/website/cue/reference/components/sources/base/http.cue b/website/cue/reference/components/sources/base/http.cue index 73ae0efbe4ac7..182f492cee352 100644 --- a/website/cue/reference/components/sources/base/http.cue +++ b/website/cue/reference/components/sources/base/http.cue @@ -316,6 +316,31 @@ base: components: sources: http: configuration: { } } } + chunked_gelf: { + description: "Options for the chunked gelf decoder." + relevant_when: "method = \"chunked_gelf\"" + required: false + type: object: options: { + pending_messages_limit: { + description: """ + The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + The default value is 1000. + """ + required: false + type: uint: default: 1000 + } + timeout_millis: { + description: """ + The timeout in milliseconds for a message to be fully received. If the timeout is reached, the + decoder will drop all the received chunks of the uncomplete message and start over. + The default value is 5 seconds. + """ + required: false + type: uint: default: 5000 + } + } + } length_delimited: { description: "Options for the length delimited decoder." relevant_when: "method = \"length_delimited\"" @@ -349,8 +374,13 @@ base: components: sources: http: configuration: { type: string: enum: { bytes: "Byte frames are passed through as-is according to the underlying I/O boundaries (for example, split between messages or stream segments)." character_delimited: "Byte frames which are delimited by a chosen character." - length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." - newline_delimited: "Byte frames which are delimited by a newline character." + chunked_gelf: """ + Byte frames which are chunked GELF messages. + + [chunked_gelf]: https://go2docs.graylog.org/current/getting_in_log_data/gelf.html + """ + length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." + newline_delimited: "Byte frames which are delimited by a newline character." octet_counting: """ Byte frames according to the [octet counting][octet_counting] format. diff --git a/website/cue/reference/components/sources/base/http_client.cue b/website/cue/reference/components/sources/base/http_client.cue index 38485fb373cd2..8aa20dacd8f51 100644 --- a/website/cue/reference/components/sources/base/http_client.cue +++ b/website/cue/reference/components/sources/base/http_client.cue @@ -304,6 +304,31 @@ base: components: sources: http_client: configuration: { } } } + chunked_gelf: { + description: "Options for the chunked gelf decoder." + relevant_when: "method = \"chunked_gelf\"" + required: false + type: object: options: { + pending_messages_limit: { + description: """ + The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + The default value is 1000. + """ + required: false + type: uint: default: 1000 + } + timeout_millis: { + description: """ + The timeout in milliseconds for a message to be fully received. If the timeout is reached, the + decoder will drop all the received chunks of the uncomplete message and start over. + The default value is 5 seconds. + """ + required: false + type: uint: default: 5000 + } + } + } length_delimited: { description: "Options for the length delimited decoder." relevant_when: "method = \"length_delimited\"" @@ -339,8 +364,13 @@ base: components: sources: http_client: configuration: { enum: { bytes: "Byte frames are passed through as-is according to the underlying I/O boundaries (for example, split between messages or stream segments)." character_delimited: "Byte frames which are delimited by a chosen character." - length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." - newline_delimited: "Byte frames which are delimited by a newline character." + chunked_gelf: """ + Byte frames which are chunked GELF messages. + + [chunked_gelf]: https://go2docs.graylog.org/current/getting_in_log_data/gelf.html + """ + length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." + newline_delimited: "Byte frames which are delimited by a newline character." octet_counting: """ Byte frames according to the [octet counting][octet_counting] format. diff --git a/website/cue/reference/components/sources/base/http_server.cue b/website/cue/reference/components/sources/base/http_server.cue index 3837266a94199..7cb2df523676c 100644 --- a/website/cue/reference/components/sources/base/http_server.cue +++ b/website/cue/reference/components/sources/base/http_server.cue @@ -316,6 +316,31 @@ base: components: sources: http_server: configuration: { } } } + chunked_gelf: { + description: "Options for the chunked gelf decoder." + relevant_when: "method = \"chunked_gelf\"" + required: false + type: object: options: { + pending_messages_limit: { + description: """ + The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + The default value is 1000. + """ + required: false + type: uint: default: 1000 + } + timeout_millis: { + description: """ + The timeout in milliseconds for a message to be fully received. If the timeout is reached, the + decoder will drop all the received chunks of the uncomplete message and start over. + The default value is 5 seconds. + """ + required: false + type: uint: default: 5000 + } + } + } length_delimited: { description: "Options for the length delimited decoder." relevant_when: "method = \"length_delimited\"" @@ -349,8 +374,13 @@ base: components: sources: http_server: configuration: { type: string: enum: { bytes: "Byte frames are passed through as-is according to the underlying I/O boundaries (for example, split between messages or stream segments)." character_delimited: "Byte frames which are delimited by a chosen character." - length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." - newline_delimited: "Byte frames which are delimited by a newline character." + chunked_gelf: """ + Byte frames which are chunked GELF messages. + + [chunked_gelf]: https://go2docs.graylog.org/current/getting_in_log_data/gelf.html + """ + length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." + newline_delimited: "Byte frames which are delimited by a newline character." octet_counting: """ Byte frames according to the [octet counting][octet_counting] format. diff --git a/website/cue/reference/components/sources/base/kafka.cue b/website/cue/reference/components/sources/base/kafka.cue index f431399ba5c37..89b55000e4f3b 100644 --- a/website/cue/reference/components/sources/base/kafka.cue +++ b/website/cue/reference/components/sources/base/kafka.cue @@ -337,6 +337,31 @@ base: components: sources: kafka: configuration: { } } } + chunked_gelf: { + description: "Options for the chunked gelf decoder." + relevant_when: "method = \"chunked_gelf\"" + required: false + type: object: options: { + pending_messages_limit: { + description: """ + The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + The default value is 1000. + """ + required: false + type: uint: default: 1000 + } + timeout_millis: { + description: """ + The timeout in milliseconds for a message to be fully received. If the timeout is reached, the + decoder will drop all the received chunks of the uncomplete message and start over. + The default value is 5 seconds. + """ + required: false + type: uint: default: 5000 + } + } + } length_delimited: { description: "Options for the length delimited decoder." relevant_when: "method = \"length_delimited\"" @@ -372,8 +397,13 @@ base: components: sources: kafka: configuration: { enum: { bytes: "Byte frames are passed through as-is according to the underlying I/O boundaries (for example, split between messages or stream segments)." character_delimited: "Byte frames which are delimited by a chosen character." - length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." - newline_delimited: "Byte frames which are delimited by a newline character." + chunked_gelf: """ + Byte frames which are chunked GELF messages. + + [chunked_gelf]: https://go2docs.graylog.org/current/getting_in_log_data/gelf.html + """ + length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." + newline_delimited: "Byte frames which are delimited by a newline character." octet_counting: """ Byte frames according to the [octet counting][octet_counting] format. diff --git a/website/cue/reference/components/sources/base/nats.cue b/website/cue/reference/components/sources/base/nats.cue index 36cf215f3f55a..f44aa2ef83671 100644 --- a/website/cue/reference/components/sources/base/nats.cue +++ b/website/cue/reference/components/sources/base/nats.cue @@ -356,6 +356,31 @@ base: components: sources: nats: configuration: { } } } + chunked_gelf: { + description: "Options for the chunked gelf decoder." + relevant_when: "method = \"chunked_gelf\"" + required: false + type: object: options: { + pending_messages_limit: { + description: """ + The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + The default value is 1000. + """ + required: false + type: uint: default: 1000 + } + timeout_millis: { + description: """ + The timeout in milliseconds for a message to be fully received. If the timeout is reached, the + decoder will drop all the received chunks of the uncomplete message and start over. + The default value is 5 seconds. + """ + required: false + type: uint: default: 5000 + } + } + } length_delimited: { description: "Options for the length delimited decoder." relevant_when: "method = \"length_delimited\"" @@ -391,8 +416,13 @@ base: components: sources: nats: configuration: { enum: { bytes: "Byte frames are passed through as-is according to the underlying I/O boundaries (for example, split between messages or stream segments)." character_delimited: "Byte frames which are delimited by a chosen character." - length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." - newline_delimited: "Byte frames which are delimited by a newline character." + chunked_gelf: """ + Byte frames which are chunked GELF messages. + + [chunked_gelf]: https://go2docs.graylog.org/current/getting_in_log_data/gelf.html + """ + length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." + newline_delimited: "Byte frames which are delimited by a newline character." octet_counting: """ Byte frames according to the [octet counting][octet_counting] format. diff --git a/website/cue/reference/components/sources/base/pulsar.cue b/website/cue/reference/components/sources/base/pulsar.cue index f12f205944955..e3c775a1018cf 100644 --- a/website/cue/reference/components/sources/base/pulsar.cue +++ b/website/cue/reference/components/sources/base/pulsar.cue @@ -367,6 +367,31 @@ base: components: sources: pulsar: configuration: { } } } + chunked_gelf: { + description: "Options for the chunked gelf decoder." + relevant_when: "method = \"chunked_gelf\"" + required: false + type: object: options: { + pending_messages_limit: { + description: """ + The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + The default value is 1000. + """ + required: false + type: uint: default: 1000 + } + timeout_millis: { + description: """ + The timeout in milliseconds for a message to be fully received. If the timeout is reached, the + decoder will drop all the received chunks of the uncomplete message and start over. + The default value is 5 seconds. + """ + required: false + type: uint: default: 5000 + } + } + } length_delimited: { description: "Options for the length delimited decoder." relevant_when: "method = \"length_delimited\"" @@ -402,8 +427,13 @@ base: components: sources: pulsar: configuration: { enum: { bytes: "Byte frames are passed through as-is according to the underlying I/O boundaries (for example, split between messages or stream segments)." character_delimited: "Byte frames which are delimited by a chosen character." - length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." - newline_delimited: "Byte frames which are delimited by a newline character." + chunked_gelf: """ + Byte frames which are chunked GELF messages. + + [chunked_gelf]: https://go2docs.graylog.org/current/getting_in_log_data/gelf.html + """ + length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." + newline_delimited: "Byte frames which are delimited by a newline character." octet_counting: """ Byte frames according to the [octet counting][octet_counting] format. diff --git a/website/cue/reference/components/sources/base/redis.cue b/website/cue/reference/components/sources/base/redis.cue index 039ba3c0fe1b1..534fd827d4579 100644 --- a/website/cue/reference/components/sources/base/redis.cue +++ b/website/cue/reference/components/sources/base/redis.cue @@ -274,6 +274,31 @@ base: components: sources: redis: configuration: { } } } + chunked_gelf: { + description: "Options for the chunked gelf decoder." + relevant_when: "method = \"chunked_gelf\"" + required: false + type: object: options: { + pending_messages_limit: { + description: """ + The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + The default value is 1000. + """ + required: false + type: uint: default: 1000 + } + timeout_millis: { + description: """ + The timeout in milliseconds for a message to be fully received. If the timeout is reached, the + decoder will drop all the received chunks of the uncomplete message and start over. + The default value is 5 seconds. + """ + required: false + type: uint: default: 5000 + } + } + } length_delimited: { description: "Options for the length delimited decoder." relevant_when: "method = \"length_delimited\"" @@ -309,8 +334,13 @@ base: components: sources: redis: configuration: { enum: { bytes: "Byte frames are passed through as-is according to the underlying I/O boundaries (for example, split between messages or stream segments)." character_delimited: "Byte frames which are delimited by a chosen character." - length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." - newline_delimited: "Byte frames which are delimited by a newline character." + chunked_gelf: """ + Byte frames which are chunked GELF messages. + + [chunked_gelf]: https://go2docs.graylog.org/current/getting_in_log_data/gelf.html + """ + length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." + newline_delimited: "Byte frames which are delimited by a newline character." octet_counting: """ Byte frames according to the [octet counting][octet_counting] format. diff --git a/website/cue/reference/components/sources/base/socket.cue b/website/cue/reference/components/sources/base/socket.cue index 55cbd60be49d7..279bf33ded75a 100644 --- a/website/cue/reference/components/sources/base/socket.cue +++ b/website/cue/reference/components/sources/base/socket.cue @@ -276,6 +276,31 @@ base: components: sources: socket: configuration: { } } } + chunked_gelf: { + description: "Options for the chunked gelf decoder." + relevant_when: "method = \"chunked_gelf\"" + required: false + type: object: options: { + pending_messages_limit: { + description: """ + The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + The default value is 1000. + """ + required: false + type: uint: default: 1000 + } + timeout_millis: { + description: """ + The timeout in milliseconds for a message to be fully received. If the timeout is reached, the + decoder will drop all the received chunks of the uncomplete message and start over. + The default value is 5 seconds. + """ + required: false + type: uint: default: 5000 + } + } + } length_delimited: { description: "Options for the length delimited decoder." relevant_when: "method = \"length_delimited\"" @@ -309,8 +334,13 @@ base: components: sources: socket: configuration: { type: string: enum: { bytes: "Byte frames are passed through as-is according to the underlying I/O boundaries (for example, split between messages or stream segments)." character_delimited: "Byte frames which are delimited by a chosen character." - length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." - newline_delimited: "Byte frames which are delimited by a newline character." + chunked_gelf: """ + Byte frames which are chunked GELF messages. + + [chunked_gelf]: https://go2docs.graylog.org/current/getting_in_log_data/gelf.html + """ + length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." + newline_delimited: "Byte frames which are delimited by a newline character." octet_counting: """ Byte frames according to the [octet counting][octet_counting] format. diff --git a/website/cue/reference/components/sources/base/stdin.cue b/website/cue/reference/components/sources/base/stdin.cue index 7e5fdefe6c44e..1736f8b633361 100644 --- a/website/cue/reference/components/sources/base/stdin.cue +++ b/website/cue/reference/components/sources/base/stdin.cue @@ -259,6 +259,31 @@ base: components: sources: stdin: configuration: { } } } + chunked_gelf: { + description: "Options for the chunked gelf decoder." + relevant_when: "method = \"chunked_gelf\"" + required: false + type: object: options: { + pending_messages_limit: { + description: """ + The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + The default value is 1000. + """ + required: false + type: uint: default: 1000 + } + timeout_millis: { + description: """ + The timeout in milliseconds for a message to be fully received. If the timeout is reached, the + decoder will drop all the received chunks of the uncomplete message and start over. + The default value is 5 seconds. + """ + required: false + type: uint: default: 5000 + } + } + } length_delimited: { description: "Options for the length delimited decoder." relevant_when: "method = \"length_delimited\"" @@ -292,8 +317,13 @@ base: components: sources: stdin: configuration: { type: string: enum: { bytes: "Byte frames are passed through as-is according to the underlying I/O boundaries (for example, split between messages or stream segments)." character_delimited: "Byte frames which are delimited by a chosen character." - length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." - newline_delimited: "Byte frames which are delimited by a newline character." + chunked_gelf: """ + Byte frames which are chunked GELF messages. + + [chunked_gelf]: https://go2docs.graylog.org/current/getting_in_log_data/gelf.html + """ + length_delimited: "Byte frames which are prefixed by an unsigned big-endian 32-bit integer indicating the length." + newline_delimited: "Byte frames which are delimited by a newline character." octet_counting: """ Byte frames according to the [octet counting][octet_counting] format. From 6f8a214ec295e6b173ae72b7042f6ca1b9b9c67b Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Tue, 16 Jul 2024 17:41:12 +0200 Subject: [PATCH 25/60] fix: clippy lints --- lib/codecs/src/decoding/framing/chunked_gelf.rs | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index fa42989fc0a0f..101280a06d56a 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -14,8 +14,6 @@ use vector_config::configurable_component; const GELF_MAGIC: [u8; 2] = [0x1e, 0x0f]; const GELF_MAX_TOTAL_CHUNKS: u8 = 128; -const DEFAULT_CHUNKS: [Bytes; GELF_MAX_TOTAL_CHUNKS as usize] = - [const { Bytes::new() }; GELF_MAX_TOTAL_CHUNKS as usize]; const DEFAULT_TIMEOUT_MILLIS: u64 = 5000; const DEFAULT_PENDING_MESSAGES_LIMIT: usize = 1000; @@ -90,12 +88,16 @@ impl MessageState { pub const fn new(total_chunks: u8, timeout_task: JoinHandle<()>) -> Self { Self { total_chunks, - chunks: DEFAULT_CHUNKS, + chunks: Self::default_chunks(), chunks_bitmap: 0, timeout_task, } } + pub const fn default_chunks() -> [Bytes; GELF_MAX_TOTAL_CHUNKS as usize] { + [const { Bytes::new() }; GELF_MAX_TOTAL_CHUNKS as usize] + } + pub fn is_chunk_present(&self, sequence_number: u8) -> bool { let chunk_bitmap_id = 1 << sequence_number; self.chunks_bitmap & chunk_bitmap_id != 0 @@ -220,11 +222,11 @@ impl ChunkedGelfDecoder { // We need to spawn a task that will clear the message state after a certain time // otherwise we will have a memory leak due to messages that never complete let state = Arc::clone(&self.state); - let timeout = self.timeout.clone(); + let timeout = self.timeout; let timeout_handle = tokio::spawn(async move { tokio::time::sleep(timeout).await; let mut state_lock = state.lock().unwrap(); - if let Some(_) = state_lock.remove(&message_id) { + if state_lock.remove(&message_id).is_some() { let message = format!("Message was not fully received within the timeout window of {}ms. Discarding it.", timeout.as_millis()); warn!( @@ -275,10 +277,9 @@ impl ChunkedGelfDecoder { let magic = src.get(0..2); if magic.is_some_and(|magic| magic == GELF_MAGIC) { src.advance(2); - let frame = self.decode_chunk(src); - return frame; + self.decode_chunk(src) } else { - return Ok(Some(src)); + Ok(Some(src)) } } } From d5b753f40ec57fc69d04b1aec9df39d677aca596 Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Wed, 17 Jul 2024 09:57:13 +0200 Subject: [PATCH 26/60] docs: Fix documentation grammar --- lib/codecs/src/decoding/framing/chunked_gelf.rs | 8 ++++---- website/cue/reference/components/sources/base/amqp.cue | 8 ++++---- .../components/sources/base/aws_kinesis_firehose.cue | 8 ++++---- website/cue/reference/components/sources/base/aws_s3.cue | 8 ++++---- website/cue/reference/components/sources/base/aws_sqs.cue | 8 ++++---- .../reference/components/sources/base/datadog_agent.cue | 8 ++++---- .../cue/reference/components/sources/base/demo_logs.cue | 8 ++++---- website/cue/reference/components/sources/base/exec.cue | 8 ++++---- .../reference/components/sources/base/file_descriptor.cue | 8 ++++---- .../cue/reference/components/sources/base/gcp_pubsub.cue | 8 ++++---- .../cue/reference/components/sources/base/heroku_logs.cue | 8 ++++---- website/cue/reference/components/sources/base/http.cue | 8 ++++---- .../cue/reference/components/sources/base/http_client.cue | 8 ++++---- .../cue/reference/components/sources/base/http_server.cue | 8 ++++---- website/cue/reference/components/sources/base/kafka.cue | 8 ++++---- website/cue/reference/components/sources/base/nats.cue | 8 ++++---- website/cue/reference/components/sources/base/pulsar.cue | 8 ++++---- website/cue/reference/components/sources/base/redis.cue | 8 ++++---- website/cue/reference/components/sources/base/socket.cue | 8 ++++---- website/cue/reference/components/sources/base/stdin.cue | 8 ++++---- 20 files changed, 80 insertions(+), 80 deletions(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index 101280a06d56a..b1cb2328dca73 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -29,7 +29,7 @@ const fn default_pending_messages_limit() -> usize { #[configurable_component] #[derive(Debug, Clone, Default, PartialEq, Eq)] pub struct ChunkedGelfDecoderConfig { - /// Options for the chunked gelf decoder. + /// Options for the chunked GELF decoder. #[serde(default, skip_serializing_if = "vector_core::serde::is_default")] pub chunked_gelf: ChunkedGelfDecoderOptions, } @@ -48,8 +48,8 @@ impl ChunkedGelfDecoderConfig { #[configurable_component] #[derive(Clone, Debug, Derivative, PartialEq, Eq)] pub struct ChunkedGelfDecoderOptions { - /// The timeout in milliseconds for a message to be fully received. If the timeout is reached, the - /// decoder will drop all the received chunks of the uncomplete message and start over. + /// The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + /// decoder drops all the received chunks of the incomplete message and starts over. /// The default value is 5 seconds. #[serde( default = "default_timeout_millis", @@ -57,7 +57,7 @@ pub struct ChunkedGelfDecoderOptions { )] pub timeout_millis: u64, - /// The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + /// The maximum number of pending incomplete messages. If this limit is reached, the decoder starts /// dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. /// The default value is 1000. #[serde( diff --git a/website/cue/reference/components/sources/base/amqp.cue b/website/cue/reference/components/sources/base/amqp.cue index ca655a51068c9..1bdd0f5b745f0 100644 --- a/website/cue/reference/components/sources/base/amqp.cue +++ b/website/cue/reference/components/sources/base/amqp.cue @@ -310,13 +310,13 @@ base: components: sources: amqp: configuration: { } } chunked_gelf: { - description: "Options for the chunked gelf decoder." + description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { pending_messages_limit: { description: """ - The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. The default value is 1000. """ @@ -325,8 +325,8 @@ base: components: sources: amqp: configuration: { } timeout_millis: { description: """ - The timeout in milliseconds for a message to be fully received. If the timeout is reached, the - decoder will drop all the received chunks of the uncomplete message and start over. + The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + decoder drops all the received chunks of the incomplete message and starts over. The default value is 5 seconds. """ required: false diff --git a/website/cue/reference/components/sources/base/aws_kinesis_firehose.cue b/website/cue/reference/components/sources/base/aws_kinesis_firehose.cue index eff1f5f71c40c..1ab11509b9c62 100644 --- a/website/cue/reference/components/sources/base/aws_kinesis_firehose.cue +++ b/website/cue/reference/components/sources/base/aws_kinesis_firehose.cue @@ -308,13 +308,13 @@ base: components: sources: aws_kinesis_firehose: configuration: { } } chunked_gelf: { - description: "Options for the chunked gelf decoder." + description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { pending_messages_limit: { description: """ - The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. The default value is 1000. """ @@ -323,8 +323,8 @@ base: components: sources: aws_kinesis_firehose: configuration: { } timeout_millis: { description: """ - The timeout in milliseconds for a message to be fully received. If the timeout is reached, the - decoder will drop all the received chunks of the uncomplete message and start over. + The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + decoder drops all the received chunks of the incomplete message and starts over. The default value is 5 seconds. """ required: false diff --git a/website/cue/reference/components/sources/base/aws_s3.cue b/website/cue/reference/components/sources/base/aws_s3.cue index 5012e5099e0eb..19dceccf490e4 100644 --- a/website/cue/reference/components/sources/base/aws_s3.cue +++ b/website/cue/reference/components/sources/base/aws_s3.cue @@ -408,13 +408,13 @@ base: components: sources: aws_s3: configuration: { } } chunked_gelf: { - description: "Options for the chunked gelf decoder." + description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { pending_messages_limit: { description: """ - The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. The default value is 1000. """ @@ -423,8 +423,8 @@ base: components: sources: aws_s3: configuration: { } timeout_millis: { description: """ - The timeout in milliseconds for a message to be fully received. If the timeout is reached, the - decoder will drop all the received chunks of the uncomplete message and start over. + The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + decoder drops all the received chunks of the incomplete message and starts over. The default value is 5 seconds. """ required: false diff --git a/website/cue/reference/components/sources/base/aws_sqs.cue b/website/cue/reference/components/sources/base/aws_sqs.cue index d7792abd9cc3a..5691b9dd23f93 100644 --- a/website/cue/reference/components/sources/base/aws_sqs.cue +++ b/website/cue/reference/components/sources/base/aws_sqs.cue @@ -412,13 +412,13 @@ base: components: sources: aws_sqs: configuration: { } } chunked_gelf: { - description: "Options for the chunked gelf decoder." + description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { pending_messages_limit: { description: """ - The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. The default value is 1000. """ @@ -427,8 +427,8 @@ base: components: sources: aws_sqs: configuration: { } timeout_millis: { description: """ - The timeout in milliseconds for a message to be fully received. If the timeout is reached, the - decoder will drop all the received chunks of the uncomplete message and start over. + The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + decoder drops all the received chunks of the incomplete message and starts over. The default value is 5 seconds. """ required: false diff --git a/website/cue/reference/components/sources/base/datadog_agent.cue b/website/cue/reference/components/sources/base/datadog_agent.cue index e154d3d82b632..157d5476a008e 100644 --- a/website/cue/reference/components/sources/base/datadog_agent.cue +++ b/website/cue/reference/components/sources/base/datadog_agent.cue @@ -305,13 +305,13 @@ base: components: sources: datadog_agent: configuration: { } } chunked_gelf: { - description: "Options for the chunked gelf decoder." + description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { pending_messages_limit: { description: """ - The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. The default value is 1000. """ @@ -320,8 +320,8 @@ base: components: sources: datadog_agent: configuration: { } timeout_millis: { description: """ - The timeout in milliseconds for a message to be fully received. If the timeout is reached, the - decoder will drop all the received chunks of the uncomplete message and start over. + The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + decoder drops all the received chunks of the incomplete message and starts over. The default value is 5 seconds. """ required: false diff --git a/website/cue/reference/components/sources/base/demo_logs.cue b/website/cue/reference/components/sources/base/demo_logs.cue index bc7b938508221..28610fd26af22 100644 --- a/website/cue/reference/components/sources/base/demo_logs.cue +++ b/website/cue/reference/components/sources/base/demo_logs.cue @@ -301,13 +301,13 @@ base: components: sources: demo_logs: configuration: { } } chunked_gelf: { - description: "Options for the chunked gelf decoder." + description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { pending_messages_limit: { description: """ - The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. The default value is 1000. """ @@ -316,8 +316,8 @@ base: components: sources: demo_logs: configuration: { } timeout_millis: { description: """ - The timeout in milliseconds for a message to be fully received. If the timeout is reached, the - decoder will drop all the received chunks of the uncomplete message and start over. + The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + decoder drops all the received chunks of the incomplete message and starts over. The default value is 5 seconds. """ required: false diff --git a/website/cue/reference/components/sources/base/exec.cue b/website/cue/reference/components/sources/base/exec.cue index ed59eb864edf8..867bf5e55ff82 100644 --- a/website/cue/reference/components/sources/base/exec.cue +++ b/website/cue/reference/components/sources/base/exec.cue @@ -289,13 +289,13 @@ base: components: sources: exec: configuration: { } } chunked_gelf: { - description: "Options for the chunked gelf decoder." + description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { pending_messages_limit: { description: """ - The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. The default value is 1000. """ @@ -304,8 +304,8 @@ base: components: sources: exec: configuration: { } timeout_millis: { description: """ - The timeout in milliseconds for a message to be fully received. If the timeout is reached, the - decoder will drop all the received chunks of the uncomplete message and start over. + The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + decoder drops all the received chunks of the incomplete message and starts over. The default value is 5 seconds. """ required: false diff --git a/website/cue/reference/components/sources/base/file_descriptor.cue b/website/cue/reference/components/sources/base/file_descriptor.cue index 0ed97d8c9c61c..cc41d35581b16 100644 --- a/website/cue/reference/components/sources/base/file_descriptor.cue +++ b/website/cue/reference/components/sources/base/file_descriptor.cue @@ -267,13 +267,13 @@ base: components: sources: file_descriptor: configuration: { } } chunked_gelf: { - description: "Options for the chunked gelf decoder." + description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { pending_messages_limit: { description: """ - The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. The default value is 1000. """ @@ -282,8 +282,8 @@ base: components: sources: file_descriptor: configuration: { } timeout_millis: { description: """ - The timeout in milliseconds for a message to be fully received. If the timeout is reached, the - decoder will drop all the received chunks of the uncomplete message and start over. + The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + decoder drops all the received chunks of the incomplete message and starts over. The default value is 5 seconds. """ required: false diff --git a/website/cue/reference/components/sources/base/gcp_pubsub.cue b/website/cue/reference/components/sources/base/gcp_pubsub.cue index 248057af5e2e7..05816470d4fe7 100644 --- a/website/cue/reference/components/sources/base/gcp_pubsub.cue +++ b/website/cue/reference/components/sources/base/gcp_pubsub.cue @@ -344,13 +344,13 @@ base: components: sources: gcp_pubsub: configuration: { } } chunked_gelf: { - description: "Options for the chunked gelf decoder." + description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { pending_messages_limit: { description: """ - The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. The default value is 1000. """ @@ -359,8 +359,8 @@ base: components: sources: gcp_pubsub: configuration: { } timeout_millis: { description: """ - The timeout in milliseconds for a message to be fully received. If the timeout is reached, the - decoder will drop all the received chunks of the uncomplete message and start over. + The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + decoder drops all the received chunks of the incomplete message and starts over. The default value is 5 seconds. """ required: false diff --git a/website/cue/reference/components/sources/base/heroku_logs.cue b/website/cue/reference/components/sources/base/heroku_logs.cue index 54b9e5eea48b6..d465f64a21d0b 100644 --- a/website/cue/reference/components/sources/base/heroku_logs.cue +++ b/website/cue/reference/components/sources/base/heroku_logs.cue @@ -302,13 +302,13 @@ base: components: sources: heroku_logs: configuration: { } } chunked_gelf: { - description: "Options for the chunked gelf decoder." + description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { pending_messages_limit: { description: """ - The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. The default value is 1000. """ @@ -317,8 +317,8 @@ base: components: sources: heroku_logs: configuration: { } timeout_millis: { description: """ - The timeout in milliseconds for a message to be fully received. If the timeout is reached, the - decoder will drop all the received chunks of the uncomplete message and start over. + The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + decoder drops all the received chunks of the incomplete message and starts over. The default value is 5 seconds. """ required: false diff --git a/website/cue/reference/components/sources/base/http.cue b/website/cue/reference/components/sources/base/http.cue index 182f492cee352..d513aa75d4e1c 100644 --- a/website/cue/reference/components/sources/base/http.cue +++ b/website/cue/reference/components/sources/base/http.cue @@ -317,13 +317,13 @@ base: components: sources: http: configuration: { } } chunked_gelf: { - description: "Options for the chunked gelf decoder." + description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { pending_messages_limit: { description: """ - The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. The default value is 1000. """ @@ -332,8 +332,8 @@ base: components: sources: http: configuration: { } timeout_millis: { description: """ - The timeout in milliseconds for a message to be fully received. If the timeout is reached, the - decoder will drop all the received chunks of the uncomplete message and start over. + The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + decoder drops all the received chunks of the incomplete message and starts over. The default value is 5 seconds. """ required: false diff --git a/website/cue/reference/components/sources/base/http_client.cue b/website/cue/reference/components/sources/base/http_client.cue index 8aa20dacd8f51..cb4752807db84 100644 --- a/website/cue/reference/components/sources/base/http_client.cue +++ b/website/cue/reference/components/sources/base/http_client.cue @@ -305,13 +305,13 @@ base: components: sources: http_client: configuration: { } } chunked_gelf: { - description: "Options for the chunked gelf decoder." + description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { pending_messages_limit: { description: """ - The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. The default value is 1000. """ @@ -320,8 +320,8 @@ base: components: sources: http_client: configuration: { } timeout_millis: { description: """ - The timeout in milliseconds for a message to be fully received. If the timeout is reached, the - decoder will drop all the received chunks of the uncomplete message and start over. + The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + decoder drops all the received chunks of the incomplete message and starts over. The default value is 5 seconds. """ required: false diff --git a/website/cue/reference/components/sources/base/http_server.cue b/website/cue/reference/components/sources/base/http_server.cue index 7cb2df523676c..bb55504c42cc0 100644 --- a/website/cue/reference/components/sources/base/http_server.cue +++ b/website/cue/reference/components/sources/base/http_server.cue @@ -317,13 +317,13 @@ base: components: sources: http_server: configuration: { } } chunked_gelf: { - description: "Options for the chunked gelf decoder." + description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { pending_messages_limit: { description: """ - The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. The default value is 1000. """ @@ -332,8 +332,8 @@ base: components: sources: http_server: configuration: { } timeout_millis: { description: """ - The timeout in milliseconds for a message to be fully received. If the timeout is reached, the - decoder will drop all the received chunks of the uncomplete message and start over. + The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + decoder drops all the received chunks of the incomplete message and starts over. The default value is 5 seconds. """ required: false diff --git a/website/cue/reference/components/sources/base/kafka.cue b/website/cue/reference/components/sources/base/kafka.cue index 89b55000e4f3b..909078dc4b597 100644 --- a/website/cue/reference/components/sources/base/kafka.cue +++ b/website/cue/reference/components/sources/base/kafka.cue @@ -338,13 +338,13 @@ base: components: sources: kafka: configuration: { } } chunked_gelf: { - description: "Options for the chunked gelf decoder." + description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { pending_messages_limit: { description: """ - The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. The default value is 1000. """ @@ -353,8 +353,8 @@ base: components: sources: kafka: configuration: { } timeout_millis: { description: """ - The timeout in milliseconds for a message to be fully received. If the timeout is reached, the - decoder will drop all the received chunks of the uncomplete message and start over. + The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + decoder drops all the received chunks of the incomplete message and starts over. The default value is 5 seconds. """ required: false diff --git a/website/cue/reference/components/sources/base/nats.cue b/website/cue/reference/components/sources/base/nats.cue index f44aa2ef83671..7fca13ac650c4 100644 --- a/website/cue/reference/components/sources/base/nats.cue +++ b/website/cue/reference/components/sources/base/nats.cue @@ -357,13 +357,13 @@ base: components: sources: nats: configuration: { } } chunked_gelf: { - description: "Options for the chunked gelf decoder." + description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { pending_messages_limit: { description: """ - The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. The default value is 1000. """ @@ -372,8 +372,8 @@ base: components: sources: nats: configuration: { } timeout_millis: { description: """ - The timeout in milliseconds for a message to be fully received. If the timeout is reached, the - decoder will drop all the received chunks of the uncomplete message and start over. + The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + decoder drops all the received chunks of the incomplete message and starts over. The default value is 5 seconds. """ required: false diff --git a/website/cue/reference/components/sources/base/pulsar.cue b/website/cue/reference/components/sources/base/pulsar.cue index e3c775a1018cf..b9085c291ae2a 100644 --- a/website/cue/reference/components/sources/base/pulsar.cue +++ b/website/cue/reference/components/sources/base/pulsar.cue @@ -368,13 +368,13 @@ base: components: sources: pulsar: configuration: { } } chunked_gelf: { - description: "Options for the chunked gelf decoder." + description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { pending_messages_limit: { description: """ - The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. The default value is 1000. """ @@ -383,8 +383,8 @@ base: components: sources: pulsar: configuration: { } timeout_millis: { description: """ - The timeout in milliseconds for a message to be fully received. If the timeout is reached, the - decoder will drop all the received chunks of the uncomplete message and start over. + The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + decoder drops all the received chunks of the incomplete message and starts over. The default value is 5 seconds. """ required: false diff --git a/website/cue/reference/components/sources/base/redis.cue b/website/cue/reference/components/sources/base/redis.cue index 534fd827d4579..c07dbc628de10 100644 --- a/website/cue/reference/components/sources/base/redis.cue +++ b/website/cue/reference/components/sources/base/redis.cue @@ -275,13 +275,13 @@ base: components: sources: redis: configuration: { } } chunked_gelf: { - description: "Options for the chunked gelf decoder." + description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { pending_messages_limit: { description: """ - The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. The default value is 1000. """ @@ -290,8 +290,8 @@ base: components: sources: redis: configuration: { } timeout_millis: { description: """ - The timeout in milliseconds for a message to be fully received. If the timeout is reached, the - decoder will drop all the received chunks of the uncomplete message and start over. + The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + decoder drops all the received chunks of the incomplete message and starts over. The default value is 5 seconds. """ required: false diff --git a/website/cue/reference/components/sources/base/socket.cue b/website/cue/reference/components/sources/base/socket.cue index 279bf33ded75a..1183bc68f19eb 100644 --- a/website/cue/reference/components/sources/base/socket.cue +++ b/website/cue/reference/components/sources/base/socket.cue @@ -277,13 +277,13 @@ base: components: sources: socket: configuration: { } } chunked_gelf: { - description: "Options for the chunked gelf decoder." + description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { pending_messages_limit: { description: """ - The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. The default value is 1000. """ @@ -292,8 +292,8 @@ base: components: sources: socket: configuration: { } timeout_millis: { description: """ - The timeout in milliseconds for a message to be fully received. If the timeout is reached, the - decoder will drop all the received chunks of the uncomplete message and start over. + The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + decoder drops all the received chunks of the incomplete message and starts over. The default value is 5 seconds. """ required: false diff --git a/website/cue/reference/components/sources/base/stdin.cue b/website/cue/reference/components/sources/base/stdin.cue index 1736f8b633361..10de0273f8195 100644 --- a/website/cue/reference/components/sources/base/stdin.cue +++ b/website/cue/reference/components/sources/base/stdin.cue @@ -260,13 +260,13 @@ base: components: sources: stdin: configuration: { } } chunked_gelf: { - description: "Options for the chunked gelf decoder." + description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { pending_messages_limit: { description: """ - The maximum number of pending uncomplete messages. If this limit is reached, the decoder will start + The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. The default value is 1000. """ @@ -275,8 +275,8 @@ base: components: sources: stdin: configuration: { } timeout_millis: { description: """ - The timeout in milliseconds for a message to be fully received. If the timeout is reached, the - decoder will drop all the received chunks of the uncomplete message and start over. + The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + decoder drops all the received chunks of the incomplete message and starts over. The default value is 5 seconds. """ required: false From 1b7bfe1e22adaa006582a6b427062b6e8f2a9df5 Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Mon, 22 Jul 2024 10:23:04 +0200 Subject: [PATCH 27/60] fix: clippy lints --- src/sources/socket/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/sources/socket/mod.rs b/src/sources/socket/mod.rs index c06fb141440fe..9a62733a7b7de 100644 --- a/src/sources/socket/mod.rs +++ b/src/sources/socket/mod.rs @@ -1433,7 +1433,7 @@ mod test { ////////////// UNIX DATAGRAM TESTS ////////////// #[cfg(unix)] async fn send_lines_unix_datagram(path: PathBuf, lines: &[&str]) { - let packets = lines.into_iter().map(|line| Bytes::from(line.to_string())); + let packets = lines.iter().map(|line| Bytes::from(line.to_string())); send_packets_unix_datagram(path, packets).await; } From 770f1e33bef5d33bd54b664dd432f639b2e5eb3d Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Mon, 22 Jul 2024 10:29:20 +0200 Subject: [PATCH 28/60] fix: Add 'unchunked' to spellcheck list --- .github/actions/spelling/allow.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/actions/spelling/allow.txt b/.github/actions/spelling/allow.txt index b46bd40407b1b..640a3d764d911 100644 --- a/.github/actions/spelling/allow.txt +++ b/.github/actions/spelling/allow.txt @@ -455,6 +455,7 @@ timespan timestamped tzdata ubuntu +unchunked upstreaminfo useragents usergroups From 7a03832a3d6631446567ed7a74c5cefcb54f8c52 Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Sat, 27 Jul 2024 13:13:30 +0200 Subject: [PATCH 29/60] Update lib/codecs/src/decoding/framing/chunked_gelf.rs Co-authored-by: Jesse Szwedko --- lib/codecs/src/decoding/framing/chunked_gelf.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index b1cb2328dca73..f9395fe79858a 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -205,7 +205,7 @@ impl ChunkedGelfDecoder { return Ok(None); } - let mut state_lock = self.state.lock().unwrap(); + let mut state_lock = self.state.lock().expect("poisoned lock"); if state_lock.len() >= self.pending_messages_limit { warn!( From 8c66ea2d819da9e649b710aa1ffb68b7d861314d Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Sun, 28 Jul 2024 19:09:51 +0200 Subject: [PATCH 30/60] docs(chunked gelf): Remove default value for chunked gelf options in the description --- website/cue/reference/components/sources/base/amqp.cue | 2 -- .../reference/components/sources/base/aws_kinesis_firehose.cue | 2 -- website/cue/reference/components/sources/base/aws_s3.cue | 2 -- website/cue/reference/components/sources/base/aws_sqs.cue | 2 -- website/cue/reference/components/sources/base/datadog_agent.cue | 2 -- website/cue/reference/components/sources/base/demo_logs.cue | 2 -- website/cue/reference/components/sources/base/exec.cue | 2 -- .../cue/reference/components/sources/base/file_descriptor.cue | 2 -- website/cue/reference/components/sources/base/gcp_pubsub.cue | 2 -- website/cue/reference/components/sources/base/heroku_logs.cue | 2 -- website/cue/reference/components/sources/base/http.cue | 2 -- website/cue/reference/components/sources/base/http_client.cue | 2 -- website/cue/reference/components/sources/base/http_server.cue | 2 -- website/cue/reference/components/sources/base/kafka.cue | 2 -- website/cue/reference/components/sources/base/nats.cue | 2 -- website/cue/reference/components/sources/base/pulsar.cue | 2 -- website/cue/reference/components/sources/base/redis.cue | 2 -- website/cue/reference/components/sources/base/socket.cue | 2 -- website/cue/reference/components/sources/base/stdin.cue | 2 -- 19 files changed, 38 deletions(-) diff --git a/website/cue/reference/components/sources/base/amqp.cue b/website/cue/reference/components/sources/base/amqp.cue index 7a2450b510f2a..1962308059509 100644 --- a/website/cue/reference/components/sources/base/amqp.cue +++ b/website/cue/reference/components/sources/base/amqp.cue @@ -339,7 +339,6 @@ base: components: sources: amqp: configuration: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. - The default value is 1000. """ required: false type: uint: default: 1000 @@ -348,7 +347,6 @@ base: components: sources: amqp: configuration: { description: """ The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. - The default value is 5 seconds. """ required: false type: uint: default: 5000 diff --git a/website/cue/reference/components/sources/base/aws_kinesis_firehose.cue b/website/cue/reference/components/sources/base/aws_kinesis_firehose.cue index 47761fc548b7d..55940af9226f4 100644 --- a/website/cue/reference/components/sources/base/aws_kinesis_firehose.cue +++ b/website/cue/reference/components/sources/base/aws_kinesis_firehose.cue @@ -337,7 +337,6 @@ base: components: sources: aws_kinesis_firehose: configuration: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. - The default value is 1000. """ required: false type: uint: default: 1000 @@ -346,7 +345,6 @@ base: components: sources: aws_kinesis_firehose: configuration: { description: """ The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. - The default value is 5 seconds. """ required: false type: uint: default: 5000 diff --git a/website/cue/reference/components/sources/base/aws_s3.cue b/website/cue/reference/components/sources/base/aws_s3.cue index 4883b9e5a8445..4a0c898d8cfa8 100644 --- a/website/cue/reference/components/sources/base/aws_s3.cue +++ b/website/cue/reference/components/sources/base/aws_s3.cue @@ -437,7 +437,6 @@ base: components: sources: aws_s3: configuration: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. - The default value is 1000. """ required: false type: uint: default: 1000 @@ -446,7 +445,6 @@ base: components: sources: aws_s3: configuration: { description: """ The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. - The default value is 5 seconds. """ required: false type: uint: default: 5000 diff --git a/website/cue/reference/components/sources/base/aws_sqs.cue b/website/cue/reference/components/sources/base/aws_sqs.cue index 124e2896ed5de..eea757080d0e5 100644 --- a/website/cue/reference/components/sources/base/aws_sqs.cue +++ b/website/cue/reference/components/sources/base/aws_sqs.cue @@ -441,7 +441,6 @@ base: components: sources: aws_sqs: configuration: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. - The default value is 1000. """ required: false type: uint: default: 1000 @@ -450,7 +449,6 @@ base: components: sources: aws_sqs: configuration: { description: """ The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. - The default value is 5 seconds. """ required: false type: uint: default: 5000 diff --git a/website/cue/reference/components/sources/base/datadog_agent.cue b/website/cue/reference/components/sources/base/datadog_agent.cue index aea3a74d51647..9b810572a2be1 100644 --- a/website/cue/reference/components/sources/base/datadog_agent.cue +++ b/website/cue/reference/components/sources/base/datadog_agent.cue @@ -334,7 +334,6 @@ base: components: sources: datadog_agent: configuration: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. - The default value is 1000. """ required: false type: uint: default: 1000 @@ -343,7 +342,6 @@ base: components: sources: datadog_agent: configuration: { description: """ The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. - The default value is 5 seconds. """ required: false type: uint: default: 5000 diff --git a/website/cue/reference/components/sources/base/demo_logs.cue b/website/cue/reference/components/sources/base/demo_logs.cue index 0b3d10433ddd3..e0ce2ec5d896c 100644 --- a/website/cue/reference/components/sources/base/demo_logs.cue +++ b/website/cue/reference/components/sources/base/demo_logs.cue @@ -330,7 +330,6 @@ base: components: sources: demo_logs: configuration: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. - The default value is 1000. """ required: false type: uint: default: 1000 @@ -339,7 +338,6 @@ base: components: sources: demo_logs: configuration: { description: """ The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. - The default value is 5 seconds. """ required: false type: uint: default: 5000 diff --git a/website/cue/reference/components/sources/base/exec.cue b/website/cue/reference/components/sources/base/exec.cue index e2caf373a436f..c3cb213c30bde 100644 --- a/website/cue/reference/components/sources/base/exec.cue +++ b/website/cue/reference/components/sources/base/exec.cue @@ -318,7 +318,6 @@ base: components: sources: exec: configuration: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. - The default value is 1000. """ required: false type: uint: default: 1000 @@ -327,7 +326,6 @@ base: components: sources: exec: configuration: { description: """ The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. - The default value is 5 seconds. """ required: false type: uint: default: 5000 diff --git a/website/cue/reference/components/sources/base/file_descriptor.cue b/website/cue/reference/components/sources/base/file_descriptor.cue index 1f6874759d73a..09733ec5f98c9 100644 --- a/website/cue/reference/components/sources/base/file_descriptor.cue +++ b/website/cue/reference/components/sources/base/file_descriptor.cue @@ -296,7 +296,6 @@ base: components: sources: file_descriptor: configuration: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. - The default value is 1000. """ required: false type: uint: default: 1000 @@ -305,7 +304,6 @@ base: components: sources: file_descriptor: configuration: { description: """ The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. - The default value is 5 seconds. """ required: false type: uint: default: 5000 diff --git a/website/cue/reference/components/sources/base/gcp_pubsub.cue b/website/cue/reference/components/sources/base/gcp_pubsub.cue index 35977bc47d076..033e9f5ca7967 100644 --- a/website/cue/reference/components/sources/base/gcp_pubsub.cue +++ b/website/cue/reference/components/sources/base/gcp_pubsub.cue @@ -373,7 +373,6 @@ base: components: sources: gcp_pubsub: configuration: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. - The default value is 1000. """ required: false type: uint: default: 1000 @@ -382,7 +381,6 @@ base: components: sources: gcp_pubsub: configuration: { description: """ The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. - The default value is 5 seconds. """ required: false type: uint: default: 5000 diff --git a/website/cue/reference/components/sources/base/heroku_logs.cue b/website/cue/reference/components/sources/base/heroku_logs.cue index 974df7619ce7f..473559ddcbed3 100644 --- a/website/cue/reference/components/sources/base/heroku_logs.cue +++ b/website/cue/reference/components/sources/base/heroku_logs.cue @@ -331,7 +331,6 @@ base: components: sources: heroku_logs: configuration: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. - The default value is 1000. """ required: false type: uint: default: 1000 @@ -340,7 +339,6 @@ base: components: sources: heroku_logs: configuration: { description: """ The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. - The default value is 5 seconds. """ required: false type: uint: default: 5000 diff --git a/website/cue/reference/components/sources/base/http.cue b/website/cue/reference/components/sources/base/http.cue index ca46105cc7a3d..cb2ea487f6cc8 100644 --- a/website/cue/reference/components/sources/base/http.cue +++ b/website/cue/reference/components/sources/base/http.cue @@ -346,7 +346,6 @@ base: components: sources: http: configuration: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. - The default value is 1000. """ required: false type: uint: default: 1000 @@ -355,7 +354,6 @@ base: components: sources: http: configuration: { description: """ The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. - The default value is 5 seconds. """ required: false type: uint: default: 5000 diff --git a/website/cue/reference/components/sources/base/http_client.cue b/website/cue/reference/components/sources/base/http_client.cue index dd5f6fa1ede9b..a3684a35f1b00 100644 --- a/website/cue/reference/components/sources/base/http_client.cue +++ b/website/cue/reference/components/sources/base/http_client.cue @@ -334,7 +334,6 @@ base: components: sources: http_client: configuration: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. - The default value is 1000. """ required: false type: uint: default: 1000 @@ -343,7 +342,6 @@ base: components: sources: http_client: configuration: { description: """ The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. - The default value is 5 seconds. """ required: false type: uint: default: 5000 diff --git a/website/cue/reference/components/sources/base/http_server.cue b/website/cue/reference/components/sources/base/http_server.cue index 56ea4afc7ef17..8ab69e9826a9d 100644 --- a/website/cue/reference/components/sources/base/http_server.cue +++ b/website/cue/reference/components/sources/base/http_server.cue @@ -346,7 +346,6 @@ base: components: sources: http_server: configuration: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. - The default value is 1000. """ required: false type: uint: default: 1000 @@ -355,7 +354,6 @@ base: components: sources: http_server: configuration: { description: """ The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. - The default value is 5 seconds. """ required: false type: uint: default: 5000 diff --git a/website/cue/reference/components/sources/base/kafka.cue b/website/cue/reference/components/sources/base/kafka.cue index 07cd78265d68e..059520799e6c3 100644 --- a/website/cue/reference/components/sources/base/kafka.cue +++ b/website/cue/reference/components/sources/base/kafka.cue @@ -367,7 +367,6 @@ base: components: sources: kafka: configuration: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. - The default value is 1000. """ required: false type: uint: default: 1000 @@ -376,7 +375,6 @@ base: components: sources: kafka: configuration: { description: """ The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. - The default value is 5 seconds. """ required: false type: uint: default: 5000 diff --git a/website/cue/reference/components/sources/base/nats.cue b/website/cue/reference/components/sources/base/nats.cue index 6769e4c82b283..957a1048f2aad 100644 --- a/website/cue/reference/components/sources/base/nats.cue +++ b/website/cue/reference/components/sources/base/nats.cue @@ -386,7 +386,6 @@ base: components: sources: nats: configuration: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. - The default value is 1000. """ required: false type: uint: default: 1000 @@ -395,7 +394,6 @@ base: components: sources: nats: configuration: { description: """ The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. - The default value is 5 seconds. """ required: false type: uint: default: 5000 diff --git a/website/cue/reference/components/sources/base/pulsar.cue b/website/cue/reference/components/sources/base/pulsar.cue index fdde2e9cb2b90..9d3faae882bbe 100644 --- a/website/cue/reference/components/sources/base/pulsar.cue +++ b/website/cue/reference/components/sources/base/pulsar.cue @@ -397,7 +397,6 @@ base: components: sources: pulsar: configuration: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. - The default value is 1000. """ required: false type: uint: default: 1000 @@ -406,7 +405,6 @@ base: components: sources: pulsar: configuration: { description: """ The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. - The default value is 5 seconds. """ required: false type: uint: default: 5000 diff --git a/website/cue/reference/components/sources/base/redis.cue b/website/cue/reference/components/sources/base/redis.cue index 1084a000bf7e6..440a020e2b019 100644 --- a/website/cue/reference/components/sources/base/redis.cue +++ b/website/cue/reference/components/sources/base/redis.cue @@ -304,7 +304,6 @@ base: components: sources: redis: configuration: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. - The default value is 1000. """ required: false type: uint: default: 1000 @@ -313,7 +312,6 @@ base: components: sources: redis: configuration: { description: """ The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. - The default value is 5 seconds. """ required: false type: uint: default: 5000 diff --git a/website/cue/reference/components/sources/base/socket.cue b/website/cue/reference/components/sources/base/socket.cue index 7db5eccc83ee7..dc601aac493bf 100644 --- a/website/cue/reference/components/sources/base/socket.cue +++ b/website/cue/reference/components/sources/base/socket.cue @@ -306,7 +306,6 @@ base: components: sources: socket: configuration: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. - The default value is 1000. """ required: false type: uint: default: 1000 @@ -315,7 +314,6 @@ base: components: sources: socket: configuration: { description: """ The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. - The default value is 5 seconds. """ required: false type: uint: default: 5000 diff --git a/website/cue/reference/components/sources/base/stdin.cue b/website/cue/reference/components/sources/base/stdin.cue index e573e98afddf6..5c72a42f2ec5c 100644 --- a/website/cue/reference/components/sources/base/stdin.cue +++ b/website/cue/reference/components/sources/base/stdin.cue @@ -289,7 +289,6 @@ base: components: sources: stdin: configuration: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. - The default value is 1000. """ required: false type: uint: default: 1000 @@ -298,7 +297,6 @@ base: components: sources: stdin: configuration: { description: """ The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. - The default value is 5 seconds. """ required: false type: uint: default: 5000 From 09fb7bc3d64c52d305d767482bb008414a83248e Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Sun, 28 Jul 2024 21:01:36 +0200 Subject: [PATCH 31/60] feat(decoding): Return concrete error types in chunked gelf --- .../src/decoding/framing/chunked_gelf.rs | 247 +++++++++++------- lib/codecs/src/decoding/framing/mod.rs | 32 ++- 2 files changed, 178 insertions(+), 101 deletions(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index f9395fe79858a..1757c20979ef9 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -1,8 +1,10 @@ -use crate::BytesDecoder; +use crate::{BytesDecoder, StreamDecodingError}; -use super::BoxedFramingError; +use super::{BoxedFramingError, FramingError}; use bytes::{Buf, Bytes, BytesMut}; use derivative::Derivative; +use snafu::{ensure, Snafu}; +use std::any::Any; use std::collections::HashMap; use std::sync::{Arc, Mutex}; use std::time::Duration; @@ -50,7 +52,6 @@ impl ChunkedGelfDecoderConfig { pub struct ChunkedGelfDecoderOptions { /// The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the /// decoder drops all the received chunks of the incomplete message and starts over. - /// The default value is 5 seconds. #[serde( default = "default_timeout_millis", skip_serializing_if = "vector_core::serde::is_default" @@ -59,7 +60,6 @@ pub struct ChunkedGelfDecoderOptions { /// The maximum number of pending incomplete messages. If this limit is reached, the decoder starts /// dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. - /// The default value is 1000. #[serde( default = "default_pending_messages_limit", skip_serializing_if = "vector_core::serde::is_default" @@ -128,6 +128,49 @@ impl MessageState { } } +#[derive(Debug, Snafu)] +pub enum ChunkedGelfDecoderError { + #[snafu(display("Invalid chunk header with less than 10 bytes: {header:?}"))] + InvalidChunkHeader { header: Bytes }, + #[snafu(display("Invalid total chunks value {total_chunks} for message with id {message_id} and sequence number {sequence_number}"))] + InvalidTotalChunks { + message_id: u64, + sequence_number: u8, + total_chunks: u8, + }, + #[snafu(display("Sequence number {sequence_number} is greater than the total chunks value {total_chunks} for message with id {message_id}"))] + InvalidSequenceNumber { + message_id: u64, + sequence_number: u8, + total_chunks: u8, + }, + #[snafu(display("Pending messages limit of {pending_messages_limit} reached while processing message with id {message_id} and sequence number {sequence_number}"))] + PendingMessagesLimitReached { + message_id: u64, + sequence_number: u8, + pending_messages_limit: usize, + }, + #[snafu(display("Received message with id {message_id} and sequence number {sequence_number} has different total chunks values: original total chunks value {original_total_chunks}, received total chunks value {received_total_chunks}"))] + TotalChunksMismatch { + message_id: u64, + sequence_number: u8, + original_total_chunks: u8, + received_total_chunks: u8, + }, +} + +impl StreamDecodingError for ChunkedGelfDecoderError { + fn can_continue(&self) -> bool { + true + } +} + +impl FramingError for ChunkedGelfDecoderError { + fn as_any(&self) -> &dyn Any { + self as &dyn Any + } +} + /// A codec for handling GELF messages that may be chunked. The implementation is based on [Graylog's GELF documentation](https://go2docs.graylog.org/5-0/getting_in_log_data/gelf.html#GELFviaUDP) /// and [Graylog's go-gelf library](https://github.com/Graylog2/go-gelf/blob/v1/gelf/reader.go). #[derive(Debug, Clone)] @@ -155,7 +198,10 @@ impl ChunkedGelfDecoder { } /// Decode a GELF chunk - pub fn decode_chunk(&mut self, mut chunk: Bytes) -> Result, BoxedFramingError> { + pub fn decode_chunk( + &mut self, + mut chunk: Bytes, + ) -> Result, ChunkedGelfDecoderError> { // Encoding scheme: // // +------------+-----------------+--------------+----------------------+ @@ -170,53 +216,53 @@ impl ChunkedGelfDecoder { // and return the complete payload. // We need 10 bytes to read the message id, sequence number and total chunks - if chunk.remaining() < 10 { - let src_display = format!("{chunk:?}"); - warn!(message = "Received malformed chunk headers (message ID, sequence number and total chunks) with less than 10 bytes. Ignoring it.", - src = src_display, - remaining = chunk.remaining(), - internal_log_rate_limit = true - ); - return Ok(None); - } + // if chunk.remaining() < 10 { + // let src_display = format!("{chunk:?}"); + // warn!(message = "Received malformed chunk headers (message ID, sequence number and total chunks) with less than 10 bytes. Ignoring it.", + // src = src_display, + // remaining = chunk.remaining(), + // internal_log_rate_limit = true + // ); + // return Ok(None); + // } + + ensure!( + chunk.remaining() >= 10, + InvalidChunkHeaderSnafu { header: chunk } + ); + let message_id = chunk.get_u64(); let sequence_number = chunk.get_u8(); let total_chunks = chunk.get_u8(); - if total_chunks == 0 || total_chunks > GELF_MAX_TOTAL_CHUNKS { - warn!( - message = "Received a chunk with an invalid total chunks value. Ignoring it.", - message_id = message_id, - sequence_number = sequence_number, - total_chunks = total_chunks, - internal_log_rate_limit = true - ); - return Ok(None); - } + ensure!( + total_chunks > 0 && total_chunks <= GELF_MAX_TOTAL_CHUNKS, + InvalidTotalChunksSnafu { + message_id, + sequence_number, + total_chunks + } + ); - if sequence_number >= total_chunks { - warn!( - message = "Received a chunk with a sequence number greater than total chunks. Ignoring it.", - message_id = message_id, - sequence_number = sequence_number, - total_chunks = total_chunks, - internal_log_rate_limit = true - ); - return Ok(None); - } + ensure!( + sequence_number < total_chunks, + InvalidSequenceNumberSnafu { + message_id, + sequence_number, + total_chunks + } + ); let mut state_lock = self.state.lock().expect("poisoned lock"); - if state_lock.len() >= self.pending_messages_limit { - warn!( - message = "Received a chunk but reached the pending messages limit. Ignoring it.", - message_id = message_id, - sequence_number = sequence_number, - pending_messages_limit = self.pending_messages_limit, - internal_log_rate_limit = true - ); - return Ok(None); - } + ensure!( + state_lock.len() < self.pending_messages_limit, + PendingMessagesLimitReachedSnafu { + message_id, + sequence_number, + pending_messages_limit: self.pending_messages_limit + } + ); let message_state = state_lock.entry(message_id).or_insert_with(|| { // We need to spawn a task that will clear the message state after a certain time @@ -240,15 +286,15 @@ impl ChunkedGelfDecoder { MessageState::new(total_chunks, timeout_handle) }); - if message_state.total_chunks != total_chunks { - warn!(message_id = "Received a chunk with a different total chunks than the original. Ignoring it.", - message_id = message_id, - original_total_chunks = message_state.total_chunks, - received_total_chunks = total_chunks, - internal_log_rate_limit = true - ); - return Ok(None); - } + ensure!( + message_state.total_chunks == total_chunks, + TotalChunksMismatchSnafu { + message_id, + sequence_number, + original_total_chunks: message_state.total_chunks, + received_total_chunks: total_chunks + } + ); if message_state.is_chunk_present(sequence_number) { info!( @@ -273,7 +319,10 @@ impl ChunkedGelfDecoder { /// Decode a GELF message that may be chunked or not. The source bytes are expected to be /// datagram-based (or message-based), so it must not contain multiple GELF messages /// delimited by '\0', such as it would be in a stream-based protocol. - pub fn decode_message(&mut self, mut src: Bytes) -> Result, BoxedFramingError> { + pub fn decode_message( + &mut self, + mut src: Bytes, + ) -> Result, ChunkedGelfDecoderError> { let magic = src.get(0..2); if magic.is_some_and(|magic| magic == GELF_MAGIC) { src.advance(2); @@ -300,29 +349,28 @@ impl Decoder for ChunkedGelfDecoder { return Ok(None); } - // TODO: add a PR comment here stating that this will never call the decode_message since - // the bytes decoder will always return a Ok(None) in this method, but leaving this - // here for consistency. Would be better to add a unreachable/panic here if the inner decoder returns - // the Some variant? - self.bytes_decoder + Ok(self + .bytes_decoder .decode(src)? .and_then(|frame| self.decode_message(frame).transpose()) - .transpose() + .transpose()?) } fn decode_eof(&mut self, buf: &mut BytesMut) -> Result, Self::Error> { if buf.is_empty() { return Ok(None); } - self.bytes_decoder + Ok(self + .bytes_decoder .decode_eof(buf)? .and_then(|frame| self.decode_message(frame).transpose()) - .transpose() + .transpose()?) } } #[cfg(test)] mod tests { + use super::*; use bytes::{BufMut, BytesMut}; use rstest::{fixture, rstest}; @@ -416,6 +464,13 @@ mod tests { ) } + fn downcast_framing_error(error: &Box) -> &ChunkedGelfDecoderError { + error + .as_any() + .downcast_ref::() + .expect("Expected ChunkedGelfDecoderError to be downcasted") + } + #[rstest] #[tokio::test] async fn decode_chunked(two_chunks_message: ([BytesMut; 2], String)) { @@ -513,7 +568,7 @@ mod tests { tokio::time::sleep(Duration::from_millis(DEFAULT_TIMEOUT_MILLIS + 1)).await; assert!(decoder.state.lock().unwrap().is_empty()); assert!(logs_contain( - "Message was not fully received within the timeout window. Discarding it." + "Message was not fully received within the timeout window of 5000ms. Discarding it." )); let frame = decoder.decode_eof(&mut chunks[1]).unwrap(); @@ -537,17 +592,21 @@ mod tests { #[tokio::test] #[traced_test] - async fn decode_chunk_with_malformed_header() { + async fn decode_chunk_with_invalid_header() { let mut src = BytesMut::new(); src.extend_from_slice(&GELF_MAGIC); - // Malformed chunk header with less than 10 bytes - let malformed_chunk = [0x12, 0x34]; - src.extend_from_slice(&malformed_chunk); + // Invalid chunk header with less than 10 bytes + let invalid_chunk = [0x12, 0x34]; + src.extend_from_slice(&invalid_chunk); let mut decoder = ChunkedGelfDecoder::default(); + let frame = decoder.decode_eof(&mut src); - let frame = decoder.decode_eof(&mut src).unwrap(); - assert!(frame.is_none()); - assert!(logs_contain("Received malformed chunk headers (message ID, sequence number and total chunks) with less than 10 bytes. Ignoring it.")); + let error = frame.expect_err("Expected an error"); + let downcasted_error = downcast_framing_error(&error); + matches!( + downcasted_error, + ChunkedGelfDecoderError::InvalidChunkHeader { .. } + ); } #[tokio::test] @@ -560,11 +619,13 @@ mod tests { let mut chunk = create_chunk(message_id, sequence_number, invalid_total_chunks, payload); let mut decoder = ChunkedGelfDecoder::default(); - let frame = decoder.decode_eof(&mut chunk).unwrap(); - assert!(frame.is_none()); - assert!(logs_contain( - "Received a chunk with an invalid total chunks value. Ignoring it." - )); + let frame = decoder.decode_eof(&mut chunk); + let error = frame.expect_err("Expected an error"); + let downcasted_error = downcast_framing_error(&error); + matches!( + downcasted_error, + ChunkedGelfDecoderError::InvalidTotalChunks { .. } + ); } #[tokio::test] @@ -577,11 +638,13 @@ mod tests { let mut chunk = create_chunk(message_id, invalid_sequence_number, total_chunks, payload); let mut decoder = ChunkedGelfDecoder::default(); - let frame = decoder.decode_eof(&mut chunk).unwrap(); - assert!(frame.is_none()); - assert!(logs_contain( - "Received a chunk with a sequence number greater than total chunks. Ignoring it." - )); + let frame = decoder.decode_eof(&mut chunk); + let error = frame.expect_err("Expected an error"); + let downcasted_error = downcast_framing_error(&error); + matches!( + downcasted_error, + ChunkedGelfDecoderError::InvalidSequenceNumber { .. } + ); } #[rstest] @@ -600,12 +663,14 @@ mod tests { assert!(frame.is_none()); assert!(decoder.state.lock().unwrap().len() == 1); - let frame = decoder.decode_eof(&mut three_chunks[0]).unwrap(); - assert!(frame.is_none()); + let frame = decoder.decode_eof(&mut three_chunks[0]); + let error = frame.expect_err("Expected an error"); + let downcasted_error = downcast_framing_error(&error); + matches!( + downcasted_error, + ChunkedGelfDecoderError::PendingMessagesLimitReached { .. } + ); assert!(decoder.state.lock().unwrap().len() == 1); - assert!(logs_contain( - "Received a chunk but reached the pending messages limit. Ignoring it." - )); } #[rstest] @@ -624,11 +689,13 @@ mod tests { let frame = decoder.decode_eof(&mut first_chunk).unwrap(); assert!(frame.is_none()); - let frame = decoder.decode_eof(&mut second_chunk).unwrap(); - assert!(frame.is_none()); - assert!(logs_contain( - "Received a chunk with a different total chunks than the original. Ignoring it." - )); + let frame = decoder.decode_eof(&mut second_chunk); + let error = frame.expect_err("Expected an error"); + let downcasted_error = downcast_framing_error(&error); + matches!( + downcasted_error, + ChunkedGelfDecoderError::TotalChunksMismatch { .. } + ); } #[rstest] diff --git a/lib/codecs/src/decoding/framing/mod.rs b/lib/codecs/src/decoding/framing/mod.rs index 49b615ed152b3..b0bcab93415f0 100644 --- a/lib/codecs/src/decoding/framing/mod.rs +++ b/lib/codecs/src/decoding/framing/mod.rs @@ -10,7 +10,7 @@ mod length_delimited; mod newline_delimited; mod octet_counting; -use std::fmt::Debug; +use std::{any::Any, fmt::Debug}; use ::bytes::Bytes; pub use character_delimited::{ @@ -36,23 +36,33 @@ use super::StreamDecodingError; /// It requires conformance to `TcpError` so that we can determine whether the /// error is recoverable or if trying to continue will lead to hanging up the /// TCP source indefinitely. -pub trait FramingError: std::error::Error + StreamDecodingError + Send + Sync {} +pub trait FramingError: std::error::Error + StreamDecodingError + Send + Sync + Any { + /// Coerces the error to a `dyn Any`. + /// This is useful for downcasting the error to a concrete type, as we are dealing + /// with Box instead of a concrete `FramingError` enum. + fn as_any(&self) -> &dyn Any; +} impl std::error::Error for BoxedFramingError {} -impl FramingError for std::io::Error {} - -impl FramingError for LinesCodecError {} +impl FramingError for std::io::Error { + fn as_any(&self) -> &dyn Any { + self as &dyn Any + } +} -impl From for BoxedFramingError { - fn from(error: std::io::Error) -> Self { - Box::new(error) +impl FramingError for LinesCodecError { + fn as_any(&self) -> &dyn Any { + self as &dyn Any } } -impl From for BoxedFramingError { - fn from(error: LinesCodecError) -> Self { - Box::new(error) +impl From for BoxedFramingError +where + T: FramingError + 'static, +{ + fn from(value: T) -> Self { + Box::new(value) } } From 67d9992deaf5d504619a5dcfa628cefe0b79c4d3 Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Sun, 4 Aug 2024 20:54:52 +0200 Subject: [PATCH 32/60] test: add assert to matches! macros --- .../src/decoding/framing/chunked_gelf.rs | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index 1757c20979ef9..e512743ed9ddd 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -603,10 +603,10 @@ mod tests { let error = frame.expect_err("Expected an error"); let downcasted_error = downcast_framing_error(&error); - matches!( + assert!(matches!( downcasted_error, ChunkedGelfDecoderError::InvalidChunkHeader { .. } - ); + )); } #[tokio::test] @@ -622,10 +622,10 @@ mod tests { let frame = decoder.decode_eof(&mut chunk); let error = frame.expect_err("Expected an error"); let downcasted_error = downcast_framing_error(&error); - matches!( + assert!(matches!( downcasted_error, ChunkedGelfDecoderError::InvalidTotalChunks { .. } - ); + )); } #[tokio::test] @@ -641,10 +641,10 @@ mod tests { let frame = decoder.decode_eof(&mut chunk); let error = frame.expect_err("Expected an error"); let downcasted_error = downcast_framing_error(&error); - matches!( + assert!(matches!( downcasted_error, ChunkedGelfDecoderError::InvalidSequenceNumber { .. } - ); + )); } #[rstest] @@ -666,10 +666,10 @@ mod tests { let frame = decoder.decode_eof(&mut three_chunks[0]); let error = frame.expect_err("Expected an error"); let downcasted_error = downcast_framing_error(&error); - matches!( + assert!(matches!( downcasted_error, ChunkedGelfDecoderError::PendingMessagesLimitReached { .. } - ); + )); assert!(decoder.state.lock().unwrap().len() == 1); } @@ -692,10 +692,10 @@ mod tests { let frame = decoder.decode_eof(&mut second_chunk); let error = frame.expect_err("Expected an error"); let downcasted_error = downcast_framing_error(&error); - matches!( + assert!(matches!( downcasted_error, ChunkedGelfDecoderError::TotalChunksMismatch { .. } - ); + )); } #[rstest] From 2adaeb5e51b13587ff002c638723eac0fa0bfcd4 Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Sun, 4 Aug 2024 21:09:53 +0200 Subject: [PATCH 33/60] feat: change timeout_millis u64 to timeout_secs f64 --- .../src/decoding/framing/chunked_gelf.rs | 31 +++++++++---------- 1 file changed, 15 insertions(+), 16 deletions(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index e512743ed9ddd..a66d5bf92e861 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -16,11 +16,11 @@ use vector_config::configurable_component; const GELF_MAGIC: [u8; 2] = [0x1e, 0x0f]; const GELF_MAX_TOTAL_CHUNKS: u8 = 128; -const DEFAULT_TIMEOUT_MILLIS: u64 = 5000; +const DEFAULT_TIMEOUT_SECS: f64 = 5.0; const DEFAULT_PENDING_MESSAGES_LIMIT: usize = 1000; -const fn default_timeout_millis() -> u64 { - DEFAULT_TIMEOUT_MILLIS +const fn default_timeout_secs() -> f64 { + DEFAULT_TIMEOUT_SECS } const fn default_pending_messages_limit() -> usize { @@ -29,10 +29,9 @@ const fn default_pending_messages_limit() -> usize { /// Config used to build a `ChunkedGelfDecoder`. #[configurable_component] -#[derive(Debug, Clone, Default, PartialEq, Eq)] +#[derive(Debug, Clone, Default)] pub struct ChunkedGelfDecoderConfig { /// Options for the chunked GELF decoder. - #[serde(default, skip_serializing_if = "vector_core::serde::is_default")] pub chunked_gelf: ChunkedGelfDecoderOptions, } @@ -40,7 +39,7 @@ impl ChunkedGelfDecoderConfig { /// Build the `ChunkedGelfDecoder` from this configuration. pub fn build(&self) -> ChunkedGelfDecoder { ChunkedGelfDecoder::new( - self.chunked_gelf.timeout_millis, + self.chunked_gelf.timeout_secs, self.chunked_gelf.pending_messages_limit, ) } @@ -48,15 +47,15 @@ impl ChunkedGelfDecoderConfig { /// Options for building a `ChunkedGelfDecoder`. #[configurable_component] -#[derive(Clone, Debug, Derivative, PartialEq, Eq)] +#[derive(Clone, Debug, Derivative)] pub struct ChunkedGelfDecoderOptions { /// The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the /// decoder drops all the received chunks of the incomplete message and starts over. #[serde( - default = "default_timeout_millis", + default = "default_timeout_secs", skip_serializing_if = "vector_core::serde::is_default" )] - pub timeout_millis: u64, + pub timeout_secs: f64, /// The maximum number of pending incomplete messages. If this limit is reached, the decoder starts /// dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. @@ -70,7 +69,7 @@ pub struct ChunkedGelfDecoderOptions { impl Default for ChunkedGelfDecoderOptions { fn default() -> Self { Self { - timeout_millis: default_timeout_millis(), + timeout_secs: default_timeout_secs(), pending_messages_limit: default_pending_messages_limit(), } } @@ -188,11 +187,11 @@ pub struct ChunkedGelfDecoder { impl ChunkedGelfDecoder { /// Creates a new `ChunkedGelfDecoder`. - pub fn new(timeout_millis: u64, pending_messages_limit: usize) -> Self { + pub fn new(timeout_secs: f64, pending_messages_limit: usize) -> Self { Self { bytes_decoder: BytesDecoder::new(), state: Arc::new(Mutex::new(HashMap::new())), - timeout: Duration::from_millis(timeout_millis), + timeout: Duration::from_secs_f64(timeout_secs), pending_messages_limit, } } @@ -335,7 +334,7 @@ impl ChunkedGelfDecoder { impl Default for ChunkedGelfDecoder { fn default() -> Self { - Self::new(DEFAULT_TIMEOUT_MILLIS, DEFAULT_PENDING_MESSAGES_LIMIT) + Self::new(DEFAULT_TIMEOUT_SECS, DEFAULT_PENDING_MESSAGES_LIMIT) } } @@ -565,7 +564,7 @@ mod tests { assert!(!decoder.state.lock().unwrap().is_empty()); // The message state should be cleared after a certain time - tokio::time::sleep(Duration::from_millis(DEFAULT_TIMEOUT_MILLIS + 1)).await; + tokio::time::sleep(Duration::from_secs_f64(DEFAULT_TIMEOUT_SECS + 1.0)).await; assert!(decoder.state.lock().unwrap().is_empty()); assert!(logs_contain( "Message was not fully received within the timeout window of 5000ms. Discarding it." @@ -574,7 +573,7 @@ mod tests { let frame = decoder.decode_eof(&mut chunks[1]).unwrap(); assert!(frame.is_none()); - tokio::time::sleep(Duration::from_millis(DEFAULT_TIMEOUT_MILLIS + 1)).await; + tokio::time::sleep(Duration::from_secs_f64(DEFAULT_TIMEOUT_SECS + 1.0)).await; assert!(decoder.state.lock().unwrap().is_empty()); assert!(logs_contain( "Message was not fully received within the timeout window of 5000ms. Discarding it" @@ -657,7 +656,7 @@ mod tests { let pending_messages_limit = 1; let (mut two_chunks, _) = two_chunks_message; let (mut three_chunks, _) = three_chunks_message; - let mut decoder = ChunkedGelfDecoder::new(DEFAULT_TIMEOUT_MILLIS, pending_messages_limit); + let mut decoder = ChunkedGelfDecoder::new(DEFAULT_TIMEOUT_SECS, pending_messages_limit); let frame = decoder.decode_eof(&mut two_chunks[0]).unwrap(); assert!(frame.is_none()); From 049c8b62d1a1f7cee30f300f0ba229383954d7e4 Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Sun, 4 Aug 2024 21:13:58 +0200 Subject: [PATCH 34/60] style: use tracing macros as the style guides dictates --- .../src/decoding/framing/chunked_gelf.rs | 25 ++++++------------- 1 file changed, 7 insertions(+), 18 deletions(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index a66d5bf92e861..c9c6bcb8ec6b6 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -49,7 +49,7 @@ impl ChunkedGelfDecoderConfig { #[configurable_component] #[derive(Clone, Debug, Derivative)] pub struct ChunkedGelfDecoderOptions { - /// The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + /// The timeout, in seconds, for a message to be fully received. If the timeout is reached, the /// decoder drops all the received chunks of the incomplete message and starts over. #[serde( default = "default_timeout_secs", @@ -215,16 +215,6 @@ impl ChunkedGelfDecoder { // and return the complete payload. // We need 10 bytes to read the message id, sequence number and total chunks - // if chunk.remaining() < 10 { - // let src_display = format!("{chunk:?}"); - // warn!(message = "Received malformed chunk headers (message ID, sequence number and total chunks) with less than 10 bytes. Ignoring it.", - // src = src_display, - // remaining = chunk.remaining(), - // internal_log_rate_limit = true - // ); - // return Ok(None); - // } - ensure!( chunk.remaining() >= 10, InvalidChunkHeaderSnafu { header: chunk } @@ -272,13 +262,12 @@ impl ChunkedGelfDecoder { tokio::time::sleep(timeout).await; let mut state_lock = state.lock().unwrap(); if state_lock.remove(&message_id).is_some() { - let message = format!("Message was not fully received within the timeout window of {}ms. Discarding it.", - timeout.as_millis()); warn!( - message = message, message_id = message_id, - timeout = timeout.as_millis(), - internal_log_rate_limit = true + timeout = timeout.as_secs_f64(), + internal_log_rate_limit = true, + format!("Message was not fully received within the timeout window of {}s. Discarding it.", + timeout.as_secs_f64()) ); } }); @@ -297,10 +286,10 @@ impl ChunkedGelfDecoder { if message_state.is_chunk_present(sequence_number) { info!( - message = "Received a duplicate chunk. Ignoring it.", message_id = message_id, sequence_number = sequence_number, - internal_log_rate_limit = true + internal_log_rate_limit = true, + "Received a duplicate chunk. Ignoring it." ); return Ok(None); } From 0b41a8afc3f6bc13e663ea5de03ba490ef2bdebd Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Sun, 4 Aug 2024 21:31:39 +0200 Subject: [PATCH 35/60] test: fix broken test --- src/sources/socket/mod.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/sources/socket/mod.rs b/src/sources/socket/mod.rs index 4ab558a0b3ba8..089fc819d8d86 100644 --- a/src/sources/socket/mod.rs +++ b/src/sources/socket/mod.rs @@ -1145,14 +1145,14 @@ mod test { let seed = 42; let mut rng = SmallRng::seed_from_u64(seed); let max_size = 300; - let big_message = "This is a very large message".repeat(1000); - let another_big_message = "This is another very large message".repeat(1000); + let big_message = "This is a very large message".repeat(500); + let another_big_message = "This is another very large message".repeat(500); let mut chunks = get_gelf_chunks(big_message.as_str(), max_size, &mut rng); let mut another_chunks = get_gelf_chunks(another_big_message.as_str(), max_size, &mut rng); chunks.append(&mut another_chunks); chunks.shuffle(&mut rng); - + dbg!(chunks.len()); send_packets_udp(address, chunks); let events = collect_n(rx, 2).await; From d050b3e140bd08bb728cd5c5492804492d2142c9 Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Sun, 4 Aug 2024 21:32:13 +0200 Subject: [PATCH 36/60] style: modify tracing logs usage according to style guide --- lib/codecs/src/decoding/framing/chunked_gelf.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index c9c6bcb8ec6b6..e295d38b19f7b 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -264,10 +264,9 @@ impl ChunkedGelfDecoder { if state_lock.remove(&message_id).is_some() { warn!( message_id = message_id, - timeout = timeout.as_secs_f64(), + timeout_secs = timeout.as_secs_f64(), internal_log_rate_limit = true, - format!("Message was not fully received within the timeout window of {}s. Discarding it.", - timeout.as_secs_f64()) + "Message was not fully received within the timeout window. Discarding it." ); } }); From 8b74992a600c91e5ea9d0b051326f3b590af5ecc Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Sun, 4 Aug 2024 21:50:08 +0200 Subject: [PATCH 37/60] test: fix broken udp socket test --- src/sources/socket/mod.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/sources/socket/mod.rs b/src/sources/socket/mod.rs index 089fc819d8d86..453de5777a977 100644 --- a/src/sources/socket/mod.rs +++ b/src/sources/socket/mod.rs @@ -1157,11 +1157,11 @@ mod test { let events = collect_n(rx, 2).await; assert_eq!( - events[0].as_log()[log_schema().message_key().unwrap().to_string()], + events[1].as_log()[log_schema().message_key().unwrap().to_string()], big_message.into() ); assert_eq!( - events[1].as_log()[log_schema().message_key().unwrap().to_string()], + events[0].as_log()[log_schema().message_key().unwrap().to_string()], another_big_message.into() ); }) From 833196469fa81cf02920be80d4a66cf26bdc8a11 Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Sun, 4 Aug 2024 21:53:31 +0200 Subject: [PATCH 38/60] test: fix broken chunked gelf framing test --- lib/codecs/src/decoding/framing/chunked_gelf.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index e295d38b19f7b..b68a4bd8ae599 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -555,7 +555,7 @@ mod tests { tokio::time::sleep(Duration::from_secs_f64(DEFAULT_TIMEOUT_SECS + 1.0)).await; assert!(decoder.state.lock().unwrap().is_empty()); assert!(logs_contain( - "Message was not fully received within the timeout window of 5000ms. Discarding it." + "Message was not fully received within the timeout window. Discarding it." )); let frame = decoder.decode_eof(&mut chunks[1]).unwrap(); @@ -564,7 +564,7 @@ mod tests { tokio::time::sleep(Duration::from_secs_f64(DEFAULT_TIMEOUT_SECS + 1.0)).await; assert!(decoder.state.lock().unwrap().is_empty()); assert!(logs_contain( - "Message was not fully received within the timeout window of 5000ms. Discarding it" + "Message was not fully received within the timeout window. Discarding it" )); } From c70a64583b57680a444cedb27e6010a3b27a9e2d Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Mon, 5 Aug 2024 16:49:26 +0200 Subject: [PATCH 39/60] fix: clippy lints --- lib/codecs/src/decoding/framing/chunked_gelf.rs | 2 +- src/sources/socket/mod.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index b68a4bd8ae599..4d23fe6b0493e 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -451,7 +451,7 @@ mod tests { ) } - fn downcast_framing_error(error: &Box) -> &ChunkedGelfDecoderError { + fn downcast_framing_error(error: &BoxedFramingError) -> &ChunkedGelfDecoderError { error .as_any() .downcast_ref::() diff --git a/src/sources/socket/mod.rs b/src/sources/socket/mod.rs index 453de5777a977..d57ab820602ed 100644 --- a/src/sources/socket/mod.rs +++ b/src/sources/socket/mod.rs @@ -1152,7 +1152,7 @@ mod test { get_gelf_chunks(another_big_message.as_str(), max_size, &mut rng); chunks.append(&mut another_chunks); chunks.shuffle(&mut rng); - dbg!(chunks.len()); + send_packets_udp(address, chunks); let events = collect_n(rx, 2).await; From 12fb1e2d2eafeb09c0d23e7099b812a4fe788d51 Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Sun, 1 Sep 2024 21:35:54 +0200 Subject: [PATCH 40/60] refactor: cleanup some code --- .../src/decoding/framing/chunked_gelf.rs | 21 +++++++------------ 1 file changed, 7 insertions(+), 14 deletions(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index 4d23fe6b0493e..b0c99f96bc28e 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -14,7 +14,7 @@ use tokio_util::codec::Decoder; use tracing::{info, warn}; use vector_config::configurable_component; -const GELF_MAGIC: [u8; 2] = [0x1e, 0x0f]; +const GELF_MAGIC: &[u8] = &[0x1e, 0x0f]; const GELF_MAX_TOTAL_CHUNKS: u8 = 128; const DEFAULT_TIMEOUT_SECS: f64 = 5.0; const DEFAULT_PENDING_MESSAGES_LIMIT: usize = 1000; @@ -48,6 +48,7 @@ impl ChunkedGelfDecoderConfig { /// Options for building a `ChunkedGelfDecoder`. #[configurable_component] #[derive(Clone, Debug, Derivative)] +#[derivative(Default)] pub struct ChunkedGelfDecoderOptions { /// The timeout, in seconds, for a message to be fully received. If the timeout is reached, the /// decoder drops all the received chunks of the incomplete message and starts over. @@ -55,6 +56,7 @@ pub struct ChunkedGelfDecoderOptions { default = "default_timeout_secs", skip_serializing_if = "vector_core::serde::is_default" )] + #[derivative(Default(value = "default_timeout_secs()"))] pub timeout_secs: f64, /// The maximum number of pending incomplete messages. If this limit is reached, the decoder starts @@ -63,18 +65,10 @@ pub struct ChunkedGelfDecoderOptions { default = "default_pending_messages_limit", skip_serializing_if = "vector_core::serde::is_default" )] + #[derivative(Default(value = "default_pending_messages_limit()"))] pub pending_messages_limit: usize, } -impl Default for ChunkedGelfDecoderOptions { - fn default() -> Self { - Self { - timeout_secs: default_timeout_secs(), - pending_messages_limit: default_pending_messages_limit(), - } - } -} - #[derive(Debug)] struct MessageState { total_chunks: u8, @@ -310,8 +304,7 @@ impl ChunkedGelfDecoder { &mut self, mut src: Bytes, ) -> Result, ChunkedGelfDecoderError> { - let magic = src.get(0..2); - if magic.is_some_and(|magic| magic == GELF_MAGIC) { + if src.starts_with(GELF_MAGIC) { src.advance(2); self.decode_chunk(src) } else { @@ -370,7 +363,7 @@ mod tests { payload: &str, ) -> BytesMut { let mut chunk = BytesMut::new(); - chunk.put_slice(&GELF_MAGIC); + chunk.put_slice(GELF_MAGIC); chunk.put_u64(message_id); chunk.put_u8(sequence_number); chunk.put_u8(total_chunks); @@ -581,7 +574,7 @@ mod tests { #[traced_test] async fn decode_chunk_with_invalid_header() { let mut src = BytesMut::new(); - src.extend_from_slice(&GELF_MAGIC); + src.extend_from_slice(GELF_MAGIC); // Invalid chunk header with less than 10 bytes let invalid_chunk = [0x12, 0x34]; src.extend_from_slice(&invalid_chunk); From e76a75080e0f977bc44c1a0765dd0a0f7c7e155d Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Sun, 1 Sep 2024 21:45:10 +0200 Subject: [PATCH 41/60] refactor: cleanup --- lib/codecs/src/decoding/framing/chunked_gelf.rs | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index b0c99f96bc28e..2f6b20e7b3e10 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -81,32 +81,28 @@ impl MessageState { pub const fn new(total_chunks: u8, timeout_task: JoinHandle<()>) -> Self { Self { total_chunks, - chunks: Self::default_chunks(), + chunks: [const { Bytes::new() }; GELF_MAX_TOTAL_CHUNKS as usize], chunks_bitmap: 0, timeout_task, } } - pub const fn default_chunks() -> [Bytes; GELF_MAX_TOTAL_CHUNKS as usize] { - [const { Bytes::new() }; GELF_MAX_TOTAL_CHUNKS as usize] - } - - pub fn is_chunk_present(&self, sequence_number: u8) -> bool { + fn is_chunk_present(&self, sequence_number: u8) -> bool { let chunk_bitmap_id = 1 << sequence_number; self.chunks_bitmap & chunk_bitmap_id != 0 } - pub fn add_chunk(&mut self, sequence_number: u8, chunk: Bytes) { + fn add_chunk(&mut self, sequence_number: u8, chunk: Bytes) { let chunk_bitmap_id = 1 << sequence_number; self.chunks[sequence_number as usize] = chunk; self.chunks_bitmap |= chunk_bitmap_id; } - pub fn is_complete(&self) -> bool { + fn is_complete(&self) -> bool { self.chunks_bitmap.count_ones() == self.total_chunks as u32 } - pub fn retrieve_message(&mut self) -> Option { + fn retrieve_message(&mut self) -> Option { if self.is_complete() { self.timeout_task.abort(); let chunks = &self.chunks[0..self.total_chunks as usize]; From 5c8d38c7e29c6fa63da1720c622ce5097402acff Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Tue, 3 Sep 2024 23:00:45 +0200 Subject: [PATCH 42/60] docs: update components docs --- website/cue/reference/components/sources/base/amqp.cue | 8 ++++---- .../components/sources/base/aws_kinesis_firehose.cue | 8 ++++---- website/cue/reference/components/sources/base/aws_s3.cue | 8 ++++---- website/cue/reference/components/sources/base/aws_sqs.cue | 8 ++++---- .../reference/components/sources/base/datadog_agent.cue | 8 ++++---- .../cue/reference/components/sources/base/demo_logs.cue | 8 ++++---- website/cue/reference/components/sources/base/exec.cue | 8 ++++---- .../reference/components/sources/base/file_descriptor.cue | 8 ++++---- .../cue/reference/components/sources/base/gcp_pubsub.cue | 8 ++++---- .../cue/reference/components/sources/base/heroku_logs.cue | 8 ++++---- website/cue/reference/components/sources/base/http.cue | 8 ++++---- .../cue/reference/components/sources/base/http_client.cue | 8 ++++---- .../cue/reference/components/sources/base/http_server.cue | 8 ++++---- website/cue/reference/components/sources/base/kafka.cue | 8 ++++---- website/cue/reference/components/sources/base/nats.cue | 8 ++++---- website/cue/reference/components/sources/base/pulsar.cue | 8 ++++---- website/cue/reference/components/sources/base/redis.cue | 8 ++++---- website/cue/reference/components/sources/base/socket.cue | 8 ++++---- website/cue/reference/components/sources/base/stdin.cue | 8 ++++---- 19 files changed, 76 insertions(+), 76 deletions(-) diff --git a/website/cue/reference/components/sources/base/amqp.cue b/website/cue/reference/components/sources/base/amqp.cue index de5d934f9cdc5..dd126fa2d884e 100644 --- a/website/cue/reference/components/sources/base/amqp.cue +++ b/website/cue/reference/components/sources/base/amqp.cue @@ -333,7 +333,7 @@ base: components: sources: amqp: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: false + required: true type: object: options: { pending_messages_limit: { description: """ @@ -343,13 +343,13 @@ base: components: sources: amqp: configuration: { required: false type: uint: default: 1000 } - timeout_millis: { + timeout_secs: { description: """ - The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + The timeout, in seconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. """ required: false - type: uint: default: 5000 + type: float: default: 5.0 } } } diff --git a/website/cue/reference/components/sources/base/aws_kinesis_firehose.cue b/website/cue/reference/components/sources/base/aws_kinesis_firehose.cue index 245f4c9cb5579..d6e53577eb7d3 100644 --- a/website/cue/reference/components/sources/base/aws_kinesis_firehose.cue +++ b/website/cue/reference/components/sources/base/aws_kinesis_firehose.cue @@ -331,7 +331,7 @@ base: components: sources: aws_kinesis_firehose: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: false + required: true type: object: options: { pending_messages_limit: { description: """ @@ -341,13 +341,13 @@ base: components: sources: aws_kinesis_firehose: configuration: { required: false type: uint: default: 1000 } - timeout_millis: { + timeout_secs: { description: """ - The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + The timeout, in seconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. """ required: false - type: uint: default: 5000 + type: float: default: 5.0 } } } diff --git a/website/cue/reference/components/sources/base/aws_s3.cue b/website/cue/reference/components/sources/base/aws_s3.cue index e9227e28dce9a..d6669885d96f4 100644 --- a/website/cue/reference/components/sources/base/aws_s3.cue +++ b/website/cue/reference/components/sources/base/aws_s3.cue @@ -431,7 +431,7 @@ base: components: sources: aws_s3: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: false + required: true type: object: options: { pending_messages_limit: { description: """ @@ -441,13 +441,13 @@ base: components: sources: aws_s3: configuration: { required: false type: uint: default: 1000 } - timeout_millis: { + timeout_secs: { description: """ - The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + The timeout, in seconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. """ required: false - type: uint: default: 5000 + type: float: default: 5.0 } } } diff --git a/website/cue/reference/components/sources/base/aws_sqs.cue b/website/cue/reference/components/sources/base/aws_sqs.cue index 9d5e06aec6018..768ab56759635 100644 --- a/website/cue/reference/components/sources/base/aws_sqs.cue +++ b/website/cue/reference/components/sources/base/aws_sqs.cue @@ -435,7 +435,7 @@ base: components: sources: aws_sqs: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: false + required: true type: object: options: { pending_messages_limit: { description: """ @@ -445,13 +445,13 @@ base: components: sources: aws_sqs: configuration: { required: false type: uint: default: 1000 } - timeout_millis: { + timeout_secs: { description: """ - The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + The timeout, in seconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. """ required: false - type: uint: default: 5000 + type: float: default: 5.0 } } } diff --git a/website/cue/reference/components/sources/base/datadog_agent.cue b/website/cue/reference/components/sources/base/datadog_agent.cue index c4f8e6861aee3..18f30ae85de5a 100644 --- a/website/cue/reference/components/sources/base/datadog_agent.cue +++ b/website/cue/reference/components/sources/base/datadog_agent.cue @@ -328,7 +328,7 @@ base: components: sources: datadog_agent: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: false + required: true type: object: options: { pending_messages_limit: { description: """ @@ -338,13 +338,13 @@ base: components: sources: datadog_agent: configuration: { required: false type: uint: default: 1000 } - timeout_millis: { + timeout_secs: { description: """ - The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + The timeout, in seconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. """ required: false - type: uint: default: 5000 + type: float: default: 5.0 } } } diff --git a/website/cue/reference/components/sources/base/demo_logs.cue b/website/cue/reference/components/sources/base/demo_logs.cue index e54bb09a30857..7f253f5e9dbf5 100644 --- a/website/cue/reference/components/sources/base/demo_logs.cue +++ b/website/cue/reference/components/sources/base/demo_logs.cue @@ -324,7 +324,7 @@ base: components: sources: demo_logs: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: false + required: true type: object: options: { pending_messages_limit: { description: """ @@ -334,13 +334,13 @@ base: components: sources: demo_logs: configuration: { required: false type: uint: default: 1000 } - timeout_millis: { + timeout_secs: { description: """ - The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + The timeout, in seconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. """ required: false - type: uint: default: 5000 + type: float: default: 5.0 } } } diff --git a/website/cue/reference/components/sources/base/exec.cue b/website/cue/reference/components/sources/base/exec.cue index f3fed43d32456..2452ff71affa5 100644 --- a/website/cue/reference/components/sources/base/exec.cue +++ b/website/cue/reference/components/sources/base/exec.cue @@ -312,7 +312,7 @@ base: components: sources: exec: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: false + required: true type: object: options: { pending_messages_limit: { description: """ @@ -322,13 +322,13 @@ base: components: sources: exec: configuration: { required: false type: uint: default: 1000 } - timeout_millis: { + timeout_secs: { description: """ - The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + The timeout, in seconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. """ required: false - type: uint: default: 5000 + type: float: default: 5.0 } } } diff --git a/website/cue/reference/components/sources/base/file_descriptor.cue b/website/cue/reference/components/sources/base/file_descriptor.cue index 32d6458015a89..aba7a62e0ec63 100644 --- a/website/cue/reference/components/sources/base/file_descriptor.cue +++ b/website/cue/reference/components/sources/base/file_descriptor.cue @@ -290,7 +290,7 @@ base: components: sources: file_descriptor: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: false + required: true type: object: options: { pending_messages_limit: { description: """ @@ -300,13 +300,13 @@ base: components: sources: file_descriptor: configuration: { required: false type: uint: default: 1000 } - timeout_millis: { + timeout_secs: { description: """ - The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + The timeout, in seconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. """ required: false - type: uint: default: 5000 + type: float: default: 5.0 } } } diff --git a/website/cue/reference/components/sources/base/gcp_pubsub.cue b/website/cue/reference/components/sources/base/gcp_pubsub.cue index 130d5e2a584b6..6ae9cf0307590 100644 --- a/website/cue/reference/components/sources/base/gcp_pubsub.cue +++ b/website/cue/reference/components/sources/base/gcp_pubsub.cue @@ -367,7 +367,7 @@ base: components: sources: gcp_pubsub: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: false + required: true type: object: options: { pending_messages_limit: { description: """ @@ -377,13 +377,13 @@ base: components: sources: gcp_pubsub: configuration: { required: false type: uint: default: 1000 } - timeout_millis: { + timeout_secs: { description: """ - The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + The timeout, in seconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. """ required: false - type: uint: default: 5000 + type: float: default: 5.0 } } } diff --git a/website/cue/reference/components/sources/base/heroku_logs.cue b/website/cue/reference/components/sources/base/heroku_logs.cue index 4e8748aae949a..c61984e57311a 100644 --- a/website/cue/reference/components/sources/base/heroku_logs.cue +++ b/website/cue/reference/components/sources/base/heroku_logs.cue @@ -325,7 +325,7 @@ base: components: sources: heroku_logs: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: false + required: true type: object: options: { pending_messages_limit: { description: """ @@ -335,13 +335,13 @@ base: components: sources: heroku_logs: configuration: { required: false type: uint: default: 1000 } - timeout_millis: { + timeout_secs: { description: """ - The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + The timeout, in seconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. """ required: false - type: uint: default: 5000 + type: float: default: 5.0 } } } diff --git a/website/cue/reference/components/sources/base/http.cue b/website/cue/reference/components/sources/base/http.cue index 967cccab041b4..bcf15acd5dc37 100644 --- a/website/cue/reference/components/sources/base/http.cue +++ b/website/cue/reference/components/sources/base/http.cue @@ -340,7 +340,7 @@ base: components: sources: http: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: false + required: true type: object: options: { pending_messages_limit: { description: """ @@ -350,13 +350,13 @@ base: components: sources: http: configuration: { required: false type: uint: default: 1000 } - timeout_millis: { + timeout_secs: { description: """ - The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + The timeout, in seconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. """ required: false - type: uint: default: 5000 + type: float: default: 5.0 } } } diff --git a/website/cue/reference/components/sources/base/http_client.cue b/website/cue/reference/components/sources/base/http_client.cue index f5674d31c3f24..613770a9c83e3 100644 --- a/website/cue/reference/components/sources/base/http_client.cue +++ b/website/cue/reference/components/sources/base/http_client.cue @@ -328,7 +328,7 @@ base: components: sources: http_client: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: false + required: true type: object: options: { pending_messages_limit: { description: """ @@ -338,13 +338,13 @@ base: components: sources: http_client: configuration: { required: false type: uint: default: 1000 } - timeout_millis: { + timeout_secs: { description: """ - The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + The timeout, in seconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. """ required: false - type: uint: default: 5000 + type: float: default: 5.0 } } } diff --git a/website/cue/reference/components/sources/base/http_server.cue b/website/cue/reference/components/sources/base/http_server.cue index 589cf16bfd05b..66f3f01285173 100644 --- a/website/cue/reference/components/sources/base/http_server.cue +++ b/website/cue/reference/components/sources/base/http_server.cue @@ -340,7 +340,7 @@ base: components: sources: http_server: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: false + required: true type: object: options: { pending_messages_limit: { description: """ @@ -350,13 +350,13 @@ base: components: sources: http_server: configuration: { required: false type: uint: default: 1000 } - timeout_millis: { + timeout_secs: { description: """ - The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + The timeout, in seconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. """ required: false - type: uint: default: 5000 + type: float: default: 5.0 } } } diff --git a/website/cue/reference/components/sources/base/kafka.cue b/website/cue/reference/components/sources/base/kafka.cue index e6b621758cd76..c3cef087e9885 100644 --- a/website/cue/reference/components/sources/base/kafka.cue +++ b/website/cue/reference/components/sources/base/kafka.cue @@ -361,7 +361,7 @@ base: components: sources: kafka: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: false + required: true type: object: options: { pending_messages_limit: { description: """ @@ -371,13 +371,13 @@ base: components: sources: kafka: configuration: { required: false type: uint: default: 1000 } - timeout_millis: { + timeout_secs: { description: """ - The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + The timeout, in seconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. """ required: false - type: uint: default: 5000 + type: float: default: 5.0 } } } diff --git a/website/cue/reference/components/sources/base/nats.cue b/website/cue/reference/components/sources/base/nats.cue index 147e4cca9ff78..1cca22dbbb46f 100644 --- a/website/cue/reference/components/sources/base/nats.cue +++ b/website/cue/reference/components/sources/base/nats.cue @@ -380,7 +380,7 @@ base: components: sources: nats: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: false + required: true type: object: options: { pending_messages_limit: { description: """ @@ -390,13 +390,13 @@ base: components: sources: nats: configuration: { required: false type: uint: default: 1000 } - timeout_millis: { + timeout_secs: { description: """ - The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + The timeout, in seconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. """ required: false - type: uint: default: 5000 + type: float: default: 5.0 } } } diff --git a/website/cue/reference/components/sources/base/pulsar.cue b/website/cue/reference/components/sources/base/pulsar.cue index 03ca87c19a714..d7bd84a2ad6f5 100644 --- a/website/cue/reference/components/sources/base/pulsar.cue +++ b/website/cue/reference/components/sources/base/pulsar.cue @@ -391,7 +391,7 @@ base: components: sources: pulsar: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: false + required: true type: object: options: { pending_messages_limit: { description: """ @@ -401,13 +401,13 @@ base: components: sources: pulsar: configuration: { required: false type: uint: default: 1000 } - timeout_millis: { + timeout_secs: { description: """ - The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + The timeout, in seconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. """ required: false - type: uint: default: 5000 + type: float: default: 5.0 } } } diff --git a/website/cue/reference/components/sources/base/redis.cue b/website/cue/reference/components/sources/base/redis.cue index bb8603299c460..fecbec8d88eb0 100644 --- a/website/cue/reference/components/sources/base/redis.cue +++ b/website/cue/reference/components/sources/base/redis.cue @@ -298,7 +298,7 @@ base: components: sources: redis: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: false + required: true type: object: options: { pending_messages_limit: { description: """ @@ -308,13 +308,13 @@ base: components: sources: redis: configuration: { required: false type: uint: default: 1000 } - timeout_millis: { + timeout_secs: { description: """ - The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + The timeout, in seconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. """ required: false - type: uint: default: 5000 + type: float: default: 5.0 } } } diff --git a/website/cue/reference/components/sources/base/socket.cue b/website/cue/reference/components/sources/base/socket.cue index c7b73c35f2a79..63b00a6f41778 100644 --- a/website/cue/reference/components/sources/base/socket.cue +++ b/website/cue/reference/components/sources/base/socket.cue @@ -300,7 +300,7 @@ base: components: sources: socket: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: false + required: true type: object: options: { pending_messages_limit: { description: """ @@ -310,13 +310,13 @@ base: components: sources: socket: configuration: { required: false type: uint: default: 1000 } - timeout_millis: { + timeout_secs: { description: """ - The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + The timeout, in seconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. """ required: false - type: uint: default: 5000 + type: float: default: 5.0 } } } diff --git a/website/cue/reference/components/sources/base/stdin.cue b/website/cue/reference/components/sources/base/stdin.cue index 993381a0945ec..bae27713e72d1 100644 --- a/website/cue/reference/components/sources/base/stdin.cue +++ b/website/cue/reference/components/sources/base/stdin.cue @@ -283,7 +283,7 @@ base: components: sources: stdin: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: false + required: true type: object: options: { pending_messages_limit: { description: """ @@ -293,13 +293,13 @@ base: components: sources: stdin: configuration: { required: false type: uint: default: 1000 } - timeout_millis: { + timeout_secs: { description: """ - The timeout, in milliseconds, for a message to be fully received. If the timeout is reached, the + The timeout, in seconds, for a message to be fully received. If the timeout is reached, the decoder drops all the received chunks of the incomplete message and starts over. """ required: false - type: uint: default: 5000 + type: float: default: 5.0 } } } From 6c66ff04aea9e31ce7105f1a28ef45e9f58411e1 Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Fri, 11 Oct 2024 00:00:06 +0200 Subject: [PATCH 43/60] fix: remove malformed comment --- lib/codecs/src/decoding/framing/mod.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/lib/codecs/src/decoding/framing/mod.rs b/lib/codecs/src/decoding/framing/mod.rs index b0bcab93415f0..86740a50129f2 100644 --- a/lib/codecs/src/decoding/framing/mod.rs +++ b/lib/codecs/src/decoding/framing/mod.rs @@ -38,8 +38,7 @@ use super::StreamDecodingError; /// TCP source indefinitely. pub trait FramingError: std::error::Error + StreamDecodingError + Send + Sync + Any { /// Coerces the error to a `dyn Any`. - /// This is useful for downcasting the error to a concrete type, as we are dealing - /// with Box instead of a concrete `FramingError` enum. + /// This is useful for downcasting the error to a concrete type fn as_any(&self) -> &dyn Any; } From 72414d54820436d818ecac49830bb14a47fa1ba6 Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Fri, 11 Oct 2024 00:14:02 +0200 Subject: [PATCH 44/60] fix: default chunked gelf options --- lib/codecs/src/decoding/framing/chunked_gelf.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index 2f6b20e7b3e10..090db6bd20918 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -32,6 +32,7 @@ const fn default_pending_messages_limit() -> usize { #[derive(Debug, Clone, Default)] pub struct ChunkedGelfDecoderConfig { /// Options for the chunked GELF decoder. + #[serde(default)] pub chunked_gelf: ChunkedGelfDecoderOptions, } From 85edb0014787b2b1e340078b2a275603b2f0f7f6 Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Fri, 18 Oct 2024 17:48:37 +0200 Subject: [PATCH 45/60] feat: default pending_messages_limit to None --- .../src/decoding/framing/chunked_gelf.rs | 49 ++++++++----------- 1 file changed, 20 insertions(+), 29 deletions(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index 090db6bd20918..a85d0dec86ac5 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -17,16 +17,11 @@ use vector_config::configurable_component; const GELF_MAGIC: &[u8] = &[0x1e, 0x0f]; const GELF_MAX_TOTAL_CHUNKS: u8 = 128; const DEFAULT_TIMEOUT_SECS: f64 = 5.0; -const DEFAULT_PENDING_MESSAGES_LIMIT: usize = 1000; const fn default_timeout_secs() -> f64 { DEFAULT_TIMEOUT_SECS } -const fn default_pending_messages_limit() -> usize { - DEFAULT_PENDING_MESSAGES_LIMIT -} - /// Config used to build a `ChunkedGelfDecoder`. #[configurable_component] #[derive(Debug, Clone, Default)] @@ -61,13 +56,11 @@ pub struct ChunkedGelfDecoderOptions { pub timeout_secs: f64, /// The maximum number of pending incomplete messages. If this limit is reached, the decoder starts - /// dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. - #[serde( - default = "default_pending_messages_limit", - skip_serializing_if = "vector_core::serde::is_default" - )] - #[derivative(Default(value = "default_pending_messages_limit()"))] - pub pending_messages_limit: usize, + /// dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. + /// If this option is not set, the decoder does not limit the number of pending messages and the memory usage + /// of pending messages can grow unbounded. This matches Graylog Server's behavior. + #[serde(default, skip_serializing_if = "vector_core::serde::is_default")] + pub pending_messages_limit: Option, } #[derive(Debug)] @@ -173,12 +166,12 @@ pub struct ChunkedGelfDecoder { bytes_decoder: BytesDecoder, state: Arc>>, timeout: Duration, - pending_messages_limit: usize, + pending_messages_limit: Option, } impl ChunkedGelfDecoder { /// Creates a new `ChunkedGelfDecoder`. - pub fn new(timeout_secs: f64, pending_messages_limit: usize) -> Self { + pub fn new(timeout_secs: f64, pending_messages_limit: Option) -> Self { Self { bytes_decoder: BytesDecoder::new(), state: Arc::new(Mutex::new(HashMap::new())), @@ -235,14 +228,16 @@ impl ChunkedGelfDecoder { let mut state_lock = self.state.lock().expect("poisoned lock"); - ensure!( - state_lock.len() < self.pending_messages_limit, - PendingMessagesLimitReachedSnafu { - message_id, - sequence_number, - pending_messages_limit: self.pending_messages_limit - } - ); + if let Some(pending_messages_limit) = self.pending_messages_limit { + ensure!( + state_lock.len() < pending_messages_limit, + PendingMessagesLimitReachedSnafu { + message_id, + sequence_number, + pending_messages_limit + } + ); + } let message_state = state_lock.entry(message_id).or_insert_with(|| { // We need to spawn a task that will clear the message state after a certain time @@ -312,7 +307,7 @@ impl ChunkedGelfDecoder { impl Default for ChunkedGelfDecoder { fn default() -> Self { - Self::new(DEFAULT_TIMEOUT_SECS, DEFAULT_PENDING_MESSAGES_LIMIT) + Self::new(DEFAULT_TIMEOUT_SECS, None) } } @@ -568,7 +563,6 @@ mod tests { } #[tokio::test] - #[traced_test] async fn decode_chunk_with_invalid_header() { let mut src = BytesMut::new(); src.extend_from_slice(GELF_MAGIC); @@ -587,7 +581,6 @@ mod tests { } #[tokio::test] - #[traced_test] async fn decode_chunk_with_invalid_total_chunks() { let message_id = 1u64; let sequence_number = 1u8; @@ -606,7 +599,6 @@ mod tests { } #[tokio::test] - #[traced_test] async fn decode_chunk_with_invalid_sequence_number() { let message_id = 1u64; let total_chunks = 2u8; @@ -626,7 +618,6 @@ mod tests { #[rstest] #[tokio::test] - #[traced_test] async fn decode_when_reached_pending_messages_limit( two_chunks_message: ([BytesMut; 2], String), three_chunks_message: ([BytesMut; 3], String), @@ -634,7 +625,8 @@ mod tests { let pending_messages_limit = 1; let (mut two_chunks, _) = two_chunks_message; let (mut three_chunks, _) = three_chunks_message; - let mut decoder = ChunkedGelfDecoder::new(DEFAULT_TIMEOUT_SECS, pending_messages_limit); + let mut decoder = + ChunkedGelfDecoder::new(DEFAULT_TIMEOUT_SECS, Some(pending_messages_limit)); let frame = decoder.decode_eof(&mut two_chunks[0]).unwrap(); assert!(frame.is_none()); @@ -652,7 +644,6 @@ mod tests { #[rstest] #[tokio::test] - #[traced_test] async fn decode_chunk_with_different_total_chunks() { let message_id = 1u64; let sequence_number = 0u8; From b214fdb3e23352c5e353529280fda968909b1b5d Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Fri, 18 Oct 2024 17:54:35 +0200 Subject: [PATCH 46/60] feat: remove skip_serialization for timeout_secs --- lib/codecs/src/decoding/framing/chunked_gelf.rs | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index a85d0dec86ac5..cc3e24e4bcf63 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -48,10 +48,7 @@ impl ChunkedGelfDecoderConfig { pub struct ChunkedGelfDecoderOptions { /// The timeout, in seconds, for a message to be fully received. If the timeout is reached, the /// decoder drops all the received chunks of the incomplete message and starts over. - #[serde( - default = "default_timeout_secs", - skip_serializing_if = "vector_core::serde::is_default" - )] + #[serde(default = "default_timeout_secs")] #[derivative(Default(value = "default_timeout_secs()"))] pub timeout_secs: f64, From f1e5996f4138a98f321f94a0aba8f02655c0018d Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Fri, 18 Oct 2024 18:04:23 +0200 Subject: [PATCH 47/60] Update lib/codecs/src/decoding/framing/chunked_gelf.rs Co-authored-by: Jesse Szwedko --- lib/codecs/src/decoding/framing/chunked_gelf.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index cc3e24e4bcf63..bf22ad6a307d6 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -243,7 +243,7 @@ impl ChunkedGelfDecoder { let timeout = self.timeout; let timeout_handle = tokio::spawn(async move { tokio::time::sleep(timeout).await; - let mut state_lock = state.lock().unwrap(); + let mut state_lock = state.lock().expect("poisoned lock"); if state_lock.remove(&message_id).is_some() { warn!( message_id = message_id, From 77ec8f553cb0aa9016534dce521fff9571afafdd Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Fri, 18 Oct 2024 18:07:56 +0200 Subject: [PATCH 48/60] feat: change duplicated chunk log error to debug --- lib/codecs/src/decoding/framing/chunked_gelf.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index bf22ad6a307d6..52bf81f474050 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -11,7 +11,7 @@ use std::time::Duration; use tokio; use tokio::task::JoinHandle; use tokio_util::codec::Decoder; -use tracing::{info, warn}; +use tracing::{debug, info, warn}; use vector_config::configurable_component; const GELF_MAGIC: &[u8] = &[0x1e, 0x0f]; @@ -267,7 +267,7 @@ impl ChunkedGelfDecoder { ); if message_state.is_chunk_present(sequence_number) { - info!( + debug!( message_id = message_id, sequence_number = sequence_number, internal_log_rate_limit = true, From 2ab9bf3e9d8222ccc6b6b9c7000343469dabc859 Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Fri, 18 Oct 2024 18:35:13 +0200 Subject: [PATCH 49/60] test: add shuffle messages test --- Cargo.lock | 1 + Cargo.toml | 3 +- lib/codecs/Cargo.toml | 1 + .../src/decoding/framing/chunked_gelf.rs | 61 ++++++++++++++++++- 4 files changed, 62 insertions(+), 4 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 8294f5e547af6..9cf9d7fbda340 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2190,6 +2190,7 @@ dependencies = [ "ordered-float 4.4.0", "prost 0.12.6", "prost-reflect", + "rand 0.8.5", "regex", "rstest", "serde", diff --git a/Cargo.toml b/Cargo.toml index af1b822488b46..6a54d8ae3ce71 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -149,6 +149,7 @@ prost = { version = "0.12", default-features = false, features = ["std"] } prost-build = { version = "0.12", default-features = false } prost-reflect = { version = "0.14", features = ["serde"], default-features = false } prost-types = { version = "0.12", default-features = false } +rand = { version = "0.8.5", default-features = false, features = ["small_rng"] } serde_json = { version = "1.0.128", default-features = false, features = ["raw_value", "std"] } serde = { version = "1.0.210", default-features = false, features = ["alloc", "derive", "rc"] } tokio = { version = "1.39.2", default-features = false, features = ["full"] } @@ -338,7 +339,7 @@ paste = "1.0.15" percent-encoding = { version = "2.3.1", default-features = false } postgres-openssl = { version = "0.5.0", default-features = false, features = ["runtime"], optional = true } pulsar = { version = "6.3.0", default-features = false, features = ["tokio-runtime", "auth-oauth2", "flate2", "lz4", "snap", "zstd"], optional = true } -rand = { version = "0.8.5", default-features = false, features = ["small_rng"] } +rand.workspace = true rand_distr = { version = "0.4.3", default-features = false } rdkafka = { version = "0.35.0", default-features = false, features = ["curl-static", "tokio", "libz", "ssl", "zstd"], optional = true } redis = { version = "0.24.0", default-features = false, features = ["connection-manager", "tokio-comp", "tokio-native-tls-comp"], optional = true } diff --git a/lib/codecs/Cargo.toml b/lib/codecs/Cargo.toml index 1de29c77ed0f9..00cc2b3b610e7 100644 --- a/lib/codecs/Cargo.toml +++ b/lib/codecs/Cargo.toml @@ -22,6 +22,7 @@ memchr = { version = "2", default-features = false } ordered-float = { version = "4.4.0", default-features = false } prost.workspace = true prost-reflect.workspace = true +rand.workspace = true regex = { version = "1.11.0", default-features = false, features = ["std", "perf"] } serde.workspace = true serde_json.workspace = true diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index 52bf81f474050..746344887fc3b 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -11,7 +11,7 @@ use std::time::Duration; use tokio; use tokio::task::JoinHandle; use tokio_util::codec::Decoder; -use tracing::{debug, info, warn}; +use tracing::{debug, warn}; use vector_config::configurable_component; const GELF_MAGIC: &[u8] = &[0x1e, 0x0f]; @@ -342,6 +342,7 @@ mod tests { use super::*; use bytes::{BufMut, BytesMut}; + use rand::{rngs::SmallRng, seq::SliceRandom, SeedableRng}; use rstest::{fixture, rstest}; use tracing_test::traced_test; @@ -522,6 +523,60 @@ mod tests { assert_eq!(frame, Some(Bytes::from(expected_chunked_message))); } + #[tokio::test] + async fn decode_shuffled_messages() { + let mut rng = SmallRng::seed_from_u64(420); + let total_chunks = 100u8; + let first_message_id = 1u64; + let first_payload = "first payload"; + let second_message_id = 2u64; + let second_payload = "second payload"; + let first_message_chunks = (0..total_chunks).map(|sequence_number| { + create_chunk( + first_message_id, + sequence_number, + total_chunks, + first_payload, + ) + }); + let second_message_chunks = (0..total_chunks).map(|sequence_number| { + create_chunk( + second_message_id, + sequence_number, + total_chunks, + second_payload, + ) + }); + let expected_first_message = first_payload.repeat(total_chunks as usize); + let expected_second_message = second_payload.repeat(total_chunks as usize); + let mut merged_chunks = first_message_chunks + .chain(second_message_chunks) + .collect::>(); + merged_chunks.shuffle(&mut rng); + let mut decoder = ChunkedGelfDecoder::default(); + + let mut count = 0; + let first_retrieved_message = loop { + assert!(count < 2 * total_chunks as usize); + if let Some(message) = decoder.decode_eof(&mut merged_chunks[count]).unwrap() { + break message; + } else { + count = count + 1; + } + }; + let second_retrieved_message = loop { + assert!(count < 2 * total_chunks as usize); + if let Some(message) = decoder.decode_eof(&mut merged_chunks[count]).unwrap() { + break message; + } else { + count = count + 1; + } + }; + + assert_eq!(second_retrieved_message, expected_first_message); + assert_eq!(first_retrieved_message, expected_second_message); + } + #[rstest] #[tokio::test(start_paused = true)] #[traced_test] @@ -615,7 +670,7 @@ mod tests { #[rstest] #[tokio::test] - async fn decode_when_reached_pending_messages_limit( + async fn decode_reached_pending_messages_limit( two_chunks_message: ([BytesMut; 2], String), three_chunks_message: ([BytesMut; 3], String), ) { @@ -666,7 +721,7 @@ mod tests { #[rstest] #[tokio::test] #[traced_test] - async fn decode_when_duplicated_chunk(two_chunks_message: ([BytesMut; 2], String)) { + async fn decode_duplicated_chunk(two_chunks_message: ([BytesMut; 2], String)) { let (mut chunks, _) = two_chunks_message; let mut decoder = ChunkedGelfDecoder::default(); From ef3f0a7a519aa89535044efe7784ffe1b30122e9 Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Fri, 18 Oct 2024 18:41:01 +0200 Subject: [PATCH 50/60] fix: clippy lint --- lib/codecs/src/decoding/framing/chunked_gelf.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index 746344887fc3b..507dbe749908f 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -561,7 +561,7 @@ mod tests { if let Some(message) = decoder.decode_eof(&mut merged_chunks[count]).unwrap() { break message; } else { - count = count + 1; + count += 1; } }; let second_retrieved_message = loop { @@ -569,7 +569,7 @@ mod tests { if let Some(message) = decoder.decode_eof(&mut merged_chunks[count]).unwrap() { break message; } else { - count = count + 1; + count += 1 } }; From 308146b389ef4caf779343347f8b00bfb68096d3 Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Fri, 18 Oct 2024 18:54:55 +0200 Subject: [PATCH 51/60] docs: update components docs --- website/cue/reference/components/sources/base/amqp.cue | 8 +++++--- .../components/sources/base/aws_kinesis_firehose.cue | 8 +++++--- website/cue/reference/components/sources/base/aws_s3.cue | 8 +++++--- website/cue/reference/components/sources/base/aws_sqs.cue | 8 +++++--- .../reference/components/sources/base/datadog_agent.cue | 8 +++++--- .../cue/reference/components/sources/base/demo_logs.cue | 8 +++++--- website/cue/reference/components/sources/base/exec.cue | 8 +++++--- .../reference/components/sources/base/file_descriptor.cue | 8 +++++--- .../cue/reference/components/sources/base/gcp_pubsub.cue | 8 +++++--- .../cue/reference/components/sources/base/heroku_logs.cue | 8 +++++--- website/cue/reference/components/sources/base/http.cue | 8 +++++--- .../cue/reference/components/sources/base/http_client.cue | 8 +++++--- .../cue/reference/components/sources/base/http_server.cue | 8 +++++--- website/cue/reference/components/sources/base/kafka.cue | 8 +++++--- website/cue/reference/components/sources/base/nats.cue | 8 +++++--- website/cue/reference/components/sources/base/pulsar.cue | 8 +++++--- website/cue/reference/components/sources/base/redis.cue | 8 +++++--- website/cue/reference/components/sources/base/socket.cue | 8 +++++--- website/cue/reference/components/sources/base/stdin.cue | 8 +++++--- 19 files changed, 95 insertions(+), 57 deletions(-) diff --git a/website/cue/reference/components/sources/base/amqp.cue b/website/cue/reference/components/sources/base/amqp.cue index a51bc82c7db30..e2f522d60dc1b 100644 --- a/website/cue/reference/components/sources/base/amqp.cue +++ b/website/cue/reference/components/sources/base/amqp.cue @@ -333,15 +333,17 @@ base: components: sources: amqp: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: true + required: false type: object: options: { pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts - dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. + If this option is not set, the decoder does not limit the number of pending messages and the memory usage + of pending messages can grow unbounded. This matches Graylog Server's behavior. """ required: false - type: uint: default: 1000 + type: uint: {} } timeout_secs: { description: """ diff --git a/website/cue/reference/components/sources/base/aws_kinesis_firehose.cue b/website/cue/reference/components/sources/base/aws_kinesis_firehose.cue index 6a8ffc48d6a33..6a0a75191c4c3 100644 --- a/website/cue/reference/components/sources/base/aws_kinesis_firehose.cue +++ b/website/cue/reference/components/sources/base/aws_kinesis_firehose.cue @@ -331,15 +331,17 @@ base: components: sources: aws_kinesis_firehose: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: true + required: false type: object: options: { pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts - dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. + If this option is not set, the decoder does not limit the number of pending messages and the memory usage + of pending messages can grow unbounded. This matches Graylog Server's behavior. """ required: false - type: uint: default: 1000 + type: uint: {} } timeout_secs: { description: """ diff --git a/website/cue/reference/components/sources/base/aws_s3.cue b/website/cue/reference/components/sources/base/aws_s3.cue index 4ae30f7b887aa..d6098e31217f2 100644 --- a/website/cue/reference/components/sources/base/aws_s3.cue +++ b/website/cue/reference/components/sources/base/aws_s3.cue @@ -431,15 +431,17 @@ base: components: sources: aws_s3: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: true + required: false type: object: options: { pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts - dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. + If this option is not set, the decoder does not limit the number of pending messages and the memory usage + of pending messages can grow unbounded. This matches Graylog Server's behavior. """ required: false - type: uint: default: 1000 + type: uint: {} } timeout_secs: { description: """ diff --git a/website/cue/reference/components/sources/base/aws_sqs.cue b/website/cue/reference/components/sources/base/aws_sqs.cue index 893d6d35891df..469efeda0a59b 100644 --- a/website/cue/reference/components/sources/base/aws_sqs.cue +++ b/website/cue/reference/components/sources/base/aws_sqs.cue @@ -435,15 +435,17 @@ base: components: sources: aws_sqs: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: true + required: false type: object: options: { pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts - dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. + If this option is not set, the decoder does not limit the number of pending messages and the memory usage + of pending messages can grow unbounded. This matches Graylog Server's behavior. """ required: false - type: uint: default: 1000 + type: uint: {} } timeout_secs: { description: """ diff --git a/website/cue/reference/components/sources/base/datadog_agent.cue b/website/cue/reference/components/sources/base/datadog_agent.cue index 41b7f95acc58c..d404f6c95b87d 100644 --- a/website/cue/reference/components/sources/base/datadog_agent.cue +++ b/website/cue/reference/components/sources/base/datadog_agent.cue @@ -328,15 +328,17 @@ base: components: sources: datadog_agent: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: true + required: false type: object: options: { pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts - dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. + If this option is not set, the decoder does not limit the number of pending messages and the memory usage + of pending messages can grow unbounded. This matches Graylog Server's behavior. """ required: false - type: uint: default: 1000 + type: uint: {} } timeout_secs: { description: """ diff --git a/website/cue/reference/components/sources/base/demo_logs.cue b/website/cue/reference/components/sources/base/demo_logs.cue index 7f253f5e9dbf5..acd6e79832338 100644 --- a/website/cue/reference/components/sources/base/demo_logs.cue +++ b/website/cue/reference/components/sources/base/demo_logs.cue @@ -324,15 +324,17 @@ base: components: sources: demo_logs: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: true + required: false type: object: options: { pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts - dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. + If this option is not set, the decoder does not limit the number of pending messages and the memory usage + of pending messages can grow unbounded. This matches Graylog Server's behavior. """ required: false - type: uint: default: 1000 + type: uint: {} } timeout_secs: { description: """ diff --git a/website/cue/reference/components/sources/base/exec.cue b/website/cue/reference/components/sources/base/exec.cue index 2452ff71affa5..ee93f511acf81 100644 --- a/website/cue/reference/components/sources/base/exec.cue +++ b/website/cue/reference/components/sources/base/exec.cue @@ -312,15 +312,17 @@ base: components: sources: exec: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: true + required: false type: object: options: { pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts - dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. + If this option is not set, the decoder does not limit the number of pending messages and the memory usage + of pending messages can grow unbounded. This matches Graylog Server's behavior. """ required: false - type: uint: default: 1000 + type: uint: {} } timeout_secs: { description: """ diff --git a/website/cue/reference/components/sources/base/file_descriptor.cue b/website/cue/reference/components/sources/base/file_descriptor.cue index aba7a62e0ec63..7a6cb0078142c 100644 --- a/website/cue/reference/components/sources/base/file_descriptor.cue +++ b/website/cue/reference/components/sources/base/file_descriptor.cue @@ -290,15 +290,17 @@ base: components: sources: file_descriptor: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: true + required: false type: object: options: { pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts - dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. + If this option is not set, the decoder does not limit the number of pending messages and the memory usage + of pending messages can grow unbounded. This matches Graylog Server's behavior. """ required: false - type: uint: default: 1000 + type: uint: {} } timeout_secs: { description: """ diff --git a/website/cue/reference/components/sources/base/gcp_pubsub.cue b/website/cue/reference/components/sources/base/gcp_pubsub.cue index e35ab6ab5e18a..5a8fe6414d22d 100644 --- a/website/cue/reference/components/sources/base/gcp_pubsub.cue +++ b/website/cue/reference/components/sources/base/gcp_pubsub.cue @@ -367,15 +367,17 @@ base: components: sources: gcp_pubsub: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: true + required: false type: object: options: { pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts - dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. + If this option is not set, the decoder does not limit the number of pending messages and the memory usage + of pending messages can grow unbounded. This matches Graylog Server's behavior. """ required: false - type: uint: default: 1000 + type: uint: {} } timeout_secs: { description: """ diff --git a/website/cue/reference/components/sources/base/heroku_logs.cue b/website/cue/reference/components/sources/base/heroku_logs.cue index 191caf5104ced..880fdfddf2667 100644 --- a/website/cue/reference/components/sources/base/heroku_logs.cue +++ b/website/cue/reference/components/sources/base/heroku_logs.cue @@ -325,15 +325,17 @@ base: components: sources: heroku_logs: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: true + required: false type: object: options: { pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts - dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. + If this option is not set, the decoder does not limit the number of pending messages and the memory usage + of pending messages can grow unbounded. This matches Graylog Server's behavior. """ required: false - type: uint: default: 1000 + type: uint: {} } timeout_secs: { description: """ diff --git a/website/cue/reference/components/sources/base/http.cue b/website/cue/reference/components/sources/base/http.cue index 0c5f3f1d34ad4..01b406f41d755 100644 --- a/website/cue/reference/components/sources/base/http.cue +++ b/website/cue/reference/components/sources/base/http.cue @@ -340,15 +340,17 @@ base: components: sources: http: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: true + required: false type: object: options: { pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts - dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. + If this option is not set, the decoder does not limit the number of pending messages and the memory usage + of pending messages can grow unbounded. This matches Graylog Server's behavior. """ required: false - type: uint: default: 1000 + type: uint: {} } timeout_secs: { description: """ diff --git a/website/cue/reference/components/sources/base/http_client.cue b/website/cue/reference/components/sources/base/http_client.cue index 31c2a4351e075..1379f86a2f80f 100644 --- a/website/cue/reference/components/sources/base/http_client.cue +++ b/website/cue/reference/components/sources/base/http_client.cue @@ -328,15 +328,17 @@ base: components: sources: http_client: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: true + required: false type: object: options: { pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts - dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. + If this option is not set, the decoder does not limit the number of pending messages and the memory usage + of pending messages can grow unbounded. This matches Graylog Server's behavior. """ required: false - type: uint: default: 1000 + type: uint: {} } timeout_secs: { description: """ diff --git a/website/cue/reference/components/sources/base/http_server.cue b/website/cue/reference/components/sources/base/http_server.cue index 58a448338e33d..0d3305d5437df 100644 --- a/website/cue/reference/components/sources/base/http_server.cue +++ b/website/cue/reference/components/sources/base/http_server.cue @@ -340,15 +340,17 @@ base: components: sources: http_server: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: true + required: false type: object: options: { pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts - dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. + If this option is not set, the decoder does not limit the number of pending messages and the memory usage + of pending messages can grow unbounded. This matches Graylog Server's behavior. """ required: false - type: uint: default: 1000 + type: uint: {} } timeout_secs: { description: """ diff --git a/website/cue/reference/components/sources/base/kafka.cue b/website/cue/reference/components/sources/base/kafka.cue index 5047155067ebc..c80487b5f63cf 100644 --- a/website/cue/reference/components/sources/base/kafka.cue +++ b/website/cue/reference/components/sources/base/kafka.cue @@ -361,15 +361,17 @@ base: components: sources: kafka: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: true + required: false type: object: options: { pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts - dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. + If this option is not set, the decoder does not limit the number of pending messages and the memory usage + of pending messages can grow unbounded. This matches Graylog Server's behavior. """ required: false - type: uint: default: 1000 + type: uint: {} } timeout_secs: { description: """ diff --git a/website/cue/reference/components/sources/base/nats.cue b/website/cue/reference/components/sources/base/nats.cue index a37f40f9c2376..dd31bdd690fd8 100644 --- a/website/cue/reference/components/sources/base/nats.cue +++ b/website/cue/reference/components/sources/base/nats.cue @@ -380,15 +380,17 @@ base: components: sources: nats: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: true + required: false type: object: options: { pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts - dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. + If this option is not set, the decoder does not limit the number of pending messages and the memory usage + of pending messages can grow unbounded. This matches Graylog Server's behavior. """ required: false - type: uint: default: 1000 + type: uint: {} } timeout_secs: { description: """ diff --git a/website/cue/reference/components/sources/base/pulsar.cue b/website/cue/reference/components/sources/base/pulsar.cue index d7bd84a2ad6f5..f5911f27a8878 100644 --- a/website/cue/reference/components/sources/base/pulsar.cue +++ b/website/cue/reference/components/sources/base/pulsar.cue @@ -391,15 +391,17 @@ base: components: sources: pulsar: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: true + required: false type: object: options: { pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts - dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. + If this option is not set, the decoder does not limit the number of pending messages and the memory usage + of pending messages can grow unbounded. This matches Graylog Server's behavior. """ required: false - type: uint: default: 1000 + type: uint: {} } timeout_secs: { description: """ diff --git a/website/cue/reference/components/sources/base/redis.cue b/website/cue/reference/components/sources/base/redis.cue index fecbec8d88eb0..8f6a8ec3199a6 100644 --- a/website/cue/reference/components/sources/base/redis.cue +++ b/website/cue/reference/components/sources/base/redis.cue @@ -298,15 +298,17 @@ base: components: sources: redis: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: true + required: false type: object: options: { pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts - dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. + If this option is not set, the decoder does not limit the number of pending messages and the memory usage + of pending messages can grow unbounded. This matches Graylog Server's behavior. """ required: false - type: uint: default: 1000 + type: uint: {} } timeout_secs: { description: """ diff --git a/website/cue/reference/components/sources/base/socket.cue b/website/cue/reference/components/sources/base/socket.cue index 0640fb66d4e46..df107f45c7c29 100644 --- a/website/cue/reference/components/sources/base/socket.cue +++ b/website/cue/reference/components/sources/base/socket.cue @@ -300,15 +300,17 @@ base: components: sources: socket: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: true + required: false type: object: options: { pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts - dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. + If this option is not set, the decoder does not limit the number of pending messages and the memory usage + of pending messages can grow unbounded. This matches Graylog Server's behavior. """ required: false - type: uint: default: 1000 + type: uint: {} } timeout_secs: { description: """ diff --git a/website/cue/reference/components/sources/base/stdin.cue b/website/cue/reference/components/sources/base/stdin.cue index bae27713e72d1..cab46ba037fff 100644 --- a/website/cue/reference/components/sources/base/stdin.cue +++ b/website/cue/reference/components/sources/base/stdin.cue @@ -283,15 +283,17 @@ base: components: sources: stdin: configuration: { chunked_gelf: { description: "Options for the chunked GELF decoder." relevant_when: "method = \"chunked_gelf\"" - required: true + required: false type: object: options: { pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts - dropping chunks of new messages. This limit ensures the memory usage of the decoder's state is bounded. + dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. + If this option is not set, the decoder does not limit the number of pending messages and the memory usage + of pending messages can grow unbounded. This matches Graylog Server's behavior. """ required: false - type: uint: default: 1000 + type: uint: {} } timeout_secs: { description: """ From 16f82289055de60d367c80ee90dd0c239fee72ff Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Fri, 18 Oct 2024 18:59:07 +0200 Subject: [PATCH 52/60] docs: improve chunked gelf decoder option documentation --- lib/codecs/src/decoding/framing/chunked_gelf.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index 507dbe749908f..0e75def442325 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -47,7 +47,7 @@ impl ChunkedGelfDecoderConfig { #[derivative(Default)] pub struct ChunkedGelfDecoderOptions { /// The timeout, in seconds, for a message to be fully received. If the timeout is reached, the - /// decoder drops all the received chunks of the incomplete message and starts over. + /// decoder drops all the received chunks of the timed out message. #[serde(default = "default_timeout_secs")] #[derivative(Default(value = "default_timeout_secs()"))] pub timeout_secs: f64, From e3f824578c98a6d95e0e906b839f5d81393231b2 Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Fri, 18 Oct 2024 19:00:02 +0200 Subject: [PATCH 53/60] docs: improve chunked gelf decoder option documentation --- lib/codecs/src/decoding/framing/chunked_gelf.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index 0e75def442325..bca093a29d6a2 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -55,7 +55,7 @@ pub struct ChunkedGelfDecoderOptions { /// The maximum number of pending incomplete messages. If this limit is reached, the decoder starts /// dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. /// If this option is not set, the decoder does not limit the number of pending messages and the memory usage - /// of pending messages can grow unbounded. This matches Graylog Server's behavior. + /// of its messages buffer can grow unbounded. This matches Graylog Server's behavior. #[serde(default, skip_serializing_if = "vector_core::serde::is_default")] pub pending_messages_limit: Option, } From cecfa8c970115d6682e851c716c20a43ce462a6d Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Fri, 18 Oct 2024 19:04:02 +0200 Subject: [PATCH 54/60] docs: update components docs --- website/cue/reference/components/sources/base/amqp.cue | 4 ++-- .../components/sources/base/aws_kinesis_firehose.cue | 4 ++-- website/cue/reference/components/sources/base/aws_s3.cue | 4 ++-- website/cue/reference/components/sources/base/aws_sqs.cue | 4 ++-- .../cue/reference/components/sources/base/datadog_agent.cue | 4 ++-- website/cue/reference/components/sources/base/demo_logs.cue | 4 ++-- website/cue/reference/components/sources/base/exec.cue | 4 ++-- .../cue/reference/components/sources/base/file_descriptor.cue | 4 ++-- website/cue/reference/components/sources/base/gcp_pubsub.cue | 4 ++-- website/cue/reference/components/sources/base/heroku_logs.cue | 4 ++-- website/cue/reference/components/sources/base/http.cue | 4 ++-- website/cue/reference/components/sources/base/http_client.cue | 4 ++-- website/cue/reference/components/sources/base/http_server.cue | 4 ++-- website/cue/reference/components/sources/base/kafka.cue | 4 ++-- website/cue/reference/components/sources/base/nats.cue | 4 ++-- website/cue/reference/components/sources/base/pulsar.cue | 4 ++-- website/cue/reference/components/sources/base/redis.cue | 4 ++-- website/cue/reference/components/sources/base/socket.cue | 4 ++-- website/cue/reference/components/sources/base/stdin.cue | 4 ++-- 19 files changed, 38 insertions(+), 38 deletions(-) diff --git a/website/cue/reference/components/sources/base/amqp.cue b/website/cue/reference/components/sources/base/amqp.cue index e2f522d60dc1b..45d22c6e180bf 100644 --- a/website/cue/reference/components/sources/base/amqp.cue +++ b/website/cue/reference/components/sources/base/amqp.cue @@ -340,7 +340,7 @@ base: components: sources: amqp: configuration: { The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. If this option is not set, the decoder does not limit the number of pending messages and the memory usage - of pending messages can grow unbounded. This matches Graylog Server's behavior. + of its messages buffer can grow unbounded. This matches Graylog Server's behavior. """ required: false type: uint: {} @@ -348,7 +348,7 @@ base: components: sources: amqp: configuration: { timeout_secs: { description: """ The timeout, in seconds, for a message to be fully received. If the timeout is reached, the - decoder drops all the received chunks of the incomplete message and starts over. + decoder drops all the received chunks of the timed out message. """ required: false type: float: default: 5.0 diff --git a/website/cue/reference/components/sources/base/aws_kinesis_firehose.cue b/website/cue/reference/components/sources/base/aws_kinesis_firehose.cue index 6a0a75191c4c3..096f9b8694fd1 100644 --- a/website/cue/reference/components/sources/base/aws_kinesis_firehose.cue +++ b/website/cue/reference/components/sources/base/aws_kinesis_firehose.cue @@ -338,7 +338,7 @@ base: components: sources: aws_kinesis_firehose: configuration: { The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. If this option is not set, the decoder does not limit the number of pending messages and the memory usage - of pending messages can grow unbounded. This matches Graylog Server's behavior. + of its messages buffer can grow unbounded. This matches Graylog Server's behavior. """ required: false type: uint: {} @@ -346,7 +346,7 @@ base: components: sources: aws_kinesis_firehose: configuration: { timeout_secs: { description: """ The timeout, in seconds, for a message to be fully received. If the timeout is reached, the - decoder drops all the received chunks of the incomplete message and starts over. + decoder drops all the received chunks of the timed out message. """ required: false type: float: default: 5.0 diff --git a/website/cue/reference/components/sources/base/aws_s3.cue b/website/cue/reference/components/sources/base/aws_s3.cue index d6098e31217f2..3e200fa336a94 100644 --- a/website/cue/reference/components/sources/base/aws_s3.cue +++ b/website/cue/reference/components/sources/base/aws_s3.cue @@ -438,7 +438,7 @@ base: components: sources: aws_s3: configuration: { The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. If this option is not set, the decoder does not limit the number of pending messages and the memory usage - of pending messages can grow unbounded. This matches Graylog Server's behavior. + of its messages buffer can grow unbounded. This matches Graylog Server's behavior. """ required: false type: uint: {} @@ -446,7 +446,7 @@ base: components: sources: aws_s3: configuration: { timeout_secs: { description: """ The timeout, in seconds, for a message to be fully received. If the timeout is reached, the - decoder drops all the received chunks of the incomplete message and starts over. + decoder drops all the received chunks of the timed out message. """ required: false type: float: default: 5.0 diff --git a/website/cue/reference/components/sources/base/aws_sqs.cue b/website/cue/reference/components/sources/base/aws_sqs.cue index 469efeda0a59b..f7252b5dcd2d6 100644 --- a/website/cue/reference/components/sources/base/aws_sqs.cue +++ b/website/cue/reference/components/sources/base/aws_sqs.cue @@ -442,7 +442,7 @@ base: components: sources: aws_sqs: configuration: { The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. If this option is not set, the decoder does not limit the number of pending messages and the memory usage - of pending messages can grow unbounded. This matches Graylog Server's behavior. + of its messages buffer can grow unbounded. This matches Graylog Server's behavior. """ required: false type: uint: {} @@ -450,7 +450,7 @@ base: components: sources: aws_sqs: configuration: { timeout_secs: { description: """ The timeout, in seconds, for a message to be fully received. If the timeout is reached, the - decoder drops all the received chunks of the incomplete message and starts over. + decoder drops all the received chunks of the timed out message. """ required: false type: float: default: 5.0 diff --git a/website/cue/reference/components/sources/base/datadog_agent.cue b/website/cue/reference/components/sources/base/datadog_agent.cue index d404f6c95b87d..4d4eafee8d8b4 100644 --- a/website/cue/reference/components/sources/base/datadog_agent.cue +++ b/website/cue/reference/components/sources/base/datadog_agent.cue @@ -335,7 +335,7 @@ base: components: sources: datadog_agent: configuration: { The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. If this option is not set, the decoder does not limit the number of pending messages and the memory usage - of pending messages can grow unbounded. This matches Graylog Server's behavior. + of its messages buffer can grow unbounded. This matches Graylog Server's behavior. """ required: false type: uint: {} @@ -343,7 +343,7 @@ base: components: sources: datadog_agent: configuration: { timeout_secs: { description: """ The timeout, in seconds, for a message to be fully received. If the timeout is reached, the - decoder drops all the received chunks of the incomplete message and starts over. + decoder drops all the received chunks of the timed out message. """ required: false type: float: default: 5.0 diff --git a/website/cue/reference/components/sources/base/demo_logs.cue b/website/cue/reference/components/sources/base/demo_logs.cue index acd6e79832338..f4b9c324f2311 100644 --- a/website/cue/reference/components/sources/base/demo_logs.cue +++ b/website/cue/reference/components/sources/base/demo_logs.cue @@ -331,7 +331,7 @@ base: components: sources: demo_logs: configuration: { The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. If this option is not set, the decoder does not limit the number of pending messages and the memory usage - of pending messages can grow unbounded. This matches Graylog Server's behavior. + of its messages buffer can grow unbounded. This matches Graylog Server's behavior. """ required: false type: uint: {} @@ -339,7 +339,7 @@ base: components: sources: demo_logs: configuration: { timeout_secs: { description: """ The timeout, in seconds, for a message to be fully received. If the timeout is reached, the - decoder drops all the received chunks of the incomplete message and starts over. + decoder drops all the received chunks of the timed out message. """ required: false type: float: default: 5.0 diff --git a/website/cue/reference/components/sources/base/exec.cue b/website/cue/reference/components/sources/base/exec.cue index ee93f511acf81..59be6415a0c1c 100644 --- a/website/cue/reference/components/sources/base/exec.cue +++ b/website/cue/reference/components/sources/base/exec.cue @@ -319,7 +319,7 @@ base: components: sources: exec: configuration: { The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. If this option is not set, the decoder does not limit the number of pending messages and the memory usage - of pending messages can grow unbounded. This matches Graylog Server's behavior. + of its messages buffer can grow unbounded. This matches Graylog Server's behavior. """ required: false type: uint: {} @@ -327,7 +327,7 @@ base: components: sources: exec: configuration: { timeout_secs: { description: """ The timeout, in seconds, for a message to be fully received. If the timeout is reached, the - decoder drops all the received chunks of the incomplete message and starts over. + decoder drops all the received chunks of the timed out message. """ required: false type: float: default: 5.0 diff --git a/website/cue/reference/components/sources/base/file_descriptor.cue b/website/cue/reference/components/sources/base/file_descriptor.cue index 7a6cb0078142c..d20353ce4d14e 100644 --- a/website/cue/reference/components/sources/base/file_descriptor.cue +++ b/website/cue/reference/components/sources/base/file_descriptor.cue @@ -297,7 +297,7 @@ base: components: sources: file_descriptor: configuration: { The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. If this option is not set, the decoder does not limit the number of pending messages and the memory usage - of pending messages can grow unbounded. This matches Graylog Server's behavior. + of its messages buffer can grow unbounded. This matches Graylog Server's behavior. """ required: false type: uint: {} @@ -305,7 +305,7 @@ base: components: sources: file_descriptor: configuration: { timeout_secs: { description: """ The timeout, in seconds, for a message to be fully received. If the timeout is reached, the - decoder drops all the received chunks of the incomplete message and starts over. + decoder drops all the received chunks of the timed out message. """ required: false type: float: default: 5.0 diff --git a/website/cue/reference/components/sources/base/gcp_pubsub.cue b/website/cue/reference/components/sources/base/gcp_pubsub.cue index 5a8fe6414d22d..d972a14186ea8 100644 --- a/website/cue/reference/components/sources/base/gcp_pubsub.cue +++ b/website/cue/reference/components/sources/base/gcp_pubsub.cue @@ -374,7 +374,7 @@ base: components: sources: gcp_pubsub: configuration: { The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. If this option is not set, the decoder does not limit the number of pending messages and the memory usage - of pending messages can grow unbounded. This matches Graylog Server's behavior. + of its messages buffer can grow unbounded. This matches Graylog Server's behavior. """ required: false type: uint: {} @@ -382,7 +382,7 @@ base: components: sources: gcp_pubsub: configuration: { timeout_secs: { description: """ The timeout, in seconds, for a message to be fully received. If the timeout is reached, the - decoder drops all the received chunks of the incomplete message and starts over. + decoder drops all the received chunks of the timed out message. """ required: false type: float: default: 5.0 diff --git a/website/cue/reference/components/sources/base/heroku_logs.cue b/website/cue/reference/components/sources/base/heroku_logs.cue index 880fdfddf2667..730e4e9d868a8 100644 --- a/website/cue/reference/components/sources/base/heroku_logs.cue +++ b/website/cue/reference/components/sources/base/heroku_logs.cue @@ -332,7 +332,7 @@ base: components: sources: heroku_logs: configuration: { The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. If this option is not set, the decoder does not limit the number of pending messages and the memory usage - of pending messages can grow unbounded. This matches Graylog Server's behavior. + of its messages buffer can grow unbounded. This matches Graylog Server's behavior. """ required: false type: uint: {} @@ -340,7 +340,7 @@ base: components: sources: heroku_logs: configuration: { timeout_secs: { description: """ The timeout, in seconds, for a message to be fully received. If the timeout is reached, the - decoder drops all the received chunks of the incomplete message and starts over. + decoder drops all the received chunks of the timed out message. """ required: false type: float: default: 5.0 diff --git a/website/cue/reference/components/sources/base/http.cue b/website/cue/reference/components/sources/base/http.cue index 01b406f41d755..96e7594a7a6bd 100644 --- a/website/cue/reference/components/sources/base/http.cue +++ b/website/cue/reference/components/sources/base/http.cue @@ -347,7 +347,7 @@ base: components: sources: http: configuration: { The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. If this option is not set, the decoder does not limit the number of pending messages and the memory usage - of pending messages can grow unbounded. This matches Graylog Server's behavior. + of its messages buffer can grow unbounded. This matches Graylog Server's behavior. """ required: false type: uint: {} @@ -355,7 +355,7 @@ base: components: sources: http: configuration: { timeout_secs: { description: """ The timeout, in seconds, for a message to be fully received. If the timeout is reached, the - decoder drops all the received chunks of the incomplete message and starts over. + decoder drops all the received chunks of the timed out message. """ required: false type: float: default: 5.0 diff --git a/website/cue/reference/components/sources/base/http_client.cue b/website/cue/reference/components/sources/base/http_client.cue index 1379f86a2f80f..7301a345fe774 100644 --- a/website/cue/reference/components/sources/base/http_client.cue +++ b/website/cue/reference/components/sources/base/http_client.cue @@ -335,7 +335,7 @@ base: components: sources: http_client: configuration: { The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. If this option is not set, the decoder does not limit the number of pending messages and the memory usage - of pending messages can grow unbounded. This matches Graylog Server's behavior. + of its messages buffer can grow unbounded. This matches Graylog Server's behavior. """ required: false type: uint: {} @@ -343,7 +343,7 @@ base: components: sources: http_client: configuration: { timeout_secs: { description: """ The timeout, in seconds, for a message to be fully received. If the timeout is reached, the - decoder drops all the received chunks of the incomplete message and starts over. + decoder drops all the received chunks of the timed out message. """ required: false type: float: default: 5.0 diff --git a/website/cue/reference/components/sources/base/http_server.cue b/website/cue/reference/components/sources/base/http_server.cue index 0d3305d5437df..1437fd60b4aa4 100644 --- a/website/cue/reference/components/sources/base/http_server.cue +++ b/website/cue/reference/components/sources/base/http_server.cue @@ -347,7 +347,7 @@ base: components: sources: http_server: configuration: { The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. If this option is not set, the decoder does not limit the number of pending messages and the memory usage - of pending messages can grow unbounded. This matches Graylog Server's behavior. + of its messages buffer can grow unbounded. This matches Graylog Server's behavior. """ required: false type: uint: {} @@ -355,7 +355,7 @@ base: components: sources: http_server: configuration: { timeout_secs: { description: """ The timeout, in seconds, for a message to be fully received. If the timeout is reached, the - decoder drops all the received chunks of the incomplete message and starts over. + decoder drops all the received chunks of the timed out message. """ required: false type: float: default: 5.0 diff --git a/website/cue/reference/components/sources/base/kafka.cue b/website/cue/reference/components/sources/base/kafka.cue index c80487b5f63cf..a1d9fdc0a4062 100644 --- a/website/cue/reference/components/sources/base/kafka.cue +++ b/website/cue/reference/components/sources/base/kafka.cue @@ -368,7 +368,7 @@ base: components: sources: kafka: configuration: { The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. If this option is not set, the decoder does not limit the number of pending messages and the memory usage - of pending messages can grow unbounded. This matches Graylog Server's behavior. + of its messages buffer can grow unbounded. This matches Graylog Server's behavior. """ required: false type: uint: {} @@ -376,7 +376,7 @@ base: components: sources: kafka: configuration: { timeout_secs: { description: """ The timeout, in seconds, for a message to be fully received. If the timeout is reached, the - decoder drops all the received chunks of the incomplete message and starts over. + decoder drops all the received chunks of the timed out message. """ required: false type: float: default: 5.0 diff --git a/website/cue/reference/components/sources/base/nats.cue b/website/cue/reference/components/sources/base/nats.cue index dd31bdd690fd8..7984d06b8a9f7 100644 --- a/website/cue/reference/components/sources/base/nats.cue +++ b/website/cue/reference/components/sources/base/nats.cue @@ -387,7 +387,7 @@ base: components: sources: nats: configuration: { The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. If this option is not set, the decoder does not limit the number of pending messages and the memory usage - of pending messages can grow unbounded. This matches Graylog Server's behavior. + of its messages buffer can grow unbounded. This matches Graylog Server's behavior. """ required: false type: uint: {} @@ -395,7 +395,7 @@ base: components: sources: nats: configuration: { timeout_secs: { description: """ The timeout, in seconds, for a message to be fully received. If the timeout is reached, the - decoder drops all the received chunks of the incomplete message and starts over. + decoder drops all the received chunks of the timed out message. """ required: false type: float: default: 5.0 diff --git a/website/cue/reference/components/sources/base/pulsar.cue b/website/cue/reference/components/sources/base/pulsar.cue index f5911f27a8878..ae357cdadf506 100644 --- a/website/cue/reference/components/sources/base/pulsar.cue +++ b/website/cue/reference/components/sources/base/pulsar.cue @@ -398,7 +398,7 @@ base: components: sources: pulsar: configuration: { The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. If this option is not set, the decoder does not limit the number of pending messages and the memory usage - of pending messages can grow unbounded. This matches Graylog Server's behavior. + of its messages buffer can grow unbounded. This matches Graylog Server's behavior. """ required: false type: uint: {} @@ -406,7 +406,7 @@ base: components: sources: pulsar: configuration: { timeout_secs: { description: """ The timeout, in seconds, for a message to be fully received. If the timeout is reached, the - decoder drops all the received chunks of the incomplete message and starts over. + decoder drops all the received chunks of the timed out message. """ required: false type: float: default: 5.0 diff --git a/website/cue/reference/components/sources/base/redis.cue b/website/cue/reference/components/sources/base/redis.cue index 8f6a8ec3199a6..9bb6b91aee72b 100644 --- a/website/cue/reference/components/sources/base/redis.cue +++ b/website/cue/reference/components/sources/base/redis.cue @@ -305,7 +305,7 @@ base: components: sources: redis: configuration: { The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. If this option is not set, the decoder does not limit the number of pending messages and the memory usage - of pending messages can grow unbounded. This matches Graylog Server's behavior. + of its messages buffer can grow unbounded. This matches Graylog Server's behavior. """ required: false type: uint: {} @@ -313,7 +313,7 @@ base: components: sources: redis: configuration: { timeout_secs: { description: """ The timeout, in seconds, for a message to be fully received. If the timeout is reached, the - decoder drops all the received chunks of the incomplete message and starts over. + decoder drops all the received chunks of the timed out message. """ required: false type: float: default: 5.0 diff --git a/website/cue/reference/components/sources/base/socket.cue b/website/cue/reference/components/sources/base/socket.cue index df107f45c7c29..2ceff30d4af34 100644 --- a/website/cue/reference/components/sources/base/socket.cue +++ b/website/cue/reference/components/sources/base/socket.cue @@ -307,7 +307,7 @@ base: components: sources: socket: configuration: { The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. If this option is not set, the decoder does not limit the number of pending messages and the memory usage - of pending messages can grow unbounded. This matches Graylog Server's behavior. + of its messages buffer can grow unbounded. This matches Graylog Server's behavior. """ required: false type: uint: {} @@ -315,7 +315,7 @@ base: components: sources: socket: configuration: { timeout_secs: { description: """ The timeout, in seconds, for a message to be fully received. If the timeout is reached, the - decoder drops all the received chunks of the incomplete message and starts over. + decoder drops all the received chunks of the timed out message. """ required: false type: float: default: 5.0 diff --git a/website/cue/reference/components/sources/base/stdin.cue b/website/cue/reference/components/sources/base/stdin.cue index cab46ba037fff..e9f354f3b58ac 100644 --- a/website/cue/reference/components/sources/base/stdin.cue +++ b/website/cue/reference/components/sources/base/stdin.cue @@ -290,7 +290,7 @@ base: components: sources: stdin: configuration: { The maximum number of pending incomplete messages. If this limit is reached, the decoder starts dropping chunks of new messages, ensuring the memory usage of the decoder's state is bounded. If this option is not set, the decoder does not limit the number of pending messages and the memory usage - of pending messages can grow unbounded. This matches Graylog Server's behavior. + of its messages buffer can grow unbounded. This matches Graylog Server's behavior. """ required: false type: uint: {} @@ -298,7 +298,7 @@ base: components: sources: stdin: configuration: { timeout_secs: { description: """ The timeout, in seconds, for a message to be fully received. If the timeout is reached, the - decoder drops all the received chunks of the incomplete message and starts over. + decoder drops all the received chunks of the timed out message. """ required: false type: float: default: 5.0 From 2139ae4a9c41b6674dc56851e4a9a955d9345601 Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Fri, 25 Oct 2024 19:11:33 +0200 Subject: [PATCH 55/60] feat: Implement max_chunk_length and max_message_length --- .../src/decoding/framing/chunked_gelf.rs | 186 ++++++++++++++---- 1 file changed, 153 insertions(+), 33 deletions(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index bca093a29d6a2..cc9ba50581fbf 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -37,6 +37,8 @@ impl ChunkedGelfDecoderConfig { ChunkedGelfDecoder::new( self.chunked_gelf.timeout_secs, self.chunked_gelf.pending_messages_limit, + self.chunked_gelf.max_chunk_length, + self.chunked_gelf.max_message_length, ) } } @@ -58,6 +60,27 @@ pub struct ChunkedGelfDecoderOptions { /// of its messages buffer can grow unbounded. This matches Graylog Server's behavior. #[serde(default, skip_serializing_if = "vector_core::serde::is_default")] pub pending_messages_limit: Option, + + /// The maximum length of a single GELF chunk, in bytes. Chunks longer than this length will + /// be dropped. If this option is not set, the decoder does not limit the length of chunks and + /// the per-chunk memory is unbounded. + /// + /// This limit takes only into account the chunk's payload and the GELF header bytes are excluded from the calculation. + #[serde(skip_serializing_if = "vector_core::serde::is_default")] + pub max_chunk_length: Option, + + /// The maximum length of a single GELF message, in bytes. Messages longer than this length will + /// be dropped. If this option is not set, the decoder does not limit the length of messages and + /// the per-message memory is unbounded. + /// + /// Note that a message can be composed of multiple chunks and this limit is applied to the whole + /// message, not to individual chunks. This length should always be greater than the `max_chunk_length`. + /// This option is useful to limit the memory usage of the decoders's chunk buffer. + /// + /// This limit takes only into account the message's payload and the GELF header bytes are excluded from the calculation. + /// The message's payload is the concatenation of all the chunks' payloads. + #[serde(skip_serializing_if = "vector_core::serde::is_default")] + pub max_message_length: Option, } #[derive(Debug)] @@ -65,6 +88,7 @@ struct MessageState { total_chunks: u8, chunks: [Bytes; GELF_MAX_TOTAL_CHUNKS as usize], chunks_bitmap: u128, + current_message_length: usize, timeout_task: JoinHandle<()>, } @@ -74,6 +98,7 @@ impl MessageState { total_chunks, chunks: [const { Bytes::new() }; GELF_MAX_TOTAL_CHUNKS as usize], chunks_bitmap: 0, + current_message_length: 0, timeout_task, } } @@ -85,15 +110,20 @@ impl MessageState { fn add_chunk(&mut self, sequence_number: u8, chunk: Bytes) { let chunk_bitmap_id = 1 << sequence_number; - self.chunks[sequence_number as usize] = chunk; self.chunks_bitmap |= chunk_bitmap_id; + self.current_message_length += chunk.remaining(); + self.chunks[sequence_number as usize] = chunk; } fn is_complete(&self) -> bool { self.chunks_bitmap.count_ones() == self.total_chunks as u32 } - fn retrieve_message(&mut self) -> Option { + fn current_message_length(&self) -> usize { + self.current_message_length + } + + fn retrieve_message(&self) -> Option { if self.is_complete() { self.timeout_task.abort(); let chunks = &self.chunks[0..self.total_chunks as usize]; @@ -108,35 +138,49 @@ impl MessageState { } } -#[derive(Debug, Snafu)] +#[derive(Debug, Snafu, PartialEq, Eq)] pub enum ChunkedGelfDecoderError { - #[snafu(display("Invalid chunk header with less than 10 bytes: {header:?}"))] + #[snafu(display("Invalid chunk header with less than 10 bytes: 0x{header:0x}"))] InvalidChunkHeader { header: Bytes }, - #[snafu(display("Invalid total chunks value {total_chunks} for message with id {message_id} and sequence number {sequence_number}"))] + #[snafu(display("Received chunk with message id {message_id} and sequence number {sequence_number} has an invalid total chunks value of {total_chunks}. It must be between 1 and {GELF_MAX_TOTAL_CHUNKS}."))] InvalidTotalChunks { message_id: u64, sequence_number: u8, total_chunks: u8, }, - #[snafu(display("Sequence number {sequence_number} is greater than the total chunks value {total_chunks} for message with id {message_id}"))] + #[snafu(display("Received chunk with message id {message_id} and sequence number {sequence_number} has a sequence number greater than its total chunks value of {total_chunks}"))] InvalidSequenceNumber { message_id: u64, sequence_number: u8, total_chunks: u8, }, - #[snafu(display("Pending messages limit of {pending_messages_limit} reached while processing message with id {message_id} and sequence number {sequence_number}"))] + #[snafu(display("Pending messages limit of {pending_messages_limit} reached while processing chunk with message id {message_id} and sequence number {sequence_number}"))] PendingMessagesLimitReached { message_id: u64, sequence_number: u8, pending_messages_limit: usize, }, - #[snafu(display("Received message with id {message_id} and sequence number {sequence_number} has different total chunks values: original total chunks value {original_total_chunks}, received total chunks value {received_total_chunks}"))] + #[snafu(display("Received chunk with message id {message_id} and sequence number {sequence_number} has different total chunks values: original total chunks value is {original_total_chunks} and received total chunks value is {received_total_chunks}"))] TotalChunksMismatch { message_id: u64, sequence_number: u8, original_total_chunks: u8, received_total_chunks: u8, }, + #[snafu(display("Received chunk with message id {message_id} and sequence number {sequence_number} has exceeded the maximum chunk length: got {chunk_length} bytes and max chunk length is {max_chunk_length} bytes"))] + MaxChunkLengthExceeded { + message_id: u64, + sequence_number: u8, + chunk_length: usize, + max_chunk_length: usize, + }, + #[snafu(display("Message with id {message_id} has exceeded the maximum message length and it will be dropped: got {message_length} bytes and max message length is {max_message_length} bytes. Discarding all buffered chunks of that message"))] + MaxMessageLengthExceeded { + message_id: u64, + sequence_number: u8, + message_length: usize, + max_message_length: usize, + }, } impl StreamDecodingError for ChunkedGelfDecoderError { @@ -164,16 +208,25 @@ pub struct ChunkedGelfDecoder { state: Arc>>, timeout: Duration, pending_messages_limit: Option, + max_chunk_length: Option, + max_message_length: Option, } impl ChunkedGelfDecoder { /// Creates a new `ChunkedGelfDecoder`. - pub fn new(timeout_secs: f64, pending_messages_limit: Option) -> Self { + pub fn new( + timeout_secs: f64, + pending_messages_limit: Option, + max_chunk_length: Option, + max_message_length: Option, + ) -> Self { Self { bytes_decoder: BytesDecoder::new(), state: Arc::new(Mutex::new(HashMap::new())), timeout: Duration::from_secs_f64(timeout_secs), pending_messages_limit, + max_chunk_length, + max_message_length, } } @@ -276,8 +329,34 @@ impl ChunkedGelfDecoder { return Ok(None); } + if let Some(max_chunk_length) = self.max_chunk_length { + let chunk_length = chunk.remaining(); + ensure!( + chunk_length <= max_chunk_length, + MaxChunkLengthExceededSnafu { + message_id, + sequence_number, + chunk_length, + max_chunk_length + } + ); + } + message_state.add_chunk(sequence_number, chunk); + if let Some(max_message_length) = self.max_message_length { + let message_length = message_state.current_message_length(); + if message_length > max_message_length { + state_lock.remove(&message_id); + return Err(ChunkedGelfDecoderError::MaxMessageLengthExceeded { + message_id, + sequence_number, + message_length, + max_message_length, + }); + } + } + if let Some(message) = message_state.retrieve_message() { state_lock.remove(&message_id); Ok(Some(message)) @@ -304,7 +383,7 @@ impl ChunkedGelfDecoder { impl Default for ChunkedGelfDecoder { fn default() -> Self { - Self::new(DEFAULT_TIMEOUT_SECS, None) + Self::new(DEFAULT_TIMEOUT_SECS, None, None, None) } } @@ -626,10 +705,12 @@ mod tests { let error = frame.expect_err("Expected an error"); let downcasted_error = downcast_framing_error(&error); - assert!(matches!( - downcasted_error, - ChunkedGelfDecoderError::InvalidChunkHeader { .. } - )); + assert_eq!( + *downcasted_error, + ChunkedGelfDecoderError::InvalidChunkHeader { + header: Bytes::from_static(&[0x12, 0x34]) + } + ); } #[tokio::test] @@ -644,10 +725,14 @@ mod tests { let frame = decoder.decode_eof(&mut chunk); let error = frame.expect_err("Expected an error"); let downcasted_error = downcast_framing_error(&error); - assert!(matches!( - downcasted_error, - ChunkedGelfDecoderError::InvalidTotalChunks { .. } - )); + assert_eq!( + *downcasted_error, + ChunkedGelfDecoderError::InvalidTotalChunks { + message_id: 1, + sequence_number: 1, + total_chunks: 129, + } + ); } #[tokio::test] @@ -662,10 +747,14 @@ mod tests { let frame = decoder.decode_eof(&mut chunk); let error = frame.expect_err("Expected an error"); let downcasted_error = downcast_framing_error(&error); - assert!(matches!( - downcasted_error, - ChunkedGelfDecoderError::InvalidSequenceNumber { .. } - )); + assert_eq!( + *downcasted_error, + ChunkedGelfDecoderError::InvalidSequenceNumber { + message_id: 1, + sequence_number: 3, + total_chunks: 2, + }, + ); } #[rstest] @@ -674,11 +763,9 @@ mod tests { two_chunks_message: ([BytesMut; 2], String), three_chunks_message: ([BytesMut; 3], String), ) { - let pending_messages_limit = 1; let (mut two_chunks, _) = two_chunks_message; let (mut three_chunks, _) = three_chunks_message; - let mut decoder = - ChunkedGelfDecoder::new(DEFAULT_TIMEOUT_SECS, Some(pending_messages_limit)); + let mut decoder = ChunkedGelfDecoder::new(DEFAULT_TIMEOUT_SECS, Some(1), None, None); let frame = decoder.decode_eof(&mut two_chunks[0]).unwrap(); assert!(frame.is_none()); @@ -687,10 +774,14 @@ mod tests { let frame = decoder.decode_eof(&mut three_chunks[0]); let error = frame.expect_err("Expected an error"); let downcasted_error = downcast_framing_error(&error); - assert!(matches!( - downcasted_error, - ChunkedGelfDecoderError::PendingMessagesLimitReached { .. } - )); + assert_eq!( + *downcasted_error, + ChunkedGelfDecoderError::PendingMessagesLimitReached { + message_id: 2u64, + sequence_number: 0u8, + pending_messages_limit: 1, + } + ); assert!(decoder.state.lock().unwrap().len() == 1); } @@ -712,10 +803,39 @@ mod tests { let frame = decoder.decode_eof(&mut second_chunk); let error = frame.expect_err("Expected an error"); let downcasted_error = downcast_framing_error(&error); - assert!(matches!( - downcasted_error, - ChunkedGelfDecoderError::TotalChunksMismatch { .. } - )); + assert_eq!( + *downcasted_error, + ChunkedGelfDecoderError::TotalChunksMismatch { + message_id: 1, + sequence_number: 1, + original_total_chunks: 2, + received_total_chunks: 3, + } + ); + } + + #[rstest] + #[tokio::test] + async fn decode_chunk_greater_than_max_chunk_length() { + let message_id = 1u64; + let sequence_number = 0u8; + let total_chunks = 1u8; + let payload = "foo"; + let mut chunk = create_chunk(message_id, sequence_number, total_chunks, &payload); + let mut decoder = ChunkedGelfDecoder::new(DEFAULT_TIMEOUT_SECS, None, Some(1), None); + + let frame = decoder.decode_eof(&mut chunk); + let error = frame.expect_err("Expected an error"); + let downcasted_error = downcast_framing_error(&error); + assert_eq!( + *downcasted_error, + ChunkedGelfDecoderError::MaxChunkLengthExceeded { + message_id: 1, + sequence_number: 0, + chunk_length: 3, + max_chunk_length: 1, + } + ); } #[rstest] From f0ea0008e03bfbccab01acace4f8ac06b70fbdde Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Fri, 25 Oct 2024 19:20:11 +0200 Subject: [PATCH 56/60] fix: clippy lint --- lib/codecs/src/decoding/framing/chunked_gelf.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index cc9ba50581fbf..a69eac5ea86c8 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -821,7 +821,7 @@ mod tests { let sequence_number = 0u8; let total_chunks = 1u8; let payload = "foo"; - let mut chunk = create_chunk(message_id, sequence_number, total_chunks, &payload); + let mut chunk = create_chunk(message_id, sequence_number, total_chunks, payload); let mut decoder = ChunkedGelfDecoder::new(DEFAULT_TIMEOUT_SECS, None, Some(1), None); let frame = decoder.decode_eof(&mut chunk); From f4630e92b839f53b0959598cbdcc3595c88b88a2 Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Sat, 26 Oct 2024 10:48:02 +0200 Subject: [PATCH 57/60] test: add test for max_message_length --- .../src/decoding/framing/chunked_gelf.rs | 38 ++++++++++++++++--- 1 file changed, 32 insertions(+), 6 deletions(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index a69eac5ea86c8..835c34b5f7b14 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -703,7 +703,7 @@ mod tests { let mut decoder = ChunkedGelfDecoder::default(); let frame = decoder.decode_eof(&mut src); - let error = frame.expect_err("Expected an error"); + let error = frame.unwrap_err(); let downcasted_error = downcast_framing_error(&error); assert_eq!( *downcasted_error, @@ -723,7 +723,7 @@ mod tests { let mut decoder = ChunkedGelfDecoder::default(); let frame = decoder.decode_eof(&mut chunk); - let error = frame.expect_err("Expected an error"); + let error = frame.unwrap_err(); let downcasted_error = downcast_framing_error(&error); assert_eq!( *downcasted_error, @@ -745,7 +745,7 @@ mod tests { let mut decoder = ChunkedGelfDecoder::default(); let frame = decoder.decode_eof(&mut chunk); - let error = frame.expect_err("Expected an error"); + let error = frame.unwrap_err(); let downcasted_error = downcast_framing_error(&error); assert_eq!( *downcasted_error, @@ -772,7 +772,7 @@ mod tests { assert!(decoder.state.lock().unwrap().len() == 1); let frame = decoder.decode_eof(&mut three_chunks[0]); - let error = frame.expect_err("Expected an error"); + let error = frame.unwrap_err(); let downcasted_error = downcast_framing_error(&error); assert_eq!( *downcasted_error, @@ -801,7 +801,7 @@ mod tests { assert!(frame.is_none()); let frame = decoder.decode_eof(&mut second_chunk); - let error = frame.expect_err("Expected an error"); + let error = frame.unwrap_err(); let downcasted_error = downcast_framing_error(&error); assert_eq!( *downcasted_error, @@ -825,7 +825,7 @@ mod tests { let mut decoder = ChunkedGelfDecoder::new(DEFAULT_TIMEOUT_SECS, None, Some(1), None); let frame = decoder.decode_eof(&mut chunk); - let error = frame.expect_err("Expected an error"); + let error = frame.unwrap_err(); let downcasted_error = downcast_framing_error(&error); assert_eq!( *downcasted_error, @@ -836,6 +836,32 @@ mod tests { max_chunk_length: 1, } ); + assert_eq!(decoder.state.lock().unwrap().len(), 0); + } + + #[rstest] + #[tokio::test] + async fn decode_message_greater_than_max_message_length( + two_chunks_message: ([BytesMut; 2], String), + ) { + let (mut chunks, _) = two_chunks_message; + let mut decoder = ChunkedGelfDecoder::new(DEFAULT_TIMEOUT_SECS, None, None, Some(5)); + + let frame = decoder.decode_eof(&mut chunks[0]).unwrap(); + assert!(frame.is_none()); + let frame = decoder.decode_eof(&mut chunks[1]); + let error = frame.unwrap_err(); + let downcasted_error = downcast_framing_error(&error); + assert_eq!( + *downcasted_error, + ChunkedGelfDecoderError::MaxMessageLengthExceeded { + message_id: 1, + sequence_number: 1, + message_length: 6, + max_message_length: 5, + } + ); + assert_eq!(decoder.state.lock().unwrap().len(), 0); } #[rstest] From eeaaf31ced625f4d491989aac0d45ca7bf0320af Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Sat, 26 Oct 2024 11:20:13 +0200 Subject: [PATCH 58/60] feat: add serde(default) into new configs --- Cargo.lock | 32 ++++++++++++++++--- .../src/decoding/framing/chunked_gelf.rs | 4 +-- 2 files changed, 30 insertions(+), 6 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 5259d73832987..204cbc4fac3db 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2190,6 +2190,7 @@ dependencies = [ "ordered-float 4.4.0", "prost 0.12.6", "prost-reflect", + "rand 0.8.5", "regex", "rstest", "serde", @@ -2201,6 +2202,7 @@ dependencies = [ "tokio", "tokio-util", "tracing 0.1.40", + "tracing-test", "uuid", "vector-common", "vector-config", @@ -4437,7 +4439,7 @@ dependencies = [ "httpdate", "itoa", "pin-project-lite", - "socket2 0.4.10", + "socket2 0.5.7", "tokio", "tower-service", "tracing 0.1.40", @@ -7243,7 +7245,7 @@ checksum = "22505a5c94da8e3b7c2996394d1c933236c4d743e81a410bcca4e6989fc066a4" dependencies = [ "bytes 1.8.0", "heck 0.5.0", - "itertools 0.11.0", + "itertools 0.12.1", "log", "multimap", "once_cell", @@ -7276,7 +7278,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "81bddcdb20abf9501610992b6759a4c888aef7d1a7247ef75e2404275ac24af1" dependencies = [ "anyhow", - "itertools 0.11.0", + "itertools 0.12.1", "proc-macro2 1.0.89", "quote 1.0.37", "syn 2.0.85", @@ -7289,7 +7291,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e9552f850d5f0964a4e4d0bf306459ac29323ddfbae05e35a7c0d35cb0803cc5" dependencies = [ "anyhow", - "itertools 0.11.0", + "itertools 0.13.0", "proc-macro2 1.0.89", "quote 1.0.37", "syn 2.0.85", @@ -10041,6 +10043,7 @@ dependencies = [ "serde", "serde_json", "sharded-slab", + "smallvec", "thread_local", "tracing 0.1.40", "tracing-core 0.1.32", @@ -10048,6 +10051,27 @@ dependencies = [ "tracing-serde", ] +[[package]] +name = "tracing-test" +version = "0.2.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "557b891436fe0d5e0e363427fc7f217abf9ccd510d5136549847bdcbcd011d68" +dependencies = [ + "tracing-core 0.1.32", + "tracing-subscriber", + "tracing-test-macro", +] + +[[package]] +name = "tracing-test-macro" +version = "0.2.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "04659ddb06c87d233c566112c1c9c5b9e98256d9af50ec3bc9c8327f873a7568" +dependencies = [ + "quote 1.0.37", + "syn 2.0.85", +] + [[package]] name = "tracing-tower" version = "0.1.0" diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index 835c34b5f7b14..54c88863a9edf 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -66,7 +66,7 @@ pub struct ChunkedGelfDecoderOptions { /// the per-chunk memory is unbounded. /// /// This limit takes only into account the chunk's payload and the GELF header bytes are excluded from the calculation. - #[serde(skip_serializing_if = "vector_core::serde::is_default")] + #[serde(default, skip_serializing_if = "vector_core::serde::is_default")] pub max_chunk_length: Option, /// The maximum length of a single GELF message, in bytes. Messages longer than this length will @@ -79,7 +79,7 @@ pub struct ChunkedGelfDecoderOptions { /// /// This limit takes only into account the message's payload and the GELF header bytes are excluded from the calculation. /// The message's payload is the concatenation of all the chunks' payloads. - #[serde(skip_serializing_if = "vector_core::serde::is_default")] + #[serde(default, skip_serializing_if = "vector_core::serde::is_default")] pub max_message_length: Option, } From ecc61da7028ba49a9034ea89d6fb17367f31986b Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Sat, 26 Oct 2024 11:27:43 +0200 Subject: [PATCH 59/60] docs: update component docs --- .../components/sources/base/amqp.cue | 27 +++++++++++++++++++ .../sources/base/aws_kinesis_firehose.cue | 27 +++++++++++++++++++ .../components/sources/base/aws_s3.cue | 27 +++++++++++++++++++ .../components/sources/base/aws_sqs.cue | 27 +++++++++++++++++++ .../components/sources/base/datadog_agent.cue | 27 +++++++++++++++++++ .../components/sources/base/demo_logs.cue | 27 +++++++++++++++++++ .../components/sources/base/exec.cue | 27 +++++++++++++++++++ .../sources/base/file_descriptor.cue | 27 +++++++++++++++++++ .../components/sources/base/gcp_pubsub.cue | 27 +++++++++++++++++++ .../components/sources/base/heroku_logs.cue | 27 +++++++++++++++++++ .../components/sources/base/http.cue | 27 +++++++++++++++++++ .../components/sources/base/http_client.cue | 27 +++++++++++++++++++ .../components/sources/base/http_server.cue | 27 +++++++++++++++++++ .../components/sources/base/kafka.cue | 27 +++++++++++++++++++ .../components/sources/base/nats.cue | 27 +++++++++++++++++++ .../components/sources/base/pulsar.cue | 27 +++++++++++++++++++ .../components/sources/base/redis.cue | 27 +++++++++++++++++++ .../components/sources/base/socket.cue | 27 +++++++++++++++++++ .../components/sources/base/stdin.cue | 27 +++++++++++++++++++ 19 files changed, 513 insertions(+) diff --git a/website/cue/reference/components/sources/base/amqp.cue b/website/cue/reference/components/sources/base/amqp.cue index 45d22c6e180bf..efd8675348c6e 100644 --- a/website/cue/reference/components/sources/base/amqp.cue +++ b/website/cue/reference/components/sources/base/amqp.cue @@ -335,6 +335,33 @@ base: components: sources: amqp: configuration: { relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { + max_chunk_length: { + description: """ + The maximum length of a single GELF chunk, in bytes. Chunks longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of chunks and + the per-chunk memory is unbounded. + + This limit takes only into account the chunk's payload and the GELF header bytes are excluded from the calculation. + """ + required: false + type: uint: {} + } + max_message_length: { + description: """ + The maximum length of a single GELF message, in bytes. Messages longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of messages and + the per-message memory is unbounded. + + Note that a message can be composed of multiple chunks and this limit is applied to the whole + message, not to individual chunks. This length should always be greater than the `max_chunk_length`. + This option is useful to limit the memory usage of the decoders's chunk buffer. + + This limit takes only into account the message's payload and the GELF header bytes are excluded from the calculation. + The message's payload is the concatenation of all the chunks' payloads. + """ + required: false + type: uint: {} + } pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts diff --git a/website/cue/reference/components/sources/base/aws_kinesis_firehose.cue b/website/cue/reference/components/sources/base/aws_kinesis_firehose.cue index 096f9b8694fd1..ef112080255c4 100644 --- a/website/cue/reference/components/sources/base/aws_kinesis_firehose.cue +++ b/website/cue/reference/components/sources/base/aws_kinesis_firehose.cue @@ -333,6 +333,33 @@ base: components: sources: aws_kinesis_firehose: configuration: { relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { + max_chunk_length: { + description: """ + The maximum length of a single GELF chunk, in bytes. Chunks longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of chunks and + the per-chunk memory is unbounded. + + This limit takes only into account the chunk's payload and the GELF header bytes are excluded from the calculation. + """ + required: false + type: uint: {} + } + max_message_length: { + description: """ + The maximum length of a single GELF message, in bytes. Messages longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of messages and + the per-message memory is unbounded. + + Note that a message can be composed of multiple chunks and this limit is applied to the whole + message, not to individual chunks. This length should always be greater than the `max_chunk_length`. + This option is useful to limit the memory usage of the decoders's chunk buffer. + + This limit takes only into account the message's payload and the GELF header bytes are excluded from the calculation. + The message's payload is the concatenation of all the chunks' payloads. + """ + required: false + type: uint: {} + } pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts diff --git a/website/cue/reference/components/sources/base/aws_s3.cue b/website/cue/reference/components/sources/base/aws_s3.cue index 3e200fa336a94..feffca6a37193 100644 --- a/website/cue/reference/components/sources/base/aws_s3.cue +++ b/website/cue/reference/components/sources/base/aws_s3.cue @@ -433,6 +433,33 @@ base: components: sources: aws_s3: configuration: { relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { + max_chunk_length: { + description: """ + The maximum length of a single GELF chunk, in bytes. Chunks longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of chunks and + the per-chunk memory is unbounded. + + This limit takes only into account the chunk's payload and the GELF header bytes are excluded from the calculation. + """ + required: false + type: uint: {} + } + max_message_length: { + description: """ + The maximum length of a single GELF message, in bytes. Messages longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of messages and + the per-message memory is unbounded. + + Note that a message can be composed of multiple chunks and this limit is applied to the whole + message, not to individual chunks. This length should always be greater than the `max_chunk_length`. + This option is useful to limit the memory usage of the decoders's chunk buffer. + + This limit takes only into account the message's payload and the GELF header bytes are excluded from the calculation. + The message's payload is the concatenation of all the chunks' payloads. + """ + required: false + type: uint: {} + } pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts diff --git a/website/cue/reference/components/sources/base/aws_sqs.cue b/website/cue/reference/components/sources/base/aws_sqs.cue index f7252b5dcd2d6..28a58f46e7aef 100644 --- a/website/cue/reference/components/sources/base/aws_sqs.cue +++ b/website/cue/reference/components/sources/base/aws_sqs.cue @@ -437,6 +437,33 @@ base: components: sources: aws_sqs: configuration: { relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { + max_chunk_length: { + description: """ + The maximum length of a single GELF chunk, in bytes. Chunks longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of chunks and + the per-chunk memory is unbounded. + + This limit takes only into account the chunk's payload and the GELF header bytes are excluded from the calculation. + """ + required: false + type: uint: {} + } + max_message_length: { + description: """ + The maximum length of a single GELF message, in bytes. Messages longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of messages and + the per-message memory is unbounded. + + Note that a message can be composed of multiple chunks and this limit is applied to the whole + message, not to individual chunks. This length should always be greater than the `max_chunk_length`. + This option is useful to limit the memory usage of the decoders's chunk buffer. + + This limit takes only into account the message's payload and the GELF header bytes are excluded from the calculation. + The message's payload is the concatenation of all the chunks' payloads. + """ + required: false + type: uint: {} + } pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts diff --git a/website/cue/reference/components/sources/base/datadog_agent.cue b/website/cue/reference/components/sources/base/datadog_agent.cue index 4d4eafee8d8b4..64dd7f7fb232f 100644 --- a/website/cue/reference/components/sources/base/datadog_agent.cue +++ b/website/cue/reference/components/sources/base/datadog_agent.cue @@ -330,6 +330,33 @@ base: components: sources: datadog_agent: configuration: { relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { + max_chunk_length: { + description: """ + The maximum length of a single GELF chunk, in bytes. Chunks longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of chunks and + the per-chunk memory is unbounded. + + This limit takes only into account the chunk's payload and the GELF header bytes are excluded from the calculation. + """ + required: false + type: uint: {} + } + max_message_length: { + description: """ + The maximum length of a single GELF message, in bytes. Messages longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of messages and + the per-message memory is unbounded. + + Note that a message can be composed of multiple chunks and this limit is applied to the whole + message, not to individual chunks. This length should always be greater than the `max_chunk_length`. + This option is useful to limit the memory usage of the decoders's chunk buffer. + + This limit takes only into account the message's payload and the GELF header bytes are excluded from the calculation. + The message's payload is the concatenation of all the chunks' payloads. + """ + required: false + type: uint: {} + } pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts diff --git a/website/cue/reference/components/sources/base/demo_logs.cue b/website/cue/reference/components/sources/base/demo_logs.cue index f4b9c324f2311..4660526ecef0a 100644 --- a/website/cue/reference/components/sources/base/demo_logs.cue +++ b/website/cue/reference/components/sources/base/demo_logs.cue @@ -326,6 +326,33 @@ base: components: sources: demo_logs: configuration: { relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { + max_chunk_length: { + description: """ + The maximum length of a single GELF chunk, in bytes. Chunks longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of chunks and + the per-chunk memory is unbounded. + + This limit takes only into account the chunk's payload and the GELF header bytes are excluded from the calculation. + """ + required: false + type: uint: {} + } + max_message_length: { + description: """ + The maximum length of a single GELF message, in bytes. Messages longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of messages and + the per-message memory is unbounded. + + Note that a message can be composed of multiple chunks and this limit is applied to the whole + message, not to individual chunks. This length should always be greater than the `max_chunk_length`. + This option is useful to limit the memory usage of the decoders's chunk buffer. + + This limit takes only into account the message's payload and the GELF header bytes are excluded from the calculation. + The message's payload is the concatenation of all the chunks' payloads. + """ + required: false + type: uint: {} + } pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts diff --git a/website/cue/reference/components/sources/base/exec.cue b/website/cue/reference/components/sources/base/exec.cue index 59be6415a0c1c..61857b7d28a87 100644 --- a/website/cue/reference/components/sources/base/exec.cue +++ b/website/cue/reference/components/sources/base/exec.cue @@ -314,6 +314,33 @@ base: components: sources: exec: configuration: { relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { + max_chunk_length: { + description: """ + The maximum length of a single GELF chunk, in bytes. Chunks longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of chunks and + the per-chunk memory is unbounded. + + This limit takes only into account the chunk's payload and the GELF header bytes are excluded from the calculation. + """ + required: false + type: uint: {} + } + max_message_length: { + description: """ + The maximum length of a single GELF message, in bytes. Messages longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of messages and + the per-message memory is unbounded. + + Note that a message can be composed of multiple chunks and this limit is applied to the whole + message, not to individual chunks. This length should always be greater than the `max_chunk_length`. + This option is useful to limit the memory usage of the decoders's chunk buffer. + + This limit takes only into account the message's payload and the GELF header bytes are excluded from the calculation. + The message's payload is the concatenation of all the chunks' payloads. + """ + required: false + type: uint: {} + } pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts diff --git a/website/cue/reference/components/sources/base/file_descriptor.cue b/website/cue/reference/components/sources/base/file_descriptor.cue index d20353ce4d14e..b26a2af12b18c 100644 --- a/website/cue/reference/components/sources/base/file_descriptor.cue +++ b/website/cue/reference/components/sources/base/file_descriptor.cue @@ -292,6 +292,33 @@ base: components: sources: file_descriptor: configuration: { relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { + max_chunk_length: { + description: """ + The maximum length of a single GELF chunk, in bytes. Chunks longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of chunks and + the per-chunk memory is unbounded. + + This limit takes only into account the chunk's payload and the GELF header bytes are excluded from the calculation. + """ + required: false + type: uint: {} + } + max_message_length: { + description: """ + The maximum length of a single GELF message, in bytes. Messages longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of messages and + the per-message memory is unbounded. + + Note that a message can be composed of multiple chunks and this limit is applied to the whole + message, not to individual chunks. This length should always be greater than the `max_chunk_length`. + This option is useful to limit the memory usage of the decoders's chunk buffer. + + This limit takes only into account the message's payload and the GELF header bytes are excluded from the calculation. + The message's payload is the concatenation of all the chunks' payloads. + """ + required: false + type: uint: {} + } pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts diff --git a/website/cue/reference/components/sources/base/gcp_pubsub.cue b/website/cue/reference/components/sources/base/gcp_pubsub.cue index d972a14186ea8..cd02efc81f8f7 100644 --- a/website/cue/reference/components/sources/base/gcp_pubsub.cue +++ b/website/cue/reference/components/sources/base/gcp_pubsub.cue @@ -369,6 +369,33 @@ base: components: sources: gcp_pubsub: configuration: { relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { + max_chunk_length: { + description: """ + The maximum length of a single GELF chunk, in bytes. Chunks longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of chunks and + the per-chunk memory is unbounded. + + This limit takes only into account the chunk's payload and the GELF header bytes are excluded from the calculation. + """ + required: false + type: uint: {} + } + max_message_length: { + description: """ + The maximum length of a single GELF message, in bytes. Messages longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of messages and + the per-message memory is unbounded. + + Note that a message can be composed of multiple chunks and this limit is applied to the whole + message, not to individual chunks. This length should always be greater than the `max_chunk_length`. + This option is useful to limit the memory usage of the decoders's chunk buffer. + + This limit takes only into account the message's payload and the GELF header bytes are excluded from the calculation. + The message's payload is the concatenation of all the chunks' payloads. + """ + required: false + type: uint: {} + } pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts diff --git a/website/cue/reference/components/sources/base/heroku_logs.cue b/website/cue/reference/components/sources/base/heroku_logs.cue index 730e4e9d868a8..ff45fbc204286 100644 --- a/website/cue/reference/components/sources/base/heroku_logs.cue +++ b/website/cue/reference/components/sources/base/heroku_logs.cue @@ -327,6 +327,33 @@ base: components: sources: heroku_logs: configuration: { relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { + max_chunk_length: { + description: """ + The maximum length of a single GELF chunk, in bytes. Chunks longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of chunks and + the per-chunk memory is unbounded. + + This limit takes only into account the chunk's payload and the GELF header bytes are excluded from the calculation. + """ + required: false + type: uint: {} + } + max_message_length: { + description: """ + The maximum length of a single GELF message, in bytes. Messages longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of messages and + the per-message memory is unbounded. + + Note that a message can be composed of multiple chunks and this limit is applied to the whole + message, not to individual chunks. This length should always be greater than the `max_chunk_length`. + This option is useful to limit the memory usage of the decoders's chunk buffer. + + This limit takes only into account the message's payload and the GELF header bytes are excluded from the calculation. + The message's payload is the concatenation of all the chunks' payloads. + """ + required: false + type: uint: {} + } pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts diff --git a/website/cue/reference/components/sources/base/http.cue b/website/cue/reference/components/sources/base/http.cue index 96e7594a7a6bd..fcc7a7396b61a 100644 --- a/website/cue/reference/components/sources/base/http.cue +++ b/website/cue/reference/components/sources/base/http.cue @@ -342,6 +342,33 @@ base: components: sources: http: configuration: { relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { + max_chunk_length: { + description: """ + The maximum length of a single GELF chunk, in bytes. Chunks longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of chunks and + the per-chunk memory is unbounded. + + This limit takes only into account the chunk's payload and the GELF header bytes are excluded from the calculation. + """ + required: false + type: uint: {} + } + max_message_length: { + description: """ + The maximum length of a single GELF message, in bytes. Messages longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of messages and + the per-message memory is unbounded. + + Note that a message can be composed of multiple chunks and this limit is applied to the whole + message, not to individual chunks. This length should always be greater than the `max_chunk_length`. + This option is useful to limit the memory usage of the decoders's chunk buffer. + + This limit takes only into account the message's payload and the GELF header bytes are excluded from the calculation. + The message's payload is the concatenation of all the chunks' payloads. + """ + required: false + type: uint: {} + } pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts diff --git a/website/cue/reference/components/sources/base/http_client.cue b/website/cue/reference/components/sources/base/http_client.cue index 7301a345fe774..8282ea348a16e 100644 --- a/website/cue/reference/components/sources/base/http_client.cue +++ b/website/cue/reference/components/sources/base/http_client.cue @@ -330,6 +330,33 @@ base: components: sources: http_client: configuration: { relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { + max_chunk_length: { + description: """ + The maximum length of a single GELF chunk, in bytes. Chunks longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of chunks and + the per-chunk memory is unbounded. + + This limit takes only into account the chunk's payload and the GELF header bytes are excluded from the calculation. + """ + required: false + type: uint: {} + } + max_message_length: { + description: """ + The maximum length of a single GELF message, in bytes. Messages longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of messages and + the per-message memory is unbounded. + + Note that a message can be composed of multiple chunks and this limit is applied to the whole + message, not to individual chunks. This length should always be greater than the `max_chunk_length`. + This option is useful to limit the memory usage of the decoders's chunk buffer. + + This limit takes only into account the message's payload and the GELF header bytes are excluded from the calculation. + The message's payload is the concatenation of all the chunks' payloads. + """ + required: false + type: uint: {} + } pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts diff --git a/website/cue/reference/components/sources/base/http_server.cue b/website/cue/reference/components/sources/base/http_server.cue index 1437fd60b4aa4..38e9658082e2b 100644 --- a/website/cue/reference/components/sources/base/http_server.cue +++ b/website/cue/reference/components/sources/base/http_server.cue @@ -342,6 +342,33 @@ base: components: sources: http_server: configuration: { relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { + max_chunk_length: { + description: """ + The maximum length of a single GELF chunk, in bytes. Chunks longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of chunks and + the per-chunk memory is unbounded. + + This limit takes only into account the chunk's payload and the GELF header bytes are excluded from the calculation. + """ + required: false + type: uint: {} + } + max_message_length: { + description: """ + The maximum length of a single GELF message, in bytes. Messages longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of messages and + the per-message memory is unbounded. + + Note that a message can be composed of multiple chunks and this limit is applied to the whole + message, not to individual chunks. This length should always be greater than the `max_chunk_length`. + This option is useful to limit the memory usage of the decoders's chunk buffer. + + This limit takes only into account the message's payload and the GELF header bytes are excluded from the calculation. + The message's payload is the concatenation of all the chunks' payloads. + """ + required: false + type: uint: {} + } pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts diff --git a/website/cue/reference/components/sources/base/kafka.cue b/website/cue/reference/components/sources/base/kafka.cue index a1d9fdc0a4062..a23a8a2ec9ae0 100644 --- a/website/cue/reference/components/sources/base/kafka.cue +++ b/website/cue/reference/components/sources/base/kafka.cue @@ -363,6 +363,33 @@ base: components: sources: kafka: configuration: { relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { + max_chunk_length: { + description: """ + The maximum length of a single GELF chunk, in bytes. Chunks longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of chunks and + the per-chunk memory is unbounded. + + This limit takes only into account the chunk's payload and the GELF header bytes are excluded from the calculation. + """ + required: false + type: uint: {} + } + max_message_length: { + description: """ + The maximum length of a single GELF message, in bytes. Messages longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of messages and + the per-message memory is unbounded. + + Note that a message can be composed of multiple chunks and this limit is applied to the whole + message, not to individual chunks. This length should always be greater than the `max_chunk_length`. + This option is useful to limit the memory usage of the decoders's chunk buffer. + + This limit takes only into account the message's payload and the GELF header bytes are excluded from the calculation. + The message's payload is the concatenation of all the chunks' payloads. + """ + required: false + type: uint: {} + } pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts diff --git a/website/cue/reference/components/sources/base/nats.cue b/website/cue/reference/components/sources/base/nats.cue index 7984d06b8a9f7..6e1c0e5fa8685 100644 --- a/website/cue/reference/components/sources/base/nats.cue +++ b/website/cue/reference/components/sources/base/nats.cue @@ -382,6 +382,33 @@ base: components: sources: nats: configuration: { relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { + max_chunk_length: { + description: """ + The maximum length of a single GELF chunk, in bytes. Chunks longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of chunks and + the per-chunk memory is unbounded. + + This limit takes only into account the chunk's payload and the GELF header bytes are excluded from the calculation. + """ + required: false + type: uint: {} + } + max_message_length: { + description: """ + The maximum length of a single GELF message, in bytes. Messages longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of messages and + the per-message memory is unbounded. + + Note that a message can be composed of multiple chunks and this limit is applied to the whole + message, not to individual chunks. This length should always be greater than the `max_chunk_length`. + This option is useful to limit the memory usage of the decoders's chunk buffer. + + This limit takes only into account the message's payload and the GELF header bytes are excluded from the calculation. + The message's payload is the concatenation of all the chunks' payloads. + """ + required: false + type: uint: {} + } pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts diff --git a/website/cue/reference/components/sources/base/pulsar.cue b/website/cue/reference/components/sources/base/pulsar.cue index ae357cdadf506..d3d5b5f03b1ed 100644 --- a/website/cue/reference/components/sources/base/pulsar.cue +++ b/website/cue/reference/components/sources/base/pulsar.cue @@ -393,6 +393,33 @@ base: components: sources: pulsar: configuration: { relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { + max_chunk_length: { + description: """ + The maximum length of a single GELF chunk, in bytes. Chunks longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of chunks and + the per-chunk memory is unbounded. + + This limit takes only into account the chunk's payload and the GELF header bytes are excluded from the calculation. + """ + required: false + type: uint: {} + } + max_message_length: { + description: """ + The maximum length of a single GELF message, in bytes. Messages longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of messages and + the per-message memory is unbounded. + + Note that a message can be composed of multiple chunks and this limit is applied to the whole + message, not to individual chunks. This length should always be greater than the `max_chunk_length`. + This option is useful to limit the memory usage of the decoders's chunk buffer. + + This limit takes only into account the message's payload and the GELF header bytes are excluded from the calculation. + The message's payload is the concatenation of all the chunks' payloads. + """ + required: false + type: uint: {} + } pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts diff --git a/website/cue/reference/components/sources/base/redis.cue b/website/cue/reference/components/sources/base/redis.cue index 9bb6b91aee72b..f4d5599370b8b 100644 --- a/website/cue/reference/components/sources/base/redis.cue +++ b/website/cue/reference/components/sources/base/redis.cue @@ -300,6 +300,33 @@ base: components: sources: redis: configuration: { relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { + max_chunk_length: { + description: """ + The maximum length of a single GELF chunk, in bytes. Chunks longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of chunks and + the per-chunk memory is unbounded. + + This limit takes only into account the chunk's payload and the GELF header bytes are excluded from the calculation. + """ + required: false + type: uint: {} + } + max_message_length: { + description: """ + The maximum length of a single GELF message, in bytes. Messages longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of messages and + the per-message memory is unbounded. + + Note that a message can be composed of multiple chunks and this limit is applied to the whole + message, not to individual chunks. This length should always be greater than the `max_chunk_length`. + This option is useful to limit the memory usage of the decoders's chunk buffer. + + This limit takes only into account the message's payload and the GELF header bytes are excluded from the calculation. + The message's payload is the concatenation of all the chunks' payloads. + """ + required: false + type: uint: {} + } pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts diff --git a/website/cue/reference/components/sources/base/socket.cue b/website/cue/reference/components/sources/base/socket.cue index 2ceff30d4af34..c11046ac2a13f 100644 --- a/website/cue/reference/components/sources/base/socket.cue +++ b/website/cue/reference/components/sources/base/socket.cue @@ -302,6 +302,33 @@ base: components: sources: socket: configuration: { relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { + max_chunk_length: { + description: """ + The maximum length of a single GELF chunk, in bytes. Chunks longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of chunks and + the per-chunk memory is unbounded. + + This limit takes only into account the chunk's payload and the GELF header bytes are excluded from the calculation. + """ + required: false + type: uint: {} + } + max_message_length: { + description: """ + The maximum length of a single GELF message, in bytes. Messages longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of messages and + the per-message memory is unbounded. + + Note that a message can be composed of multiple chunks and this limit is applied to the whole + message, not to individual chunks. This length should always be greater than the `max_chunk_length`. + This option is useful to limit the memory usage of the decoders's chunk buffer. + + This limit takes only into account the message's payload and the GELF header bytes are excluded from the calculation. + The message's payload is the concatenation of all the chunks' payloads. + """ + required: false + type: uint: {} + } pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts diff --git a/website/cue/reference/components/sources/base/stdin.cue b/website/cue/reference/components/sources/base/stdin.cue index e9f354f3b58ac..bc706702a5aef 100644 --- a/website/cue/reference/components/sources/base/stdin.cue +++ b/website/cue/reference/components/sources/base/stdin.cue @@ -285,6 +285,33 @@ base: components: sources: stdin: configuration: { relevant_when: "method = \"chunked_gelf\"" required: false type: object: options: { + max_chunk_length: { + description: """ + The maximum length of a single GELF chunk, in bytes. Chunks longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of chunks and + the per-chunk memory is unbounded. + + This limit takes only into account the chunk's payload and the GELF header bytes are excluded from the calculation. + """ + required: false + type: uint: {} + } + max_message_length: { + description: """ + The maximum length of a single GELF message, in bytes. Messages longer than this length will + be dropped. If this option is not set, the decoder does not limit the length of messages and + the per-message memory is unbounded. + + Note that a message can be composed of multiple chunks and this limit is applied to the whole + message, not to individual chunks. This length should always be greater than the `max_chunk_length`. + This option is useful to limit the memory usage of the decoders's chunk buffer. + + This limit takes only into account the message's payload and the GELF header bytes are excluded from the calculation. + The message's payload is the concatenation of all the chunks' payloads. + """ + required: false + type: uint: {} + } pending_messages_limit: { description: """ The maximum number of pending incomplete messages. If this limit is reached, the decoder starts From dc07697599b91e384a3022546ce412db4ddf323c Mon Sep 17 00:00:00 2001 From: Jorge Hermo Date: Sat, 26 Oct 2024 11:45:56 +0200 Subject: [PATCH 60/60] test: fix broken max chunk length test --- .../src/decoding/framing/chunked_gelf.rs | 26 +++++++++---------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/lib/codecs/src/decoding/framing/chunked_gelf.rs b/lib/codecs/src/decoding/framing/chunked_gelf.rs index 54c88863a9edf..131ff116eb9b1 100644 --- a/lib/codecs/src/decoding/framing/chunked_gelf.rs +++ b/lib/codecs/src/decoding/framing/chunked_gelf.rs @@ -276,6 +276,19 @@ impl ChunkedGelfDecoder { } ); + if let Some(max_chunk_length) = self.max_chunk_length { + let chunk_length = chunk.remaining(); + ensure!( + chunk_length <= max_chunk_length, + MaxChunkLengthExceededSnafu { + message_id, + sequence_number, + chunk_length, + max_chunk_length + } + ); + } + let mut state_lock = self.state.lock().expect("poisoned lock"); if let Some(pending_messages_limit) = self.pending_messages_limit { @@ -329,19 +342,6 @@ impl ChunkedGelfDecoder { return Ok(None); } - if let Some(max_chunk_length) = self.max_chunk_length { - let chunk_length = chunk.remaining(); - ensure!( - chunk_length <= max_chunk_length, - MaxChunkLengthExceededSnafu { - message_id, - sequence_number, - chunk_length, - max_chunk_length - } - ); - } - message_state.add_chunk(sequence_number, chunk); if let Some(max_message_length) = self.max_message_length {