diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index 0ecbdb630aa5..5c290bfec99d 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -208,6 +208,7 @@ go_library( "@com_github_gogo_protobuf//proto", "@com_github_google_btree//:btree", "@com_github_kr_pretty//:pretty", + "@com_github_prometheus_client_golang//prometheus", "@io_etcd_go_etcd_raft_v3//:raft", "@io_etcd_go_etcd_raft_v3//raftpb", "@io_etcd_go_etcd_raft_v3//tracker", diff --git a/pkg/kv/kvserver/metrics.go b/pkg/kv/kvserver/metrics.go index 8a47f9e2e975..e071d5e2c3b1 100644 --- a/pkg/kv/kvserver/metrics.go +++ b/pkg/kv/kvserver/metrics.go @@ -31,6 +31,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/slidingwindow" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/prometheus/client_golang/prometheus" "go.etcd.io/etcd/raft/v3/raftpb" ) @@ -1752,7 +1753,7 @@ type StoreMetrics struct { RaftCommandCommitLatency *metric.Histogram RaftHandleReadyLatency *metric.Histogram RaftApplyCommittedLatency *metric.Histogram - RaftSchedulerLatency *metric.Histogram + RaftSchedulerLatency *metric.HistogramV2 RaftTimeoutCampaign *metric.Counter // Raft message metrics. @@ -2255,8 +2256,10 @@ func newStoreMetrics(histogramWindow time.Duration) *StoreMetrics { RaftCommandCommitLatency: metric.NewLatency(metaRaftCommandCommitLatency, histogramWindow), RaftHandleReadyLatency: metric.NewLatency(metaRaftHandleReadyLatency, histogramWindow), RaftApplyCommittedLatency: metric.NewLatency(metaRaftApplyCommittedLatency, histogramWindow), - RaftSchedulerLatency: metric.NewLatency(metaRaftSchedulerLatency, histogramWindow), - RaftTimeoutCampaign: metric.NewCounter(metaRaftTimeoutCampaign), + RaftSchedulerLatency: metric.NewHistogramV2(metaRaftSchedulerLatency, histogramWindow, prometheus.HistogramOpts{ + Buckets: metric.IOLatencyBuckets, + }), + RaftTimeoutCampaign: metric.NewCounter(metaRaftTimeoutCampaign), // Raft message metrics. RaftRcvdMessages: [...]*metric.Counter{ diff --git a/pkg/kv/kvserver/scheduler.go b/pkg/kv/kvserver/scheduler.go index dbfdddb843d6..2ed446471219 100644 --- a/pkg/kv/kvserver/scheduler.go +++ b/pkg/kv/kvserver/scheduler.go @@ -169,7 +169,7 @@ type raftScheduleState struct { type raftScheduler struct { ambientContext log.AmbientContext processor raftProcessor - latency *metric.Histogram + latency *metric.HistogramV2 numWorkers int mu struct { diff --git a/pkg/server/status/BUILD.bazel b/pkg/server/status/BUILD.bazel index 216b2b0da11d..a4ddbc1fa350 100644 --- a/pkg/server/status/BUILD.bazel +++ b/pkg/server/status/BUILD.bazel @@ -68,6 +68,8 @@ go_library( "@com_github_codahale_hdrhistogram//:hdrhistogram", "@com_github_dustin_go_humanize//:go-humanize", "@com_github_elastic_gosigar//:gosigar", + "@com_github_prometheus_client_golang//prometheus", + "@com_github_prometheus_client_model//go", "@com_github_shirou_gopsutil_v3//net", ] + select({ "@io_bazel_rules_go//go/platform:aix": [ diff --git a/pkg/server/status/recorder.go b/pkg/server/status/recorder.go index 464eaa160eb8..3667b5eb6741 100644 --- a/pkg/server/status/recorder.go +++ b/pkg/server/status/recorder.go @@ -44,9 +44,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/system" "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" - "github.com/codahale/hdrhistogram" - humanize "github.com/dustin/go-humanize" + "github.com/dustin/go-humanize" "github.com/elastic/gosigar" + prometheusgo "github.com/prometheus/client_model/go" ) const ( @@ -531,12 +531,10 @@ func extractValue(name string, mtr interface{}, fn func(string, float64)) error // TODO(tschottdorf,ajwerner): consider moving this switch to a single // interface implemented by the individual metric types. type ( - float64Valuer interface{ Value() float64 } - int64Valuer interface{ Value() int64 } - int64Counter interface{ Count() int64 } - histogramValuer interface { - Windowed() (*hdrhistogram.Histogram, time.Duration) - } + float64Valuer interface{ Value() float64 } + int64Valuer interface{ Value() int64 } + int64Counter interface{ Count() int64 } + prometheusMetricValuer interface{ ToPrometheusMetric() *prometheusgo.Metric } ) switch mtr := mtr.(type) { case float64: @@ -547,7 +545,7 @@ func extractValue(name string, mtr interface{}, fn func(string, float64)) error fn(name, float64(mtr.Value())) case int64Counter: fn(name, float64(mtr.Count())) - case histogramValuer: + case *metric.Histogram: // TODO(mrtracy): Where should this comment go for better // visibility? // @@ -567,6 +565,21 @@ func extractValue(name string, mtr interface{}, fn func(string, float64)) error fn(name+pt.suffix, float64(curr.ValueAtQuantile(pt.quantile))) } fn(name+"-count", float64(curr.TotalCount())) + case *metric.HistogramV2: + // NB: this branch is intentionally at the bottom since all metrics implement it. + cur := mtr.Windowed() + var m prometheusgo.Metric + _ = cur.Write(&m) + hist := m.Histogram + n := float64(*hist.SampleCount) + fn(name+"-count", n) + fn(name+"-avg", *hist.SampleSum/n) + // TODO(obs-inf): add quantiles like for the hdrhistogram. + case prometheusMetricValuer: + // TODO we should be able to handle all non-histogram branches using this, i.e. + // can delete the float, int, etc, cases above. + _ = mtr.ToPrometheusMetric() + default: return errors.Errorf("cannot extract value for type %T", mtr) } diff --git a/pkg/util/metric/BUILD.bazel b/pkg/util/metric/BUILD.bazel index cf5b88fac4c5..5db5d0bfe2dc 100644 --- a/pkg/util/metric/BUILD.bazel +++ b/pkg/util/metric/BUILD.bazel @@ -14,8 +14,8 @@ go_library( "registry.go", "rule.go", "rule_registry.go", - "sliding_histogram.go", "test_helpers.go", + "tick_helper.go", ], embed = [":metric_go_proto"], importpath = "github.com/cockroachdb/cockroach/pkg/util/metric", diff --git a/pkg/util/metric/metric.go b/pkg/util/metric/metric.go index 2ce62329a5d3..f7d1c833b67e 100644 --- a/pkg/util/metric/metric.go +++ b/pkg/util/metric/metric.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/codahale/hdrhistogram" "github.com/gogo/protobuf/proto" + "github.com/prometheus/client_golang/prometheus" prometheusgo "github.com/prometheus/client_model/go" metrics "github.com/rcrowley/go-metrics" ) @@ -185,7 +186,8 @@ type Histogram struct { mu struct { syncutil.Mutex cumulative *hdrhistogram.Histogram - sliding *slidingHistogram + *tickHelper + sliding *hdrhistogram.WindowedHistogram } } @@ -194,13 +196,20 @@ type Histogram struct { // track nonnegative values up to 'maxVal' with 'sigFigs' decimal points of // precision. func NewHistogram(metadata Metadata, duration time.Duration, maxVal int64, sigFigs int) *Histogram { - dHist := newSlidingHistogram(duration, maxVal, sigFigs) h := &Histogram{ Metadata: metadata, maxVal: maxVal, } + wHist := hdrhistogram.NewWindowed(histWrapNum, 0, maxVal, sigFigs) h.mu.cumulative = hdrhistogram.New(0, maxVal, sigFigs) - h.mu.sliding = dHist + h.mu.sliding = wHist + h.mu.tickHelper = &tickHelper{ + nextT: now(), + tickInterval: duration / histWrapNum, + onTick: func() { + wHist.Rotate() + }, + } return h } @@ -222,7 +231,9 @@ func NewLatency(metadata Metadata, histogramWindow time.Duration) *Histogram { func (h *Histogram) Windowed() (*hdrhistogram.Histogram, time.Duration) { h.mu.Lock() defer h.mu.Unlock() - return cloneHistogram(h.mu.sliding.Current()), h.mu.sliding.duration + // TODO(obs-inf): not sure we should multiply by histWrapNum here, but it + // has been the behavior for a long time. + return cloneHistogram(h.mu.sliding.Merge()), histWrapNum * h.mu.tickInterval } // Snapshot returns a copy of the cumulative (i.e. all-time samples) histogram @@ -240,8 +251,8 @@ func (h *Histogram) RecordValue(v int64) { h.mu.Lock() defer h.mu.Unlock() - if h.mu.sliding.RecordValue(v) != nil { - _ = h.mu.sliding.RecordValue(h.maxVal) + 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) @@ -265,7 +276,7 @@ func (h *Histogram) Min() int64 { // Inspect calls the closure with the empty string and the receiver. func (h *Histogram) Inspect(f func(interface{})) { h.mu.Lock() - maybeTick(h.mu.sliding) + maybeTick(h.mu.tickHelper) h.mu.Unlock() f(h) } @@ -280,7 +291,7 @@ func (h *Histogram) ToPrometheusMetric() *prometheusgo.Metric { hist := &prometheusgo.Histogram{} h.mu.Lock() - maybeTick(h.mu.sliding) + maybeTick(h.mu.tickHelper) bars := h.mu.cumulative.Distribution() hist.Bucket = make([]*prometheusgo.Bucket, 0, len(bars)) @@ -319,6 +330,155 @@ func (h *Histogram) GetMetadata() Metadata { return baseMetadata } +// IOLatencyBuckets are prometheus histogram buckets suitable for a histogram +// that records a quantity (nanosecond-denominated) in which most measurements +// resemble those of typical disk latencies, i.e. which are in the micro- and +// millisecond range during normal operation. +var IOLatencyBuckets = []float64{ + // Generated via TestHistogramBuckets/IOLatencyBuckets. + 10000.000000, // 10µs + 26826.957953, // 26.826µs + 71968.567300, // 71.968µs + 193069.772888, // 193.069µs + 517947.467923, // 517.947µs + 1389495.494373, // 1.389495ms + 3727593.720315, // 3.727593ms + 10000000.000000, // 9.999999ms + 26826957.952797, // 26.826957ms + 71968567.300115, // 71.968567ms + 193069772.888325, // 193.069772ms + 517947467.923120, // 517.947467ms + 1389495494.373135, // 1.389495494s + 3727593720.314933, // 3.72759372s + 9999999999.999981, // 9.999999999s +} + +// NetworkLatencyBuckets are prometheus histogram buckets suitable for a histogram +// that records a quantity (nanosecond-denominated) in which most measurements +// behave like network latencies, i.e. most measurements are in the ms to sub-second +// range during normal operation. +var NetworkLatencyBuckets = []float64{ + // Generated via TestHistogramBuckets/NetworkLatencyBuckets. + 500000.000000, // 500µs + 860513.842995, // 860.513µs + 1480968.147973, // 1.480968ms + 2548787.184731, // 2.548787ms + 4386533.310619, // 4.386533ms + 7549345.273094, // 7.549345ms + 12992632.226094, // 12.992632ms + 22360679.774998, // 22.360679ms + 38483348.970335, // 38.483348ms + 66230909.027573, // 66.230909ms + 113985228.104760, // 113.985228ms + 196171733.362212, // 196.171733ms + 337616984.325077, // 337.616984ms + 581048177.284016, // 581.048177ms + 999999999.999999, // 999.999999ms, +} + +// NewHistogramV2 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 NewHistogramV2( + meta Metadata, windowDuration time.Duration, opts prometheus.HistogramOpts, +) *HistogramV2 { + // TODO(obs-inf): prometheus supports labeled histograms but they require more + // plumbing and don't fit into the PrometheusObservable interface any more. + cum := prometheus.NewHistogram(opts) + h := &HistogramV2{ + Metadata: meta, + cum: cum, + } + h.windowed.tickHelper = &tickHelper{ + nextT: now(), + tickInterval: windowDuration, + onTick: func() { + h.windowed.prev = h.windowed.cur + h.windowed.cur = prometheus.NewHistogram(opts) + }, + } + h.windowed.tickHelper.onTick() + return h +} + +var _ periodic = (*HistogramV2)(nil) +var _ PrometheusExportable = (*HistogramV2)(nil) + +type HistogramV2 struct { + Metadata + cum prometheus.Histogram + + // TODO(obs-inf): the way we implement windowed histograms is not great. If + // the windowed histogram is pulled right after a tick, it will be mostly + // empty. We could add a third bucket and represent the merged view of the two + // most recent buckets to avoid that. Or we could "just" double the rotation + // interval (so that the histogram really collects for 20s when we expect to + // persist the contents every 10s). Really it would make more sense to + // explicitly rotate the histogram atomically with collecting its contents, + // but that is now how we have set it up right now. It should be doable + // though, since there is only one consumer of windowed histograms - our + // internal timeseries system. + windowed struct { + // prometheus.Histogram is thread safe, so we only + // need an RLock to record into it. But write lock + // is held while rotating. + syncutil.RWMutex + *tickHelper + prev, cur prometheus.Histogram + } +} + +func (h *HistogramV2) nextTick() time.Time { + h.windowed.RLock() + defer h.windowed.RUnlock() + return h.windowed.nextTick() +} + +func (h *HistogramV2) tick() { + h.windowed.Lock() + defer h.windowed.Unlock() + h.windowed.tick() +} + +func (h *HistogramV2) Windowed() prometheus.Histogram { + h.windowed.RLock() + defer h.windowed.RUnlock() + return h.windowed.cur +} + +func (h *HistogramV2) RecordValue(n int64) { + v := float64(n) + h.cum.Observe(v) + + h.windowed.RLock() + defer h.windowed.RUnlock() + h.windowed.cur.Observe(v) + +} + +func (h *HistogramV2) GetType() *prometheusgo.MetricType { + return prometheusgo.MetricType_HISTOGRAM.Enum() +} + +func (h *HistogramV2) ToPrometheusMetric() *prometheusgo.Metric { + m := &prometheusgo.Metric{} + if err := h.cum.Write(m); err != nil { + panic(err) // TODD + } + return m +} + +func (h *HistogramV2) GetMetadata() Metadata { + return h.Metadata +} + +func (h *HistogramV2) Inspect(f func(interface{})) { + h.windowed.Lock() + maybeTick(&h.windowed) + h.windowed.Unlock() + f(h) +} + // A Counter holds a single mutable atomic value. type Counter struct { Metadata diff --git a/pkg/util/metric/metric_test.go b/pkg/util/metric/metric_test.go index 78f4aa040dcc..619886306d8e 100644 --- a/pkg/util/metric/metric_test.go +++ b/pkg/util/metric/metric_test.go @@ -13,15 +13,19 @@ package metric import ( "bytes" "encoding/json" + "fmt" "math" "reflect" + "strings" "sync" "testing" "time" _ "github.com/cockroachdb/cockroach/pkg/util/log" // for flags "github.com/kr/pretty" + "github.com/prometheus/client_golang/prometheus" prometheusgo "github.com/prometheus/client_model/go" + "github.com/stretchr/testify/require" ) func testMarshal(t *testing.T, m json.Marshaler, exp string) { @@ -135,6 +139,58 @@ func TestHistogramPrometheus(t *testing.T) { } } +func TestHistogramBuckets(t *testing.T) { + verifyAndPrint := func(t *testing.T, exp, act []float64) { + t.Helper() + var buf strings.Builder + for idx, f := range exp { + if idx == 0 { + fmt.Fprintf(&buf, "// Generated via %s.", t.Name()) + } + fmt.Fprintf(&buf, "\n%f, // %s", f, time.Duration(f)) + } + t.Logf("%s", &buf) + require.InDeltaSlice(t, exp, act, 1 /* delta */, "Please update the bucket boundaries for %s", t.Name()) + } + t.Run("IOLatencyBuckets", func(t *testing.T) { + exp := prometheus.ExponentialBucketsRange(10e3, 10e9, 15) + verifyAndPrint(t, exp, IOLatencyBuckets) + }) + + t.Run("NetworkLatencyBuckets", func(t *testing.T) { + exp := prometheus.ExponentialBucketsRange(500e3, 1e9, 15) + verifyAndPrint(t, exp, NetworkLatencyBuckets) + }) +} + +func TestNewHistogramV2Rotate(t *testing.T) { + defer TestingSetNow(nil)() + setNow(0) + + h := NewHistogramV2(emptyMetadata, 10*time.Second, prometheus.HistogramOpts{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) + // But cumulative histogram has history (if i > 0). + require.EqualValues(t, i, *h.ToPrometheusMetric().Histogram.SampleCount) + + // 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) + } + // Tick. This rotates the histogram. + setNow(time.Duration(i+1) * 10 * time.Second) + // Go to beginning. + } +} + func TestHistogramRotate(t *testing.T) { defer TestingSetNow(nil)() setNow(0) diff --git a/pkg/util/metric/sliding_histogram.go b/pkg/util/metric/sliding_histogram.go deleted file mode 100644 index 20262befc1ef..000000000000 --- a/pkg/util/metric/sliding_histogram.go +++ /dev/null @@ -1,61 +0,0 @@ -// Copyright 2015 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/codahale/hdrhistogram" -) - -var _ periodic = &slidingHistogram{} - -// A deprecatedWindowedHistogram is a wrapper around an -// hdrhistogram.WindowedHistogram. The caller must enforce proper -// synchronization. -type slidingHistogram struct { - windowed *hdrhistogram.WindowedHistogram - nextT time.Time - duration time.Duration -} - -// newSlidingHistogram creates a new windowed HDRHistogram with the given -// parameters. Data is kept in the active window for approximately the given -// duration. See the documentation for hdrhistogram.WindowedHistogram for -// details. -func newSlidingHistogram(duration time.Duration, maxVal int64, sigFigs int) *slidingHistogram { - if duration <= 0 { - panic("cannot create a sliding histogram with nonpositive duration") - } - return &slidingHistogram{ - nextT: now(), - duration: duration, - windowed: hdrhistogram.NewWindowed(histWrapNum, 0, maxVal, sigFigs), - } -} - -func (h *slidingHistogram) tick() { - h.nextT = h.nextT.Add(h.duration / histWrapNum) - h.windowed.Rotate() -} - -func (h *slidingHistogram) nextTick() time.Time { - return h.nextT -} - -func (h *slidingHistogram) Current() *hdrhistogram.Histogram { - maybeTick(h) - return h.windowed.Merge() -} - -func (h *slidingHistogram) RecordValue(v int64) error { - return h.windowed.Current.RecordValue(v) -} diff --git a/pkg/util/metric/tick_helper.go b/pkg/util/metric/tick_helper.go new file mode 100644 index 000000000000..fbc6ab19e96a --- /dev/null +++ b/pkg/util/metric/tick_helper.go @@ -0,0 +1,34 @@ +// Copyright 2015 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" + +var _ periodic = &tickHelper{} + +// tickHelper is used by metrics that are at heart cumulative, but wish to also +// maintain a windowed version to work around limitations of our internal +// timeseries database. +type tickHelper struct { + nextT time.Time + tickInterval time.Duration + + onTick func() +} + +func (s *tickHelper) nextTick() time.Time { + return s.nextT +} + +func (s *tickHelper) tick() { + s.nextT = s.nextT.Add(s.tickInterval) + s.onTick() +}