diff --git a/CHANGELOG.md b/CHANGELOG.md index 2664b517584..23a35d383a7 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,6 +2,10 @@ ## Unreleased +**Breaking Changes**: + +- Switch Statsd histogram metrics to distribution metrics. ([#5378](https://github.com/getsentry/relay/pull/5378)) + **Features**: - Support comparing release versions without build code. ([#5376](https://github.com/getsentry/relay/pull/5376)) diff --git a/relay-cardinality/src/redis/limiter.rs b/relay-cardinality/src/redis/limiter.rs index 57d047ab810..4ac3e774b8a 100644 --- a/relay-cardinality/src/redis/limiter.rs +++ b/relay-cardinality/src/redis/limiter.rs @@ -12,7 +12,7 @@ use crate::{ script::{CardinalityScript, CardinalityScriptResult, Status}, state::{LimitState, RedisEntry}, }, - statsd::{CardinalityLimiterHistograms, CardinalityLimiterTimers}, + statsd::{CardinalityLimiterDistributions, CardinalityLimiterTimers}, }; use relay_common::time::UnixTimestamp; @@ -78,7 +78,7 @@ impl RedisSetLimiter { } metric!( - histogram(CardinalityLimiterHistograms::RedisCheckHashes) = num_hashes, + distribution(CardinalityLimiterDistributions::RedisCheckHashes) = num_hashes, id = state.id(), ); @@ -90,7 +90,7 @@ impl RedisSetLimiter { .zip(results) .inspect(|(_, result)| { metric!( - histogram(CardinalityLimiterHistograms::RedisSetCardinality) = + distribution(CardinalityLimiterDistributions::RedisSetCardinality) = result.cardinality as u64, id = state.id(), ); diff --git a/relay-cardinality/src/statsd.rs b/relay-cardinality/src/statsd.rs index 924014cc993..3061e51cb43 100644 --- a/relay-cardinality/src/statsd.rs +++ b/relay-cardinality/src/statsd.rs @@ -1,6 +1,6 @@ use relay_statsd::TimerMetric; #[cfg(feature = "redis")] -use relay_statsd::{CounterMetric, HistogramMetric, SetMetric}; +use relay_statsd::{CounterMetric, DistributionMetric, SetMetric}; /// Counter metrics for the Relay Cardinality Limiter. #[cfg(feature = "redis")] @@ -87,7 +87,7 @@ impl TimerMetric for CardinalityLimiterTimers { } #[cfg(feature = "redis")] -pub enum CardinalityLimiterHistograms { +pub enum CardinalityLimiterDistributions { /// Amount of hashes sent to Redis to check the cardinality. /// /// This metric is tagged with: @@ -103,7 +103,7 @@ pub enum CardinalityLimiterHistograms { } #[cfg(feature = "redis")] -impl HistogramMetric for CardinalityLimiterHistograms { +impl DistributionMetric for CardinalityLimiterDistributions { fn name(&self) -> &'static str { match *self { #[cfg(feature = "redis")] diff --git a/relay-kafka/src/producer/mod.rs b/relay-kafka/src/producer/mod.rs index ce83e30575f..53c0dd1a008 100644 --- a/relay-kafka/src/producer/mod.rs +++ b/relay-kafka/src/producer/mod.rs @@ -18,7 +18,7 @@ use crate::config::{KafkaParams, KafkaTopic}; use crate::debounced::Debounced; use crate::limits::KafkaRateLimits; use crate::producer::utils::KafkaHeaders; -use crate::statsd::{KafkaCounters, KafkaGauges, KafkaHistograms}; +use crate::statsd::{KafkaCounters, KafkaDistributions, KafkaGauges}; mod utils; use utils::{Context, ThreadedProducer}; @@ -217,7 +217,7 @@ impl Producer { let producer_name = producer.context().producer_name(); metric!( - histogram(KafkaHistograms::KafkaMessageSize) = payload.len() as u64, + distribution(KafkaDistributions::KafkaMessageSize) = payload.len() as u64, variant = variant, topic = topic_name, producer_name = producer_name diff --git a/relay-kafka/src/statsd.rs b/relay-kafka/src/statsd.rs index 728f11a6074..890b9ec17c6 100644 --- a/relay-kafka/src/statsd.rs +++ b/relay-kafka/src/statsd.rs @@ -32,7 +32,7 @@ //! # Results in: producer_name="unknown" //! ``` -use relay_statsd::{CounterMetric, GaugeMetric, HistogramMetric}; +use relay_statsd::{CounterMetric, DistributionMetric, GaugeMetric}; pub enum KafkaCounters { /// Number of messages that failed to be enqueued in the Kafka producer's memory buffer. @@ -84,7 +84,7 @@ impl CounterMetric for KafkaCounters { } } -pub enum KafkaHistograms { +pub enum KafkaDistributions { /// Size of emitted kafka message in bytes. /// /// This metric is tagged with: @@ -94,7 +94,7 @@ pub enum KafkaHistograms { KafkaMessageSize, } -impl HistogramMetric for KafkaHistograms { +impl DistributionMetric for KafkaDistributions { fn name(&self) -> &'static str { match self { Self::KafkaMessageSize => "kafka.message_size", diff --git a/relay-server/src/endpoints/common.rs b/relay-server/src/endpoints/common.rs index b723746d97e..83b595c7587 100644 --- a/relay-server/src/endpoints/common.rs +++ b/relay-server/src/endpoints/common.rs @@ -14,7 +14,7 @@ use crate::service::ServiceState; use crate::services::buffer::ProjectKeyPair; use crate::services::outcome::{DiscardItemType, DiscardReason, Outcome}; use crate::services::processor::{BucketSource, MetricData, ProcessMetrics}; -use crate::statsd::{RelayCounters, RelayHistograms}; +use crate::statsd::{RelayCounters, RelayDistributions}; use crate::utils::{self, ApiErrorResponse, FormDataIter}; #[derive(Clone, Copy, Debug, thiserror::Error)] @@ -380,7 +380,7 @@ fn emit_envelope_metrics(envelope: &Envelope) { let is_container = if item.is_container() { "true" } else { "false" }; metric!( - histogram(RelayHistograms::EnvelopeItemSize) = item.payload().len() as u64, + distribution(RelayDistributions::EnvelopeItemSize) = item.payload().len() as u64, item_type = item_type, is_container = is_container, ); diff --git a/relay-server/src/processing/utils/event.rs b/relay-server/src/processing/utils/event.rs index 218368a68a7..28c82b9e0af 100644 --- a/relay-server/src/processing/utils/event.rs +++ b/relay-server/src/processing/utils/event.rs @@ -36,8 +36,8 @@ use crate::envelope::{Envelope, EnvelopeHeaders, Item}; use crate::processing::Context; use crate::services::processor::{MINIMUM_CLOCK_DRIFT, ProcessingError}; use crate::services::projects::project::ProjectInfo; -use crate::statsd::{RelayCounters, RelayHistograms, RelayTimers}; -use crate::utils::{self}; +use crate::statsd::{RelayCounters, RelayDistributions, RelayTimers}; +use crate::utils; /// Returns the data category if there is an event. /// @@ -130,7 +130,7 @@ pub fn finalize<'a>( let span_count = inner_event.spans.value().map(Vec::len).unwrap_or(0) as u64; metric!( - histogram(RelayHistograms::EventSpans) = span_count, + distribution(RelayDistributions::EventSpans) = span_count, sdk = client_name, platform = platform, ); diff --git a/relay-server/src/services/buffer/envelope_buffer/mod.rs b/relay-server/src/services/buffer/envelope_buffer/mod.rs index 7e17472d6a9..30671150338 100644 --- a/relay-server/src/services/buffer/envelope_buffer/mod.rs +++ b/relay-server/src/services/buffer/envelope_buffer/mod.rs @@ -20,7 +20,7 @@ use crate::services::buffer::envelope_store::sqlite::SqliteEnvelopeStoreError; use crate::services::buffer::stack_provider::memory::MemoryStackProvider; use crate::services::buffer::stack_provider::sqlite::SqliteStackProvider; use crate::services::buffer::stack_provider::{StackCreationType, StackProvider}; -use crate::statsd::{RelayGauges, RelayHistograms, RelayTimers}; +use crate::statsd::{RelayDistributions, RelayGauges, RelayTimers}; use crate::utils::MemoryChecker; /// Polymorphic envelope buffering interface. @@ -79,7 +79,7 @@ impl PolymorphicEnvelopeBuffer { /// Adds an envelope to the buffer. pub async fn push(&mut self, envelope: Box) -> Result<(), EnvelopeBufferError> { relay_statsd::metric!( - histogram(RelayHistograms::BufferEnvelopeBodySize) = + distribution(RelayDistributions::BufferEnvelopeBodySize) = envelope.items().map(Item::len).sum::() as u64, partition_id = self.partition_tag() ); @@ -576,7 +576,7 @@ where false => "false", }; relay_statsd::metric!( - histogram(RelayHistograms::BufferEnvelopesCount) = total_count, + distribution(RelayDistributions::BufferEnvelopesCount) = total_count, initialized = initialized, stack_type = self.stack_provider.stack_type(), partition_id = &self.partition_tag diff --git a/relay-server/src/services/buffer/envelope_store/sqlite.rs b/relay-server/src/services/buffer/envelope_store/sqlite.rs index b584064bbaf..6af40b586f0 100644 --- a/relay-server/src/services/buffer/envelope_store/sqlite.rs +++ b/relay-server/src/services/buffer/envelope_store/sqlite.rs @@ -8,7 +8,7 @@ use crate::envelope::EnvelopeError; use crate::Envelope; use crate::services::buffer::common::ProjectKeyPair; -use crate::statsd::{RelayGauges, RelayHistograms, RelayTimers}; +use crate::statsd::{RelayDistributions, RelayGauges, RelayTimers}; use bytes::{Buf, Bytes}; use chrono::{DateTime, Utc}; use futures::stream::StreamExt; @@ -141,7 +141,7 @@ impl<'a> TryFrom<&'a Envelope> for DatabaseEnvelope { let serialized_envelope = value.to_vec()?; relay_statsd::metric!( - histogram(RelayHistograms::BufferEnvelopeSize) = serialized_envelope.len() as u64 + distribution(RelayDistributions::BufferEnvelopeSize) = serialized_envelope.len() as u64 ); let encoded_envelope = @@ -149,7 +149,7 @@ impl<'a> TryFrom<&'a Envelope> for DatabaseEnvelope { zstd::encode_all(serialized_envelope.as_slice(), Self::COMPRESSION_LEVEL)? }); relay_statsd::metric!( - histogram(RelayHistograms::BufferEnvelopeSizeCompressed) = + distribution(RelayDistributions::BufferEnvelopeSizeCompressed) = encoded_envelope.len() as u64 ); diff --git a/relay-server/src/services/processor.rs b/relay-server/src/services/processor.rs index 837758e6793..e12bfbc9f9d 100644 --- a/relay-server/src/services/processor.rs +++ b/relay-server/src/services/processor.rs @@ -67,7 +67,7 @@ use crate::services::projects::project::{ProjectInfo, ProjectState}; use crate::services::upstream::{ SendRequest, Sign, SignatureType, UpstreamRelay, UpstreamRequest, UpstreamRequestError, }; -use crate::statsd::{RelayCounters, RelayHistograms, RelayTimers}; +use crate::statsd::{RelayCounters, RelayDistributions, RelayTimers}; use crate::utils::{self, CheckLimits, EnvelopeLimiter, SamplingResult}; use crate::{http, processing}; use relay_threading::AsyncPool; @@ -2770,7 +2770,7 @@ impl EnvelopeProcessorService { let dsn = PartialDsn::outbound(scoping, upstream); relay_statsd::metric!( - histogram(RelayHistograms::PartitionKeys) = u64::from(partition_key) + distribution(RelayDistributions::PartitionKeys) = u64::from(partition_key) ); let mut num_batches = 0; @@ -2789,14 +2789,16 @@ impl EnvelopeProcessorService { .scope(*scoping); relay_statsd::metric!( - histogram(RelayHistograms::BucketsPerBatch) = batch.len() as u64 + distribution(RelayDistributions::BucketsPerBatch) = batch.len() as u64 ); self.submit_upstream(cogs, Submit::Envelope(envelope.into_processed())); num_batches += 1; } - relay_statsd::metric!(histogram(RelayHistograms::BatchesPerPartition) = num_batches); + relay_statsd::metric!( + distribution(RelayDistributions::BatchesPerPartition) = num_batches + ); } } @@ -2874,7 +2876,7 @@ impl EnvelopeProcessorService { } if partition_splits > 0 { - metric!(histogram(RelayHistograms::PartitionSplits) = partition_splits); + metric!(distribution(RelayDistributions::PartitionSplits) = partition_splits); } self.send_global_partition(partition_key, &mut partition); @@ -3139,7 +3141,9 @@ impl UpstreamRequest for SendEnvelope { fn build(&mut self, builder: &mut http::RequestBuilder) -> Result<(), http::HttpError> { let envelope_body = self.body.clone(); - metric!(histogram(RelayHistograms::UpstreamEnvelopeBodySize) = envelope_body.len() as u64); + metric!( + distribution(RelayDistributions::UpstreamEnvelopeBodySize) = envelope_body.len() as u64 + ); let meta = &self.envelope.meta(); let shard = self.envelope.partition_key().map(|p| p.to_string()); @@ -3355,7 +3359,9 @@ impl UpstreamRequest for SendMetricsRequest { } fn build(&mut self, builder: &mut http::RequestBuilder) -> Result<(), http::HttpError> { - metric!(histogram(RelayHistograms::UpstreamMetricsBodySize) = self.encoded.len() as u64); + metric!( + distribution(RelayDistributions::UpstreamMetricsBodySize) = self.encoded.len() as u64 + ); builder .content_encoding(self.http_encoding) diff --git a/relay-server/src/services/projects/cache/state.rs b/relay-server/src/services/projects/cache/state.rs index 729a3a2230a..2163f17efa5 100644 --- a/relay-server/src/services/projects/cache/state.rs +++ b/relay-server/src/services/projects/cache/state.rs @@ -12,7 +12,7 @@ use relay_statsd::metric; use crate::services::projects::project::{ProjectState, Revision}; use crate::services::projects::source::SourceProjectState; -use crate::statsd::{RelayHistograms, RelayTimers}; +use crate::statsd::{RelayDistributions, RelayTimers}; use crate::utils::{RetryBackoff, UniqueScheduledQueue}; /// The backing storage for a project cache. @@ -90,11 +90,12 @@ impl ProjectStore { }; metric!( - histogram(RelayHistograms::ProjectStateCacheSize) = self.shared.projects.len() as u64, + distribution(RelayDistributions::ProjectStateCacheSize) = + self.shared.projects.len() as u64, storage = "shared" ); metric!( - histogram(RelayHistograms::ProjectStateCacheSize) = self.private.len() as u64, + distribution(RelayDistributions::ProjectStateCacheSize) = self.private.len() as u64, storage = "private" ); diff --git a/relay-server/src/services/projects/source/redis.rs b/relay-server/src/services/projects/source/redis.rs index 4abc11de683..692584ddff9 100644 --- a/relay-server/src/services/projects/source/redis.rs +++ b/relay-server/src/services/projects/source/redis.rs @@ -7,7 +7,7 @@ use std::sync::Arc; use crate::services::projects::project::{ParsedProjectState, ProjectState, Revision}; use crate::services::projects::source::SourceProjectState; -use crate::statsd::{RelayCounters, RelayHistograms, RelayTimers}; +use crate::statsd::{RelayCounters, RelayDistributions, RelayTimers}; use relay_redis::redis::cmd; #[derive(Clone, Debug)] @@ -33,11 +33,11 @@ fn parse_redis_response(raw_response: &[u8]) -> Result { metric!( - histogram(RelayHistograms::ProjectStateSizeBytesCompressed) = + distribution(RelayDistributions::ProjectStateSizeBytesCompressed) = raw_response.len() as f64 ); metric!( - histogram(RelayHistograms::ProjectStateSizeBytesDecompressed) = + distribution(RelayDistributions::ProjectStateSizeBytesDecompressed) = decoded.len() as f64 ); decoded.as_slice() diff --git a/relay-server/src/services/projects/source/upstream.rs b/relay-server/src/services/projects/source/upstream.rs index 0491c41e86b..75900cad476 100644 --- a/relay-server/src/services/projects/source/upstream.rs +++ b/relay-server/src/services/projects/source/upstream.rs @@ -24,7 +24,7 @@ use crate::services::projects::source::{FetchProjectState, SourceProjectState}; use crate::services::upstream::{ Method, RequestPriority, SendQuery, UpstreamQuery, UpstreamRelay, UpstreamRequestError, }; -use crate::statsd::{RelayCounters, RelayHistograms, RelayTimers}; +use crate::statsd::{RelayCounters, RelayDistributions, RelayTimers}; use crate::utils::{RetryBackoff, SleepHandle}; /// A query to retrieve a batch of project states from upstream. @@ -289,7 +289,7 @@ impl UpstreamProjectSourceService { .filter(|(id, channel)| { if channel.expired() { metric!( - histogram(RelayHistograms::ProjectStateAttempts) = channel.attempts, + distribution(RelayDistributions::ProjectStateAttempts) = channel.attempts, result = "timeout", ); metric!( @@ -315,7 +315,10 @@ impl UpstreamProjectSourceService { let total_count = cache_channels.len() + nocache_channels.len(); - metric!(histogram(RelayHistograms::ProjectStatePending) = self.state_channels.len() as u64); + metric!( + distribution(RelayDistributions::ProjectStatePending) = + self.state_channels.len() as u64 + ); relay_log::debug!( "updating project states for {}/{} projects (attempt {})", @@ -373,7 +376,7 @@ impl UpstreamProjectSourceService { } relay_log::debug!("sending request of size {}", channels_batch.len()); metric!( - histogram(RelayHistograms::ProjectStateRequestBatchSize) = + distribution(RelayDistributions::ProjectStateRequestBatchSize) = channels_batch.len() as u64 ); @@ -449,7 +452,7 @@ impl UpstreamProjectSourceService { // Count number of project states returned (via http requests). metric!( - histogram(RelayHistograms::ProjectStateReceived) = + distribution(RelayDistributions::ProjectStateReceived) = response.configs.len() as u64 ); for (key, mut channel) in channels_batch { @@ -484,7 +487,8 @@ impl UpstreamProjectSourceService { }; metric!( - histogram(RelayHistograms::ProjectStateAttempts) = channel.attempts, + distribution(RelayDistributions::ProjectStateAttempts) = + channel.attempts, result = result, ); metric!( @@ -515,7 +519,7 @@ impl UpstreamProjectSourceService { } metric!( - histogram(RelayHistograms::ProjectStatePending) = + distribution(RelayDistributions::ProjectStatePending) = self.state_channels.len() as u64 ); // Put the channels back into the queue, we will retry again shortly. diff --git a/relay-server/src/services/upstream.rs b/relay-server/src/services/upstream.rs index b18a3e1f3a3..140c3a062c8 100644 --- a/relay-server/src/services/upstream.rs +++ b/relay-server/src/services/upstream.rs @@ -34,7 +34,7 @@ use tokio::sync::mpsc; use tokio::time::Instant; use crate::http::{HttpError, Request, RequestBuilder, Response, StatusCode}; -use crate::statsd::{RelayHistograms, RelayTimers}; +use crate::statsd::{RelayDistributions, RelayTimers}; use crate::utils::{self, ApiErrorResponse, RelayErrorAction, RetryBackoff}; /// Rate limits returned by the upstream. @@ -570,7 +570,7 @@ where let body = self.body()?; relay_statsd::metric!( - histogram(RelayHistograms::UpstreamQueryBodySize) = body.len() as u64 + distribution(RelayDistributions::UpstreamQueryBodySize) = body.len() as u64 ); builder @@ -705,7 +705,7 @@ fn emit_response_metrics( ); relay_statsd::metric!( - histogram(RelayHistograms::UpstreamRetries) = entry.retries as u64, + distribution(RelayDistributions::UpstreamRetries) = entry.retries as u64, result = description, status_code = status_str, route = entry.request.route(), @@ -1043,7 +1043,7 @@ impl UpstreamQueue { RequestPriority::Low => self.low.push_back(entry), } relay_statsd::metric!( - histogram(RelayHistograms::UpstreamMessageQueueSize) = self.len() as u64, + distribution(RelayDistributions::UpstreamMessageQueueSize) = self.len() as u64, priority = priority.name(), attempt = "first" ); @@ -1065,7 +1065,7 @@ impl UpstreamQueue { self.next_retry = Instant::now() + self.retry_interval; relay_statsd::metric!( - histogram(RelayHistograms::UpstreamMessageQueueSize) = self.len() as u64, + distribution(RelayDistributions::UpstreamMessageQueueSize) = self.len() as u64, priority = priority.name(), attempt = "retry" ); diff --git a/relay-server/src/statsd.rs b/relay-server/src/statsd.rs index e08d1b0d59d..1302b0a3879 100644 --- a/relay-server/src/statsd.rs +++ b/relay-server/src/statsd.rs @@ -1,4 +1,4 @@ -use relay_statsd::{CounterMetric, GaugeMetric, HistogramMetric, TimerMetric}; +use relay_statsd::{CounterMetric, DistributionMetric, GaugeMetric, TimerMetric}; #[cfg(doc)] use relay_system::RuntimeMetrics; @@ -219,7 +219,7 @@ impl CounterMetric for RuntimeCounters { } /// Histogram metrics used by Relay. -pub enum RelayHistograms { +pub enum RelayDistributions { /// The number of bytes received by Relay for each individual envelope item type. /// /// This metric is tagged with: @@ -336,37 +336,33 @@ pub enum RelayHistograms { PartitionSplits, } -impl HistogramMetric for RelayHistograms { +impl DistributionMetric for RelayDistributions { fn name(&self) -> &'static str { match self { - RelayHistograms::EnvelopeItemSize => "event.item_size", - RelayHistograms::EventSpans => "event.spans", - RelayHistograms::BatchesPerPartition => "metrics.buckets.batches_per_partition", - RelayHistograms::BucketsPerBatch => "metrics.buckets.per_batch", - RelayHistograms::BufferEnvelopesCount => "buffer.envelopes_count", - RelayHistograms::BufferEnvelopeBodySize => "buffer.envelope_body_size", - RelayHistograms::BufferEnvelopeSize => "buffer.envelope_size", - RelayHistograms::BufferEnvelopeSizeCompressed => "buffer.envelope_size.compressed", - RelayHistograms::ProjectStatePending => "project_state.pending", - RelayHistograms::ProjectStateAttempts => "project_state.attempts", - RelayHistograms::ProjectStateRequestBatchSize => "project_state.request.batch_size", - RelayHistograms::ProjectStateReceived => "project_state.received", - RelayHistograms::ProjectStateCacheSize => "project_cache.size", + Self::EnvelopeItemSize => "event.item_size", + Self::EventSpans => "event.spans", + Self::BatchesPerPartition => "metrics.buckets.batches_per_partition", + Self::BucketsPerBatch => "metrics.buckets.per_batch", + Self::BufferEnvelopesCount => "buffer.envelopes_count", + Self::BufferEnvelopeBodySize => "buffer.envelope_body_size", + Self::BufferEnvelopeSize => "buffer.envelope_size", + Self::BufferEnvelopeSizeCompressed => "buffer.envelope_size.compressed", + Self::ProjectStatePending => "project_state.pending", + Self::ProjectStateAttempts => "project_state.attempts", + Self::ProjectStateRequestBatchSize => "project_state.request.batch_size", + Self::ProjectStateReceived => "project_state.received", + Self::ProjectStateCacheSize => "project_cache.size", #[cfg(feature = "processing")] - RelayHistograms::ProjectStateSizeBytesCompressed => { - "project_state.size_bytes.compressed" - } + Self::ProjectStateSizeBytesCompressed => "project_state.size_bytes.compressed", #[cfg(feature = "processing")] - RelayHistograms::ProjectStateSizeBytesDecompressed => { - "project_state.size_bytes.decompressed" - } - RelayHistograms::UpstreamMessageQueueSize => "http_queue.size", - RelayHistograms::UpstreamRetries => "upstream.retries", - RelayHistograms::UpstreamQueryBodySize => "upstream.query.body_size", - RelayHistograms::UpstreamEnvelopeBodySize => "upstream.envelope.body_size", - RelayHistograms::UpstreamMetricsBodySize => "upstream.metrics.body_size", - RelayHistograms::PartitionKeys => "metrics.buckets.partition_keys", - RelayHistograms::PartitionSplits => "partition_splits", + Self::ProjectStateSizeBytesDecompressed => "project_state.size_bytes.decompressed", + Self::UpstreamMessageQueueSize => "http_queue.size", + Self::UpstreamRetries => "upstream.retries", + Self::UpstreamQueryBodySize => "upstream.query.body_size", + Self::UpstreamEnvelopeBodySize => "upstream.envelope.body_size", + Self::UpstreamMetricsBodySize => "upstream.metrics.body_size", + Self::PartitionKeys => "metrics.buckets.partition_keys", + Self::PartitionSplits => "partition_splits", } } } diff --git a/relay-statsd/src/lib.rs b/relay-statsd/src/lib.rs index 97e2727905d..1477220f986 100644 --- a/relay-statsd/src/lib.rs +++ b/relay-statsd/src/lib.rs @@ -3,7 +3,7 @@ //! ## Defining Metrics //! //! In order to use metrics, one needs to first define one of the metric traits on a custom enum. -//! The following types of metrics are available: `counter`, `timer`, `gauge`, `histogram`, and +//! The following types of metrics are available: `counter`, `timer`, `gauge`, `distribution`, and //! `set`. For explanations on what that means see [Metric Types]. //! //! The metric traits serve only to provide a type safe metric name. All metric types have exactly @@ -448,20 +448,19 @@ pub trait CounterMetric { fn name(&self) -> &'static str; } -/// A metric for capturing histograms. +/// A metric for capturing distributions. /// -/// Histograms are values whose distribution is calculated by the server. The distribution -/// calculated for histograms is often similar to that of timers. Histograms can be thought of as a +/// A distribution is often similar to timers. Distributions can be thought of as a /// more general (not limited to timing things) form of timers. /// /// ## Example /// /// ``` -/// use relay_statsd::{metric, HistogramMetric}; +/// use relay_statsd::{metric, DistributionMetric}; /// /// struct QueueSize; /// -/// impl HistogramMetric for QueueSize { +/// impl DistributionMetric for QueueSize { /// fn name(&self) -> &'static str { /// "queue_size" /// } @@ -471,18 +470,18 @@ pub trait CounterMetric { /// let queue = VecDeque::new(); /// # let _hint: &VecDeque<()> = &queue; /// -/// // record a histogram value -/// metric!(histogram(QueueSize) = queue.len() as u64); +/// // record a distribution value +/// metric!(distribution(QueueSize) = queue.len() as u64); /// /// // record with tags /// metric!( -/// histogram(QueueSize) = queue.len() as u64, +/// distribution(QueueSize) = queue.len() as u64, /// server = "server1", /// host = "host1", /// ); /// ``` -pub trait HistogramMetric { - /// Returns the histogram metric name that will be sent to statsd. +pub trait DistributionMetric { + /// Returns the distribution metric name that will be sent to statsd. fn name(&self) -> &'static str; } @@ -615,12 +614,12 @@ macro_rules! metric { }) }; - // histogram - (histogram($id:expr) = $value:expr $(, $($k:ident).* = $v:expr)* $(,)?) => { + // distribution + (distribution($id:expr) = $value:expr $(, $($k:ident).* = $v:expr)* $(,)?) => { $crate::with_client(|client| { use $crate::_pred::*; client.send_metric( - client.histogram_with_tags(&$crate::HistogramMetric::name(&$id), $value) + client.distribution_with_tags(&$crate::DistributionMetric::name(&$id), $value) $(.with_tag(stringify!($($k).*), $v))* ) }) @@ -642,9 +641,9 @@ macro_rules! metric { $crate::with_client(|client| { use $crate::_pred::*; client.send_metric( - // NOTE: cadence histograms support Duration out of the box and converts it to nanos, + // NOTE: cadence distribution support Duration out of the box and converts it to nanos, // but we want milliseconds for historical reasons. - client.histogram_with_tags(&$crate::TimerMetric::name(&$id), $value.as_nanos() as f64 / 1e6) + client.distribution_with_tags(&$crate::TimerMetric::name(&$id), $value.as_nanos() as f64 / 1e6) $(.with_tag(stringify!($($k).*), $v))* ) }) @@ -666,7 +665,7 @@ mod tests { use cadence::{NopMetricSink, StatsdClient}; use crate::{ - CounterMetric, GaugeMetric, HistogramMetric, MetricsClient, SetMetric, TimerMetric, + CounterMetric, DistributionMetric, GaugeMetric, MetricsClient, SetMetric, TimerMetric, set_client, with_capturing_test_client, with_client, }; @@ -692,11 +691,11 @@ mod tests { } } - struct TestHistogram; + struct TestDistribution; - impl HistogramMetric for TestHistogram { + impl DistributionMetric for TestDistribution { fn name(&self) -> &'static str { - "histogram" + "distribution" } } @@ -791,17 +790,17 @@ mod tests { } #[test] - fn test_histogram_tags_with_dots() { + fn test_distribution_tags_with_dots() { let captures = with_capturing_test_client(|| { metric!( - histogram(TestHistogram) = 123, + distribution(TestDistribution) = 123, hc.project_id = "567", server = "server1", ); }); assert_eq!( captures, - ["histogram:123|h|#hc.project_id:567,server:server1"] + ["distribution:123|d|#hc.project_id:567,server:server1"] ); } @@ -829,7 +828,7 @@ mod tests { }); assert_eq!( captures, - ["timer:100000|h|#hc.project_id:567,server:server1"] + ["timer:100000|d|#hc.project_id:567,server:server1"] ); } @@ -846,7 +845,7 @@ mod tests { ) }); // just check the tags to not make this flaky - assert!(captures[0].ends_with("|h|#hc.project_id:567,server:server1")); + assert!(captures[0].ends_with("|d|#hc.project_id:567,server:server1")); } #[test] @@ -857,7 +856,7 @@ mod tests { }); // for "short" durations, precision is preserved: - assert_eq!(captures, ["timer:86400000.000001|h"]); // h is for histogram, not hours + assert_eq!(captures, ["timer:86400000.000001|d"]); let one_year = Duration::from_secs(60 * 60 * 24 * 365); let captures = with_capturing_test_client(|| { @@ -865,6 +864,6 @@ mod tests { }); // for very long durations, precision is lost: - assert_eq!(captures, ["timer:31536000000|h"]); + assert_eq!(captures, ["timer:31536000000|d"]); } } diff --git a/tools/document-metrics/src/main.rs b/tools/document-metrics/src/main.rs index 5ad189053dc..7e3b3ea5f4d 100644 --- a/tools/document-metrics/src/main.rs +++ b/tools/document-metrics/src/main.rs @@ -23,7 +23,7 @@ enum SchemaFormat { enum MetricType { Timer, Counter, - Histogram, + Distribution, Set, Gauge, } @@ -134,8 +134,8 @@ fn get_metric_type(imp: &mut syn::ItemImpl) -> Option { Some(MetricType::Timer) } else if trait_name == "CounterMetric" { Some(MetricType::Counter) - } else if trait_name == "HistogramMetric" { - Some(MetricType::Histogram) + } else if trait_name == "DistributionMetric" { + Some(MetricType::Distribution) } else if trait_name == "SetMetric" { Some(MetricType::Set) } else if trait_name == "GaugeMetric" {