diff --git a/model/labels/labels_common.go b/model/labels/labels_common.go index 5f46d6c35f..4d5cee7ed1 100644 --- a/model/labels/labels_common.go +++ b/model/labels/labels_common.go @@ -215,7 +215,7 @@ func (b *Builder) Range(f func(l Label)) { // Take a copy of add and del, so they are unaffected by calls to Set() or Del(). origAdd, origDel := append(addStack[:0], b.add...), append(delStack[:0], b.del...) b.base.Range(func(l Label) { - if !slices.Contains(origDel, l.Name) && !contains(origAdd, l.Name) { + if !slices.Contains(origDel, l.Name) && !Contains(origAdd, l.Name) { f(l) } }) @@ -224,7 +224,8 @@ func (b *Builder) Range(f func(l Label)) { } } -func contains(s []Label, n string) bool { +// Contains returns true if the label name n is present in s, otherwise false. +func Contains(s []Label, n string) bool { for _, a := range s { if a.Name == n { return true diff --git a/model/labels/labels_dedupelabels.go b/model/labels/labels_dedupelabels.go index edc6ff8e82..ccef3649c4 100644 --- a/model/labels/labels_dedupelabels.go +++ b/model/labels/labels_dedupelabels.go @@ -464,6 +464,18 @@ func New(ls ...Label) Labels { return Labels{syms: syms.nameTable, data: yoloString(buf)} } +// NewFromSorted returns sorted Labels from the given +// sorted labels. In case of slices it returns the +// input. +func NewFromSorted(ls []Label) Labels { + syms := NewSymbolTable() + var stackSpace [16]int + size, nums := mapLabelsToNumbers(syms, ls, stackSpace[:]) + buf := make([]byte, size) + marshalNumbersToSizedBuffer(nums, buf) + return Labels{syms: syms.nameTable, data: yoloString(buf)} +} + // FromStrings creates new labels from pairs of strings. func FromStrings(ss ...string) Labels { if len(ss)%2 != 0 { diff --git a/model/labels/labels_slicelabels.go b/model/labels/labels_slicelabels.go index a6e5654fa7..684a0fedb3 100644 --- a/model/labels/labels_slicelabels.go +++ b/model/labels/labels_slicelabels.go @@ -279,6 +279,13 @@ func New(ls ...Label) Labels { return set } +// NewFromSorted returns sorted Labels from the given +// sorted labels. In case of slices it returns the +// input. This function does NOT copy! +func NewFromSorted(ls []Label) Labels { + return ls +} + // FromStrings creates new labels from pairs of strings. func FromStrings(ss ...string) Labels { if len(ss)%2 != 0 { @@ -423,7 +430,7 @@ func (b *Builder) Labels() Labels { } res := make(Labels, 0, expectedSize) for _, l := range b.base { - if slices.Contains(b.del, l.Name) || contains(b.add, l.Name) { + if slices.Contains(b.del, l.Name) || Contains(b.add, l.Name) { continue } res = append(res, l) diff --git a/model/labels/labels_stringlabels.go b/model/labels/labels_stringlabels.go index 4b9bfd15af..9d459ebef0 100644 --- a/model/labels/labels_stringlabels.go +++ b/model/labels/labels_stringlabels.go @@ -314,6 +314,16 @@ func New(ls ...Label) Labels { return Labels{data: yoloString(buf)} } +// NewFromSorted returns sorted Labels from the given +// sorted labels. In case of slices it returns the +// input. +func NewFromSorted(ls []Label) Labels { + size := labelsSize(ls) + buf := make([]byte, size) + marshalLabelsToSizedBuffer(ls, buf) + return Labels{data: yoloString(buf)} +} + // FromStrings creates new labels from pairs of strings. func FromStrings(ss ...string) Labels { if len(ss)%2 != 0 { diff --git a/storage/remote/otlptranslator/prometheusremotewrite/combined_appender.go b/storage/remote/otlptranslator/prometheusremotewrite/combined_appender.go new file mode 100644 index 0000000000..bbe8eb57b4 --- /dev/null +++ b/storage/remote/otlptranslator/prometheusremotewrite/combined_appender.go @@ -0,0 +1,214 @@ +// Copyright 2025 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 prometheusremotewrite + +import ( + "errors" + "fmt" + "log/slog" + + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + + "github.com/prometheus/prometheus/model/exemplar" + "github.com/prometheus/prometheus/model/histogram" + modelLabels "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/model/metadata" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite/labels" +) + +// CombinedAppender is similar to storage.Appender, but combines updates to +// metadata, created timestamps, exemplars and samples into a single call. +type CombinedAppender interface { + // AppendSample appends a sample and related exemplars, metadata, and + // created timestamp to the storage. + AppendSample(metricFamilyName string, ls labels.Labels, meta metadata.Metadata, t, ct int64, v float64, es []exemplar.Exemplar) error + // AppendSample appends a histogram and related exemplars, metadata, and + // created timestamp to the storage. + AppendHistogram(metricFamilyName string, ls labels.Labels, meta metadata.Metadata, t, ct int64, h *histogram.Histogram, es []exemplar.Exemplar) error + // Commit finalizes the ongoing transaction in storage. + Commit() error +} + +// NewCombinedAppender creates a combined appender that sets start times and +// updates metadata for each series only once, and appends samples and +// exemplars for each call. +func NewCombinedAppender(app storage.Appender, logger *slog.Logger, reg prometheus.Registerer, ingestCTZeroSample bool) CombinedAppender { + return &combinedAppender{ + app: app, + logger: logger, + ingestCTZeroSample: ingestCTZeroSample, + refs: make(map[uint64]labelsRef), + samplesAppendedWithoutMetadata: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Namespace: "prometheus", + Subsystem: "api", + Name: "otlp_without_metadata_appended_samples_total", + Help: "The total number of received OTLP data points which were ingested without corresponding metadata.", + }), + outOfOrderExemplars: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Namespace: "prometheus", + Subsystem: "api", + Name: "otlp_out_of_order_exemplars_total", + Help: "The total number of received OTLP exemplars which were rejected because they were out of order.", + }), + } +} + +type labelsRef struct { + ref storage.SeriesRef + ls modelLabels.Labels +} + +type combinedAppender struct { + app storage.Appender + logger *slog.Logger + samplesAppendedWithoutMetadata prometheus.Counter + outOfOrderExemplars prometheus.Counter + ingestCTZeroSample bool + // Used to ensure we only update metadata and created timestamps once, and to share storage.SeriesRefs. + // To detect hash collision it also stores the labels. + // There is no overflow/conflict list, the TSDB will handle that part. + refs map[uint64]labelsRef +} + +func (b *combinedAppender) AppendSample(_ string, rawls labels.Labels, meta metadata.Metadata, t, ct int64, v float64, es []exemplar.Exemplar) (err error) { + ls := modelLabels.NewFromSorted(rawls) + hash := ls.Hash() + lref, exists := b.refs[hash] + ref := lref.ref + if exists && !modelLabels.Equal(lref.ls, ls) { + // Hash collision, this is a new series. + exists = false + } + if !exists { + if ct != 0 && b.ingestCTZeroSample { + ref, err = b.app.AppendCTZeroSample(ref, ls, t, ct) + if err != nil && !errors.Is(err, storage.ErrOutOfOrderCT) { + // Even for the first sample OOO is a common scenario because + // we can't tell if a CT was already ingested in a previous request. + // We ignore the error. + b.logger.Debug("Error when appending float CT from OTLP", "err", err, "series", ls.String(), "created_timestamp", ct, "timestamp", t) + } + } + } + ref, err = b.app.Append(ref, ls, t, v) + if err != nil { + // Although AppendHistogram does not currently return ErrDuplicateSampleForTimestamp there is + // a note indicating its inclusion in the future. + if errors.Is(err, storage.ErrOutOfOrderSample) || + errors.Is(err, storage.ErrOutOfBounds) || + errors.Is(err, storage.ErrDuplicateSampleForTimestamp) { + b.logger.Error("Error when appending float sample from OTLP", "err", err.Error(), "series", ls.String(), "timestamp", t) + } + } + + if ref == 0 { + // We cannot update metadata or add exemplars on non existent series. + return + } + + if !exists { + b.refs[hash] = labelsRef{ + ref: ref, + ls: ls, + } + // If this is the first time we see this series, set the metadata. + ref, err = b.app.UpdateMetadata(0, ls, meta) + if err != nil { + b.samplesAppendedWithoutMetadata.Add(1) + b.logger.Debug("Error while updating metadata from OTLP", "err", err) + } + } + + b.appendExemplars(ref, ls, es) + + return +} + +func (b *combinedAppender) AppendHistogram(_ string, rawls labels.Labels, meta metadata.Metadata, t, ct int64, h *histogram.Histogram, es []exemplar.Exemplar) (err error) { + ls := modelLabels.NewFromSorted(rawls) + hash := ls.Hash() + lref, exists := b.refs[hash] + ref := lref.ref + if exists && !modelLabels.Equal(lref.ls, ls) { + // Hash collision, this is a new series. + exists = false + } + if !exists { + if ct != 0 && b.ingestCTZeroSample { + ref, err = b.app.AppendHistogramCTZeroSample(ref, ls, t, ct, h, nil) + if err != nil && !errors.Is(err, storage.ErrOutOfOrderCT) { + // Even for the first sample OOO is a common scenario because + // we can't tell if a CT was already ingested in a previous request. + // We ignore the error. + b.logger.Debug("Error when appending Histogram CT from OTLP", "err", err, "series", ls.String(), "created_timestamp", ct, "timestamp", t) + } + } + } + + ref, err = b.app.AppendHistogram(ref, ls, t, h, nil) + if err != nil { + // Although AppendHistogram does not currently return ErrDuplicateSampleForTimestamp there is + // a note indicating its inclusion in the future. + if errors.Is(err, storage.ErrOutOfOrderSample) || + errors.Is(err, storage.ErrOutOfBounds) || + errors.Is(err, storage.ErrDuplicateSampleForTimestamp) { + b.logger.Error("Error when appending histogram sample from OTLP", "err", err.Error(), "series", ls.String(), "timestamp", t) + } + } + + if ref == 0 { + // We cannot update metadata or add exemplars on non existent series. + return + } + + if !exists { + b.refs[hash] = labelsRef{ + ref: ref, + ls: ls, + } + // If this is the first time we see this series, set the metadata. + ref, err = b.app.UpdateMetadata(0, ls, meta) + if err != nil { + b.samplesAppendedWithoutMetadata.Add(1) + b.logger.Debug("Error while updating metadata from OTLP", "err", err) + } + } + + b.appendExemplars(ref, ls, es) + + return +} + +func (b *combinedAppender) Commit() error { + return b.app.Commit() +} + +func (b *combinedAppender) appendExemplars(ref storage.SeriesRef, ls modelLabels.Labels, es []exemplar.Exemplar) storage.SeriesRef { + var err error + for _, e := range es { + if ref, err = b.app.AppendExemplar(ref, ls, e); err != nil { + switch { + case errors.Is(err, storage.ErrOutOfOrderExemplar): + b.outOfOrderExemplars.Add(1) + b.logger.Debug("Out of order exemplar from OTLP", "series", ls.String(), "exemplar", fmt.Sprintf("%+v", e)) + default: + // Since exemplar storage is still experimental, we don't fail the request on ingestion errors + b.logger.Debug("Error while adding exemplar from OTLP", "series", ls.String(), "exemplar", fmt.Sprintf("%+v", e), "err", err) + } + } + } + return ref +} diff --git a/storage/remote/otlptranslator/prometheusremotewrite/combined_appender_test.go b/storage/remote/otlptranslator/prometheusremotewrite/combined_appender_test.go new file mode 100644 index 0000000000..b3b80756a7 --- /dev/null +++ b/storage/remote/otlptranslator/prometheusremotewrite/combined_appender_test.go @@ -0,0 +1,471 @@ +// Copyright 2025 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 prometheusremotewrite + +import ( + "context" + "math" + "testing" + "time" + + "github.com/google/go-cmp/cmp" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/common/model" + "github.com/prometheus/common/promslog" + "github.com/stretchr/testify/require" + + "github.com/prometheus/prometheus/model/exemplar" + "github.com/prometheus/prometheus/model/histogram" + modelLabels "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/model/metadata" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite/labels" + "github.com/prometheus/prometheus/tsdb" + "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/prometheus/prometheus/tsdb/tsdbutil" + "github.com/prometheus/prometheus/util/testutil" +) + +type mockCombinedAppender struct { + pendingSamples []combinedSample + pendingHistograms []combinedHistogram + + samples []combinedSample + histograms []combinedHistogram +} + +type combinedSample struct { + metricFamilyName string + ls labels.Labels + meta metadata.Metadata + t int64 + ct int64 + v float64 + es []exemplar.Exemplar +} + +type combinedHistogram struct { + metricFamilyName string + ls labels.Labels + meta metadata.Metadata + t int64 + ct int64 + h *histogram.Histogram + es []exemplar.Exemplar +} + +func (m *mockCombinedAppender) AppendSample(metricFamilyName string, ls labels.Labels, meta metadata.Metadata, t, ct int64, v float64, es []exemplar.Exemplar) error { + m.pendingSamples = append(m.pendingSamples, combinedSample{ + metricFamilyName: metricFamilyName, + ls: ls, + meta: meta, + t: t, + ct: ct, + v: v, + es: es, + }) + return nil +} + +func (m *mockCombinedAppender) AppendHistogram(metricFamilyName string, ls labels.Labels, meta metadata.Metadata, t, ct int64, h *histogram.Histogram, es []exemplar.Exemplar) error { + m.pendingHistograms = append(m.pendingHistograms, combinedHistogram{ + metricFamilyName: metricFamilyName, + ls: ls, + meta: meta, + t: t, + ct: ct, + h: h, + es: es, + }) + return nil +} + +func (m *mockCombinedAppender) Commit() error { + m.samples = append(m.samples, m.pendingSamples...) + m.pendingSamples = m.pendingSamples[:0] + m.histograms = append(m.histograms, m.pendingHistograms...) + m.pendingHistograms = m.pendingHistograms[:0] + return nil +} + +func requireEqual(t testing.TB, expected, actual interface{}, msgAndArgs ...interface{}) { + testutil.RequireEqualWithOptions(t, expected, actual, []cmp.Option{cmp.AllowUnexported(combinedSample{}, combinedHistogram{})}, msgAndArgs...) +} + +// TestCombinedAppenderOnTSDB runs some basic tests on a real TSDB to check +// that the combinedAppender works on a real TSDB. +func TestCombinedAppenderOnTSDB(t *testing.T) { + t.Run("ingestCTZeroSample=false", func(t *testing.T) { testCombinedAppenderOnTSDB(t, false) }) + + t.Run("ingestCTZeroSample=true", func(t *testing.T) { testCombinedAppenderOnTSDB(t, true) }) +} + +func testCombinedAppenderOnTSDB(t *testing.T, ingestCTZeroSample bool) { + t.Helper() + + now := time.Now() + + testExemplars := []exemplar.Exemplar{ + { + Labels: modelLabels.FromStrings("tracid", "122"), + Value: 1337, + }, + { + Labels: modelLabels.FromStrings("tracid", "132"), + Value: 7777, + }, + } + expectedExemplars := []exemplar.QueryResult{ + { + SeriesLabels: modelLabels.FromStrings( + model.MetricNameLabel, "test_bytes_total", + "foo", "bar", + ), + Exemplars: testExemplars, + }, + } + + testCases := map[string]struct { + appendFunc func(CombinedAppender) error + expectedSamples []sample + expectedExemplars []exemplar.QueryResult + }{ + "single float sample, zero CT": { + appendFunc: func(app CombinedAppender) error { + return app.AppendSample("test_bytes_total", labels.FromStrings( + model.MetricNameLabel, "test_bytes_total", + "foo", "bar", + ), metadata.Metadata{ + Type: model.MetricTypeCounter, + Unit: "bytes", + Help: "some help", + }, now.UnixMilli(), 0, 42.0, testExemplars) + }, + expectedSamples: []sample{ + { + t: now.UnixMilli(), + f: 42.0, + }, + }, + expectedExemplars: expectedExemplars, + }, + "single float sample, very old CT": { + appendFunc: func(app CombinedAppender) error { + return app.AppendSample("test_bytes_total", labels.FromStrings( + model.MetricNameLabel, "test_bytes_total", + "foo", "bar", + ), metadata.Metadata{ + Type: model.MetricTypeCounter, + Unit: "bytes", + Help: "some help", + }, now.UnixMilli(), 1, 42.0, nil) + }, + expectedSamples: []sample{ + { + t: now.UnixMilli(), + f: 42.0, + }, + }, + }, + "single float sample, normal CT": { + appendFunc: func(app CombinedAppender) error { + return app.AppendSample("test_bytes_total", labels.FromStrings( + model.MetricNameLabel, "test_bytes_total", + "foo", "bar", + ), metadata.Metadata{ + Type: model.MetricTypeCounter, + Unit: "bytes", + Help: "some help", + }, now.UnixMilli(), now.Add(-2*time.Minute).UnixMilli(), 42.0, nil) + }, + expectedSamples: []sample{ + { + ctZero: true, + t: now.Add(-2 * time.Minute).UnixMilli(), + }, + { + t: now.UnixMilli(), + f: 42.0, + }, + }, + }, + "single float sample, CT name time as sample": { + appendFunc: func(app CombinedAppender) error { + return app.AppendSample("test_bytes_total", labels.FromStrings( + model.MetricNameLabel, "test_bytes_total", + "foo", "bar", + ), metadata.Metadata{ + Type: model.MetricTypeCounter, + Unit: "bytes", + Help: "some help", + }, now.UnixMilli(), now.UnixMilli(), 42.0, nil) + }, + expectedSamples: []sample{ + { + t: now.UnixMilli(), + f: 42.0, + }, + }, + }, + "single float sample, CT in the future of the sample": { + appendFunc: func(app CombinedAppender) error { + return app.AppendSample("test_bytes_total", labels.FromStrings( + model.MetricNameLabel, "test_bytes_total", + "foo", "bar", + ), metadata.Metadata{ + Type: model.MetricTypeCounter, + Unit: "bytes", + Help: "some help", + }, now.UnixMilli(), now.Add(time.Minute).UnixMilli(), 42.0, nil) + }, + expectedSamples: []sample{ + { + t: now.UnixMilli(), + f: 42.0, + }, + }, + }, + "single histogram sample, zero CT": { + appendFunc: func(app CombinedAppender) error { + return app.AppendHistogram("test_bytes_total", labels.FromStrings( + model.MetricNameLabel, "test_bytes_total", + "foo", "bar", + ), metadata.Metadata{ + Type: model.MetricTypeCounter, + Unit: "bytes", + Help: "some help", + }, now.UnixMilli(), 0, tsdbutil.GenerateTestHistogram(42), testExemplars) + }, + expectedSamples: []sample{ + { + t: now.UnixMilli(), + h: tsdbutil.GenerateTestHistogram(42), + }, + }, + expectedExemplars: expectedExemplars, + }, + "single histogram sample, very old CT": { + appendFunc: func(app CombinedAppender) error { + return app.AppendHistogram("test_bytes_total", labels.FromStrings( + model.MetricNameLabel, "test_bytes_total", + "foo", "bar", + ), metadata.Metadata{ + Type: model.MetricTypeCounter, + Unit: "bytes", + Help: "some help", + }, now.UnixMilli(), 1, tsdbutil.GenerateTestHistogram(42), nil) + }, + expectedSamples: []sample{ + { + t: now.UnixMilli(), + h: tsdbutil.GenerateTestHistogram(42), + }, + }, + }, + "single histogram sample, normal CT": { + appendFunc: func(app CombinedAppender) error { + return app.AppendHistogram("test_bytes_total", labels.FromStrings( + model.MetricNameLabel, "test_bytes_total", + "foo", "bar", + ), metadata.Metadata{ + Type: model.MetricTypeCounter, + Unit: "bytes", + Help: "some help", + }, now.UnixMilli(), now.Add(-2*time.Minute).UnixMilli(), tsdbutil.GenerateTestHistogram(42), nil) + }, + expectedSamples: []sample{ + { + ctZero: true, + t: now.Add(-2 * time.Minute).UnixMilli(), + h: &histogram.Histogram{}, + }, + { + t: now.UnixMilli(), + h: tsdbutil.GenerateTestHistogram(42), + }, + }, + }, + "single histogram sample, CT name time as sample": { + appendFunc: func(app CombinedAppender) error { + return app.AppendHistogram("test_bytes_total", labels.FromStrings( + model.MetricNameLabel, "test_bytes_total", + "foo", "bar", + ), metadata.Metadata{ + Type: model.MetricTypeCounter, + Unit: "bytes", + Help: "some help", + }, now.UnixMilli(), now.UnixMilli(), tsdbutil.GenerateTestHistogram(42), nil) + }, + expectedSamples: []sample{ + { + t: now.UnixMilli(), + h: tsdbutil.GenerateTestHistogram(42), + }, + }, + }, + "single histogram sample, CT in the future of the sample": { + appendFunc: func(app CombinedAppender) error { + return app.AppendHistogram("test_bytes_total", labels.FromStrings( + model.MetricNameLabel, "test_bytes_total", + "foo", "bar", + ), metadata.Metadata{ + Type: model.MetricTypeCounter, + Unit: "bytes", + Help: "some help", + }, now.UnixMilli(), now.Add(time.Minute).UnixMilli(), tsdbutil.GenerateTestHistogram(42), nil) + }, + expectedSamples: []sample{ + { + t: now.UnixMilli(), + h: tsdbutil.GenerateTestHistogram(42), + }, + }, + }, + "multiple float samples": { + appendFunc: func(app CombinedAppender) error { + err := app.AppendSample("test_bytes_total", labels.FromStrings( + model.MetricNameLabel, "test_bytes_total", + "foo", "bar", + ), metadata.Metadata{ + Type: model.MetricTypeCounter, + Unit: "bytes", + Help: "some help", + }, now.UnixMilli(), 0, 42.0, nil) + if err != nil { + return err + } + return app.AppendSample("test_bytes_total", labels.FromStrings( + model.MetricNameLabel, "test_bytes_total", + "foo", "bar", + ), metadata.Metadata{ + Type: model.MetricTypeCounter, + Unit: "bytes", + Help: "some help", + }, now.Add(15*time.Second).UnixMilli(), 0, 62.0, nil) + }, + expectedSamples: []sample{ + { + t: now.UnixMilli(), + f: 42.0, + }, + { + t: now.Add(15 * time.Second).UnixMilli(), + f: 62.0, + }, + }, + }, + "multiple histogram samples": { + appendFunc: func(app CombinedAppender) error { + err := app.AppendHistogram("test_bytes_total", labels.FromStrings( + model.MetricNameLabel, "test_bytes_total", + "foo", "bar", + ), metadata.Metadata{ + Type: model.MetricTypeCounter, + Unit: "bytes", + Help: "some help", + }, now.UnixMilli(), 0, tsdbutil.GenerateTestHistogram(42), nil) + if err != nil { + return err + } + return app.AppendHistogram("test_bytes_total", labels.FromStrings( + model.MetricNameLabel, "test_bytes_total", + "foo", "bar", + ), metadata.Metadata{ + Type: model.MetricTypeCounter, + Unit: "bytes", + Help: "some help", + }, now.Add(15*time.Second).UnixMilli(), 0, tsdbutil.GenerateTestHistogram(62), nil) + }, + expectedSamples: []sample{ + { + t: now.UnixMilli(), + h: tsdbutil.GenerateTestHistogram(42), + }, + { + t: now.Add(15 * time.Second).UnixMilli(), + h: tsdbutil.GenerateTestHistogram(62), + }, + }, + }, + } + + for name, tc := range testCases { + t.Run(name, func(t *testing.T) { + dir := t.TempDir() + opts := tsdb.DefaultOptions() + opts.EnableExemplarStorage = true + opts.MaxExemplars = 100 + opts.EnableNativeHistograms = true + db, err := tsdb.Open(dir, promslog.NewNopLogger(), prometheus.NewRegistry(), opts, nil) + require.NoError(t, err) + + t.Cleanup(func() { db.Close() }) + + ctx := context.Background() + capp := NewCombinedAppender(db.Appender(ctx), promslog.NewNopLogger(), prometheus.NewRegistry(), ingestCTZeroSample) + + require.NoError(t, tc.appendFunc(capp)) + + require.NoError(t, capp.Commit()) + + q, err := db.Querier(int64(math.MinInt64), int64(math.MaxInt64)) + require.NoError(t, err) + + ss := q.Select(ctx, false, &storage.SelectHints{ + Start: int64(math.MinInt64), + End: int64(math.MaxInt64), + }, modelLabels.MustNewMatcher(modelLabels.MatchEqual, model.MetricNameLabel, "test_bytes_total")) + + require.NoError(t, ss.Err()) + + require.True(t, ss.Next()) + series := ss.At() + it := series.Iterator(nil) + for _, sample := range tc.expectedSamples { + if !ingestCTZeroSample && sample.ctZero { + continue + } + if sample.h == nil { + require.Equal(t, chunkenc.ValFloat, it.Next()) + ts, v := it.At() + require.Equal(t, sample.t, ts) + require.Equal(t, sample.f, v) + } else { + require.Equal(t, chunkenc.ValHistogram, it.Next()) + ts, h := it.AtHistogram(nil) + require.Equal(t, sample.t, ts) + require.Equal(t, sample.h.Count, h.Count) + } + } + require.False(t, ss.Next()) + + eq, err := db.ExemplarQuerier(ctx) + require.NoError(t, err) + exResult, err := eq.Select(int64(math.MinInt64), int64(math.MaxInt64), []*modelLabels.Matcher{modelLabels.MustNewMatcher(modelLabels.MatchEqual, model.MetricNameLabel, "test_bytes_total")}) + require.NoError(t, err) + if tc.expectedExemplars == nil { + tc.expectedExemplars = []exemplar.QueryResult{} + } + require.Equal(t, tc.expectedExemplars, exResult) + }) + } +} + +type sample struct { + ctZero bool + + t int64 + f float64 + h *histogram.Histogram +} diff --git a/storage/remote/otlptranslator/prometheusremotewrite/helper.go b/storage/remote/otlptranslator/prometheusremotewrite/helper.go index 691e48de1c..573f30a0d4 100644 --- a/storage/remote/otlptranslator/prometheusremotewrite/helper.go +++ b/storage/remote/otlptranslator/prometheusremotewrite/helper.go @@ -23,22 +23,24 @@ import ( "log" "math" "slices" - "sort" "strconv" "strings" "time" "unicode/utf8" - "github.com/cespare/xxhash/v2" "github.com/prometheus/common/model" "github.com/prometheus/otlptranslator" "go.opentelemetry.io/collector/pdata/pcommon" "go.opentelemetry.io/collector/pdata/pmetric" conventions "go.opentelemetry.io/collector/semconv/v1.6.1" + "github.com/prometheus/prometheus/model/exemplar" + modelLabels "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/model/metadata" "github.com/prometheus/prometheus/model/timestamp" "github.com/prometheus/prometheus/model/value" "github.com/prometheus/prometheus/prompb" + "github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite/labels" ) const ( @@ -62,177 +64,115 @@ const ( defaultLookbackDelta = 5 * time.Minute ) -type bucketBoundsData struct { - ts *prompb.TimeSeries - bound float64 -} - -// byBucketBoundsData enables the usage of sort.Sort() with a slice of bucket bounds. -type byBucketBoundsData []bucketBoundsData - -func (m byBucketBoundsData) Len() int { return len(m) } -func (m byBucketBoundsData) Less(i, j int) bool { return m[i].bound < m[j].bound } -func (m byBucketBoundsData) Swap(i, j int) { m[i], m[j] = m[j], m[i] } - -// ByLabelName enables the usage of sort.Sort() with a slice of labels. -type ByLabelName []prompb.Label - -func (a ByLabelName) Len() int { return len(a) } -func (a ByLabelName) Less(i, j int) bool { return a[i].Name < a[j].Name } -func (a ByLabelName) Swap(i, j int) { a[i], a[j] = a[j], a[i] } - -// timeSeriesSignature returns a hashed label set signature. -// The label slice should not contain duplicate label names; this method sorts the slice by label name before creating -// the signature. -// The algorithm is the same as in Prometheus' labels.StableHash function. -func timeSeriesSignature(labels []prompb.Label) uint64 { - sort.Sort(ByLabelName(labels)) - - // Use xxhash.Sum64(b) for fast path as it's faster. - b := make([]byte, 0, 1024) - for i, v := range labels { - if len(b)+len(v.Name)+len(v.Value)+2 >= cap(b) { - // If labels entry is 1KB+ do not allocate whole entry. - h := xxhash.New() - _, _ = h.Write(b) - for _, v := range labels[i:] { - _, _ = h.WriteString(v.Name) - _, _ = h.Write(seps) - _, _ = h.WriteString(v.Value) - _, _ = h.Write(seps) - } - return h.Sum64() - } - - b = append(b, v.Name...) - b = append(b, seps[0]) - b = append(b, v.Value...) - b = append(b, seps[0]) - } - return xxhash.Sum64(b) -} - -var seps = []byte{'\xff'} - // createAttributes creates a slice of Prometheus Labels with OTLP attributes and pairs of string values. // Unpaired string values are ignored. String pairs overwrite OTLP labels if collisions happen and // if logOnOverwrite is true, the overwrite is logged. Resulting label names are sanitized. // If settings.PromoteResourceAttributes is not empty, it's a set of resource attributes that should be promoted to labels. -func createAttributes(resource pcommon.Resource, attributes pcommon.Map, scope scope, settings Settings, - ignoreAttrs []string, logOnOverwrite bool, metadata prompb.MetricMetadata, extras ...string, -) ([]prompb.Label, error) { +func (c *PrometheusConverter) createAttributes(resource pcommon.Resource, attributes pcommon.Map, scope scope, settings Settings, + ignoreAttrs []string, logOnOverwrite bool, meta metadata.Metadata, extras ...string, +) (labels.Labels, error) { resourceAttrs := resource.Attributes() serviceName, haveServiceName := resourceAttrs.Get(conventions.AttributeServiceName) instance, haveInstanceID := resourceAttrs.Get(conventions.AttributeServiceInstanceID) - promotedAttrs := settings.PromoteResourceAttributes.promotedAttributes(resourceAttrs) - promoteScope := settings.PromoteScopeMetadata && scope.name != "" - scopeLabelCount := 0 - if promoteScope { - // Include name, version and schema URL. - scopeLabelCount = scope.attributes.Len() + 3 - } - - // Calculate the maximum possible number of labels we could return so we can preallocate l. - maxLabelCount := attributes.Len() + len(settings.ExternalLabels) + len(promotedAttrs) + scopeLabelCount + len(extras)/2 - - if haveServiceName { - maxLabelCount++ - } - if haveInstanceID { - maxLabelCount++ - } - if settings.EnableTypeAndUnitLabels { - maxLabelCount += 2 - } // Ensure attributes are sorted by key for consistent merging of keys which // collide when sanitized. - labels := make([]prompb.Label, 0, maxLabelCount) + c.scratchBuilder.Reset() + // XXX: Should we always drop service namespace/service name/service instance ID from the labels // (as they get mapped to other Prometheus labels)? attributes.Range(func(key string, value pcommon.Value) bool { if !slices.Contains(ignoreAttrs, key) { - labels = append(labels, prompb.Label{Name: key, Value: value.AsString()}) + c.scratchBuilder.Add(key, value.AsString()) } return true }) - sort.Stable(ByLabelName(labels)) + c.scratchBuilder.Sort() + sortedLabels := c.scratchBuilder.Labels() - // map ensures no duplicate label names. - l := make(map[string]string, maxLabelCount) labelNamer := otlptranslator.LabelNamer{UTF8Allowed: settings.AllowUTF8} - for _, label := range labels { - finalKey, err := labelNamer.Build(label.Name) - if err != nil { - return nil, err - } - if existingValue, alreadyExists := l[finalKey]; alreadyExists { - l[finalKey] = existingValue + ";" + label.Value - } else { - l[finalKey] = label.Value - } - } - for _, lbl := range promotedAttrs { - normalized, err := labelNamer.Build(lbl.Name) - if err != nil { - return nil, err - } - if _, exists := l[normalized]; !exists { - l[normalized] = lbl.Value - } - } - if promoteScope { - var rangeErr error - scope.attributes.Range(func(k string, v pcommon.Value) bool { - name, err := labelNamer.Build("otel_scope_" + k) - if err != nil { - rangeErr = err - return false + if settings.AllowUTF8 { + // UTF8 is allowed, so conflicts aren't possible. + c.builder.Reset(sortedLabels) + } else { + // Now that we have sorted and filtered the labels, build the actual list + // of labels, and handle conflicts by appending values. + c.builder.Reset(labels.EmptyLabels()) + var sortErr error + sortedLabels.Range(func(l modelLabels.Label) { + finalKey, err := labelNamer.Build(l.Name) + if err != nil && sortErr == nil { + sortErr = err + return + } + if existingValue := c.builder.Get(finalKey); existingValue != "" { + c.builder.Set(finalKey, existingValue+";"+l.Value) + } else { + c.builder.Set(finalKey, l.Value) } - l[name] = v.AsString() - return true }) - if rangeErr != nil { - return nil, rangeErr + if sortErr != nil { + return nil, sortErr } - // Scope Name, Version and Schema URL are added after attributes to ensure they are not overwritten by attributes. - l["otel_scope_name"] = scope.name - l["otel_scope_version"] = scope.version - l["otel_scope_schema_url"] = scope.schemaURL } if settings.EnableTypeAndUnitLabels { unitNamer := otlptranslator.UnitNamer{UTF8Allowed: settings.AllowUTF8} - if metadata.Type != prompb.MetricMetadata_UNKNOWN { - l["__type__"] = strings.ToLower(metadata.Type.String()) + if meta.Type != model.MetricTypeUnknown { + c.builder.Set("__type__", strings.ToLower(string(meta.Type))) } - if metadata.Unit != "" { - l["__unit__"] = unitNamer.Build(metadata.Unit) + if meta.Unit != "" { + c.builder.Set("__unit__", unitNamer.Build(meta.Unit)) } } + err := settings.PromoteResourceAttributes.addPromotedAttributes(c.builder, resourceAttrs, settings.AllowUTF8) + if err != nil { + return nil, err + } + if promoteScope { + // Scope Name, Version and Schema URL are added after attributes to ensure they are not overwritten by attributes. + c.builder.Set("otel_scope_name", scope.name) + c.builder.Set("otel_scope_version", scope.version) + c.builder.Set("otel_scope_schema_url", scope.schemaURL) + var scopeErr error + scope.attributes.Range(func(k string, v pcommon.Value) bool { + name := "otel_scope_" + k + var err error + name, err = labelNamer.Build(name) + if err != nil && scopeErr == nil { + scopeErr = err + return false + } + c.builder.Set(name, v.AsString()) + return true + }) + if scopeErr != nil { + return nil, scopeErr + } + } // Map service.name + service.namespace to job. if haveServiceName { val := serviceName.AsString() if serviceNamespace, ok := resourceAttrs.Get(conventions.AttributeServiceNamespace); ok { val = fmt.Sprintf("%s/%s", serviceNamespace.AsString(), val) } - l[model.JobLabel] = val + c.builder.Set(model.JobLabel, val) } // Map service.instance.id to instance. if haveInstanceID { - l[model.InstanceLabel] = instance.AsString() + c.builder.Set(model.InstanceLabel, instance.AsString()) } for key, value := range settings.ExternalLabels { // External labels have already been sanitized. - if _, alreadyExists := l[key]; alreadyExists { + if existingValue := c.builder.Get(key); existingValue != "" { // Skip external labels if they are overridden by metric attributes. continue } - l[key] = value + c.builder.Set(key, value) } for i := 0; i < len(extras); i += 2 { @@ -241,8 +181,7 @@ func createAttributes(resource pcommon.Resource, attributes pcommon.Map, scope s } name := extras[i] - _, found := l[name] - if found && logOnOverwrite { + if existingValue := c.builder.Get(name); existingValue != "" && logOnOverwrite { log.Println("label " + name + " is overwritten. Check if Prometheus reserved labels are used.") } // internal labels should be maintained. @@ -253,15 +192,10 @@ func createAttributes(resource pcommon.Resource, attributes pcommon.Map, scope s return nil, err } } - l[name] = extras[i+1] + c.builder.Set(name, extras[i+1]) } - labels = labels[:0] - for k, v := range l { - labels = append(labels, prompb.Label{Name: k, Value: v}) - } - - return labels, nil + return c.builder.Labels(), nil } func aggregationTemporality(metric pmetric.Metric) (pmetric.AggregationTemporality, bool, error) { @@ -287,7 +221,7 @@ func aggregationTemporality(metric pmetric.Metric) (pmetric.AggregationTemporali // However, work is under way to resolve this shortcoming through a feature called native histograms custom buckets: // https://github.com/prometheus/prometheus/issues/13485. func (c *PrometheusConverter) addHistogramDataPoints(ctx context.Context, dataPoints pmetric.HistogramDataPointSlice, - resource pcommon.Resource, settings Settings, metadata prompb.MetricMetadata, scope scope, + resource pcommon.Resource, settings Settings, baseName string, scope scope, meta metadata.Metadata, ) error { for x := 0; x < dataPoints.Len(); x++ { if err := c.everyN.checkContext(ctx); err != nil { @@ -296,7 +230,8 @@ func (c *PrometheusConverter) addHistogramDataPoints(ctx context.Context, dataPo pt := dataPoints.At(x) timestamp := convertTimeStamp(pt.Timestamp()) - baseLabels, err := createAttributes(resource, pt.Attributes(), scope, settings, nil, false, metadata) + startTimestamp := convertTimeStamp(pt.StartTimestamp()) + baseLabels, err := c.createAttributes(resource, pt.Attributes(), scope, settings, nil, false, meta) if err != nil { return err } @@ -305,35 +240,36 @@ func (c *PrometheusConverter) addHistogramDataPoints(ctx context.Context, dataPo // omitted if pt.HasSum() { // treat sum as a sample in an individual TimeSeries - sum := &prompb.Sample{ - Value: pt.Sum(), - Timestamp: timestamp, - } + val := pt.Sum() if pt.Flags().NoRecordedValue() { - sum.Value = math.Float64frombits(value.StaleNaN) + val = math.Float64frombits(value.StaleNaN) } - sumlabels := createLabels(metadata.MetricFamilyName+sumStr, baseLabels) - c.addSample(sum, sumlabels) + sumlabels := c.addLabels(baseName+sumStr, baseLabels) + if err := c.appender.AppendSample(baseName, sumlabels, meta, timestamp, startTimestamp, val, nil); err != nil { + return err + } } // treat count as a sample in an individual TimeSeries - count := &prompb.Sample{ - Value: float64(pt.Count()), - Timestamp: timestamp, - } + val := float64(pt.Count()) if pt.Flags().NoRecordedValue() { - count.Value = math.Float64frombits(value.StaleNaN) + val = math.Float64frombits(value.StaleNaN) } - countlabels := createLabels(metadata.MetricFamilyName+countStr, baseLabels) - c.addSample(count, countlabels) + countlabels := c.addLabels(baseName+countStr, baseLabels) + if err := c.appender.AppendSample(baseName, countlabels, meta, timestamp, startTimestamp, val, nil); err != nil { + return err + } + exemplars, err := c.getPromExemplars(ctx, pt.Exemplars()) + if err != nil { + return err + } + nextExemplarIdx := 0 // cumulative count for conversion to cumulative histogram var cumulativeCount uint64 - var bucketBounds []bucketBoundsData - // process each bound, based on histograms proto definition, # of buckets = # of explicit bounds + 1 for i := 0; i < pt.ExplicitBounds().Len() && i < pt.BucketCounts().Len(); i++ { if err := c.everyN.checkContext(ctx); err != nil { @@ -342,116 +278,110 @@ func (c *PrometheusConverter) addHistogramDataPoints(ctx context.Context, dataPo bound := pt.ExplicitBounds().At(i) cumulativeCount += pt.BucketCounts().At(i) - bucket := &prompb.Sample{ - Value: float64(cumulativeCount), - Timestamp: timestamp, + + // Find exemplars that belong to this bucket. Both exemplars and + // buckets are sorted in ascending order. + var currentBucketExemplars []exemplar.Exemplar + for ; nextExemplarIdx < len(exemplars); nextExemplarIdx++ { + ex := exemplars[nextExemplarIdx] + if ex.Value > bound { + // This exemplar belongs in a higher bucket. + break + } + currentBucketExemplars = append(currentBucketExemplars, ex) } + val := float64(cumulativeCount) if pt.Flags().NoRecordedValue() { - bucket.Value = math.Float64frombits(value.StaleNaN) + val = math.Float64frombits(value.StaleNaN) } boundStr := strconv.FormatFloat(bound, 'f', -1, 64) - labels := createLabels(metadata.MetricFamilyName+bucketStr, baseLabels, leStr, boundStr) - ts := c.addSample(bucket, labels) - - bucketBounds = append(bucketBounds, bucketBoundsData{ts: ts, bound: bound}) + labels := c.addLabels(baseName+bucketStr, baseLabels, leStr, boundStr) + if err := c.appender.AppendSample(baseName, labels, meta, timestamp, startTimestamp, val, currentBucketExemplars); err != nil { + return err + } } // add le=+Inf bucket - infBucket := &prompb.Sample{ - Timestamp: timestamp, - } + val = float64(pt.Count()) if pt.Flags().NoRecordedValue() { - infBucket.Value = math.Float64frombits(value.StaleNaN) - } else { - infBucket.Value = float64(pt.Count()) + val = math.Float64frombits(value.StaleNaN) } - infLabels := createLabels(metadata.MetricFamilyName+bucketStr, baseLabels, leStr, pInfStr) - ts := c.addSample(infBucket, infLabels) - - bucketBounds = append(bucketBounds, bucketBoundsData{ts: ts, bound: math.Inf(1)}) - if err := c.addExemplars(ctx, pt, bucketBounds); err != nil { + infLabels := c.addLabels(baseName+bucketStr, baseLabels, leStr, pInfStr) + if err := c.appender.AppendSample(baseName, infLabels, meta, timestamp, startTimestamp, val, exemplars[nextExemplarIdx:]); err != nil { return err } - startTimestamp := pt.StartTimestamp() - if settings.ExportCreatedMetric && startTimestamp != 0 { - labels := createLabels(metadata.MetricFamilyName+createdSuffix, baseLabels) - c.addTimeSeriesIfNeeded(labels, startTimestamp, pt.Timestamp()) + if settings.ExportCreatedMetric && pt.StartTimestamp() != 0 { + labels := c.addLabels(baseName+createdSuffix, baseLabels) + if c.timeSeriesIsNew(labels) { + if err := c.appender.AppendSample(baseName, labels, meta, timestamp, 0, float64(startTimestamp), nil); err != nil { + return err + } + } } } return nil } -type exemplarType interface { - pmetric.ExponentialHistogramDataPoint | pmetric.HistogramDataPoint | pmetric.NumberDataPoint - Exemplars() pmetric.ExemplarSlice -} - -func getPromExemplars[T exemplarType](ctx context.Context, everyN *everyNTimes, pt T) ([]prompb.Exemplar, error) { - promExemplars := make([]prompb.Exemplar, 0, pt.Exemplars().Len()) - for i := 0; i < pt.Exemplars().Len(); i++ { - if err := everyN.checkContext(ctx); err != nil { +func (c *PrometheusConverter) getPromExemplars(ctx context.Context, exemplars pmetric.ExemplarSlice) ([]exemplar.Exemplar, error) { + if exemplars.Len() == 0 { + return nil, nil + } + outputExemplars := make([]exemplar.Exemplar, 0, exemplars.Len()) + for i := 0; i < exemplars.Len(); i++ { + if err := c.everyN.checkContext(ctx); err != nil { return nil, err } - exemplar := pt.Exemplars().At(i) + ex := exemplars.At(i) exemplarRunes := 0 - promExemplar := prompb.Exemplar{ - Timestamp: timestamp.FromTime(exemplar.Timestamp().AsTime()), + ts := timestamp.FromTime(ex.Timestamp().AsTime()) + newExemplar := exemplar.Exemplar{ + Ts: ts, + HasTs: ts != 0, } - switch exemplar.ValueType() { + c.scratchBuilder.Reset() + switch ex.ValueType() { case pmetric.ExemplarValueTypeInt: - promExemplar.Value = float64(exemplar.IntValue()) + newExemplar.Value = float64(ex.IntValue()) case pmetric.ExemplarValueTypeDouble: - promExemplar.Value = exemplar.DoubleValue() + newExemplar.Value = ex.DoubleValue() default: - return nil, fmt.Errorf("unsupported exemplar value type: %v", exemplar.ValueType()) + return nil, fmt.Errorf("unsupported exemplar value type: %v", ex.ValueType()) } - if traceID := exemplar.TraceID(); !traceID.IsEmpty() { + if traceID := ex.TraceID(); !traceID.IsEmpty() { val := hex.EncodeToString(traceID[:]) exemplarRunes += utf8.RuneCountInString(traceIDKey) + utf8.RuneCountInString(val) - promLabel := prompb.Label{ - Name: traceIDKey, - Value: val, - } - promExemplar.Labels = append(promExemplar.Labels, promLabel) + c.scratchBuilder.Add(traceIDKey, val) } - if spanID := exemplar.SpanID(); !spanID.IsEmpty() { + if spanID := ex.SpanID(); !spanID.IsEmpty() { val := hex.EncodeToString(spanID[:]) exemplarRunes += utf8.RuneCountInString(spanIDKey) + utf8.RuneCountInString(val) - promLabel := prompb.Label{ - Name: spanIDKey, - Value: val, - } - promExemplar.Labels = append(promExemplar.Labels, promLabel) + c.scratchBuilder.Add(spanIDKey, val) } - attrs := exemplar.FilteredAttributes() - labelsFromAttributes := make([]prompb.Label, 0, attrs.Len()) + attrs := ex.FilteredAttributes() attrs.Range(func(key string, value pcommon.Value) bool { - val := value.AsString() - exemplarRunes += utf8.RuneCountInString(key) + utf8.RuneCountInString(val) - promLabel := prompb.Label{ - Name: key, - Value: val, - } - - labelsFromAttributes = append(labelsFromAttributes, promLabel) - + exemplarRunes += utf8.RuneCountInString(key) + utf8.RuneCountInString(value.AsString()) return true }) - if exemplarRunes <= maxExemplarRunes { - // only append filtered attributes if it does not cause exemplar - // labels to exceed the max number of runes - promExemplar.Labels = append(promExemplar.Labels, labelsFromAttributes...) - } - promExemplars = append(promExemplars, promExemplar) + // Only append filtered attributes if it does not cause exemplar + // labels to exceed the max number of runes. + if exemplarRunes <= maxExemplarRunes { + attrs.Range(func(key string, value pcommon.Value) bool { + c.scratchBuilder.Add(key, value.AsString()) + return true + }) + } + c.scratchBuilder.Sort() + newExemplar.Labels = modelLabels.NewFromSorted(c.scratchBuilder.Labels()) + outputExemplars = append(outputExemplars, newExemplar) } - return promExemplars, nil + return outputExemplars, nil } // findMinAndMaxTimestamps returns the minimum of minTimestamp and the earliest timestamp in metric and @@ -500,7 +430,7 @@ func findMinAndMaxTimestamps(metric pmetric.Metric, minTimestamp, maxTimestamp p } func (c *PrometheusConverter) addSummaryDataPoints(ctx context.Context, dataPoints pmetric.SummaryDataPointSlice, resource pcommon.Resource, - settings Settings, metadata prompb.MetricMetadata, scope scope, + settings Settings, baseName string, scope scope, meta metadata.Metadata, ) error { for x := 0; x < dataPoints.Len(); x++ { if err := c.everyN.checkContext(ctx); err != nil { @@ -509,75 +439,73 @@ func (c *PrometheusConverter) addSummaryDataPoints(ctx context.Context, dataPoin pt := dataPoints.At(x) timestamp := convertTimeStamp(pt.Timestamp()) - baseLabels, err := createAttributes(resource, pt.Attributes(), scope, settings, nil, false, metadata) + startTimestamp := convertTimeStamp(pt.StartTimestamp()) + baseLabels, err := c.createAttributes(resource, pt.Attributes(), scope, settings, nil, false, meta) if err != nil { return err } // treat sum as a sample in an individual TimeSeries - sum := &prompb.Sample{ - Value: pt.Sum(), - Timestamp: timestamp, - } + val := pt.Sum() if pt.Flags().NoRecordedValue() { - sum.Value = math.Float64frombits(value.StaleNaN) + val = math.Float64frombits(value.StaleNaN) } // sum and count of the summary should append suffix to baseName - sumlabels := createLabels(metadata.MetricFamilyName+sumStr, baseLabels) - c.addSample(sum, sumlabels) + sumlabels := c.addLabels(baseName+sumStr, baseLabels) + if err := c.appender.AppendSample(baseName, sumlabels, meta, timestamp, startTimestamp, val, nil); err != nil { + return err + } // treat count as a sample in an individual TimeSeries - count := &prompb.Sample{ - Value: float64(pt.Count()), - Timestamp: timestamp, - } + val = float64(pt.Count()) if pt.Flags().NoRecordedValue() { - count.Value = math.Float64frombits(value.StaleNaN) + val = math.Float64frombits(value.StaleNaN) + } + countlabels := c.addLabels(baseName+countStr, baseLabels) + if err := c.appender.AppendSample(baseName, countlabels, meta, timestamp, startTimestamp, val, nil); err != nil { + return err } - countlabels := createLabels(metadata.MetricFamilyName+countStr, baseLabels) - c.addSample(count, countlabels) // process each percentile/quantile for i := 0; i < pt.QuantileValues().Len(); i++ { qt := pt.QuantileValues().At(i) - quantile := &prompb.Sample{ - Value: qt.Value(), - Timestamp: timestamp, - } + val = qt.Value() if pt.Flags().NoRecordedValue() { - quantile.Value = math.Float64frombits(value.StaleNaN) + val = math.Float64frombits(value.StaleNaN) } percentileStr := strconv.FormatFloat(qt.Quantile(), 'f', -1, 64) - qtlabels := createLabels(metadata.MetricFamilyName, baseLabels, quantileStr, percentileStr) - c.addSample(quantile, qtlabels) + qtlabels := c.addLabels(baseName, baseLabels, quantileStr, percentileStr) + if err := c.appender.AppendSample(baseName, qtlabels, meta, timestamp, startTimestamp, val, nil); err != nil { + return err + } } - startTimestamp := pt.StartTimestamp() - if settings.ExportCreatedMetric && startTimestamp != 0 { - createdLabels := createLabels(metadata.MetricFamilyName+createdSuffix, baseLabels) - c.addTimeSeriesIfNeeded(createdLabels, startTimestamp, pt.Timestamp()) + if settings.ExportCreatedMetric && pt.StartTimestamp() != 0 { + createdLabels := c.addLabels(baseName+createdSuffix, baseLabels) + if c.timeSeriesIsNew(createdLabels) { + if err := c.appender.AppendSample(baseName, createdLabels, meta, timestamp, 0, float64(startTimestamp), nil); err != nil { + return err + } + } } } return nil } -// createLabels returns a copy of baseLabels, adding to it the pair model.MetricNameLabel=name. +// addLabels returns a copy of baseLabels, adding to it the pair model.MetricNameLabel=name. // If extras are provided, corresponding label pairs are also added to the returned slice. // If extras is uneven length, the last (unpaired) extra will be ignored. -func createLabels(name string, baseLabels []prompb.Label, extras ...string) []prompb.Label { - extraLabelCount := len(extras) / 2 - labels := make([]prompb.Label, len(baseLabels), len(baseLabels)+extraLabelCount+1) // +1 for name - copy(labels, baseLabels) +func (c *PrometheusConverter) addLabels(name string, baseLabels labels.Labels, extras ...string) labels.Labels { + c.builder.Reset(baseLabels) n := len(extras) n -= n % 2 for extrasIdx := 0; extrasIdx < n; extrasIdx += 2 { - labels = append(labels, prompb.Label{Name: extras[extrasIdx], Value: extras[extrasIdx+1]}) + c.builder.Set(extras[extrasIdx], extras[extrasIdx+1]) } - - labels = append(labels, prompb.Label{Name: model.MetricNameLabel, Value: name}) - return labels + c.builder.Set(model.MetricNameLabel, name) + return c.builder.Labels() } // addTypeAndUnitLabels appends type and unit labels to the given labels slice. @@ -596,57 +524,35 @@ func addTypeAndUnitLabels(labels []prompb.Label, metadata prompb.MetricMetadata, // getOrCreateTimeSeries returns the time series corresponding to the label set if existent, and false. // Otherwise it creates a new one and returns that, and true. -func (c *PrometheusConverter) getOrCreateTimeSeries(lbls []prompb.Label) (*prompb.TimeSeries, bool) { - h := timeSeriesSignature(lbls) - ts := c.unique[h] - if ts != nil { - if isSameMetric(ts, lbls) { +func (c *PrometheusConverter) timeSeriesIsNew(lbls labels.Labels) bool { + h := lbls.Hash() + uLabels, ok := c.unique[h] + if ok { + if labels.Equal(uLabels, lbls) { // We already have this metric - return ts, false + return false } // Look for a matching conflict - for _, cTS := range c.conflicts[h] { - if isSameMetric(cTS, lbls) { + for _, cLabels := range c.conflicts[h] { + if labels.Equal(cLabels, lbls) { // We already have this metric - return cTS, false + return false } } // New conflict - ts = &prompb.TimeSeries{ - Labels: lbls, - } - c.conflicts[h] = append(c.conflicts[h], ts) - return ts, true + c.conflicts[h] = append(c.conflicts[h], uLabels) + return true } // This metric is new - ts = &prompb.TimeSeries{ - Labels: lbls, - } - c.unique[h] = ts - return ts, true -} - -// addTimeSeriesIfNeeded adds a corresponding time series if it doesn't already exist. -// If the time series doesn't already exist, it gets added with startTimestamp for its value and timestamp for its timestamp, -// both converted to milliseconds. -func (c *PrometheusConverter) addTimeSeriesIfNeeded(lbls []prompb.Label, startTimestamp, timestamp pcommon.Timestamp) { - ts, created := c.getOrCreateTimeSeries(lbls) - if created { - ts.Samples = []prompb.Sample{ - { - // convert ns to ms - Value: float64(convertTimeStamp(startTimestamp)), - Timestamp: convertTimeStamp(timestamp), - }, - } - } + c.unique[h] = uLabels + return true } // addResourceTargetInfo converts the resource to the target info metric. -func addResourceTargetInfo(resource pcommon.Resource, settings Settings, earliestTimestamp, latestTimestamp time.Time, converter *PrometheusConverter) error { +func (c *PrometheusConverter) addResourceTargetInfo(resource pcommon.Resource, settings Settings, earliestTimestamp, latestTimestamp time.Time) error { if settings.DisableTargetInfo { return nil } @@ -679,17 +585,21 @@ func addResourceTargetInfo(resource pcommon.Resource, settings Settings, earlies // Do not pass identifying attributes as ignoreAttrs below. identifyingAttrs = nil } - labels, err := createAttributes(resource, attributes, scope{}, settings, identifyingAttrs, false, prompb.MetricMetadata{}, model.MetricNameLabel, name) + meta := metadata.Metadata{ + Type: model.MetricTypeGauge, + Help: "Target metadata", + } + // TODO: should target info have the __type__ metadata label? + lbls, err := c.createAttributes(resource, attributes, scope{}, settings, identifyingAttrs, false, metadata.Metadata{}, model.MetricNameLabel, name) if err != nil { return err } haveIdentifier := false - for _, l := range labels { + lbls.Range(func(l modelLabels.Label) { if l.Name == model.JobLabel || l.Name == model.InstanceLabel { haveIdentifier = true - break } - } + }) if !haveIdentifier { // We need at least one identifying label to generate target_info. @@ -704,18 +614,13 @@ func addResourceTargetInfo(resource pcommon.Resource, settings Settings, earlies settings.LookbackDelta = defaultLookbackDelta } interval := settings.LookbackDelta / 2 - ts, _ := converter.getOrCreateTimeSeries(labels) - for timestamp := earliestTimestamp; timestamp.Before(latestTimestamp); timestamp = timestamp.Add(interval) { - ts.Samples = append(ts.Samples, prompb.Sample{ - Value: float64(1), - Timestamp: timestamp.UnixMilli(), - }) + timestamp := earliestTimestamp + for ; timestamp.Before(latestTimestamp); timestamp = timestamp.Add(interval) { + if err := c.appender.AppendSample(name, lbls, meta, timestamp.UnixMilli(), 0, float64(1), nil); err != nil { + return err + } } - ts.Samples = append(ts.Samples, prompb.Sample{ - Value: float64(1), - Timestamp: latestTimestamp.UnixMilli(), - }) - return nil + return c.appender.AppendSample(name, lbls, meta, latestTimestamp.UnixMilli(), 0, float64(1), nil) } // convertTimeStamp converts OTLP timestamp in ns to timestamp in ms. diff --git a/storage/remote/otlptranslator/prometheusremotewrite/helper_test.go b/storage/remote/otlptranslator/prometheusremotewrite/helper_test.go index 9ab37e26fd..cd2478c131 100644 --- a/storage/remote/otlptranslator/prometheusremotewrite/helper_test.go +++ b/storage/remote/otlptranslator/prometheusremotewrite/helper_test.go @@ -27,7 +27,9 @@ import ( "go.opentelemetry.io/collector/pdata/pmetric" "github.com/prometheus/prometheus/config" + "github.com/prometheus/prometheus/model/metadata" "github.com/prometheus/prometheus/prompb" + "github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite/labels" "github.com/prometheus/prometheus/util/testutil" ) @@ -71,134 +73,62 @@ func TestCreateAttributes(t *testing.T) { promoteScope bool ignoreResourceAttributes []string ignoreAttrs []string - expectedLabels []prompb.Label + expectedLabels labels.Labels }{ { name: "Successful conversion without resource attribute promotion and without scope promotion", scope: defaultScope, promoteScope: false, promoteResourceAttributes: nil, - expectedLabels: []prompb.Label{ - { - Name: "__name__", - Value: "test_metric", - }, - { - Name: "instance", - Value: "service ID", - }, - { - Name: "job", - Value: "service name", - }, - { - Name: "metric_attr", - Value: "metric value", - }, - { - Name: "metric_attr_other", - Value: "metric value other", - }, - }, + expectedLabels: labels.FromStrings( + "__name__", "test_metric", + "instance", "service ID", + "job", "service name", + "metric_attr", "metric value", + "metric_attr_other", "metric value other", + ), }, { name: "Successful conversion without resource attribute promotion and with scope promotion", scope: defaultScope, promoteScope: true, promoteResourceAttributes: nil, - expectedLabels: []prompb.Label{ - { - Name: "__name__", - Value: "test_metric", - }, - { - Name: "instance", - Value: "service ID", - }, - { - Name: "job", - Value: "service name", - }, - { - Name: "metric_attr", - Value: "metric value", - }, - { - Name: "metric_attr_other", - Value: "metric value other", - }, - { - Name: "otel_scope_name", - Value: defaultScope.name, - }, - { - Name: "otel_scope_schema_url", - Value: defaultScope.schemaURL, - }, - { - Name: "otel_scope_version", - Value: defaultScope.version, - }, - { - Name: "otel_scope_attr1", - Value: "value1", - }, - { - Name: "otel_scope_attr2", - Value: "value2", - }, - }, + expectedLabels: labels.FromStrings( + "__name__", "test_metric", + "instance", "service ID", + "job", "service name", + "metric_attr", "metric value", + "metric_attr_other", "metric value other", + "otel_scope_name", defaultScope.name, + "otel_scope_schema_url", defaultScope.schemaURL, + "otel_scope_version", defaultScope.version, + "otel_scope_attr1", "value1", + "otel_scope_attr2", "value2", + ), }, { name: "Successful conversion without resource attribute promotion and with scope promotion, but without scope", scope: scope{}, promoteResourceAttributes: nil, promoteScope: true, - expectedLabels: []prompb.Label{ - { - Name: "__name__", - Value: "test_metric", - }, - { - Name: "instance", - Value: "service ID", - }, - { - Name: "job", - Value: "service name", - }, - { - Name: "metric_attr", - Value: "metric value", - }, - { - Name: "metric_attr_other", - Value: "metric value other", - }, - }, + expectedLabels: labels.FromStrings( + "__name__", "test_metric", + "instance", "service ID", + "job", "service name", + "metric_attr", "metric value", + "metric_attr_other", "metric value other", + ), }, { name: "Successful conversion with some attributes ignored", promoteResourceAttributes: nil, ignoreAttrs: []string{"metric-attr-other"}, - expectedLabels: []prompb.Label{ - { - Name: "__name__", - Value: "test_metric", - }, - { - Name: "instance", - Value: "service ID", - }, - { - Name: "job", - Value: "service name", - }, - { - Name: "metric_attr", - Value: "metric value", - }, - }, + expectedLabels: labels.FromStrings( + "__name__", "test_metric", + "instance", "service ID", + "job", "service name", + "metric_attr", "metric value", + ), }, { name: "Successful conversion with some attributes ignored and with scope promotion", @@ -206,260 +136,95 @@ func TestCreateAttributes(t *testing.T) { promoteScope: true, promoteResourceAttributes: nil, ignoreAttrs: []string{"metric-attr-other"}, - expectedLabels: []prompb.Label{ - { - Name: "__name__", - Value: "test_metric", - }, - { - Name: "instance", - Value: "service ID", - }, - { - Name: "job", - Value: "service name", - }, - { - Name: "metric_attr", - Value: "metric value", - }, - { - Name: "otel_scope_name", - Value: defaultScope.name, - }, - { - Name: "otel_scope_version", - Value: defaultScope.version, - }, - { - Name: "otel_scope_schema_url", - Value: defaultScope.schemaURL, - }, - { - Name: "otel_scope_attr1", - Value: "value1", - }, - { - Name: "otel_scope_attr2", - Value: "value2", - }, - }, + expectedLabels: labels.FromStrings( + "__name__", "test_metric", + "instance", "service ID", + "job", "service name", + "metric_attr", "metric value", + "otel_scope_name", defaultScope.name, + "otel_scope_schema_url", defaultScope.schemaURL, + "otel_scope_version", defaultScope.version, + "otel_scope_attr1", "value1", + "otel_scope_attr2", "value2", + ), }, { name: "Successful conversion with resource attribute promotion and with scope promotion", scope: defaultScope, promoteResourceAttributes: []string{"non-existent-attr", "existent-attr"}, promoteScope: true, - expectedLabels: []prompb.Label{ - { - Name: "__name__", - Value: "test_metric", - }, - { - Name: "instance", - Value: "service ID", - }, - { - Name: "job", - Value: "service name", - }, - { - Name: "metric_attr", - Value: "metric value", - }, - { - Name: "metric_attr_other", - Value: "metric value other", - }, - { - Name: "existent_attr", - Value: "resource value", - }, - { - Name: "otel_scope_name", - Value: defaultScope.name, - }, - { - Name: "otel_scope_version", - Value: defaultScope.version, - }, - { - Name: "otel_scope_schema_url", - Value: defaultScope.schemaURL, - }, - { - Name: "otel_scope_attr1", - Value: "value1", - }, - { - Name: "otel_scope_attr2", - Value: "value2", - }, - }, + expectedLabels: labels.FromStrings( + "__name__", "test_metric", + "instance", "service ID", + "job", "service name", + "metric_attr", "metric value", + "metric_attr_other", "metric value other", + "existent_attr", "resource value", + "otel_scope_name", defaultScope.name, + "otel_scope_schema_url", defaultScope.schemaURL, + "otel_scope_version", defaultScope.version, + "otel_scope_attr1", "value1", + "otel_scope_attr2", "value2", + ), }, { name: "Successful conversion with resource attribute promotion and with scope promotion, conflicting resource attributes are ignored", scope: defaultScope, promoteScope: true, promoteResourceAttributes: []string{"non-existent-attr", "existent-attr", "metric-attr", "job", "instance"}, - expectedLabels: []prompb.Label{ - { - Name: "__name__", - Value: "test_metric", - }, - { - Name: "instance", - Value: "service ID", - }, - { - Name: "job", - Value: "service name", - }, - { - Name: "existent_attr", - Value: "resource value", - }, - { - Name: "metric_attr", - Value: "metric value", - }, - { - Name: "metric_attr_other", - Value: "metric value other", - }, - { - Name: "otel_scope_name", - Value: defaultScope.name, - }, - { - Name: "otel_scope_version", - Value: defaultScope.version, - }, - { - Name: "otel_scope_schema_url", - Value: defaultScope.schemaURL, - }, - { - Name: "otel_scope_attr1", - Value: "value1", - }, - { - Name: "otel_scope_attr2", - Value: "value2", - }, - }, + expectedLabels: labels.FromStrings( + "__name__", "test_metric", + "instance", "service ID", + "job", "service name", + "existent_attr", "resource value", + "metric_attr", "metric value", + "metric_attr_other", "metric value other", + "otel_scope_name", defaultScope.name, + "otel_scope_schema_url", defaultScope.schemaURL, + "otel_scope_version", defaultScope.version, + "otel_scope_attr1", "value1", + "otel_scope_attr2", "value2", + ), }, { name: "Successful conversion with resource attribute promotion and with scope promotion, attributes are only promoted once", scope: defaultScope, promoteScope: true, promoteResourceAttributes: []string{"existent-attr", "existent-attr"}, - expectedLabels: []prompb.Label{ - { - Name: "__name__", - Value: "test_metric", - }, - { - Name: "instance", - Value: "service ID", - }, - { - Name: "job", - Value: "service name", - }, - { - Name: "existent_attr", - Value: "resource value", - }, - { - Name: "metric_attr", - Value: "metric value", - }, - { - Name: "metric_attr_other", - Value: "metric value other", - }, - { - Name: "otel_scope_name", - Value: defaultScope.name, - }, - { - Name: "otel_scope_version", - Value: defaultScope.version, - }, - { - Name: "otel_scope_schema_url", - Value: defaultScope.schemaURL, - }, - { - Name: "otel_scope_attr1", - Value: "value1", - }, - { - Name: "otel_scope_attr2", - Value: "value2", - }, - }, + expectedLabels: labels.FromStrings( + "__name__", "test_metric", + "instance", "service ID", + "job", "service name", + "existent_attr", "resource value", + "metric_attr", "metric value", + "metric_attr_other", "metric value other", + "otel_scope_name", defaultScope.name, + "otel_scope_schema_url", defaultScope.schemaURL, + "otel_scope_version", defaultScope.version, + "otel_scope_attr1", "value1", + "otel_scope_attr2", "value2", + ), }, { name: "Successful conversion promoting all resource attributes and with scope promotion", scope: defaultScope, promoteAllResourceAttributes: true, promoteScope: true, - expectedLabels: []prompb.Label{ - { - Name: "__name__", - Value: "test_metric", - }, - { - Name: "instance", - Value: "service ID", - }, - { - Name: "job", - Value: "service name", - }, - { - Name: "existent_attr", - Value: "resource value", - }, - { - Name: "metric_attr", - Value: "metric value", - }, - { - Name: "metric_attr_other", - Value: "metric value other", - }, - { - Name: "service_name", - Value: "service name", - }, - { - Name: "service_instance_id", - Value: "service ID", - }, - { - Name: "otel_scope_name", - Value: defaultScope.name, - }, - { - Name: "otel_scope_version", - Value: defaultScope.version, - }, - { - Name: "otel_scope_schema_url", - Value: defaultScope.schemaURL, - }, - { - Name: "otel_scope_attr1", - Value: "value1", - }, - { - Name: "otel_scope_attr2", - Value: "value2", - }, - }, + expectedLabels: labels.FromStrings( + "__name__", "test_metric", + "instance", "service ID", + "job", "service name", + "existent_attr", "resource value", + "metric_attr", "metric value", + "metric_attr_other", "metric value other", + "service_name", "service name", + "service_instance_id", "service ID", + "otel_scope_name", defaultScope.name, + "otel_scope_schema_url", defaultScope.schemaURL, + "otel_scope_version", defaultScope.version, + "otel_scope_attr1", "value1", + "otel_scope_attr2", "value2", + ), }, { name: "Successful conversion promoting all resource attributes and with scope promotion, ignoring 'service.instance.id'", @@ -469,60 +234,25 @@ func TestCreateAttributes(t *testing.T) { ignoreResourceAttributes: []string{ "service.instance.id", }, - expectedLabels: []prompb.Label{ - { - Name: "__name__", - Value: "test_metric", - }, - { - Name: "instance", - Value: "service ID", - }, - { - Name: "job", - Value: "service name", - }, - { - Name: "existent_attr", - Value: "resource value", - }, - { - Name: "metric_attr", - Value: "metric value", - }, - { - Name: "metric_attr_other", - Value: "metric value other", - }, - { - Name: "service_name", - Value: "service name", - }, - { - Name: "otel_scope_name", - Value: defaultScope.name, - }, - { - Name: "otel_scope_version", - Value: defaultScope.version, - }, - { - Name: "otel_scope_schema_url", - Value: defaultScope.schemaURL, - }, - { - Name: "otel_scope_attr1", - Value: "value1", - }, - { - Name: "otel_scope_attr2", - Value: "value2", - }, - }, + expectedLabels: labels.FromStrings( + "__name__", "test_metric", + "instance", "service ID", + "job", "service name", + "existent_attr", "resource value", + "metric_attr", "metric value", + "metric_attr_other", "metric value other", + "service_name", "service name", + "otel_scope_name", defaultScope.name, + "otel_scope_schema_url", defaultScope.schemaURL, + "otel_scope_version", defaultScope.version, + "otel_scope_attr1", "value1", + "otel_scope_attr2", "value2", + ), }, } for _, tc := range testCases { t.Run(tc.name, func(t *testing.T) { + c := NewPrometheusConverter(&mockCombinedAppender{}) settings := Settings{ PromoteResourceAttributes: NewPromoteResourceAttributes(config.OTLPConfig{ PromoteAllResourceAttributes: tc.promoteAllResourceAttributes, @@ -531,10 +261,10 @@ func TestCreateAttributes(t *testing.T) { }), PromoteScopeMetadata: tc.promoteScope, } - lbls, err := createAttributes(resource, attrs, tc.scope, settings, tc.ignoreAttrs, false, prompb.MetricMetadata{}, model.MetricNameLabel, "test_metric") + lbls, err := c.createAttributes(resource, attrs, tc.scope, settings, tc.ignoreAttrs, false, metadata.Metadata{}, model.MetricNameLabel, "test_metric") require.NoError(t, err) - require.ElementsMatch(t, lbls, tc.expectedLabels) + testutil.RequireEqual(t, lbls, tc.expectedLabels) }) } } @@ -575,7 +305,7 @@ func TestPrometheusConverter_AddSummaryDataPoints(t *testing.T) { metric func() pmetric.Metric scope scope promoteScope bool - want func() map[uint64]*prompb.TimeSeries + want func() []combinedSample }{ { name: "summary with start time and without scope promotion", @@ -592,34 +322,33 @@ func TestPrometheusConverter_AddSummaryDataPoints(t *testing.T) { }, scope: defaultScope, promoteScope: false, - want: func() map[uint64]*prompb.TimeSeries { - countLabels := []prompb.Label{ - {Name: model.MetricNameLabel, Value: "test_summary" + countStr}, - } - sumLabels := []prompb.Label{ - {Name: model.MetricNameLabel, Value: "test_summary" + sumStr}, - } - createdLabels := []prompb.Label{ - {Name: model.MetricNameLabel, Value: "test_summary" + createdSuffix}, - } - return map[uint64]*prompb.TimeSeries{ - timeSeriesSignature(countLabels): { - Labels: countLabels, - Samples: []prompb.Sample{ - {Value: 0, Timestamp: convertTimeStamp(ts)}, - }, + want: func() []combinedSample { + return []combinedSample{ + { + metricFamilyName: "test_summary", + ls: labels.FromStrings( + model.MetricNameLabel, "test_summary"+sumStr, + ), + t: convertTimeStamp(ts), + ct: convertTimeStamp(ts), + v: 0, }, - timeSeriesSignature(sumLabels): { - Labels: sumLabels, - Samples: []prompb.Sample{ - {Value: 0, Timestamp: convertTimeStamp(ts)}, - }, + { + metricFamilyName: "test_summary", + ls: labels.FromStrings( + model.MetricNameLabel, "test_summary"+countStr, + ), + t: convertTimeStamp(ts), + ct: convertTimeStamp(ts), + v: 0, }, - timeSeriesSignature(createdLabels): { - Labels: createdLabels, - Samples: []prompb.Sample{ - {Value: float64(convertTimeStamp(ts)), Timestamp: convertTimeStamp(ts)}, - }, + { + metricFamilyName: "test_summary", + ls: labels.FromStrings( + model.MetricNameLabel, "test_summary"+createdSuffix, + ), + t: convertTimeStamp(ts), + v: float64(convertTimeStamp(ts)), }, } }, @@ -639,59 +368,38 @@ func TestPrometheusConverter_AddSummaryDataPoints(t *testing.T) { }, scope: defaultScope, promoteScope: true, - want: func() map[uint64]*prompb.TimeSeries { - scopeLabels := []prompb.Label{ - { - Name: "otel_scope_attr1", - Value: "value1", - }, - { - Name: "otel_scope_attr2", - Value: "value2", - }, - { - Name: "otel_scope_name", - Value: defaultScope.name, - }, - { - Name: "otel_scope_schema_url", - Value: defaultScope.schemaURL, - }, - { - Name: "otel_scope_version", - Value: defaultScope.version, - }, + want: func() []combinedSample { + scopeLabels := []string{ + "otel_scope_attr1", "value1", + "otel_scope_attr2", "value2", + "otel_scope_name", defaultScope.name, + "otel_scope_schema_url", defaultScope.schemaURL, + "otel_scope_version", defaultScope.version, } - countLabels := append([]prompb.Label{ - {Name: model.MetricNameLabel, Value: "test_summary" + countStr}, - }, scopeLabels...) - sumLabels := append([]prompb.Label{ - {Name: model.MetricNameLabel, Value: "test_summary" + sumStr}, - }, scopeLabels...) - createdLabels := append([]prompb.Label{ + return []combinedSample{ { - Name: model.MetricNameLabel, - Value: "test_summary" + createdSuffix, + metricFamilyName: "test_summary", + ls: labels.FromStrings(append(scopeLabels, + model.MetricNameLabel, "test_summary"+sumStr)...), + t: convertTimeStamp(ts), + ct: convertTimeStamp(ts), + v: 0, }, - }, scopeLabels...) - return map[uint64]*prompb.TimeSeries{ - timeSeriesSignature(countLabels): { - Labels: countLabels, - Samples: []prompb.Sample{ - {Value: 0, Timestamp: convertTimeStamp(ts)}, - }, + { + metricFamilyName: "test_summary", + ls: labels.FromStrings(append(scopeLabels, + model.MetricNameLabel, "test_summary"+countStr)...), + t: convertTimeStamp(ts), + ct: convertTimeStamp(ts), + v: 0, }, - timeSeriesSignature(sumLabels): { - Labels: sumLabels, - Samples: []prompb.Sample{ - {Value: 0, Timestamp: convertTimeStamp(ts)}, - }, - }, - timeSeriesSignature(createdLabels): { - Labels: createdLabels, - Samples: []prompb.Sample{ - {Value: float64(convertTimeStamp(ts)), Timestamp: convertTimeStamp(ts)}, - }, + { + metricFamilyName: "test_summary", + ls: labels.FromStrings(append(scopeLabels, + model.MetricNameLabel, "test_summary"+createdSuffix, + )...), + t: convertTimeStamp(ts), + v: float64(convertTimeStamp(ts)), }, } }, @@ -710,25 +418,85 @@ func TestPrometheusConverter_AddSummaryDataPoints(t *testing.T) { }, scope: defaultScope, promoteScope: false, - want: func() map[uint64]*prompb.TimeSeries { - countLabels := []prompb.Label{ - {Name: model.MetricNameLabel, Value: "test_summary" + countStr}, - } - sumLabels := []prompb.Label{ - {Name: model.MetricNameLabel, Value: "test_summary" + sumStr}, - } - return map[uint64]*prompb.TimeSeries{ - timeSeriesSignature(countLabels): { - Labels: countLabels, - Samples: []prompb.Sample{ - {Value: 0, Timestamp: convertTimeStamp(ts)}, - }, + want: func() []combinedSample { + return []combinedSample{ + { + metricFamilyName: "test_summary", + ls: labels.FromStrings( + model.MetricNameLabel, "test_summary"+sumStr, + ), + t: convertTimeStamp(ts), + v: 0, }, - timeSeriesSignature(sumLabels): { - Labels: sumLabels, - Samples: []prompb.Sample{ - {Value: 0, Timestamp: convertTimeStamp(ts)}, - }, + { + metricFamilyName: "test_summary", + ls: labels.FromStrings( + model.MetricNameLabel, "test_summary"+countStr, + ), + t: convertTimeStamp(ts), + v: 0, + }, + } + }, + }, + { + name: "summary without start time and without scope promotion and some quantiles", + metric: func() pmetric.Metric { + metric := pmetric.NewMetric() + metric.SetName("test_summary") + metric.SetEmptySummary() + + dp := metric.Summary().DataPoints().AppendEmpty() + dp.SetTimestamp(ts) + dp.SetCount(50) + dp.SetSum(100) + dp.QuantileValues().EnsureCapacity(2) + h := dp.QuantileValues().AppendEmpty() + h.SetQuantile(0.5) + h.SetValue(30) + n := dp.QuantileValues().AppendEmpty() + n.SetQuantile(0.9) + n.SetValue(40) + + return metric + }, + scope: defaultScope, + promoteScope: false, + want: func() []combinedSample { + return []combinedSample{ + { + metricFamilyName: "test_summary", + ls: labels.FromStrings( + model.MetricNameLabel, "test_summary"+sumStr, + ), + t: convertTimeStamp(ts), + v: 100, + }, + { + metricFamilyName: "test_summary", + ls: labels.FromStrings( + model.MetricNameLabel, "test_summary"+countStr, + ), + t: convertTimeStamp(ts), + v: 50, + }, + { + metricFamilyName: "test_summary", + ls: labels.FromStrings( + model.MetricNameLabel, "test_summary", + quantileStr, "0.5", + ), + t: convertTimeStamp(ts), + v: 30, + }, + { + metricFamilyName: "test_summary", + ls: labels.FromStrings( + model.MetricNameLabel, "test_summary", + quantileStr, "0.9", + ), + t: convertTimeStamp(ts), + v: 40, }, } }, @@ -737,7 +505,8 @@ func TestPrometheusConverter_AddSummaryDataPoints(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { metric := tt.metric() - converter := NewPrometheusConverter() + mockAppender := &mockCombinedAppender{} + converter := NewPrometheusConverter(mockAppender) converter.addSummaryDataPoints( context.Background(), @@ -747,11 +516,13 @@ func TestPrometheusConverter_AddSummaryDataPoints(t *testing.T) { ExportCreatedMetric: true, PromoteScopeMetadata: tt.promoteScope, }, - prompb.MetricMetadata{MetricFamilyName: metric.Name()}, + metric.Name(), tt.scope, + metadata.Metadata{}, ) + require.NoError(t, mockAppender.Commit()) - testutil.RequireEqual(t, tt.want(), converter.unique) + requireEqual(t, tt.want(), mockAppender.samples) require.Empty(t, converter.conflicts) }) } @@ -775,7 +546,7 @@ func TestPrometheusConverter_AddHistogramDataPoints(t *testing.T) { metric func() pmetric.Metric scope scope promoteScope bool - want func() map[uint64]*prompb.TimeSeries + want func() []combinedSample }{ { name: "histogram with start time and without scope promotion", @@ -792,35 +563,34 @@ func TestPrometheusConverter_AddHistogramDataPoints(t *testing.T) { }, scope: defaultScope, promoteScope: false, - want: func() map[uint64]*prompb.TimeSeries { - countLabels := []prompb.Label{ - {Name: model.MetricNameLabel, Value: "test_hist" + countStr}, - } - createdLabels := []prompb.Label{ - {Name: model.MetricNameLabel, Value: "test_hist" + createdSuffix}, - } - infLabels := []prompb.Label{ - {Name: model.MetricNameLabel, Value: "test_hist_bucket"}, - {Name: model.BucketLabel, Value: "+Inf"}, - } - return map[uint64]*prompb.TimeSeries{ - timeSeriesSignature(countLabels): { - Labels: countLabels, - Samples: []prompb.Sample{ - {Value: 0, Timestamp: convertTimeStamp(ts)}, - }, + want: func() []combinedSample { + return []combinedSample{ + { + metricFamilyName: "test_hist", + ls: labels.FromStrings( + model.MetricNameLabel, "test_hist"+countStr, + ), + t: convertTimeStamp(ts), + ct: convertTimeStamp(ts), + v: 0, }, - timeSeriesSignature(infLabels): { - Labels: infLabels, - Samples: []prompb.Sample{ - {Value: 0, Timestamp: convertTimeStamp(ts)}, - }, + { + metricFamilyName: "test_hist", + ls: labels.FromStrings( + model.MetricNameLabel, "test_hist_bucket", + model.BucketLabel, "+Inf", + ), + t: convertTimeStamp(ts), + ct: convertTimeStamp(ts), + v: 0, }, - timeSeriesSignature(createdLabels): { - Labels: createdLabels, - Samples: []prompb.Sample{ - {Value: float64(convertTimeStamp(ts)), Timestamp: convertTimeStamp(ts)}, - }, + { + metricFamilyName: "test_hist", + ls: labels.FromStrings( + model.MetricNameLabel, "test_hist"+createdSuffix, + ), + t: convertTimeStamp(ts), + v: float64(convertTimeStamp(ts)), }, } }, @@ -840,57 +610,38 @@ func TestPrometheusConverter_AddHistogramDataPoints(t *testing.T) { }, scope: defaultScope, promoteScope: true, - want: func() map[uint64]*prompb.TimeSeries { - scopeLabels := []prompb.Label{ - { - Name: "otel_scope_attr1", - Value: "value1", - }, - { - Name: "otel_scope_attr2", - Value: "value2", - }, - { - Name: "otel_scope_name", - Value: defaultScope.name, - }, - { - Name: "otel_scope_schema_url", - Value: defaultScope.schemaURL, - }, - { - Name: "otel_scope_version", - Value: defaultScope.version, - }, + want: func() []combinedSample { + scopeLabels := []string{ + "otel_scope_attr1", "value1", + "otel_scope_attr2", "value2", + "otel_scope_name", defaultScope.name, + "otel_scope_schema_url", defaultScope.schemaURL, + "otel_scope_version", defaultScope.version, } - countLabels := append([]prompb.Label{ - {Name: model.MetricNameLabel, Value: "test_hist" + countStr}, - }, scopeLabels...) - infLabels := append([]prompb.Label{ - {Name: model.MetricNameLabel, Value: "test_hist_bucket"}, - {Name: model.BucketLabel, Value: "+Inf"}, - }, scopeLabels...) - createdLabels := append([]prompb.Label{ - {Name: model.MetricNameLabel, Value: "test_hist" + createdSuffix}, - }, scopeLabels...) - return map[uint64]*prompb.TimeSeries{ - timeSeriesSignature(countLabels): { - Labels: countLabels, - Samples: []prompb.Sample{ - {Value: 0, Timestamp: convertTimeStamp(ts)}, - }, + return []combinedSample{ + { + metricFamilyName: "test_hist", + ls: labels.FromStrings(append(scopeLabels, + model.MetricNameLabel, "test_hist"+countStr)...), + t: convertTimeStamp(ts), + ct: convertTimeStamp(ts), + v: 0, }, - timeSeriesSignature(infLabels): { - Labels: infLabels, - Samples: []prompb.Sample{ - {Value: 0, Timestamp: convertTimeStamp(ts)}, - }, + { + metricFamilyName: "test_hist", + ls: labels.FromStrings(append(scopeLabels, + model.MetricNameLabel, "test_hist_bucket", + model.BucketLabel, "+Inf")...), + t: convertTimeStamp(ts), + ct: convertTimeStamp(ts), + v: 0, }, - timeSeriesSignature(createdLabels): { - Labels: createdLabels, - Samples: []prompb.Sample{ - {Value: float64(convertTimeStamp(ts)), Timestamp: convertTimeStamp(ts)}, - }, + { + metricFamilyName: "test_hist", + ls: labels.FromStrings(append(scopeLabels, + model.MetricNameLabel, "test_hist"+createdSuffix)...), + t: convertTimeStamp(ts), + v: float64(convertTimeStamp(ts)), }, } }, @@ -907,26 +658,24 @@ func TestPrometheusConverter_AddHistogramDataPoints(t *testing.T) { return metric }, - want: func() map[uint64]*prompb.TimeSeries { - labels := []prompb.Label{ - {Name: model.MetricNameLabel, Value: "test_hist" + countStr}, - } - infLabels := []prompb.Label{ - {Name: model.MetricNameLabel, Value: "test_hist_bucket"}, - {Name: model.BucketLabel, Value: "+Inf"}, - } - return map[uint64]*prompb.TimeSeries{ - timeSeriesSignature(infLabels): { - Labels: infLabels, - Samples: []prompb.Sample{ - {Value: 0, Timestamp: convertTimeStamp(ts)}, - }, + want: func() []combinedSample { + return []combinedSample{ + { + metricFamilyName: "test_hist", + ls: labels.FromStrings( + model.MetricNameLabel, "test_hist"+countStr, + ), + t: convertTimeStamp(ts), + v: 0, }, - timeSeriesSignature(labels): { - Labels: labels, - Samples: []prompb.Sample{ - {Value: 0, Timestamp: convertTimeStamp(ts)}, - }, + { + metricFamilyName: "test_hist", + ls: labels.FromStrings( + model.MetricNameLabel, "test_hist_bucket", + model.BucketLabel, "+Inf", + ), + t: convertTimeStamp(ts), + v: 0, }, } }, @@ -935,7 +684,8 @@ func TestPrometheusConverter_AddHistogramDataPoints(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { metric := tt.metric() - converter := NewPrometheusConverter() + mockAppender := &mockCombinedAppender{} + converter := NewPrometheusConverter(mockAppender) converter.addHistogramDataPoints( context.Background(), @@ -945,11 +695,13 @@ func TestPrometheusConverter_AddHistogramDataPoints(t *testing.T) { ExportCreatedMetric: true, PromoteScopeMetadata: tt.promoteScope, }, - prompb.MetricMetadata{MetricFamilyName: metric.Name()}, + metric.Name(), tt.scope, + metadata.Metadata{}, ) + require.NoError(t, mockAppender.Commit()) - require.Equal(t, tt.want(), converter.unique) + requireEqual(t, tt.want(), mockAppender.samples) require.Empty(t, converter.conflicts) }) } @@ -957,35 +709,35 @@ func TestPrometheusConverter_AddHistogramDataPoints(t *testing.T) { func TestGetPromExemplars(t *testing.T) { ctx := context.Background() - everyN := &everyNTimes{n: 1} + c := NewPrometheusConverter(&mockCombinedAppender{}) t.Run("Exemplars with int value", func(t *testing.T) { - pt := pmetric.NewNumberDataPoint() - exemplar := pt.Exemplars().AppendEmpty() + es := pmetric.NewExemplarSlice() + exemplar := es.AppendEmpty() exemplar.SetTimestamp(pcommon.Timestamp(time.Now().UnixNano())) exemplar.SetIntValue(42) - exemplars, err := getPromExemplars(ctx, everyN, pt) + exemplars, err := c.getPromExemplars(ctx, es) require.NoError(t, err) require.Len(t, exemplars, 1) require.Equal(t, float64(42), exemplars[0].Value) }) t.Run("Exemplars with double value", func(t *testing.T) { - pt := pmetric.NewNumberDataPoint() - exemplar := pt.Exemplars().AppendEmpty() + es := pmetric.NewExemplarSlice() + exemplar := es.AppendEmpty() exemplar.SetTimestamp(pcommon.Timestamp(time.Now().UnixNano())) exemplar.SetDoubleValue(69.420) - exemplars, err := getPromExemplars(ctx, everyN, pt) + exemplars, err := c.getPromExemplars(ctx, es) require.NoError(t, err) require.Len(t, exemplars, 1) require.Equal(t, 69.420, exemplars[0].Value) }) t.Run("Exemplars with unsupported value type", func(t *testing.T) { - pt := pmetric.NewNumberDataPoint() - exemplar := pt.Exemplars().AppendEmpty() + es := pmetric.NewExemplarSlice() + exemplar := es.AppendEmpty() exemplar.SetTimestamp(pcommon.Timestamp(time.Now().UnixNano())) - _, err := getPromExemplars(ctx, everyN, pt) + _, err := c.getPromExemplars(ctx, es) require.Error(t, err) }) } diff --git a/storage/remote/otlptranslator/prometheusremotewrite/histograms.go b/storage/remote/otlptranslator/prometheusremotewrite/histograms.go index f4199fd1c2..7e6fae37cd 100644 --- a/storage/remote/otlptranslator/prometheusremotewrite/histograms.go +++ b/storage/remote/otlptranslator/prometheusremotewrite/histograms.go @@ -26,8 +26,8 @@ import ( "go.opentelemetry.io/collector/pdata/pmetric" "github.com/prometheus/prometheus/model/histogram" + "github.com/prometheus/prometheus/model/metadata" "github.com/prometheus/prometheus/model/value" - "github.com/prometheus/prometheus/prompb" "github.com/prometheus/prometheus/util/annotations" ) @@ -36,7 +36,8 @@ const defaultZeroThreshold = 1e-128 // addExponentialHistogramDataPoints adds OTel exponential histogram data points to the corresponding time series // as native histogram samples. func (c *PrometheusConverter) addExponentialHistogramDataPoints(ctx context.Context, dataPoints pmetric.ExponentialHistogramDataPointSlice, - resource pcommon.Resource, settings Settings, metadata prompb.MetricMetadata, temporality pmetric.AggregationTemporality, scope scope, + resource pcommon.Resource, settings Settings, promName string, temporality pmetric.AggregationTemporality, + scope scope, meta metadata.Metadata, ) (annotations.Annotations, error) { var annots annotations.Annotations for x := 0; x < dataPoints.Len(); x++ { @@ -46,34 +47,36 @@ func (c *PrometheusConverter) addExponentialHistogramDataPoints(ctx context.Cont pt := dataPoints.At(x) - histogram, ws, err := exponentialToNativeHistogram(pt, temporality) + hp, ws, err := exponentialToNativeHistogram(pt, temporality) annots.Merge(ws) if err != nil { return annots, err } - lbls, err := createAttributes( + lbls, err := c.createAttributes( resource, pt.Attributes(), scope, settings, nil, true, - metadata, + meta, model.MetricNameLabel, - metadata.MetricFamilyName, + promName, ) - if err != nil { - return nil, err - } - ts, _ := c.getOrCreateTimeSeries(lbls) - ts.Histograms = append(ts.Histograms, histogram) - - exemplars, err := getPromExemplars[pmetric.ExponentialHistogramDataPoint](ctx, &c.everyN, pt) if err != nil { return annots, err } - ts.Exemplars = append(ts.Exemplars, exemplars...) + ts := convertTimeStamp(pt.Timestamp()) + ct := convertTimeStamp(pt.StartTimestamp()) + exemplars, err := c.getPromExemplars(ctx, pt.Exemplars()) + if err != nil { + return annots, err + } + // OTel exponential histograms are always Int Histograms. + if err = c.appender.AppendHistogram(promName, lbls, meta, ts, ct, hp, exemplars); err != nil { + return annots, err + } } return annots, nil @@ -81,11 +84,11 @@ func (c *PrometheusConverter) addExponentialHistogramDataPoints(ctx context.Cont // exponentialToNativeHistogram translates an OTel Exponential Histogram data point // to a Prometheus Native Histogram. -func exponentialToNativeHistogram(p pmetric.ExponentialHistogramDataPoint, temporality pmetric.AggregationTemporality) (prompb.Histogram, annotations.Annotations, error) { +func exponentialToNativeHistogram(p pmetric.ExponentialHistogramDataPoint, temporality pmetric.AggregationTemporality) (*histogram.Histogram, annotations.Annotations, error) { var annots annotations.Annotations scale := p.Scale() if scale < -4 { - return prompb.Histogram{}, annots, + return nil, annots, fmt.Errorf("cannot convert exponential to native histogram."+ " Scale must be >= -4, was %d", scale) } @@ -108,41 +111,36 @@ func exponentialToNativeHistogram(p pmetric.ExponentialHistogramDataPoint, tempo // need to know here if it was used for the detection. // Ref: https://github.com/open-telemetry/opentelemetry-collector-contrib/pull/28663#issuecomment-1810577303 // Counter reset detection in Prometheus: https://github.com/prometheus/prometheus/blob/f997c72f294c0f18ca13fa06d51889af04135195/tsdb/chunkenc/histogram.go#L232 - resetHint := prompb.Histogram_UNKNOWN + resetHint := histogram.UnknownCounterReset if temporality == pmetric.AggregationTemporalityDelta { // If the histogram has delta temporality, set the reset hint to gauge to avoid unnecessary chunk cutting. // We're in an early phase of implementing delta support (proposal: https://github.com/prometheus/proposals/pull/48/). // This might be changed to a different hint name as gauge type might be misleading for samples that should be // summed over time. - resetHint = prompb.Histogram_GAUGE + resetHint = histogram.GaugeType } - - h := prompb.Histogram{ - ResetHint: resetHint, - Schema: scale, - - ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: p.ZeroCount()}, + h := &histogram.Histogram{ + CounterResetHint: resetHint, + Schema: scale, // TODO use zero_threshold, if set, see // https://github.com/open-telemetry/opentelemetry-proto/pull/441 - ZeroThreshold: defaultZeroThreshold, - - PositiveSpans: pSpans, - PositiveDeltas: pDeltas, - NegativeSpans: nSpans, - NegativeDeltas: nDeltas, - - Timestamp: convertTimeStamp(p.Timestamp()), + ZeroThreshold: defaultZeroThreshold, + ZeroCount: p.ZeroCount(), + PositiveSpans: pSpans, + PositiveBuckets: pDeltas, + NegativeSpans: nSpans, + NegativeBuckets: nDeltas, } if p.Flags().NoRecordedValue() { h.Sum = math.Float64frombits(value.StaleNaN) - h.Count = &prompb.Histogram_CountInt{CountInt: value.StaleNaN} + h.Count = value.StaleNaN } else { if p.HasSum() { h.Sum = p.Sum() } - h.Count = &prompb.Histogram_CountInt{CountInt: p.Count()} + h.Count = p.Count() if p.Count() == 0 && h.Sum != 0 { annots.Add(fmt.Errorf("exponential histogram data point has zero count, but non-zero sum: %f", h.Sum)) } @@ -167,13 +165,13 @@ func exponentialToNativeHistogram(p pmetric.ExponentialHistogramDataPoint, tempo // // When converting from OTel Explicit Histograms to Native Histograms with Custom Buckets, // the bucket indexes are not scaled, and the indices are not adjusted by 1. -func convertBucketsLayout(bucketCounts []uint64, offset, scaleDown int32, adjustOffset bool) ([]prompb.BucketSpan, []int64) { +func convertBucketsLayout(bucketCounts []uint64, offset, scaleDown int32, adjustOffset bool) ([]histogram.Span, []int64) { if len(bucketCounts) == 0 { return nil, nil } var ( - spans []prompb.BucketSpan + spans []histogram.Span deltas []int64 count int64 prevCount int64 @@ -196,7 +194,7 @@ func convertBucketsLayout(bucketCounts []uint64, offset, scaleDown int32, adjust initialOffset = initialOffset>>scaleDown + 1 } - spans = append(spans, prompb.BucketSpan{ + spans = append(spans, histogram.Span{ Offset: initialOffset, Length: 0, }) @@ -217,7 +215,7 @@ func convertBucketsLayout(bucketCounts []uint64, offset, scaleDown int32, adjust // We have to create a new span, because we have found a gap // of more than two buckets. The constant 2 is copied from the logic in // https://github.com/prometheus/client_golang/blob/27f0506d6ebbb117b6b697d0552ee5be2502c5f2/prometheus/histogram.go#L1296 - spans = append(spans, prompb.BucketSpan{ + spans = append(spans, histogram.Span{ Offset: gap, Length: 0, }) @@ -239,7 +237,7 @@ func convertBucketsLayout(bucketCounts []uint64, offset, scaleDown int32, adjust // We have to create a new span, because we have found a gap // of more than two buckets. The constant 2 is copied from the logic in // https://github.com/prometheus/client_golang/blob/27f0506d6ebbb117b6b697d0552ee5be2502c5f2/prometheus/histogram.go#L1296 - spans = append(spans, prompb.BucketSpan{ + spans = append(spans, histogram.Span{ Offset: gap, Length: 0, }) @@ -256,7 +254,8 @@ func convertBucketsLayout(bucketCounts []uint64, offset, scaleDown int32, adjust } func (c *PrometheusConverter) addCustomBucketsHistogramDataPoints(ctx context.Context, dataPoints pmetric.HistogramDataPointSlice, - resource pcommon.Resource, settings Settings, metadata prompb.MetricMetadata, temporality pmetric.AggregationTemporality, scope scope, + resource pcommon.Resource, settings Settings, promName string, temporality pmetric.AggregationTemporality, + scope scope, meta metadata.Metadata, ) (annotations.Annotations, error) { var annots annotations.Annotations @@ -267,41 +266,41 @@ func (c *PrometheusConverter) addCustomBucketsHistogramDataPoints(ctx context.Co pt := dataPoints.At(x) - histogram, ws, err := explicitHistogramToCustomBucketsHistogram(pt, temporality) + hp, ws, err := explicitHistogramToCustomBucketsHistogram(pt, temporality) annots.Merge(ws) if err != nil { return annots, err } - lbls, err := createAttributes( + lbls, err := c.createAttributes( resource, pt.Attributes(), scope, settings, nil, true, - metadata, + meta, model.MetricNameLabel, - metadata.MetricFamilyName, + promName, ) - if err != nil { - return nil, err - } - - ts, _ := c.getOrCreateTimeSeries(lbls) - ts.Histograms = append(ts.Histograms, histogram) - - exemplars, err := getPromExemplars[pmetric.HistogramDataPoint](ctx, &c.everyN, pt) if err != nil { return annots, err } - ts.Exemplars = append(ts.Exemplars, exemplars...) + ts := convertTimeStamp(pt.Timestamp()) + ct := convertTimeStamp(pt.StartTimestamp()) + exemplars, err := c.getPromExemplars(ctx, pt.Exemplars()) + if err != nil { + return annots, err + } + if err = c.appender.AppendHistogram(promName, lbls, meta, ts, ct, hp, exemplars); err != nil { + return annots, err + } } return annots, nil } -func explicitHistogramToCustomBucketsHistogram(p pmetric.HistogramDataPoint, temporality pmetric.AggregationTemporality) (prompb.Histogram, annotations.Annotations, error) { +func explicitHistogramToCustomBucketsHistogram(p pmetric.HistogramDataPoint, temporality pmetric.AggregationTemporality) (*histogram.Histogram, annotations.Annotations, error) { var annots annotations.Annotations buckets := p.BucketCounts().AsRaw() @@ -318,23 +317,22 @@ func explicitHistogramToCustomBucketsHistogram(p pmetric.HistogramDataPoint, tem // need to know here if it was used for the detection. // Ref: https://github.com/open-telemetry/opentelemetry-collector-contrib/pull/28663#issuecomment-1810577303 // Counter reset detection in Prometheus: https://github.com/prometheus/prometheus/blob/f997c72f294c0f18ca13fa06d51889af04135195/tsdb/chunkenc/histogram.go#L232 - resetHint := prompb.Histogram_UNKNOWN + resetHint := histogram.UnknownCounterReset if temporality == pmetric.AggregationTemporalityDelta { // If the histogram has delta temporality, set the reset hint to gauge to avoid unnecessary chunk cutting. // We're in an early phase of implementing delta support (proposal: https://github.com/prometheus/proposals/pull/48/). // This might be changed to a different hint name as gauge type might be misleading for samples that should be // summed over time. - resetHint = prompb.Histogram_GAUGE + resetHint = histogram.GaugeType } // TODO(carrieedwards): Add setting to limit maximum bucket count - h := prompb.Histogram{ - ResetHint: resetHint, - Schema: histogram.CustomBucketsSchema, - - PositiveSpans: positiveSpans, - PositiveDeltas: positiveDeltas, + h := &histogram.Histogram{ + CounterResetHint: resetHint, + Schema: histogram.CustomBucketsSchema, + PositiveSpans: positiveSpans, + PositiveBuckets: positiveDeltas, // Note: OTel explicit histograms have an implicit +Inf bucket, which has a lower bound // of the last element in the explicit_bounds array. // This is similar to the custom_values array in native histograms with custom buckets. @@ -342,18 +340,16 @@ func explicitHistogramToCustomBucketsHistogram(p pmetric.HistogramDataPoint, tem // can be mapped directly to the custom_values array. // See: https://github.com/open-telemetry/opentelemetry-proto/blob/d7770822d70c7bd47a6891fc9faacc66fc4af3d3/opentelemetry/proto/metrics/v1/metrics.proto#L469 CustomValues: p.ExplicitBounds().AsRaw(), - - Timestamp: convertTimeStamp(p.Timestamp()), } if p.Flags().NoRecordedValue() { h.Sum = math.Float64frombits(value.StaleNaN) - h.Count = &prompb.Histogram_CountInt{CountInt: value.StaleNaN} + h.Count = value.StaleNaN } else { if p.HasSum() { h.Sum = p.Sum() } - h.Count = &prompb.Histogram_CountInt{CountInt: p.Count()} + h.Count = p.Count() if p.Count() == 0 && h.Sum != 0 { annots.Add(fmt.Errorf("histogram data point has zero count, but non-zero sum: %f", h.Sum)) } diff --git a/storage/remote/otlptranslator/prometheusremotewrite/histograms_test.go b/storage/remote/otlptranslator/prometheusremotewrite/histograms_test.go index 97f7dc9bca..3873cf94fe 100644 --- a/storage/remote/otlptranslator/prometheusremotewrite/histograms_test.go +++ b/storage/remote/otlptranslator/prometheusremotewrite/histograms_test.go @@ -28,11 +28,14 @@ import ( "go.opentelemetry.io/collector/pdata/pcommon" "go.opentelemetry.io/collector/pdata/pmetric" - "github.com/prometheus/prometheus/prompb" + "github.com/prometheus/prometheus/model/exemplar" + "github.com/prometheus/prometheus/model/histogram" + "github.com/prometheus/prometheus/model/metadata" + "github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite/labels" ) type expectedBucketLayout struct { - wantSpans []prompb.BucketSpan + wantSpans []histogram.Span wantDeltas []int64 } @@ -52,7 +55,7 @@ func TestConvertBucketsLayout(t *testing.T) { }, wantLayout: map[int32]expectedBucketLayout{ 0: { - wantSpans: []prompb.BucketSpan{ + wantSpans: []histogram.Span{ { Offset: 1, Length: 4, @@ -61,7 +64,7 @@ func TestConvertBucketsLayout(t *testing.T) { wantDeltas: []int64{4, -1, -1, -1}, }, 1: { - wantSpans: []prompb.BucketSpan{ + wantSpans: []histogram.Span{ { Offset: 1, Length: 2, @@ -71,7 +74,7 @@ func TestConvertBucketsLayout(t *testing.T) { wantDeltas: []int64{7, -4}, }, 2: { - wantSpans: []prompb.BucketSpan{ + wantSpans: []histogram.Span{ { Offset: 1, Length: 1, @@ -92,7 +95,7 @@ func TestConvertBucketsLayout(t *testing.T) { }, wantLayout: map[int32]expectedBucketLayout{ 0: { - wantSpans: []prompb.BucketSpan{ + wantSpans: []histogram.Span{ { Offset: 2, Length: 4, @@ -101,7 +104,7 @@ func TestConvertBucketsLayout(t *testing.T) { wantDeltas: []int64{4, -1, -1, -1}, }, 1: { - wantSpans: []prompb.BucketSpan{ + wantSpans: []histogram.Span{ { Offset: 1, Length: 3, @@ -110,7 +113,7 @@ func TestConvertBucketsLayout(t *testing.T) { wantDeltas: []int64{4, 1, -4}, // 0+4, 3+2, 1+0 = 4, 5, 1 }, 2: { - wantSpans: []prompb.BucketSpan{ + wantSpans: []histogram.Span{ { Offset: 1, Length: 2, @@ -130,7 +133,7 @@ func TestConvertBucketsLayout(t *testing.T) { }, wantLayout: map[int32]expectedBucketLayout{ 0: { - wantSpans: []prompb.BucketSpan{ + wantSpans: []histogram.Span{ { Offset: 5, Length: 4, @@ -143,7 +146,7 @@ func TestConvertBucketsLayout(t *testing.T) { wantDeltas: []int64{4, -2, -2, 2, -1}, }, 1: { - wantSpans: []prompb.BucketSpan{ + wantSpans: []histogram.Span{ { Offset: 3, Length: 2, @@ -158,7 +161,7 @@ func TestConvertBucketsLayout(t *testing.T) { wantDeltas: []int64{6, -4, -1}, }, 2: { - wantSpans: []prompb.BucketSpan{ + wantSpans: []histogram.Span{ { Offset: 2, Length: 1, @@ -185,7 +188,7 @@ func TestConvertBucketsLayout(t *testing.T) { }, wantLayout: map[int32]expectedBucketLayout{ 0: { - wantSpans: []prompb.BucketSpan{ + wantSpans: []histogram.Span{ { Offset: 5, Length: 4, @@ -198,7 +201,7 @@ func TestConvertBucketsLayout(t *testing.T) { wantDeltas: []int64{4, -2, -2, 2, -1}, }, 1: { - wantSpans: []prompb.BucketSpan{ + wantSpans: []histogram.Span{ { Offset: 3, Length: 2, @@ -213,7 +216,7 @@ func TestConvertBucketsLayout(t *testing.T) { wantDeltas: []int64{6, -4, -1}, }, 2: { - wantSpans: []prompb.BucketSpan{ + wantSpans: []histogram.Span{ { Offset: 2, Length: 4, @@ -236,7 +239,7 @@ func TestConvertBucketsLayout(t *testing.T) { }, wantLayout: map[int32]expectedBucketLayout{ 0: { - wantSpans: []prompb.BucketSpan{ + wantSpans: []histogram.Span{ { Offset: -1, Length: 2, @@ -249,7 +252,7 @@ func TestConvertBucketsLayout(t *testing.T) { wantDeltas: []int64{3, -2, 0}, }, 1: { - wantSpans: []prompb.BucketSpan{ + wantSpans: []histogram.Span{ { Offset: 0, Length: 3, @@ -260,7 +263,7 @@ func TestConvertBucketsLayout(t *testing.T) { wantDeltas: []int64{4, -4, 1}, }, 2: { - wantSpans: []prompb.BucketSpan{ + wantSpans: []histogram.Span{ { Offset: 0, Length: 2, @@ -282,7 +285,7 @@ func TestConvertBucketsLayout(t *testing.T) { }, wantLayout: map[int32]expectedBucketLayout{ 0: { - wantSpans: []prompb.BucketSpan{ + wantSpans: []histogram.Span{ { Offset: -1, Length: 6, @@ -291,7 +294,7 @@ func TestConvertBucketsLayout(t *testing.T) { wantDeltas: []int64{3, -2, -1, 1, -1, 1}, }, 1: { - wantSpans: []prompb.BucketSpan{ + wantSpans: []histogram.Span{ { Offset: 0, Length: 3, @@ -302,7 +305,7 @@ func TestConvertBucketsLayout(t *testing.T) { wantDeltas: []int64{4, -3, 0}, }, 2: { - wantSpans: []prompb.BucketSpan{ + wantSpans: []histogram.Span{ { Offset: 0, Length: 2, @@ -324,7 +327,7 @@ func TestConvertBucketsLayout(t *testing.T) { }, wantLayout: map[int32]expectedBucketLayout{ 0: { - wantSpans: []prompb.BucketSpan{ + wantSpans: []histogram.Span{ { Offset: -1, Length: 7, @@ -333,7 +336,7 @@ func TestConvertBucketsLayout(t *testing.T) { wantDeltas: []int64{3, -3, 0, 1, -1, 0, 1}, }, 1: { - wantSpans: []prompb.BucketSpan{ + wantSpans: []histogram.Span{ { Offset: 0, Length: 4, @@ -344,7 +347,7 @@ func TestConvertBucketsLayout(t *testing.T) { wantDeltas: []int64{3, -2, -1, 1}, }, 2: { - wantSpans: []prompb.BucketSpan{ + wantSpans: []histogram.Span{ { Offset: 0, Length: 3, @@ -379,8 +382,8 @@ func TestConvertBucketsLayout(t *testing.T) { for scaleDown, wantLayout := range tt.wantLayout { t.Run(fmt.Sprintf("%s-scaleby-%d", tt.name, scaleDown), func(t *testing.T) { gotSpans, gotDeltas := convertBucketsLayout(tt.buckets().BucketCounts().AsRaw(), tt.buckets().Offset(), scaleDown, true) - require.Equal(t, wantLayout.wantSpans, gotSpans) - require.Equal(t, wantLayout.wantDeltas, gotDeltas) + requireEqual(t, wantLayout.wantSpans, gotSpans) + requireEqual(t, wantLayout.wantDeltas, gotDeltas) }) } } @@ -418,7 +421,7 @@ func TestExponentialToNativeHistogram(t *testing.T) { tests := []struct { name string exponentialHist func() pmetric.ExponentialHistogramDataPoint - wantNativeHist func() prompb.Histogram + wantNativeHist func() *histogram.Histogram wantErrMessage string }{ { @@ -440,18 +443,17 @@ func TestExponentialToNativeHistogram(t *testing.T) { return pt }, - wantNativeHist: func() prompb.Histogram { - return prompb.Histogram{ - Count: &prompb.Histogram_CountInt{CountInt: 4}, - Sum: 10.1, - Schema: 1, - ZeroThreshold: defaultZeroThreshold, - ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: 1}, - NegativeSpans: []prompb.BucketSpan{{Offset: 2, Length: 2}}, - NegativeDeltas: []int64{1, 0}, - PositiveSpans: []prompb.BucketSpan{{Offset: 2, Length: 2}}, - PositiveDeltas: []int64{1, 0}, - Timestamp: 500, + wantNativeHist: func() *histogram.Histogram { + return &histogram.Histogram{ + Count: 4, + Sum: 10.1, + Schema: 1, + ZeroThreshold: defaultZeroThreshold, + ZeroCount: 1, + NegativeSpans: []histogram.Span{{Offset: 2, Length: 2}}, + NegativeBuckets: []int64{1, 0}, + PositiveSpans: []histogram.Span{{Offset: 2, Length: 2}}, + PositiveBuckets: []int64{1, 0}, } }, }, @@ -474,17 +476,16 @@ func TestExponentialToNativeHistogram(t *testing.T) { return pt }, - wantNativeHist: func() prompb.Histogram { - return prompb.Histogram{ - Count: &prompb.Histogram_CountInt{CountInt: 4}, - Schema: 1, - ZeroThreshold: defaultZeroThreshold, - ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: 1}, - NegativeSpans: []prompb.BucketSpan{{Offset: 2, Length: 2}}, - NegativeDeltas: []int64{1, 0}, - PositiveSpans: []prompb.BucketSpan{{Offset: 2, Length: 2}}, - PositiveDeltas: []int64{1, 0}, - Timestamp: 500, + wantNativeHist: func() *histogram.Histogram { + return &histogram.Histogram{ + Count: 4, + Schema: 1, + ZeroThreshold: defaultZeroThreshold, + ZeroCount: 1, + NegativeSpans: []histogram.Span{{Offset: 2, Length: 2}}, + NegativeBuckets: []int64{1, 0}, + PositiveSpans: []histogram.Span{{Offset: 2, Length: 2}}, + PositiveBuckets: []int64{1, 0}, } }, }, @@ -515,18 +516,17 @@ func TestExponentialToNativeHistogram(t *testing.T) { pt.Negative().SetOffset(2) return pt }, - wantNativeHist: func() prompb.Histogram { - return prompb.Histogram{ - Count: &prompb.Histogram_CountInt{CountInt: 6}, - Sum: 10.1, - Schema: 8, - ZeroThreshold: defaultZeroThreshold, - ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: 1}, - PositiveSpans: []prompb.BucketSpan{{Offset: 2, Length: 3}}, - PositiveDeltas: []int64{1, 0, 0}, // 1, 1, 1 - NegativeSpans: []prompb.BucketSpan{{Offset: 3, Length: 3}}, - NegativeDeltas: []int64{1, 0, 0}, // 1, 1, 1 - Timestamp: 500, + wantNativeHist: func() *histogram.Histogram { + return &histogram.Histogram{ + Count: 6, + Sum: 10.1, + Schema: 8, + ZeroThreshold: defaultZeroThreshold, + ZeroCount: 1, + PositiveSpans: []histogram.Span{{Offset: 2, Length: 3}}, + PositiveBuckets: []int64{1, 0, 0}, // 1, 1, 1 + NegativeSpans: []histogram.Span{{Offset: 3, Length: 3}}, + NegativeBuckets: []int64{1, 0, 0}, // 1, 1, 1 } }, }, @@ -547,18 +547,17 @@ func TestExponentialToNativeHistogram(t *testing.T) { pt.Negative().SetOffset(2) return pt }, - wantNativeHist: func() prompb.Histogram { - return prompb.Histogram{ - Count: &prompb.Histogram_CountInt{CountInt: 6}, - Sum: 10.1, - Schema: 8, - ZeroThreshold: defaultZeroThreshold, - ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: 1}, - PositiveSpans: []prompb.BucketSpan{{Offset: 1, Length: 2}}, - PositiveDeltas: []int64{1, 1}, // 0+1, 1+1 = 1, 2 - NegativeSpans: []prompb.BucketSpan{{Offset: 2, Length: 2}}, - NegativeDeltas: []int64{2, -1}, // 1+1, 1+0 = 2, 1 - Timestamp: 500, + wantNativeHist: func() *histogram.Histogram { + return &histogram.Histogram{ + Count: 6, + Sum: 10.1, + Schema: 8, + ZeroThreshold: defaultZeroThreshold, + ZeroCount: 1, + PositiveSpans: []histogram.Span{{Offset: 1, Length: 2}}, + PositiveBuckets: []int64{1, 1}, // 0+1, 1+1 = 1, 2 + NegativeSpans: []histogram.Span{{Offset: 2, Length: 2}}, + NegativeBuckets: []int64{2, -1}, // 1+1, 1+0 = 2, 1 } }, }, @@ -599,20 +598,18 @@ func validateExponentialHistogramCount(t *testing.T, h pmetric.ExponentialHistog require.Equal(t, h.Count(), actualCount, "exponential histogram count mismatch") } -func validateNativeHistogramCount(t *testing.T, h prompb.Histogram) { - require.NotNil(t, h.Count) - require.IsType(t, &prompb.Histogram_CountInt{}, h.Count) - want := h.Count.(*prompb.Histogram_CountInt).CountInt +func validateNativeHistogramCount(t *testing.T, h *histogram.Histogram) { + want := h.Count var ( actualCount uint64 prevBucket int64 ) - for _, delta := range h.PositiveDeltas { + for _, delta := range h.PositiveBuckets { prevBucket += delta actualCount += uint64(prevBucket) } prevBucket = 0 - for _, delta := range h.NegativeDeltas { + for _, delta := range h.NegativeBuckets { prevBucket += delta actualCount += uint64(prevBucket) } @@ -636,7 +633,7 @@ func TestPrometheusConverter_addExponentialHistogramDataPoints(t *testing.T) { metric func() pmetric.Metric scope scope promoteScope bool - wantSeries func() map[uint64]*prompb.TimeSeries + wantSeries func() []combinedHistogram }{ { name: "histogram data points with same labels and without scope promotion", @@ -665,36 +662,43 @@ func TestPrometheusConverter_addExponentialHistogramDataPoints(t *testing.T) { }, scope: defaultScope, promoteScope: false, - wantSeries: func() map[uint64]*prompb.TimeSeries { - labels := []prompb.Label{ - {Name: model.MetricNameLabel, Value: "test_hist"}, - {Name: "attr", Value: "test_attr"}, - } - return map[uint64]*prompb.TimeSeries{ - timeSeriesSignature(labels): { - Labels: labels, - Histograms: []prompb.Histogram{ - { - Count: &prompb.Histogram_CountInt{CountInt: 7}, - Schema: 1, - ZeroThreshold: defaultZeroThreshold, - ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: 0}, - PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 2}}, - PositiveDeltas: []int64{4, -2}, - }, - { - Count: &prompb.Histogram_CountInt{CountInt: 4}, - Schema: 1, - ZeroThreshold: defaultZeroThreshold, - ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: 0}, - PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 3}}, - PositiveDeltas: []int64{4, -2, -1}, - }, + wantSeries: func() []combinedHistogram { + lbls := labels.FromStrings( + model.MetricNameLabel, "test_hist", + "attr", "test_attr", + ) + return []combinedHistogram{ + { + metricFamilyName: "test_hist", + ls: lbls, + meta: metadata.Metadata{}, + t: 0, + ct: 0, + h: &histogram.Histogram{ + Count: 7, + Schema: 1, + ZeroThreshold: defaultZeroThreshold, + ZeroCount: 0, + PositiveSpans: []histogram.Span{{Offset: 0, Length: 2}}, + PositiveBuckets: []int64{4, -2}, }, - Exemplars: []prompb.Exemplar{ - {Value: 1}, - {Value: 2}, + es: []exemplar.Exemplar{{Value: 1}}, + }, + { + metricFamilyName: "test_hist", + ls: lbls, + meta: metadata.Metadata{}, + t: 0, + ct: 0, + h: &histogram.Histogram{ + Count: 4, + Schema: 1, + ZeroThreshold: defaultZeroThreshold, + ZeroCount: 0, + PositiveSpans: []histogram.Span{{Offset: 0, Length: 3}}, + PositiveBuckets: []int64{4, -2, -1}, }, + es: []exemplar.Exemplar{{Value: 2}}, }, } }, @@ -726,41 +730,48 @@ func TestPrometheusConverter_addExponentialHistogramDataPoints(t *testing.T) { }, scope: defaultScope, promoteScope: true, - wantSeries: func() map[uint64]*prompb.TimeSeries { - labels := []prompb.Label{ - {Name: model.MetricNameLabel, Value: "test_hist"}, - {Name: "attr", Value: "test_attr"}, - {Name: "otel_scope_name", Value: defaultScope.name}, - {Name: "otel_scope_schema_url", Value: defaultScope.schemaURL}, - {Name: "otel_scope_version", Value: defaultScope.version}, - {Name: "otel_scope_attr1", Value: "value1"}, - {Name: "otel_scope_attr2", Value: "value2"}, - } - return map[uint64]*prompb.TimeSeries{ - timeSeriesSignature(labels): { - Labels: labels, - Histograms: []prompb.Histogram{ - { - Count: &prompb.Histogram_CountInt{CountInt: 7}, - Schema: 1, - ZeroThreshold: defaultZeroThreshold, - ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: 0}, - PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 2}}, - PositiveDeltas: []int64{4, -2}, - }, - { - Count: &prompb.Histogram_CountInt{CountInt: 4}, - Schema: 1, - ZeroThreshold: defaultZeroThreshold, - ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: 0}, - PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 3}}, - PositiveDeltas: []int64{4, -2, -1}, - }, + wantSeries: func() []combinedHistogram { + lbls := labels.FromStrings( + model.MetricNameLabel, "test_hist", + "attr", "test_attr", + "otel_scope_name", defaultScope.name, + "otel_scope_schema_url", defaultScope.schemaURL, + "otel_scope_version", defaultScope.version, + "otel_scope_attr1", "value1", + "otel_scope_attr2", "value2", + ) + return []combinedHistogram{ + { + metricFamilyName: "test_hist", + ls: lbls, + meta: metadata.Metadata{}, + t: 0, + ct: 0, + h: &histogram.Histogram{ + Count: 7, + Schema: 1, + ZeroThreshold: defaultZeroThreshold, + ZeroCount: 0, + PositiveSpans: []histogram.Span{{Offset: 0, Length: 2}}, + PositiveBuckets: []int64{4, -2}, }, - Exemplars: []prompb.Exemplar{ - {Value: 1}, - {Value: 2}, + es: []exemplar.Exemplar{{Value: 1}}, + }, + { + metricFamilyName: "test_hist", + ls: lbls, + meta: metadata.Metadata{}, + t: 0, + ct: 0, + h: &histogram.Histogram{ + Count: 4, + Schema: 1, + ZeroThreshold: defaultZeroThreshold, + ZeroCount: 0, + PositiveSpans: []histogram.Span{{Offset: 0, Length: 3}}, + PositiveBuckets: []int64{4, -2, -1}, }, + es: []exemplar.Exemplar{{Value: 2}}, }, } }, @@ -792,48 +803,48 @@ func TestPrometheusConverter_addExponentialHistogramDataPoints(t *testing.T) { }, scope: defaultScope, promoteScope: false, - wantSeries: func() map[uint64]*prompb.TimeSeries { - labels := []prompb.Label{ - {Name: model.MetricNameLabel, Value: "test_hist"}, - {Name: "attr", Value: "test_attr"}, - } - labelsAnother := []prompb.Label{ - {Name: model.MetricNameLabel, Value: "test_hist"}, - {Name: "attr", Value: "test_attr_two"}, - } + wantSeries: func() []combinedHistogram { + lbls := labels.FromStrings( + model.MetricNameLabel, "test_hist", + "attr", "test_attr", + ) + labelsAnother := labels.FromStrings( + model.MetricNameLabel, "test_hist", + "attr", "test_attr_two", + ) - return map[uint64]*prompb.TimeSeries{ - timeSeriesSignature(labels): { - Labels: labels, - Histograms: []prompb.Histogram{ - { - Count: &prompb.Histogram_CountInt{CountInt: 7}, - Schema: 1, - ZeroThreshold: defaultZeroThreshold, - ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: 0}, - PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 2}}, - PositiveDeltas: []int64{4, -2}, - }, - }, - Exemplars: []prompb.Exemplar{ - {Value: 1}, + return []combinedHistogram{ + { + metricFamilyName: "test_hist", + ls: lbls, + meta: metadata.Metadata{}, + t: 0, + ct: 0, + h: &histogram.Histogram{ + Count: 7, + Schema: 1, + ZeroThreshold: defaultZeroThreshold, + ZeroCount: 0, + PositiveSpans: []histogram.Span{{Offset: 0, Length: 2}}, + PositiveBuckets: []int64{4, -2}, }, + es: []exemplar.Exemplar{{Value: 1}}, }, - timeSeriesSignature(labelsAnother): { - Labels: labelsAnother, - Histograms: []prompb.Histogram{ - { - Count: &prompb.Histogram_CountInt{CountInt: 4}, - Schema: 1, - ZeroThreshold: defaultZeroThreshold, - ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: 0}, - NegativeSpans: []prompb.BucketSpan{{Offset: 0, Length: 3}}, - NegativeDeltas: []int64{4, -2, -1}, - }, - }, - Exemplars: []prompb.Exemplar{ - {Value: 2}, + { + metricFamilyName: "test_hist", + ls: labelsAnother, + meta: metadata.Metadata{}, + t: 0, + ct: 0, + h: &histogram.Histogram{ + Count: 4, + Schema: 1, + ZeroThreshold: defaultZeroThreshold, + ZeroCount: 0, + NegativeSpans: []histogram.Span{{Offset: 0, Length: 3}}, + NegativeBuckets: []int64{4, -2, -1}, }, + es: []exemplar.Exemplar{{Value: 2}}, }, } }, @@ -843,7 +854,8 @@ func TestPrometheusConverter_addExponentialHistogramDataPoints(t *testing.T) { t.Run(tt.name, func(t *testing.T) { metric := tt.metric() - converter := NewPrometheusConverter() + mockAppender := &mockCombinedAppender{} + converter := NewPrometheusConverter(mockAppender) namer := otlptranslator.MetricNamer{ WithMetricSuffixes: true, } @@ -857,14 +869,17 @@ func TestPrometheusConverter_addExponentialHistogramDataPoints(t *testing.T) { ExportCreatedMetric: true, PromoteScopeMetadata: tt.promoteScope, }, - prompb.MetricMetadata{MetricFamilyName: name}, + name, pmetric.AggregationTemporalityCumulative, tt.scope, + metadata.Metadata{}, ) require.NoError(t, err) require.Empty(t, annots) - require.Equal(t, tt.wantSeries(), converter.unique) + require.NoError(t, mockAppender.Commit()) + + requireEqual(t, tt.wantSeries(), mockAppender.histograms) require.Empty(t, converter.conflicts) }) } @@ -880,7 +895,7 @@ func TestConvertExplicitHistogramBucketsToNHCBLayout(t *testing.T) { name: "zero offset", buckets: []uint64{4, 3, 2, 1}, wantLayout: expectedBucketLayout{ - wantSpans: []prompb.BucketSpan{ + wantSpans: []histogram.Span{ { Offset: 0, Length: 4, @@ -893,7 +908,7 @@ func TestConvertExplicitHistogramBucketsToNHCBLayout(t *testing.T) { name: "leading empty buckets", buckets: []uint64{0, 0, 1, 1, 2, 3}, wantLayout: expectedBucketLayout{ - wantSpans: []prompb.BucketSpan{ + wantSpans: []histogram.Span{ { Offset: 2, Length: 4, @@ -906,7 +921,7 @@ func TestConvertExplicitHistogramBucketsToNHCBLayout(t *testing.T) { name: "trailing empty buckets", buckets: []uint64{0, 0, 1, 1, 2, 3, 0, 0}, // TODO: add tests for 3 trailing buckets wantLayout: expectedBucketLayout{ - wantSpans: []prompb.BucketSpan{ + wantSpans: []histogram.Span{ { Offset: 2, Length: 6, @@ -919,7 +934,7 @@ func TestConvertExplicitHistogramBucketsToNHCBLayout(t *testing.T) { name: "bucket gap of 2", buckets: []uint64{1, 2, 0, 0, 2}, wantLayout: expectedBucketLayout{ - wantSpans: []prompb.BucketSpan{ + wantSpans: []histogram.Span{ { Offset: 0, Length: 5, @@ -932,7 +947,7 @@ func TestConvertExplicitHistogramBucketsToNHCBLayout(t *testing.T) { name: "bucket gap > 2", buckets: []uint64{1, 2, 0, 0, 0, 2, 4, 4}, wantLayout: expectedBucketLayout{ - wantSpans: []prompb.BucketSpan{ + wantSpans: []histogram.Span{ { Offset: 0, Length: 2, @@ -949,7 +964,7 @@ func TestConvertExplicitHistogramBucketsToNHCBLayout(t *testing.T) { name: "multiple bucket gaps", buckets: []uint64{0, 0, 1, 2, 0, 0, 0, 2, 4, 4, 0, 0}, wantLayout: expectedBucketLayout{ - wantSpans: []prompb.BucketSpan{ + wantSpans: []histogram.Span{ { Offset: 2, Length: 2, @@ -1008,7 +1023,7 @@ func TestHistogramToCustomBucketsHistogram(t *testing.T) { tests := []struct { name string hist func() pmetric.HistogramDataPoint - wantNativeHist func() prompb.Histogram + wantNativeHist func() *histogram.Histogram wantErrMessage string }{ { @@ -1024,15 +1039,14 @@ func TestHistogramToCustomBucketsHistogram(t *testing.T) { pt.ExplicitBounds().FromRaw([]float64{0, 1}) return pt }, - wantNativeHist: func() prompb.Histogram { - return prompb.Histogram{ - Count: &prompb.Histogram_CountInt{CountInt: 2}, - Sum: 10.1, - Schema: -53, - PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 2}}, - PositiveDeltas: []int64{1, 0}, - CustomValues: []float64{0, 1}, - Timestamp: 500, + wantNativeHist: func() *histogram.Histogram { + return &histogram.Histogram{ + Count: 2, + Sum: 10.1, + Schema: -53, + PositiveSpans: []histogram.Span{{Offset: 0, Length: 2}}, + PositiveBuckets: []int64{1, 0}, + CustomValues: []float64{0, 1}, } }, }, @@ -1048,14 +1062,13 @@ func TestHistogramToCustomBucketsHistogram(t *testing.T) { pt.ExplicitBounds().FromRaw([]float64{0, 1}) return pt }, - wantNativeHist: func() prompb.Histogram { - return prompb.Histogram{ - Count: &prompb.Histogram_CountInt{CountInt: 4}, - Schema: -53, - PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 2}}, - PositiveDeltas: []int64{2, 0}, - CustomValues: []float64{0, 1}, - Timestamp: 500, + wantNativeHist: func() *histogram.Histogram { + return &histogram.Histogram{ + Count: 4, + Schema: -53, + PositiveSpans: []histogram.Span{{Offset: 0, Length: 2}}, + PositiveBuckets: []int64{2, 0}, + CustomValues: []float64{0, 1}, } }, }, @@ -1094,7 +1107,7 @@ func TestPrometheusConverter_addCustomBucketsHistogramDataPoints(t *testing.T) { metric func() pmetric.Metric scope scope promoteScope bool - wantSeries func() map[uint64]*prompb.TimeSeries + wantSeries func() []combinedHistogram }{ { name: "histogram data points with same labels and without scope promotion", @@ -1123,36 +1136,43 @@ func TestPrometheusConverter_addCustomBucketsHistogramDataPoints(t *testing.T) { }, scope: defaultScope, promoteScope: false, - wantSeries: func() map[uint64]*prompb.TimeSeries { - labels := []prompb.Label{ - {Name: model.MetricNameLabel, Value: "test_hist_to_nhcb"}, - {Name: "attr", Value: "test_attr"}, - } - return map[uint64]*prompb.TimeSeries{ - timeSeriesSignature(labels): { - Labels: labels, - Histograms: []prompb.Histogram{ - { - Count: &prompb.Histogram_CountInt{CountInt: 3}, - Sum: 3, - Schema: -53, - PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 3}}, - PositiveDeltas: []int64{2, -2, 1}, - CustomValues: []float64{5, 10}, - }, - { - Count: &prompb.Histogram_CountInt{CountInt: 11}, - Sum: 5, - Schema: -53, - PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 3}}, - PositiveDeltas: []int64{3, 5, -8}, - CustomValues: []float64{0, 1}, - }, + wantSeries: func() []combinedHistogram { + lbls := labels.FromStrings( + model.MetricNameLabel, "test_hist_to_nhcb", + "attr", "test_attr", + ) + return []combinedHistogram{ + { + metricFamilyName: "test_hist_to_nhcb", + ls: lbls, + meta: metadata.Metadata{}, + t: 0, + ct: 0, + h: &histogram.Histogram{ + Count: 3, + Sum: 3, + Schema: -53, + PositiveSpans: []histogram.Span{{Offset: 0, Length: 3}}, + PositiveBuckets: []int64{2, -2, 1}, + CustomValues: []float64{5, 10}, }, - Exemplars: []prompb.Exemplar{ - {Value: 1}, - {Value: 2}, + es: []exemplar.Exemplar{{Value: 1}}, + }, + { + metricFamilyName: "test_hist_to_nhcb", + ls: lbls, + meta: metadata.Metadata{}, + t: 0, + ct: 0, + h: &histogram.Histogram{ + Count: 11, + Sum: 5, + Schema: -53, + PositiveSpans: []histogram.Span{{Offset: 0, Length: 3}}, + PositiveBuckets: []int64{3, 5, -8}, + CustomValues: []float64{0, 1}, }, + es: []exemplar.Exemplar{{Value: 2}}, }, } }, @@ -1184,41 +1204,48 @@ func TestPrometheusConverter_addCustomBucketsHistogramDataPoints(t *testing.T) { }, scope: defaultScope, promoteScope: true, - wantSeries: func() map[uint64]*prompb.TimeSeries { - labels := []prompb.Label{ - {Name: model.MetricNameLabel, Value: "test_hist_to_nhcb"}, - {Name: "attr", Value: "test_attr"}, - {Name: "otel_scope_name", Value: defaultScope.name}, - {Name: "otel_scope_schema_url", Value: defaultScope.schemaURL}, - {Name: "otel_scope_version", Value: defaultScope.version}, - {Name: "otel_scope_attr1", Value: "value1"}, - {Name: "otel_scope_attr2", Value: "value2"}, - } - return map[uint64]*prompb.TimeSeries{ - timeSeriesSignature(labels): { - Labels: labels, - Histograms: []prompb.Histogram{ - { - Count: &prompb.Histogram_CountInt{CountInt: 3}, - Sum: 3, - Schema: -53, - PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 3}}, - PositiveDeltas: []int64{2, -2, 1}, - CustomValues: []float64{5, 10}, - }, - { - Count: &prompb.Histogram_CountInt{CountInt: 11}, - Sum: 5, - Schema: -53, - PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 3}}, - PositiveDeltas: []int64{3, 5, -8}, - CustomValues: []float64{0, 1}, - }, + wantSeries: func() []combinedHistogram { + lbls := labels.FromStrings( + model.MetricNameLabel, "test_hist_to_nhcb", + "attr", "test_attr", + "otel_scope_name", defaultScope.name, + "otel_scope_schema_url", defaultScope.schemaURL, + "otel_scope_version", defaultScope.version, + "otel_scope_attr1", "value1", + "otel_scope_attr2", "value2", + ) + return []combinedHistogram{ + { + metricFamilyName: "test_hist_to_nhcb", + ls: lbls, + meta: metadata.Metadata{}, + t: 0, + ct: 0, + h: &histogram.Histogram{ + Count: 3, + Sum: 3, + Schema: -53, + PositiveSpans: []histogram.Span{{Offset: 0, Length: 3}}, + PositiveBuckets: []int64{2, -2, 1}, + CustomValues: []float64{5, 10}, }, - Exemplars: []prompb.Exemplar{ - {Value: 1}, - {Value: 2}, + es: []exemplar.Exemplar{{Value: 1}}, + }, + { + metricFamilyName: "test_hist_to_nhcb", + ls: lbls, + meta: metadata.Metadata{}, + t: 0, + ct: 0, + h: &histogram.Histogram{ + Count: 11, + Sum: 5, + Schema: -53, + PositiveSpans: []histogram.Span{{Offset: 0, Length: 3}}, + PositiveBuckets: []int64{3, 5, -8}, + CustomValues: []float64{0, 1}, }, + es: []exemplar.Exemplar{{Value: 2}}, }, } }, @@ -1250,48 +1277,48 @@ func TestPrometheusConverter_addCustomBucketsHistogramDataPoints(t *testing.T) { }, scope: defaultScope, promoteScope: false, - wantSeries: func() map[uint64]*prompb.TimeSeries { - labels := []prompb.Label{ - {Name: model.MetricNameLabel, Value: "test_hist_to_nhcb"}, - {Name: "attr", Value: "test_attr"}, - } - labelsAnother := []prompb.Label{ - {Name: model.MetricNameLabel, Value: "test_hist_to_nhcb"}, - {Name: "attr", Value: "test_attr_two"}, - } + wantSeries: func() []combinedHistogram { + lbls := labels.FromStrings( + model.MetricNameLabel, "test_hist_to_nhcb", + "attr", "test_attr", + ) + labelsAnother := labels.FromStrings( + model.MetricNameLabel, "test_hist_to_nhcb", + "attr", "test_attr_two", + ) - return map[uint64]*prompb.TimeSeries{ - timeSeriesSignature(labels): { - Labels: labels, - Histograms: []prompb.Histogram{ - { - Count: &prompb.Histogram_CountInt{CountInt: 6}, - Sum: 3, - Schema: -53, - PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 2}}, - PositiveDeltas: []int64{4, -2}, - CustomValues: []float64{0, 1}, - }, - }, - Exemplars: []prompb.Exemplar{ - {Value: 1}, + return []combinedHistogram{ + { + metricFamilyName: "test_hist_to_nhcb", + ls: lbls, + meta: metadata.Metadata{}, + t: 0, + ct: 0, + h: &histogram.Histogram{ + Count: 6, + Sum: 3, + Schema: -53, + PositiveSpans: []histogram.Span{{Offset: 0, Length: 2}}, + PositiveBuckets: []int64{4, -2}, + CustomValues: []float64{0, 1}, }, + es: []exemplar.Exemplar{{Value: 1}}, }, - timeSeriesSignature(labelsAnother): { - Labels: labelsAnother, - Histograms: []prompb.Histogram{ - { - Count: &prompb.Histogram_CountInt{CountInt: 11}, - Sum: 5, - Schema: -53, - PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 2}}, - PositiveDeltas: []int64{3, 5}, - CustomValues: []float64{0, 1}, - }, - }, - Exemplars: []prompb.Exemplar{ - {Value: 2}, + { + metricFamilyName: "test_hist_to_nhcb", + ls: labelsAnother, + meta: metadata.Metadata{}, + t: 0, + ct: 0, + h: &histogram.Histogram{ + Count: 11, + Sum: 5, + Schema: -53, + PositiveSpans: []histogram.Span{{Offset: 0, Length: 2}}, + PositiveBuckets: []int64{3, 5}, + CustomValues: []float64{0, 1}, }, + es: []exemplar.Exemplar{{Value: 2}}, }, } }, @@ -1301,7 +1328,8 @@ func TestPrometheusConverter_addCustomBucketsHistogramDataPoints(t *testing.T) { t.Run(tt.name, func(t *testing.T) { metric := tt.metric() - converter := NewPrometheusConverter() + mockAppender := &mockCombinedAppender{} + converter := NewPrometheusConverter(mockAppender) namer := otlptranslator.MetricNamer{ WithMetricSuffixes: true, } @@ -1316,15 +1344,18 @@ func TestPrometheusConverter_addCustomBucketsHistogramDataPoints(t *testing.T) { ConvertHistogramsToNHCB: true, PromoteScopeMetadata: tt.promoteScope, }, - prompb.MetricMetadata{MetricFamilyName: name}, + name, pmetric.AggregationTemporalityCumulative, tt.scope, + metadata.Metadata{}, ) require.NoError(t, err) require.Empty(t, annots) - require.Equal(t, tt.wantSeries(), converter.unique) + require.NoError(t, mockAppender.Commit()) + + requireEqual(t, tt.wantSeries(), mockAppender.histograms) require.Empty(t, converter.conflicts) }) } diff --git a/storage/remote/otlptranslator/prometheusremotewrite/labels/labels.go b/storage/remote/otlptranslator/prometheusremotewrite/labels/labels.go new file mode 100644 index 0000000000..2442add46d --- /dev/null +++ b/storage/remote/otlptranslator/prometheusremotewrite/labels/labels.go @@ -0,0 +1,318 @@ +// Copyright 2025 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. + +// This file is copied from model/labels/labels_slicelabels.go and +// labels_common.go. Unused functions are removed. + +package labels + +import ( + "slices" + "strings" + "unsafe" + + "github.com/cespare/xxhash/v2" + + common "github.com/prometheus/prometheus/model/labels" +) + +const sep = '\xff' // Used between labels in `Bytes` and `Hash`. +var seps = []byte{sep} // Used with Hash, which has no WriteByte method. + +// Labels is a sorted set of labels. Order has to be guaranteed upon +// instantiation. +type Labels []common.Label + +func (ls Labels) Len() int { return len(ls) } +func (ls Labels) Swap(i, j int) { ls[i], ls[j] = ls[j], ls[i] } +func (ls Labels) Less(i, j int) bool { return ls[i].Name < ls[j].Name } + +// Hash returns a hash value for the label set. +// Note: the result is not guaranteed to be consistent across different runs of Prometheus. +func (ls Labels) Hash() uint64 { + // Use xxhash.Sum64(b) for fast path as it's faster. + b := make([]byte, 0, 1024) + for i, v := range ls { + if len(b)+len(v.Name)+len(v.Value)+2 >= cap(b) { + // If labels entry is 1KB+ do not allocate whole entry. + h := xxhash.New() + _, _ = h.Write(b) + for _, v := range ls[i:] { + _, _ = h.WriteString(v.Name) + _, _ = h.Write(seps) + _, _ = h.WriteString(v.Value) + _, _ = h.Write(seps) + } + return h.Sum64() + } + + b = append(b, v.Name...) + b = append(b, sep) + b = append(b, v.Value...) + b = append(b, sep) + } + return xxhash.Sum64(b) +} + +// Get returns the value for the label with the given name. +// Returns an empty string if the label doesn't exist. +func (ls Labels) Get(name string) string { + for _, l := range ls { + if l.Name == name { + return l.Value + } + } + return "" +} + +// Has returns true if the label with the given name is present. +func (ls Labels) Has(name string) bool { + for _, l := range ls { + if l.Name == name { + return true + } + } + return false +} + +// Equal returns whether the two label sets are equal. +func Equal(ls, o Labels) bool { + return slices.Equal(ls, o) +} + +// EmptyLabels returns n empty Labels value, for convenience. +func EmptyLabels() Labels { + return Labels{} +} + +// FromStrings creates new labels from pairs of strings. +func FromStrings(ss ...string) Labels { + if len(ss)%2 != 0 { + panic("invalid number of strings") + } + res := make(Labels, 0, len(ss)/2) + for i := 0; i < len(ss); i += 2 { + res = append(res, common.Label{Name: ss[i], Value: ss[i+1]}) + } + + slices.SortFunc(res, func(a, b common.Label) int { return strings.Compare(a.Name, b.Name) }) + return res +} + +// Range calls f on each label. +func (ls Labels) Range(f func(l common.Label)) { + for _, l := range ls { + f(l) + } +} + +// Builder allows modifying Labels. +type Builder struct { + base Labels + del []string + add []common.Label +} + +// NewBuilder returns a new LabelsBuilder. +func NewBuilder(base Labels) *Builder { + b := &Builder{ + del: make([]string, 0, 5), + add: make([]common.Label, 0, 5), + } + b.Reset(base) + return b +} + +// Reset clears all current state for the builder. +func (b *Builder) Reset(base Labels) { + b.base = base + b.del = b.del[:0] + b.add = b.add[:0] + b.base.Range(func(l common.Label) { + if l.Value == "" { + b.del = append(b.del, l.Name) + } + }) +} + +// Del deletes the label of the given name. +func (b *Builder) Del(ns ...string) *Builder { + for _, n := range ns { + for i, a := range b.add { + if a.Name == n { + b.add = append(b.add[:i], b.add[i+1:]...) + } + } + b.del = append(b.del, n) + } + return b +} + +// Keep removes all labels from the base except those with the given names. +func (b *Builder) Keep(ns ...string) *Builder { + b.base.Range(func(l common.Label) { + if slices.Contains(ns, l.Name) { + return + } + b.del = append(b.del, l.Name) + }) + return b +} + +// Set the name/value pair as a label. A value of "" means delete that label. +func (b *Builder) Set(n, v string) *Builder { + if v == "" { + // Empty labels are the same as missing labels. + return b.Del(n) + } + for i, a := range b.add { + if a.Name == n { + b.add[i].Value = v + return b + } + } + b.add = append(b.add, common.Label{Name: n, Value: v}) + + return b +} + +func (b *Builder) Get(n string) string { + // Del() removes entries from .add but Set() does not remove from .del, so check .add first. + for _, a := range b.add { + if a.Name == n { + return a.Value + } + } + if slices.Contains(b.del, n) { + return "" + } + return b.base.Get(n) +} + +// Range calls f on each label in the Builder. +func (b *Builder) Range(f func(l common.Label)) { + // Stack-based arrays to avoid heap allocation in most cases. + var addStack [128]common.Label + var delStack [128]string + // Take a copy of add and del, so they are unaffected by calls to Set() or Del(). + origAdd, origDel := append(addStack[:0], b.add...), append(delStack[:0], b.del...) + b.base.Range(func(l common.Label) { + if !slices.Contains(origDel, l.Name) && !common.Contains(origAdd, l.Name) { + f(l) + } + }) + for _, a := range origAdd { + f(a) + } +} + +// Labels returns the labels from the builder. +// If no modifications were made, the original labels are returned. +func (b *Builder) Labels() Labels { + if len(b.del) == 0 && len(b.add) == 0 { + return b.base + } + + expectedSize := len(b.base) + len(b.add) - len(b.del) + if expectedSize < 1 { + expectedSize = 1 + } + res := make(Labels, 0, expectedSize) + for _, l := range b.base { + if slices.Contains(b.del, l.Name) || common.Contains(b.add, l.Name) { + continue + } + res = append(res, l) + } + if len(b.add) > 0 { // Base is already in order, so we only need to sort if we add to it. + res = append(res, b.add...) + slices.SortFunc(res, func(a, b common.Label) int { return strings.Compare(a.Name, b.Name) }) + } + return res +} + +// ScratchBuilder allows efficient construction of a Labels from scratch. +type ScratchBuilder struct { + add Labels +} + +// SymbolTable is no-op, just for api parity with dedupelabels. +type SymbolTable struct{} + +func NewSymbolTable() *SymbolTable { return nil } + +func (t *SymbolTable) Len() int { return 0 } + +// NewScratchBuilder creates a ScratchBuilder initialized for Labels with n entries. +func NewScratchBuilder(n int) ScratchBuilder { + return ScratchBuilder{add: make([]common.Label, 0, n)} +} + +// NewBuilderWithSymbolTable creates a Builder, for api parity with dedupelabels. +func NewBuilderWithSymbolTable(_ *SymbolTable) *Builder { + return NewBuilder(EmptyLabels()) +} + +// NewScratchBuilderWithSymbolTable creates a ScratchBuilder, for api parity with dedupelabels. +func NewScratchBuilderWithSymbolTable(_ *SymbolTable, n int) ScratchBuilder { + return NewScratchBuilder(n) +} + +func (b *ScratchBuilder) SetSymbolTable(_ *SymbolTable) { + // no-op +} + +func (b *ScratchBuilder) Reset() { + b.add = b.add[:0] +} + +// Add a name/value pair. +// Note if you Add the same name twice you will get a duplicate label, which is invalid. +func (b *ScratchBuilder) Add(name, value string) { + b.add = append(b.add, common.Label{Name: name, Value: value}) +} + +// UnsafeAddBytes adds a name/value pair, using []byte instead of string. +// The '-tags stringlabels' version of this function is unsafe, hence the name. +// This version is safe - it copies the strings immediately - but we keep the same name so everything compiles. +func (b *ScratchBuilder) UnsafeAddBytes(name, value []byte) { + b.add = append(b.add, common.Label{Name: string(name), Value: string(value)}) +} + +// Sort the labels added so far by name. +func (b *ScratchBuilder) Sort() { + slices.SortFunc(b.add, func(a, b common.Label) int { return strings.Compare(a.Name, b.Name) }) +} + +// Assign is for when you already have a Labels which you want this ScratchBuilder to return. +func (b *ScratchBuilder) Assign(ls Labels) { + b.add = append(b.add[:0], ls...) // Copy on top of our slice, so we don't retain the input slice. +} + +// Labels returns the name/value pairs added so far as a Labels object. +// Note: if you want them sorted, call Sort() first. +func (b *ScratchBuilder) Labels() Labels { + // Copy the slice, so the next use of ScratchBuilder doesn't overwrite. + return append([]common.Label{}, b.add...) +} + +// Overwrite the newly-built Labels out to ls. +// Callers must ensure that there are no other references to ls, or any strings fetched from it. +func (b *ScratchBuilder) Overwrite(ls *Labels) { + *ls = append((*ls)[:0], b.add...) +} + +// SizeOfLabels returns the approximate space required for n copies of a label. +func SizeOfLabels(name, value string, n uint64) uint64 { + return (uint64(len(name)) + uint64(unsafe.Sizeof(name)) + uint64(len(value)) + uint64(unsafe.Sizeof(value))) * n +} diff --git a/storage/remote/otlptranslator/prometheusremotewrite/labels/labels_test.go b/storage/remote/otlptranslator/prometheusremotewrite/labels/labels_test.go new file mode 100644 index 0000000000..adeab87be1 --- /dev/null +++ b/storage/remote/otlptranslator/prometheusremotewrite/labels/labels_test.go @@ -0,0 +1,424 @@ +// Copyright 2025 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 labels + +import ( + "fmt" + "net/http" + "strconv" + "strings" + "testing" + + "github.com/stretchr/testify/require" + + common "github.com/prometheus/prometheus/model/labels" +) + +func TestLabels_Equal(t *testing.T) { + labels := FromStrings( + "aaa", "111", + "bbb", "222") + + tests := []struct { + compared Labels + expected bool + }{ + { + compared: FromStrings( + "aaa", "111", + "bbb", "222", + "ccc", "333"), + expected: false, + }, + { + compared: FromStrings( + "aaa", "111", + "bar", "222"), + expected: false, + }, + { + compared: FromStrings( + "aaa", "111", + "bbb", "233"), + expected: false, + }, + { + compared: FromStrings( + "aaa", "111", + "bbb", "222"), + expected: true, + }, + } + + for i, test := range tests { + got := Equal(labels, test.compared) + require.Equal(t, test.expected, got, "unexpected comparison result for test case %d", i) + } +} + +func TestLabels_FromStrings(t *testing.T) { + labels := FromStrings("aaa", "111", "bbb", "222") + x := 0 + labels.Range(func(l common.Label) { + switch x { + case 0: + require.Equal(t, common.Label{Name: "aaa", Value: "111"}, l, "unexpected value") + case 1: + require.Equal(t, common.Label{Name: "bbb", Value: "222"}, l, "unexpected value") + default: + t.Fatalf("unexpected labelset value %d: %v", x, l) + } + x++ + }) + + require.Panics(t, func() { FromStrings("aaa", "111", "bbb") }) //nolint:staticcheck // Ignore SA5012, error is intentional test. +} + +func TestLabels_Has(t *testing.T) { + tests := []struct { + input string + expected bool + }{ + { + input: "foo", + expected: false, + }, + { + input: "aaa", + expected: true, + }, + } + + labelsSet := FromStrings( + "aaa", "111", + "bbb", "222") + + for i, test := range tests { + got := labelsSet.Has(test.input) + require.Equal(t, test.expected, got, "unexpected comparison result for test case %d", i) + } +} + +func TestLabels_Get(t *testing.T) { + require.Empty(t, FromStrings("aaa", "111", "bbb", "222").Get("foo")) + require.Equal(t, "111", FromStrings("aaaa", "111", "bbb", "222").Get("aaaa")) + require.Equal(t, "222", FromStrings("aaaa", "111", "bbb", "222").Get("bbb")) +} + +func ScratchBuilderForBenchmark() ScratchBuilder { + // (Only relevant to -tags dedupelabels: stuff the symbol table before adding the real labels, to avoid having everything fitting into 1 byte.) + b := NewScratchBuilder(256) + for i := 0; i < 256; i++ { + b.Add(fmt.Sprintf("name%d", i), fmt.Sprintf("value%d", i)) + } + b.Labels() + b.Reset() + return b +} + +func NewForBenchmark(ls ...common.Label) Labels { + b := ScratchBuilderForBenchmark() + for _, l := range ls { + b.Add(l.Name, l.Value) + } + b.Sort() + return b.Labels() +} + +func FromStringsForBenchmark(ss ...string) Labels { + if len(ss)%2 != 0 { + panic("invalid number of strings") + } + b := ScratchBuilderForBenchmark() + for i := 0; i < len(ss); i += 2 { + b.Add(ss[i], ss[i+1]) + } + b.Sort() + return b.Labels() +} + +// BenchmarkLabels_Get was written to check whether a binary search can improve the performance vs the linear search implementation +// The results have shown that binary search would only be better when searching last labels in scenarios with more than 10 labels. +// In the following list, `old` is the linear search while `new` is the binary search implementation (without calling sort.Search, which performs even worse here) +// +// name old time/op new time/op delta +// Labels_Get/with_5_labels/get_first_label 5.12ns ± 0% 14.24ns ± 0% ~ (p=1.000 n=1+1) +// Labels_Get/with_5_labels/get_middle_label 13.5ns ± 0% 18.5ns ± 0% ~ (p=1.000 n=1+1) +// Labels_Get/with_5_labels/get_last_label 21.9ns ± 0% 18.9ns ± 0% ~ (p=1.000 n=1+1) +// Labels_Get/with_10_labels/get_first_label 5.11ns ± 0% 19.47ns ± 0% ~ (p=1.000 n=1+1) +// Labels_Get/with_10_labels/get_middle_label 26.2ns ± 0% 19.3ns ± 0% ~ (p=1.000 n=1+1) +// Labels_Get/with_10_labels/get_last_label 42.8ns ± 0% 23.4ns ± 0% ~ (p=1.000 n=1+1) +// Labels_Get/with_30_labels/get_first_label 5.10ns ± 0% 24.63ns ± 0% ~ (p=1.000 n=1+1) +// Labels_Get/with_30_labels/get_middle_label 75.8ns ± 0% 29.7ns ± 0% ~ (p=1.000 n=1+1) +// Labels_Get/with_30_labels/get_last_label 169ns ± 0% 29ns ± 0% ~ (p=1.000 n=1+1) +func BenchmarkLabels_Get(b *testing.B) { + maxLabels := 30 + allLabels := make([]common.Label, maxLabels) + for i := 0; i < maxLabels; i++ { + allLabels[i] = common.Label{Name: strings.Repeat(string('a'+byte(i)), 5+(i%5))} + } + for _, size := range []int{5, 10, maxLabels} { + b.Run(fmt.Sprintf("with %d labels", size), func(b *testing.B) { + labels := NewForBenchmark(allLabels[:size]...) + for _, scenario := range []struct { + desc, label string + }{ + {"first label", allLabels[0].Name}, + {"middle label", allLabels[size/2].Name}, + {"last label", allLabels[size-1].Name}, + {"not-found label", "benchmark"}, + } { + b.Run(scenario.desc, func(b *testing.B) { + b.Run("get", func(b *testing.B) { + for i := 0; i < b.N; i++ { + _ = labels.Get(scenario.label) + } + }) + b.Run("has", func(b *testing.B) { + for i := 0; i < b.N; i++ { + _ = labels.Has(scenario.label) + } + }) + }) + } + }) + } +} + +var comparisonBenchmarkScenarios = []struct { + desc string + base, other Labels +}{ + { + "equal", + FromStringsForBenchmark("a_label_name", "a_label_value", "another_label_name", "another_label_value"), + FromStringsForBenchmark("a_label_name", "a_label_value", "another_label_name", "another_label_value"), + }, + { + "not equal", + FromStringsForBenchmark("a_label_name", "a_label_value", "another_label_name", "another_label_value"), + FromStringsForBenchmark("a_label_name", "a_label_value", "another_label_name", "a_different_label_value"), + }, + { + "different sizes", + FromStringsForBenchmark("a_label_name", "a_label_value", "another_label_name", "another_label_value"), + FromStringsForBenchmark("a_label_name", "a_label_value"), + }, + { + "lots", + FromStringsForBenchmark("aaa", "bbb", "ccc", "ddd", "eee", "fff", "ggg", "hhh", "iii", "jjj", "kkk", "lll", "mmm", "nnn", "ooo", "ppp", "qqq", "rrz"), + FromStringsForBenchmark("aaa", "bbb", "ccc", "ddd", "eee", "fff", "ggg", "hhh", "iii", "jjj", "kkk", "lll", "mmm", "nnn", "ooo", "ppp", "qqq", "rrr"), + }, + { + "real long equal", + FromStringsForBenchmark("__name__", "kube_pod_container_status_last_terminated_exitcode", "cluster", "prod-af-north-0", " container", "prometheus", "instance", "kube-state-metrics-0:kube-state-metrics:ksm", "job", "kube-state-metrics/kube-state-metrics", " namespace", "observability-prometheus", "pod", "observability-prometheus-0", "uid", "d3ec90b2-4975-4607-b45d-b9ad64bb417e"), + FromStringsForBenchmark("__name__", "kube_pod_container_status_last_terminated_exitcode", "cluster", "prod-af-north-0", " container", "prometheus", "instance", "kube-state-metrics-0:kube-state-metrics:ksm", "job", "kube-state-metrics/kube-state-metrics", " namespace", "observability-prometheus", "pod", "observability-prometheus-0", "uid", "d3ec90b2-4975-4607-b45d-b9ad64bb417e"), + }, + { + "real long different end", + FromStringsForBenchmark("__name__", "kube_pod_container_status_last_terminated_exitcode", "cluster", "prod-af-north-0", " container", "prometheus", "instance", "kube-state-metrics-0:kube-state-metrics:ksm", "job", "kube-state-metrics/kube-state-metrics", " namespace", "observability-prometheus", "pod", "observability-prometheus-0", "uid", "d3ec90b2-4975-4607-b45d-b9ad64bb417e"), + FromStringsForBenchmark("__name__", "kube_pod_container_status_last_terminated_exitcode", "cluster", "prod-af-north-0", " container", "prometheus", "instance", "kube-state-metrics-0:kube-state-metrics:ksm", "job", "kube-state-metrics/kube-state-metrics", " namespace", "observability-prometheus", "pod", "observability-prometheus-0", "uid", "deadbeef-0000-1111-2222-b9ad64bb417e"), + }, +} + +func BenchmarkLabels_Equals(b *testing.B) { + for _, scenario := range comparisonBenchmarkScenarios { + b.Run(scenario.desc, func(b *testing.B) { + b.ResetTimer() + for i := 0; i < b.N; i++ { + _ = Equal(scenario.base, scenario.other) + } + }) + } +} + +func TestBuilder(t *testing.T) { + reuseBuilder := NewBuilderWithSymbolTable(NewSymbolTable()) + for i, tcase := range []struct { + base Labels + del []string + keep []string + set []common.Label + want Labels + }{ + { + base: FromStrings("aaa", "111"), + want: FromStrings("aaa", "111"), + }, + { + base: EmptyLabels(), + set: []common.Label{{Name: "aaa", Value: "444"}, {Name: "bbb", Value: "555"}, {Name: "ccc", Value: "666"}}, + want: FromStrings("aaa", "444", "bbb", "555", "ccc", "666"), + }, + { + base: FromStrings("aaa", "111", "bbb", "222", "ccc", "333"), + set: []common.Label{{Name: "aaa", Value: "444"}, {Name: "bbb", Value: "555"}, {Name: "ccc", Value: "666"}}, + want: FromStrings("aaa", "444", "bbb", "555", "ccc", "666"), + }, + { + base: FromStrings("aaa", "111", "bbb", "222", "ccc", "333"), + del: []string{"bbb"}, + want: FromStrings("aaa", "111", "ccc", "333"), + }, + { + set: []common.Label{{Name: "aaa", Value: "111"}, {Name: "bbb", Value: "222"}, {Name: "ccc", Value: "333"}}, + del: []string{"bbb"}, + want: FromStrings("aaa", "111", "ccc", "333"), + }, + { + base: FromStrings("aaa", "111"), + set: []common.Label{{Name: "bbb", Value: "222"}}, + want: FromStrings("aaa", "111", "bbb", "222"), + }, + { + base: FromStrings("aaa", "111"), + set: []common.Label{{Name: "bbb", Value: "222"}, {Name: "bbb", Value: "333"}}, + want: FromStrings("aaa", "111", "bbb", "333"), + }, + { + base: FromStrings("aaa", "111", "bbb", "222", "ccc", "333"), + del: []string{"bbb"}, + set: []common.Label{{Name: "ddd", Value: "444"}}, + want: FromStrings("aaa", "111", "ccc", "333", "ddd", "444"), + }, + { // Blank value is interpreted as delete. + base: FromStrings("aaa", "111", "bbb", "", "ccc", "333"), + want: FromStrings("aaa", "111", "ccc", "333"), + }, + { + base: FromStrings("aaa", "111", "bbb", "222", "ccc", "333"), + set: []common.Label{{Name: "bbb", Value: ""}}, + want: FromStrings("aaa", "111", "ccc", "333"), + }, + { + base: FromStrings("aaa", "111", "bbb", "222", "ccc", "333"), + keep: []string{"bbb"}, + want: FromStrings("bbb", "222"), + }, + { + base: FromStrings("aaa", "111", "bbb", "222", "ccc", "333"), + keep: []string{"aaa", "ccc"}, + want: FromStrings("aaa", "111", "ccc", "333"), + }, + { + base: FromStrings("aaa", "111", "bbb", "222", "ccc", "333"), + del: []string{"bbb"}, + set: []common.Label{{Name: "ddd", Value: "444"}}, + keep: []string{"aaa", "ddd"}, + want: FromStrings("aaa", "111", "ddd", "444"), + }, + } { + test := func(t *testing.T, b *Builder) { + for _, lbl := range tcase.set { + b.Set(lbl.Name, lbl.Value) + } + if len(tcase.keep) > 0 { + b.Keep(tcase.keep...) + } + b.Del(tcase.del...) + require.True(t, Equal(tcase.want, b.Labels())) + + // Check what happens when we call Range and mutate the builder. + b.Range(func(l common.Label) { + if l.Name == "aaa" || l.Name == "bbb" { + b.Del(l.Name) + } + }) + // require.Equal(t, tcase.want.BytesWithoutLabels(nil, "aaa", "bbb"), b.Labels().Bytes(nil)) + } + t.Run(fmt.Sprintf("NewBuilder %d", i), func(t *testing.T) { + test(t, NewBuilder(tcase.base)) + }) + t.Run(fmt.Sprintf("NewSymbolTable %d", i), func(t *testing.T) { + b := NewBuilderWithSymbolTable(NewSymbolTable()) + b.Reset(tcase.base) + test(t, b) + }) + t.Run(fmt.Sprintf("reuseBuilder %d", i), func(t *testing.T) { + reuseBuilder.Reset(tcase.base) + test(t, reuseBuilder) + }) + } + t.Run("set_after_del", func(t *testing.T) { + b := NewBuilder(FromStrings("aaa", "111")) + b.Del("bbb") + b.Set("bbb", "222") + require.Equal(t, FromStrings("aaa", "111", "bbb", "222"), b.Labels()) + require.Equal(t, "222", b.Get("bbb")) + }) +} + +func TestScratchBuilder(t *testing.T) { + for i, tcase := range []struct { + add []common.Label + want Labels + }{ + { + add: []common.Label{}, + want: EmptyLabels(), + }, + { + add: []common.Label{{Name: "aaa", Value: "111"}}, + want: FromStrings("aaa", "111"), + }, + { + add: []common.Label{{Name: "aaa", Value: "111"}, {Name: "bbb", Value: "222"}, {Name: "ccc", Value: "333"}}, + want: FromStrings("aaa", "111", "bbb", "222", "ccc", "333"), + }, + { + add: []common.Label{{Name: "bbb", Value: "222"}, {Name: "aaa", Value: "111"}, {Name: "ccc", Value: "333"}}, + want: FromStrings("aaa", "111", "bbb", "222", "ccc", "333"), + }, + { + add: []common.Label{{Name: "ddd", Value: "444"}}, + want: FromStrings("ddd", "444"), + }, + } { + t.Run(strconv.Itoa(i), func(t *testing.T) { + b := NewScratchBuilder(len(tcase.add)) + for _, lbl := range tcase.add { + b.Add(lbl.Name, lbl.Value) + } + b.Sort() + require.True(t, Equal(tcase.want, b.Labels())) + b.Assign(tcase.want) + require.True(t, Equal(tcase.want, b.Labels())) + }) + } +} + +var benchmarkLabels = []common.Label{ + {Name: "job", Value: "node"}, + {Name: "instance", Value: "123.123.1.211:9090"}, + {Name: "path", Value: "/api/v1/namespaces//deployments/"}, + {Name: "method", Value: http.MethodGet}, + {Name: "namespace", Value: "system"}, + {Name: "status", Value: "500"}, + {Name: "prometheus", Value: "prometheus-core-1"}, + {Name: "datacenter", Value: "eu-west-1"}, + {Name: "pod_name", Value: "abcdef-99999-defee"}, +} + +func BenchmarkBuilder(b *testing.B) { + var l Labels + builder := NewBuilder(EmptyLabels()) + for i := 0; i < b.N; i++ { + builder.Reset(EmptyLabels()) + for _, l := range benchmarkLabels { + builder.Set(l.Name, l.Value) + } + l = builder.Labels() + } + require.Equal(b, 9, l.Len()) +} diff --git a/storage/remote/otlptranslator/prometheusremotewrite/metrics_to_prw.go b/storage/remote/otlptranslator/prometheusremotewrite/metrics_to_prw.go index 0eef4e1567..d1e9f363f2 100644 --- a/storage/remote/otlptranslator/prometheusremotewrite/metrics_to_prw.go +++ b/storage/remote/otlptranslator/prometheusremotewrite/metrics_to_prw.go @@ -21,7 +21,6 @@ import ( "errors" "fmt" "math" - "sort" "time" "github.com/prometheus/otlptranslator" @@ -30,7 +29,8 @@ import ( "go.uber.org/multierr" "github.com/prometheus/prometheus/config" - "github.com/prometheus/prometheus/prompb" + "github.com/prometheus/prometheus/model/metadata" + "github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite/labels" "github.com/prometheus/prometheus/util/annotations" ) @@ -59,16 +59,21 @@ type Settings struct { // PrometheusConverter converts from OTel write format to Prometheus remote write format. type PrometheusConverter struct { - unique map[uint64]*prompb.TimeSeries - conflicts map[uint64][]*prompb.TimeSeries - everyN everyNTimes - metadata []prompb.MetricMetadata + unique map[uint64]labels.Labels + conflicts map[uint64][]labels.Labels + everyN everyNTimes + scratchBuilder labels.ScratchBuilder + builder *labels.Builder + appender CombinedAppender } -func NewPrometheusConverter() *PrometheusConverter { +func NewPrometheusConverter(appender CombinedAppender) *PrometheusConverter { return &PrometheusConverter{ - unique: map[uint64]*prompb.TimeSeries{}, - conflicts: map[uint64][]*prompb.TimeSeries{}, + unique: map[uint64]labels.Labels{}, + conflicts: map[uint64][]labels.Labels{}, + scratchBuilder: labels.NewScratchBuilder(0), + builder: labels.NewBuilder(labels.EmptyLabels()), + appender: appender, } } @@ -121,6 +126,7 @@ func (c *PrometheusConverter) FromMetrics(ctx context.Context, md pmetric.Metric WithMetricSuffixes: settings.AddMetricSuffixes, UTF8Allowed: settings.AllowUTF8, } + unitNamer := otlptranslator.UnitNamer{} c.everyN = everyNTimes{n: 128} resourceMetricsSlice := md.ResourceMetrics() @@ -131,7 +137,6 @@ func (c *PrometheusConverter) FromMetrics(ctx context.Context, md pmetric.Metric numMetrics += scopeMetricsSlice.At(j).Metrics().Len() } } - c.metadata = make([]prompb.MetricMetadata, 0, numMetrics) for i := 0; i < resourceMetricsSlice.Len(); i++ { resourceMetrics := resourceMetricsSlice.At(i) @@ -176,13 +181,11 @@ func (c *PrometheusConverter) FromMetrics(ctx context.Context, md pmetric.Metric errs = multierr.Append(errs, err) continue } - metadata := prompb.MetricMetadata{ - Type: otelMetricTypeToPromMetricType(metric), - MetricFamilyName: promName, - Help: metric.Description(), - Unit: metric.Unit(), + meta := metadata.Metadata{ + Type: otelMetricTypeToPromMetricType(metric), + Unit: unitNamer.Build(metric.Unit()), + Help: metric.Description(), } - c.metadata = append(c.metadata, metadata) // handle individual metrics based on type //exhaustive:enforce @@ -193,7 +196,7 @@ func (c *PrometheusConverter) FromMetrics(ctx context.Context, md pmetric.Metric errs = multierr.Append(errs, fmt.Errorf("empty data points. %s is dropped", metric.Name())) break } - if err := c.addGaugeNumberDataPoints(ctx, dataPoints, resource, settings, metadata, scope); err != nil { + if err := c.addGaugeNumberDataPoints(ctx, dataPoints, resource, settings, promName, scope, meta); err != nil { errs = multierr.Append(errs, err) if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) { return @@ -205,7 +208,7 @@ func (c *PrometheusConverter) FromMetrics(ctx context.Context, md pmetric.Metric errs = multierr.Append(errs, fmt.Errorf("empty data points. %s is dropped", metric.Name())) break } - if err := c.addSumNumberDataPoints(ctx, dataPoints, resource, metric, settings, metadata, scope); err != nil { + if err := c.addSumNumberDataPoints(ctx, dataPoints, resource, metric, settings, promName, scope, meta); err != nil { errs = multierr.Append(errs, err) if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) { return @@ -219,7 +222,7 @@ func (c *PrometheusConverter) FromMetrics(ctx context.Context, md pmetric.Metric } if settings.ConvertHistogramsToNHCB { ws, err := c.addCustomBucketsHistogramDataPoints( - ctx, dataPoints, resource, settings, metadata, temporality, scope, + ctx, dataPoints, resource, settings, promName, temporality, scope, meta, ) annots.Merge(ws) if err != nil { @@ -229,7 +232,7 @@ func (c *PrometheusConverter) FromMetrics(ctx context.Context, md pmetric.Metric } } } else { - if err := c.addHistogramDataPoints(ctx, dataPoints, resource, settings, metadata, scope); err != nil { + if err := c.addHistogramDataPoints(ctx, dataPoints, resource, settings, promName, scope, meta); err != nil { errs = multierr.Append(errs, err) if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) { return @@ -247,9 +250,10 @@ func (c *PrometheusConverter) FromMetrics(ctx context.Context, md pmetric.Metric dataPoints, resource, settings, - metadata, + promName, temporality, scope, + meta, ) annots.Merge(ws) if err != nil { @@ -264,7 +268,7 @@ func (c *PrometheusConverter) FromMetrics(ctx context.Context, md pmetric.Metric errs = multierr.Append(errs, fmt.Errorf("empty data points. %s is dropped", metric.Name())) break } - if err := c.addSummaryDataPoints(ctx, dataPoints, resource, settings, metadata, scope); err != nil { + if err := c.addSummaryDataPoints(ctx, dataPoints, resource, settings, promName, scope, meta); err != nil { errs = multierr.Append(errs, err) if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) { return @@ -278,72 +282,16 @@ func (c *PrometheusConverter) FromMetrics(ctx context.Context, md pmetric.Metric if earliestTimestamp < pcommon.Timestamp(math.MaxUint64) { // We have at least one metric sample for this resource. // Generate a corresponding target_info series. - err := addResourceTargetInfo(resource, settings, earliestTimestamp.AsTime(), latestTimestamp.AsTime(), c) - if err != nil { + if err := c.addResourceTargetInfo(resource, settings, earliestTimestamp.AsTime(), latestTimestamp.AsTime()); err != nil { errs = multierr.Append(errs, err) } } } - return annots, errs -} + err := c.appender.Commit() + errs = multierr.Append(errs, err) -func isSameMetric(ts *prompb.TimeSeries, lbls []prompb.Label) bool { - if len(ts.Labels) != len(lbls) { - return false - } - for i, l := range ts.Labels { - if l.Name != ts.Labels[i].Name || l.Value != ts.Labels[i].Value { - return false - } - } - return true -} - -// addExemplars adds exemplars for the dataPoint. For each exemplar, if it can find a bucket bound corresponding to its value, -// the exemplar is added to the bucket bound's time series, provided that the time series' has samples. -func (c *PrometheusConverter) addExemplars(ctx context.Context, dataPoint pmetric.HistogramDataPoint, bucketBounds []bucketBoundsData) error { - if len(bucketBounds) == 0 { - return nil - } - - exemplars, err := getPromExemplars(ctx, &c.everyN, dataPoint) - if err != nil { - return err - } - if len(exemplars) == 0 { - return nil - } - - sort.Sort(byBucketBoundsData(bucketBounds)) - for _, exemplar := range exemplars { - for _, bound := range bucketBounds { - if err := c.everyN.checkContext(ctx); err != nil { - return err - } - if len(bound.ts.Samples) > 0 && exemplar.Value <= bound.bound { - bound.ts.Exemplars = append(bound.ts.Exemplars, exemplar) - break - } - } - } - - return nil -} - -// addSample finds a TimeSeries that corresponds to lbls, and adds sample to it. -// If there is no corresponding TimeSeries already, it's created. -// The corresponding TimeSeries is returned. -// If either lbls is nil/empty or sample is nil, nothing is done. -func (c *PrometheusConverter) addSample(sample *prompb.Sample, lbls []prompb.Label) *prompb.TimeSeries { - if sample == nil || len(lbls) == 0 { - // This shouldn't happen - return nil - } - - ts, _ := c.getOrCreateTimeSeries(lbls) - ts.Samples = append(ts.Samples, *sample) - return ts + return } func NewPromoteResourceAttributes(otlpCfg config.OTLPConfig) *PromoteResourceAttributes { @@ -361,30 +309,43 @@ func NewPromoteResourceAttributes(otlpCfg config.OTLPConfig) *PromoteResourceAtt } } -// promotedAttributes returns labels for promoted resourceAttributes. -func (s *PromoteResourceAttributes) promotedAttributes(resourceAttributes pcommon.Map) []prompb.Label { +// addPromotedAttributes adds labels for promoted resourceAttributes to the builder. +func (s *PromoteResourceAttributes) addPromotedAttributes(builder *labels.Builder, resourceAttributes pcommon.Map, allowUTF8 bool) error { if s == nil { return nil } - var promotedAttrs []prompb.Label + labelNamer := otlptranslator.LabelNamer{UTF8Allowed: allowUTF8} if s.promoteAll { - promotedAttrs = make([]prompb.Label, 0, resourceAttributes.Len()) + var err error resourceAttributes.Range(func(name string, value pcommon.Value) bool { if _, exists := s.attrs[name]; !exists { - promotedAttrs = append(promotedAttrs, prompb.Label{Name: name, Value: value.AsString()}) - } - return true - }) - } else { - promotedAttrs = make([]prompb.Label, 0, len(s.attrs)) - resourceAttributes.Range(func(name string, value pcommon.Value) bool { - if _, exists := s.attrs[name]; exists { - promotedAttrs = append(promotedAttrs, prompb.Label{Name: name, Value: value.AsString()}) + var normalized string + normalized, err = labelNamer.Build(name) + if err != nil { + return false + } + if builder.Get(normalized) == "" { + builder.Set(normalized, value.AsString()) + } } return true }) + return err } - sort.Stable(ByLabelName(promotedAttrs)) - return promotedAttrs + var err error + resourceAttributes.Range(func(name string, value pcommon.Value) bool { + if _, exists := s.attrs[name]; exists { + var normalized string + normalized, err = labelNamer.Build(name) + if err != nil { + return false + } + if builder.Get(normalized) == "" { + builder.Set(normalized, value.AsString()) + } + } + return true + }) + return err } diff --git a/storage/remote/otlptranslator/prometheusremotewrite/metrics_to_prw_test.go b/storage/remote/otlptranslator/prometheusremotewrite/metrics_to_prw_test.go index d264c81af8..ba5ad431e5 100644 --- a/storage/remote/otlptranslator/prometheusremotewrite/metrics_to_prw_test.go +++ b/storage/remote/otlptranslator/prometheusremotewrite/metrics_to_prw_test.go @@ -19,19 +19,20 @@ package prometheusremotewrite import ( "context" "fmt" - "sort" "testing" "time" + "github.com/prometheus/common/model" "github.com/prometheus/otlptranslator" "github.com/stretchr/testify/require" "go.opentelemetry.io/collector/pdata/pcommon" "go.opentelemetry.io/collector/pdata/pmetric" "go.opentelemetry.io/collector/pdata/pmetric/pmetricotlp" - "github.com/prometheus/prometheus/model/labels" - "github.com/prometheus/prometheus/prompb" - "github.com/prometheus/prometheus/util/testutil" + "github.com/prometheus/prometheus/model/histogram" + modelLabels "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/model/metadata" + "github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite/labels" ) func TestFromMetrics(t *testing.T) { @@ -77,9 +78,9 @@ func TestFromMetrics(t *testing.T) { }, } { t.Run(tc.name, func(t *testing.T) { - converter := NewPrometheusConverter() + mockAppender := &mockCombinedAppender{} + converter := NewPrometheusConverter(mockAppender) payload, wantPromMetrics := createExportRequest(5, 128, 128, 2, 0, tc.settings, tc.temporality) - var expMetadata []prompb.MetricMetadata seenFamilyNames := map[string]struct{}{} for _, wantMetric := range wantPromMetrics { if _, exists := seenFamilyNames[wantMetric.familyName]; exists { @@ -90,12 +91,6 @@ func TestFromMetrics(t *testing.T) { } seenFamilyNames[wantMetric.familyName] = struct{}{} - expMetadata = append(expMetadata, prompb.MetricMetadata{ - Type: wantMetric.metricType, - MetricFamilyName: wantMetric.familyName, - Help: wantMetric.description, - Unit: wantMetric.unit, - }) } annots, err := converter.FromMetrics( @@ -106,16 +101,13 @@ func TestFromMetrics(t *testing.T) { require.NoError(t, err) require.Empty(t, annots) - testutil.RequireEqual(t, expMetadata, converter.Metadata()) - - ts := converter.TimeSeries() + ts := mockAppender.samples require.Len(t, ts, 1536+1) // +1 for the target_info. tgtInfoCount := 0 for _, s := range ts { - b := labels.NewScratchBuilder(2) - lbls := s.ToLabels(&b, nil) - if lbls.Get(labels.MetricName) == "target_info" { + lbls := s.ls + if lbls.Get(modelLabels.MetricName) == "target_info" { tgtInfoCount++ require.Equal(t, "test-namespace/test-service", lbls.Get("job")) require.Equal(t, "id1234", lbls.Get("instance")) @@ -156,7 +148,8 @@ func TestFromMetrics(t *testing.T) { generateAttributes(h.Attributes(), "series", 1) - converter := NewPrometheusConverter() + mockAppender := &mockCombinedAppender{} + converter := NewPrometheusConverter(mockAppender) annots, err := converter.FromMetrics( context.Background(), request.Metrics(), @@ -165,24 +158,19 @@ func TestFromMetrics(t *testing.T) { require.NoError(t, err) require.Empty(t, annots) - series := converter.TimeSeries() - if convertHistogramsToNHCB { - require.Len(t, series[0].Histograms, 1) - require.Empty(t, series[0].Samples) + require.Len(t, mockAppender.histograms, 1) + require.Empty(t, mockAppender.samples) } else { - require.Len(t, series, 3) - for i := range series { - require.Len(t, series[i].Samples, 1) - require.Nil(t, series[i].Histograms) - } + require.Empty(t, mockAppender.histograms) + require.Len(t, mockAppender.samples, 3) } }) } t.Run("context cancellation", func(t *testing.T) { settings := Settings{} - converter := NewPrometheusConverter() + converter := NewPrometheusConverter(&mockCombinedAppender{}) ctx, cancel := context.WithCancel(context.Background()) // Verify that converter.FromMetrics respects cancellation. cancel() @@ -195,7 +183,7 @@ func TestFromMetrics(t *testing.T) { t.Run("context timeout", func(t *testing.T) { settings := Settings{} - converter := NewPrometheusConverter() + converter := NewPrometheusConverter(&mockCombinedAppender{}) // Verify that converter.FromMetrics respects timeout. ctx, cancel := context.WithTimeout(context.Background(), 0) t.Cleanup(cancel) @@ -228,7 +216,7 @@ func TestFromMetrics(t *testing.T) { generateAttributes(h.Attributes(), "series", 10) } - converter := NewPrometheusConverter() + converter := NewPrometheusConverter(&mockCombinedAppender{}) annots, err := converter.FromMetrics(context.Background(), request.Metrics(), Settings{}) require.NoError(t, err) require.NotEmpty(t, annots) @@ -261,7 +249,7 @@ func TestFromMetrics(t *testing.T) { generateAttributes(h.Attributes(), "series", 10) } - converter := NewPrometheusConverter() + converter := NewPrometheusConverter(&mockCombinedAppender{}) annots, err := converter.FromMetrics( context.Background(), request.Metrics(), @@ -292,7 +280,6 @@ func TestFromMetrics(t *testing.T) { metrics := rm.ScopeMetrics().AppendEmpty().Metrics() ts := pcommon.NewTimestampFromTime(time.Now()) - var expMetadata []prompb.MetricMetadata for i := range 3 { m := metrics.AppendEmpty() m.SetEmptyGauge() @@ -308,68 +295,65 @@ func TestFromMetrics(t *testing.T) { generateAttributes(point.Attributes(), "series", 2) curTs = curTs.Add(defaultLookbackDelta / 4) } - - namer := otlptranslator.MetricNamer{} - name, err := namer.Build(TranslatorMetricFromOtelMetric(m)) - require.NoError(t, err) - expMetadata = append(expMetadata, prompb.MetricMetadata{ - Type: otelMetricTypeToPromMetricType(m), - MetricFamilyName: name, - Help: m.Description(), - Unit: m.Unit(), - }) } - converter := NewPrometheusConverter() - annots, err := converter.FromMetrics( - context.Background(), - request.Metrics(), - Settings{ - LookbackDelta: defaultLookbackDelta, - }, - ) + mockAppender := &mockCombinedAppender{} + converter := NewPrometheusConverter(mockAppender) + settings := Settings{ + LookbackDelta: defaultLookbackDelta, + } + + annots, err := converter.FromMetrics(context.Background(), request.Metrics(), settings) require.NoError(t, err) require.Empty(t, annots) - testutil.RequireEqual(t, expMetadata, converter.Metadata()) - - timeSeries := converter.TimeSeries() - tgtInfoCount := 0 - for _, s := range timeSeries { - b := labels.NewScratchBuilder(2) - lbls := s.ToLabels(&b, nil) - if lbls.Get(labels.MetricName) != "target_info" { - continue - } - - tgtInfoCount++ - require.Equal(t, "test-namespace/test-service", lbls.Get("job")) - require.Equal(t, "id1234", lbls.Get("instance")) - require.False(t, lbls.Has("service_name")) - require.False(t, lbls.Has("service_namespace")) - require.False(t, lbls.Has("service_instance_id")) - // There should be a target_info sample at the earliest metric timestamp, then two spaced lookback delta/2 apart, - // then one at the latest metric timestamp. - testutil.RequireEqual(t, []prompb.Sample{ - { - Value: 1, - Timestamp: ts.AsTime().UnixMilli(), - }, - { - Value: 1, - Timestamp: ts.AsTime().Add(defaultLookbackDelta / 2).UnixMilli(), - }, - { - Value: 1, - Timestamp: ts.AsTime().Add(defaultLookbackDelta).UnixMilli(), - }, - { - Value: 1, - Timestamp: ts.AsTime().Add(defaultLookbackDelta + defaultLookbackDelta/4).UnixMilli(), - }, - }, s.Samples) + require.Len(t, mockAppender.samples, 22) + // There should be a target_info sample at the earliest metric timestamp, then two spaced lookback delta/2 apart, + // then one at the latest metric timestamp. + targetInfoLabels := labels.FromStrings( + "__name__", "target_info", + "instance", "id1234", + "job", "test-namespace/test-service", + "resource_name_1", "value-1", + "resource_name_2", "value-2", + "resource_name_3", "value-3", + "resource_name_4", "value-4", + "resource_name_5", "value-5", + ) + targetInfoMeta := metadata.Metadata{ + Type: model.MetricTypeGauge, + Help: "Target metadata", } - require.Equal(t, 1, tgtInfoCount) + requireEqual(t, []combinedSample{ + { + metricFamilyName: "target_info", + v: 1, + t: ts.AsTime().UnixMilli(), + ls: targetInfoLabels, + meta: targetInfoMeta, + }, + { + metricFamilyName: "target_info", + v: 1, + t: ts.AsTime().Add(defaultLookbackDelta / 2).UnixMilli(), + ls: targetInfoLabels, + meta: targetInfoMeta, + }, + { + metricFamilyName: "target_info", + v: 1, + t: ts.AsTime().Add(defaultLookbackDelta).UnixMilli(), + ls: targetInfoLabels, + meta: targetInfoMeta, + }, + { + metricFamilyName: "target_info", + v: 1, + t: ts.AsTime().Add(defaultLookbackDelta + defaultLookbackDelta/4).UnixMilli(), + ls: targetInfoLabels, + meta: targetInfoMeta, + }, + }, mockAppender.samples[len(mockAppender.samples)-4:]) }) } @@ -377,12 +361,13 @@ func TestTemporality(t *testing.T) { ts := time.Unix(100, 0) tests := []struct { - name string - allowDelta bool - convertToNHCB bool - inputSeries []pmetric.Metric - expectedSeries []prompb.TimeSeries - expectedError string + name string + allowDelta bool + convertToNHCB bool + inputSeries []pmetric.Metric + expectedSamples []combinedSample + expectedHistograms []combinedHistogram + expectedError string }{ { name: "all cumulative when delta not allowed", @@ -391,9 +376,9 @@ func TestTemporality(t *testing.T) { createOtelSum("test_metric_1", pmetric.AggregationTemporalityCumulative, ts), createOtelSum("test_metric_2", pmetric.AggregationTemporalityCumulative, ts), }, - expectedSeries: []prompb.TimeSeries{ - createPromFloatSeries("test_metric_1", ts), - createPromFloatSeries("test_metric_2", ts), + expectedSamples: []combinedSample{ + createPromFloatSeries("test_metric_1", ts, model.MetricTypeCounter), + createPromFloatSeries("test_metric_2", ts, model.MetricTypeCounter), }, }, { @@ -403,9 +388,9 @@ func TestTemporality(t *testing.T) { createOtelSum("test_metric_1", pmetric.AggregationTemporalityDelta, ts), createOtelSum("test_metric_2", pmetric.AggregationTemporalityDelta, ts), }, - expectedSeries: []prompb.TimeSeries{ - createPromFloatSeries("test_metric_1", ts), - createPromFloatSeries("test_metric_2", ts), + expectedSamples: []combinedSample{ + createPromFloatSeries("test_metric_1", ts, model.MetricTypeUnknown), + createPromFloatSeries("test_metric_2", ts, model.MetricTypeUnknown), }, }, { @@ -415,9 +400,9 @@ func TestTemporality(t *testing.T) { createOtelSum("test_metric_1", pmetric.AggregationTemporalityDelta, ts), createOtelSum("test_metric_2", pmetric.AggregationTemporalityCumulative, ts), }, - expectedSeries: []prompb.TimeSeries{ - createPromFloatSeries("test_metric_1", ts), - createPromFloatSeries("test_metric_2", ts), + expectedSamples: []combinedSample{ + createPromFloatSeries("test_metric_1", ts, model.MetricTypeUnknown), + createPromFloatSeries("test_metric_2", ts, model.MetricTypeCounter), }, }, { @@ -427,8 +412,8 @@ func TestTemporality(t *testing.T) { createOtelSum("test_metric_1", pmetric.AggregationTemporalityCumulative, ts), createOtelSum("test_metric_2", pmetric.AggregationTemporalityDelta, ts), }, - expectedSeries: []prompb.TimeSeries{ - createPromFloatSeries("test_metric_1", ts), + expectedSamples: []combinedSample{ + createPromFloatSeries("test_metric_1", ts, model.MetricTypeCounter), }, expectedError: `invalid temporality and type combination for metric "test_metric_2"`, }, @@ -439,8 +424,8 @@ func TestTemporality(t *testing.T) { createOtelSum("test_metric_1", pmetric.AggregationTemporalityCumulative, ts), createOtelSum("test_metric_2", pmetric.AggregationTemporalityUnspecified, ts), }, - expectedSeries: []prompb.TimeSeries{ - createPromFloatSeries("test_metric_1", ts), + expectedSamples: []combinedSample{ + createPromFloatSeries("test_metric_1", ts, model.MetricTypeCounter), }, expectedError: `invalid temporality and type combination for metric "test_metric_2"`, }, @@ -450,8 +435,8 @@ func TestTemporality(t *testing.T) { inputSeries: []pmetric.Metric{ createOtelExponentialHistogram("test_histogram", pmetric.AggregationTemporalityCumulative, ts), }, - expectedSeries: []prompb.TimeSeries{ - createPromNativeHistogramSeries("test_histogram", prompb.Histogram_UNKNOWN, ts), + expectedHistograms: []combinedHistogram{ + createPromNativeHistogramSeries("test_histogram", histogram.UnknownCounterReset, ts, model.MetricTypeHistogram), }, }, { @@ -461,9 +446,9 @@ func TestTemporality(t *testing.T) { createOtelExponentialHistogram("test_histogram_1", pmetric.AggregationTemporalityDelta, ts), createOtelExponentialHistogram("test_histogram_2", pmetric.AggregationTemporalityCumulative, ts), }, - expectedSeries: []prompb.TimeSeries{ - createPromNativeHistogramSeries("test_histogram_1", prompb.Histogram_GAUGE, ts), - createPromNativeHistogramSeries("test_histogram_2", prompb.Histogram_UNKNOWN, ts), + expectedHistograms: []combinedHistogram{ + createPromNativeHistogramSeries("test_histogram_1", histogram.GaugeType, ts, model.MetricTypeUnknown), + createPromNativeHistogramSeries("test_histogram_2", histogram.UnknownCounterReset, ts, model.MetricTypeHistogram), }, }, { @@ -473,8 +458,8 @@ func TestTemporality(t *testing.T) { createOtelExponentialHistogram("test_histogram_1", pmetric.AggregationTemporalityDelta, ts), createOtelExponentialHistogram("test_histogram_2", pmetric.AggregationTemporalityCumulative, ts), }, - expectedSeries: []prompb.TimeSeries{ - createPromNativeHistogramSeries("test_histogram_2", prompb.Histogram_UNKNOWN, ts), + expectedHistograms: []combinedHistogram{ + createPromNativeHistogramSeries("test_histogram_2", histogram.UnknownCounterReset, ts, model.MetricTypeHistogram), }, expectedError: `invalid temporality and type combination for metric "test_histogram_1"`, }, @@ -485,8 +470,8 @@ func TestTemporality(t *testing.T) { inputSeries: []pmetric.Metric{ createOtelExplicitHistogram("test_histogram", pmetric.AggregationTemporalityCumulative, ts), }, - expectedSeries: []prompb.TimeSeries{ - createPromNHCBSeries("test_histogram", prompb.Histogram_UNKNOWN, ts), + expectedHistograms: []combinedHistogram{ + createPromNHCBSeries("test_histogram", histogram.UnknownCounterReset, ts, model.MetricTypeHistogram), }, }, { @@ -497,9 +482,9 @@ func TestTemporality(t *testing.T) { createOtelExplicitHistogram("test_histogram_1", pmetric.AggregationTemporalityDelta, ts), createOtelExplicitHistogram("test_histogram_2", pmetric.AggregationTemporalityCumulative, ts), }, - expectedSeries: []prompb.TimeSeries{ - createPromNHCBSeries("test_histogram_1", prompb.Histogram_GAUGE, ts), - createPromNHCBSeries("test_histogram_2", prompb.Histogram_UNKNOWN, ts), + expectedHistograms: []combinedHistogram{ + createPromNHCBSeries("test_histogram_1", histogram.GaugeType, ts, model.MetricTypeUnknown), + createPromNHCBSeries("test_histogram_2", histogram.UnknownCounterReset, ts, model.MetricTypeHistogram), }, }, { @@ -510,8 +495,8 @@ func TestTemporality(t *testing.T) { createOtelExplicitHistogram("test_histogram_1", pmetric.AggregationTemporalityDelta, ts), createOtelExplicitHistogram("test_histogram_2", pmetric.AggregationTemporalityCumulative, ts), }, - expectedSeries: []prompb.TimeSeries{ - createPromNHCBSeries("test_histogram_2", prompb.Histogram_UNKNOWN, ts), + expectedHistograms: []combinedHistogram{ + createPromNHCBSeries("test_histogram_2", histogram.UnknownCounterReset, ts, model.MetricTypeHistogram), }, expectedError: `invalid temporality and type combination for metric "test_histogram_1"`, }, @@ -523,8 +508,8 @@ func TestTemporality(t *testing.T) { createOtelExplicitHistogram("test_histogram_1", pmetric.AggregationTemporalityDelta, ts), createOtelExplicitHistogram("test_histogram_2", pmetric.AggregationTemporalityCumulative, ts), }, - expectedSeries: createPromClassicHistogramSeries("test_histogram_2", ts), - expectedError: `invalid temporality and type combination for metric "test_histogram_1"`, + expectedSamples: createPromClassicHistogramSeries("test_histogram_2", ts, model.MetricTypeHistogram), + expectedError: `invalid temporality and type combination for metric "test_histogram_1"`, }, { name: "delta histogram with buckets and convertToNHCB=false when allowed", @@ -534,9 +519,9 @@ func TestTemporality(t *testing.T) { createOtelExplicitHistogram("test_histogram_1", pmetric.AggregationTemporalityDelta, ts), createOtelExplicitHistogram("test_histogram_2", pmetric.AggregationTemporalityCumulative, ts), }, - expectedSeries: append( - createPromClassicHistogramSeries("test_histogram_1", ts), - createPromClassicHistogramSeries("test_histogram_2", ts)..., + expectedSamples: append( + createPromClassicHistogramSeries("test_histogram_1", ts, model.MetricTypeUnknown), + createPromClassicHistogramSeries("test_histogram_2", ts, model.MetricTypeHistogram)..., ), }, { @@ -544,15 +529,15 @@ func TestTemporality(t *testing.T) { inputSeries: []pmetric.Metric{ createOtelSummary("test_summary_1", ts), }, - expectedSeries: createPromSummarySeries("test_summary_1", ts), + expectedSamples: createPromSummarySeries("test_summary_1", ts), }, { name: "gauge does not have temporality", inputSeries: []pmetric.Metric{ createOtelGauge("test_gauge_1", ts), }, - expectedSeries: []prompb.TimeSeries{ - createPromFloatSeries("test_gauge_1", ts), + expectedSamples: []combinedSample{ + createPromFloatSeries("test_gauge_1", ts, model.MetricTypeGauge), }, }, { @@ -560,8 +545,7 @@ func TestTemporality(t *testing.T) { inputSeries: []pmetric.Metric{ createOtelEmptyType("test_empty"), }, - expectedSeries: []prompb.TimeSeries{}, - expectedError: `could not get aggregation temporality for test_empty as it has unsupported metric type Empty`, + expectedError: `could not get aggregation temporality for test_empty as it has unsupported metric type Empty`, }, } @@ -575,7 +559,8 @@ func TestTemporality(t *testing.T) { s.CopyTo(sm.Metrics().AppendEmpty()) } - c := NewPrometheusConverter() + mockAppender := &mockCombinedAppender{} + c := NewPrometheusConverter(mockAppender) settings := Settings{ AllowDeltaTemporality: tc.allowDelta, ConvertHistogramsToNHCB: tc.convertToNHCB, @@ -589,10 +574,9 @@ func TestTemporality(t *testing.T) { require.NoError(t, err) } - series := c.TimeSeries() - // Sort series to make the test deterministic. - testutil.RequireEqual(t, sortTimeSeries(tc.expectedSeries), sortTimeSeries(series)) + requireEqual(t, tc.expectedSamples, mockAppender.samples) + requireEqual(t, tc.expectedHistograms, mockAppender.histograms) }) } } @@ -603,6 +587,7 @@ func createOtelSum(name string, temporality pmetric.AggregationTemporality, ts t m.SetName(name) sum := m.SetEmptySum() sum.SetAggregationTemporality(temporality) + sum.SetIsMonotonic(true) dp := sum.DataPoints().AppendEmpty() dp.SetDoubleValue(5) dp.SetTimestamp(pcommon.NewTimestampFromTime(ts)) @@ -610,16 +595,15 @@ func createOtelSum(name string, temporality pmetric.AggregationTemporality, ts t return m } -func createPromFloatSeries(name string, ts time.Time) prompb.TimeSeries { - return prompb.TimeSeries{ - Labels: []prompb.Label{ - {Name: "__name__", Value: name}, - {Name: "test_label", Value: "test_value"}, +func createPromFloatSeries(name string, ts time.Time, typ model.MetricType) combinedSample { + return combinedSample{ + metricFamilyName: name, + ls: labels.FromStrings("__name__", name, "test_label", "test_value"), + t: ts.UnixMilli(), + v: 5, + meta: metadata.Metadata{ + Type: typ, }, - Samples: []prompb.Sample{{ - Value: 5, - Timestamp: ts.UnixMilli(), - }}, } } @@ -649,22 +633,21 @@ func createOtelExponentialHistogram(name string, temporality pmetric.Aggregation return m } -func createPromNativeHistogramSeries(name string, hint prompb.Histogram_ResetHint, ts time.Time) prompb.TimeSeries { - return prompb.TimeSeries{ - Labels: []prompb.Label{ - {Name: "__name__", Value: name}, - {Name: "test_label", Value: "test_value"}, +func createPromNativeHistogramSeries(name string, hint histogram.CounterResetHint, ts time.Time, typ model.MetricType) combinedHistogram { + return combinedHistogram{ + metricFamilyName: name, + ls: labels.FromStrings("__name__", name, "test_label", "test_value"), + t: ts.UnixMilli(), + meta: metadata.Metadata{ + Type: typ, }, - Histograms: []prompb.Histogram{ - { - Count: &prompb.Histogram_CountInt{CountInt: 1}, - Sum: 5, - Schema: 0, - ZeroThreshold: 1e-128, - ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: 0}, - Timestamp: ts.UnixMilli(), - ResetHint: hint, - }, + h: &histogram.Histogram{ + Count: 1, + Sum: 5, + Schema: 0, + ZeroThreshold: 1e-128, + ZeroCount: 0, + CounterResetHint: hint, }, } } @@ -685,72 +668,77 @@ func createOtelExplicitHistogram(name string, temporality pmetric.AggregationTem return m } -func createPromNHCBSeries(name string, hint prompb.Histogram_ResetHint, ts time.Time) prompb.TimeSeries { - return prompb.TimeSeries{ - Labels: []prompb.Label{ - {Name: "__name__", Value: name}, - {Name: "test_label", Value: "test_value"}, +func createPromNHCBSeries(name string, hint histogram.CounterResetHint, ts time.Time, typ model.MetricType) combinedHistogram { + return combinedHistogram{ + metricFamilyName: name, + ls: labels.FromStrings("__name__", name, "test_label", "test_value"), + meta: metadata.Metadata{ + Type: typ, }, - Histograms: []prompb.Histogram{ - { - Count: &prompb.Histogram_CountInt{CountInt: 20}, - Sum: 30, - Schema: -53, - ZeroThreshold: 0, - ZeroCount: nil, - PositiveSpans: []prompb.BucketSpan{ - { - Length: 3, - }, + t: ts.UnixMilli(), + h: &histogram.Histogram{ + Count: 20, + Sum: 30, + Schema: -53, + ZeroThreshold: 0, + PositiveSpans: []histogram.Span{ + { + Length: 3, }, - PositiveDeltas: []int64{10, 0, -10}, - CustomValues: []float64{1, 2}, - Timestamp: ts.UnixMilli(), - ResetHint: hint, }, + PositiveBuckets: []int64{10, 0, -10}, + CustomValues: []float64{1, 2}, + CounterResetHint: hint, }, } } -func createPromClassicHistogramSeries(name string, ts time.Time) []prompb.TimeSeries { - return []prompb.TimeSeries{ +func createPromClassicHistogramSeries(name string, ts time.Time, typ model.MetricType) []combinedSample { + return []combinedSample{ { - Labels: []prompb.Label{ - {Name: "__name__", Value: name + "_bucket"}, - {Name: "le", Value: "1"}, - {Name: "test_label", Value: "test_value"}, + metricFamilyName: name, + ls: labels.FromStrings("__name__", name+"_sum", "test_label", "test_value"), + t: ts.UnixMilli(), + v: 30, + meta: metadata.Metadata{ + Type: typ, }, - Samples: []prompb.Sample{{Value: 10, Timestamp: ts.UnixMilli()}}, }, { - Labels: []prompb.Label{ - {Name: "__name__", Value: name + "_bucket"}, - {Name: "le", Value: "2"}, - {Name: "test_label", Value: "test_value"}, + metricFamilyName: name, + ls: labels.FromStrings("__name__", name+"_count", "test_label", "test_value"), + t: ts.UnixMilli(), + v: 20, + meta: metadata.Metadata{ + Type: typ, }, - Samples: []prompb.Sample{{Value: 20, Timestamp: ts.UnixMilli()}}, }, { - Labels: []prompb.Label{ - {Name: "__name__", Value: name + "_bucket"}, - {Name: "le", Value: "+Inf"}, - {Name: "test_label", Value: "test_value"}, + metricFamilyName: name, + ls: labels.FromStrings("__name__", name+"_bucket", "le", "1", "test_label", "test_value"), + t: ts.UnixMilli(), + v: 10, + meta: metadata.Metadata{ + Type: typ, }, - Samples: []prompb.Sample{{Value: 20, Timestamp: ts.UnixMilli()}}, }, { - Labels: []prompb.Label{ - {Name: "__name__", Value: name + "_count"}, - {Name: "test_label", Value: "test_value"}, + metricFamilyName: name, + ls: labels.FromStrings("__name__", name+"_bucket", "le", "2", "test_label", "test_value"), + t: ts.UnixMilli(), + v: 20, + meta: metadata.Metadata{ + Type: typ, }, - Samples: []prompb.Sample{{Value: 20, Timestamp: ts.UnixMilli()}}, }, { - Labels: []prompb.Label{ - {Name: "__name__", Value: name + "_sum"}, - {Name: "test_label", Value: "test_value"}, + metricFamilyName: name, + ls: labels.FromStrings("__name__", name+"_bucket", "le", "+Inf", "test_label", "test_value"), + t: ts.UnixMilli(), + v: 20, + meta: metadata.Metadata{ + Type: typ, }, - Samples: []prompb.Sample{{Value: 30, Timestamp: ts.UnixMilli()}}, }, } } @@ -771,38 +759,34 @@ func createOtelSummary(name string, ts time.Time) pmetric.Metric { return m } -func createPromSummarySeries(name string, ts time.Time) []prompb.TimeSeries { - return []prompb.TimeSeries{ +func createPromSummarySeries(name string, ts time.Time) []combinedSample { + return []combinedSample{ { - Labels: []prompb.Label{ - {Name: "__name__", Value: name + "_sum"}, - {Name: "test_label", Value: "test_value"}, + metricFamilyName: name, + ls: labels.FromStrings("__name__", name+"_sum", "test_label", "test_value"), + t: ts.UnixMilli(), + v: 18, + meta: metadata.Metadata{ + Type: model.MetricTypeSummary, }, - Samples: []prompb.Sample{{ - Value: 18, - Timestamp: ts.UnixMilli(), - }}, }, { - Labels: []prompb.Label{ - {Name: "__name__", Value: name + "_count"}, - {Name: "test_label", Value: "test_value"}, + metricFamilyName: name, + ls: labels.FromStrings("__name__", name+"_count", "test_label", "test_value"), + t: ts.UnixMilli(), + v: 9, + meta: metadata.Metadata{ + Type: model.MetricTypeSummary, }, - Samples: []prompb.Sample{{ - Value: 9, - Timestamp: ts.UnixMilli(), - }}, }, { - Labels: []prompb.Label{ - {Name: "__name__", Value: name}, - {Name: "quantile", Value: "0.5"}, - {Name: "test_label", Value: "test_value"}, + metricFamilyName: name, + ls: labels.FromStrings("__name__", name, "quantile", "0.5", "test_label", "test_value"), + t: ts.UnixMilli(), + v: 2, + meta: metadata.Metadata{ + Type: model.MetricTypeSummary, }, - Samples: []prompb.Sample{{ - Value: 2, - Timestamp: ts.UnixMilli(), - }}, }, } } @@ -814,20 +798,6 @@ func createOtelEmptyType(name string) pmetric.Metric { return m } -func sortTimeSeries(series []prompb.TimeSeries) []prompb.TimeSeries { - for i := range series { - sort.Slice(series[i].Labels, func(j, k int) bool { - return series[i].Labels[j].Name < series[i].Labels[k].Name - }) - } - - sort.Slice(series, func(i, j int) bool { - return fmt.Sprint(series[i].Labels) < fmt.Sprint(series[j].Labels) - }) - - return series -} - func TestTranslatorMetricFromOtelMetric(t *testing.T) { tests := []struct { name string @@ -968,7 +938,7 @@ func BenchmarkPrometheusConverter_FromMetrics(b *testing.B) { b.Run(fmt.Sprintf("histogram count: %v", histogramCount), func(b *testing.B) { nonHistogramCounts := []int{0, 1000} - if resourceAttributeCount == 0 && histogramCount == 0 { + if histogramCount == 0 { // Don't bother running a scenario where we'll generate no series. nonHistogramCounts = []int{1000} } @@ -992,12 +962,12 @@ func BenchmarkPrometheusConverter_FromMetrics(b *testing.B) { b.ResetTimer() for range b.N { - converter := NewPrometheusConverter() + mockAppender := &mockCombinedAppender{} + converter := NewPrometheusConverter(mockAppender) annots, err := converter.FromMetrics(context.Background(), payload.Metrics(), settings) require.NoError(b, err) require.Empty(b, annots) - require.NotNil(b, converter.TimeSeries()) - require.NotNil(b, converter.Metadata()) + require.Positive(b, len(mockAppender.samples)+len(mockAppender.histograms)) } }) } @@ -1014,7 +984,7 @@ func BenchmarkPrometheusConverter_FromMetrics(b *testing.B) { type wantPrometheusMetric struct { name string familyName string - metricType prompb.MetricMetadata_MetricType + metricType model.MetricType description string unit string } @@ -1061,11 +1031,11 @@ func createExportRequest(resourceAttributeCount, histogramCount, nonHistogramCou generateAttributes(h.Attributes(), "series", labelsPerMetric) generateExemplars(h.Exemplars(), exemplarsPerSeries, ts) - metricType := prompb.MetricMetadata_HISTOGRAM + metricType := model.MetricTypeHistogram if temporality != pmetric.AggregationTemporalityCumulative { // We're in an early phase of implementing delta support (proposal: https://github.com/prometheus/proposals/pull/48/) // We don't have a proper way to flag delta metrics yet, therefore marking the metric type as unknown for now. - metricType = prompb.MetricMetadata_UNKNOWN + metricType = model.MetricTypeUnknown } wantPromMetrics = append(wantPromMetrics, wantPrometheusMetric{ name: fmt.Sprintf("histogram_%d%s_bucket", i, suffix), @@ -1103,11 +1073,11 @@ func createExportRequest(resourceAttributeCount, histogramCount, nonHistogramCou generateAttributes(point.Attributes(), "series", labelsPerMetric) generateExemplars(point.Exemplars(), exemplarsPerSeries, ts) - metricType := prompb.MetricMetadata_GAUGE + metricType := model.MetricTypeGauge if temporality != pmetric.AggregationTemporalityCumulative { // We're in an early phase of implementing delta support (proposal: https://github.com/prometheus/proposals/pull/48/) // We don't have a proper way to flag delta metrics yet, therefore marking the metric type as unknown for now. - metricType = prompb.MetricMetadata_UNKNOWN + metricType = model.MetricTypeUnknown } wantPromMetrics = append(wantPromMetrics, wantPrometheusMetric{ name: fmt.Sprintf("non_monotonic_sum_%d%s", i, suffix), @@ -1137,11 +1107,11 @@ func createExportRequest(resourceAttributeCount, histogramCount, nonHistogramCou counterSuffix = suffix + "_total" } - metricType := prompb.MetricMetadata_COUNTER + metricType := model.MetricTypeCounter if temporality != pmetric.AggregationTemporalityCumulative { // We're in an early phase of implementing delta support (proposal: https://github.com/prometheus/proposals/pull/48/) // We don't have a proper way to flag delta metrics yet, therefore marking the metric type as unknown for now. - metricType = prompb.MetricMetadata_UNKNOWN + metricType = model.MetricTypeUnknown } wantPromMetrics = append(wantPromMetrics, wantPrometheusMetric{ name: fmt.Sprintf("monotonic_sum_%d%s", i, counterSuffix), @@ -1167,7 +1137,7 @@ func createExportRequest(resourceAttributeCount, histogramCount, nonHistogramCou wantPromMetrics = append(wantPromMetrics, wantPrometheusMetric{ name: fmt.Sprintf("gauge_%d%s", i, suffix), familyName: fmt.Sprintf("gauge_%d%s", i, suffix), - metricType: prompb.MetricMetadata_GAUGE, + metricType: model.MetricTypeGauge, unit: "unit", description: "gauge", }) diff --git a/storage/remote/otlptranslator/prometheusremotewrite/number_data_points.go b/storage/remote/otlptranslator/prometheusremotewrite/number_data_points.go index 6992ce20e0..8a762d504c 100644 --- a/storage/remote/otlptranslator/prometheusremotewrite/number_data_points.go +++ b/storage/remote/otlptranslator/prometheusremotewrite/number_data_points.go @@ -24,12 +24,12 @@ import ( "go.opentelemetry.io/collector/pdata/pcommon" "go.opentelemetry.io/collector/pdata/pmetric" + "github.com/prometheus/prometheus/model/metadata" "github.com/prometheus/prometheus/model/value" - "github.com/prometheus/prometheus/prompb" ) func (c *PrometheusConverter) addGaugeNumberDataPoints(ctx context.Context, dataPoints pmetric.NumberDataPointSlice, - resource pcommon.Resource, settings Settings, metadata prompb.MetricMetadata, scope scope, + resource pcommon.Resource, settings Settings, name string, scope scope, meta metadata.Metadata, ) error { for x := 0; x < dataPoints.Len(); x++ { if err := c.everyN.checkContext(ctx); err != nil { @@ -37,42 +37,42 @@ func (c *PrometheusConverter) addGaugeNumberDataPoints(ctx context.Context, data } pt := dataPoints.At(x) - labels, err := createAttributes( + labels, err := c.createAttributes( resource, pt.Attributes(), scope, settings, nil, true, - metadata, + meta, model.MetricNameLabel, - metadata.MetricFamilyName, + name, ) if err != nil { return err } - sample := &prompb.Sample{ - // convert ns to ms - Timestamp: convertTimeStamp(pt.Timestamp()), - } + var val float64 switch pt.ValueType() { case pmetric.NumberDataPointValueTypeInt: - sample.Value = float64(pt.IntValue()) + val = float64(pt.IntValue()) case pmetric.NumberDataPointValueTypeDouble: - sample.Value = pt.DoubleValue() + val = pt.DoubleValue() } if pt.Flags().NoRecordedValue() { - sample.Value = math.Float64frombits(value.StaleNaN) + val = math.Float64frombits(value.StaleNaN) + } + ts := convertTimeStamp(pt.Timestamp()) + ct := convertTimeStamp(pt.StartTimestamp()) + if err := c.appender.AppendSample(name, labels, meta, ts, ct, val, nil); err != nil { + return err } - - c.addSample(sample, labels) } return nil } func (c *PrometheusConverter) addSumNumberDataPoints(ctx context.Context, dataPoints pmetric.NumberDataPointSlice, - resource pcommon.Resource, metric pmetric.Metric, settings Settings, metadata prompb.MetricMetadata, scope scope, + resource pcommon.Resource, metric pmetric.Metric, settings Settings, name string, scope scope, meta metadata.Metadata, ) error { for x := 0; x < dataPoints.Len(); x++ { if err := c.everyN.checkContext(ctx); err != nil { @@ -80,59 +80,51 @@ func (c *PrometheusConverter) addSumNumberDataPoints(ctx context.Context, dataPo } pt := dataPoints.At(x) - lbls, err := createAttributes( + lbls, err := c.createAttributes( resource, pt.Attributes(), scope, settings, nil, true, - metadata, + meta, model.MetricNameLabel, - metadata.MetricFamilyName, + name, ) + if err != nil { + return nil + } + var val float64 + switch pt.ValueType() { + case pmetric.NumberDataPointValueTypeInt: + val = float64(pt.IntValue()) + case pmetric.NumberDataPointValueTypeDouble: + val = pt.DoubleValue() + } + if pt.Flags().NoRecordedValue() { + val = math.Float64frombits(value.StaleNaN) + } + ts := convertTimeStamp(pt.Timestamp()) + ct := convertTimeStamp(pt.StartTimestamp()) + exemplars, err := c.getPromExemplars(ctx, pt.Exemplars()) if err != nil { return err } - sample := &prompb.Sample{ - // convert ns to ms - Timestamp: convertTimeStamp(pt.Timestamp()), - } - switch pt.ValueType() { - case pmetric.NumberDataPointValueTypeInt: - sample.Value = float64(pt.IntValue()) - case pmetric.NumberDataPointValueTypeDouble: - sample.Value = pt.DoubleValue() - } - if pt.Flags().NoRecordedValue() { - sample.Value = math.Float64frombits(value.StaleNaN) - } - - ts := c.addSample(sample, lbls) - if ts != nil { - exemplars, err := getPromExemplars[pmetric.NumberDataPoint](ctx, &c.everyN, pt) - if err != nil { - return err - } - ts.Exemplars = append(ts.Exemplars, exemplars...) + if err := c.appender.AppendSample(name, lbls, meta, ts, ct, val, exemplars); err != nil { + return err } // add created time series if needed - if settings.ExportCreatedMetric && metric.Sum().IsMonotonic() { - startTimestamp := pt.StartTimestamp() - if startTimestamp == 0 { - return nil - } - - createdLabels := make([]prompb.Label, len(lbls)) - copy(createdLabels, lbls) - for i, l := range createdLabels { - if l.Name == model.MetricNameLabel { - createdLabels[i].Value = metadata.MetricFamilyName + createdSuffix - break + if settings.ExportCreatedMetric && metric.Sum().IsMonotonic() && pt.StartTimestamp() != 0 { + c.builder.Reset(lbls) + // Add created suffix to the metric name for CT series. + c.builder.Set(model.MetricNameLabel, name+createdSuffix) + ls := c.builder.Labels() + if c.timeSeriesIsNew(ls) { + if err := c.appender.AppendSample(name, ls, meta, ts, 0, float64(ct), nil); err != nil { + return err } } - c.addTimeSeriesIfNeeded(createdLabels, startTimestamp, pt.Timestamp()) } } diff --git a/storage/remote/otlptranslator/prometheusremotewrite/number_data_points_test.go b/storage/remote/otlptranslator/prometheusremotewrite/number_data_points_test.go index b166700b69..de700aa7cc 100644 --- a/storage/remote/otlptranslator/prometheusremotewrite/number_data_points_test.go +++ b/storage/remote/otlptranslator/prometheusremotewrite/number_data_points_test.go @@ -26,7 +26,9 @@ import ( "go.opentelemetry.io/collector/pdata/pcommon" "go.opentelemetry.io/collector/pdata/pmetric" - "github.com/prometheus/prometheus/prompb" + "github.com/prometheus/prometheus/model/exemplar" + "github.com/prometheus/prometheus/model/metadata" + "github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite/labels" ) func TestPrometheusConverter_addGaugeNumberDataPoints(t *testing.T) { @@ -47,7 +49,7 @@ func TestPrometheusConverter_addGaugeNumberDataPoints(t *testing.T) { metric func() pmetric.Metric scope scope promoteScope bool - want func() map[uint64]*prompb.TimeSeries + want func() []combinedSample }{ { name: "gauge without scope promotion", @@ -60,19 +62,17 @@ func TestPrometheusConverter_addGaugeNumberDataPoints(t *testing.T) { }, scope: defaultScope, promoteScope: false, - want: func() map[uint64]*prompb.TimeSeries { - labels := []prompb.Label{ - {Name: model.MetricNameLabel, Value: "test"}, - } - return map[uint64]*prompb.TimeSeries{ - timeSeriesSignature(labels): { - Labels: labels, - Samples: []prompb.Sample{ - { - Value: 1, - Timestamp: convertTimeStamp(pcommon.Timestamp(ts)), - }, - }, + want: func() []combinedSample { + lbls := labels.FromStrings( + model.MetricNameLabel, "test", + ) + return []combinedSample{ + { + metricFamilyName: "test", + ls: lbls, + meta: metadata.Metadata{}, + t: convertTimeStamp(pcommon.Timestamp(ts)), + v: 1, }, } }, @@ -88,24 +88,22 @@ func TestPrometheusConverter_addGaugeNumberDataPoints(t *testing.T) { }, scope: defaultScope, promoteScope: true, - want: func() map[uint64]*prompb.TimeSeries { - labels := []prompb.Label{ - {Name: model.MetricNameLabel, Value: "test"}, - {Name: "otel_scope_name", Value: defaultScope.name}, - {Name: "otel_scope_schema_url", Value: defaultScope.schemaURL}, - {Name: "otel_scope_version", Value: defaultScope.version}, - {Name: "otel_scope_attr1", Value: "value1"}, - {Name: "otel_scope_attr2", Value: "value2"}, - } - return map[uint64]*prompb.TimeSeries{ - timeSeriesSignature(labels): { - Labels: labels, - Samples: []prompb.Sample{ - { - Value: 1, - Timestamp: convertTimeStamp(pcommon.Timestamp(ts)), - }, - }, + want: func() []combinedSample { + lbls := labels.FromStrings( + model.MetricNameLabel, "test", + "otel_scope_name", defaultScope.name, + "otel_scope_schema_url", defaultScope.schemaURL, + "otel_scope_version", defaultScope.version, + "otel_scope_attr1", "value1", + "otel_scope_attr2", "value2", + ) + return []combinedSample{ + { + metricFamilyName: "test", + ls: lbls, + meta: metadata.Metadata{}, + t: convertTimeStamp(pcommon.Timestamp(ts)), + v: 1, }, } }, @@ -114,7 +112,8 @@ func TestPrometheusConverter_addGaugeNumberDataPoints(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { metric := tt.metric() - converter := NewPrometheusConverter() + mockAppender := &mockCombinedAppender{} + converter := NewPrometheusConverter(mockAppender) converter.addGaugeNumberDataPoints( context.Background(), @@ -124,11 +123,13 @@ func TestPrometheusConverter_addGaugeNumberDataPoints(t *testing.T) { ExportCreatedMetric: true, PromoteScopeMetadata: tt.promoteScope, }, - prompb.MetricMetadata{MetricFamilyName: metric.Name()}, + metric.Name(), tt.scope, + metadata.Metadata{}, ) + require.NoError(t, mockAppender.Commit()) - require.Equal(t, tt.want(), converter.unique) + requireEqual(t, tt.want(), mockAppender.samples) require.Empty(t, converter.conflicts) }) } @@ -152,7 +153,7 @@ func TestPrometheusConverter_addSumNumberDataPoints(t *testing.T) { metric func() pmetric.Metric scope scope promoteScope bool - want func() map[uint64]*prompb.TimeSeries + want func() []combinedSample }{ { name: "sum without scope promotion", @@ -166,19 +167,17 @@ func TestPrometheusConverter_addSumNumberDataPoints(t *testing.T) { }, scope: defaultScope, promoteScope: false, - want: func() map[uint64]*prompb.TimeSeries { - labels := []prompb.Label{ - {Name: model.MetricNameLabel, Value: "test"}, - } - return map[uint64]*prompb.TimeSeries{ - timeSeriesSignature(labels): { - Labels: labels, - Samples: []prompb.Sample{ - { - Value: 1, - Timestamp: convertTimeStamp(ts), - }, - }, + want: func() []combinedSample { + lbls := labels.FromStrings( + model.MetricNameLabel, "test", + ) + return []combinedSample{ + { + metricFamilyName: "test", + ls: lbls, + meta: metadata.Metadata{}, + t: convertTimeStamp(ts), + v: 1, }, } }, @@ -195,24 +194,22 @@ func TestPrometheusConverter_addSumNumberDataPoints(t *testing.T) { }, scope: defaultScope, promoteScope: true, - want: func() map[uint64]*prompb.TimeSeries { - labels := []prompb.Label{ - {Name: model.MetricNameLabel, Value: "test"}, - {Name: "otel_scope_name", Value: defaultScope.name}, - {Name: "otel_scope_schema_url", Value: defaultScope.schemaURL}, - {Name: "otel_scope_version", Value: defaultScope.version}, - {Name: "otel_scope_attr1", Value: "value1"}, - {Name: "otel_scope_attr2", Value: "value2"}, - } - return map[uint64]*prompb.TimeSeries{ - timeSeriesSignature(labels): { - Labels: labels, - Samples: []prompb.Sample{ - { - Value: 1, - Timestamp: convertTimeStamp(ts), - }, - }, + want: func() []combinedSample { + lbls := labels.FromStrings( + model.MetricNameLabel, "test", + "otel_scope_name", defaultScope.name, + "otel_scope_schema_url", defaultScope.schemaURL, + "otel_scope_version", defaultScope.version, + "otel_scope_attr1", "value1", + "otel_scope_attr2", "value2", + ) + return []combinedSample{ + { + metricFamilyName: "test", + ls: lbls, + meta: metadata.Metadata{}, + t: convertTimeStamp(ts), + v: 1, }, } }, @@ -231,18 +228,18 @@ func TestPrometheusConverter_addSumNumberDataPoints(t *testing.T) { }, scope: defaultScope, promoteScope: false, - want: func() map[uint64]*prompb.TimeSeries { - labels := []prompb.Label{ - {Name: model.MetricNameLabel, Value: "test"}, - } - return map[uint64]*prompb.TimeSeries{ - timeSeriesSignature(labels): { - Labels: labels, - Samples: []prompb.Sample{{ - Value: 1, - Timestamp: convertTimeStamp(ts), - }}, - Exemplars: []prompb.Exemplar{ + want: func() []combinedSample { + lbls := labels.FromStrings( + model.MetricNameLabel, "test", + ) + return []combinedSample{ + { + metricFamilyName: "test", + ls: lbls, + meta: metadata.Metadata{}, + t: convertTimeStamp(ts), + v: 1, + es: []exemplar.Exemplar{ {Value: 2}, }, }, @@ -266,25 +263,28 @@ func TestPrometheusConverter_addSumNumberDataPoints(t *testing.T) { }, scope: defaultScope, promoteScope: false, - want: func() map[uint64]*prompb.TimeSeries { - labels := []prompb.Label{ - {Name: model.MetricNameLabel, Value: "test_sum"}, - } - createdLabels := []prompb.Label{ - {Name: model.MetricNameLabel, Value: "test_sum" + createdSuffix}, - } - return map[uint64]*prompb.TimeSeries{ - timeSeriesSignature(labels): { - Labels: labels, - Samples: []prompb.Sample{ - {Value: 1, Timestamp: convertTimeStamp(ts)}, - }, + want: func() []combinedSample { + lbls := labels.FromStrings( + model.MetricNameLabel, "test_sum", + ) + createdLabels := labels.FromStrings( + model.MetricNameLabel, "test_sum"+createdSuffix, + ) + return []combinedSample{ + { + metricFamilyName: "test_sum", + ls: lbls, + meta: metadata.Metadata{}, + t: convertTimeStamp(ts), + ct: convertTimeStamp(ts), + v: 1, }, - timeSeriesSignature(createdLabels): { - Labels: createdLabels, - Samples: []prompb.Sample{ - {Value: float64(convertTimeStamp(ts)), Timestamp: convertTimeStamp(ts)}, - }, + { + metricFamilyName: "test_sum", + ls: createdLabels, + meta: metadata.Metadata{}, + t: convertTimeStamp(ts), + v: float64(convertTimeStamp(ts)), }, } }, @@ -304,16 +304,17 @@ func TestPrometheusConverter_addSumNumberDataPoints(t *testing.T) { }, scope: defaultScope, promoteScope: false, - want: func() map[uint64]*prompb.TimeSeries { - labels := []prompb.Label{ - {Name: model.MetricNameLabel, Value: "test_sum"}, - } - return map[uint64]*prompb.TimeSeries{ - timeSeriesSignature(labels): { - Labels: labels, - Samples: []prompb.Sample{ - {Value: 0, Timestamp: convertTimeStamp(ts)}, - }, + want: func() []combinedSample { + lbls := labels.FromStrings( + model.MetricNameLabel, "test_sum", + ) + return []combinedSample{ + { + metricFamilyName: "test_sum", + ls: lbls, + meta: metadata.Metadata{}, + t: convertTimeStamp(ts), + v: 0, }, } }, @@ -333,16 +334,17 @@ func TestPrometheusConverter_addSumNumberDataPoints(t *testing.T) { }, scope: defaultScope, promoteScope: false, - want: func() map[uint64]*prompb.TimeSeries { - labels := []prompb.Label{ - {Name: model.MetricNameLabel, Value: "test_sum"}, - } - return map[uint64]*prompb.TimeSeries{ - timeSeriesSignature(labels): { - Labels: labels, - Samples: []prompb.Sample{ - {Value: 0, Timestamp: convertTimeStamp(ts)}, - }, + want: func() []combinedSample { + lbls := labels.FromStrings( + model.MetricNameLabel, "test_sum", + ) + return []combinedSample{ + { + metricFamilyName: "test_sum", + ls: lbls, + meta: metadata.Metadata{}, + t: convertTimeStamp(ts), + v: 0, }, } }, @@ -351,7 +353,8 @@ func TestPrometheusConverter_addSumNumberDataPoints(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { metric := tt.metric() - converter := NewPrometheusConverter() + mockAppender := &mockCombinedAppender{} + converter := NewPrometheusConverter(mockAppender) converter.addSumNumberDataPoints( context.Background(), @@ -362,11 +365,13 @@ func TestPrometheusConverter_addSumNumberDataPoints(t *testing.T) { ExportCreatedMetric: true, PromoteScopeMetadata: tt.promoteScope, }, - prompb.MetricMetadata{MetricFamilyName: metric.Name()}, + metric.Name(), tt.scope, + metadata.Metadata{}, ) + require.NoError(t, mockAppender.Commit()) - require.Equal(t, tt.want(), converter.unique) + requireEqual(t, tt.want(), mockAppender.samples) require.Empty(t, converter.conflicts) }) } diff --git a/storage/remote/otlptranslator/prometheusremotewrite/otlp_to_openmetrics_metadata.go b/storage/remote/otlptranslator/prometheusremotewrite/otlp_to_openmetrics_metadata.go index 716a6cd6f9..49f96e0019 100644 --- a/storage/remote/otlptranslator/prometheusremotewrite/otlp_to_openmetrics_metadata.go +++ b/storage/remote/otlptranslator/prometheusremotewrite/otlp_to_openmetrics_metadata.go @@ -17,42 +17,41 @@ package prometheusremotewrite import ( + "github.com/prometheus/common/model" "go.opentelemetry.io/collector/pdata/pmetric" - - "github.com/prometheus/prometheus/prompb" ) -func otelMetricTypeToPromMetricType(otelMetric pmetric.Metric) prompb.MetricMetadata_MetricType { +func otelMetricTypeToPromMetricType(otelMetric pmetric.Metric) model.MetricType { switch otelMetric.Type() { case pmetric.MetricTypeGauge: - return prompb.MetricMetadata_GAUGE + return model.MetricTypeGauge case pmetric.MetricTypeSum: - metricType := prompb.MetricMetadata_GAUGE + metricType := model.MetricTypeGauge if otelMetric.Sum().IsMonotonic() { - metricType = prompb.MetricMetadata_COUNTER + metricType = model.MetricTypeCounter } // We're in an early phase of implementing delta support (proposal: https://github.com/prometheus/proposals/pull/48/) // We don't have a proper way to flag delta metrics yet, therefore marking the metric type as unknown for now. if otelMetric.Sum().AggregationTemporality() == pmetric.AggregationTemporalityDelta { - metricType = prompb.MetricMetadata_UNKNOWN + metricType = model.MetricTypeUnknown } return metricType case pmetric.MetricTypeHistogram: // We're in an early phase of implementing delta support (proposal: https://github.com/prometheus/proposals/pull/48/) // We don't have a proper way to flag delta metrics yet, therefore marking the metric type as unknown for now. if otelMetric.Histogram().AggregationTemporality() == pmetric.AggregationTemporalityDelta { - return prompb.MetricMetadata_UNKNOWN + return model.MetricTypeUnknown } - return prompb.MetricMetadata_HISTOGRAM + return model.MetricTypeHistogram case pmetric.MetricTypeSummary: - return prompb.MetricMetadata_SUMMARY + return model.MetricTypeSummary case pmetric.MetricTypeExponentialHistogram: if otelMetric.ExponentialHistogram().AggregationTemporality() == pmetric.AggregationTemporalityDelta { // We're in an early phase of implementing delta support (proposal: https://github.com/prometheus/proposals/pull/48/) // We don't have a proper way to flag delta metrics yet, therefore marking the metric type as unknown for now. - return prompb.MetricMetadata_UNKNOWN + return model.MetricTypeUnknown } - return prompb.MetricMetadata_HISTOGRAM + return model.MetricTypeHistogram } - return prompb.MetricMetadata_UNKNOWN + return model.MetricTypeUnknown } diff --git a/storage/remote/otlptranslator/prometheusremotewrite/timeseries.go b/storage/remote/otlptranslator/prometheusremotewrite/timeseries.go deleted file mode 100644 index abffbe6105..0000000000 --- a/storage/remote/otlptranslator/prometheusremotewrite/timeseries.go +++ /dev/null @@ -1,46 +0,0 @@ -// Copyright 2024 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. -// Provenance-includes-location: -// https://github.com/open-telemetry/opentelemetry-collector-contrib/blob/95e8f8fdc2a9dc87230406c9a3cf02be4fd68bea/pkg/translator/prometheusremotewrite/metrics_to_prw.go -// Provenance-includes-license: Apache-2.0 -// Provenance-includes-copyright: Copyright The OpenTelemetry Authors. - -package prometheusremotewrite - -import ( - "github.com/prometheus/prometheus/prompb" -) - -// TimeSeries returns a slice of the prompb.TimeSeries that were converted from OTel format. -func (c *PrometheusConverter) TimeSeries() []prompb.TimeSeries { - conflicts := 0 - for _, ts := range c.conflicts { - conflicts += len(ts) - } - allTS := make([]prompb.TimeSeries, 0, len(c.unique)+conflicts) - for _, ts := range c.unique { - allTS = append(allTS, *ts) - } - for _, cTS := range c.conflicts { - for _, ts := range cTS { - allTS = append(allTS, *ts) - } - } - - return allTS -} - -// Metadata returns a slice of the prompb.Metadata that were converted from OTel format. -func (c *PrometheusConverter) Metadata() []prompb.MetricMetadata { - return c.metadata -} diff --git a/storage/remote/write_handler.go b/storage/remote/write_handler.go index 81e2681088..b62aa1580c 100644 --- a/storage/remote/write_handler.go +++ b/storage/remote/write_handler.go @@ -535,25 +535,28 @@ type OTLPOptions struct { LookbackDelta time.Duration // Add type and unit labels to the metrics. EnableTypeAndUnitLabels bool + // IngestCTZeroSample enables writing zero samples based on the start time + // of metrics. + IngestCTZeroSample bool } // NewOTLPWriteHandler creates a http.Handler that accepts OTLP write requests and // writes them to the provided appendable. -func NewOTLPWriteHandler(logger *slog.Logger, _ prometheus.Registerer, appendable storage.Appendable, configFunc func() config.Config, opts OTLPOptions) http.Handler { +func NewOTLPWriteHandler(logger *slog.Logger, reg prometheus.Registerer, appendable storage.Appendable, configFunc func() config.Config, opts OTLPOptions) http.Handler { if opts.NativeDelta && opts.ConvertDelta { // This should be validated when iterating through feature flags, so not expected to fail here. panic("cannot enable native delta ingestion and delta2cumulative conversion at the same time") } ex := &rwExporter{ - writeHandler: &writeHandler{ - logger: logger, - appendable: appendable, - }, + logger: logger, + appendable: appendable, config: configFunc, allowDeltaTemporality: opts.NativeDelta, lookbackDelta: opts.LookbackDelta, + ingestCTZeroSample: opts.IngestCTZeroSample, enableTypeAndUnitLabels: opts.EnableTypeAndUnitLabels, + reg: reg, } wh := &otlpWriteHandler{logger: logger, defaultConsumer: ex} @@ -587,18 +590,24 @@ func NewOTLPWriteHandler(logger *slog.Logger, _ prometheus.Registerer, appendabl } type rwExporter struct { - *writeHandler + logger *slog.Logger + appendable storage.Appendable config func() config.Config allowDeltaTemporality bool lookbackDelta time.Duration + ingestCTZeroSample bool enableTypeAndUnitLabels bool + reg prometheus.Registerer } func (rw *rwExporter) ConsumeMetrics(ctx context.Context, md pmetric.Metrics) error { otlpCfg := rw.config().OTLPConfig - - converter := otlptranslator.NewPrometheusConverter() - + app := &timeLimitAppender{ + Appender: rw.appendable.Appender(ctx), + maxTime: timestamp.FromTime(time.Now().Add(maxAheadTime)), + } + combinedAppender := otlptranslator.NewCombinedAppender(app, rw.logger, rw.reg, rw.ingestCTZeroSample) + converter := otlptranslator.NewPrometheusConverter(combinedAppender) annots, err := converter.FromMetrics(ctx, md, otlptranslator.Settings{ AddMetricSuffixes: otlpCfg.TranslationStrategy.ShouldAddSuffixes(), AllowUTF8: !otlpCfg.TranslationStrategy.ShouldEscape(), @@ -610,18 +619,18 @@ func (rw *rwExporter) ConsumeMetrics(ctx context.Context, md pmetric.Metrics) er LookbackDelta: rw.lookbackDelta, EnableTypeAndUnitLabels: rw.enableTypeAndUnitLabels, }) - if err != nil { - rw.logger.Warn("Error translating OTLP metrics to Prometheus write request", "err", err) - } + + defer func() { + if err != nil { + _ = app.Rollback() + return + } + err = app.Commit() + }() ws, _ := annots.AsStrings("", 0, 0) if len(ws) > 0 { rw.logger.Warn("Warnings translating OTLP metrics to Prometheus write request", "warnings", ws) } - - err = rw.write(ctx, &prompb.WriteRequest{ - Timeseries: converter.TimeSeries(), - Metadata: converter.Metadata(), - }) return err } diff --git a/storage/remote/write_test.go b/storage/remote/write_test.go index 9c347ae8e7..0e6727037c 100644 --- a/storage/remote/write_test.go +++ b/storage/remote/write_test.go @@ -44,6 +44,7 @@ import ( "github.com/prometheus/prometheus/config" "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/storage" ) @@ -381,14 +382,16 @@ func TestWriteStorageApplyConfig_PartialUpdate(t *testing.T) { require.NoError(t, s.Close()) } -func TestOTLPWriteHandler(t *testing.T) { +func TestOTLPWriteHandlerNaming(t *testing.T) { timestamp := time.Now() - exportRequest := generateOTLPWriteRequest(timestamp) + var zeroTime time.Time + exportRequest := generateOTLPWriteRequest(timestamp, zeroTime) for _, testCase := range []struct { name string otlpCfg config.OTLPConfig typeAndUnitLabels bool expectedSamples []mockSample + expectedMetadata []mockMetadata }{ { name: "NoTranslation/NoTypeAndUnitLabels", @@ -397,24 +400,70 @@ func TestOTLPWriteHandler(t *testing.T) { }, expectedSamples: []mockSample{ { - l: labels.New(labels.Label{Name: "__name__", Value: "test.counter"}, - labels.Label{Name: "foo.bar", Value: "baz"}, - labels.Label{Name: "instance", Value: "test-instance"}, - labels.Label{Name: "job", Value: "test-service"}), + l: labels.FromStrings(model.MetricNameLabel, "test.counter", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"), t: timestamp.UnixMilli(), v: 10.0, }, { - l: labels.New( - labels.Label{Name: "__name__", Value: "target_info"}, - labels.Label{Name: "host.name", Value: "test-host"}, - labels.Label{Name: "instance", Value: "test-instance"}, - labels.Label{Name: "job", Value: "test-service"}, - ), + l: labels.FromStrings(model.MetricNameLabel, "target_info", "host.name", "test-host", "instance", "test-instance", "job", "test-service"), t: timestamp.UnixMilli(), v: 1, }, }, + expectedMetadata: []mockMetadata{ + { + l: labels.FromStrings(model.MetricNameLabel, "test.counter", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeCounter, Unit: "bytes", Help: "test-counter-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.gauge", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeGauge, Unit: "bytes", Help: "test-gauge-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_sum", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_count", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "0"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "1"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "2"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "3"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "4"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "5"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "+Inf"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.exponential.histogram", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-exponential-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "target_info", "host.name", "test-host", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeGauge, Unit: "", Help: "Target metadata"}, + }, + }, }, { name: "NoTranslation/WithTypeAndUnitLabels", @@ -424,26 +473,71 @@ func TestOTLPWriteHandler(t *testing.T) { typeAndUnitLabels: true, expectedSamples: []mockSample{ { - l: labels.New(labels.Label{Name: "__name__", Value: "test.counter"}, - labels.Label{Name: "__type__", Value: "counter"}, - labels.Label{Name: "__unit__", Value: "bytes"}, - labels.Label{Name: "foo.bar", Value: "baz"}, - labels.Label{Name: "instance", Value: "test-instance"}, - labels.Label{Name: "job", Value: "test-service"}), + l: labels.FromStrings(model.MetricNameLabel, "test.counter", "__type__", "counter", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"), t: timestamp.UnixMilli(), v: 10.0, }, { - l: labels.New( - labels.Label{Name: "__name__", Value: "target_info"}, - labels.Label{Name: "host.name", Value: "test-host"}, - labels.Label{Name: "instance", Value: "test-instance"}, - labels.Label{Name: "job", Value: "test-service"}, - ), + l: labels.FromStrings(model.MetricNameLabel, "target_info", "host.name", "test-host", "instance", "test-instance", "job", "test-service"), t: timestamp.UnixMilli(), v: 1, }, }, + expectedMetadata: []mockMetadata{ + { + // Metadata labels follow series labels. + l: labels.FromStrings(model.MetricNameLabel, "test.counter", "__type__", "counter", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeCounter, Unit: "bytes", Help: "test-counter-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.gauge", "__type__", "gauge", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeGauge, Unit: "bytes", Help: "test-gauge-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_sum", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_count", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "0"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "1"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "2"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "3"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "4"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "5"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "+Inf"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.exponential.histogram", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-exponential-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "target_info", "host.name", "test-host", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeGauge, Unit: "", Help: "Target metadata"}, + }, + }, }, { name: "UnderscoreEscapingWithSuffixes/NoTypeAndUnitLabels", @@ -452,24 +546,71 @@ func TestOTLPWriteHandler(t *testing.T) { }, expectedSamples: []mockSample{ { - l: labels.New(labels.Label{Name: "__name__", Value: "test_counter_bytes_total"}, - labels.Label{Name: "foo_bar", Value: "baz"}, - labels.Label{Name: "instance", Value: "test-instance"}, - labels.Label{Name: "job", Value: "test-service"}), + l: labels.FromStrings(model.MetricNameLabel, "test_counter_bytes_total", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"), t: timestamp.UnixMilli(), v: 10.0, }, { - l: labels.New( - labels.Label{Name: "__name__", Value: "target_info"}, - labels.Label{Name: "host_name", Value: "test-host"}, - labels.Label{Name: "instance", Value: "test-instance"}, - labels.Label{Name: "job", Value: "test-service"}, - ), + l: labels.FromStrings(model.MetricNameLabel, "target_info", "host_name", "test-host", "instance", "test-instance", "job", "test-service"), t: timestamp.UnixMilli(), v: 1, }, }, + expectedMetadata: []mockMetadata{ + // All get _bytes unit suffix and counter also gets _total. + { + l: labels.FromStrings(model.MetricNameLabel, "test_counter_bytes_total", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeCounter, Unit: "bytes", Help: "test-counter-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test_gauge_bytes", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeGauge, Unit: "bytes", Help: "test-gauge-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_sum", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_count", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_bucket", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "0"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_bucket", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "1"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_bucket", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "2"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_bucket", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "3"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_bucket", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "4"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_bucket", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "5"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_bucket", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "+Inf"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test_exponential_histogram_bytes", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-exponential-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "target_info", "host_name", "test-host", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeGauge, Unit: "", Help: "Target metadata"}, + }, + }, }, { name: "UnderscoreEscapingWithoutSuffixes", @@ -478,24 +619,70 @@ func TestOTLPWriteHandler(t *testing.T) { }, expectedSamples: []mockSample{ { - l: labels.New(labels.Label{Name: "__name__", Value: "test_counter"}, - labels.Label{Name: "foo_bar", Value: "baz"}, - labels.Label{Name: "instance", Value: "test-instance"}, - labels.Label{Name: "job", Value: "test-service"}), + l: labels.FromStrings(model.MetricNameLabel, "test_counter", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"), t: timestamp.UnixMilli(), v: 10.0, }, { - l: labels.New( - labels.Label{Name: "__name__", Value: "target_info"}, - labels.Label{Name: "host_name", Value: "test-host"}, - labels.Label{Name: "instance", Value: "test-instance"}, - labels.Label{Name: "job", Value: "test-service"}, - ), + l: labels.FromStrings(model.MetricNameLabel, "target_info", "host_name", "test-host", "instance", "test-instance", "job", "test-service"), t: timestamp.UnixMilli(), v: 1, }, }, + expectedMetadata: []mockMetadata{ + { + l: labels.FromStrings(model.MetricNameLabel, "test_counter", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeCounter, Unit: "bytes", Help: "test-counter-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test_gauge", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeGauge, Unit: "bytes", Help: "test-gauge-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test_histogram_sum", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test_histogram_count", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bucket", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "0"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bucket", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "1"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bucket", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "2"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bucket", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "3"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bucket", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "4"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bucket", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "5"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bucket", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "+Inf"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test_exponential_histogram", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-exponential-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "target_info", "host_name", "test-host", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeGauge, Unit: "", Help: "Target metadata"}, + }, + }, }, { name: "UnderscoreEscapingWithSuffixes/WithTypeAndUnitLabels", @@ -525,6 +712,60 @@ func TestOTLPWriteHandler(t *testing.T) { v: 1, }, }, + expectedMetadata: []mockMetadata{ + { + l: labels.FromStrings(model.MetricNameLabel, "test_counter_bytes_total", "__type__", "counter", "__unit__", "bytes", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeCounter, Unit: "bytes", Help: "test-counter-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test_gauge_bytes", "__type__", "gauge", "__unit__", "bytes", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeGauge, Unit: "bytes", Help: "test-gauge-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_sum", "__type__", "histogram", "__unit__", "bytes", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_count", "__type__", "histogram", "__unit__", "bytes", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_bucket", "__type__", "histogram", "__unit__", "bytes", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "0"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_bucket", "__type__", "histogram", "__unit__", "bytes", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "1"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_bucket", "__type__", "histogram", "__unit__", "bytes", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "2"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_bucket", "__type__", "histogram", "__unit__", "bytes", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "3"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_bucket", "__type__", "histogram", "__unit__", "bytes", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "4"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_bucket", "__type__", "histogram", "__unit__", "bytes", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "5"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_bucket", "__type__", "histogram", "__unit__", "bytes", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "+Inf"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test_exponential_histogram_bytes", "__type__", "histogram", "__unit__", "bytes", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-exponential-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "target_info", "host_name", "test-host", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeGauge, Unit: "", Help: "Target metadata"}, + }, + }, }, { name: "NoUTF8EscapingWithSuffixes/NoTypeAndUnitLabels", @@ -533,24 +774,71 @@ func TestOTLPWriteHandler(t *testing.T) { }, expectedSamples: []mockSample{ { - l: labels.New(labels.Label{Name: "__name__", Value: "test.counter_bytes_total"}, - labels.Label{Name: "foo.bar", Value: "baz"}, - labels.Label{Name: "instance", Value: "test-instance"}, - labels.Label{Name: "job", Value: "test-service"}), + l: labels.FromStrings(model.MetricNameLabel, "test.counter_bytes_total", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"), t: timestamp.UnixMilli(), v: 10.0, }, { - l: labels.New( - labels.Label{Name: "__name__", Value: "target_info"}, - labels.Label{Name: "host.name", Value: "test-host"}, - labels.Label{Name: "instance", Value: "test-instance"}, - labels.Label{Name: "job", Value: "test-service"}, - ), + l: labels.FromStrings(model.MetricNameLabel, "target_info", "host.name", "test-host", "instance", "test-instance", "job", "test-service"), t: timestamp.UnixMilli(), v: 1, }, }, + expectedMetadata: []mockMetadata{ + // All get _bytes unit suffix and counter also gets _total. + { + l: labels.FromStrings(model.MetricNameLabel, "test.counter_bytes_total", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeCounter, Unit: "bytes", Help: "test-counter-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.gauge_bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeGauge, Unit: "bytes", Help: "test-gauge-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_sum", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_count", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "0"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "1"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "2"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "3"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "4"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "5"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "+Inf"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.exponential.histogram_bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-exponential-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "target_info", "host.name", "test-host", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeGauge, Unit: "", Help: "Target metadata"}, + }, + }, }, { name: "NoUTF8EscapingWithSuffixes/WithTypeAndUnitLabels", @@ -560,40 +848,268 @@ func TestOTLPWriteHandler(t *testing.T) { typeAndUnitLabels: true, expectedSamples: []mockSample{ { - l: labels.New(labels.Label{Name: "__name__", Value: "test.counter_bytes_total"}, - labels.Label{Name: "__type__", Value: "counter"}, - labels.Label{Name: "__unit__", Value: "bytes"}, - labels.Label{Name: "foo.bar", Value: "baz"}, - labels.Label{Name: "instance", Value: "test-instance"}, - labels.Label{Name: "job", Value: "test-service"}), + l: labels.FromStrings(model.MetricNameLabel, "test.counter_bytes_total", "__type__", "counter", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"), t: timestamp.UnixMilli(), v: 10.0, }, { - l: labels.New( - labels.Label{Name: "__name__", Value: "target_info"}, - labels.Label{Name: "host.name", Value: "test-host"}, - labels.Label{Name: "instance", Value: "test-instance"}, - labels.Label{Name: "job", Value: "test-service"}, - ), + l: labels.FromStrings(model.MetricNameLabel, "target_info", "host.name", "test-host", "instance", "test-instance", "job", "test-service"), t: timestamp.UnixMilli(), v: 1, }, }, + expectedMetadata: []mockMetadata{ + // All get _bytes unit suffix and counter also gets _total. + { + l: labels.FromStrings(model.MetricNameLabel, "test.counter_bytes_total", "__type__", "counter", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeCounter, Unit: "bytes", Help: "test-counter-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.gauge_bytes", "__type__", "gauge", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeGauge, Unit: "bytes", Help: "test-gauge-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_sum", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_count", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_bucket", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "0"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_bucket", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "1"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_bucket", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "2"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_bucket", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "3"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_bucket", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "4"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_bucket", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "5"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_bucket", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "+Inf"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.exponential.histogram_bytes", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-exponential-histogram-description"}, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "target_info", "host.name", "test-host", "instance", "test-instance", "job", "test-service"), + m: metadata.Metadata{Type: model.MetricTypeGauge, Unit: "", Help: "Target metadata"}, + }, + }, }, } { t.Run(testCase.name, func(t *testing.T) { - appendable := handleOTLP(t, exportRequest, testCase.otlpCfg, testCase.typeAndUnitLabels) + otlpOpts := OTLPOptions{ + EnableTypeAndUnitLabels: testCase.typeAndUnitLabels, + } + appendable := handleOTLP(t, exportRequest, testCase.otlpCfg, otlpOpts) for _, sample := range testCase.expectedSamples { requireContainsSample(t, appendable.samples, sample) } + for _, meta := range testCase.expectedMetadata { + requireContainsMetadata(t, appendable.metadata, meta) + } require.Len(t, appendable.samples, 12) // 1 (counter) + 1 (gauge) + 1 (target_info) + 7 (hist_bucket) + 2 (hist_sum, hist_count) require.Len(t, appendable.histograms, 1) // 1 (exponential histogram) + require.Len(t, appendable.metadata, 13) // for each float and histogram sample require.Len(t, appendable.exemplars, 1) // 1 (exemplar) }) } } +// Check that start time is ingested if ingestCTZeroSample is enabled +// and the start time is actually set (non-zero). +func TestOTLPWRiteHandlerStartTime(t *testing.T) { + timestamp := time.Now() + startTime := timestamp.Add(-1 * time.Millisecond) + var zeroTime time.Time + + expectedSamples := []mockSample{ + { + l: labels.FromStrings(model.MetricNameLabel, "test.counter", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"), + t: timestamp.UnixMilli(), + v: 10.0, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.gauge", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"), + t: timestamp.UnixMilli(), + v: 10.0, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_sum", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"), + t: timestamp.UnixMilli(), + v: 30.0, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_count", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"), + t: timestamp.UnixMilli(), + v: 12.0, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "0"), + t: timestamp.UnixMilli(), + v: 2.0, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "1"), + t: timestamp.UnixMilli(), + v: 4.0, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "2"), + t: timestamp.UnixMilli(), + v: 6.0, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "3"), + t: timestamp.UnixMilli(), + v: 8.0, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "4"), + t: timestamp.UnixMilli(), + v: 10.0, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "5"), + t: timestamp.UnixMilli(), + v: 12.0, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "+Inf"), + t: timestamp.UnixMilli(), + v: 12.0, + }, + { + l: labels.FromStrings(model.MetricNameLabel, "target_info", "host.name", "test-host", "instance", "test-instance", "job", "test-service"), + t: timestamp.UnixMilli(), + v: 1.0, + }, + } + expectedHistograms := []mockHistogram{ + { + l: labels.FromStrings(model.MetricNameLabel, "test.exponential.histogram", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"), + t: timestamp.UnixMilli(), + h: &histogram.Histogram{ + Schema: 2, + ZeroThreshold: 1e-128, + ZeroCount: 2, + Count: 10, + Sum: 30, + PositiveSpans: []histogram.Span{{Offset: 1, Length: 5}}, + PositiveBuckets: []int64{2, 0, 0, 0, 0}, + }, + }, + } + + expectedSamplesWithCTZero := make([]mockSample, 0, len(expectedSamples)*2-1) // All samples will get CT zero, except target_info. + for _, s := range expectedSamples { + if s.l.Get(model.MetricNameLabel) != "target_info" { + expectedSamplesWithCTZero = append(expectedSamplesWithCTZero, mockSample{ + l: s.l.Copy(), + t: startTime.UnixMilli(), + v: 0, + }) + } + expectedSamplesWithCTZero = append(expectedSamplesWithCTZero, s) + } + expectedHistogramsWithCTZero := make([]mockHistogram, 0, len(expectedHistograms)*2) + for _, s := range expectedHistograms { + if s.l.Get(model.MetricNameLabel) != "target_info" { + expectedHistogramsWithCTZero = append(expectedHistogramsWithCTZero, mockHistogram{ + l: s.l.Copy(), + t: startTime.UnixMilli(), + h: &histogram.Histogram{}, + }) + } + expectedHistogramsWithCTZero = append(expectedHistogramsWithCTZero, s) + } + + for _, testCase := range []struct { + name string + otlpOpts OTLPOptions + startTime time.Time + expectCTZero bool + expectedSamples []mockSample + expectedHistograms []mockHistogram + }{ + { + name: "IngestCTZero=false/startTime=0", + otlpOpts: OTLPOptions{ + IngestCTZeroSample: false, + }, + startTime: zeroTime, + expectedSamples: expectedSamples, + expectedHistograms: expectedHistograms, + }, + { + name: "IngestCTZero=true/startTime=0", + otlpOpts: OTLPOptions{ + IngestCTZeroSample: true, + }, + startTime: zeroTime, + expectedSamples: expectedSamples, + expectedHistograms: expectedHistograms, + }, + { + name: "IngestCTZero=false/startTime=ts-1ms", + otlpOpts: OTLPOptions{ + IngestCTZeroSample: false, + }, + startTime: startTime, + expectedSamples: expectedSamples, + expectedHistograms: expectedHistograms, + }, + { + name: "IngestCTZero=true/startTime=ts-1ms", + otlpOpts: OTLPOptions{ + IngestCTZeroSample: true, + }, + startTime: startTime, + expectedSamples: expectedSamplesWithCTZero, + expectedHistograms: expectedHistogramsWithCTZero, + }, + } { + t.Run(testCase.name, func(t *testing.T) { + exportRequest := generateOTLPWriteRequest(timestamp, testCase.startTime) + appendable := handleOTLP(t, exportRequest, config.OTLPConfig{ + TranslationStrategy: otlptranslator.NoTranslation, + }, testCase.otlpOpts) + for i, expect := range testCase.expectedSamples { + actual := appendable.samples[i] + require.True(t, labels.Equal(expect.l, actual.l), "sample labels,pos=%v", i) + require.Equal(t, expect.t, actual.t, "sample timestamp,pos=%v", i) + require.Equal(t, expect.v, actual.v, "sample value,pos=%v", i) + } + for i, expect := range testCase.expectedHistograms { + actual := appendable.histograms[i] + require.True(t, labels.Equal(expect.l, actual.l), "histogram labels,pos=%v", i) + require.Equal(t, expect.t, actual.t, "histogram timestamp,pos=%v", i) + require.True(t, expect.h.Equals(actual.h), "histogram value,pos=%v", i) + } + require.Len(t, appendable.samples, len(testCase.expectedSamples)) + require.Len(t, appendable.histograms, len(testCase.expectedHistograms)) + }) + } +} + func requireContainsSample(t *testing.T, actual []mockSample, expected mockSample) { t.Helper() @@ -607,7 +1123,20 @@ func requireContainsSample(t *testing.T, actual []mockSample, expected mockSampl "actual : %v", expected, actual)) } -func handleOTLP(t *testing.T, exportRequest pmetricotlp.ExportRequest, otlpCfg config.OTLPConfig, typeAndUnitLabels bool) *mockAppendable { +func requireContainsMetadata(t *testing.T, actual []mockMetadata, expected mockMetadata) { + t.Helper() + + for _, got := range actual { + if labels.Equal(expected.l, got.l) && expected.m.Type == got.m.Type && expected.m.Unit == got.m.Unit && expected.m.Help == got.m.Help { + return + } + } + require.Fail(t, fmt.Sprintf("Metadata not found: \n"+ + "expected: %v\n"+ + "actual : %v", expected, actual)) +} + +func handleOTLP(t *testing.T, exportRequest pmetricotlp.ExportRequest, otlpCfg config.OTLPConfig, otlpOpts OTLPOptions) *mockAppendable { buf, err := exportRequest.MarshalProto() require.NoError(t, err) @@ -615,12 +1144,13 @@ func handleOTLP(t *testing.T, exportRequest pmetricotlp.ExportRequest, otlpCfg c require.NoError(t, err) req.Header.Set("Content-Type", "application/x-protobuf") + log := slog.New(slog.NewTextHandler(os.Stderr, &slog.HandlerOptions{Level: slog.LevelWarn})) appendable := &mockAppendable{} - handler := NewOTLPWriteHandler(nil, nil, appendable, func() config.Config { + handler := NewOTLPWriteHandler(log, nil, appendable, func() config.Config { return config.Config{ OTLPConfig: otlpCfg, } - }, OTLPOptions{EnableTypeAndUnitLabels: typeAndUnitLabels}) + }, otlpOpts) recorder := httptest.NewRecorder() handler.ServeHTTP(recorder, req) @@ -630,7 +1160,7 @@ func handleOTLP(t *testing.T, exportRequest pmetricotlp.ExportRequest, otlpCfg c return appendable } -func generateOTLPWriteRequest(timestamp time.Time) pmetricotlp.ExportRequest { +func generateOTLPWriteRequest(timestamp, startTime time.Time) pmetricotlp.ExportRequest { d := pmetric.NewMetrics() // Generate One Counter, One Gauge, One Histogram, One Exponential-Histogram @@ -655,6 +1185,7 @@ func generateOTLPWriteRequest(timestamp time.Time) pmetricotlp.ExportRequest { counterDataPoint := counterMetric.Sum().DataPoints().AppendEmpty() counterDataPoint.SetTimestamp(pcommon.NewTimestampFromTime(timestamp)) + counterDataPoint.SetStartTimestamp(pcommon.NewTimestampFromTime(startTime)) counterDataPoint.SetDoubleValue(10.0) counterDataPoint.Attributes().PutStr("foo.bar", "baz") @@ -674,6 +1205,7 @@ func generateOTLPWriteRequest(timestamp time.Time) pmetricotlp.ExportRequest { gaugeDataPoint := gaugeMetric.Gauge().DataPoints().AppendEmpty() gaugeDataPoint.SetTimestamp(pcommon.NewTimestampFromTime(timestamp)) + gaugeDataPoint.SetStartTimestamp(pcommon.NewTimestampFromTime(startTime)) gaugeDataPoint.SetDoubleValue(10.0) gaugeDataPoint.Attributes().PutStr("foo.bar", "baz") @@ -687,9 +1219,10 @@ func generateOTLPWriteRequest(timestamp time.Time) pmetricotlp.ExportRequest { histogramDataPoint := histogramMetric.Histogram().DataPoints().AppendEmpty() histogramDataPoint.SetTimestamp(pcommon.NewTimestampFromTime(timestamp)) + histogramDataPoint.SetStartTimestamp(pcommon.NewTimestampFromTime(startTime)) histogramDataPoint.ExplicitBounds().FromRaw([]float64{0.0, 1.0, 2.0, 3.0, 4.0, 5.0}) histogramDataPoint.BucketCounts().FromRaw([]uint64{2, 2, 2, 2, 2, 2}) - histogramDataPoint.SetCount(10) + histogramDataPoint.SetCount(12) histogramDataPoint.SetSum(30.0) histogramDataPoint.Attributes().PutStr("foo.bar", "baz") @@ -703,6 +1236,7 @@ func generateOTLPWriteRequest(timestamp time.Time) pmetricotlp.ExportRequest { exponentialHistogramDataPoint := exponentialHistogramMetric.ExponentialHistogram().DataPoints().AppendEmpty() exponentialHistogramDataPoint.SetTimestamp(pcommon.NewTimestampFromTime(timestamp)) + exponentialHistogramDataPoint.SetStartTimestamp(pcommon.NewTimestampFromTime(startTime)) exponentialHistogramDataPoint.SetScale(2.0) exponentialHistogramDataPoint.Positive().BucketCounts().FromRaw([]uint64{2, 2, 2, 2, 2}) exponentialHistogramDataPoint.SetZeroCount(2) diff --git a/web/api/v1/api.go b/web/api/v1/api.go index a67c0aa525..4feae5aed2 100644 --- a/web/api/v1/api.go +++ b/web/api/v1/api.go @@ -317,6 +317,7 @@ func NewAPI( ConvertDelta: otlpDeltaToCumulative, NativeDelta: otlpNativeDeltaIngestion, LookbackDelta: lookbackDelta, + IngestCTZeroSample: ctZeroIngestionEnabled, EnableTypeAndUnitLabels: enableTypeAndUnitLabels, }) }