From fd6a28a7da2b0a40b525ee73e33d69cecffc8cf5 Mon Sep 17 00:00:00 2001 From: bwplotka Date: Mon, 1 Dec 2025 11:06:21 +0000 Subject: [PATCH] refactor(scrape): simplified scrapeLoop constructors & tests; add teststorage.Appender mock Signed-off-by: bwplotka --- model/histogram/float_histogram.go | 2 +- model/histogram/histogram.go | 2 +- model/metadata/metadata.go | 31 +- scrape/helpers_test.go | 192 +-- scrape/manager.go | 22 +- scrape/manager_test.go | 98 +- scrape/scrape.go | 457 +++---- scrape/scrape_append.go | 187 +++ scrape/scrape_append_test.go | 260 ++++ scrape/scrape_test.go | 1813 +++++++++++++--------------- scrape/target.go | 136 +-- scrape/target_test.go | 243 +--- util/teststorage/appender.go | 346 ++++++ util/teststorage/appender_test.go | 59 + 14 files changed, 1956 insertions(+), 1892 deletions(-) create mode 100644 scrape/scrape_append.go create mode 100644 scrape/scrape_append_test.go create mode 100644 util/teststorage/appender.go create mode 100644 util/teststorage/appender_test.go diff --git a/model/histogram/float_histogram.go b/model/histogram/float_histogram.go index 91fcac1cfb..0acf9cb28f 100644 --- a/model/histogram/float_histogram.go +++ b/model/histogram/float_histogram.go @@ -484,7 +484,7 @@ func (h *FloatHistogram) Sub(other *FloatHistogram) (res *FloatHistogram, counte // supposed to be used according to the schema. func (h *FloatHistogram) Equals(h2 *FloatHistogram) bool { if h2 == nil { - return false + return h == nil } if h.Schema != h2.Schema || diff --git a/model/histogram/histogram.go b/model/histogram/histogram.go index 5fc68ef9d0..aa9f696be6 100644 --- a/model/histogram/histogram.go +++ b/model/histogram/histogram.go @@ -247,7 +247,7 @@ func (h *Histogram) CumulativeBucketIterator() BucketIterator[uint64] { // supposed to be used according to the schema. func (h *Histogram) Equals(h2 *Histogram) bool { if h2 == nil { - return false + return h == nil } if h.Schema != h2.Schema || h.Count != h2.Count || diff --git a/model/metadata/metadata.go b/model/metadata/metadata.go index 1b7e63e0f3..c3a21bf997 100644 --- a/model/metadata/metadata.go +++ b/model/metadata/metadata.go @@ -1,4 +1,4 @@ -// Copyright 2022 The Prometheus Authors +// Copyright The Prometheus Authors // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. // You may obtain a copy of the License at @@ -13,7 +13,11 @@ package metadata -import "github.com/prometheus/common/model" +import ( + "strings" + + "github.com/prometheus/common/model" +) // Metadata stores a series' metadata information. type Metadata struct { @@ -21,3 +25,26 @@ type Metadata struct { Unit string `json:"unit"` Help string `json:"help"` } + +// IsEmpty returns true if metadata structure is empty, including unknown type case. +func (m Metadata) IsEmpty() bool { + return (m.Type == "" || m.Type == model.MetricTypeUnknown) && m.Unit == "" && m.Help == "" +} + +// Equals returns true if m is semantically the same as other metadata. +func (m Metadata) Equals(other Metadata) bool { + if strings.Compare(m.Unit, other.Unit) != 0 || strings.Compare(m.Help, other.Help) != 0 { + return false + } + + if m.Type == "" || m.Type == model.MetricTypeUnknown { + if m.Type != "" && m.Type != model.MetricTypeUnknown { + return false + } + } else { + if m.Type != other.Type { + return false + } + } + return true +} diff --git a/scrape/helpers_test.go b/scrape/helpers_test.go index ff7a7bf65a..94e3e77d67 100644 --- a/scrape/helpers_test.go +++ b/scrape/helpers_test.go @@ -1,4 +1,4 @@ -// Copyright 2013 The Prometheus Authors +// Copyright The Prometheus Authors // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. // You may obtain a copy of the License at @@ -17,10 +17,6 @@ import ( "bytes" "context" "encoding/binary" - "fmt" - "math" - "strings" - "sync" "testing" "github.com/gogo/protobuf/proto" @@ -32,8 +28,12 @@ import ( "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/model/metadata" "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/util/teststorage" ) +// For readability. +type sample = teststorage.Sample + type nopAppendable struct{} func (nopAppendable) Appender(context.Context) storage.Appender { @@ -71,188 +71,6 @@ func (nopAppender) AppendSTZeroSample(storage.SeriesRef, labels.Labels, int64, i func (nopAppender) Commit() error { return nil } func (nopAppender) Rollback() error { return nil } -type floatSample struct { - metric labels.Labels - t int64 - f float64 -} - -func equalFloatSamples(a, b floatSample) bool { - // Compare Float64bits so NaN values which are exactly the same will compare equal. - return labels.Equal(a.metric, b.metric) && a.t == b.t && math.Float64bits(a.f) == math.Float64bits(b.f) -} - -type histogramSample struct { - metric labels.Labels - t int64 - h *histogram.Histogram - fh *histogram.FloatHistogram -} - -type metadataEntry struct { - m metadata.Metadata - metric labels.Labels -} - -func metadataEntryEqual(a, b metadataEntry) bool { - if !labels.Equal(a.metric, b.metric) { - return false - } - if a.m.Type != b.m.Type { - return false - } - if a.m.Unit != b.m.Unit { - return false - } - if a.m.Help != b.m.Help { - return false - } - return true -} - -type collectResultAppendable struct { - *collectResultAppender -} - -func (a *collectResultAppendable) Appender(context.Context) storage.Appender { - return a -} - -// collectResultAppender records all samples that were added through the appender. -// It can be used as its zero value or be backed by another appender it writes samples through. -type collectResultAppender struct { - mtx sync.Mutex - - next storage.Appender - resultFloats []floatSample - pendingFloats []floatSample - rolledbackFloats []floatSample - resultHistograms []histogramSample - pendingHistograms []histogramSample - rolledbackHistograms []histogramSample - resultExemplars []exemplar.Exemplar - pendingExemplars []exemplar.Exemplar - resultMetadata []metadataEntry - pendingMetadata []metadataEntry -} - -func (*collectResultAppender) SetOptions(*storage.AppendOptions) {} - -func (a *collectResultAppender) Append(ref storage.SeriesRef, lset labels.Labels, t int64, v float64) (storage.SeriesRef, error) { - a.mtx.Lock() - defer a.mtx.Unlock() - a.pendingFloats = append(a.pendingFloats, floatSample{ - metric: lset, - t: t, - f: v, - }) - - if a.next == nil { - if ref == 0 { - // Use labels hash as a stand-in for unique series reference, to avoid having to track all series. - ref = storage.SeriesRef(lset.Hash()) - } - return ref, nil - } - - ref, err := a.next.Append(ref, lset, t, v) - if err != nil { - return 0, err - } - return ref, nil -} - -func (a *collectResultAppender) AppendExemplar(ref storage.SeriesRef, l labels.Labels, e exemplar.Exemplar) (storage.SeriesRef, error) { - a.mtx.Lock() - defer a.mtx.Unlock() - a.pendingExemplars = append(a.pendingExemplars, e) - if a.next == nil { - return 0, nil - } - - return a.next.AppendExemplar(ref, l, e) -} - -func (a *collectResultAppender) AppendHistogram(ref storage.SeriesRef, l labels.Labels, t int64, h *histogram.Histogram, fh *histogram.FloatHistogram) (storage.SeriesRef, error) { - a.mtx.Lock() - defer a.mtx.Unlock() - a.pendingHistograms = append(a.pendingHistograms, histogramSample{h: h, fh: fh, t: t, metric: l}) - if a.next == nil { - return 0, nil - } - - return a.next.AppendHistogram(ref, l, t, h, fh) -} - -func (a *collectResultAppender) AppendHistogramSTZeroSample(ref storage.SeriesRef, l labels.Labels, _, st int64, h *histogram.Histogram, _ *histogram.FloatHistogram) (storage.SeriesRef, error) { - if h != nil { - return a.AppendHistogram(ref, l, st, &histogram.Histogram{}, nil) - } - return a.AppendHistogram(ref, l, st, nil, &histogram.FloatHistogram{}) -} - -func (a *collectResultAppender) UpdateMetadata(ref storage.SeriesRef, l labels.Labels, m metadata.Metadata) (storage.SeriesRef, error) { - a.mtx.Lock() - defer a.mtx.Unlock() - a.pendingMetadata = append(a.pendingMetadata, metadataEntry{metric: l, m: m}) - if a.next == nil { - if ref == 0 { - ref = storage.SeriesRef(l.Hash()) - } - return ref, nil - } - - return a.next.UpdateMetadata(ref, l, m) -} - -func (a *collectResultAppender) AppendSTZeroSample(ref storage.SeriesRef, l labels.Labels, _, st int64) (storage.SeriesRef, error) { - return a.Append(ref, l, st, 0.0) -} - -func (a *collectResultAppender) Commit() error { - a.mtx.Lock() - defer a.mtx.Unlock() - a.resultFloats = append(a.resultFloats, a.pendingFloats...) - a.resultExemplars = append(a.resultExemplars, a.pendingExemplars...) - a.resultHistograms = append(a.resultHistograms, a.pendingHistograms...) - a.resultMetadata = append(a.resultMetadata, a.pendingMetadata...) - a.pendingFloats = nil - a.pendingExemplars = nil - a.pendingHistograms = nil - a.pendingMetadata = nil - if a.next == nil { - return nil - } - return a.next.Commit() -} - -func (a *collectResultAppender) Rollback() error { - a.mtx.Lock() - defer a.mtx.Unlock() - a.rolledbackFloats = a.pendingFloats - a.rolledbackHistograms = a.pendingHistograms - a.pendingFloats = nil - a.pendingHistograms = nil - if a.next == nil { - return nil - } - return a.next.Rollback() -} - -func (a *collectResultAppender) String() string { - var sb strings.Builder - for _, s := range a.resultFloats { - sb.WriteString(fmt.Sprintf("committed: %s %f %d\n", s.metric, s.f, s.t)) - } - for _, s := range a.pendingFloats { - sb.WriteString(fmt.Sprintf("pending: %s %f %d\n", s.metric, s.f, s.t)) - } - for _, s := range a.rolledbackFloats { - sb.WriteString(fmt.Sprintf("rolledback: %s %f %d\n", s.metric, s.f, s.t)) - } - return sb.String() -} - // protoMarshalDelimited marshals a MetricFamily into a delimited // Prometheus proto exposition format bytes (known as `encoding=delimited`) // diff --git a/scrape/manager.go b/scrape/manager.go index c63d7d0eae..3aa97c9e79 100644 --- a/scrape/manager.go +++ b/scrape/manager.go @@ -1,4 +1,4 @@ -// Copyright 2013 The Prometheus Authors +// Copyright The Prometheus Authors // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. // You may obtain a copy of the License at @@ -38,8 +38,8 @@ import ( "github.com/prometheus/prometheus/util/pool" ) -// NewManager is the Manager constructor. -func NewManager(o *Options, logger *slog.Logger, newScrapeFailureLogger func(string) (*logging.JSONFileLogger, error), app storage.Appendable, registerer prometheus.Registerer) (*Manager, error) { +// NewManager is the Manager constructor using Appendable. +func NewManager(o *Options, logger *slog.Logger, newScrapeFailureLogger func(string) (*logging.JSONFileLogger, error), appendable storage.Appendable, registerer prometheus.Registerer) (*Manager, error) { if o == nil { o = &Options{} } @@ -53,7 +53,7 @@ func NewManager(o *Options, logger *slog.Logger, newScrapeFailureLogger func(str } m := &Manager{ - append: app, + appendable: appendable, opts: o, logger: logger, newScrapeFailureLogger: newScrapeFailureLogger, @@ -79,15 +79,15 @@ type Options struct { // Option to enable appending of scraped Metadata to the TSDB/other appenders. Individual appenders // can decide what to do with metadata, but for practical purposes this flag exists so that metadata // can be written to the WAL and thus read for remote write. - // TODO: implement some form of metadata storage AppendMetadata bool // Option to increase the interval used by scrape manager to throttle target groups updates. DiscoveryReloadInterval model.Duration + // Option to enable the ingestion of the created timestamp as a synthetic zero sample. // See: https://github.com/prometheus/proposals/blob/main/proposals/2023-06-13_created-timestamp.md EnableStartTimestampZeroIngestion bool - // EnableTypeAndUnitLabels + // EnableTypeAndUnitLabels represents type-and-unit-labels feature flag. EnableTypeAndUnitLabels bool // Optional HTTP client options to use when scraping. @@ -100,9 +100,11 @@ type Options struct { // Manager maintains a set of scrape pools and manages start/stop cycles // when receiving new target groups from the discovery manager. type Manager struct { - opts *Options - logger *slog.Logger - append storage.Appendable + opts *Options + logger *slog.Logger + + appendable storage.Appendable + graceShut chan struct{} offsetSeed uint64 // Global offsetSeed seed is used to spread scrape workload across HA setup. @@ -183,7 +185,7 @@ func (m *Manager) reload() { continue } m.metrics.targetScrapePools.Inc() - sp, err := newScrapePool(scrapeConfig, m.append, m.offsetSeed, m.logger.With("scrape_pool", setName), m.buffers, m.opts, m.metrics) + sp, err := newScrapePool(scrapeConfig, m.appendable, m.offsetSeed, m.logger.With("scrape_pool", setName), m.buffers, m.opts, m.metrics) if err != nil { m.metrics.targetScrapePoolsFailed.Inc() m.logger.Error("error creating new scrape pool", "err", err, "scrape_pool", setName) diff --git a/scrape/manager_test.go b/scrape/manager_test.go index 1ec4875d19..c7f66bacf9 100644 --- a/scrape/manager_test.go +++ b/scrape/manager_test.go @@ -1,4 +1,4 @@ -// Copyright 2013 The Prometheus Authors +// Copyright The Prometheus Authors // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. // You may obtain a copy of the License at @@ -51,6 +51,7 @@ import ( "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/tsdbutil" "github.com/prometheus/prometheus/util/runutil" + "github.com/prometheus/prometheus/util/teststorage" "github.com/prometheus/prometheus/util/testutil" ) @@ -764,7 +765,7 @@ func TestManagerSTZeroIngestion(t *testing.T) { for _, testWithST := range []bool{false, true} { t.Run(fmt.Sprintf("withST=%v", testWithST), func(t *testing.T) { for _, testSTZeroIngest := range []bool{false, true} { - t.Run(fmt.Sprintf("ctZeroIngest=%v", testSTZeroIngest), func(t *testing.T) { + t.Run(fmt.Sprintf("stZeroIngest=%v", testSTZeroIngest), func(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -777,11 +778,11 @@ func TestManagerSTZeroIngestion(t *testing.T) { // TODO(bwplotka): Add more types than just counter? encoded := prepareTestEncodedCounter(t, testFormat, expectedMetricName, expectedSampleValue, sampleTs, stTs) - app := &collectResultAppender{} + app := teststorage.NewAppender() discoveryManager, scrapeManager := runManagers(t, ctx, &Options{ EnableStartTimestampZeroIngestion: testSTZeroIngest, skipOffsetting: true, - }, &collectResultAppendable{app}) + }, app) defer scrapeManager.Stop() server := setupTestServer(t, config.ScrapeProtocolsHeaders[testFormat], encoded) @@ -806,11 +807,8 @@ scrape_configs: ctx, cancel = context.WithTimeout(ctx, 1*time.Minute) defer cancel() require.NoError(t, runutil.Retry(100*time.Millisecond, ctx.Done(), func() error { - app.mtx.Lock() - defer app.mtx.Unlock() - // Check if scrape happened and grab the relevant samples. - if len(app.resultFloats) > 0 { + if app.ResultSamplesGreaterThan(0) { return nil } return errors.New("expected some float samples, got none") @@ -818,32 +816,32 @@ scrape_configs: // Verify results. // Verify what we got vs expectations around ST injection. - samples := findSamplesForMetric(app.resultFloats, expectedMetricName) + got := findSamplesForMetric(app.PendingSamples, expectedMetricName) if testWithST && testSTZeroIngest { - require.Len(t, samples, 2) - require.Equal(t, 0.0, samples[0].f) - require.Equal(t, timestamp.FromTime(stTs), samples[0].t) - require.Equal(t, expectedSampleValue, samples[1].f) - require.Equal(t, timestamp.FromTime(sampleTs), samples[1].t) + require.Len(t, got, 2) + require.Equal(t, 0.0, got[0].V) + require.Equal(t, timestamp.FromTime(stTs), got[0].T) + require.Equal(t, expectedSampleValue, got[1].V) + require.Equal(t, timestamp.FromTime(sampleTs), got[1].T) } else { - require.Len(t, samples, 1) - require.Equal(t, expectedSampleValue, samples[0].f) - require.Equal(t, timestamp.FromTime(sampleTs), samples[0].t) + require.Len(t, got, 1) + require.Equal(t, expectedSampleValue, got[0].V) + require.Equal(t, timestamp.FromTime(sampleTs), got[0].T) } // Verify what we got vs expectations around additional _created series for OM text. // enableSTZeroInjection also kills that _created line. - createdSeriesSamples := findSamplesForMetric(app.resultFloats, expectedCreatedMetricName) + stSeriesSaender := findSamplesForMetric(app.PendingSamples, expectedCreatedMetricName) if testFormat == config.OpenMetricsText1_0_0 && testWithST && !testSTZeroIngest { // For OM Text, when counter has ST, and feature flag disabled we should see _created lines. - require.Len(t, createdSeriesSamples, 1) + require.Len(t, stSeriesSaender, 1) // Conversion taken from common/expfmt.writeOpenMetricsFloat. // We don't check the st timestamp as explicit ts was not implemented in expfmt.Encoder, // but exists in OM https://github.com/prometheus/OpenMetrics/blob/v1.0.0/specification/OpenMetrics.md#:~:text=An%20example%20with%20a%20Metric%20with%20no%20labels%2C%20and%20a%20MetricPoint%20with%20a%20timestamp%20and%20a%20created // We can implement this, but we want to potentially get rid of OM 1.0 ST lines - require.Equal(t, float64(timestamppb.New(stTs).AsTime().UnixNano())/1e9, createdSeriesSamples[0].f) + require.Equal(t, float64(timestamppb.New(stTs).AsTime().UnixNano())/1e9, stSeriesSaender[0].V) } else { - require.Empty(t, createdSeriesSamples) + require.Empty(t, stSeriesSaender) } }) } @@ -885,9 +883,9 @@ func prepareTestEncodedCounter(t *testing.T, format config.ScrapeProtocol, mName } } -func findSamplesForMetric(floats []floatSample, metricName string) (ret []floatSample) { +func findSamplesForMetric(floats []sample, metricName string) (ret []sample) { for _, f := range floats { - if f.metric.Get(model.MetricNameLabel) == metricName { + if f.L.Get(model.MetricNameLabel) == metricName { ret = append(ret, f) } } @@ -964,11 +962,11 @@ func TestManagerSTZeroIngestionHistogram(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - app := &collectResultAppender{} + app := teststorage.NewAppender() discoveryManager, scrapeManager := runManagers(t, ctx, &Options{ EnableStartTimestampZeroIngestion: tc.enableSTZeroIngestion, skipOffsetting: true, - }, &collectResultAppendable{app}) + }, app) defer scrapeManager.Stop() once := sync.Once{} @@ -1012,43 +1010,33 @@ scrape_configs: `, serverURL.Host) applyConfig(t, testConfig, scrapeManager, discoveryManager) - var got []histogramSample - // Wait for one scrape. ctx, cancel = context.WithTimeout(ctx, 1*time.Minute) defer cancel() require.NoError(t, runutil.Retry(100*time.Millisecond, ctx.Done(), func() error { - app.mtx.Lock() - defer app.mtx.Unlock() - - // Check if scrape happened and grab the relevant histograms, they have to be there - or it's a bug - // and it's not worth waiting. - for _, h := range app.resultHistograms { - if h.metric.Get(model.MetricNameLabel) == mName { - got = append(got, h) - } - } - if len(app.resultHistograms) > 0 { + if app.ResultSamplesGreaterThan(0) { return nil } return errors.New("expected some histogram samples, got none") }), "after 1 minute") + got := findSamplesForMetric(app.PendingSamples, mName) + // Check for zero samples, assuming we only injected always one histogram sample. // Did it contain ST to inject? If yes, was ST zero enabled? if tc.inputHistSample.CreatedTimestamp.IsValid() && tc.enableSTZeroIngestion { require.Len(t, got, 2) // Zero sample. - require.Equal(t, histogram.Histogram{}, *got[0].h) + require.Equal(t, histogram.Histogram{}, *got[0].H) // Quick soft check to make sure it's the same sample or at least not zero. - require.Equal(t, tc.inputHistSample.GetSampleSum(), got[1].h.Sum) + require.Equal(t, tc.inputHistSample.GetSampleSum(), got[1].H.Sum) return } // Expect only one, valid sample. require.Len(t, got, 1) // Quick soft check to make sure it's the same sample or at least not zero. - require.Equal(t, tc.inputHistSample.GetSampleSum(), got[0].h.Sum) + require.Equal(t, tc.inputHistSample.GetSampleSum(), got[0].H.Sum) }) } } @@ -1083,11 +1071,11 @@ func TestNHCBAndSTZeroIngestion(t *testing.T) { ctx := t.Context() - app := &collectResultAppender{} + app := teststorage.NewAppender() discoveryManager, scrapeManager := runManagers(t, ctx, &Options{ EnableStartTimestampZeroIngestion: true, skipOffsetting: true, - }, &collectResultAppendable{app}) + }, app) defer scrapeManager.Stop() once := sync.Once{} @@ -1146,33 +1134,19 @@ scrape_configs: return exists }, 5*time.Second, 100*time.Millisecond, "scrape pool should be created for job 'test'") - // Helper function to get matching histograms to avoid race conditions. - getMatchingHistograms := func() []histogramSample { - app.mtx.Lock() - defer app.mtx.Unlock() - - var got []histogramSample - for _, h := range app.resultHistograms { - if h.metric.Get(model.MetricNameLabel) == mName { - got = append(got, h) - } - } - return got - } - require.Eventually(t, func() bool { - return len(getMatchingHistograms()) > 0 + return app.ResultSamplesGreaterThan(0) }, 1*time.Minute, 100*time.Millisecond, "expected histogram samples, got none") // Verify that samples were ingested (proving both features work together). - got := getMatchingHistograms() + got := findSamplesForMetric(app.PendingSamples, mName) // With ST zero ingestion enabled and a created timestamp present, we expect 2 samples: // one zero sample and one actual sample. require.Len(t, got, 2, "expected 2 histogram samples (zero sample + actual sample)") - require.Equal(t, histogram.Histogram{}, *got[0].h, "first sample should be zero sample") - require.InDelta(t, expectedHistogramSum, got[1].h.Sum, 1e-9, "second sample should retain the expected sum") - require.Len(t, app.resultExemplars, 2, "expected 2 exemplars from histogram buckets") + require.Equal(t, histogram.Histogram{}, *got[0].H, "first sample should be zero sample") + require.InDelta(t, expectedHistogramSum, got[1].H.Sum, 1e-9, "second sample should retain the expected sum") + require.Len(t, got[1].ES, 2, "expected 2 exemplars on second histogram") } func applyConfig( diff --git a/scrape/scrape.go b/scrape/scrape.go index bbb93c8801..77f88efa1a 100644 --- a/scrape/scrape.go +++ b/scrape/scrape.go @@ -1,4 +1,4 @@ -// Copyright 2016 The Prometheus Authors +// Copyright The Prometheus Authors // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. // You may obtain a copy of the License at @@ -81,9 +81,10 @@ type FailureLogger interface { // scrapePool manages scrapes for sets of targets. type scrapePool struct { appendable storage.Appendable - logger *slog.Logger - cancel context.CancelFunc - httpOpts []config_util.HTTPClientOption + + logger *slog.Logger + cancel context.CancelFunc + httpOpts []config_util.HTTPClientOption // mtx must not be taken after targetMtx. mtx sync.Mutex @@ -147,7 +148,17 @@ const maxAheadTime = 10 * time.Minute // returning an empty label set is interpreted as "drop". type labelsMutator func(labels.Labels) labels.Labels -func newScrapePool(cfg *config.ScrapeConfig, app storage.Appendable, offsetSeed uint64, logger *slog.Logger, buffers *pool.Pool, options *Options, metrics *scrapeMetrics) (*scrapePool, error) { +// scrapeLoopAppendAdapter allows support for multiple storage.Appender versions. +// NOTE(bwplotka): Required for: https://github.com/prometheus/prometheus/pull/17610 +type scrapeLoopAppendAdapter interface { + Commit() error + Rollback() error + + addReportSample(s reportSample, t int64, v float64, b *labels.Builder, rejectOOO bool) error + append(b []byte, contentType string, ts time.Time) (total, added, seriesAdded int, err error) +} + +func newScrapePool(cfg *config.ScrapeConfig, appendable storage.Appendable, offsetSeed uint64, logger *slog.Logger, buffers *pool.Pool, options *Options, metrics *scrapeMetrics) (*scrapePool, error) { if logger == nil { logger = promslog.NewNopLogger() } @@ -169,7 +180,7 @@ func newScrapePool(cfg *config.ScrapeConfig, app storage.Appendable, offsetSeed ctx, cancel := context.WithCancel(context.Background()) sp := &scrapePool{ cancel: cancel, - appendable: app, + appendable: appendable, config: cfg, client: client, activeTargets: map[uint64]*Target{}, @@ -183,55 +194,75 @@ func newScrapePool(cfg *config.ScrapeConfig, app storage.Appendable, offsetSeed escapingScheme: escapingScheme, } sp.newLoop = func(opts scrapeLoopOptions) loop { - // Update the targets retrieval function for metadata to a new scrape cache. - cache := opts.cache - if cache == nil { - cache = newScrapeCache(metrics) - } - opts.target.SetMetadataStore(cache) - - return newScrapeLoop( - ctx, - opts.scraper, - logger.With("target", opts.target), - buffers, - func(l labels.Labels) labels.Labels { + // NOTE: Formatting matches scrapeLoop fields order for readability. + sl := &scrapeLoop{ + buffers: buffers, + appendable: appendable, + sampleMutator: func(l labels.Labels) labels.Labels { return mutateSampleLabels(l, opts.target, opts.honorLabels, opts.mrc) }, - func(l labels.Labels) labels.Labels { return mutateReportSampleLabels(l, opts.target) }, - func(ctx context.Context) storage.Appender { return app.Appender(ctx) }, - cache, - sp.symbolTable, - offsetSeed, - opts.honorTimestamps, - opts.trackTimestampsStaleness, - opts.enableCompression, - opts.sampleLimit, - opts.bucketLimit, - opts.maxSchema, - opts.labelLimits, - opts.interval, - opts.timeout, - opts.alwaysScrapeClassicHist, - opts.convertClassicHistToNHCB, - cfg.ScrapeNativeHistogramsEnabled(), - options.EnableStartTimestampZeroIngestion, - options.EnableTypeAndUnitLabels, - options.ExtraMetrics, - options.AppendMetadata, - opts.target, - options.PassMetadataInContext, - metrics, - options.skipOffsetting, - sp.validationScheme, - sp.escapingScheme, - opts.fallbackScrapeProtocol, - ) + reportSampleMutator: func(l labels.Labels) labels.Labels { return mutateReportSampleLabels(l, opts.target) }, + offsetSeed: offsetSeed, + metrics: metrics, + + symbolTable: sp.symbolTable, + validationScheme: sp.validationScheme, + escapingScheme: sp.escapingScheme, + + enableNativeHistogramScraping: cfg.ScrapeNativeHistogramsEnabled(), + + enableSTZeroIngestion: options.EnableStartTimestampZeroIngestion, + enableTypeAndUnitLabels: options.EnableTypeAndUnitLabels, + reportExtraMetrics: options.ExtraMetrics, + appendMetadataToWAL: options.AppendMetadata, + skipOffsetting: options.skipOffsetting, + + scrapeLoopOptions: opts, + } + sl.init(ctx, options.PassMetadataInContext) + return sl } sp.metrics.targetScrapePoolTargetLimit.WithLabelValues(sp.config.JobName).Set(float64(sp.config.TargetLimit)) return sp, nil } +// init prepares scrapeLoop after raw construction. +// NOTE: While newScrapeLoop constructor pattern would be safer, it has proven to be +// highly not readable (too many params). Instead, we follow init pattern. +func (sl *scrapeLoop) init(ctx context.Context, passMetadataInContext bool) { + if sl.l == nil { + sl.l = promslog.NewNopLogger() + } + sl.parentCtx = ctx + sl.stopped = make(chan struct{}) + if sl.buffers == nil { + sl.buffers = pool.New(1e3, 1e6, 3, func(sz int) any { return make([]byte, 0, sz) }) + } + if sl.cache == nil { + sl.cache = newScrapeCache(sl.metrics) + if sl.target != nil { + // Update the targets retrieval function for metadata to a new scrape cache. + sl.target.SetMetadataStore(sl.cache) + // TODO(bwplotka): Not sure why, but doing this before sl.target.SetMetadataStore(sl.cache) blocks goroutines... + // Debug, something is odd. + sl.l = sl.l.With("target", sl.target) + } + } + + appenderCtx := ctx + if passMetadataInContext { + // Store the cache and target in the context. This is then used by downstream OTel Collector + // to lookup the metadata required to process the samples. Not used by Prometheus itself. + // TODO(gouthamve) We're using a dedicated context because using the parentCtx caused a memory + // leak. We should ideally fix the main leak. See: https://github.com/prometheus/prometheus/pull/10590 + // TODO(bwplotka): Remove once OpenTelemetry collector uses AppenderV2 (add issue) + appenderCtx = ContextWithMetricMetadataStore(appenderCtx, sl.cache) + appenderCtx = ContextWithTarget(appenderCtx, sl.target) + } + sl.appenderCtx = appenderCtx + sl.ctx, sl.cancel = context.WithCancel(ctx) +} + func (sp *scrapePool) ActiveTargets() []*Target { sp.targetMtx.Lock() defer sp.targetMtx.Unlock() @@ -392,6 +423,8 @@ func (sp *scrapePool) restartLoops(reuseCache bool) { } t := sp.activeTargets[fp] + // Update the targets retrieval function for metadata to a new target. + t.SetMetadataStore(cache) targetInterval, targetTimeout, err := t.intervalAndTimeout(interval, timeout) var ( s = &targetScraper{ @@ -753,39 +786,6 @@ func mutateReportSampleLabels(lset labels.Labels, target *Target) labels.Labels return lb.Labels() } -// appender returns an appender for ingested samples from the target. -func appender(app storage.Appender, sampleLimit, bucketLimit int, maxSchema int32) storage.Appender { - app = &timeLimitAppender{ - Appender: app, - maxTime: timestamp.FromTime(time.Now().Add(maxAheadTime)), - } - - // The sampleLimit is applied after metrics are potentially dropped via relabeling. - if sampleLimit > 0 { - app = &limitAppender{ - Appender: app, - limit: sampleLimit, - } - } - - if bucketLimit > 0 { - app = &bucketLimitAppender{ - Appender: app, - limit: bucketLimit, - } - } - - if maxSchema < histogram.ExponentialSchemaMax { - app = &maxSchemaAppender{ - Appender: app, - maxSchema: maxSchema, - } - } - - return app -} - -// A scraper retrieves samples and accepts a status report at the end. type scraper interface { scrape(ctx context.Context) (*http.Response, error) readResponse(ctx context.Context, resp *http.Response, w io.Writer) (string, error) @@ -931,55 +931,49 @@ type cacheEntry struct { } type scrapeLoop struct { - scraper scraper - l *slog.Logger - scrapeFailureLogger FailureLogger - scrapeFailureLoggerMtx sync.RWMutex - cache *scrapeCache - lastScrapeSize int - buffers *pool.Pool - offsetSeed uint64 - honorTimestamps bool - trackTimestampsStaleness bool - enableCompression bool - forcedErr error - forcedErrMtx sync.Mutex - sampleLimit int - bucketLimit int - maxSchema int32 - labelLimits *labelLimits - interval time.Duration - timeout time.Duration - validationScheme model.ValidationScheme - escapingScheme model.EscapingScheme - - alwaysScrapeClassicHist bool - convertClassicHistToNHCB bool - enableSTZeroIngestion bool - enableTypeAndUnitLabels bool - fallbackScrapeProtocol string - - enableNativeHistogramScraping bool - - appender func(ctx context.Context) storage.Appender - symbolTable *labels.SymbolTable + // Parameters. + ctx context.Context + cancel func() + stopped chan struct{} + parentCtx context.Context + appenderCtx context.Context + l *slog.Logger + buffers *pool.Pool + appendable storage.Appendable sampleMutator labelsMutator reportSampleMutator labelsMutator + offsetSeed uint64 + metrics *scrapeMetrics - parentCtx context.Context - appenderCtx context.Context - ctx context.Context - cancel func() - stopped chan struct{} + // Scrape pool shared data. + symbolTable *labels.SymbolTable + validationScheme model.ValidationScheme + escapingScheme model.EscapingScheme + // Options inherited from config.ScrapeConfig. + enableNativeHistogramScraping bool + + // Options inherited from scrape.Options. + enableSTZeroIngestion bool + enableTypeAndUnitLabels bool + reportExtraMetrics bool + appendMetadataToWAL bool + skipOffsetting bool // For testability. + + // Common options. + scrapeLoopOptions + + // error injection through setForcedError. + forcedErr error + forcedErrMtx sync.Mutex + + // Special logger set on setScrapeFailureLogger + scrapeFailureLoggerMtx sync.RWMutex + scrapeFailureLogger FailureLogger + + // Locally cached data. + lastScrapeSize int disabledEndOfRunStalenessMarkers atomic.Bool - - reportExtraMetrics bool - appendMetadataToWAL bool - - metrics *scrapeMetrics - - skipOffsetting bool // For testability. } // scrapeCache tracks mappings of exposed metric strings to label sets and @@ -1004,8 +998,8 @@ type scrapeCache struct { seriesCur map[storage.SeriesRef]*cacheEntry seriesPrev map[storage.SeriesRef]*cacheEntry - // TODO(bwplotka): Consider moving Metadata API to use WAL instead of scrape loop to - // avoid locking (using metadata API can block scraping). + // TODO(bwplotka): Consider moving metadata caching to head. See + // https://github.com/prometheus/prometheus/issues/17619. metaMtx sync.Mutex // Mutex is needed due to api touching it when metadata is queried. metadata map[string]*metaEntry // metadata by metric family name. @@ -1240,101 +1234,6 @@ func (c *scrapeCache) LengthMetadata() int { return len(c.metadata) } -func newScrapeLoop(ctx context.Context, - sc scraper, - l *slog.Logger, - buffers *pool.Pool, - sampleMutator labelsMutator, - reportSampleMutator labelsMutator, - appender func(ctx context.Context) storage.Appender, - cache *scrapeCache, - symbolTable *labels.SymbolTable, - offsetSeed uint64, - honorTimestamps bool, - trackTimestampsStaleness bool, - enableCompression bool, - sampleLimit int, - bucketLimit int, - maxSchema int32, - labelLimits *labelLimits, - interval time.Duration, - timeout time.Duration, - alwaysScrapeClassicHist bool, - convertClassicHistToNHCB bool, - enableNativeHistogramScraping bool, - enableSTZeroIngestion bool, - enableTypeAndUnitLabels bool, - reportExtraMetrics bool, - appendMetadataToWAL bool, - target *Target, - passMetadataInContext bool, - metrics *scrapeMetrics, - skipOffsetting bool, - validationScheme model.ValidationScheme, - escapingScheme model.EscapingScheme, - fallbackScrapeProtocol string, -) *scrapeLoop { - if l == nil { - l = promslog.NewNopLogger() - } - if buffers == nil { - buffers = pool.New(1e3, 1e6, 3, func(sz int) any { return make([]byte, 0, sz) }) - } - if cache == nil { - cache = newScrapeCache(metrics) - } - - appenderCtx := ctx - - if passMetadataInContext { - // Store the cache and target in the context. This is then used by downstream OTel Collector - // to lookup the metadata required to process the samples. Not used by Prometheus itself. - // TODO(gouthamve) We're using a dedicated context because using the parentCtx caused a memory - // leak. We should ideally fix the main leak. See: https://github.com/prometheus/prometheus/pull/10590 - appenderCtx = ContextWithMetricMetadataStore(appenderCtx, cache) - appenderCtx = ContextWithTarget(appenderCtx, target) - } - - sl := &scrapeLoop{ - scraper: sc, - buffers: buffers, - cache: cache, - appender: appender, - symbolTable: symbolTable, - sampleMutator: sampleMutator, - reportSampleMutator: reportSampleMutator, - stopped: make(chan struct{}), - offsetSeed: offsetSeed, - l: l, - parentCtx: ctx, - appenderCtx: appenderCtx, - honorTimestamps: honorTimestamps, - trackTimestampsStaleness: trackTimestampsStaleness, - enableCompression: enableCompression, - sampleLimit: sampleLimit, - bucketLimit: bucketLimit, - maxSchema: maxSchema, - labelLimits: labelLimits, - interval: interval, - timeout: timeout, - alwaysScrapeClassicHist: alwaysScrapeClassicHist, - convertClassicHistToNHCB: convertClassicHistToNHCB, - enableSTZeroIngestion: enableSTZeroIngestion, - enableTypeAndUnitLabels: enableTypeAndUnitLabels, - fallbackScrapeProtocol: fallbackScrapeProtocol, - enableNativeHistogramScraping: enableNativeHistogramScraping, - reportExtraMetrics: reportExtraMetrics, - appendMetadataToWAL: appendMetadataToWAL, - metrics: metrics, - skipOffsetting: skipOffsetting, - validationScheme: validationScheme, - escapingScheme: escapingScheme, - } - sl.ctx, sl.cancel = context.WithCancel(ctx) - - return sl -} - func (sl *scrapeLoop) setScrapeFailureLogger(l FailureLogger) { sl.scrapeFailureLoggerMtx.Lock() defer sl.scrapeFailureLoggerMtx.Unlock() @@ -1411,6 +1310,11 @@ mainLoop: } } +func (sl *scrapeLoop) appender() scrapeLoopAppendAdapter { + // TODO(bwplotka): Here we will inject AppenderV2 eventually. See https://github.com/prometheus/prometheus/pull/17610 + return &scrapeLoopAppender{scrapeLoop: sl, Appender: sl.appendable.Appender(sl.appenderCtx)} +} + // scrapeAndReport performs a scrape and then appends the result to the storage // together with reporting metrics, by using as few appenders as possible. // In the happy scenario, a single appender is used. @@ -1432,20 +1336,20 @@ func (sl *scrapeLoop) scrapeAndReport(last, appendTime time.Time, errc chan<- er var total, added, seriesAdded, bytesRead int var err, appErr, scrapeErr error - app := sl.appender(sl.appenderCtx) + sla := sl.appender() defer func() { if err != nil { - app.Rollback() + _ = sla.Rollback() return } - err = app.Commit() + err = sla.Commit() if err != nil { sl.l.Error("Scrape commit failed", "err", err) } }() defer func() { - if err = sl.report(app, appendTime, time.Since(start), total, added, seriesAdded, bytesRead, scrapeErr); err != nil { + if err = sl.report(sla, appendTime, time.Since(start), total, added, seriesAdded, bytesRead, scrapeErr); err != nil { sl.l.Warn("Appending scrape report failed", "err", err) } }() @@ -1453,9 +1357,9 @@ func (sl *scrapeLoop) scrapeAndReport(last, appendTime time.Time, errc chan<- er if forcedErr := sl.getForcedError(); forcedErr != nil { scrapeErr = forcedErr // Add stale markers. - if _, _, _, err := sl.append(app, []byte{}, "", appendTime); err != nil { - app.Rollback() - app = sl.appender(sl.appenderCtx) + if _, _, _, err := sla.append([]byte{}, "", appendTime); err != nil { + _ = sla.Rollback() + sla = sl.appender() sl.l.Warn("Append failed", "err", err) } if errc != nil { @@ -1505,16 +1409,16 @@ func (sl *scrapeLoop) scrapeAndReport(last, appendTime time.Time, errc chan<- er // A failed scrape is the same as an empty scrape, // we still call sl.append to trigger stale markers. - total, added, seriesAdded, appErr = sl.append(app, b, contentType, appendTime) + total, added, seriesAdded, appErr = sla.append(b, contentType, appendTime) if appErr != nil { - app.Rollback() - app = sl.appender(sl.appenderCtx) + _ = sla.Rollback() + sla = sl.appender() sl.l.Debug("Append failed", "err", appErr) // The append failed, probably due to a parse error or sample limit. // Call sl.append again with an empty scrape to trigger stale markers. - if _, _, _, err := sl.append(app, []byte{}, "", appendTime); err != nil { - app.Rollback() - app = sl.appender(sl.appenderCtx) + if _, _, _, err := sla.append([]byte{}, "", appendTime); err != nil { + _ = sla.Rollback() + sla = sl.appender() sl.l.Warn("Append failed", "err", err) } } @@ -1584,24 +1488,24 @@ func (sl *scrapeLoop) endOfRunStaleness(last time.Time, ticker *time.Ticker, int // If the target has since been recreated and scraped, the // stale markers will be out of order and ignored. // sl.context would have been cancelled, hence using sl.appenderCtx. - app := sl.appender(sl.appenderCtx) + sla := sl.appender() var err error defer func() { if err != nil { - app.Rollback() + _ = sla.Rollback() return } - err = app.Commit() + err = sla.Commit() if err != nil { sl.l.Warn("Stale commit failed", "err", err) } }() - if _, _, _, err = sl.append(app, []byte{}, "", staleTime); err != nil { - app.Rollback() - app = sl.appender(sl.appenderCtx) + if _, _, _, err = sla.append([]byte{}, "", staleTime); err != nil { + _ = sla.Rollback() + sla = sl.appender() sl.l.Warn("Stale append failed", "err", err) } - if err = sl.reportStale(app, staleTime); err != nil { + if err = sl.reportStale(sla, staleTime); err != nil { sl.l.Warn("Stale report failed", "err", err) } } @@ -1646,12 +1550,22 @@ func (sl *scrapeLoop) updateStaleMarkers(app storage.Appender, defTime int64) (e return err } -func (sl *scrapeLoop) append(app storage.Appender, b []byte, contentType string, ts time.Time) (total, added, seriesAdded int, err error) { +type scrapeLoopAppender struct { + *scrapeLoop + + storage.Appender +} + +var _ scrapeLoopAppendAdapter = &scrapeLoopAppender{} + +// append for the deprecated storage.Appender flow. +// This is only for downstream project migration purposes and will be removed soon. +func (sl *scrapeLoopAppender) append(b []byte, contentType string, ts time.Time) (total, added, seriesAdded int, err error) { defTime := timestamp.FromTime(ts) if len(b) == 0 { // Empty scrape. Just update the stale makers and swap the cache (but don't flush it). - err = sl.updateStaleMarkers(app, defTime) + err = sl.updateStaleMarkers(sl.Appender, defTime) sl.cache.iterDone(false) return total, added, seriesAdded, err } @@ -1694,7 +1608,7 @@ func (sl *scrapeLoop) append(app storage.Appender, b []byte, contentType string, exemplars := make([]exemplar.Exemplar, 0, 1) // Take an appender with limits. - app = appender(app, sl.sampleLimit, sl.bucketLimit, sl.maxSchema) + app := appenderWithLimits(sl.Appender, sl.sampleLimit, sl.bucketLimit, sl.maxSchema) defer func() { if err != nil { @@ -1915,7 +1829,7 @@ loop: // In majority cases we can trust that the current series/histogram is matching the lastMeta and lastMFName. // However, optional TYPE etc metadata and broken OM text can break this, detect those cases here. // TODO(bwplotka): Consider moving this to parser as many parser users end up doing this (e.g. ST and NHCB parsing). - if isSeriesPartOfFamily(lset.Get(labels.MetricName), lastMFName, lastMeta.Type) { + if isSeriesPartOfFamily(lset.Get(model.MetricNameLabel), lastMFName, lastMeta.Type) { if _, merr := app.UpdateMetadata(ref, lset, lastMeta.Metadata); merr != nil { // No need to fail the scrape on errors appending metadata. sl.l.Debug("Error when appending metadata in scrape loop", "ref", fmt.Sprintf("%d", ref), "metadata", fmt.Sprintf("%+v", lastMeta.Metadata), "err", merr) @@ -2027,7 +1941,7 @@ func isSeriesPartOfFamily(mName string, mfName []byte, typ model.MetricType) boo // during normal operation (e.g., accidental cardinality explosion, sudden traffic spikes). // Current case ordering prevents exercising other cases when limits are exceeded. // Remaining error cases typically occur only a few times, often during initial setup. -func (sl *scrapeLoop) checkAddError(met []byte, err error, sampleLimitErr, bucketLimitErr *error, appErrs *appendErrors) (bool, error) { +func (sl *scrapeLoop) checkAddError(met []byte, err error, sampleLimitErr, bucketLimitErr *error, appErrs *appendErrors) (sampleAdded bool, _ error) { switch { case err == nil: return true, nil @@ -2139,7 +2053,7 @@ var ( } ) -func (sl *scrapeLoop) report(app storage.Appender, start time.Time, duration time.Duration, scraped, added, seriesAdded, bytes int, scrapeErr error) (err error) { +func (sl *scrapeLoop) report(sla scrapeLoopAppendAdapter, start time.Time, duration time.Duration, scraped, added, seriesAdded, bytes int, scrapeErr error) (err error) { sl.scraper.Report(start, duration, scrapeErr) ts := timestamp.FromTime(start) @@ -2150,71 +2064,70 @@ func (sl *scrapeLoop) report(app storage.Appender, start time.Time, duration tim } b := labels.NewBuilderWithSymbolTable(sl.symbolTable) - if err = sl.addReportSample(app, scrapeHealthMetric, ts, health, b); err != nil { + if err = sla.addReportSample(scrapeHealthMetric, ts, health, b, false); err != nil { return err } - if err = sl.addReportSample(app, scrapeDurationMetric, ts, duration.Seconds(), b); err != nil { + if err = sla.addReportSample(scrapeDurationMetric, ts, duration.Seconds(), b, false); err != nil { return err } - if err = sl.addReportSample(app, scrapeSamplesMetric, ts, float64(scraped), b); err != nil { + if err = sla.addReportSample(scrapeSamplesMetric, ts, float64(scraped), b, false); err != nil { return err } - if err = sl.addReportSample(app, samplesPostRelabelMetric, ts, float64(added), b); err != nil { + if err = sla.addReportSample(samplesPostRelabelMetric, ts, float64(added), b, false); err != nil { return err } - if err = sl.addReportSample(app, scrapeSeriesAddedMetric, ts, float64(seriesAdded), b); err != nil { + if err = sla.addReportSample(scrapeSeriesAddedMetric, ts, float64(seriesAdded), b, false); err != nil { return err } if sl.reportExtraMetrics { - if err = sl.addReportSample(app, scrapeTimeoutMetric, ts, sl.timeout.Seconds(), b); err != nil { + if err = sla.addReportSample(scrapeTimeoutMetric, ts, sl.timeout.Seconds(), b, false); err != nil { return err } - if err = sl.addReportSample(app, scrapeSampleLimitMetric, ts, float64(sl.sampleLimit), b); err != nil { + if err = sla.addReportSample(scrapeSampleLimitMetric, ts, float64(sl.sampleLimit), b, false); err != nil { return err } - if err = sl.addReportSample(app, scrapeBodySizeBytesMetric, ts, float64(bytes), b); err != nil { + if err = sla.addReportSample(scrapeBodySizeBytesMetric, ts, float64(bytes), b, false); err != nil { return err } } return err } -func (sl *scrapeLoop) reportStale(app storage.Appender, start time.Time) (err error) { +func (sl *scrapeLoop) reportStale(sla scrapeLoopAppendAdapter, start time.Time) (err error) { ts := timestamp.FromTime(start) - app.SetOptions(&storage.AppendOptions{DiscardOutOfOrder: true}) stale := math.Float64frombits(value.StaleNaN) b := labels.NewBuilder(labels.EmptyLabels()) - if err = sl.addReportSample(app, scrapeHealthMetric, ts, stale, b); err != nil { + if err = sla.addReportSample(scrapeHealthMetric, ts, stale, b, true); err != nil { return err } - if err = sl.addReportSample(app, scrapeDurationMetric, ts, stale, b); err != nil { + if err = sla.addReportSample(scrapeDurationMetric, ts, stale, b, true); err != nil { return err } - if err = sl.addReportSample(app, scrapeSamplesMetric, ts, stale, b); err != nil { + if err = sla.addReportSample(scrapeSamplesMetric, ts, stale, b, true); err != nil { return err } - if err = sl.addReportSample(app, samplesPostRelabelMetric, ts, stale, b); err != nil { + if err = sla.addReportSample(samplesPostRelabelMetric, ts, stale, b, true); err != nil { return err } - if err = sl.addReportSample(app, scrapeSeriesAddedMetric, ts, stale, b); err != nil { + if err = sla.addReportSample(scrapeSeriesAddedMetric, ts, stale, b, true); err != nil { return err } if sl.reportExtraMetrics { - if err = sl.addReportSample(app, scrapeTimeoutMetric, ts, stale, b); err != nil { + if err = sla.addReportSample(scrapeTimeoutMetric, ts, stale, b, true); err != nil { return err } - if err = sl.addReportSample(app, scrapeSampleLimitMetric, ts, stale, b); err != nil { + if err = sla.addReportSample(scrapeSampleLimitMetric, ts, stale, b, true); err != nil { return err } - if err = sl.addReportSample(app, scrapeBodySizeBytesMetric, ts, stale, b); err != nil { + if err = sla.addReportSample(scrapeBodySizeBytesMetric, ts, stale, b, true); err != nil { return err } } return err } -func (sl *scrapeLoop) addReportSample(app storage.Appender, s reportSample, t int64, v float64, b *labels.Builder) error { +func (sl *scrapeLoopAppender) addReportSample(s reportSample, t int64, v float64, b *labels.Builder, rejectOOO bool) error { ce, ok, _ := sl.cache.get(s.name) var ref storage.SeriesRef var lset labels.Labels @@ -2226,18 +2139,22 @@ func (sl *scrapeLoop) addReportSample(app storage.Appender, s reportSample, t in // with scraped metrics in the cache. // We have to drop it when building the actual metric. b.Reset(labels.EmptyLabels()) - b.Set(labels.MetricName, string(s.name[:len(s.name)-1])) + b.Set(model.MetricNameLabel, string(s.name[:len(s.name)-1])) lset = sl.reportSampleMutator(b.Labels()) } - ref, err := app.Append(ref, lset, t, v) + opt := storage.AppendOptions{DiscardOutOfOrder: rejectOOO} + sl.SetOptions(&opt) + ref, err := sl.Append(ref, lset, t, v) + opt.DiscardOutOfOrder = false + sl.SetOptions(&opt) switch { case err == nil: if !ok { sl.cache.addRef(s.name, ref, lset, lset.Hash()) // We only need to add metadata once a scrape target appears. if sl.appendMetadataToWAL { - if _, merr := app.UpdateMetadata(ref, lset, s.Metadata); merr != nil { + if _, merr := sl.UpdateMetadata(ref, lset, s.Metadata); merr != nil { sl.l.Debug("Error when appending metadata in addReportSample", "ref", fmt.Sprintf("%d", ref), "metadata", fmt.Sprintf("%+v", s.Metadata), "err", merr) } } diff --git a/scrape/scrape_append.go b/scrape/scrape_append.go new file mode 100644 index 0000000000..c84a75d4da --- /dev/null +++ b/scrape/scrape_append.go @@ -0,0 +1,187 @@ +// Copyright The Prometheus Authors +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package scrape + +import ( + "time" + + "github.com/prometheus/prometheus/model/histogram" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/model/timestamp" + "github.com/prometheus/prometheus/model/value" + "github.com/prometheus/prometheus/storage" +) + +// appenderWithLimits returns an appender with additional validation. +func appenderWithLimits(app storage.Appender, sampleLimit, bucketLimit int, maxSchema int32) storage.Appender { + app = &timeLimitAppender{ + Appender: app, + maxTime: timestamp.FromTime(time.Now().Add(maxAheadTime)), + } + + // The sampleLimit is applied after metrics are potentially dropped via relabeling. + if sampleLimit > 0 { + app = &limitAppender{ + Appender: app, + limit: sampleLimit, + } + } + + if bucketLimit > 0 { + app = &bucketLimitAppender{ + Appender: app, + limit: bucketLimit, + } + } + + if maxSchema < histogram.ExponentialSchemaMax { + app = &maxSchemaAppender{ + Appender: app, + maxSchema: maxSchema, + } + } + + return app +} + +// limitAppender limits the number of total appended samples in a batch. +type limitAppender struct { + storage.Appender + + limit int + i int +} + +func (app *limitAppender) Append(ref storage.SeriesRef, lset labels.Labels, t int64, v float64) (storage.SeriesRef, error) { + // Bypass sample_limit checks only if we have a staleness marker for a known series (ref value is non-zero). + // This ensures that if a series is already in TSDB then we always write the marker. + if ref == 0 || !value.IsStaleNaN(v) { + app.i++ + if app.i > app.limit { + return 0, errSampleLimit + } + } + ref, err := app.Appender.Append(ref, lset, t, v) + if err != nil { + return 0, err + } + return ref, nil +} + +func (app *limitAppender) AppendHistogram(ref storage.SeriesRef, lset labels.Labels, t int64, h *histogram.Histogram, fh *histogram.FloatHistogram) (storage.SeriesRef, error) { + // Bypass sample_limit checks only if we have a staleness marker for a known series (ref value is non-zero). + // This ensures that if a series is already in TSDB then we always write the marker. + if ref == 0 || (h != nil && !value.IsStaleNaN(h.Sum)) || (fh != nil && !value.IsStaleNaN(fh.Sum)) { + app.i++ + if app.i > app.limit { + return 0, errSampleLimit + } + } + ref, err := app.Appender.AppendHistogram(ref, lset, t, h, fh) + if err != nil { + return 0, err + } + return ref, nil +} + +type timeLimitAppender struct { + storage.Appender + + maxTime int64 +} + +func (app *timeLimitAppender) Append(ref storage.SeriesRef, lset labels.Labels, t int64, v float64) (storage.SeriesRef, error) { + if t > app.maxTime { + return 0, storage.ErrOutOfBounds + } + + ref, err := app.Appender.Append(ref, lset, t, v) + if err != nil { + return 0, err + } + return ref, nil +} + +// bucketLimitAppender limits the number of total appended samples in a batch. +type bucketLimitAppender struct { + storage.Appender + + limit int +} + +func (app *bucketLimitAppender) AppendHistogram(ref storage.SeriesRef, lset labels.Labels, t int64, h *histogram.Histogram, fh *histogram.FloatHistogram) (storage.SeriesRef, error) { + var err error + if h != nil { + // Return with an early error if the histogram has too many buckets and the + // schema is not exponential, in which case we can't reduce the resolution. + if len(h.PositiveBuckets)+len(h.NegativeBuckets) > app.limit && !histogram.IsExponentialSchema(h.Schema) { + return 0, errBucketLimit + } + for len(h.PositiveBuckets)+len(h.NegativeBuckets) > app.limit { + if h.Schema <= histogram.ExponentialSchemaMin { + return 0, errBucketLimit + } + if err = h.ReduceResolution(h.Schema - 1); err != nil { + return 0, err + } + } + } + if fh != nil { + // Return with an early error if the histogram has too many buckets and the + // schema is not exponential, in which case we can't reduce the resolution. + if len(fh.PositiveBuckets)+len(fh.NegativeBuckets) > app.limit && !histogram.IsExponentialSchema(fh.Schema) { + return 0, errBucketLimit + } + for len(fh.PositiveBuckets)+len(fh.NegativeBuckets) > app.limit { + if fh.Schema <= histogram.ExponentialSchemaMin { + return 0, errBucketLimit + } + if err = fh.ReduceResolution(fh.Schema - 1); err != nil { + return 0, err + } + } + } + if ref, err = app.Appender.AppendHistogram(ref, lset, t, h, fh); err != nil { + return 0, err + } + return ref, nil +} + +type maxSchemaAppender struct { + storage.Appender + + maxSchema int32 +} + +func (app *maxSchemaAppender) AppendHistogram(ref storage.SeriesRef, lset labels.Labels, t int64, h *histogram.Histogram, fh *histogram.FloatHistogram) (storage.SeriesRef, error) { + var err error + if h != nil { + if histogram.IsExponentialSchemaReserved(h.Schema) && h.Schema > app.maxSchema { + if err = h.ReduceResolution(app.maxSchema); err != nil { + return 0, err + } + } + } + if fh != nil { + if histogram.IsExponentialSchemaReserved(fh.Schema) && fh.Schema > app.maxSchema { + if err = fh.ReduceResolution(app.maxSchema); err != nil { + return 0, err + } + } + } + if ref, err = app.Appender.AppendHistogram(ref, lset, t, h, fh); err != nil { + return 0, err + } + return ref, nil +} diff --git a/scrape/scrape_append_test.go b/scrape/scrape_append_test.go new file mode 100644 index 0000000000..39eda24a3f --- /dev/null +++ b/scrape/scrape_append_test.go @@ -0,0 +1,260 @@ +// Copyright The Prometheus Authors +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package scrape + +import ( + "fmt" + "testing" + "time" + + "github.com/prometheus/common/model" + "github.com/stretchr/testify/require" + + "github.com/prometheus/prometheus/model/histogram" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/model/timestamp" + "github.com/prometheus/prometheus/util/teststorage" +) + +func TestBucketLimitAppender(t *testing.T) { + example := histogram.Histogram{ + Schema: 0, + Count: 21, + Sum: 33, + ZeroThreshold: 0.001, + ZeroCount: 3, + PositiveSpans: []histogram.Span{ + {Offset: 0, Length: 3}, + }, + PositiveBuckets: []int64{3, 0, 0}, + NegativeSpans: []histogram.Span{ + {Offset: 0, Length: 3}, + }, + NegativeBuckets: []int64{3, 0, 0}, + } + + bigGap := histogram.Histogram{ + Schema: 0, + Count: 21, + Sum: 33, + ZeroThreshold: 0.001, + ZeroCount: 3, + PositiveSpans: []histogram.Span{ + {Offset: 1, Length: 1}, // in (1, 2] + {Offset: 2, Length: 1}, // in (8, 16] + }, + PositiveBuckets: []int64{1, 0}, // 1, 1 + } + + customBuckets := histogram.Histogram{ + Schema: histogram.CustomBucketsSchema, + Count: 9, + Sum: 33, + PositiveSpans: []histogram.Span{ + {Offset: 0, Length: 3}, + }, + PositiveBuckets: []int64{3, 0, 0}, + CustomValues: []float64{1, 2, 3}, + } + + cases := []struct { + h histogram.Histogram + limit int + expectError bool + expectBucketCount int + expectSchema int32 + }{ + { + h: example, + limit: 3, + expectError: true, + }, + { + h: example, + limit: 4, + expectError: false, + expectBucketCount: 4, + expectSchema: -1, + }, + { + h: example, + limit: 10, + expectError: false, + expectBucketCount: 6, + expectSchema: 0, + }, + { + h: bigGap, + limit: 1, + expectError: false, + expectBucketCount: 1, + expectSchema: -2, + }, + { + h: customBuckets, + limit: 2, + expectError: true, + }, + { + h: customBuckets, + limit: 3, + expectError: false, + expectBucketCount: 3, + expectSchema: histogram.CustomBucketsSchema, + }, + } + + appTest := teststorage.NewAppender() + + for _, c := range cases { + for _, floatHisto := range []bool{true, false} { + t.Run(fmt.Sprintf("floatHistogram=%t", floatHisto), func(t *testing.T) { + app := &bucketLimitAppender{Appender: appTest.Appender(t.Context()), limit: c.limit} + ts := int64(10 * time.Minute / time.Millisecond) + lbls := labels.FromStrings("__name__", "sparse_histogram_series") + var err error + if floatHisto { + fh := c.h.Copy().ToFloat(nil) + _, err = app.AppendHistogram(0, lbls, ts, nil, fh) + if c.expectError { + require.Error(t, err) + } else { + require.Equal(t, c.expectSchema, fh.Schema) + require.Equal(t, c.expectBucketCount, len(fh.NegativeBuckets)+len(fh.PositiveBuckets)) + require.NoError(t, err) + } + } else { + h := c.h.Copy() + _, err = app.AppendHistogram(0, lbls, ts, h, nil) + if c.expectError { + require.Error(t, err) + } else { + require.Equal(t, c.expectSchema, h.Schema) + require.Equal(t, c.expectBucketCount, len(h.NegativeBuckets)+len(h.PositiveBuckets)) + require.NoError(t, err) + } + } + require.NoError(t, app.Commit()) + }) + } + } +} + +func TestMaxSchemaAppender(t *testing.T) { + example := histogram.Histogram{ + Schema: 0, + Count: 21, + Sum: 33, + ZeroThreshold: 0.001, + ZeroCount: 3, + PositiveSpans: []histogram.Span{ + {Offset: 0, Length: 3}, + }, + PositiveBuckets: []int64{3, 0, 0}, + NegativeSpans: []histogram.Span{ + {Offset: 0, Length: 3}, + }, + NegativeBuckets: []int64{3, 0, 0}, + } + + customBuckets := histogram.Histogram{ + Schema: histogram.CustomBucketsSchema, + Count: 9, + Sum: 33, + PositiveSpans: []histogram.Span{ + {Offset: 0, Length: 3}, + }, + PositiveBuckets: []int64{3, 0, 0}, + CustomValues: []float64{1, 2, 3}, + } + + cases := []struct { + h histogram.Histogram + maxSchema int32 + expectSchema int32 + }{ + { + h: example, + maxSchema: -1, + expectSchema: -1, + }, + { + h: example, + maxSchema: 0, + expectSchema: 0, + }, + { + h: customBuckets, + maxSchema: -1, + expectSchema: histogram.CustomBucketsSchema, + }, + } + + appTest := teststorage.NewAppender() + + for _, c := range cases { + for _, floatHisto := range []bool{true, false} { + t.Run(fmt.Sprintf("floatHistogram=%t", floatHisto), func(t *testing.T) { + app := &maxSchemaAppender{Appender: appTest.Appender(t.Context()), maxSchema: c.maxSchema} + ts := int64(10 * time.Minute / time.Millisecond) + lbls := labels.FromStrings("__name__", "sparse_histogram_series") + var err error + if floatHisto { + fh := c.h.Copy().ToFloat(nil) + _, err = app.AppendHistogram(0, lbls, ts, nil, fh) + require.Equal(t, c.expectSchema, fh.Schema) + require.NoError(t, err) + } else { + h := c.h.Copy() + _, err = app.AppendHistogram(0, lbls, ts, h, nil) + require.Equal(t, c.expectSchema, h.Schema) + require.NoError(t, err) + } + require.NoError(t, app.Commit()) + }) + } + } +} + +// Test sample_limit when a scrape contains Native Histograms. +func TestAppendWithSampleLimitAndNativeHistogram(t *testing.T) { + appTest := teststorage.NewAppender() + + now := time.Now() + app := appenderWithLimits(appTest.Appender(t.Context()), 2, 0, 0) + + // sample_limit is set to 2, so first two scrapes should work + _, err := app.Append(0, labels.FromStrings(model.MetricNameLabel, "foo"), timestamp.FromTime(now), 1) + require.NoError(t, err) + + // Second sample, should be ok. + _, err = app.AppendHistogram( + 0, + labels.FromStrings(model.MetricNameLabel, "my_histogram1"), + timestamp.FromTime(now), + &histogram.Histogram{}, + nil, + ) + require.NoError(t, err) + + // This is third sample with sample_limit=2, it should trigger errSampleLimit. + _, err = app.AppendHistogram( + 0, + labels.FromStrings(model.MetricNameLabel, "my_histogram2"), + timestamp.FromTime(now), + &histogram.Histogram{}, + nil, + ) + require.ErrorIs(t, err, errSampleLimit) +} diff --git a/scrape/scrape_test.go b/scrape/scrape_test.go index 5ccdb80019..759f5c100a 100644 --- a/scrape/scrape_test.go +++ b/scrape/scrape_test.go @@ -1,4 +1,4 @@ -// Copyright 2016 The Prometheus Authors +// Copyright The Prometheus Authors // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. // You may obtain a copy of the License at @@ -108,9 +108,7 @@ func TestStorageHandlesOutOfOrderTimestamps(t *testing.T) { // Test with default OutOfOrderTimeWindow (0) t.Run("Out-Of-Order Sample Disabled", func(t *testing.T) { s := teststorage.New(t) - t.Cleanup(func() { - _ = s.Close() - }) + t.Cleanup(func() { _ = s.Close() }) runScrapeLoopTest(t, s, false) }) @@ -118,19 +116,16 @@ func TestStorageHandlesOutOfOrderTimestamps(t *testing.T) { // Test with specific OutOfOrderTimeWindow (600000) t.Run("Out-Of-Order Sample Enabled", func(t *testing.T) { s := teststorage.New(t, 600000) - t.Cleanup(func() { - _ = s.Close() - }) + t.Cleanup(func() { _ = s.Close() }) runScrapeLoopTest(t, s, true) }) } func runScrapeLoopTest(t *testing.T, s *teststorage.TestStorage, expectOutOfOrder bool) { - // Create an appender for adding samples to the storage. - app := s.Appender(context.Background()) - capp := &collectResultAppender{next: app} - sl := newBasicScrapeLoop(t, context.Background(), nil, func(context.Context) storage.Appender { return capp }, 0) + sl, _, appTest := newTestScrapeLoop(t) + // Inject storage, so we can query later. + appTest.Next = s // Current time for generating timestamps. now := time.Now() @@ -141,37 +136,35 @@ func runScrapeLoopTest(t *testing.T, s *teststorage.TestStorage, expectOutOfOrde timestampOutOfOrder := now.Add(-5 * time.Minute) timestampInorder2 := now.Add(5 * time.Minute) - slApp := sl.appender(context.Background()) - _, _, _, err := sl.append(slApp, []byte(`metric_total{a="1",b="1"} 1`), "text/plain", timestampInorder1) + slApp := sl.appender() + _, _, _, err := slApp.append([]byte(`metric_total{a="1",b="1"} 1`), "text/plain", timestampInorder1) require.NoError(t, err) - _, _, _, err = sl.append(slApp, []byte(`metric_total{a="1",b="1"} 2`), "text/plain", timestampOutOfOrder) + _, _, _, err = slApp.append([]byte(`metric_total{a="1",b="1"} 2`), "text/plain", timestampOutOfOrder) require.NoError(t, err) - _, _, _, err = sl.append(slApp, []byte(`metric_total{a="1",b="1"} 3`), "text/plain", timestampInorder2) + _, _, _, err = slApp.append([]byte(`metric_total{a="1",b="1"} 3`), "text/plain", timestampInorder2) require.NoError(t, err) require.NoError(t, slApp.Commit()) // Query the samples back from the storage. - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() q, err := s.Querier(time.Time{}.UnixNano(), time.Now().UnixNano()) require.NoError(t, err) - defer q.Close() + t.Cleanup(func() { _ = q.Close() }) // Use a matcher to filter the metric name. - series := q.Select(ctx, false, nil, labels.MustNewMatcher(labels.MatchRegexp, "__name__", "metric_total")) + series := q.Select(t.Context(), false, nil, labels.MustNewMatcher(labels.MatchRegexp, "__name__", "metric_total")) - var results []floatSample + var results []sample for series.Next() { it := series.At().Iterator(nil) for it.Next() == chunkenc.ValFloat { t, v := it.At() - results = append(results, floatSample{ - metric: series.At().Labels(), - t: t, - f: v, + results = append(results, sample{ + L: series.At().Labels(), + T: t, + V: v, }) } require.NoError(t, it.Err()) @@ -179,16 +172,16 @@ func runScrapeLoopTest(t *testing.T, s *teststorage.TestStorage, expectOutOfOrde require.NoError(t, series.Err()) // Define the expected results - want := []floatSample{ + want := []sample{ { - metric: labels.FromStrings("__name__", "metric_total", "a", "1", "b", "1"), - t: timestamp.FromTime(timestampInorder1), - f: 1, + L: labels.FromStrings("__name__", "metric_total", "a", "1", "b", "1"), + T: timestamp.FromTime(timestampInorder1), + V: 1, }, { - metric: labels.FromStrings("__name__", "metric_total", "a", "1", "b", "1"), - t: timestamp.FromTime(timestampInorder2), - f: 3, + L: labels.FromStrings("__name__", "metric_total", "a", "1", "b", "1"), + T: timestamp.FromTime(timestampInorder2), + V: 3, }, } @@ -200,7 +193,7 @@ func runScrapeLoopTest(t *testing.T, s *teststorage.TestStorage, expectOutOfOrde } // Regression test against https://github.com/prometheus/prometheus/issues/15831. -func TestScrapeAppendMetadataUpdate(t *testing.T) { +func TestScrapeAppend_MetadataUpdate(t *testing.T) { const ( scrape1 = `# TYPE test_metric counter # HELP test_metric some help text @@ -223,60 +216,58 @@ test_metric2{foo="bar"} 22 # EOF` ) - // Create an appender for adding samples to the storage. - capp := &collectResultAppender{next: nopAppender{}} - sl := newBasicScrapeLoop(t, context.Background(), nil, func(context.Context) storage.Appender { return capp }, 0) + sl, _, appTest := newTestScrapeLoop(t) now := time.Now() - slApp := sl.appender(context.Background()) - _, _, _, err := sl.append(slApp, []byte(scrape1), "application/openmetrics-text", now) + slApp := sl.appender() + _, _, _, err := slApp.append([]byte(scrape1), "application/openmetrics-text", now) require.NoError(t, err) require.NoError(t, slApp.Commit()) - testutil.RequireEqualWithOptions(t, []metadataEntry{ - {metric: labels.FromStrings("__name__", "test_metric_total"), m: metadata.Metadata{Type: "counter", Unit: "metric", Help: "some help text"}}, - {metric: labels.FromStrings("__name__", "test_metric2", "foo", "bar"), m: metadata.Metadata{Type: "gauge", Unit: "", Help: "other help text"}}, - }, capp.resultMetadata, []cmp.Option{cmp.Comparer(metadataEntryEqual)}) - capp.resultMetadata = nil + testutil.RequireEqual(t, []sample{ + {L: labels.FromStrings("__name__", "test_metric_total"), M: metadata.Metadata{Type: "counter", Unit: "metric", Help: "some help text"}}, + {L: labels.FromStrings("__name__", "test_metric2", "foo", "bar"), M: metadata.Metadata{Type: "gauge", Unit: "", Help: "other help text"}}, + {L: labels.FromStrings("__name__", "test_metric3_metric4", "foo", "bar")}, + }, appTest.ResultMetadata()) + appTest.ResultReset() - // Next (the same) scrape should not add new metadata entries. - slApp = sl.appender(context.Background()) - _, _, _, err = sl.append(slApp, []byte(scrape1), "application/openmetrics-text", now.Add(15*time.Second)) + // Next (the same) scrape should not new metadata entries. + slApp = sl.appender() + _, _, _, err = slApp.append([]byte(scrape1), "application/openmetrics-text", now.Add(15*time.Second)) require.NoError(t, err) require.NoError(t, slApp.Commit()) - testutil.RequireEqualWithOptions(t, []metadataEntry(nil), capp.resultMetadata, []cmp.Option{cmp.Comparer(metadataEntryEqual)}) + testutil.RequireEqual(t, []sample{ + {L: labels.FromStrings("__name__", "test_metric_total")}, + {L: labels.FromStrings("__name__", "test_metric2", "foo", "bar")}, + {L: labels.FromStrings("__name__", "test_metric3_metric4", "foo", "bar")}, + }, appTest.ResultMetadata()) + appTest.ResultReset() - slApp = sl.appender(context.Background()) - _, _, _, err = sl.append(slApp, []byte(scrape2), "application/openmetrics-text", now.Add(15*time.Second)) + slApp = sl.appender() + _, _, _, err = slApp.append([]byte(scrape2), "application/openmetrics-text", now.Add(15*time.Second)) require.NoError(t, err) require.NoError(t, slApp.Commit()) - testutil.RequireEqualWithOptions(t, []metadataEntry{ - {metric: labels.FromStrings("__name__", "test_metric_total"), m: metadata.Metadata{Type: "counter", Unit: "metric", Help: "different help text"}}, // Here, technically we should have no unit, but it's a known limitation of the current implementation. - {metric: labels.FromStrings("__name__", "test_metric2", "foo", "bar"), m: metadata.Metadata{Type: "gauge", Unit: "metric2", Help: "other help text"}}, - }, capp.resultMetadata, []cmp.Option{cmp.Comparer(metadataEntryEqual)}) + testutil.RequireEqual(t, []sample{ + {L: labels.FromStrings("__name__", "test_metric_total"), M: metadata.Metadata{Type: "counter", Unit: "metric", Help: "different help text"}}, // Here, technically we should have no unit, but it's a known limitation of the current implementation. + {L: labels.FromStrings("__name__", "test_metric2", "foo", "bar"), M: metadata.Metadata{Type: "gauge", Unit: "metric2", Help: "other help text"}}, + {L: labels.FromStrings("__name__", "test_metric3_metric4", "foo", "bar")}, // Stale marker. + }, appTest.ResultMetadata()) + appTest.ResultReset() } -type nopScraper struct { - scraper -} +func TestScrapeReportMetadata(t *testing.T) { + sl, _, appTest := newTestScrapeLoop(t) + slApp := sl.appender() -func (nopScraper) Report(time.Time, time.Duration, error) {} - -func TestScrapeReportMetadataUpdate(t *testing.T) { - // Create an appender for adding samples to the storage. - capp := &collectResultAppender{next: nopAppender{}} - sl := newBasicScrapeLoop(t, context.Background(), nopScraper{}, func(context.Context) storage.Appender { return capp }, 0) now := time.Now() - slApp := sl.appender(context.Background()) - require.NoError(t, sl.report(slApp, now, 2*time.Second, 1, 1, 1, 512, nil)) require.NoError(t, slApp.Commit()) - testutil.RequireEqualWithOptions(t, []metadataEntry{ - {metric: labels.FromStrings("__name__", "up"), m: scrapeHealthMetric.Metadata}, - {metric: labels.FromStrings("__name__", "scrape_duration_seconds"), m: scrapeDurationMetric.Metadata}, - {metric: labels.FromStrings("__name__", "scrape_samples_scraped"), m: scrapeSamplesMetric.Metadata}, - {metric: labels.FromStrings("__name__", "scrape_samples_post_metric_relabeling"), m: samplesPostRelabelMetric.Metadata}, - {metric: labels.FromStrings("__name__", "scrape_series_added"), m: scrapeSeriesAddedMetric.Metadata}, - }, capp.resultMetadata, []cmp.Option{cmp.Comparer(metadataEntryEqual)}) + testutil.RequireEqual(t, []sample{ + {L: labels.FromStrings("__name__", "up"), M: scrapeHealthMetric.Metadata}, + {L: labels.FromStrings("__name__", "scrape_duration_seconds"), M: scrapeDurationMetric.Metadata}, + {L: labels.FromStrings("__name__", "scrape_samples_scraped"), M: scrapeSamplesMetric.Metadata}, + {L: labels.FromStrings("__name__", "scrape_samples_post_metric_relabeling"), M: samplesPostRelabelMetric.Metadata}, + {L: labels.FromStrings("__name__", "scrape_series_added"), M: scrapeSeriesAddedMetric.Metadata}, + }, appTest.ResultMetadata()) } func TestIsSeriesPartOfFamily(t *testing.T) { @@ -551,9 +542,8 @@ func TestScrapePoolReload(t *testing.T) { // one terminated. for i := range numTargets { - labels := labels.FromStrings(model.AddressLabel, fmt.Sprintf("example.com:%d", i)) t := &Target{ - labels: labels, + labels: labels.FromStrings(model.AddressLabel, fmt.Sprintf("example.com:%d", i)), scrapeConfig: &config.ScrapeConfig{}, } l := &testLoop{} @@ -577,7 +567,7 @@ func TestScrapePoolReload(t *testing.T) { reloadTime := time.Now() go func() { - sp.reload(reloadCfg) + _ = sp.reload(reloadCfg) close(done) }() @@ -796,7 +786,7 @@ func TestScrapePoolTargetLimit(t *testing.T) { validateErrorMessage(false) } -func TestScrapePoolAppender(t *testing.T) { +func TestScrapePoolAppenderWithLimits(t *testing.T) { cfg := &config.ScrapeConfig{ MetricNameValidationScheme: model.UTF8Validation, MetricNameEscapingScheme: model.AllowUTF8, @@ -810,7 +800,7 @@ func TestScrapePoolAppender(t *testing.T) { appl, ok := loop.(*scrapeLoop) require.True(t, ok, "Expected scrapeLoop but got %T", loop) - wrapped := appender(appl.appender(context.Background()), 0, 0, histogram.ExponentialSchemaMax) + wrapped := appenderWithLimits(appl.appendable.Appender(context.Background()), 0, 0, histogram.ExponentialSchemaMax) tl, ok := wrapped.(*timeLimitAppender) require.True(t, ok, "Expected timeLimitAppender but got %T", wrapped) @@ -826,7 +816,7 @@ func TestScrapePoolAppender(t *testing.T) { appl, ok = loop.(*scrapeLoop) require.True(t, ok, "Expected scrapeLoop but got %T", loop) - wrapped = appender(appl.appender(context.Background()), sampleLimit, 0, histogram.ExponentialSchemaMax) + wrapped = appenderWithLimits(appl.appendable.Appender(context.Background()), sampleLimit, 0, histogram.ExponentialSchemaMax) sl, ok := wrapped.(*limitAppender) require.True(t, ok, "Expected limitAppender but got %T", wrapped) @@ -837,7 +827,7 @@ func TestScrapePoolAppender(t *testing.T) { _, ok = tl.Appender.(nopAppender) require.True(t, ok, "Expected base appender but got %T", tl.Appender) - wrapped = appender(appl.appender(context.Background()), sampleLimit, 100, histogram.ExponentialSchemaMax) + wrapped = appenderWithLimits(appl.appendable.Appender(context.Background()), sampleLimit, 100, histogram.ExponentialSchemaMax) bl, ok := wrapped.(*bucketLimitAppender) require.True(t, ok, "Expected bucketLimitAppender but got %T", wrapped) @@ -851,7 +841,7 @@ func TestScrapePoolAppender(t *testing.T) { _, ok = tl.Appender.(nopAppender) require.True(t, ok, "Expected base appender but got %T", tl.Appender) - wrapped = appender(appl.appender(context.Background()), sampleLimit, 100, 0) + wrapped = appenderWithLimits(appl.appendable.Appender(context.Background()), sampleLimit, 100, 0) ml, ok := wrapped.(*maxSchemaAppender) require.True(t, ok, "Expected maxSchemaAppender but got %T", wrapped) @@ -907,7 +897,7 @@ func TestScrapePoolRaces(t *testing.T) { for range 20 { time.Sleep(10 * time.Millisecond) - sp.reload(newConfig()) + _ = sp.reload(newConfig()) } sp.stop() } @@ -964,48 +954,56 @@ func TestScrapePoolScrapeLoopsStarted(t *testing.T) { } } -func newBasicScrapeLoop(t testing.TB, ctx context.Context, scraper scraper, app func(ctx context.Context) storage.Appender, interval time.Duration) *scrapeLoop { - return newBasicScrapeLoopWithFallback(t, ctx, scraper, app, interval, "") -} +// newTestScrapeLoop is the initial scrape loop for all tests. +// It returns scrapeLoop but also mock scraper (for injections) and appender +// (for injections and append result testing). +// +// NOTE: Try to NOT add more parameter to this function. Try to NOT add more +// newTestScrapeLoop-like constructors. It should be flexible enough with scrapeLoop +// used for initial options. +func newTestScrapeLoop(t testing.TB, opts ...func(sl *scrapeLoop)) (_ *scrapeLoop, scraper *testScraper, appTest *teststorage.Appender) { + scraper = &testScraper{} + appTest = teststorage.NewAppender() -func newBasicScrapeLoopWithFallback(t testing.TB, ctx context.Context, scraper scraper, app func(ctx context.Context) storage.Appender, interval time.Duration, fallback string) *scrapeLoop { - return newScrapeLoop(ctx, - scraper, - nil, nil, - nopMutator, - nopMutator, - app, - nil, - labels.NewSymbolTable(), - 0, - true, - false, - true, - 0, 0, histogram.ExponentialSchemaMax, - nil, - interval, - time.Hour, - false, - false, - false, - false, - false, - false, - true, - nil, - false, - newTestScrapeMetrics(t), - false, - model.UTF8Validation, - model.NoEscaping, - fallback, - ) + ctx := t.Context() + sl := &scrapeLoop{ + appendable: appTest, + sampleMutator: nopMutator, + reportSampleMutator: nopMutator, + validationScheme: model.UTF8Validation, + symbolTable: labels.NewSymbolTable(), + metrics: newTestScrapeMetrics(t), + scrapeLoopOptions: scrapeLoopOptions{ + interval: 10 * time.Millisecond, + bucketLimit: int(histogram.ExponentialSchemaMax), + timeout: 1 * time.Hour, + honorTimestamps: true, + enableCompression: true, + }, + appendMetadataToWAL: true, // Tests assumes it's enabled. + } + for _, o := range opts { + o(sl) + } + // Use sl.ctx as context injection. + if sl.ctx != nil { + ctx = sl.ctx + } + + // Validate user opts for convenience. + require.Nil(t, sl.parentCtx, "newTestScrapeLoop does not support injecting non-ctx contexts") + require.Nil(t, sl.appenderCtx, "newTestScrapeLoop does not support injecting non-ctx contexts") + require.Nil(t, sl.cancel, "newTestScrapeLoop does not support injecting non-ctx contexts") + require.Nil(t, sl.scraper, "newTestScrapeLoop does not support injecting scraper, it's mocked, use returned scraper") + sl.scraper = scraper + sl.init(ctx, true) + return sl, scraper, appTest } func TestScrapeLoopStopBeforeRun(t *testing.T) { t.Parallel() - scraper := &testScraper{} - sl := newBasicScrapeLoop(t, context.Background(), scraper, nil, 1) + + sl, scraper, _ := newTestScrapeLoop(t) // The scrape pool synchronizes on stopping scrape loops. However, new scrape // loops are started asynchronously. Thus it's possible, that a loop is stopped @@ -1053,26 +1051,22 @@ func TestScrapeLoopStopBeforeRun(t *testing.T) { func nopMutator(l labels.Labels) labels.Labels { return l } func TestScrapeLoopStop(t *testing.T) { - var ( - signal = make(chan struct{}, 1) - appender = &collectResultAppender{} - scraper = &testScraper{} - app = func(context.Context) storage.Appender { return appender } - ) + signal := make(chan struct{}, 1) - // Since we're writing samples directly below we need to provide a protocol fallback. - sl := newBasicScrapeLoopWithFallback(t, context.Background(), scraper, app, 10*time.Millisecond, "text/plain") + sl, scraper, appTest := newTestScrapeLoop(t, func(sl *scrapeLoop) { + // Since we're writing samples directly below we need to provide a protocol fallback. + sl.fallbackScrapeProtocol = "text/plain" + }) // Terminate loop after 2 scrapes. numScrapes := 0 - scraper.scrapeFunc = func(ctx context.Context, w io.Writer) error { numScrapes++ if numScrapes == 2 { go sl.stop() <-sl.ctx.Done() } - w.Write([]byte("metric_a 42\n")) + _, _ = w.Write([]byte("metric_a 42\n")) return ctx.Err() } @@ -1089,21 +1083,21 @@ func TestScrapeLoopStop(t *testing.T) { // We expected 1 actual sample for each scrape plus 5 for report samples. // At least 2 scrapes were made, plus the final stale markers. - require.GreaterOrEqual(t, len(appender.resultFloats), 6*3, "Expected at least 3 scrapes with 6 samples each.") - require.Zero(t, len(appender.resultFloats)%6, "There is a scrape with missing samples.") + require.GreaterOrEqual(t, len(appTest.ResultSamples), 6*3, "Expected at least 3 scrapes with 6 samples each.") + require.Zero(t, len(appTest.ResultSamples)%6, "There is a scrape with missing samples.") // All samples in a scrape must have the same timestamp. var ts int64 - for i, s := range appender.resultFloats { + for i, s := range appTest.ResultSamples { switch { case i%6 == 0: - ts = s.t - case s.t != ts: + ts = s.T + case s.T != ts: t.Fatalf("Unexpected multiple timestamps within single scrape") } } // All samples from the last scrape must be stale markers. - for _, s := range appender.resultFloats[len(appender.resultFloats)-5:] { - require.True(t, value.IsStaleNaN(s.f), "Appended last sample not as expected. Wanted: stale NaN Got: %x", math.Float64bits(s.f)) + for _, s := range appTest.ResultSamples[len(appTest.ResultSamples)-5:] { + require.True(t, value.IsStaleNaN(s.V), "Appended last sample not as expected. Wanted: stale NaN Got: %x", math.Float64bits(s.V)) } } @@ -1112,45 +1106,12 @@ func TestScrapeLoopRun(t *testing.T) { var ( signal = make(chan struct{}, 1) errc = make(chan error) - - scraper = &testScraper{} - app = func(context.Context) storage.Appender { return &nopAppender{} } - scrapeMetrics = newTestScrapeMetrics(t) - ) - - ctx, cancel := context.WithCancel(context.Background()) - sl := newScrapeLoop(ctx, - scraper, - nil, nil, - nopMutator, - nopMutator, - app, - nil, - nil, - 0, - true, - false, - true, - 0, 0, histogram.ExponentialSchemaMax, - nil, - time.Second, - time.Hour, - false, - false, - false, - false, - false, - false, - false, - nil, - false, - scrapeMetrics, - false, - model.UTF8Validation, - model.NoEscaping, - "", ) + ctx, cancel := context.WithCancel(t.Context()) + sl, scraper, _ := newTestScrapeLoop(t, func(sl *scrapeLoop) { + sl.ctx = ctx + }) // The loop must terminate during the initial offset if the context // is canceled. scraper.offsetDur = time.Hour @@ -1186,9 +1147,11 @@ func TestScrapeLoopRun(t *testing.T) { return nil } - ctx, cancel = context.WithCancel(context.Background()) - sl = newBasicScrapeLoop(t, ctx, scraper, app, time.Second) - sl.timeout = 100 * time.Millisecond + ctx, cancel = context.WithCancel(t.Context()) + sl, scraper, _ = newTestScrapeLoop(t, func(sl *scrapeLoop) { + sl.ctx = ctx + sl.timeout = 100 * time.Millisecond + }) go func() { sl.run(errc) @@ -1222,16 +1185,14 @@ func TestScrapeLoopForcedErr(t *testing.T) { var ( signal = make(chan struct{}, 1) errc = make(chan error) - - scraper = &testScraper{} - app = func(context.Context) storage.Appender { return &nopAppender{} } ) - ctx, cancel := context.WithCancel(context.Background()) - sl := newBasicScrapeLoop(t, ctx, scraper, app, time.Second) - + ctx, cancel := context.WithCancel(t.Context()) forcedErr := errors.New("forced err") - sl.setForcedError(forcedErr) + sl, scraper, _ := newTestScrapeLoop(t, func(sl *scrapeLoop) { + sl.ctx = ctx + sl.setForcedError(forcedErr) + }) scraper.scrapeFunc = func(context.Context, io.Writer) error { require.FailNow(t, "Should not be scraped.") @@ -1259,50 +1220,10 @@ func TestScrapeLoopForcedErr(t *testing.T) { } func TestScrapeLoopMetadata(t *testing.T) { - var ( - signal = make(chan struct{}) - scraper = &testScraper{} - scrapeMetrics = newTestScrapeMetrics(t) - cache = newScrapeCache(scrapeMetrics) - ) - defer close(signal) + sl, _, _ := newTestScrapeLoop(t) - ctx, cancel := context.WithCancel(context.Background()) - sl := newScrapeLoop(ctx, - scraper, - nil, nil, - nopMutator, - nopMutator, - func(context.Context) storage.Appender { return nopAppender{} }, - cache, - labels.NewSymbolTable(), - 0, - true, - false, - true, - 0, 0, histogram.ExponentialSchemaMax, - nil, - 0, - 0, - false, - false, - false, - false, - false, - false, - false, - nil, - false, - scrapeMetrics, - false, - model.UTF8Validation, - model.NoEscaping, - "", - ) - defer cancel() - - slApp := sl.appender(ctx) - total, _, _, err := sl.append(slApp, []byte(`# TYPE test_metric counter + slApp := sl.appender() + total, _, _, err := slApp.append([]byte(`# TYPE test_metric counter # HELP test_metric some help text # UNIT test_metric metric test_metric_total 1 @@ -1313,50 +1234,38 @@ test_metric_total 1 require.NoError(t, slApp.Commit()) require.Equal(t, 1, total) - md, ok := cache.GetMetadata("test_metric") + md, ok := sl.cache.GetMetadata("test_metric") require.True(t, ok, "expected metadata to be present") require.Equal(t, model.MetricTypeCounter, md.Type, "unexpected metric type") require.Equal(t, "some help text", md.Help) require.Equal(t, "metric", md.Unit) - md, ok = cache.GetMetadata("test_metric_no_help") + md, ok = sl.cache.GetMetadata("test_metric_no_help") require.True(t, ok, "expected metadata to be present") require.Equal(t, model.MetricTypeGauge, md.Type, "unexpected metric type") require.Empty(t, md.Help) require.Empty(t, md.Unit) - md, ok = cache.GetMetadata("test_metric_no_type") + md, ok = sl.cache.GetMetadata("test_metric_no_type") require.True(t, ok, "expected metadata to be present") require.Equal(t, model.MetricTypeUnknown, md.Type, "unexpected metric type") require.Equal(t, "other help text", md.Help) require.Empty(t, md.Unit) } -func simpleTestScrapeLoop(t testing.TB) (context.Context, *scrapeLoop) { - // Need a full storage for correct Add/AddFast semantics. - s := teststorage.New(t) - t.Cleanup(func() { s.Close() }) - - ctx, cancel := context.WithCancel(context.Background()) - sl := newBasicScrapeLoop(t, ctx, &testScraper{}, s.Appender, 0) - t.Cleanup(func() { cancel() }) - - return ctx, sl -} - func TestScrapeLoopSeriesAdded(t *testing.T) { - ctx, sl := simpleTestScrapeLoop(t) + sl, _, _ := newTestScrapeLoop(t) - slApp := sl.appender(ctx) - total, added, seriesAdded, err := sl.append(slApp, []byte("test_metric 1\n"), "text/plain", time.Time{}) + slApp := sl.appender() + total, added, seriesAdded, err := slApp.append([]byte("test_metric 1\n"), "text/plain", time.Time{}) require.NoError(t, err) require.NoError(t, slApp.Commit()) require.Equal(t, 1, total) require.Equal(t, 1, added) require.Equal(t, 1, seriesAdded) - slApp = sl.appender(ctx) - total, added, seriesAdded, err = sl.append(slApp, []byte("test_metric 1\n"), "text/plain", time.Time{}) + slApp = sl.appender() + total, added, seriesAdded, err = slApp.append([]byte("test_metric 1\n"), "text/plain", time.Time{}) require.NoError(t, slApp.Commit()) require.NoError(t, err) require.Equal(t, 1, total) @@ -1365,10 +1274,6 @@ func TestScrapeLoopSeriesAdded(t *testing.T) { } func TestScrapeLoopFailWithInvalidLabelsAfterRelabel(t *testing.T) { - s := teststorage.New(t) - defer s.Close() - ctx := t.Context() - target := &Target{ labels: labels.FromStrings("pod_label_invalid_012\xff", "test"), } @@ -1379,13 +1284,14 @@ func TestScrapeLoopFailWithInvalidLabelsAfterRelabel(t *testing.T) { Replacement: "$1", NameValidationScheme: model.UTF8Validation, }} - sl := newBasicScrapeLoop(t, ctx, &testScraper{}, s.Appender, 0) - sl.sampleMutator = func(l labels.Labels) labels.Labels { - return mutateSampleLabels(l, target, true, relabelConfig) - } + sl, _, _ := newTestScrapeLoop(t, func(sl *scrapeLoop) { + sl.sampleMutator = func(l labels.Labels) labels.Labels { + return mutateSampleLabels(l, target, true, relabelConfig) + } + }) - slApp := sl.appender(ctx) - total, added, seriesAdded, err := sl.append(slApp, []byte("test_metric 1\n"), "text/plain", time.Time{}) + slApp := sl.appender() + total, added, seriesAdded, err := slApp.append([]byte("test_metric 1\n"), "text/plain", time.Time{}) require.ErrorContains(t, err, "invalid metric name or label names") require.NoError(t, slApp.Rollback()) require.Equal(t, 1, total) @@ -1394,17 +1300,12 @@ func TestScrapeLoopFailWithInvalidLabelsAfterRelabel(t *testing.T) { } func TestScrapeLoopFailLegacyUnderUTF8(t *testing.T) { - // Test that scrapes fail when default validation is utf8 but scrape config is - // legacy. - s := teststorage.New(t) - defer s.Close() - ctx := t.Context() + sl, _, _ := newTestScrapeLoop(t, func(sl *scrapeLoop) { + sl.validationScheme = model.LegacyValidation + }) - sl := newBasicScrapeLoop(t, ctx, &testScraper{}, s.Appender, 0) - sl.validationScheme = model.LegacyValidation - - slApp := sl.appender(ctx) - total, added, seriesAdded, err := sl.append(slApp, []byte("{\"test.metric\"} 1\n"), "text/plain", time.Time{}) + slApp := sl.appender() + total, added, seriesAdded, err := slApp.append([]byte("{\"test.metric\"} 1\n"), "text/plain", time.Time{}) require.ErrorContains(t, err, "invalid metric name or label names") require.NoError(t, slApp.Rollback()) require.Equal(t, 1, total) @@ -1412,10 +1313,12 @@ func TestScrapeLoopFailLegacyUnderUTF8(t *testing.T) { require.Equal(t, 0, seriesAdded) // When scrapeloop has validation set to UTF-8, the metric is allowed. - sl.validationScheme = model.UTF8Validation + sl, _, _ = newTestScrapeLoop(t, func(sl *scrapeLoop) { + sl.validationScheme = model.UTF8Validation + }) - slApp = sl.appender(ctx) - total, added, seriesAdded, err = sl.append(slApp, []byte("{\"test.metric\"} 1\n"), "text/plain", time.Time{}) + slApp = sl.appender() + total, added, seriesAdded, err = slApp.append([]byte("{\"test.metric\"} 1\n"), "text/plain", time.Time{}) require.NoError(t, err) require.Equal(t, 1, total) require.Equal(t, 1, added) @@ -1434,12 +1337,12 @@ func readTextParseTestMetrics(t testing.TB) []byte { func makeTestGauges(n int) []byte { sb := bytes.Buffer{} - fmt.Fprintf(&sb, "# TYPE metric_a gauge\n") - fmt.Fprintf(&sb, "# HELP metric_a help text\n") + _, _ = fmt.Fprintf(&sb, "# TYPE metric_a gauge\n") + _, _ = fmt.Fprintf(&sb, "# HELP metric_a help text\n") for i := range n { - fmt.Fprintf(&sb, "metric_a{foo=\"%d\",bar=\"%d\"} 1\n", i, i*100) + _, _ = fmt.Fprintf(&sb, "metric_a{foo=\"%d\",bar=\"%d\"} 1\n", i, i*100) } - fmt.Fprintf(&sb, "# EOF\n") + _, _ = fmt.Fprintf(&sb, "# EOF\n") return sb.Bytes() } @@ -1536,16 +1439,21 @@ func BenchmarkScrapeLoopAppend(b *testing.B) { {name: "PromProto", contentType: "application/vnd.google.protobuf", parsable: metricsProto}, } { b.Run(fmt.Sprintf("fmt=%v", bcase.name), func(b *testing.B) { - ctx, sl := simpleTestScrapeLoop(b) + // Need a full storage for correct Add/AddFast semantics. + s := teststorage.New(b) + b.Cleanup(func() { _ = s.Close() }) - slApp := sl.appender(ctx) + sl, _, _ := newTestScrapeLoop(b, func(sl *scrapeLoop) { + sl.appendable = s + }) + slApp := sl.appender() ts := time.Time{} b.ReportAllocs() b.ResetTimer() for b.Loop() { ts = ts.Add(time.Second) - _, _, _, err := sl.append(slApp, bcase.parsable, bcase.contentType, ts) + _, _, _, err := slApp.append(bcase.parsable, bcase.contentType, ts) if err != nil { b.Fatal(err) } @@ -1558,28 +1466,26 @@ func BenchmarkScrapeLoopAppend(b *testing.B) { func TestSetOptionsHandlingStaleness(t *testing.T) { s := teststorage.New(t, 600000) - defer s.Close() + t.Cleanup(func() { _ = s.Close() }) signal := make(chan struct{}, 1) - ctx, cancel := context.WithCancel(context.Background()) + ctx, cancel := context.WithCancel(t.Context()) defer cancel() // Function to run the scrape loop runScrapeLoop := func(ctx context.Context, t *testing.T, cue int, action func(*scrapeLoop)) { - var ( - scraper = &testScraper{} - app = func(ctx context.Context) storage.Appender { - return s.Appender(ctx) - } - ) - sl := newBasicScrapeLoop(t, ctx, scraper, app, 10*time.Millisecond) + sl, scraper, _ := newTestScrapeLoop(t, func(sl *scrapeLoop) { + sl.ctx = ctx + sl.appendable = s + }) + numScrapes := 0 scraper.scrapeFunc = func(_ context.Context, w io.Writer) error { numScrapes++ if numScrapes == cue { action(sl) } - fmt.Fprintf(w, "metric_a{a=\"1\",b=\"1\"} %d\n", 42+numScrapes) + _, _ = fmt.Fprintf(w, "metric_a{a=\"1\",b=\"1\"} %d\n", 42+numScrapes) return nil } sl.run(nil) @@ -1604,25 +1510,25 @@ func TestSetOptionsHandlingStaleness(t *testing.T) { t.Fatalf("Scrape wasn't stopped.") } - ctx1, cancel := context.WithCancel(context.Background()) + ctx1, cancel := context.WithCancel(t.Context()) defer cancel() q, err := s.Querier(0, time.Now().UnixNano()) require.NoError(t, err) - defer q.Close() + t.Cleanup(func() { _ = q.Close() }) series := q.Select(ctx1, false, nil, labels.MustNewMatcher(labels.MatchRegexp, "__name__", "metric_a")) - var results []floatSample + var results []sample for series.Next() { it := series.At().Iterator(nil) for it.Next() == chunkenc.ValFloat { t, v := it.At() - results = append(results, floatSample{ - metric: series.At().Labels(), - t: t, - f: v, + results = append(results, sample{ + L: series.At().Labels(), + T: t, + V: v, }) } require.NoError(t, it.Err()) @@ -1630,7 +1536,7 @@ func TestSetOptionsHandlingStaleness(t *testing.T) { require.NoError(t, series.Err()) var c int for _, s := range results { - if value.IsStaleNaN(s.f) { + if value.IsStaleNaN(s.V) { c++ } } @@ -1638,25 +1544,23 @@ func TestSetOptionsHandlingStaleness(t *testing.T) { } func TestScrapeLoopRunCreatesStaleMarkersOnFailedScrape(t *testing.T) { - appender := &collectResultAppender{} - var ( - signal = make(chan struct{}, 1) - scraper = &testScraper{} - app = func(context.Context) storage.Appender { return appender } - ) + signal := make(chan struct{}, 1) + + ctx, cancel := context.WithCancel(t.Context()) + sl, scraper, appTest := newTestScrapeLoop(t, func(sl *scrapeLoop) { + sl.ctx = ctx + // Since we're writing samples directly below we need to provide a protocol fallback. + sl.fallbackScrapeProtocol = "text/plain" + }) - ctx, cancel := context.WithCancel(context.Background()) - // Since we're writing samples directly below we need to provide a protocol fallback. - sl := newBasicScrapeLoopWithFallback(t, ctx, scraper, app, 10*time.Millisecond, "text/plain") // Succeed once, several failures, then stop. numScrapes := 0 - scraper.scrapeFunc = func(_ context.Context, w io.Writer) error { numScrapes++ switch numScrapes { case 1: - w.Write([]byte("metric_a 42\n")) + _, _ = w.Write([]byte("metric_a 42\n")) return nil case 5: cancel() @@ -1675,36 +1579,36 @@ func TestScrapeLoopRunCreatesStaleMarkersOnFailedScrape(t *testing.T) { require.FailNow(t, "Scrape wasn't stopped.") } - // 1 successfully scraped sample, 1 stale marker after first fail, 5 report samples for - // each scrape successful or not. - require.Len(t, appender.resultFloats, 27, "Appended samples not as expected:\n%s", appender) - require.Equal(t, 42.0, appender.resultFloats[0].f, "Appended first sample not as expected") - require.True(t, value.IsStaleNaN(appender.resultFloats[6].f), - "Appended second sample not as expected. Wanted: stale NaN Got: %x", math.Float64bits(appender.resultFloats[6].f)) + // 1 successfully scraped sample + // 1 stale marker after first fail + // 5x 5 report samples for each scrape successful or not. + require.Len(t, appTest.ResultSamples, 27, "Appended samples not as expected:\n%s", appTest) + require.Equal(t, 42.0, appTest.ResultSamples[0].V, "Appended first sample not as expected") + require.True(t, value.IsStaleNaN(appTest.ResultSamples[6].V), + "Appended second sample not as expected. Wanted: stale NaN Got: %x", math.Float64bits(appTest.ResultSamples[6].V)) } func TestScrapeLoopRunCreatesStaleMarkersOnParseFailure(t *testing.T) { - appender := &collectResultAppender{} - var ( - signal = make(chan struct{}, 1) - scraper = &testScraper{} - app = func(context.Context) storage.Appender { return appender } - numScrapes = 0 - ) + signal := make(chan struct{}, 1) - ctx, cancel := context.WithCancel(context.Background()) - // Since we're writing samples directly below we need to provide a protocol fallback. - sl := newBasicScrapeLoopWithFallback(t, ctx, scraper, app, 10*time.Millisecond, "text/plain") + ctx, cancel := context.WithCancel(t.Context()) + sl, scraper, appTest := newTestScrapeLoop(t, func(sl *scrapeLoop) { + sl.ctx = ctx + // Since we're writing samples directly below we need to provide a protocol fallback. + sl.fallbackScrapeProtocol = "text/plain" + }) // Succeed once, several failures, then stop. + numScrapes := 0 scraper.scrapeFunc = func(_ context.Context, w io.Writer) error { numScrapes++ + switch numScrapes { case 1: - w.Write([]byte("metric_a 42\n")) + _, _ = w.Write([]byte("metric_a 42\n")) return nil case 2: - w.Write([]byte("7&-\n")) + _, _ = w.Write([]byte("7&-\n")) return nil case 3: cancel() @@ -1719,46 +1623,46 @@ func TestScrapeLoopRunCreatesStaleMarkersOnParseFailure(t *testing.T) { select { case <-signal: + // TODO(bwplotka): Prone to flakiness, depend on atomic numScrapes. case <-time.After(5 * time.Second): require.FailNow(t, "Scrape wasn't stopped.") } - // 1 successfully scraped sample, 1 stale marker after first fail, 5 report samples for - // each scrape successful or not. - require.Len(t, appender.resultFloats, 17, "Appended samples not as expected:\n%s", appender) - require.Equal(t, 42.0, appender.resultFloats[0].f, "Appended first sample not as expected") - require.True(t, value.IsStaleNaN(appender.resultFloats[6].f), - "Appended second sample not as expected. Wanted: stale NaN Got: %x", math.Float64bits(appender.resultFloats[6].f)) + // 1 successfully scraped sample + // 1 stale marker after first fail + // 3x 5 report samples for each scrape successful or not. + require.Len(t, appTest.ResultSamples, 17, "Appended samples not as expected:\n%s", appTest) + require.Equal(t, 42.0, appTest.ResultSamples[0].V, "Appended first sample not as expected") + require.True(t, value.IsStaleNaN(appTest.ResultSamples[6].V), + "Appended second sample not as expected. Wanted: stale NaN Got: %x", math.Float64bits(appTest.ResultSamples[6].V)) } // If we have a target with sample_limit set and scrape initially works but then we hit the sample_limit error, // then we don't expect to see any StaleNaNs appended for the series that disappeared due to sample_limit error. func TestScrapeLoopRunCreatesStaleMarkersOnSampleLimit(t *testing.T) { - appender := &collectResultAppender{} - var ( - signal = make(chan struct{}, 1) - scraper = &testScraper{} - app = func(_ context.Context) storage.Appender { return appender } - numScrapes = 0 - ) + signal := make(chan struct{}, 1) - ctx, cancel := context.WithCancel(context.Background()) - // Since we're writing samples directly below we need to provide a protocol fallback. - sl := newBasicScrapeLoopWithFallback(t, ctx, scraper, app, 10*time.Millisecond, "text/plain") - sl.sampleLimit = 4 + ctx, cancel := context.WithCancel(t.Context()) + sl, scraper, appTest := newTestScrapeLoop(t, func(sl *scrapeLoop) { + sl.ctx = ctx + // Since we're writing samples directly below we need to provide a protocol fallback. + sl.fallbackScrapeProtocol = "text/plain" + sl.sampleLimit = 4 + }) // Succeed once, several failures, then stop. + numScrapes := 0 scraper.scrapeFunc = func(_ context.Context, w io.Writer) error { numScrapes++ switch numScrapes { case 1: - w.Write([]byte("metric_a 10\nmetric_b 10\nmetric_c 10\nmetric_d 10\n")) + _, _ = w.Write([]byte("metric_a 10\nmetric_b 10\nmetric_c 10\nmetric_d 10\n")) return nil case 2: - w.Write([]byte("metric_a 20\nmetric_b 20\nmetric_c 20\nmetric_d 20\nmetric_e 999\n")) + _, _ = w.Write([]byte("metric_a 20\nmetric_b 20\nmetric_c 20\nmetric_d 20\nmetric_e 999\n")) return nil case 3: - w.Write([]byte("metric_a 30\nmetric_b 30\nmetric_c 30\nmetric_d 30\n")) + _, _ = w.Write([]byte("metric_a 30\nmetric_b 30\nmetric_c 30\nmetric_d 30\n")) return nil case 4: cancel() @@ -1782,44 +1686,44 @@ func TestScrapeLoopRunCreatesStaleMarkersOnSampleLimit(t *testing.T) { // #2 - sample_limit exceeded - no samples appended, only 5 report series // #3 - success - 4 samples appended + 5 report series // #4 - scrape canceled - 4 StaleNaNs appended because of scrape error + 5 report series - require.Len(t, appender.resultFloats, (4+5)+5+(4+5)+(4+5), "Appended samples not as expected:\n%s", appender) + require.Len(t, appTest.ResultSamples, (4+5)+5+(4+5)+(4+5), "Appended samples not as expected:\n%s", appTest) // Expect first 4 samples to be metric_X [0-3]. for i := range 4 { - require.Equal(t, 10.0, appender.resultFloats[i].f, "Appended %d sample not as expected", i) + require.Equal(t, 10.0, appTest.ResultSamples[i].V, "Appended %d sample not as expected", i) } // Next 5 samples are report series [4-8]. // Next 5 samples are report series for the second scrape [9-13]. // Expect first 4 samples to be metric_X from the third scrape [14-17]. for i := 14; i <= 17; i++ { - require.Equal(t, 30.0, appender.resultFloats[i].f, "Appended %d sample not as expected", i) + require.Equal(t, 30.0, appTest.ResultSamples[i].V, "Appended %d sample not as expected", i) } // Next 5 samples are report series [18-22]. // Next 5 samples are report series [23-26]. for i := 23; i <= 26; i++ { - require.True(t, value.IsStaleNaN(appender.resultFloats[i].f), - "Appended second sample not as expected. Wanted: stale NaN Got: %x", math.Float64bits(appender.resultFloats[i].f)) + require.True(t, value.IsStaleNaN(appTest.ResultSamples[i].V), + "Appended second sample not as expected. Wanted: stale NaN Got: %x", math.Float64bits(appTest.ResultSamples[i].V)) } } func TestScrapeLoopCache(t *testing.T) { s := teststorage.New(t) - defer s.Close() + t.Cleanup(func() { _ = s.Close() }) - appender := &collectResultAppender{} - var ( - signal = make(chan struct{}, 1) - scraper = &testScraper{} - app = func(ctx context.Context) storage.Appender { appender.next = s.Appender(ctx); return appender } - ) + signal := make(chan struct{}, 1) - ctx, cancel := context.WithCancel(context.Background()) - // Decreasing the scrape interval could make the test fail, as multiple scrapes might be initiated at identical millisecond timestamps. - // See https://github.com/prometheus/prometheus/issues/12727. - // Since we're writing samples directly below we need to provide a protocol fallback. - sl := newBasicScrapeLoopWithFallback(t, ctx, scraper, app, 100*time.Millisecond, "text/plain") + ctx, cancel := context.WithCancel(t.Context()) + sl, scraper, appTest := newTestScrapeLoop(t, func(sl *scrapeLoop) { + sl.l = promslog.New(&promslog.Config{}) + sl.ctx = ctx + // Since we're writing samples directly below we need to provide a protocol fallback. + sl.fallbackScrapeProtocol = "text/plain" + // Decreasing the scrape interval could make the test fail, as multiple scrapes might be initiated at identical millisecond timestamps. + // See https://github.com/prometheus/prometheus/issues/12727. + sl.interval = 100 * time.Millisecond + }) + appTest.Next = s numScrapes := 0 - scraper.scrapeFunc = func(_ context.Context, w io.Writer) error { switch numScrapes { case 1, 2: @@ -1837,10 +1741,10 @@ func TestScrapeLoopCache(t *testing.T) { numScrapes++ switch numScrapes { case 1: - w.Write([]byte("metric_a 42\nmetric_b 43\n")) + _, _ = w.Write([]byte("metric_a 42\nmetric_b 43\n")) return nil case 3: - w.Write([]byte("metric_a 44\n")) + _, _ = w.Write([]byte("metric_a 44\n")) return nil case 4: cancel() @@ -1859,29 +1763,23 @@ func TestScrapeLoopCache(t *testing.T) { require.FailNow(t, "Scrape wasn't stopped.") } - // 1 successfully scraped sample, 1 stale marker after first fail, 5 report samples for - // each scrape successful or not. - require.Len(t, appender.resultFloats, 26, "Appended samples not as expected:\n%s", appender) + // 3 successfully scraped samples + // 3 stale marker after samples were missing. + // 4x 5 report samples for each scrape successful or not. + require.Len(t, appTest.ResultSamples, 26, "Appended samples not as expected:\n%s", appTest) } func TestScrapeLoopCacheMemoryExhaustionProtection(t *testing.T) { s := teststorage.New(t) - defer s.Close() + t.Cleanup(func() { _ = s.Close() }) - sapp := s.Appender(context.Background()) - - appender := &collectResultAppender{next: sapp} - var ( - signal = make(chan struct{}, 1) - scraper = &testScraper{} - app = func(context.Context) storage.Appender { return appender } - ) - - ctx, cancel := context.WithCancel(context.Background()) - sl := newBasicScrapeLoop(t, ctx, scraper, app, 10*time.Millisecond) + signal := make(chan struct{}, 1) + ctx, cancel := context.WithCancel(t.Context()) + sl, scraper, _ := newTestScrapeLoop(t, func(sl *scrapeLoop) { + sl.ctx = ctx + }) numScrapes := 0 - scraper.scrapeFunc = func(_ context.Context, w io.Writer) error { numScrapes++ if numScrapes < 5 { @@ -1889,7 +1787,7 @@ func TestScrapeLoopCacheMemoryExhaustionProtection(t *testing.T) { for i := range 500 { s = fmt.Sprintf("%smetric_%d_%d 42\n", s, i, numScrapes) } - w.Write([]byte(s + "&")) + _, _ = w.Write([]byte(s + "&")) } else { cancel() } @@ -1964,37 +1862,36 @@ func TestScrapeLoopAppend(t *testing.T) { } for _, test := range tests { - app := &collectResultAppender{} - discoveryLabels := &Target{ labels: labels.FromStrings(test.discoveryLabels...), } - sl := newBasicScrapeLoop(t, context.Background(), nil, func(context.Context) storage.Appender { return app }, 0) - sl.sampleMutator = func(l labels.Labels) labels.Labels { - return mutateSampleLabels(l, discoveryLabels, test.honorLabels, nil) - } - sl.reportSampleMutator = func(l labels.Labels) labels.Labels { - return mutateReportSampleLabels(l, discoveryLabels) - } + sl, _, appTest := newTestScrapeLoop(t, func(sl *scrapeLoop) { + sl.sampleMutator = func(l labels.Labels) labels.Labels { + return mutateSampleLabels(l, discoveryLabels, test.honorLabels, nil) + } + sl.reportSampleMutator = func(l labels.Labels) labels.Labels { + return mutateReportSampleLabels(l, discoveryLabels) + } + }) now := time.Now() - slApp := sl.appender(context.Background()) - _, _, _, err := sl.append(slApp, []byte(test.scrapeLabels), "text/plain", now) + slApp := sl.appender() + _, _, _, err := slApp.append([]byte(test.scrapeLabels), "text/plain", now) require.NoError(t, err) require.NoError(t, slApp.Commit()) - expected := []floatSample{ + expected := []sample{ { - metric: test.expLset, - t: timestamp.FromTime(now), - f: test.expValue, + L: test.expLset, + T: timestamp.FromTime(now), + V: test.expValue, }, } t.Logf("Test:%s", test.title) - requireEqual(t, expected, app.resultFloats) + requireEqual(t, expected, appTest.ResultSamples) } } @@ -2002,13 +1899,12 @@ func requireEqual(t *testing.T, expected, actual any, msgAndArgs ...any) { t.Helper() testutil.RequireEqualWithOptions(t, expected, actual, []cmp.Option{ - cmp.Comparer(equalFloatSamples), - cmp.AllowUnexported(histogramSample{}), + cmp.Comparer(func(a, b sample) bool { return a.Equal(b) }), // StaleNaN samples are generated by iterating over a map, which means that the order // of samples might be different on every test run. Sort series by label to avoid // test failures because of that. - cmpopts.SortSlices(func(a, b floatSample) int { - return labels.Compare(a.metric, b.metric) + cmpopts.SortSlices(func(a, b sample) int { + return labels.Compare(a.L, b.L) }), }, msgAndArgs...) @@ -2066,32 +1962,31 @@ func TestScrapeLoopAppendForConflictingPrefixedLabels(t *testing.T) { for name, tc := range testcases { t.Run(name, func(t *testing.T) { - app := &collectResultAppender{} - sl := newBasicScrapeLoop(t, context.Background(), nil, func(context.Context) storage.Appender { return app }, 0) - sl.sampleMutator = func(l labels.Labels) labels.Labels { - return mutateSampleLabels(l, &Target{labels: labels.FromStrings(tc.targetLabels...)}, false, nil) - } - slApp := sl.appender(context.Background()) - _, _, _, err := sl.append(slApp, []byte(tc.exposedLabels), "text/plain", time.Date(2000, 1, 1, 1, 0, 0, 0, time.UTC)) + sl, _, appTest := newTestScrapeLoop(t, func(sl *scrapeLoop) { + sl.sampleMutator = func(l labels.Labels) labels.Labels { + return mutateSampleLabels(l, &Target{labels: labels.FromStrings(tc.targetLabels...)}, false, nil) + } + }) + + slApp := sl.appender() + _, _, _, err := slApp.append([]byte(tc.exposedLabels), "text/plain", time.Date(2000, 1, 1, 1, 0, 0, 0, time.UTC)) require.NoError(t, err) require.NoError(t, slApp.Commit()) - requireEqual(t, []floatSample{ + requireEqual(t, []sample{ { - metric: labels.FromStrings(tc.expected...), - t: timestamp.FromTime(time.Date(2000, 1, 1, 1, 0, 0, 0, time.UTC)), - f: 0, + L: labels.FromStrings(tc.expected...), + T: timestamp.FromTime(time.Date(2000, 1, 1, 1, 0, 0, 0, time.UTC)), + V: 0, }, - }, app.resultFloats) + }, appTest.ResultSamples) }) } } func TestScrapeLoopAppendCacheEntryButErrNotFound(t *testing.T) { - // collectResultAppender's AddFast always returns ErrNotFound if we don't give it a next. - app := &collectResultAppender{} - sl := newBasicScrapeLoop(t, context.Background(), nil, func(context.Context) storage.Appender { return app }, 0) + sl, _, appTest := newTestScrapeLoop(t) fakeRef := storage.SeriesRef(1) expValue := float64(1) @@ -2101,7 +1996,8 @@ func TestScrapeLoopAppendCacheEntryButErrNotFound(t *testing.T) { require.NoError(t, warning) var lset labels.Labels - p.Next() + _, err := p.Next() + require.NoError(t, err) p.Labels(&lset) hash := lset.Hash() @@ -2109,34 +2005,33 @@ func TestScrapeLoopAppendCacheEntryButErrNotFound(t *testing.T) { sl.cache.addRef(metric, fakeRef, lset, hash) now := time.Now() - slApp := sl.appender(context.Background()) - _, _, _, err := sl.append(slApp, metric, "text/plain", now) + slApp := sl.appender() + _, _, _, err = slApp.append(metric, "text/plain", now) require.NoError(t, err) require.NoError(t, slApp.Commit()) - expected := []floatSample{ + expected := []sample{ { - metric: lset, - t: timestamp.FromTime(now), - f: expValue, + L: lset, + T: timestamp.FromTime(now), + V: expValue, }, } - require.Equal(t, expected, app.resultFloats) + require.Equal(t, expected, appTest.ResultSamples) } func TestScrapeLoopAppendSampleLimit(t *testing.T) { - resApp := &collectResultAppender{} - app := &limitAppender{Appender: resApp, limit: 1} - - sl := newBasicScrapeLoop(t, context.Background(), nil, func(context.Context) storage.Appender { return app }, 0) - sl.sampleMutator = func(l labels.Labels) labels.Labels { - if l.Has("deleteme") { - return labels.EmptyLabels() + sl, _, appTest := newTestScrapeLoop(t, func(sl *scrapeLoop) { + sl.sampleMutator = func(l labels.Labels) labels.Labels { + if l.Has("deleteme") { + return labels.EmptyLabels() + } + return l } - return l - } - sl.sampleLimit = app.limit + sl.sampleLimit = 1 // Same as limitAppender.limit + }) + // appTest.Prev = &limitAppender{Appender: nopAppender{}, limit: 1} // Get the value of the Counter before performing the append. beforeMetric := dto.Metric{} @@ -2146,8 +2041,8 @@ func TestScrapeLoopAppendSampleLimit(t *testing.T) { beforeMetricValue := beforeMetric.GetCounter().GetValue() now := time.Now() - slApp := sl.appender(context.Background()) - total, added, seriesAdded, err := sl.append(app, []byte("metric_a 1\nmetric_b 1\nmetric_c 1\n"), "text/plain", now) + slApp := sl.appender() + total, added, seriesAdded, err := slApp.append([]byte("metric_a 1\nmetric_b 1\nmetric_c 1\n"), "text/plain", now) require.ErrorIs(t, err, errSampleLimit) require.NoError(t, slApp.Rollback()) require.Equal(t, 3, total) @@ -2160,23 +2055,23 @@ func TestScrapeLoopAppendSampleLimit(t *testing.T) { err = sl.metrics.targetScrapeSampleLimit.Write(&metric) require.NoError(t, err) - value := metric.GetCounter().GetValue() - change := value - beforeMetricValue + v := metric.GetCounter().GetValue() + change := v - beforeMetricValue require.Equal(t, 1.0, change, "Unexpected change of sample limit metric: %f", change) // And verify that we got the samples that fit under the limit. - want := []floatSample{ + want := []sample{ { - metric: labels.FromStrings(model.MetricNameLabel, "metric_a"), - t: timestamp.FromTime(now), - f: 1, + L: labels.FromStrings(model.MetricNameLabel, "metric_a"), + T: timestamp.FromTime(now), + V: 1, }, } - requireEqual(t, want, resApp.rolledbackFloats, "Appended samples not as expected:\n%s", appender) + requireEqual(t, want, appTest.RolledbackSamples, "Appended samples not as expected:\n%s", appTest) now = time.Now() - slApp = sl.appender(context.Background()) - total, added, seriesAdded, err = sl.append(slApp, []byte("metric_a 1\nmetric_b 1\nmetric_c{deleteme=\"yes\"} 1\nmetric_d 1\nmetric_e 1\nmetric_f 1\nmetric_g 1\nmetric_h{deleteme=\"yes\"} 1\nmetric_i{deleteme=\"yes\"} 1\n"), "text/plain", now) + slApp = sl.appender() + total, added, seriesAdded, err = slApp.append([]byte("metric_a 1\nmetric_b 1\nmetric_c{deleteme=\"yes\"} 1\nmetric_d 1\nmetric_e 1\nmetric_f 1\nmetric_g 1\nmetric_h{deleteme=\"yes\"} 1\nmetric_i{deleteme=\"yes\"} 1\n"), "text/plain", now) require.ErrorIs(t, err, errSampleLimit) require.NoError(t, slApp.Rollback()) require.Equal(t, 9, total) @@ -2185,17 +2080,18 @@ func TestScrapeLoopAppendSampleLimit(t *testing.T) { } func TestScrapeLoop_HistogramBucketLimit(t *testing.T) { - resApp := &collectResultAppender{} - app := &bucketLimitAppender{Appender: resApp, limit: 2} - - sl := newBasicScrapeLoop(t, context.Background(), nil, func(context.Context) storage.Appender { return app }, 0) - sl.enableNativeHistogramScraping = true - sl.sampleMutator = func(l labels.Labels) labels.Labels { - if l.Has("deleteme") { - return labels.EmptyLabels() + sl, _, _ := newTestScrapeLoop(t, func(sl *scrapeLoop) { + sl.enableNativeHistogramScraping = true + sl.sampleMutator = func(l labels.Labels) labels.Labels { + if l.Has("deleteme") { + return labels.EmptyLabels() + } + return l } - return l - } + }) + + // appTest.Prev = &bucketLimitAppender{Appender: nopAppender{}, limit: 2} + slApp := sl.appender() metric := dto.Metric{} err := sl.metrics.targetScrapeNativeHistogramBucketLimit.Write(&metric) @@ -2214,7 +2110,7 @@ func TestScrapeLoop_HistogramBucketLimit(t *testing.T) { []string{"size"}, ) registry := prometheus.NewRegistry() - registry.Register(nativeHistogram) + require.NoError(t, registry.Register(nativeHistogram)) nativeHistogram.WithLabelValues("S").Observe(1.0) nativeHistogram.WithLabelValues("M").Observe(1.0) nativeHistogram.WithLabelValues("L").Observe(1.0) @@ -2230,7 +2126,7 @@ func TestScrapeLoop_HistogramBucketLimit(t *testing.T) { require.NoError(t, err) now := time.Now() - total, added, seriesAdded, err := sl.append(app, msg, "application/vnd.google.protobuf", now) + total, added, seriesAdded, err := slApp.append(msg, "application/vnd.google.protobuf", now) require.NoError(t, err) require.Equal(t, 3, total) require.Equal(t, 3, added) @@ -2253,7 +2149,7 @@ func TestScrapeLoop_HistogramBucketLimit(t *testing.T) { require.NoError(t, err) now = time.Now() - total, added, seriesAdded, err = sl.append(app, msg, "application/vnd.google.protobuf", now) + total, added, seriesAdded, err = slApp.append(msg, "application/vnd.google.protobuf", now) require.NoError(t, err) require.Equal(t, 3, total) require.Equal(t, 3, added) @@ -2276,11 +2172,11 @@ func TestScrapeLoop_HistogramBucketLimit(t *testing.T) { require.NoError(t, err) now = time.Now() - total, added, seriesAdded, err = sl.append(app, msg, "application/vnd.google.protobuf", now) + total, added, seriesAdded, err = slApp.append(msg, "application/vnd.google.protobuf", now) if !errors.Is(err, errBucketLimit) { t.Fatalf("Did not see expected histogram bucket limit error: %s", err) } - require.NoError(t, app.Rollback()) + require.NoError(t, slApp.Rollback()) require.Equal(t, 3, total) require.Equal(t, 3, added) require.Equal(t, 0, seriesAdded) @@ -2296,148 +2192,144 @@ func TestScrapeLoop_ChangingMetricString(t *testing.T) { // IDs when the string representation of a metric changes across a scrape. Thus // we use a real storage appender here. s := teststorage.New(t) - defer s.Close() + t.Cleanup(func() { _ = s.Close() }) - capp := &collectResultAppender{} - sl := newBasicScrapeLoop(t, context.Background(), nil, func(context.Context) storage.Appender { return capp }, 0) + sl, _, appTest := newTestScrapeLoop(t) now := time.Now() - slApp := sl.appender(context.Background()) - _, _, _, err := sl.append(slApp, []byte(`metric_a{a="1",b="1"} 1`), "text/plain", now) + slApp := sl.appender() + _, _, _, err := slApp.append([]byte(`metric_a{a="1",b="1"} 1`), "text/plain", now) require.NoError(t, err) require.NoError(t, slApp.Commit()) - slApp = sl.appender(context.Background()) - _, _, _, err = sl.append(slApp, []byte(`metric_a{b="1",a="1"} 2`), "text/plain", now.Add(time.Minute)) + slApp = sl.appender() + _, _, _, err = slApp.append([]byte(`metric_a{b="1",a="1"} 2`), "text/plain", now.Add(time.Minute)) require.NoError(t, err) require.NoError(t, slApp.Commit()) - want := []floatSample{ + want := []sample{ { - metric: labels.FromStrings("__name__", "metric_a", "a", "1", "b", "1"), - t: timestamp.FromTime(now), - f: 1, + L: labels.FromStrings("__name__", "metric_a", "a", "1", "b", "1"), + T: timestamp.FromTime(now), + V: 1, }, { - metric: labels.FromStrings("__name__", "metric_a", "a", "1", "b", "1"), - t: timestamp.FromTime(now.Add(time.Minute)), - f: 2, + L: labels.FromStrings("__name__", "metric_a", "a", "1", "b", "1"), + T: timestamp.FromTime(now.Add(time.Minute)), + V: 2, }, } - require.Equal(t, want, capp.resultFloats, "Appended samples not as expected:\n%s", appender) + require.Equal(t, want, appTest.ResultSamples, "Appended samples not as expected:\n%s", appTest) } func TestScrapeLoopAppendFailsWithNoContentType(t *testing.T) { - app := &collectResultAppender{} - - // Explicitly setting the lack of fallback protocol here to make it obvious. - sl := newBasicScrapeLoopWithFallback(t, context.Background(), nil, func(context.Context) storage.Appender { return app }, 0, "") + sl, _, _ := newTestScrapeLoop(t, func(sl *scrapeLoop) { + // Explicitly setting the lack of fallback protocol here to make it obvious. + sl.fallbackScrapeProtocol = "" + }) now := time.Now() - slApp := sl.appender(context.Background()) - _, _, _, err := sl.append(slApp, []byte("metric_a 1\n"), "", now) + slApp := sl.appender() + _, _, _, err := slApp.append([]byte("metric_a 1\n"), "", now) // We expect the appropriate error. require.ErrorContains(t, err, "non-compliant scrape target sending blank Content-Type and no fallback_scrape_protocol specified for target", "Expected \"non-compliant scrape\" error but got: %s", err) } +// TestScrapeLoopAppendEmptyWithNoContentType ensures we there are no errors when we get a blank scrape or just want to append a stale marker. func TestScrapeLoopAppendEmptyWithNoContentType(t *testing.T) { - // This test ensures we there are no errors when we get a blank scrape or just want to append a stale marker. - app := &collectResultAppender{} - - // Explicitly setting the lack of fallback protocol here to make it obvious. - sl := newBasicScrapeLoopWithFallback(t, context.Background(), nil, func(context.Context) storage.Appender { return app }, 0, "") + sl, _, _ := newTestScrapeLoop(t, func(sl *scrapeLoop) { + // Explicitly setting the lack of fallback protocol here to make it obvious. + sl.fallbackScrapeProtocol = "" + }) now := time.Now() - slApp := sl.appender(context.Background()) - _, _, _, err := sl.append(slApp, []byte(""), "", now) + slApp := sl.appender() + _, _, _, err := slApp.append([]byte(""), "", now) require.NoError(t, err) require.NoError(t, slApp.Commit()) } func TestScrapeLoopAppendStaleness(t *testing.T) { - app := &collectResultAppender{} - - sl := newBasicScrapeLoop(t, context.Background(), nil, func(context.Context) storage.Appender { return app }, 0) + sl, _, appTest := newTestScrapeLoop(t) now := time.Now() - slApp := sl.appender(context.Background()) - _, _, _, err := sl.append(slApp, []byte("metric_a 1\n"), "text/plain", now) + slApp := sl.appender() + _, _, _, err := slApp.append([]byte("metric_a 1\n"), "text/plain", now) require.NoError(t, err) require.NoError(t, slApp.Commit()) - slApp = sl.appender(context.Background()) - _, _, _, err = sl.append(slApp, []byte(""), "", now.Add(time.Second)) + slApp = sl.appender() + _, _, _, err = slApp.append([]byte(""), "", now.Add(time.Second)) require.NoError(t, err) require.NoError(t, slApp.Commit()) - want := []floatSample{ + want := []sample{ { - metric: labels.FromStrings(model.MetricNameLabel, "metric_a"), - t: timestamp.FromTime(now), - f: 1, + L: labels.FromStrings(model.MetricNameLabel, "metric_a"), + T: timestamp.FromTime(now), + V: 1, }, { - metric: labels.FromStrings(model.MetricNameLabel, "metric_a"), - t: timestamp.FromTime(now.Add(time.Second)), - f: math.Float64frombits(value.StaleNaN), + L: labels.FromStrings(model.MetricNameLabel, "metric_a"), + T: timestamp.FromTime(now.Add(time.Second)), + V: math.Float64frombits(value.StaleNaN), }, } - requireEqual(t, want, app.resultFloats, "Appended samples not as expected:\n%s", appender) + requireEqual(t, want, appTest.ResultSamples, "Appended samples not as expected:\n%s", appTest) } func TestScrapeLoopAppendNoStalenessIfTimestamp(t *testing.T) { - app := &collectResultAppender{} - sl := newBasicScrapeLoop(t, context.Background(), nil, func(context.Context) storage.Appender { return app }, 0) + sl, _, appTest := newTestScrapeLoop(t) now := time.Now() - slApp := sl.appender(context.Background()) - _, _, _, err := sl.append(slApp, []byte("metric_a 1 1000\n"), "text/plain", now) + slApp := sl.appender() + _, _, _, err := slApp.append([]byte("metric_a 1 1000\n"), "text/plain", now) require.NoError(t, err) require.NoError(t, slApp.Commit()) - slApp = sl.appender(context.Background()) - _, _, _, err = sl.append(slApp, []byte(""), "", now.Add(time.Second)) + slApp = sl.appender() + _, _, _, err = slApp.append([]byte(""), "", now.Add(time.Second)) require.NoError(t, err) require.NoError(t, slApp.Commit()) - want := []floatSample{ + want := []sample{ { - metric: labels.FromStrings(model.MetricNameLabel, "metric_a"), - t: 1000, - f: 1, + L: labels.FromStrings(model.MetricNameLabel, "metric_a"), + T: 1000, + V: 1, }, } - require.Equal(t, want, app.resultFloats, "Appended samples not as expected:\n%s", appender) + require.Equal(t, want, appTest.ResultSamples, "Appended samples not as expected:\n%s", appTest) } func TestScrapeLoopAppendStalenessIfTrackTimestampStaleness(t *testing.T) { - app := &collectResultAppender{} - sl := newBasicScrapeLoop(t, context.Background(), nil, func(context.Context) storage.Appender { return app }, 0) - sl.trackTimestampsStaleness = true + sl, _, appTest := newTestScrapeLoop(t, func(sl *scrapeLoop) { + sl.trackTimestampsStaleness = true + }) now := time.Now() - slApp := sl.appender(context.Background()) - _, _, _, err := sl.append(slApp, []byte("metric_a 1 1000\n"), "text/plain", now) + slApp := sl.appender() + _, _, _, err := slApp.append([]byte("metric_a 1 1000\n"), "text/plain", now) require.NoError(t, err) require.NoError(t, slApp.Commit()) - slApp = sl.appender(context.Background()) - _, _, _, err = sl.append(slApp, []byte(""), "", now.Add(time.Second)) + slApp = sl.appender() + _, _, _, err = slApp.append([]byte(""), "", now.Add(time.Second)) require.NoError(t, err) require.NoError(t, slApp.Commit()) - want := []floatSample{ + want := []sample{ { - metric: labels.FromStrings(model.MetricNameLabel, "metric_a"), - t: 1000, - f: 1, + L: labels.FromStrings(model.MetricNameLabel, "metric_a"), + T: 1000, + V: 1, }, { - metric: labels.FromStrings(model.MetricNameLabel, "metric_a"), - t: timestamp.FromTime(now.Add(time.Second)), - f: math.Float64frombits(value.StaleNaN), + L: labels.FromStrings(model.MetricNameLabel, "metric_a"), + T: timestamp.FromTime(now.Add(time.Second)), + V: math.Float64frombits(value.StaleNaN), }, } - requireEqual(t, want, app.resultFloats, "Appended samples not as expected:\n%s", appender) + requireEqual(t, want, appTest.ResultSamples, "Appended samples not as expected:\n%s", appTest) } func TestScrapeLoopAppendExemplar(t *testing.T) { @@ -2448,18 +2340,16 @@ func TestScrapeLoopAppendExemplar(t *testing.T) { scrapeText string contentType string discoveryLabels []string - floats []floatSample - histograms []histogramSample - exemplars []exemplar.Exemplar + samples []sample }{ { title: "Metric without exemplars", scrapeText: "metric_total{n=\"1\"} 0\n# EOF", contentType: "application/openmetrics-text", discoveryLabels: []string{"n", "2"}, - floats: []floatSample{{ - metric: labels.FromStrings("__name__", "metric_total", "exported_n", "1", "n", "2"), - f: 0, + samples: []sample{{ + L: labels.FromStrings("__name__", "metric_total", "exported_n", "1", "n", "2"), + V: 0, }}, }, { @@ -2467,26 +2357,24 @@ func TestScrapeLoopAppendExemplar(t *testing.T) { scrapeText: "metric_total{n=\"1\"} 0 # {a=\"abc\"} 1.0\n# EOF", contentType: "application/openmetrics-text", discoveryLabels: []string{"n", "2"}, - floats: []floatSample{{ - metric: labels.FromStrings("__name__", "metric_total", "exported_n", "1", "n", "2"), - f: 0, + samples: []sample{{ + L: labels.FromStrings("__name__", "metric_total", "exported_n", "1", "n", "2"), + V: 0, + ES: []exemplar.Exemplar{ + {Labels: labels.FromStrings("a", "abc"), Value: 1}, + }, }}, - exemplars: []exemplar.Exemplar{ - {Labels: labels.FromStrings("a", "abc"), Value: 1}, - }, }, { title: "Metric with exemplars and TS", scrapeText: "metric_total{n=\"1\"} 0 # {a=\"abc\"} 1.0 10000\n# EOF", contentType: "application/openmetrics-text", discoveryLabels: []string{"n", "2"}, - floats: []floatSample{{ - metric: labels.FromStrings("__name__", "metric_total", "exported_n", "1", "n", "2"), - f: 0, + samples: []sample{{ + L: labels.FromStrings("__name__", "metric_total", "exported_n", "1", "n", "2"), + V: 0, + ES: []exemplar.Exemplar{{Labels: labels.FromStrings("a", "abc"), Value: 1, Ts: 10000000, HasTs: true}}, }}, - exemplars: []exemplar.Exemplar{ - {Labels: labels.FromStrings("a", "abc"), Value: 1, Ts: 10000000, HasTs: true}, - }, }, { title: "Two metrics and exemplars", @@ -2494,17 +2382,15 @@ func TestScrapeLoopAppendExemplar(t *testing.T) { metric_total{n="2"} 2 # {t="2"} 2.0 20000 # EOF`, contentType: "application/openmetrics-text", - floats: []floatSample{{ - metric: labels.FromStrings("__name__", "metric_total", "n", "1"), - f: 1, + samples: []sample{{ + L: labels.FromStrings("__name__", "metric_total", "n", "1"), + V: 1, + ES: []exemplar.Exemplar{{Labels: labels.FromStrings("t", "1"), Value: 1, Ts: 10000000, HasTs: true}}, }, { - metric: labels.FromStrings("__name__", "metric_total", "n", "2"), - f: 2, + L: labels.FromStrings("__name__", "metric_total", "n", "2"), + V: 2, + ES: []exemplar.Exemplar{{Labels: labels.FromStrings("t", "2"), Value: 2, Ts: 20000000, HasTs: true}}, }}, - exemplars: []exemplar.Exemplar{ - {Labels: labels.FromStrings("t", "1"), Value: 1, Ts: 10000000, HasTs: true}, - {Labels: labels.FromStrings("t", "2"), Value: 2, Ts: 20000000, HasTs: true}, - }, }, { title: "Native histogram with three exemplars from classic buckets", @@ -2596,10 +2482,10 @@ metric: < `, contentType: "application/vnd.google.protobuf", - histograms: []histogramSample{{ - t: 1234568, - metric: labels.FromStrings("__name__", "test_histogram"), - h: &histogram.Histogram{ + samples: []sample{{ + T: 1234568, + L: labels.FromStrings("__name__", "test_histogram"), + H: &histogram.Histogram{ Count: 175, ZeroCount: 2, Sum: 0.0008280461746287094, @@ -2616,12 +2502,12 @@ metric: < PositiveBuckets: []int64{1, 2, -1, -1}, NegativeBuckets: []int64{1, 3, -2, -1, 1}, }, + ES: []exemplar.Exemplar{ + // Native histogram exemplars are arranged by timestamp, and those with missing timestamps are dropped. + {Labels: labels.FromStrings("dummyID", "58215"), Value: -0.00019, Ts: 1625851055146, HasTs: true}, + {Labels: labels.FromStrings("dummyID", "59727"), Value: -0.00039, Ts: 1625851155146, HasTs: true}, + }, }}, - exemplars: []exemplar.Exemplar{ - // Native histogram exemplars are arranged by timestamp, and those with missing timestamps are dropped. - {Labels: labels.FromStrings("dummyID", "58215"), Value: -0.00019, Ts: 1625851055146, HasTs: true}, - {Labels: labels.FromStrings("dummyID", "59727"), Value: -0.00039, Ts: 1625851155146, HasTs: true}, - }, }, { title: "Native histogram with three exemplars scraped as classic histogram", @@ -2714,46 +2600,46 @@ metric: < `, alwaysScrapeClassicHist: true, contentType: "application/vnd.google.protobuf", - floats: []floatSample{ - {metric: labels.FromStrings("__name__", "test_histogram_count"), t: 1234568, f: 175}, - {metric: labels.FromStrings("__name__", "test_histogram_sum"), t: 1234568, f: 0.0008280461746287094}, - {metric: labels.FromStrings("__name__", "test_histogram_bucket", "le", "-0.0004899999999999998"), t: 1234568, f: 2}, - {metric: labels.FromStrings("__name__", "test_histogram_bucket", "le", "-0.0003899999999999998"), t: 1234568, f: 4}, - {metric: labels.FromStrings("__name__", "test_histogram_bucket", "le", "-0.0002899999999999998"), t: 1234568, f: 16}, - {metric: labels.FromStrings("__name__", "test_histogram_bucket", "le", "-0.0001899999999999998"), t: 1234568, f: 32}, - {metric: labels.FromStrings("__name__", "test_histogram_bucket", "le", "+Inf"), t: 1234568, f: 175}, - }, - histograms: []histogramSample{{ - t: 1234568, - metric: labels.FromStrings("__name__", "test_histogram"), - h: &histogram.Histogram{ - Count: 175, - ZeroCount: 2, - Sum: 0.0008280461746287094, - ZeroThreshold: 2.938735877055719e-39, - Schema: 3, - PositiveSpans: []histogram.Span{ - {Offset: -161, Length: 1}, - {Offset: 8, Length: 3}, + samples: []sample{ + {L: labels.FromStrings("__name__", "test_histogram_count"), T: 1234568, V: 175}, + {L: labels.FromStrings("__name__", "test_histogram_sum"), T: 1234568, V: 0.0008280461746287094}, + {L: labels.FromStrings("__name__", "test_histogram_bucket", "le", "-0.0004899999999999998"), T: 1234568, V: 2}, + {L: labels.FromStrings("__name__", "test_histogram_bucket", "le", "-0.0003899999999999998"), T: 1234568, V: 4}, + {L: labels.FromStrings("__name__", "test_histogram_bucket", "le", "-0.0002899999999999998"), T: 1234568, V: 16}, + {L: labels.FromStrings("__name__", "test_histogram_bucket", "le", "-0.0001899999999999998"), T: 1234568, V: 32}, + {L: labels.FromStrings("__name__", "test_histogram_bucket", "le", "+Inf"), T: 1234568, V: 175}, + { + T: 1234568, + L: labels.FromStrings("__name__", "test_histogram"), + H: &histogram.Histogram{ + Count: 175, + ZeroCount: 2, + Sum: 0.0008280461746287094, + ZeroThreshold: 2.938735877055719e-39, + Schema: 3, + PositiveSpans: []histogram.Span{ + {Offset: -161, Length: 1}, + {Offset: 8, Length: 3}, + }, + NegativeSpans: []histogram.Span{ + {Offset: -162, Length: 1}, + {Offset: 23, Length: 4}, + }, + PositiveBuckets: []int64{1, 2, -1, -1}, + NegativeBuckets: []int64{1, 3, -2, -1, 1}, }, - NegativeSpans: []histogram.Span{ - {Offset: -162, Length: 1}, - {Offset: 23, Length: 4}, + ES: []exemplar.Exemplar{ + // Native histogram one is arranged by timestamp. + // Exemplars with missing timestamps are dropped for native histograms. + {Labels: labels.FromStrings("dummyID", "58215"), Value: -0.00019, Ts: 1625851055146, HasTs: true}, + {Labels: labels.FromStrings("dummyID", "59727"), Value: -0.00039, Ts: 1625851155146, HasTs: true}, + // Classic histogram one is in order of appearance. + // Exemplars with missing timestamps are supported for classic histograms. + {Labels: labels.FromStrings("dummyID", "59727"), Value: -0.00039, Ts: 1625851155146, HasTs: true}, + {Labels: labels.FromStrings("dummyID", "5617"), Value: -0.00029, Ts: 1234568, HasTs: false}, + {Labels: labels.FromStrings("dummyID", "58215"), Value: -0.00019, Ts: 1625851055146, HasTs: true}, }, - PositiveBuckets: []int64{1, 2, -1, -1}, - NegativeBuckets: []int64{1, 3, -2, -1, 1}, }, - }}, - exemplars: []exemplar.Exemplar{ - // Native histogram one is arranged by timestamp. - // Exemplars with missing timestamps are dropped for native histograms. - {Labels: labels.FromStrings("dummyID", "58215"), Value: -0.00019, Ts: 1625851055146, HasTs: true}, - {Labels: labels.FromStrings("dummyID", "59727"), Value: -0.00039, Ts: 1625851155146, HasTs: true}, - // Classic histogram one is in order of appearance. - // Exemplars with missing timestamps are supported for classic histograms. - {Labels: labels.FromStrings("dummyID", "59727"), Value: -0.00039, Ts: 1625851155146, HasTs: true}, - {Labels: labels.FromStrings("dummyID", "5617"), Value: -0.00029, Ts: 1234568, HasTs: false}, - {Labels: labels.FromStrings("dummyID", "58215"), Value: -0.00019, Ts: 1625851055146, HasTs: true}, }, }, { @@ -2829,10 +2715,10 @@ metric: < > `, - histograms: []histogramSample{{ - t: 1234568, - metric: labels.FromStrings("__name__", "test_histogram"), - h: &histogram.Histogram{ + samples: []sample{{ + T: 1234568, + L: labels.FromStrings("__name__", "test_histogram"), + H: &histogram.Histogram{ Count: 175, ZeroCount: 2, Sum: 0.0008280461746287094, @@ -2849,12 +2735,12 @@ metric: < PositiveBuckets: []int64{1, 2, -1, -1}, NegativeBuckets: []int64{1, 3, -2, -1, 1}, }, + ES: []exemplar.Exemplar{ + // Exemplars with missing timestamps are dropped for native histograms. + {Labels: labels.FromStrings("dummyID", "58242"), Value: -0.00019, Ts: 1625851055146, HasTs: true}, + {Labels: labels.FromStrings("dummyID", "59732"), Value: -0.00039, Ts: 1625851155146, HasTs: true}, + }, }}, - exemplars: []exemplar.Exemplar{ - // Exemplars with missing timestamps are dropped for native histograms. - {Labels: labels.FromStrings("dummyID", "58242"), Value: -0.00019, Ts: 1625851055146, HasTs: true}, - {Labels: labels.FromStrings("dummyID", "59732"), Value: -0.00039, Ts: 1625851155146, HasTs: true}, - }, }, { title: "Native histogram with exemplars but ingestion disabled", @@ -2929,45 +2815,45 @@ metric: < > `, - floats: []floatSample{ - {metric: labels.FromStrings("__name__", "test_histogram_count"), t: 1234568, f: 175}, - {metric: labels.FromStrings("__name__", "test_histogram_sum"), t: 1234568, f: 0.0008280461746287094}, - {metric: labels.FromStrings("__name__", "test_histogram_bucket", "le", "+Inf"), t: 1234568, f: 175}, + samples: []sample{ + {L: labels.FromStrings("__name__", "test_histogram_count"), T: 1234568, V: 175}, + {L: labels.FromStrings("__name__", "test_histogram_sum"), T: 1234568, V: 0.0008280461746287094}, + {L: labels.FromStrings("__name__", "test_histogram_bucket", "le", "+Inf"), T: 1234568, V: 175}, }, }, } for _, test := range tests { t.Run(test.title, func(t *testing.T) { - app := &collectResultAppender{} - discoveryLabels := &Target{ labels: labels.FromStrings(test.discoveryLabels...), } - sl := newBasicScrapeLoop(t, context.Background(), nil, func(context.Context) storage.Appender { return app }, 0) - sl.enableNativeHistogramScraping = test.enableNativeHistogramsIngestion - sl.sampleMutator = func(l labels.Labels) labels.Labels { - return mutateSampleLabels(l, discoveryLabels, false, nil) - } - sl.reportSampleMutator = func(l labels.Labels) labels.Labels { - return mutateReportSampleLabels(l, discoveryLabels) - } - sl.alwaysScrapeClassicHist = test.alwaysScrapeClassicHist + sl, _, appTest := newTestScrapeLoop(t, func(sl *scrapeLoop) { + sl.enableNativeHistogramScraping = test.enableNativeHistogramsIngestion + sl.sampleMutator = func(l labels.Labels) labels.Labels { + return mutateSampleLabels(l, discoveryLabels, false, nil) + } + sl.reportSampleMutator = func(l labels.Labels) labels.Labels { + return mutateReportSampleLabels(l, discoveryLabels) + } + sl.alwaysScrapeClassicHist = test.alwaysScrapeClassicHist + }) + slApp := sl.appender() now := time.Now() - for i := range test.floats { - if test.floats[i].t != 0 { + for i := range test.samples { + if test.samples[i].T != 0 { continue } - test.floats[i].t = timestamp.FromTime(now) - } + test.samples[i].T = timestamp.FromTime(now) - // We need to set the timestamp for expected exemplars that does not have a timestamp. - for i := range test.exemplars { - if test.exemplars[i].Ts == 0 { - test.exemplars[i].Ts = timestamp.FromTime(now) + // We need to set the timestamp for expected exemplars that does not have a timestamp. + for j := range test.samples[i].ES { + if test.samples[i].ES[j].Ts == 0 { + test.samples[i].ES[j].Ts = timestamp.FromTime(now) + } } } @@ -2978,12 +2864,10 @@ metric: < buf.WriteString(test.scrapeText) } - _, _, _, err := sl.append(app, buf.Bytes(), test.contentType, now) + _, _, _, err := slApp.append(buf.Bytes(), test.contentType, now) require.NoError(t, err) - require.NoError(t, app.Commit()) - requireEqual(t, test.floats, app.resultFloats) - requireEqual(t, test.histograms, app.resultHistograms) - requireEqual(t, test.exemplars, app.resultExemplars) + require.NoError(t, slApp.Commit()) + requireEqual(t, test.samples, appTest.ResultSamples) }) } } @@ -3012,12 +2896,12 @@ func TestScrapeLoopAppendExemplarSeries(t *testing.T) { scrapeText := []string{`metric_total{n="1"} 1 # {t="1"} 1.0 10000 # EOF`, `metric_total{n="1"} 2 # {t="2"} 2.0 20000 # EOF`} - samples := []floatSample{{ - metric: labels.FromStrings("__name__", "metric_total", "n", "1"), - f: 1, + samples := []sample{{ + L: labels.FromStrings("__name__", "metric_total", "n", "1"), + V: 1, }, { - metric: labels.FromStrings("__name__", "metric_total", "n", "1"), - f: 2, + L: labels.FromStrings("__name__", "metric_total", "n", "1"), + V: 2, }} exemplars := []exemplar.Exemplar{ {Labels: labels.FromStrings("t", "1"), Value: 1, Ts: 10000000, HasTs: true}, @@ -3027,21 +2911,22 @@ func TestScrapeLoopAppendExemplarSeries(t *testing.T) { labels: labels.FromStrings(), } - app := &collectResultAppender{} + sl, _, appTest := newTestScrapeLoop(t, func(sl *scrapeLoop) { + sl.sampleMutator = func(l labels.Labels) labels.Labels { + return mutateSampleLabels(l, discoveryLabels, false, nil) + } + sl.reportSampleMutator = func(l labels.Labels) labels.Labels { + return mutateReportSampleLabels(l, discoveryLabels) + } + }) - sl := newBasicScrapeLoop(t, context.Background(), nil, func(context.Context) storage.Appender { return app }, 0) - sl.sampleMutator = func(l labels.Labels) labels.Labels { - return mutateSampleLabels(l, discoveryLabels, false, nil) - } - sl.reportSampleMutator = func(l labels.Labels) labels.Labels { - return mutateReportSampleLabels(l, discoveryLabels) - } + slApp := sl.appender() now := time.Now() for i := range samples { ts := now.Add(time.Second * time.Duration(i)) - samples[i].t = timestamp.FromTime(ts) + samples[i].T = timestamp.FromTime(ts) } // We need to set the timestamp for expected exemplars that does not have a timestamp. @@ -3053,60 +2938,49 @@ func TestScrapeLoopAppendExemplarSeries(t *testing.T) { } for i, st := range scrapeText { - _, _, _, err := sl.append(app, []byte(st), "application/openmetrics-text", timestamp.Time(samples[i].t)) + _, _, _, err := slApp.append([]byte(st), "application/openmetrics-text", timestamp.Time(samples[i].T)) require.NoError(t, err) - require.NoError(t, app.Commit()) + require.NoError(t, slApp.Commit()) } - requireEqual(t, samples, app.resultFloats) - requireEqual(t, exemplars, app.resultExemplars) + requireEqual(t, samples, appTest.ResultSamples) } func TestScrapeLoopRunReportsTargetDownOnScrapeError(t *testing.T) { - var ( - scraper = &testScraper{} - appender = &collectResultAppender{} - app = func(context.Context) storage.Appender { return appender } - ) - - ctx, cancel := context.WithCancel(context.Background()) - sl := newBasicScrapeLoop(t, ctx, scraper, app, 10*time.Millisecond) - + ctx, cancel := context.WithCancel(t.Context()) + sl, scraper, appTest := newTestScrapeLoop(t, func(sl *scrapeLoop) { + sl.ctx = ctx + }) scraper.scrapeFunc = func(context.Context, io.Writer) error { cancel() return errors.New("scrape failed") } sl.run(nil) - require.Equal(t, 0.0, appender.resultFloats[0].f, "bad 'up' value") + require.Equal(t, 0.0, appTest.ResultSamples[0].V, "bad 'up' value") } func TestScrapeLoopRunReportsTargetDownOnInvalidUTF8(t *testing.T) { - var ( - scraper = &testScraper{} - appender = &collectResultAppender{} - app = func(context.Context) storage.Appender { return appender } - ) - - ctx, cancel := context.WithCancel(context.Background()) - sl := newBasicScrapeLoop(t, ctx, scraper, app, 10*time.Millisecond) - + ctx, cancel := context.WithCancel(t.Context()) + sl, scraper, appTest := newTestScrapeLoop(t, func(sl *scrapeLoop) { + sl.ctx = ctx + }) scraper.scrapeFunc = func(_ context.Context, w io.Writer) error { cancel() - w.Write([]byte("a{l=\"\xff\"} 1\n")) + _, _ = w.Write([]byte("a{l=\"\xff\"} 1\n")) return nil } sl.run(nil) - require.Equal(t, 0.0, appender.resultFloats[0].f, "bad 'up' value") + require.Equal(t, 0.0, appTest.ResultSamples[0].V, "bad 'up' value") } type errorAppender struct { - collectResultAppender + storage.Appender } -func (app *errorAppender) Append(ref storage.SeriesRef, lset labels.Labels, t int64, v float64) (storage.SeriesRef, error) { - switch lset.Get(model.MetricNameLabel) { +func (app *errorAppender) Append(ref storage.SeriesRef, ls labels.Labels, t int64, v float64) (storage.SeriesRef, error) { + switch ls.Get(model.MetricNameLabel) { case "out_of_order": return 0, storage.ErrOutOfOrderSample case "amend": @@ -3114,48 +2988,43 @@ func (app *errorAppender) Append(ref storage.SeriesRef, lset labels.Labels, t in case "out_of_bounds": return 0, storage.ErrOutOfBounds default: - return app.collectResultAppender.Append(ref, lset, t, v) + return app.Appender.Append(ref, ls, t, v) } } func TestScrapeLoopAppendGracefullyIfAmendOrOutOfOrderOrOutOfBounds(t *testing.T) { - app := &errorAppender{} - sl := newBasicScrapeLoop(t, context.Background(), nil, func(context.Context) storage.Appender { return app }, 0) + sl, _, appTest := newTestScrapeLoop(t) + // appTest.Prev = &errorAppender{Appender: nopAppender{}} now := time.Unix(1, 0) - slApp := sl.appender(context.Background()) - total, added, seriesAdded, err := sl.append(slApp, []byte("out_of_order 1\namend 1\nnormal 1\nout_of_bounds 1\n"), "text/plain", now) + slApp := sl.appender() + total, added, seriesAdded, err := slApp.append([]byte("out_of_order 1\namend 1\nnormal 1\nout_of_bounds 1\n"), "text/plain", now) require.NoError(t, err) require.NoError(t, slApp.Commit()) - want := []floatSample{ + want := []sample{ { - metric: labels.FromStrings(model.MetricNameLabel, "normal"), - t: timestamp.FromTime(now), - f: 1, + L: labels.FromStrings(model.MetricNameLabel, "normal"), + T: timestamp.FromTime(now), + V: 1, }, } - requireEqual(t, want, app.resultFloats, "Appended samples not as expected:\n%s", appender) + requireEqual(t, want, appTest.ResultSamples, "Appended samples not as expected:\n%s", appTest) require.Equal(t, 4, total) require.Equal(t, 4, added) require.Equal(t, 1, seriesAdded) } func TestScrapeLoopOutOfBoundsTimeError(t *testing.T) { - app := &collectResultAppender{} - sl := newBasicScrapeLoop(t, context.Background(), nil, - func(context.Context) storage.Appender { - return &timeLimitAppender{ - Appender: app, - maxTime: timestamp.FromTime(time.Now().Add(10 * time.Minute)), - } - }, - 0, - ) + sl, _, _ := newTestScrapeLoop(t) + //appTest.Prev = &timeLimitAppender{ + // Appender: nopAppender{}, + // maxTime: timestamp.FromTime(time.Now().Add(10 * time.Minute)), + //} now := time.Now().Add(20 * time.Minute) - slApp := sl.appender(context.Background()) - total, added, seriesAdded, err := sl.append(slApp, []byte("normal 1\n"), "text/plain", now) + slApp := sl.appender() + total, added, seriesAdded, err := slApp.append([]byte("normal 1\n"), "text/plain", now) require.NoError(t, err) require.NoError(t, slApp.Commit()) require.Equal(t, 1, total) @@ -3252,7 +3121,7 @@ func TestRequestTraceparentHeader(t *testing.T) { resp, err := ts.scrape(context.Background()) require.NoError(t, err) require.NotNil(t, resp) - defer resp.Body.Close() + t.Cleanup(func() { _ = resp.Body.Close() }) } func TestTargetScraperScrapeOK(t *testing.T) { @@ -3299,7 +3168,7 @@ func TestTargetScraperScrapeOK(t *testing.T) { } else { w.Header().Set("Content-Type", `text/plain; version=0.0.4`) } - w.Write([]byte("metric_a 1\nmetric_b 2\n")) + _, _ = w.Write([]byte("metric_a 1\nmetric_b 2\n")) }), ) defer server.Close() @@ -3476,11 +3345,11 @@ func TestTargetScraperBodySizeLimit(t *testing.T) { if gzipResponse { w.Header().Set("Content-Encoding", "gzip") gw := gzip.NewWriter(w) - defer gw.Close() - gw.Write([]byte(responseBody)) + defer func() { _ = gw.Close() }() + _, _ = gw.Write([]byte(responseBody)) return } - w.Write([]byte(responseBody)) + _, _ = w.Write([]byte(responseBody)) }), ) defer server.Close() @@ -3574,66 +3443,62 @@ func (ts *testScraper) readResponse(ctx context.Context, _ *http.Response, w io. func TestScrapeLoop_RespectTimestamps(t *testing.T) { s := teststorage.New(t) - defer s.Close() + t.Cleanup(func() { _ = s.Close() }) - app := s.Appender(context.Background()) - capp := &collectResultAppender{next: app} - sl := newBasicScrapeLoop(t, context.Background(), nil, func(context.Context) storage.Appender { return capp }, 0) + sl, _, appTest := newTestScrapeLoop(t) + appTest.Next = s now := time.Now() - slApp := sl.appender(context.Background()) - _, _, _, err := sl.append(slApp, []byte(`metric_a{a="1",b="1"} 1 0`), "text/plain", now) + slApp := sl.appender() + _, _, _, err := slApp.append([]byte(`metric_a{a="1",b="1"} 1 0`), "text/plain", now) require.NoError(t, err) require.NoError(t, slApp.Commit()) - want := []floatSample{ + want := []sample{ { - metric: labels.FromStrings("__name__", "metric_a", "a", "1", "b", "1"), - t: 0, - f: 1, + L: labels.FromStrings("__name__", "metric_a", "a", "1", "b", "1"), + T: 0, + V: 1, }, } - require.Equal(t, want, capp.resultFloats, "Appended samples not as expected:\n%s", appender) + require.Equal(t, want, appTest.ResultSamples, "Appended samples not as expected:\n%s", appTest) } func TestScrapeLoop_DiscardTimestamps(t *testing.T) { s := teststorage.New(t) - defer s.Close() + t.Cleanup(func() { _ = s.Close() }) - app := s.Appender(context.Background()) - - capp := &collectResultAppender{next: app} - - sl := newBasicScrapeLoop(t, context.Background(), nil, func(context.Context) storage.Appender { return capp }, 0) - sl.honorTimestamps = false + sl, _, appTest := newTestScrapeLoop(t, func(sl *scrapeLoop) { + sl.honorTimestamps = false + }) + appTest.Next = s now := time.Now() - slApp := sl.appender(context.Background()) - _, _, _, err := sl.append(slApp, []byte(`metric_a{a="1",b="1"} 1 0`), "text/plain", now) + slApp := sl.appender() + _, _, _, err := slApp.append([]byte(`metric_a{a="1",b="1"} 1 0`), "text/plain", now) require.NoError(t, err) require.NoError(t, slApp.Commit()) - want := []floatSample{ + want := []sample{ { - metric: labels.FromStrings("__name__", "metric_a", "a", "1", "b", "1"), - t: timestamp.FromTime(now), - f: 1, + L: labels.FromStrings("__name__", "metric_a", "a", "1", "b", "1"), + T: timestamp.FromTime(now), + V: 1, }, } - require.Equal(t, want, capp.resultFloats, "Appended samples not as expected:\n%s", appender) + require.Equal(t, want, appTest.ResultSamples, "Appended samples not as expected:\n%s", appTest) } func TestScrapeLoopDiscardDuplicateLabels(t *testing.T) { s := teststorage.New(t) - defer s.Close() + t.Cleanup(func() { _ = s.Close() }) - ctx, cancel := context.WithCancel(context.Background()) - sl := newBasicScrapeLoop(t, ctx, &testScraper{}, s.Appender, 0) - defer cancel() + sl, _, appTest := newTestScrapeLoop(t) + appTest.Next = s // We add a good and a bad metric to check that both are discarded. - slApp := sl.appender(ctx) - _, _, _, err := sl.append(slApp, []byte("test_metric{le=\"500\"} 1\ntest_metric{le=\"600\",le=\"700\"} 1\n"), "text/plain", time.Time{}) + slApp := sl.appender() + _, _, _, err := slApp.append([]byte("test_metric{le=\"500\"} 1\ntest_metric{le=\"600\",le=\"700\"} 1\n"), "text/plain", time.Time{}) require.Error(t, err) require.NoError(t, slApp.Rollback()) // We need to cycle staleness cache maps after a manual rollback. Otherwise they will have old entries in them, @@ -3642,19 +3507,19 @@ func TestScrapeLoopDiscardDuplicateLabels(t *testing.T) { q, err := s.Querier(time.Time{}.UnixNano(), 0) require.NoError(t, err) - series := q.Select(ctx, false, nil, labels.MustNewMatcher(labels.MatchRegexp, "__name__", ".*")) + series := q.Select(sl.ctx, false, nil, labels.MustNewMatcher(labels.MatchRegexp, "__name__", ".*")) require.False(t, series.Next(), "series found in tsdb") require.NoError(t, series.Err()) // We add a good metric to check that it is recorded. - slApp = sl.appender(ctx) - _, _, _, err = sl.append(slApp, []byte("test_metric{le=\"500\"} 1\n"), "text/plain", time.Time{}) + slApp = sl.appender() + _, _, _, err = slApp.append([]byte("test_metric{le=\"500\"} 1\n"), "text/plain", time.Time{}) require.NoError(t, err) require.NoError(t, slApp.Commit()) q, err = s.Querier(time.Time{}.UnixNano(), 0) require.NoError(t, err) - series = q.Select(ctx, false, nil, labels.MustNewMatcher(labels.MatchEqual, "le", "500")) + series = q.Select(sl.ctx, false, nil, labels.MustNewMatcher(labels.MatchEqual, "le", "500")) require.True(t, series.Next(), "series not found in tsdb") require.NoError(t, series.Err()) require.False(t, series.Next(), "more than one series found in tsdb") @@ -3662,29 +3527,27 @@ func TestScrapeLoopDiscardDuplicateLabels(t *testing.T) { func TestScrapeLoopDiscardUnnamedMetrics(t *testing.T) { s := teststorage.New(t) - defer s.Close() + t.Cleanup(func() { _ = s.Close() }) - app := s.Appender(context.Background()) - - ctx, cancel := context.WithCancel(context.Background()) - sl := newBasicScrapeLoop(t, context.Background(), &testScraper{}, func(context.Context) storage.Appender { return app }, 0) - sl.sampleMutator = func(l labels.Labels) labels.Labels { - if l.Has("drop") { - return labels.FromStrings("no", "name") // This label set will trigger an error. + sl, _, appTest := newTestScrapeLoop(t, func(sl *scrapeLoop) { + sl.sampleMutator = func(l labels.Labels) labels.Labels { + if l.Has("drop") { + return labels.FromStrings("no", "name") // This label set will trigger an error. + } + return l } - return l - } - defer cancel() + }) + appTest.Next = s - slApp := sl.appender(context.Background()) - _, _, _, err := sl.append(slApp, []byte("nok 1\nnok2{drop=\"drop\"} 1\n"), "text/plain", time.Time{}) + slApp := sl.appender() + _, _, _, err := slApp.append([]byte("nok 1\nnok2{drop=\"drop\"} 1\n"), "text/plain", time.Time{}) require.Error(t, err) require.NoError(t, slApp.Rollback()) require.Equal(t, errNameLabelMandatory, err) q, err := s.Querier(time.Time{}.UnixNano(), 0) require.NoError(t, err) - series := q.Select(ctx, false, nil, labels.MustNewMatcher(labels.MatchRegexp, "__name__", ".*")) + series := q.Select(sl.ctx, false, nil, labels.MustNewMatcher(labels.MatchRegexp, "__name__", ".*")) require.False(t, series.Next(), "series found in tsdb") require.NoError(t, series.Err()) } @@ -3924,7 +3787,7 @@ func TestReuseScrapeCache(t *testing.T) { for i, s := range steps { initCacheAddr := cacheAddr(sp) - sp.reload(s.newConfig) + require.NoError(t, sp.reload(s.newConfig)) for fp, newCacheAddr := range cacheAddr(sp) { if s.keep { require.Equal(t, initCacheAddr[fp], newCacheAddr, "step %d: old cache and new cache are not the same", i) @@ -3933,7 +3796,7 @@ func TestReuseScrapeCache(t *testing.T) { } } initCacheAddr = cacheAddr(sp) - sp.reload(s.newConfig) + require.NoError(t, sp.reload(s.newConfig)) for fp, newCacheAddr := range cacheAddr(sp) { require.Equal(t, initCacheAddr[fp], newCacheAddr, "step %d: reloading the exact config invalidates the cache", i) } @@ -3942,14 +3805,13 @@ func TestReuseScrapeCache(t *testing.T) { func TestScrapeAddFast(t *testing.T) { s := teststorage.New(t) - defer s.Close() + t.Cleanup(func() { _ = s.Close() }) - ctx, cancel := context.WithCancel(context.Background()) - sl := newBasicScrapeLoop(t, ctx, &testScraper{}, s.Appender, 0) - defer cancel() + sl, _, appTest := newTestScrapeLoop(t) + appTest.Next = s - slApp := sl.appender(ctx) - _, _, _, err := sl.append(slApp, []byte("up 1\n"), "text/plain", time.Time{}) + slApp := sl.appender() + _, _, _, err := slApp.append([]byte("up 1\n"), "text/plain", time.Time{}) require.NoError(t, err) require.NoError(t, slApp.Commit()) @@ -3959,8 +3821,8 @@ func TestScrapeAddFast(t *testing.T) { v.ref++ } - slApp = sl.appender(ctx) - _, _, _, err = sl.append(slApp, []byte("up 1\n"), "text/plain", time.Time{}.Add(time.Second)) + slApp = sl.appender() + _, _, _, err = slApp.append([]byte("up 1\n"), "text/plain", time.Time{}.Add(time.Second)) require.NoError(t, err) require.NoError(t, slApp.Commit()) } @@ -3991,7 +3853,7 @@ func TestReuseCacheRace(t *testing.T) { if time.Since(start) > 5*time.Second { break } - sp.reload(&config.ScrapeConfig{ + require.NoError(t, sp.reload(&config.ScrapeConfig{ JobName: "Prometheus", ScrapeTimeout: model.Duration(1 * time.Millisecond), ScrapeInterval: model.Duration(1 * time.Millisecond), @@ -3999,39 +3861,42 @@ func TestReuseCacheRace(t *testing.T) { SampleLimit: i, MetricNameValidationScheme: model.UTF8Validation, MetricNameEscapingScheme: model.AllowUTF8, - }) + })) } } func TestCheckAddError(t *testing.T) { var appErrs appendErrors sl := scrapeLoop{l: promslog.NewNopLogger(), metrics: newTestScrapeMetrics(t)} - sl.checkAddError(nil, storage.ErrOutOfOrderSample, nil, nil, &appErrs) + // TODO: Check err etc + _, _ = sl.checkAddError(nil, storage.ErrOutOfOrderSample, nil, nil, &appErrs) require.Equal(t, 1, appErrs.numOutOfOrder) + + // TODO(bwplotka): Test partial error check and other cases } func TestScrapeReportSingleAppender(t *testing.T) { t.Parallel() s := teststorage.New(t) - defer s.Close() + t.Cleanup(func() { _ = s.Close() }) - var ( - signal = make(chan struct{}, 1) - scraper = &testScraper{} - ) + signal := make(chan struct{}, 1) - ctx, cancel := context.WithCancel(context.Background()) - // Since we're writing samples directly below we need to provide a protocol fallback. - sl := newBasicScrapeLoopWithFallback(t, ctx, scraper, s.Appender, 10*time.Millisecond, "text/plain") + ctx, cancel := context.WithCancel(t.Context()) + sl, scraper, _ := newTestScrapeLoop(t, func(sl *scrapeLoop) { + sl.ctx = ctx + sl.appendable = s + // Since we're writing samples directly below we need to provide a protocol fallback. + sl.fallbackScrapeProtocol = "text/plain" + }) numScrapes := 0 - scraper.scrapeFunc = func(_ context.Context, w io.Writer) error { numScrapes++ if numScrapes%4 == 0 { return errors.New("scrape failed") } - w.Write([]byte("metric_a 44\nmetric_b 44\nmetric_c 44\nmetric_d 44\n")) + _, _ = w.Write([]byte("metric_a 44\nmetric_b 44\nmetric_c 44\nmetric_d 44\n")) return nil } @@ -4055,7 +3920,7 @@ func TestScrapeReportSingleAppender(t *testing.T) { } require.Equal(t, 0, c%9, "Appended samples not as expected: %d", c) - q.Close() + require.NoError(t, q.Close()) } cancel() @@ -4068,7 +3933,7 @@ func TestScrapeReportSingleAppender(t *testing.T) { func TestScrapeReportLimit(t *testing.T) { s := teststorage.New(t) - defer s.Close() + t.Cleanup(func() { _ = s.Close() }) cfg := &config.ScrapeConfig{ JobName: "test", @@ -4106,7 +3971,7 @@ func TestScrapeReportLimit(t *testing.T) { ctx := t.Context() q, err := s.Querier(time.Time{}.UnixNano(), time.Now().UnixNano()) require.NoError(t, err) - defer q.Close() + t.Cleanup(func() { _ = q.Close() }) series := q.Select(ctx, false, nil, labels.MustNewMatcher(labels.MatchRegexp, "__name__", "up")) var found bool @@ -4124,7 +3989,7 @@ func TestScrapeReportLimit(t *testing.T) { func TestScrapeUTF8(t *testing.T) { s := teststorage.New(t) - defer s.Close() + t.Cleanup(func() { _ = s.Close() }) cfg := &config.ScrapeConfig{ JobName: "test", @@ -4160,7 +4025,7 @@ func TestScrapeUTF8(t *testing.T) { ctx := t.Context() q, err := s.Querier(time.Time{}.UnixNano(), time.Now().UnixNano()) require.NoError(t, err) - defer q.Close() + t.Cleanup(func() { _ = q.Close() }) series := q.Select(ctx, false, nil, labels.MustNewMatcher(labels.MatchRegexp, "__name__", "with.dots")) require.True(t, series.Next(), "series not found in tsdb") @@ -4232,23 +4097,22 @@ func TestScrapeLoopLabelLimit(t *testing.T) { } for _, test := range tests { - app := &collectResultAppender{} - discoveryLabels := &Target{ labels: labels.FromStrings(test.discoveryLabels...), } - sl := newBasicScrapeLoop(t, context.Background(), nil, func(context.Context) storage.Appender { return app }, 0) - sl.sampleMutator = func(l labels.Labels) labels.Labels { - return mutateSampleLabels(l, discoveryLabels, false, nil) - } - sl.reportSampleMutator = func(l labels.Labels) labels.Labels { - return mutateReportSampleLabels(l, discoveryLabels) - } - sl.labelLimits = &test.labelLimits + sl, _, _ := newTestScrapeLoop(t, func(sl *scrapeLoop) { + sl.sampleMutator = func(l labels.Labels) labels.Labels { + return mutateSampleLabels(l, discoveryLabels, false, nil) + } + sl.reportSampleMutator = func(l labels.Labels) labels.Labels { + return mutateReportSampleLabels(l, discoveryLabels) + } + sl.labelLimits = &test.labelLimits + }) - slApp := sl.appender(context.Background()) - _, _, _, err := sl.append(slApp, []byte(test.scrapeLabels), "text/plain", time.Now()) + slApp := sl.appender() + _, _, _, err := slApp.append([]byte(test.scrapeLabels), "text/plain", time.Now()) t.Logf("Test:%s", test.title) if test.expectErr { @@ -4263,7 +4127,7 @@ func TestScrapeLoopLabelLimit(t *testing.T) { func TestTargetScrapeIntervalAndTimeoutRelabel(t *testing.T) { interval, _ := model.ParseDuration("2s") timeout, _ := model.ParseDuration("500ms") - config := &config.ScrapeConfig{ + cfg := &config.ScrapeConfig{ ScrapeInterval: interval, ScrapeTimeout: timeout, MetricNameValidationScheme: model.UTF8Validation, @@ -4287,7 +4151,7 @@ func TestTargetScrapeIntervalAndTimeoutRelabel(t *testing.T) { }, }, } - sp, _ := newScrapePool(config, &nopAppendable{}, 0, nil, nil, &Options{}, newTestScrapeMetrics(t)) + sp, _ := newScrapePool(cfg, &nopAppendable{}, 0, nil, nil, &Options{}, newTestScrapeMetrics(t)) tgts := []*targetgroup.Group{ { Targets: []model.LabelSet{{model.AddressLabel: "127.0.0.1:9090"}}, @@ -4303,10 +4167,10 @@ func TestTargetScrapeIntervalAndTimeoutRelabel(t *testing.T) { // Testing whether we can remove trailing .0 from histogram 'le' and summary 'quantile' labels. func TestLeQuantileReLabel(t *testing.T) { - simpleStorage := teststorage.New(t) - defer simpleStorage.Close() + s := teststorage.New(t) + t.Cleanup(func() { _ = s.Close() }) - config := &config.ScrapeConfig{ + cfg := &config.ScrapeConfig{ JobName: "test", MetricRelabelConfigs: []*relabel.Config{ { @@ -4373,7 +4237,7 @@ test_summary_count 199 ts, scrapedTwice := newScrapableServer(metricsText) defer ts.Close() - sp, err := newScrapePool(config, simpleStorage, 0, nil, nil, &Options{}, newTestScrapeMetrics(t)) + sp, err := newScrapePool(cfg, s, 0, nil, nil, &Options{}, newTestScrapeMetrics(t)) require.NoError(t, err) defer sp.stop() @@ -4393,9 +4257,9 @@ test_summary_count 199 } ctx := t.Context() - q, err := simpleStorage.Querier(time.Time{}.UnixNano(), time.Now().UnixNano()) + q, err := s.Querier(time.Time{}.UnixNano(), time.Now().UnixNano()) require.NoError(t, err) - defer q.Close() + t.Cleanup(func() { _ = q.Close() }) checkValues := func(labelName string, expectedValues []string, series storage.SeriesSet) { foundLeValues := map[string]bool{} @@ -4441,12 +4305,12 @@ func TestConvertClassicHistogramsToNHCB(t *testing.T) { } b := &bytes.Buffer{} if withMetadata { - template.Must(template.New("").Parse(` + require.NoError(t, template.Must(template.New("").Parse(` # HELP {{.name}} This is a histogram with default buckets # TYPE {{.name}} histogram -`)).Execute(b, data) +`)).Execute(b, data)) } - template.Must(template.New("").Parse(` + require.NoError(t, template.Must(template.New("").Parse(` {{.name}}_bucket{address="0.0.0.0",port="5001",le="0.005"} 0 {{.name}}_bucket{address="0.0.0.0",port="5001",le="0.01"} 0 {{.name}}_bucket{address="0.0.0.0",port="5001",le="0.025"} 0 @@ -4461,7 +4325,7 @@ func TestConvertClassicHistogramsToNHCB(t *testing.T) { {{.name}}_bucket{address="0.0.0.0",port="5001",le="+Inf"} 1 {{.name}}_sum{address="0.0.0.0",port="5001"} 10 {{.name}}_count{address="0.0.0.0",port="5001"} 1 -`)).Execute(b, data) +`)).Execute(b, data)) return b.String() } genTestCounterProto := func(name string, value int) string { @@ -4831,14 +4695,17 @@ metric: < t.Run(fmt.Sprintf("%s with %s", name, metricsTextName), func(t *testing.T) { t.Parallel() - simpleStorage := teststorage.New(t) - defer simpleStorage.Close() + s := teststorage.New(t) + t.Cleanup(func() { _ = s.Close() }) - sl := newBasicScrapeLoop(t, context.Background(), nil, func(ctx context.Context) storage.Appender { return simpleStorage.Appender(ctx) }, 0) - sl.alwaysScrapeClassicHist = tc.alwaysScrapeClassicHistograms - sl.convertClassicHistToNHCB = tc.convertClassicHistToNHCB - sl.enableNativeHistogramScraping = true - app := simpleStorage.Appender(context.Background()) + sl, _, _ := newTestScrapeLoop(t, func(sl *scrapeLoop) { + sl.appendable = s + sl.alwaysScrapeClassicHist = tc.alwaysScrapeClassicHistograms + sl.convertClassicHistToNHCB = tc.convertClassicHistToNHCB + sl.enableNativeHistogramScraping = true + }) + + slApp := sl.appender() var content []byte contentType := metricsText.contentType @@ -4862,13 +4729,14 @@ metric: < default: t.Error("unexpected content type") } - sl.append(app, content, contentType, time.Now()) - require.NoError(t, app.Commit()) + _, _, _, err := slApp.append(content, contentType, time.Now()) + require.NoError(t, err) + require.NoError(t, slApp.Commit()) ctx := t.Context() - q, err := simpleStorage.Querier(time.Time{}.UnixNano(), time.Now().UnixNano()) + q, err := s.Querier(time.Time{}.UnixNano(), time.Now().UnixNano()) require.NoError(t, err) - defer q.Close() + t.Cleanup(func() { _ = q.Close() }) var series storage.SeriesSet @@ -4910,10 +4778,10 @@ metric: < } func TestTypeUnitReLabel(t *testing.T) { - simpleStorage := teststorage.New(t) - defer simpleStorage.Close() + s := teststorage.New(t) + t.Cleanup(func() { _ = s.Close() }) - config := &config.ScrapeConfig{ + cfg := &config.ScrapeConfig{ JobName: "test", MetricRelabelConfigs: []*relabel.Config{ { @@ -4958,7 +4826,7 @@ disk_usage_bytes 456 ts, scrapedTwice := newScrapableServer(metricsText) defer ts.Close() - sp, err := newScrapePool(config, simpleStorage, 0, nil, nil, &Options{}, newTestScrapeMetrics(t)) + sp, err := newScrapePool(cfg, s, 0, nil, nil, &Options{}, newTestScrapeMetrics(t)) require.NoError(t, err) defer sp.stop() @@ -4978,9 +4846,9 @@ disk_usage_bytes 456 } ctx := t.Context() - q, err := simpleStorage.Querier(time.Time{}.UnixNano(), time.Now().UnixNano()) + q, err := s.Querier(time.Time{}.UnixNano(), time.Now().UnixNano()) require.NoError(t, err) - defer q.Close() + t.Cleanup(func() { _ = q.Close() }) series := q.Select(ctx, false, nil, labels.MustNewMatcher(labels.MatchRegexp, "__name__", ".*_total$")) for series.Next() { @@ -4996,26 +4864,24 @@ disk_usage_bytes 456 } func TestScrapeLoopRunCreatesStaleMarkersOnFailedScrapeForTimestampedMetrics(t *testing.T) { - appender := &collectResultAppender{} - var ( - signal = make(chan struct{}, 1) - scraper = &testScraper{} - app = func(context.Context) storage.Appender { return appender } - ) + signal := make(chan struct{}, 1) + + ctx, cancel := context.WithCancel(t.Context()) + sl, scraper, appTest := newTestScrapeLoop(t, func(sl *scrapeLoop) { + sl.ctx = ctx + // Since we're writing samples directly below we need to provide a protocol fallback. + sl.fallbackScrapeProtocol = "text/plain" + sl.trackTimestampsStaleness = true + }) - ctx, cancel := context.WithCancel(context.Background()) - // Since we're writing samples directly below we need to provide a protocol fallback. - sl := newBasicScrapeLoopWithFallback(t, ctx, scraper, app, 10*time.Millisecond, "text/plain") - sl.trackTimestampsStaleness = true // Succeed once, several failures, then stop. numScrapes := 0 - scraper.scrapeFunc = func(_ context.Context, w io.Writer) error { numScrapes++ switch numScrapes { case 1: - fmt.Fprintf(w, "metric_a 42 %d\n", time.Now().UnixNano()/int64(time.Millisecond)) + _, _ = fmt.Fprintf(w, "metric_a 42 %d\n", time.Now().UnixNano()/int64(time.Millisecond)) return nil case 5: cancel() @@ -5035,15 +4901,15 @@ func TestScrapeLoopRunCreatesStaleMarkersOnFailedScrapeForTimestampedMetrics(t * } // 1 successfully scraped sample, 1 stale marker after first fail, 5 report samples for // each scrape successful or not. - require.Len(t, appender.resultFloats, 27, "Appended samples not as expected:\n%s", appender) - require.Equal(t, 42.0, appender.resultFloats[0].f, "Appended first sample not as expected") - require.True(t, value.IsStaleNaN(appender.resultFloats[6].f), - "Appended second sample not as expected. Wanted: stale NaN Got: %x", math.Float64bits(appender.resultFloats[6].f)) + require.Len(t, appTest.ResultSamples, 27, "Appended samples not as expected:\n%s", appTest) + require.Equal(t, 42.0, appTest.ResultSamples[0].V, "Appended first sample not as expected") + require.True(t, value.IsStaleNaN(appTest.ResultSamples[6].V), + "Appended second sample not as expected. Wanted: stale NaN Got: %x", math.Float64bits(appTest.ResultSamples[6].V)) } func TestScrapeLoopCompression(t *testing.T) { - simpleStorage := teststorage.New(t) - defer simpleStorage.Close() + s := teststorage.New(t) + t.Cleanup(func() { _ = s.Close() }) metricsText := makeTestGauges(10) @@ -5065,12 +4931,12 @@ func TestScrapeLoopCompression(t *testing.T) { ts := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { require.Equal(t, tc.acceptEncoding, r.Header.Get("Accept-Encoding"), "invalid value of the Accept-Encoding header") - fmt.Fprint(w, string(metricsText)) + _, _ = fmt.Fprint(w, string(metricsText)) close(scraped) })) defer ts.Close() - config := &config.ScrapeConfig{ + cfg := &config.ScrapeConfig{ JobName: "test", SampleLimit: 100, Scheme: "http", @@ -5081,7 +4947,7 @@ func TestScrapeLoopCompression(t *testing.T) { MetricNameEscapingScheme: model.AllowUTF8, } - sp, err := newScrapePool(config, simpleStorage, 0, nil, nil, &Options{}, newTestScrapeMetrics(t)) + sp, err := newScrapePool(cfg, s, 0, nil, nil, &Options{}, newTestScrapeMetrics(t)) require.NoError(t, err) defer sp.stop() @@ -5191,11 +5057,11 @@ func BenchmarkTargetScraperGzip(b *testing.B) { gw := gzip.NewWriter(&buf) for j := 0; j < scenarios[i].metricsCount; j++ { name = fmt.Sprintf("go_memstats_alloc_bytes_total_%d", j) - fmt.Fprintf(gw, "# HELP %s Total number of bytes allocated, even if freed.\n", name) - fmt.Fprintf(gw, "# TYPE %s counter\n", name) - fmt.Fprintf(gw, "%s %d\n", name, i*j) + _, _ = fmt.Fprintf(gw, "# HELP %s Total number of bytes allocated, even if freed.\n", name) + _, _ = fmt.Fprintf(gw, "# TYPE %s counter\n", name) + _, _ = fmt.Fprintf(gw, "%s %d\n", name, i*j) } - gw.Close() + require.NoError(b, gw.Close()) scenarios[i].body = buf.Bytes() } @@ -5204,7 +5070,7 @@ func BenchmarkTargetScraperGzip(b *testing.B) { w.Header().Set("Content-Encoding", "gzip") for _, scenario := range scenarios { if strconv.Itoa(scenario.metricsCount) == r.URL.Query()["count"][0] { - w.Write(scenario.body) + _, _ = w.Write(scenario.body) return } } @@ -5253,10 +5119,10 @@ func BenchmarkTargetScraperGzip(b *testing.B) { // When a scrape contains multiple instances for the same time series we should increment // prometheus_target_scrapes_sample_duplicate_timestamp_total metric. func TestScrapeLoopSeriesAddedDuplicates(t *testing.T) { - ctx, sl := simpleTestScrapeLoop(t) + sl, _, _ := newTestScrapeLoop(t) - slApp := sl.appender(ctx) - total, added, seriesAdded, err := sl.append(slApp, []byte("test_metric 1\ntest_metric 2\ntest_metric 3\n"), "text/plain", time.Time{}) + slApp := sl.appender() + total, added, seriesAdded, err := slApp.append([]byte("test_metric 1\ntest_metric 2\ntest_metric 3\n"), "text/plain", time.Time{}) require.NoError(t, err) require.NoError(t, slApp.Commit()) require.Equal(t, 3, total) @@ -5264,8 +5130,8 @@ func TestScrapeLoopSeriesAddedDuplicates(t *testing.T) { require.Equal(t, 1, seriesAdded) require.Equal(t, 2.0, prom_testutil.ToFloat64(sl.metrics.targetScrapeSampleDuplicate)) - slApp = sl.appender(ctx) - total, added, seriesAdded, err = sl.append(slApp, []byte("test_metric 1\ntest_metric 1\ntest_metric 1\n"), "text/plain", time.Time{}) + slApp = sl.appender() + total, added, seriesAdded, err = slApp.append([]byte("test_metric 1\ntest_metric 1\ntest_metric 1\n"), "text/plain", time.Time{}) require.NoError(t, err) require.NoError(t, slApp.Commit()) require.Equal(t, 3, total) @@ -5274,8 +5140,8 @@ func TestScrapeLoopSeriesAddedDuplicates(t *testing.T) { require.Equal(t, 4.0, prom_testutil.ToFloat64(sl.metrics.targetScrapeSampleDuplicate)) // When different timestamps are supplied, multiple samples are accepted. - slApp = sl.appender(ctx) - total, added, seriesAdded, err = sl.append(slApp, []byte("test_metric 1 1001\ntest_metric 1 1002\ntest_metric 1 1003\n"), "text/plain", time.Time{}) + slApp = sl.appender() + total, added, seriesAdded, err = slApp.append([]byte("test_metric 1 1001\ntest_metric 1 1002\ntest_metric 1 1003\n"), "text/plain", time.Time{}) require.NoError(t, err) require.NoError(t, slApp.Commit()) require.Equal(t, 3, total) @@ -5325,7 +5191,7 @@ func testNativeHistogramMaxSchemaSet(t *testing.T, minBucketFactor string, expec }, ) registry := prometheus.NewRegistry() - registry.Register(nativeHistogram) + require.NoError(t, registry.Register(nativeHistogram)) nativeHistogram.Observe(1.0) nativeHistogram.Observe(1.0) nativeHistogram.Observe(1.0) @@ -5342,7 +5208,7 @@ func testNativeHistogramMaxSchemaSet(t *testing.T, minBucketFactor string, expec // Create a HTTP server to serve /metrics via ProtoBuf metricsServer := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, _ *http.Request) { w.Header().Set("Content-Type", `application/vnd.google.protobuf; proto=io.prometheus.client.MetricFamily; encoding=delimited`) - w.Write(buffer) + _, _ = w.Write(buffer) })) defer metricsServer.Close() @@ -5361,18 +5227,17 @@ scrape_configs: `, minBucketFactor, strings.ReplaceAll(metricsServer.URL, "http://", "")) s := teststorage.New(t) - defer s.Close() + t.Cleanup(func() { _ = s.Close() }) reg := prometheus.NewRegistry() mng, err := NewManager(&Options{DiscoveryReloadInterval: model.Duration(10 * time.Millisecond)}, nil, nil, s, reg) require.NoError(t, err) cfg, err := config.Load(configStr, promslog.NewNopLogger()) require.NoError(t, err) - mng.ApplyConfig(cfg) + require.NoError(t, mng.ApplyConfig(cfg)) tsets := make(chan map[string][]*targetgroup.Group) go func() { - err = mng.Run(tsets) - require.NoError(t, err) + require.NoError(t, mng.Run(tsets)) }() defer mng.Stop() @@ -5447,7 +5312,7 @@ func TestTargetScrapeConfigWithLabels(t *testing.T) { require.Equal(t, expectedPath, r.URL.Path) w.Header().Set("Content-Type", `text/plain; version=0.0.4`) - w.Write([]byte("metric_a 1\nmetric_b 2\n")) + _, _ = w.Write([]byte("metric_a 1\nmetric_b 2\n")) }), ) t.Cleanup(server.Close) @@ -5595,7 +5460,7 @@ func newScrapableServer(scrapeText string) (s *httptest.Server, scrapedTwice cha scrapedTwice = make(chan bool) return httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, _ *http.Request) { - fmt.Fprint(w, scrapeText) + _, _ = fmt.Fprint(w, scrapeText) scrapes++ if scrapes == 2 { close(scrapedTwice) @@ -5607,7 +5472,7 @@ func newScrapableServer(scrapeText string) (s *httptest.Server, scrapedTwice cha func TestScrapePoolScrapeAfterReload(t *testing.T) { h := httptest.NewServer(http.HandlerFunc( func(w http.ResponseWriter, _ *http.Request) { - w.Write([]byte{0x42, 0x42}) + _, _ = w.Write([]byte{0x42, 0x42}) }, )) t.Cleanup(h.Close) @@ -5657,46 +5522,42 @@ func TestScrapeAppendWithParseError(t *testing.T) { # EOF` ) - sl := newBasicScrapeLoop(t, context.Background(), nil, nil, 0) - sl.cache = newScrapeCache(sl.metrics) + sl, _, appTest := newTestScrapeLoop(t) + slApp := sl.appender() now := time.Now() - capp := &collectResultAppender{next: nopAppender{}} - _, _, _, err := sl.append(capp, []byte(scrape1), "application/openmetrics-text", now) + _, _, _, err := slApp.append([]byte(scrape1), "application/openmetrics-text", now) require.Error(t, err) - _, _, _, err = sl.append(capp, nil, "application/openmetrics-text", now) + _, _, _, err = slApp.append(nil, "application/openmetrics-text", now) require.NoError(t, err) - require.Empty(t, capp.resultFloats) + require.Empty(t, appTest.ResultSamples) - capp = &collectResultAppender{next: nopAppender{}} - _, _, _, err = sl.append(capp, []byte(scrape2), "application/openmetrics-text", now.Add(15*time.Second)) + _, _, _, err = slApp.append([]byte(scrape2), "application/openmetrics-text", now.Add(15*time.Second)) require.NoError(t, err) - require.NoError(t, capp.Commit()) + require.NoError(t, slApp.Commit()) - want := []floatSample{ + want := []sample{ { - metric: labels.FromStrings(model.MetricNameLabel, "metric_a"), - t: timestamp.FromTime(now.Add(15 * time.Second)), - f: 11, + L: labels.FromStrings(model.MetricNameLabel, "metric_a"), + T: timestamp.FromTime(now.Add(15 * time.Second)), + V: 11, }, } - requireEqual(t, want, capp.resultFloats, "Appended samples not as expected:\n%s", capp) + requireEqual(t, want, appTest.ResultSamples, "Appended samples not as expected:\n%s", appTest) } // This test covers a case where there's a target with sample_limit set and the some of exporter samples // changes between scrapes. func TestScrapeLoopAppendSampleLimitWithDisappearingSeries(t *testing.T) { const sampleLimit = 4 - resApp := &collectResultAppender{} - sl := newBasicScrapeLoop(t, context.Background(), nil, func(_ context.Context) storage.Appender { - return resApp - }, 0) - sl.sampleLimit = sampleLimit + + sl, _, appTest := newTestScrapeLoop(t, func(sl *scrapeLoop) { + sl.sampleLimit = sampleLimit + }) now := time.Now() - slApp := sl.appender(context.Background()) - samplesScraped, samplesAfterRelabel, createdSeries, err := sl.append( - slApp, + slApp := sl.appender() + samplesScraped, samplesAfterRelabel, createdSeries, err := slApp.append( // Start with 3 samples, all accepted. []byte("metric_a 1\nmetric_b 1\nmetric_c 1\n"), "text/plain", @@ -5707,29 +5568,28 @@ func TestScrapeLoopAppendSampleLimitWithDisappearingSeries(t *testing.T) { require.Equal(t, 3, samplesScraped) // All on scrape. require.Equal(t, 3, samplesAfterRelabel) // This is series after relabeling. require.Equal(t, 3, createdSeries) // Newly added to TSDB. - want := []floatSample{ + want := []sample{ { - metric: labels.FromStrings(model.MetricNameLabel, "metric_a"), - t: timestamp.FromTime(now), - f: 1, + L: labels.FromStrings(model.MetricNameLabel, "metric_a"), + T: timestamp.FromTime(now), + V: 1, }, { - metric: labels.FromStrings(model.MetricNameLabel, "metric_b"), - t: timestamp.FromTime(now), - f: 1, + L: labels.FromStrings(model.MetricNameLabel, "metric_b"), + T: timestamp.FromTime(now), + V: 1, }, { - metric: labels.FromStrings(model.MetricNameLabel, "metric_c"), - t: timestamp.FromTime(now), - f: 1, + L: labels.FromStrings(model.MetricNameLabel, "metric_c"), + T: timestamp.FromTime(now), + V: 1, }, } - requireEqual(t, want, resApp.resultFloats, "Appended samples not as expected:\n%s", slApp) + requireEqual(t, want, appTest.ResultSamples, "Appended samples not as expected:\n%s", slApp) now = now.Add(time.Minute) - slApp = sl.appender(context.Background()) - samplesScraped, samplesAfterRelabel, createdSeries, err = sl.append( - slApp, + slApp = sl.appender() + samplesScraped, samplesAfterRelabel, createdSeries, err = slApp.append( // Start exporting 3 more samples, so we're over the limit now. []byte("metric_a 1\nmetric_b 1\nmetric_c 1\nmetric_d 1\nmetric_e 1\nmetric_f 1\n"), "text/plain", @@ -5740,13 +5600,12 @@ func TestScrapeLoopAppendSampleLimitWithDisappearingSeries(t *testing.T) { require.Equal(t, 6, samplesScraped) require.Equal(t, 6, samplesAfterRelabel) require.Equal(t, 1, createdSeries) // We've added one series before hitting the limit. - requireEqual(t, want, resApp.resultFloats, "Appended samples not as expected:\n%s", slApp) + requireEqual(t, want, appTest.ResultSamples, "Appended samples not as expected:\n%s", slApp) sl.cache.iterDone(false) now = now.Add(time.Minute) - slApp = sl.appender(context.Background()) - samplesScraped, samplesAfterRelabel, createdSeries, err = sl.append( - slApp, + slApp = sl.appender() + samplesScraped, samplesAfterRelabel, createdSeries, err = slApp.append( // Remove all samples except first 2. []byte("metric_a 1\nmetric_b 1\n"), "text/plain", @@ -5763,45 +5622,43 @@ func TestScrapeLoopAppendSampleLimitWithDisappearingSeries(t *testing.T) { // - Append with stale marker for metric_d - this series was added during second scrape before we hit the sample_limit. // We should NOT see: // - Appends with stale markers for metric_e & metric_f - both over the limit during second scrape and so they never made it into TSDB. - want = append(want, []floatSample{ + want = append(want, []sample{ { - metric: labels.FromStrings(model.MetricNameLabel, "metric_a"), - t: timestamp.FromTime(now), - f: 1, + L: labels.FromStrings(model.MetricNameLabel, "metric_a"), + T: timestamp.FromTime(now), + V: 1, }, { - metric: labels.FromStrings(model.MetricNameLabel, "metric_b"), - t: timestamp.FromTime(now), - f: 1, + L: labels.FromStrings(model.MetricNameLabel, "metric_b"), + T: timestamp.FromTime(now), + V: 1, }, { - metric: labels.FromStrings(model.MetricNameLabel, "metric_c"), - t: timestamp.FromTime(now), - f: math.Float64frombits(value.StaleNaN), + L: labels.FromStrings(model.MetricNameLabel, "metric_c"), + T: timestamp.FromTime(now), + V: math.Float64frombits(value.StaleNaN), }, { - metric: labels.FromStrings(model.MetricNameLabel, "metric_d"), - t: timestamp.FromTime(now), - f: math.Float64frombits(value.StaleNaN), + L: labels.FromStrings(model.MetricNameLabel, "metric_d"), + T: timestamp.FromTime(now), + V: math.Float64frombits(value.StaleNaN), }, }...) - requireEqual(t, want, resApp.resultFloats, "Appended samples not as expected:\n%s", slApp) + requireEqual(t, want, appTest.ResultSamples, "Appended samples not as expected:\n%s", slApp) } // This test covers a case where there's a target with sample_limit set and each scrape sees a completely // different set of samples. func TestScrapeLoopAppendSampleLimitReplaceAllSamples(t *testing.T) { const sampleLimit = 4 - resApp := &collectResultAppender{} - sl := newBasicScrapeLoop(t, context.Background(), nil, func(_ context.Context) storage.Appender { - return resApp - }, 0) - sl.sampleLimit = sampleLimit + + sl, _, appTest := newTestScrapeLoop(t, func(sl *scrapeLoop) { + sl.sampleLimit = sampleLimit + }) now := time.Now() - slApp := sl.appender(context.Background()) - samplesScraped, samplesAfterRelabel, createdSeries, err := sl.append( - slApp, + slApp := sl.appender() + samplesScraped, samplesAfterRelabel, createdSeries, err := slApp.append( // Start with 4 samples, all accepted. []byte("metric_a 1\nmetric_b 1\nmetric_c 1\nmetric_d 1\n"), "text/plain", @@ -5812,34 +5669,33 @@ func TestScrapeLoopAppendSampleLimitReplaceAllSamples(t *testing.T) { require.Equal(t, 4, samplesScraped) // All on scrape. require.Equal(t, 4, samplesAfterRelabel) // This is series after relabeling. require.Equal(t, 4, createdSeries) // Newly added to TSDB. - want := []floatSample{ + want := []sample{ { - metric: labels.FromStrings(model.MetricNameLabel, "metric_a"), - t: timestamp.FromTime(now), - f: 1, + L: labels.FromStrings(model.MetricNameLabel, "metric_a"), + T: timestamp.FromTime(now), + V: 1, }, { - metric: labels.FromStrings(model.MetricNameLabel, "metric_b"), - t: timestamp.FromTime(now), - f: 1, + L: labels.FromStrings(model.MetricNameLabel, "metric_b"), + T: timestamp.FromTime(now), + V: 1, }, { - metric: labels.FromStrings(model.MetricNameLabel, "metric_c"), - t: timestamp.FromTime(now), - f: 1, + L: labels.FromStrings(model.MetricNameLabel, "metric_c"), + T: timestamp.FromTime(now), + V: 1, }, { - metric: labels.FromStrings(model.MetricNameLabel, "metric_d"), - t: timestamp.FromTime(now), - f: 1, + L: labels.FromStrings(model.MetricNameLabel, "metric_d"), + T: timestamp.FromTime(now), + V: 1, }, } - requireEqual(t, want, resApp.resultFloats, "Appended samples not as expected:\n%s", slApp) + requireEqual(t, want, appTest.ResultSamples, "Appended samples not as expected:\n%s", slApp) now = now.Add(time.Minute) - slApp = sl.appender(context.Background()) - samplesScraped, samplesAfterRelabel, createdSeries, err = sl.append( - slApp, + slApp = sl.appender() + samplesScraped, samplesAfterRelabel, createdSeries, err = slApp.append( // Replace all samples with new time series. []byte("metric_e 1\nmetric_f 1\nmetric_g 1\nmetric_h 1\n"), "text/plain", @@ -5854,60 +5710,55 @@ func TestScrapeLoopAppendSampleLimitReplaceAllSamples(t *testing.T) { // We expect to see: // - 4 appends for new samples. // - 4 appends with staleness markers for old samples. - want = append(want, []floatSample{ + want = append(want, []sample{ { - metric: labels.FromStrings(model.MetricNameLabel, "metric_e"), - t: timestamp.FromTime(now), - f: 1, + L: labels.FromStrings(model.MetricNameLabel, "metric_e"), + T: timestamp.FromTime(now), + V: 1, }, { - metric: labels.FromStrings(model.MetricNameLabel, "metric_f"), - t: timestamp.FromTime(now), - f: 1, + L: labels.FromStrings(model.MetricNameLabel, "metric_f"), + T: timestamp.FromTime(now), + V: 1, }, { - metric: labels.FromStrings(model.MetricNameLabel, "metric_g"), - t: timestamp.FromTime(now), - f: 1, + L: labels.FromStrings(model.MetricNameLabel, "metric_g"), + T: timestamp.FromTime(now), + V: 1, }, { - metric: labels.FromStrings(model.MetricNameLabel, "metric_h"), - t: timestamp.FromTime(now), - f: 1, + L: labels.FromStrings(model.MetricNameLabel, "metric_h"), + T: timestamp.FromTime(now), + V: 1, }, { - metric: labels.FromStrings(model.MetricNameLabel, "metric_a"), - t: timestamp.FromTime(now), - f: math.Float64frombits(value.StaleNaN), + L: labels.FromStrings(model.MetricNameLabel, "metric_a"), + T: timestamp.FromTime(now), + V: math.Float64frombits(value.StaleNaN), }, { - metric: labels.FromStrings(model.MetricNameLabel, "metric_b"), - t: timestamp.FromTime(now), - f: math.Float64frombits(value.StaleNaN), + L: labels.FromStrings(model.MetricNameLabel, "metric_b"), + T: timestamp.FromTime(now), + V: math.Float64frombits(value.StaleNaN), }, { - metric: labels.FromStrings(model.MetricNameLabel, "metric_c"), - t: timestamp.FromTime(now), - f: math.Float64frombits(value.StaleNaN), + L: labels.FromStrings(model.MetricNameLabel, "metric_c"), + T: timestamp.FromTime(now), + V: math.Float64frombits(value.StaleNaN), }, { - metric: labels.FromStrings(model.MetricNameLabel, "metric_d"), - t: timestamp.FromTime(now), - f: math.Float64frombits(value.StaleNaN), + L: labels.FromStrings(model.MetricNameLabel, "metric_d"), + T: timestamp.FromTime(now), + V: math.Float64frombits(value.StaleNaN), }, }...) - requireEqual(t, want, resApp.resultFloats, "Appended samples not as expected:\n%s", slApp) + requireEqual(t, want, appTest.ResultSamples, "Appended samples not as expected:\n%s", slApp) } func TestScrapeLoopDisableStalenessMarkerInjection(t *testing.T) { - var ( - loopDone = atomic.NewBool(false) - appender = &collectResultAppender{} - scraper = &testScraper{} - app = func(_ context.Context) storage.Appender { return appender } - ) + loopDone := atomic.NewBool(false) - sl := newBasicScrapeLoop(t, context.Background(), scraper, app, 10*time.Millisecond) + sl, scraper, appTest := newTestScrapeLoop(t) scraper.scrapeFunc = func(ctx context.Context, w io.Writer) error { if _, err := w.Write([]byte("metric_a 42\n")); err != nil { return err @@ -5923,9 +5774,7 @@ func TestScrapeLoopDisableStalenessMarkerInjection(t *testing.T) { // Wait for some samples to be appended. require.Eventually(t, func() bool { - appender.mtx.Lock() - defer appender.mtx.Unlock() - return len(appender.resultFloats) > 2 + return appTest.ResultSamplesGreaterThan(2) }, 5*time.Second, 100*time.Millisecond, "Scrape loop didn't append any samples.") // Disable end of run staleness markers and stop the loop. @@ -5936,9 +5785,9 @@ func TestScrapeLoopDisableStalenessMarkerInjection(t *testing.T) { }, 5*time.Second, 100*time.Millisecond, "Scrape loop didn't stop.") // No stale markers should be appended, since they were disabled. - for _, s := range appender.resultFloats { - if value.IsStaleNaN(s.f) { - t.Fatalf("Got stale NaN samples while end of run staleness is disabled: %x", math.Float64bits(s.f)) + for _, s := range appTest.ResultSamples { + if value.IsStaleNaN(s.V) { + t.Fatalf("Got stale NaN samples while end of run staleness is disabled: %x", math.Float64bits(s.V)) } } } diff --git a/scrape/target.go b/scrape/target.go index 2aabff20e2..63ff7a2a43 100644 --- a/scrape/target.go +++ b/scrape/target.go @@ -1,4 +1,4 @@ -// Copyright 2013 The Prometheus Authors +// Copyright The Prometheus Authors // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. // You may obtain a copy of the License at @@ -26,11 +26,8 @@ import ( "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/discovery/targetgroup" - "github.com/prometheus/prometheus/model/histogram" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/model/relabel" - "github.com/prometheus/prometheus/model/value" - "github.com/prometheus/prometheus/storage" ) // TargetHealth describes the health state of a target. @@ -323,137 +320,6 @@ var ( errBucketLimit = errors.New("histogram bucket limit exceeded") ) -// limitAppender limits the number of total appended samples in a batch. -type limitAppender struct { - storage.Appender - - limit int - i int -} - -func (app *limitAppender) Append(ref storage.SeriesRef, lset labels.Labels, t int64, v float64) (storage.SeriesRef, error) { - // Bypass sample_limit checks only if we have a staleness marker for a known series (ref value is non-zero). - // This ensures that if a series is already in TSDB then we always write the marker. - if ref == 0 || !value.IsStaleNaN(v) { - app.i++ - if app.i > app.limit { - return 0, errSampleLimit - } - } - ref, err := app.Appender.Append(ref, lset, t, v) - if err != nil { - return 0, err - } - return ref, nil -} - -func (app *limitAppender) AppendHistogram(ref storage.SeriesRef, lset labels.Labels, t int64, h *histogram.Histogram, fh *histogram.FloatHistogram) (storage.SeriesRef, error) { - // Bypass sample_limit checks only if we have a staleness marker for a known series (ref value is non-zero). - // This ensures that if a series is already in TSDB then we always write the marker. - if ref == 0 || (h != nil && !value.IsStaleNaN(h.Sum)) || (fh != nil && !value.IsStaleNaN(fh.Sum)) { - app.i++ - if app.i > app.limit { - return 0, errSampleLimit - } - } - ref, err := app.Appender.AppendHistogram(ref, lset, t, h, fh) - if err != nil { - return 0, err - } - return ref, nil -} - -type timeLimitAppender struct { - storage.Appender - - maxTime int64 -} - -func (app *timeLimitAppender) Append(ref storage.SeriesRef, lset labels.Labels, t int64, v float64) (storage.SeriesRef, error) { - if t > app.maxTime { - return 0, storage.ErrOutOfBounds - } - - ref, err := app.Appender.Append(ref, lset, t, v) - if err != nil { - return 0, err - } - return ref, nil -} - -// bucketLimitAppender limits the number of total appended samples in a batch. -type bucketLimitAppender struct { - storage.Appender - - limit int -} - -func (app *bucketLimitAppender) AppendHistogram(ref storage.SeriesRef, lset labels.Labels, t int64, h *histogram.Histogram, fh *histogram.FloatHistogram) (storage.SeriesRef, error) { - var err error - if h != nil { - // Return with an early error if the histogram has too many buckets and the - // schema is not exponential, in which case we can't reduce the resolution. - if len(h.PositiveBuckets)+len(h.NegativeBuckets) > app.limit && !histogram.IsExponentialSchema(h.Schema) { - return 0, errBucketLimit - } - for len(h.PositiveBuckets)+len(h.NegativeBuckets) > app.limit { - if h.Schema <= histogram.ExponentialSchemaMin { - return 0, errBucketLimit - } - if err = h.ReduceResolution(h.Schema - 1); err != nil { - return 0, err - } - } - } - if fh != nil { - // Return with an early error if the histogram has too many buckets and the - // schema is not exponential, in which case we can't reduce the resolution. - if len(fh.PositiveBuckets)+len(fh.NegativeBuckets) > app.limit && !histogram.IsExponentialSchema(fh.Schema) { - return 0, errBucketLimit - } - for len(fh.PositiveBuckets)+len(fh.NegativeBuckets) > app.limit { - if fh.Schema <= histogram.ExponentialSchemaMin { - return 0, errBucketLimit - } - if err = fh.ReduceResolution(fh.Schema - 1); err != nil { - return 0, err - } - } - } - if ref, err = app.Appender.AppendHistogram(ref, lset, t, h, fh); err != nil { - return 0, err - } - return ref, nil -} - -type maxSchemaAppender struct { - storage.Appender - - maxSchema int32 -} - -func (app *maxSchemaAppender) AppendHistogram(ref storage.SeriesRef, lset labels.Labels, t int64, h *histogram.Histogram, fh *histogram.FloatHistogram) (storage.SeriesRef, error) { - var err error - if h != nil { - if histogram.IsExponentialSchemaReserved(h.Schema) && h.Schema > app.maxSchema { - if err = h.ReduceResolution(app.maxSchema); err != nil { - return 0, err - } - } - } - if fh != nil { - if histogram.IsExponentialSchemaReserved(fh.Schema) && fh.Schema > app.maxSchema { - if err = fh.ReduceResolution(app.maxSchema); err != nil { - return 0, err - } - } - } - if ref, err = app.Appender.AppendHistogram(ref, lset, t, h, fh); err != nil { - return 0, err - } - return ref, nil -} - // PopulateDiscoveredLabels sets base labels on lb from target and group labels and scrape configuration, before relabeling. func PopulateDiscoveredLabels(lb *labels.Builder, cfg *config.ScrapeConfig, tLabels, tgLabels model.LabelSet) { lb.Reset(labels.EmptyLabels()) diff --git a/scrape/target_test.go b/scrape/target_test.go index 582b198c79..63cf97bff5 100644 --- a/scrape/target_test.go +++ b/scrape/target_test.go @@ -1,4 +1,4 @@ -// Copyright 2013 The Prometheus Authors +// Copyright The Prometheus Authors // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. // You may obtain a copy of the License at @@ -14,7 +14,6 @@ package scrape import ( - "context" "crypto/tls" "crypto/x509" "fmt" @@ -33,10 +32,7 @@ import ( "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/discovery/targetgroup" - "github.com/prometheus/prometheus/model/histogram" "github.com/prometheus/prometheus/model/labels" - "github.com/prometheus/prometheus/model/timestamp" - "github.com/prometheus/prometheus/storage" ) const ( @@ -522,240 +518,3 @@ scrape_configs: }) } } - -func TestBucketLimitAppender(t *testing.T) { - example := histogram.Histogram{ - Schema: 0, - Count: 21, - Sum: 33, - ZeroThreshold: 0.001, - ZeroCount: 3, - PositiveSpans: []histogram.Span{ - {Offset: 0, Length: 3}, - }, - PositiveBuckets: []int64{3, 0, 0}, - NegativeSpans: []histogram.Span{ - {Offset: 0, Length: 3}, - }, - NegativeBuckets: []int64{3, 0, 0}, - } - - bigGap := histogram.Histogram{ - Schema: 0, - Count: 21, - Sum: 33, - ZeroThreshold: 0.001, - ZeroCount: 3, - PositiveSpans: []histogram.Span{ - {Offset: 1, Length: 1}, // in (1, 2] - {Offset: 2, Length: 1}, // in (8, 16] - }, - PositiveBuckets: []int64{1, 0}, // 1, 1 - } - - customBuckets := histogram.Histogram{ - Schema: histogram.CustomBucketsSchema, - Count: 9, - Sum: 33, - PositiveSpans: []histogram.Span{ - {Offset: 0, Length: 3}, - }, - PositiveBuckets: []int64{3, 0, 0}, - CustomValues: []float64{1, 2, 3}, - } - - cases := []struct { - h histogram.Histogram - limit int - expectError bool - expectBucketCount int - expectSchema int32 - }{ - { - h: example, - limit: 3, - expectError: true, - }, - { - h: example, - limit: 4, - expectError: false, - expectBucketCount: 4, - expectSchema: -1, - }, - { - h: example, - limit: 10, - expectError: false, - expectBucketCount: 6, - expectSchema: 0, - }, - { - h: bigGap, - limit: 1, - expectError: false, - expectBucketCount: 1, - expectSchema: -2, - }, - { - h: customBuckets, - limit: 2, - expectError: true, - }, - { - h: customBuckets, - limit: 3, - expectError: false, - expectBucketCount: 3, - expectSchema: histogram.CustomBucketsSchema, - }, - } - - resApp := &collectResultAppender{} - - for _, c := range cases { - for _, floatHisto := range []bool{true, false} { - t.Run(fmt.Sprintf("floatHistogram=%t", floatHisto), func(t *testing.T) { - app := &bucketLimitAppender{Appender: resApp, limit: c.limit} - ts := int64(10 * time.Minute / time.Millisecond) - lbls := labels.FromStrings("__name__", "sparse_histogram_series") - var err error - if floatHisto { - fh := c.h.Copy().ToFloat(nil) - _, err = app.AppendHistogram(0, lbls, ts, nil, fh) - if c.expectError { - require.Error(t, err) - } else { - require.Equal(t, c.expectSchema, fh.Schema) - require.Equal(t, c.expectBucketCount, len(fh.NegativeBuckets)+len(fh.PositiveBuckets)) - require.NoError(t, err) - } - } else { - h := c.h.Copy() - _, err = app.AppendHistogram(0, lbls, ts, h, nil) - if c.expectError { - require.Error(t, err) - } else { - require.Equal(t, c.expectSchema, h.Schema) - require.Equal(t, c.expectBucketCount, len(h.NegativeBuckets)+len(h.PositiveBuckets)) - require.NoError(t, err) - } - } - require.NoError(t, app.Commit()) - }) - } - } -} - -func TestMaxSchemaAppender(t *testing.T) { - example := histogram.Histogram{ - Schema: 0, - Count: 21, - Sum: 33, - ZeroThreshold: 0.001, - ZeroCount: 3, - PositiveSpans: []histogram.Span{ - {Offset: 0, Length: 3}, - }, - PositiveBuckets: []int64{3, 0, 0}, - NegativeSpans: []histogram.Span{ - {Offset: 0, Length: 3}, - }, - NegativeBuckets: []int64{3, 0, 0}, - } - - customBuckets := histogram.Histogram{ - Schema: histogram.CustomBucketsSchema, - Count: 9, - Sum: 33, - PositiveSpans: []histogram.Span{ - {Offset: 0, Length: 3}, - }, - PositiveBuckets: []int64{3, 0, 0}, - CustomValues: []float64{1, 2, 3}, - } - - cases := []struct { - h histogram.Histogram - maxSchema int32 - expectSchema int32 - }{ - { - h: example, - maxSchema: -1, - expectSchema: -1, - }, - { - h: example, - maxSchema: 0, - expectSchema: 0, - }, - { - h: customBuckets, - maxSchema: -1, - expectSchema: histogram.CustomBucketsSchema, - }, - } - - resApp := &collectResultAppender{} - - for _, c := range cases { - for _, floatHisto := range []bool{true, false} { - t.Run(fmt.Sprintf("floatHistogram=%t", floatHisto), func(t *testing.T) { - app := &maxSchemaAppender{Appender: resApp, maxSchema: c.maxSchema} - ts := int64(10 * time.Minute / time.Millisecond) - lbls := labels.FromStrings("__name__", "sparse_histogram_series") - var err error - if floatHisto { - fh := c.h.Copy().ToFloat(nil) - _, err = app.AppendHistogram(0, lbls, ts, nil, fh) - require.Equal(t, c.expectSchema, fh.Schema) - require.NoError(t, err) - } else { - h := c.h.Copy() - _, err = app.AppendHistogram(0, lbls, ts, h, nil) - require.Equal(t, c.expectSchema, h.Schema) - require.NoError(t, err) - } - require.NoError(t, app.Commit()) - }) - } - } -} - -// Test sample_limit when a scrape containst Native Histograms. -func TestAppendWithSampleLimitAndNativeHistogram(t *testing.T) { - const sampleLimit = 2 - resApp := &collectResultAppender{} - sl := newBasicScrapeLoop(t, context.Background(), nil, func(_ context.Context) storage.Appender { - return resApp - }, 0) - sl.sampleLimit = sampleLimit - - now := time.Now() - app := appender(sl.appender(context.Background()), sl.sampleLimit, sl.bucketLimit, sl.maxSchema) - - // sample_limit is set to 2, so first two scrapes should work - _, err := app.Append(0, labels.FromStrings(model.MetricNameLabel, "foo"), timestamp.FromTime(now), 1) - require.NoError(t, err) - - // Second sample, should be ok. - _, err = app.AppendHistogram( - 0, - labels.FromStrings(model.MetricNameLabel, "my_histogram1"), - timestamp.FromTime(now), - &histogram.Histogram{}, - nil, - ) - require.NoError(t, err) - - // This is third sample with sample_limit=2, it should trigger errSampleLimit. - _, err = app.AppendHistogram( - 0, - labels.FromStrings(model.MetricNameLabel, "my_histogram2"), - timestamp.FromTime(now), - &histogram.Histogram{}, - nil, - ) - require.ErrorIs(t, err, errSampleLimit) -} diff --git a/util/teststorage/appender.go b/util/teststorage/appender.go new file mode 100644 index 0000000000..58a08c0daf --- /dev/null +++ b/util/teststorage/appender.go @@ -0,0 +1,346 @@ +// Copyright The Prometheus Authors +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package teststorage + +import ( + "bytes" + "context" + "fmt" + "math" + "strings" + "sync" + + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/exemplar" + "github.com/prometheus/prometheus/model/histogram" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/model/metadata" + "github.com/prometheus/prometheus/storage" +) + +// Sample represents test, combined sample for mocking storage.AppenderV2. +type Sample struct { + MF string + L labels.Labels + M metadata.Metadata + ST, T int64 + V float64 + H *histogram.Histogram + FH *histogram.FloatHistogram + ES []exemplar.Exemplar +} + +func (s Sample) String() string { + b := bytes.Buffer{} + if s.M.Help != "" { + _, _ = fmt.Fprintf(&b, "HELP %s\n", s.M.Help) + } + if s.M.Type != model.MetricTypeUnknown && s.M.Type != "" { + _, _ = fmt.Fprintf(&b, "type@%s ", s.M.Type) + } + if s.M.Unit != "" { + _, _ = fmt.Fprintf(&b, "unit@%s ", s.M.Unit) + } + h := "" + if s.H != nil { + h = s.H.String() + } + + fh := "" + if s.FH != nil { + fh = s.FH.String() + } + _, _ = fmt.Fprintf(&b, "%s %v%v%v st@%v t@%v\n", s.L.String(), s.V, h, fh, s.ST, s.T) + return b.String() +} + +func (s Sample) exemplarsEqual(other []exemplar.Exemplar) bool { + if len(s.ES) != len(other) { + return false + } + for i := range s.ES { + if !s.ES[i].Equals(other[i]) { + return false + } + } + return true +} + +func (s Sample) Equal(other Sample) bool { + return strings.Compare(s.MF, other.MF) == 0 && + labels.Equal(s.L, other.L) && + s.M.Equals(other.M) && + s.ST == other.ST && + s.T == other.T && + math.Float64bits(s.V) == math.Float64bits(s.V) && // Compare Float64bits so NaN values which are exactly the same will compare equal. + s.H.Equals(other.H) && + s.FH.Equals(other.FH) && + s.exemplarsEqual(other.ES) +} + +// Appender is a storage.Appender mock. +// It allows: +// * recording all samples that were added through the appender. +// * optionally backed by another appender it writes samples through (Next). +// * optionally runs another appender before result recording e.g. to simulate chained validation (Prev) +type Appender struct { + Prev storage.Appendable // Optional appender to run before the result collection. + Next storage.Appendable // Optional appender to run after results are collected (e.g. TestStorage). + + AppendErr error // Inject appender error on every Append, AppendHistogram and ST zero calls. + AppendExemplarsError error // Inject exemplar error. + CommitErr error // Inject commit error. + + mtx sync.Mutex // mutex for result writes and ResultSamplesGreaterThan read. + + // Recorded results. + PendingSamples []Sample + ResultSamples []Sample + RolledbackSamples []Sample +} + +func (a *Appender) ResultReset() { + a.PendingSamples = a.PendingSamples[:0] + a.ResultSamples = a.ResultSamples[:0] + a.RolledbackSamples = a.RolledbackSamples[:0] +} + +func (a *Appender) ResultSamplesGreaterThan(than int) bool { + a.mtx.Lock() + defer a.mtx.Unlock() + return len(a.ResultSamples) > than +} + +// ResultMetadata returns ResultSamples with samples only containing L and M. +// This is for compatibility with tests that only focus on metadata. +// +// TODO: Rewrite tests to test metadata on ResultSamples instead. +func (a *Appender) ResultMetadata() []Sample { + var ret []Sample + for _, s := range a.ResultSamples { + ret = append(ret, Sample{L: s.L, M: s.M}) + } + return ret +} + +func (a *Appender) String() string { + var sb strings.Builder + sb.WriteString("committed:\n") + for _, s := range a.ResultSamples { + sb.WriteString("\n") + sb.WriteString(s.String()) + } + sb.WriteString("pending:\n") + for _, s := range a.PendingSamples { + sb.WriteString("\n") + sb.WriteString(s.String()) + } + sb.WriteString("rolledback:\n") + for _, s := range a.RolledbackSamples { + sb.WriteString("\n") + sb.WriteString(s.String()) + } + return sb.String() +} + +func NewAppender() *Appender { + return &Appender{} +} + +type appender struct { + prev storage.Appender + next storage.Appender + + *Appender +} + +func (a *Appender) Appender(ctx context.Context) storage.Appender { + ret := &appender{Appender: a} + if a.Prev != nil { + ret.prev = a.Prev.Appender(ctx) + } + if a.Next != nil { + ret.next = a.Next.Appender(ctx) + } + return ret +} + +func (a *appender) SetOptions(*storage.AppendOptions) {} + +func (a *appender) Append(ref storage.SeriesRef, ls labels.Labels, t int64, v float64) (storage.SeriesRef, error) { + if a.Prev != nil { + if _, err := a.prev.Append(ref, ls, t, v); err != nil { + return 0, err + } + } + + if a.AppendErr != nil { + return 0, a.AppendErr + } + + a.mtx.Lock() + a.PendingSamples = append(a.PendingSamples, Sample{L: ls, T: t, V: v}) + a.mtx.Unlock() + + if a.next != nil { + return a.next.Append(ref, ls, t, v) + } + + if ref == 0 { + // Use labels hash as a stand-in for unique series reference, to avoid having to track all series. + ref = storage.SeriesRef(ls.Hash()) + } + return ref, nil +} + +func (a *appender) AppendHistogram(ref storage.SeriesRef, ls labels.Labels, t int64, h *histogram.Histogram, fh *histogram.FloatHistogram) (storage.SeriesRef, error) { + if a.Prev != nil { + if _, err := a.prev.AppendHistogram(ref, ls, t, h, fh); err != nil { + return 0, err + } + } + + if a.AppendErr != nil { + return 0, a.AppendErr + } + + a.mtx.Lock() + a.PendingSamples = append(a.PendingSamples, Sample{L: ls, T: t, H: h, FH: fh}) + a.mtx.Unlock() + + if a.next != nil { + return a.next.AppendHistogram(ref, ls, t, h, fh) + } + + if ref == 0 { + // Use labels hash as a stand-in for unique series reference, to avoid having to track all series. + ref = storage.SeriesRef(ls.Hash()) + } + return ref, nil +} + +func (a *appender) AppendExemplar(ref storage.SeriesRef, l labels.Labels, e exemplar.Exemplar) (storage.SeriesRef, error) { + if a.Prev != nil { + if _, err := a.prev.AppendExemplar(ref, l, e); err != nil { + return 0, err + } + } + + if a.AppendExemplarsError != nil { + return 0, a.AppendExemplarsError + } + + a.mtx.Lock() + // NOTE(bwplotka): Eventually exemplar has to be attached to a series and soon + // the AppenderV2 will guarantee that for TSDB. Assume this from the mock perspective + // with the naive attaching. See: https://github.com/prometheus/prometheus/pull/17610 + i := 0 + for range len(a.PendingSamples) { + if ref == storage.SeriesRef(a.PendingSamples[i].L.Hash()) { + a.PendingSamples[i].ES = append(a.PendingSamples[i].ES, e) + break + } + i++ + } + a.mtx.Unlock() + if i >= len(a.PendingSamples) { + return 0, fmt.Errorf("teststorage.appender: exemplar appender without series; ref %v; l %v; exemplar: %v", ref, l, e) + } + + if a.next != nil { + return a.next.AppendExemplar(ref, l, e) + } + return ref, nil +} + +func (a *appender) AppendSTZeroSample(ref storage.SeriesRef, l labels.Labels, _, st int64) (storage.SeriesRef, error) { + return a.Append(ref, l, st, 0.0) +} + +func (a *appender) AppendHistogramSTZeroSample(ref storage.SeriesRef, l labels.Labels, _, st int64, h *histogram.Histogram, _ *histogram.FloatHistogram) (storage.SeriesRef, error) { + if h != nil { + return a.AppendHistogram(ref, l, st, &histogram.Histogram{}, nil) + } + return a.AppendHistogram(ref, l, st, nil, &histogram.FloatHistogram{}) +} + +func (a *appender) UpdateMetadata(ref storage.SeriesRef, l labels.Labels, m metadata.Metadata) (storage.SeriesRef, error) { + if a.Prev != nil { + if _, err := a.prev.UpdateMetadata(ref, l, m); err != nil { + return 0, err + } + } + + a.mtx.Lock() + // NOTE(bwplotka): Eventually exemplar has to be attached to a series and soon + // the AppenderV2 will guarantee that for TSDB. Assume this from the mock perspective + // with the naive attaching. See: https://github.com/prometheus/prometheus/pull/17610 + i := 0 + for range len(a.PendingSamples) { + if ref == storage.SeriesRef(a.PendingSamples[i].L.Hash()) { + a.PendingSamples[i].M = m + break + } + i++ + } + a.mtx.Unlock() + if i >= len(a.PendingSamples) { + return 0, fmt.Errorf("teststorage.appender: metadata update without series; ref %v; l %v; m: %v", ref, l, m) + } + + if a.next != nil { + return a.next.UpdateMetadata(ref, l, m) + } + return ref, nil +} + +func (a *appender) Commit() error { + if a.Prev != nil { + if err := a.prev.Commit(); err != nil { + return err + } + } + + if a.CommitErr != nil { + return a.CommitErr + } + + a.mtx.Lock() + a.ResultSamples = append(a.ResultSamples, a.PendingSamples...) + a.PendingSamples = a.PendingSamples[:0] + a.mtx.Unlock() + + if a.next != nil { + return a.next.Commit() + } + return nil +} + +func (a *appender) Rollback() error { + if a.prev != nil { + if err := a.prev.Rollback(); err != nil { + return err + } + } + + a.mtx.Lock() + a.RolledbackSamples = append(a.RolledbackSamples, a.PendingSamples...) + a.PendingSamples = a.PendingSamples[:0] + a.mtx.Unlock() + + if a.next != nil { + return a.next.Rollback() + } + return nil +} diff --git a/util/teststorage/appender_test.go b/util/teststorage/appender_test.go new file mode 100644 index 0000000000..f01c0c4581 --- /dev/null +++ b/util/teststorage/appender_test.go @@ -0,0 +1,59 @@ +// Copyright The Prometheus Authors +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package teststorage + +import ( + "fmt" + "testing" + + "github.com/google/go-cmp/cmp" + "github.com/stretchr/testify/require" + + "github.com/prometheus/prometheus/model/exemplar" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/model/metadata" + "github.com/prometheus/prometheus/util/testutil" +) + +// TestSample_RequireEqual ensures standard testutil.RequireEqual is enough for comparisons. +// This is thanks to the fact metadata has now Equals method. +// TODO(bwplotka): Adding (*labels.Labels) Equals(other *labels.Labels) would remove the +// need of testutil.RequireEqual fully. Do it. +func TestSample_RequireEqual(t *testing.T) { + a := []Sample{ + {}, + {L: labels.FromStrings("__name__", "test_metric_total"), M: metadata.Metadata{Type: "counter", Unit: "metric", Help: "some help text"}}, + {L: labels.FromStrings("__name__", "test_metric2", "foo", "bar"), M: metadata.Metadata{Type: "gauge", Unit: "", Help: "other help text"}, V: 123.123}, + {ES: []exemplar.Exemplar{{Labels: labels.FromStrings("__name__", "yolo")}}}, + } + testutil.RequireEqual(t, a, a) + + b1 := []Sample{ + {}, + {L: labels.FromStrings("__name__", "test_metric_total"), M: metadata.Metadata{Type: "counter", Unit: "metric", Help: "some help text"}}, + {L: labels.FromStrings("__name__", "test_metric2_diff", "foo", "bar"), M: metadata.Metadata{Type: "gauge", Unit: "", Help: "other help text"}, V: 123.123}, // test_metric2_diff is different. + {ES: []exemplar.Exemplar{{Labels: labels.FromStrings("__name__", "yolo")}}}, + } + requireNotEqual(t, a, b1) +} + +func requireNotEqual(t testing.TB, a, b any) { + t.Helper() + if !cmp.Equal(a, b, cmp.Comparer(labels.Equal)) { + return + } + require.Fail(t, fmt.Sprintf("Equal, but expected not: \n"+ + "a: %s\n"+ + "b: %s", a, b)) +}