Skip to content

Commit

Permalink
metric: migrate all histograms to use prometheus-backed version
Browse files Browse the repository at this point in the history
In a previous change, a new prometheus-backed histogram library was
introdced to help standardize histogram buckets across the codebase.
This change migrates all existing histograms to use the new library.

In this change, `NewLatency()` is removed in favor of explicitly defining
which buckets to use between `NetworkLatencyBuckets` and
`IOLatencyBuckets` when calling `NewHistogram()`. For all histograms
that were previously created using the `NewLatency()` func, I tried to
place them in appropriate buckets with the new library. For cases where
it was unclear, I chose `IOLatencyBuckets` as it allows for a larger
range of values.

related: #85990

Release justification: low risk, high benefit

Release note (ops change): This change introduces a new histogram
implementation that will reduce the total number of buckets and
standardize them across all usage. This should help increase the
usability of histograms when exxported to a UI (i.e. Grafana) and reduce
the storage overhead.

After applying this patch it is expected to see fewer buckets in
prometheus/grafana, but still  have similar values for histogram
percentiles due to the use of interpolated values by Prometheus.
  • Loading branch information
aadityasondhi committed Aug 31, 2022
1 parent cb57def commit ddbd1cd
Show file tree
Hide file tree
Showing 34 changed files with 556 additions and 614 deletions.
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/changefeed_processors.go
Original file line number Diff line number Diff line change
Expand Up @@ -817,7 +817,7 @@ func (j *jobState) checkpointCompleted(ctx context.Context, checkpointDuration t

j.metrics.CheckpointHistNanos.RecordValue(checkpointDuration.Nanoseconds())
j.lastProgressUpdate = j.ts.Now()
j.checkpointDuration = time.Duration(j.metrics.CheckpointHistNanos.Snapshot().Mean())
j.checkpointDuration = time.Duration(j.metrics.CheckpointHistNanos.Mean())
j.progressUpdatesSkipped = false
}

Expand Down
52 changes: 19 additions & 33 deletions pkg/ccl/changefeedccl/metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -284,14 +284,6 @@ func (w *wrappingCostController) getBackfillRangeCallback() func(int64) (func(),
return w.inner.getBackfillRangeCallback()
}

const (
changefeedCheckpointHistMaxLatency = 30 * time.Second
changefeedBatchHistMaxLatency = 30 * time.Second
changefeedFlushHistMaxLatency = 1 * time.Minute
admitLatencyMaxValue = 1 * time.Minute
commitLatencyMaxValue = 10 * time.Minute
)

var (
metaChangefeedForwardedResolvedMessages = metric.Metadata{
Name: "changefeed.forwarded_resolved_messages",
Expand Down Expand Up @@ -450,20 +442,15 @@ func newAggregateMetrics(histogramWindow time.Duration) *AggMetrics {
a := &AggMetrics{
ErrorRetries: b.Counter(metaChangefeedErrorRetries),
EmittedMessages: b.Counter(metaChangefeedEmittedMessages),
MessageSize: b.Histogram(metaMessageSize,
histogramWindow, 10<<20 /* 10MB max message size */, 1),
EmittedBytes: b.Counter(metaChangefeedEmittedBytes),
FlushedBytes: b.Counter(metaChangefeedFlushedBytes),
Flushes: b.Counter(metaChangefeedFlushes),

BatchHistNanos: b.Histogram(metaChangefeedBatchHistNanos,
histogramWindow, changefeedBatchHistMaxLatency.Nanoseconds(), 1),
FlushHistNanos: b.Histogram(metaChangefeedFlushHistNanos,
histogramWindow, changefeedFlushHistMaxLatency.Nanoseconds(), 2),
CommitLatency: b.Histogram(metaCommitLatency,
histogramWindow, commitLatencyMaxValue.Nanoseconds(), 1),
AdmitLatency: b.Histogram(metaAdmitLatency, histogramWindow,
admitLatencyMaxValue.Nanoseconds(), 1),
MessageSize: b.Histogram(metaMessageSize, histogramWindow, metric.DataSize16MBBuckets),
EmittedBytes: b.Counter(metaChangefeedEmittedBytes),
FlushedBytes: b.Counter(metaChangefeedFlushedBytes),
Flushes: b.Counter(metaChangefeedFlushes),

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),
BackfillCount: b.Gauge(metaChangefeedBackfillCount),
BackfillPendingRanges: b.Gauge(metaChangefeedBackfillPendingRanges),
RunningCount: b.Gauge(metaChangefeedRunning),
Expand Down Expand Up @@ -566,17 +553,16 @@ 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,
changefeedCheckpointHistMaxLatency.Nanoseconds(), 2),
FrontierUpdates: metric.NewCounter(metaChangefeedFrontierUpdates),
ThrottleMetrics: cdcutils.MakeMetrics(histogramWindow),
ReplanCount: metric.NewCounter(metaChangefeedReplanCount),
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),
}

