From 9b3e2369514cac22e58c5601b90c33e0453a722b Mon Sep 17 00:00:00 2001 From: Bob Weinand Date: Thu, 25 Sep 2025 14:56:58 +0200 Subject: [PATCH 1/5] Introduce TraceData to unify text and binary data Also move Span structures to v04, to make space for v1 spans (which will eventually become the new default). TraceData is also going to be used in the V1 implementation, to carry around byte arrays and strings alike, separate from the indexed offsets into the big vector. Signed-off-by: Bob Weinand --- data-pipeline-ffi/src/trace_exporter.rs | 2 +- .../benches/span_concentrator_bench.rs | 2 +- .../src/span_concentrator/aggregation.rs | 21 +- data-pipeline/src/span_concentrator/mod.rs | 9 +- data-pipeline/src/span_concentrator/tests.rs | 2 +- data-pipeline/src/stats_exporter.rs | 2 +- data-pipeline/src/trace_exporter/mod.rs | 10 +- data-pipeline/src/trace_exporter/stats.rs | 8 +- .../src/trace_exporter/trace_serializer.rs | 10 +- datadog-sidecar-ffi/src/span.rs | 2 +- datadog-sidecar-ffi/tests/span.rs | 2 +- .../src/service/tracing/trace_flusher.rs | 5 +- .../src/msgpack_decoder/decode/buffer.rs | 63 +++ .../src/msgpack_decoder/decode/map.rs | 12 +- .../src/msgpack_decoder/decode/meta_struct.rs | 30 +- .../src/msgpack_decoder/decode/metrics.rs | 16 +- .../src/msgpack_decoder/decode/mod.rs | 1 + .../src/msgpack_decoder/decode/number.rs | 51 +- .../src/msgpack_decoder/decode/span_event.rs | 111 ++-- .../src/msgpack_decoder/decode/span_link.rs | 39 +- .../src/msgpack_decoder/decode/string.rs | 63 +-- .../src/msgpack_decoder/v04/mod.rs | 43 +- .../src/msgpack_decoder/v04/span.rs | 32 +- .../src/msgpack_decoder/v05/mod.rs | 108 ++-- .../src/msgpack_encoder/v04/mod.rs | 29 +- .../src/msgpack_encoder/v04/span.rs | 16 +- datadog-trace-utils/src/send_data/mod.rs | 132 ++--- datadog-trace-utils/src/span/mod.rs | 494 +++--------------- datadog-trace-utils/src/span/trace_utils.rs | 21 +- datadog-trace-utils/src/span/v04/mod.rs | 341 ++++++++++++ datadog-trace-utils/src/span/v05/mod.rs | 12 +- datadog-trace-utils/src/test_utils/mod.rs | 2 +- datadog-trace-utils/src/trace_utils.rs | 62 +-- datadog-trace-utils/src/tracer_payload.rs | 61 ++- 34 files changed, 919 insertions(+), 895 deletions(-) create mode 100644 datadog-trace-utils/src/msgpack_decoder/decode/buffer.rs create mode 100644 datadog-trace-utils/src/span/v04/mod.rs diff --git a/data-pipeline-ffi/src/trace_exporter.rs b/data-pipeline-ffi/src/trace_exporter.rs index 2f666c7f65..827bb4a7dc 100644 --- a/data-pipeline-ffi/src/trace_exporter.rs +++ b/data-pipeline-ffi/src/trace_exporter.rs @@ -544,7 +544,7 @@ pub unsafe extern "C" fn ddog_trace_exporter_send( mod tests { use super::*; use crate::error::ddog_trace_exporter_error_free; - use datadog_trace_utils::span::SpanSlice; + use datadog_trace_utils::span::v04::SpanSlice; use httpmock::prelude::*; use httpmock::MockServer; use std::{borrow::Borrow, mem::MaybeUninit}; diff --git a/data-pipeline/benches/span_concentrator_bench.rs b/data-pipeline/benches/span_concentrator_bench.rs index 5fe1dfd918..a939e0f5d0 100644 --- a/data-pipeline/benches/span_concentrator_bench.rs +++ b/data-pipeline/benches/span_concentrator_bench.rs @@ -7,7 +7,7 @@ use std::{ use criterion::{criterion_group, Criterion}; use data_pipeline::span_concentrator::SpanConcentrator; -use datadog_trace_utils::span::SpanBytes; +use datadog_trace_utils::span::v04::SpanBytes; fn get_bucket_start(now: SystemTime, n: u64) -> i64 { let start = now.duration_since(time::UNIX_EPOCH).unwrap() + Duration::from_secs(10 * n); diff --git a/data-pipeline/src/span_concentrator/aggregation.rs b/data-pipeline/src/span_concentrator/aggregation.rs index ad75a65d37..96758f24d8 100644 --- a/data-pipeline/src/span_concentrator/aggregation.rs +++ b/data-pipeline/src/span_concentrator/aggregation.rs @@ -4,10 +4,10 @@ //! This includes the aggregation key to group spans together and the computation of stats from a //! span. use datadog_trace_protobuf::pb; -use datadog_trace_utils::span::trace_utils; -use datadog_trace_utils::span::Span; -use datadog_trace_utils::span::SpanText; +use datadog_trace_utils::span::v04::Span; +use datadog_trace_utils::span::{trace_utils, SpanText, TraceData}; use hashbrown::HashMap; +use std::borrow::Borrow; const TAG_STATUS_CODE: &str = "http.status_code"; const TAG_SYNTHETICS: &str = "synthetics"; @@ -101,10 +101,7 @@ impl<'a> BorrowedAggregationKey<'a> { /// /// If `peer_tags_keys` is not empty then the peer tags of the span will be included in the /// key. - pub(super) fn from_span(span: &'a Span, peer_tag_keys: &'a [String]) -> Self - where - T: SpanText, - { + pub(super) fn from_span(span: &'a Span, peer_tag_keys: &'a [String]) -> Self { let span_kind = span .meta .get(TAG_SPANKIND) @@ -176,7 +173,7 @@ impl From for OwnedAggregationKey { /// Return the status code of a span based on the metrics and meta tags. fn get_status_code(span: &Span) -> u32 where - T: SpanText, + T: TraceData, { if let Some(status_code) = span.metrics.get(TAG_STATUS_CODE) { *status_code as u32 @@ -205,7 +202,7 @@ fn get_peer_tags<'k, 'v, T>( peer_tag_keys: &'k [String], ) -> Vec<(&'k str, &'v str)> where - T: SpanText, + T: TraceData, { peer_tag_keys .iter() @@ -228,7 +225,7 @@ impl GroupedStats { /// Update the stats of a GroupedStats by inserting a span. fn insert(&mut self, value: &Span) where - T: SpanText, + T: TraceData, { self.hits += 1; self.duration += value.duration as u64; @@ -266,7 +263,7 @@ impl StatsBucket { /// not exist it creates it. pub(super) fn insert(&mut self, key: BorrowedAggregationKey<'_>, value: &Span) where - T: SpanText, + T: TraceData, { self.data.entry_ref(&key).or_default().insert(value); } @@ -326,7 +323,7 @@ fn encode_grouped_stats(key: OwnedAggregationKey, group: GroupedStats) -> pb::Cl #[cfg(test)] mod tests { - use datadog_trace_utils::span::{SpanBytes, SpanSlice}; + use datadog_trace_utils::span::v04::{SpanBytes, SpanSlice}; use super::*; use std::{collections::HashMap, hash::Hash}; diff --git a/data-pipeline/src/span_concentrator/mod.rs b/data-pipeline/src/span_concentrator/mod.rs index 02332d7b5b..1d22af68f1 100644 --- a/data-pipeline/src/span_concentrator/mod.rs +++ b/data-pipeline/src/span_concentrator/mod.rs @@ -1,11 +1,12 @@ // Copyright 2024-Present Datadog, Inc. https://www.datadoghq.com/ // SPDX-License-Identifier: Apache-2.0 //! This module implements the SpanConcentrator used to aggregate spans into stats +use std::borrow::Borrow; use std::collections::HashMap; use std::time::{self, Duration, SystemTime}; use datadog_trace_protobuf::pb; -use datadog_trace_utils::span::{trace_utils, Span, SpanText}; +use datadog_trace_utils::span::{trace_utils, v04::Span, TraceData}; use aggregation::{BorrowedAggregationKey, StatsBucket}; @@ -27,7 +28,7 @@ fn align_timestamp(t: u64, bucket_size: u64) -> u64 { /// Return true if the span has a span.kind that is eligible for stats computation fn compute_stats_for_span_kind(span: &Span, span_kinds_stats_computed: &[String]) -> bool where - T: SpanText, + T: TraceData, { !span_kinds_stats_computed.is_empty() && span.meta.get("span.kind").is_some_and(|span_kind| { @@ -38,7 +39,7 @@ where /// Return true if the span should be ignored for stats computation fn should_ignore_span(span: &Span, span_kinds_stats_computed: &[String]) -> bool where - T: SpanText, + T: TraceData, { !(trace_utils::has_top_level(span) || trace_utils::is_measured(span) @@ -121,7 +122,7 @@ impl SpanConcentrator { /// computation. pub fn add_span(&mut self, span: &Span) where - T: SpanText, + T: TraceData, { // If the span is eligible for stats computation if !should_ignore_span(span, self.span_kinds_stats_computed.as_slice()) { diff --git a/data-pipeline/src/span_concentrator/tests.rs b/data-pipeline/src/span_concentrator/tests.rs index 318a2780fe..03a671e8d9 100644 --- a/data-pipeline/src/span_concentrator/tests.rs +++ b/data-pipeline/src/span_concentrator/tests.rs @@ -4,7 +4,7 @@ use crate::span_concentrator::aggregation::OwnedAggregationKey; use super::*; -use datadog_trace_utils::span::{trace_utils::compute_top_level_span, SpanSlice}; +use datadog_trace_utils::span::{trace_utils::compute_top_level_span, v04::SpanSlice}; use rand::{thread_rng, Rng}; const BUCKET_SIZE: u64 = Duration::from_secs(2).as_nanos() as u64; diff --git a/data-pipeline/src/stats_exporter.rs b/data-pipeline/src/stats_exporter.rs index f80ceef67c..778dfdf436 100644 --- a/data-pipeline/src/stats_exporter.rs +++ b/data-pipeline/src/stats_exporter.rs @@ -188,7 +188,7 @@ pub fn stats_url_from_agent_url(agent_url: &str) -> anyhow::Result { #[cfg(test)] mod tests { use super::*; - use datadog_trace_utils::span::{trace_utils, SpanSlice}; + use datadog_trace_utils::span::{trace_utils, v04::SpanSlice}; use datadog_trace_utils::test_utils::poll_for_mock_hit; use httpmock::prelude::*; use httpmock::MockServer; diff --git a/data-pipeline/src/trace_exporter/mod.rs b/data-pipeline/src/trace_exporter/mod.rs index 6162656643..3ab0e44708 100644 --- a/data-pipeline/src/trace_exporter/mod.rs +++ b/data-pipeline/src/trace_exporter/mod.rs @@ -34,7 +34,7 @@ use datadog_trace_utils::msgpack_decoder; use datadog_trace_utils::send_with_retry::{ send_with_retry, RetryStrategy, SendWithRetryError, SendWithRetryResult, }; -use datadog_trace_utils::span::{Span, SpanText}; +use datadog_trace_utils::span::{v04::Span, TraceData}; use datadog_trace_utils::trace_utils::TracerHeaderTags; use ddcommon::MutexExt; use ddcommon::{hyper_migration, Endpoint}; @@ -577,7 +577,7 @@ impl TraceExporter { /// # Returns /// * Ok(String): The response from the agent /// * Err(TraceExporterError): An error detailing what went wrong in the process - pub fn send_trace_chunks( + pub fn send_trace_chunks( &self, trace_chunks: Vec>>, ) -> Result { @@ -658,7 +658,7 @@ impl TraceExporter { self.handle_send_result(result, chunks, payload_len).await } - fn send_trace_chunks_inner( + fn send_trace_chunks_inner( &self, mut traces: Vec>>, ) -> Result { @@ -982,8 +982,8 @@ mod tests { use self::error::AgentErrorKind; use super::*; use datadog_trace_utils::msgpack_encoder; + use datadog_trace_utils::span::v04::SpanBytes; use datadog_trace_utils::span::v05; - use datadog_trace_utils::span::SpanBytes; use httpmock::prelude::*; use httpmock::MockServer; use std::collections::HashMap; @@ -1993,7 +1993,7 @@ mod single_threaded_tests { use super::*; use crate::agent_info; use datadog_trace_utils::msgpack_encoder; - use datadog_trace_utils::span::SpanBytes; + use datadog_trace_utils::span::v04::SpanBytes; use httpmock::prelude::*; use std::time::Duration; use tokio::time::sleep; diff --git a/data-pipeline/src/trace_exporter/stats.rs b/data-pipeline/src/trace_exporter/stats.rs index b3c5021274..c36b43e497 100644 --- a/data-pipeline/src/trace_exporter/stats.rs +++ b/data-pipeline/src/trace_exporter/stats.rs @@ -200,9 +200,9 @@ pub(crate) fn handle_stats_enabled( /// Add all spans from the given iterator into the stats concentrator /// # Panic /// Will panic if another thread panicked will holding the lock on `stats_concentrator` -fn add_spans_to_stats( +fn add_spans_to_stats( stats_concentrator: &Mutex, - traces: &[Vec>], + traces: &[Vec>], ) { let mut stats_concentrator = stats_concentrator.lock_or_panic(); @@ -213,8 +213,8 @@ fn add_spans_to_stats( } /// Process traces for stats computation and update header tags accordingly -pub(crate) fn process_traces_for_stats( - traces: &mut Vec>>, +pub(crate) fn process_traces_for_stats( + traces: &mut Vec>>, header_tags: &mut datadog_trace_utils::trace_utils::TracerHeaderTags, client_side_stats: &ArcSwap, client_computed_top_level: bool, diff --git a/data-pipeline/src/trace_exporter/trace_serializer.rs b/data-pipeline/src/trace_exporter/trace_serializer.rs index 2e4359fca6..dc121c989c 100644 --- a/data-pipeline/src/trace_exporter/trace_serializer.rs +++ b/data-pipeline/src/trace_exporter/trace_serializer.rs @@ -8,7 +8,7 @@ use crate::trace_exporter::error::TraceExporterError; use crate::trace_exporter::TraceExporterOutputFormat; use datadog_trace_utils::msgpack_decoder::decode::error::DecodeError; use datadog_trace_utils::msgpack_encoder; -use datadog_trace_utils::span::{Span, SpanText}; +use datadog_trace_utils::span::{v04::Span, TraceData}; use datadog_trace_utils::trace_utils::{self, TracerHeaderTags}; use datadog_trace_utils::tracer_payload; use ddcommon::header::{ @@ -46,7 +46,7 @@ impl<'a> TraceSerializer<'a> { } /// Prepare traces payload and HTTP headers for sending to agent - pub(super) fn prepare_traces_payload( + pub(super) fn prepare_traces_payload( &self, traces: Vec>>, header_tags: TracerHeaderTags, @@ -64,7 +64,7 @@ impl<'a> TraceSerializer<'a> { } /// Collect trace chunks based on output format - fn collect_and_process_traces( + fn collect_and_process_traces( &self, traces: Vec>>, ) -> Result, TraceExporterError> { @@ -97,7 +97,7 @@ impl<'a> TraceSerializer<'a> { } /// Serialize payload to msgpack format - fn serialize_payload( + fn serialize_payload( &self, payload: &tracer_payload::TraceChunks, ) -> Result, TraceExporterError> { @@ -114,7 +114,7 @@ impl<'a> TraceSerializer<'a> { mod tests { use super::*; use crate::trace_exporter::agent_response::AgentResponsePayloadVersion; - use datadog_trace_utils::span::SpanBytes; + use datadog_trace_utils::span::v04::SpanBytes; use datadog_trace_utils::trace_utils::TracerHeaderTags; use ddcommon::header::{ APPLICATION_MSGPACK_STR, DATADOG_SEND_REAL_HTTP_STATUS_STR, DATADOG_TRACE_COUNT_STR, diff --git a/datadog-sidecar-ffi/src/span.rs b/datadog-sidecar-ffi/src/span.rs index 8b7fbb9960..d25510fae8 100644 --- a/datadog-sidecar-ffi/src/span.rs +++ b/datadog-sidecar-ffi/src/span.rs @@ -1,7 +1,7 @@ // Copyright 2024-Present Datadog, Inc. https://www.datadoghq.com/ // SPDX-License-Identifier: Apache-2.0 -use datadog_trace_utils::span::{ +use datadog_trace_utils::span::v04::{ AttributeAnyValueBytes, AttributeArrayValueBytes, SpanBytes, SpanEventBytes, SpanLinkBytes, }; use ddcommon_ffi::slice::{AsBytes, CharSlice}; diff --git a/datadog-sidecar-ffi/tests/span.rs b/datadog-sidecar-ffi/tests/span.rs index e4cb81c46a..64508cd38e 100644 --- a/datadog-sidecar-ffi/tests/span.rs +++ b/datadog-sidecar-ffi/tests/span.rs @@ -2,7 +2,7 @@ // SPDX-License-Identifier: Apache-2.0 use datadog_sidecar_ffi::span::*; -use datadog_trace_utils::span::*; +use datadog_trace_utils::span::v04::*; use ddcommon_ffi::slice::*; use std::collections::HashMap; use tinybytes::*; diff --git a/datadog-sidecar/src/service/tracing/trace_flusher.rs b/datadog-sidecar/src/service/tracing/trace_flusher.rs index 3a3524faf9..093a4a9b21 100644 --- a/datadog-sidecar/src/service/tracing/trace_flusher.rs +++ b/datadog-sidecar/src/service/tracing/trace_flusher.rs @@ -352,9 +352,8 @@ mod tests { }; let send_data_1 = create_send_data(size, &target_endpoint); - - let send_data_2 = send_data_1.clone(); - let send_data_3 = send_data_1.clone(); + let send_data_2 = create_send_data(size, &target_endpoint); + let send_data_3 = create_send_data(size, &target_endpoint); trace_flusher.enqueue(send_data_1); trace_flusher.enqueue(send_data_2); diff --git a/datadog-trace-utils/src/msgpack_decoder/decode/buffer.rs b/datadog-trace-utils/src/msgpack_decoder/decode/buffer.rs new file mode 100644 index 0000000000..76df2f1c1f --- /dev/null +++ b/datadog-trace-utils/src/msgpack_decoder/decode/buffer.rs @@ -0,0 +1,63 @@ +// Copyright 2024-Present Datadog, Inc. https://www.datadoghq.com/ +// SPDX-License-Identifier: Apache-2.0 + +use crate::msgpack_decoder::decode::error::DecodeError; +use crate::span::TraceData; +use rmp::decode; +use rmp::decode::DecodeStringError; + +use std::borrow::Borrow; +use std::ops::Deref; + +/// Read a string from `buf`. +/// +/// # Errors +/// Fails if the buffer doesn't contain a valid utf8 msgpack string. +#[inline] +pub fn read_string_ref_nomut(buf: &[u8]) -> Result<(&str, &[u8]), DecodeError> { + decode::read_str_from_slice(buf).map_err(|e| match e { + DecodeStringError::InvalidMarkerRead(e) => DecodeError::InvalidFormat(e.to_string()), + DecodeStringError::InvalidDataRead(e) => DecodeError::InvalidConversion(e.to_string()), + DecodeStringError::TypeMismatch(marker) => { + DecodeError::InvalidType(format!("Type mismatch at marker {marker:?}")) + } + DecodeStringError::InvalidUtf8(_, e) => DecodeError::Utf8Error(e.to_string()), + _ => DecodeError::IOError, + }) +} + +/// Internal Buffer used to wrap msgpack data for decoding. +/// Provides a couple accessors to extract data from the buffer. +pub struct Buffer(T::Bytes); + +impl Buffer { + pub fn new(data: T::Bytes) -> Self { + Buffer(data) + } + + /// Returns a mutable reference to the underlying slice. + pub fn as_mut_slice(&mut self) -> &mut &'static [u8] { + T::get_mut_slice(&mut self.0) + } + + /// Tries to extract a slice of `bytes` from the buffer and advances the buffer. + pub fn try_slice_and_advance(&mut self, bytes: usize) -> Option { + T::try_slice_and_advance(&mut self.0, bytes) + } + + /// Read a string from the slices `buf`. + /// + /// # Errors + /// Fails if the buffer doesn't contain a valid utf8 msgpack string. + pub fn read_string(&mut self) -> Result { + T::read_string(&mut self.0) + } +} + +impl Deref for Buffer { + type Target = [u8]; + + fn deref(&self) -> &Self::Target { + self.0.borrow() + } +} diff --git a/datadog-trace-utils/src/msgpack_decoder/decode/map.rs b/datadog-trace-utils/src/msgpack_decoder/decode/map.rs index c6dd22fb1d..9641d80864 100644 --- a/datadog-trace-utils/src/msgpack_decoder/decode/map.rs +++ b/datadog-trace-utils/src/msgpack_decoder/decode/map.rs @@ -1,7 +1,8 @@ // Copyright 2024-Present Datadog, Inc. https://www.datadoghq.com/ // SPDX-License-Identifier: Apache-2.0 -use crate::msgpack_decoder::decode::error::DecodeError; +use crate::msgpack_decoder::decode::{buffer::Buffer, error::DecodeError}; +use crate::span::TraceData; use rmp::{decode, decode::RmpRead, Marker}; use std::collections::HashMap; @@ -33,14 +34,14 @@ use std::collections::HashMap; /// * `V` - The type of the values in the map. /// * `F` - The type of the function used to read key-value pairs from the buffer. #[inline] -pub fn read_map<'a, K, V, F>( +pub fn read_map( len: usize, - buf: &mut &'a [u8], + buf: &mut B, read_pair: F, ) -> Result, DecodeError> where K: std::hash::Hash + Eq, - F: Fn(&mut &'a [u8]) -> Result<(K, V), DecodeError>, + F: Fn(&mut B) -> Result<(K, V), DecodeError>, { let mut map = HashMap::with_capacity(len); for _ in 0..len { @@ -67,7 +68,8 @@ where /// - The buffer does not contain a map. /// - There is an error reading from the buffer. #[inline] -pub fn read_map_len(buf: &mut &[u8]) -> Result { +pub fn read_map_len(buf: &mut Buffer) -> Result { + let buf = buf.as_mut_slice(); match decode::read_marker(buf) .map_err(|_| DecodeError::InvalidFormat("Unable to read marker for map".to_owned()))? { diff --git a/datadog-trace-utils/src/msgpack_decoder/decode/meta_struct.rs b/datadog-trace-utils/src/msgpack_decoder/decode/meta_struct.rs index 145200877c..c32104d7b4 100644 --- a/datadog-trace-utils/src/msgpack_decoder/decode/meta_struct.rs +++ b/datadog-trace-utils/src/msgpack_decoder/decode/meta_struct.rs @@ -1,33 +1,35 @@ // Copyright 2024-Present Datadog, Inc. https://www.datadoghq.com/ // SPDX-License-Identifier: Apache-2.0 +use crate::msgpack_decoder::decode::buffer::Buffer; use crate::msgpack_decoder::decode::error::DecodeError; use crate::msgpack_decoder::decode::map::{read_map, read_map_len}; -use crate::msgpack_decoder::decode::string::{handle_null_marker, read_string_ref}; +use crate::msgpack_decoder::decode::string::handle_null_marker; +use crate::span::TraceData; use rmp::decode; use std::collections::HashMap; -use tinybytes::Bytes; -fn read_byte_array_len(buf: &mut &[u8]) -> Result { - decode::read_bin_len(buf).map_err(|_| { +fn read_byte_array_len(buf: &mut Buffer) -> Result { + decode::read_bin_len(buf.as_mut_slice()).map_err(|_| { DecodeError::InvalidFormat("Unable to read binary len for meta_struct".to_owned()) }) } #[inline] -pub fn read_meta_struct<'a>(buf: &mut &'a [u8]) -> Result, DecodeError> { +pub fn read_meta_struct( + buf: &mut Buffer, +) -> Result, DecodeError> { if handle_null_marker(buf) { return Ok(HashMap::default()); } - fn read_meta_struct_pair<'a>(buf: &mut &'a [u8]) -> Result<(&'a str, Bytes), DecodeError> { - let key = read_string_ref(buf)?; + fn read_meta_struct_pair( + buf: &mut Buffer, + ) -> Result<(T::Text, T::Bytes), DecodeError> { + let key = buf.read_string()?; let byte_array_len = read_byte_array_len(buf)? as usize; - let slice = buf.get(0..byte_array_len); - if let Some(slice) = slice { - let data = Bytes::copy_from_slice(slice); - *buf = &buf[byte_array_len..]; + if let Some(data) = buf.try_slice_and_advance(byte_array_len) { Ok((key, data)) } else { Err(DecodeError::InvalidFormat( @@ -43,13 +45,15 @@ pub fn read_meta_struct<'a>(buf: &mut &'a [u8]) -> Result::new(serialized.as_ref()); let res = read_meta_struct(&mut slice).unwrap(); assert_eq!(res.get("key").unwrap().to_vec(), vec![1, 2, 3, 4]); @@ -60,7 +64,7 @@ mod tests { let meta = HashMap::from([("key".to_string(), vec![1, 2, 3, 4])]); let serialized = rmp_serde::to_vec_named(&meta).unwrap(); - let mut slice = serialized.as_ref(); + let mut slice = Buffer::::new(serialized.as_ref()); let res = read_meta_struct(&mut slice); assert!(res.is_err()); diff --git a/datadog-trace-utils/src/msgpack_decoder/decode/metrics.rs b/datadog-trace-utils/src/msgpack_decoder/decode/metrics.rs index 5b84ea16be..8dfc8cefed 100644 --- a/datadog-trace-utils/src/msgpack_decoder/decode/metrics.rs +++ b/datadog-trace-utils/src/msgpack_decoder/decode/metrics.rs @@ -1,21 +1,25 @@ // Copyright 2024-Present Datadog, Inc. https://www.datadoghq.com/ // SPDX-License-Identifier: Apache-2.0 +use crate::msgpack_decoder::decode::buffer::Buffer; use crate::msgpack_decoder::decode::error::DecodeError; use crate::msgpack_decoder::decode::map::{read_map, read_map_len}; -use crate::msgpack_decoder::decode::number::read_number_slice; -use crate::msgpack_decoder::decode::string::{handle_null_marker, read_string_ref}; +use crate::msgpack_decoder::decode::number::read_number; +use crate::msgpack_decoder::decode::string::handle_null_marker; +use crate::span::TraceData; use std::collections::HashMap; #[inline] -pub fn read_metric_pair<'a>(buf: &mut &'a [u8]) -> Result<(&'a str, f64), DecodeError> { - let key = read_string_ref(buf)?; - let v = read_number_slice(buf)?; +pub fn read_metric_pair(buf: &mut Buffer) -> Result<(T::Text, f64), DecodeError> { + let key = buf.read_string()?; + let v = read_number(buf)?; Ok((key, v)) } #[inline] -pub fn read_metrics<'a>(buf: &mut &'a [u8]) -> Result, DecodeError> { +pub fn read_metrics( + buf: &mut Buffer, +) -> Result, DecodeError> { if handle_null_marker(buf) { return Ok(HashMap::default()); } diff --git a/datadog-trace-utils/src/msgpack_decoder/decode/mod.rs b/datadog-trace-utils/src/msgpack_decoder/decode/mod.rs index 156f8614d1..48f0d75506 100644 --- a/datadog-trace-utils/src/msgpack_decoder/decode/mod.rs +++ b/datadog-trace-utils/src/msgpack_decoder/decode/mod.rs @@ -1,6 +1,7 @@ // Copyright 2024-Present Datadog, Inc. https://www.datadoghq.com/ // SPDX-License-Identifier: Apache-2.0 +pub mod buffer; pub mod error; pub mod map; pub mod meta_struct; diff --git a/datadog-trace-utils/src/msgpack_decoder/decode/number.rs b/datadog-trace-utils/src/msgpack_decoder/decode/number.rs index 10ef05b02d..d057ba9545 100644 --- a/datadog-trace-utils/src/msgpack_decoder/decode/number.rs +++ b/datadog-trace-utils/src/msgpack_decoder/decode/number.rs @@ -1,7 +1,9 @@ // Copyright 2024-Present Datadog, Inc. https://www.datadoghq.com/ // SPDX-License-Identifier: Apache-2.0 +use crate::msgpack_decoder::decode::buffer::Buffer; use crate::msgpack_decoder::decode::error::DecodeError; +use crate::span::TraceData; use rmp::{decode::RmpRead, Marker}; use std::fmt; @@ -146,7 +148,7 @@ impl TryFrom for f64 { } } -fn read_number(buf: &mut &[u8], allow_null: bool) -> Result { +fn read_num(buf: &mut &[u8], allow_null: bool) -> Result { match rmp::decode::read_marker(buf) .map_err(|_| DecodeError::InvalidFormat("Unable to read marker for number".to_owned()))? { @@ -194,22 +196,23 @@ fn read_number(buf: &mut &[u8], allow_null: bool) -> Result } /// Read a msgpack encoded number from `buf`. -pub fn read_number_slice>( - buf: &mut &[u8], -) -> Result { - read_number(buf, false)?.try_into() +pub fn read_number>( + buf: &mut Buffer, +) -> Result { + read_num(buf.as_mut_slice(), false)?.try_into() } /// Read a msgpack encoded number from `buf` and return 0 if null. -pub fn read_nullable_number_slice>( - buf: &mut &[u8], -) -> Result { - read_number(buf, true)?.try_into() +pub fn read_nullable_number>( + buf: &mut Buffer, +) -> Result { + read_num(buf.as_mut_slice(), true)?.try_into() } #[cfg(test)] mod tests { use super::*; + use crate::span::SliceData; use serde_json::json; use std::f64; @@ -219,8 +222,8 @@ mod tests { let expected_value = 42; let val = json!(expected_value); rmp_serde::encode::write_named(&mut buf, &val).unwrap(); - let mut slice = buf.as_slice(); - let result: u8 = read_number_slice(&mut slice).unwrap(); + let mut slice = Buffer::::new(buf.as_slice()); + let result: u8 = read_number(&mut slice).unwrap(); assert_eq!(result, expected_value); } @@ -230,8 +233,8 @@ mod tests { let expected_value = 42; let val = json!(expected_value); rmp_serde::encode::write_named(&mut buf, &val).unwrap(); - let mut slice = buf.as_slice(); - let result: i8 = read_number_slice(&mut slice).unwrap(); + let mut slice = Buffer::::new(buf.as_slice()); + let result: i8 = read_number(&mut slice).unwrap(); assert_eq!(result, expected_value); } @@ -241,18 +244,18 @@ mod tests { let expected_value = 42.98; let val = json!(expected_value); rmp_serde::encode::write_named(&mut buf, &val).unwrap(); - let mut slice = buf.as_slice(); - let result: f64 = read_number_slice(&mut slice).unwrap(); + let mut slice = Buffer::::new(buf.as_slice()); + let result: f64 = read_number(&mut slice).unwrap(); assert_eq!(result, expected_value); } #[test] - fn test_decoding_null_through_read_number_slice_raises_exception() { + fn test_decoding_null_through_read_number_raises_exception() { let mut buf = Vec::new(); let val = json!(null); rmp_serde::encode::write_named(&mut buf, &val).unwrap(); - let mut slice = buf.as_slice(); - let result: Result = read_number_slice(&mut slice); + let mut slice = Buffer::::new(buf.as_slice()); + let result: Result = read_number(&mut slice); assert!(matches!(result, Err(DecodeError::InvalidType(_)))); assert_eq!( @@ -266,8 +269,8 @@ mod tests { let mut buf = Vec::new(); let val = json!(null); rmp_serde::encode::write_named(&mut buf, &val).unwrap(); - let mut slice = buf.as_slice(); - let result: u8 = read_nullable_number_slice(&mut slice).unwrap(); + let mut slice = Buffer::::new(buf.as_slice()); + let result: u8 = read_nullable_number(&mut slice).unwrap(); assert_eq!(result, 0); } @@ -276,8 +279,8 @@ mod tests { let mut buf = Vec::new(); let val = json!(null); rmp_serde::encode::write_named(&mut buf, &val).unwrap(); - let mut slice = buf.as_slice(); - let result: i8 = read_nullable_number_slice(&mut slice).unwrap(); + let mut slice = Buffer::::new(buf.as_slice()); + let result: i8 = read_nullable_number(&mut slice).unwrap(); assert_eq!(result, 0); } @@ -286,8 +289,8 @@ mod tests { let mut buf = Vec::new(); let val = json!(null); rmp_serde::encode::write_named(&mut buf, &val).unwrap(); - let mut slice = buf.as_slice(); - let result: f64 = read_nullable_number_slice(&mut slice).unwrap(); + let mut slice = Buffer::::new(buf.as_slice()); + let result: f64 = read_nullable_number(&mut slice).unwrap(); assert_eq!(result, 0.0); } diff --git a/datadog-trace-utils/src/msgpack_decoder/decode/span_event.rs b/datadog-trace-utils/src/msgpack_decoder/decode/span_event.rs index 4b3cb0cc43..6befce9f40 100644 --- a/datadog-trace-utils/src/msgpack_decoder/decode/span_event.rs +++ b/datadog-trace-utils/src/msgpack_decoder/decode/span_event.rs @@ -1,10 +1,13 @@ // Copyright 2025-Present Datadog, Inc. https://www.datadoghq.com/ // SPDX-License-Identifier: Apache-2.0 +use crate::msgpack_decoder::decode::buffer::Buffer; use crate::msgpack_decoder::decode::error::DecodeError; -use crate::msgpack_decoder::decode::number::read_number_slice; -use crate::msgpack_decoder::decode::string::{handle_null_marker, read_string_ref}; -use crate::span::{AttributeAnyValueSlice, AttributeArrayValueSlice, SpanEventSlice}; +use crate::msgpack_decoder::decode::number::read_number; +use crate::msgpack_decoder::decode::string::handle_null_marker; +use crate::span::v04::{AttributeAnyValue, AttributeArrayValue, SpanEvent}; +use crate::span::TraceData; +use std::borrow::Borrow; use std::collections::HashMap; use std::str::FromStr; @@ -24,18 +27,18 @@ use std::str::FromStr; /// This function will return an error if: /// - The marker for the array length cannot be read. /// - Any `SpanEvent` cannot be decoded. -pub(crate) fn read_span_events<'a>( - buf: &mut &'a [u8], -) -> Result>, DecodeError> { +pub(crate) fn read_span_events( + buf: &mut Buffer, +) -> Result>, DecodeError> { if handle_null_marker(buf) { return Ok(Vec::default()); } - let len = rmp::decode::read_array_len(buf).map_err(|_| { + let len = rmp::decode::read_array_len(buf.as_mut_slice()).map_err(|_| { DecodeError::InvalidType("Unable to get array len for span events".to_owned()) })?; - let mut vec: Vec = Vec::with_capacity(len as usize); + let mut vec: Vec> = Vec::with_capacity(len as usize); for _ in 0..len { vec.push(decode_span_event(buf)?); } @@ -63,15 +66,15 @@ impl FromStr for SpanEventKey { } } -fn decode_span_event<'a>(buf: &mut &'a [u8]) -> Result, DecodeError> { - let mut event = SpanEventSlice::default(); - let event_size = rmp::decode::read_map_len(buf) +fn decode_span_event(buf: &mut Buffer) -> Result, DecodeError> { + let mut event = SpanEvent::default(); + let event_size = rmp::decode::read_map_len(buf.as_mut_slice()) .map_err(|_| DecodeError::InvalidType("Unable to get map len for event size".to_owned()))?; for _ in 0..event_size { - match read_string_ref(buf)?.parse::()? { - SpanEventKey::TimeUnixNano => event.time_unix_nano = read_number_slice(buf)?, - SpanEventKey::Name => event.name = read_string_ref(buf)?, + match buf.read_string()?.borrow().parse::()? { + SpanEventKey::TimeUnixNano => event.time_unix_nano = read_number(buf)?, + SpanEventKey::Name => event.name = buf.read_string()?, SpanEventKey::Attributes => event.attributes = read_attributes_map(buf)?, } } @@ -79,16 +82,16 @@ fn decode_span_event<'a>(buf: &mut &'a [u8]) -> Result, Decod Ok(event) } -fn read_attributes_map<'a>( - buf: &mut &'a [u8], -) -> Result>, DecodeError> { - let len = rmp::decode::read_map_len(buf) +fn read_attributes_map( + buf: &mut Buffer, +) -> Result>, DecodeError> { + let len = rmp::decode::read_map_len(buf.as_mut_slice()) .map_err(|_| DecodeError::InvalidType("Unable to get map len for attributes".to_owned()))?; #[allow(clippy::expect_used)] let mut map = HashMap::with_capacity(len.try_into().expect("Unable to cast map len to usize")); for _ in 0..len { - let key = read_string_ref(buf)?; + let key = buf.read_string()?; let value = decode_attribute_any(buf)?; map.insert(key, value); } @@ -121,9 +124,11 @@ impl FromStr for AttributeAnyKey { } } -fn decode_attribute_any<'a>(buf: &mut &'a [u8]) -> Result, DecodeError> { - let mut attribute: Option = None; - let attribute_size = rmp::decode::read_map_len(buf).map_err(|_| { +fn decode_attribute_any( + buf: &mut Buffer, +) -> Result, DecodeError> { + let mut attribute: Option> = None; + let attribute_size = rmp::decode::read_map_len(buf.as_mut_slice()).map_err(|_| { DecodeError::InvalidType("Unable to get map len for attribute size".to_owned()) })?; @@ -135,15 +140,15 @@ fn decode_attribute_any<'a>(buf: &mut &'a [u8]) -> Result = None; for _ in 0..attribute_size { - match read_string_ref(buf)?.parse::()? { - AttributeAnyKey::Type => attribute_type = Some(read_number_slice(buf)?), + match buf.read_string()?.borrow().parse::()? { + AttributeAnyKey::Type => attribute_type = Some(read_number(buf)?), AttributeAnyKey::SingleValue(key) => { - attribute = Some(AttributeAnyValueSlice::SingleValue(get_attribute_from_key( + attribute = Some(AttributeAnyValue::SingleValue(get_attribute_from_key( buf, key, )?)) } AttributeAnyKey::ArrayValue => { - attribute = Some(AttributeAnyValueSlice::Array(read_attributes_array(buf)?)) + attribute = Some(AttributeAnyValue::Array(read_attributes_array(buf)?)) } } } @@ -168,14 +173,14 @@ fn decode_attribute_any<'a>(buf: &mut &'a [u8]) -> Result( - buf: &mut &'a [u8], -) -> Result>, DecodeError> { +fn read_attributes_array( + buf: &mut Buffer, +) -> Result>, DecodeError> { if handle_null_marker(buf) { return Ok(Vec::default()); } - let map_len = rmp::decode::read_map_len(buf).map_err(|_| { + let map_len = rmp::decode::read_map_len(buf.as_mut_slice()).map_err(|_| { DecodeError::InvalidType( "Unable to get map len for event attributes array_value object".to_owned(), ) @@ -187,20 +192,20 @@ fn read_attributes_array<'a>( )); } - let key = read_string_ref(buf)?; - if key != "values" { + let key = buf.read_string()?; + if key.borrow() != "values" { return Err(DecodeError::InvalidFormat( "Expected 'values' field in event attributes array_value object".to_owned(), )); } - let len = rmp::decode::read_array_len(buf).map_err(|_| { + let len = rmp::decode::read_array_len(buf.as_mut_slice()).map_err(|_| { DecodeError::InvalidType( "Unable to get array len for event attributes values field".to_owned(), ) })?; - let mut vec: Vec = Vec::with_capacity(len as usize); + let mut vec: Vec> = Vec::with_capacity(len as usize); if len > 0 { let first = decode_attribute_array(buf, None)?; let array_type = (&first).into(); @@ -238,41 +243,35 @@ impl FromStr for AttributeArrayKey { } } -fn get_attribute_from_key<'a>( - buf: &mut &'a [u8], +fn get_attribute_from_key( + buf: &mut Buffer, key: AttributeArrayKey, -) -> Result, DecodeError> { +) -> Result, DecodeError> { match key { - AttributeArrayKey::StringValue => { - Ok(AttributeArrayValueSlice::String(read_string_ref(buf)?)) - } + AttributeArrayKey::StringValue => Ok(AttributeArrayValue::String(buf.read_string()?)), AttributeArrayKey::BoolValue => { - let boolean = rmp::decode::read_bool(buf); + let boolean = rmp::decode::read_bool(buf.as_mut_slice()); if let Ok(value) = boolean { match value { - true => Ok(AttributeArrayValueSlice::Boolean(true)), - false => Ok(AttributeArrayValueSlice::Boolean(false)), + true => Ok(AttributeArrayValue::Boolean(true)), + false => Ok(AttributeArrayValue::Boolean(false)), } } else { Err(DecodeError::InvalidType("Invalid boolean field".to_owned())) } } - AttributeArrayKey::IntValue => { - Ok(AttributeArrayValueSlice::Integer(read_number_slice(buf)?)) - } - AttributeArrayKey::DoubleValue => { - Ok(AttributeArrayValueSlice::Double(read_number_slice(buf)?)) - } + AttributeArrayKey::IntValue => Ok(AttributeArrayValue::Integer(read_number(buf)?)), + AttributeArrayKey::DoubleValue => Ok(AttributeArrayValue::Double(read_number(buf)?)), _ => Err(DecodeError::InvalidFormat("Invalid attribute".to_owned())), } } -fn decode_attribute_array<'a>( - buf: &mut &'a [u8], +fn decode_attribute_array( + buf: &mut Buffer, array_type: Option, -) -> Result, DecodeError> { - let mut attribute: Option = None; - let attribute_size = rmp::decode::read_map_len(buf).map_err(|_| { +) -> Result, DecodeError> { + let mut attribute: Option> = None; + let attribute_size = rmp::decode::read_map_len(buf.as_mut_slice()).map_err(|_| { DecodeError::InvalidType("Unable to get map len for attribute size".to_owned()) })?; @@ -284,8 +283,8 @@ fn decode_attribute_array<'a>( let mut attribute_type: Option = None; for _ in 0..attribute_size { - match read_string_ref(buf)?.parse::()? { - AttributeArrayKey::Type => attribute_type = Some(read_number_slice(buf)?), + match buf.read_string()?.borrow().parse::()? { + AttributeArrayKey::Type => attribute_type = Some(read_number(buf)?), key => attribute = Some(get_attribute_from_key(buf, key)?), } } diff --git a/datadog-trace-utils/src/msgpack_decoder/decode/span_link.rs b/datadog-trace-utils/src/msgpack_decoder/decode/span_link.rs index acb3efb15d..fffacbe3ac 100644 --- a/datadog-trace-utils/src/msgpack_decoder/decode/span_link.rs +++ b/datadog-trace-utils/src/msgpack_decoder/decode/span_link.rs @@ -1,12 +1,13 @@ // Copyright 2024-Present Datadog, Inc. https://www.datadoghq.com/ // SPDX-License-Identifier: Apache-2.0 +use crate::msgpack_decoder::decode::buffer::Buffer; use crate::msgpack_decoder::decode::error::DecodeError; -use crate::msgpack_decoder::decode::number::read_number_slice; -use crate::msgpack_decoder::decode::string::{ - handle_null_marker, read_str_map_to_strings, read_string_ref, -}; -use crate::span::SpanLinkSlice; +use crate::msgpack_decoder::decode::number::read_number; +use crate::msgpack_decoder::decode::string::{handle_null_marker, read_str_map_to_strings}; +use crate::span::v04::SpanLink; +use crate::span::TraceData; +use std::borrow::Borrow; use std::str::FromStr; /// Reads a slice of bytes and decodes it into a vector of `SpanLink` objects. @@ -26,18 +27,18 @@ use std::str::FromStr; /// - The marker for the array length cannot be read. /// - Any `SpanLink` cannot be decoded. /// ``` -pub(crate) fn read_span_links<'a>( - buf: &mut &'a [u8], -) -> Result>, DecodeError> { +pub(crate) fn read_span_links( + buf: &mut Buffer, +) -> Result>, DecodeError> { if handle_null_marker(buf) { return Ok(Vec::default()); } - let len = rmp::decode::read_array_len(buf).map_err(|_| { + let len = rmp::decode::read_array_len(buf.as_mut_slice()).map_err(|_| { DecodeError::InvalidType("Unable to get array len for span links".to_owned()) })?; - let mut vec: Vec = Vec::with_capacity(len as usize); + let mut vec: Vec> = Vec::with_capacity(len as usize); for _ in 0..len { vec.push(decode_span_link(buf)?); } @@ -71,19 +72,19 @@ impl FromStr for SpanLinkKey { } } -fn decode_span_link<'a>(buf: &mut &'a [u8]) -> Result, DecodeError> { - let mut span = SpanLinkSlice::default(); - let span_size = rmp::decode::read_map_len(buf) +fn decode_span_link(buf: &mut Buffer) -> Result, DecodeError> { + let mut span = SpanLink::default(); + let span_size = rmp::decode::read_map_len(buf.as_mut_slice()) .map_err(|_| DecodeError::InvalidType("Unable to get map len for span size".to_owned()))?; for _ in 0..span_size { - match read_string_ref(buf)?.parse::()? { - SpanLinkKey::TraceId => span.trace_id = read_number_slice(buf)?, - SpanLinkKey::TraceIdHigh => span.trace_id_high = read_number_slice(buf)?, - SpanLinkKey::SpanId => span.span_id = read_number_slice(buf)?, + match buf.read_string()?.borrow().parse::()? { + SpanLinkKey::TraceId => span.trace_id = read_number(buf)?, + SpanLinkKey::TraceIdHigh => span.trace_id_high = read_number(buf)?, + SpanLinkKey::SpanId => span.span_id = read_number(buf)?, SpanLinkKey::Attributes => span.attributes = read_str_map_to_strings(buf)?, - SpanLinkKey::Tracestate => span.tracestate = read_string_ref(buf)?, - SpanLinkKey::Flags => span.flags = read_number_slice(buf)?, + SpanLinkKey::Tracestate => span.tracestate = buf.read_string()?, + SpanLinkKey::Flags => span.flags = read_number(buf)?, } } diff --git a/datadog-trace-utils/src/msgpack_decoder/decode/string.rs b/datadog-trace-utils/src/msgpack_decoder/decode/string.rs index 215fdd941a..475dd7ae88 100644 --- a/datadog-trace-utils/src/msgpack_decoder/decode/string.rs +++ b/datadog-trace-utils/src/msgpack_decoder/decode/string.rs @@ -1,53 +1,25 @@ // Copyright 2024-Present Datadog, Inc. https://www.datadoghq.com/ // SPDX-License-Identifier: Apache-2.0 +use crate::msgpack_decoder::decode::buffer::Buffer; use crate::msgpack_decoder::decode::error::DecodeError; +use crate::span::TraceData; use rmp::decode; -use rmp::decode::DecodeStringError; use std::collections::HashMap; // https://docs.rs/rmp/latest/rmp/enum.Marker.html#variant.Null (0xc0 == 192) const NULL_MARKER: &u8 = &0xc0; -/// Read a string from `buf`. -/// -/// # Errors -/// Fails if the buffer doesn't contain a valid utf8 msgpack string. -#[inline] -pub fn read_string_ref_nomut(buf: &[u8]) -> Result<(&str, &[u8]), DecodeError> { - decode::read_str_from_slice(buf).map_err(|e| match e { - DecodeStringError::InvalidMarkerRead(e) => DecodeError::InvalidFormat(e.to_string()), - DecodeStringError::InvalidDataRead(e) => DecodeError::InvalidConversion(e.to_string()), - DecodeStringError::TypeMismatch(marker) => { - DecodeError::InvalidType(format!("Type mismatch at marker {marker:?}")) - } - DecodeStringError::InvalidUtf8(_, e) => DecodeError::Utf8Error(e.to_string()), - _ => DecodeError::IOError, - }) -} - -/// Read a string from the slices `buf`. -/// -/// # Errors -/// Fails if the buffer doesn't contain a valid utf8 msgpack string. -#[inline] -pub fn read_string_ref<'a>(buf: &mut &'a [u8]) -> Result<&'a str, DecodeError> { - read_string_ref_nomut(buf).map(|(str, newbuf)| { - *buf = newbuf; - str - }) -} - /// Read a nullable string from the slices `buf`. /// /// # Errors /// Fails if the buffer doesn't contain a valid utf8 msgpack string or a null marker. #[inline] -pub fn read_nullable_string<'a>(buf: &mut &'a [u8]) -> Result<&'a str, DecodeError> { +pub fn read_nullable_string(buf: &mut Buffer) -> Result { if handle_null_marker(buf) { - Ok("") + Ok(T::Text::default()) } else { - read_string_ref(buf) + buf.read_string() } } @@ -57,17 +29,17 @@ pub fn read_nullable_string<'a>(buf: &mut &'a [u8]) -> Result<&'a str, DecodeErr /// Fails if the buffer does not contain a valid map length prefix, /// or if any key or value is not a valid utf8 msgpack string. #[inline] -pub fn read_str_map_to_strings<'a>( - buf: &mut &'a [u8], -) -> Result, DecodeError> { - let len = decode::read_map_len(buf) +pub fn read_str_map_to_strings( + buf: &mut Buffer, +) -> Result, DecodeError> { + let len = decode::read_map_len(buf.as_mut_slice()) .map_err(|_| DecodeError::InvalidFormat("Unable to get map len for str map".to_owned()))?; #[allow(clippy::expect_used)] let mut map = HashMap::with_capacity(len.try_into().expect("Unable to cast map len to usize")); for _ in 0..len { - let key = read_string_ref(buf)?; - let value = read_string_ref(buf)?; + let key = buf.read_string()?; + let value = buf.read_string()?; map.insert(key, value); } Ok(map) @@ -79,9 +51,9 @@ pub fn read_str_map_to_strings<'a>( /// Fails if the buffer does not contain a valid map length prefix, /// or if any key or value is not a valid utf8 msgpack string. #[inline] -pub fn read_nullable_str_map_to_strings<'a>( - buf: &mut &'a [u8], -) -> Result, DecodeError> { +pub fn read_nullable_str_map_to_strings( + buf: &mut Buffer, +) -> Result, DecodeError> { if handle_null_marker(buf) { return Ok(HashMap::default()); } @@ -95,9 +67,10 @@ pub fn read_nullable_str_map_to_strings<'a>( /// # Returns /// A boolean indicating whether the next value is null or not. #[inline] -pub fn handle_null_marker(buf: &mut &[u8]) -> bool { - if buf.first() == Some(NULL_MARKER) { - *buf = &buf[1..]; +pub fn handle_null_marker(buf: &mut Buffer) -> bool { + let slice = buf.as_mut_slice(); + if slice.first() == Some(NULL_MARKER) { + *slice = &slice[1..]; true } else { false diff --git a/datadog-trace-utils/src/msgpack_decoder/v04/mod.rs b/datadog-trace-utils/src/msgpack_decoder/v04/mod.rs index 09d4541dde..c9e9630b1e 100644 --- a/datadog-trace-utils/src/msgpack_decoder/v04/mod.rs +++ b/datadog-trace-utils/src/msgpack_decoder/v04/mod.rs @@ -4,8 +4,10 @@ pub(crate) mod span; use self::span::decode_span; +use crate::msgpack_decoder::decode::buffer::Buffer; use crate::msgpack_decoder::decode::error::DecodeError; -use crate::span::{SpanBytes, SpanSlice}; +use crate::span::v04::{Span, SpanBytes, SpanSlice}; +use crate::span::TraceData; /// Decodes a Bytes buffer into a `Vec>` object, also represented as a vector of /// `TracerPayloadV04` objects. @@ -50,21 +52,7 @@ use crate::span::{SpanBytes, SpanSlice}; /// assert_eq!("test-span", decoded_span.name.as_str()); /// ``` pub fn from_bytes(data: tinybytes::Bytes) -> Result<(Vec>, usize), DecodeError> { - let mut parsed_data = data.clone(); - let (traces_ref, size) = from_slice(unsafe { parsed_data.as_mut_slice() })?; - - #[allow(clippy::unwrap_used)] - let traces_owned = traces_ref - .iter() - .map(|trace| { - trace - .iter() - // Safe to unwrap since the spans use subslices of the `data` slice - .map(|span| span.try_to_bytes(&data).unwrap()) - .collect() - }) - .collect(); - Ok((traces_owned, size)) + from_buffer(&mut Buffer::new(data)) } /// Decodes a slice of bytes into a `Vec>` object. @@ -109,11 +97,19 @@ pub fn from_bytes(data: tinybytes::Bytes) -> Result<(Vec>, usize) /// let decoded_span = &decoded_traces[0][0]; /// assert_eq!("test-span", decoded_span.name); /// ``` -pub fn from_slice(mut data: &[u8]) -> Result<(Vec>>, usize), DecodeError> { - let trace_count = rmp::decode::read_array_len(&mut data).map_err(|_| { +pub fn from_slice(data: &[u8]) -> Result<(Vec>>, usize), DecodeError> { + from_buffer(&mut Buffer::new(data)) +} + +#[allow(clippy::type_complexity)] +pub fn from_buffer( + data: &mut Buffer, +) -> Result<(Vec>>, usize), DecodeError> { + let trace_count = rmp::decode::read_array_len(data.as_mut_slice()).map_err(|_| { DecodeError::InvalidFormat("Unable to read array len for trace count".to_owned()) })?; + // Intentionally skip the size of the array (as it will be recomputed after coalescing). let start_len = data.len(); #[allow(clippy::expect_used)] @@ -125,9 +121,12 @@ pub fn from_slice(mut data: &[u8]) -> Result<(Vec>>, usize), D .expect("Unable to cast trace_count to usize"), ), |mut traces, _| { - let span_count = rmp::decode::read_array_len(&mut data).map_err(|_| { - DecodeError::InvalidFormat("Unable to read array len for span count".to_owned()) - })?; + let span_count = + rmp::decode::read_array_len(data.as_mut_slice()).map_err(|_| { + DecodeError::InvalidFormat( + "Unable to read array len for span count".to_owned(), + ) + })?; let trace = (0..span_count).try_fold( Vec::with_capacity( @@ -136,7 +135,7 @@ pub fn from_slice(mut data: &[u8]) -> Result<(Vec>>, usize), D .expect("Unable to cast span_count to usize"), ), |mut trace, _| { - let span = decode_span(&mut data)?; + let span = decode_span(data)?; trace.push(span); Ok(trace) }, diff --git a/datadog-trace-utils/src/msgpack_decoder/v04/span.rs b/datadog-trace-utils/src/msgpack_decoder/v04/span.rs index d7f4e1feb6..a68f7eebc1 100644 --- a/datadog-trace-utils/src/msgpack_decoder/v04/span.rs +++ b/datadog-trace-utils/src/msgpack_decoder/v04/span.rs @@ -1,15 +1,17 @@ // Copyright 2024-Present Datadog, Inc. https://www.datadoghq.com/ // SPDX-License-Identifier: Apache-2.0 +use crate::msgpack_decoder::decode::buffer::Buffer; use crate::msgpack_decoder::decode::error::DecodeError; -use crate::msgpack_decoder::decode::number::read_nullable_number_slice; +use crate::msgpack_decoder::decode::number::read_nullable_number; use crate::msgpack_decoder::decode::span_event::read_span_events; use crate::msgpack_decoder::decode::span_link::read_span_links; use crate::msgpack_decoder::decode::string::{ - read_nullable_str_map_to_strings, read_nullable_string, read_string_ref, + read_nullable_str_map_to_strings, read_nullable_string, }; use crate::msgpack_decoder::decode::{meta_struct::read_meta_struct, metrics::read_metrics}; -use crate::span::{SpanKey, SpanSlice}; +use crate::span::{v04::Span, SpanKey, TraceData}; +use std::borrow::Borrow; /// Decodes a slice of bytes into a `Span` object. /// @@ -27,10 +29,10 @@ use crate::span::{SpanKey, SpanSlice}; /// This function will return an error if: /// - The map length cannot be read. /// - Any key or value cannot be decoded. -pub fn decode_span<'a>(buffer: &mut &'a [u8]) -> Result, DecodeError> { - let mut span = SpanSlice::default(); +pub fn decode_span(buffer: &mut Buffer) -> Result, DecodeError> { + let mut span = Span::::default(); - let span_size = rmp::decode::read_map_len(buffer).map_err(|_| { + let span_size = rmp::decode::read_map_len(buffer.as_mut_slice()).map_err(|_| { DecodeError::InvalidFormat("Unable to get map len for span size".to_owned()) })?; @@ -43,8 +45,10 @@ pub fn decode_span<'a>(buffer: &mut &'a [u8]) -> Result, DecodeErr // Safety: read_string_ref checks utf8 validity, so we don't do it again when creating the // BytesStrings -fn fill_span<'a>(span: &mut SpanSlice<'a>, buf: &mut &'a [u8]) -> Result<(), DecodeError> { - let key = read_string_ref(buf)? +fn fill_span(span: &mut Span, buf: &mut Buffer) -> Result<(), DecodeError> { + let key = buf + .read_string()? + .borrow() .parse::() .map_err(|e| DecodeError::InvalidFormat(e.message))?; @@ -52,12 +56,12 @@ fn fill_span<'a>(span: &mut SpanSlice<'a>, buf: &mut &'a [u8]) -> Result<(), Dec SpanKey::Service => span.service = read_nullable_string(buf)?, SpanKey::Name => span.name = read_nullable_string(buf)?, SpanKey::Resource => span.resource = read_nullable_string(buf)?, - SpanKey::TraceId => span.trace_id = read_nullable_number_slice(buf)?, - SpanKey::SpanId => span.span_id = read_nullable_number_slice(buf)?, - SpanKey::ParentId => span.parent_id = read_nullable_number_slice(buf)?, - SpanKey::Start => span.start = read_nullable_number_slice(buf)?, - SpanKey::Duration => span.duration = read_nullable_number_slice(buf)?, - SpanKey::Error => span.error = read_nullable_number_slice(buf)?, + SpanKey::TraceId => span.trace_id = read_nullable_number(buf)?, + SpanKey::SpanId => span.span_id = read_nullable_number(buf)?, + SpanKey::ParentId => span.parent_id = read_nullable_number(buf)?, + SpanKey::Start => span.start = read_nullable_number(buf)?, + SpanKey::Duration => span.duration = read_nullable_number(buf)?, + SpanKey::Error => span.error = read_nullable_number(buf)?, SpanKey::Type => span.r#type = read_nullable_string(buf)?, SpanKey::Meta => span.meta = read_nullable_str_map_to_strings(buf)?, SpanKey::Metrics => span.metrics = read_metrics(buf)?, diff --git a/datadog-trace-utils/src/msgpack_decoder/v05/mod.rs b/datadog-trace-utils/src/msgpack_decoder/v05/mod.rs index f94d74d750..cc89796dcb 100644 --- a/datadog-trace-utils/src/msgpack_decoder/v05/mod.rs +++ b/datadog-trace-utils/src/msgpack_decoder/v05/mod.rs @@ -3,11 +3,10 @@ use crate::msgpack_decoder::decode::error::DecodeError; use crate::msgpack_decoder::decode::{ - map::read_map_len, - number::read_number_slice, - string::{handle_null_marker, read_string_ref}, + buffer::Buffer, map::read_map_len, number::read_number, string::handle_null_marker, }; -use crate::span::{SpanBytes, SpanSlice}; +use crate::span::v04::{Span, SpanBytes, SpanSlice}; +use crate::span::TraceData; use std::collections::HashMap; const PAYLOAD_LEN: u32 = 2; @@ -68,21 +67,7 @@ const SPAN_ELEM_COUNT: u32 = 12; /// assert_eq!("", decoded_span.name.as_str()); /// ``` pub fn from_bytes(data: tinybytes::Bytes) -> Result<(Vec>, usize), DecodeError> { - let mut parsed_data = data.clone(); - let (traces_ref, size) = from_slice(unsafe { parsed_data.as_mut_slice() })?; - - #[allow(clippy::unwrap_used)] - let traces_owned = traces_ref - .iter() - .map(|trace| { - trace - .iter() - // Safe to unwrap since the spans use subslices of the `data` slice - .map(|span| span.try_to_bytes(&data).unwrap()) - .collect() - }) - .collect(); - Ok((traces_owned, size)) + from_buffer(&mut Buffer::new(data)) } /// Decodes a slice of bytes into a `Vec>` object. @@ -139,8 +124,15 @@ pub fn from_bytes(data: tinybytes::Bytes) -> Result<(Vec>, usize) /// let decoded_span = &decoded_traces[0][0]; /// assert_eq!("", decoded_span.name); /// ``` -pub fn from_slice(mut data: &[u8]) -> Result<(Vec>>, usize), DecodeError> { - let data_elem = rmp::decode::read_array_len(&mut data) +pub fn from_slice(data: &[u8]) -> Result<(Vec>>, usize), DecodeError> { + from_buffer(&mut Buffer::new(data)) +} + +#[allow(clippy::type_complexity)] +fn from_buffer( + data: &mut Buffer, +) -> Result<(Vec>>, usize), DecodeError> { + let data_elem = rmp::decode::read_array_len(data.as_mut_slice()) .map_err(|_| DecodeError::InvalidFormat("Unable to read payload len".to_string()))?; if data_elem != PAYLOAD_LEN { @@ -149,21 +141,21 @@ pub fn from_slice(mut data: &[u8]) -> Result<(Vec>>, usize), D )); } - let dict = deserialize_dict(&mut data)?; + let dict = deserialize_dict(data)?; - let trace_count = rmp::decode::read_array_len(&mut data) + let trace_count = rmp::decode::read_array_len(data.as_mut_slice()) .map_err(|_| DecodeError::InvalidFormat("Unable to read trace len".to_string()))?; - let mut traces: Vec> = Vec::with_capacity(trace_count as usize); + let mut traces: Vec>> = Vec::with_capacity(trace_count as usize); let start_len = data.len(); for _ in 0..trace_count { - let span_count = rmp::decode::read_array_len(&mut data) + let span_count = rmp::decode::read_array_len(data.as_mut_slice()) .map_err(|_| DecodeError::InvalidFormat("Unable to read span len".to_string()))?; - let mut trace: Vec = Vec::with_capacity(span_count as usize); + let mut trace: Vec> = Vec::with_capacity(span_count as usize); for _ in 0..span_count { - let span = deserialize_span(&mut data, &dict)?; + let span = deserialize_span(data, &dict)?; trace.push(span); } traces.push(trace); @@ -171,21 +163,24 @@ pub fn from_slice(mut data: &[u8]) -> Result<(Vec>>, usize), D Ok((traces, start_len - data.len())) } -fn deserialize_dict<'a>(data: &mut &'a [u8]) -> Result, DecodeError> { - let dict_len = rmp::decode::read_array_len(data) +fn deserialize_dict(data: &mut Buffer) -> Result, DecodeError> { + let dict_len = rmp::decode::read_array_len(data.as_mut_slice()) .map_err(|_| DecodeError::InvalidFormat("Unable to read dictionary len".to_string()))?; - let mut dict: Vec<&'a str> = Vec::with_capacity(dict_len as usize); + let mut dict: Vec = Vec::with_capacity(dict_len as usize); for _ in 0..dict_len { - let str = read_string_ref(data)?; + let str = data.read_string()?; dict.push(str); } Ok(dict) } -fn deserialize_span<'a>(data: &mut &[u8], dict: &[&'a str]) -> Result, DecodeError> { - let mut span = SpanSlice::default(); - let span_len = rmp::decode::read_array_len(data) +fn deserialize_span( + data: &mut Buffer, + dict: &[T::Text], +) -> Result, DecodeError> { + let mut span = Span::default(); + let span_len = rmp::decode::read_array_len(data.as_mut_slice()) .map_err(|_| DecodeError::InvalidFormat("Unable to read dictionary len".to_string()))?; if span_len != SPAN_ELEM_COUNT { @@ -197,12 +192,12 @@ fn deserialize_span<'a>(data: &mut &[u8], dict: &[&'a str]) -> Result(data: &mut &[u8], dict: &[&'a str]) -> Result(data: &mut &[u8], dict: &[&'a str]) -> Result<&'a str, DecodeError> { - let index: u32 = read_number_slice(data)?; +fn get_from_dict( + data: &mut Buffer, + dict: &[T::Text], +) -> Result { + let index: u32 = read_number(data)?; match dict.get(index as usize) { - Some(value) => Ok(value), + Some(value) => Ok(value.clone()), None => Err(DecodeError::InvalidFormat( "Unable to locate string in the dictionary".to_string(), )), } } -fn read_indexed_map_to_bytes_strings<'a>( - buf: &mut &[u8], - dict: &[&'a str], -) -> Result, DecodeError> { - let len = rmp::decode::read_map_len(buf) +fn read_indexed_map_to_bytes_strings( + buf: &mut Buffer, + dict: &[T::Text], +) -> Result, DecodeError> { + let len = rmp::decode::read_map_len(buf.as_mut_slice()) .map_err(|_| DecodeError::InvalidFormat("Unable to get map len for str map".to_owned()))?; #[allow(clippy::expect_used)] @@ -237,10 +235,10 @@ fn read_indexed_map_to_bytes_strings<'a>( Ok(map) } -fn read_metrics<'a>( - buf: &mut &[u8], - dict: &[&'a str], -) -> Result, DecodeError> { +fn read_metrics( + buf: &mut Buffer, + dict: &[T::Text], +) -> Result, DecodeError> { if handle_null_marker(buf) { return Ok(HashMap::default()); } @@ -250,7 +248,7 @@ fn read_metrics<'a>( let mut map = HashMap::with_capacity(len); for _ in 0..len { let k = get_from_dict(buf, dict)?; - let v = read_number_slice(buf)?; + let v = read_number(buf)?; map.insert(k, v); } Ok(map) @@ -259,7 +257,9 @@ fn read_metrics<'a>( #[cfg(test)] mod tests { use super::*; + use crate::span::SliceData; use std::collections::HashMap; + type V05Span = ( u8, u8, @@ -296,7 +296,7 @@ mod tests { fn deserialize_dict_test() { let dict = vec!["foo", "bar", "baz"]; let mpack = rmp_serde::to_vec(&dict).unwrap(); - let mut payload = mpack.as_ref(); + let mut payload = Buffer::::new(mpack.as_ref()); let result = deserialize_dict(&mut payload).unwrap(); assert_eq!(dict, result); diff --git a/datadog-trace-utils/src/msgpack_encoder/v04/mod.rs b/datadog-trace-utils/src/msgpack_encoder/v04/mod.rs index 69f576d8a8..81a723d049 100644 --- a/datadog-trace-utils/src/msgpack_encoder/v04/mod.rs +++ b/datadog-trace-utils/src/msgpack_encoder/v04/mod.rs @@ -1,13 +1,14 @@ // Copyright 2021-Present Datadog, Inc. https://www.datadoghq.com/ // SPDX-License-Identifier: Apache-2.0 -use crate::span::{Span, SpanText}; +use crate::span::v04::Span; +use crate::span::TraceData; use rmp::encode::{write_array_len, ByteBuf, RmpWrite, ValueWriteError}; mod span; #[inline(always)] -fn to_writer]>>( +fn to_writer]>>( writer: &mut W, traces: &[S], ) -> Result<(), ValueWriteError> { @@ -44,10 +45,10 @@ fn to_writer]>>( /// /// ``` /// use datadog_trace_utils::msgpack_encoder::v04::write_to_slice; -/// use datadog_trace_utils::span::Span; +/// use datadog_trace_utils::span::v04::SpanSlice; /// /// let mut buffer = vec![0u8; 1024]; -/// let span = Span { +/// let span = SpanSlice { /// name: "test-span", /// ..Default::default() /// }; @@ -55,7 +56,7 @@ fn to_writer]>>( /// /// write_to_slice(&mut &mut buffer[..], &traces).expect("Encoding failed"); /// ``` -pub fn write_to_slice]>>( +pub fn write_to_slice]>>( slice: &mut &mut [u8], traces: &[S], ) -> Result<(), ValueWriteError> { @@ -76,9 +77,9 @@ pub fn write_to_slice]>>( /// /// ``` /// use datadog_trace_utils::msgpack_encoder::v04::to_vec; -/// use datadog_trace_utils::span::Span; +/// use datadog_trace_utils::span::v04::SpanSlice; /// -/// let span = Span { +/// let span = SpanSlice { /// name: "test-span", /// ..Default::default() /// }; @@ -87,7 +88,7 @@ pub fn write_to_slice]>>( /// /// assert!(!encoded.is_empty()); /// ``` -pub fn to_vec]>>(traces: &[S]) -> Vec { +pub fn to_vec]>>(traces: &[S]) -> Vec { to_vec_with_capacity(traces, 0) } @@ -106,9 +107,9 @@ pub fn to_vec]>>(traces: &[S]) -> Vec { /// /// ``` /// use datadog_trace_utils::msgpack_encoder::v04::to_vec_with_capacity; -/// use datadog_trace_utils::span::Span; +/// use datadog_trace_utils::span::v04::SpanSlice; /// -/// let span = Span { +/// let span = SpanSlice { /// name: "test-span", /// ..Default::default() /// }; @@ -117,7 +118,7 @@ pub fn to_vec]>>(traces: &[S]) -> Vec { /// /// assert!(encoded.capacity() >= 1024); /// ``` -pub fn to_vec_with_capacity]>>( +pub fn to_vec_with_capacity]>>( traces: &[S], capacity: u32, ) -> Vec { @@ -165,9 +166,9 @@ impl std::io::Write for CountLength { /// /// ``` /// use datadog_trace_utils::msgpack_encoder::v04::to_len; -/// use datadog_trace_utils::span::Span; +/// use datadog_trace_utils::span::v04::SpanSlice; /// -/// let span = Span { +/// let span = SpanSlice { /// name: "test-span", /// ..Default::default() /// }; @@ -176,7 +177,7 @@ impl std::io::Write for CountLength { /// /// assert!(encoded_len > 0); /// ``` -pub fn to_len]>>(traces: &[S]) -> u32 { +pub fn to_len]>>(traces: &[S]) -> u32 { let mut counter = CountLength(0); #[allow(clippy::expect_used)] to_writer(&mut counter, traces).expect("infallible: CountLength never fails"); diff --git a/datadog-trace-utils/src/msgpack_encoder/v04/span.rs b/datadog-trace-utils/src/msgpack_encoder/v04/span.rs index 0a0be27d2c..6ceb10c03f 100644 --- a/datadog-trace-utils/src/msgpack_encoder/v04/span.rs +++ b/datadog-trace-utils/src/msgpack_encoder/v04/span.rs @@ -1,11 +1,13 @@ // Copyright 2021-Present Datadog, Inc. https://www.datadoghq.com/ // SPDX-License-Identifier: Apache-2.0 -use crate::span::{AttributeAnyValue, AttributeArrayValue, Span, SpanEvent, SpanLink, SpanText}; +use crate::span::v04::{AttributeAnyValue, AttributeArrayValue, Span, SpanEvent, SpanLink}; +use crate::span::TraceData; use rmp::encode::{ write_bin, write_bool, write_f64, write_i64, write_sint, write_str, write_u32, write_u64, write_u8, RmpWrite, ValueWriteError, }; +use std::borrow::Borrow; /// Encodes a `SpanLink` object into a slice of bytes. /// @@ -15,13 +17,13 @@ use rmp::encode::{ /// /// # Returns /// -/// * `Ok(()))` - Nothing if successful. +/// * `Ok(())` - Nothing if successful. /// * `Err(ValueWriteError)` - An error if the writing fails. /// /// # Errors /// /// This function will return any error emitted by the writer. -pub fn encode_span_links( +pub fn encode_span_links( writer: &mut W, span_links: &[SpanLink], ) -> Result<(), ValueWriteError> { @@ -82,7 +84,7 @@ pub fn encode_span_links( /// # Errors /// /// This function will return any error emitted by the writer. -pub fn encode_span_events( +pub fn encode_span_events( writer: &mut W, span_events: &[SpanEvent], ) -> Result<(), ValueWriteError> { @@ -106,7 +108,7 @@ pub fn encode_span_events( for (k, attribute) in event.attributes.iter() { write_str(writer, k.borrow())?; - fn write_array_value( + fn write_array_value( writer: &mut W, value: &AttributeArrayValue, ) -> Result<(), ValueWriteError> { @@ -181,7 +183,7 @@ pub fn encode_span_events( /// /// This function will return any error emitted by the writer. #[inline(always)] -pub fn encode_span( +pub fn encode_span( writer: &mut W, span: &Span, ) -> Result<(), ValueWriteError> { @@ -256,7 +258,7 @@ pub fn encode_span( rmp::encode::write_map_len(writer, span.meta_struct.len() as u32)?; for (k, v) in span.meta_struct.iter() { write_str(writer, k.borrow())?; - write_bin(writer, v.as_ref())?; + write_bin(writer, v.borrow())?; } } diff --git a/datadog-trace-utils/src/send_data/mod.rs b/datadog-trace-utils/src/send_data/mod.rs index 82147a25b4..583a7d923e 100644 --- a/datadog-trace-utils/src/send_data/mod.rs +++ b/datadog-trace-utils/src/send_data/mod.rs @@ -26,7 +26,7 @@ use std::io::Write; #[cfg(feature = "compression")] use zstd::stream::write::Encoder; -#[derive(Debug, Clone)] +#[derive(Debug)] /// `SendData` is a structure that holds the data to be sent to a target endpoint. /// It includes the payloads to be sent, the size of the data, the target endpoint, /// headers for the request, and a retry strategy for sending the data. @@ -79,7 +79,6 @@ pub enum Compression { None, } -#[derive(Clone)] pub struct SendDataBuilder { pub(crate) tracer_payloads: TracerPayloadCollection, pub(crate) size: usize, @@ -217,25 +216,13 @@ impl SendData { self.retry_strategy = retry_strategy; } - /// Returns a clone of the SendData with the user-defined endpoint. - /// - /// # Arguments - /// - /// * `endpoint`: The new endpoint to be used. - pub fn with_endpoint(&self, endpoint: Endpoint) -> SendData { - SendData { - target: endpoint, - ..self.clone() - } - } - /// Sends the data to the target endpoint. /// /// # Returns /// /// A `SendDataResult` instance containing the result of the operation. pub async fn send(&self) -> SendDataResult { - self.send_internal(None).await + self.send_internal(None, None).await } /// Sends the data to the target endpoint. @@ -243,15 +230,23 @@ impl SendData { /// # Returns /// /// A `SendDataResult` instance containing the result of the operation. - pub async fn send_proxy(&self, http_proxy: Option<&str>) -> SendDataResult { - self.send_internal(http_proxy).await + pub async fn send_proxy( + &self, + http_proxy: Option<&str>, + endpoint: Option, + ) -> SendDataResult { + self.send_internal(http_proxy, endpoint).await } - async fn send_internal(&self, http_proxy: Option<&str>) -> SendDataResult { + async fn send_internal( + &self, + http_proxy: Option<&str>, + endpoint: Option, + ) -> SendDataResult { if self.use_protobuf() { - self.send_with_protobuf(http_proxy).await + self.send_with_protobuf(http_proxy, endpoint).await } else { - self.send_with_msgpack(http_proxy).await + self.send_with_msgpack(http_proxy, endpoint).await } } @@ -261,12 +256,13 @@ impl SendData { payload: Vec, headers: HashMap<&'static str, String>, http_proxy: Option<&str>, + endpoint: Option<&Endpoint>, ) -> (SendWithRetryResult, u64, u64) { #[allow(clippy::unwrap_used)] let payload_len = u64::try_from(payload.len()).unwrap(); ( send_with_retry( - &self.target, + endpoint.unwrap_or(&self.target), payload, &headers, &self.retry_strategy, @@ -304,7 +300,11 @@ impl SendData { } } - async fn send_with_protobuf(&self, http_proxy: Option<&str>) -> SendDataResult { + async fn send_with_protobuf( + &self, + http_proxy: Option<&str>, + endpoint: Option, + ) -> SendDataResult { let mut result = SendDataResult::default(); #[allow(clippy::unwrap_used)] @@ -331,7 +331,13 @@ impl SendData { request_headers.insert(CONTENT_TYPE.as_str(), APPLICATION_PROTOBUF_STR.to_string()); let (response, bytes_sent, chunks) = self - .send_payload(chunks, final_payload, request_headers, http_proxy) + .send_payload( + chunks, + final_payload, + request_headers, + http_proxy, + endpoint.as_ref(), + ) .await; result.update(response, bytes_sent, chunks); @@ -342,7 +348,11 @@ impl SendData { } } - async fn send_with_msgpack(&self, http_proxy: Option<&str>) -> SendDataResult { + async fn send_with_msgpack( + &self, + http_proxy: Option<&str>, + endpoint: Option, + ) -> SendDataResult { let mut result = SendDataResult::default(); let mut futures = FuturesUnordered::new(); @@ -360,7 +370,13 @@ impl SendData { Err(e) => return result.error(anyhow!(e)), }; - futures.push(self.send_payload(chunks, payload, headers, http_proxy)); + futures.push(self.send_payload( + chunks, + payload, + headers, + http_proxy, + endpoint.as_ref(), + )); } } TracerPayloadCollection::V04(payload) => { @@ -372,7 +388,13 @@ impl SendData { let payload = msgpack_encoder::v04::to_vec(payload); - futures.push(self.send_payload(chunks, payload, headers, http_proxy)); + futures.push(self.send_payload( + chunks, + payload, + headers, + http_proxy, + endpoint.as_ref(), + )); } TracerPayloadCollection::V05(payload) => { #[allow(clippy::unwrap_used)] @@ -386,7 +408,13 @@ impl SendData { Err(e) => return result.error(anyhow!(e)), }; - futures.push(self.send_payload(chunks, payload, headers, http_proxy)); + futures.push(self.send_payload( + chunks, + payload, + headers, + http_proxy, + endpoint.as_ref(), + )); } } @@ -993,56 +1021,6 @@ mod tests { assert_eq!(res.responses_count_per_code.len(), 0); } - #[test] - fn test_with_endpoint() { - let header_tags = HEADER_TAGS; - let payload = setup_payload(&header_tags); - let original_endpoint = Endpoint { - api_key: Some(std::borrow::Cow::Borrowed("original-key")), - url: "http://originalexample.com/".parse::().unwrap(), - timeout_ms: 1000, - ..Endpoint::default() - }; - - let original_data = SendData::new( - 100, - TracerPayloadCollection::V07(vec![payload]), - header_tags, - &original_endpoint, - ); - - let new_endpoint = Endpoint { - api_key: Some(std::borrow::Cow::Borrowed("new-key")), - url: "http://newexample.com/".parse::().unwrap(), - timeout_ms: 2000, - ..Endpoint::default() - }; - - let new_data = original_data.with_endpoint(new_endpoint.clone()); - - assert_eq!(new_data.target.api_key, new_endpoint.api_key); - assert_eq!(new_data.target.url, new_endpoint.url); - assert_eq!(new_data.target.timeout_ms, new_endpoint.timeout_ms); - - assert_eq!(new_data.size, original_data.size); - assert_eq!(new_data.headers, original_data.headers); - assert_eq!(new_data.retry_strategy, original_data.retry_strategy); - assert_eq!( - new_data.tracer_payloads.size(), - original_data.tracer_payloads.size() - ); - - assert_eq!(original_data.target.api_key, original_endpoint.api_key); - assert_eq!(original_data.target.url, original_endpoint.url); - assert_eq!( - original_data.target.timeout_ms, - original_endpoint.timeout_ms - ); - - #[cfg(feature = "compression")] - assert!(matches!(new_data.compression, Compression::None)); - } - #[test] fn test_builder() { let header_tags = HEADER_TAGS; diff --git a/datadog-trace-utils/src/span/mod.rs b/datadog-trace-utils/src/span/mod.rs index 62057b2aae..76aa518e9e 100644 --- a/datadog-trace-utils/src/span/mod.rs +++ b/datadog-trace-utils/src/span/mod.rs @@ -2,19 +2,20 @@ // SPDX-License-Identifier: Apache-2.0 pub mod trace_utils; +pub mod v04; pub mod v05; -use serde::ser::SerializeStruct; +use crate::msgpack_decoder::decode::buffer::read_string_ref_nomut; +use crate::msgpack_decoder::decode::error::DecodeError; +use crate::span::v05::dict::SharedDict; use serde::Serialize; use std::borrow::Borrow; -use std::collections::HashMap; use std::fmt; +use std::fmt::Debug; use std::hash::Hash; +use std::marker::PhantomData; use std::str::FromStr; use tinybytes::{Bytes, BytesString}; -use v05::dict::SharedDict; - -use crate::tracer_payload::TraceChunks; #[derive(Debug, PartialEq)] pub enum SpanKey { @@ -63,7 +64,7 @@ impl FromStr for SpanKey { /// Trait representing the requirements for a type to be used as a Span "string" type. /// Note: Borrow is not required by the derived traits, but allows to access HashMap elements /// from a static str and check if the string is empty. -pub trait SpanText: Eq + Hash + Borrow + Serialize + Default + Clone { +pub trait SpanText: Debug + Eq + Hash + Borrow + Serialize + Default + Clone { fn from_static_str(value: &'static str) -> Self; } @@ -79,340 +80,94 @@ impl SpanText for BytesString { } } -/// Checks if the `value` represents an empty string. Used to skip serializing empty strings -/// with serde. -fn is_empty_str>(value: &T) -> bool { - value.borrow().is_empty() -} - -/// The generic representation of a V04 span. -/// -/// `T` is the type used to represent strings in the span, it can be either owned (e.g. BytesString) -/// or borrowed (e.g. &str). To define a generic function taking any `Span` you can use the -/// [`SpanValue`] trait: -/// ``` -/// use datadog_trace_utils::span::{Span, SpanText}; -/// fn foo(span: Span) { -/// let _ = span.meta.get("foo"); -/// } -/// ``` -#[derive(Clone, Debug, Default, PartialEq, Serialize)] -pub struct Span -where - T: SpanText, -{ - pub service: T, - pub name: T, - pub resource: T, - #[serde(skip_serializing_if = "is_empty_str")] - pub r#type: T, - pub trace_id: u64, - pub span_id: u64, - #[serde(skip_serializing_if = "is_default")] - pub parent_id: u64, - pub start: i64, - pub duration: i64, - #[serde(skip_serializing_if = "is_default")] - pub error: i32, - #[serde(skip_serializing_if = "HashMap::is_empty")] - pub meta: HashMap, - #[serde(skip_serializing_if = "HashMap::is_empty")] - pub metrics: HashMap, - // TODO: APMSP-1941 - Replace `Bytes` with a wrapper that borrows the underlying - // slice and serializes to bytes in MessagePack. - #[serde(skip_serializing_if = "HashMap::is_empty")] - pub meta_struct: HashMap, - #[serde(skip_serializing_if = "Vec::is_empty")] - pub span_links: Vec>, - #[serde(skip_serializing_if = "Vec::is_empty")] - pub span_events: Vec>, +pub trait SpanBytes: Debug + Eq + Hash + Borrow<[u8]> + Serialize + Default + Clone { + fn from_static_bytes(value: &'static [u8]) -> Self; } -/// The generic representation of a V04 span link. -/// `T` is the type used to represent strings in the span link. -#[derive(Clone, Debug, Default, PartialEq, Serialize)] -pub struct SpanLink -where - T: SpanText, -{ - pub trace_id: u64, - pub trace_id_high: u64, - pub span_id: u64, - #[serde(skip_serializing_if = "HashMap::is_empty")] - pub attributes: HashMap, - #[serde(skip_serializing_if = "is_empty_str")] - pub tracestate: T, - #[serde(skip_serializing_if = "is_default")] - pub flags: u32, -} - -/// The generic representation of a V04 span event. -/// `T` is the type used to represent strings in the span event. -#[derive(Clone, Debug, Default, PartialEq, Serialize)] -pub struct SpanEvent -where - T: SpanText, -{ - pub time_unix_nano: u64, - pub name: T, - #[serde(skip_serializing_if = "HashMap::is_empty")] - pub attributes: HashMap>, +impl SpanBytes for &[u8] { + fn from_static_bytes(value: &'static [u8]) -> Self { + value + } } -#[derive(Clone, Debug, PartialEq)] -pub enum AttributeAnyValue -where - T: SpanText, -{ - SingleValue(AttributeArrayValue), - Array(Vec>), +impl SpanBytes for Bytes { + fn from_static_bytes(value: &'static [u8]) -> Self { + Bytes::from_static(value) + } } -#[derive(Serialize)] -struct ArrayValueWrapper<'a, T: SpanText> { - values: &'a Vec>, -} +/// Trait representing a tuple of (Text, Bytes) types used for different underlying data structures. +/// Note: The functions are internal to the msgpack decoder and should not be used directly: they're +/// only exposed here due to the inavailability of min_specialization in stable Rust. +pub trait TraceData: Default + Debug + Clone + PartialEq + Serialize { + type Text: SpanText; + type Bytes: SpanBytes; -impl Serialize for AttributeAnyValue -where - T: SpanText, -{ - fn serialize(&self, serializer: S) -> Result - where - S: serde::Serializer, - { - let mut state = serializer.serialize_struct("AttributeAnyValue", 2)?; + fn get_mut_slice(buf: &mut Self::Bytes) -> &mut &'static [u8]; - match self { - AttributeAnyValue::SingleValue(attribute) => { - serialize_attribute_array::(&mut state, attribute)?; - } - AttributeAnyValue::Array(value) => { - let value_type: u8 = self.into(); - state.serialize_field("type", &value_type)?; - let wrapped_value = ArrayValueWrapper { values: value }; - state.serialize_field("array_value", &wrapped_value)?; - } - } + fn try_slice_and_advance(buf: &mut Self::Bytes, bytes: usize) -> Option; - state.end() - } + fn read_string(buf: &mut Self::Bytes) -> Result; } -impl From<&AttributeAnyValue> for u8 -where - T: SpanText, -{ - fn from(attribute: &AttributeAnyValue) -> u8 { - match attribute { - AttributeAnyValue::SingleValue(value) => value.into(), - AttributeAnyValue::Array(_) => 4, - } - } -} - -#[derive(Clone, Debug, PartialEq)] -pub enum AttributeArrayValue -where - T: SpanText, -{ - String(T), - Boolean(bool), - Integer(i64), - Double(f64), -} - -impl Serialize for AttributeArrayValue -where - T: SpanText, -{ - fn serialize(&self, serializer: S) -> Result - where - S: serde::Serializer, - { - let mut state = serializer.serialize_struct("AttributeArrayValue", 2)?; - serialize_attribute_array::(&mut state, self)?; - state.end() - } -} +/// TraceData implementation using `Bytes` and `BytesString`. +#[derive(Default, Debug, Clone, PartialEq, Serialize)] +pub struct TinyData; +impl TraceData for TinyData { + type Text = BytesString; + type Bytes = Bytes; -fn serialize_attribute_array( - state: &mut S::SerializeStruct, - attribute: &AttributeArrayValue, -) -> Result<(), ::Error> -where - T: SpanText, - S: serde::Serializer, -{ - let attribute_type: u8 = attribute.into(); - state.serialize_field("type", &attribute_type)?; - match attribute { - AttributeArrayValue::String(value) => state.serialize_field("string_value", value), - AttributeArrayValue::Boolean(value) => state.serialize_field("bool_value", value), - AttributeArrayValue::Integer(value) => state.serialize_field("int_value", value), - AttributeArrayValue::Double(value) => state.serialize_field("double_value", value), + #[inline] + fn get_mut_slice(buf: &mut Bytes) -> &mut &'static [u8] { + unsafe { buf.as_mut_slice() } } -} -impl From<&AttributeArrayValue> for u8 -where - T: SpanText, -{ - fn from(attribute: &AttributeArrayValue) -> u8 { - match attribute { - AttributeArrayValue::String(_) => 0, - AttributeArrayValue::Boolean(_) => 1, - AttributeArrayValue::Integer(_) => 2, - AttributeArrayValue::Double(_) => 3, + #[inline] + fn try_slice_and_advance(buf: &mut Bytes, bytes: usize) -> Option { + let data = buf.slice_ref(&buf[0..bytes])?; + unsafe { + // SAFETY: forwarding the buffer requires that buf is borrowed from static. + *buf.as_mut_slice() = &buf.as_mut_slice()[bytes..]; } + Some(data) } -} - -pub type SpanBytes = Span; -pub type SpanLinkBytes = SpanLink; -pub type SpanEventBytes = SpanEvent; -pub type AttributeAnyValueBytes = AttributeAnyValue; -pub type AttributeArrayValueBytes = AttributeArrayValue; - -pub type SpanSlice<'a> = Span<&'a str>; -pub type SpanLinkSlice<'a> = SpanLink<&'a str>; -pub type SpanEventSlice<'a> = SpanEvent<&'a str>; -pub type AttributeAnyValueSlice<'a> = AttributeAnyValue<&'a str>; -pub type AttributeArrayValueSlice<'a> = AttributeArrayValue<&'a str>; - -pub type TraceChunksBytes = TraceChunks; - -pub type SharedDictBytes = SharedDict; -impl SpanSlice<'_> { - /// Converts a borrowed `SpanSlice` into an owned `SpanBytes`, by resolving all internal - /// references into slices of the provided `Bytes` buffer. Returns `None` if any slice is - /// out of bounds or invalid. - pub fn try_to_bytes(&self, bytes: &Bytes) -> Option { - Some(SpanBytes { - service: BytesString::try_from_bytes_slice(bytes, self.service)?, - name: BytesString::try_from_bytes_slice(bytes, self.name)?, - resource: BytesString::try_from_bytes_slice(bytes, self.resource)?, - r#type: BytesString::try_from_bytes_slice(bytes, self.r#type)?, - trace_id: self.trace_id, - span_id: self.span_id, - parent_id: self.parent_id, - start: self.start, - duration: self.duration, - error: self.error, - meta: self - .meta - .iter() - .map(|(k, v)| { - Some(( - BytesString::try_from_bytes_slice(bytes, k)?, - BytesString::try_from_bytes_slice(bytes, v)?, - )) - }) - .collect::>>()?, - metrics: self - .metrics - .iter() - .map(|(k, v)| Some((BytesString::try_from_bytes_slice(bytes, k)?, *v))) - .collect::>>()?, - meta_struct: self - .meta_struct - .iter() - .map(|(k, v)| Some((BytesString::try_from_bytes_slice(bytes, k)?, v.clone()))) - .collect::>>()?, - span_links: self - .span_links - .iter() - .map(|link| link.try_to_bytes(bytes)) - .collect::>>()?, - span_events: self - .span_events - .iter() - .map(|event| event.try_to_bytes(bytes)) - .collect::>>()?, + #[inline] + fn read_string(buf: &mut Bytes) -> Result { + // Note: we need to pass a &'static lifetime here, otherwise it'll complain + read_string_ref_nomut(unsafe { buf.as_mut_slice() }).map(|(str, newbuf)| { + let string = BytesString::from_bytes_slice(buf, str); + *unsafe { buf.as_mut_slice() } = newbuf; + string }) } } -impl SpanLinkSlice<'_> { - /// Converts a borrowed `SpanLinkSlice` into an owned `SpanLinkBytes`, using the provided - /// `Bytes` buffer to resolve all referenced strings. Returns `None` if conversion fails due - /// to invalid slice ranges. - pub fn try_to_bytes(&self, bytes: &Bytes) -> Option { - Some(SpanLinkBytes { - trace_id: self.trace_id, - trace_id_high: self.trace_id_high, - span_id: self.span_id, - attributes: self - .attributes - .iter() - .map(|(k, v)| { - Some(( - BytesString::try_from_bytes_slice(bytes, k)?, - BytesString::try_from_bytes_slice(bytes, v)?, - )) - }) - .collect::>>()?, - tracestate: BytesString::try_from_bytes_slice(bytes, self.tracestate)?, - flags: self.flags, - }) - } -} +/// TraceData implementation using `&str` and `&[u8]`. +#[derive(Default, Debug, Clone, PartialEq, Serialize)] +pub struct SliceData<'a>(PhantomData<&'a u8>); +impl<'a> TraceData for SliceData<'a> { + type Text = &'a str; + type Bytes = &'a [u8]; -impl SpanEventSlice<'_> { - /// Converts a borrowed `SpanEventSlice` into an owned `SpanEventBytes`, resolving references - /// into the provided `Bytes` buffer. Fails with `None` if any slice is invalid or cannot be - /// converted. - pub fn try_to_bytes(&self, bytes: &Bytes) -> Option { - Some(SpanEventBytes { - time_unix_nano: self.time_unix_nano, - name: BytesString::try_from_bytes_slice(bytes, self.name)?, - attributes: self - .attributes - .iter() - .map(|(k, v)| { - Some(( - BytesString::try_from_bytes_slice(bytes, k)?, - v.try_to_bytes(bytes)?, - )) - }) - .collect::>>()?, - }) + #[inline] + fn get_mut_slice<'b>(buf: &'b mut &'a [u8]) -> &'b mut &'static [u8] { + unsafe { std::mem::transmute::<&'b mut &[u8], &'b mut &'static [u8]>(buf) } } -} -impl AttributeAnyValueSlice<'_> { - /// Converts a borrowed `AttributeAnyValueSlice` into its owned `AttributeAnyValueBytes` - /// representation, using the provided `Bytes` buffer. Recursively processes inner values if - /// it's an array. - pub fn try_to_bytes(&self, bytes: &Bytes) -> Option { - match self { - AttributeAnyValue::SingleValue(value) => { - Some(AttributeAnyValue::SingleValue(value.try_to_bytes(bytes)?)) - } - AttributeAnyValue::Array(value) => Some(AttributeAnyValue::Array( - value - .iter() - .map(|attribute| attribute.try_to_bytes(bytes)) - .collect::>>()?, - )), - } + #[inline] + fn try_slice_and_advance(buf: &mut &'a [u8], bytes: usize) -> Option<&'a [u8]> { + let slice = buf.get(0..bytes)?; + *buf = &buf[bytes..]; + Some(slice) } -} -impl AttributeArrayValueSlice<'_> { - /// Converts a single `AttributeArrayValueSlice` item into its owned form - /// (`AttributeArrayValueBytes`), borrowing data from the provided `Bytes` buffer when - /// necessary. - pub fn try_to_bytes(&self, bytes: &Bytes) -> Option { - match self { - AttributeArrayValue::String(value) => Some(AttributeArrayValue::String( - BytesString::try_from_bytes_slice(bytes, value)?, - )), - AttributeArrayValue::Boolean(value) => Some(AttributeArrayValue::Boolean(*value)), - AttributeArrayValue::Integer(value) => Some(AttributeArrayValue::Integer(*value)), - AttributeArrayValue::Double(value) => Some(AttributeArrayValue::Double(*value)), - } + #[inline] + fn read_string(buf: &mut &'a [u8]) -> Result<&'a str, DecodeError> { + read_string_ref_nomut(buf).map(|(str, newbuf)| { + *buf = newbuf; + str + }) } } @@ -435,113 +190,4 @@ impl fmt::Display for SpanKeyParseError { } impl std::error::Error for SpanKeyParseError {} -fn is_default(t: &T) -> bool { - t == &T::default() -} - -#[cfg(test)] -mod tests { - use super::{AttributeAnyValue, AttributeArrayValue, Span, SpanEvent, SpanLink}; - use crate::msgpack_decoder::v04::span::decode_span; - use std::collections::HashMap; - - #[test] - fn skip_serializing_empty_fields_test() { - let expected = b"\x87\xa7service\xa0\xa4name\xa0\xa8resource\xa0\xa8trace_id\x00\xa7span_id\x00\xa5start\x00\xa8duration\x00"; - let val: Span<&str> = Span::default(); - let serialized = rmp_serde::encode::to_vec_named(&val).unwrap(); - assert_eq!(expected, serialized.as_slice()); - } - - #[test] - fn serialize_deserialize_test() { - let span: Span<&str> = Span { - name: "tracing.operation", - resource: "MyEndpoint", - span_links: vec![SpanLink { - trace_id: 42, - attributes: HashMap::from([("span", "link")]), - tracestate: "running", - ..Default::default() - }], - span_events: vec![SpanEvent { - time_unix_nano: 1727211691770716000, - name: "exception", - attributes: HashMap::from([ - ( - "exception.message", - AttributeAnyValue::SingleValue(AttributeArrayValue::String( - "Cannot divide by zero", - )), - ), - ( - "exception.type", - AttributeAnyValue::SingleValue(AttributeArrayValue::String("RuntimeError")), - ), - ( - "exception.escaped", - AttributeAnyValue::SingleValue(AttributeArrayValue::Boolean(false)), - ), - ( - "exception.count", - AttributeAnyValue::SingleValue(AttributeArrayValue::Integer(1)), - ), - ( - "exception.lines", - AttributeAnyValue::Array(vec![ - AttributeArrayValue::String(" File \"\", line 1, in "), - AttributeArrayValue::String(" File \"\", line 1, in divide"), - AttributeArrayValue::String("RuntimeError: Cannot divide by zero"), - ]), - ), - ]), - }], - ..Default::default() - }; - - let serialized = rmp_serde::encode::to_vec_named(&span).unwrap(); - let mut serialized_slice = serialized.as_ref(); - let deserialized = decode_span(&mut serialized_slice).unwrap(); - - assert_eq!(span.name, deserialized.name); - assert_eq!(span.resource, deserialized.resource); - assert_eq!( - span.span_links[0].trace_id, - deserialized.span_links[0].trace_id - ); - assert_eq!( - span.span_links[0].tracestate, - deserialized.span_links[0].tracestate - ); - assert_eq!(span.span_events[0].name, deserialized.span_events[0].name); - assert_eq!( - span.span_events[0].time_unix_nano, - deserialized.span_events[0].time_unix_nano - ); - for attribut in &deserialized.span_events[0].attributes { - assert!(span.span_events[0].attributes.contains_key(attribut.0)) - } - } - - #[test] - fn serialize_event_test() { - // `expected` is created by transforming the span into bytes - // and passing each bytes through `escaped_default` - let expected = b"\x88\xa7service\xa0\xa4name\xa0\xa8resource\xa0\xa8trace_id\x00\xa7span_id\x00\xa5start\x00\xa8duration\x00\xabspan_events\x91\x83\xaetime_unix_nano\xcf\x17\xf8I\xe1\xeb\xe5\x1f`\xa4name\xa4test\xaaattributes\x81\xaatest.event\x82\xa4type\x03\xacdouble_value\xcb@\x10\xcc\xcc\xcc\xcc\xcc\xcd"; - - let span: Span<&str> = Span { - span_events: vec![SpanEvent { - time_unix_nano: 1727211691770716000, - name: "test", - attributes: HashMap::from([( - "test.event", - AttributeAnyValue::SingleValue(AttributeArrayValue::Double(4.2)), - )]), - }], - ..Default::default() - }; - - let serialized = rmp_serde::encode::to_vec_named(&span).unwrap(); - assert_eq!(expected, serialized.as_slice()); - } -} +pub type SharedDictBytes = SharedDict; diff --git a/datadog-trace-utils/src/span/trace_utils.rs b/datadog-trace-utils/src/span/trace_utils.rs index 6a0ae7c11f..8a0c263903 100644 --- a/datadog-trace-utils/src/span/trace_utils.rs +++ b/datadog-trace-utils/src/span/trace_utils.rs @@ -3,7 +3,7 @@ //! Trace-utils functionalities implementation for tinybytes based spans -use super::{Span, SpanText}; +use super::{v04::Span, SpanText, TraceData}; use std::collections::HashMap; /// Span metric the mini agent must set for the backend to recognize top level span @@ -15,10 +15,11 @@ const PARTIAL_VERSION_KEY: &str = "_dd.partial_version"; fn set_top_level_span(span: &mut Span, is_top_level: bool) where - T: SpanText, + T: TraceData, { if is_top_level { - span.metrics.insert(T::from_static_str(TOP_LEVEL_KEY), 1.0); + span.metrics + .insert(T::Text::from_static_str(TOP_LEVEL_KEY), 1.0); } else { span.metrics.remove(TOP_LEVEL_KEY); } @@ -32,9 +33,9 @@ where /// ancestor of other spans belonging to this service and attached to it). pub fn compute_top_level_span(trace: &mut [Span]) where - T: SpanText, + T: TraceData, { - let mut span_id_to_service: HashMap = HashMap::new(); + let mut span_id_to_service: HashMap = HashMap::new(); for span in trace.iter() { span_id_to_service.insert(span.span_id, span.service.clone()); } @@ -60,7 +61,7 @@ where } /// Return true if the span has a top level key set -pub fn has_top_level(span: &Span) -> bool { +pub fn has_top_level(span: &Span) -> bool { span.metrics .get(TRACER_TOP_LEVEL_KEY) .is_some_and(|v| *v == 1.0) @@ -68,7 +69,7 @@ pub fn has_top_level(span: &Span) -> bool { } /// Returns true if a span should be measured (i.e., it should get trace metrics calculated). -pub fn is_measured(span: &Span) -> bool { +pub fn is_measured(span: &Span) -> bool { span.metrics.get(MEASURED_KEY).is_some_and(|v| *v == 1.0) } @@ -77,7 +78,7 @@ pub fn is_measured(span: &Span) -> bool { /// When incomplete, a partial snapshot has a metric _dd.partial_version which is a positive /// integer. The metric usually increases each time a new version of the same span is sent by /// the tracer -pub fn is_partial_snapshot(span: &Span) -> bool { +pub fn is_partial_snapshot(span: &Span) -> bool { span.metrics .get(PARTIAL_VERSION_KEY) .is_some_and(|v| *v >= 0.0) @@ -102,7 +103,7 @@ const SAMPLING_ANALYTICS_RATE_KEY: &str = "_dd1.sr.eausr"; /// dropped and the latter to the spans dropped. pub fn drop_chunks(traces: &mut Vec>>) -> DroppedP0Stats where - T: SpanText, + T: TraceData, { let mut dropped_p0_traces = 0; let mut dropped_p0_spans = 0; @@ -156,7 +157,7 @@ where #[cfg(test)] mod tests { use super::*; - use crate::span::SpanBytes; + use crate::span::v04::SpanBytes; fn create_test_span( trace_id: u64, diff --git a/datadog-trace-utils/src/span/v04/mod.rs b/datadog-trace-utils/src/span/v04/mod.rs new file mode 100644 index 0000000000..3a133b861e --- /dev/null +++ b/datadog-trace-utils/src/span/v04/mod.rs @@ -0,0 +1,341 @@ +// Copyright 2023-Present Datadog, Inc. https://www.datadoghq.com/ +// SPDX-License-Identifier: Apache-2.0 + +use crate::span::{SliceData, SpanKeyParseError, TinyData, TraceData}; +use crate::tracer_payload::TraceChunks; +use serde::ser::SerializeStruct; +use serde::Serialize; +use std::borrow::Borrow; +use std::collections::HashMap; +use std::str::FromStr; + +#[derive(Debug, PartialEq)] +pub enum SpanKey { + Service, + Name, + Resource, + TraceId, + SpanId, + ParentId, + Start, + Duration, + Error, + Meta, + Metrics, + Type, + MetaStruct, + SpanLinks, + SpanEvents, +} + +impl FromStr for SpanKey { + type Err = SpanKeyParseError; + + fn from_str(s: &str) -> Result { + match s { + "service" => Ok(SpanKey::Service), + "name" => Ok(SpanKey::Name), + "resource" => Ok(SpanKey::Resource), + "trace_id" => Ok(SpanKey::TraceId), + "span_id" => Ok(SpanKey::SpanId), + "parent_id" => Ok(SpanKey::ParentId), + "start" => Ok(SpanKey::Start), + "duration" => Ok(SpanKey::Duration), + "error" => Ok(SpanKey::Error), + "meta" => Ok(SpanKey::Meta), + "metrics" => Ok(SpanKey::Metrics), + "type" => Ok(SpanKey::Type), + "meta_struct" => Ok(SpanKey::MetaStruct), + "span_links" => Ok(SpanKey::SpanLinks), + "span_events" => Ok(SpanKey::SpanEvents), + _ => Err(SpanKeyParseError::new(format!("Invalid span key: {s}"))), + } + } +} + +/// Checks if the `value` represents an empty string. Used to skip serializing empty strings +/// with serde. +fn is_empty_str>(value: &T) -> bool { + value.borrow().is_empty() +} + +/// The generic representation of a V04 span. +/// +/// `T` is the type used to represent strings in the span, it can be either owned (e.g. BytesString) +/// or borrowed (e.g. &str). To define a generic function taking any `Span` you can use the +/// [`SpanValue`] trait: +/// ``` +/// use datadog_trace_utils::span::{v04::Span, TraceData}; +/// fn foo(span: Span) { +/// let _ = span.meta.get("foo"); +/// } +/// ``` +#[derive(Clone, Debug, Default, PartialEq, Serialize)] +pub struct Span { + pub service: T::Text, + pub name: T::Text, + pub resource: T::Text, + #[serde(skip_serializing_if = "is_empty_str")] + pub r#type: T::Text, + pub trace_id: u64, + pub span_id: u64, + #[serde(skip_serializing_if = "is_default")] + pub parent_id: u64, + pub start: i64, + pub duration: i64, + #[serde(skip_serializing_if = "is_default")] + pub error: i32, + #[serde(skip_serializing_if = "HashMap::is_empty")] + pub meta: HashMap, + #[serde(skip_serializing_if = "HashMap::is_empty")] + pub metrics: HashMap, + #[serde(skip_serializing_if = "HashMap::is_empty")] + pub meta_struct: HashMap, + #[serde(skip_serializing_if = "Vec::is_empty")] + pub span_links: Vec>, + #[serde(skip_serializing_if = "Vec::is_empty")] + pub span_events: Vec>, +} + +/// The generic representation of a V04 span link. +/// `T` is the type used to represent strings in the span link. +#[derive(Clone, Debug, Default, PartialEq, Serialize)] +pub struct SpanLink { + pub trace_id: u64, + pub trace_id_high: u64, + pub span_id: u64, + #[serde(skip_serializing_if = "HashMap::is_empty")] + pub attributes: HashMap, + #[serde(skip_serializing_if = "is_empty_str")] + pub tracestate: T::Text, + #[serde(skip_serializing_if = "is_default")] + pub flags: u32, +} + +/// The generic representation of a V04 span event. +/// `T` is the type used to represent strings in the span event. +#[derive(Clone, Debug, Default, PartialEq, Serialize)] +pub struct SpanEvent { + pub time_unix_nano: u64, + pub name: T::Text, + #[serde(skip_serializing_if = "HashMap::is_empty")] + pub attributes: HashMap>, +} + +#[derive(Clone, Debug, PartialEq)] +pub enum AttributeAnyValue { + SingleValue(AttributeArrayValue), + Array(Vec>), +} + +#[derive(Serialize)] +struct ArrayValueWrapper<'a, T: TraceData> { + values: &'a Vec>, +} + +impl Serialize for AttributeAnyValue { + fn serialize(&self, serializer: S) -> Result + where + S: serde::Serializer, + { + let mut state = serializer.serialize_struct("AttributeAnyValue", 2)?; + + match self { + AttributeAnyValue::SingleValue(attribute) => { + serialize_attribute_array::(&mut state, attribute)?; + } + AttributeAnyValue::Array(value) => { + let value_type: u8 = self.into(); + state.serialize_field("type", &value_type)?; + let wrapped_value = ArrayValueWrapper { values: value }; + state.serialize_field("array_value", &wrapped_value)?; + } + } + + state.end() + } +} + +impl From<&AttributeAnyValue> for u8 { + fn from(attribute: &AttributeAnyValue) -> u8 { + match attribute { + AttributeAnyValue::SingleValue(value) => value.into(), + AttributeAnyValue::Array(_) => 4, + } + } +} + +#[derive(Clone, Debug, PartialEq)] +pub enum AttributeArrayValue { + String(T::Text), + Boolean(bool), + Integer(i64), + Double(f64), +} + +impl Serialize for AttributeArrayValue { + fn serialize(&self, serializer: S) -> Result + where + S: serde::Serializer, + { + let mut state = serializer.serialize_struct("AttributeArrayValue", 2)?; + serialize_attribute_array::(&mut state, self)?; + state.end() + } +} + +fn serialize_attribute_array( + state: &mut S::SerializeStruct, + attribute: &AttributeArrayValue, +) -> Result<(), ::Error> +where + T: TraceData, + S: serde::Serializer, +{ + let attribute_type: u8 = attribute.into(); + state.serialize_field("type", &attribute_type)?; + match attribute { + AttributeArrayValue::String(value) => state.serialize_field("string_value", value), + AttributeArrayValue::Boolean(value) => state.serialize_field("bool_value", value), + AttributeArrayValue::Integer(value) => state.serialize_field("int_value", value), + AttributeArrayValue::Double(value) => state.serialize_field("double_value", value), + } +} + +impl From<&AttributeArrayValue> for u8 { + fn from(attribute: &AttributeArrayValue) -> u8 { + match attribute { + AttributeArrayValue::String(_) => 0, + AttributeArrayValue::Boolean(_) => 1, + AttributeArrayValue::Integer(_) => 2, + AttributeArrayValue::Double(_) => 3, + } + } +} + +fn is_default(t: &T) -> bool { + t == &T::default() +} + +pub type SpanBytes = Span; +pub type SpanLinkBytes = SpanLink; +pub type SpanEventBytes = SpanEvent; +pub type AttributeAnyValueBytes = AttributeAnyValue; +pub type AttributeArrayValueBytes = AttributeArrayValue; + +pub type SpanSlice<'a> = Span>; +pub type SpanLinkSlice<'a> = SpanLink>; +pub type SpanEventSlice<'a> = SpanEvent>; +pub type AttributeAnyValueSlice<'a> = AttributeAnyValue>; +pub type AttributeArrayValueSlice<'a> = AttributeArrayValue>; + +pub type TraceChunksBytes = TraceChunks; + +#[cfg(test)] +mod tests { + use super::{AttributeAnyValue, AttributeArrayValue, Span, SpanEvent, SpanLink}; + use crate::msgpack_decoder::decode::buffer::Buffer; + use crate::msgpack_decoder::v04::span::decode_span; + use crate::span::SliceData; + use std::collections::HashMap; + + #[test] + fn skip_serializing_empty_fields_test() { + let expected = b"\x87\xa7service\xa0\xa4name\xa0\xa8resource\xa0\xa8trace_id\x00\xa7span_id\x00\xa5start\x00\xa8duration\x00"; + let val: Span> = Span::default(); + let serialized = rmp_serde::encode::to_vec_named(&val).unwrap(); + assert_eq!(expected, serialized.as_slice()); + } + + #[test] + fn serialize_deserialize_test() { + let span: Span> = Span { + name: "tracing.operation", + resource: "MyEndpoint", + span_links: vec![SpanLink { + trace_id: 42, + attributes: HashMap::from([("span", "link")]), + tracestate: "running", + ..Default::default() + }], + span_events: vec![SpanEvent { + time_unix_nano: 1727211691770716000, + name: "exception", + attributes: HashMap::from([ + ( + "exception.message", + AttributeAnyValue::SingleValue(AttributeArrayValue::String( + "Cannot divide by zero", + )), + ), + ( + "exception.type", + AttributeAnyValue::SingleValue(AttributeArrayValue::String("RuntimeError")), + ), + ( + "exception.escaped", + AttributeAnyValue::SingleValue(AttributeArrayValue::Boolean(false)), + ), + ( + "exception.count", + AttributeAnyValue::SingleValue(AttributeArrayValue::Integer(1)), + ), + ( + "exception.lines", + AttributeAnyValue::Array(vec![ + AttributeArrayValue::String(" File \"\", line 1, in "), + AttributeArrayValue::String(" File \"\", line 1, in divide"), + AttributeArrayValue::String("RuntimeError: Cannot divide by zero"), + ]), + ), + ]), + }], + ..Default::default() + }; + + let serialized = rmp_serde::encode::to_vec_named(&span).unwrap(); + let mut serialized_slice = Buffer::>::new(serialized.as_ref()); + let deserialized = decode_span(&mut serialized_slice).unwrap(); + + assert_eq!(span.name, deserialized.name); + assert_eq!(span.resource, deserialized.resource); + assert_eq!( + span.span_links[0].trace_id, + deserialized.span_links[0].trace_id + ); + assert_eq!( + span.span_links[0].tracestate, + deserialized.span_links[0].tracestate + ); + assert_eq!(span.span_events[0].name, deserialized.span_events[0].name); + assert_eq!( + span.span_events[0].time_unix_nano, + deserialized.span_events[0].time_unix_nano + ); + for attribut in &deserialized.span_events[0].attributes { + assert!(span.span_events[0].attributes.contains_key(attribut.0)) + } + } + + #[test] + fn serialize_event_test() { + // `expected` is created by transforming the span into bytes + // and passing each bytes through `escaped_default` + let expected = b"\x88\xa7service\xa0\xa4name\xa0\xa8resource\xa0\xa8trace_id\x00\xa7span_id\x00\xa5start\x00\xa8duration\x00\xabspan_events\x91\x83\xaetime_unix_nano\xcf\x17\xf8I\xe1\xeb\xe5\x1f`\xa4name\xa4test\xaaattributes\x81\xaatest.event\x82\xa4type\x03\xacdouble_value\xcb@\x10\xcc\xcc\xcc\xcc\xcc\xcd"; + + let span: Span> = Span { + span_events: vec![SpanEvent { + time_unix_nano: 1727211691770716000, + name: "test", + attributes: HashMap::from([( + "test.event", + AttributeAnyValue::SingleValue(AttributeArrayValue::Double(4.2)), + )]), + }], + ..Default::default() + }; + + let serialized = rmp_serde::encode::to_vec_named(&span).unwrap(); + assert_eq!(expected, serialized.as_slice()); + } +} diff --git a/datadog-trace-utils/src/span/v05/mod.rs b/datadog-trace-utils/src/span/v05/mod.rs index 61a179dfde..c48d92322a 100644 --- a/datadog-trace-utils/src/span/v05/mod.rs +++ b/datadog-trace-utils/src/span/v05/mod.rs @@ -3,7 +3,7 @@ pub mod dict; -use crate::span::{v05::dict::SharedDict, SpanText}; +use crate::span::{v05::dict::SharedDict, TraceData}; use anyhow::Result; use serde::Serialize; use std::collections::HashMap; @@ -28,9 +28,9 @@ pub struct Span { pub r#type: u32, } -pub fn from_span( - span: &crate::span::Span, - dict: &mut SharedDict, +pub fn from_v04_span( + span: &crate::span::v04::Span, + dict: &mut SharedDict, ) -> Result { Ok(Span { service: dict.get_or_insert(&span.service)?, @@ -63,7 +63,7 @@ pub fn from_span( #[cfg(test)] mod tests { use super::*; - use crate::span::SpanBytes; + use crate::span::v04::SpanBytes; use tinybytes::BytesString; #[test] @@ -90,7 +90,7 @@ mod tests { }; let mut dict = SharedDict::default(); - let v05_span = from_span(&span, &mut dict).unwrap(); + let v05_span = from_v04_span(&span, &mut dict).unwrap(); let dict = dict.dict(); diff --git a/datadog-trace-utils/src/test_utils/mod.rs b/datadog-trace-utils/src/test_utils/mod.rs index cdac5fcfe1..94769d8e48 100644 --- a/datadog-trace-utils/src/test_utils/mod.rs +++ b/datadog-trace-utils/src/test_utils/mod.rs @@ -15,7 +15,7 @@ use std::collections::HashMap; use std::time::Duration; use crate::send_data::SendData; -use crate::span::SpanBytes; +use crate::span::v04::SpanBytes; use crate::span::{v05, SharedDictBytes}; use crate::trace_utils::TracerHeaderTags; use crate::tracer_payload::TracerPayloadCollection; diff --git a/datadog-trace-utils/src/trace_utils.rs b/datadog-trace-utils/src/trace_utils.rs index 369f459e97..a5e3b27368 100644 --- a/datadog-trace-utils/src/trace_utils.rs +++ b/datadog-trace-utils/src/trace_utils.rs @@ -4,7 +4,7 @@ pub use crate::send_data::send_data_result::SendDataResult; pub use crate::send_data::SendData; use crate::span::v05::dict::SharedDict; -use crate::span::{v05, SpanText}; +use crate::span::{v05, TraceData}; pub use crate::tracer_header_tags::TracerHeaderTags; use crate::tracer_payload::TracerPayloadCollection; use crate::tracer_payload::{self, TraceChunks}; @@ -598,8 +598,8 @@ macro_rules! parse_root_span_tags { } } -pub fn collect_trace_chunks( - traces: Vec>>, +pub fn collect_trace_chunks( + traces: Vec>>, use_v05_format: bool, ) -> anyhow::Result> { if use_v05_format { @@ -609,7 +609,7 @@ pub fn collect_trace_chunks( let v05_trace = trace.iter().try_fold( Vec::with_capacity(trace.len()), |mut acc, span| -> anyhow::Result> { - acc.push(v05::from_span(span, &mut shared_dict)?); + acc.push(v05::from_v04_span(span, &mut shared_dict)?); Ok(acc) }, )?; @@ -722,36 +722,32 @@ mod tests { #[test] fn test_coalescing_does_not_exceed_max_size() { - let dummy = SendData::new( - MAX_PAYLOAD_SIZE / 5 + 1, - TracerPayloadCollection::V07(vec![pb::TracerPayload { - container_id: "".to_string(), - language_name: "".to_string(), - language_version: "".to_string(), - tracer_version: "".to_string(), - runtime_id: "".to_string(), - chunks: vec![pb::TraceChunk { - priority: 0, - origin: "".to_string(), - spans: vec![], + fn dummy() -> SendData { + SendData::new( + MAX_PAYLOAD_SIZE / 5 + 1, + TracerPayloadCollection::V07(vec![pb::TracerPayload { + container_id: "".to_string(), + language_name: "".to_string(), + language_version: "".to_string(), + tracer_version: "".to_string(), + runtime_id: "".to_string(), + chunks: vec![pb::TraceChunk { + priority: 0, + origin: "".to_string(), + spans: vec![], + tags: Default::default(), + dropped_trace: false, + }], tags: Default::default(), - dropped_trace: false, - }], - tags: Default::default(), - env: "".to_string(), - hostname: "".to_string(), - app_version: "".to_string(), - }]), - TracerHeaderTags::default(), - &Endpoint::default(), - ); - let coalesced = coalesce_send_data(vec![ - dummy.clone(), - dummy.clone(), - dummy.clone(), - dummy.clone(), - dummy.clone(), - ]); + env: "".to_string(), + hostname: "".to_string(), + app_version: "".to_string(), + }]), + TracerHeaderTags::default(), + &Endpoint::default(), + ) + } + let coalesced = coalesce_send_data(vec![dummy(), dummy(), dummy(), dummy(), dummy()]); assert_eq!( 5, coalesced diff --git a/datadog-trace-utils/src/tracer_payload.rs b/datadog-trace-utils/src/tracer_payload.rs index 7bf6b91335..86dd23f4a1 100644 --- a/datadog-trace-utils/src/tracer_payload.rs +++ b/datadog-trace-utils/src/tracer_payload.rs @@ -1,7 +1,7 @@ // Copyright 2024-Present Datadog, Inc. https://www.datadoghq.com/ // SPDX-License-Identifier: Apache-2.0 -use crate::span::{v05, Span, SpanBytes, SpanText}; +use crate::span::{v04, v05, TinyData, TraceData}; use crate::trace_utils::collect_trace_chunks; use crate::{msgpack_decoder, trace_utils::cmp_send_data_payloads}; use datadog_trace_protobuf::pb; @@ -9,7 +9,7 @@ use std::cmp::Ordering; use std::iter::Iterator; use tinybytes::{self, BytesString}; -pub type TracerPayloadV04 = Vec; +pub type TracerPayloadV04 = Vec; pub type TracerPayloadV05 = Vec; #[derive(Debug, Clone)] @@ -17,19 +17,19 @@ pub type TracerPayloadV05 = Vec; pub enum TraceEncoding { /// v0.4 encoding (TracerPayloadV04). V04, - /// v054 encoding (TracerPayloadV04). + /// v0.5 encoding (TracerPayloadV05). V05, } -#[derive(Debug, Clone)] -pub enum TraceChunks { +#[derive(Debug)] +pub enum TraceChunks { /// Collection of TraceChunkSpan. - V04(Vec>>), + V04(Vec>>), /// Collection of TraceChunkSpan with de-duplicated strings. - V05((Vec, Vec>)), + V05((Vec, Vec>)), } -impl TraceChunks { +impl TraceChunks { pub fn into_tracer_payload_collection(self) -> TracerPayloadCollection { match self { TraceChunks::V04(traces) => TracerPayloadCollection::V04(traces), @@ -38,7 +38,7 @@ impl TraceChunks { } } -impl TraceChunks { +impl TraceChunks { /// Returns the number of traces in the chunk pub fn size(&self) -> usize { match self { @@ -48,15 +48,15 @@ impl TraceChunks { } } -#[derive(Debug, Clone)] +#[derive(Debug)] /// Enum representing a general abstraction for a collection of tracer payloads. pub enum TracerPayloadCollection { /// Collection of TracerPayloads. V07(Vec), /// Collection of TraceChunkSpan. - V04(Vec>), + V04(Vec>), /// Collection of TraceChunkSpan with de-duplicated strings. - V05((Vec, Vec>)), + V05((Vec, Vec>)), } impl TracerPayloadCollection { @@ -222,7 +222,7 @@ impl TraceChunkProcessor for DefaultTraceChunkProcessor { pub fn decode_to_trace_chunks( data: tinybytes::Bytes, encoding_type: TraceEncoding, -) -> Result<(TraceChunks, usize), anyhow::Error> { +) -> Result<(TraceChunks, usize), anyhow::Error> { let (data, size) = match encoding_type { TraceEncoding::V04 => msgpack_decoder::v04::from_bytes(data), TraceEncoding::V05 => msgpack_decoder::v05::from_bytes(data), @@ -238,7 +238,7 @@ pub fn decode_to_trace_chunks( #[cfg(test)] mod tests { use super::*; - use crate::span::SpanBytes; + use crate::span::v04::SpanBytes; use crate::test_utils::create_test_no_alloc_span; use datadog_trace_protobuf::pb; use serde_json::json; @@ -277,34 +277,43 @@ mod tests { #[test] fn test_append_traces_v07() { + let mut two_traces = create_dummy_collection_v07(); + two_traces.append(&mut create_dummy_collection_v07()); + let mut trace = create_dummy_collection_v07(); - let empty = TracerPayloadCollection::V07(vec![]); + let mut empty = TracerPayloadCollection::V07(vec![]); - trace.append(&mut trace.clone()); + trace.append(&mut create_dummy_collection_v07()); assert_eq!(2, trace.size()); - trace.append(&mut trace.clone()); + trace.append(&mut two_traces); assert_eq!(4, trace.size()); - trace.append(&mut empty.clone()); + trace.append(&mut empty); assert_eq!(4, trace.size()); } #[test] fn test_append_traces_v04() { - let mut trace = - TracerPayloadCollection::V04(vec![vec![create_test_no_alloc_span(0, 1, 0, 2, true)]]); + fn create_trace() -> TracerPayloadCollection { + TracerPayloadCollection::V04(vec![vec![create_test_no_alloc_span(0, 1, 0, 2, true)]]) + } + + let mut two_traces = create_trace(); + two_traces.append(&mut create_trace()); + + let mut trace = create_trace(); - let empty = TracerPayloadCollection::V04(vec![]); + let mut empty = TracerPayloadCollection::V04(vec![]); - trace.append(&mut trace.clone()); + trace.append(&mut create_trace()); assert_eq!(2, trace.size()); - trace.append(&mut trace.clone()); + trace.append(&mut two_traces); assert_eq!(4, trace.size()); - trace.append(&mut empty.clone()); + trace.append(&mut empty); assert_eq!(4, trace.size()); } @@ -312,7 +321,7 @@ mod tests { fn test_merge_traces() { let mut trace = create_dummy_collection_v07(); - trace.append(&mut trace.clone()); + trace.append(&mut create_dummy_collection_v07()); assert_eq!(2, trace.size()); trace.merge(); @@ -428,7 +437,7 @@ mod tests { #[test] fn test_try_into_meta_metrics_success() { let dummy_trace = create_trace(); - let expected = vec![dummy_trace.clone()]; + let expected = vec![create_trace()]; let payload = rmp_serde::to_vec_named(&expected).unwrap(); let payload = tinybytes::Bytes::from(payload); From d207172014f85720880c8bd944fcf7260cbd5ce0 Mon Sep 17 00:00:00 2001 From: Bob Weinand Date: Tue, 30 Sep 2025 13:23:16 +0200 Subject: [PATCH 2/5] Rename to BytesData Signed-off-by: Bob Weinand --- datadog-trace-utils/src/span/mod.rs | 11 +++++------ datadog-trace-utils/src/span/v04/mod.rs | 14 +++++++------- .../src/test_utils/datadog_test_agent.rs | 2 +- datadog-trace-utils/src/tracer_payload.rs | 6 +++--- 4 files changed, 16 insertions(+), 17 deletions(-) diff --git a/datadog-trace-utils/src/span/mod.rs b/datadog-trace-utils/src/span/mod.rs index 76aa518e9e..5a3599b2c9 100644 --- a/datadog-trace-utils/src/span/mod.rs +++ b/datadog-trace-utils/src/span/mod.rs @@ -99,7 +99,7 @@ impl SpanBytes for Bytes { /// Trait representing a tuple of (Text, Bytes) types used for different underlying data structures. /// Note: The functions are internal to the msgpack decoder and should not be used directly: they're /// only exposed here due to the inavailability of min_specialization in stable Rust. -pub trait TraceData: Default + Debug + Clone + PartialEq + Serialize { +pub trait TraceData: Default + Clone + Debug + PartialEq + Serialize { type Text: SpanText; type Bytes: SpanBytes; @@ -111,9 +111,9 @@ pub trait TraceData: Default + Debug + Clone + PartialEq + Serialize { } /// TraceData implementation using `Bytes` and `BytesString`. -#[derive(Default, Debug, Clone, PartialEq, Serialize)] -pub struct TinyData; -impl TraceData for TinyData { +#[derive(Clone, Default, Debug, PartialEq, Serialize)] +pub struct BytesData; +impl TraceData for BytesData { type Text = BytesString; type Bytes = Bytes; @@ -134,7 +134,6 @@ impl TraceData for TinyData { #[inline] fn read_string(buf: &mut Bytes) -> Result { - // Note: we need to pass a &'static lifetime here, otherwise it'll complain read_string_ref_nomut(unsafe { buf.as_mut_slice() }).map(|(str, newbuf)| { let string = BytesString::from_bytes_slice(buf, str); *unsafe { buf.as_mut_slice() } = newbuf; @@ -144,7 +143,7 @@ impl TraceData for TinyData { } /// TraceData implementation using `&str` and `&[u8]`. -#[derive(Default, Debug, Clone, PartialEq, Serialize)] +#[derive(Clone, Default, Debug, PartialEq, Serialize)] pub struct SliceData<'a>(PhantomData<&'a u8>); impl<'a> TraceData for SliceData<'a> { type Text = &'a str; diff --git a/datadog-trace-utils/src/span/v04/mod.rs b/datadog-trace-utils/src/span/v04/mod.rs index 3a133b861e..53403e9506 100644 --- a/datadog-trace-utils/src/span/v04/mod.rs +++ b/datadog-trace-utils/src/span/v04/mod.rs @@ -1,7 +1,7 @@ // Copyright 2023-Present Datadog, Inc. https://www.datadoghq.com/ // SPDX-License-Identifier: Apache-2.0 -use crate::span::{SliceData, SpanKeyParseError, TinyData, TraceData}; +use crate::span::{BytesData, SliceData, SpanKeyParseError, TraceData}; use crate::tracer_payload::TraceChunks; use serde::ser::SerializeStruct; use serde::Serialize; @@ -217,11 +217,11 @@ fn is_default(t: &T) -> bool { t == &T::default() } -pub type SpanBytes = Span; -pub type SpanLinkBytes = SpanLink; -pub type SpanEventBytes = SpanEvent; -pub type AttributeAnyValueBytes = AttributeAnyValue; -pub type AttributeArrayValueBytes = AttributeArrayValue; +pub type SpanBytes = Span; +pub type SpanLinkBytes = SpanLink; +pub type SpanEventBytes = SpanEvent; +pub type AttributeAnyValueBytes = AttributeAnyValue; +pub type AttributeArrayValueBytes = AttributeArrayValue; pub type SpanSlice<'a> = Span>; pub type SpanLinkSlice<'a> = SpanLink>; @@ -229,7 +229,7 @@ pub type SpanEventSlice<'a> = SpanEvent>; pub type AttributeAnyValueSlice<'a> = AttributeAnyValue>; pub type AttributeArrayValueSlice<'a> = AttributeArrayValue>; -pub type TraceChunksBytes = TraceChunks; +pub type TraceChunksBytes = TraceChunks; #[cfg(test)] mod tests { diff --git a/datadog-trace-utils/src/test_utils/datadog_test_agent.rs b/datadog-trace-utils/src/test_utils/datadog_test_agent.rs index 367b939946..6b0ac922da 100644 --- a/datadog-trace-utils/src/test_utils/datadog_test_agent.rs +++ b/datadog-trace-utils/src/test_utils/datadog_test_agent.rs @@ -216,7 +216,7 @@ impl DatadogAgentContainerBuilder { /// /// let data = SendData::new( /// 100, -/// TracerPayloadCollection::V04(vec![trace.clone()]), +/// TracerPayloadCollection::V04(vec![trace]), /// TracerHeaderTags::default(), /// &endpoint, /// ); diff --git a/datadog-trace-utils/src/tracer_payload.rs b/datadog-trace-utils/src/tracer_payload.rs index 86dd23f4a1..43ca1077e1 100644 --- a/datadog-trace-utils/src/tracer_payload.rs +++ b/datadog-trace-utils/src/tracer_payload.rs @@ -1,7 +1,7 @@ // Copyright 2024-Present Datadog, Inc. https://www.datadoghq.com/ // SPDX-License-Identifier: Apache-2.0 -use crate::span::{v04, v05, TinyData, TraceData}; +use crate::span::{v04, v05, BytesData, TraceData}; use crate::trace_utils::collect_trace_chunks; use crate::{msgpack_decoder, trace_utils::cmp_send_data_payloads}; use datadog_trace_protobuf::pb; @@ -29,7 +29,7 @@ pub enum TraceChunks { V05((Vec, Vec>)), } -impl TraceChunks { +impl TraceChunks { pub fn into_tracer_payload_collection(self) -> TracerPayloadCollection { match self { TraceChunks::V04(traces) => TracerPayloadCollection::V04(traces), @@ -222,7 +222,7 @@ impl TraceChunkProcessor for DefaultTraceChunkProcessor { pub fn decode_to_trace_chunks( data: tinybytes::Bytes, encoding_type: TraceEncoding, -) -> Result<(TraceChunks, usize), anyhow::Error> { +) -> Result<(TraceChunks, usize), anyhow::Error> { let (data, size) = match encoding_type { TraceEncoding::V04 => msgpack_decoder::v04::from_bytes(data), TraceEncoding::V05 => msgpack_decoder::v05::from_bytes(data), From c7da672ae383c22b04f03d8429ea799b907636c7 Mon Sep 17 00:00:00 2001 From: Bob Weinand Date: Tue, 30 Sep 2025 17:19:09 +0200 Subject: [PATCH 3/5] Small note Signed-off-by: Bob Weinand --- datadog-trace-utils/src/span/mod.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/datadog-trace-utils/src/span/mod.rs b/datadog-trace-utils/src/span/mod.rs index 5a3599b2c9..ea17eefc1d 100644 --- a/datadog-trace-utils/src/span/mod.rs +++ b/datadog-trace-utils/src/span/mod.rs @@ -99,6 +99,7 @@ impl SpanBytes for Bytes { /// Trait representing a tuple of (Text, Bytes) types used for different underlying data structures. /// Note: The functions are internal to the msgpack decoder and should not be used directly: they're /// only exposed here due to the inavailability of min_specialization in stable Rust. +/// Also note that the Clone and PartialEq bounds are only present for tests. pub trait TraceData: Default + Clone + Debug + PartialEq + Serialize { type Text: SpanText; type Bytes: SpanBytes; From 2f927528d4892bc794a5296d193661103b5438b5 Mon Sep 17 00:00:00 2001 From: Bob Weinand Date: Tue, 30 Sep 2025 20:13:27 +0200 Subject: [PATCH 4/5] Remove accidentally duplicated SpanKey Signed-off-by: Bob Weinand --- .../src/msgpack_decoder/v04/span.rs | 2 +- datadog-trace-utils/src/span/mod.rs | 45 ------------------- 2 files changed, 1 insertion(+), 46 deletions(-) diff --git a/datadog-trace-utils/src/msgpack_decoder/v04/span.rs b/datadog-trace-utils/src/msgpack_decoder/v04/span.rs index a68f7eebc1..54796d9f60 100644 --- a/datadog-trace-utils/src/msgpack_decoder/v04/span.rs +++ b/datadog-trace-utils/src/msgpack_decoder/v04/span.rs @@ -10,7 +10,7 @@ use crate::msgpack_decoder::decode::string::{ read_nullable_str_map_to_strings, read_nullable_string, }; use crate::msgpack_decoder::decode::{meta_struct::read_meta_struct, metrics::read_metrics}; -use crate::span::{v04::Span, SpanKey, TraceData}; +use crate::span::{v04::Span, v04::SpanKey, TraceData}; use std::borrow::Borrow; /// Decodes a slice of bytes into a `Span` object. diff --git a/datadog-trace-utils/src/span/mod.rs b/datadog-trace-utils/src/span/mod.rs index ea17eefc1d..3a22805742 100644 --- a/datadog-trace-utils/src/span/mod.rs +++ b/datadog-trace-utils/src/span/mod.rs @@ -14,53 +14,8 @@ use std::fmt; use std::fmt::Debug; use std::hash::Hash; use std::marker::PhantomData; -use std::str::FromStr; use tinybytes::{Bytes, BytesString}; -#[derive(Debug, PartialEq)] -pub enum SpanKey { - Service, - Name, - Resource, - TraceId, - SpanId, - ParentId, - Start, - Duration, - Error, - Meta, - Metrics, - Type, - MetaStruct, - SpanLinks, - SpanEvents, -} - -impl FromStr for SpanKey { - type Err = SpanKeyParseError; - - fn from_str(s: &str) -> Result { - match s { - "service" => Ok(SpanKey::Service), - "name" => Ok(SpanKey::Name), - "resource" => Ok(SpanKey::Resource), - "trace_id" => Ok(SpanKey::TraceId), - "span_id" => Ok(SpanKey::SpanId), - "parent_id" => Ok(SpanKey::ParentId), - "start" => Ok(SpanKey::Start), - "duration" => Ok(SpanKey::Duration), - "error" => Ok(SpanKey::Error), - "meta" => Ok(SpanKey::Meta), - "metrics" => Ok(SpanKey::Metrics), - "type" => Ok(SpanKey::Type), - "meta_struct" => Ok(SpanKey::MetaStruct), - "span_links" => Ok(SpanKey::SpanLinks), - "span_events" => Ok(SpanKey::SpanEvents), - _ => Err(SpanKeyParseError::new(format!("Invalid span key: {s}"))), - } - } -} - /// Trait representing the requirements for a type to be used as a Span "string" type. /// Note: Borrow is not required by the derived traits, but allows to access HashMap elements /// from a static str and check if the string is empty. From e58506c4c99537054cf624dcce5de9b9664bc5cb Mon Sep 17 00:00:00 2001 From: Bob Weinand Date: Mon, 6 Oct 2025 20:40:32 +0200 Subject: [PATCH 5/5] Try TraceProjection (incomplete) Signed-off-by: Bob Weinand --- .../src/msgpack_decoder/mod.rs | 1 + .../src/msgpack_decoder/v1/mod.rs | 120 +++ .../src/msgpack_decoder/v1/span.rs | 110 +++ datadog-trace-utils/src/span/mod.rs | 7 + datadog-trace-utils/src/span/table.rs | 91 ++ datadog-trace-utils/src/span/trace.rs | 869 ++++++++++++++++++ datadog-trace-utils/src/span/trace_utils.rs | 33 +- datadog-trace-utils/src/span/v05/mod.rs | 119 ++- datadog-trace-utils/src/span/v1/mod.rs | 137 +++ datadog-trace-utils/src/tracer_payload.rs | 11 +- 10 files changed, 1469 insertions(+), 29 deletions(-) create mode 100644 datadog-trace-utils/src/msgpack_decoder/v1/mod.rs create mode 100644 datadog-trace-utils/src/msgpack_decoder/v1/span.rs create mode 100644 datadog-trace-utils/src/span/table.rs create mode 100644 datadog-trace-utils/src/span/trace.rs create mode 100644 datadog-trace-utils/src/span/v1/mod.rs diff --git a/datadog-trace-utils/src/msgpack_decoder/mod.rs b/datadog-trace-utils/src/msgpack_decoder/mod.rs index 6a07361389..d22f2fd788 100644 --- a/datadog-trace-utils/src/msgpack_decoder/mod.rs +++ b/datadog-trace-utils/src/msgpack_decoder/mod.rs @@ -4,3 +4,4 @@ pub mod decode; pub mod v04; pub mod v05; +pub mod v1; diff --git a/datadog-trace-utils/src/msgpack_decoder/v1/mod.rs b/datadog-trace-utils/src/msgpack_decoder/v1/mod.rs new file mode 100644 index 0000000000..d439645547 --- /dev/null +++ b/datadog-trace-utils/src/msgpack_decoder/v1/mod.rs @@ -0,0 +1,120 @@ +// Copyright 2024-Present Datadog, Inc. https://www.datadoghq.com/ +// SPDX-License-Identifier: Apache-2.0 + +pub(crate) mod span; + +use self::span::decode_span; +use crate::msgpack_decoder::decode::buffer::Buffer; +use crate::msgpack_decoder::decode::error::DecodeError; +use crate::span::v04::{Span, SpanBytes, SpanSlice}; +use crate::span::{TraceData, Traces, TracesBytes, TracesSlice}; + +/// Decodes a Bytes buffer into a `Vec>` object, also represented as a vector of +/// `TracerPayloadV04` objects. +/// +/// # Arguments +/// +/// * `data` - A tinybytes Bytes buffer containing the encoded data. Bytes are expected to be +/// encoded msgpack data containing a list of a list of v04 spans. +/// +/// # Returns +/// +/// * `Ok(Vec, usize)` - A vector of decoded `Vec` objects if +/// successful. and the number of bytes in the slice used by the decoder. +/// * `Err(DecodeError)` - An error if the decoding process fails. +/// +/// # Errors +/// +/// This function will return an error if: +/// - The array length for trace count or span count cannot be read. +/// - Any span cannot be decoded. +/// +/// # Examples +/// +/// ``` +/// use datadog_trace_protobuf::pb::Span; +/// use datadog_trace_utils::msgpack_decoder::v04::from_bytes; +/// use rmp_serde::to_vec_named; +/// use tinybytes; +/// +/// let span = Span { +/// name: "test-span".to_owned(), +/// ..Default::default() +/// }; +/// let encoded_data = to_vec_named(&vec![vec![span]]).unwrap(); +/// let encoded_data_as_tinybytes = tinybytes::Bytes::from(encoded_data); +/// let (decoded_traces, _payload_size) = +/// from_bytes(encoded_data_as_tinybytes).expect("Decoding failed"); +/// +/// assert_eq!(1, decoded_traces.len()); +/// assert_eq!(1, decoded_traces[0].len()); +/// let decoded_span = &decoded_traces[0][0]; +/// assert_eq!("test-span", decoded_span.name.as_str()); +/// ``` +pub fn from_bytes(data: tinybytes::Bytes) -> Result<(TracesBytes, usize), DecodeError> { + from_buffer(&mut Buffer::new(data)) +} + +/// Decodes a slice of bytes into a `Vec>` object. +/// The resulting spans have the same lifetime as the initial buffer. +/// +/// # Arguments +/// +/// * `data` - A slice of bytes containing the encoded data. Bytes are expected to be encoded +/// msgpack data containing a list of a list of v04 spans. +/// +/// # Returns +/// +/// * `Ok(Vec, usize)` - A vector of decoded `Vec` objects if +/// successful. and the number of bytes in the slice used by the decoder. +/// * `Err(DecodeError)` - An error if the decoding process fails. +/// +/// # Errors +/// +/// This function will return an error if: +/// - The array length for trace count or span count cannot be read. +/// - Any span cannot be decoded. +/// +/// # Examples +/// +/// ``` +/// use datadog_trace_protobuf::pb::Span; +/// use datadog_trace_utils::msgpack_decoder::v04::from_slice; +/// use rmp_serde::to_vec_named; +/// use tinybytes; +/// +/// let span = Span { +/// name: "test-span".to_owned(), +/// ..Default::default() +/// }; +/// let encoded_data = to_vec_named(&vec![vec![span]]).unwrap(); +/// let encoded_data_as_tinybytes = tinybytes::Bytes::from(encoded_data); +/// let (decoded_traces, _payload_size) = +/// from_slice(&encoded_data_as_tinybytes).expect("Decoding failed"); +/// +/// assert_eq!(1, decoded_traces.len()); +/// assert_eq!(1, decoded_traces[0].len()); +/// let decoded_span = &decoded_traces[0][0]; +/// assert_eq!("test-span", decoded_span.name); +/// ``` +pub fn from_slice(data: &[u8]) -> Result<(TracesSlice<'_>, usize), DecodeError> { + from_buffer(&mut Buffer::new(data)) +} + +#[allow(clippy::type_complexity)] +pub fn from_buffer( + data: &mut Buffer, +) -> Result<(Traces, usize), DecodeError> { + let trace_count = rmp::decode::read_array_len(data.as_mut_slice()).map_err(|_| { + DecodeError::InvalidFormat("Unable to read array len for trace count".to_owned()) + })?; + + let traces = Traces::default(); + + // Intentionally skip the size of the array (as it will be recomputed after coalescing). + let start_len = data.len(); + + + + Ok((traces, start_len - data.len())) +} diff --git a/datadog-trace-utils/src/msgpack_decoder/v1/span.rs b/datadog-trace-utils/src/msgpack_decoder/v1/span.rs new file mode 100644 index 0000000000..a68f7eebc1 --- /dev/null +++ b/datadog-trace-utils/src/msgpack_decoder/v1/span.rs @@ -0,0 +1,110 @@ +// Copyright 2024-Present Datadog, Inc. https://www.datadoghq.com/ +// SPDX-License-Identifier: Apache-2.0 + +use crate::msgpack_decoder::decode::buffer::Buffer; +use crate::msgpack_decoder::decode::error::DecodeError; +use crate::msgpack_decoder::decode::number::read_nullable_number; +use crate::msgpack_decoder::decode::span_event::read_span_events; +use crate::msgpack_decoder::decode::span_link::read_span_links; +use crate::msgpack_decoder::decode::string::{ + read_nullable_str_map_to_strings, read_nullable_string, +}; +use crate::msgpack_decoder::decode::{meta_struct::read_meta_struct, metrics::read_metrics}; +use crate::span::{v04::Span, SpanKey, TraceData}; +use std::borrow::Borrow; + +/// Decodes a slice of bytes into a `Span` object. +/// +/// # Arguments +/// +/// * `buf` - A mutable reference to a slice of bytes containing the encoded data. +/// +/// # Returns +/// +/// * `Ok(Span)` - A decoded `Span` object if successful. +/// * `Err(DecodeError)` - An error if the decoding process fails. +/// +/// # Errors +/// +/// This function will return an error if: +/// - The map length cannot be read. +/// - Any key or value cannot be decoded. +pub fn decode_span(buffer: &mut Buffer) -> Result, DecodeError> { + let mut span = Span::::default(); + + let span_size = rmp::decode::read_map_len(buffer.as_mut_slice()).map_err(|_| { + DecodeError::InvalidFormat("Unable to get map len for span size".to_owned()) + })?; + + for _ in 0..span_size { + fill_span(&mut span, buffer)?; + } + + Ok(span) +} + +// Safety: read_string_ref checks utf8 validity, so we don't do it again when creating the +// BytesStrings +fn fill_span(span: &mut Span, buf: &mut Buffer) -> Result<(), DecodeError> { + let key = buf + .read_string()? + .borrow() + .parse::() + .map_err(|e| DecodeError::InvalidFormat(e.message))?; + + match key { + SpanKey::Service => span.service = read_nullable_string(buf)?, + SpanKey::Name => span.name = read_nullable_string(buf)?, + SpanKey::Resource => span.resource = read_nullable_string(buf)?, + SpanKey::TraceId => span.trace_id = read_nullable_number(buf)?, + SpanKey::SpanId => span.span_id = read_nullable_number(buf)?, + SpanKey::ParentId => span.parent_id = read_nullable_number(buf)?, + SpanKey::Start => span.start = read_nullable_number(buf)?, + SpanKey::Duration => span.duration = read_nullable_number(buf)?, + SpanKey::Error => span.error = read_nullable_number(buf)?, + SpanKey::Type => span.r#type = read_nullable_string(buf)?, + SpanKey::Meta => span.meta = read_nullable_str_map_to_strings(buf)?, + SpanKey::Metrics => span.metrics = read_metrics(buf)?, + SpanKey::MetaStruct => span.meta_struct = read_meta_struct(buf)?, + SpanKey::SpanLinks => span.span_links = read_span_links(buf)?, + SpanKey::SpanEvents => span.span_events = read_span_events(buf)?, + } + Ok(()) +} + +#[cfg(test)] +mod tests { + use super::SpanKey; + use crate::span::SpanKeyParseError; + use std::str::FromStr; + + #[test] + fn test_span_key_from_str() { + assert_eq!(SpanKey::from_str("service").unwrap(), SpanKey::Service); + assert_eq!(SpanKey::from_str("name").unwrap(), SpanKey::Name); + assert_eq!(SpanKey::from_str("resource").unwrap(), SpanKey::Resource); + assert_eq!(SpanKey::from_str("trace_id").unwrap(), SpanKey::TraceId); + assert_eq!(SpanKey::from_str("span_id").unwrap(), SpanKey::SpanId); + assert_eq!(SpanKey::from_str("parent_id").unwrap(), SpanKey::ParentId); + assert_eq!(SpanKey::from_str("start").unwrap(), SpanKey::Start); + assert_eq!(SpanKey::from_str("duration").unwrap(), SpanKey::Duration); + assert_eq!(SpanKey::from_str("error").unwrap(), SpanKey::Error); + assert_eq!(SpanKey::from_str("meta").unwrap(), SpanKey::Meta); + assert_eq!(SpanKey::from_str("metrics").unwrap(), SpanKey::Metrics); + assert_eq!(SpanKey::from_str("type").unwrap(), SpanKey::Type); + assert_eq!( + SpanKey::from_str("meta_struct").unwrap(), + SpanKey::MetaStruct + ); + assert_eq!(SpanKey::from_str("span_links").unwrap(), SpanKey::SpanLinks); + assert_eq!( + SpanKey::from_str("span_events").unwrap(), + SpanKey::SpanEvents + ); + + let invalid_result = SpanKey::from_str("invalid_key"); + let msg = format!("SpanKeyParseError: Invalid span key: {}", "invalid_key"); + assert!(matches!(invalid_result, Err(SpanKeyParseError { .. }))); + assert_eq!(invalid_result.unwrap_err().to_string(), msg); + } +} diff --git a/datadog-trace-utils/src/span/mod.rs b/datadog-trace-utils/src/span/mod.rs index 3a22805742..0e6451f390 100644 --- a/datadog-trace-utils/src/span/mod.rs +++ b/datadog-trace-utils/src/span/mod.rs @@ -1,9 +1,14 @@ // Copyright 2023-Present Datadog, Inc. https://www.datadoghq.com/ // SPDX-License-Identifier: Apache-2.0 +pub mod table; pub mod trace_utils; pub mod v04; pub mod v05; +pub mod v1; + +mod trace; +pub use trace::*; use crate::msgpack_decoder::decode::buffer::read_string_ref_nomut; use crate::msgpack_decoder::decode::error::DecodeError; @@ -14,6 +19,7 @@ use std::fmt; use std::fmt::Debug; use std::hash::Hash; use std::marker::PhantomData; +use std::ops::Deref; use tinybytes::{Bytes, BytesString}; /// Trait representing the requirements for a type to be used as a Span "string" type. @@ -126,6 +132,7 @@ impl<'a> TraceData for SliceData<'a> { } } + #[derive(Debug)] pub struct SpanKeyParseError { pub message: String, diff --git a/datadog-trace-utils/src/span/table.rs b/datadog-trace-utils/src/span/table.rs new file mode 100644 index 0000000000..bcead8071c --- /dev/null +++ b/datadog-trace-utils/src/span/table.rs @@ -0,0 +1,91 @@ +use std::collections::HashMap; +use std::marker::PhantomData; +use crate::span::TraceData; + +trait TraceDataType { + type Data; +} +#[derive(Debug, Default, Eq, PartialEq, Hash)] +pub struct TraceDataBytes; +impl TraceDataType for TraceDataBytes { + type Data = T::Bytes; +} +#[derive(Debug, Default, Eq, PartialEq, Hash)] +pub struct TraceDataText; +impl TraceDataType for TraceDataText { + type Data = T::Text; +} + +#[derive(Copy, Debug, Default, Eq, PartialEq, Hash)] +#[repr(transparent)] +pub struct TraceDataRef { + index: u32, + _phantom: PhantomData, +} + +impl TraceDataRef { + fn new(r#ref: u32) -> Self { + Self { + index: r#ref, + _phantom: PhantomData, + } + } +} + +pub type TraceStringRef = TraceDataRef; +pub type TraceBytesRef = TraceDataRef; + +struct StaticDataValue { + value: T, + rc: u32, +} + +pub struct StaticDataVec { + vec: Vec>>, + // This HashMap is probably the bottleneck. However we are required to ensure every string only exists once. + table: HashMap, TraceDataRef>, +} + +impl Default for StaticDataVec { + fn default() -> Self { + Self { + vec: vec![StaticDataValue { + value: D::Data::::default(), + rc: 1 << 30, // so that we can just have TraceDataRef::new(0) as default without the rc ever reaching 0 + }], + table: HashMap::from([(D::Data::::default(), TraceDataRef::new(0))]), + } + } +} + +impl StaticDataVec { + pub fn get(&self, r#ref: TraceDataRef) -> &D::Data:: { + &self.vec[r#ref.index as usize].value + } + + pub fn add(&mut self, value: D::Data::) -> TraceDataRef { + if let Some(r#ref) = self.table.get(&value) { + self.vec[r#ref.index as usize].rc += 1; + return *r#ref; + } + let index = self.vec.len() as u32; + self.table.insert(value.clone(), TraceDataRef::new(index)); + self.vec.push(StaticDataValue { + value, + rc: 1, + }); + TraceDataRef::new(index) + } + + pub fn update(&mut self, r#ref: &mut TraceDataRef, value: D::Data::) { + let entry = &mut self.vec[r#ref.index as usize]; + if entry.rc == 1 { + self.table.remove(&entry.value); + self.table.insert(value.clone(), *r#ref); + entry.value = value; + } else { + entry.rc -= 1; + *r#ref = self.add(entry.value); + } + } +} diff --git a/datadog-trace-utils/src/span/trace.rs b/datadog-trace-utils/src/span/trace.rs new file mode 100644 index 0000000000..af308ecf32 --- /dev/null +++ b/datadog-trace-utils/src/span/trace.rs @@ -0,0 +1,869 @@ +use std::marker::PhantomData; +use datadog_trace_protobuf::pb::idx::SpanKind; +use crate::span::TraceData; + +pub trait TraceProjector +where + AttributeArray: AttributeArrayOp, + TraceAttributes: TraceAttributesOp, + TraceAttributes: TraceAttributesOp, + TraceAttributes: TraceAttributesOp, + TraceValue: TraceValueOp, + TraceValue: TraceValueOp, + TraceValue: TraceValueOp, + TraceValue: TraceValueOp, + TraceValue: TraceValueOp, + TraceValue: TraceValueOp, + TraceValue: TraceValueOp, + TraceValue: TraceValueOp, + ChunkValue: TraceValueOp, + ChunkValue: TraceValueOp, + ChunkValue: TraceValueOp, + ChunkValue: TraceValueOp, + ChunkValue: TraceValueOp, + SpanValue: TraceValueOp, + SpanValue: TraceValueOp, + SpanValue: TraceValueOp, + SpanValue: TraceValueOp, + SpanValue: TraceValueOp, + SpanValue: TraceValueOp, + SpanValue: TraceValueOp, + SpanValue: TraceValueOp, + SpanValue: TraceValueOp, + SpanValue: TraceValueOp, + SpanValue: TraceValueOp, + SpanValue: TraceValueOp, + SpanValue: TraceValueOp, + // TODO and mut variants +{ + // Safety note: Not only may it not be transferred across threads, but it also must not do callbacks (to avoid two mut references) + type Storage: ?Send; + type TraceRef; + type ChunkRef; + type SpanRef; + type SpanLinkRef; + type SpanEventRef; + type AttributeRef; + + fn project(&mut self) -> Traces; + + fn chunk_iterator(trace: &Self::TraceRef) -> std::slice::Iter; + fn span_iterator(chunk: &Self::ChunkRef) -> std::slice::Iter; + fn span_link_iterator(span: &Self::SpanRef) -> std::slice::Iter; + fn span_events_iterator(span: &Self::SpanRef) -> std::slice::Iter; +} + +const IMMUT: u8 = 0; +pub const MUT: u8 = 1; + +unsafe fn as_mut(v: &T) -> &mut T { + &mut *(v as *const _ as *mut _) +} + +pub struct TraceValue<'a, T: TraceProjector, D: TraceData, const Type: u8, const Mut: u8 = IMMUT> { + traces: &'a Traces<'a, T, D, Mut>, +} + +pub struct ChunkValue<'a, T: TraceProjector, D: TraceData, const Type: u8, const Mut: u8 = IMMUT> { + storage: &'a T::Storage, + chunk: &'a T::ChunkRef, +} + +impl<'a, T: TraceProjector, D: TraceData, const Type: u8, const Mut: u8> ChunkValue<'a, T, D, Type, MUT> { + pub fn storage(&self) -> &'a T::Storage { + self.storage + } + + pub fn chunk(&self) -> &'a T::ChunkRef { + self.storage + } +} + +impl<'a, T: TraceProjector, D: TraceData, const Type: u8> ChunkValue<'a, T, D, Type, MUT> { + pub fn as_mut(&mut self) -> (&'a mut T::Storage, &'a mut T::ChunkRef) { + // SATEFY: As given by invariants on TraceProjector::Storage / this being MUT + ( + &mut unsafe { *(self.storage as *const _ as *mut _) }, + &mut unsafe { *(self.chunk as *const _ as *mut _) }, + ) + } +} + +pub struct SpanValue<'a, T: TraceProjector, D: TraceData, const Type: u8, const Mut: u8 = IMMUT> { + storage: &'a T::Storage, + span: &'a T::SpanRef, +} + +impl<'a, T: TraceProjector, D: TraceData, const Type: u8> SpanValue<'a, T, D, Type, MUT> { + pub fn mut_storage(&mut self) -> &'a mut T::Storage { + // SATEFY: As given by invariants on TraceProjector::Storage + &mut unsafe { *(self.storage as *const _ as *mut _) } + } + + pub fn mut_span(&mut self) -> &'a mut T::SpanRef { + // SATEFY: Exclusive by this being MUT + &mut unsafe { *(self.span as *const _ as *mut _) } + } +} + +pub struct SpanLinkValue<'a, T: TraceProjector, D: TraceData, const Type: u8, const Mut: u8 = IMMUT> { + storage: &'a T::Storage, + link: &'a T::SpanLinkRef, +} + +impl<'a, T: TraceProjector, D: TraceData, const Type: u8> SpanLinkValue<'a, T, D, Type, MUT> { + pub fn mut_storage(&mut self) -> &'a mut T::Storage { + // SATEFY: As given by invariants on TraceProjector::Storage + &mut unsafe { *(self.storage as *const _ as *mut _) } + } + + pub fn mut_span(&mut self) -> &'a mut T::SpanLinkRef { + // SATEFY: Exclusive by this being MUT + &mut unsafe { *(self.link as *const _ as *mut _) } + } +} + +pub struct SpanEventValue<'a, T: TraceProjector, D: TraceData, const Type: u8, const Mut: u8 = IMMUT> { + storage: &'a T::Storage, + event: &'a T::SpanEventRef, +} + +impl<'a, T: TraceProjector, D: TraceData, const Type: u8> SpanEventValue<'a, T, D, Type, MUT> { + pub fn mut_storage(&mut self) -> &'a mut T::Storage { + // SATEFY: As given by invariants on TraceProjector::Storage + &mut unsafe { *(self.storage as *const _ as *mut _) } + } + + pub fn mut_span(&mut self) -> &'a mut T::SpanRef { + // SATEFY: Exclusive by this being MUT + &mut unsafe { *(self.event as *const _ as *mut _) } + } +} + +pub trait TraceValueDataType { + type Value; +} + +pub trait TraceValueOp: TraceValueDataType { + fn get(storage: &S, chunk: &C) -> Self::Value; +} + +pub trait TraceValueMutOp: TraceValueOp { + fn set(storage: &mut S, chunk: &mut C, value: >::Value); +} + +impl, T: TraceProjector, D: TraceData, const Type: u8, const Mut: u8> SpanValue { + fn get_self(&self) { + TraceValueOp::get(&self.storage, &self.span) + } +} + +impl, T: TraceProjector, D: TraceData, const Type: u8> SpanValue { + fn set_self(&self, value: >::Value) { + unsafe { TraceValueMutOp::set(as_mut(self.storage), as_mut(self.span), value) } + } +} + +pub enum TraceValueType { + ContainerId = 1, + LanguageName = 2, + LanguageVersion = 3, + TracerVersion = 4, + RuntimeId = 5, + Env = 6, + Hostname = 7, + AppVersion = 8, +} + +impl TraceValueDataType for TraceValue { + type Value = String; +} + +#[derive(Debug, Copy, Clone)] +pub struct Traces<'a, T: TraceProjector, D: TraceData, const Mut: u8 = IMMUT> { + storage: &'a T::Storage, // pin? +} +pub type TracesMut<'a, T, D> = Traces<'a, T, D, MUT>; + +impl, D: TraceData, const Mut: u8> Traces { + fn value(&self) -> TraceValue { + TraceValue { + traces: self, + } + } + + pub fn container_id(&self) -> D::Text { + self.value::<{ TraceValueType::ContainerId as u8 }>().get() + } + + pub fn language_name(&self) -> D::Text { + self.value::<{ TraceValueType::LanguageName as u8 }>().get() + } + + pub fn language_version(&self) -> D::Text { + self.value::<{ TraceValueType::LanguageVersion as u8 }>().get() + } + + pub fn tracer_version(&self) -> D::Text { + self.value::<{ TraceValueType::TracerVersion as u8 }>().get() + } + + pub fn runtime_id(&self) -> D::Text { + self.value::<{ TraceValueType::RuntimeId as u8 }>().get() + } + + pub fn env(&self) -> D::Text { + self.value::<{ TraceValueType::Env as u8 }>().get() + } + + pub fn hostname(&self) -> D::Text { + self.value::<{ TraceValueType::Hostname as u8 }>().get() + } + + pub fn app_version(&self) -> D::Text { + self.value::<{ TraceValueType::AppVersion as u8 }>().get() + } + + pub fn attributes(&self) -> TraceAttributes { + TraceAttributes { + storage: self.storage, + container: self, + } + } + + pub fn chunks(&self) -> Nested { + Nested { + storage: self.storage, + container: self, + _phantom: PhantomData, + } + } +} + +impl , D: TraceData> TracesMut<'_, T, D> { + pub fn set_container_id>(&mut self, value: I) { + self.value::<{ TraceValueType::ContainerId as u8 }>().set(value.into()) + } + + pub fn set_language_name>(&mut self, value: I) { + self.value::<{ TraceValueType::LanguageName as u8 }>().set(value.into()) + } + + pub fn set_language_version>(&mut self, value: I) { + self.value::<{ TraceValueType::LanguageVersion as u8 }>().set(value.into()) + } + + pub fn set_tracer_version>(&mut self, value: I) { + self.value::<{ TraceValueType::TracerVersion as u8 }>().set(value.into()) + } + + pub fn set_runtime_id>(&mut self, value: I) { + self.value::<{ TraceValueType::RuntimeId as u8 }>().set(value.into()) + } + + pub fn set_env>(&mut self, value: I) { + self.value::<{ TraceValueType::Env as u8 }>().set(value.into()) + } + + pub fn set_hostname>(&mut self, value: I) { + self.value::<{ TraceValueType::Hostname as u8 }>().set(value.into()) + } + + pub fn set_app_version>(&mut self, value: I) { + self.value::<{ TraceValueType::AppVersion as u8 }>().set(value.into()) + } +} + +pub struct Nested<'a, T: TraceProjector, D: TraceData, C, I, const Mut: u8 = IMMUT> { + storage: &'a T::Storage, + container: &'a C, + _phantom: PhantomData, +} + +struct NestedIterator<'a, T: TraceProjector, D: TraceData, I: Iterator> { + storage: &'a T::Storage, + it: I, +} + +impl, D: TraceData, I: Iterator> Iterator for NestedIterator { + type Item = TraceChunk; + + fn next(&mut self) -> Option { + self.it.next().map(|chunk| { + TraceChunk { + storage: self.storage, + chunk, + } + }) + } +} + +impl<'a, T: TraceProjector, D: TraceData> IntoIterator for &'a mut Nested<'a, T, D, T::TraceRef, T::ChunkRef> { + type Item = TraceChunk; + type IntoIter = NestedIterator<'a, T, D, std::slice::Iter<'a, T::ChunkRef>>; + + fn into_iter(self) -> Self::IntoIter { + NestedIterator { + storage: self.storage, + it: T::chunk_iterator(self.container) + } + } +} + +pub enum ChunkValueType { + Priority = 1, + Origin = 2, + DroppedTrace = 3, + TraceId = 4, + SamplingMechanism = 5, +} + +impl TraceValueDataType for ChunkValue { + type Value = i32; +} + +impl TraceValueDataType for ChunkValue { + type Value = String; +} + +impl TraceValueDataType for ChunkValue { + type Value = bool; +} + +impl TraceValueDataType for ChunkValue { + type Value = u128; +} + +impl TraceValueDataType for ChunkValue { + type Value = u32; +} + +#[derive(Debug, Copy, Clone)] +pub struct TraceChunk, D: TraceData, const Mut: u8 = IMMUT> { + storage: T::Storage, + chunk: T::ChunkRef, +} + +impl, D: TraceData, const Mut: u8> TraceChunk { + fn value(&self) -> ChunkValue { + ChunkValue { + chunk: self.chunk, + storage: self.storage, + } + } + + pub fn priority(&self) -> ChunkValue { + self.value() + } + + pub fn origin(&self) -> ChunkValue { + self.value() + } + + pub fn dropped_trace(&self) -> ChunkValue { + self.value() + } + + pub fn trace_id(&self) -> ChunkValue { + self.value() + } + + pub fn sampling_mechanism(&self) -> ChunkValue { + self.value() + } + + pub fn attributes(&self) -> TraceAttributes { + TraceAttributes { + storage: self.storage, + container: self, + } + } + + pub fn spans(&self) -> Nested { + Nested { + storage: self.storage, + container: self, + _phantom: PhantomData, + } + } +} + + +impl, D: TraceData, I: Iterator> Iterator for NestedIterator { + type Item = TraceChunk; + + fn next(&mut self) -> Option { + self.it.next().map(|chunk| { + TraceChunk { + storage: self.storage, + chunk, + } + }) + } +} + +impl<'a, T: TraceProjector, D: TraceData> IntoIterator for &'a mut Nested { + type Item = TraceChunk; + type IntoIter = NestedIterator<'a, T, D, std::slice::Iter<'a, T::SpanRef>>; + + fn into_iter(self) -> Self::IntoIter { + NestedIterator { + storage: self.storage, + it: T::span_iterator(self.container) + } + } +} + +pub enum SpanValueType { + Service = 1, + Name = 2, + Resource = 3, + Type = 4, + SpanId = 5, + ParentId = 6, + Start = 7, + Duration = 8, + Error = 9, + Env = 10, + Version = 11, + Component = 12, + Kind = 13, +} + +impl TraceValueDataType for SpanValue { + type Value = String; +} + +impl TraceValueDataType for SpanValue { + type Value = String; +} + +impl TraceValueDataType for SpanValue { + type Value = String; +} + +impl TraceValueDataType for SpanValue { + type Value = String; +} + +impl TraceValueDataType for SpanValue { + type Value = u64; +} + +impl TraceValueDataType for SpanValue { + type Value = u64; +} + +impl TraceValueDataType for SpanValue { + type Value = i64; +} + +impl TraceValueDataType for SpanValue { + type Value = i64; +} + +impl TraceValueDataType for SpanValue { + type Value = bool; +} + +impl TraceValueDataType for SpanValue { + type Value = String; +} + +impl TraceValueDataType for SpanValue { + type Value = String; +} + +impl TraceValueDataType for SpanValue { + type Value = String; +} + +impl TraceValueDataType for SpanValue { + type Value = SpanKind; +} + + +/// The generic representation of a V04 span. +/// +/// `T` is the type used to represent strings in the span, it can be either owned (e.g. BytesString) +/// or borrowed (e.g. &str). To define a generic function taking any `Span` you can use the +/// [`SpanValue`] trait: +/// ``` +/// use datadog_trace_utils::span::{Span, SpanText}; +/// fn foo(span: Span) { +/// let _ = span.attributes.get("foo"); +/// } +/// ``` +#[derive(Debug, Copy, Clone)] +pub struct Span, D: TraceData, const Mut: u8 = IMMUT> { + storage: T::Storage, + span: T::SpanRef, +} + +impl, D: TraceData, const Mut: u8> Span { + fn value(&self) -> SpanValue { + SpanValue { + span: self.span, + storage: self.storage, + } + } + + pub fn service(&self) -> SpanValue { + self.value() + } + + pub fn name(&self) -> SpanValue { + self.value() + } + + pub fn resource(&self) -> SpanValue { + self.value() + } + + pub fn r#type(&self) -> SpanValue { + self.value() + } + + pub fn span_id(&self) -> SpanValue { + self.value() + } + + pub fn parent_id(&self) -> SpanValue { + self.value() + } + + pub fn start(&self) -> SpanValue { + self.value() + } + + pub fn duration(&self) -> SpanValue { + self.value() + } + + pub fn error(&self) -> SpanValue { + self.value() + } + + pub fn env(&self) -> SpanValue { + self.value() + } + + pub fn version(&self) -> SpanValue { + self.value() + } + + pub fn component(&self) -> SpanValue { + self.value() + } + + pub fn kind(&self) -> SpanValue { + self.value() + } + + + pub fn attributes(&self) -> TraceAttributes { + TraceAttributes { + storage: self.storage, + container: self, + } + } + + pub fn span_links(&self) -> Nested { + Nested { + storage: self.storage, + container: self, + _phantom: PhantomData, + } + } + + pub fn span_events(&self) -> Nested { + Nested { + storage: self.storage, + container: self, + _phantom: PhantomData, + } + } +} + +pub enum SpanLinkType { + TraceId = 1, + SpanId = 2, + TraceState = 3, + Flags = 4, +} + +impl TraceValueDataType for SpanLinkValue { + type Value = u128; +} + +impl TraceValueDataType for SpanLinkValue { + type Value = u64; +} + +impl TraceValueDataType for SpanLinkValue { + type Value = D::Text; +} + +impl TraceValueDataType for SpanLinkValue { + type Value = u32; +} + + +/// The generic representation of a V04 span link. +/// `T` is the type used to represent strings in the span link. +#[derive(Debug)] +pub struct SpanLink, D: TraceData, const Mut: u8 = IMMUT> { + storage: T::Storage, + link: T::SpanLinkRef, +} + +impl, D: TraceData, const Mut: u8> SpanLink { + fn value(&self) -> SpanLinkValue { + SpanLinkValue { + link: self.link, + storage: self.storage, + } + } + + pub fn trace_id(&self) -> SpanLinkValue { + self.value() + } + + pub fn span_id(&self) -> SpanLinkValue { + self.value() + } + + pub fn trace_state(&self) -> SpanLinkValue { + self.value() + } + + pub fn flags(&self) -> SpanLinkValue { + self.value() + } + + pub fn attributes(&self) -> TraceAttributes { + TraceAttributes { + storage: self.storage, + container: self, + } + } +} + +pub enum SpanEventType { + TimeUnixNano = 1, + Name = 2, +} + +impl TraceValueDataType for SpanEventValue { + type Value = u64; +} + +impl TraceValueDataType for SpanEventValue { + type Value = D::Text; +} + + +/// The generic representation of a V04 span event. +/// `T` is the type used to represent strings in the span event. +#[derive(Debug)] +pub struct SpanEvent, D: TraceData, const Mut: u8 = IMMUT> { + storage: T::Storage, + event: T::SpanEventRef, +} + +impl, D: TraceData, const Mut: u8> SpanEvent { + fn value(&self) -> SpanEventValue { + SpanEventValue { + storage: self.storage, + event: self.event, + } + } + + pub fn time_unix_nano(&self) -> SpanEventValue { + self.value() + } + + pub fn name(&self) -> SpanEventValue { + self.value() + } +} + +enum AttributeAnyValueType { + String = 1, + Bytes = 2, + Boolean = 3, + Integer = 4, + Double = 5, + Array = 6, + Map = 7, +} + +pub struct AttributeInnerValue<'a, T: TraceProjector, D: TraceData, const Type: u8, const Mut: u8 = IMMUT> { + pub storage: &'a mut T::Storage, + pub container: &'a mut T::AttributeRef, +} + +impl TraceValueDataType for AttributeInnerValue { + type Value = D::Text; +} + +impl TraceValueDataType for AttributeInnerValue { + type Value = D::Bytes; +} + +impl TraceValueDataType for AttributeInnerValue { + type Value = bool; +} + +impl TraceValueDataType for AttributeInnerValue { + type Value = i64; +} + +impl TraceValueDataType for AttributeInnerValue { + type Value = f64; +} + +pub struct AttributeArray<'a, T: TraceProjector, D: TraceData, const Mut: u8 = IMMUT> { + storage: &'a T::Storage, + container: &'a T::AttributeRef, +} + +pub trait AttributeArrayOp, D: TraceData>: Iterator> { + fn append(&self) -> AttributeAnyValue; + fn len(&self) -> usize; + fn get(&self, index: usize) -> AttributeAnyValue; +} + +#[derive(Debug, PartialEq)] +pub enum AttributeAnyValue<'a, T: TraceProjector, D: TraceData> { + String(AttributeInnerValue<'a, T, D, { AttributeAnyValueType::String as u8 }>), + Bytes(AttributeInnerValue<'a, T, D, { AttributeAnyValueType::Bytes as u8 }>), + Boolean(AttributeInnerValue<'a, T, D, { AttributeAnyValueType::Boolean as u8 }>), + Integer(AttributeInnerValue<'a, T, D, { AttributeAnyValueType::Integer as u8 }>), + Double(AttributeInnerValue<'a, T, D, { AttributeAnyValueType::Double as u8 }>), + Array(AttributeArray<'a, T, D>), + Map(TraceAttributes<'a, T, D, T::AttributeRef>) +} + +pub struct TraceAttributes<'a, T: TraceProjector, D: TraceData, C, const Mut: u8 = IMMUT> { + storage: &'a T::Storage, + container: &'a C, +} + +trait TraceAttributesOp, D: TraceData> +{ + fn set(&mut self, key: &str, value: AttributeAnyValueType) -> AttributeAnyValue; + fn get(&self, key: &str) -> Option>; + fn remove(&mut self, key: &str); +} + +impl, D: TraceData, C> TraceAttributes where Self: TraceAttributesOp, AttributeInnerValue: TraceValueMutOp +{ + fn set_string(&mut self, key: &str, value: D::Text) { + let AttributeAnyValue::String(inner) = self.set(key, AttributeAnyValueType::String) else { unreachable!() }; + inner.set(value); + } +} + +impl, D: TraceData, C> TraceAttributes where Self: TraceAttributesOp, AttributeInnerValue: TraceValueOp +{ + fn get_string(&self, key: &str) -> Option { + if let Some(AttributeAnyValue::String(inner)) = self.get(key) { + Some(inner.get()) + } else { + None + } + } +} + +impl TraceAttributes where Self: TraceAttributesOp, AttributeInnerValue: TraceValueOp { + fn set_bytes(&mut self, key: &str, value: D::Bytes) { + let AttributeAnyValue::Bytes(inner) = self.set(key, AttributeAnyValueType::Bytes) else { unreachable!() }; + inner.set(value); + } + + fn get_bytes(&self, key: &str) -> Option { + if let Some(AttributeAnyValue::Bytes(inner)) = self.get(key) { + Some(inner.get()) + } else { + None + } + } +} + +impl TraceAttributes where Self: TraceAttributesOp, AttributeInnerValue: TraceValueOp { + + fn set_bool(&mut self, key: &str, value: bool) { + let AttributeAnyValue::Boolean(inner) = self.set(key, AttributeAnyValueType::Boolean) else { unreachable!() }; + inner.set(value); + } + + fn get_bool(&self, key: &str) -> Option { + if let Some(AttributeAnyValue::Boolean(inner)) = self.get(key) { + Some(inner.get()) + } else { + None + } + } +} + +impl TraceAttributes where Self: TraceAttributesOp, AttributeInnerValue: TraceValueOp { + + fn set_int(&mut self, key: &str, value: i64) { + let AttributeAnyValue::Integer(inner) = self.set(key, AttributeAnyValueType::Integer) else { unreachable!() }; + inner.set(value); + } + + fn get_int(&self, key: &str) -> Option { + if let Some(AttributeAnyValue::Integer(inner)) = self.get(key) { + Some(inner.get()) + } else { + None + } + } +} + +impl TraceAttributes where Self: TraceAttributesOp, AttributeInnerValue: TraceValueOp { + + fn set_double(&mut self, key: &str, value: f64) { + let AttributeAnyValue::Double(inner) = self.set(key, AttributeAnyValueType::Double) else { unreachable!() }; + inner.set(value); + } + + fn get_double(&self, key: &str) -> Option { + if let Some(AttributeAnyValue::Double(inner)) = self.get(key) { + Some(inner.get()) + } else { + None + } + } +} + +impl, D: TraceData, C> TraceAttributes where Self: TraceAttributesOp { + + fn set_array(&mut self, key: &str) -> AttributeArray { + let AttributeAnyValue::Array(inner) = self.set(key, AttributeAnyValueType::Array) else { unreachable!() }; + inner + } + + fn get_array(&self, key: &str) -> Option> { + if let Some(AttributeAnyValue::Array(inner)) = self.get(key) { + Some(inner) + } else { + None + } + } + + fn set_map(&mut self, key: &str) -> TraceAttributes { + let AttributeAnyValue::Map(inner) = self.set(key, AttributeAnyValueType::Map) else { unreachable!() }; + inner + } + + fn get_map(&self, key: &str) -> Option> { + if let Some(AttributeAnyValue::Map(inner)) = self.get(key) { + Some(inner) + } else { + None + } + } +} diff --git a/datadog-trace-utils/src/span/trace_utils.rs b/datadog-trace-utils/src/span/trace_utils.rs index 8a0c263903..55bae6c4b1 100644 --- a/datadog-trace-utils/src/span/trace_utils.rs +++ b/datadog-trace-utils/src/span/trace_utils.rs @@ -3,7 +3,7 @@ //! Trace-utils functionalities implementation for tinybytes based spans -use super::{v04::Span, SpanText, TraceData}; +use super::{Span, SpanText, TraceData, TraceProjector, TraceAttributesOp}; use std::collections::HashMap; /// Span metric the mini agent must set for the backend to recognize top level span @@ -13,15 +13,11 @@ const TRACER_TOP_LEVEL_KEY: &str = "_dd.top_level"; const MEASURED_KEY: &str = "_dd.measured"; const PARTIAL_VERSION_KEY: &str = "_dd.partial_version"; -fn set_top_level_span(span: &mut Span, is_top_level: bool) -where - T: TraceData, -{ +fn set_top_level_span, D: TraceData>(span: &mut Span, is_top_level: bool) { if is_top_level { - span.metrics - .insert(T::Text::from_static_str(TOP_LEVEL_KEY), 1.0); + span.attributes().set_double(TOP_LEVEL_KEY, 1.0); } else { - span.metrics.remove(TOP_LEVEL_KEY); + span.attributes().remove(TOP_LEVEL_KEY); } } @@ -31,23 +27,20 @@ where /// - OR its parent is unknown (other part of the code, distributed trace) /// - OR its parent belongs to another service (in that case it's a "local root" being the highest /// ancestor of other spans belonging to this service and attached to it). -pub fn compute_top_level_span(trace: &mut [Span]) -where - T: TraceData, -{ - let mut span_id_to_service: HashMap = HashMap::new(); +pub fn compute_top_level_span, D: TraceData>(trace: &mut [Span]) { + let mut span_id_to_service: HashMap = HashMap::new(); for span in trace.iter() { - span_id_to_service.insert(span.span_id, span.service.clone()); + span_id_to_service.insert(span.span_id().get(), span.service().get().clone()); } for span in trace.iter_mut() { - let parent_id = span.parent_id; + let parent_id = span.parent_id().get(); if parent_id == 0 { set_top_level_span(span, true); continue; } match span_id_to_service.get(&parent_id) { Some(parent_span_service) => { - if !parent_span_service.eq(&span.service) { + if !parent_span_service.eq(&span.service().get()) { // parent is not in the same service set_top_level_span(span, true) } @@ -61,7 +54,7 @@ where } /// Return true if the span has a top level key set -pub fn has_top_level(span: &Span) -> bool { +pub fn has_top_level, D: TraceData>(span: &Span) -> bool { span.metrics .get(TRACER_TOP_LEVEL_KEY) .is_some_and(|v| *v == 1.0) @@ -69,7 +62,7 @@ pub fn has_top_level(span: &Span) -> bool { } /// Returns true if a span should be measured (i.e., it should get trace metrics calculated). -pub fn is_measured(span: &Span) -> bool { +pub fn is_measured, D: TraceData>(span: &Span) -> bool { span.metrics.get(MEASURED_KEY).is_some_and(|v| *v == 1.0) } @@ -78,7 +71,7 @@ pub fn is_measured(span: &Span) -> bool { /// When incomplete, a partial snapshot has a metric _dd.partial_version which is a positive /// integer. The metric usually increases each time a new version of the same span is sent by /// the tracer -pub fn is_partial_snapshot(span: &Span) -> bool { +pub fn is_partial_snapshot, D: TraceData>(span: &Span) -> bool { span.metrics .get(PARTIAL_VERSION_KEY) .is_some_and(|v| *v >= 0.0) @@ -101,7 +94,7 @@ const SAMPLING_ANALYTICS_RATE_KEY: &str = "_dd1.sr.eausr"; /// /// A tuple containing the dropped p0 stats, the first value correspond the amount of traces /// dropped and the latter to the spans dropped. -pub fn drop_chunks(traces: &mut Vec>>) -> DroppedP0Stats +pub fn drop_chunks, D: TraceData>(traces: &mut Vec>>) -> DroppedP0Stats where T: TraceData, { diff --git a/datadog-trace-utils/src/span/v05/mod.rs b/datadog-trace-utils/src/span/v05/mod.rs index c48d92322a..2d70ea3a10 100644 --- a/datadog-trace-utils/src/span/v05/mod.rs +++ b/datadog-trace-utils/src/span/v05/mod.rs @@ -3,35 +3,135 @@ pub mod dict; -use crate::span::{v05::dict::SharedDict, TraceData}; +use crate::span::{v05::dict::SharedDict, TraceData, TraceProjector, TraceValueType, TraceValue, TraceValueOp, Traces, SpanValue, SpanValueType, TraceAttributes, TraceAttributesOp, AttributeAnyValue, AttributeAnyValueType, AttributeInnerValue, MUT, TraceValueMutOp}; use anyhow::Result; use serde::Serialize; use std::collections::HashMap; +use std::ops::Deref; +use std::slice::Iter; +use crate::span::table::{StaticDataVec, TraceDataText, TraceStringRef}; /// Structure that represent a TraceChunk Span which String fields are interned in a shared /// dictionary. The number of elements is fixed by the spec and they all need to be serialized, in /// case of adding more items the constant msgpack_decoder::v05::SPAN_ELEM_COUNT need to be /// updated. -#[derive(Clone, Debug, Default, PartialEq, Serialize)] +#[derive(Debug, Default, PartialEq, Serialize)] pub struct Span { - pub service: u32, - pub name: u32, - pub resource: u32, + pub service: TraceStringRef, + pub name: TraceStringRef, + pub resource: TraceStringRef, pub trace_id: u64, pub span_id: u64, pub parent_id: u64, pub start: i64, pub duration: i64, pub error: i32, - pub meta: HashMap, - pub metrics: HashMap, - pub r#type: u32, + pub meta: HashMap, + pub metrics: HashMap, + pub r#type: TraceStringRef, } +pub struct ChunkCollection { + pub dict: StaticDataVec, + pub chunks: Vec>, +} + +impl<'a, D: TraceData> TraceProjector for &'a ChunkCollection { + type Storage = StaticDataVec; + type TraceRef = Vec>; + type ChunkRef = Vec; + type SpanRef = Span; + type SpanLinkRef = (); + type SpanEventRef = (); + type AttributeRef = Span; + + fn project(self) -> Traces { + Traces { + storage: self, + } + } + + fn chunk_iterator(trace: Self::TraceRef) -> Iter> { + trace.chunks.iter() + } + + fn span_iterator(chunk: Self::ChunkRef) -> Iter { + chunk.iter() + } + + fn span_link_iterator(span: Self::SpanRef) -> Iter<::Target> { + [].iter() + } + + fn span_events_iterator(span: Self::SpanRef) -> Iter<::Target> { + [].iter() + } +} + +impl<'a, D: TraceData> TraceValueOp for TraceValue<&'a mut ChunkCollection, D, { TraceValueType::ContainerId as u8 }> { + fn set>(&self, value: I) { + todo!() + } + + fn get(&self) -> &str { + todo!() + } +} + +impl<'a, D: TraceData> TraceValueMutOp for SpanValue<&'a ChunkCollection, D, { SpanValueType::Service as u8 }, MUT> { + fn set(storage: &mut &'a mut StaticDataVec, span: &'a mut Span, value: D::Text) { + storage.update(&mut span.service, value) + } +} + +impl<'a, D: TraceData> TraceValueOp for SpanValue<&'a ChunkCollection, D, { SpanValueType::Service as u8 }> { + fn get(&self) -> D::Text { + self.storage.get(self.span.service) + } +} + +fn inner_value<'a, D: TraceData, const Type: u8>(span: &'a mut Span, dict: &'a mut StaticDataVec) -> AttributeInnerValue<&'a ChunkCollection, D, Type> { + AttributeInnerValue { + storage: dict, + container: span, + } +} + +impl<'a, D: TraceData> TraceAttributesOp<&'a ChunkCollection, D> for TraceAttributes<&'a ChunkCollection, D, &'a ChunkCollection> { + fn set(&self, key: &str, value: AttributeAnyValueType) -> AttributeAnyValue<&'a ChunkCollection, D> { + let span = &mut self.container.chunks[0][0]; + match value { + AttributeAnyValueType::String => AttributeAnyValue::String(inner_value(span, self.storage)), + AttributeAnyValueType::Bytes => AttributeAnyValue::Bytes(inner_value(span, self.storage)), + AttributeAnyValueType::Boolean => AttributeAnyValue::Boolean(inner_value(span, self.storage)), + AttributeAnyValueType::Integer => AttributeAnyValue::Integer(inner_value(span, self.storage)), + AttributeAnyValueType::Double => AttributeAnyValue::Double(inner_value(span, self.storage)), + AttributeAnyValueType::Array => {} + AttributeAnyValueType::Map => {} + } + } + + fn get(&self, key: &str) -> Option, D>> { + todo!() + } + + fn remove(&self, key: &str) { + todo!() + } +} + +impl, D: TraceData> TraceAttributes { + pub fn set(&self, key: &str, value: AttributeAnyValue) { + self.container.set_attribute(self, key, value); + } +} + + pub fn from_v04_span( span: &crate::span::v04::Span, dict: &mut SharedDict, ) -> Result { + /* Ok(Span { service: dict.get_or_insert(&span.service)?, name: dict.get_or_insert(&span.name)?, @@ -58,6 +158,9 @@ pub fn from_v04_span( )?, r#type: dict.get_or_insert(&span.r#type)?, }) + + */ + Ok(Span::default()) } #[cfg(test)] diff --git a/datadog-trace-utils/src/span/v1/mod.rs b/datadog-trace-utils/src/span/v1/mod.rs new file mode 100644 index 0000000000..983104b7f0 --- /dev/null +++ b/datadog-trace-utils/src/span/v1/mod.rs @@ -0,0 +1,137 @@ +use std::borrow::Borrow; +use std::collections::HashMap; +use datadog_trace_protobuf::pb::idx::SpanKind; +use crate::span::{BytesData, SliceData, TraceData, table::*}; + + + +/// Checks if the `value` represents an empty string. Used to skip serializing empty strings +/// with serde. +fn is_empty_str>(value: &T) -> bool { + value.borrow().is_empty() +} + +#[derive(Default)] +struct TraceStaticData { + pub strings: StaticDataVec, + pub bytes: StaticDataVec, +} + +impl TraceStaticData { + pub fn get_string(&self, r#ref: TraceStringRef) -> &T::Text { + self.strings.get(r#ref) + } + + pub fn add_string(&mut self, value: T::Text) -> TraceStringRef { + self.strings.add(value) + } + + pub fn get_bytes(&self, r#ref: TraceBytesRef) -> &T::Bytes { + self.bytes.get(r#ref) + } + + pub fn add_bytes(&mut self, value: T::Bytes) -> TraceBytesRef { + self.bytes.add(value) + } +} + +#[derive(Default, Debug)] +pub struct Traces { + pub static_data: TraceStaticData, + pub container_id: TraceStringRef, + pub language_name: TraceStringRef, + pub language_version: TraceStringRef, + pub tracer_version: TraceStringRef, + pub runtime_id: TraceStringRef, + pub env: TraceStringRef, + pub hostname: TraceStringRef, + pub app_version: TraceStringRef, + pub attributes: HashMap, + pub chunks: Vec, +} + +#[derive(Debug, Default)] +pub struct TraceChunk { + pub priority: i32, + pub origin: TraceStringRef, + pub attributes: HashMap, + pub spans: Vec, + pub dropped_trace: bool, + pub trace_id: u128, + pub sampling_mechanism: u32, +} + +/// The generic representation of a V04 span. +/// +/// `T` is the type used to represent strings in the span, it can be either owned (e.g. BytesString) +/// or borrowed (e.g. &str). To define a generic function taking any `Span` you can use the +/// [`SpanValue`] trait: +/// ``` +/// use datadog_trace_utils::span::{Span, SpanText}; +/// fn foo(span: Span) { +/// let _ = span.attributes.get("foo"); +/// } +/// ``` +#[derive(Debug, Default, PartialEq)] +pub struct Span { + pub service: TraceStringRef, + pub name: TraceStringRef, + pub resource: TraceStringRef, + #[serde(skip_serializing_if = "is_empty_str")] + pub r#type: TraceStringRef, + pub span_id: u64, + #[serde(skip_serializing_if = "is_default")] + pub parent_id: u64, + pub start: i64, + pub duration: i64, + #[serde(skip_serializing_if = "is_default")] + pub error: bool, + #[serde(skip_serializing_if = "HashMap::is_empty")] + pub attributes: HashMap, + #[serde(skip_serializing_if = "Vec::is_empty")] + pub span_links: Vec, + #[serde(skip_serializing_if = "Vec::is_empty")] + pub span_events: Vec, + pub env: TraceStringRef, + pub version: TraceStringRef, + pub component: TraceStringRef, + pub kind: SpanKind, +} + +/// The generic representation of a V04 span link. +/// `T` is the type used to represent strings in the span link. +#[derive(Debug, Default, PartialEq)] +pub struct SpanLink { + pub trace_id: u128, + pub span_id: u64, + #[serde(skip_serializing_if = "HashMap::is_empty")] + pub attributes: HashMap, + #[serde(skip_serializing_if = "is_empty_str")] + pub tracestate: TraceStringRef, + #[serde(skip_serializing_if = "is_default")] + pub flags: u32, +} + +/// The generic representation of a V04 span event. +/// `T` is the type used to represent strings in the span event. +#[derive(Debug, Default, PartialEq)] +pub struct SpanEvent { + pub time_unix_nano: u64, + pub name: TraceStringRef, + #[serde(skip_serializing_if = "HashMap::is_empty")] + pub attributes: HashMap, +} + +#[derive(Debug, PartialEq)] +pub enum AttributeAnyValue { + String(TraceStringRef), + Bytes(TraceBytesRef), + Boolean(bool), + Integer(i64), + Double(f64), + Array(Vec), + Map(HashMap) +} + +pub type TracesSlice<'a> = Traces>; +pub type TracesBytes = Traces; diff --git a/datadog-trace-utils/src/tracer_payload.rs b/datadog-trace-utils/src/tracer_payload.rs index 43ca1077e1..88224b3395 100644 --- a/datadog-trace-utils/src/tracer_payload.rs +++ b/datadog-trace-utils/src/tracer_payload.rs @@ -1,7 +1,7 @@ // Copyright 2024-Present Datadog, Inc. https://www.datadoghq.com/ // SPDX-License-Identifier: Apache-2.0 -use crate::span::{v04, v05, BytesData, TraceData}; +use crate::span::{v04, v05, v1, BytesData, TraceData}; use crate::trace_utils::collect_trace_chunks; use crate::{msgpack_decoder, trace_utils::cmp_send_data_payloads}; use datadog_trace_protobuf::pb; @@ -19,6 +19,8 @@ pub enum TraceEncoding { V04, /// v0.5 encoding (TracerPayloadV05). V05, + /// v1 encoding (TracerPayload). + V1, } #[derive(Debug)] @@ -27,6 +29,8 @@ pub enum TraceChunks { V04(Vec>>), /// Collection of TraceChunkSpan with de-duplicated strings. V05((Vec, Vec>)), + /// A full V1 trace. + V1(Traces) } impl TraceChunks { @@ -34,6 +38,7 @@ impl TraceChunks { match self { TraceChunks::V04(traces) => TracerPayloadCollection::V04(traces), TraceChunks::V05(traces) => TracerPayloadCollection::V05(traces), + TraceChunks::V1(traces) => TracerPayloadCollection::V1(traces), } } } @@ -44,6 +49,7 @@ impl TraceChunks { match self { TraceChunks::V04(traces) => traces.len(), TraceChunks::V05((_, traces)) => traces.len(), + TraceChunks::V1(traces) => traces.chunks.iter().map(|c| c.spans.len()).sum() } } } @@ -51,6 +57,8 @@ impl TraceChunks { #[derive(Debug)] /// Enum representing a general abstraction for a collection of tracer payloads. pub enum TracerPayloadCollection { + /// Collection of TracerPayloads. + V1(v1::Traces), /// Collection of TracerPayloads. V07(Vec), /// Collection of TraceChunkSpan. @@ -139,6 +147,7 @@ impl TracerPayloadCollection { } TracerPayloadCollection::V04(collection) => collection.len(), TracerPayloadCollection::V05((_, collection)) => collection.len(), + TracerPayloadCollection::V1(traces) => traces.chunks.iter().map(|c| c.spans.len()).sum() } } }