diff --git a/pkg/ccl/changefeedccl/metrics.go b/pkg/ccl/changefeedccl/metrics.go index 4b042273df8a..45578bc6645e 100644 --- a/pkg/ccl/changefeedccl/metrics.go +++ b/pkg/ccl/changefeedccl/metrics.go @@ -28,6 +28,14 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/timeutil" ) +const ( + changefeedCheckpointHistMaxLatency = 30 * time.Second + changefeedBatchHistMaxLatency = 30 * time.Second + changefeedFlushHistMaxLatency = 1 * time.Minute + admitLatencyMaxValue = 1 * time.Minute + commitLatencyMaxValue = 10 * time.Minute +) + // max length for the scope name. const maxSLIScopeNameLen = 128 @@ -488,16 +496,46 @@ func newAggregateMetrics(histogramWindow time.Duration) *AggMetrics { ErrorRetries: b.Counter(metaChangefeedErrorRetries), EmittedMessages: b.Counter(metaChangefeedEmittedMessages), FilteredMessages: b.Counter(metaChangefeedFilteredMessages), - MessageSize: b.Histogram(metaMessageSize, histogramWindow, metric.DataSize16MBBuckets), + MessageSize: b.Histogram(metric.HistogramOptions{ + Metadata: metaMessageSize, + Duration: histogramWindow, + MaxVal: 10 << 20, /* 10MB max message size */ + SigFigs: 1, + Buckets: metric.DataSize16MBBuckets, + }), EmittedBytes: b.Counter(metaChangefeedEmittedBytes), FlushedBytes: b.Counter(metaChangefeedFlushedBytes), Flushes: b.Counter(metaChangefeedFlushes), SizeBasedFlushes: b.Counter(metaSizeBasedFlushes), - BatchHistNanos: b.Histogram(metaChangefeedBatchHistNanos, histogramWindow, metric.BatchProcessLatencyBuckets), - FlushHistNanos: b.Histogram(metaChangefeedFlushHistNanos, histogramWindow, metric.BatchProcessLatencyBuckets), - CommitLatency: b.Histogram(metaCommitLatency, histogramWindow, metric.BatchProcessLatencyBuckets), - AdmitLatency: b.Histogram(metaAdmitLatency, histogramWindow, metric.BatchProcessLatencyBuckets), + BatchHistNanos: b.Histogram(metric.HistogramOptions{ + Metadata: metaChangefeedBatchHistNanos, + Duration: histogramWindow, + MaxVal: changefeedBatchHistMaxLatency.Nanoseconds(), + SigFigs: 1, + Buckets: metric.BatchProcessLatencyBuckets, + }), + FlushHistNanos: b.Histogram(metric.HistogramOptions{ + Metadata: metaChangefeedFlushHistNanos, + Duration: histogramWindow, + MaxVal: changefeedFlushHistMaxLatency.Nanoseconds(), + SigFigs: 2, + Buckets: metric.BatchProcessLatencyBuckets, + }), + CommitLatency: b.Histogram(metric.HistogramOptions{ + Metadata: metaCommitLatency, + Duration: histogramWindow, + MaxVal: commitLatencyMaxValue.Nanoseconds(), + SigFigs: 1, + Buckets: metric.BatchProcessLatencyBuckets, + }), + AdmitLatency: b.Histogram(metric.HistogramOptions{ + Metadata: metaAdmitLatency, + Duration: histogramWindow, + MaxVal: admitLatencyMaxValue.Nanoseconds(), + SigFigs: 1, + Buckets: metric.BatchProcessLatencyBuckets, + }), BackfillCount: b.Gauge(metaChangefeedBackfillCount), BackfillPendingRanges: b.Gauge(metaChangefeedBackfillPendingRanges), RunningCount: b.Gauge(metaChangefeedRunning), @@ -572,12 +610,12 @@ type Metrics struct { Failures *metric.Counter ResolvedMessages *metric.Counter QueueTimeNanos *metric.Counter - CheckpointHistNanos *metric.Histogram + CheckpointHistNanos metric.IHistogram FrontierUpdates *metric.Counter ThrottleMetrics cdcutils.Metrics ReplanCount *metric.Counter - ParallelConsumerFlushNanos *metric.Histogram - ParallelConsumerConsumeNanos *metric.Histogram + ParallelConsumerFlushNanos metric.IHistogram + ParallelConsumerConsumeNanos metric.IHistogram ParallelConsumerInFlightEvents *metric.Gauge mu struct { @@ -599,18 +637,36 @@ func (m *Metrics) getSLIMetrics(scope string) (*sliMetrics, error) { // MakeMetrics makes the metrics for changefeed monitoring. func MakeMetrics(histogramWindow time.Duration) metric.Struct { m := &Metrics{ - AggMetrics: newAggregateMetrics(histogramWindow), - KVFeedMetrics: kvevent.MakeMetrics(histogramWindow), - SchemaFeedMetrics: schemafeed.MakeMetrics(histogramWindow), - ResolvedMessages: metric.NewCounter(metaChangefeedForwardedResolvedMessages), - Failures: metric.NewCounter(metaChangefeedFailures), - QueueTimeNanos: metric.NewCounter(metaEventQueueTime), - CheckpointHistNanos: metric.NewHistogram(metaChangefeedCheckpointHistNanos, histogramWindow, metric.IOLatencyBuckets), - FrontierUpdates: metric.NewCounter(metaChangefeedFrontierUpdates), - ThrottleMetrics: cdcutils.MakeMetrics(histogramWindow), - ReplanCount: metric.NewCounter(metaChangefeedReplanCount), - ParallelConsumerFlushNanos: metric.NewHistogram(metaChangefeedEventConsumerFlushNanos, histogramWindow, metric.IOLatencyBuckets), - ParallelConsumerConsumeNanos: metric.NewHistogram(metaChangefeedEventConsumerConsumeNanos, histogramWindow, metric.IOLatencyBuckets), + AggMetrics: newAggregateMetrics(histogramWindow), + KVFeedMetrics: kvevent.MakeMetrics(histogramWindow), + SchemaFeedMetrics: schemafeed.MakeMetrics(histogramWindow), + ResolvedMessages: metric.NewCounter(metaChangefeedForwardedResolvedMessages), + Failures: metric.NewCounter(metaChangefeedFailures), + QueueTimeNanos: metric.NewCounter(metaEventQueueTime), + CheckpointHistNanos: metric.NewHistogram(metric.HistogramOptions{ + Metadata: metaChangefeedCheckpointHistNanos, + Duration: histogramWindow, + MaxVal: changefeedCheckpointHistMaxLatency.Nanoseconds(), + SigFigs: 2, + Buckets: metric.IOLatencyBuckets, + }), + FrontierUpdates: metric.NewCounter(metaChangefeedFrontierUpdates), + ThrottleMetrics: cdcutils.MakeMetrics(histogramWindow), + ReplanCount: metric.NewCounter(metaChangefeedReplanCount), + // Below two metrics were never implemented using the hdr histogram. Set ForceUsePrometheus + // to true. + ParallelConsumerFlushNanos: metric.NewHistogram(metric.HistogramOptions{ + Metadata: metaChangefeedEventConsumerFlushNanos, + Duration: histogramWindow, + Buckets: metric.IOLatencyBuckets, + Mode: metric.HistogramModePrometheus, + }), + ParallelConsumerConsumeNanos: metric.NewHistogram(metric.HistogramOptions{ + Metadata: metaChangefeedEventConsumerConsumeNanos, + Duration: histogramWindow, + Buckets: metric.IOLatencyBuckets, + Mode: metric.HistogramModePrometheus, + }), ParallelConsumerInFlightEvents: metric.NewGauge(metaChangefeedEventConsumerInFlightEvents), } diff --git a/pkg/ccl/sqlproxyccl/connector.go b/pkg/ccl/sqlproxyccl/connector.go index 8329efecc8ae..6a891a7ab53f 100644 --- a/pkg/ccl/sqlproxyccl/connector.go +++ b/pkg/ccl/sqlproxyccl/connector.go @@ -77,7 +77,7 @@ type connector struct { // DialTenantLatency tracks how long it takes to retrieve the address for // a tenant and set up a tcp connection to the address. - DialTenantLatency *metric.Histogram + DialTenantLatency metric.IHistogram // DialTenantRetries counts how often dialing a tenant is retried. DialTenantRetries *metric.Counter diff --git a/pkg/ccl/sqlproxyccl/connector_test.go b/pkg/ccl/sqlproxyccl/connector_test.go index 8a28955b71a6..6fa21c56afe3 100644 --- a/pkg/ccl/sqlproxyccl/connector_test.go +++ b/pkg/ccl/sqlproxyccl/connector_test.go @@ -380,9 +380,12 @@ func TestConnector_dialTenantCluster(t *testing.T) { c := &connector{ TenantID: roachpb.MustMakeTenantID(42), - DialTenantLatency: metric.NewHistogram( - metaDialTenantLatency, time.Millisecond, metric.NetworkLatencyBuckets, - ), + DialTenantLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePrometheus, + Metadata: metaDialTenantLatency, + Duration: time.Millisecond, + Buckets: metric.NetworkLatencyBuckets, + }), DialTenantRetries: metric.NewCounter(metaDialTenantRetries), } dc := &testTenantDirectoryCache{} @@ -460,9 +463,12 @@ func TestConnector_dialTenantCluster(t *testing.T) { defer cancel() c := &connector{ - DialTenantLatency: metric.NewHistogram( - metaDialTenantLatency, time.Millisecond, metric.NetworkLatencyBuckets, - ), + DialTenantLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaDialTenantLatency, + Duration: time.Millisecond, + Buckets: metric.NetworkLatencyBuckets, + }), DialTenantRetries: metric.NewCounter(metaDialTenantRetries), } c.testingKnobs.lookupAddr = func(ctx context.Context) (string, error) { @@ -491,9 +497,12 @@ func TestConnector_dialTenantCluster(t *testing.T) { var reportFailureFnCount int c := &connector{ TenantID: roachpb.MustMakeTenantID(42), - DialTenantLatency: metric.NewHistogram( - metaDialTenantLatency, time.Millisecond, metric.NetworkLatencyBuckets, - ), + DialTenantLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaDialTenantLatency, + Duration: time.Millisecond, + Buckets: metric.NetworkLatencyBuckets, + }), DialTenantRetries: metric.NewCounter(metaDialTenantRetries), } c.DirectoryCache = &testTenantDirectoryCache{ diff --git a/pkg/ccl/sqlproxyccl/metrics.go b/pkg/ccl/sqlproxyccl/metrics.go index 2fe0d8e16131..2432f9b1f668 100644 --- a/pkg/ccl/sqlproxyccl/metrics.go +++ b/pkg/ccl/sqlproxyccl/metrics.go @@ -23,19 +23,19 @@ type metrics struct { RoutingErrCount *metric.Counter RefusedConnCount *metric.Counter SuccessfulConnCount *metric.Counter - ConnectionLatency *metric.Histogram + ConnectionLatency metric.IHistogram AuthFailedCount *metric.Counter ExpiredClientConnCount *metric.Counter - DialTenantLatency *metric.Histogram + DialTenantLatency metric.IHistogram DialTenantRetries *metric.Counter ConnMigrationSuccessCount *metric.Counter ConnMigrationErrorFatalCount *metric.Counter ConnMigrationErrorRecoverableCount *metric.Counter ConnMigrationAttemptedCount *metric.Counter - ConnMigrationAttemptedLatency *metric.Histogram - ConnMigrationTransferResponseMessageSize *metric.Histogram + ConnMigrationAttemptedLatency metric.IHistogram + ConnMigrationTransferResponseMessageSize metric.IHistogram QueryCancelReceivedPGWire *metric.Counter QueryCancelReceivedHTTP *metric.Counter @@ -49,6 +49,16 @@ func (metrics) MetricStruct() {} var _ metric.Struct = metrics{} +const ( + // maxExpectedTransferResponseMessageSize corresponds to maximum expected + // response message size for the SHOW TRANSFER STATE query. We choose 16MB + // here to match the defaultMaxReadBufferSize used for ingesting SQL + // statements in the SQL server (see pkg/sql/pgwire/pgwirebase/encoding.go). + // + // This will be used to tune sql.session_transfer.max_session_size. + maxExpectedTransferResponseMessageSize = 1 << 24 // 16MB +) + var ( metaCurConnCount = metric.Metadata{ Name: "proxy.sql.conns", @@ -213,18 +223,20 @@ func makeProxyMetrics() metrics { RoutingErrCount: metric.NewCounter(metaRoutingErrCount), RefusedConnCount: metric.NewCounter(metaRefusedConnCount), SuccessfulConnCount: metric.NewCounter(metaSuccessfulConnCount), - ConnectionLatency: metric.NewHistogram( - metaConnMigrationAttemptedCount, - base.DefaultHistogramWindowInterval(), - metric.NetworkLatencyBuckets, - ), + ConnectionLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaConnMigrationAttemptedCount, + Duration: base.DefaultHistogramWindowInterval(), + Buckets: metric.NetworkLatencyBuckets, + }), AuthFailedCount: metric.NewCounter(metaAuthFailedCount), ExpiredClientConnCount: metric.NewCounter(metaExpiredClientConnCount), // Connector metrics. - DialTenantLatency: metric.NewHistogram( - metaDialTenantLatency, - base.DefaultHistogramWindowInterval(), - metric.NetworkLatencyBuckets, + DialTenantLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaDialTenantLatency, + Duration: base.DefaultHistogramWindowInterval(), + Buckets: metric.NetworkLatencyBuckets}, ), DialTenantRetries: metric.NewCounter(metaDialTenantRetries), // Connection migration metrics. @@ -232,16 +244,19 @@ func makeProxyMetrics() metrics { ConnMigrationErrorFatalCount: metric.NewCounter(metaConnMigrationErrorFatalCount), ConnMigrationErrorRecoverableCount: metric.NewCounter(metaConnMigrationErrorRecoverableCount), ConnMigrationAttemptedCount: metric.NewCounter(metaConnMigrationAttemptedCount), - ConnMigrationAttemptedLatency: metric.NewHistogram( - metaConnMigrationAttemptedLatency, - base.DefaultHistogramWindowInterval(), - metric.NetworkLatencyBuckets, - ), - ConnMigrationTransferResponseMessageSize: metric.NewHistogram( - metaConnMigrationTransferResponseMessageSize, - base.DefaultHistogramWindowInterval(), - metric.DataSize16MBBuckets, - ), + ConnMigrationAttemptedLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaConnMigrationAttemptedLatency, + Duration: base.DefaultHistogramWindowInterval(), + Buckets: metric.NetworkLatencyBuckets, + }), + ConnMigrationTransferResponseMessageSize: metric.NewHistogram(metric.HistogramOptions{ + Metadata: metaConnMigrationTransferResponseMessageSize, + Duration: base.DefaultHistogramWindowInterval(), + Buckets: metric.DataSize16MBBuckets, + MaxVal: maxExpectedTransferResponseMessageSize, + SigFigs: 1, + }), QueryCancelReceivedPGWire: metric.NewCounter(metaQueryCancelReceivedPGWire), QueryCancelReceivedHTTP: metric.NewCounter(metaQueryCancelReceivedHTTP), QueryCancelIgnored: metric.NewCounter(metaQueryCancelIgnored), diff --git a/pkg/ccl/streamingccl/streamingest/metrics.go b/pkg/ccl/streamingccl/streamingest/metrics.go index f9f28f38a247..c2858cd5ad36 100644 --- a/pkg/ccl/streamingccl/streamingest/metrics.go +++ b/pkg/ccl/streamingccl/streamingest/metrics.go @@ -15,6 +15,12 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/metric" ) +const ( + streamingFlushHistMaxLatency = 1 * time.Minute + streamingAdmitLatencyMaxValue = 3 * time.Minute + streamingCommitLatencyMaxValue = 10 * time.Minute +) + var ( metaReplicationEventsIngested = metric.Metadata{ Name: "replication.events_ingested", @@ -120,9 +126,9 @@ type Metrics struct { Flushes *metric.Counter JobProgressUpdates *metric.Counter ResolvedEvents *metric.Counter - FlushHistNanos *metric.Histogram - CommitLatency *metric.Histogram - AdmitLatency *metric.Histogram + FlushHistNanos metric.IHistogram + CommitLatency metric.IHistogram + AdmitLatency metric.IHistogram RunningCount *metric.Gauge EarliestDataCheckpointSpan *metric.Gauge LatestDataCheckpointSpan *metric.Gauge @@ -143,12 +149,30 @@ func MakeMetrics(histogramWindow time.Duration) metric.Struct { Flushes: metric.NewCounter(metaReplicationFlushes), ResolvedEvents: metric.NewCounter(metaReplicationResolvedEventsIngested), JobProgressUpdates: metric.NewCounter(metaJobProgressUpdates), - FlushHistNanos: metric.NewHistogram(metaReplicationFlushHistNanos, - histogramWindow, metric.BatchProcessLatencyBuckets), - CommitLatency: metric.NewHistogram(metaReplicationCommitLatency, - histogramWindow, metric.BatchProcessLatencyBuckets), - AdmitLatency: metric.NewHistogram(metaReplicationAdmitLatency, - histogramWindow, metric.BatchProcessLatencyBuckets), + FlushHistNanos: metric.NewHistogram(metric.HistogramOptions{ + Metadata: metaReplicationFlushHistNanos, + Duration: histogramWindow, + Buckets: metric.BatchProcessLatencyBuckets, + MaxVal: streamingFlushHistMaxLatency.Nanoseconds(), + SigFigs: 1, + Mode: metric.HistogramModePreferHdrLatency, + }), + CommitLatency: metric.NewHistogram(metric.HistogramOptions{ + Metadata: metaReplicationCommitLatency, + Duration: histogramWindow, + Buckets: metric.BatchProcessLatencyBuckets, + MaxVal: streamingCommitLatencyMaxValue.Nanoseconds(), + SigFigs: 1, + Mode: metric.HistogramModePreferHdrLatency, + }), + AdmitLatency: metric.NewHistogram(metric.HistogramOptions{ + Metadata: metaReplicationAdmitLatency, + Duration: histogramWindow, + Buckets: metric.BatchProcessLatencyBuckets, + MaxVal: streamingAdmitLatencyMaxValue.Nanoseconds(), + SigFigs: 1, + Mode: metric.HistogramModePreferHdrLatency, + }), RunningCount: metric.NewGauge(metaStreamsRunning), EarliestDataCheckpointSpan: metric.NewGauge(metaEarliestDataCheckpointSpan), LatestDataCheckpointSpan: metric.NewGauge(metaLatestDataCheckpointSpan), diff --git a/pkg/kv/bulk/bulk_metrics.go b/pkg/kv/bulk/bulk_metrics.go index f3390d54733e..7cbbc748a20b 100644 --- a/pkg/kv/bulk/bulk_metrics.go +++ b/pkg/kv/bulk/bulk_metrics.go @@ -20,7 +20,7 @@ import ( // Metrics contains pointers to the metrics for // monitoring bulk operations. type Metrics struct { - MaxBytesHist *metric.Histogram + MaxBytesHist metric.IHistogram CurBytesCount *metric.Gauge } @@ -44,10 +44,20 @@ var ( } ) +// See pkg/sql/mem_metrics.go +// log10int64times1000 = log10(math.MaxInt64) * 1000, rounded up somewhat +const log10int64times1000 = 19 * 1000 + // MakeBulkMetrics instantiates the metrics holder for bulk operation monitoring. func MakeBulkMetrics(histogramWindow time.Duration) Metrics { return Metrics{ - MaxBytesHist: metric.NewHistogram(metaMemMaxBytes, histogramWindow, metric.MemoryUsage64MBBuckets), + MaxBytesHist: metric.NewHistogram(metric.HistogramOptions{ + Metadata: metaMemMaxBytes, + Duration: histogramWindow, + MaxVal: log10int64times1000, + SigFigs: 3, + Buckets: metric.MemoryUsage64MBBuckets, + }), CurBytesCount: metric.NewGauge(metaMemCurBytes), } } diff --git a/pkg/kv/kvclient/kvcoord/txn_metrics.go b/pkg/kv/kvclient/kvcoord/txn_metrics.go index eb6313012717..fcca64aa74ae 100644 --- a/pkg/kv/kvclient/kvcoord/txn_metrics.go +++ b/pkg/kv/kvclient/kvcoord/txn_metrics.go @@ -31,14 +31,14 @@ type TxnMetrics struct { RefreshMemoryLimitExceeded *metric.Counter RefreshAutoRetries *metric.Counter - Durations *metric.Histogram + Durations metric.IHistogram TxnsWithCondensedIntents *metric.Counter TxnsWithCondensedIntentsGauge *metric.Gauge TxnsRejectedByLockSpanBudget *metric.Counter // Restarts is the number of times we had to restart the transaction. - Restarts *metric.Histogram + Restarts metric.IHistogram // Counts of restart types. RestartsWriteTooOld telemetry.CounterWithMetric @@ -264,21 +264,32 @@ var ( // windowed portions retain data for approximately histogramWindow. func MakeTxnMetrics(histogramWindow time.Duration) TxnMetrics { return TxnMetrics{ - Aborts: metric.NewCounter(metaAbortsRates), - Commits: metric.NewCounter(metaCommitsRates), - Commits1PC: metric.NewCounter(metaCommits1PCRates), - ParallelCommits: metric.NewCounter(metaParallelCommitsRates), - CommitWaits: metric.NewCounter(metaCommitWaitCount), - RefreshSuccess: metric.NewCounter(metaRefreshSuccess), - RefreshFail: metric.NewCounter(metaRefreshFail), - RefreshFailWithCondensedSpans: metric.NewCounter(metaRefreshFailWithCondensedSpans), - RefreshMemoryLimitExceeded: metric.NewCounter(metaRefreshMemoryLimitExceeded), - RefreshAutoRetries: metric.NewCounter(metaRefreshAutoRetries), - Durations: metric.NewHistogram(metaDurationsHistograms, histogramWindow, metric.IOLatencyBuckets), - TxnsWithCondensedIntents: metric.NewCounter(metaTxnsWithCondensedIntentSpans), - TxnsWithCondensedIntentsGauge: metric.NewGauge(metaTxnsWithCondensedIntentSpansGauge), - TxnsRejectedByLockSpanBudget: metric.NewCounter(metaTxnsRejectedByLockSpanBudget), - Restarts: metric.NewHistogram(metaRestartsHistogram, histogramWindow, metric.Count1KBuckets), + Aborts: metric.NewCounter(metaAbortsRates), + Commits: metric.NewCounter(metaCommitsRates), + Commits1PC: metric.NewCounter(metaCommits1PCRates), + ParallelCommits: metric.NewCounter(metaParallelCommitsRates), + CommitWaits: metric.NewCounter(metaCommitWaitCount), + RefreshSuccess: metric.NewCounter(metaRefreshSuccess), + RefreshFail: metric.NewCounter(metaRefreshFail), + RefreshFailWithCondensedSpans: metric.NewCounter(metaRefreshFailWithCondensedSpans), + RefreshMemoryLimitExceeded: metric.NewCounter(metaRefreshMemoryLimitExceeded), + RefreshAutoRetries: metric.NewCounter(metaRefreshAutoRetries), + Durations: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaDurationsHistograms, + Duration: histogramWindow, + Buckets: metric.IOLatencyBuckets, + }), + TxnsWithCondensedIntents: metric.NewCounter(metaTxnsWithCondensedIntentSpans), + TxnsWithCondensedIntentsGauge: metric.NewGauge(metaTxnsWithCondensedIntentSpansGauge), + TxnsRejectedByLockSpanBudget: metric.NewCounter(metaTxnsRejectedByLockSpanBudget), + Restarts: metric.NewHistogram(metric.HistogramOptions{ + Metadata: metaRestartsHistogram, + Duration: histogramWindow, + MaxVal: 100, + SigFigs: 3, + Buckets: metric.Count1KBuckets, + }), RestartsWriteTooOld: telemetry.NewCounterWithMetric(metaRestartsWriteTooOld), RestartsWriteTooOldMulti: telemetry.NewCounterWithMetric(metaRestartsWriteTooOldMulti), RestartsSerializable: telemetry.NewCounterWithMetric(metaRestartsSerializable), diff --git a/pkg/kv/kvprober/kvprober.go b/pkg/kv/kvprober/kvprober.go index 2e0d84529fc9..0ec07006efe1 100644 --- a/pkg/kv/kvprober/kvprober.go +++ b/pkg/kv/kvprober/kvprober.go @@ -140,10 +140,10 @@ var ( type Metrics struct { ReadProbeAttempts *metric.Counter ReadProbeFailures *metric.Counter - ReadProbeLatency *metric.Histogram + ReadProbeLatency metric.IHistogram WriteProbeAttempts *metric.Counter WriteProbeFailures *metric.Counter - WriteProbeLatency *metric.Histogram + WriteProbeLatency metric.IHistogram WriteProbeQuarantineOldestDuration *metric.Gauge ProbePlanAttempts *metric.Counter ProbePlanFailures *metric.Counter @@ -229,14 +229,20 @@ func NewProber(opts Opts) *Prober { metrics: Metrics{ ReadProbeAttempts: metric.NewCounter(metaReadProbeAttempts), ReadProbeFailures: metric.NewCounter(metaReadProbeFailures), - ReadProbeLatency: metric.NewHistogram( - metaReadProbeLatency, opts.HistogramWindowInterval, metric.NetworkLatencyBuckets, - ), + ReadProbeLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaReadProbeLatency, + Duration: opts.HistogramWindowInterval, + Buckets: metric.NetworkLatencyBuckets, + }), WriteProbeAttempts: metric.NewCounter(metaWriteProbeAttempts), WriteProbeFailures: metric.NewCounter(metaWriteProbeFailures), - WriteProbeLatency: metric.NewHistogram( - metaWriteProbeLatency, opts.HistogramWindowInterval, metric.NetworkLatencyBuckets, - ), + WriteProbeLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaWriteProbeLatency, + Duration: opts.HistogramWindowInterval, + Buckets: metric.NetworkLatencyBuckets, + }), WriteProbeQuarantineOldestDuration: metric.NewFunctionalGauge( metaWriteProbeQuarantineOldestDuration, func() int64 { return qPool.oldestDuration().Nanoseconds() }, diff --git a/pkg/kv/kvserver/liveness/liveness.go b/pkg/kv/kvserver/liveness/liveness.go index 3411ea1bbd78..a11f3abc4114 100644 --- a/pkg/kv/kvserver/liveness/liveness.go +++ b/pkg/kv/kvserver/liveness/liveness.go @@ -144,7 +144,7 @@ type Metrics struct { HeartbeatSuccesses *metric.Counter HeartbeatFailures telemetry.CounterWithMetric EpochIncrements telemetry.CounterWithMetric - HeartbeatLatency *metric.Histogram + HeartbeatLatency metric.IHistogram } // IsLiveCallback is invoked when a node's IsLive state changes to true. @@ -310,9 +310,12 @@ func NewNodeLiveness(opts NodeLivenessOptions) *NodeLiveness { HeartbeatSuccesses: metric.NewCounter(metaHeartbeatSuccesses), HeartbeatFailures: telemetry.NewCounterWithMetric(metaHeartbeatFailures), EpochIncrements: telemetry.NewCounterWithMetric(metaEpochIncrements), - HeartbeatLatency: metric.NewHistogram( - metaHeartbeatLatency, opts.HistogramWindowInterval, metric.NetworkLatencyBuckets, - ), + HeartbeatLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaHeartbeatLatency, + Duration: opts.HistogramWindowInterval, + Buckets: metric.NetworkLatencyBuckets, + }), } nl.mu.nodes = make(map[roachpb.NodeID]Record) nl.heartbeatToken <- struct{}{} diff --git a/pkg/kv/kvserver/logstore/logstore.go b/pkg/kv/kvserver/logstore/logstore.go index f1784431b0f5..e1cd72d34483 100644 --- a/pkg/kv/kvserver/logstore/logstore.go +++ b/pkg/kv/kvserver/logstore/logstore.go @@ -90,7 +90,7 @@ type AppendStats struct { // Metrics contains metrics specific to the log storage. type Metrics struct { - RaftLogCommitLatency *metric.Histogram + RaftLogCommitLatency metric.IHistogram } // LogStore is a stub of a separated Raft log storage. diff --git a/pkg/kv/kvserver/logstore/logstore_bench_test.go b/pkg/kv/kvserver/logstore/logstore_bench_test.go index 0c3f8e0473e2..f09472e27385 100644 --- a/pkg/kv/kvserver/logstore/logstore_bench_test.go +++ b/pkg/kv/kvserver/logstore/logstore_bench_test.go @@ -60,7 +60,12 @@ func runBenchmarkLogStore_StoreEntries(b *testing.B, bytes int64) { EntryCache: ec, Settings: cluster.MakeTestingClusterSettings(), Metrics: Metrics{ - RaftLogCommitLatency: metric.NewHistogram(metric.Metadata{}, 10*time.Second, metric.IOLatencyBuckets), + RaftLogCommitLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePrometheus, + Metadata: metric.Metadata{}, + Duration: 10 * time.Second, + Buckets: metric.IOLatencyBuckets, + }), }, } diff --git a/pkg/kv/kvserver/metrics.go b/pkg/kv/kvserver/metrics.go index 38953afe5d50..5476a6c17e9a 100644 --- a/pkg/kv/kvserver/metrics.go +++ b/pkg/kv/kvserver/metrics.go @@ -1850,15 +1850,15 @@ type StoreMetrics struct { // Raft processing metrics. RaftTicks *metric.Counter - RaftQuotaPoolPercentUsed *metric.Histogram + RaftQuotaPoolPercentUsed metric.IHistogram RaftWorkingDurationNanos *metric.Counter RaftTickingDurationNanos *metric.Counter RaftCommandsApplied *metric.Counter - RaftLogCommitLatency *metric.Histogram - RaftCommandCommitLatency *metric.Histogram - RaftHandleReadyLatency *metric.Histogram - RaftApplyCommittedLatency *metric.Histogram - RaftSchedulerLatency *metric.Histogram + RaftLogCommitLatency metric.IHistogram + RaftCommandCommitLatency metric.IHistogram + RaftHandleReadyLatency metric.IHistogram + RaftApplyCommittedLatency metric.IHistogram + RaftSchedulerLatency metric.IHistogram RaftTimeoutCampaign *metric.Counter // Raft message metrics. @@ -1990,8 +1990,8 @@ type StoreMetrics struct { ReplicaCircuitBreakerCumTripped *metric.Counter // Replica batch evaluation metrics. - ReplicaReadBatchEvaluationLatency *metric.Histogram - ReplicaWriteBatchEvaluationLatency *metric.Histogram + ReplicaReadBatchEvaluationLatency metric.IHistogram + ReplicaWriteBatchEvaluationLatency metric.IHistogram ReplicaReadBatchDroppedLatchesBeforeEval *metric.Counter ReplicaReadBatchWithoutInterleavingIter *metric.Counter @@ -2377,27 +2377,46 @@ func newStoreMetrics(histogramWindow time.Duration) *StoreMetrics { // Raft processing metrics. RaftTicks: metric.NewCounter(metaRaftTicks), - RaftQuotaPoolPercentUsed: metric.NewHistogram( - metaRaftQuotaPoolPercentUsed, histogramWindow, metric.Percent100Buckets, - ), + RaftQuotaPoolPercentUsed: metric.NewHistogram(metric.HistogramOptions{ + Metadata: metaRaftQuotaPoolPercentUsed, + Duration: histogramWindow, + MaxVal: 100, + SigFigs: 1, + Buckets: metric.Percent100Buckets, + }), RaftWorkingDurationNanos: metric.NewCounter(metaRaftWorkingDurationNanos), RaftTickingDurationNanos: metric.NewCounter(metaRaftTickingDurationNanos), RaftCommandsApplied: metric.NewCounter(metaRaftCommandsApplied), - RaftLogCommitLatency: metric.NewHistogram( - metaRaftLogCommitLatency, histogramWindow, metric.IOLatencyBuckets, - ), - RaftCommandCommitLatency: metric.NewHistogram( - metaRaftCommandCommitLatency, histogramWindow, metric.IOLatencyBuckets, - ), - RaftHandleReadyLatency: metric.NewHistogram( - metaRaftHandleReadyLatency, histogramWindow, metric.IOLatencyBuckets, - ), - RaftApplyCommittedLatency: metric.NewHistogram( - metaRaftApplyCommittedLatency, histogramWindow, metric.IOLatencyBuckets, - ), - RaftSchedulerLatency: metric.NewHistogram( - metaRaftSchedulerLatency, histogramWindow, metric.IOLatencyBuckets, - ), + RaftLogCommitLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaRaftLogCommitLatency, + Duration: histogramWindow, + Buckets: metric.IOLatencyBuckets, + }), + RaftCommandCommitLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaRaftCommandCommitLatency, + Duration: histogramWindow, + Buckets: metric.IOLatencyBuckets, + }), + RaftHandleReadyLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaRaftHandleReadyLatency, + Duration: histogramWindow, + Buckets: metric.IOLatencyBuckets, + }), + RaftApplyCommittedLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaRaftApplyCommittedLatency, + Duration: histogramWindow, + Buckets: metric.IOLatencyBuckets, + }), + RaftSchedulerLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaRaftSchedulerLatency, + Duration: histogramWindow, + Buckets: metric.IOLatencyBuckets, + }), RaftTimeoutCampaign: metric.NewCounter(metaRaftTimeoutCampaign), // Raft message metrics. @@ -2538,12 +2557,18 @@ func newStoreMetrics(histogramWindow time.Duration) *StoreMetrics { ReplicaCircuitBreakerCumTripped: metric.NewCounter(metaReplicaCircuitBreakerCumTripped), // Replica batch evaluation. - ReplicaReadBatchEvaluationLatency: metric.NewHistogram( - metaReplicaReadBatchEvaluationLatency, histogramWindow, metric.IOLatencyBuckets, - ), - ReplicaWriteBatchEvaluationLatency: metric.NewHistogram( - metaReplicaWriteBatchEvaluationLatency, histogramWindow, metric.IOLatencyBuckets, - ), + ReplicaReadBatchEvaluationLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaReplicaReadBatchEvaluationLatency, + Duration: histogramWindow, + Buckets: metric.IOLatencyBuckets, + }), + ReplicaWriteBatchEvaluationLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaReplicaWriteBatchEvaluationLatency, + Duration: histogramWindow, + Buckets: metric.IOLatencyBuckets, + }), FlushUtilization: metric.NewGaugeFloat64(metaStorageFlushUtilization), FsyncLatency: metric.NewManualWindowHistogram(metaStorageFsyncLatency, pebble.FsyncLatencyBuckets), diff --git a/pkg/kv/kvserver/scheduler.go b/pkg/kv/kvserver/scheduler.go index 85db3cefa1fb..062bb4562843 100644 --- a/pkg/kv/kvserver/scheduler.go +++ b/pkg/kv/kvserver/scheduler.go @@ -181,7 +181,7 @@ type raftScheduleState struct { type raftScheduler struct { ambientContext log.AmbientContext processor raftProcessor - latency *metric.Histogram + latency metric.IHistogram numWorkers int maxTicks int diff --git a/pkg/kv/kvserver/txnwait/metrics.go b/pkg/kv/kvserver/txnwait/metrics.go index 2e9d1d2a2055..4610fd8e375c 100644 --- a/pkg/kv/kvserver/txnwait/metrics.go +++ b/pkg/kv/kvserver/txnwait/metrics.go @@ -22,8 +22,8 @@ type Metrics struct { PusherWaiting *metric.Gauge QueryWaiting *metric.Gauge PusherSlow *metric.Gauge - PusherWaitTime *metric.Histogram - QueryWaitTime *metric.Histogram + PusherWaitTime metric.IHistogram + QueryWaitTime metric.IHistogram DeadlocksTotal *metric.Counter } @@ -66,27 +66,31 @@ func NewMetrics(histogramWindowInterval time.Duration) *Metrics { }, ), - PusherWaitTime: metric.NewHistogram( - metric.Metadata{ + PusherWaitTime: metric.NewHistogram(metric.HistogramOptions{ + Metadata: metric.Metadata{ Name: "txnwaitqueue.pusher.wait_time", Help: "Histogram of durations spent in queue by pushers", Measurement: "Pusher wait time", Unit: metric.Unit_NANOSECONDS, }, - histogramWindowInterval, - metric.LongRunning60mLatencyBuckets, - ), + MaxVal: time.Hour.Nanoseconds(), + SigFigs: 1, + Duration: histogramWindowInterval, + Buckets: metric.LongRunning60mLatencyBuckets, + }), - QueryWaitTime: metric.NewHistogram( - metric.Metadata{ + QueryWaitTime: metric.NewHistogram(metric.HistogramOptions{ + Metadata: metric.Metadata{ Name: "txnwaitqueue.query.wait_time", Help: "Histogram of durations spent in queue by queries", Measurement: "Query wait time", Unit: metric.Unit_NANOSECONDS, }, - histogramWindowInterval, - metric.LongRunning60mLatencyBuckets, - ), + MaxVal: time.Hour.Nanoseconds(), + SigFigs: 1, + Duration: histogramWindowInterval, + Buckets: metric.LongRunning60mLatencyBuckets, + }), DeadlocksTotal: metric.NewCounter( metric.Metadata{ diff --git a/pkg/rpc/clock_offset.go b/pkg/rpc/clock_offset.go index bd5877f26b6b..99daad030a72 100644 --- a/pkg/rpc/clock_offset.go +++ b/pkg/rpc/clock_offset.go @@ -29,7 +29,7 @@ import ( type RemoteClockMetrics struct { ClockOffsetMeanNanos *metric.Gauge ClockOffsetStdDevNanos *metric.Gauge - LatencyHistogramNanos *metric.Histogram + LatencyHistogramNanos metric.IHistogram } // avgLatencyMeasurementAge determines how to exponentially weight the @@ -136,9 +136,12 @@ func newRemoteClockMonitor( r.metrics = RemoteClockMetrics{ ClockOffsetMeanNanos: metric.NewGauge(metaClockOffsetMeanNanos), ClockOffsetStdDevNanos: metric.NewGauge(metaClockOffsetStdDevNanos), - LatencyHistogramNanos: metric.NewHistogram( - metaLatencyHistogramNanos, histogramWindowInterval, metric.IOLatencyBuckets, - ), + LatencyHistogramNanos: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaLatencyHistogramNanos, + Duration: histogramWindowInterval, + Buckets: metric.IOLatencyBuckets, + }), } return &r } diff --git a/pkg/server/node.go b/pkg/server/node.go index 76019595ca04..2259a512bd75 100644 --- a/pkg/server/node.go +++ b/pkg/server/node.go @@ -167,7 +167,7 @@ var ( ) type nodeMetrics struct { - Latency *metric.Histogram + Latency metric.IHistogram Success *metric.Counter Err *metric.Counter DiskStalls *metric.Counter @@ -178,9 +178,12 @@ type nodeMetrics struct { func makeNodeMetrics(reg *metric.Registry, histogramWindow time.Duration) nodeMetrics { nm := nodeMetrics{ - Latency: metric.NewHistogram( - metaExecLatency, histogramWindow, metric.IOLatencyBuckets, - ), + Latency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: metaExecLatency, + Duration: histogramWindow, + Buckets: metric.IOLatencyBuckets, + }), Success: metric.NewCounter(metaExecSuccess), Err: metric.NewCounter(metaExecError), DiskStalls: metric.NewCounter(metaDiskStalls), diff --git a/pkg/server/status/recorder_test.go b/pkg/server/status/recorder_test.go index 82016bd8902c..da77954343af 100644 --- a/pkg/server/status/recorder_test.go +++ b/pkg/server/status/recorder_test.go @@ -385,7 +385,12 @@ func TestMetricsRecorder(t *testing.T) { c.Inc((data.val)) addExpected(reg.prefix, data.name, reg.source, 100, data.val, reg.isNode) case "histogram": - h := metric.NewHistogram(metric.Metadata{Name: reg.prefix + data.name}, time.Second, []float64{1.0, 10.0, 100.0, 1000.0}) + h := metric.NewHistogram(metric.HistogramOptions{ + Metadata: metric.Metadata{Name: reg.prefix + data.name}, + Duration: time.Second, + Buckets: []float64{1.0, 10.0, 100.0, 1000.0}, + Mode: metric.HistogramModePrometheus, + }) reg.reg.AddMetric(h) h.RecordValue(data.val) for _, q := range recordHistogramQuantiles { diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index b593acf1f863..ae3dffb05c35 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -452,21 +452,36 @@ func makeMetrics(internal bool) Metrics { SQLOptPlanCacheHits: metric.NewCounter(getMetricMeta(MetaSQLOptPlanCacheHits, internal)), SQLOptPlanCacheMisses: metric.NewCounter(getMetricMeta(MetaSQLOptPlanCacheMisses, internal)), // TODO(mrtracy): See HistogramWindowInterval in server/config.go for the 6x factor. - DistSQLExecLatency: metric.NewHistogram( - getMetricMeta(MetaDistSQLExecLatency, internal), 6*metricsSampleInterval, metric.IOLatencyBuckets, - ), - SQLExecLatency: metric.NewHistogram( - getMetricMeta(MetaSQLExecLatency, internal), 6*metricsSampleInterval, metric.IOLatencyBuckets, - ), - DistSQLServiceLatency: metric.NewHistogram( - getMetricMeta(MetaDistSQLServiceLatency, internal), 6*metricsSampleInterval, metric.IOLatencyBuckets, - ), - SQLServiceLatency: metric.NewHistogram( - getMetricMeta(MetaSQLServiceLatency, internal), 6*metricsSampleInterval, metric.IOLatencyBuckets, - ), - SQLTxnLatency: metric.NewHistogram( - getMetricMeta(MetaSQLTxnLatency, internal), 6*metricsSampleInterval, metric.IOLatencyBuckets, - ), + DistSQLExecLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: getMetricMeta(MetaDistSQLExecLatency, internal), + Duration: 6 * metricsSampleInterval, + Buckets: metric.IOLatencyBuckets, + }), + SQLExecLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: getMetricMeta(MetaSQLExecLatency, internal), + Duration: 6 * metricsSampleInterval, + Buckets: metric.IOLatencyBuckets, + }), + DistSQLServiceLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: getMetricMeta(MetaDistSQLServiceLatency, internal), + Duration: 6 * metricsSampleInterval, + Buckets: metric.IOLatencyBuckets, + }), + SQLServiceLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: getMetricMeta(MetaSQLServiceLatency, internal), + Duration: 6 * metricsSampleInterval, + Buckets: metric.IOLatencyBuckets, + }), + SQLTxnLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: getMetricMeta(MetaSQLTxnLatency, internal), + Duration: 6 * metricsSampleInterval, + Buckets: metric.IOLatencyBuckets, + }), SQLTxnsOpen: metric.NewGauge(getMetricMeta(MetaSQLTxnsOpen, internal)), SQLActiveStatements: metric.NewGauge(getMetricMeta(MetaSQLActiveQueries, internal)), SQLContendedTxns: metric.NewCounter(getMetricMeta(MetaSQLTxnContended, internal)), @@ -490,28 +505,38 @@ func makeMetrics(internal bool) Metrics { func makeServerMetrics(cfg *ExecutorConfig) ServerMetrics { return ServerMetrics{ StatsMetrics: StatsMetrics{ - SQLStatsMemoryMaxBytesHist: metric.NewHistogram( - MetaSQLStatsMemMaxBytes, - cfg.HistogramWindowInterval, - metric.MemoryUsage64MBBuckets, - ), + SQLStatsMemoryMaxBytesHist: metric.NewHistogram(metric.HistogramOptions{ + Metadata: MetaSQLStatsMemMaxBytes, + Duration: cfg.HistogramWindowInterval, + MaxVal: log10int64times1000, + SigFigs: 3, + Buckets: metric.MemoryUsage64MBBuckets, + }), SQLStatsMemoryCurBytesCount: metric.NewGauge(MetaSQLStatsMemCurBytes), - ReportedSQLStatsMemoryMaxBytesHist: metric.NewHistogram( - MetaReportedSQLStatsMemMaxBytes, - cfg.HistogramWindowInterval, - metric.MemoryUsage64MBBuckets, - ), + ReportedSQLStatsMemoryMaxBytesHist: metric.NewHistogram(metric.HistogramOptions{ + Metadata: MetaReportedSQLStatsMemMaxBytes, + Duration: cfg.HistogramWindowInterval, + MaxVal: log10int64times1000, + SigFigs: 3, + Buckets: metric.MemoryUsage64MBBuckets, + }), ReportedSQLStatsMemoryCurBytesCount: metric.NewGauge(MetaReportedSQLStatsMemCurBytes), DiscardedStatsCount: metric.NewCounter(MetaDiscardedSQLStats), SQLStatsFlushStarted: metric.NewCounter(MetaSQLStatsFlushStarted), SQLStatsFlushFailure: metric.NewCounter(MetaSQLStatsFlushFailure), - SQLStatsFlushDuration: metric.NewHistogram( - MetaSQLStatsFlushDuration, 6*metricsSampleInterval, metric.IOLatencyBuckets, - ), + SQLStatsFlushDuration: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: MetaSQLStatsFlushDuration, + Duration: 6 * metricsSampleInterval, + Buckets: metric.IOLatencyBuckets, + }), SQLStatsRemovedRows: metric.NewCounter(MetaSQLStatsRemovedRows), - SQLTxnStatsCollectionOverhead: metric.NewHistogram( - MetaSQLTxnStatsCollectionOverhead, 6*metricsSampleInterval, metric.IOLatencyBuckets, - ), + SQLTxnStatsCollectionOverhead: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: MetaSQLTxnStatsCollectionOverhead, + Duration: 6 * metricsSampleInterval, + Buckets: metric.IOLatencyBuckets, + }), }, ContentionSubsystemMetrics: txnidcache.NewMetrics(), InsightsMetrics: insights.NewMetrics(), diff --git a/pkg/sql/execinfra/metrics.go b/pkg/sql/execinfra/metrics.go index 17ad207d88db..30123ce27ee6 100644 --- a/pkg/sql/execinfra/metrics.go +++ b/pkg/sql/execinfra/metrics.go @@ -24,11 +24,11 @@ type DistSQLMetrics struct { ContendedQueriesCount *metric.Counter FlowsActive *metric.Gauge FlowsTotal *metric.Counter - MaxBytesHist *metric.Histogram + MaxBytesHist metric.IHistogram CurBytesCount *metric.Gauge VecOpenFDs *metric.Gauge CurDiskBytesCount *metric.Gauge - MaxDiskBytesHist *metric.Histogram + MaxDiskBytesHist metric.IHistogram QueriesSpilled *metric.Counter SpilledBytesWritten *metric.Counter SpilledBytesRead *metric.Counter @@ -120,6 +120,10 @@ var ( } ) +// See pkg/sql/mem_metrics.go +// log10int64times1000 = log10(math.MaxInt64) * 1000, rounded up somewhat +const log10int64times1000 = 19 * 1000 + // MakeDistSQLMetrics instantiates the metrics holder for DistSQL monitoring. func MakeDistSQLMetrics(histogramWindow time.Duration) DistSQLMetrics { return DistSQLMetrics{ @@ -128,14 +132,25 @@ func MakeDistSQLMetrics(histogramWindow time.Duration) DistSQLMetrics { ContendedQueriesCount: metric.NewCounter(metaContendedQueriesCount), FlowsActive: metric.NewGauge(metaFlowsActive), FlowsTotal: metric.NewCounter(metaFlowsTotal), - MaxBytesHist: metric.NewHistogram(metaMemMaxBytes, histogramWindow, metric.MemoryUsage64MBBuckets), - CurBytesCount: metric.NewGauge(metaMemCurBytes), - VecOpenFDs: metric.NewGauge(metaVecOpenFDs), - CurDiskBytesCount: metric.NewGauge(metaDiskCurBytes), - MaxDiskBytesHist: metric.NewHistogram(metaDiskMaxBytes, histogramWindow, metric.MemoryUsage64MBBuckets), - QueriesSpilled: metric.NewCounter(metaQueriesSpilled), - SpilledBytesWritten: metric.NewCounter(metaSpilledBytesWritten), - SpilledBytesRead: metric.NewCounter(metaSpilledBytesRead), + MaxBytesHist: metric.NewHistogram(metric.HistogramOptions{ + Metadata: metaMemMaxBytes, + Duration: histogramWindow, + MaxVal: log10int64times1000, + SigFigs: 3, + Buckets: metric.MemoryUsage64MBBuckets, + }), + CurBytesCount: metric.NewGauge(metaMemCurBytes), + VecOpenFDs: metric.NewGauge(metaVecOpenFDs), + CurDiskBytesCount: metric.NewGauge(metaDiskCurBytes), + MaxDiskBytesHist: metric.NewHistogram(metric.HistogramOptions{ + Metadata: metaDiskMaxBytes, + Duration: histogramWindow, + MaxVal: log10int64times1000, + SigFigs: 3, + Buckets: metric.MemoryUsage64MBBuckets}), + QueriesSpilled: metric.NewCounter(metaQueriesSpilled), + SpilledBytesWritten: metric.NewCounter(metaSpilledBytesWritten), + SpilledBytesRead: metric.NewCounter(metaSpilledBytesRead), } } diff --git a/pkg/sql/executor_statement_metrics.go b/pkg/sql/executor_statement_metrics.go index e9cfcdf12c95..3eb34a83eb75 100644 --- a/pkg/sql/executor_statement_metrics.go +++ b/pkg/sql/executor_statement_metrics.go @@ -37,11 +37,11 @@ type EngineMetrics struct { SQLOptPlanCacheHits *metric.Counter SQLOptPlanCacheMisses *metric.Counter - DistSQLExecLatency *metric.Histogram - SQLExecLatency *metric.Histogram - DistSQLServiceLatency *metric.Histogram - SQLServiceLatency *metric.Histogram - SQLTxnLatency *metric.Histogram + DistSQLExecLatency metric.IHistogram + SQLExecLatency metric.IHistogram + DistSQLServiceLatency metric.IHistogram + SQLServiceLatency metric.IHistogram + SQLTxnLatency metric.IHistogram SQLTxnsOpen *metric.Gauge SQLActiveStatements *metric.Gauge SQLContendedTxns *metric.Counter @@ -70,20 +70,20 @@ func (EngineMetrics) MetricStruct() {} // StatsMetrics groups metrics related to SQL Stats collection. type StatsMetrics struct { - SQLStatsMemoryMaxBytesHist *metric.Histogram + SQLStatsMemoryMaxBytesHist metric.IHistogram SQLStatsMemoryCurBytesCount *metric.Gauge - ReportedSQLStatsMemoryMaxBytesHist *metric.Histogram + ReportedSQLStatsMemoryMaxBytesHist metric.IHistogram ReportedSQLStatsMemoryCurBytesCount *metric.Gauge DiscardedStatsCount *metric.Counter SQLStatsFlushStarted *metric.Counter SQLStatsFlushFailure *metric.Counter - SQLStatsFlushDuration *metric.Histogram + SQLStatsFlushDuration metric.IHistogram SQLStatsRemovedRows *metric.Counter - SQLTxnStatsCollectionOverhead *metric.Histogram + SQLTxnStatsCollectionOverhead metric.IHistogram } // StatsMetrics is part of the metric.Struct interface. diff --git a/pkg/sql/mem_metrics.go b/pkg/sql/mem_metrics.go index db2198d6bfd8..248aff4ddc88 100644 --- a/pkg/sql/mem_metrics.go +++ b/pkg/sql/mem_metrics.go @@ -19,7 +19,7 @@ import ( // BaseMemoryMetrics contains a max histogram and a current count of the // bytes allocated by a sql endpoint. type BaseMemoryMetrics struct { - MaxBytesHist *metric.Histogram + MaxBytesHist metric.IHistogram CurBytesCount *metric.Gauge } @@ -30,9 +30,9 @@ type BaseMemoryMetrics struct { // - "internal" for activities related to leases, schema changes, etc. type MemoryMetrics struct { BaseMemoryMetrics - TxnMaxBytesHist *metric.Histogram + TxnMaxBytesHist metric.IHistogram TxnCurBytesCount *metric.Gauge - SessionMaxBytesHist *metric.Histogram + SessionMaxBytesHist metric.IHistogram SessionCurBytesCount *metric.Gauge } @@ -41,6 +41,22 @@ func (MemoryMetrics) MetricStruct() {} var _ metric.Struct = MemoryMetrics{} +// TODO(knz): Until #10014 is addressed, the UI graphs don't have a +// log scale on the Y axis and the histograms are thus displayed using +// a manual log scale: we store the logarithm in the value in the DB +// and plot that logarithm in the UI. +// +// We could, but do not, store the full value in the DB and compute +// the log in the UI, because the current histogram implementation +// does not deal well with large maxima (#10015). +// +// Since the DB stores an integer, we scale the values by 1000 so that +// a modicum of precision is restored when exponentiating the value. +// + +// log10int64times1000 = log10(math.MaxInt64) * 1000, rounded up somewhat +const log10int64times1000 = 19 * 1000 + func makeMemMetricMetadata(name, help string) metric.Metadata { return metric.Metadata{ Name: name, @@ -57,7 +73,13 @@ func MakeBaseMemMetrics(endpoint string, histogramWindow time.Duration) BaseMemo MetaMemMaxBytes := makeMemMetricMetadata(prefix+".max", "Memory usage per sql statement for "+endpoint) MetaMemCurBytes := makeMemMetricMetadata(prefix+".current", "Current sql statement memory usage for "+endpoint) return BaseMemoryMetrics{ - MaxBytesHist: metric.NewHistogram(MetaMemMaxBytes, histogramWindow, metric.MemoryUsage64MBBuckets), + MaxBytesHist: metric.NewHistogram(metric.HistogramOptions{ + Metadata: MetaMemMaxBytes, + Duration: histogramWindow, + MaxVal: log10int64times1000, + SigFigs: 3, + Buckets: metric.MemoryUsage64MBBuckets, + }), CurBytesCount: metric.NewGauge(MetaMemCurBytes), } } @@ -71,10 +93,20 @@ func MakeMemMetrics(endpoint string, histogramWindow time.Duration) MemoryMetric MetaMemMaxSessionBytes := makeMemMetricMetadata(prefix+".session.max", "Memory usage per sql session for "+endpoint) MetaMemSessionCurBytes := makeMemMetricMetadata(prefix+".session.current", "Current sql session memory usage for "+endpoint) return MemoryMetrics{ - BaseMemoryMetrics: base, - TxnMaxBytesHist: metric.NewHistogram(MetaMemMaxTxnBytes, histogramWindow, metric.MemoryUsage64MBBuckets), - TxnCurBytesCount: metric.NewGauge(MetaMemTxnCurBytes), - SessionMaxBytesHist: metric.NewHistogram(MetaMemMaxSessionBytes, histogramWindow, metric.MemoryUsage64MBBuckets), + BaseMemoryMetrics: base, + TxnMaxBytesHist: metric.NewHistogram(metric.HistogramOptions{ + Metadata: MetaMemMaxTxnBytes, + Duration: histogramWindow, + MaxVal: log10int64times1000, + SigFigs: 3, + Buckets: metric.MemoryUsage64MBBuckets}), + TxnCurBytesCount: metric.NewGauge(MetaMemTxnCurBytes), + SessionMaxBytesHist: metric.NewHistogram(metric.HistogramOptions{ + Metadata: MetaMemMaxSessionBytes, + Duration: histogramWindow, + MaxVal: log10int64times1000, + SigFigs: 3, + Buckets: metric.MemoryUsage64MBBuckets}), SessionCurBytesCount: metric.NewGauge(MetaMemSessionCurBytes), } diff --git a/pkg/sql/pgwire/pre_serve.go b/pkg/sql/pgwire/pre_serve.go index e9d2a975c29e..b87b61ad406b 100644 --- a/pkg/sql/pgwire/pre_serve.go +++ b/pkg/sql/pgwire/pre_serve.go @@ -183,7 +183,7 @@ type tenantIndependentMetrics struct { PreServeBytesOutCount *metric.Counter PreServeConnFailures *metric.Counter PreServeNewConns *metric.Counter - PreServeMaxBytes *metric.Histogram + PreServeMaxBytes metric.IHistogram PreServeCurBytes *metric.Gauge } @@ -193,8 +193,13 @@ func makeTenantIndependentMetrics(histogramWindow time.Duration) tenantIndepende PreServeBytesOutCount: metric.NewCounter(MetaPreServeBytesOut), PreServeNewConns: metric.NewCounter(MetaPreServeNewConns), PreServeConnFailures: metric.NewCounter(MetaPreServeConnFailures), - PreServeMaxBytes: metric.NewHistogram(MetaPreServeMaxBytes, histogramWindow, metric.MemoryUsage64MBBuckets), - PreServeCurBytes: metric.NewGauge(MetaPreServeCurBytes), + PreServeMaxBytes: metric.NewHistogram(metric.HistogramOptions{ + Metadata: MetaPreServeMaxBytes, + Duration: histogramWindow, + Buckets: metric.MemoryUsage64MBBuckets, + Mode: metric.HistogramModePrometheus, + }), + PreServeCurBytes: metric.NewGauge(MetaPreServeCurBytes), } } diff --git a/pkg/sql/pgwire/server.go b/pkg/sql/pgwire/server.go index 1e07fc426b36..56d9a0b14ccc 100644 --- a/pkg/sql/pgwire/server.go +++ b/pkg/sql/pgwire/server.go @@ -256,7 +256,7 @@ type tenantSpecificMetrics struct { BytesOutCount *metric.Counter Conns *metric.Gauge NewConns *metric.Counter - ConnLatency *metric.Histogram + ConnLatency metric.IHistogram ConnFailures *metric.Counter PGWireCancelTotalCount *metric.Counter PGWireCancelIgnoredCount *metric.Counter @@ -273,9 +273,12 @@ func makeTenantSpecificMetrics( BytesOutCount: metric.NewCounter(MetaBytesOut), Conns: metric.NewGauge(MetaConns), NewConns: metric.NewCounter(MetaNewConns), - ConnLatency: metric.NewHistogram( - MetaConnLatency, histogramWindow, metric.IOLatencyBuckets, - ), + ConnLatency: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: MetaConnLatency, + Duration: histogramWindow, + Buckets: metric.IOLatencyBuckets, + }), ConnFailures: metric.NewCounter(MetaConnFailures), PGWireCancelTotalCount: metric.NewCounter(MetaPGWireCancelTotal), PGWireCancelIgnoredCount: metric.NewCounter(MetaPGWireCancelIgnored), diff --git a/pkg/sql/sqlstats/persistedsqlstats/provider.go b/pkg/sql/sqlstats/persistedsqlstats/provider.go index f6581e68eac1..3a95761a8499 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/provider.go +++ b/pkg/sql/sqlstats/persistedsqlstats/provider.go @@ -43,7 +43,7 @@ type Config struct { // Metrics. FlushCounter *metric.Counter - FlushDuration *metric.Histogram + FlushDuration metric.IHistogram FailureCounter *metric.Counter // Testing knobs. diff --git a/pkg/sql/sqlstats/sslocal/sql_stats.go b/pkg/sql/sqlstats/sslocal/sql_stats.go index 90c3350961eb..92fe6dcf3904 100644 --- a/pkg/sql/sqlstats/sslocal/sql_stats.go +++ b/pkg/sql/sqlstats/sslocal/sql_stats.go @@ -75,7 +75,7 @@ func newSQLStats( uniqueStmtFingerprintLimit *settings.IntSetting, uniqueTxnFingerprintLimit *settings.IntSetting, curMemBytesCount *metric.Gauge, - maxMemBytesHist *metric.Histogram, + maxMemBytesHist metric.IHistogram, insightsWriter insights.WriterProvider, parentMon *mon.BytesMonitor, flushTarget Sink, diff --git a/pkg/sql/sqlstats/sslocal/sslocal_provider.go b/pkg/sql/sqlstats/sslocal/sslocal_provider.go index b508533ada09..e375c9c00e27 100644 --- a/pkg/sql/sqlstats/sslocal/sslocal_provider.go +++ b/pkg/sql/sqlstats/sslocal/sslocal_provider.go @@ -35,7 +35,7 @@ func New( maxStmtFingerprints *settings.IntSetting, maxTxnFingerprints *settings.IntSetting, curMemoryBytesCount *metric.Gauge, - maxMemoryBytesHist *metric.Histogram, + maxMemoryBytesHist metric.IHistogram, insightsWriter insights.WriterProvider, pool *mon.BytesMonitor, reportingSink Sink, diff --git a/pkg/sql/ttl/ttljob/ttljob_metrics.go b/pkg/sql/ttl/ttljob/ttljob_metrics.go index d9b450a8d481..ae526f574226 100644 --- a/pkg/sql/ttl/ttljob/ttljob_metrics.go +++ b/pkg/sql/ttl/ttljob/ttljob_metrics.go @@ -96,41 +96,48 @@ func (m *RowLevelTTLAggMetrics) loadMetrics(labelMetrics bool, relation string) } func makeRowLevelTTLAggMetrics(histogramWindowInterval time.Duration) metric.Struct { + sigFigs := 2 b := aggmetric.MakeBuilder("relation") ret := &RowLevelTTLAggMetrics{ - SpanTotalDuration: b.Histogram( - metric.Metadata{ + SpanTotalDuration: b.Histogram(metric.HistogramOptions{ + Metadata: metric.Metadata{ Name: "jobs.row_level_ttl.span_total_duration", Help: "Duration for processing a span during row level TTL.", Measurement: "nanoseconds", Unit: metric.Unit_NANOSECONDS, MetricType: io_prometheus_client.MetricType_HISTOGRAM, }, - histogramWindowInterval, - metric.LongRunning60mLatencyBuckets, - ), - SelectDuration: b.Histogram( - metric.Metadata{ + MaxVal: time.Hour.Nanoseconds(), + SigFigs: sigFigs, + Duration: histogramWindowInterval, + Buckets: metric.LongRunning60mLatencyBuckets, + }), + SelectDuration: b.Histogram(metric.HistogramOptions{ + Metadata: metric.Metadata{ Name: "jobs.row_level_ttl.select_duration", Help: "Duration for select requests during row level TTL.", Measurement: "nanoseconds", Unit: metric.Unit_NANOSECONDS, MetricType: io_prometheus_client.MetricType_HISTOGRAM, }, - histogramWindowInterval, - metric.BatchProcessLatencyBuckets, - ), - DeleteDuration: b.Histogram( - metric.Metadata{ + MaxVal: time.Minute.Nanoseconds(), + SigFigs: sigFigs, + Duration: histogramWindowInterval, + Buckets: metric.BatchProcessLatencyBuckets, + }), + DeleteDuration: b.Histogram(metric.HistogramOptions{ + Metadata: metric.Metadata{ Name: "jobs.row_level_ttl.delete_duration", Help: "Duration for delete requests during row level TTL.", Measurement: "nanoseconds", Unit: metric.Unit_NANOSECONDS, MetricType: io_prometheus_client.MetricType_HISTOGRAM, }, - histogramWindowInterval, - metric.BatchProcessLatencyBuckets, - ), + MaxVal: time.Minute.Nanoseconds(), + SigFigs: sigFigs, + Duration: histogramWindowInterval, + Buckets: metric.BatchProcessLatencyBuckets, + }), RowSelections: b.Counter( metric.Metadata{ Name: "jobs.row_level_ttl.rows_selected", diff --git a/pkg/util/admission/work_queue.go b/pkg/util/admission/work_queue.go index be9b4fabc3d4..d6211bee01ab 100644 --- a/pkg/util/admission/work_queue.go +++ b/pkg/util/admission/work_queue.go @@ -1561,7 +1561,7 @@ type workQueueMetricsSingle struct { Requested *metric.Counter Admitted *metric.Counter Errored *metric.Counter - WaitDurations *metric.Histogram + WaitDurations metric.IHistogram WaitQueueLength *metric.Gauge } @@ -1622,9 +1622,12 @@ func makeWorkQueueMetricsSingle(name string) workQueueMetricsSingle { Requested: metric.NewCounter(addName(name, requestedMeta)), Admitted: metric.NewCounter(addName(name, admittedMeta)), Errored: metric.NewCounter(addName(name, erroredMeta)), - WaitDurations: metric.NewHistogram( - addName(name, waitDurationsMeta), base.DefaultHistogramWindowInterval(), metric.IOLatencyBuckets, - ), + WaitDurations: metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePreferHdrLatency, + Metadata: addName(name, waitDurationsMeta), + Duration: base.DefaultHistogramWindowInterval(), + Buckets: metric.IOLatencyBuckets, + }), WaitQueueLength: metric.NewGauge(addName(name, waitQueueLengthMeta)), } } diff --git a/pkg/util/metric/BUILD.bazel b/pkg/util/metric/BUILD.bazel index 7d0473c33b40..e94462787b40 100644 --- a/pkg/util/metric/BUILD.bazel +++ b/pkg/util/metric/BUILD.bazel @@ -8,6 +8,7 @@ go_library( srcs = [ "doc.go", "graphite_exporter.go", + "hdrhistogram.go", "histogram_buckets.go", "metric.go", "prometheus_exporter.go", @@ -22,10 +23,13 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/util/metric", visibility = ["//visibility:public"], deps = [ + "//pkg/util", + "//pkg/util/envutil", "//pkg/util/log", "//pkg/util/syncutil", "//pkg/util/timeutil", "@com_github_cockroachdb_errors//:errors", + "@com_github_codahale_hdrhistogram//:hdrhistogram", "@com_github_gogo_protobuf//proto", "@com_github_prometheus_client_golang//prometheus", "@com_github_prometheus_client_golang//prometheus/graphite", diff --git a/pkg/util/metric/aggmetric/BUILD.bazel b/pkg/util/metric/aggmetric/BUILD.bazel index daf77ba5d5ff..5984de7e5154 100644 --- a/pkg/util/metric/aggmetric/BUILD.bazel +++ b/pkg/util/metric/aggmetric/BUILD.bazel @@ -17,7 +17,6 @@ go_library( "@com_github_cockroachdb_errors//:errors", "@com_github_gogo_protobuf//proto", "@com_github_google_btree//:btree", - "@com_github_prometheus_client_golang//prometheus", "@com_github_prometheus_client_model//go", ], ) diff --git a/pkg/util/metric/aggmetric/agg_metric.go b/pkg/util/metric/aggmetric/agg_metric.go index c9afb965d64f..ab5ad03ce5b6 100644 --- a/pkg/util/metric/aggmetric/agg_metric.go +++ b/pkg/util/metric/aggmetric/agg_metric.go @@ -15,7 +15,6 @@ package aggmetric import ( "strings" - "time" "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -50,10 +49,8 @@ func (b Builder) Counter(metadata metric.Metadata) *AggCounter { } // Histogram constructs a new AggHistogram with the Builder's labels. -func (b Builder) Histogram( - metadata metric.Metadata, duration time.Duration, buckets []float64, -) *AggHistogram { - return NewHistogram(metadata, duration, buckets, b.labels...) +func (b Builder) Histogram(opts metric.HistogramOptions) *AggHistogram { + return NewHistogram(opts, b.labels...) } type childSet struct { diff --git a/pkg/util/metric/aggmetric/agg_metric_test.go b/pkg/util/metric/aggmetric/agg_metric_test.go index 842a30504889..a139347d38db 100644 --- a/pkg/util/metric/aggmetric/agg_metric_test.go +++ b/pkg/util/metric/aggmetric/agg_metric_test.go @@ -13,6 +13,7 @@ package aggmetric_test import ( "bufio" "bytes" + "fmt" "sort" "strings" "testing" @@ -63,10 +64,15 @@ func TestAggMetric(t *testing.T) { Name: "baz_gauge", }, "tenant_id") r.AddMetric(f) - - h := aggmetric.NewHistogram(metric.Metadata{ - Name: "histo_gram", - }, base.DefaultHistogramWindowInterval(), metric.Count1KBuckets, "tenant_id") + h := aggmetric.NewHistogram(metric.HistogramOptions{ + Metadata: metric.Metadata{ + Name: "histo_gram", + }, + Duration: base.DefaultHistogramWindowInterval(), + MaxVal: 100, + SigFigs: 1, + Buckets: metric.Count1KBuckets, + }, "tenant_id") r.AddMetric(h) tenant2 := roachpb.MustMakeTenantID(2) @@ -87,18 +93,28 @@ func TestAggMetric(t *testing.T) { g3.Inc(3) g3.Dec(1) f2.Update(1.5) + fmt.Println(r) f3.Update(2.5) h2.RecordValue(10) h3.RecordValue(90) - echotest.Require(t, writePrometheusMetrics(t), datapathutils.TestDataPath(t, "basic.txt")) + testFile := "basic.txt" + if metric.HdrEnabled() { + testFile = "basic_hdr.txt" + } + echotest.Require(t, writePrometheusMetrics(t), datapathutils.TestDataPath(t, testFile)) }) t.Run("destroy", func(t *testing.T) { + fmt.Println(r) g3.Unlink() c2.Unlink() f3.Unlink() h3.Unlink() - echotest.Require(t, writePrometheusMetrics(t), datapathutils.TestDataPath(t, "destroy.txt")) + testFile := "destroy.txt" + if metric.HdrEnabled() { + testFile = "destroy_hdr.txt" + } + echotest.Require(t, writePrometheusMetrics(t), datapathutils.TestDataPath(t, testFile)) }) t.Run("panic on already exists", func(t *testing.T) { @@ -119,7 +135,11 @@ func TestAggMetric(t *testing.T) { c2 = c.AddChild(tenant2.String()) f3 = f.AddChild(tenant3.String()) h3 = h.AddChild(tenant3.String()) - echotest.Require(t, writePrometheusMetrics(t), datapathutils.TestDataPath(t, "add_after_destroy.txt")) + testFile := "add_after_destroy.txt" + if metric.HdrEnabled() { + testFile = "add_after_destroy_hdr.txt" + } + echotest.Require(t, writePrometheusMetrics(t), datapathutils.TestDataPath(t, testFile)) }) t.Run("panic on label length mismatch", func(t *testing.T) { @@ -135,8 +155,13 @@ func TestAggMetricBuilder(t *testing.T) { c := b.Counter(metric.Metadata{Name: "foo_counter"}) g := b.Gauge(metric.Metadata{Name: "bar_gauge"}) f := b.GaugeFloat64(metric.Metadata{Name: "baz_gauge"}) - h := b.Histogram(metric.Metadata{Name: "histo_gram"}, - base.DefaultHistogramWindowInterval(), metric.Count1KBuckets) + h := b.Histogram(metric.HistogramOptions{ + Metadata: metric.Metadata{Name: "histo_gram"}, + Duration: base.DefaultHistogramWindowInterval(), + MaxVal: 100, + SigFigs: 1, + Buckets: metric.Count1KBuckets, + }) for i := 5; i < 10; i++ { tenantLabel := roachpb.MustMakeTenantID(uint64(i)).String() diff --git a/pkg/util/metric/aggmetric/histogram.go b/pkg/util/metric/aggmetric/histogram.go index 68d30fab17a2..1c07cc1ef3d3 100644 --- a/pkg/util/metric/aggmetric/histogram.go +++ b/pkg/util/metric/aggmetric/histogram.go @@ -11,10 +11,7 @@ package aggmetric import ( - "time" - "github.com/cockroachdb/cockroach/pkg/util/metric" - "github.com/prometheus/client_golang/prometheus" io_prometheus_client "github.com/prometheus/client_model/go" ) @@ -23,8 +20,8 @@ import ( // children, while its children are additionally exported to prometheus via the // PrometheusIterable interface. type AggHistogram struct { - h metric.Histogram - create func() *metric.Histogram + h metric.IHistogram + create func() metric.IHistogram childSet } @@ -34,14 +31,12 @@ var _ metric.PrometheusExportable = (*AggHistogram)(nil) var _ metric.WindowedHistogram = (*AggHistogram)(nil) // NewHistogram constructs a new AggHistogram. -func NewHistogram( - metadata metric.Metadata, duration time.Duration, buckets []float64, childLabels ...string, -) *AggHistogram { - create := func() *metric.Histogram { - return metric.NewHistogram(metadata, duration, buckets) +func NewHistogram(opts metric.HistogramOptions, childLabels ...string) *AggHistogram { + create := func() metric.IHistogram { + return metric.NewHistogram(opts) } a := &AggHistogram{ - h: *create(), + h: create(), create: create, } a.init(childLabels) @@ -96,19 +91,13 @@ func (a *AggHistogram) ToPrometheusMetric() *io_prometheus_client.Metric { return a.h.ToPrometheusMetric() } -// Windowed returns a copy of the current windowed histogram data and its -// rotation interval. -func (a *AggHistogram) Windowed() prometheus.Histogram { - return a.h.Windowed() -} - // AddChild adds a Counter to this AggCounter. This method panics if a Counter // already exists for this set of labelVals. func (a *AggHistogram) AddChild(labelVals ...string) *Histogram { child := &Histogram{ parent: a, labelValuesSlice: labelValuesSlice(labelVals), - h: *a.create(), + h: a.create(), } a.add(child) return child @@ -121,7 +110,7 @@ func (a *AggHistogram) AddChild(labelVals ...string) *Histogram { type Histogram struct { parent *AggHistogram labelValuesSlice - h metric.Histogram + h metric.IHistogram } // ToPrometheusMetric constructs a prometheus metric for this Histogram. diff --git a/pkg/util/metric/aggmetric/testdata/add_after_destroy_hdr.txt b/pkg/util/metric/aggmetric/testdata/add_after_destroy_hdr.txt new file mode 100644 index 000000000000..ffcbd4571a33 --- /dev/null +++ b/pkg/util/metric/aggmetric/testdata/add_after_destroy_hdr.txt @@ -0,0 +1,23 @@ +echo +---- +bar_gauge 4 +bar_gauge{tenant_id="2"} 2 +bar_gauge{tenant_id="3"} 0 +baz_gauge 4 +baz_gauge{tenant_id="2"} 1.5 +baz_gauge{tenant_id="3"} 0 +foo_counter 6 +foo_counter{tenant_id="2"} 0 +foo_counter{tenant_id="3"} 4 +histo_gram_bucket{le="+Inf"} 2 +histo_gram_bucket{le="10"} 1 +histo_gram_bucket{le="91"} 2 +histo_gram_bucket{tenant_id="2",le="+Inf"} 1 +histo_gram_bucket{tenant_id="2",le="10"} 1 +histo_gram_bucket{tenant_id="3",le="+Inf"} 0 +histo_gram_count 2 +histo_gram_count{tenant_id="2"} 1 +histo_gram_count{tenant_id="3"} 0 +histo_gram_sum 101 +histo_gram_sum{tenant_id="2"} 10 +histo_gram_sum{tenant_id="3"} 0 \ No newline at end of file diff --git a/pkg/util/metric/aggmetric/testdata/basic_hdr.txt b/pkg/util/metric/aggmetric/testdata/basic_hdr.txt new file mode 100644 index 000000000000..a796b8ef3406 --- /dev/null +++ b/pkg/util/metric/aggmetric/testdata/basic_hdr.txt @@ -0,0 +1,24 @@ +echo +---- +bar_gauge 4 +bar_gauge{tenant_id="2"} 2 +bar_gauge{tenant_id="3"} 2 +baz_gauge 4 +baz_gauge{tenant_id="2"} 1.5 +baz_gauge{tenant_id="3"} 2.5 +foo_counter 6 +foo_counter{tenant_id="2"} 2 +foo_counter{tenant_id="3"} 4 +histo_gram_bucket{le="+Inf"} 2 +histo_gram_bucket{le="10"} 1 +histo_gram_bucket{le="91"} 2 +histo_gram_bucket{tenant_id="2",le="+Inf"} 1 +histo_gram_bucket{tenant_id="2",le="10"} 1 +histo_gram_bucket{tenant_id="3",le="+Inf"} 1 +histo_gram_bucket{tenant_id="3",le="91"} 1 +histo_gram_count 2 +histo_gram_count{tenant_id="2"} 1 +histo_gram_count{tenant_id="3"} 1 +histo_gram_sum 101 +histo_gram_sum{tenant_id="2"} 10 +histo_gram_sum{tenant_id="3"} 91 \ No newline at end of file diff --git a/pkg/util/metric/aggmetric/testdata/destroy_hdr.txt b/pkg/util/metric/aggmetric/testdata/destroy_hdr.txt new file mode 100644 index 000000000000..dd17b7aae5bb --- /dev/null +++ b/pkg/util/metric/aggmetric/testdata/destroy_hdr.txt @@ -0,0 +1,17 @@ +echo +---- +bar_gauge 4 +bar_gauge{tenant_id="2"} 2 +baz_gauge 4 +baz_gauge{tenant_id="2"} 1.5 +foo_counter 6 +foo_counter{tenant_id="3"} 4 +histo_gram_bucket{le="+Inf"} 2 +histo_gram_bucket{le="10"} 1 +histo_gram_bucket{le="91"} 2 +histo_gram_bucket{tenant_id="2",le="+Inf"} 1 +histo_gram_bucket{tenant_id="2",le="10"} 1 +histo_gram_count 2 +histo_gram_count{tenant_id="2"} 1 +histo_gram_sum 101 +histo_gram_sum{tenant_id="2"} 10 \ No newline at end of file diff --git a/pkg/util/metric/hdrhistogram.go b/pkg/util/metric/hdrhistogram.go new file mode 100644 index 000000000000..e33582b9c2e3 --- /dev/null +++ b/pkg/util/metric/hdrhistogram.go @@ -0,0 +1,248 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package metric + +import ( + "time" + + "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/codahale/hdrhistogram" + prometheusgo "github.com/prometheus/client_model/go" +) + +const ( + // HdrHistogramMaxLatency is the maximum value tracked in latency histograms. Higher + // values will be recorded as this value instead. + HdrHistogramMaxLatency = 10 * time.Second + + // The number of histograms to keep in rolling window. + hdrHistogramHistWrapNum = 2 // TestSampleInterval is passed to histograms during tests which don't +) + +// A HdrHistogram collects observed values by keeping bucketed counts. For +// convenience, internally two sets of buckets are kept: A cumulative set (i.e. +// data is never evicted) and a windowed set (which keeps only recently +// collected samples). +// +// Top-level methods generally apply to the cumulative buckets; the windowed +// variant is exposed through the Windowed method. +// +// TODO(#96357): remove HdrHistogram model entirely once the Prometheus +// backed histogram and its bucket boundaries have been reliably proven in +// production. +type HdrHistogram struct { + Metadata + maxVal int64 + mu struct { + syncutil.Mutex + cumulative *hdrhistogram.Histogram + *tickHelper + sliding *hdrhistogram.WindowedHistogram + } +} + +var _ IHistogram = &HdrHistogram{} +var _ PrometheusExportable = &HdrHistogram{} +var _ Iterable = &HdrHistogram{} + +// NewHdrHistogram initializes a given Histogram. The contained windowed histogram +// rotates every 'duration'; both the windowed and the cumulative histogram +// track nonnegative values up to 'maxVal' with 'sigFigs' decimal points of +// precision. +func NewHdrHistogram( + metadata Metadata, duration time.Duration, maxVal int64, sigFigs int, +) *HdrHistogram { + h := &HdrHistogram{ + Metadata: metadata, + maxVal: maxVal, + } + wHist := hdrhistogram.NewWindowed(hdrHistogramHistWrapNum, 0, maxVal, sigFigs) + h.mu.cumulative = hdrhistogram.New(0, maxVal, sigFigs) + h.mu.sliding = wHist + h.mu.tickHelper = &tickHelper{ + nextT: now(), + tickInterval: duration / hdrHistogramHistWrapNum, + onTick: func() { + wHist.Rotate() + }, + } + return h +} + +// NewHdrLatency is a convenience function which returns a histogram with +// suitable defaults for latency tracking. Values are expressed in ns, +// are truncated into the interval [0, HdrHistogramMaxLatency] and are recorded +// with one digit of precision (i.e. errors of <10ms at 100ms, <6s at 60s). +// +// The windowed portion of the Histogram retains values for approximately +// histogramWindow. +func NewHdrLatency(metadata Metadata, histogramWindow time.Duration) *HdrHistogram { + return NewHdrHistogram( + metadata, histogramWindow, HdrHistogramMaxLatency.Nanoseconds(), 1, + ) +} + +// RecordValue adds the given value to the histogram. Recording a value in +// excess of the configured maximum value for that histogram results in +// recording the maximum value instead. +func (h *HdrHistogram) RecordValue(v int64) { + h.mu.Lock() + defer h.mu.Unlock() + + if h.mu.sliding.Current.RecordValue(v) != nil { + _ = h.mu.sliding.Current.RecordValue(h.maxVal) + } + if h.mu.cumulative.RecordValue(v) != nil { + _ = h.mu.cumulative.RecordValue(h.maxVal) + } +} + +// TotalCount returns the (cumulative) number of samples. +func (h *HdrHistogram) TotalCount() int64 { + h.mu.Lock() + defer h.mu.Unlock() + return h.mu.cumulative.TotalCount() +} + +// Min returns the minimum. +func (h *HdrHistogram) Min() int64 { + h.mu.Lock() + defer h.mu.Unlock() + return h.mu.cumulative.Min() +} + +// Inspect calls the closure with the empty string and the receiver. +func (h *HdrHistogram) Inspect(f func(interface{})) { + h.mu.Lock() + maybeTick(h.mu.tickHelper) + h.mu.Unlock() + f(h) +} + +// GetType returns the prometheus type enum for this metric. +func (h *HdrHistogram) GetType() *prometheusgo.MetricType { + return prometheusgo.MetricType_HISTOGRAM.Enum() +} + +// ToPrometheusMetric returns a filled-in prometheus metric of the right type. +func (h *HdrHistogram) ToPrometheusMetric() *prometheusgo.Metric { + hist := &prometheusgo.Histogram{} + + h.mu.Lock() + maybeTick(h.mu.tickHelper) + bars := h.mu.cumulative.Distribution() + hist.Bucket = make([]*prometheusgo.Bucket, 0, len(bars)) + + var cumCount uint64 + var sum float64 + for _, bar := range bars { + if bar.Count == 0 { + // No need to expose trivial buckets. + continue + } + upperBound := float64(bar.To) + sum += upperBound * float64(bar.Count) + + cumCount += uint64(bar.Count) + curCumCount := cumCount // need a new alloc thanks to bad proto code + + hist.Bucket = append(hist.Bucket, &prometheusgo.Bucket{ + CumulativeCount: &curCumCount, + UpperBound: &upperBound, + }) + } + hist.SampleCount = &cumCount + hist.SampleSum = &sum // can do better here; we approximate in the loop + h.mu.Unlock() + + return &prometheusgo.Metric{ + Histogram: hist, + } +} + +// TotalCountWindowed implements the WindowedHistogram interface. +func (h *HdrHistogram) TotalCountWindowed() int64 { + return int64(h.ToPrometheusMetricWindowed().Histogram.GetSampleCount()) +} + +// TotalSumWindowed implements the WindowedHistogram interface. +func (h *HdrHistogram) TotalSumWindowed() float64 { + return h.ToPrometheusMetricWindowed().Histogram.GetSampleSum() +} + +func (h *HdrHistogram) toPrometheusMetricWindowedLocked() *prometheusgo.Metric { + hist := &prometheusgo.Histogram{} + + maybeTick(h.mu.tickHelper) + bars := h.mu.sliding.Current.Distribution() + hist.Bucket = make([]*prometheusgo.Bucket, 0, len(bars)) + + var cumCount uint64 + var sum float64 + for _, bar := range bars { + if bar.Count == 0 { + // No need to expose trivial buckets. + continue + } + upperBound := float64(bar.To) + sum += upperBound * float64(bar.Count) + + cumCount += uint64(bar.Count) + curCumCount := cumCount // need a new alloc thanks to bad proto code + + hist.Bucket = append(hist.Bucket, &prometheusgo.Bucket{ + CumulativeCount: &curCumCount, + UpperBound: &upperBound, + }) + } + hist.SampleCount = &cumCount + hist.SampleSum = &sum // can do better here; we approximate in the loop + + return &prometheusgo.Metric{ + Histogram: hist, + } +} + +// ToPrometheusMetricWindowed returns a filled-in prometheus metric of the right type. +func (h *HdrHistogram) ToPrometheusMetricWindowed() *prometheusgo.Metric { + h.mu.Lock() + defer h.mu.Unlock() + return h.toPrometheusMetricWindowedLocked() +} + +// GetMetadata returns the metric's metadata including the Prometheus +// MetricType. +func (h *HdrHistogram) GetMetadata() Metadata { + baseMetadata := h.Metadata + baseMetadata.MetricType = prometheusgo.MetricType_HISTOGRAM + return baseMetadata +} + +func (h *HdrHistogram) ValueAtQuantileWindowed(q float64) float64 { + h.mu.Lock() + defer h.mu.Unlock() + + return ValueAtQuantileWindowed(h.toPrometheusMetricWindowedLocked().Histogram, q) +} + +func (h *HdrHistogram) Mean() float64 { + h.mu.Lock() + defer h.mu.Unlock() + + return h.mu.cumulative.Mean() +} + +func (h *HdrHistogram) TotalSum() float64 { + h.mu.Lock() + defer h.mu.Unlock() + + return h.ToPrometheusMetric().GetSummary().GetSampleSum() +} diff --git a/pkg/util/metric/metric.go b/pkg/util/metric/metric.go index a6b794422e4b..afd19cf83638 100644 --- a/pkg/util/metric/metric.go +++ b/pkg/util/metric/metric.go @@ -17,6 +17,8 @@ import ( "sync/atomic" "time" + "github.com/cockroachdb/cockroach/pkg/util" + "github.com/cockroachdb/cockroach/pkg/util/envutil" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/gogo/protobuf/proto" @@ -25,11 +27,9 @@ import ( "github.com/rcrowley/go-metrics" ) -const ( - // TestSampleInterval is passed to histograms during tests which don't - // want to concern themselves with supplying a "correct" interval. - TestSampleInterval = time.Duration(math.MaxInt64) -) +// TestSampleInterval is passed to histograms during tests which don't +// want to concern themselves with supplying a "correct" interval. +const TestSampleInterval = time.Duration(math.MaxInt64) // Iterable provides a method for synchronized access to interior objects. type Iterable interface { @@ -176,10 +176,81 @@ func maybeTick(m periodic) { } } +// useHdrHistogramsEnvVar can be used to switch all histograms to use the +// legacy HDR histograms (except for those that explicitly force the use +// of the newer Prometheus via HistogramModePrometheus). HDR Histograms +// dynamically generate bucket boundaries, which can lead to hundreds of +// buckets. This can cause performance issues with timeseries databases +// like Prometheus. +const useHdrHistogramsEnvVar = "COCKROACH_ENABLE_HDR_HISTOGRAMS" + +var hdrEnabled = util.ConstantWithMetamorphicTestBool(useHdrHistogramsEnvVar, envutil.EnvOrDefaultBool(useHdrHistogramsEnvVar, false)) + +// HdrEnabled returns whether or not the HdrHistogram model is enabled +// in the metric package. Primarily useful in tests where we want to validate +// different outputs depending on whether or not HDR is enabled. +func HdrEnabled() bool { + return hdrEnabled +} + +type HistogramMode byte + +const ( + // HistogramModePrometheus will force the constructed histogram to use + // the Prometheus histogram model, regardless of the value of + // useHdrHistogramsEnvVar. This option should be used for all + // newly defined histograms moving forward. + // + // NB: If neither this mode nor the HistogramModePreferHdrLatency mode + // is set, MaxVal and SigFigs must be defined to maintain backwards + // compatibility with the legacy HdrHistogram model. + HistogramModePrometheus HistogramMode = iota + 1 + // HistogramModePreferHdrLatency will cause the returned histogram to + // use the HdrHistgoram model and be configured with suitable defaults + // for latency tracking iff useHdrHistogramsEnvVar is enabled. + // + // NB: If this option is set, no MaxVal or SigFigs are required in the + // HistogramOptions to maintain backwards compatibility with the legacy + // HdrHistogram model, since suitable defaults are used for both. + HistogramModePreferHdrLatency +) + +type HistogramOptions struct { + // Metadata is the metric Metadata associated with the histogram. + Metadata Metadata + // Duration is the histogram's window duration. + Duration time.Duration + // MaxVal is only relevant to the HdrHistogram, and represents the + // highest trackable value in the resulting histogram buckets. + MaxVal int64 + // SigFigs is only relevant to the HdrHistogram, and represents + // the number of significant figures to be used to determine the + // degree of accuracy used in measurements. + SigFigs int + // Buckets are only relevant to Prometheus histograms, and represent + // the pre-defined histogram bucket boundaries to be used. + Buckets []float64 + // Mode defines the type of histogram to be used. See individual + // comments on each HistogramMode value for details. + Mode HistogramMode +} + +func NewHistogram(opt HistogramOptions) IHistogram { + if hdrEnabled && opt.Mode != HistogramModePrometheus { + if opt.Mode == HistogramModePreferHdrLatency { + return NewHdrLatency(opt.Metadata, opt.Duration) + } else { + return NewHdrHistogram(opt.Metadata, opt.Duration, opt.MaxVal, opt.SigFigs) + } + } else { + return newHistogram(opt.Metadata, opt.Duration, opt.Buckets) + } +} + // NewHistogram is a prometheus-backed histogram. Depending on the value of // opts.Buckets, this is suitable for recording any kind of quantity. Common // sensible choices are {IO,Network}LatencyBuckets. -func NewHistogram(meta Metadata, windowDuration time.Duration, buckets []float64) *Histogram { +func newHistogram(meta Metadata, windowDuration time.Duration, buckets []float64) *Histogram { // TODO(obs-inf): prometheus supports labeled histograms but they require more // plumbing and don't fit into the PrometheusObservable interface any more. opts := prometheus.HistogramOpts{ @@ -236,6 +307,21 @@ type Histogram struct { } } +type IHistogram interface { + Iterable + PrometheusExportable + WindowedHistogram + + RecordValue(n int64) + TotalCount() int64 + TotalSum() float64 + TotalCountWindowed() int64 + TotalSumWindowed() float64 + Mean() float64 +} + +var _ IHistogram = &Histogram{} + func (h *Histogram) nextTick() time.Time { h.windowed.RLock() defer h.windowed.RUnlock() @@ -326,7 +412,8 @@ func (h *Histogram) TotalSumWindowed() float64 { // Mean returns the (cumulative) mean of samples. func (h *Histogram) Mean() float64 { - return h.TotalSum() / float64(h.TotalCount()) + pm := h.ToPrometheusMetric() + return pm.Histogram.GetSampleSum() / float64(pm.Histogram.GetSampleCount()) } // ValueAtQuantileWindowed implements the WindowedHistogram interface. diff --git a/pkg/util/metric/metric_ext_test.go b/pkg/util/metric/metric_ext_test.go index a06cdfc0cb63..60c14ba25bc5 100644 --- a/pkg/util/metric/metric_ext_test.go +++ b/pkg/util/metric/metric_ext_test.go @@ -25,7 +25,12 @@ func TestHistogramPrometheus(t *testing.T) { // Regression test against https://github.com/cockroachdb/cockroach/pull/88331. // The output includes buckets for which the upper bound equals the previous // bucket's upper bound. - h := metric.NewHistogram(metric.Metadata{}, time.Second, []float64{1, 2, 3, 4, 5, 6, 10, 20, 30}) + h := metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePrometheus, + Metadata: metric.Metadata{}, + Duration: time.Second, + Buckets: []float64{1, 2, 3, 4, 5, 6, 10, 20, 30}, + }) h.RecordValue(1) h.RecordValue(5) h.RecordValue(5) diff --git a/pkg/util/metric/metric_test.go b/pkg/util/metric/metric_test.go index 2a273bc7fa85..d8a451ec06ad 100644 --- a/pkg/util/metric/metric_test.go +++ b/pkg/util/metric/metric_test.go @@ -112,17 +112,18 @@ func TestHistogram(t *testing.T) { return &n } - h := NewHistogram( - Metadata{}, - time.Hour, - []float64{ + h := NewHistogram(HistogramOptions{ + Mode: HistogramModePrometheus, + Metadata: Metadata{}, + Duration: time.Hour, + Buckets: []float64{ 1.0, 5.0, 10.0, 25.0, 100.0, }, - ) + }) // should return 0 if no observations are made require.Equal(t, 0.0, h.ValueAtQuantileWindowed(0)) @@ -236,23 +237,24 @@ func TestNewHistogramRotate(t *testing.T) { defer TestingSetNow(nil)() setNow(0) - h := NewHistogram(emptyMetadata, 10*time.Second, nil) + h := NewHistogram(HistogramOptions{ + Mode: HistogramModePrometheus, + Metadata: emptyMetadata, + Duration: 10 * time.Second, + Buckets: nil, + }) for i := 0; i < 4; i++ { // Windowed histogram is initially empty. h.Inspect(func(interface{}) {}) // triggers ticking - var m prometheusgo.Metric - require.NoError(t, h.Windowed().Write(&m)) - require.Zero(t, *m.Histogram.SampleSum) + require.Zero(t, h.TotalSumWindowed()) // But cumulative histogram has history (if i > 0). - require.EqualValues(t, i, *h.ToPrometheusMetric().Histogram.SampleCount) + require.EqualValues(t, i, h.TotalCount()) // Add a measurement and verify it's there. { h.RecordValue(12345) f := float64(12345) - var m prometheusgo.Metric - require.NoError(t, h.Windowed().Write(&m)) - require.Equal(t, *m.Histogram.SampleSum, f) + require.Equal(t, h.TotalSumWindowed(), f) } // Tick. This rotates the histogram. setNow(time.Duration(i+1) * 10 * time.Second) diff --git a/pkg/util/metric/registry_test.go b/pkg/util/metric/registry_test.go index 5d2b2a6c4e88..48f4aba216bd 100644 --- a/pkg/util/metric/registry_test.go +++ b/pkg/util/metric/registry_test.go @@ -76,14 +76,19 @@ func TestRegistry(t *testing.T) { topCounter := NewCounter(Metadata{Name: "top.counter"}) r.AddMetric(topCounter) - r.AddMetric(NewHistogram(Metadata{Name: "top.histogram"}, time.Minute, Count1KBuckets)) + r.AddMetric(NewHistogram(HistogramOptions{ + Mode: HistogramModePrometheus, + Metadata: Metadata{Name: "top.histogram"}, + Duration: time.Minute, + Buckets: Count1KBuckets, + })) r.AddMetric(NewGauge(Metadata{Name: "bottom.gauge"})) ms := &struct { StructGauge *Gauge StructGauge64 *GaugeFloat64 StructCounter *Counter - StructHistogram *Histogram + StructHistogram IHistogram NestedStructGauge NestedStruct ArrayStructCounters [4]*Counter // Ensure that nil struct values in arrays are safe. @@ -92,7 +97,7 @@ func TestRegistry(t *testing.T) { privateStructGauge *Gauge privateStructGauge64 *GaugeFloat64 privateStructCounter *Counter - privateStructHistogram *Histogram + privateStructHistogram IHistogram privateNestedStructGauge NestedStruct privateArrayStructCounters [2]*Counter NotAMetric int @@ -100,10 +105,15 @@ func TestRegistry(t *testing.T) { ReallyNotAMetric *Registry DefinitelyNotAnArrayOfMetrics [2]int }{ - StructGauge: NewGauge(Metadata{Name: "struct.gauge"}), - StructGauge64: NewGaugeFloat64(Metadata{Name: "struct.gauge64"}), - StructCounter: NewCounter(Metadata{Name: "struct.counter"}), - StructHistogram: NewHistogram(Metadata{Name: "struct.histogram"}, time.Minute, Count1KBuckets), + StructGauge: NewGauge(Metadata{Name: "struct.gauge"}), + StructGauge64: NewGaugeFloat64(Metadata{Name: "struct.gauge64"}), + StructCounter: NewCounter(Metadata{Name: "struct.counter"}), + StructHistogram: NewHistogram(HistogramOptions{ + Mode: HistogramModePrometheus, + Metadata: Metadata{Name: "struct.histogram"}, + Duration: time.Minute, + Buckets: Count1KBuckets, + }), NestedStructGauge: NestedStruct{ NestedStructGauge: NewGauge(Metadata{Name: "nested.struct.gauge"}), }, @@ -119,10 +129,15 @@ func TestRegistry(t *testing.T) { NestedStructGauge: NewGauge(Metadata{Name: "nested.struct.array.1.gauge"}), }, }, - privateStructGauge: NewGauge(Metadata{Name: "private.struct.gauge"}), - privateStructGauge64: NewGaugeFloat64(Metadata{Name: "private.struct.gauge64"}), - privateStructCounter: NewCounter(Metadata{Name: "private.struct.counter"}), - privateStructHistogram: NewHistogram(Metadata{Name: "private.struct.histogram"}, time.Minute, Count1KBuckets), + privateStructGauge: NewGauge(Metadata{Name: "private.struct.gauge"}), + privateStructGauge64: NewGaugeFloat64(Metadata{Name: "private.struct.gauge64"}), + privateStructCounter: NewCounter(Metadata{Name: "private.struct.counter"}), + privateStructHistogram: NewHistogram(HistogramOptions{ + Mode: HistogramModePrometheus, + Metadata: Metadata{Name: "private.struct.histogram"}, + Duration: time.Minute, + Buckets: Count1KBuckets, + }), privateNestedStructGauge: NestedStruct{ NestedStructGauge: NewGauge(Metadata{Name: "private.nested.struct.gauge"}), }, diff --git a/pkg/util/mon/bytes_usage.go b/pkg/util/mon/bytes_usage.go index 4ea0489afa20..5ba2493730dd 100644 --- a/pkg/util/mon/bytes_usage.go +++ b/pkg/util/mon/bytes_usage.go @@ -195,7 +195,7 @@ type BytesMonitor struct { // maxBytesHist is the metric object used to track the high watermark of bytes // allocated by the monitor during its lifetime. - maxBytesHist *metric.Histogram + maxBytesHist metric.IHistogram } // name identifies this monitor in logging messages. @@ -273,7 +273,7 @@ func NewMonitor( name redact.RedactableString, res Resource, curCount *metric.Gauge, - maxHist *metric.Histogram, + maxHist metric.IHistogram, increment int64, noteworthy int64, settings *cluster.Settings, @@ -289,7 +289,7 @@ func NewMonitorWithLimit( res Resource, limit int64, curCount *metric.Gauge, - maxHist *metric.Histogram, + maxHist metric.IHistogram, increment int64, noteworthy int64, settings *cluster.Settings, @@ -386,7 +386,7 @@ func NewUnlimitedMonitor( name redact.RedactableString, res Resource, curCount *metric.Gauge, - maxHist *metric.Histogram, + maxHist metric.IHistogram, noteworthy int64, settings *cluster.Settings, ) *BytesMonitor { @@ -485,7 +485,7 @@ func (mm *BytesMonitor) AllocBytes() int64 { } // SetMetrics sets the metric objects for the monitor. -func (mm *BytesMonitor) SetMetrics(curCount *metric.Gauge, maxHist *metric.Histogram) { +func (mm *BytesMonitor) SetMetrics(curCount *metric.Gauge, maxHist metric.IHistogram) { mm.mu.Lock() defer mm.mu.Unlock() mm.mu.curBytesCount = curCount diff --git a/pkg/util/schedulerlatency/scheduler_latency_test.go b/pkg/util/schedulerlatency/scheduler_latency_test.go index 1ec15ed46987..2539e6f62adb 100644 --- a/pkg/util/schedulerlatency/scheduler_latency_test.go +++ b/pkg/util/schedulerlatency/scheduler_latency_test.go @@ -170,7 +170,12 @@ func TestComputeSchedulerPercentileAgainstPrometheus(t *testing.T) { } // Compare values against metric.Histogram (prometheus-based implementation) - promhist := metric.NewHistogram(metric.Metadata{}, time.Hour, hist.Buckets) + promhist := metric.NewHistogram(metric.HistogramOptions{ + Mode: metric.HistogramModePrometheus, + Metadata: metric.Metadata{}, + Duration: time.Hour, + Buckets: hist.Buckets, + }) for i := 0; i < len(hist.Counts); i++ { for j := 0; j < int(hist.Counts[i]); j++ { // Since the scheduler buckets are non-inclusive of Upper Bound and prometheus