m.mu.resolved = make(map[int]hlc.Timestamp)
Expand Down
10 changes: 7 additions & 3 deletions pkg/ccl/sqlproxyccl/connector_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -375,7 +375,9 @@ func TestConnector_dialTenantCluster(t *testing.T) {
defer cancel()

c := &connector{
DialTenantLatency: metric.NewLatency(metaDialTenantLatency, time.Millisecond),
DialTenantLatency: metric.NewHistogram(
metaDialTenantLatency, time.Millisecond, metric.NetworkLatencyBuckets,
),
DialTenantRetries: metric.NewCounter(metaDialTenantRetries),
}
c.testingKnobs.lookupAddr = func(ctx context.Context) (string, error) {
Expand Down Expand Up @@ -403,8 +405,10 @@ func TestConnector_dialTenantCluster(t *testing.T) {

var reportFailureFnCount int
c := &connector{
TenantID: roachpb.MakeTenantID(42),
DialTenantLatency: metric.NewLatency(metaDialTenantLatency, time.Millisecond),
TenantID: roachpb.MakeTenantID(42),
DialTenantLatency: metric.NewHistogram(
metaDialTenantLatency, time.Millisecond, metric.NetworkLatencyBuckets,
),
DialTenantRetries: metric.NewCounter(metaDialTenantRetries),
}
c.DirectoryCache = &testTenantDirectoryCache{
Expand Down
22 changes: 7 additions & 15 deletions pkg/ccl/sqlproxyccl/metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,16 +50,6 @@ 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",
Expand Down Expand Up @@ -224,32 +214,34 @@ func makeProxyMetrics() metrics {
RoutingErrCount: metric.NewCounter(metaRoutingErrCount),
RefusedConnCount: metric.NewCounter(metaRefusedConnCount),
SuccessfulConnCount: metric.NewCounter(metaSuccessfulConnCount),
ConnectionLatency: metric.NewLatency(
ConnectionLatency: metric.NewHistogram(
metaConnMigrationAttemptedCount,
base.DefaultHistogramWindowInterval(),
metric.NetworkLatencyBuckets,
),
AuthFailedCount: metric.NewCounter(metaAuthFailedCount),
ExpiredClientConnCount: metric.NewCounter(metaExpiredClientConnCount),
// Connector metrics.
DialTenantLatency: metric.NewLatency(
DialTenantLatency: metric.NewHistogram(
metaDialTenantLatency,
base.DefaultHistogramWindowInterval(),
metric.NetworkLatencyBuckets,
),
DialTenantRetries: metric.NewCounter(metaDialTenantRetries),
// Connection migration metrics.
ConnMigrationSuccessCount: metric.NewCounter(metaConnMigrationSuccessCount),
ConnMigrationErrorFatalCount: metric.NewCounter(metaConnMigrationErrorFatalCount),
ConnMigrationErrorRecoverableCount: metric.NewCounter(metaConnMigrationErrorRecoverableCount),
ConnMigrationAttemptedCount: metric.NewCounter(metaConnMigrationAttemptedCount),
ConnMigrationAttemptedLatency: metric.NewLatency(
ConnMigrationAttemptedLatency: metric.NewHistogram(
metaConnMigrationAttemptedLatency,
base.DefaultHistogramWindowInterval(),
metric.NetworkLatencyBuckets,
),
ConnMigrationTransferResponseMessageSize: metric.NewHistogram(
metaConnMigrationTransferResponseMessageSize,
base.DefaultHistogramWindowInterval(),
maxExpectedTransferResponseMessageSize,
1,
metric.DataSize16MBBuckets,
),
QueryCancelReceivedPGWire: metric.NewCounter(metaQueryCancelReceivedPGWire),
QueryCancelReceivedHTTP: metric.NewCounter(metaQueryCancelReceivedHTTP),
Expand Down
12 changes: 3 additions & 9 deletions pkg/ccl/streamingccl/streamingest/metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,12 +15,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/metric"
)

const (
streamingFlushHistMaxLatency = 1 * time.Minute
streamingAdmitLatencyMaxValue = 3 * time.Minute
streamingCommitLatencyMaxValue = 10 * time.Minute
)

var (
metaStreamingEventsIngested = metric.Metadata{
Name: "streaming.events_ingested",
Expand Down Expand Up @@ -135,11 +129,11 @@ func MakeMetrics(histogramWindow time.Duration) metric.Struct {
ResolvedEvents: metric.NewCounter(metaStreamingResolvedEventsIngested),
JobProgressUpdates: metric.NewCounter(metaJobProgressUpdates),
FlushHistNanos: metric.NewHistogram(metaStreamingFlushHistNanos,
histogramWindow, streamingFlushHistMaxLatency.Nanoseconds(), 1),
histogramWindow, metric.BatchProcessLatencyBuckets),
CommitLatency: metric.NewHistogram(metaStreamingCommitLatency,
histogramWindow, streamingCommitLatencyMaxValue.Nanoseconds(), 1),
histogramWindow, metric.BatchProcessLatencyBuckets),
AdmitLatency: metric.NewHistogram(metaStreamingAdmitLatency,
histogramWindow, streamingAdmitLatencyMaxValue.Nanoseconds(), 1),
histogramWindow, metric.BatchProcessLatencyBuckets),
RunningCount: metric.NewGauge(metaStreamsRunning),
EarliestDataCheckpointSpan: metric.NewGauge(metaEarliestDataCheckpointSpan),
LatestDataCheckpointSpan: metric.NewGauge(metaLatestDataCheckpointSpan),
Expand Down
6 changes: 1 addition & 5 deletions pkg/kv/bulk/bulk_metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,14 +43,10 @@ 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, log10int64times1000, 3),
MaxBytesHist: metric.NewHistogram(metaMemMaxBytes, histogramWindow, metric.MemoryUsage64MBBuckets),
CurBytesCount: metric.NewGauge(metaMemCurBytes),
}
}
33 changes: 9 additions & 24 deletions pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1073,15 +1073,12 @@ func checkTxnMetrics(
commits, commits1PC, aborts, restarts int64,
) {
testutils.SucceedsSoon(t, func() error {
return checkTxnMetricsOnce(t, metrics, name, commits, commits1PC, aborts, restarts)
return checkTxnMetricsOnce(metrics, name, commits, commits1PC, aborts, restarts)
})
}

func checkTxnMetricsOnce(
t *testing.T,
metrics kvcoord.TxnMetrics,
name string,
commits, commits1PC, aborts, restarts int64,
metrics kvcoord.TxnMetrics, name string, commits, commits1PC, aborts, restarts int64,
) error {
testcases := []struct {
name string
Expand All @@ -1091,6 +1088,7 @@ func checkTxnMetricsOnce(
{"commits1PC", metrics.Commits1PC.Count(), commits1PC},
{"aborts", metrics.Aborts.Count(), aborts},
{"durations", metrics.Durations.TotalCount(), commits + aborts},
{"restarts", metrics.Restarts.TotalCount(), restarts},
}

for _, tc := range testcases {
Expand All @@ -1099,22 +1097,6 @@ func checkTxnMetricsOnce(
}
}

// Handle restarts separately, because that's a histogram. Though the
// histogram is approximate, we're recording so few distinct values
// that we should be okay.
dist := metrics.Restarts.Snapshot().Distribution()
var actualRestarts int64
for _, b := range dist {
if b.From == b.To {
actualRestarts += b.From * b.Count
} else {
t.Fatalf("unexpected value in histogram: %d-%d", b.From, b.To)
}
}
if a, e := actualRestarts, restarts; a != e {
return errors.Errorf("%s: actual restarts %d != expected %d", name, a, e)
}

return nil
}

Expand Down Expand Up @@ -1324,10 +1306,13 @@ func TestTxnDurations(t *testing.T) {
t.Fatalf("durations %d != expected %d", a, e)
}

// Metrics lose fidelity, so we can't compare incr directly.
if min, thresh := hist.Min(), (incr - 10).Nanoseconds(); min < thresh {
t.Fatalf("min %d < %d", min, thresh)
for _, b := range hist.ToPrometheusMetric().GetHistogram().GetBucket() {
thresh := incr.Nanoseconds()
if *b.UpperBound < float64(thresh) && *b.CumulativeCount != 0 {
t.Fatalf("expected no values in bucket: %f", *b.UpperBound)
}
}

}

// TestTxnCommitWait tests the commit-wait sleep phase of transactions under
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvclient/kvcoord/txn_metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -274,11 +274,11 @@ func MakeTxnMetrics(histogramWindow time.Duration) TxnMetrics {
RefreshFailWithCondensedSpans: metric.NewCounter(metaRefreshFailWithCondensedSpans),
RefreshMemoryLimitExceeded: metric.NewCounter(metaRefreshMemoryLimitExceeded),
RefreshAutoRetries: metric.NewCounter(metaRefreshAutoRetries),
Durations: metric.NewLatency(metaDurationsHistograms, histogramWindow),
Durations: metric.NewHistogram(metaDurationsHistograms, histogramWindow, metric.IOLatencyBuckets),
TxnsWithCondensedIntents: metric.NewCounter(metaTxnsWithCondensedIntentSpans),
TxnsWithCondensedIntentsGauge: metric.NewGauge(metaTxnsWithCondensedIntentSpansGauge),
TxnsRejectedByLockSpanBudget: metric.NewCounter(metaTxnsRejectedByLockSpanBudget),
Restarts: metric.NewHistogram(metaRestartsHistogram, histogramWindow, 100, 3),
Restarts: metric.NewHistogram(metaRestartsHistogram, histogramWindow, metric.Count1KBuckets),
RestartsWriteTooOld: telemetry.NewCounterWithMetric(metaRestartsWriteTooOld),
RestartsWriteTooOldMulti: telemetry.NewCounterWithMetric(metaRestartsWriteTooOldMulti),
RestartsSerializable: telemetry.NewCounterWithMetric(metaRestartsSerializable),
Expand Down
16 changes: 10 additions & 6 deletions pkg/kv/kvprober/kvprober.go
Original file line number Diff line number Diff line change
Expand Up @@ -214,14 +214,18 @@ func NewProber(opts Opts) *Prober {
writePlanner: newMeta2Planner(opts.DB, opts.Settings, func() time.Duration { return writeInterval.Get(&opts.Settings.SV) }),

metrics: Metrics{
ReadProbeAttempts: metric.NewCounter(metaReadProbeAttempts),
ReadProbeFailures: metric.NewCounter(metaReadProbeFailures),
ReadProbeLatency: metric.NewLatency(metaReadProbeLatency, opts.HistogramWindowInterval),
ReadProbeAttempts: metric.NewCounter(metaReadProbeAttempts),
ReadProbeFailures: metric.NewCounter(metaReadProbeFailures),
ReadProbeLatency: metric.NewHistogram(
metaReadProbeLatency, opts.HistogramWindowInterval, metric.NetworkLatencyBuckets,
),
WriteProbeAttempts: metric.NewCounter(metaWriteProbeAttempts),
WriteProbeFailures: metric.NewCounter(metaWriteProbeFailures),
WriteProbeLatency: metric.NewLatency(metaWriteProbeLatency, opts.HistogramWindowInterval),
ProbePlanAttempts: metric.NewCounter(metaProbePlanAttempts),
ProbePlanFailures: metric.NewCounter(metaProbePlanFailures),
WriteProbeLatency: metric.NewHistogram(
metaWriteProbeLatency, opts.HistogramWindowInterval, metric.NetworkLatencyBuckets,
),
ProbePlanAttempts: metric.NewCounter(metaProbePlanAttempts),
ProbePlanFailures: metric.NewCounter(metaProbePlanFailures),
},
tracer: opts.Tracer,
}
Expand Down
4 changes: 3 additions & 1 deletion pkg/kv/kvserver/liveness/liveness.go
Original file line number Diff line number Diff line change
Expand Up @@ -309,7 +309,9 @@ func NewNodeLiveness(opts NodeLivenessOptions) *NodeLiveness {
HeartbeatSuccesses: metric.NewCounter(metaHeartbeatSuccesses),
HeartbeatFailures: telemetry.NewCounterWithMetric(metaHeartbeatFailures),
EpochIncrements: telemetry.NewCounterWithMetric(metaEpochIncrements),
HeartbeatLatency: metric.NewLatency(metaHeartbeatLatency, opts.HistogramWindowInterval),
HeartbeatLatency: metric.NewHistogram(
metaHeartbeatLatency, opts.HistogramWindowInterval, metric.NetworkLatencyBuckets,
),
}
nl.mu.nodes = make(map[roachpb.NodeID]Record)
nl.heartbeatToken <- struct{}{}
Expand Down
39 changes: 26 additions & 13 deletions pkg/kv/kvserver/metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -2245,18 +2245,27 @@ func newStoreMetrics(histogramWindow time.Duration) *StoreMetrics {
// Raft processing metrics.
RaftTicks: metric.NewCounter(metaRaftTicks),
RaftQuotaPoolPercentUsed: metric.NewHistogram(
// NB: this results in 64 buckets (i.e. 64 timeseries in prometheus).
metaRaftQuotaPoolPercentUsed, histogramWindow, 100 /* maxVal */, 1, /* sigFigs */
metaRaftQuotaPoolPercentUsed, histogramWindow, metric.Percent100Buckets,
),
RaftWorkingDurationNanos: metric.NewCounter(metaRaftWorkingDurationNanos),
RaftTickingDurationNanos: metric.NewCounter(metaRaftTickingDurationNanos),
RaftCommandsApplied: metric.NewCounter(metaRaftCommandsApplied),
RaftLogCommitLatency: metric.NewLatency(metaRaftLogCommitLatency, histogramWindow),
RaftCommandCommitLatency: metric.NewLatency(metaRaftCommandCommitLatency, histogramWindow),
RaftHandleReadyLatency: metric.NewLatency(metaRaftHandleReadyLatency, histogramWindow),
RaftApplyCommittedLatency: metric.NewLatency(metaRaftApplyCommittedLatency, histogramWindow),
RaftSchedulerLatency: metric.NewLatency(metaRaftSchedulerLatency, histogramWindow),
RaftTimeoutCampaign: metric.NewCounter(metaRaftTimeoutCampaign),
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,
),
RaftTimeoutCampaign: metric.NewCounter(metaRaftTimeoutCampaign),

// Raft message metrics.
RaftRcvdMessages: [...]*metric.Counter{
Expand Down Expand Up @@ -2394,8 +2403,12 @@ func newStoreMetrics(histogramWindow time.Duration) *StoreMetrics {
ReplicaCircuitBreakerCumTripped: metric.NewCounter(metaReplicaCircuitBreakerCumTripped),

// Replica batch evaluation.
ReplicaReadBatchEvaluationLatency: metric.NewLatency(metaReplicaReadBatchEvaluationLatency, histogramWindow),
ReplicaWriteBatchEvaluationLatency: metric.NewLatency(metaReplicaWriteBatchEvaluationLatency, histogramWindow),
ReplicaReadBatchEvaluationLatency: metric.NewHistogram(
metaReplicaReadBatchEvaluationLatency, histogramWindow, metric.IOLatencyBuckets,
),
ReplicaWriteBatchEvaluationLatency: metric.NewHistogram(
metaReplicaWriteBatchEvaluationLatency, histogramWindow, metric.IOLatencyBuckets,
),
}

{
Expand Down
6 changes: 2 additions & 4 deletions pkg/kv/kvserver/txnwait/metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -74,8 +74,7 @@ func NewMetrics(histogramWindowInterval time.Duration) *Metrics {
Unit: metric.Unit_NANOSECONDS,
},
histogramWindowInterval,
time.Hour.Nanoseconds(),
1,
metric.LongRunning60mLatencyBuckets,
),

QueryWaitTime: metric.NewHistogram(
Expand All @@ -86,8 +85,7 @@ func NewMetrics(histogramWindowInterval time.Duration) *Metrics {
Unit: metric.Unit_NANOSECONDS,
},
histogramWindowInterval,
time.Hour.Nanoseconds(),
1,
metric.LongRunning60mLatencyBuckets,
),

DeadlocksTotal: metric.NewCounter(
Expand Down
Loading

0 comments on commit ddbd1cd

Please sign in to comment.