From 8b74d7f2272ddf561f20d4b93656d27ddc4ece4b Mon Sep 17 00:00:00 2001 From: Mike Del Tito Date: Mon, 12 Feb 2024 12:12:58 -0500 Subject: [PATCH 01/19] fix(mezmo-sink): remove ARC, replace sleep timer with manual check Two pieces were found that impact the throughput possible by this destination. - Adaptive concurrency does not work with the `PartitionBatchSink`. Regardless of the status of the underlying service, the concurrency never ticks up and the request permit semaphore is never properly released. - Under high throughput, the number of partitions in the request buffer causes the "linger" timeout on the partition to never fire. This removes ARC completely for the `PartitionBatchSink`, and switches to an explicit check of the batch timeout rather than a `Sleep` future. Ref: LOG-19184 --- src/sinks/util/http.rs | 9 ++--- src/sinks/util/mod.rs | 4 +-- src/sinks/util/service.rs | 75 +++++++++++++++++++++++++++++++++++++++ src/sinks/util/sink.rs | 57 +++++++++++++++-------------- 4 files changed, 113 insertions(+), 32 deletions(-) diff --git a/src/sinks/util/http.rs b/src/sinks/util/http.rs index deea66f23..e89cce6f0 100644 --- a/src/sinks/util/http.rs +++ b/src/sinks/util/http.rs @@ -26,8 +26,8 @@ use vector_lib::{ByteSizeOf, EstimatedJsonEncodedSizeOf}; use super::{ retries::{RetryAction, RetryLogic}, sink::{self, Response as _}, - uri, Batch, EncodedEvent, Partition, TowerBatchedSink, TowerPartitionSink, TowerRequestConfig, - TowerRequestSettings, + uri, Batch, EncodedEvent, Partition, TowerBatchedSink, TowerPartitionSinkWithoutArc, + TowerRequestConfig, TowerRequestSettings, }; use crate::{ config::SinkContext, @@ -229,7 +229,7 @@ where { sink: Arc, #[pin] - inner: TowerPartitionSink< + inner: TowerPartitionSinkWithoutArc< MezmoHttpBatchLoggingService< BoxFuture<'static, crate::Result>>, B::Output, @@ -300,7 +300,8 @@ where HttpBatchService::new(client, request_builder), cx.mezmo_ctx, ); - let inner = request_settings.partition_sink(retry_logic, svc, batch, batch_timeout); + let inner = + request_settings.partition_sink_without_arc(retry_logic, svc, batch, batch_timeout); let encoder = sink.build_encoder(); Self { diff --git a/src/sinks/util/mod.rs b/src/sinks/util/mod.rs index f0a233b70..5d13626a6 100644 --- a/src/sinks/util/mod.rs +++ b/src/sinks/util/mod.rs @@ -46,8 +46,8 @@ pub use compressor::Compressor; pub use normalizer::Normalizer; pub use request_builder::{IncrementalRequestBuilder, RequestBuilder}; pub use service::{ - Concurrency, ServiceBuilderExt, TowerBatchedSink, TowerPartitionSink, TowerRequestConfig, - TowerRequestLayer, TowerRequestSettings, + Concurrency, ServiceBuilderExt, TowerBatchedSink, TowerPartitionSink, + TowerPartitionSinkWithoutArc, TowerRequestConfig, TowerRequestLayer, TowerRequestSettings, }; pub use sink::{BatchSink, PartitionBatchSink, StreamSink}; use snafu::Snafu; diff --git a/src/sinks/util/service.rs b/src/sinks/util/service.rs index 0b01cfabf..dd902a5ac 100644 --- a/src/sinks/util/service.rs +++ b/src/sinks/util/service.rs @@ -38,8 +38,10 @@ mod map; pub mod net; pub type Svc = RateLimit, Timeout>, L>>; +pub type SvcWithoutArc = RateLimit, Timeout>>; pub type TowerBatchedSink = BatchSink, B>; pub type TowerPartitionSink = PartitionBatchSink, B, K>; +pub type TowerPartitionSinkWithoutArc = PartitionBatchSink, B, K>; // Distributed service types pub type DistributedService = RateLimit< @@ -60,6 +62,12 @@ pub trait ServiceBuilderExt { settings: TowerRequestSettings, retry_logic: RL, ) -> ServiceBuilder, L>>; + + fn settings_without_arc( + self, + settings: TowerRequestSettings, + retry_logic: RL, + ) -> ServiceBuilder, L>>; } impl ServiceBuilderExt for ServiceBuilder { @@ -81,6 +89,18 @@ impl ServiceBuilderExt for ServiceBuilder { _pd: std::marker::PhantomData, }) } + + fn settings_without_arc( + self, + settings: TowerRequestSettings, + retry_logic: RL, + ) -> ServiceBuilder, L>> { + self.layer(TowerRequestLayerWithoutArc { + settings, + retry_logic, + _pd: std::marker::PhantomData, + }) + } } /// Middleware settings for outbound requests. @@ -316,6 +336,30 @@ impl TowerRequestSettings { PartitionBatchSink::new(service, batch, batch_timeout) } + pub fn partition_sink_without_arc( + &self, + retry_logic: RL, + service: S, + batch: B, + batch_timeout: Duration, + ) -> TowerPartitionSinkWithoutArc + where + RL: RetryLogic, + S: Service + Clone + Send + 'static, + S::Error: Into + Send + Sync + 'static, + S::Response: Send + Response, + S::Future: Send + 'static, + B: Batch, + B::Input: Partition, + B::Output: Send + Clone + 'static, + K: Hash + Eq + Clone + Send + 'static, + { + let service = ServiceBuilder::new() + .settings_without_arc(self.clone(), retry_logic) + .service(service); + PartitionBatchSink::new(service, batch, batch_timeout) + } + /// Note: This has been deprecated, please do not use when creating new Sinks. pub fn batch_sink( &self, @@ -431,6 +475,37 @@ where } } +#[derive(Debug, Clone)] +pub struct TowerRequestLayerWithoutArc { + settings: TowerRequestSettings, + retry_logic: L, + _pd: PhantomData, +} + +impl Layer for TowerRequestLayerWithoutArc +where + S: Service + Send + 'static, + S::Response: Send + 'static, + S::Error: Into + Send + Sync + 'static, + S::Future: Send + 'static, + RL: RetryLogic + Send + 'static, + Request: Clone + Send + 'static, +{ + type Service = SvcWithoutArc; + + fn layer(&self, inner: S) -> Self::Service { + let policy = self.settings.retry_policy(self.retry_logic.clone()); + ServiceBuilder::new() + .rate_limit( + self.settings.rate_limit_num, + self.settings.rate_limit_duration, + ) + .retry(policy) + .timeout(self.settings.timeout) + .service(inner) + } +} + #[cfg(test)] mod tests { use std::sync::{ diff --git a/src/sinks/util/sink.rs b/src/sinks/util/sink.rs index e8a31d13f..c7f722a22 100644 --- a/src/sinks/util/sink.rs +++ b/src/sinks/util/sink.rs @@ -31,6 +31,8 @@ //! from the sink. A oneshot channel is used to tie them back into the sink to allow //! it to notify the consumer that the request has succeeded. +use futures::{future::BoxFuture, stream::FuturesUnordered, FutureExt, Sink, Stream, TryFutureExt}; +use pin_project::pin_project; use std::{ collections::HashMap, fmt, @@ -38,14 +40,9 @@ use std::{ marker::PhantomData, pin::Pin, task::{ready, Context, Poll}, + time::{Duration, Instant}, }; - -use futures::{future::BoxFuture, stream::FuturesUnordered, FutureExt, Sink, Stream, TryFutureExt}; -use pin_project::pin_project; -use tokio::{ - sync::oneshot, - time::{sleep, Duration, Sleep}, -}; +use tokio::sync::oneshot; use tower::{Service, ServiceBuilder}; use tracing::Instrument; use vector_lib::internal_event::{ @@ -191,7 +188,7 @@ where batch: StatefulBatch>, partitions: HashMap>>, timeout: Duration, - lingers: HashMap>>, + lingers: HashMap, in_flight: Option>>, closing: bool, } @@ -267,8 +264,8 @@ where let batch = self.batch.fresh(); self.partitions.insert(partition.clone(), batch); - let delay = sleep(self.timeout); - self.lingers.insert(partition.clone(), Box::pin(delay)); + let deadline = Instant::now() + self.timeout; + self.lingers.insert(partition.clone(), deadline); }; if let PushResult::Overflow(item) = batch.push(item) { @@ -290,23 +287,27 @@ where let this = self.as_mut().project(); let mut partitions_ready = vec![]; for (partition, batch) in this.partitions.iter() { - if ((*this.closing && !batch.is_empty()) - || batch.was_full() - || matches!( - this.lingers - .get_mut(partition) - .expect("linger should exists for poll_flush") - .poll_unpin(cx), - Poll::Ready(()) - )) - && this - .in_flight - .as_mut() - .and_then(|map| map.get_mut(partition)) - .map(|req| matches!(req.poll_unpin(cx), Poll::Ready(()))) - .unwrap_or(true) + let linger_deadline_met = Instant::now() + >= *this + .lingers + .get(partition) + .expect("linger should exist for poll_flush"); + + let in_flight_ready = this + .in_flight + .as_mut() + .and_then(|map| map.get_mut(partition)) + .map(|req| matches!(req.poll_unpin(cx), Poll::Ready(()))) + .unwrap_or(true); + + if ((*this.closing && !batch.is_empty()) || batch.was_full() || linger_deadline_met) + && in_flight_ready { partitions_ready.push(partition.clone()); + } else if !linger_deadline_met { + // If the deadline is not met, force a wakeup to ensure this partition + // is polled again and the logic above re-evaluated. + cx.waker().wake_by_ref(); } } let mut batch_consumed = false; @@ -316,6 +317,7 @@ where Poll::Ready(Err(error)) => return Poll::Ready(Err(error)), Poll::Pending => false, }; + if service_ready { trace!("Service ready; Sending batch."); @@ -575,7 +577,10 @@ mod tests { use bytes::Bytes; use futures::{future, stream, task::noop_waker_ref, SinkExt, StreamExt}; - use tokio::{task::yield_now, time::Instant}; + use tokio::{ + task::yield_now, + time::{sleep, Instant}, + }; use vector_lib::{ finalization::{BatchNotifier, BatchStatus, EventFinalizer, EventFinalizers}, json_size::JsonSize, From e3e77f0a544d874625557ae68c066c3dec4e17f9 Mon Sep 17 00:00:00 2001 From: LogDNA Bot Date: Wed, 14 Feb 2024 14:00:29 +0000 Subject: [PATCH 02/19] chore(release): 3.7.4 [skip ci] ## [3.7.4](https://github.com/answerbook/vector/compare/v3.7.3...v3.7.4) (2024-02-14) ### Bug Fixes * **mezmo-sink**: remove ARC, replace sleep timer with manual check [8b74d7f](https://github.com/answerbook/vector/commit/8b74d7f2272ddf561f20d4b93656d27ddc4ece4b) - Mike Del Tito [LOG-19184](https://logdna.atlassian.net/browse/LOG-19184) ### Miscellaneous * Merge pull request #420 from answerbook/feature/LOG-19184 [273f6a5](https://github.com/answerbook/vector/commit/273f6a5f19585db8c1dfde9fe6684b133f2ad917) - GitHub [LOG-19184](https://logdna.atlassian.net/browse/LOG-19184) --- MEZMO_CHANGELOG.md | 12 ++++++++++++ package.json | 2 +- 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/MEZMO_CHANGELOG.md b/MEZMO_CHANGELOG.md index 20bab1d36..25913a75c 100644 --- a/MEZMO_CHANGELOG.md +++ b/MEZMO_CHANGELOG.md @@ -1,3 +1,15 @@ +## [3.7.4](https://github.com/answerbook/vector/compare/v3.7.3...v3.7.4) (2024-02-14) + + +### Bug Fixes + +* **mezmo-sink**: remove ARC, replace sleep timer with manual check [8b74d7f](https://github.com/answerbook/vector/commit/8b74d7f2272ddf561f20d4b93656d27ddc4ece4b) - Mike Del Tito [LOG-19184](https://logdna.atlassian.net/browse/LOG-19184) + + +### Miscellaneous + +* Merge pull request #420 from answerbook/feature/LOG-19184 [273f6a5](https://github.com/answerbook/vector/commit/273f6a5f19585db8c1dfde9fe6684b133f2ad917) - GitHub [LOG-19184](https://logdna.atlassian.net/browse/LOG-19184) + ## [3.7.3](https://github.com/answerbook/vector/compare/v3.7.2...v3.7.3) (2024-02-08) diff --git a/package.json b/package.json index eaad0187c..6ad65d3e8 100644 --- a/package.json +++ b/package.json @@ -1,6 +1,6 @@ { "name": "vector", - "version": "3.7.3", + "version": "3.7.4", "description": "Vector is a high-performance, end-to-end (agent & aggregator) observability data pipeline", "repository": { "type": "git", From df7f3f6cf9481766f53ab7cb4325693ce84825d4 Mon Sep 17 00:00:00 2001 From: Adam Holmberg Date: Thu, 15 Feb 2024 16:29:35 -0600 Subject: [PATCH 03/19] feat: spread vector nodes using topo spread constraints This spreads vector nodes more evenly among our worker nodes, instead of following the default STS scheduling, which always puts the zero pod on the first node in the range of candidates. ref: LOG-18832 --- deployment/kubernetes/mtp-vector.yaml.envsubst | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/deployment/kubernetes/mtp-vector.yaml.envsubst b/deployment/kubernetes/mtp-vector.yaml.envsubst index b71efd350..71b282a58 100644 --- a/deployment/kubernetes/mtp-vector.yaml.envsubst +++ b/deployment/kubernetes/mtp-vector.yaml.envsubst @@ -245,6 +245,13 @@ spec: sources: - configMap: name: vector + topologySpreadConstraints: + - maxSkew: 1 + topologyKey: kubernetes.io/hostname + whenUnsatisfiable: ScheduleAnyway + labelSelector: + matchLabels: + app: vector affinity: nodeAffinity: requiredDuringSchedulingIgnoredDuringExecution: From 676ef24bd22e8f1006e2b819821881c8ce4cccf6 Mon Sep 17 00:00:00 2001 From: LogDNA Bot Date: Tue, 20 Feb 2024 21:14:34 +0000 Subject: [PATCH 04/19] chore(release): 3.8.0 [skip ci] # [3.8.0](https://github.com/answerbook/vector/compare/v3.7.4...v3.8.0) (2024-02-20) ### Features * spread vector nodes using topo spread constraints [df7f3f6](https://github.com/answerbook/vector/commit/df7f3f6cf9481766f53ab7cb4325693ce84825d4) - Adam Holmberg [LOG-18832](https://logdna.atlassian.net/browse/LOG-18832) ### Miscellaneous * Merge pull request #421 from answerbook/holmberg/LOG-18832 [6aa2489](https://github.com/answerbook/vector/commit/6aa2489a43843266e54881fa869456fe4f46b0d2) - GitHub [LOG-18832](https://logdna.atlassian.net/browse/LOG-18832) --- MEZMO_CHANGELOG.md | 12 ++++++++++++ package.json | 2 +- 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/MEZMO_CHANGELOG.md b/MEZMO_CHANGELOG.md index 25913a75c..67e342482 100644 --- a/MEZMO_CHANGELOG.md +++ b/MEZMO_CHANGELOG.md @@ -1,3 +1,15 @@ +# [3.8.0](https://github.com/answerbook/vector/compare/v3.7.4...v3.8.0) (2024-02-20) + + +### Features + +* spread vector nodes using topo spread constraints [df7f3f6](https://github.com/answerbook/vector/commit/df7f3f6cf9481766f53ab7cb4325693ce84825d4) - Adam Holmberg [LOG-18832](https://logdna.atlassian.net/browse/LOG-18832) + + +### Miscellaneous + +* Merge pull request #421 from answerbook/holmberg/LOG-18832 [6aa2489](https://github.com/answerbook/vector/commit/6aa2489a43843266e54881fa869456fe4f46b0d2) - GitHub [LOG-18832](https://logdna.atlassian.net/browse/LOG-18832) + ## [3.7.4](https://github.com/answerbook/vector/compare/v3.7.3...v3.7.4) (2024-02-14) diff --git a/package.json b/package.json index 6ad65d3e8..4f5c7e539 100644 --- a/package.json +++ b/package.json @@ -1,6 +1,6 @@ { "name": "vector", - "version": "3.7.4", + "version": "3.8.0", "description": "Vector is a high-performance, end-to-end (agent & aggregator) observability data pipeline", "repository": { "type": "git", From 4326471d8ee7a406fe52f8c450c1d2ba8eaaa61b Mon Sep 17 00:00:00 2001 From: kwabena Date: Fri, 23 Feb 2024 09:56:40 -0500 Subject: [PATCH 05/19] refactor(sinks): Restore healthcheck and response user error logging (#422) User logging of healthcheck and response errors was lost for some sinks during recent changes. The issue was reported for HTTP only but restoration was done for sinks used in the mezmo data pipeline service. Ref: LOG-19146 --- src/sinks/aws_cloudwatch_logs/config.rs | 4 +- src/sinks/aws_cloudwatch_logs/healthcheck.rs | 28 ++++++++-- .../aws_cloudwatch_logs/integration_tests.rs | 3 +- src/sinks/clickhouse/config.rs | 18 ++++++- src/sinks/datadog/events/config.rs | 2 +- src/sinks/datadog/logs/config.rs | 4 +- src/sinks/datadog/metrics/config.rs | 4 +- src/sinks/datadog/mod.rs | 20 ++++++-- src/sinks/datadog/traces/config.rs | 2 +- src/sinks/elasticsearch/common.rs | 20 ++++++-- src/sinks/elasticsearch/config.rs | 2 +- src/sinks/gcp/pubsub.rs | 22 +++++++- src/sinks/gcp/stackdriver/logs/config.rs | 21 ++++++-- src/sinks/gcp/stackdriver/metrics/config.rs | 2 +- src/sinks/honeycomb/config.rs | 19 +++++-- src/sinks/http/config.rs | 33 +++++++++--- src/sinks/loki/config.rs | 4 +- src/sinks/loki/healthcheck.rs | 19 ++++++- src/sinks/loki/tests.rs | 6 ++- src/sinks/prometheus/remote_write/config.rs | 14 ++++- src/sinks/redis/config.rs | 11 ++-- src/sinks/splunk_hec/common/util.rs | 51 ++++++++++++++----- src/sinks/splunk_hec/logs/config.rs | 1 + src/sinks/splunk_hec/metrics/config.rs | 1 + src/sinks/sumo_logic/config.rs | 8 ++- src/sinks/sumo_logic/healthcheck.rs | 18 +++++-- src/sinks/util/http.rs | 26 ++++++++++ 27 files changed, 293 insertions(+), 70 deletions(-) diff --git a/src/sinks/aws_cloudwatch_logs/config.rs b/src/sinks/aws_cloudwatch_logs/config.rs index 38be9a044..e9cb2e08c 100644 --- a/src/sinks/aws_cloudwatch_logs/config.rs +++ b/src/sinks/aws_cloudwatch_logs/config.rs @@ -180,12 +180,12 @@ impl SinkConfig for CloudwatchLogsSinkConfig { self.clone(), client.clone(), std::sync::Arc::new(smithy_client), - cx, + cx.clone(), )); let transformer = self.encoding.transformer(); let serializer = self.encoding.build()?; let encoder = Encoder::<()>::new(serializer); - let healthcheck = healthcheck(self.clone(), client).boxed(); + let healthcheck = healthcheck(self.clone(), client, cx.clone()).boxed(); let sink = CloudwatchSink { batcher_settings, diff --git a/src/sinks/aws_cloudwatch_logs/healthcheck.rs b/src/sinks/aws_cloudwatch_logs/healthcheck.rs index 7a953b2a9..dba248cc9 100644 --- a/src/sinks/aws_cloudwatch_logs/healthcheck.rs +++ b/src/sinks/aws_cloudwatch_logs/healthcheck.rs @@ -3,7 +3,11 @@ use aws_sdk_cloudwatchlogs::types::SdkError; use aws_sdk_cloudwatchlogs::Client as CloudwatchLogsClient; use snafu::Snafu; -use crate::sinks::aws_cloudwatch_logs::config::CloudwatchLogsSinkConfig; +use crate::{ + config::SinkContext, mezmo::user_trace::MezmoUserLog, + sinks::aws_cloudwatch_logs::config::CloudwatchLogsSinkConfig, user_log_error, +}; +use vrl::value::Value; #[allow(clippy::large_enum_variant)] #[derive(Debug, Snafu)] @@ -23,6 +27,7 @@ enum HealthcheckError { pub async fn healthcheck( config: CloudwatchLogsSinkConfig, client: CloudwatchLogsClient, + cx: SinkContext, ) -> crate::Result<()> { let group_name = config.group_name.get_ref().to_owned(); let expected_group_name = group_name.clone(); @@ -43,13 +48,18 @@ pub async fn healthcheck( if name == expected_group_name { Ok(()) } else { - Err(HealthcheckError::GroupNameMismatch { + let err = HealthcheckError::GroupNameMismatch { expected: expected_group_name, name, - } - .into()) + }; + user_log_error!(cx.mezmo_ctx, Value::from(format!("{}", err))); + Err(err.into()) } } else { + user_log_error!( + cx.mezmo_ctx, + Value::from(format!("{}", HealthcheckError::GroupNameError)) + ); Err(HealthcheckError::GroupNameError.into()) } } @@ -61,10 +71,18 @@ pub async fn healthcheck( info!("Skipping healthcheck log group check: `group_name` will be created if missing."); Ok(()) } else { + user_log_error!( + cx.mezmo_ctx, + Value::from(format!("{}", HealthcheckError::NoLogGroup)) + ); Err(HealthcheckError::NoLogGroup.into()) } } }, - Err(source) => Err(HealthcheckError::DescribeLogGroupsFailed { source }.into()), + Err(source) => { + let err = HealthcheckError::DescribeLogGroupsFailed { source }; + user_log_error!(cx.mezmo_ctx, Value::from(format!("{}", err))); + Err(err.into()) + } } } diff --git a/src/sinks/aws_cloudwatch_logs/integration_tests.rs b/src/sinks/aws_cloudwatch_logs/integration_tests.rs index 3091cab25..b19a602bb 100644 --- a/src/sinks/aws_cloudwatch_logs/integration_tests.rs +++ b/src/sinks/aws_cloudwatch_logs/integration_tests.rs @@ -449,9 +449,10 @@ async fn cloudwatch_healthcheck() { auth: Default::default(), acknowledgements: Default::default(), }; + let cx = SinkContext::default(); let client = config.create_client(&ProxyConfig::default()).await.unwrap(); - healthcheck(config, client).await.unwrap(); + healthcheck(config, client, cx).await.unwrap(); } async fn create_client_test() -> CloudwatchLogsClient { diff --git a/src/sinks/clickhouse/config.rs b/src/sinks/clickhouse/config.rs index 0c8cc1646..6cd3d9052 100644 --- a/src/sinks/clickhouse/config.rs +++ b/src/sinks/clickhouse/config.rs @@ -116,7 +116,7 @@ impl SinkConfig for ClickhouseConfig { self.table.clone(), ); - let healthcheck = Box::pin(healthcheck(client, endpoint, auth)); + let healthcheck = Box::pin(healthcheck(client, endpoint, auth, cx)); Ok((VectorSink::from_event_streamsink(sink), healthcheck)) } @@ -139,7 +139,12 @@ fn get_healthcheck_uri(endpoint: &Uri) -> String { uri } -async fn healthcheck(client: HttpClient, endpoint: Uri, auth: Option) -> crate::Result<()> { +async fn healthcheck( + client: HttpClient, + endpoint: Uri, + auth: Option, + cx: SinkContext, +) -> crate::Result<()> { let uri = get_healthcheck_uri(&endpoint); let mut request = Request::get(uri).body(Body::empty()).unwrap(); @@ -148,6 +153,15 @@ async fn healthcheck(client: HttpClient, endpoint: Uri, auth: Option) -> c } let response = client.send(request).await?; + let status = response.status(); + + if status.is_client_error() || status.is_server_error() { + let msg = Value::from(format!( + "Error returned from destination with status code: {}", + status + )); + user_log_error!(cx.mezmo_ctx, msg); + } match response.status() { StatusCode::OK => Ok(()), diff --git a/src/sinks/datadog/events/config.rs b/src/sinks/datadog/events/config.rs index 166881667..02dad4bba 100644 --- a/src/sinks/datadog/events/config.rs +++ b/src/sinks/datadog/events/config.rs @@ -90,7 +90,7 @@ impl DatadogEventsConfig { impl SinkConfig for DatadogEventsConfig { async fn build(&self, cx: SinkContext) -> crate::Result<(VectorSink, Healthcheck)> { let client = self.build_client(cx.proxy())?; - let healthcheck = self.dd_common.build_healthcheck(client.clone())?; + let healthcheck = self.dd_common.build_healthcheck(client.clone(), cx)?; let sink = self.build_sink(client)?; Ok((sink, healthcheck)) diff --git a/src/sinks/datadog/logs/config.rs b/src/sinks/datadog/logs/config.rs index 83e411de0..7ff3cfaa1 100644 --- a/src/sinks/datadog/logs/config.rs +++ b/src/sinks/datadog/logs/config.rs @@ -158,7 +158,9 @@ impl SinkConfig for DatadogLogsConfig { async fn build(&self, cx: SinkContext) -> crate::Result<(VectorSink, Healthcheck)> { let client = self.create_client(&cx.proxy)?; - let healthcheck = self.dd_common.build_healthcheck(client.clone())?; + let healthcheck = self + .dd_common + .build_healthcheck(client.clone(), cx.clone())?; let app_name_slug = cx.app_name_slug.clone(); let sink = self.build_processor(client, cx, app_name_slug)?; diff --git a/src/sinks/datadog/metrics/config.rs b/src/sinks/datadog/metrics/config.rs index db4c060a9..d127e43ce 100644 --- a/src/sinks/datadog/metrics/config.rs +++ b/src/sinks/datadog/metrics/config.rs @@ -181,7 +181,9 @@ impl_generate_config_from_default!(DatadogMetricsConfig); impl SinkConfig for DatadogMetricsConfig { async fn build(&self, cx: SinkContext) -> crate::Result<(VectorSink, Healthcheck)> { let client = self.build_client(&cx.proxy)?; - let healthcheck = self.dd_common.build_healthcheck(client.clone())?; + let healthcheck = self + .dd_common + .build_healthcheck(client.clone(), cx.clone())?; let sink = self.build_sink(client, cx)?; Ok((sink, healthcheck)) diff --git a/src/sinks/datadog/mod.rs b/src/sinks/datadog/mod.rs index 0b8225e3c..ff226ccc7 100644 --- a/src/sinks/datadog/mod.rs +++ b/src/sinks/datadog/mod.rs @@ -9,9 +9,11 @@ use vrl::value::Value; use crate::{ common::datadog::{get_api_base_endpoint, DD_US_SITE}, + config::SinkContext, http::{HttpClient, HttpError}, - mezmo::user_trace::UserLoggingError, + mezmo::user_trace::{MezmoUserLog, UserLoggingError}, sinks::HealthcheckError, + user_log_error, }; use super::Healthcheck; @@ -95,13 +97,13 @@ impl Default for DatadogCommonConfig { impl DatadogCommonConfig { /// Returns a `Healthcheck` which is a future that will be used to ensure the /// `/api/v1/validate` endpoint is reachable. - fn build_healthcheck(&self, client: HttpClient) -> crate::Result { + fn build_healthcheck(&self, client: HttpClient, cx: SinkContext) -> crate::Result { let validate_endpoint = get_api_validate_endpoint(self.endpoint.as_ref(), self.site.as_str())?; let api_key: String = self.default_api_key.clone().into(); - Ok(build_healthcheck_future(client, validate_endpoint, api_key).boxed()) + Ok(build_healthcheck_future(client, validate_endpoint, api_key, cx).boxed()) } } @@ -110,6 +112,7 @@ async fn build_healthcheck_future( client: HttpClient, validate_endpoint: Uri, api_key: String, + cx: SinkContext, ) -> crate::Result<()> { let request = Request::get(validate_endpoint) .header("DD-API-KEY", api_key) @@ -117,8 +120,17 @@ async fn build_healthcheck_future( .unwrap(); let response = client.send(request).await?; + let status = response.status(); + + if status.is_client_error() || status.is_server_error() { + let msg = Value::from(format!( + "Error returned from destination with status code: {}", + status + )); + user_log_error!(cx.mezmo_ctx, msg); + } - match response.status() { + match status { StatusCode::OK => Ok(()), other => Err(HealthcheckError::UnexpectedStatus { status: other }.into()), } diff --git a/src/sinks/datadog/traces/config.rs b/src/sinks/datadog/traces/config.rs index b3fe500ba..f48125719 100644 --- a/src/sinks/datadog/traces/config.rs +++ b/src/sinks/datadog/traces/config.rs @@ -215,7 +215,7 @@ impl DatadogTracesConfig { impl SinkConfig for DatadogTracesConfig { async fn build(&self, cx: SinkContext) -> crate::Result<(VectorSink, Healthcheck)> { let client = self.build_client(&cx.proxy)?; - let healthcheck = self.dd_common.build_healthcheck(client.clone())?; + let healthcheck = self.dd_common.build_healthcheck(client.clone(), cx)?; let sink = self.build_sink(client)?; Ok((sink, healthcheck)) diff --git a/src/sinks/elasticsearch/common.rs b/src/sinks/elasticsearch/common.rs index 27b71e527..fda935b29 100644 --- a/src/sinks/elasticsearch/common.rs +++ b/src/sinks/elasticsearch/common.rs @@ -13,18 +13,21 @@ use super::{ InvalidHostSnafu, Request, }; use crate::{ + config::SinkContext, http::{HttpClient, MaybeAuth}, + mezmo::user_trace::MezmoUserLog, sinks::{ elasticsearch::{ ElasticsearchAuthConfig, ElasticsearchCommonMode, ElasticsearchConfig, ParseError, }, - util::auth::Auth, - util::{http::RequestConfig, TowerRequestConfig, UriSerde}, + util::{auth::Auth, http::RequestConfig, TowerRequestConfig, UriSerde}, HealthcheckError, }, tls::TlsSettings, transforms::metric_to_log::MetricToLog, + user_log_error, }; +use vrl::value::Value; #[derive(Debug, Clone)] pub struct ElasticsearchCommon { @@ -233,7 +236,7 @@ impl ElasticsearchCommon { Ok(commons.remove(0)) } - pub async fn healthcheck(self, client: HttpClient) -> crate::Result<()> { + pub async fn healthcheck(self, client: HttpClient, cx: SinkContext) -> crate::Result<()> { match get( &self.base_url, &self.auth, @@ -245,7 +248,16 @@ impl ElasticsearchCommon { .status() { StatusCode::OK => Ok(()), - status => Err(HealthcheckError::UnexpectedStatus { status }.into()), + status => { + user_log_error!( + cx.mezmo_ctx, + Value::from(format!( + "Error returned from destination with status code: {}", + status, + )) + ); + Err(HealthcheckError::UnexpectedStatus { status }.into()) + } } } } diff --git a/src/sinks/elasticsearch/config.rs b/src/sinks/elasticsearch/config.rs index efed7e5ce..7a9cc4fec 100644 --- a/src/sinks/elasticsearch/config.rs +++ b/src/sinks/elasticsearch/config.rs @@ -519,7 +519,7 @@ impl SinkConfig for ElasticsearchConfig { let healthcheck = futures::future::select_ok( commons .into_iter() - .map(move |common| common.healthcheck(client.clone()).boxed()), + .map(move |common| common.healthcheck(client.clone(), cx.clone()).boxed()), ) .map_ok(|((), _)| ()) .boxed(); diff --git a/src/sinks/gcp/pubsub.rs b/src/sinks/gcp/pubsub.rs index f4d411fa6..599114c48 100644 --- a/src/sinks/gcp/pubsub.rs +++ b/src/sinks/gcp/pubsub.rs @@ -15,6 +15,7 @@ use crate::{ event::Event, gcp::{GcpAuthConfig, GcpAuthenticator, Scope, PUBSUB_URL}, http::HttpClient, + mezmo::user_trace::MezmoUserLog, sinks::{ gcs_common::config::healthcheck_response, util::{ @@ -24,6 +25,7 @@ use crate::{ Healthcheck, UriParseSnafu, VectorSink, }, tls::{TlsConfig, TlsSettings}, + user_log_error, }; #[derive(Debug, Snafu)] @@ -130,7 +132,8 @@ impl SinkConfig for PubsubConfig { let tls_settings = TlsSettings::from_options(&self.tls)?; let client = HttpClient::new(tls_settings, cx.proxy())?; - let healthcheck = healthcheck(client.clone(), sink.uri("")?, sink.auth.clone()).boxed(); + let healthcheck = + healthcheck(client.clone(), sink.uri("")?, sink.auth.clone(), cx.clone()).boxed(); if let Some(creds) = &sink.auth { creds.spawn_regenerate_token(); } @@ -260,6 +263,7 @@ async fn healthcheck( client: HttpClient, uri: Uri, auth: Option, + cx: SinkContext, ) -> crate::Result<()> { let mut request = Request::get(uri).body(Body::empty()).unwrap(); @@ -268,9 +272,23 @@ async fn healthcheck( auth.apply(&mut request); let response = client.send(request).await?; + let status = response.status(); + if status.is_client_error() || status.is_server_error() { + let msg = Value::from(format!( + "Error returned from destination with status code: {}", + status + )); + user_log_error!(cx.mezmo_ctx, msg); + } healthcheck_response(response, HealthcheckError::TopicNotFound.into()) } - None => Err(HealthcheckError::InvalidAuth.into()), + None => { + user_log_error!( + cx.mezmo_ctx, + Value::from(format!("{}", HealthcheckError::InvalidAuth)) + ); + Err(HealthcheckError::InvalidAuth.into()) + } } } diff --git a/src/sinks/gcp/stackdriver/logs/config.rs b/src/sinks/gcp/stackdriver/logs/config.rs index 9e3a82eb4..07f466dc2 100644 --- a/src/sinks/gcp/stackdriver/logs/config.rs +++ b/src/sinks/gcp/stackdriver/logs/config.rs @@ -12,6 +12,7 @@ use crate::{ BoxedRawValue, RealtimeSizeBasedDefaultBatchSettings, }, }, + user_log_error, }; use http::{Request, Uri}; use hyper::Body; @@ -234,11 +235,11 @@ impl SinkConfig for StackdriverConfig { let service = ServiceBuilder::new() .settings(request_limits, http_response_retry_logic()) - .service(service); + .service(MezmoLoggingService::new(service, cx.mezmo_ctx.clone())); let sink = StackdriverLogsSink::new(service, batch_settings, request_builder); - let healthcheck = healthcheck(client, auth.clone(), uri).boxed(); + let healthcheck = healthcheck(client, auth.clone(), uri, cx.clone()).boxed(); auth.spawn_regenerate_token(); @@ -257,7 +258,12 @@ impl SinkConfig for StackdriverConfig { } } -async fn healthcheck(client: HttpClient, auth: GcpAuthenticator, uri: Uri) -> crate::Result<()> { +async fn healthcheck( + client: HttpClient, + auth: GcpAuthenticator, + uri: Uri, + cx: SinkContext, +) -> crate::Result<()> { let entries: Vec = Vec::new(); let events = serde_json::json!({ "entries": entries }); @@ -273,6 +279,15 @@ async fn healthcheck(client: HttpClient, auth: GcpAuthenticator, uri: Uri) -> cr let request = request.map(Body::from); let response = client.send(request).await?; + let status = response.status(); + + if status.is_client_error() || status.is_server_error() { + let msg = Value::from(format!( + "Error returned from destination with status code: {}", + status + )); + user_log_error!(cx.mezmo_ctx, msg); + } healthcheck_response(response, HealthcheckError::NotFound.into()) } diff --git a/src/sinks/gcp/stackdriver/metrics/config.rs b/src/sinks/gcp/stackdriver/metrics/config.rs index 99d03d114..b8aa40bcb 100644 --- a/src/sinks/gcp/stackdriver/metrics/config.rs +++ b/src/sinks/gcp/stackdriver/metrics/config.rs @@ -119,7 +119,7 @@ impl SinkConfig for StackdriverConfig { let service = ServiceBuilder::new() .settings(request_limits, http_response_retry_logic()) - .service(service); + .service(MezmoLoggingService::new(service, cx.mezmo_ctx.clone())); let sink = StackdriverMetricsSink::new(service, batch_settings, request_builder); diff --git a/src/sinks/honeycomb/config.rs b/src/sinks/honeycomb/config.rs index 6dda01074..64570ee90 100644 --- a/src/sinks/honeycomb/config.rs +++ b/src/sinks/honeycomb/config.rs @@ -117,11 +117,11 @@ impl SinkConfig for HoneycombConfig { let service = ServiceBuilder::new() .settings(request_limits, http_response_retry_logic()) - .service(service); + .service(MezmoLoggingService::new(service, cx.mezmo_ctx.clone())); let sink = HoneycombSink::new(service, batch_settings, request_builder); - let healthcheck = healthcheck(uri, self.api_key.clone(), client).boxed(); + let healthcheck = healthcheck(uri, self.api_key.clone(), client, cx.clone()).boxed(); Ok((VectorSink::from_event_streamsink(sink), healthcheck)) } @@ -144,7 +144,12 @@ impl HoneycombConfig { } } -async fn healthcheck(uri: Uri, api_key: SensitiveString, client: HttpClient) -> crate::Result<()> { +async fn healthcheck( + uri: Uri, + api_key: SensitiveString, + client: HttpClient, + cx: SinkContext, +) -> crate::Result<()> { let request = Request::post(uri).header(HTTP_HEADER_HONEYCOMB, api_key.inner()); let body = crate::serde::json::to_bytes(&Vec::::new()) .unwrap() @@ -157,6 +162,14 @@ async fn healthcheck(uri: Uri, api_key: SensitiveString, client: HttpClient) -> let status = res.status(); let body = hyper::body::to_bytes(res.into_body()).await?; + if status.is_client_error() || status.is_server_error() { + let msg = Value::from(format!( + "Error returned from destination with status code: {}", + status + )); + user_log_error!(cx.mezmo_ctx, msg); + } + if status == StatusCode::BAD_REQUEST { Ok(()) } else if status == StatusCode::UNAUTHORIZED { diff --git a/src/sinks/http/config.rs b/src/sinks/http/config.rs index 91add9f4e..e8edef5c8 100644 --- a/src/sinks/http/config.rs +++ b/src/sinks/http/config.rs @@ -11,6 +11,7 @@ use vector_lib::codecs::{ use crate::{ codecs::{EncodingConfigWithFraming, SinkType}, http::{Auth, HttpClient, MaybeAuth}, + mezmo::user_trace::{MezmoLoggingService, MezmoUserLog}, sinks::{ prelude::*, util::{ @@ -172,7 +173,12 @@ impl GenerateConfig for HttpSinkConfig { } } -async fn healthcheck(uri: UriSerde, auth: Option, client: HttpClient) -> crate::Result<()> { +async fn healthcheck( + uri: UriSerde, + auth: Option, + client: HttpClient, + cx: SinkContext, +) -> crate::Result<()> { let auth = auth.choose_one(&uri.auth)?; let uri = uri.with_default_parts(); let mut request = Request::head(&uri.uri).body(Body::empty()).unwrap(); @@ -182,8 +188,17 @@ async fn healthcheck(uri: UriSerde, auth: Option, client: HttpClient) -> c } let response = client.send(request).await?; + let status = response.status(); + + if status.is_client_error() || status.is_server_error() { + let msg = Value::from(format!( + "Error returned from destination with status code: {}", + status + )); + user_log_error!(cx.mezmo_ctx, msg); + } - match response.status() { + match status { StatusCode::OK => Ok(()), status => Err(HealthcheckError::UnexpectedStatus { status }.into()), } @@ -244,12 +259,17 @@ impl SinkConfig for HttpSinkConfig { let (payload_prefix, payload_suffix) = validate_payload_wrapper(&self.payload_prefix, &self.payload_suffix, &encoder)?; + let cx_clone = cx.clone(); let client = self.build_http_client(&cx)?; let healthcheck = match cx.healthcheck.uri { - Some(healthcheck_uri) => { - healthcheck(healthcheck_uri, self.auth.clone(), client.clone()).boxed() - } + Some(healthcheck_uri) => healthcheck( + healthcheck_uri, + self.auth.clone(), + client.clone(), + cx_clone.clone(), + ) + .boxed(), None => future::ok(()).boxed(), }; @@ -293,8 +313,7 @@ impl SinkConfig for HttpSinkConfig { let service = ServiceBuilder::new() .settings(request_limits, http_response_retry_logic()) - .service(service); - + .service(MezmoLoggingService::new(service, cx_clone.mezmo_ctx)); let sink = HttpSink::new(service, batch_settings, request_builder); Ok((VectorSink::from_event_streamsink(sink), healthcheck)) diff --git a/src/sinks/loki/config.rs b/src/sinks/loki/config.rs index 74ce25479..6ef8f720b 100644 --- a/src/sinks/loki/config.rs +++ b/src/sinks/loki/config.rs @@ -194,7 +194,7 @@ impl SinkConfig for LokiConfig { } } - let client = self.build_client(cx)?; + let client = self.build_client(cx.clone())?; let config = LokiConfig { auth: self.auth.choose_one(&self.endpoint.auth)?, @@ -203,7 +203,7 @@ impl SinkConfig for LokiConfig { let sink = LokiSink::new(config.clone(), client.clone())?; - let healthcheck = healthcheck(config, client).boxed(); + let healthcheck = healthcheck(config, client, cx.clone()).boxed(); Ok((VectorSink::from_event_streamsink(sink), healthcheck)) } diff --git a/src/sinks/loki/healthcheck.rs b/src/sinks/loki/healthcheck.rs index af0995526..406eacd03 100644 --- a/src/sinks/loki/healthcheck.rs +++ b/src/sinks/loki/healthcheck.rs @@ -1,5 +1,8 @@ use super::config::LokiConfig; -use crate::http::HttpClient; +use crate::{ + config::SinkContext, http::HttpClient, mezmo::user_trace::MezmoUserLog, user_log_error, +}; +use vrl::value::Value; async fn fetch_status( endpoint: &str, @@ -19,7 +22,11 @@ async fn fetch_status( Ok(client.send(req).await?.status()) } -pub async fn healthcheck(config: LokiConfig, client: HttpClient) -> crate::Result<()> { +pub async fn healthcheck( + config: LokiConfig, + client: HttpClient, + cx: SinkContext, +) -> crate::Result<()> { let status = match fetch_status("ready", &config, &client).await? { // Issue https://github.com/vectordotdev/vector/issues/6463 http::StatusCode::NOT_FOUND => { @@ -29,6 +36,14 @@ pub async fn healthcheck(config: LokiConfig, client: HttpClient) -> crate::Resul status => status, }; + if status.is_client_error() || status.is_server_error() { + let msg = Value::from(format!( + "Error returned from destination with status code: {}", + status + )); + user_log_error!(cx.mezmo_ctx, msg); + } + match status { http::StatusCode::OK => Ok(()), _ => Err(format!("A non-successful status returned: {}", status).into()), diff --git a/src/sinks/loki/tests.rs b/src/sinks/loki/tests.rs index 281e61682..41f0b61b4 100644 --- a/src/sinks/loki/tests.rs +++ b/src/sinks/loki/tests.rs @@ -112,8 +112,9 @@ async fn healthcheck_includes_auth() { let tls = TlsSettings::from_options(&config.tls).expect("could not create TLS settings"); let proxy = ProxyConfig::default(); let client = HttpClient::new(tls, &proxy).expect("could not create HTTP client"); + let cx = SinkContext::default(); - healthcheck(config.clone(), client) + healthcheck(config.clone(), client, cx) .await .expect("healthcheck failed"); @@ -141,8 +142,9 @@ async fn healthcheck_grafana_cloud() { let tls = TlsSettings::from_options(&config.tls).expect("could not create TLS settings"); let proxy = ProxyConfig::default(); let client = HttpClient::new(tls, &proxy).expect("could not create HTTP client"); + let cx = SinkContext::default(); - healthcheck(config, client) + healthcheck(config, client, cx) .await .expect("healthcheck failed"); } diff --git a/src/sinks/prometheus/remote_write/config.rs b/src/sinks/prometheus/remote_write/config.rs index 8f9f06564..ffc71dd12 100644 --- a/src/sinks/prometheus/remote_write/config.rs +++ b/src/sinks/prometheus/remote_write/config.rs @@ -175,6 +175,7 @@ impl SinkConfig for RemoteWriteConfig { endpoint.clone(), self.compression, auth.clone(), + cx.clone(), ) .boxed(); @@ -186,7 +187,7 @@ impl SinkConfig for RemoteWriteConfig { }; let service = ServiceBuilder::new() .settings(request_settings, http_response_retry_logic()) - .service(service); + .service(MezmoLoggingService::new(service, cx.mezmo_ctx)); let sink = RemoteWriteSink { tenant_id: self.tenant_id.clone(), @@ -216,13 +217,22 @@ async fn healthcheck( endpoint: Uri, compression: Compression, auth: Option, + cx: SinkContext, ) -> crate::Result<()> { let body = bytes::Bytes::new(); let request = build_request(http::Method::GET, &endpoint, compression, body, None, auth).await?; let response = client.send(request).await?; + let status = response.status(); + if status.is_client_error() || status.is_server_error() { + let msg = Value::from(format!( + "Error returned from destination with status code: {}", + status + )); + user_log_error!(cx.mezmo_ctx, msg); + } - match response.status() { + match status { http::StatusCode::OK => Ok(()), other => Err(HealthcheckError::UnexpectedStatus { status: other }.into()), } diff --git a/src/sinks/redis/config.rs b/src/sinks/redis/config.rs index 0504cda3c..4db874da4 100644 --- a/src/sinks/redis/config.rs +++ b/src/sinks/redis/config.rs @@ -128,12 +128,12 @@ impl GenerateConfig for RedisSinkConfig { #[async_trait::async_trait] #[typetag::serde(name = "redis")] impl SinkConfig for RedisSinkConfig { - async fn build(&self, _cx: SinkContext) -> crate::Result<(VectorSink, Healthcheck)> { + async fn build(&self, cx: SinkContext) -> crate::Result<(VectorSink, Healthcheck)> { if self.key.is_empty() { return Err("`key` cannot be empty.".into()); } let conn = self.build_client().await.context(RedisCreateFailedSnafu)?; - let healthcheck = RedisSinkConfig::healthcheck(conn.clone()).boxed(); + let healthcheck = RedisSinkConfig::healthcheck(conn.clone(), cx).boxed(); let sink = RedisSink::new(self, conn)?; Ok((super::VectorSink::from_event_streamsink(sink), healthcheck)) } @@ -153,10 +153,13 @@ impl RedisSinkConfig { client.get_tokio_connection_manager().await } - async fn healthcheck(mut conn: ConnectionManager) -> crate::Result<()> { + async fn healthcheck(mut conn: ConnectionManager, cx: SinkContext) -> crate::Result<()> { redis::cmd("PING") .query_async(&mut conn) .await - .map_err(Into::into) + .map_err(|err| { + user_log_error!(cx.mezmo_ctx, Value::from(format!("{}", err))); + err.into() + }) } } diff --git a/src/sinks/splunk_hec/common/util.rs b/src/sinks/splunk_hec/common/util.rs index 8b9875ffd..f90a88b4e 100644 --- a/src/sinks/splunk_hec/common/util.rs +++ b/src/sinks/splunk_hec/common/util.rs @@ -17,7 +17,7 @@ use crate::{ config::SinkContext, http::HttpClient, internal_events::TemplateRenderingError, - mezmo::user_trace::MezmoHttpBatchLoggingService, + mezmo::user_trace::{MezmoHttpBatchLoggingService, MezmoUserLog}, sinks::{ self, util::{http::HttpBatchService, SinkBatchSettings}, @@ -25,7 +25,9 @@ use crate::{ }, template::Template, tls::{TlsConfig, TlsSettings}, + user_log_error, }; +use vrl::value::Value; #[derive(Clone, Copy, Debug, Default)] pub struct SplunkHecDefaultBatchSettings; @@ -96,6 +98,7 @@ pub async fn build_healthcheck( endpoint: String, token: String, client: HttpClient, + cx: SinkContext, ) -> crate::Result<()> { let uri = build_uri(endpoint.as_str(), "/services/collector/health/1.0", None) .context(UriParseSnafu)?; @@ -106,7 +109,15 @@ pub async fn build_healthcheck( .unwrap(); let response = client.send(request).await?; - match response.status() { + let status = response.status(); + if status.is_client_error() || status.is_server_error() { + let msg = Value::from(format!( + "Error returned from destination with status code: {}", + status + )); + user_log_error!(cx.mezmo_ctx, msg); + } + match status { StatusCode::OK => Ok(()), StatusCode::BAD_REQUEST => Err(HealthcheckError::InvalidToken.into()), StatusCode::SERVICE_UNAVAILABLE => Err(HealthcheckError::QueuesFull.into()), @@ -183,13 +194,16 @@ mod tests { Mock, MockServer, ResponseTemplate, }; - use crate::sinks::{ - splunk_hec::common::{ - build_healthcheck, build_uri, create_client, - service::{HttpRequestBuilder, MetadataFields}, - EndpointTarget, HOST_FIELD, SOURCE_FIELD, + use crate::{ + config::SinkContext, + sinks::{ + splunk_hec::common::{ + build_healthcheck, build_uri, create_client, + service::{HttpRequestBuilder, MetadataFields}, + EndpointTarget, HOST_FIELD, SOURCE_FIELD, + }, + util::Compression, }, - util::Compression, }; #[tokio::test] @@ -203,8 +217,9 @@ mod tests { .mount(&mock_server) .await; + let cx = SinkContext::default(); let client = create_client(&None, &ProxyConfig::default()).unwrap(); - let healthcheck = build_healthcheck(mock_server.uri(), "token".to_string(), client); + let healthcheck = build_healthcheck(mock_server.uri(), "token".to_string(), client, cx); assert!(healthcheck.await.is_ok()) } @@ -220,8 +235,9 @@ mod tests { .mount(&mock_server) .await; + let cx = SinkContext::default(); let client = create_client(&None, &ProxyConfig::default()).unwrap(); - let healthcheck = build_healthcheck(mock_server.uri(), "token".to_string(), client); + let healthcheck = build_healthcheck(mock_server.uri(), "token".to_string(), client, cx); assert_eq!( &healthcheck.await.unwrap_err().to_string(), @@ -240,8 +256,9 @@ mod tests { .mount(&mock_server) .await; + let cx = SinkContext::default(); let client = create_client(&None, &ProxyConfig::default()).unwrap(); - let healthcheck = build_healthcheck(mock_server.uri(), "token".to_string(), client); + let healthcheck = build_healthcheck(mock_server.uri(), "token".to_string(), client, cx); assert_eq!( &healthcheck.await.unwrap_err().to_string(), @@ -260,8 +277,9 @@ mod tests { .mount(&mock_server) .await; + let cx = SinkContext::default(); let client = create_client(&None, &ProxyConfig::default()).unwrap(); - let healthcheck = build_healthcheck(mock_server.uri(), "token".to_string(), client); + let healthcheck = build_healthcheck(mock_server.uri(), "token".to_string(), client, cx); assert_eq!( &healthcheck.await.unwrap_err().to_string(), @@ -414,7 +432,7 @@ mod integration_tests { integration_test_helpers::{get_token, splunk_hec_address}, }; use crate::{ - assert_downcast_matches, sinks::splunk_hec::common::HealthcheckError, + assert_downcast_matches, config::SinkContext, sinks::splunk_hec::common::HealthcheckError, test_util::retry_until, }; @@ -423,9 +441,10 @@ mod integration_tests { let client = create_client(&None, &ProxyConfig::default()).unwrap(); let address = splunk_hec_address(); let token = get_token().await; + let cx = SinkContext::default(); retry_until( - || build_healthcheck(address.clone(), token.clone(), client.clone()), + || build_healthcheck(address.clone(), token.clone(), client.clone(), cx.clone()), Duration::from_millis(500), Duration::from_secs(30), ) @@ -435,10 +454,12 @@ mod integration_tests { #[tokio::test] async fn splunk_healthcheck_server_not_listening() { let client = create_client(&None, &ProxyConfig::default()).unwrap(); + let cx = SinkContext::default(); let healthcheck = build_healthcheck( "http://localhost:1111/".to_string(), get_token().await, client, + cx.clone(), ); healthcheck.await.unwrap_err(); @@ -447,10 +468,12 @@ mod integration_tests { #[tokio::test] async fn splunk_healthcheck_server_unavailable() { let client = create_client(&None, &ProxyConfig::default()).unwrap(); + let cx = SinkContext::default(); let healthcheck = build_healthcheck( "http://localhost:5503/".to_string(), get_token().await, client, + cx.clone(), ); let unhealthy = warp::any() diff --git a/src/sinks/splunk_hec/logs/config.rs b/src/sinks/splunk_hec/logs/config.rs index 0507175fc..f84c4ec2c 100644 --- a/src/sinks/splunk_hec/logs/config.rs +++ b/src/sinks/splunk_hec/logs/config.rs @@ -198,6 +198,7 @@ impl SinkConfig for HecLogsSinkConfig { self.endpoint.clone(), self.default_token.inner().to_owned(), client.clone(), + cx.clone(), ) .boxed(); let sink = self.build_processor(client, cx)?; diff --git a/src/sinks/splunk_hec/metrics/config.rs b/src/sinks/splunk_hec/metrics/config.rs index 660b7ac84..edf8e1a42 100644 --- a/src/sinks/splunk_hec/metrics/config.rs +++ b/src/sinks/splunk_hec/metrics/config.rs @@ -150,6 +150,7 @@ impl SinkConfig for HecMetricsSinkConfig { self.endpoint.clone(), self.default_token.inner().to_owned(), client.clone(), + cx.clone(), ) .boxed(); let sink = self.build_processor(client, cx)?; diff --git a/src/sinks/sumo_logic/config.rs b/src/sinks/sumo_logic/config.rs index 36483af28..8161bfa5f 100644 --- a/src/sinks/sumo_logic/config.rs +++ b/src/sinks/sumo_logic/config.rs @@ -183,8 +183,12 @@ impl SinkConfig for SumoLogicSinkConfig { let request_limits = self.request.unwrap_with(&Default::default()); let client = self.build_client(ctx.clone())?; - let healthcheck = - healthcheck(client.clone(), SumoLogicCredentials::from(self).into()).boxed(); + let healthcheck = healthcheck( + client.clone(), + SumoLogicCredentials::from(self).into(), + ctx.clone(), + ) + .boxed(); let service = ServiceBuilder::new() .settings(request_limits, SumoLogicRetry) .service(MezmoLoggingService::new( diff --git a/src/sinks/sumo_logic/healthcheck.rs b/src/sinks/sumo_logic/healthcheck.rs index 3fa0391e0..561e032cb 100644 --- a/src/sinks/sumo_logic/healthcheck.rs +++ b/src/sinks/sumo_logic/healthcheck.rs @@ -1,21 +1,33 @@ use http::Request; use std::sync::Arc; -use crate::{http::HttpClient, sinks::HealthcheckError}; +use crate::{ + config::SinkContext, http::HttpClient, mezmo::user_trace::MezmoUserLog, + sinks::HealthcheckError, user_log_error, +}; +use vrl::value::Value; use super::config::SumoLogicCredentials; pub(crate) async fn healthcheck( client: HttpClient, credentials: Arc, + cx: SinkContext, ) -> crate::Result<()> { let uri = credentials.build_uri()?; let request = Request::post(uri).body(hyper::Body::empty())?; let response = client.send(request).await?; - - match response.status() { + let status = response.status(); + if status.is_client_error() || status.is_server_error() { + let msg = Value::from(format!( + "Error returned from destination with status code: {}", + status + )); + user_log_error!(cx.mezmo_ctx, msg); + } + match status { status if status.is_success() => Ok(()), other => Err(HealthcheckError::UnexpectedStatus { status: other }.into()), } diff --git a/src/sinks/util/http.rs b/src/sinks/util/http.rs index e89cce6f0..ef658ab3c 100644 --- a/src/sinks/util/http.rs +++ b/src/sinks/util/http.rs @@ -785,6 +785,32 @@ where } } +impl UserLoggingResponse for HttpResponse { + fn log_msg(&self) -> Option { + if !self.http_response.status().is_success() { + Some( + format!( + "Error returned from destination with status code: {}", + self.http_response.status() + ) + .into(), + ) + } else { + None + } + } +} + +impl UserLoggingError for crate::Error { + fn log_msg(&self) -> Option { + let msg = match self.downcast_ref::() { + Some(err) => Value::from(format!("{}", err)), + None => Value::from("Request failed".to_string()), + }; + Some(msg) + } +} + #[cfg(test)] mod test { #![allow(clippy::print_stderr)] //tests From fc3cd8a84fe2893fe01e203ee4ece48fb5305047 Mon Sep 17 00:00:00 2001 From: kwabena Date: Fri, 23 Feb 2024 16:19:24 -0500 Subject: [PATCH 06/19] fix(sinks): Enable user error logging for azure blob (#424) Add user error logging for azure blob sink Fix GCP cloud storage error logging Ref: LOG-19360 --- src/sinks/azure_blob/config.rs | 11 ++++++++--- src/sinks/azure_blob/integration_tests.rs | 2 +- src/sinks/azure_common/config.rs | 3 +++ src/sinks/gcs_common/service.rs | 5 +++-- 4 files changed, 15 insertions(+), 6 deletions(-) diff --git a/src/sinks/azure_blob/config.rs b/src/sinks/azure_blob/config.rs index 4f3d027df..09a587bc9 100644 --- a/src/sinks/azure_blob/config.rs +++ b/src/sinks/azure_blob/config.rs @@ -8,6 +8,7 @@ use vector_lib::sensitive_string::SensitiveString; use vrl::value::Value; use super::request_builder::AzureBlobRequestOptions; +use crate::mezmo::user_trace::MezmoLoggingService; use crate::{ codecs::{Encoder, EncodingConfigWithFraming, SinkType, Transformer}, config::{AcknowledgementsConfig, DataType, GenerateConfig, Input, SinkConfig, SinkContext}, @@ -188,9 +189,9 @@ impl SinkConfig for AzureBlobSinkConfig { let healthcheck = azure_common::config::build_healthcheck( self.container_name.clone(), client.clone(), - cx, + cx.clone(), )?; - let sink = self.build_processor(client)?; + let sink = self.build_processor(client, cx)?; Ok((sink, healthcheck)) } @@ -211,13 +212,17 @@ impl AzureBlobSinkConfig { pub fn build_processor( &self, client: Option>, + cx: SinkContext, ) -> crate::Result { let request_limits = self .request .unwrap_with(&TowerRequestConfig::default().rate_limit_num(250)); let service = ServiceBuilder::new() .settings(request_limits, AzureBlobRetryLogic) - .service(AzureBlobService::new(client)); + .service(MezmoLoggingService::new( + AzureBlobService::new(client), + cx.mezmo_ctx, + )); // Configure our partitioning/batching. let batcher_settings = self.batch.into_batcher_settings()?; diff --git a/src/sinks/azure_blob/integration_tests.rs b/src/sinks/azure_blob/integration_tests.rs index 14c5261a1..84b34ec19 100644 --- a/src/sinks/azure_blob/integration_tests.rs +++ b/src/sinks/azure_blob/integration_tests.rs @@ -259,7 +259,7 @@ impl AzureBlobSinkConfig { ) .expect("Failed to create client"); - self.build_processor(Some(client)) + self.build_processor(Some(client), SinkContext::default()) .expect("Failed to create sink") } diff --git a/src/sinks/azure_common/config.rs b/src/sinks/azure_common/config.rs index 6a697bd5d..5a5ea8271 100644 --- a/src/sinks/azure_common/config.rs +++ b/src/sinks/azure_common/config.rs @@ -15,6 +15,7 @@ use vector_lib::{ }; use vrl::value::Value; +use crate::mezmo::user_trace::UserLoggingResponse; use crate::{ config::SinkContext, event::{EventFinalizers, EventStatus, Finalizable}, @@ -90,6 +91,8 @@ impl DriverResponse for AzureBlobResponse { } } +impl UserLoggingResponse for AzureBlobResponse {} + #[derive(Debug, Snafu)] pub enum HealthcheckError { #[snafu(display("Invalid connection string specified"))] diff --git a/src/sinks/gcs_common/service.rs b/src/sinks/gcs_common/service.rs index a5449b2bb..b87baeb4e 100644 --- a/src/sinks/gcs_common/service.rs +++ b/src/sinks/gcs_common/service.rs @@ -114,11 +114,12 @@ impl UserLoggingResponse for GcsResponse { fn log_msg(&self) -> Option { match &self.inner { Some(response) => { - if response.status().is_success() { + let status = response.status(); + if status.is_client_error() || status.is_server_error() { Some( format!( "Error returned from destination with status code: {}", - response.status() + status ) .into(), ) From 2f567709e89c153cb59a0db06e28c7c56a2dc202 Mon Sep 17 00:00:00 2001 From: LogDNA Bot Date: Fri, 23 Feb 2024 21:45:12 +0000 Subject: [PATCH 07/19] chore(release): 3.8.1 [skip ci] ## [3.8.1](https://github.com/answerbook/vector/compare/v3.8.0...v3.8.1) (2024-02-23) ### Bug Fixes * **sinks**: Enable user error logging for azure blob (#424) [fc3cd8a](https://github.com/answerbook/vector/commit/fc3cd8a84fe2893fe01e203ee4ece48fb5305047) - GitHub [LOG-19360](https://logdna.atlassian.net/browse/LOG-19360) ### Code Refactoring * **sinks**: Restore healthcheck and response user error logging (#422) [4326471](https://github.com/answerbook/vector/commit/4326471d8ee7a406fe52f8c450c1d2ba8eaaa61b) - GitHub [LOG-19146](https://logdna.atlassian.net/browse/LOG-19146) --- MEZMO_CHANGELOG.md | 12 ++++++++++++ package.json | 2 +- 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/MEZMO_CHANGELOG.md b/MEZMO_CHANGELOG.md index 67e342482..2e09de56e 100644 --- a/MEZMO_CHANGELOG.md +++ b/MEZMO_CHANGELOG.md @@ -1,3 +1,15 @@ +## [3.8.1](https://github.com/answerbook/vector/compare/v3.8.0...v3.8.1) (2024-02-23) + + +### Bug Fixes + +* **sinks**: Enable user error logging for azure blob (#424) [fc3cd8a](https://github.com/answerbook/vector/commit/fc3cd8a84fe2893fe01e203ee4ece48fb5305047) - GitHub [LOG-19360](https://logdna.atlassian.net/browse/LOG-19360) + + +### Code Refactoring + +* **sinks**: Restore healthcheck and response user error logging (#422) [4326471](https://github.com/answerbook/vector/commit/4326471d8ee7a406fe52f8c450c1d2ba8eaaa61b) - GitHub [LOG-19146](https://logdna.atlassian.net/browse/LOG-19146) + # [3.8.0](https://github.com/answerbook/vector/compare/v3.7.4...v3.8.0) (2024-02-20) diff --git a/package.json b/package.json index 4f5c7e539..fcc1e1fde 100644 --- a/package.json +++ b/package.json @@ -1,6 +1,6 @@ { "name": "vector", - "version": "3.8.0", + "version": "3.8.1", "description": "Vector is a high-performance, end-to-end (agent & aggregator) observability data pipeline", "repository": { "type": "git", From e88c421ba9818b1ab7678be413064dfdc2d36915 Mon Sep 17 00:00:00 2001 From: Dan Hable Date: Sat, 24 Feb 2024 09:00:44 -0600 Subject: [PATCH 08/19] feat(aggregate-v2): Track and expose metadata on event Track a handful of properties about the aggregation and expose them on the produced event. Using these values, a consumer can determine if the event was flushed early, how many events are rolled up into the aggregate and the timeframe that the aggregate would apply to. Ref: LOG-19291 --- src/transforms/mezmo_aggregate_v2/mod.rs | 73 ++++- src/transforms/mezmo_aggregate_v2/tests.rs | 304 +++++++++++++++------ 2 files changed, 274 insertions(+), 103 deletions(-) diff --git a/src/transforms/mezmo_aggregate_v2/mod.rs b/src/transforms/mezmo_aggregate_v2/mod.rs index 79b1713fd..99a1dc1f0 100644 --- a/src/transforms/mezmo_aggregate_v2/mod.rs +++ b/src/transforms/mezmo_aggregate_v2/mod.rs @@ -58,16 +58,21 @@ struct AggregateWindow { } impl AggregateWindow { - const fn new(event: Event, window_start: i64, window_size: i64) -> Self { + fn new(event: Event, window_start: i64, window_size: i64) -> Self { let size_ms = window_start..window_start + window_size; - Self { - size_ms, - event, - flushed: false, - } - } - const fn from_parts(size_ms: Range, event: Event) -> Self { + let mut event = event; + let log_event = event.as_mut_log(); + log_event.insert(".metadata.aggregate.event_count", Value::from(1)); + log_event.insert( + ".metadata.aggregate.start_timestamp", + Value::from(size_ms.start), + ); + log_event.insert( + ".metadata.aggregate.end_timestamp", + Value::from(size_ms.end), + ); + Self { size_ms, event, @@ -101,7 +106,10 @@ impl AggregateWindow { None => (false, self), Some(flush_condition) => { let Self { - size_ms, mut event, .. + size_ms, + mut event, + flushed, + .. } = self; if let Some(Event::Log(prev_event)) = prev_event { let prev_event = prev_event.value().clone(); @@ -109,10 +117,39 @@ impl AggregateWindow { } let (should_flush, mut event) = flush_condition.check(event); event.as_mut_log().remove("%previous"); - (should_flush, Self::from_parts(size_ms, event)) + + let event = Self { + size_ms, + event, + flushed, + }; + (should_flush, event) } } } + + fn increment_event_count(&mut self) { + match self + .event + .as_mut_log() + .get_mut(".metadata.aggregate.event_count") + { + Some(Value::Integer(count)) => *count += 1, + _ => panic!( + "missing event_count metadata: size_ms={:?},flushed={},event={:?}", + self.size_ms, + self.flushed, + serde_json::to_string(&self.event) + ), + } + } + + fn set_flushed(&mut self, ts: i64) { + self.event + .as_mut_log() + .insert(".metadata.aggregate.flush_timestamp", Value::from(ts)); + self.flushed = true; + } } // Create a simple struct that either returns the time from the system clock, @@ -216,12 +253,17 @@ impl MezmoAggregateV2 { hasher.finish() } - /// Executes the aggregation VRL program againt the current accumulated event and the new event. + /// Executes the aggregation VRL program against the current accumulated event and the new event. fn run_merge_vrl(&mut self, accum_event: Event, new_event: Event) -> Result { let (accum_value, mut accum_meta) = accum_event.into_log().into_parts(); let (new_value, new_meta) = new_event.into_log().into_parts(); accum_meta.merge(new_meta); + let aggregate_meta = accum_value + .get(".metadata.aggregate") + .map(ToOwned::to_owned) + .expect("accumulated event should always contain aggregate metadata"); + let mut vrl_target = VrlTarget::LogEvent( Value::from(btreemap! { "accum" => accum_value, @@ -231,10 +273,11 @@ impl MezmoAggregateV2 { ); let timezone = TimeZone::parse("UTC").unwrap(); - let value = + let mut value = self.vrl_runtime .resolve(&mut vrl_target, &self.event_merge_program, &timezone)?; self.vrl_runtime.clear(); + value.insert(".metadata.aggregate", aggregate_meta); Ok(Event::from(LogEvent::from_parts(value, accum_meta))) } @@ -297,6 +340,7 @@ impl MezmoAggregateV2 { Err(e) => error!("dropping event; failed to execute VRL program on event to aggregate: {e}"), Ok(new_acc) => window.event = new_acc }; + window.increment_event_count(); } } @@ -370,16 +414,17 @@ impl MezmoAggregateV2 { // flush_condition check to use when checking the oldest aggregation window. let mut to_flush = new_window_list.drain(0..flush_end); let retained = to_flush.next_back(); - for datum in to_flush { + for mut datum in to_flush { if !datum.flushed { + datum.set_flushed(current_time); output.push(datum.event); } } if let Some(mut retain) = retained { if !retain.flushed { + retain.set_flushed(current_time); output.push(retain.event.clone()); - retain.flushed = true; new_window_list.push_front(retain); } } diff --git a/src/transforms/mezmo_aggregate_v2/tests.rs b/src/transforms/mezmo_aggregate_v2/tests.rs index 552a1d7df..963e55856 100644 --- a/src/transforms/mezmo_aggregate_v2/tests.rs +++ b/src/transforms/mezmo_aggregate_v2/tests.rs @@ -102,11 +102,7 @@ fn log_event(json_event: impl AsRef) -> Event { Event::from(log_event) } -fn counter_event( - name: impl Into, - tags: Option>, - value: f64, -) -> Event { +fn counter(name: impl Into, tags: Option>, value: f64) -> Event { let name = name.into(); let tags = match tags { None => "{}".to_string(), @@ -130,13 +126,31 @@ fn counter_event( log_event(counter) } +fn set_aggregate_meta(event: Event, field: &str, value: i64) -> Event { + let mut event = event; + let field = format!(".metadata.aggregate.{field}"); + event.as_mut_log().insert(&*field, Value::from(value)); + event +} + +fn not_flushed(event: Event, start_ts: i64, end_ts: i64, count: i64) -> Event { + let event = set_aggregate_meta(event, "start_timestamp", start_ts); + let event = set_aggregate_meta(event, "end_timestamp", end_ts); + set_aggregate_meta(event, "event_count", count) +} + +fn as_flushed(event: Event, start_ts: i64, end_ts: i64, flush_ts: i64, count: i64) -> Event { + let event = not_flushed(event, start_ts, end_ts, count); + set_aggregate_meta(event, "flush_timestamp", flush_ts) +} + fn counter_event_custom_timestamp( name: impl Into, tags: Option>, value: f64, timestamp: u32, ) -> Event { - let mut event = counter_event(name, tags, value); + let mut event = counter(name, tags, value); let metadata = json!({ "timestamp": timestamp }); @@ -205,7 +219,7 @@ fn generate_config() { #[tokio::test] async fn record_single_metric() { let mut target = new_aggregator(None, default_aggregator_limits(), None).await; - target.record(counter_event("a", None, 10.0)); + target.record(counter("a", None, 10.0)); assert_eq!(target.data.len(), 1); let key = metric_event_key("a", None); @@ -218,7 +232,10 @@ async fn record_single_metric() { .. } = val.get(0).unwrap(); assert_eq!(5, actual_size_ms.end - actual_size_ms.start); - assert_eq!(*actual_event, counter_event("a", None, 10.0)); + assert_eq!( + *actual_event, + not_flushed(counter("a", None, 10.0), 1, 6, 1) + ); } #[tokio::test] @@ -237,8 +254,8 @@ async fn record_overlapping_windows() { assert_eq!(actual.len(), 2, "number of sliding windows didn't match"); assert_windows_eq( vec![ - counter_event_custom_timestamp("a", None, 7.0, 1), - counter_event_custom_timestamp("a", None, 4.0, 3), + not_flushed(counter_event_custom_timestamp("a", None, 7.0, 1), 1, 6, 2), + not_flushed(counter_event_custom_timestamp("a", None, 4.0, 3), 3, 8, 1), ], actual, ); @@ -247,7 +264,7 @@ async fn record_overlapping_windows() { #[tokio::test] async fn record_nonoverlapping_windows() { let mut target = new_aggregator(None, aggregator_limits_custom_window_size(10), None).await; - target.record(counter_event("a", None, 3.0)); + target.record(counter("a", None, 3.0)); target.record(counter_event_custom_timestamp("a", None, 4.0, 15)); assert_eq!( target.data.len(), @@ -260,8 +277,13 @@ async fn record_nonoverlapping_windows() { assert_eq!(actual.len(), 2, "number of sliding windows didn't match"); assert_windows_eq( vec![ - counter_event("a", None, 3.0), - counter_event_custom_timestamp("a", None, 4.0, 15), + not_flushed(counter("a", None, 3.0), 1, 6, 1), + not_flushed( + counter_event_custom_timestamp("a", None, 4.0, 15), + 15, + 20, + 1, + ), ], actual, ); @@ -270,16 +292,8 @@ async fn record_nonoverlapping_windows() { #[tokio::test] async fn record_group_by_tags() { let mut target = new_aggregator(None, aggregator_limits_custom_window_size(0), None).await; - target.record(counter_event( - "a", - Some(btreemap! { "host" => "host-1"}), - 3.0, - )); - target.record(counter_event( - "a", - Some(btreemap! { "host" => "host-2"}), - 2.0, - )); + target.record(counter("a", Some(btreemap! { "host" => "host-1"}), 3.0)); + target.record(counter("a", Some(btreemap! { "host" => "host-2"}), 2.0)); target.record(counter_event_custom_timestamp( "a", Some(btreemap! { "host" => "host-1"}), @@ -298,8 +312,18 @@ async fn record_group_by_tags() { assert_windows_eq( vec![ // timestamp of event 1 is updated after merge with event 2 - counter_event_custom_timestamp("a", Some(btreemap! { "host" => "host-1"}), 7.0, 3), - counter_event_custom_timestamp("a", Some(btreemap! { "host" => "host-1"}), 4.0, 3), + not_flushed( + counter_event_custom_timestamp("a", Some(btreemap! { "host" => "host-1"}), 7.0, 3), + 1, + 6, + 2, + ), + not_flushed( + counter_event_custom_timestamp("a", Some(btreemap! { "host" => "host-1"}), 4.0, 3), + 3, + 8, + 1, + ), ], actual_events, ); @@ -308,10 +332,11 @@ async fn record_group_by_tags() { let host_1_key = metric_event_key("a", Some(btreemap! { "host" => "host-2"})); let actual_events = target.data.get(&host_1_key).unwrap(); assert_windows_eq( - vec![counter_event( - "a", - Some(btreemap! { "host" => "host-2"}), - 2.0, + vec![not_flushed( + counter("a", Some(btreemap! { "host" => "host-2"}), 2.0), + 1, + 6, + 1, )], actual_events, ); @@ -320,9 +345,9 @@ async fn record_group_by_tags() { #[tokio::test] async fn record_drops_events_when_cardinality_is_exceeded() { let mut target = new_aggregator(None, AggregatorLimits::new(200, 2, 5000, 5), None).await; - target.record(counter_event("a", None, 3.0)); - target.record(counter_event("b", None, 5.0)); - target.record(counter_event("c", None, 6.0)); + target.record(counter("a", None, 3.0)); + target.record(counter("b", None, 5.0)); + target.record(counter("c", None, 6.0)); assert_eq!( target.data.len(), 2, @@ -340,8 +365,8 @@ async fn record_drops_events_when_cardinality_is_exceeded() { #[tokio::test] async fn record_skips_creating_window() { let mut target = new_aggregator(None, aggregator_limits_custom_window_size(10), None).await; - target.record(counter_event("a", None, 3.0)); - target.record(counter_event("b", None, 7.0)); + target.record(counter("a", None, 3.0)); + target.record(counter("b", None, 7.0)); // event overlaps existing window but does not allocate new window target.record(counter_event_custom_timestamp("a", None, 6.0, 4)); assert_eq!( @@ -355,7 +380,12 @@ async fn record_skips_creating_window() { assert_eq!(actual.len(), 1, "number of sliding windows didn't match"); // timestamp is updated after merge of event metadata assert_windows_eq( - vec![counter_event_custom_timestamp("a", None, 9.0, 4)], + vec![not_flushed( + counter_event_custom_timestamp("a", None, 9.0, 4), + 1, + 6, + 2, + )], actual, ); } @@ -363,8 +393,8 @@ async fn record_skips_creating_window() { #[tokio::test] async fn record_creates_new_windows_when_event_exceeds_min_window() { let mut target = new_aggregator(None, aggregator_limits_custom_window_size(10), None).await; - target.record(counter_event("a", None, 3.0)); - target.record(counter_event("b", None, 7.0)); + target.record(counter("a", None, 3.0)); + target.record(counter("b", None, 7.0)); // use explicit timestamp instead of shared atomic value target.record(counter_event_custom_timestamp("a", None, 6.0, 15)); assert_eq!( @@ -378,8 +408,13 @@ async fn record_creates_new_windows_when_event_exceeds_min_window() { assert_eq!(actual.len(), 2, "number of sliding windows didn't match"); assert_windows_eq( vec![ - counter_event("a", None, 3.0), - counter_event_custom_timestamp("a", None, 6.0, 15), + not_flushed(counter("a", None, 3.0), 1, 6, 1), + not_flushed( + counter_event_custom_timestamp("a", None, 6.0, 15), + 15, + 20, + 1, + ), ], actual, ); @@ -396,8 +431,8 @@ async fn flush_when_empty() { #[tokio::test] async fn flush_no_expired() { let mut target = new_aggregator(None, default_aggregator_limits(), None).await; - target.record(counter_event("a", None, 3.0)); - target.record(counter_event("b", None, 3.0)); + target.record(counter("a", None, 3.0)); + target.record(counter("b", None, 3.0)); let mut res = vec![]; target.flush_finalized(&mut res); @@ -407,16 +442,19 @@ async fn flush_no_expired() { #[tokio::test] async fn flush_only_expired() { let mut target = new_aggregator(None, default_aggregator_limits(), None).await; - target.record(counter_event("a", None, 3.0)); - target.record(counter_event("b", None, 3.0)); + target.record(counter("a", None, 3.0)); + target.record(counter("b", None, 3.0)); target.clock.increment_by(10); - target.record(counter_event("a", None, 3.0)); + target.record(counter("a", None, 3.0)); let mut actual_events = vec![]; target.flush_finalized(&mut actual_events); fix_event_ordering(&mut actual_events); assert_events_eq( - vec![counter_event("a", None, 3.0), counter_event("b", None, 3.0)], + vec![ + as_flushed(counter("a", None, 3.0), 1, 6, 11, 1), + as_flushed(counter("b", None, 3.0), 1, 6, 11, 1), + ], actual_events, ); } @@ -429,14 +467,17 @@ async fn flush_on_conditional_value() { None, ) .await; - target.record(counter_event("a", None, 3.0)); - target.record(counter_event("b", None, 3.0)); - target.record(counter_event("a", None, 3.0)); + target.record(counter("a", None, 3.0)); + target.record(counter("b", None, 3.0)); + target.record(counter("a", None, 3.0)); let mut actual_events = vec![]; target.flush_finalized(&mut actual_events); fix_event_ordering(&mut actual_events); - assert_events_eq(vec![counter_event("a", None, 6.0)], actual_events); + assert_events_eq( + vec![as_flushed(counter("a", None, 6.0), 1, 6, 1, 2)], + actual_events, + ); } #[tokio::test] @@ -447,22 +488,10 @@ async fn flush_on_conditional_tag() { None, ) .await; - target.record(counter_event( - "a", - Some(btreemap! { "region" => "foo"}), - 2.0, - )); - target.record(counter_event( - "a", - Some(btreemap! { "region" => "bar"}), - 4.0, - )); - target.record(counter_event( - "b", - Some(btreemap! { "region" => "foo"}), - 6.0, - )); - target.record(counter_event("b", None, 8.0)); + target.record(counter("a", Some(btreemap! { "region" => "foo"}), 2.0)); + target.record(counter("a", Some(btreemap! { "region" => "bar"}), 4.0)); + target.record(counter("b", Some(btreemap! { "region" => "foo"}), 6.0)); + target.record(counter("b", None, 8.0)); assert_eq!(target.data.len(), 4); let mut actual_events = vec![]; @@ -470,8 +499,20 @@ async fn flush_on_conditional_tag() { fix_event_ordering(&mut actual_events); assert_events_eq( vec![ - counter_event("a", Some(btreemap! { "region" => "foo"}), 2.0), - counter_event("b", Some(btreemap! { "region" => "foo"}), 6.0), + as_flushed( + counter("a", Some(btreemap! { "region" => "foo"}), 2.0), + 1, + 6, + 1, + 1, + ), + as_flushed( + counter("b", Some(btreemap! { "region" => "foo"}), 6.0), + 1, + 6, + 1, + 1, + ), ], actual_events, ); @@ -491,9 +532,9 @@ async fn flush_using_prev_value() { None, ) .await; - target.record(counter_event("a", None, 1.0)); + target.record(counter("a", None, 1.0)); target.clock.increment_by(1); - target.record(counter_event("a", None, 1.0)); + target.record(counter("a", None, 1.0)); // Assert that the internal state of the aggregate windows match what we expect for // two events that have not exceeded the trigger condition nor has the window elapsed. @@ -541,7 +582,7 @@ async fn flush_using_prev_value() { // Now record a large event that should trigger the flush condition. This should // flush even without the window expiring. (clock = 6) - target.record(counter_event("a", None, 100.0)); + target.record(counter("a", None, 100.0)); target.flush_finalized(&mut actual_events); assert_eq!(actual_events.len(), 1); let event = actual_events @@ -563,8 +604,8 @@ async fn flush_using_prev_value() { #[tokio::test] async fn flushes_excess_windows_to_stay_within_window_limits() { let mut target = new_aggregator(None, AggregatorLimits::new(2, 5000, 0, 5), None).await; - target.record(counter_event("a", None, 3.0)); - target.record(counter_event("b", None, 3.0)); + target.record(counter("a", None, 3.0)); + target.record(counter("b", None, 3.0)); // use explicit timestamps to force new window allocations target.record(counter_event_custom_timestamp("a", None, 4.0, 12)); target.record(counter_event_custom_timestamp("a", None, 5.0, 13)); @@ -578,10 +619,22 @@ async fn flushes_excess_windows_to_stay_within_window_limits() { fix_event_ordering(&mut actual_events); assert_events_eq( vec![ - counter_event("a", None, 3.0), - counter_event_custom_timestamp("a", None, 22.0, 12), - counter_event_custom_timestamp("a", None, 18.0, 13), - counter_event("b", None, 3.0), + as_flushed(counter("a", None, 3.0), 1, 6, 15, 1), + as_flushed( + counter_event_custom_timestamp("a", None, 22.0, 12), + 12, + 17, + 15, + 4, + ), + as_flushed( + counter_event_custom_timestamp("a", None, 18.0, 13), + 13, + 18, + 15, + 3, + ), + as_flushed(counter("b", None, 3.0), 1, 6, 15, 1), ], actual_events, ); @@ -591,7 +644,7 @@ async fn flushes_excess_windows_to_stay_within_window_limits() { async fn window_alloc_limit_over_time() { let mut target = new_aggregator(None, AggregatorLimits::new(10, 10, 3, 5), None).await; for _ in 0..6 { - target.record(counter_event("a", None, 1.0)); + target.record(counter("a", None, 1.0)); target.clock.increment_by(2); } @@ -621,9 +674,9 @@ async fn window_alloc_limit_over_time() { */ assert_eq!( vec![ - (1..6, counter_event("a", None, 3.0)), - (5..10, counter_event("a", None, 3.0)), - (9..14, counter_event("a", None, 2.0)), + (1..6, not_flushed(counter("a", None, 3.0), 1, 6, 3)), + (5..10, not_flushed(counter("a", None, 3.0), 5, 10, 3)), + (9..14, not_flushed(counter("a", None, 2.0), 9, 14, 2)), ], actual ); @@ -636,8 +689,8 @@ async fn with_initial_state() { let limits = AggregatorLimits::new(1, 5000, 1, 5); let mut target = new_aggregator(None, limits.clone(), state_persistence_base_path).await; - target.record(counter_event("a", None, 3.0)); - target.record(counter_event("b", None, 3.0)); + target.record(counter("a", None, 3.0)); + target.record(counter("b", None, 3.0)); let mut res = vec![]; let initial_data = target.data.clone(); @@ -653,15 +706,18 @@ async fn with_initial_state() { ); let mut new_res = vec![]; - new_target.record(counter_event("a", None, 3.0)); - new_target.record(counter_event("b", None, 3.0)); + new_target.record(counter("a", None, 3.0)); + new_target.record(counter("b", None, 3.0)); new_target.clock.increment_by(10); new_target.flush_finalized(&mut new_res); assert!(!new_res.is_empty()); fix_event_ordering(&mut new_res); assert_events_eq( - vec![counter_event("a", None, 6.0), counter_event("b", None, 6.0)], + vec![ + as_flushed(counter("a", None, 6.0), 1, 6, 11, 2), + as_flushed(counter("b", None, 6.0), 1, 6, 11, 2), + ], new_res, ); } @@ -724,8 +780,43 @@ async fn tumbling_aggregate_behavior() { assert_eq!(1, flushed_events.len()); assert_eq!( vec![ - (1..6, true, log_event(r#"{ "id": "a", "value": 5 }"#)), - (6..11, false, log_event(r#"{ "id": "a", "value": 1 }"#)) + ( + 1..6, + true, + log_event( + r#" + { + "metadata": { + "aggregate": { + "start_timestamp": 1, + "end_timestamp": 6, + "flush_timestamp": 6, + "event_count": 5 + } + }, + "id": "a", + "value": 5 + }"# + ) + ), + ( + 6..11, + false, + log_event( + r#" + { + "metadata": { + "aggregate": { + "start_timestamp": 6, + "end_timestamp": 11, + "event_count": 1 + } + }, + "id": "a", + "value": 1 + }"# + ) + ) ], target .data @@ -758,8 +849,43 @@ async fn tumbling_aggregate_behavior() { assert_eq!(1, flushed_events.len()); assert_eq!( vec![ - (6..11, true, log_event(r#"{ "id": "a", "value": 5 }"#)), - (11..16, false, log_event(r#"{ "id": "a", "value": 1 }"#)) + ( + 6..11, + true, + log_event( + r#" + { + "metadata": { + "aggregate": { + "start_timestamp": 6, + "end_timestamp": 11, + "flush_timestamp": 11, + "event_count": 5 + } + }, + "id": "a", + "value": 5 + }"# + ) + ), + ( + 11..16, + false, + log_event( + r#" + { + "metadata": { + "aggregate": { + "start_timestamp": 11, + "end_timestamp": 16, + "event_count": 1 + } + }, + "id": "a", + "value": 1 + }"# + ) + ) ], target .data From caf8ec693262f0c7d81dcb259d5a3659d92ce678 Mon Sep 17 00:00:00 2001 From: LogDNA Bot Date: Mon, 26 Feb 2024 14:53:26 +0000 Subject: [PATCH 09/19] chore(release): 3.9.0 [skip ci] # [3.9.0](https://github.com/answerbook/vector/compare/v3.8.1...v3.9.0) (2024-02-26) ### Features * **aggregate-v2**: Track and expose metadata on event [e88c421](https://github.com/answerbook/vector/commit/e88c421ba9818b1ab7678be413064dfdc2d36915) - Dan Hable [LOG-19291](https://logdna.atlassian.net/browse/LOG-19291) --- MEZMO_CHANGELOG.md | 7 +++++++ package.json | 2 +- 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/MEZMO_CHANGELOG.md b/MEZMO_CHANGELOG.md index 2e09de56e..49af4d385 100644 --- a/MEZMO_CHANGELOG.md +++ b/MEZMO_CHANGELOG.md @@ -1,3 +1,10 @@ +# [3.9.0](https://github.com/answerbook/vector/compare/v3.8.1...v3.9.0) (2024-02-26) + + +### Features + +* **aggregate-v2**: Track and expose metadata on event [e88c421](https://github.com/answerbook/vector/commit/e88c421ba9818b1ab7678be413064dfdc2d36915) - Dan Hable [LOG-19291](https://logdna.atlassian.net/browse/LOG-19291) + ## [3.8.1](https://github.com/answerbook/vector/compare/v3.8.0...v3.8.1) (2024-02-23) diff --git a/package.json b/package.json index fcc1e1fde..74f87014c 100644 --- a/package.json +++ b/package.json @@ -1,6 +1,6 @@ { "name": "vector", - "version": "3.8.1", + "version": "3.9.0", "description": "Vector is a high-performance, end-to-end (agent & aggregator) observability data pipeline", "repository": { "type": "git", From d933d2800edfde28e2eb19d6c27a84b5431238e4 Mon Sep 17 00:00:00 2001 From: dominic-mcallister-logdna <98114655+dominic-mcallister-logdna@users.noreply.github.com> Date: Tue, 27 Feb 2024 14:14:14 -0500 Subject: [PATCH 10/19] feat(azure blob): add tagging and file consolidation summary: the azure blob sink outputs many small files (under 10 MB) to stave off memory pressure in a SaaS environment. Adding the ability for processing thread to consolidate said files into larger merge files. Handles multiple directory outputs via a base_path indicated by the user and places the merged files into said directories. Handles all 3 outputs -- json, ndjson, and text to be consolidated. To aid the processing, the ability to add tags to documents has been included in the sink ref: LOG-19336 ref: LOG-19337 --- .github/workflows/integration-test-mezmo.yml | 2 +- scripts/integration/azure/test.yaml | 2 +- src/sinks/azure_blob/config.rs | 54 ++ .../file_consolidation_processor.rs | 749 ++++++++++++++++++ .../azure_blob/file_consolidator_async.rs | 196 +++++ src/sinks/azure_blob/integration_tests.rs | 68 +- .../azure_blob/integration_tests_mezmo.rs | 693 +++++++++++++++- src/sinks/azure_blob/mod.rs | 4 + src/sinks/azure_blob/request_builder.rs | 3 + src/sinks/azure_blob/test.rs | 67 ++ src/sinks/azure_common/config.rs | 2 + src/sinks/azure_common/service.rs | 11 +- src/sinks/azure_common/sink.rs | 22 +- 13 files changed, 1838 insertions(+), 35 deletions(-) create mode 100644 src/sinks/azure_blob/file_consolidation_processor.rs create mode 100644 src/sinks/azure_blob/file_consolidator_async.rs diff --git a/.github/workflows/integration-test-mezmo.yml b/.github/workflows/integration-test-mezmo.yml index 88e3932c7..35ff0e447 100644 --- a/.github/workflows/integration-test-mezmo.yml +++ b/.github/workflows/integration-test-mezmo.yml @@ -48,7 +48,7 @@ jobs: include: # Integrations that we are not using/supporting are disabled - test: 'aws' - # - test: 'azure' + - test: 'azure' # - test: 'clickhouse' # - test: 'datadog-agent' # - test: 'datadog-logs' diff --git a/scripts/integration/azure/test.yaml b/scripts/integration/azure/test.yaml index fe9226e2b..bbb48f067 100644 --- a/scripts/integration/azure/test.yaml +++ b/scripts/integration/azure/test.yaml @@ -9,7 +9,7 @@ env: LOGSTASH_ADDRESS: 0.0.0.0:8081 matrix: - version: [3.14.0] + version: [3.29.0] # changes to these files/paths will invoke the integration test in CI # expressions are evaluated using https://github.com/micromatch/picomatch diff --git a/src/sinks/azure_blob/config.rs b/src/sinks/azure_blob/config.rs index 4f3d027df..6b3bfd350 100644 --- a/src/sinks/azure_blob/config.rs +++ b/src/sinks/azure_blob/config.rs @@ -1,3 +1,4 @@ +use std::collections::BTreeMap; use std::sync::Arc; use azure_storage_blobs::prelude::*; @@ -26,6 +27,12 @@ use crate::{ user_log_error, Result, }; +// MEZMO: added dependencies for s3-sink file consolidation +use crate::sinks::azure_blob::file_consolidator_async::{ + FileConsolidationConfig, FileConsolidatorAsync, +}; +use gethostname::gethostname; + /// Configuration for the `azure_blob` sink. #[configurable_component(sink( "azure_blob", @@ -142,6 +149,16 @@ pub struct AzureBlobSinkConfig { skip_serializing_if = "crate::serde::skip_serializing_if_default" )] pub(super) acknowledgements: AcknowledgementsConfig, + + // MEZMO: added configuration for azure-sink file consolidation + #[configurable(derived)] + #[serde(default)] + pub file_consolidation_config: FileConsolidationConfig, + + /// allow the customer to specify tags to be added to the document + #[configurable(derived)] + #[serde(skip_serializing_if = "Option::is_none")] + pub tags: Option>, } pub fn default_blob_prefix() -> Template { @@ -163,6 +180,8 @@ impl GenerateConfig for AzureBlobSinkConfig { batch: BatchConfig::default(), request: TowerRequestConfig::default(), acknowledgements: Default::default(), + file_consolidation_config: Default::default(), + tags: Default::default(), }) .unwrap() } @@ -242,13 +261,18 @@ impl AzureBlobSinkConfig { blob_append_uuid, encoder: (transformer, encoder), compression: self.compression, + tags: self.tags.clone(), }; + // MEZMO: added new file consolidation process for azure blobs + let consolidation_process = self.build_consolidation_process(); + let sink = AzureBlobSink::new( service, request_options, self.key_partitioner()?, batcher_settings, + consolidation_process, ); Ok(VectorSink::from_event_streamsink(sink)) @@ -257,4 +281,34 @@ impl AzureBlobSinkConfig { pub fn key_partitioner(&self) -> crate::Result { Ok(KeyPartitioner::new(self.blob_prefix.clone())) } + + // MEZMO: added process to define setup for azure-blob-sink file consolidation + fn build_consolidation_process(&self) -> Option { + // we can perform consolidation assuming that the process itself is requested via the configuration + // we only want to handle this process on the primary instance of the statefulset + // so we don't have to worry about contention between instances of sinks + let host_name = gethostname().into_string().unwrap(); + if !host_name.ends_with("-0") || !self.file_consolidation_config.enabled { + info!( + message = "azure-blob-sink file consolidation process disabled", + host_name, + config.enabled = self.file_consolidation_config.enabled, + ); + return None; + } else { + info!( + message = "azure-blob-sink file consolidation enabled", + host_name, + config.enabled = self.file_consolidation_config.enabled, + ); + } + + // grab the connection options and config so we can return a new FileConsolidator + let consolidator = FileConsolidatorAsync::new( + self.connection_string.clone(), + self.container_name.clone(), + self.file_consolidation_config.clone(), + ); + Some(consolidator) + } } diff --git a/src/sinks/azure_blob/file_consolidation_processor.rs b/src/sinks/azure_blob/file_consolidation_processor.rs new file mode 100644 index 000000000..ed91bdb65 --- /dev/null +++ b/src/sinks/azure_blob/file_consolidation_processor.rs @@ -0,0 +1,749 @@ +use azure_storage_blobs::prelude::*; +use bytes::{Bytes, BytesMut}; +use flate2::read::GzDecoder; +use futures::StreamExt; + +pub use super::config::AzureBlobSinkConfig; + +use std::{ + cmp, + io::Read, + sync::Arc, + time::{SystemTime, UNIX_EPOCH}, +}; + +use std::io; +use std::io::Cursor; + +use std::collections::HashMap; +use std::path::Path; +use uuid::Uuid; + +use base64::{engine::general_purpose, Engine as _}; + +const ONE_MEGABYTE_USIZE: usize = 1024 * 1024; +const MAX_BLOCKS_IN_PUT_BLOCK: usize = 50_000; + +// handles consolidating the small files within AWS into much larger files +#[derive(Debug)] +pub struct FileConsolidationProcessor<'a> { + container_client: &'a Arc, + container_name: String, + base_path: String, + requested_size_bytes: u64, + output_format: String, +} + +// handles consolidating all the smaller files generated into larger files of the bucket +// a few assumptions: +// 1. the files are tagged with mezmo keys as found in @get_files_to_consolidate +// to keep us from accidentally messing with files the customer had in the bucket +// 2. the files themselves aren't huge (currently the sink limits to 10 MB files) +// so no file size containts are enforced locally for memory issues across the instance +impl<'a> FileConsolidationProcessor<'a> { + pub const fn new( + container_client: &'a Arc, + container_name: String, + base_path: String, + requested_size_bytes: u64, + output_format: String, + ) -> Self { + FileConsolidationProcessor { + container_client, + container_name, + base_path, + requested_size_bytes, + output_format, + } + } + + pub async fn run(self) { + let files_to_consolidate: Vec = match get_files_to_consolidate( + self.container_client, + self.container_name.clone(), + self.base_path.clone(), + self.output_format.clone(), + ) + .await + { + Ok(f) => f, + Err(e) => { + error!( + ?e, + "container={}, base_path={}, Failed to retrieve files to consolidate", + self.container_name.clone(), + self.base_path.clone(), + ); + let empty_files: Vec = Vec::new(); + empty_files + } + }; + + // customers are specifying a base path to consolidate + let files_by_dir: HashMap> = + group_files_by_directory(files_to_consolidate); + + for dir in files_by_dir.keys() { + let mut files = files_by_dir.get(dir).unwrap().clone(); + + // break the files into groups so we can generate a file of requested file size + while files.len() > 1 { + let mut upload_file_parts: Vec = + splice_files_list(self.requested_size_bytes, &mut files); + + // build the new file properties and expiration time + // make sure the process hasn't ran so fast (really should just unit tests) + // that we accidentally overwrite a merge file. + let mut time_since_epoch: u64; + let mut new_file_key: String; + + loop { + time_since_epoch = SystemTime::now() + .duration_since(UNIX_EPOCH) + .unwrap() + .as_secs(); + + // if there's a directory, the trailing slash is not present so add it + let newfile_dir = if dir.is_empty() { + dir.clone() + } else { + format!("{}/", dir.clone()) + }; + + new_file_key = format!("{}merged_{}.log", newfile_dir, time_since_epoch); + + info!( + "container={}, Determining if merge file already exists, file={}", + self.container_name.clone(), + new_file_key.clone(), + ); + + if let Some(response) = self + .container_client + .blob_client(new_file_key.clone()) + .get() + .into_stream() + .next() + .await + { + match response { + Ok(_d) => { + info!( + "container={}, Merged file already exists, file={}", + self.container_name.clone(), + new_file_key.clone(), + ); + tokio::time::sleep(tokio::time::Duration::from_secs(1)).await; + } + Err(_e) => { + // the file doesn't exist, break the loop and move on. + break; + } + } + } else { + // not response from the stream, assuming file doesn't exist + break; + } + } + info!( + "container={}, Starting consolidated file={}", + self.container_name.clone(), + new_file_key.clone(), + ); + + // start grabbing each file and upload it as part of the block + // don't worry about failures here as blocks will expire after 7 days if not consumed. + // https://learn.microsoft.com/en-us/rest/api/storageservices/understanding-block-blobs--append-blobs--and-page-blobs + + // keep track of the processed files to delete + let mut files_to_delete: Vec = Vec::new(); + let is_standard_json_file: bool = self.output_format == "json"; + + let mut block_parts: Vec = Vec::new(); + while let Some(file) = upload_file_parts.pop() { + let prepend_char: Option = + if is_standard_json_file && !files_to_delete.is_empty() { + Some(',') + } else if !files_to_delete.is_empty() { + Some('\n') + } else { + None + }; + + let data = match download_file_as_bytes( + self.container_client, + self.container_name.clone(), + &file, + is_standard_json_file && !block_parts.is_empty(), //trim_open_bracket + is_standard_json_file && !upload_file_parts.is_empty(), //trim_close_bracket + prepend_char, + ) + .await + { + Ok(d) => d, + Err(err) => { + error!( + ?err, + "container={}, Failed to download file={} for merge file={}", + self.container_name.clone(), + file.key.clone(), + new_file_key.clone(), + ); + continue; + } + }; + + // block id is required, must be unique, and has to be a base64 string + let block_id = general_purpose::URL_SAFE_NO_PAD.encode(Uuid::new_v4()); + match self + .container_client + .blob_client(new_file_key.clone()) + .put_block(block_id.clone(), data) + .into_future() + .await + { + Ok(_response) => { + info!( + "container={}, Uploaded block file={} for merge file={}", + self.container_name.clone(), + file.key.clone(), + new_file_key.clone(), + ); + } + Err(err) => { + error!( + ?err, + "container={}, Failed block file={} for merge file={}", + self.container_name.clone(), + file.key.clone(), + new_file_key.clone(), + ); + continue; + } + }; + + // keep track of the blobs that have been successfully uploaded + // note: they're uncommitted right now as they're just uploaded parts + files_to_delete.push(file.key.clone()); + block_parts.push(BlobBlockType::new_uncommitted(block_id.clone())); + } // end handle individual files + + // complete the file with all the parts + if !block_parts.is_empty() { + let mut tags: Tags = Tags::new(); + tags.insert("mezmo_pipeline_merged", "true"); + + let content_type = match self.output_format.as_str() { + "json" => BlobContentType::from("application/json"), + "ndjson" => BlobContentType::from("application/x-ndjson"), + "text" => BlobContentType::from("text/plain"), + _ => BlobContentType::from("application/x-log"), + }; + + match self + .container_client + .blob_client(new_file_key.clone()) + .put_block_list(BlockList { + blocks: block_parts, + }) + .tags(tags) + .content_type(content_type) + .into_future() + .await + { + Ok(_response) => { + info!( + "container={}, Completed merge file={}", + self.container_name.clone(), + new_file_key.clone(), + ); + } + Err(err) => { + error!( + ?err, + "container={}, Failed to complete merge file={}", + self.container_name.clone(), + new_file_key.clone(), + ); + continue; + } + }; + } + + // remove all the files from azure that have been merged into the larger file + for file in files_to_delete { + match self + .container_client + .blob_client(file.clone()) + .delete() + .into_future() + .await + { + Ok(_) => { + info!( + message = format!( + "File={} removed from container={} after merge successful file consolidation into {}", + file.clone(), + self.container_name.clone(), + new_file_key.clone() + ) + ) + } + Err(e) => error!( + ?e, + "container={}, Failed to delete file={} which was merged into {}", + self.container_name.clone(), + file.clone(), + new_file_key.clone() + ), + }; + } + } // end else multipart logic + } // end files to consolidate loop + } //end run +} + +// helper class for the files that we're consolidating into a single file +#[derive(Debug, Clone)] +pub struct ConsolidationFile { + pub compressed: bool, + pub size: u64, + pub key: String, +} + +impl ConsolidationFile { + pub const fn new(compressed: bool, size: u64, key: String) -> ConsolidationFile { + ConsolidationFile { + compressed, + size, + key, + } + } +} + +/* + handles taking in a list of files and grabbing however many + files which combined is the requested size. + @requested_size_bytes: the total size of data requested + @files: the list of files to pick from. + @@returns: a vector of consolidation files +*/ +fn splice_files_list( + requested_size_bytes: u64, + files: &mut Vec, +) -> Vec { + let mut total_bytes: u64 = 0; + + // azure only allows so many blocks in a single file, + // so make sure we don't overshoot that. + let max_files: usize = cmp::min(files.len(), MAX_BLOCKS_IN_PUT_BLOCK); + for i in 0..max_files { + total_bytes += files[i].size; + + // grab files up until the bytes requested + // and make sure to stop at the max of the api + if total_bytes >= requested_size_bytes { + return files.drain(0..i + 1).collect(); + } + } + + std::mem::take(files) +} + +/* + Handles reading the blob container and evaluating the files + which can be merged into larger files + @container_client: the azure blob client + @base_path: the base path for the files + @file_type: the type of files to be merged + @@returns: Vector, the files which can be merged. +*/ +pub async fn get_files_to_consolidate( + container_client: &Arc, + container_name: String, + base_path: String, + file_type: String, +) -> Result, &'static str> { + let mut files_to_consolidate: Vec = Vec::new(); + + // the azure API has the ability to list blobs by tag, + // but its not yet available in the rust version + let mut stream = container_client + .list_blobs() + .prefix(base_path.clone()) + .include_tags(true) + .into_stream(); + + while let Some(value) = stream.next().await { + // there's a method on the api to search blobs by particular tags + // but the results are only for positive results and only include + // the tags searched and doesn't include the file stats + let blobs_response = match value { + Ok(b) => b, + Err(e) => { + error!( + ?e, + "container={}, base_path={}, Failed to retrieve the next stream of blobs", + container_name.clone(), + base_path.clone(), + ); + continue; + } + }; + + let mut blobs: Vec<&Blob> = blobs_response.blobs.blobs().collect(); + blobs.sort_by(|x, y| y.properties.creation_time.cmp(&x.properties.creation_time)); + + for b in blobs { + if let Some(t) = b.tags.clone() { + // set defaults and resolve via tags + let mut mezmo_merged_file = false; + let mut mezmo_produced_file = false; + let mut can_combine = false; + + for tag in t.tag_set.tags { + match tag.key.as_str() { + "mezmo_pipeline_merged" => mezmo_merged_file = true, + "mezmo_pipeline_azure_sink" => mezmo_produced_file = true, + "mezmo_pipeline_azure_type" => { + can_combine = tag.value.as_str() == file_type; + } + _ => (), + } + } + + if !mezmo_merged_file && mezmo_produced_file && can_combine { + let compressed = + b.properties.content_encoding.clone().unwrap_or_default() == "gzip"; + let size = b.properties.content_length; + let key = b.name.clone(); + + files_to_consolidate.push(ConsolidationFile::new(compressed, size, key)); + } + } + } + } + + Ok(files_to_consolidate) +} + +/* + Handles downloading the byte data from the provided file + If the file is compressed, handles also decompressing the document + via gzip compression. + In an effort to merge json files, allow the truncation of brackets + so they can be merged directly into a byte buffer representing a new + json block. + @container_client: the azure blob client + @file: the file to download + @trim_open_bracket: whether to trim leading brackets of json files + @trim_close_bracket: whether to trim close brackets of json files + @prepend_char: any character to prepend to the stream + @@returns: Bytes, the byte data representing the new file +*/ +async fn download_file_as_bytes( + container_client: &Arc, + container_name: String, + file: &ConsolidationFile, + trim_open_bracket: bool, + trim_close_bracket: bool, + prepend_char: Option, +) -> Result { + let b: Bytes = + download_bytes(container_client, container_name.clone(), file.key.clone()).await?; + + let mut vec: Vec; + if file.compressed { + vec = decompress_gzip(&b); + } else { + vec = b.to_vec(); + } + + if trim_open_bracket && !vec.is_empty() { + let i: usize = 0; + while i < vec.len() { + let c = char::from(vec[i]); + if c.is_whitespace() { + vec.remove(0); + continue; + } + if c == '[' { + vec.remove(0); + break; + } + } + } + + if trim_close_bracket && !vec.is_empty() { + loop { + let i = vec.len() - 1; + let c = char::from(vec[i]); + if c.is_whitespace() { + vec.remove(i); + continue; + } + if c == ']' { + vec.remove(i); + break; + } + + break; + } + } + + if prepend_char.is_some() { + vec.insert(0, prepend_char.unwrap() as u8); + } + + let mut buf = BytesMut::with_capacity(0); + buf.extend_from_slice(&vec); + Ok(buf.freeze()) +} + +/* + Handles gzip decompression of the bytes provided. + @bytes: the byte representation of the file + @@returns: the vector representing the decompressed bytes +*/ +fn decompress_gzip(bytes: &Bytes) -> Vec { + //place the bytes into a buffer that'll decode gzip + let cursor = Cursor::new(bytes); + let in_gz = GzDecoder::new(cursor); + let mut in_buf = io::BufReader::with_capacity(ONE_MEGABYTE_USIZE, in_gz); + + // https://web.mit.edu/rust-lang_v1.25/arch/amd64_ubuntu1404/share/doc/rust/html/std/io/struct.BufReader.html + let mut vec: Vec = Vec::new(); + _ = in_buf.read_to_end(&mut vec); + vec +} + +/* + Handles retrieval of the azure blob from storage + @container_client: the azure blob client + @key: the file key + @@returns: the byte data of the file +*/ +async fn download_bytes( + container_client: &Arc, + container_name: String, + key: String, +) -> Result { + static FAILURE: &str = "Failed to download bytes"; + + let mut stream = container_client + .blob_client(key.clone()) + .get() + .into_stream(); + + let mut bytes_mut = BytesMut::with_capacity(0); + + while let Some(response) = stream.next().await { + match response { + Ok(r) => { + let body: Bytes = r.data.collect().await.unwrap(); + bytes_mut.extend_from_slice(&body); + } + Err(e) => { + error!( + ?e, + "container={}, key={}, Failed to retrieve bytes for the file", + container_name.clone(), + key.clone(), + ); + + return Err(FAILURE); + } + } + } + + Ok(bytes_mut.freeze()) +} + +fn group_files_by_directory( + list: Vec, +) -> HashMap> { + let mut hm: HashMap> = HashMap::new(); + for f in &list { + let p = Path::new(&f.key); + + let base_path = match p.parent() { + Some(p_base) => p_base.to_string_lossy().to_string(), + None => "".to_string(), + }; + + if !hm.contains_key(&base_path) { + let vec: Vec = Vec::new(); + hm.insert(base_path.clone(), vec); + } + + let m = hm.get_mut(&base_path).unwrap(); + m.push(f.clone()); + } + + hm +} + +#[cfg(test)] +mod tests { + use bytes::{Bytes, BytesMut}; + use flate2::read::GzEncoder; + use flate2::Compression; + use std::io::Read; + + use crate::sinks::azure_blob::file_consolidation_processor::decompress_gzip; + use crate::sinks::azure_blob::file_consolidation_processor::group_files_by_directory; + use crate::sinks::azure_blob::file_consolidation_processor::splice_files_list; + use crate::sinks::azure_blob::file_consolidation_processor::ConsolidationFile; + + #[test] + fn splice_empty_list() { + let mut files: Vec = Vec::new(); + + let result = splice_files_list(1000, &mut files); + assert_eq!(files.len(), 0); + assert_eq!(result.len(), 0); + } + + #[test] + fn splice_single_item() { + let mut files: Vec = Vec::new(); + for i in 0..1 { + files.push(ConsolidationFile { + compressed: false, + size: 10, + key: i.to_string().to_owned(), + }); + } + + let result = splice_files_list(9, &mut files); + assert_eq!(files.len(), 0); + assert_eq!(result.len(), 1); + } + + #[test] + fn splice_partial_list() { + let mut files: Vec = Vec::new(); + for i in 0..10 { + files.push(ConsolidationFile { + compressed: false, + size: 10, + key: i.to_string().to_owned(), + }); + } + + let result = splice_files_list(40, &mut files); + assert_eq!(files.len(), 6); + assert_eq!(result.len(), 4); + } + + #[test] + fn splice_entire_list() { + let mut files: Vec = Vec::new(); + for i in 0..10 { + files.push(ConsolidationFile { + compressed: false, + size: 10, + key: i.to_string().to_owned(), + }); + } + + let result = splice_files_list(1000, &mut files); + assert_eq!(files.len(), 0); + assert_eq!(result.len(), 10); + } + + #[test] + fn decompress_document_data() { + let hello_world = "hello world".to_owned(); + + // compress the text + let mut ret_vec = [0; 100]; + let mut bytestring = hello_world.as_bytes(); + let mut gz = GzEncoder::new(&mut bytestring, Compression::fast()); + let count = gz.read(&mut ret_vec).unwrap(); + let vec = ret_vec[0..count].to_vec(); + + let mut bytes_mut = BytesMut::with_capacity(0); + bytes_mut.extend_from_slice(&vec); + let bytes = Bytes::from(bytes_mut); + + //decompress + let decompressed = decompress_gzip(&bytes); + let s = std::str::from_utf8(&decompressed).unwrap(); + assert_eq!(hello_world, s); + } + + #[test] + fn test_group_files_by_directory() { + let vec: Vec = vec![ + create_consolidation_file("base_file1.log"), + create_consolidation_file("base_file2.log"), + create_consolidation_file("/file1.log"), + create_consolidation_file("/sub/file2.log"), + create_consolidation_file("/sub/file3.log"), + create_consolidation_file("/sub/file4.log"), + create_consolidation_file("/sub/sub/file5.log"), + create_consolidation_file("/sub/sub/file6.log"), + create_consolidation_file("/sub/sub/file7.log"), + create_consolidation_file("/sub/sub/file8.log"), + ]; + + let group = group_files_by_directory(vec); + let keys = group.keys(); + + assert_eq!(4, keys.len()); + + assert_eq!( + vec!["base_file1.log".to_string(), "base_file2.log".to_string()], + group + .get("") + .unwrap() + .iter() + .map(|cf| cf.key.clone()) + .collect::>() + ); + + assert_eq!( + vec!["/file1.log"], + group + .get("/") + .unwrap() + .iter() + .map(|cf| cf.key.clone()) + .collect::>() + ); + + assert_eq!( + vec!["/sub/file2.log", "/sub/file3.log", "/sub/file4.log"], + group + .get("/sub") + .unwrap() + .iter() + .map(|cf| cf.key.clone()) + .collect::>() + ); + + assert_eq!( + vec![ + "/sub/sub/file5.log", + "/sub/sub/file6.log", + "/sub/sub/file7.log", + "/sub/sub/file8.log", + ], + group + .get("/sub/sub") + .unwrap() + .iter() + .map(|cf| cf.key.clone()) + .collect::>() + ); + } + + fn create_consolidation_file(p: &str) -> ConsolidationFile { + ConsolidationFile { + compressed: false, + size: 10, + key: p.to_string().to_owned(), + } + } +} diff --git a/src/sinks/azure_blob/file_consolidator_async.rs b/src/sinks/azure_blob/file_consolidator_async.rs new file mode 100644 index 000000000..e38cea83e --- /dev/null +++ b/src/sinks/azure_blob/file_consolidator_async.rs @@ -0,0 +1,196 @@ +use crate::sinks::azure_blob::file_consolidation_processor::FileConsolidationProcessor; +use tokio::task::JoinHandle; +use vector_lib::configurable::configurable_component; +use vector_lib::sensitive_string::SensitiveString; + +/// File Consolidation +/// Depending on the configuration of the sink and the throughput of data, +/// Azure may receive hundreds and thousands of files. This is unmanageable from +/// the customer perspective. Instead of increasing the memory or disk footprint +/// locally, allow everything to process and later on combine all the files +/// +/// Assumption(s): +/// 1. All files within the blob directory are of the same format configured +/// to the sink +#[configurable_component] +// #[derive(Clone, Debug, Copy)] +#[derive(Clone, Debug)] +#[serde(deny_unknown_fields)] +pub struct FileConsolidationConfig { + /// boolean indicating if the consolidation process is enabled + pub enabled: bool, + + /// Indicates the file consolidation should occur every 'X' milliseconds + pub process_every_ms: u64, + + /// Indicates the size of the consolidation file that is produced + pub requested_size_bytes: u64, + + /// Indicates the output format (text, json, ndjson) + pub output_format: String, + + /// Indicates the base path to start consolidation + pub base_path: String, +} + +impl Default for FileConsolidationConfig { + fn default() -> Self { + Self { + enabled: false, + process_every_ms: 600000, // 10 min + requested_size_bytes: 500000000, // 500 MB + output_format: "ndjson".to_owned(), + base_path: "".to_owned(), + } + } +} + +// handles consolidating the small files within AWS into much larger files +#[derive(Debug, Default)] +pub struct FileConsolidatorAsync { + connection_string: Option, + container_name: String, + file_consolidation_config: FileConsolidationConfig, + join_handle: Option>, +} + +impl AsRef for FileConsolidatorAsync { + fn as_ref(&self) -> &FileConsolidatorAsync { + self + } +} + +impl FileConsolidatorAsync { + pub const fn new( + connection_string: Option, + container_name: String, + file_consolidation_config: FileConsolidationConfig, + ) -> FileConsolidatorAsync { + FileConsolidatorAsync { + connection_string, + container_name, + file_consolidation_config, + join_handle: None, + } + } + + pub fn start(&mut self) -> bool { + // default situation so the config isn't enabled + if !self.file_consolidation_config.enabled { + return false; + } + + if self.join_handle.is_some() { + info!( + message = + "container_name={}, base_path={} Thread for azure-blob file consolidation already in progress", + container_name = self.container_name, + base_path = self.file_consolidation_config.base_path.clone(), + ); + return false; + } + + info!( + message = "container_name={}, base_path={}, Initiating thread for azure-blob file consolidation", + container_name = self.container_name, + base_path = self.file_consolidation_config.base_path.clone(), + ); + + const TEN_MINUTES_MS: u64 = 10 * 60 * 1000; + + let process_every_ms = if self.file_consolidation_config.process_every_ms > 0 { + self.file_consolidation_config.process_every_ms + } else { + TEN_MINUTES_MS + }; + + let box_connection_string = Box::new(self.connection_string.as_ref().unwrap().clone()); + let box_container_name = Box::new(self.container_name.clone()); + let box_base_path = Box::new(self.file_consolidation_config.base_path.clone()); + let box_requested_size_bytes = + Box::new(self.file_consolidation_config.requested_size_bytes); + let box_output_format = Box::new(self.file_consolidation_config.output_format.clone()); + + let spawned = tokio::spawn(async move { + let client = crate::sinks::azure_common::config::build_client( + Some(box_connection_string.inner().to_string()), + None, + *box_container_name.clone(), + None, + ) + .unwrap(); + + loop { + let start_time = tokio::time::Instant::now(); + + info!( + message = + "container_name={}, base_path={}, Starting azure-blob file consolidation", + bucket = *box_container_name.clone(), + base_path = *box_base_path.clone(), + ); + + let processor = FileConsolidationProcessor::new( + &client, + *box_container_name.clone(), + *box_base_path.clone(), + *box_requested_size_bytes, + *box_output_format.clone(), + ); + + processor.run().await; + info!( + message = + "container_name={}, base_path={}, Completed azure-blob file consolidation", + container_name = *box_container_name.clone(), + base_path = *box_base_path.clone(), + ); + + // determine how long this action took to complete and await + // the duration necessary to restart on the requested interval + let elapsed = start_time.elapsed().as_millis(); + let diff = process_every_ms - elapsed as u64; + if diff > 0 { + info!( + message = + "container_name={}, base_path={}, processing time={} ms, restarting in {} ms", + container_name = *box_container_name.clone(), + base_path = *box_base_path.clone(), + elapsed, + diff + ); + + tokio::time::sleep(tokio::time::Duration::from_millis(diff)).await; + } + } + }); + + self.join_handle = Some(spawned); + true + } + + pub fn stop(&mut self) -> bool { + // default situation so the config isn't enabled + if !self.file_consolidation_config.enabled { + return false; + } + + info!( + message = "Triggering shutdown for azure-blob file consolidation", + container_name = self.container_name.clone(), + base_path = self.file_consolidation_config.base_path.clone() + ); + + if let Some(h) = self.join_handle.take() { + h.abort(); + } + + info!( + message = "Shutdown for azure-blob file consolidation complete", + container_name = self.container_name.clone(), + base_path = self.file_consolidation_config.base_path.clone() + ); + + true + } +} diff --git a/src/sinks/azure_blob/integration_tests.rs b/src/sinks/azure_blob/integration_tests.rs index 14c5261a1..1c451bdba 100644 --- a/src/sinks/azure_blob/integration_tests.rs +++ b/src/sinks/azure_blob/integration_tests.rs @@ -1,6 +1,9 @@ +use bytes::Bytes; use std::{ + collections::BTreeMap, io::{BufRead, BufReader}, num::NonZeroU32, + sync::Arc, }; use azure_core::{error::HttpError, prelude::Range}; @@ -133,8 +136,6 @@ async fn azure_blob_insert_json_into_blob() { } #[tokio::test] -// This test will fail with Azurite blob emulator because of this issue: -// https://github.com/Azure/Azurite/issues/629 async fn azure_blob_insert_lines_into_blob_gzip() { let blob_prefix = format!("lines-gzip/into/blob/{}", random_string(10)); let config = AzureBlobSinkConfig::new_emulator().await; @@ -158,8 +159,6 @@ async fn azure_blob_insert_lines_into_blob_gzip() { #[ignore] #[tokio::test] -// This test will fail with Azurite blob emulator because of this issue: -// https://github.com/Azure/Azurite/issues/629 async fn azure_blob_insert_json_into_blob_gzip() { let blob_prefix = format!("json-gzip/into/blob/{}", random_string(10)); let config = AzureBlobSinkConfig::new_emulator().await; @@ -243,6 +242,8 @@ impl AzureBlobSinkConfig { batch: Default::default(), request: TowerRequestConfig::default(), acknowledgements: Default::default(), + file_consolidation_config: Default::default(), + tags: Default::default(), }; config.ensure_container().await; @@ -336,6 +337,65 @@ impl AzureBlobSinkConfig { } } + pub async fn put_blob( + &self, + filename: String, + content_type: &str, + encoding: &str, + file_tags: Option>, + data: Bytes, + ) { + let client = azure_common::config::build_client( + self.connection_string.clone().map(Into::into), + self.storage_account.clone().map(Into::into), + self.container_name.clone(), + self.endpoint.clone(), + ) + .unwrap(); + + let mut tags: Tags = Tags::new(); + if file_tags.is_some() { + for (key, value) in file_tags.unwrap().iter() { + tags.insert(key, value); + } + } + + client + .blob_client(filename.clone()) + .put_block_blob(data) + .content_type(content_type.to_string()) + .content_encoding(encoding.to_string()) + .tags(tags) + .into_future() + .await + .unwrap(); + } + + pub async fn get_tags(&self, blob: String) -> azure_storage_blobs::prelude::Tags { + let client = azure_common::config::build_client( + self.connection_string.clone().map(Into::into), + self.storage_account.clone().map(Into::into), + self.container_name.clone(), + self.endpoint.clone(), + ) + .unwrap(); + + let response = client.blob_client(blob).get_tags().await.unwrap(); + response.tags + } + + pub async fn get_client(&self) -> Arc { + let client = azure_common::config::build_client( + self.connection_string.clone().map(Into::into), + self.storage_account.clone().map(Into::into), + self.container_name.clone(), + self.endpoint.clone(), + ) + .unwrap(); + + client + } + async fn ensure_container(&self) { let client = azure_common::config::build_client( self.connection_string.clone().map(Into::into), diff --git a/src/sinks/azure_blob/integration_tests_mezmo.rs b/src/sinks/azure_blob/integration_tests_mezmo.rs index fd69161b7..6a5cb7707 100644 --- a/src/sinks/azure_blob/integration_tests_mezmo.rs +++ b/src/sinks/azure_blob/integration_tests_mezmo.rs @@ -2,41 +2,43 @@ // of the event to be based on the `message` property. #![cfg(all(test, feature = "azure-blob-integration-tests"))] +use bytes::{Bytes, BytesMut}; +use flate2::read::GzEncoder; +use flate2::Compression; +use std::io::Read; + use crate::template::Template; use crate::test_util::{random_message_object_events_with_stream, random_string}; use assay::assay; +use std::{collections::BTreeMap, thread, time}; use vector_lib::codecs::{JsonSerializerConfig, MetricTagValues, NewlineDelimitedEncoderConfig}; // Use THEIR implementation - reduces code copying use super::integration_tests::AzureBlobSinkConfig; use crate::mezmo::reshape_log_event_by_message; +use super::file_consolidation_processor::{get_files_to_consolidate, FileConsolidationProcessor}; +use super::file_consolidator_async::{FileConsolidationConfig, FileConsolidatorAsync}; + #[assay( env = [ ("MEZMO_RESHAPE_MESSAGE", "0"), ] )] async fn azure_blob_mezmo_message_reshaping_does_not_happen() { - let blob_prefix = format!("json/into/blob/{}", random_string(10)); - let config = AzureBlobSinkConfig::new_emulator().await; - let config = AzureBlobSinkConfig { - blob_prefix: Template::try_from(blob_prefix.clone()).unwrap(), - encoding: ( - Some(NewlineDelimitedEncoderConfig::new()), - JsonSerializerConfig::new(MetricTagValues::Single), - ) - .into(), - ..config - }; + let config = get_test_config(None, None).await; let (events, stream) = random_message_object_events_with_stream(100, 10, None); config.run_assert(stream).await; - let blobs = config.list_blobs(blob_prefix).await; + let blobs = config.list_blobs(config.blob_prefix.to_string()).await; assert_eq!(blobs.len(), 1); assert!(blobs[0].clone().ends_with(".log")); let (blob, blob_lines) = config.get_blob(blobs[0].clone()).await; - assert_eq!(blob.properties.content_type, String::from("text/plain")); + assert_eq!( + blob.properties.content_type, + String::from("application/x-ndjson") + ); let expected = events .iter() .map(|event| serde_json::to_string(&event.as_log()).unwrap()) @@ -54,26 +56,19 @@ async fn azure_blob_mezmo_message_reshaping_does_not_happen() { ] )] async fn azure_blob_mezmo_message_reshaping_happens() { - let blob_prefix = format!("json/into/blob/{}", random_string(10)); - let config = AzureBlobSinkConfig::new_emulator().await; - let config = AzureBlobSinkConfig { - blob_prefix: Template::try_from(blob_prefix.clone()).unwrap(), - encoding: ( - Some(NewlineDelimitedEncoderConfig::new()), - JsonSerializerConfig::new(MetricTagValues::Single), - ) - .into(), - ..config - }; + let config = get_test_config(None, None).await; let (mut events, stream) = random_message_object_events_with_stream(100, 3, None); config.run_assert(stream).await; - let blobs = config.list_blobs(blob_prefix).await; + let blobs = config.list_blobs(config.blob_prefix.to_string()).await; assert_eq!(blobs.len(), 1); assert!(blobs[0].clone().ends_with(".log")); let (blob, blob_lines) = config.get_blob(blobs[0].clone()).await; - assert_eq!(blob.properties.content_type, String::from("text/plain")); + assert_eq!( + blob.properties.content_type, + String::from("application/x-ndjson") + ); let expected = events .iter_mut() .map(|event| { @@ -84,3 +79,649 @@ async fn azure_blob_mezmo_message_reshaping_happens() { assert_eq!(expected, blob_lines, "Events were properly reshaped"); } + +#[assay( + env = [ + ("MEZMO_RESHAPE_MESSAGE", "2"), + ] + )] +async fn azure_blob_mezmo_tags_are_added() { + let mut tags = BTreeMap::new(); + tags.insert("tag1".to_string(), "value of tag1".to_string()); + tags.insert("tag2".to_string(), "value of tag2".to_string()); + + let config = get_test_config(None, Some(tags)).await; + let (_, stream) = random_message_object_events_with_stream(100, 3, None); + + config.run_assert(stream).await; + + let blobs = config.list_blobs(config.blob_prefix.to_string()).await; + assert_eq!(blobs.len(), 1); + assert!(blobs[0].clone().ends_with(".log")); + + let tags = config.get_tags(blobs[0].clone()).await; + assert_eq!(tags.tag_set.tags.len(), 2); + + assert_eq!( + vec!["tag1".to_owned(), "tag2".to_owned()], + tags.tag_set + .tags + .iter() + .map(|t| t.key.clone()) + .collect::>() + ); + assert_eq!( + vec!["value of tag1".to_owned(), "value of tag2".to_owned()], + tags.tag_set + .tags + .iter() + .map(|t| t.value.clone()) + .collect::>() + ); +} + +#[tokio::test] +async fn azure_file_consolidator_disabled_run() { + // testing the default scenario where the consolidator is disabled + let mut fc: FileConsolidatorAsync = Default::default(); + + let started = fc.start(); + assert!(!started, "started false"); + + thread::sleep(time::Duration::from_millis(500)); + + let stopped = fc.stop(); + assert!(!stopped, "stopped false"); +} + +#[tokio::test] +async fn azure_file_consolidator_enabled_run() { + let config = get_test_config(None, None).await; + + let mut fc = FileConsolidatorAsync::new( + config.connection_string, + config.container_name, + FileConsolidationConfig { + enabled: true, + process_every_ms: 10, + requested_size_bytes: 512000000, + base_path: "".to_owned(), + output_format: "ndjson".to_owned(), + }, + ); + + let started = fc.start(); + assert!(started, "started true"); + + thread::sleep(time::Duration::from_millis(500)); + + let stopped = fc.stop(); + assert!(stopped, "stopped true"); +} + +#[tokio::test] +async fn azure_file_consolidation_process_no_files() { + let config = get_test_config(None, None).await; + let client = config.get_client().await; + let container_name = config.container_name.clone(); + let base_path = config.blob_prefix.clone().to_string(); + let requested_size_bytes = 512000000; + let output_format = "ndjson".to_owned(); + + let files = get_files_to_consolidate( + &client, + container_name.clone(), + base_path.clone(), + output_format.clone(), + ) + .await + .unwrap(); + assert_eq!(files.len(), 0); + + let fcp = FileConsolidationProcessor::new( + &client, + container_name, + base_path, + requested_size_bytes, + output_format, + ); + fcp.run().await; +} + +#[tokio::test] +async fn azure_file_consolidation_process_files_no_tags() { + let config = get_test_config(None, None).await; + let client = config.get_client().await; + let container_name = config.container_name.clone(); + let base_path = config.blob_prefix.clone().to_string(); + let requested_size_bytes = 512000000; + let output_format = "ndjson".to_owned(); + + for _ in 0..3 { + let (_, stream) = random_message_object_events_with_stream(100, 3, None); + config.run_assert(stream).await; + } + + // three files + let blobs = config.list_blobs(config.blob_prefix.to_string()).await; + assert_eq!(blobs.len(), 3); + + // no files found for consolidation + let files = get_files_to_consolidate( + &client, + container_name.clone(), + config.blob_prefix.clone().to_string(), + output_format.clone(), + ) + .await + .unwrap(); + assert_eq!(files.len(), 0); + + let fcp = FileConsolidationProcessor::new( + &client, + container_name, + base_path.clone(), + requested_size_bytes, + output_format, + ); + fcp.run().await; + + // the files haven't changed + let blobs = config.list_blobs(config.blob_prefix.to_string()).await; + assert_eq!(blobs.len(), 3); +} + +#[tokio::test] +async fn azure_file_consolidation_process_tag_filters() { + let config = get_test_config(None, None).await; + let client = config.get_client().await; + let container_name = config.container_name; + let prefix = format!("json/into/blob/{}/", random_string(10)); + + let mut formats: BTreeMap = BTreeMap::new(); + formats.insert("ndjson".to_owned(), 5); + formats.insert("json".to_owned(), 7); + formats.insert("text".to_owned(), 3); + formats.insert("something".to_owned(), 1); + + // create 3 files of each type + for f in formats.keys() { + let mut tags_data: BTreeMap = BTreeMap::new(); + tags_data.insert("mezmo_pipeline_azure_sink".to_owned(), "true".to_owned()); + tags_data.insert("mezmo_pipeline_azure_type".to_owned(), f.to_string()); + let tags = Some(tags_data); + + let config = get_test_config(Some(prefix.clone()), tags).await; + for _ in 0..formats[f] { + let (_, stream) = random_message_object_events_with_stream(100, 1, None); + config.run_assert(stream).await; + } + } + + for f in formats.keys() { + let files = + get_files_to_consolidate(&client, container_name.clone(), prefix.clone(), f.clone()) + .await + .unwrap(); + assert_eq!(files.len(), formats[f]); + } +} + +#[tokio::test] +async fn azure_file_consolidation_process_text_files() { + let prefix = format!("json/into/blob/{}/text-files/", random_string(10)); + let config = get_test_config(Some(prefix.clone()), None).await; + let output_format = "text".to_owned(); + + let lines = vec![ + "this is some plaintext", + "written to individual files", + "for the purpose", + "of unit testing", + "successful processes", + ]; + + for line in lines.iter() { + let mut tags: BTreeMap = BTreeMap::new(); + tags.insert("mezmo_pipeline_azure_sink".to_owned(), "true".to_owned()); + tags.insert( + "mezmo_pipeline_azure_type".to_owned(), + output_format.clone(), + ); + + let filename = format!("{}{}.log", prefix.clone(), random_string(10)); + let data = line.as_bytes(); + config + .put_blob(filename, "text/plain", "None", Some(tags), data.into()) + .await; + + // forcing a sleep so the file sorting is tested correctly as the + // emulator only keeps time to the second + thread::sleep(time::Duration::from_millis(1000)); + } + + let client = config.get_client().await; + let container_name = config.container_name.clone(); + let base_path = prefix.clone(); + let requested_size_bytes = 512000000; + + let files = get_files_to_consolidate( + &client, + container_name.clone(), + base_path.clone(), + output_format.clone(), + ) + .await + .unwrap(); + assert_eq!(files.len(), 5); + + let fcp = FileConsolidationProcessor::new( + &client, + container_name.clone(), + base_path.clone(), + requested_size_bytes, + output_format.clone(), + ); + fcp.run().await; + + // no more files should be found + let files = get_files_to_consolidate( + &client, + container_name.clone(), + prefix.clone(), + output_format.clone(), + ) + .await + .unwrap(); + assert_eq!(files.len(), 0); + + // list the files + let blobs = config.list_blobs(prefix.clone()).await; + assert_eq!(blobs.len(), 1); + assert!(blobs[0].clone().ends_with(".log")); + assert!(blobs[0].clone().contains("merged")); + + let (blob, blob_lines) = config.get_blob(blobs[0].clone()).await; + assert_eq!(blob.properties.content_type, String::from("text/plain")); + assert_eq!(blob_lines.len(), lines.len()); + assert_eq!(blob_lines, lines); +} + +#[tokio::test] +async fn azure_file_consolidation_process_json_files() { + let prefix = format!("json/into/blob/{}/json-files/", random_string(10)); + let config = get_test_config(Some(prefix.clone()), None).await; + + let lines = vec![ + "[{ \"message\": \"this is some json\" }]", + "[{ \"message\": \"written to individual files\" }]", + "[{ \"message\": \"for the purpose\" }]", + "[{ \"message\": \"of unit testing\" }]", + "[{ \"message\": \"successful processes\" }]", + ]; + + for line in lines.iter() { + let mut tags: BTreeMap = BTreeMap::new(); + tags.insert("mezmo_pipeline_azure_sink".to_owned(), "true".to_owned()); + tags.insert("mezmo_pipeline_azure_type".to_owned(), "json".to_string()); + + let filename = format!("{}{}.log", prefix.clone(), random_string(10)); + let data = line.as_bytes(); + config + .put_blob( + filename, + "application/json", + "None", + Some(tags), + data.into(), + ) + .await; + + // forcing a sleep so the file sorting is tested correctly as the + // emulator only keeps time to the second + thread::sleep(time::Duration::from_millis(1000)); + } + + let client = config.get_client().await; + let container_name = config.container_name.clone(); + let base_path = prefix.clone(); + let requested_size_bytes = 512000000; + let output_format = "json".to_owned(); + + let files = get_files_to_consolidate( + &client, + container_name.clone(), + base_path.clone(), + output_format.clone(), + ) + .await + .unwrap(); + assert_eq!(files.len(), 5); + + let fcp = FileConsolidationProcessor::new( + &client, + container_name.clone(), + base_path.clone(), + requested_size_bytes, + output_format.clone(), + ); + fcp.run().await; + + // no more files should be found + let files = get_files_to_consolidate( + &client, + container_name.clone(), + prefix.clone(), + output_format.clone(), + ) + .await + .unwrap(); + assert_eq!(files.len(), 0); + + // list the files + let blobs = config.list_blobs(prefix.clone()).await; + assert_eq!(blobs.len(), 1); + assert!(blobs[0].clone().ends_with(".log")); + assert!(blobs[0].clone().contains("merged")); + + let (blob, blob_lines) = config.get_blob(blobs[0].clone()).await; + assert_eq!( + blob.properties.content_type, + String::from("application/json") + ); + + // all the json files concatinated together as a new json string + assert_eq!(blob_lines.len(), 1); + assert_eq!( + blob_lines[0], + "[{ \"message\": \"this is some json\" },{ \"message\": \"written to individual files\" },{ \"message\": \"for the purpose\" },{ \"message\": \"of unit testing\" },{ \"message\": \"successful processes\" }]", + ); +} + +#[tokio::test] +async fn azure_file_consolidation_process_ndjson_files() { + let prefix = format!("json/into/blob/{}/ndjson-files/", random_string(10)); + let config = get_test_config(Some(prefix.clone()), None).await; + + let lines = vec![ + "{\"message\": \"this is some json\"}", + "{\"message\": \"that is building\"}", + "{\"message\": \"individual files\"}", + "{\"message\": \"that may have some data\"}", + "{\"message\": \"for testing purposes\"}", + ]; + + for line in lines.iter() { + let mut tags: BTreeMap = BTreeMap::new(); + tags.insert("mezmo_pipeline_azure_sink".to_owned(), "true".to_owned()); + tags.insert("mezmo_pipeline_azure_type".to_owned(), "ndjson".to_string()); + + let filename = format!("{}{}.log", prefix.clone(), random_string(10)); + let data = line.as_bytes(); + config + .put_blob( + filename, + "application/x-ndjson", + "None", + Some(tags), + data.into(), + ) + .await; + + // forcing a sleep so the file sorting is tested correctly as the + // emulator only keeps time to the second + thread::sleep(time::Duration::from_millis(1000)); + } + + let client = config.get_client().await; + let container_name = config.container_name.clone(); + let base_path = prefix.clone(); + let requested_size_bytes = 512000000; + let output_format = "ndjson".to_owned(); + + let files = get_files_to_consolidate( + &client, + container_name.clone(), + base_path.clone(), + output_format.clone(), + ) + .await + .unwrap(); + assert_eq!(files.len(), 5); + + let fcp = FileConsolidationProcessor::new( + &client, + container_name.clone(), + base_path.clone(), + requested_size_bytes, + output_format.clone(), + ); + fcp.run().await; + + // no more files should be found + let files = get_files_to_consolidate( + &client, + container_name.clone(), + prefix.clone(), + output_format.clone(), + ) + .await + .unwrap(); + assert_eq!(files.len(), 0); + + // list the files + let blobs = config.list_blobs(prefix.clone()).await; + assert_eq!(blobs.len(), 1); + assert!(blobs[0].clone().ends_with(".log")); + assert!(blobs[0].clone().contains("merged")); + + let (blob, blob_lines) = config.get_blob(blobs[0].clone()).await; + assert_eq!( + blob.properties.content_type, + String::from("application/x-ndjson") + ); + + // all the json lines concatinated together as newlines + assert_eq!(blob_lines.len(), 5); + assert_eq!(blob_lines, lines); +} + +#[tokio::test] +async fn azure_file_consolidation_process_compressed_files() { + let prefix = format!("json/into/blob/{}/compressed-files/", random_string(10)); + let config = get_test_config(Some(prefix.clone()), None).await; + let output_format = "text".to_owned(); + + let lines = vec![ + "this is some text", + "that will be compressed", + "so we prove that gzip", + "files are handled appropriately", + "when merged together", + ]; + + for line in lines.iter() { + let mut tags: BTreeMap = BTreeMap::new(); + tags.insert("mezmo_pipeline_azure_sink".to_owned(), "true".to_owned()); + tags.insert( + "mezmo_pipeline_azure_type".to_owned(), + output_format.clone(), + ); + + let filename = format!("{}{}.log.gz", prefix.clone(), random_string(10)); + + // compress the data + let mut ret_vec = [0; 100]; + let mut bytestring = line.as_bytes(); + let mut gz = GzEncoder::new(&mut bytestring, Compression::fast()); + let count = gz.read(&mut ret_vec).unwrap(); + let vec = ret_vec[0..count].to_vec(); + + let mut bytes_mut = BytesMut::with_capacity(0); + bytes_mut.extend_from_slice(&vec); + let data = Bytes::from(bytes_mut); + + config + .put_blob(filename, "text/plain", "gzip", Some(tags), data.into()) + .await; + + // forcing a sleep so the file sorting is tested correctly as the + // emulator only keeps time to the second + thread::sleep(time::Duration::from_millis(1000)); + } + + let client = config.get_client().await; + let container_name = config.container_name.clone(); + let base_path = prefix.clone(); + let requested_size_bytes = 512000000; + + let files = get_files_to_consolidate( + &client, + container_name.clone(), + base_path.clone(), + output_format.clone(), + ) + .await + .unwrap(); + assert_eq!(files.len(), 5); + + let fcp = FileConsolidationProcessor::new( + &client, + container_name.clone(), + base_path.clone(), + requested_size_bytes, + output_format.clone(), + ); + fcp.run().await; + + // no more files should be found + let files = get_files_to_consolidate( + &client, + container_name.clone(), + prefix.clone(), + output_format.clone(), + ) + .await + .unwrap(); + assert_eq!(files.len(), 0); + + // list the files + let blobs = config.list_blobs(prefix.clone()).await; + assert_eq!(blobs.len(), 1); + assert!(blobs[0].clone().ends_with(".log")); + assert!(blobs[0].clone().contains("merged")); + + let (blob, blob_lines) = config.get_blob(blobs[0].clone()).await; + assert_eq!(blob.properties.content_type, String::from("text/plain")); + + // all the json lines concatinated together as newlines + assert_eq!(blob_lines.len(), 5); + assert_eq!(blob_lines, lines); +} + +#[tokio::test] +async fn azure_file_consolidation_process_file_size_limits() { + let prefix = format!("json/into/blob/{}/max-file-sizes/", random_string(10)); + let config = get_test_config(Some(prefix.clone()), None).await; + let output_format = "text".to_owned(); + + // create 10x 10 KB files + for _ in 0..10 { + let hundred_bytes = random_string(100); + + let mut five_mb = BytesMut::new(); + for _ in 0..100 { + five_mb.extend_from_slice(hundred_bytes.as_bytes()); + five_mb.extend_from_slice(b"\n"); + } + + let data = Bytes::from(five_mb.to_vec()); + + let mut tags: BTreeMap = BTreeMap::new(); + tags.insert("mezmo_pipeline_azure_sink".to_owned(), "true".to_owned()); + tags.insert( + "mezmo_pipeline_azure_type".to_owned(), + output_format.clone(), + ); + + let filename = format!("{}{}.log", prefix.clone(), random_string(10)); + config + .put_blob(filename, "text/plain", "None", Some(tags), data.into()) + .await; + } + + let client = config.get_client().await; + let container_name = config.container_name.clone(); + let base_path = prefix.clone(); + let requested_size_bytes = 40000; // 40 KB + + let files = get_files_to_consolidate( + &client, + container_name.clone(), + base_path.clone(), + output_format.clone(), + ) + .await + .unwrap(); + assert_eq!(files.len(), 10); + + let fcp = FileConsolidationProcessor::new( + &client, + container_name.clone(), + base_path.clone(), + requested_size_bytes, + output_format.clone(), + ); + fcp.run().await; + + // no more files should be found + let files = get_files_to_consolidate( + &client, + container_name.clone(), + prefix.clone(), + output_format.clone(), + ) + .await + .unwrap(); + assert_eq!(files.len(), 0); + + // list the files + let blobs = config.list_blobs(prefix.clone()).await; + assert_eq!(blobs.len(), 3); + + let mut lines_count = 0; + for b in blobs { + assert!(b.clone().ends_with(".log")); + assert!(b.clone().contains("merged")); + + let (blob, blob_lines) = config.get_blob(b.clone()).await; + assert_eq!(blob.properties.content_type, String::from("text/plain")); + + lines_count += blob_lines.len(); + } + + // each file plus newlines between + assert_eq!(lines_count, 1007); +} + +async fn get_test_config( + prefix: Option, + tags: Option>, +) -> AzureBlobSinkConfig { + let blob_prefix = if prefix.is_some() { + prefix.unwrap() + } else { + format!("unittest/{}/", random_string(10)) + }; + let config = AzureBlobSinkConfig::new_emulator().await; + AzureBlobSinkConfig { + blob_prefix: Template::try_from(blob_prefix.clone()).unwrap(), + encoding: ( + Some(NewlineDelimitedEncoderConfig::new()), + JsonSerializerConfig::new(MetricTagValues::Single), + ) + .into(), + tags, + ..config + } +} diff --git a/src/sinks/azure_blob/mod.rs b/src/sinks/azure_blob/mod.rs index f3124668a..9c6ea1767 100644 --- a/src/sinks/azure_blob/mod.rs +++ b/src/sinks/azure_blob/mod.rs @@ -1,6 +1,10 @@ mod config; mod request_builder; +// MEZMO: added files for azure-blob-sink file consolidation +pub mod file_consolidation_processor; +pub mod file_consolidator_async; + #[cfg(all(test, feature = "azure-blob-integration-tests"))] pub mod integration_tests; mod integration_tests_mezmo; diff --git a/src/sinks/azure_blob/request_builder.rs b/src/sinks/azure_blob/request_builder.rs index 507874ab4..233c84c72 100644 --- a/src/sinks/azure_blob/request_builder.rs +++ b/src/sinks/azure_blob/request_builder.rs @@ -1,5 +1,6 @@ use bytes::Bytes; use chrono::Utc; +use std::collections::BTreeMap; use uuid::Uuid; use vector_lib::codecs::encoding::Framer; use vector_lib::request_metadata::RequestMetadata; @@ -24,6 +25,7 @@ pub struct AzureBlobRequestOptions { pub blob_append_uuid: bool, pub encoder: (Transformer, Encoder), pub compression: Compression, + pub tags: Option>, } impl RequestBuilder<(String, Vec)> for AzureBlobRequestOptions { @@ -96,6 +98,7 @@ impl RequestBuilder<(String, Vec)> for AzureBlobRequestOptions { content_type: self.encoder.1.content_type(), metadata: azure_metadata, request_metadata, + tags: self.tags.clone(), } } } diff --git a/src/sinks/azure_blob/test.rs b/src/sinks/azure_blob/test.rs index dc0736690..7e1e30c90 100644 --- a/src/sinks/azure_blob/test.rs +++ b/src/sinks/azure_blob/test.rs @@ -29,6 +29,8 @@ fn default_config(encoding: EncodingConfigWithFraming) -> AzureBlobSinkConfig { batch: Default::default(), request: Default::default(), acknowledgements: Default::default(), + file_consolidation_config: Default::default(), + tags: Default::default(), } } @@ -37,6 +39,66 @@ fn generate_config() { crate::test_util::test_generate_config::(); } +#[test] +fn azure_blob_build_request_with_tags() { + use std::collections::BTreeMap; + + let log = Event::Log(LogEvent::from("test message")); + let compression = Compression::None; + let container_name = String::from("logs"); + let sink_config = AzureBlobSinkConfig { + blob_prefix: "blob".try_into().unwrap(), + container_name: container_name.clone(), + ..default_config((None::, TextSerializerConfig::default()).into()) + }; + let blob_time_format = String::from(""); + let blob_append_uuid = false; + + let key = sink_config + .key_partitioner() + .unwrap() + .partition(&log) + .expect("key wasn't provided"); + + let mut tags = BTreeMap::new(); + tags.insert("tag1".to_string(), "value of tag1".to_string()); + tags.insert("tag2".to_string(), "value of tag2".to_string()); + + let request_options = AzureBlobRequestOptions { + container_name, + blob_time_format, + blob_append_uuid, + encoder: ( + Default::default(), + Encoder::::new( + NewlineDelimitedEncoder::new().into(), + TextSerializerConfig::default().build().into(), + ), + ), + compression, + tags: Some(tags), + }; + + let mut byte_size = GroupedCountByteSize::new_untagged(); + byte_size.add_event(&log, log.estimated_json_encoded_size_of()); + + let (metadata, request_metadata_builder, _events) = + request_options.split_input((key, vec![log])); + + let payload = EncodeResult::uncompressed(Bytes::new(), byte_size); + let request_metadata = request_metadata_builder.build(&payload); + let request = request_options.build_request(metadata, request_metadata, payload); + + assert_eq!(request.metadata.partition_key, "blob.log".to_string()); + assert_eq!(request.content_encoding, None); + assert_eq!(request.content_type, "text/plain"); + + assert!(request.tags.is_some()); + let t = request.tags.unwrap(); + assert_eq!(t.get("tag1").unwrap(), "value of tag1"); + assert_eq!(t.get("tag2").unwrap(), "value of tag2"); +} + #[test] fn azure_blob_build_request_without_compression() { let log = Event::Log(LogEvent::from("test message")); @@ -68,6 +130,7 @@ fn azure_blob_build_request_without_compression() { ), ), compression, + tags: None, }; let mut byte_size = GroupedCountByteSize::new_untagged(); @@ -116,6 +179,7 @@ fn azure_blob_build_request_with_compression() { ), ), compression, + tags: None, }; let mut byte_size = GroupedCountByteSize::new_untagged(); @@ -164,6 +228,7 @@ fn azure_blob_build_request_with_time_format() { ), ), compression, + tags: None, }; let mut byte_size = GroupedCountByteSize::new_untagged(); @@ -182,6 +247,7 @@ fn azure_blob_build_request_with_time_format() { ); assert_eq!(request.content_encoding, None); assert_eq!(request.content_type, "text/plain"); + assert_eq!(request.tags, None); } #[test] @@ -215,6 +281,7 @@ fn azure_blob_build_request_with_uuid() { ), ), compression, + tags: None, }; let mut byte_size = GroupedCountByteSize::new_untagged(); diff --git a/src/sinks/azure_common/config.rs b/src/sinks/azure_common/config.rs index 6a697bd5d..c3d534514 100644 --- a/src/sinks/azure_common/config.rs +++ b/src/sinks/azure_common/config.rs @@ -8,6 +8,7 @@ use bytes::Bytes; use futures::FutureExt; use http::StatusCode; use snafu::Snafu; +use std::collections::BTreeMap; use vector_lib::stream::DriverResponse; use vector_lib::{ json_size::JsonSize, @@ -30,6 +31,7 @@ pub struct AzureBlobRequest { pub content_type: &'static str, pub metadata: AzureBlobMetadata, pub request_metadata: RequestMetadata, + pub tags: Option>, } impl Finalizable for AzureBlobRequest { diff --git a/src/sinks/azure_common/service.rs b/src/sinks/azure_common/service.rs index 117090589..e872df74f 100644 --- a/src/sinks/azure_common/service.rs +++ b/src/sinks/azure_common/service.rs @@ -42,9 +42,18 @@ impl Service for AzureBlobService { Some(client) => { let client = client.blob_client(request.metadata.partition_key.as_str()); let byte_size = request.blob_data.len(); + + let mut tags: Tags = Tags::new(); + if request.tags.is_some() { + for (key, value) in request.tags.unwrap().iter() { + tags.insert(key, value); + } + } + let blob = client .put_block_blob(request.blob_data) - .content_type(request.content_type); + .content_type(request.content_type) + .tags(tags); let blob = match request.content_encoding { Some(encoding) => blob.content_encoding(encoding), None => blob, diff --git a/src/sinks/azure_common/sink.rs b/src/sinks/azure_common/sink.rs index 484447423..3fdf2414e 100644 --- a/src/sinks/azure_common/sink.rs +++ b/src/sinks/azure_common/sink.rs @@ -2,11 +2,16 @@ use std::fmt; use crate::sinks::{prelude::*, util::partitioner::KeyPartitioner}; +// MEZMO: added dependency for s3-sink file consolidation +use crate::sinks::azure_blob::file_consolidator_async::FileConsolidatorAsync; + pub struct AzureBlobSink { service: Svc, request_builder: RB, partitioner: KeyPartitioner, batcher_settings: BatcherSettings, + // MEZMO: added property for azure-blob-sink file consolidation + file_consolidator: Option, } impl AzureBlobSink { @@ -15,12 +20,14 @@ impl AzureBlobSink { request_builder: RB, partitioner: KeyPartitioner, batcher_settings: BatcherSettings, + file_consolidator: Option, ) -> Self { Self { service, request_builder, partitioner, batcher_settings, + file_consolidator, } } } @@ -41,7 +48,12 @@ where let request_builder = self.request_builder; - input + // MEZMO: added file consolidation processing + // initiate the file consolidation process if necessary + let mut file_consolidator = self.file_consolidator.unwrap_or_default(); + file_consolidator.start(); + + let result = input .batched_partitioned(partitioner, || settings.as_byte_size_config()) .filter_map(|(key, batch)| async move { // We don't need to emit an error here if the event is dropped since this will occur if the template @@ -62,7 +74,13 @@ where .into_driver(self.service) .protocol("https") .run() - .await + .await; + + // MEZMO: added file consolidation processing + //stop the file consolidation process if necessary + file_consolidator.stop(); + + result } } From 277d7fa6b6e14154defc1711357e881dbdd07f38 Mon Sep 17 00:00:00 2001 From: LogDNA Bot Date: Thu, 29 Feb 2024 17:06:45 +0000 Subject: [PATCH 11/19] chore(release): 3.10.0 [skip ci] # [3.10.0](https://github.com/answerbook/vector/compare/v3.9.0...v3.10.0) (2024-02-29) ### Features * **azure blob**: add tagging and file consolidation [d933d28](https://github.com/answerbook/vector/commit/d933d2800edfde28e2eb19d6c27a84b5431238e4) - dominic-mcallister-logdna [LOG-19336](https://logdna.atlassian.net/browse/LOG-19336) [LOG-19337](https://logdna.atlassian.net/browse/LOG-19337) ### Miscellaneous * Merge pull request #426 from answerbook/dominic/LOG-19232 [a5acdec](https://github.com/answerbook/vector/commit/a5acdec594f76de949900f6f31dcd1f300f6f2dd) - GitHub [LOG-19232](https://logdna.atlassian.net/browse/LOG-19232) --- MEZMO_CHANGELOG.md | 12 ++++++++++++ package.json | 2 +- 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/MEZMO_CHANGELOG.md b/MEZMO_CHANGELOG.md index 49af4d385..9e81e4baf 100644 --- a/MEZMO_CHANGELOG.md +++ b/MEZMO_CHANGELOG.md @@ -1,3 +1,15 @@ +# [3.10.0](https://github.com/answerbook/vector/compare/v3.9.0...v3.10.0) (2024-02-29) + + +### Features + +* **azure blob**: add tagging and file consolidation [d933d28](https://github.com/answerbook/vector/commit/d933d2800edfde28e2eb19d6c27a84b5431238e4) - dominic-mcallister-logdna [LOG-19336](https://logdna.atlassian.net/browse/LOG-19336) [LOG-19337](https://logdna.atlassian.net/browse/LOG-19337) + + +### Miscellaneous + +* Merge pull request #426 from answerbook/dominic/LOG-19232 [a5acdec](https://github.com/answerbook/vector/commit/a5acdec594f76de949900f6f31dcd1f300f6f2dd) - GitHub [LOG-19232](https://logdna.atlassian.net/browse/LOG-19232) + # [3.9.0](https://github.com/answerbook/vector/compare/v3.8.1...v3.9.0) (2024-02-26) diff --git a/package.json b/package.json index 74f87014c..aae0a580d 100644 --- a/package.json +++ b/package.json @@ -1,6 +1,6 @@ { "name": "vector", - "version": "3.9.0", + "version": "3.10.0", "description": "Vector is a high-performance, end-to-end (agent & aggregator) observability data pipeline", "repository": { "type": "git", From a837636de79d9339b03d16c4e95e906f4cdca587 Mon Sep 17 00:00:00 2001 From: Mike Del Tito Date: Thu, 29 Feb 2024 16:54:29 -0500 Subject: [PATCH 12/19] feat(classification): identify and annotate json string messages For profiling, it was determined that a major gap we have is the inability to pull annotations like `app` and `host` from the event itself. We currently rely on the request to carry that information in most cases. In other cases, the source (like splunk-hec) has a known/fixed format that includes this information, and so we annotate events during formatting off the queue. Rather than going down the path of parsing all events/fields and trying to map parsed data back into annotations, the decision was to handle this for JSON only to start. From internal testing this will support a large number of use-cases for profiling. With this change, when the `message` is either an object or a JSON string, we will look for app/host/level annotations in the object. The fields we look for are configurable with some sane defaults. Ref: LOG-19433 --- src/transforms/mezmo_log_classification.rs | 234 +++++++++++++++++---- 1 file changed, 193 insertions(+), 41 deletions(-) diff --git a/src/transforms/mezmo_log_classification.rs b/src/transforms/mezmo_log_classification.rs index 93a558f87..b4c1f3c43 100644 --- a/src/transforms/mezmo_log_classification.rs +++ b/src/transforms/mezmo_log_classification.rs @@ -3,7 +3,7 @@ use crate::{ schema::Definition, DataType, Input, LogNamespace, OutputId, TransformConfig, TransformContext, }, - event::Event, + event::{Event, LogEvent}, transforms::{TaskTransform, Transform}, }; use futures::StreamExt; @@ -21,6 +21,10 @@ use std::{ sync::OnceLock, }; +const DEFAULT_APP_FIELDS: [&str; 3] = ["app", "application", "container"]; +const DEFAULT_HOST_FIELDS: [&str; 2] = ["host", "hostname"]; +const DEFAULT_LEVEL_FIELDS: [&str; 2] = ["level", "log_level"]; + const DEFAULT_LOG_EVENT_TYPES: [&str; 67] = [ "HTTPD_COMBINEDLOG", "HTTPD_COMMONLOG", @@ -113,12 +117,25 @@ fn grok_patterns() -> &'static BTreeMap { #[derive(Clone, Debug, Default)] #[serde(deny_unknown_fields)] pub struct LogClassificationConfig { - /// When a [[LogEvent]] ".message" property is an object, look for matches in these fields. - /// Fields are evaluated in the order they are defined in the configuration, and the - /// first valid (string) field will be used to attempt to classify the event. - /// Note that these fields are relative to the message field rather than the root of the event. + /// When a [[LogEvent]] ".message" property is an object or is parsed as JSON, look for + /// matches in these fields. Fields are evaluated in the order they are defined in the + /// configuration, and the first valid (string) field will be used to attempt to classify + /// the event. Note that these fields are relative to the message field rather than the + /// root of the event. line_fields: Option>, + /// A list of object fields to consider for the "app" annotation + #[serde(default = "default_app_fields")] + app_fields: Vec, + + /// A list of object fields to consider for the "host" annotation + #[serde(default = "default_host_fields")] + host_fields: Vec, + + /// A list of object fields to consider for the "level" annotation + #[serde(default = "default_level_fields")] + level_fields: Vec, + /// List of Grok patterns to match on #[serde(default = "default_grok_patterns")] grok_patterns: Vec, @@ -131,6 +148,18 @@ fn default_grok_patterns() -> Vec { .collect() } +fn default_app_fields() -> Vec { + DEFAULT_APP_FIELDS.iter().map(|s| s.to_string()).collect() +} + +fn default_host_fields() -> Vec { + DEFAULT_HOST_FIELDS.iter().map(|s| s.to_string()).collect() +} + +fn default_level_fields() -> Vec { + DEFAULT_LEVEL_FIELDS.iter().map(|s| s.to_string()).collect() +} + impl_generate_config_from_default!(LogClassificationConfig); #[async_trait::async_trait] @@ -157,6 +186,9 @@ impl TransformConfig for LogClassificationConfig { pub struct LogClassification { patterns: Vec, line_fields: Vec, + app_fields: Vec, + host_fields: Vec, + level_fields: Vec, } impl LogClassification { @@ -164,6 +196,9 @@ impl LogClassification { LogClassification { patterns: config.grok_patterns.clone(), line_fields: config.line_fields.clone().unwrap_or_default(), + app_fields: config.app_fields.clone(), + host_fields: config.host_fields.clone(), + level_fields: config.level_fields.clone(), } } @@ -185,66 +220,98 @@ impl LogClassification { None } + fn match_from_line_fields( + &self, + value: &Value, + matches: &mut Vec, + message_key: &mut String, + ) { + for line_field in self.line_fields.iter() { + let value = value.get(line_field.as_str()); + if let Some(value) = value { + // Only consider fields containing string values. + // The first string field we encounter will be used, regardless + // of whether or not there are other string fields that may potentially + // match one of the patterns. + if !value.is_bytes() { + continue; + } + + // We identified a line field that is a string. + // We mark it as the message_key, regardless of whether there's a match in + // the classification + *message_key = format!("{message_key}{line_field}"); + + let line = value.to_string_lossy(); + if let Some(event_type) = self.match_event_type(&line) { + matches.push(event_type); + } + + break; + } + } + } + + fn annotate_from_fields(&self, value: &Value, log: &mut LogEvent) { + for field in self.app_fields.iter() { + if let Some(val) = value.get(field.as_str()) { + log.insert(annotation_path(vec!["app"]).as_str(), val.clone()); + } + } + for field in self.host_fields.iter() { + if let Some(val) = value.get(field.as_str()) { + log.insert(annotation_path(vec!["host"]).as_str(), val.clone()); + } + } + for field in self.level_fields.iter() { + if let Some(val) = value.get(field.as_str()) { + log.insert(annotation_path(vec!["level"]).as_str(), val.clone()); + } + } + } + fn transform_one(&mut self, mut event: Event) -> Option { let log = event.as_mut_log(); if let Some(message) = log.get(log_schema().message_key_target_path().unwrap()) { - let mut matches = vec![]; let mut message_key = log_schema().message_key().unwrap().to_string(); + let mut matches = Vec::new(); + let mut message_size = value_size(message) as i64; if message_size.is_negative() { warn!("total_bytes for message exceeded i64 limit, using i64::MAX instead"); message_size = i64::MAX; } - // For object messages, look for a valid field from `line_fields` in order. - // Otherwise just look for matches in the message (string). + // For object messages, look for a valid string field from `line_fields` in order. + // Otherwise just look for matches in the message string. If none are found, + // attempt to parse as JSON including annotations from the object. // NOTE: array values for `message` are not explicitly handled here, as it is // expected the events are already unrolled when hitting this transform. if message.is_object() { - for line_field in self.line_fields.iter() { - let value = message.get(line_field.as_str()); - if let Some(value) = value { - // Only consider fields containing string values. - // The first string field we encounter will be used, regardless - // of whether or not there are other string fields that may potentially - // match one of the patterns. - if !value.is_bytes() { - continue; - } - - // We identified a line field that is a string. - // We mark it as the message_key, regardless of whether there's a match in - // the classification - message_key = format!("{message_key}{line_field}"); - - let line = value.to_string_lossy(); - if let Some(event_type) = self.match_event_type(&line) { - matches.push(event_type); - } - - break; - } - } + self.match_from_line_fields(message, &mut matches, &mut message_key); + self.annotate_from_fields(&message.clone(), log); } else if message.is_bytes() { - if let Some(event_type) = self.match_event_type(&message.to_string_lossy()) { + let message_str = &message.to_string_lossy(); + if let Some(event_type) = self.match_event_type(message_str) { matches.push(event_type); + } else if let Some(json) = try_parse_json(message_str) { + let value = Value::from(json); + self.match_from_line_fields(&value, &mut matches, &mut message_key); + self.annotate_from_fields(&value, log); } }; - let classification_path = - log_schema().annotations_key().to_string() + ".classification"; - log.insert( - (classification_path.clone() + ".total_bytes").as_str(), + annotation_path(vec!["classification", "total_bytes"]).as_str(), Value::Integer(message_size), ); log.insert( - (classification_path.clone() + ".event_count").as_str(), + annotation_path(vec!["classification", "event_count"]).as_str(), Value::Integer(1), ); log.insert( - (classification_path + ".event_types").as_str(), + annotation_path(vec!["classification", "event_types"]).as_str(), Value::Object( matches .into_iter() @@ -253,7 +320,7 @@ impl LogClassification { ), ); log.insert( - (log_schema().annotations_key().to_string() + ".message_key").as_str(), + annotation_path(vec!["message_key"]).as_str(), Value::Bytes(message_key.into()), ); } @@ -262,6 +329,14 @@ impl LogClassification { } } +fn try_parse_json(s: &str) -> Option { + serde_json::from_str(s).ok() +} + +fn annotation_path(parts: Vec<&str>) -> String { + log_schema().annotations_key().to_string() + "." + parts.join(".").as_str() +} + impl TaskTransform for LogClassification { fn transform( self: Box, @@ -274,6 +349,7 @@ impl TaskTransform for LogClassification { #[cfg(test)] mod tests { + use serde_json::json; use std::collections::BTreeMap; use std::time::Duration; use tokio::sync::mpsc; @@ -346,6 +422,9 @@ mod tests { let config = LogClassificationConfig { line_fields: None, grok_patterns: default_grok_patterns(), + app_fields: default_app_fields(), + host_fields: default_host_fields(), + level_fields: default_level_fields(), }; let output = do_transform(config, event.clone().into()).await.unwrap(); @@ -364,6 +443,54 @@ mod tests { ); } + #[tokio::test] + async fn event_with_json_string_message() { + let line = r#"47.29.201.179 - - [28/Feb/2019:13:17:10 +0000] "GET /?p=1 HTTP/2.0" 200 5316 "https://domain1.com/?p=1" "Mozilla/5.0 (Windows NT 6.1) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/72.0.3626.119 Safari/537.36" "2.75"#; + let line_obj = json!({ + "foo": "bar", + "baz": 30, + "application": "test-app", + "host": "test-host", + "level": "test-level", + "line": line + }); + + let message_key = "message".to_string(); + let event = Event::Log(LogEvent::from(Value::Object( + btreemap!(message_key.clone() => Value::Bytes(line_obj.to_string().into())), + ))); + + let config = LogClassificationConfig { + line_fields: Some(vec![".line".to_string()]), + grok_patterns: default_grok_patterns(), + app_fields: default_app_fields(), + host_fields: default_host_fields(), + level_fields: default_level_fields(), + }; + let output = do_transform(config, event.clone().into()).await.unwrap(); + + let mut annotations = make_expected_annotations( + &event, + Some(".line".into()), + vec!["HTTPD_COMBINEDLOG".to_string()], + ); + + annotations.insert("app", Value::Bytes("test-app".into())); + annotations.insert("host", Value::Bytes("test-host".into())); + annotations.insert("level", Value::Bytes("test-level".into())); + + // line is retained + assert_eq!( + output.as_log().get(message_key.as_str()), + Some(Value::Bytes(line_obj.to_string().into())).as_ref() + ); + + assert_eq!( + output.as_log().get(log_schema().annotations_key()), + Some(&annotations) + ); + } + #[tokio::test] async fn event_with_array_message() { let event = Event::Log(LogEvent::from(btreemap! { @@ -376,6 +503,9 @@ mod tests { let config = LogClassificationConfig { line_fields: None, grok_patterns: default_grok_patterns(), + app_fields: default_app_fields(), + host_fields: default_host_fields(), + level_fields: default_level_fields(), }; let output = do_transform(config, event.clone().into()).await.unwrap(); @@ -400,6 +530,9 @@ mod tests { let config = LogClassificationConfig { line_fields: None, grok_patterns: default_grok_patterns(), + app_fields: default_app_fields(), + host_fields: default_host_fields(), + level_fields: default_level_fields(), }; let output = do_transform(config, event.clone().into()).await.unwrap(); @@ -428,6 +561,9 @@ mod tests { ".key3".to_string(), ]), grok_patterns: default_grok_patterns(), + app_fields: default_app_fields(), + host_fields: default_host_fields(), + level_fields: default_level_fields(), }; let output = do_transform(config, event.clone().into()).await.unwrap(); @@ -445,6 +581,9 @@ mod tests { let event = Event::Log(LogEvent::from(btreemap! { "message" => btreemap! { "foo" => "bar", + "app" => "test-app", + "hostname" => "test-host", + "log_level" => "test-level", "apache" => r#"47.29.201.179 - - [28/Feb/2019:13:17:10 +0000] "GET /?p=1 HTTP/2.0" 200 5316 "https://domain1.com/?p=1" "Mozilla/5.0 (Windows NT 6.1) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/72.0.3626.119 Safari/537.36" "2.75"#, "syslog" => r#"<161>2 2023-11-07T14:20:52.042-05:00 walker.net jeralddamore 948 ID430 - Authentication failed from 163.27.187.39 (163.27.187.39): Permission denied in replay cache code"# } @@ -454,15 +593,22 @@ mod tests { // First match wins, apache is not detected line_fields: Some(vec![".syslog".to_string(), ".apache".to_string()]), grok_patterns: default_grok_patterns(), + app_fields: default_app_fields(), + host_fields: default_host_fields(), + level_fields: default_level_fields(), }; let output = do_transform(config, event.clone().into()).await.unwrap(); - let annotations = make_expected_annotations( + let mut annotations = make_expected_annotations( &event, Some(".syslog".to_string()), vec!["SYSLOG5424LINE".to_string()], ); + annotations.insert("app", Value::Bytes("test-app".into())); + annotations.insert("host", Value::Bytes("test-host".into())); + annotations.insert("level", Value::Bytes("test-level".into())); + assert_eq!( output.as_log().get(log_schema().annotations_key()), Some(&annotations) @@ -482,6 +628,9 @@ mod tests { // The first valid field is the only field considered line_fields: Some(vec![".foo".to_string(), ".apache".to_string()]), grok_patterns: default_grok_patterns(), + app_fields: default_app_fields(), + host_fields: default_host_fields(), + level_fields: default_level_fields(), }; let output = do_transform(config, event.clone().into()).await.unwrap(); @@ -508,6 +657,9 @@ mod tests { let config = LogClassificationConfig { line_fields: None, grok_patterns: default_grok_patterns(), + app_fields: default_app_fields(), + host_fields: default_host_fields(), + level_fields: default_level_fields(), }; let output = do_transform(config, event.clone().into()).await.unwrap(); From 23251f290ac02e3ec8c34fe93082e6fe1a74041a Mon Sep 17 00:00:00 2001 From: Mike Del Tito Date: Fri, 1 Mar 2024 12:29:14 -0500 Subject: [PATCH 13/19] chore: disable integration tests on pull requests We haven't found a ton of value in these tests running on every PR, and they are somewhat expensive. Generally we should be running integration tests locally when changing a component that has them. This moves them to run on `master` only to provide at least some visibility into failing tests. Ref: LOG-19433 --- .github/workflows/integration-test-mezmo.yml | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/.github/workflows/integration-test-mezmo.yml b/.github/workflows/integration-test-mezmo.yml index 35ff0e447..94bc48be6 100644 --- a/.github/workflows/integration-test-mezmo.yml +++ b/.github/workflows/integration-test-mezmo.yml @@ -2,18 +2,14 @@ name: Integration Test Suite (Mezmo) on: workflow_dispatch: - pull_request: - branches-ignore: - - 'dependabot/**' - paths-ignore: - - 'deployment/**' - - 'test/data/**' - - 'docs/**' - push: branches: - master - ci-fix + paths-ignore: + - 'deployment/**' + - 'test/data/**' + - 'docs/**' concurrency: # For pull requests, cancel running workflows, for master, run all From 921c5be635fb6ca733a9fcd3bc29957f5a457156 Mon Sep 17 00:00:00 2001 From: LogDNA Bot Date: Fri, 1 Mar 2024 18:15:08 +0000 Subject: [PATCH 14/19] chore(release): 3.11.0 [skip ci] # [3.11.0](https://github.com/answerbook/vector/compare/v3.10.0...v3.11.0) (2024-03-01) ### Chores * disable integration tests on pull requests [23251f2](https://github.com/answerbook/vector/commit/23251f290ac02e3ec8c34fe93082e6fe1a74041a) - Mike Del Tito [LOG-19433](https://logdna.atlassian.net/browse/LOG-19433) ### Features * **classification**: identify and annotate json string messages [a837636](https://github.com/answerbook/vector/commit/a837636de79d9339b03d16c4e95e906f4cdca587) - Mike Del Tito [LOG-19433](https://logdna.atlassian.net/browse/LOG-19433) ### Miscellaneous * Merge pull request #427 from answerbook/mdeltito/LOG-19433 [6ef9311](https://github.com/answerbook/vector/commit/6ef931137b5ae2600240c39d0fa111664f047d4a) - GitHub [LOG-19433](https://logdna.atlassian.net/browse/LOG-19433) --- MEZMO_CHANGELOG.md | 17 +++++++++++++++++ package.json | 2 +- 2 files changed, 18 insertions(+), 1 deletion(-) diff --git a/MEZMO_CHANGELOG.md b/MEZMO_CHANGELOG.md index 9e81e4baf..bb49489fe 100644 --- a/MEZMO_CHANGELOG.md +++ b/MEZMO_CHANGELOG.md @@ -1,3 +1,20 @@ +# [3.11.0](https://github.com/answerbook/vector/compare/v3.10.0...v3.11.0) (2024-03-01) + + +### Chores + +* disable integration tests on pull requests [23251f2](https://github.com/answerbook/vector/commit/23251f290ac02e3ec8c34fe93082e6fe1a74041a) - Mike Del Tito [LOG-19433](https://logdna.atlassian.net/browse/LOG-19433) + + +### Features + +* **classification**: identify and annotate json string messages [a837636](https://github.com/answerbook/vector/commit/a837636de79d9339b03d16c4e95e906f4cdca587) - Mike Del Tito [LOG-19433](https://logdna.atlassian.net/browse/LOG-19433) + + +### Miscellaneous + +* Merge pull request #427 from answerbook/mdeltito/LOG-19433 [6ef9311](https://github.com/answerbook/vector/commit/6ef931137b5ae2600240c39d0fa111664f047d4a) - GitHub [LOG-19433](https://logdna.atlassian.net/browse/LOG-19433) + # [3.10.0](https://github.com/answerbook/vector/compare/v3.9.0...v3.10.0) (2024-02-29) diff --git a/package.json b/package.json index aae0a580d..742be17ce 100644 --- a/package.json +++ b/package.json @@ -1,6 +1,6 @@ { "name": "vector", - "version": "3.10.0", + "version": "3.11.0", "description": "Vector is a high-performance, end-to-end (agent & aggregator) observability data pipeline", "repository": { "type": "git", From 301ac96b00170af05aab9f3bfd4db1a0b4a896c8 Mon Sep 17 00:00:00 2001 From: dominic-mcallister-logdna <98114655+dominic-mcallister-logdna@users.noreply.github.com> Date: Mon, 11 Mar 2024 16:10:26 -0400 Subject: [PATCH 15/19] feat(s3 sink): add recursive directory consolidation summary: added the ability to recursively consolidate files on a directory by directory basis. Also added consolidation of json files which had previously been omitted from processing. ref: LOG-19448 --- src/sinks/aws_s3/config.rs | 3 +- .../aws_s3/file_consolidation_processor.rs | 900 +++++++++++------- src/sinks/aws_s3/file_consolidator_async.rs | 71 +- src/sinks/aws_s3/integration_tests_mezmo.rs | 466 +++++++-- 4 files changed, 966 insertions(+), 474 deletions(-) diff --git a/src/sinks/aws_s3/config.rs b/src/sinks/aws_s3/config.rs index 0487e7907..8a1d64f6a 100644 --- a/src/sinks/aws_s3/config.rs +++ b/src/sinks/aws_s3/config.rs @@ -302,9 +302,8 @@ impl S3SinkConfig { endpoint, proxy, self.tls.clone(), - self.file_consolidation_config, + self.file_consolidation_config.clone(), self.bucket.clone(), - self.key_prefix.clone(), ); Some(consolidator) } diff --git a/src/sinks/aws_s3/file_consolidation_processor.rs b/src/sinks/aws_s3/file_consolidation_processor.rs index 61002bd4c..3b2e73c33 100644 --- a/src/sinks/aws_s3/file_consolidation_processor.rs +++ b/src/sinks/aws_s3/file_consolidation_processor.rs @@ -8,6 +8,9 @@ use flate2::read::GzDecoder; use std::io; use std::io::Cursor; +use std::collections::HashMap; +use std::path::Path; + use aws_sdk_s3::{ model::{CompletedMultipartUpload, CompletedPart, RequestPayer}, types::ByteStream, @@ -28,8 +31,9 @@ const MAX_PARTS_MULTIPART_FILE: i32 = 10_000; pub struct FileConsolidationProcessor<'a> { s3_client: &'a S3Client, bucket: String, - key_prefix: String, requested_size_bytes: i64, + base_path: String, + output_format: String, } // handles consolidating all the smaller files generated into larger files of the bucket @@ -42,23 +46,26 @@ impl<'a> FileConsolidationProcessor<'a> { pub const fn new( s3_client: &'a S3Client, bucket: String, - key_prefix: String, requested_size_bytes: i64, + base_path: String, + output_format: String, ) -> Self { FileConsolidationProcessor { s3_client, bucket, - key_prefix, requested_size_bytes, + output_format, + base_path, } } pub async fn run(self) { // retrieve the files list from s3 that we can process - let mut files_to_consolidate: Vec = match get_files_to_consolidate( + let files_to_consolidate: Vec = match get_files_to_consolidate( self.s3_client, self.bucket.clone(), - self.key_prefix.clone(), + self.base_path.clone(), + self.output_format.clone(), ) .await { @@ -66,207 +73,224 @@ impl<'a> FileConsolidationProcessor<'a> { Err(e) => { error!( ?e, - "bucket={}, prefix={}, Failed to retrieve files to consolidate", + "bucket={}, base_path={}, Failed to retrieve files to consolidate", self.bucket.clone(), - self.key_prefix.clone(), + self.base_path.clone(), ); let empty_files: Vec = Vec::new(); empty_files } }; - // if we have no files to combine - if files_to_consolidate.len() <= 1 { - return; - } + // customers are specifying a base path to consolidate + let files_by_dir: HashMap> = + group_files_by_directory(files_to_consolidate); - // break the files into groups so we can generate a file of the - // size requested by the customer - while !files_to_consolidate.is_empty() { - // keep track of the processed files to delete - let mut files_to_delete: Vec = Vec::new(); - let mut completed_parts: Vec = Vec::new(); - - let upload_file_parts: Vec = - splice_files_list(self.requested_size_bytes, &mut files_to_consolidate); - - // build the new file properties and expiration time - // make sure the process hasn't ran so fast (really should just unit tests) - // that we accidentally overwrite a merge file. - let mut time_since_epoch: u64; - let mut new_file_key: String; - - loop { - time_since_epoch = SystemTime::now() - .duration_since(UNIX_EPOCH) - .unwrap() - .as_secs(); - new_file_key = - format!("{}merged_{}.log", self.key_prefix.clone(), time_since_epoch); - - match self - .s3_client - .get_object() - .bucket(self.bucket.clone()) - .key(new_file_key.clone()) - .send() - .await - { - Ok(_data) => { - info!( - "bucket={}, Merged file already exists, file={}", - self.bucket.clone(), - new_file_key.clone(), - ); - tokio::time::sleep(tokio::time::Duration::from_secs(1)).await; - } - Err(_e) => { - // the file doesn't exist, break the loop and move on. - break; - } - }; - } - info!( - "bucket={}, Starting consolidated file={}", - self.bucket.clone(), - new_file_key.clone(), - ); + for dir in files_by_dir.keys() { + let mut files = files_by_dir.get(dir).unwrap().clone(); - let tags = { - let mut tagging = url::form_urlencoded::Serializer::new(String::new()); - tagging.append_pair("mezmo_pipeline_merged", "true"); - tagging.finish() - }; + while files.len() > 1 { + // break the files into groups so we can generate a file of the size requested by the customer + let mut upload_file_parts: Vec = + splice_files_list(self.requested_size_bytes, &mut files); - let content_type = "text/x-log".to_owned(); + // keep track of the processed files to delete + let mut files_to_delete: Vec = Vec::new(); + let mut completed_parts: Vec = Vec::new(); - //calculate the size of all the files - //we make no assumptions about compression here if the file is gzip'd - let mut total_bytes_of_all_files: i64 = 0; - for record in upload_file_parts.iter() { - total_bytes_of_all_files += record.size; - } + // if there's a directory, the trailing slash is not present so add it + let newfile_dir = if (dir.is_empty()) || (dir.ends_with('/')) { + dir.clone() + } else { + format!("{}/", dir.clone()) + }; - // there is a mimimum size of a multipart upload so we'll just upload a single file - // if the directory has less than that amount. - if total_bytes_of_all_files <= MULTIPART_FILE_MIN_MB_I64 { - let bytes = match download_all_files_as_bytes( - self.s3_client, + // build the new file properties and expiration time + // make sure the process hasn't ran so fast (really should just unit tests) + // that we accidentally overwrite a merge file. + let mut time_since_epoch: u64; + let mut new_file_key: String; + + loop { + time_since_epoch = SystemTime::now() + .duration_since(UNIX_EPOCH) + .unwrap() + .as_secs(); + new_file_key = format!("{}merged_{}.log", newfile_dir, time_since_epoch); + + match self + .s3_client + .get_object() + .bucket(self.bucket.clone()) + .key(new_file_key.clone()) + .send() + .await + { + Ok(_data) => { + info!( + "bucket={}, Merged file already exists, file={}", + self.bucket.clone(), + new_file_key.clone(), + ); + tokio::time::sleep(tokio::time::Duration::from_secs(1)).await; + } + Err(_e) => { + // the file doesn't exist, break the loop and move on. + break; + } + }; + } + info!( + "bucket={}, Starting consolidated file={}", self.bucket.clone(), - &upload_file_parts, - &mut files_to_delete, - ) - .await - { - Ok(data) => data, - Err(e) => { - error!( - ?e, - "bucket={}, Failed to download files", - self.bucket.clone(), - ); - continue; - } + new_file_key.clone(), + ); + + let tags = { + let mut tagging = url::form_urlencoded::Serializer::new(String::new()); + tagging.append_pair("mezmo_pipeline_merged", "true"); + tagging.finish() }; - if bytes.is_empty() { - info!( - "bucket={}, Failed to download files={:?}", - self.bucket.clone(), - upload_file_parts, - ); - continue; + let content_type = "text/x-log".to_owned(); + + //calculate the size of all the files + //we make no assumptions about compression here if the file is gzip'd + let mut total_bytes_of_all_files: i64 = 0; + for record in upload_file_parts.iter() { + total_bytes_of_all_files += record.size; } - let content_md5 = BASE64_STANDARD.encode(md5::Md5::digest(bytes.clone())); - match self - .s3_client - .put_object() - .body(bytes_to_bytestream(bytes)) - .bucket(self.bucket.clone()) - .key(new_file_key.clone()) - .set_content_type(Some(content_type)) - .set_tagging(Some(tags)) - .content_md5(content_md5) - .send() - .await - { - Ok(f) => { - info!( - "bucket={}, Successfully put single consolidated file={} for files={:?}", - self.bucket.clone(), - new_file_key.clone(), - upload_file_parts - ); - f - } - Err(e) => { - error!( - ?e, - "bucket={}, Failed to put single consolidated file={}", - self.bucket.clone(), - new_file_key.clone(), - ); - continue; - } - }; - } else { - // use a multi-part upload for all the files that we need to process - // set an expiration time for the file in case things go awry (vector dies, sink reloads, etc) - // with the expiration time, the file will be auto-deleted if necessary. - // we'll build the files up to the maximum size requested - let expires = time_since_epoch + (TWO_HOURS_IN_SECONDS); - let aws_time = AwsDateTime::from_secs(expires as i64); - let multi_part_upload = match self - .s3_client - .create_multipart_upload() - .bucket(self.bucket.clone()) - .key(new_file_key.clone()) - .set_content_type(Some(content_type)) - .set_tagging(Some(tags)) - .set_expires(Some(aws_time)) - .send() + // there is a mimimum size of a multipart upload so we'll just upload a single file + // if the directory has less than that amount. + if total_bytes_of_all_files <= MULTIPART_FILE_MIN_MB_I64 { + let bytes = match download_all_files_as_bytes( + self.s3_client, + self.bucket.clone(), + self.output_format.clone(), + &mut upload_file_parts, + &mut files_to_delete, + ) .await - { - Ok(m) => { + { + Ok(data) => data, + Err(e) => { + error!( + ?e, + "bucket={}, Failed to download files", + self.bucket.clone(), + ); + continue; + } + }; + + if bytes.is_empty() { info!( - "bucket={}, Successfully created multipart doc for file={}", - self.bucket.clone(), - new_file_key.clone(), - ); - m - } - Err(e) => { - error!( - ?e, - "bucket={}, Failed to invoke multipart upload file={}", + "bucket={}, Failed to download files={:?}", self.bucket.clone(), - new_file_key.clone(), + upload_file_parts, ); continue; } - }; - - let upload_id: String = multi_part_upload.upload_id().unwrap().to_owned(); - // The minimum file size for upload parts and copy parts 5 MB, so we'll manually consolidate - // small files into one larger to fill the void. Using 5 MB files will allow us to achieve - // 50 GB total over 10_000 parts - let mut part_num: i32 = 0; - let mut buf = BytesMut::with_capacity(0); - let mut buf_files: Vec = Vec::new(); + let content_md5 = BASE64_STANDARD.encode(md5::Md5::digest(bytes.clone())); + match self + .s3_client + .put_object() + .body(bytes_to_bytestream(bytes)) + .bucket(self.bucket.clone()) + .key(new_file_key.clone()) + .set_content_type(Some(content_type)) + .set_tagging(Some(tags)) + .content_md5(content_md5) + .send() + .await + { + Ok(f) => { + info!( + "bucket={}, Successfully put single consolidated file={} for files={:?}", + self.bucket.clone(), + new_file_key.clone(), + upload_file_parts + ); + f + } + Err(e) => { + error!( + ?e, + "bucket={}, Failed to put single consolidated file={}", + self.bucket.clone(), + new_file_key.clone(), + ); + continue; + } + }; + } else { + // use a multi-part upload for all the files that we need to process + // set an expiration time for the file in case things go awry (vector dies, sink reloads, etc) + // with the expiration time, the file will be auto-deleted if necessary. + // we'll build the files up to the maximum size requested + let expires = time_since_epoch + (TWO_HOURS_IN_SECONDS); + let aws_time = AwsDateTime::from_secs(expires as i64); + let multi_part_upload = match self + .s3_client + .create_multipart_upload() + .bucket(self.bucket.clone()) + .key(new_file_key.clone()) + .set_content_type(Some(content_type)) + .set_tagging(Some(tags)) + .set_expires(Some(aws_time)) + .send() + .await + { + Ok(m) => { + info!( + "bucket={}, Successfully created multipart doc for file={}", + self.bucket.clone(), + new_file_key.clone(), + ); + m + } + Err(e) => { + error!( + ?e, + "bucket={}, Failed to invoke multipart upload file={}", + self.bucket.clone(), + new_file_key.clone(), + ); + continue; + } + }; - for file in &upload_file_parts { - // make sure to not go over the max parts and leave - // one slot for any buffer that hasn't been pushed - if (part_num + 1) >= MAX_PARTS_MULTIPART_FILE { - break; - } + let upload_id: String = multi_part_upload.upload_id().unwrap().to_owned(); + + // The minimum file size for upload parts and copy parts 5 MB, so we'll manually consolidate + // small files into one larger to fill the void. Using 5 MB files will allow us to achieve + // 50 GB total over 10_000 parts + let mut part_num: i32 = 0; + let mut buf = BytesMut::with_capacity(0); + let mut buf_files: Vec = Vec::new(); + + upload_file_parts.reverse(); //reverse the list so we can pop off in the correct order + while let Some(file) = upload_file_parts.pop() { + let (trim_open_bracket, trim_close_bracket, prepend_char) = + determine_download_properties( + self.output_format.clone(), + &buf_files, + &upload_file_parts, + ); - // if the file is compressed, we need to pull and decompress it - // so we can join it with other files - let vector = - match download_file_as_vec(self.s3_client, self.bucket.clone(), file).await + // if the file is compressed, we need to pull and decompress it + // so we can join it with other files + let file_bytes = match download_file_as_bytes( + self.s3_client, + self.bucket.clone(), + &file, + trim_open_bracket, + trim_close_bracket, + prepend_char, + ) + .await { Ok(v) => { info!("bucket={}, Downloaded file={:?}", self.bucket.clone(), file); @@ -283,16 +307,79 @@ impl<'a> FileConsolidationProcessor<'a> { } }; - buf.extend_from_slice(&vector); - buf.extend_from_slice(b"\n"); //newline between file - buf_files.push(file.key.clone()); + buf.extend_from_slice(&file_bytes); + buf_files.push(file.key.clone()); + + // if we've got the minimum for a multipart chunk, send it on to the server + if buf.len() as i64 >= MULTIPART_FILE_MIN_MB_I64 { + part_num += 1; + + // cloning the buffer so its not moved + let body = bytes_to_bytestream(Bytes::from(buf.clone())); + let upload = match self + .s3_client + .upload_part() + .bucket(self.bucket.clone()) + .key(new_file_key.clone()) + .upload_id(upload_id.clone()) + .part_number(part_num) + .body(body) + .send() + .await + { + Ok(u) => { + info!( + "bucket={}, upload_id={}, Uploaded part={} for file={}", + self.bucket.clone(), + upload_id.clone(), + part_num, + new_file_key.clone(), + ); + u + } + Err(e) => { + error!( + ?e, + "bucket={}, upload_id={}, Failed to upload new part={} for file={}", + self.bucket.clone(), + upload_id.clone(), + part_num, + new_file_key.clone(), + ); + part_num -= 1; + continue; + } + }; + + // keep track of the part for completion + completed_parts.push( + CompletedPart::builder() + .e_tag(upload.e_tag().unwrap()) + .part_number(part_num) + .build(), + ); + + for file in &buf_files { + files_to_delete.push(file.clone()) + } + + // reset the buffer by clearing the memory + buf = BytesMut::with_capacity(0); + buf_files.clear(); + } + + // make sure to not go over the max parts and leave + // one slot for any buffer that hasn't been pushed + if (part_num + 1) >= MAX_PARTS_MULTIPART_FILE { + break; + } + } - // if we've got the minimum for a multipart chunk, send it on to the server - if buf.len() as i64 >= MULTIPART_FILE_MIN_MB_I64 { + // there's still data in the buffer, so make that the final part. + // the final upload part doesn't have to be the 5 MB min + if !buf.is_empty() { part_num += 1; - // cloning the buffer so its not moved - let body = bytes_to_bytestream(Bytes::from(buf.clone())); let upload = match self .s3_client .upload_part() @@ -300,7 +387,7 @@ impl<'a> FileConsolidationProcessor<'a> { .key(new_file_key.clone()) .upload_id(upload_id.clone()) .part_number(part_num) - .body(body) + .body(bytes_to_bytestream(Bytes::from(buf))) .send() .await { @@ -321,9 +408,8 @@ impl<'a> FileConsolidationProcessor<'a> { self.bucket.clone(), upload_id.clone(), part_num, - new_file_key.clone(), + new_file_key.clone() ); - part_num -= 1; continue; } }; @@ -339,158 +425,109 @@ impl<'a> FileConsolidationProcessor<'a> { for file in &buf_files { files_to_delete.push(file.clone()) } - - // reset the buffer by clearing the memory - buf = BytesMut::with_capacity(0); - buf_files.clear(); } - } - // there's still data in the buffer, so make that the final part. - // the final upload part doesn't have to be the 5 MB min - if !buf.is_empty() { - part_num += 1; - - let upload = match self + // time to mark the entire file as complete + match self .s3_client - .upload_part() + .complete_multipart_upload() .bucket(self.bucket.clone()) .key(new_file_key.clone()) .upload_id(upload_id.clone()) - .part_number(part_num) - .body(bytes_to_bytestream(Bytes::from(buf))) + .request_payer(RequestPayer::Requester) + .multipart_upload( + CompletedMultipartUpload::builder() + .set_parts(Some(completed_parts)) + .build(), + ) .send() .await { Ok(u) => { info!( - "bucket={}, upload_id={}, Uploaded part={} for file={}", + "bucket={}, upload_id={}, Completed multipart upload for file={}", self.bucket.clone(), upload_id.clone(), - part_num, - new_file_key.clone(), + new_file_key.clone() ); u } Err(e) => { - error!( - ?e, - "bucket={}, upload_id={}, Failed to upload new part={} for file={}", - self.bucket.clone(), - upload_id.clone(), - part_num, - new_file_key.clone() - ); + error!(?e, "bucket={}, upload_id={}, Failed to complete multipart upload for file={}", self.bucket.clone(), upload_id.clone(), new_file_key.clone()); + + // completing the file didn't work out, so abort it completely. + match self + .s3_client + .abort_multipart_upload() + .bucket(self.bucket.clone()) + .key(new_file_key.clone()) + .upload_id(upload_id.clone()) + .request_payer(RequestPayer::Requester) + .send() + .await + { + Ok(_v) => info!("bucket={}, upload_id={}, Aborted multipart upload for file={}", self.bucket.clone(), upload_id.clone(), new_file_key.clone()), + Err(e) => error!(?e, "bucket={}, upload_id={}, Failed to abort multipart upload for file={}", self.bucket.clone(), upload_id.clone(), new_file_key.clone()), + }; + continue; } }; - - // keep track of the part for completion - completed_parts.push( - CompletedPart::builder() - .e_tag(upload.e_tag().unwrap()) - .part_number(part_num) - .build(), - ); - - for file in &buf_files { - files_to_delete.push(file.clone()) - } } - // time to mark the entire file as complete - match self - .s3_client - .complete_multipart_upload() - .bucket(self.bucket.clone()) - .key(new_file_key.clone()) - .upload_id(upload_id.clone()) - .request_payer(RequestPayer::Requester) - .multipart_upload( - CompletedMultipartUpload::builder() - .set_parts(Some(completed_parts)) - .build(), - ) - .send() - .await - { - Ok(u) => { - info!( - "bucket={}, upload_id={}, Completed multipart upload for file={}", - self.bucket.clone(), - upload_id.clone(), - new_file_key.clone() - ); - u - } - Err(e) => { - error!(?e, "bucket={}, upload_id={}, Failed to complete multipart upload for file={}", self.bucket.clone(), upload_id.clone(), new_file_key.clone()); - - // completing the file didn't work out, so abort it completely. - match self - .s3_client - .abort_multipart_upload() - .bucket(self.bucket.clone()) - .key(new_file_key.clone()) - .upload_id(upload_id.clone()) - .request_payer(RequestPayer::Requester) - .send() - .await - { - Ok(_v) => info!("bucket={}, upload_id={}, Aborted multipart upload for file={}", self.bucket.clone(), upload_id.clone(), new_file_key.clone()), - Err(e) => error!(?e, "bucket={}, upload_id={}, Failed to abort multipart upload for file={}", self.bucket.clone(), upload_id.clone(), new_file_key.clone()), - }; - - continue; - } - }; - } - - // remove all the files from S3 that have been merged into the larger file - for file in files_to_delete { - match self - .s3_client - .delete_object() - .bucket(self.bucket.clone()) - .key(file.clone()) - .send() - .await - { - Ok(_result) => { - info!( - message = format!( - "File={} removed from bucket={} after merge successful file consolidation", - file.clone(), - self.bucket.clone() + // remove all the files from S3 that have been merged into the larger file + for file in files_to_delete { + match self + .s3_client + .delete_object() + .bucket(self.bucket.clone()) + .key(file.clone()) + .send() + .await + { + Ok(_result) => { + info!( + message = format!( + "File={} removed from bucket={} after merge successful file consolidation", + file.clone(), + self.bucket.clone() + ) ) - ) - } - Err(e) => error!( - ?e, - "bucket={}, Failed to delete merged file={}", - self.bucket.clone(), - file.clone() - ), - }; - } // end else multipart logic - } // end files to consolidate loop + } + Err(e) => error!( + ?e, + "bucket={}, Failed to delete merged file={}", + self.bucket.clone(), + file.clone() + ), + }; + } // end else multipart logic + } // end files to consolidate loop + } // end files by directory loop } } // helper class for the files that we're consolidating into a single file -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct ConsolidationFile { pub compressed: bool, pub size: i64, pub key: String, + pub last_modified: i64, } impl ConsolidationFile { - pub const fn new(compressed: bool, size: i64, key: String) -> ConsolidationFile { + pub const fn new( + compressed: bool, + size: i64, + key: String, + last_modified: i64, + ) -> ConsolidationFile { ConsolidationFile { compressed, size, key, + last_modified, } } } @@ -527,13 +564,15 @@ fn splice_files_list( which can be merged into larger files @client: the s3 client @bucket: the s3 bucket - @key_prefix: the prefix path for the files + @base_path: the base path for the files + @requested_file_type: the type of file to find @@returns: Vector, the files which can be merged. */ pub async fn get_files_to_consolidate( client: &S3Client, bucket: String, - key_prefix: String, + base_path: String, + requested_file_type: String, ) -> Result, Error> { let mut files_to_consolidate: Vec = Vec::new(); let mut continuation_token: Option = None; @@ -542,16 +581,16 @@ pub async fn get_files_to_consolidate( let list_result = client .list_objects_v2() .bucket(bucket.clone()) - .prefix(key_prefix.clone()) + .prefix(base_path.clone()) .set_continuation_token(continuation_token) .send() .await?; if list_result.contents.is_none() { info!( - "bucket={}, prefix={}, No files found", + "bucket={}, base_path={}, No files found", bucket.clone(), - key_prefix.clone(), + base_path.clone(), ); break; } @@ -565,10 +604,7 @@ pub async fn get_files_to_consolidate( continuation_token = None; } - let mut sorted_objects = list_result.contents().unwrap().to_vec(); - sorted_objects.sort_by_key(|x| x.last_modified().unwrap().secs()); - - for key_object in sorted_objects { + for key_object in list_result.contents().unwrap() { let key = key_object.key().unwrap(); let tag_result = client @@ -578,12 +614,8 @@ pub async fn get_files_to_consolidate( .send() .await?; - // this file is the result of a previous merge let mut mezmo_merged_file = false; - // this file wasn't produced by the mezmo s3 process let mut mezmo_produced_file = false; - // not breaking down standard json files as we don't want to load download - // the whole file into memory. We're trying to straight memory copy here. let mut can_combine = false; let tags = tag_result.tag_set().unwrap_or_default(); @@ -591,12 +623,9 @@ pub async fn get_files_to_consolidate( match tag.key().unwrap_or_default() { "mezmo_pipeline_merged" => mezmo_merged_file = true, "mezmo_pipeline_s3_sink" => mezmo_produced_file = true, - "mezmo_pipeline_s3_type" => match tag.value().unwrap() { - "ndjson" => can_combine = true, - "text" => can_combine = true, - "json" => can_combine = false, - _ => can_combine = false, - }, + "mezmo_pipeline_s3_type" => { + can_combine = requested_file_type == tag.value().unwrap() + } _ => (), } } @@ -618,8 +647,14 @@ pub async fn get_files_to_consolidate( let compressed = head.content_encoding().unwrap_or_default() == "gzip"; let size = head.content_length(); let key = key.to_string(); - - files_to_consolidate.push(ConsolidationFile::new(compressed, size, key)); + let last_modified = key_object.last_modified().unwrap().secs(); + + files_to_consolidate.push(ConsolidationFile::new( + compressed, + size, + key, + last_modified, + )); } Err(e) => error!(?e, "bucket={}, Failed to head file={}", bucket.clone(), key), }; @@ -631,6 +666,7 @@ pub async fn get_files_to_consolidate( } } + files_to_consolidate.sort_by_key(|x| x.last_modified); Ok(files_to_consolidate) } @@ -642,18 +678,34 @@ pub async fn get_files_to_consolidate( via gzip compression. @client: the s3 client @bucket: the s3 bucket + @output_format: the requested format to download @files_to_delete: populated with the files which were successfully downloaded @@returns: Bytes, the byte data representing all the downloaded files */ async fn download_all_files_as_bytes( client: &S3Client, bucket: String, - files: &[ConsolidationFile], + output_format: String, + files: &mut Vec, files_to_delete: &mut Vec, ) -> Result { let mut buf = BytesMut::with_capacity(0); - for file in files.iter() { - let b: Bytes = match download_bytes(client, bucket.clone(), file.key.clone()).await { + + (*files).reverse(); // reverse the list so we can pop off it and maintain order + while let Some(file) = (*files).pop() { + let (trim_open_bracket, trim_close_bracket, prepend_char) = + determine_download_properties(output_format.clone(), files_to_delete, files); + + let b: Bytes = match download_file_as_bytes( + client, + bucket.clone(), + &file, + trim_open_bracket, + trim_close_bracket, + prepend_char, + ) + .await + { Ok(b) => b, Err(e) => { error!( @@ -666,21 +718,11 @@ async fn download_all_files_as_bytes( } }; - if file.compressed { - let decompressed = decompress_gzip(&b); - buf.extend_from_slice(&decompressed); - } else { - buf.extend_from_slice(&b); - } - - //add a newline as a separator - buf.extend_from_slice(b"\n"); - - // file downloaded successfully so mark it for potential deletion + buf.extend_from_slice(&b); files_to_delete.push(file.key.clone()); } - Ok(Bytes::from(buf)) + Ok(buf.freeze()) } /* @@ -693,22 +735,83 @@ async fn download_all_files_as_bytes( @files_to_delete: populated with the files which were successfully downloaded @@returns: Bytes, the byte data representing all the downloaded files */ -async fn download_file_as_vec( +async fn download_file_as_bytes( client: &S3Client, bucket: String, file: &ConsolidationFile, -) -> Result, Error> { + trim_open_bracket: bool, + trim_close_bracket: bool, + prepend_char: Option, +) -> Result { let b: Bytes = download_bytes(client, bucket.clone(), file.key.clone()).await?; - let mut buf = BytesMut::with_capacity(0); + let mut vec: Vec; if file.compressed { - let decompressed = decompress_gzip(&b); - buf.extend_from_slice(&decompressed); + vec = decompress_gzip(&b); } else { - buf.extend_from_slice(&b); + vec = b.to_vec(); } - Ok(buf.to_vec()) + if trim_open_bracket && !vec.is_empty() { + let i: usize = 0; + while i < vec.len() { + let c = char::from(vec[i]); + if c.is_whitespace() { + vec.remove(0); + continue; + } + if c == '[' { + vec.remove(0); + break; + } + } + } + + if trim_close_bracket && !vec.is_empty() { + loop { + let i = vec.len() - 1; + let c = char::from(vec[i]); + if c.is_whitespace() { + vec.remove(i); + continue; + } + if c == ']' { + vec.remove(i); + break; + } + + break; + } + } + + if prepend_char.is_some() { + vec.insert(0, prepend_char.unwrap() as u8); + } + + let mut buf = BytesMut::with_capacity(0); + buf.extend_from_slice(&vec); + Ok(buf.freeze()) +} + +fn determine_download_properties( + output_format: String, + files_to_delete: &Vec, + upload_file_parts: &Vec, +) -> (bool, bool, Option) { + let is_standard_json_file: bool = output_format == "json"; + + let trim_open_bracket = is_standard_json_file && !files_to_delete.is_empty(); + let trim_close_bracket = is_standard_json_file && !upload_file_parts.is_empty(); + + let prepend_char: Option = if is_standard_json_file && !files_to_delete.is_empty() { + Some(',') + } else if !files_to_delete.is_empty() { + Some('\n') + } else { + None + }; + + (trim_open_bracket, trim_close_bracket, prepend_char) } /* @@ -752,6 +855,30 @@ async fn download_bytes(client: &S3Client, bucket: String, key: String) -> Resul Ok(body) } +fn group_files_by_directory( + list: Vec, +) -> HashMap> { + let mut hm: HashMap> = HashMap::new(); + for f in list { + let p = Path::new(&f.key); + + let base_path = match p.parent() { + Some(p_base) => p_base.to_string_lossy().to_string(), + None => "".to_string(), + }; + + if !hm.contains_key(&base_path) { + let vec: Vec = Vec::new(); + hm.insert(base_path.clone(), vec); + } + + let m = hm.get_mut(&base_path).unwrap(); + m.push(f.clone()); + } + + hm +} + #[cfg(test)] mod tests { use bytes::{Bytes, BytesMut}; @@ -760,6 +887,7 @@ mod tests { use std::io::Read; use crate::sinks::aws_s3::file_consolidation_processor::decompress_gzip; + use crate::sinks::aws_s3::file_consolidation_processor::group_files_by_directory; use crate::sinks::aws_s3::file_consolidation_processor::splice_files_list; use crate::sinks::aws_s3::file_consolidation_processor::ConsolidationFile; @@ -780,6 +908,7 @@ mod tests { compressed: false, size: 10, key: i.to_string().to_owned(), + last_modified: 1, }); } @@ -796,6 +925,7 @@ mod tests { compressed: false, size: 10, key: i.to_string().to_owned(), + last_modified: 1, }); } @@ -812,6 +942,7 @@ mod tests { compressed: false, size: 10, key: i.to_string().to_owned(), + last_modified: 1, }); } @@ -840,4 +971,79 @@ mod tests { let s = std::str::from_utf8(&decompressed).unwrap(); assert_eq!(hello_world, s); } + + #[test] + fn test_group_files_by_directory() { + let vec: Vec = vec![ + create_consolidation_file("base_file1.log"), + create_consolidation_file("base_file2.log"), + create_consolidation_file("/file1.log"), + create_consolidation_file("/sub/file2.log"), + create_consolidation_file("/sub/file3.log"), + create_consolidation_file("/sub/file4.log"), + create_consolidation_file("/sub/sub/file5.log"), + create_consolidation_file("/sub/sub/file6.log"), + create_consolidation_file("/sub/sub/file7.log"), + create_consolidation_file("/sub/sub/file8.log"), + ]; + + let group = group_files_by_directory(vec); + let keys = group.keys(); + + assert_eq!(4, keys.len()); + + assert_eq!( + vec!["base_file1.log".to_string(), "base_file2.log".to_string()], + group + .get("") + .unwrap() + .iter() + .map(|cf| cf.key.clone()) + .collect::>() + ); + + assert_eq!( + vec!["/file1.log"], + group + .get("/") + .unwrap() + .iter() + .map(|cf| cf.key.clone()) + .collect::>() + ); + + assert_eq!( + vec!["/sub/file2.log", "/sub/file3.log", "/sub/file4.log"], + group + .get("/sub") + .unwrap() + .iter() + .map(|cf| cf.key.clone()) + .collect::>() + ); + + assert_eq!( + vec![ + "/sub/sub/file5.log", + "/sub/sub/file6.log", + "/sub/sub/file7.log", + "/sub/sub/file8.log", + ], + group + .get("/sub/sub") + .unwrap() + .iter() + .map(|cf| cf.key.clone()) + .collect::>() + ); + } + + fn create_consolidation_file(p: &str) -> ConsolidationFile { + ConsolidationFile { + compressed: false, + size: 10, + key: p.to_string().to_owned(), + last_modified: 1, + } + } } diff --git a/src/sinks/aws_s3/file_consolidator_async.rs b/src/sinks/aws_s3/file_consolidator_async.rs index 9387b2d19..b94e4a71a 100644 --- a/src/sinks/aws_s3/file_consolidator_async.rs +++ b/src/sinks/aws_s3/file_consolidator_async.rs @@ -8,6 +8,9 @@ use crate::sinks::aws_s3::file_consolidation_processor::FileConsolidationProcess use crate::{aws::create_client, common::s3::S3ClientBuilder}; use tokio::task::JoinHandle; +const DEFAULT_BASE_PATH: &str = ""; +const DEFAULT_OUTPUT_FORMAT: &str = "ndjson"; + /// File Consolidation /// Depending on the configuration of the sink and the throughput of data, /// S3 may receive hundreds and thousands of files. This is unmanageable from @@ -18,7 +21,7 @@ use tokio::task::JoinHandle; /// 1. All files within the bucket directory are of the same format configured /// to the sink #[configurable_component] -#[derive(Clone, Debug, Copy)] +#[derive(Clone, Debug)] #[serde(deny_unknown_fields)] pub struct FileConsolidationConfig { /// boolean indicating if the consolidation process is enabled @@ -29,6 +32,13 @@ pub struct FileConsolidationConfig { /// Indicates the size of the consolidation file that is produced pub requested_size_bytes: i64, + + /// Indicates the output format (text, json, ndjson) + /// defaults to ndjson for backwards compatibility + pub output_format: Option, + + /// Indicates the base path to start consolidation + pub base_path: Option, } impl Default for FileConsolidationConfig { @@ -37,6 +47,8 @@ impl Default for FileConsolidationConfig { enabled: false, process_every_ms: 600000, // 10 min requested_size_bytes: 500000000, // 500 MB + output_format: Some(DEFAULT_BASE_PATH.to_string()), + base_path: Some(DEFAULT_OUTPUT_FORMAT.to_string()), } } } @@ -51,7 +63,6 @@ pub struct FileConsolidatorAsync { tls_options: Option, file_consolidation_config: FileConsolidationConfig, bucket: String, - key_prefix: String, join_handle: Option>, } @@ -71,7 +82,6 @@ impl FileConsolidatorAsync { tls_options: Option, file_consolidation_config: FileConsolidationConfig, bucket: String, - key_prefix: String, ) -> FileConsolidatorAsync { FileConsolidatorAsync { auth, @@ -81,7 +91,6 @@ impl FileConsolidatorAsync { tls_options, file_consolidation_config, bucket, - key_prefix, join_handle: None, } } @@ -92,20 +101,33 @@ impl FileConsolidatorAsync { return false; } + let base_path = self + .file_consolidation_config + .base_path + .clone() + .unwrap_or(DEFAULT_BASE_PATH.to_string()) + .clone(); + let output_format = self + .file_consolidation_config + .output_format + .clone() + .unwrap_or(DEFAULT_OUTPUT_FORMAT.to_string()) + .clone(); + if self.join_handle.is_some() { info!( message = - "bucket={}, prefix={}, Thread for S3 file consolidation already in progress", + "bucket={}, base_path={}, Thread for S3 file consolidation already in progress", bucket = self.bucket, - key_prefix = self.key_prefix, + key_prefix = base_path, ); return false; } info!( - message = "bucket={}, prefix={}, Initiating thread for S3 file consolidation", + message = "bucket={}, base_path={}, Initiating thread for S3 file consolidation", bucket = self.bucket, - key_prefix = self.key_prefix, + key_prefix = base_path, ); const TEN_MINUTES_MS: u64 = 10 * 60 * 1000; @@ -117,7 +139,8 @@ impl FileConsolidatorAsync { }; let box_bucket = Box::new(self.bucket.clone()); - let box_key_prefix = Box::new(self.key_prefix.clone()); + let box_base_path = Box::new(base_path.clone()); + let box_output_format = Box::new(output_format.clone()); let box_auth = Box::new(self.auth.clone()); let box_region = Box::new(self.region.clone()); let box_endpoint = Box::new(self.endpoint.clone()); @@ -141,9 +164,9 @@ impl FileConsolidatorAsync { Err(e) => { error!( ?e, - "bucket={}, key_prefix={} Failed to create s3 client for consolidation", + "bucket={}, base_path={} Failed to create s3 client for consolidation", (*box_bucket).clone(), - (*box_key_prefix).clone() + (*box_base_path).clone(), ); return; } @@ -153,23 +176,24 @@ impl FileConsolidatorAsync { let start_time = tokio::time::Instant::now(); info!( - message = "bucket={}, prefix={}, Starting S3 file consolidation", + message = "bucket={}, base_path={}, Starting S3 file consolidation", bucket = (*box_bucket).clone(), - key_prefix = (*box_key_prefix).clone(), + base_path = (*box_base_path).clone(), ); let processor = FileConsolidationProcessor::new( &client, (*box_bucket).clone(), - (*box_key_prefix).clone(), *box_requested_size_bytes, + (*box_base_path).clone(), + (*box_output_format).clone(), ); processor.run().await; info!( - message = "bucket={}, prefix={}, Completed S3 file consolidation", + message = "bucket={}, base_path={}, Completed S3 file consolidation", bucket = (*box_bucket).clone(), - key_prefix = (*box_key_prefix).clone(), + base_path = (*box_base_path).clone(), ); // determine how long this action took to complete and await @@ -179,9 +203,9 @@ impl FileConsolidatorAsync { if diff > 0 { info!( message = - "bucket={}, prefix={}, processing time={} ms, restarting in {} ms", + "bucket={}, base_path={}, processing time={} ms, restarting in {} ms", bucket = (*box_bucket).clone(), - key_prefix = (*box_key_prefix).clone(), + base_path = (*box_base_path).clone(), elapsed, diff ); @@ -201,10 +225,17 @@ impl FileConsolidatorAsync { return false; } + let base_path = self + .file_consolidation_config + .base_path + .clone() + .unwrap_or(DEFAULT_BASE_PATH.to_string()) + .clone(); + info!( message = "Triggering shutdown for S3 file consolidation", bucket = self.bucket, - key_prefix = self.key_prefix, + base_path = base_path, ); if let Some(h) = self.join_handle.take() { @@ -214,7 +245,7 @@ impl FileConsolidatorAsync { info!( message = "Shutdown for S3 file consolidation complete", bucket = self.bucket, - key_prefix = self.key_prefix, + base_path = base_path, ); true diff --git a/src/sinks/aws_s3/integration_tests_mezmo.rs b/src/sinks/aws_s3/integration_tests_mezmo.rs index 51119328f..3d9a5d13f 100644 --- a/src/sinks/aws_s3/integration_tests_mezmo.rs +++ b/src/sinks/aws_s3/integration_tests_mezmo.rs @@ -173,9 +173,10 @@ async fn s3_file_consolidator_enabled_run() { enabled: true, process_every_ms: 10, requested_size_bytes: 512000000, + base_path: None, + output_format: None, }, bucket.clone(), - "/".to_owned(), ); let started = fc.start(); @@ -211,10 +212,17 @@ async fn s3_file_consolidation_process_no_files() { let bucket = uuid::Uuid::new_v4().to_string(); let key_prefix = "/".to_owned(); let requested_size_bytes: i64 = 10 * 1024 * 1024; + let output_format = "ndjson".to_owned(); create_bucket(&bucket, false).await; - let fcp = FileConsolidationProcessor::new(&s3_client, bucket, key_prefix, requested_size_bytes); + let fcp = FileConsolidationProcessor::new( + &s3_client, + bucket, + requested_size_bytes, + key_prefix, + output_format, + ); fcp.run().await; } @@ -227,6 +235,7 @@ async fn s3_file_consolidation_process_no_tagged_files() { let key_prefix = "/".to_string(); let requested_size_bytes: i64 = 10 * 1024 * 1024; let content_type = "text/x-log".to_string(); + let output_format = "ndjson".to_owned(); create_bucket(&bucket, false).await; put_file( @@ -244,8 +253,17 @@ async fn s3_file_consolidation_process_no_tagged_files() { assert_eq!(keys.len(), 1); // runs without errors - let fcp = FileConsolidationProcessor::new(&s3_client, bucket, key_prefix, requested_size_bytes); + let fcp = FileConsolidationProcessor::new( + &s3_client, + bucket.clone(), + requested_size_bytes, + key_prefix.clone(), + output_format, + ); fcp.run().await; + + let keys = get_keys(&bucket, key_prefix).await; + assert_eq!(keys.len(), 1); } #[tokio::test] @@ -257,6 +275,7 @@ async fn s3_file_consolidation_process_tag_filters() { let key_prefix = "/".to_string(); let requested_size_bytes: i64 = 1024 * 1024; let content_type = "text/x-log".to_string(); + let output_format = "ndjson".to_owned(); create_bucket(&bucket, false).await; @@ -333,113 +352,335 @@ async fn s3_file_consolidation_process_tag_filters() { ) .await; + // 6 keys should be persisted let keys = get_keys(&bucket, key_prefix.clone()).await; assert_eq!(keys.len(), 6); - // only s3 created files with ndjson and text will be merged - match get_files_to_consolidate(&s3_client, bucket.clone(), key_prefix.clone()).await { - Ok(files) => { - assert_eq!(files.len(), 2); - assert_eq!(files[0].size, 16); - assert_eq!(files[0].key, "/s3_type_ndjson.log"); + //validate the filter works for the requested types + let filtered_text = get_files_to_consolidate( + &s3_client, + bucket.clone(), + key_prefix.clone(), + "text".to_owned(), + ) + .await + .unwrap(); + assert_eq!(filtered_text.len(), 1); + assert_eq!(filtered_text[0].key, "/s3_type_text.log"); - assert_eq!(files[1].size, 14); - assert_eq!(files[1].key, "/s3_type_text.log"); - } - Err(err) => panic!("Retrieving files should not error: {}", err), - }; + let filtered_json = get_files_to_consolidate( + &s3_client, + bucket.clone(), + key_prefix.clone(), + "json".to_owned(), + ) + .await + .unwrap(); + assert_eq!(filtered_json.len(), 1); + assert_eq!(filtered_json[0].key, "/s3_type_json.log"); - let fcp = FileConsolidationProcessor::new( + let filtered_ndjson = get_files_to_consolidate( &s3_client, bucket.clone(), key_prefix.clone(), + "ndjson".to_owned(), + ) + .await + .unwrap(); + assert_eq!(filtered_ndjson.len(), 1); + assert_eq!(filtered_ndjson[0].key, "/s3_type_ndjson.log"); + + // run the consolidator and + let fcp = FileConsolidationProcessor::new( + &s3_client, + bucket.clone(), requested_size_bytes, + key_prefix.clone(), + output_format.clone(), ); fcp.run().await; - // validate we're down to 5 files now since 2 of them were merged + // no changes made since there was only a single of each type let keys = get_keys(&bucket, key_prefix.clone()).await; - assert_eq!(keys.len(), 5); - - // untouched files + assert_eq!(keys.len(), 6); assert!(keys.contains(&"/s3_sink.log".to_string())); + assert!(keys.contains(&"/s3_type_text.log".to_string())); assert!(keys.contains(&"/s3_type_json.log".to_string())); + assert!(keys.contains(&"/s3_type_ndjson.log".to_string())); assert!(keys.contains(&"/s3_type_unknown.log".to_string())); assert!(keys.contains(&"/previous_merge.log".to_string())); +} - // the new file that should contain the text of the docs - if let Some(k) = keys.into_iter().find(|s| is_merged_file(s)) { - let obj = get_object(&bucket, k).await; - assert_eq!(obj.content_encoding, Some("identity".to_string())); - assert_eq!(obj.content_type, Some("text/x-log".to_string())); - assert_eq!(obj.content_length, 32); // contents plus newlines +#[tokio::test] +async fn s3_file_consolidation_process_text_files() { + let _cx = SinkContext::new_test(); - let response_lines = get_lines(obj).await; - assert_eq!(response_lines.len(), 2); - assert!(response_lines.contains(&"file with ndjson".to_string())); - assert!(response_lines.contains(&"file with text".to_string())); - } else { - panic!("did not find the merged file as expected"); + let s3_client = client().await; + let bucket = uuid::Uuid::new_v4().to_string(); + let key_prefix = "/".to_string(); + let requested_size_bytes: i64 = 1024 * 1024; + let content_type = "text/x-log".to_string(); + let output_format = "text".to_owned(); + + create_bucket(&bucket, false).await; + + let mezmo_pipeline_s3_type_text_tags = + generate_tags("mezmo_pipeline_s3_type".to_string(), output_format.clone()); + + for i in 0..3 { + let filename = format!("file_{}.log", i + 1); + + let data = Bytes::from(format!("this is from file {}", i + 1)); + put_file( + filename, + data, + key_prefix.clone(), + bucket.clone(), + content_type.clone(), + None, + mezmo_pipeline_s3_type_text_tags.clone(), + ) + .await; + + // forcing a sleep so the file sorting is tested correctly as the + // emulator only keeps time to the second + thread::sleep(time::Duration::from_millis(1000)); } + + // 3 keys should be persisted + let keys = get_keys(&bucket, key_prefix.clone()).await; + assert_eq!(keys.len(), 3); + + // run the consolidator and + let fcp = FileConsolidationProcessor::new( + &s3_client, + bucket.clone(), + requested_size_bytes, + key_prefix.clone(), + output_format.clone(), + ); + fcp.run().await; + + // the files are properly consolidated into 1 file + let keys = get_keys(&bucket, key_prefix.clone()).await; + assert_eq!(keys.len(), 1); + assert!(is_merged_file(&keys[0])); + + let obj = get_object(&bucket, keys[0].clone()).await; + assert_eq!(obj.content_encoding, Some("identity".to_string())); + assert_eq!(obj.content_type, Some("text/x-log".to_string())); + assert_eq!(obj.content_length, 59); // line length plus newlines + + let response_lines = get_lines(obj).await; + assert_eq!(response_lines.len(), 3); + + assert_eq!(response_lines[0], "this is from file 1"); + assert_eq!(response_lines[1], "this is from file 2"); + assert_eq!(response_lines[2], "this is from file 3"); } #[tokio::test] -async fn s3_file_consolidation_compressed_files() { +async fn s3_file_consolidation_process_json_files() { + let _cx = SinkContext::new_test(); + + let s3_client = client().await; + let bucket = uuid::Uuid::new_v4().to_string(); + let key_prefix = "/".to_string(); + let requested_size_bytes: i64 = 1024 * 1024; + let content_type = "text/x-log".to_string(); + let output_format = "json".to_owned(); + + create_bucket(&bucket, false).await; + + let mezmo_pipeline_s3_type_text_tags = + generate_tags("mezmo_pipeline_s3_type".to_string(), output_format.clone()); + + for i in 0..3 { + let filename = format!("file_{}.log", i + 1); + let data = Bytes::from(format!("[\"this is from file {}\"]", i + 1)); + put_file( + filename, + data, + key_prefix.clone(), + bucket.clone(), + content_type.clone(), + None, + mezmo_pipeline_s3_type_text_tags.clone(), + ) + .await; + + // forcing a sleep so the file sorting is tested correctly as the + // emulator only keeps time to the second + thread::sleep(time::Duration::from_millis(1000)); + } + + // 3 keys should be persisted + let keys = get_keys(&bucket, key_prefix.clone()).await; + assert_eq!(keys.len(), 3); + + // run the consolidator and + let fcp = FileConsolidationProcessor::new( + &s3_client, + bucket.clone(), + requested_size_bytes, + key_prefix.clone(), + output_format.clone(), + ); + fcp.run().await; + + // the files are properly consolidated into 1 file + let keys = get_keys(&bucket, key_prefix.clone()).await; + assert_eq!(keys.len(), 1); + assert!(is_merged_file(&keys[0])); + + let obj = get_object(&bucket, keys[0].clone()).await; + assert_eq!(obj.content_encoding, Some("identity".to_string())); + assert_eq!(obj.content_type, Some("text/x-log".to_string())); + assert_eq!(obj.content_length, 67); //text with comma separators + + let response_lines = get_lines(obj).await; + assert_eq!(response_lines.len(), 1); + assert_eq!( + response_lines[0], + "[\"this is from file 1\",\"this is from file 2\",\"this is from file 3\"]" + ); +} + +#[tokio::test] +async fn s3_file_consolidation_process_ndjson_files() { let _cx = SinkContext::new_test(); let s3_client = client().await; let bucket = uuid::Uuid::new_v4().to_string(); let key_prefix = "/".to_string(); + let requested_size_bytes: i64 = 1024 * 1024; + let content_type = "text/x-log".to_string(); + let output_format = "ndjson".to_owned(); + + create_bucket(&bucket, false).await; + + let mezmo_pipeline_s3_type_text_tags = + generate_tags("mezmo_pipeline_s3_type".to_string(), output_format.clone()); + + for i in 0..3 { + let filename = format!("file_{}.log", i + 1); + let data = Bytes::from(format!( + "{{ \"message\": \"this is from file {}\" }}", + i + 1 + )); + put_file( + filename, + data, + key_prefix.clone(), + bucket.clone(), + content_type.clone(), + None, + mezmo_pipeline_s3_type_text_tags.clone(), + ) + .await; + + // forcing a sleep so the file sorting is tested correctly as the + // emulator only keeps time to the second + thread::sleep(time::Duration::from_millis(1000)); + } + + // 3 keys should be persisted + let keys = get_keys(&bucket, key_prefix.clone()).await; + assert_eq!(keys.len(), 3); + + // run the consolidator and + let fcp = FileConsolidationProcessor::new( + &s3_client, + bucket.clone(), + requested_size_bytes, + key_prefix.clone(), + output_format.clone(), + ); + fcp.run().await; + + // the files are properly consolidated into 1 file + let keys = get_keys(&bucket, key_prefix.clone()).await; + assert_eq!(keys.len(), 1); + assert!(is_merged_file(&keys[0])); + + let obj = get_object(&bucket, keys[0].clone()).await; + assert_eq!(obj.content_encoding, Some("identity".to_string())); + assert_eq!(obj.content_type, Some("text/x-log".to_string())); + assert_eq!(obj.content_length, 110); // line length plus newlines + + let response_lines = get_lines(obj).await; + assert_eq!(response_lines.len(), 3); + + assert_eq!( + response_lines[0], + "{ \"message\": \"this is from file 1\" }" + ); + assert_eq!( + response_lines[1], + "{ \"message\": \"this is from file 2\" }" + ); + assert_eq!( + response_lines[2], + "{ \"message\": \"this is from file 3\" }" + ); +} + +#[tokio::test] +async fn s3_file_consolidation_compressed_files() { + let _cx = SinkContext::new_test(); + + let s3_client = client().await; + let bucket = uuid::Uuid::new_v4().to_string(); + let key_prefix = "/compressed-files/".to_string(); let requested_size_bytes: i64 = 20 * 1024 * 1024; let content_type = "text/x-log".to_string(); + let output_format = "text".to_owned(); create_bucket(&bucket, false).await; // create some text lines and compress them - let ndjson = "{\"property\":\"fkcurxdqnnybrcutaogcvzvdttjzlcavsonfhuianreijaqfpaojjmolsibjzjvcphrjxzorjtvlbphepgfzy\"}".to_owned(); let text = "ozsggnwocqbrtuzwzudhakpibrkfnewnnuoeyopbmshpgcjicrmgasucmizjqycsvjladptmhtygwwystocxsphnyckeijpyfbvy".to_owned(); - - let compressed_ndjson = compress_text(&ndjson); let compressed_text = compress_text(&text); - let mezmo_pipeline_s3_type_ndjson_tags = - generate_tags("mezmo_pipeline_s3_type".to_string(), "ndjson".to_string()); let mezmo_pipeline_s3_type_text_tags = - generate_tags("mezmo_pipeline_s3_type".to_string(), "text".to_string()); + generate_tags("mezmo_pipeline_s3_type".to_string(), output_format.clone()); - put_file( - "ndjson.log".to_string(), - compressed_ndjson, - key_prefix.clone(), - bucket.clone(), - content_type.clone(), - Some("gzip".to_string()), - mezmo_pipeline_s3_type_ndjson_tags, - ) - .await; - put_file( - "text.log".to_string(), - compressed_text, - key_prefix.clone(), - bucket.clone(), - content_type.clone(), - Some("gzip".to_string()), - mezmo_pipeline_s3_type_text_tags, - ) - .await; + for i in 0..3 { + let filename = format!("file_{}.log", i + 1); + put_file( + filename, + compressed_text.clone(), + key_prefix.clone(), + bucket.clone(), + content_type.clone(), + Some("gzip".to_string()), + mezmo_pipeline_s3_type_text_tags.clone(), + ) + .await; + } let keys = get_keys(&bucket, key_prefix.clone()).await; - assert_eq!(keys.len(), 2); + assert_eq!(keys.len(), 3); // only s3 created files with ndjson and text will be merged - match get_files_to_consolidate(&s3_client, bucket.clone(), key_prefix.clone()).await { + match get_files_to_consolidate( + &s3_client, + bucket.clone(), + key_prefix.clone(), + output_format.clone(), + ) + .await + { Ok(files) => { - assert_eq!(files.len(), 2); - assert_eq!(files[0].size, 91); - assert_eq!(files[0].key, "/ndjson.log"); + assert_eq!(files.len(), 3); + assert_eq!(files[0].size, 85); + assert_eq!(files[0].key, "/compressed-files/file_1.log"); assert_eq!(files[1].size, 85); - assert_eq!(files[1].key, "/text.log"); + assert_eq!(files[1].key, "/compressed-files/file_2.log"); + + assert_eq!(files[2].size, 85); + assert_eq!(files[2].key, "/compressed-files/file_3.log"); } Err(err) => panic!("Retrieving files should not error: {}", err), }; @@ -447,29 +688,27 @@ async fn s3_file_consolidation_compressed_files() { let fcp = FileConsolidationProcessor::new( &s3_client, bucket.clone(), - key_prefix.clone(), requested_size_bytes, + key_prefix.clone(), + output_format.clone(), ); fcp.run().await; // validate we're down to 1 files now since 2 of them were merged let keys = get_keys(&bucket, key_prefix.clone()).await; assert_eq!(keys.len(), 1); + assert!(is_merged_file(&keys[0])); - // the new file that should contain the text of the docs uncompressed - if let Some(k) = keys.into_iter().find(|s| is_merged_file(s)) { - let obj = get_object(&bucket, k).await; - assert_eq!(obj.content_encoding, Some("identity".to_string())); - assert_eq!(obj.content_type, Some("text/x-log".to_string())); - assert_eq!(obj.content_length, 202); // decompressed plus newlines + let obj = get_object(&bucket, keys[0].clone()).await; + assert_eq!(obj.content_encoding, Some("identity".to_string())); + assert_eq!(obj.content_type, Some("text/x-log".to_string())); + assert_eq!(obj.content_length, 302); // decompressed plus newlines - let response_lines = get_lines(obj).await; - assert_eq!(response_lines.len(), 2); - assert!(response_lines.contains(&ndjson)); - assert!(response_lines.contains(&text)); - } else { - panic!("did not find the merged file as expected"); - } + let response_lines = get_lines(obj).await; + assert_eq!(response_lines.len(), 3); + assert_eq!(response_lines[0], text); + assert_eq!(response_lines[1], text); + assert_eq!(response_lines[2], text); } #[tokio::test] @@ -478,9 +717,10 @@ async fn s3_file_consolidation_multiple_consolidated_files() { let s3_client = client().await; let bucket = uuid::Uuid::new_v4().to_string(); - let key_prefix = "/".to_string(); + let key_prefix = "".to_string(); let requested_size_bytes: i64 = 1024 * 1024; let content_type = "text/x-log".to_string(); + let output_type = "ndjson".to_string(); create_bucket(&bucket, false).await; @@ -492,7 +732,7 @@ async fn s3_file_consolidation_multiple_consolidated_files() { } let mezmo_pipeline_s3_type_ndjson_tags = - generate_tags("mezmo_pipeline_s3_type".to_string(), "ndjson".to_string()); + generate_tags("mezmo_pipeline_s3_type".to_string(), output_type.clone()); let filename = format!("{}_generated.log", i); put_file( filename, @@ -501,7 +741,7 @@ async fn s3_file_consolidation_multiple_consolidated_files() { bucket.clone(), content_type.clone(), None, - mezmo_pipeline_s3_type_ndjson_tags, + mezmo_pipeline_s3_type_ndjson_tags.clone(), ) .await; } @@ -509,7 +749,14 @@ async fn s3_file_consolidation_multiple_consolidated_files() { let keys = get_keys(&bucket, key_prefix.clone()).await; assert_eq!(keys.len(), 5); - match get_files_to_consolidate(&s3_client, bucket.clone(), key_prefix.clone()).await { + match get_files_to_consolidate( + &s3_client, + bucket.clone(), + key_prefix.clone(), + output_type.clone(), + ) + .await + { Ok(files) => { assert_eq!(files.len(), 5); for file in files.iter() { @@ -522,8 +769,9 @@ async fn s3_file_consolidation_multiple_consolidated_files() { let fcp = FileConsolidationProcessor::new( &s3_client, bucket.clone(), - key_prefix.clone(), requested_size_bytes, + key_prefix.clone(), + output_type.clone(), ); fcp.run().await; @@ -559,6 +807,7 @@ async fn s3_file_consolidation_large_files() { let key_prefix = "unit-test/".to_string(); let requested_size_bytes: i64 = 100 * 1024 * 1024; let content_type = "text/x-log".to_string(); + let output_format = "text".to_string(); create_bucket(&bucket, false).await; @@ -573,7 +822,7 @@ async fn s3_file_consolidation_large_files() { } let mezmo_pipeline_s3_type_text_tags = - generate_tags("mezmo_pipeline_s3_type".to_string(), "text".to_string()); + generate_tags("mezmo_pipeline_s3_type".to_string(), output_format.clone()); let filename = format!("{}_generated.log", i); put_file( filename, @@ -636,30 +885,27 @@ async fn s3_file_consolidation_large_files() { let fcp = FileConsolidationProcessor::new( &s3_client, bucket.clone(), - key_prefix.clone(), requested_size_bytes, + key_prefix.clone(), + output_format.clone(), ); fcp.run().await; // validate we're down to 1 file let keys = get_keys(&bucket, key_prefix.clone()).await; assert_eq!(keys.len(), 1); + assert!(is_merged_file(&keys[0])); - // the new file that should contain the text of the docs - if let Some(k) = keys.into_iter().find(|s| is_merged_file(s)) { - let obj = get_object(&bucket, k).await; - assert_eq!(obj.content_encoding, Some("identity".to_string())); - assert_eq!(obj.content_type, Some("text/x-log".to_string())); + let obj = get_object(&bucket, keys[0].clone()).await; + assert_eq!(obj.content_encoding, Some("identity".to_string())); + assert_eq!(obj.content_type, Some("text/x-log".to_string())); - // 15 files of 5000 lines - // 1 file of 10 lines - // 1 file of 60000 lines - // newlines between each added file - let response_lines = get_lines(obj).await; - assert_eq!(response_lines.len(), 675_027); - } else { - panic!("did not find the merged file as expected"); - } + // 15 files of 5000 lines + // 1 file of 10 lines + // 1 file of 60000 lines + // newlines between each added file + let response_lines = get_lines(obj).await; + assert_eq!(response_lines.len(), 675_025); } #[tokio::test] @@ -671,6 +917,7 @@ async fn s3_file_consolidation_lots_of_10mb_files() { let key_prefix = "unit-test/".to_string(); let requested_size_bytes: i64 = 5_000_000_000; let content_type = "text/x-log".to_string(); + let output_format = "text".to_owned(); create_bucket(&bucket, false).await; @@ -683,7 +930,7 @@ async fn s3_file_consolidation_lots_of_10mb_files() { } let mezmo_pipeline_s3_type_text_tags = - generate_tags("mezmo_pipeline_s3_type".to_string(), "text".to_string()); + generate_tags("mezmo_pipeline_s3_type".to_string(), output_format.clone()); let filename = format!("10MB_{}_generated.log", i); put_file( filename, @@ -703,8 +950,9 @@ async fn s3_file_consolidation_lots_of_10mb_files() { let fcp = FileConsolidationProcessor::new( &s3_client, bucket.clone(), - key_prefix.clone(), requested_size_bytes, + key_prefix.clone(), + output_format.clone(), ); fcp.run().await; @@ -717,11 +965,11 @@ async fn s3_file_consolidation_lots_of_10mb_files() { let obj = get_object(&bucket, k).await; assert_eq!(obj.content_encoding, Some("identity".to_string())); assert_eq!(obj.content_type, Some("text/x-log".to_string())); - assert_eq!(obj.content_length, 151_500_015); + assert_eq!(obj.content_length, 151_500_000); // 15 files of 100_000 lines that are all bashed together let response_lines = get_lines(obj).await; - assert_eq!(response_lines.len(), 1_500_015); + assert_eq!(response_lines.len(), 1_500_000); } else { panic!("did not find the merged file as expected"); } @@ -735,6 +983,7 @@ async fn s3_file_consolidation_large_amount_of_files() { let bucket = uuid::Uuid::new_v4().to_string(); let key_prefix = "large-amount-of-files/".to_string(); let content_type = "text/x-log".to_string(); + let output_format = "text".to_owned(); create_bucket(&bucket, false).await; @@ -742,7 +991,7 @@ async fn s3_file_consolidation_large_amount_of_files() { // NOTE: this is an expensive test, takes ~45 seconds locally :/ for n in 1..1006 { let mezmo_pipeline_s3_type_ndjson_tags = - generate_tags("mezmo_pipeline_s3_type".to_string(), "text".to_string()); + generate_tags("mezmo_pipeline_s3_type".to_string(), output_format.clone()); let filename = format!("{}.log", n); let data = Bytes::from(format!("This is the content of {}.log", n)); @@ -760,7 +1009,14 @@ async fn s3_file_consolidation_large_amount_of_files() { } // only s3 created files with ndjson and text will be merged - match get_files_to_consolidate(&s3_client, bucket.clone(), key_prefix.clone()).await { + match get_files_to_consolidate( + &s3_client, + bucket.clone(), + key_prefix.clone(), + output_format.clone(), + ) + .await + { Ok(files) => { assert_eq!(files.len(), 1005); } From e9086582b383f9a25178a6215333521f14ae66f2 Mon Sep 17 00:00:00 2001 From: Darin Spivey Date: Tue, 12 Mar 2024 09:34:11 -0400 Subject: [PATCH 16/19] fix(classification): `SYSLOGLINE` is having false positive matches The default pattern for `SYSLOGLINE` in the rust port of Grok is very loosly-defined where most "matches" are just greedy data. This is causing lots of lines from our agent to be matched as syslog even though they are not. The rust port of Grok was originally based on a Java library, but that library has a strictly-defined syslog pattern. This commit adds the ability to override grok aliases with custom definitions so that we can use the strict pattern from the Java library as the definition for `SYSLOGLINE`. Ref: LOG-19416 --- src/transforms/mezmo_log_classification.rs | 80 +++++++++++++++++++++- 1 file changed, 78 insertions(+), 2 deletions(-) diff --git a/src/transforms/mezmo_log_classification.rs b/src/transforms/mezmo_log_classification.rs index b4c1f3c43..052a13286 100644 --- a/src/transforms/mezmo_log_classification.rs +++ b/src/transforms/mezmo_log_classification.rs @@ -25,12 +25,21 @@ const DEFAULT_APP_FIELDS: [&str; 3] = ["app", "application", "container"]; const DEFAULT_HOST_FIELDS: [&str; 2] = ["host", "hostname"]; const DEFAULT_LEVEL_FIELDS: [&str; 2] = ["level", "log_level"]; +/// Defines a custom grok pattern and alias. This can be used to override default grok patterns. +const CUSTOM_GROK_DEFINITIONS: [(&str, &str); 1] = [ + // Use the java grok pattern which is more strict that the rust port of grok + // @see: https://docs.rs/grok/2.0.0/src/grok/lib.rs.html#1-4 + // https://github.com/thekrakken/java-grok/blob/901fda38ef6d5c902355eb25cff3f4b4fc3debde/src/main/resources/patterns/linux-syslog#L2 + ("SYSLOGLINE", "(?:%{SYSLOGTIMESTAMP:timestamp}|%{TIMESTAMP_ISO8601:timestamp8601}) (?:%{SYSLOGFACILITY} )?%{SYSLOGHOST:logsource} %{SYSLOGPROG}: %{GREEDYDATA:message}") +]; + +/// List of grok aliases to compile and check against. Comparisons happen in order of this array. const DEFAULT_LOG_EVENT_TYPES: [&str; 67] = [ "HTTPD_COMBINEDLOG", "HTTPD_COMMONLOG", "HTTPD_ERRORLOG", "SYSLOG5424LINE", - "SYSLOGLINE", + "SYSLOGLINE", // This is overridden by a custom pattern. The default is not strict enough and is causing false positives. "SYSLOGPAMSESSION", "CRONLOG", "MONGO3_LOG", @@ -98,6 +107,11 @@ const DEFAULT_LOG_EVENT_TYPES: [&str; 67] = [ fn grok_patterns() -> &'static BTreeMap { let mut parser = grok::Grok::with_default_patterns(); + // Add aliases and custom grok patterns prior to referencing them during `.compile()` + for (alias, pattern) in CUSTOM_GROK_DEFINITIONS.iter() { + parser.add_pattern(alias.to_string(), pattern.to_string()); + } + static GROK_PATTERNS: OnceLock> = OnceLock::new(); GROK_PATTERNS.get_or_init(|| { let mut m = BTreeMap::new(); @@ -413,7 +427,7 @@ mod tests { #[tokio::test] async fn event_with_string_message() { - let line = r#"47.29.201.179 - - [28/Feb/2019:13:17:10 +0000] "GET /?p=1 HTTP/2.0" 200 5316 "https://domain1.com/?p=1" "Mozilla/5.0 (Windows NT 6.1) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/72.0.3626.119 Safari/537.36" "2.75"#; + let line = r#"47.29.201.179 - - [28/Feb/2019:13:17:10 +0000] "GET /?p=1 HTTP/2.0" 200 5316 "https://domain1.com/?p=1" "Mozilla/5.0 (Windows NT 6.1) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/72.0.3626.119 Safari/537.36" "2.75""#; let message_key = "message".to_string(); let event = Event::Log(LogEvent::from(Value::Object( btreemap!(message_key.clone() => Value::Bytes(line.into())), @@ -443,6 +457,68 @@ mod tests { ); } + #[tokio::test] + async fn similar_syslog_event_is_not_syslog() { + let line = r#"2024-02-27T18:41:21.75258589Z stderr F E0227 18:41:21.752167 1 scraper.go:140] "Failed to scrape node" err="Get \"https://192.168.1.102:10250/metrics/resource\": dial tcp 192.168.1.102:10250: connect: no route to host" node="linux02""#; + let message_key = "message".to_string(); + let event = Event::Log(LogEvent::from(Value::Object( + btreemap!(message_key.clone() => Value::Bytes(line.into())), + ))); + + let config = LogClassificationConfig { + line_fields: None, + grok_patterns: default_grok_patterns(), + app_fields: default_app_fields(), + host_fields: default_host_fields(), + level_fields: default_level_fields(), + }; + let output = do_transform(config, event.clone().into()).await.unwrap(); + + // There should be no match to this line, even though default patterns think it looks like syslog + let annotations = make_expected_annotations(&event, None, vec![]); + + assert_eq!( + output.as_log().get(message_key.as_str()), + Some(Value::Bytes(line.into())).as_ref() + ); + + assert_eq!( + output.as_log().get(log_schema().annotations_key()), + Some(&annotations) + ); + } + + #[tokio::test] + async fn syslog_custom_pattern_matches() { + let line = r#"2024-02-27T18:41:21.75258589Z myhost scraper.go[35870]: "Failed to scrape node" err="Get \"https://192.168.1.102:10250/metrics/resource\": dial tcp 192.168.1.102:10250: connect: no route to host" node="linux02""#; + let message_key = "message".to_string(); + let event = Event::Log(LogEvent::from(Value::Object( + btreemap!(message_key.clone() => Value::Bytes(line.into())), + ))); + + let config = LogClassificationConfig { + line_fields: None, + grok_patterns: default_grok_patterns(), + app_fields: default_app_fields(), + host_fields: default_host_fields(), + level_fields: default_level_fields(), + }; + let output = do_transform(config, event.clone().into()).await.unwrap(); + + let annotations = make_expected_annotations(&event, None, vec!["SYSLOGLINE".to_string()]); + + // line is retained + assert_eq!( + output.as_log().get(message_key.as_str()), + Some(Value::Bytes(line.into())).as_ref() + ); + + assert_eq!( + output.as_log().get(log_schema().annotations_key()), + Some(&annotations) + ); + } + #[tokio::test] async fn event_with_json_string_message() { let line = r#"47.29.201.179 - - [28/Feb/2019:13:17:10 +0000] "GET /?p=1 HTTP/2.0" 200 5316 "https://domain1.com/?p=1" "Mozilla/5.0 (Windows NT 6.1) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/72.0.3626.119 Safari/537.36" "2.75"#; From 9b4fae9d202fc0249aa738c8ff14051996bc8ffc Mon Sep 17 00:00:00 2001 From: LogDNA Bot Date: Tue, 12 Mar 2024 15:48:38 +0000 Subject: [PATCH 17/19] chore(release): 3.11.1 [skip ci] ## [3.11.1](https://github.com/answerbook/vector/compare/v3.11.0...v3.11.1) (2024-03-12) ### Bug Fixes * **classification**: `SYSLOGLINE` is having false positive matches [e908658](https://github.com/answerbook/vector/commit/e9086582b383f9a25178a6215333521f14ae66f2) - Darin Spivey [LOG-19416](https://logdna.atlassian.net/browse/LOG-19416) ### Miscellaneous * Merge pull request #429 from answerbook/darinspivey/LOG-19416 [c129f60](https://github.com/answerbook/vector/commit/c129f605b54128ae46eb40ec682ac4920bee5cb5) - GitHub [LOG-19416](https://logdna.atlassian.net/browse/LOG-19416) --- MEZMO_CHANGELOG.md | 12 ++++++++++++ package.json | 2 +- 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/MEZMO_CHANGELOG.md b/MEZMO_CHANGELOG.md index bb49489fe..ffb1d8b60 100644 --- a/MEZMO_CHANGELOG.md +++ b/MEZMO_CHANGELOG.md @@ -1,3 +1,15 @@ +## [3.11.1](https://github.com/answerbook/vector/compare/v3.11.0...v3.11.1) (2024-03-12) + + +### Bug Fixes + +* **classification**: `SYSLOGLINE` is having false positive matches [e908658](https://github.com/answerbook/vector/commit/e9086582b383f9a25178a6215333521f14ae66f2) - Darin Spivey [LOG-19416](https://logdna.atlassian.net/browse/LOG-19416) + + +### Miscellaneous + +* Merge pull request #429 from answerbook/darinspivey/LOG-19416 [c129f60](https://github.com/answerbook/vector/commit/c129f605b54128ae46eb40ec682ac4920bee5cb5) - GitHub [LOG-19416](https://logdna.atlassian.net/browse/LOG-19416) + # [3.11.0](https://github.com/answerbook/vector/compare/v3.10.0...v3.11.0) (2024-03-01) diff --git a/package.json b/package.json index 742be17ce..2c93dd923 100644 --- a/package.json +++ b/package.json @@ -1,6 +1,6 @@ { "name": "vector", - "version": "3.11.0", + "version": "3.11.1", "description": "Vector is a high-performance, end-to-end (agent & aggregator) observability data pipeline", "repository": { "type": "git", From a8b82eb2856129549f0b2620a0622d775068aad5 Mon Sep 17 00:00:00 2001 From: LogDNA Bot Date: Wed, 13 Mar 2024 02:02:53 +0000 Subject: [PATCH 18/19] chore(release): 3.12.0 [skip ci] # [3.12.0](https://github.com/answerbook/vector/compare/v3.11.1...v3.12.0) (2024-03-13) ### Features * **s3 sink**: add recursive directory consolidation [301ac96](https://github.com/answerbook/vector/commit/301ac96b00170af05aab9f3bfd4db1a0b4a896c8) - dominic-mcallister-logdna [LOG-19448](https://logdna.atlassian.net/browse/LOG-19448) ### Miscellaneous * Merge pull request #428 from answerbook/dominic/LOG-19448 [f43e028](https://github.com/answerbook/vector/commit/f43e028b0e500a84997afb60ff0026dc38ba0fb1) - GitHub [LOG-19448](https://logdna.atlassian.net/browse/LOG-19448) --- MEZMO_CHANGELOG.md | 12 ++++++++++++ package.json | 2 +- 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/MEZMO_CHANGELOG.md b/MEZMO_CHANGELOG.md index ffb1d8b60..2b44e92ce 100644 --- a/MEZMO_CHANGELOG.md +++ b/MEZMO_CHANGELOG.md @@ -1,3 +1,15 @@ +# [3.12.0](https://github.com/answerbook/vector/compare/v3.11.1...v3.12.0) (2024-03-13) + + +### Features + +* **s3 sink**: add recursive directory consolidation [301ac96](https://github.com/answerbook/vector/commit/301ac96b00170af05aab9f3bfd4db1a0b4a896c8) - dominic-mcallister-logdna [LOG-19448](https://logdna.atlassian.net/browse/LOG-19448) + + +### Miscellaneous + +* Merge pull request #428 from answerbook/dominic/LOG-19448 [f43e028](https://github.com/answerbook/vector/commit/f43e028b0e500a84997afb60ff0026dc38ba0fb1) - GitHub [LOG-19448](https://logdna.atlassian.net/browse/LOG-19448) + ## [3.11.1](https://github.com/answerbook/vector/compare/v3.11.0...v3.11.1) (2024-03-12) diff --git a/package.json b/package.json index 2c93dd923..95553c502 100644 --- a/package.json +++ b/package.json @@ -1,6 +1,6 @@ { "name": "vector", - "version": "3.11.1", + "version": "3.12.0", "description": "Vector is a high-performance, end-to-end (agent & aggregator) observability data pipeline", "repository": { "type": "git", From c3106a903c40ac61a957c97dae67ae092473b51a Mon Sep 17 00:00:00 2001 From: Mike Del Tito Date: Tue, 19 Mar 2024 08:43:30 -0400 Subject: [PATCH 19/19] fix(profiling): only track profiling annotations from classification The existing implementation intended to only track after the `mezmo_log_classification` transform, but was instead inadvertently tracking only when the component was also being tracked for billing. This fix implements usage tracking for "task" transforms which allows us to track after the classification transform, and also tightens up the logic such that we only track profiling data for component keys that _should_ be tracked. Ref: LOG-19502 --- Cargo.lock | 1 + lib/vector-core/Cargo.toml | 1 + lib/vector-core/src/fanout.rs | 5 +- lib/vector-core/src/usage_metrics/mod.rs | 94 +++++++++++++++++++++++- src/topology/builder.rs | 4 +- 5 files changed, 102 insertions(+), 3 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 28d15cc2c..67a1fa9e6 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10776,6 +10776,7 @@ dependencies = [ name = "vector-core" version = "0.1.0" dependencies = [ + "assay", "async-graphql", "async-trait", "base64 0.21.5", diff --git a/lib/vector-core/Cargo.toml b/lib/vector-core/Cargo.toml index d67bb000b..f14fee48c 100644 --- a/lib/vector-core/Cargo.toml +++ b/lib/vector-core/Cargo.toml @@ -80,6 +80,7 @@ schannel = "0.1.22" prost-build = "0.12" [dev-dependencies] +assay = "0.1.1" base64 = "0.21.5" chrono-tz = { version = "0.8.4", default-features = false } criterion = { version = "0.5.1", features = ["html_reports"] } diff --git a/lib/vector-core/src/fanout.rs b/lib/vector-core/src/fanout.rs index c95969900..63b1c322b 100644 --- a/lib/vector-core/src/fanout.rs +++ b/lib/vector-core/src/fanout.rs @@ -7,7 +7,7 @@ use tokio::sync::mpsc; use tokio_util::sync::ReusableBoxFuture; use vector_buffers::topology::channel::BufferSender; -use crate::{config::ComponentKey, event::EventArray}; +use crate::{config::ComponentKey, event::EventArray, usage_metrics::OutputUsageTracker}; pub enum ControlMessage { /// Adds a new sink to the fanout. @@ -170,10 +170,13 @@ impl Fanout { pub async fn send_stream( &mut self, events: impl Stream, + usage_tracker: Box, ) -> crate::Result<()> { tokio::pin!(events); while let Some(event_array) = events.next().await { + let usage_profile = usage_tracker.get_size_and_profile(&event_array); self.send(event_array).await?; + usage_tracker.track_output(usage_profile); } Ok(()) } diff --git a/lib/vector-core/src/usage_metrics/mod.rs b/lib/vector-core/src/usage_metrics/mod.rs index 965c67cef..3748d5318 100644 --- a/lib/vector-core/src/usage_metrics/mod.rs +++ b/lib/vector-core/src/usage_metrics/mod.rs @@ -126,6 +126,7 @@ impl UsageMetricsKey { /// Determines whether the component with this key should be tracked for profiling. /// Currently only during the analysis phase. fn is_tracked_for_profiling(&self) -> bool { + // The absence of a pipeline id means that the component is part of the analysis phase if self.pipeline_id.is_some() { return false; } @@ -134,6 +135,8 @@ impl UsageMetricsKey { return false; }; + // Only internal instances of the log classification component should be tracked + // for profiling. !internal && self.component_type == "mezmo_log_classification" } @@ -299,7 +302,10 @@ fn track_usage( }); } - if is_profile_enabled() && !usage.usage_by_annotation.is_empty() { + if is_profile_enabled() + && key.is_tracked_for_profiling() + && !usage.usage_by_annotation.is_empty() + { usage_by_annotation = Some(usage.usage_by_annotation); } @@ -817,6 +823,7 @@ pub fn get_db_config(endpoint_url: &str) -> Result UnboundedReceiver { + let (tx, rx) = mpsc::unbounded_channel::(); + let usage_profile = get_size_and_profile(&event.into()); + + track_usage(&tx, key, usage_profile); + rx + } + + fn annotation_path(parts: &[&str]) -> String { + log_schema().annotations_key().to_string() + "." + parts.join(".").as_str() + } + #[test] fn usage_metrics_key_parse() { assert_parse_ok!( @@ -972,6 +991,79 @@ mod tests { ); } + #[assay(env = [("USAGE_METRICS_PROFILE_ENABLED", "true")])] + fn track_usage_key_not_tracked_either() { + let key: UsageMetricsKey = "v1:filter-by-field:transform:comp1:pipe1:account1" + .parse() + .unwrap(); + let mut event_map: BTreeMap = BTreeMap::new(); + event_map.insert( + log_schema().message_key().unwrap().to_string(), + "foo".into(), + ); + let event: LogEvent = event_map.into(); + + let mut rx = track_usage_test(&key, event); + assert!(rx.try_recv().is_err()); + } + + #[assay(env = [("USAGE_METRICS_PROFILE_ENABLED", "true")])] + fn track_usage_key_billing_only() { + let key: UsageMetricsKey = "v1:remap:internal_transform:comp1:pipe1:account1" + .parse() + .unwrap(); + let mut event_map: BTreeMap = BTreeMap::new(); + event_map.insert( + log_schema().message_key().unwrap().to_string(), + "the message".into(), + ); + + let mut event: LogEvent = event_map.into(); + event.insert(annotation_path(vec!["app"].as_ref()).as_str(), "app-1"); + + let mut rx = track_usage_test(&key, event); + let tracked = rx.try_recv(); + assert!(tracked.is_ok()); + + let tracked = tracked.unwrap(); + assert_eq!(tracked.key, key, "should be the same key"); + assert!(tracked.billing.is_some(), "should contain billing metrics"); + assert!( + tracked.usage_by_annotation.is_none(), + "should NOT contain profiling metrics" + ); + } + + #[assay(env = [("USAGE_METRICS_PROFILE_ENABLED", "true")])] + fn track_usage_key_profiling_only() { + let key: UsageMetricsKey = "v1:mezmo_log_classification:transform:comp1:account1" + .parse() + .unwrap(); + let mut event_map: BTreeMap = BTreeMap::new(); + event_map.insert( + log_schema().message_key().unwrap().to_string(), + "the message".into(), + ); + + let mut event: LogEvent = event_map.into(); + event.insert(annotation_path(vec!["app"].as_ref()).as_str(), "app-1"); + + let mut rx = track_usage_test(&key, event); + let tracked = rx.try_recv(); + assert!(tracked.is_ok()); + + let tracked = tracked.unwrap(); + assert_eq!(tracked.key, key, "should be the same key"); + assert!( + tracked.billing.is_none(), + "should NOT contain billing metrics" + ); + assert!( + tracked.usage_by_annotation.is_some(), + "should contain profiling metrics" + ); + } + #[test] fn target_key_test() { let value: UsageMetricsKey = "v1:s3:source:comp1:pipe1:account1".parse().unwrap(); diff --git a/src/topology/builder.rs b/src/topology/builder.rs index 314b7f86d..dec774e7f 100644 --- a/src/topology/builder.rs +++ b/src/topology/builder.rs @@ -798,6 +798,7 @@ fn build_transform( node.typetag, &node.key, &node.outputs, + usage_tracker, ), } } @@ -1001,6 +1002,7 @@ fn build_task_transform( typetag: &str, key: &ComponentKey, outputs: &[TransformOutput], + usage_tracker: Box, ) -> (Task, HashMap) { let (mut fanout, control) = Fanout::new(); @@ -1049,7 +1051,7 @@ fn build_task_transform( let transform = async move { debug!("Task transform starting."); - match fanout.send_stream(stream).await { + match fanout.send_stream(stream, usage_tracker).await { Ok(()) => { debug!("Task transform finished normally."); Ok(TaskOutput::Transform)