diff --git a/scrape/helpers_test.go b/scrape/helpers_test.go index 7bc9e3f7d4..2719a467bc 100644 --- a/scrape/helpers_test.go +++ b/scrape/helpers_test.go @@ -90,6 +90,27 @@ type histogramSample struct { 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 } @@ -112,8 +133,8 @@ type collectResultAppender struct { rolledbackHistograms []histogramSample resultExemplars []exemplar.Exemplar pendingExemplars []exemplar.Exemplar - resultMetadata []metadata.Metadata - pendingMetadata []metadata.Metadata + resultMetadata []metadataEntry + pendingMetadata []metadataEntry } func (a *collectResultAppender) SetOptions(opts *storage.AppendOptions) {} @@ -173,7 +194,7 @@ func (a *collectResultAppender) AppendHistogramCTZeroSample(ref storage.SeriesRe 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, m) + a.pendingMetadata = append(a.pendingMetadata, metadataEntry{metric: l, m: m}) if ref == 0 { ref = storage.SeriesRef(rand.Uint64()) } diff --git a/scrape/scrape.go b/scrape/scrape.go index 2da07d719e..1f41f50689 100644 --- a/scrape/scrape.go +++ b/scrape/scrape.go @@ -29,6 +29,7 @@ import ( "strings" "sync" "time" + "unsafe" "github.com/klauspost/compress/gzip" config_util "github.com/prometheus/common/config" @@ -931,6 +932,7 @@ type scrapeLoop struct { // scrapeCache tracks mappings of exposed metric strings to label sets and // storage references. Additionally, it tracks staleness of series between // scrapes. +// Cache is meant to be used per a single target. type scrapeCache struct { iter uint64 // Current scrape iteration. @@ -951,8 +953,10 @@ type scrapeCache struct { seriesCur map[uint64]labels.Labels seriesPrev map[uint64]labels.Labels - metaMtx sync.Mutex - metadata map[string]*metaEntry + // TODO(bwplotka): Consider moving Metadata API to use WAL instead of scrape loop to + // avoid locking (using metadata API can block scraping). + metaMtx sync.Mutex // Mutex is needed due to api touching it when metadata is queried. + metadata map[string]*metaEntry // metadata by metric family name. metrics *scrapeMetrics } @@ -1078,73 +1082,79 @@ func (c *scrapeCache) forEachStale(f func(labels.Labels) bool) { } } -func (c *scrapeCache) setType(metric []byte, t model.MetricType) { - c.metaMtx.Lock() +func yoloString(b []byte) string { + return unsafe.String(unsafe.SliceData(b), len(b)) +} - e, ok := c.metadata[string(metric)] +func (c *scrapeCache) setType(mfName []byte, t model.MetricType) ([]byte, *metaEntry) { + c.metaMtx.Lock() + defer c.metaMtx.Unlock() + + e, ok := c.metadata[yoloString(mfName)] if !ok { e = &metaEntry{Metadata: metadata.Metadata{Type: model.MetricTypeUnknown}} - c.metadata[string(metric)] = e + c.metadata[string(mfName)] = e } if e.Type != t { e.Type = t e.lastIterChange = c.iter } e.lastIter = c.iter - - c.metaMtx.Unlock() + return mfName, e } -func (c *scrapeCache) setHelp(metric, help []byte) { +func (c *scrapeCache) setHelp(mfName, help []byte) ([]byte, *metaEntry) { c.metaMtx.Lock() + defer c.metaMtx.Unlock() - e, ok := c.metadata[string(metric)] + e, ok := c.metadata[yoloString(mfName)] if !ok { e = &metaEntry{Metadata: metadata.Metadata{Type: model.MetricTypeUnknown}} - c.metadata[string(metric)] = e + c.metadata[string(mfName)] = e } if e.Help != string(help) { e.Help = string(help) e.lastIterChange = c.iter } e.lastIter = c.iter - - c.metaMtx.Unlock() + return mfName, e } -func (c *scrapeCache) setUnit(metric, unit []byte) { +func (c *scrapeCache) setUnit(mfName, unit []byte) ([]byte, *metaEntry) { c.metaMtx.Lock() + defer c.metaMtx.Unlock() - e, ok := c.metadata[string(metric)] + e, ok := c.metadata[yoloString(mfName)] if !ok { e = &metaEntry{Metadata: metadata.Metadata{Type: model.MetricTypeUnknown}} - c.metadata[string(metric)] = e + c.metadata[string(mfName)] = e } if e.Unit != string(unit) { e.Unit = string(unit) e.lastIterChange = c.iter } e.lastIter = c.iter - - c.metaMtx.Unlock() + return mfName, e } -func (c *scrapeCache) GetMetadata(metric string) (MetricMetadata, bool) { +// GetMetadata returns metadata given the metric family name. +func (c *scrapeCache) GetMetadata(mfName string) (MetricMetadata, bool) { c.metaMtx.Lock() defer c.metaMtx.Unlock() - m, ok := c.metadata[metric] + m, ok := c.metadata[mfName] if !ok { return MetricMetadata{}, false } return MetricMetadata{ - Metric: metric, - Type: m.Type, - Help: m.Help, - Unit: m.Unit, + MetricFamily: mfName, + Type: m.Type, + Help: m.Help, + Unit: m.Unit, }, true } +// ListMetadata lists metadata. func (c *scrapeCache) ListMetadata() []MetricMetadata { c.metaMtx.Lock() defer c.metaMtx.Unlock() @@ -1153,16 +1163,16 @@ func (c *scrapeCache) ListMetadata() []MetricMetadata { for m, e := range c.metadata { res = append(res, MetricMetadata{ - Metric: m, - Type: e.Type, - Help: e.Help, - Unit: e.Unit, + MetricFamily: m, + Type: e.Type, + Help: e.Help, + Unit: e.Unit, }) } return res } -// MetadataSize returns the size of the metadata cache. +// SizeMetadata returns the size of the metadata cache. func (c *scrapeCache) SizeMetadata() (s int) { c.metaMtx.Lock() defer c.metaMtx.Unlock() @@ -1173,7 +1183,7 @@ func (c *scrapeCache) SizeMetadata() (s int) { return s } -// MetadataLen returns the number of metadata entries in the cache. +// LengthMetadata returns the number of metadata entries in the cache. func (c *scrapeCache) LengthMetadata() int { c.metaMtx.Lock() defer c.metaMtx.Unlock() @@ -1607,39 +1617,17 @@ func (sl *scrapeLoop) append(app storage.Appender, b []byte, contentType string, ) } var ( - appErrs = appendErrors{} - sampleLimitErr error - bucketLimitErr error - lset labels.Labels // escapes to heap so hoisted out of loop - e exemplar.Exemplar // escapes to heap so hoisted out of loop - meta metadata.Metadata - metadataChanged bool + appErrs = appendErrors{} + sampleLimitErr error + bucketLimitErr error + lset labels.Labels // escapes to heap so hoisted out of loop + e exemplar.Exemplar // escapes to heap so hoisted out of loop + lastMeta *metaEntry + lastMFName []byte ) exemplars := make([]exemplar.Exemplar, 0, 1) - // updateMetadata updates the current iteration's metadata object and the - // metadataChanged value if we have metadata in the scrape cache AND the - // labelset is for a new series or the metadata for this series has just - // changed. It returns a boolean based on whether the metadata was updated. - updateMetadata := func(lset labels.Labels, isNewSeries bool) bool { - if !sl.appendMetadataToWAL { - return false - } - - sl.cache.metaMtx.Lock() - defer sl.cache.metaMtx.Unlock() - metaEntry, metaOk := sl.cache.metadata[lset.Get(labels.MetricName)] - if metaOk && (isNewSeries || metaEntry.lastIterChange == sl.cache.iter) { - metadataChanged = true - meta.Type = metaEntry.Type - meta.Unit = metaEntry.Unit - meta.Help = metaEntry.Help - return true - } - return false - } - // Take an appender with limits. app = appender(app, sl.sampleLimit, sl.bucketLimit, sl.maxSchema) @@ -1669,14 +1657,18 @@ loop: break } switch et { + // TODO(bwplotka): Consider changing parser to give metadata at once instead of type, help and unit in separation, ideally on `Series()/Histogram() + // otherwise we can expose metadata without series on metadata API. case textparse.EntryType: - sl.cache.setType(p.Type()) + // TODO(bwplotka): Build meta entry directly instead of locking and updating the map. This will + // allow to properly update metadata when e.g unit was added, then removed; + lastMFName, lastMeta = sl.cache.setType(p.Type()) continue case textparse.EntryHelp: - sl.cache.setHelp(p.Help()) + lastMFName, lastMeta = sl.cache.setHelp(p.Help()) continue case textparse.EntryUnit: - sl.cache.setUnit(p.Unit()) + lastMFName, lastMeta = sl.cache.setUnit(p.Unit()) continue case textparse.EntryComment: continue @@ -1699,10 +1691,6 @@ loop: t = *parsedTimestamp } - // Zero metadata out for current iteration until it's resolved. - meta = metadata.Metadata{} - metadataChanged = false - if sl.cache.getDropped(met) { continue } @@ -1716,9 +1704,6 @@ loop: ref = ce.ref lset = ce.lset hash = ce.hash - - // Update metadata only if it changed in the current iteration. - updateMetadata(lset, false) } else { p.Metric(&lset) hash = lset.Hash() @@ -1747,9 +1732,6 @@ loop: sl.metrics.targetScrapePoolExceededLabelLimits.Inc() break loop } - - // Append metadata for new series if they were present. - updateMetadata(lset, true) } if seriesAlreadyScraped && parsedTimestamp == nil { @@ -1857,10 +1839,18 @@ loop: sl.metrics.targetScrapeExemplarOutOfOrder.Add(float64(outOfOrderExemplars)) } - if sl.appendMetadataToWAL && metadataChanged { - if _, merr := app.UpdateMetadata(ref, lset, meta); 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", meta), "err", merr) + if sl.appendMetadataToWAL && lastMeta != nil { + // Is it new series OR did metadata change for this family? + if !ok || lastMeta.lastIterChange == sl.cache.iter { + // 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. CT and NHCB parsing). + if isSeriesPartOfFamily(lset.Get(labels.MetricName), 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) + } + } } } } @@ -1896,6 +1886,71 @@ loop: return } +func isSeriesPartOfFamily(mName string, mfName []byte, typ model.MetricType) bool { + mfNameStr := yoloString(mfName) + if !strings.HasPrefix(mName, mfNameStr) { // Fast path. + return false + } + + var ( + gotMFName string + ok bool + ) + switch typ { + case model.MetricTypeCounter: + // Prometheus allows _total, cut it from mf name to support this case. + mfNameStr, _ = strings.CutSuffix(mfNameStr, "_total") + + gotMFName, ok = strings.CutSuffix(mName, "_total") + if !ok { + gotMFName = mName + } + case model.MetricTypeHistogram: + gotMFName, ok = strings.CutSuffix(mName, "_bucket") + if !ok { + gotMFName, ok = strings.CutSuffix(mName, "_sum") + if !ok { + gotMFName, ok = strings.CutSuffix(mName, "_count") + if !ok { + gotMFName = mName + } + } + } + case model.MetricTypeGaugeHistogram: + gotMFName, ok = strings.CutSuffix(mName, "_bucket") + if !ok { + gotMFName, ok = strings.CutSuffix(mName, "_gsum") + if !ok { + gotMFName, ok = strings.CutSuffix(mName, "_gcount") + if !ok { + gotMFName = mName + } + } + } + case model.MetricTypeSummary: + gotMFName, ok = strings.CutSuffix(mName, "_sum") + if !ok { + gotMFName, ok = strings.CutSuffix(mName, "_count") + if !ok { + gotMFName = mName + } + } + case model.MetricTypeInfo: + // Technically prometheus text does not support info type, but we might + // accidentally allow info type in prom parse, so support metric family names + // with the _info explicitly too. + mfNameStr, _ = strings.CutSuffix(mfNameStr, "_info") + + gotMFName, ok = strings.CutSuffix(mName, "_info") + if !ok { + gotMFName = mName + } + default: + gotMFName = mName + } + return mfNameStr == gotMFName +} + // Adds samples to the appender, checking the error, and then returns the # of samples added, // whether the caller should continue to process more samples, and any sample or bucket limit errors. func (sl *scrapeLoop) checkAddError(met []byte, err error, sampleLimitErr, bucketLimitErr *error, appErrs *appendErrors) (bool, error) { diff --git a/scrape/scrape_test.go b/scrape/scrape_test.go index a67d52e5cc..c1fca54c6a 100644 --- a/scrape/scrape_test.go +++ b/scrape/scrape_test.go @@ -50,6 +50,7 @@ import ( "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/model/relabel" "github.com/prometheus/prometheus/model/textparse" "github.com/prometheus/prometheus/model/timestamp" @@ -96,7 +97,9 @@ 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) - defer s.Close() + t.Cleanup(func() { + _ = s.Close() + }) runScrapeLoopTest(t, s, false) }) @@ -104,7 +107,9 @@ 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) - defer s.Close() + t.Cleanup(func() { + _ = s.Close() + }) runScrapeLoopTest(t, s, true) }) @@ -126,13 +131,13 @@ func runScrapeLoopTest(t *testing.T, s *teststorage.TestStorage, expectOutOfOrde timestampInorder2 := now.Add(5 * time.Minute) slApp := sl.appender(context.Background()) - _, _, _, err := sl.append(slApp, []byte(`metric_a{a="1",b="1"} 1`), "text/plain", timestampInorder1) + _, _, _, err := sl.append(slApp, []byte(`metric_total{a="1",b="1"} 1`), "text/plain", timestampInorder1) require.NoError(t, err) - _, _, _, err = sl.append(slApp, []byte(`metric_a{a="1",b="1"} 2`), "text/plain", timestampOutOfOrder) + _, _, _, err = sl.append(slApp, []byte(`metric_total{a="1",b="1"} 2`), "text/plain", timestampOutOfOrder) require.NoError(t, err) - _, _, _, err = sl.append(slApp, []byte(`metric_a{a="1",b="1"} 3`), "text/plain", timestampInorder2) + _, _, _, err = sl.append(slApp, []byte(`metric_total{a="1",b="1"} 3`), "text/plain", timestampInorder2) require.NoError(t, err) require.NoError(t, slApp.Commit()) @@ -145,7 +150,7 @@ func runScrapeLoopTest(t *testing.T, s *teststorage.TestStorage, expectOutOfOrde defer q.Close() // Use a matcher to filter the metric name. - series := q.Select(ctx, false, nil, labels.MustNewMatcher(labels.MatchRegexp, "__name__", "metric_a")) + series := q.Select(ctx, false, nil, labels.MustNewMatcher(labels.MatchRegexp, "__name__", "metric_total")) var results []floatSample for series.Next() { @@ -165,12 +170,12 @@ func runScrapeLoopTest(t *testing.T, s *teststorage.TestStorage, expectOutOfOrde // Define the expected results want := []floatSample{ { - metric: labels.FromStrings("__name__", "metric_a", "a", "1", "b", "1"), + metric: labels.FromStrings("__name__", "metric_total", "a", "1", "b", "1"), t: timestamp.FromTime(timestampInorder1), f: 1, }, { - metric: labels.FromStrings("__name__", "metric_a", "a", "1", "b", "1"), + metric: labels.FromStrings("__name__", "metric_total", "a", "1", "b", "1"), t: timestamp.FromTime(timestampInorder2), f: 3, }, @@ -183,6 +188,110 @@ func runScrapeLoopTest(t *testing.T, s *teststorage.TestStorage, expectOutOfOrde } } +// Regression test against https://github.com/prometheus/prometheus/issues/15831. +func TestScrapeAppendMetadataUpdate(t *testing.T) { + const ( + scrape1 = `# TYPE test_metric counter +# HELP test_metric some help text +# UNIT test_metric metric +test_metric_total 1 +# TYPE test_metric2 gauge +# HELP test_metric2 other help text +test_metric2{foo="bar"} 2 +# TYPE test_metric3 gauge +# HELP test_metric3 this represents tricky case of "broken" text that is not trivial to detect +test_metric3_metric4{foo="bar"} 2 +# EOF` + scrape2 = `# TYPE test_metric counter +# HELP test_metric different help text +test_metric_total 11 +# TYPE test_metric2 gauge +# HELP test_metric2 other help text +# UNIT test_metric2 metric2 +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(ctx context.Context) storage.Appender { return capp }, 0) + + now := time.Now() + slApp := sl.appender(context.Background()) + _, _, _, err := sl.append(slApp, []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 + + // 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)) + require.NoError(t, err) + require.NoError(t, slApp.Commit()) + testutil.RequireEqualWithOptions(t, []metadataEntry(nil), capp.resultMetadata, []cmp.Option{cmp.Comparer(metadataEntryEqual)}) + + slApp = sl.appender(context.Background()) + _, _, _, err = sl.append(slApp, []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)}) +} + +func TestIsSeriesPartOfFamily(t *testing.T) { + t.Run("counter", func(t *testing.T) { + require.True(t, isSeriesPartOfFamily("http_requests_total", []byte("http_requests_total"), model.MetricTypeCounter)) // Prometheus text style. + require.True(t, isSeriesPartOfFamily("http_requests_total", []byte("http_requests"), model.MetricTypeCounter)) // OM text style. + require.True(t, isSeriesPartOfFamily("http_requests_total", []byte("http_requests_total"), model.MetricTypeUnknown)) + + require.False(t, isSeriesPartOfFamily("http_requests_total", []byte("http_requests"), model.MetricTypeUnknown)) // We don't know. + require.False(t, isSeriesPartOfFamily("http_requests2_total", []byte("http_requests_total"), model.MetricTypeCounter)) + require.False(t, isSeriesPartOfFamily("http_requests_requests_total", []byte("http_requests"), model.MetricTypeCounter)) + }) + + t.Run("gauge", func(t *testing.T) { + require.True(t, isSeriesPartOfFamily("http_requests_count", []byte("http_requests_count"), model.MetricTypeGauge)) + require.True(t, isSeriesPartOfFamily("http_requests_count", []byte("http_requests_count"), model.MetricTypeUnknown)) + + require.False(t, isSeriesPartOfFamily("http_requests_count2", []byte("http_requests_count"), model.MetricTypeCounter)) + }) + + t.Run("histogram", func(t *testing.T) { + require.True(t, isSeriesPartOfFamily("http_requests_seconds_sum", []byte("http_requests_seconds"), model.MetricTypeHistogram)) + require.True(t, isSeriesPartOfFamily("http_requests_seconds_count", []byte("http_requests_seconds"), model.MetricTypeHistogram)) + require.True(t, isSeriesPartOfFamily("http_requests_seconds_bucket", []byte("http_requests_seconds"), model.MetricTypeHistogram)) + require.True(t, isSeriesPartOfFamily("http_requests_seconds", []byte("http_requests_seconds"), model.MetricTypeHistogram)) + + require.False(t, isSeriesPartOfFamily("http_requests_seconds_sum", []byte("http_requests_seconds"), model.MetricTypeUnknown)) // We don't know. + require.False(t, isSeriesPartOfFamily("http_requests_seconds2_sum", []byte("http_requests_seconds"), model.MetricTypeHistogram)) + }) + + t.Run("summary", func(t *testing.T) { + require.True(t, isSeriesPartOfFamily("http_requests_seconds_sum", []byte("http_requests_seconds"), model.MetricTypeSummary)) + require.True(t, isSeriesPartOfFamily("http_requests_seconds_count", []byte("http_requests_seconds"), model.MetricTypeSummary)) + require.True(t, isSeriesPartOfFamily("http_requests_seconds", []byte("http_requests_seconds"), model.MetricTypeSummary)) + + require.False(t, isSeriesPartOfFamily("http_requests_seconds_sum", []byte("http_requests_seconds"), model.MetricTypeUnknown)) // We don't know. + require.False(t, isSeriesPartOfFamily("http_requests_seconds2_sum", []byte("http_requests_seconds"), model.MetricTypeSummary)) + }) + + t.Run("info", func(t *testing.T) { + require.True(t, isSeriesPartOfFamily("go_build_info", []byte("go_build_info"), model.MetricTypeInfo)) // Prometheus text style. + require.True(t, isSeriesPartOfFamily("go_build_info", []byte("go_build"), model.MetricTypeInfo)) // OM text style. + require.True(t, isSeriesPartOfFamily("go_build_info", []byte("go_build_info"), model.MetricTypeUnknown)) + + require.False(t, isSeriesPartOfFamily("go_build_info", []byte("go_build"), model.MetricTypeUnknown)) // We don't know. + require.False(t, isSeriesPartOfFamily("go_build2_info", []byte("go_build_info"), model.MetricTypeInfo)) + require.False(t, isSeriesPartOfFamily("go_build_build_info", []byte("go_build_info"), model.MetricTypeInfo)) + }) +} + func TestDroppedTargetsList(t *testing.T) { var ( app = &nopAppendable{} @@ -824,7 +933,7 @@ func newBasicScrapeLoopWithFallback(t testing.TB, ctx context.Context, scraper s false, false, false, - false, + true, nil, false, newTestScrapeMetrics(t), @@ -1131,7 +1240,7 @@ func TestScrapeLoopMetadata(t *testing.T) { total, _, _, err := sl.append(slApp, []byte(`# TYPE test_metric counter # HELP test_metric some help text # UNIT test_metric metric -test_metric 1 +test_metric_total 1 # TYPE test_metric_no_help gauge # HELP test_metric_no_type other help text # EOF`), "application/openmetrics-text", time.Now()) diff --git a/scrape/target.go b/scrape/target.go index 22cde01c05..4f576504f0 100644 --- a/scrape/target.go +++ b/scrape/target.go @@ -78,17 +78,17 @@ func (t *Target) String() string { // MetricMetadataStore represents a storage for metadata. type MetricMetadataStore interface { ListMetadata() []MetricMetadata - GetMetadata(metric string) (MetricMetadata, bool) + GetMetadata(mfName string) (MetricMetadata, bool) SizeMetadata() int LengthMetadata() int } -// MetricMetadata is a piece of metadata for a metric. +// MetricMetadata is a piece of metadata for a metric family. type MetricMetadata struct { - Metric string - Type model.MetricType - Help string - Unit string + MetricFamily string + Type model.MetricType + Help string + Unit string } func (t *Target) ListMetadata() []MetricMetadata { @@ -124,14 +124,14 @@ func (t *Target) LengthMetadata() int { } // GetMetadata returns type and help metadata for the given metric. -func (t *Target) GetMetadata(metric string) (MetricMetadata, bool) { +func (t *Target) GetMetadata(mfName string) (MetricMetadata, bool) { t.mtx.RLock() defer t.mtx.RUnlock() if t.metadata == nil { return MetricMetadata{}, false } - return t.metadata.GetMetadata(metric) + return t.metadata.GetMetadata(mfName) } func (t *Target) SetMetadataStore(s MetricMetadataStore) { diff --git a/storage/remote/metadata_watcher_test.go b/storage/remote/metadata_watcher_test.go index ce9b9d022e..d939ef8efb 100644 --- a/storage/remote/metadata_watcher_test.go +++ b/storage/remote/metadata_watcher_test.go @@ -40,9 +40,9 @@ func (s *TestMetaStore) ListMetadata() []scrape.MetricMetadata { return s.Metadata } -func (s *TestMetaStore) GetMetadata(metric string) (scrape.MetricMetadata, bool) { +func (s *TestMetaStore) GetMetadata(mfName string) (scrape.MetricMetadata, bool) { for _, m := range s.Metadata { - if metric == m.Metric { + if mfName == m.MetricFamily { return m, true } } @@ -106,26 +106,26 @@ func TestWatchScrapeManager_ReadyForCollection(t *testing.T) { metadata := &TestMetaStore{ Metadata: []scrape.MetricMetadata{ { - Metric: "prometheus_tsdb_head_chunks_created_total", - Type: model.MetricTypeCounter, - Help: "Total number", - Unit: "", + MetricFamily: "prometheus_tsdb_head_chunks_created", + Type: model.MetricTypeCounter, + Help: "Total number", + Unit: "", }, { - Metric: "prometheus_remote_storage_retried_samples_total", - Type: model.MetricTypeCounter, - Help: "Total number", - Unit: "", + MetricFamily: "prometheus_remote_storage_retried_samples", + Type: model.MetricTypeCounter, + Help: "Total number", + Unit: "", }, }, } metadataDup := &TestMetaStore{ Metadata: []scrape.MetricMetadata{ { - Metric: "prometheus_tsdb_head_chunks_created_total", - Type: model.MetricTypeCounter, - Help: "Total number", - Unit: "", + MetricFamily: "prometheus_tsdb_head_chunks_created", + Type: model.MetricTypeCounter, + Help: "Total number", + Unit: "", }, }, } diff --git a/storage/remote/queue_manager.go b/storage/remote/queue_manager.go index f0ea33fb52..50f10eebe3 100644 --- a/storage/remote/queue_manager.go +++ b/storage/remote/queue_manager.go @@ -550,7 +550,7 @@ func (t *QueueManager) AppendWatcherMetadata(ctx context.Context, metadata []scr mm := make([]prompb.MetricMetadata, 0, len(metadata)) for _, entry := range metadata { mm = append(mm, prompb.MetricMetadata{ - MetricFamilyName: entry.Metric, + MetricFamilyName: entry.MetricFamily, Help: entry.Help, Type: prompb.FromMetadataType(entry.Type), Unit: entry.Unit, diff --git a/storage/remote/queue_manager_test.go b/storage/remote/queue_manager_test.go index 202c71c348..38eda81d97 100644 --- a/storage/remote/queue_manager_test.go +++ b/storage/remote/queue_manager_test.go @@ -342,10 +342,10 @@ func TestMetadataDelivery(t *testing.T) { numMetadata := 1532 for i := 0; i < numMetadata; i++ { metadata = append(metadata, scrape.MetricMetadata{ - Metric: "prometheus_remote_storage_sent_metadata_bytes_total_" + strconv.Itoa(i), - Type: model.MetricTypeCounter, - Help: "a nice help text", - Unit: "", + MetricFamily: "prometheus_remote_storage_sent_metadata_bytes_" + strconv.Itoa(i), + Type: model.MetricTypeCounter, + Help: "a nice help text", + Unit: "", }) } @@ -357,7 +357,7 @@ func TestMetadataDelivery(t *testing.T) { // fit into MaxSamplesPerSend. require.Equal(t, numMetadata/config.DefaultMetadataConfig.MaxSamplesPerSend+1, c.writesReceived) // Make sure the last samples were sent. - require.Equal(t, c.receivedMetadata[metadata[len(metadata)-1].Metric][0].MetricFamilyName, metadata[len(metadata)-1].Metric) + require.Equal(t, c.receivedMetadata[metadata[len(metadata)-1].MetricFamily][0].MetricFamilyName, metadata[len(metadata)-1].MetricFamily) } func TestWALMetadataDelivery(t *testing.T) { diff --git a/web/api/v1/api.go b/web/api/v1/api.go index 9821b26ec2..7687264426 100644 --- a/web/api/v1/api.go +++ b/web/api/v1/api.go @@ -1228,11 +1228,11 @@ func (api *API) targetMetadata(r *http.Request) apiFuncResult { if metric == "" { for _, md := range t.ListMetadata() { res = append(res, metricMetadata{ - Target: targetLabels, - Metric: md.Metric, - Type: md.Type, - Help: md.Help, - Unit: md.Unit, + Target: targetLabels, + MetricFamily: md.MetricFamily, + Type: md.Type, + Help: md.Help, + Unit: md.Unit, }) } continue @@ -1253,11 +1253,11 @@ func (api *API) targetMetadata(r *http.Request) apiFuncResult { } type metricMetadata struct { - Target labels.Labels `json:"target"` - Metric string `json:"metric,omitempty"` - Type model.MetricType `json:"type"` - Help string `json:"help"` - Unit string `json:"unit"` + Target labels.Labels `json:"target"` + MetricFamily string `json:"metric,omitempty"` + Type model.MetricType `json:"type"` + Help string `json:"help"` + Unit string `json:"unit"` } // AlertmanagerDiscovery has all the active Alertmanagers. @@ -1357,7 +1357,7 @@ func (api *API) metricMetadata(r *http.Request) apiFuncResult { if metric == "" { for _, mm := range t.ListMetadata() { m := metadata.Metadata{Type: mm.Type, Help: mm.Help, Unit: mm.Unit} - ms, ok := metrics[mm.Metric] + ms, ok := metrics[mm.MetricFamily] if limitPerMetric > 0 && len(ms) >= limitPerMetric { continue @@ -1365,7 +1365,7 @@ func (api *API) metricMetadata(r *http.Request) apiFuncResult { if !ok { ms = map[metadata.Metadata]struct{}{} - metrics[mm.Metric] = ms + metrics[mm.MetricFamily] = ms } ms[m] = struct{}{} } @@ -1374,7 +1374,7 @@ func (api *API) metricMetadata(r *http.Request) apiFuncResult { if md, ok := t.GetMetadata(metric); ok { m := metadata.Metadata{Type: md.Type, Help: md.Help, Unit: md.Unit} - ms, ok := metrics[md.Metric] + ms, ok := metrics[md.MetricFamily] if limitPerMetric > 0 && len(ms) >= limitPerMetric { continue @@ -1382,7 +1382,7 @@ func (api *API) metricMetadata(r *http.Request) apiFuncResult { if !ok { ms = map[metadata.Metadata]struct{}{} - metrics[md.Metric] = ms + metrics[md.MetricFamily] = ms } ms[m] = struct{}{} } diff --git a/web/api/v1/api_test.go b/web/api/v1/api_test.go index 37227d849d..f9bdbe3947 100644 --- a/web/api/v1/api_test.go +++ b/web/api/v1/api_test.go @@ -84,7 +84,7 @@ func (s *testMetaStore) ListMetadata() []scrape.MetricMetadata { func (s *testMetaStore) GetMetadata(metric string) (scrape.MetricMetadata, bool) { for _, m := range s.Metadata { - if metric == m.Metric { + if metric == m.MetricFamily { return m, true } } @@ -1891,10 +1891,10 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "test", metadata: []scrape.MetricMetadata{ { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Number of OS threads created.", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Number of OS threads created.", + Unit: "", }, }, }, @@ -1921,10 +1921,10 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "blackbox", metadata: []scrape.MetricMetadata{ { - Metric: "prometheus_tsdb_storage_blocks_bytes", - Type: model.MetricTypeGauge, - Help: "The number of bytes that are currently used for local storage by all blocks.", - Unit: "", + MetricFamily: "prometheus_tsdb_storage_blocks_bytes", + Type: model.MetricTypeGauge, + Help: "The number of bytes that are currently used for local storage by all blocks.", + Unit: "", }, }, }, @@ -1934,10 +1934,10 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E Target: labels.FromMap(map[string]string{ "job": "blackbox", }), - Metric: "prometheus_tsdb_storage_blocks_bytes", - Help: "The number of bytes that are currently used for local storage by all blocks.", - Type: model.MetricTypeGauge, - Unit: "", + MetricFamily: "prometheus_tsdb_storage_blocks_bytes", + Help: "The number of bytes that are currently used for local storage by all blocks.", + Type: model.MetricTypeGauge, + Unit: "", }, }, }, @@ -1949,10 +1949,10 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "test", metadata: []scrape.MetricMetadata{ { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Number of OS threads created.", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Number of OS threads created.", + Unit: "", }, }, }, @@ -1960,10 +1960,10 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "blackbox", metadata: []scrape.MetricMetadata{ { - Metric: "prometheus_tsdb_storage_blocks_bytes", - Type: model.MetricTypeGauge, - Help: "The number of bytes that are currently used for local storage by all blocks.", - Unit: "", + MetricFamily: "prometheus_tsdb_storage_blocks_bytes", + Type: model.MetricTypeGauge, + Help: "The number of bytes that are currently used for local storage by all blocks.", + Unit: "", }, }, }, @@ -1973,25 +1973,25 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E Target: labels.FromMap(map[string]string{ "job": "test", }), - Metric: "go_threads", - Help: "Number of OS threads created.", - Type: model.MetricTypeGauge, - Unit: "", + MetricFamily: "go_threads", + Help: "Number of OS threads created.", + Type: model.MetricTypeGauge, + Unit: "", }, { Target: labels.FromMap(map[string]string{ "job": "blackbox", }), - Metric: "prometheus_tsdb_storage_blocks_bytes", - Help: "The number of bytes that are currently used for local storage by all blocks.", - Type: model.MetricTypeGauge, - Unit: "", + MetricFamily: "prometheus_tsdb_storage_blocks_bytes", + Help: "The number of bytes that are currently used for local storage by all blocks.", + Type: model.MetricTypeGauge, + Unit: "", }, }, sorter: func(m interface{}) { sort.Slice(m.([]metricMetadata), func(i, j int) bool { s := m.([]metricMetadata) - return s[i].Metric < s[j].Metric + return s[i].MetricFamily < s[j].MetricFamily }) }, }, @@ -2026,16 +2026,16 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "test", metadata: []scrape.MetricMetadata{ { - Metric: "prometheus_engine_query_duration_seconds", - Type: model.MetricTypeSummary, - Help: "Query timings", - Unit: "", + MetricFamily: "prometheus_engine_query_duration_seconds", + Type: model.MetricTypeSummary, + Help: "Query timings", + Unit: "", }, { - Metric: "go_info", - Type: model.MetricTypeGauge, - Help: "Information about the Go environment.", - Unit: "", + MetricFamily: "go_info", + Type: model.MetricTypeGauge, + Help: "Information about the Go environment.", + Unit: "", }, }, }, @@ -2056,10 +2056,10 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "test", metadata: []scrape.MetricMetadata{ { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Number of OS threads created", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Number of OS threads created", + Unit: "", }, }, }, @@ -2067,10 +2067,10 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "blackbox", metadata: []scrape.MetricMetadata{ { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Number of OS threads created", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Number of OS threads created", + Unit: "", }, }, }, @@ -2089,10 +2089,10 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "test", metadata: []scrape.MetricMetadata{ { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Number of OS threads created", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Number of OS threads created", + Unit: "", }, }, }, @@ -2100,10 +2100,10 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "blackbox", metadata: []scrape.MetricMetadata{ { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Number of OS threads that were created.", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Number of OS threads that were created.", + Unit: "", }, }, }, @@ -2136,16 +2136,16 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "test", metadata: []scrape.MetricMetadata{ { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Number of OS threads created", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Number of OS threads created", + Unit: "", }, { - Metric: "prometheus_engine_query_duration_seconds", - Type: model.MetricTypeSummary, - Help: "Query Timings.", - Unit: "", + MetricFamily: "prometheus_engine_query_duration_seconds", + Type: model.MetricTypeSummary, + Help: "Query Timings.", + Unit: "", }, }, }, @@ -2153,10 +2153,10 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "blackbox", metadata: []scrape.MetricMetadata{ { - Metric: "go_gc_duration_seconds", - Type: model.MetricTypeSummary, - Help: "A summary of the GC invocation durations.", - Unit: "", + MetricFamily: "go_gc_duration_seconds", + Type: model.MetricTypeSummary, + Help: "A summary of the GC invocation durations.", + Unit: "", }, }, }, @@ -2172,22 +2172,22 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "test", metadata: []scrape.MetricMetadata{ { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Number of OS threads created", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Number of OS threads created", + Unit: "", }, { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Repeated metadata", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Repeated metadata", + Unit: "", }, { - Metric: "go_gc_duration_seconds", - Type: model.MetricTypeSummary, - Help: "A summary of the GC invocation durations.", - Unit: "", + MetricFamily: "go_gc_duration_seconds", + Type: model.MetricTypeSummary, + Help: "A summary of the GC invocation durations.", + Unit: "", }, }, }, @@ -2211,22 +2211,22 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "test", metadata: []scrape.MetricMetadata{ { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Number of OS threads created", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Number of OS threads created", + Unit: "", }, { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Repeated metadata", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Repeated metadata", + Unit: "", }, { - Metric: "go_gc_duration_seconds", - Type: model.MetricTypeSummary, - Help: "A summary of the GC invocation durations.", - Unit: "", + MetricFamily: "go_gc_duration_seconds", + Type: model.MetricTypeSummary, + Help: "A summary of the GC invocation durations.", + Unit: "", }, }, }, @@ -2244,22 +2244,22 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "test", metadata: []scrape.MetricMetadata{ { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Number of OS threads created", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Number of OS threads created", + Unit: "", }, { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Repeated metadata", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Repeated metadata", + Unit: "", }, { - Metric: "go_gc_duration_seconds", - Type: model.MetricTypeSummary, - Help: "A summary of the GC invocation durations.", - Unit: "", + MetricFamily: "go_gc_duration_seconds", + Type: model.MetricTypeSummary, + Help: "A summary of the GC invocation durations.", + Unit: "", }, }, }, @@ -2267,16 +2267,16 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "secondTarget", metadata: []scrape.MetricMetadata{ { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Number of OS threads created, but from a different target", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Number of OS threads created, but from a different target", + Unit: "", }, { - Metric: "go_gc_duration_seconds", - Type: model.MetricTypeSummary, - Help: "A summary of the GC invocation durations, but from a different target.", - Unit: "", + MetricFamily: "go_gc_duration_seconds", + Type: model.MetricTypeSummary, + Help: "A summary of the GC invocation durations, but from a different target.", + Unit: "", }, }, }, @@ -2293,10 +2293,10 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "test", metadata: []scrape.MetricMetadata{ { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Number of OS threads created", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Number of OS threads created", + Unit: "", }, }, }, @@ -2304,16 +2304,16 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "blackbox", metadata: []scrape.MetricMetadata{ { - Metric: "go_gc_duration_seconds", - Type: model.MetricTypeSummary, - Help: "A summary of the GC invocation durations.", - Unit: "", + MetricFamily: "go_gc_duration_seconds", + Type: model.MetricTypeSummary, + Help: "A summary of the GC invocation durations.", + Unit: "", }, { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Number of OS threads that were created.", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Number of OS threads that were created.", + Unit: "", }, }, }, @@ -2342,10 +2342,10 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "test", metadata: []scrape.MetricMetadata{ { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Number of OS threads created", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Number of OS threads created", + Unit: "", }, }, },