This commit is contained in:
George Krajcsovits 2025-08-05 16:56:14 +00:00 committed by GitHub
commit 1b8044c9c5
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
21 changed files with 3517 additions and 1951 deletions

View File

@ -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(). // 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...) origAdd, origDel := append(addStack[:0], b.add...), append(delStack[:0], b.del...)
b.base.Range(func(l Label) { 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) 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 { for _, a := range s {
if a.Name == n { if a.Name == n {
return true return true

View File

@ -464,6 +464,18 @@ func New(ls ...Label) Labels {
return Labels{syms: syms.nameTable, data: yoloString(buf)} 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. // FromStrings creates new labels from pairs of strings.
func FromStrings(ss ...string) Labels { func FromStrings(ss ...string) Labels {
if len(ss)%2 != 0 { if len(ss)%2 != 0 {

View File

@ -279,6 +279,13 @@ func New(ls ...Label) Labels {
return set 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. // FromStrings creates new labels from pairs of strings.
func FromStrings(ss ...string) Labels { func FromStrings(ss ...string) Labels {
if len(ss)%2 != 0 { if len(ss)%2 != 0 {
@ -423,7 +430,7 @@ func (b *Builder) Labels() Labels {
} }
res := make(Labels, 0, expectedSize) res := make(Labels, 0, expectedSize)
for _, l := range b.base { 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 continue
} }
res = append(res, l) res = append(res, l)

View File

@ -314,6 +314,16 @@ func New(ls ...Label) Labels {
return Labels{data: yoloString(buf)} 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. // FromStrings creates new labels from pairs of strings.
func FromStrings(ss ...string) Labels { func FromStrings(ss ...string) Labels {
if len(ss)%2 != 0 { if len(ss)%2 != 0 {

View File

@ -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
}

View File

@ -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
}

View File

@ -23,22 +23,24 @@ import (
"log" "log"
"math" "math"
"slices" "slices"
"sort"
"strconv" "strconv"
"strings" "strings"
"time" "time"
"unicode/utf8" "unicode/utf8"
"github.com/cespare/xxhash/v2"
"github.com/prometheus/common/model" "github.com/prometheus/common/model"
"github.com/prometheus/otlptranslator" "github.com/prometheus/otlptranslator"
"go.opentelemetry.io/collector/pdata/pcommon" "go.opentelemetry.io/collector/pdata/pcommon"
"go.opentelemetry.io/collector/pdata/pmetric" "go.opentelemetry.io/collector/pdata/pmetric"
conventions "go.opentelemetry.io/collector/semconv/v1.6.1" 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/timestamp"
"github.com/prometheus/prometheus/model/value" "github.com/prometheus/prometheus/model/value"
"github.com/prometheus/prometheus/prompb" "github.com/prometheus/prometheus/prompb"
"github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite/labels"
) )
const ( const (
@ -62,177 +64,115 @@ const (
defaultLookbackDelta = 5 * time.Minute 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. // 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 // 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 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. // 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, func (c *PrometheusConverter) createAttributes(resource pcommon.Resource, attributes pcommon.Map, scope scope, settings Settings,
ignoreAttrs []string, logOnOverwrite bool, metadata prompb.MetricMetadata, extras ...string, ignoreAttrs []string, logOnOverwrite bool, meta metadata.Metadata, extras ...string,
) ([]prompb.Label, error) { ) (labels.Labels, error) {
resourceAttrs := resource.Attributes() resourceAttrs := resource.Attributes()
serviceName, haveServiceName := resourceAttrs.Get(conventions.AttributeServiceName) serviceName, haveServiceName := resourceAttrs.Get(conventions.AttributeServiceName)
instance, haveInstanceID := resourceAttrs.Get(conventions.AttributeServiceInstanceID) instance, haveInstanceID := resourceAttrs.Get(conventions.AttributeServiceInstanceID)
promotedAttrs := settings.PromoteResourceAttributes.promotedAttributes(resourceAttrs)
promoteScope := settings.PromoteScopeMetadata && scope.name != "" 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 // Ensure attributes are sorted by key for consistent merging of keys which
// collide when sanitized. // 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 // XXX: Should we always drop service namespace/service name/service instance ID from the labels
// (as they get mapped to other Prometheus labels)? // (as they get mapped to other Prometheus labels)?
attributes.Range(func(key string, value pcommon.Value) bool { attributes.Range(func(key string, value pcommon.Value) bool {
if !slices.Contains(ignoreAttrs, key) { if !slices.Contains(ignoreAttrs, key) {
labels = append(labels, prompb.Label{Name: key, Value: value.AsString()}) c.scratchBuilder.Add(key, value.AsString())
} }
return true 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} 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 { if settings.AllowUTF8 {
normalized, err := labelNamer.Build(lbl.Name) // UTF8 is allowed, so conflicts aren't possible.
if err != nil { c.builder.Reset(sortedLabels)
return nil, err } else {
} // Now that we have sorted and filtered the labels, build the actual list
if _, exists := l[normalized]; !exists { // of labels, and handle conflicts by appending values.
l[normalized] = lbl.Value c.builder.Reset(labels.EmptyLabels())
} var sortErr error
} sortedLabels.Range(func(l modelLabels.Label) {
if promoteScope { finalKey, err := labelNamer.Build(l.Name)
var rangeErr error if err != nil && sortErr == nil {
scope.attributes.Range(func(k string, v pcommon.Value) bool { sortErr = err
name, err := labelNamer.Build("otel_scope_" + k) return
if err != nil { }
rangeErr = err if existingValue := c.builder.Get(finalKey); existingValue != "" {
return false c.builder.Set(finalKey, existingValue+";"+l.Value)
} else {
c.builder.Set(finalKey, l.Value)
} }
l[name] = v.AsString()
return true
}) })
if rangeErr != nil { if sortErr != nil {
return nil, rangeErr 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 { if settings.EnableTypeAndUnitLabels {
unitNamer := otlptranslator.UnitNamer{UTF8Allowed: settings.AllowUTF8} unitNamer := otlptranslator.UnitNamer{UTF8Allowed: settings.AllowUTF8}
if metadata.Type != prompb.MetricMetadata_UNKNOWN { if meta.Type != model.MetricTypeUnknown {
l["__type__"] = strings.ToLower(metadata.Type.String()) c.builder.Set("__type__", strings.ToLower(string(meta.Type)))
} }
if metadata.Unit != "" { if meta.Unit != "" {
l["__unit__"] = unitNamer.Build(metadata.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. // Map service.name + service.namespace to job.
if haveServiceName { if haveServiceName {
val := serviceName.AsString() val := serviceName.AsString()
if serviceNamespace, ok := resourceAttrs.Get(conventions.AttributeServiceNamespace); ok { if serviceNamespace, ok := resourceAttrs.Get(conventions.AttributeServiceNamespace); ok {
val = fmt.Sprintf("%s/%s", serviceNamespace.AsString(), val) val = fmt.Sprintf("%s/%s", serviceNamespace.AsString(), val)
} }
l[model.JobLabel] = val c.builder.Set(model.JobLabel, val)
} }
// Map service.instance.id to instance. // Map service.instance.id to instance.
if haveInstanceID { if haveInstanceID {
l[model.InstanceLabel] = instance.AsString() c.builder.Set(model.InstanceLabel, instance.AsString())
} }
for key, value := range settings.ExternalLabels { for key, value := range settings.ExternalLabels {
// External labels have already been sanitized. // 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. // Skip external labels if they are overridden by metric attributes.
continue continue
} }
l[key] = value c.builder.Set(key, value)
} }
for i := 0; i < len(extras); i += 2 { 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] name := extras[i]
_, found := l[name] if existingValue := c.builder.Get(name); existingValue != "" && logOnOverwrite {
if found && logOnOverwrite {
log.Println("label " + name + " is overwritten. Check if Prometheus reserved labels are used.") log.Println("label " + name + " is overwritten. Check if Prometheus reserved labels are used.")
} }
// internal labels should be maintained. // internal labels should be maintained.
@ -253,15 +192,10 @@ func createAttributes(resource pcommon.Resource, attributes pcommon.Map, scope s
return nil, err return nil, err
} }
} }
l[name] = extras[i+1] c.builder.Set(name, extras[i+1])
} }
labels = labels[:0] return c.builder.Labels(), nil
for k, v := range l {
labels = append(labels, prompb.Label{Name: k, Value: v})
}
return labels, nil
} }
func aggregationTemporality(metric pmetric.Metric) (pmetric.AggregationTemporality, bool, error) { 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: // However, work is under way to resolve this shortcoming through a feature called native histograms custom buckets:
// https://github.com/prometheus/prometheus/issues/13485. // https://github.com/prometheus/prometheus/issues/13485.
func (c *PrometheusConverter) addHistogramDataPoints(ctx context.Context, dataPoints pmetric.HistogramDataPointSlice, 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 { ) error {
for x := 0; x < dataPoints.Len(); x++ { for x := 0; x < dataPoints.Len(); x++ {
if err := c.everyN.checkContext(ctx); err != nil { if err := c.everyN.checkContext(ctx); err != nil {
@ -296,7 +230,8 @@ func (c *PrometheusConverter) addHistogramDataPoints(ctx context.Context, dataPo
pt := dataPoints.At(x) pt := dataPoints.At(x)
timestamp := convertTimeStamp(pt.Timestamp()) 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 { if err != nil {
return err return err
} }
@ -305,35 +240,36 @@ func (c *PrometheusConverter) addHistogramDataPoints(ctx context.Context, dataPo
// omitted // omitted
if pt.HasSum() { if pt.HasSum() {
// treat sum as a sample in an individual TimeSeries // treat sum as a sample in an individual TimeSeries
sum := &prompb.Sample{ val := pt.Sum()
Value: pt.Sum(),
Timestamp: timestamp,
}
if pt.Flags().NoRecordedValue() { if pt.Flags().NoRecordedValue() {
sum.Value = math.Float64frombits(value.StaleNaN) val = math.Float64frombits(value.StaleNaN)
} }
sumlabels := createLabels(metadata.MetricFamilyName+sumStr, baseLabels) sumlabels := c.addLabels(baseName+sumStr, baseLabels)
c.addSample(sum, sumlabels) 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 // treat count as a sample in an individual TimeSeries
count := &prompb.Sample{ val := float64(pt.Count())
Value: float64(pt.Count()),
Timestamp: timestamp,
}
if pt.Flags().NoRecordedValue() { if pt.Flags().NoRecordedValue() {
count.Value = math.Float64frombits(value.StaleNaN) val = math.Float64frombits(value.StaleNaN)
} }
countlabels := createLabels(metadata.MetricFamilyName+countStr, baseLabels) countlabels := c.addLabels(baseName+countStr, baseLabels)
c.addSample(count, countlabels) 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 // cumulative count for conversion to cumulative histogram
var cumulativeCount uint64 var cumulativeCount uint64
var bucketBounds []bucketBoundsData
// process each bound, based on histograms proto definition, # of buckets = # of explicit bounds + 1 // 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++ { for i := 0; i < pt.ExplicitBounds().Len() && i < pt.BucketCounts().Len(); i++ {
if err := c.everyN.checkContext(ctx); err != nil { 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) bound := pt.ExplicitBounds().At(i)
cumulativeCount += pt.BucketCounts().At(i) cumulativeCount += pt.BucketCounts().At(i)
bucket := &prompb.Sample{
Value: float64(cumulativeCount), // Find exemplars that belong to this bucket. Both exemplars and
Timestamp: timestamp, // 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() { if pt.Flags().NoRecordedValue() {
bucket.Value = math.Float64frombits(value.StaleNaN) val = math.Float64frombits(value.StaleNaN)
} }
boundStr := strconv.FormatFloat(bound, 'f', -1, 64) boundStr := strconv.FormatFloat(bound, 'f', -1, 64)
labels := createLabels(metadata.MetricFamilyName+bucketStr, baseLabels, leStr, boundStr) labels := c.addLabels(baseName+bucketStr, baseLabels, leStr, boundStr)
ts := c.addSample(bucket, labels) if err := c.appender.AppendSample(baseName, labels, meta, timestamp, startTimestamp, val, currentBucketExemplars); err != nil {
return err
bucketBounds = append(bucketBounds, bucketBoundsData{ts: ts, bound: bound}) }
} }
// add le=+Inf bucket // add le=+Inf bucket
infBucket := &prompb.Sample{ val = float64(pt.Count())
Timestamp: timestamp,
}
if pt.Flags().NoRecordedValue() { if pt.Flags().NoRecordedValue() {
infBucket.Value = math.Float64frombits(value.StaleNaN) val = math.Float64frombits(value.StaleNaN)
} else {
infBucket.Value = float64(pt.Count())
} }
infLabels := createLabels(metadata.MetricFamilyName+bucketStr, baseLabels, leStr, pInfStr) infLabels := c.addLabels(baseName+bucketStr, baseLabels, leStr, pInfStr)
ts := c.addSample(infBucket, infLabels) if err := c.appender.AppendSample(baseName, infLabels, meta, timestamp, startTimestamp, val, exemplars[nextExemplarIdx:]); err != nil {
bucketBounds = append(bucketBounds, bucketBoundsData{ts: ts, bound: math.Inf(1)})
if err := c.addExemplars(ctx, pt, bucketBounds); err != nil {
return err return err
} }
startTimestamp := pt.StartTimestamp() if settings.ExportCreatedMetric && pt.StartTimestamp() != 0 {
if settings.ExportCreatedMetric && startTimestamp != 0 { labels := c.addLabels(baseName+createdSuffix, baseLabels)
labels := createLabels(metadata.MetricFamilyName+createdSuffix, baseLabels) if c.timeSeriesIsNew(labels) {
c.addTimeSeriesIfNeeded(labels, startTimestamp, pt.Timestamp()) if err := c.appender.AppendSample(baseName, labels, meta, timestamp, 0, float64(startTimestamp), nil); err != nil {
return err
}
}
} }
} }
return nil return nil
} }
type exemplarType interface { func (c *PrometheusConverter) getPromExemplars(ctx context.Context, exemplars pmetric.ExemplarSlice) ([]exemplar.Exemplar, error) {
pmetric.ExponentialHistogramDataPoint | pmetric.HistogramDataPoint | pmetric.NumberDataPoint if exemplars.Len() == 0 {
Exemplars() pmetric.ExemplarSlice return nil, nil
} }
outputExemplars := make([]exemplar.Exemplar, 0, exemplars.Len())
func getPromExemplars[T exemplarType](ctx context.Context, everyN *everyNTimes, pt T) ([]prompb.Exemplar, error) { for i := 0; i < exemplars.Len(); i++ {
promExemplars := make([]prompb.Exemplar, 0, pt.Exemplars().Len()) if err := c.everyN.checkContext(ctx); err != nil {
for i := 0; i < pt.Exemplars().Len(); i++ {
if err := everyN.checkContext(ctx); err != nil {
return nil, err return nil, err
} }
exemplar := pt.Exemplars().At(i) ex := exemplars.At(i)
exemplarRunes := 0 exemplarRunes := 0
promExemplar := prompb.Exemplar{ ts := timestamp.FromTime(ex.Timestamp().AsTime())
Timestamp: timestamp.FromTime(exemplar.Timestamp().AsTime()), newExemplar := exemplar.Exemplar{
Ts: ts,
HasTs: ts != 0,
} }
switch exemplar.ValueType() { c.scratchBuilder.Reset()
switch ex.ValueType() {
case pmetric.ExemplarValueTypeInt: case pmetric.ExemplarValueTypeInt:
promExemplar.Value = float64(exemplar.IntValue()) newExemplar.Value = float64(ex.IntValue())
case pmetric.ExemplarValueTypeDouble: case pmetric.ExemplarValueTypeDouble:
promExemplar.Value = exemplar.DoubleValue() newExemplar.Value = ex.DoubleValue()
default: 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[:]) val := hex.EncodeToString(traceID[:])
exemplarRunes += utf8.RuneCountInString(traceIDKey) + utf8.RuneCountInString(val) exemplarRunes += utf8.RuneCountInString(traceIDKey) + utf8.RuneCountInString(val)
promLabel := prompb.Label{ c.scratchBuilder.Add(traceIDKey, val)
Name: traceIDKey,
Value: val,
}
promExemplar.Labels = append(promExemplar.Labels, promLabel)
} }
if spanID := exemplar.SpanID(); !spanID.IsEmpty() { if spanID := ex.SpanID(); !spanID.IsEmpty() {
val := hex.EncodeToString(spanID[:]) val := hex.EncodeToString(spanID[:])
exemplarRunes += utf8.RuneCountInString(spanIDKey) + utf8.RuneCountInString(val) exemplarRunes += utf8.RuneCountInString(spanIDKey) + utf8.RuneCountInString(val)
promLabel := prompb.Label{ c.scratchBuilder.Add(spanIDKey, val)
Name: spanIDKey,
Value: val,
}
promExemplar.Labels = append(promExemplar.Labels, promLabel)
} }
attrs := exemplar.FilteredAttributes() attrs := ex.FilteredAttributes()
labelsFromAttributes := make([]prompb.Label, 0, attrs.Len())
attrs.Range(func(key string, value pcommon.Value) bool { attrs.Range(func(key string, value pcommon.Value) bool {
val := value.AsString() exemplarRunes += utf8.RuneCountInString(key) + utf8.RuneCountInString(value.AsString())
exemplarRunes += utf8.RuneCountInString(key) + utf8.RuneCountInString(val)
promLabel := prompb.Label{
Name: key,
Value: val,
}
labelsFromAttributes = append(labelsFromAttributes, promLabel)
return true 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 // 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, 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 { ) error {
for x := 0; x < dataPoints.Len(); x++ { for x := 0; x < dataPoints.Len(); x++ {
if err := c.everyN.checkContext(ctx); err != nil { if err := c.everyN.checkContext(ctx); err != nil {
@ -509,75 +439,73 @@ func (c *PrometheusConverter) addSummaryDataPoints(ctx context.Context, dataPoin
pt := dataPoints.At(x) pt := dataPoints.At(x)
timestamp := convertTimeStamp(pt.Timestamp()) 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 { if err != nil {
return err return err
} }
// treat sum as a sample in an individual TimeSeries // treat sum as a sample in an individual TimeSeries
sum := &prompb.Sample{ val := pt.Sum()
Value: pt.Sum(),
Timestamp: timestamp,
}
if pt.Flags().NoRecordedValue() { 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 // sum and count of the summary should append suffix to baseName
sumlabels := createLabels(metadata.MetricFamilyName+sumStr, baseLabels) sumlabels := c.addLabels(baseName+sumStr, baseLabels)
c.addSample(sum, sumlabels) 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 // treat count as a sample in an individual TimeSeries
count := &prompb.Sample{ val = float64(pt.Count())
Value: float64(pt.Count()),
Timestamp: timestamp,
}
if pt.Flags().NoRecordedValue() { 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 // process each percentile/quantile
for i := 0; i < pt.QuantileValues().Len(); i++ { for i := 0; i < pt.QuantileValues().Len(); i++ {
qt := pt.QuantileValues().At(i) qt := pt.QuantileValues().At(i)
quantile := &prompb.Sample{ val = qt.Value()
Value: qt.Value(),
Timestamp: timestamp,
}
if pt.Flags().NoRecordedValue() { if pt.Flags().NoRecordedValue() {
quantile.Value = math.Float64frombits(value.StaleNaN) val = math.Float64frombits(value.StaleNaN)
} }
percentileStr := strconv.FormatFloat(qt.Quantile(), 'f', -1, 64) percentileStr := strconv.FormatFloat(qt.Quantile(), 'f', -1, 64)
qtlabels := createLabels(metadata.MetricFamilyName, baseLabels, quantileStr, percentileStr) qtlabels := c.addLabels(baseName, baseLabels, quantileStr, percentileStr)
c.addSample(quantile, qtlabels) if err := c.appender.AppendSample(baseName, qtlabels, meta, timestamp, startTimestamp, val, nil); err != nil {
return err
}
} }
startTimestamp := pt.StartTimestamp() if settings.ExportCreatedMetric && pt.StartTimestamp() != 0 {
if settings.ExportCreatedMetric && startTimestamp != 0 { createdLabels := c.addLabels(baseName+createdSuffix, baseLabels)
createdLabels := createLabels(metadata.MetricFamilyName+createdSuffix, baseLabels) if c.timeSeriesIsNew(createdLabels) {
c.addTimeSeriesIfNeeded(createdLabels, startTimestamp, pt.Timestamp()) if err := c.appender.AppendSample(baseName, createdLabels, meta, timestamp, 0, float64(startTimestamp), nil); err != nil {
return err
}
}
} }
} }
return nil 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 are provided, corresponding label pairs are also added to the returned slice.
// If extras is uneven length, the last (unpaired) extra will be ignored. // If extras is uneven length, the last (unpaired) extra will be ignored.
func createLabels(name string, baseLabels []prompb.Label, extras ...string) []prompb.Label { func (c *PrometheusConverter) addLabels(name string, baseLabels labels.Labels, extras ...string) labels.Labels {
extraLabelCount := len(extras) / 2 c.builder.Reset(baseLabels)
labels := make([]prompb.Label, len(baseLabels), len(baseLabels)+extraLabelCount+1) // +1 for name
copy(labels, baseLabels)
n := len(extras) n := len(extras)
n -= n % 2 n -= n % 2
for extrasIdx := 0; extrasIdx < n; extrasIdx += 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])
} }
c.builder.Set(model.MetricNameLabel, name)
labels = append(labels, prompb.Label{Name: model.MetricNameLabel, Value: name}) return c.builder.Labels()
return labels
} }
// addTypeAndUnitLabels appends type and unit labels to the given labels slice. // 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. // 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. // Otherwise it creates a new one and returns that, and true.
func (c *PrometheusConverter) getOrCreateTimeSeries(lbls []prompb.Label) (*prompb.TimeSeries, bool) { func (c *PrometheusConverter) timeSeriesIsNew(lbls labels.Labels) bool {
h := timeSeriesSignature(lbls) h := lbls.Hash()
ts := c.unique[h] uLabels, ok := c.unique[h]
if ts != nil { if ok {
if isSameMetric(ts, lbls) { if labels.Equal(uLabels, lbls) {
// We already have this metric // We already have this metric
return ts, false return false
} }
// Look for a matching conflict // Look for a matching conflict
for _, cTS := range c.conflicts[h] { for _, cLabels := range c.conflicts[h] {
if isSameMetric(cTS, lbls) { if labels.Equal(cLabels, lbls) {
// We already have this metric // We already have this metric
return cTS, false return false
} }
} }
// New conflict // New conflict
ts = &prompb.TimeSeries{ c.conflicts[h] = append(c.conflicts[h], uLabels)
Labels: lbls, return true
}
c.conflicts[h] = append(c.conflicts[h], ts)
return ts, true
} }
// This metric is new // This metric is new
ts = &prompb.TimeSeries{ c.unique[h] = uLabels
Labels: lbls, return true
}
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),
},
}
}
} }
// addResourceTargetInfo converts the resource to the target info metric. // 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 { if settings.DisableTargetInfo {
return nil return nil
} }
@ -679,17 +585,21 @@ func addResourceTargetInfo(resource pcommon.Resource, settings Settings, earlies
// Do not pass identifying attributes as ignoreAttrs below. // Do not pass identifying attributes as ignoreAttrs below.
identifyingAttrs = nil 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 { if err != nil {
return err return err
} }
haveIdentifier := false haveIdentifier := false
for _, l := range labels { lbls.Range(func(l modelLabels.Label) {
if l.Name == model.JobLabel || l.Name == model.InstanceLabel { if l.Name == model.JobLabel || l.Name == model.InstanceLabel {
haveIdentifier = true haveIdentifier = true
break
} }
} })
if !haveIdentifier { if !haveIdentifier {
// We need at least one identifying label to generate target_info. // 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 settings.LookbackDelta = defaultLookbackDelta
} }
interval := settings.LookbackDelta / 2 interval := settings.LookbackDelta / 2
ts, _ := converter.getOrCreateTimeSeries(labels) timestamp := earliestTimestamp
for timestamp := earliestTimestamp; timestamp.Before(latestTimestamp); timestamp = timestamp.Add(interval) { for ; timestamp.Before(latestTimestamp); timestamp = timestamp.Add(interval) {
ts.Samples = append(ts.Samples, prompb.Sample{ if err := c.appender.AppendSample(name, lbls, meta, timestamp.UnixMilli(), 0, float64(1), nil); err != nil {
Value: float64(1), return err
Timestamp: timestamp.UnixMilli(), }
})
} }
ts.Samples = append(ts.Samples, prompb.Sample{ return c.appender.AppendSample(name, lbls, meta, latestTimestamp.UnixMilli(), 0, float64(1), nil)
Value: float64(1),
Timestamp: latestTimestamp.UnixMilli(),
})
return nil
} }
// convertTimeStamp converts OTLP timestamp in ns to timestamp in ms. // convertTimeStamp converts OTLP timestamp in ns to timestamp in ms.

View File

@ -26,8 +26,8 @@ import (
"go.opentelemetry.io/collector/pdata/pmetric" "go.opentelemetry.io/collector/pdata/pmetric"
"github.com/prometheus/prometheus/model/histogram" "github.com/prometheus/prometheus/model/histogram"
"github.com/prometheus/prometheus/model/metadata"
"github.com/prometheus/prometheus/model/value" "github.com/prometheus/prometheus/model/value"
"github.com/prometheus/prometheus/prompb"
"github.com/prometheus/prometheus/util/annotations" "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 // addExponentialHistogramDataPoints adds OTel exponential histogram data points to the corresponding time series
// as native histogram samples. // as native histogram samples.
func (c *PrometheusConverter) addExponentialHistogramDataPoints(ctx context.Context, dataPoints pmetric.ExponentialHistogramDataPointSlice, 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) { ) (annotations.Annotations, error) {
var annots annotations.Annotations var annots annotations.Annotations
for x := 0; x < dataPoints.Len(); x++ { for x := 0; x < dataPoints.Len(); x++ {
@ -46,34 +47,36 @@ func (c *PrometheusConverter) addExponentialHistogramDataPoints(ctx context.Cont
pt := dataPoints.At(x) pt := dataPoints.At(x)
histogram, ws, err := exponentialToNativeHistogram(pt, temporality) hp, ws, err := exponentialToNativeHistogram(pt, temporality)
annots.Merge(ws) annots.Merge(ws)
if err != nil { if err != nil {
return annots, err return annots, err
} }
lbls, err := createAttributes( lbls, err := c.createAttributes(
resource, resource,
pt.Attributes(), pt.Attributes(),
scope, scope,
settings, settings,
nil, nil,
true, true,
metadata, meta,
model.MetricNameLabel, 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 { if err != nil {
return annots, err 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 return annots, nil
@ -81,11 +84,11 @@ func (c *PrometheusConverter) addExponentialHistogramDataPoints(ctx context.Cont
// exponentialToNativeHistogram translates an OTel Exponential Histogram data point // exponentialToNativeHistogram translates an OTel Exponential Histogram data point
// to a Prometheus Native Histogram. // 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 var annots annotations.Annotations
scale := p.Scale() scale := p.Scale()
if scale < -4 { if scale < -4 {
return prompb.Histogram{}, annots, return nil, annots,
fmt.Errorf("cannot convert exponential to native histogram."+ fmt.Errorf("cannot convert exponential to native histogram."+
" Scale must be >= -4, was %d", scale) " 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. // need to know here if it was used for the detection.
// Ref: https://github.com/open-telemetry/opentelemetry-collector-contrib/pull/28663#issuecomment-1810577303 // 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 // 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 temporality == pmetric.AggregationTemporalityDelta {
// If the histogram has delta temporality, set the reset hint to gauge to avoid unnecessary chunk cutting. // 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/). // 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 // This might be changed to a different hint name as gauge type might be misleading for samples that should be
// summed over time. // summed over time.
resetHint = prompb.Histogram_GAUGE resetHint = histogram.GaugeType
} }
h := &histogram.Histogram{
h := prompb.Histogram{ CounterResetHint: resetHint,
ResetHint: resetHint, Schema: scale,
Schema: scale,
ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: p.ZeroCount()},
// TODO use zero_threshold, if set, see // TODO use zero_threshold, if set, see
// https://github.com/open-telemetry/opentelemetry-proto/pull/441 // https://github.com/open-telemetry/opentelemetry-proto/pull/441
ZeroThreshold: defaultZeroThreshold, ZeroThreshold: defaultZeroThreshold,
ZeroCount: p.ZeroCount(),
PositiveSpans: pSpans, PositiveSpans: pSpans,
PositiveDeltas: pDeltas, PositiveBuckets: pDeltas,
NegativeSpans: nSpans, NegativeSpans: nSpans,
NegativeDeltas: nDeltas, NegativeBuckets: nDeltas,
Timestamp: convertTimeStamp(p.Timestamp()),
} }
if p.Flags().NoRecordedValue() { if p.Flags().NoRecordedValue() {
h.Sum = math.Float64frombits(value.StaleNaN) h.Sum = math.Float64frombits(value.StaleNaN)
h.Count = &prompb.Histogram_CountInt{CountInt: value.StaleNaN} h.Count = value.StaleNaN
} else { } else {
if p.HasSum() { if p.HasSum() {
h.Sum = p.Sum() h.Sum = p.Sum()
} }
h.Count = &prompb.Histogram_CountInt{CountInt: p.Count()} h.Count = p.Count()
if p.Count() == 0 && h.Sum != 0 { 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)) 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, // 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. // 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 { if len(bucketCounts) == 0 {
return nil, nil return nil, nil
} }
var ( var (
spans []prompb.BucketSpan spans []histogram.Span
deltas []int64 deltas []int64
count int64 count int64
prevCount int64 prevCount int64
@ -196,7 +194,7 @@ func convertBucketsLayout(bucketCounts []uint64, offset, scaleDown int32, adjust
initialOffset = initialOffset>>scaleDown + 1 initialOffset = initialOffset>>scaleDown + 1
} }
spans = append(spans, prompb.BucketSpan{ spans = append(spans, histogram.Span{
Offset: initialOffset, Offset: initialOffset,
Length: 0, 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 // 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 // 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 // https://github.com/prometheus/client_golang/blob/27f0506d6ebbb117b6b697d0552ee5be2502c5f2/prometheus/histogram.go#L1296
spans = append(spans, prompb.BucketSpan{ spans = append(spans, histogram.Span{
Offset: gap, Offset: gap,
Length: 0, 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 // 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 // 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 // https://github.com/prometheus/client_golang/blob/27f0506d6ebbb117b6b697d0552ee5be2502c5f2/prometheus/histogram.go#L1296
spans = append(spans, prompb.BucketSpan{ spans = append(spans, histogram.Span{
Offset: gap, Offset: gap,
Length: 0, Length: 0,
}) })
@ -256,7 +254,8 @@ func convertBucketsLayout(bucketCounts []uint64, offset, scaleDown int32, adjust
} }
func (c *PrometheusConverter) addCustomBucketsHistogramDataPoints(ctx context.Context, dataPoints pmetric.HistogramDataPointSlice, 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) { ) (annotations.Annotations, error) {
var annots annotations.Annotations var annots annotations.Annotations
@ -267,41 +266,41 @@ func (c *PrometheusConverter) addCustomBucketsHistogramDataPoints(ctx context.Co
pt := dataPoints.At(x) pt := dataPoints.At(x)
histogram, ws, err := explicitHistogramToCustomBucketsHistogram(pt, temporality) hp, ws, err := explicitHistogramToCustomBucketsHistogram(pt, temporality)
annots.Merge(ws) annots.Merge(ws)
if err != nil { if err != nil {
return annots, err return annots, err
} }
lbls, err := createAttributes( lbls, err := c.createAttributes(
resource, resource,
pt.Attributes(), pt.Attributes(),
scope, scope,
settings, settings,
nil, nil,
true, true,
metadata, meta,
model.MetricNameLabel, 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 { if err != nil {
return annots, err 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 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 var annots annotations.Annotations
buckets := p.BucketCounts().AsRaw() 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. // need to know here if it was used for the detection.
// Ref: https://github.com/open-telemetry/opentelemetry-collector-contrib/pull/28663#issuecomment-1810577303 // 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 // 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 temporality == pmetric.AggregationTemporalityDelta {
// If the histogram has delta temporality, set the reset hint to gauge to avoid unnecessary chunk cutting. // 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/). // 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 // This might be changed to a different hint name as gauge type might be misleading for samples that should be
// summed over time. // summed over time.
resetHint = prompb.Histogram_GAUGE resetHint = histogram.GaugeType
} }
// TODO(carrieedwards): Add setting to limit maximum bucket count // TODO(carrieedwards): Add setting to limit maximum bucket count
h := prompb.Histogram{ h := &histogram.Histogram{
ResetHint: resetHint, CounterResetHint: resetHint,
Schema: histogram.CustomBucketsSchema, Schema: histogram.CustomBucketsSchema,
PositiveSpans: positiveSpans,
PositiveSpans: positiveSpans, PositiveBuckets: positiveDeltas,
PositiveDeltas: positiveDeltas,
// Note: OTel explicit histograms have an implicit +Inf bucket, which has a lower bound // Note: OTel explicit histograms have an implicit +Inf bucket, which has a lower bound
// of the last element in the explicit_bounds array. // of the last element in the explicit_bounds array.
// This is similar to the custom_values array in native histograms with custom buckets. // 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. // 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 // See: https://github.com/open-telemetry/opentelemetry-proto/blob/d7770822d70c7bd47a6891fc9faacc66fc4af3d3/opentelemetry/proto/metrics/v1/metrics.proto#L469
CustomValues: p.ExplicitBounds().AsRaw(), CustomValues: p.ExplicitBounds().AsRaw(),
Timestamp: convertTimeStamp(p.Timestamp()),
} }
if p.Flags().NoRecordedValue() { if p.Flags().NoRecordedValue() {
h.Sum = math.Float64frombits(value.StaleNaN) h.Sum = math.Float64frombits(value.StaleNaN)
h.Count = &prompb.Histogram_CountInt{CountInt: value.StaleNaN} h.Count = value.StaleNaN
} else { } else {
if p.HasSum() { if p.HasSum() {
h.Sum = p.Sum() h.Sum = p.Sum()
} }
h.Count = &prompb.Histogram_CountInt{CountInt: p.Count()} h.Count = p.Count()
if p.Count() == 0 && h.Sum != 0 { if p.Count() == 0 && h.Sum != 0 {
annots.Add(fmt.Errorf("histogram data point has zero count, but non-zero sum: %f", h.Sum)) annots.Add(fmt.Errorf("histogram data point has zero count, but non-zero sum: %f", h.Sum))
} }

View File

@ -28,11 +28,14 @@ import (
"go.opentelemetry.io/collector/pdata/pcommon" "go.opentelemetry.io/collector/pdata/pcommon"
"go.opentelemetry.io/collector/pdata/pmetric" "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 { type expectedBucketLayout struct {
wantSpans []prompb.BucketSpan wantSpans []histogram.Span
wantDeltas []int64 wantDeltas []int64
} }
@ -52,7 +55,7 @@ func TestConvertBucketsLayout(t *testing.T) {
}, },
wantLayout: map[int32]expectedBucketLayout{ wantLayout: map[int32]expectedBucketLayout{
0: { 0: {
wantSpans: []prompb.BucketSpan{ wantSpans: []histogram.Span{
{ {
Offset: 1, Offset: 1,
Length: 4, Length: 4,
@ -61,7 +64,7 @@ func TestConvertBucketsLayout(t *testing.T) {
wantDeltas: []int64{4, -1, -1, -1}, wantDeltas: []int64{4, -1, -1, -1},
}, },
1: { 1: {
wantSpans: []prompb.BucketSpan{ wantSpans: []histogram.Span{
{ {
Offset: 1, Offset: 1,
Length: 2, Length: 2,
@ -71,7 +74,7 @@ func TestConvertBucketsLayout(t *testing.T) {
wantDeltas: []int64{7, -4}, wantDeltas: []int64{7, -4},
}, },
2: { 2: {
wantSpans: []prompb.BucketSpan{ wantSpans: []histogram.Span{
{ {
Offset: 1, Offset: 1,
Length: 1, Length: 1,
@ -92,7 +95,7 @@ func TestConvertBucketsLayout(t *testing.T) {
}, },
wantLayout: map[int32]expectedBucketLayout{ wantLayout: map[int32]expectedBucketLayout{
0: { 0: {
wantSpans: []prompb.BucketSpan{ wantSpans: []histogram.Span{
{ {
Offset: 2, Offset: 2,
Length: 4, Length: 4,
@ -101,7 +104,7 @@ func TestConvertBucketsLayout(t *testing.T) {
wantDeltas: []int64{4, -1, -1, -1}, wantDeltas: []int64{4, -1, -1, -1},
}, },
1: { 1: {
wantSpans: []prompb.BucketSpan{ wantSpans: []histogram.Span{
{ {
Offset: 1, Offset: 1,
Length: 3, 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 wantDeltas: []int64{4, 1, -4}, // 0+4, 3+2, 1+0 = 4, 5, 1
}, },
2: { 2: {
wantSpans: []prompb.BucketSpan{ wantSpans: []histogram.Span{
{ {
Offset: 1, Offset: 1,
Length: 2, Length: 2,
@ -130,7 +133,7 @@ func TestConvertBucketsLayout(t *testing.T) {
}, },
wantLayout: map[int32]expectedBucketLayout{ wantLayout: map[int32]expectedBucketLayout{
0: { 0: {
wantSpans: []prompb.BucketSpan{ wantSpans: []histogram.Span{
{ {
Offset: 5, Offset: 5,
Length: 4, Length: 4,
@ -143,7 +146,7 @@ func TestConvertBucketsLayout(t *testing.T) {
wantDeltas: []int64{4, -2, -2, 2, -1}, wantDeltas: []int64{4, -2, -2, 2, -1},
}, },
1: { 1: {
wantSpans: []prompb.BucketSpan{ wantSpans: []histogram.Span{
{ {
Offset: 3, Offset: 3,
Length: 2, Length: 2,
@ -158,7 +161,7 @@ func TestConvertBucketsLayout(t *testing.T) {
wantDeltas: []int64{6, -4, -1}, wantDeltas: []int64{6, -4, -1},
}, },
2: { 2: {
wantSpans: []prompb.BucketSpan{ wantSpans: []histogram.Span{
{ {
Offset: 2, Offset: 2,
Length: 1, Length: 1,
@ -185,7 +188,7 @@ func TestConvertBucketsLayout(t *testing.T) {
}, },
wantLayout: map[int32]expectedBucketLayout{ wantLayout: map[int32]expectedBucketLayout{
0: { 0: {
wantSpans: []prompb.BucketSpan{ wantSpans: []histogram.Span{
{ {
Offset: 5, Offset: 5,
Length: 4, Length: 4,
@ -198,7 +201,7 @@ func TestConvertBucketsLayout(t *testing.T) {
wantDeltas: []int64{4, -2, -2, 2, -1}, wantDeltas: []int64{4, -2, -2, 2, -1},
}, },
1: { 1: {
wantSpans: []prompb.BucketSpan{ wantSpans: []histogram.Span{
{ {
Offset: 3, Offset: 3,
Length: 2, Length: 2,
@ -213,7 +216,7 @@ func TestConvertBucketsLayout(t *testing.T) {
wantDeltas: []int64{6, -4, -1}, wantDeltas: []int64{6, -4, -1},
}, },
2: { 2: {
wantSpans: []prompb.BucketSpan{ wantSpans: []histogram.Span{
{ {
Offset: 2, Offset: 2,
Length: 4, Length: 4,
@ -236,7 +239,7 @@ func TestConvertBucketsLayout(t *testing.T) {
}, },
wantLayout: map[int32]expectedBucketLayout{ wantLayout: map[int32]expectedBucketLayout{
0: { 0: {
wantSpans: []prompb.BucketSpan{ wantSpans: []histogram.Span{
{ {
Offset: -1, Offset: -1,
Length: 2, Length: 2,
@ -249,7 +252,7 @@ func TestConvertBucketsLayout(t *testing.T) {
wantDeltas: []int64{3, -2, 0}, wantDeltas: []int64{3, -2, 0},
}, },
1: { 1: {
wantSpans: []prompb.BucketSpan{ wantSpans: []histogram.Span{
{ {
Offset: 0, Offset: 0,
Length: 3, Length: 3,
@ -260,7 +263,7 @@ func TestConvertBucketsLayout(t *testing.T) {
wantDeltas: []int64{4, -4, 1}, wantDeltas: []int64{4, -4, 1},
}, },
2: { 2: {
wantSpans: []prompb.BucketSpan{ wantSpans: []histogram.Span{
{ {
Offset: 0, Offset: 0,
Length: 2, Length: 2,
@ -282,7 +285,7 @@ func TestConvertBucketsLayout(t *testing.T) {
}, },
wantLayout: map[int32]expectedBucketLayout{ wantLayout: map[int32]expectedBucketLayout{
0: { 0: {
wantSpans: []prompb.BucketSpan{ wantSpans: []histogram.Span{
{ {
Offset: -1, Offset: -1,
Length: 6, Length: 6,
@ -291,7 +294,7 @@ func TestConvertBucketsLayout(t *testing.T) {
wantDeltas: []int64{3, -2, -1, 1, -1, 1}, wantDeltas: []int64{3, -2, -1, 1, -1, 1},
}, },
1: { 1: {
wantSpans: []prompb.BucketSpan{ wantSpans: []histogram.Span{
{ {
Offset: 0, Offset: 0,
Length: 3, Length: 3,
@ -302,7 +305,7 @@ func TestConvertBucketsLayout(t *testing.T) {
wantDeltas: []int64{4, -3, 0}, wantDeltas: []int64{4, -3, 0},
}, },
2: { 2: {
wantSpans: []prompb.BucketSpan{ wantSpans: []histogram.Span{
{ {
Offset: 0, Offset: 0,
Length: 2, Length: 2,
@ -324,7 +327,7 @@ func TestConvertBucketsLayout(t *testing.T) {
}, },
wantLayout: map[int32]expectedBucketLayout{ wantLayout: map[int32]expectedBucketLayout{
0: { 0: {
wantSpans: []prompb.BucketSpan{ wantSpans: []histogram.Span{
{ {
Offset: -1, Offset: -1,
Length: 7, Length: 7,
@ -333,7 +336,7 @@ func TestConvertBucketsLayout(t *testing.T) {
wantDeltas: []int64{3, -3, 0, 1, -1, 0, 1}, wantDeltas: []int64{3, -3, 0, 1, -1, 0, 1},
}, },
1: { 1: {
wantSpans: []prompb.BucketSpan{ wantSpans: []histogram.Span{
{ {
Offset: 0, Offset: 0,
Length: 4, Length: 4,
@ -344,7 +347,7 @@ func TestConvertBucketsLayout(t *testing.T) {
wantDeltas: []int64{3, -2, -1, 1}, wantDeltas: []int64{3, -2, -1, 1},
}, },
2: { 2: {
wantSpans: []prompb.BucketSpan{ wantSpans: []histogram.Span{
{ {
Offset: 0, Offset: 0,
Length: 3, Length: 3,
@ -379,8 +382,8 @@ func TestConvertBucketsLayout(t *testing.T) {
for scaleDown, wantLayout := range tt.wantLayout { for scaleDown, wantLayout := range tt.wantLayout {
t.Run(fmt.Sprintf("%s-scaleby-%d", tt.name, scaleDown), func(t *testing.T) { 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) gotSpans, gotDeltas := convertBucketsLayout(tt.buckets().BucketCounts().AsRaw(), tt.buckets().Offset(), scaleDown, true)
require.Equal(t, wantLayout.wantSpans, gotSpans) requireEqual(t, wantLayout.wantSpans, gotSpans)
require.Equal(t, wantLayout.wantDeltas, gotDeltas) requireEqual(t, wantLayout.wantDeltas, gotDeltas)
}) })
} }
} }
@ -418,7 +421,7 @@ func TestExponentialToNativeHistogram(t *testing.T) {
tests := []struct { tests := []struct {
name string name string
exponentialHist func() pmetric.ExponentialHistogramDataPoint exponentialHist func() pmetric.ExponentialHistogramDataPoint
wantNativeHist func() prompb.Histogram wantNativeHist func() *histogram.Histogram
wantErrMessage string wantErrMessage string
}{ }{
{ {
@ -440,18 +443,17 @@ func TestExponentialToNativeHistogram(t *testing.T) {
return pt return pt
}, },
wantNativeHist: func() prompb.Histogram { wantNativeHist: func() *histogram.Histogram {
return prompb.Histogram{ return &histogram.Histogram{
Count: &prompb.Histogram_CountInt{CountInt: 4}, Count: 4,
Sum: 10.1, Sum: 10.1,
Schema: 1, Schema: 1,
ZeroThreshold: defaultZeroThreshold, ZeroThreshold: defaultZeroThreshold,
ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: 1}, ZeroCount: 1,
NegativeSpans: []prompb.BucketSpan{{Offset: 2, Length: 2}}, NegativeSpans: []histogram.Span{{Offset: 2, Length: 2}},
NegativeDeltas: []int64{1, 0}, NegativeBuckets: []int64{1, 0},
PositiveSpans: []prompb.BucketSpan{{Offset: 2, Length: 2}}, PositiveSpans: []histogram.Span{{Offset: 2, Length: 2}},
PositiveDeltas: []int64{1, 0}, PositiveBuckets: []int64{1, 0},
Timestamp: 500,
} }
}, },
}, },
@ -474,17 +476,16 @@ func TestExponentialToNativeHistogram(t *testing.T) {
return pt return pt
}, },
wantNativeHist: func() prompb.Histogram { wantNativeHist: func() *histogram.Histogram {
return prompb.Histogram{ return &histogram.Histogram{
Count: &prompb.Histogram_CountInt{CountInt: 4}, Count: 4,
Schema: 1, Schema: 1,
ZeroThreshold: defaultZeroThreshold, ZeroThreshold: defaultZeroThreshold,
ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: 1}, ZeroCount: 1,
NegativeSpans: []prompb.BucketSpan{{Offset: 2, Length: 2}}, NegativeSpans: []histogram.Span{{Offset: 2, Length: 2}},
NegativeDeltas: []int64{1, 0}, NegativeBuckets: []int64{1, 0},
PositiveSpans: []prompb.BucketSpan{{Offset: 2, Length: 2}}, PositiveSpans: []histogram.Span{{Offset: 2, Length: 2}},
PositiveDeltas: []int64{1, 0}, PositiveBuckets: []int64{1, 0},
Timestamp: 500,
} }
}, },
}, },
@ -515,18 +516,17 @@ func TestExponentialToNativeHistogram(t *testing.T) {
pt.Negative().SetOffset(2) pt.Negative().SetOffset(2)
return pt return pt
}, },
wantNativeHist: func() prompb.Histogram { wantNativeHist: func() *histogram.Histogram {
return prompb.Histogram{ return &histogram.Histogram{
Count: &prompb.Histogram_CountInt{CountInt: 6}, Count: 6,
Sum: 10.1, Sum: 10.1,
Schema: 8, Schema: 8,
ZeroThreshold: defaultZeroThreshold, ZeroThreshold: defaultZeroThreshold,
ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: 1}, ZeroCount: 1,
PositiveSpans: []prompb.BucketSpan{{Offset: 2, Length: 3}}, PositiveSpans: []histogram.Span{{Offset: 2, Length: 3}},
PositiveDeltas: []int64{1, 0, 0}, // 1, 1, 1 PositiveBuckets: []int64{1, 0, 0}, // 1, 1, 1
NegativeSpans: []prompb.BucketSpan{{Offset: 3, Length: 3}}, NegativeSpans: []histogram.Span{{Offset: 3, Length: 3}},
NegativeDeltas: []int64{1, 0, 0}, // 1, 1, 1 NegativeBuckets: []int64{1, 0, 0}, // 1, 1, 1
Timestamp: 500,
} }
}, },
}, },
@ -547,18 +547,17 @@ func TestExponentialToNativeHistogram(t *testing.T) {
pt.Negative().SetOffset(2) pt.Negative().SetOffset(2)
return pt return pt
}, },
wantNativeHist: func() prompb.Histogram { wantNativeHist: func() *histogram.Histogram {
return prompb.Histogram{ return &histogram.Histogram{
Count: &prompb.Histogram_CountInt{CountInt: 6}, Count: 6,
Sum: 10.1, Sum: 10.1,
Schema: 8, Schema: 8,
ZeroThreshold: defaultZeroThreshold, ZeroThreshold: defaultZeroThreshold,
ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: 1}, ZeroCount: 1,
PositiveSpans: []prompb.BucketSpan{{Offset: 1, Length: 2}}, PositiveSpans: []histogram.Span{{Offset: 1, Length: 2}},
PositiveDeltas: []int64{1, 1}, // 0+1, 1+1 = 1, 2 PositiveBuckets: []int64{1, 1}, // 0+1, 1+1 = 1, 2
NegativeSpans: []prompb.BucketSpan{{Offset: 2, Length: 2}}, NegativeSpans: []histogram.Span{{Offset: 2, Length: 2}},
NegativeDeltas: []int64{2, -1}, // 1+1, 1+0 = 2, 1 NegativeBuckets: []int64{2, -1}, // 1+1, 1+0 = 2, 1
Timestamp: 500,
} }
}, },
}, },
@ -599,20 +598,18 @@ func validateExponentialHistogramCount(t *testing.T, h pmetric.ExponentialHistog
require.Equal(t, h.Count(), actualCount, "exponential histogram count mismatch") require.Equal(t, h.Count(), actualCount, "exponential histogram count mismatch")
} }
func validateNativeHistogramCount(t *testing.T, h prompb.Histogram) { func validateNativeHistogramCount(t *testing.T, h *histogram.Histogram) {
require.NotNil(t, h.Count) want := h.Count
require.IsType(t, &prompb.Histogram_CountInt{}, h.Count)
want := h.Count.(*prompb.Histogram_CountInt).CountInt
var ( var (
actualCount uint64 actualCount uint64
prevBucket int64 prevBucket int64
) )
for _, delta := range h.PositiveDeltas { for _, delta := range h.PositiveBuckets {
prevBucket += delta prevBucket += delta
actualCount += uint64(prevBucket) actualCount += uint64(prevBucket)
} }
prevBucket = 0 prevBucket = 0
for _, delta := range h.NegativeDeltas { for _, delta := range h.NegativeBuckets {
prevBucket += delta prevBucket += delta
actualCount += uint64(prevBucket) actualCount += uint64(prevBucket)
} }
@ -636,7 +633,7 @@ func TestPrometheusConverter_addExponentialHistogramDataPoints(t *testing.T) {
metric func() pmetric.Metric metric func() pmetric.Metric
scope scope scope scope
promoteScope bool promoteScope bool
wantSeries func() map[uint64]*prompb.TimeSeries wantSeries func() []combinedHistogram
}{ }{
{ {
name: "histogram data points with same labels and without scope promotion", name: "histogram data points with same labels and without scope promotion",
@ -665,36 +662,43 @@ func TestPrometheusConverter_addExponentialHistogramDataPoints(t *testing.T) {
}, },
scope: defaultScope, scope: defaultScope,
promoteScope: false, promoteScope: false,
wantSeries: func() map[uint64]*prompb.TimeSeries { wantSeries: func() []combinedHistogram {
labels := []prompb.Label{ lbls := labels.FromStrings(
{Name: model.MetricNameLabel, Value: "test_hist"}, model.MetricNameLabel, "test_hist",
{Name: "attr", Value: "test_attr"}, "attr", "test_attr",
} )
return map[uint64]*prompb.TimeSeries{ return []combinedHistogram{
timeSeriesSignature(labels): { {
Labels: labels, metricFamilyName: "test_hist",
Histograms: []prompb.Histogram{ ls: lbls,
{ meta: metadata.Metadata{},
Count: &prompb.Histogram_CountInt{CountInt: 7}, t: 0,
Schema: 1, ct: 0,
ZeroThreshold: defaultZeroThreshold, h: &histogram.Histogram{
ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: 0}, Count: 7,
PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 2}}, Schema: 1,
PositiveDeltas: []int64{4, -2}, ZeroThreshold: defaultZeroThreshold,
}, ZeroCount: 0,
{ PositiveSpans: []histogram.Span{{Offset: 0, Length: 2}},
Count: &prompb.Histogram_CountInt{CountInt: 4}, PositiveBuckets: []int64{4, -2},
Schema: 1,
ZeroThreshold: defaultZeroThreshold,
ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: 0},
PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 3}},
PositiveDeltas: []int64{4, -2, -1},
},
}, },
Exemplars: []prompb.Exemplar{ es: []exemplar.Exemplar{{Value: 1}},
{Value: 1}, },
{Value: 2}, {
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, scope: defaultScope,
promoteScope: true, promoteScope: true,
wantSeries: func() map[uint64]*prompb.TimeSeries { wantSeries: func() []combinedHistogram {
labels := []prompb.Label{ lbls := labels.FromStrings(
{Name: model.MetricNameLabel, Value: "test_hist"}, model.MetricNameLabel, "test_hist",
{Name: "attr", Value: "test_attr"}, "attr", "test_attr",
{Name: "otel_scope_name", Value: defaultScope.name}, "otel_scope_name", defaultScope.name,
{Name: "otel_scope_schema_url", Value: defaultScope.schemaURL}, "otel_scope_schema_url", defaultScope.schemaURL,
{Name: "otel_scope_version", Value: defaultScope.version}, "otel_scope_version", defaultScope.version,
{Name: "otel_scope_attr1", Value: "value1"}, "otel_scope_attr1", "value1",
{Name: "otel_scope_attr2", Value: "value2"}, "otel_scope_attr2", "value2",
} )
return map[uint64]*prompb.TimeSeries{ return []combinedHistogram{
timeSeriesSignature(labels): { {
Labels: labels, metricFamilyName: "test_hist",
Histograms: []prompb.Histogram{ ls: lbls,
{ meta: metadata.Metadata{},
Count: &prompb.Histogram_CountInt{CountInt: 7}, t: 0,
Schema: 1, ct: 0,
ZeroThreshold: defaultZeroThreshold, h: &histogram.Histogram{
ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: 0}, Count: 7,
PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 2}}, Schema: 1,
PositiveDeltas: []int64{4, -2}, ZeroThreshold: defaultZeroThreshold,
}, ZeroCount: 0,
{ PositiveSpans: []histogram.Span{{Offset: 0, Length: 2}},
Count: &prompb.Histogram_CountInt{CountInt: 4}, PositiveBuckets: []int64{4, -2},
Schema: 1,
ZeroThreshold: defaultZeroThreshold,
ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: 0},
PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 3}},
PositiveDeltas: []int64{4, -2, -1},
},
}, },
Exemplars: []prompb.Exemplar{ es: []exemplar.Exemplar{{Value: 1}},
{Value: 1}, },
{Value: 2}, {
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, scope: defaultScope,
promoteScope: false, promoteScope: false,
wantSeries: func() map[uint64]*prompb.TimeSeries { wantSeries: func() []combinedHistogram {
labels := []prompb.Label{ lbls := labels.FromStrings(
{Name: model.MetricNameLabel, Value: "test_hist"}, model.MetricNameLabel, "test_hist",
{Name: "attr", Value: "test_attr"}, "attr", "test_attr",
} )
labelsAnother := []prompb.Label{ labelsAnother := labels.FromStrings(
{Name: model.MetricNameLabel, Value: "test_hist"}, model.MetricNameLabel, "test_hist",
{Name: "attr", Value: "test_attr_two"}, "attr", "test_attr_two",
} )
return map[uint64]*prompb.TimeSeries{ return []combinedHistogram{
timeSeriesSignature(labels): { {
Labels: labels, metricFamilyName: "test_hist",
Histograms: []prompb.Histogram{ ls: lbls,
{ meta: metadata.Metadata{},
Count: &prompb.Histogram_CountInt{CountInt: 7}, t: 0,
Schema: 1, ct: 0,
ZeroThreshold: defaultZeroThreshold, h: &histogram.Histogram{
ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: 0}, Count: 7,
PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 2}}, Schema: 1,
PositiveDeltas: []int64{4, -2}, ZeroThreshold: defaultZeroThreshold,
}, ZeroCount: 0,
}, PositiveSpans: []histogram.Span{{Offset: 0, Length: 2}},
Exemplars: []prompb.Exemplar{ PositiveBuckets: []int64{4, -2},
{Value: 1},
}, },
es: []exemplar.Exemplar{{Value: 1}},
}, },
timeSeriesSignature(labelsAnother): { {
Labels: labelsAnother, metricFamilyName: "test_hist",
Histograms: []prompb.Histogram{ ls: labelsAnother,
{ meta: metadata.Metadata{},
Count: &prompb.Histogram_CountInt{CountInt: 4}, t: 0,
Schema: 1, ct: 0,
ZeroThreshold: defaultZeroThreshold, h: &histogram.Histogram{
ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: 0}, Count: 4,
NegativeSpans: []prompb.BucketSpan{{Offset: 0, Length: 3}}, Schema: 1,
NegativeDeltas: []int64{4, -2, -1}, ZeroThreshold: defaultZeroThreshold,
}, ZeroCount: 0,
}, NegativeSpans: []histogram.Span{{Offset: 0, Length: 3}},
Exemplars: []prompb.Exemplar{ NegativeBuckets: []int64{4, -2, -1},
{Value: 2},
}, },
es: []exemplar.Exemplar{{Value: 2}},
}, },
} }
}, },
@ -843,7 +854,8 @@ func TestPrometheusConverter_addExponentialHistogramDataPoints(t *testing.T) {
t.Run(tt.name, func(t *testing.T) { t.Run(tt.name, func(t *testing.T) {
metric := tt.metric() metric := tt.metric()
converter := NewPrometheusConverter() mockAppender := &mockCombinedAppender{}
converter := NewPrometheusConverter(mockAppender)
namer := otlptranslator.MetricNamer{ namer := otlptranslator.MetricNamer{
WithMetricSuffixes: true, WithMetricSuffixes: true,
} }
@ -857,14 +869,17 @@ func TestPrometheusConverter_addExponentialHistogramDataPoints(t *testing.T) {
ExportCreatedMetric: true, ExportCreatedMetric: true,
PromoteScopeMetadata: tt.promoteScope, PromoteScopeMetadata: tt.promoteScope,
}, },
prompb.MetricMetadata{MetricFamilyName: name}, name,
pmetric.AggregationTemporalityCumulative, pmetric.AggregationTemporalityCumulative,
tt.scope, tt.scope,
metadata.Metadata{},
) )
require.NoError(t, err) require.NoError(t, err)
require.Empty(t, annots) 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) require.Empty(t, converter.conflicts)
}) })
} }
@ -880,7 +895,7 @@ func TestConvertExplicitHistogramBucketsToNHCBLayout(t *testing.T) {
name: "zero offset", name: "zero offset",
buckets: []uint64{4, 3, 2, 1}, buckets: []uint64{4, 3, 2, 1},
wantLayout: expectedBucketLayout{ wantLayout: expectedBucketLayout{
wantSpans: []prompb.BucketSpan{ wantSpans: []histogram.Span{
{ {
Offset: 0, Offset: 0,
Length: 4, Length: 4,
@ -893,7 +908,7 @@ func TestConvertExplicitHistogramBucketsToNHCBLayout(t *testing.T) {
name: "leading empty buckets", name: "leading empty buckets",
buckets: []uint64{0, 0, 1, 1, 2, 3}, buckets: []uint64{0, 0, 1, 1, 2, 3},
wantLayout: expectedBucketLayout{ wantLayout: expectedBucketLayout{
wantSpans: []prompb.BucketSpan{ wantSpans: []histogram.Span{
{ {
Offset: 2, Offset: 2,
Length: 4, Length: 4,
@ -906,7 +921,7 @@ func TestConvertExplicitHistogramBucketsToNHCBLayout(t *testing.T) {
name: "trailing empty buckets", name: "trailing empty buckets",
buckets: []uint64{0, 0, 1, 1, 2, 3, 0, 0}, // TODO: add tests for 3 trailing buckets buckets: []uint64{0, 0, 1, 1, 2, 3, 0, 0}, // TODO: add tests for 3 trailing buckets
wantLayout: expectedBucketLayout{ wantLayout: expectedBucketLayout{
wantSpans: []prompb.BucketSpan{ wantSpans: []histogram.Span{
{ {
Offset: 2, Offset: 2,
Length: 6, Length: 6,
@ -919,7 +934,7 @@ func TestConvertExplicitHistogramBucketsToNHCBLayout(t *testing.T) {
name: "bucket gap of 2", name: "bucket gap of 2",
buckets: []uint64{1, 2, 0, 0, 2}, buckets: []uint64{1, 2, 0, 0, 2},
wantLayout: expectedBucketLayout{ wantLayout: expectedBucketLayout{
wantSpans: []prompb.BucketSpan{ wantSpans: []histogram.Span{
{ {
Offset: 0, Offset: 0,
Length: 5, Length: 5,
@ -932,7 +947,7 @@ func TestConvertExplicitHistogramBucketsToNHCBLayout(t *testing.T) {
name: "bucket gap > 2", name: "bucket gap > 2",
buckets: []uint64{1, 2, 0, 0, 0, 2, 4, 4}, buckets: []uint64{1, 2, 0, 0, 0, 2, 4, 4},
wantLayout: expectedBucketLayout{ wantLayout: expectedBucketLayout{
wantSpans: []prompb.BucketSpan{ wantSpans: []histogram.Span{
{ {
Offset: 0, Offset: 0,
Length: 2, Length: 2,
@ -949,7 +964,7 @@ func TestConvertExplicitHistogramBucketsToNHCBLayout(t *testing.T) {
name: "multiple bucket gaps", name: "multiple bucket gaps",
buckets: []uint64{0, 0, 1, 2, 0, 0, 0, 2, 4, 4, 0, 0}, buckets: []uint64{0, 0, 1, 2, 0, 0, 0, 2, 4, 4, 0, 0},
wantLayout: expectedBucketLayout{ wantLayout: expectedBucketLayout{
wantSpans: []prompb.BucketSpan{ wantSpans: []histogram.Span{
{ {
Offset: 2, Offset: 2,
Length: 2, Length: 2,
@ -1008,7 +1023,7 @@ func TestHistogramToCustomBucketsHistogram(t *testing.T) {
tests := []struct { tests := []struct {
name string name string
hist func() pmetric.HistogramDataPoint hist func() pmetric.HistogramDataPoint
wantNativeHist func() prompb.Histogram wantNativeHist func() *histogram.Histogram
wantErrMessage string wantErrMessage string
}{ }{
{ {
@ -1024,15 +1039,14 @@ func TestHistogramToCustomBucketsHistogram(t *testing.T) {
pt.ExplicitBounds().FromRaw([]float64{0, 1}) pt.ExplicitBounds().FromRaw([]float64{0, 1})
return pt return pt
}, },
wantNativeHist: func() prompb.Histogram { wantNativeHist: func() *histogram.Histogram {
return prompb.Histogram{ return &histogram.Histogram{
Count: &prompb.Histogram_CountInt{CountInt: 2}, Count: 2,
Sum: 10.1, Sum: 10.1,
Schema: -53, Schema: -53,
PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 2}}, PositiveSpans: []histogram.Span{{Offset: 0, Length: 2}},
PositiveDeltas: []int64{1, 0}, PositiveBuckets: []int64{1, 0},
CustomValues: []float64{0, 1}, CustomValues: []float64{0, 1},
Timestamp: 500,
} }
}, },
}, },
@ -1048,14 +1062,13 @@ func TestHistogramToCustomBucketsHistogram(t *testing.T) {
pt.ExplicitBounds().FromRaw([]float64{0, 1}) pt.ExplicitBounds().FromRaw([]float64{0, 1})
return pt return pt
}, },
wantNativeHist: func() prompb.Histogram { wantNativeHist: func() *histogram.Histogram {
return prompb.Histogram{ return &histogram.Histogram{
Count: &prompb.Histogram_CountInt{CountInt: 4}, Count: 4,
Schema: -53, Schema: -53,
PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 2}}, PositiveSpans: []histogram.Span{{Offset: 0, Length: 2}},
PositiveDeltas: []int64{2, 0}, PositiveBuckets: []int64{2, 0},
CustomValues: []float64{0, 1}, CustomValues: []float64{0, 1},
Timestamp: 500,
} }
}, },
}, },
@ -1094,7 +1107,7 @@ func TestPrometheusConverter_addCustomBucketsHistogramDataPoints(t *testing.T) {
metric func() pmetric.Metric metric func() pmetric.Metric
scope scope scope scope
promoteScope bool promoteScope bool
wantSeries func() map[uint64]*prompb.TimeSeries wantSeries func() []combinedHistogram
}{ }{
{ {
name: "histogram data points with same labels and without scope promotion", name: "histogram data points with same labels and without scope promotion",
@ -1123,36 +1136,43 @@ func TestPrometheusConverter_addCustomBucketsHistogramDataPoints(t *testing.T) {
}, },
scope: defaultScope, scope: defaultScope,
promoteScope: false, promoteScope: false,
wantSeries: func() map[uint64]*prompb.TimeSeries { wantSeries: func() []combinedHistogram {
labels := []prompb.Label{ lbls := labels.FromStrings(
{Name: model.MetricNameLabel, Value: "test_hist_to_nhcb"}, model.MetricNameLabel, "test_hist_to_nhcb",
{Name: "attr", Value: "test_attr"}, "attr", "test_attr",
} )
return map[uint64]*prompb.TimeSeries{ return []combinedHistogram{
timeSeriesSignature(labels): { {
Labels: labels, metricFamilyName: "test_hist_to_nhcb",
Histograms: []prompb.Histogram{ ls: lbls,
{ meta: metadata.Metadata{},
Count: &prompb.Histogram_CountInt{CountInt: 3}, t: 0,
Sum: 3, ct: 0,
Schema: -53, h: &histogram.Histogram{
PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 3}}, Count: 3,
PositiveDeltas: []int64{2, -2, 1}, Sum: 3,
CustomValues: []float64{5, 10}, Schema: -53,
}, PositiveSpans: []histogram.Span{{Offset: 0, Length: 3}},
{ PositiveBuckets: []int64{2, -2, 1},
Count: &prompb.Histogram_CountInt{CountInt: 11}, CustomValues: []float64{5, 10},
Sum: 5,
Schema: -53,
PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 3}},
PositiveDeltas: []int64{3, 5, -8},
CustomValues: []float64{0, 1},
},
}, },
Exemplars: []prompb.Exemplar{ es: []exemplar.Exemplar{{Value: 1}},
{Value: 1}, },
{Value: 2}, {
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, scope: defaultScope,
promoteScope: true, promoteScope: true,
wantSeries: func() map[uint64]*prompb.TimeSeries { wantSeries: func() []combinedHistogram {
labels := []prompb.Label{ lbls := labels.FromStrings(
{Name: model.MetricNameLabel, Value: "test_hist_to_nhcb"}, model.MetricNameLabel, "test_hist_to_nhcb",
{Name: "attr", Value: "test_attr"}, "attr", "test_attr",
{Name: "otel_scope_name", Value: defaultScope.name}, "otel_scope_name", defaultScope.name,
{Name: "otel_scope_schema_url", Value: defaultScope.schemaURL}, "otel_scope_schema_url", defaultScope.schemaURL,
{Name: "otel_scope_version", Value: defaultScope.version}, "otel_scope_version", defaultScope.version,
{Name: "otel_scope_attr1", Value: "value1"}, "otel_scope_attr1", "value1",
{Name: "otel_scope_attr2", Value: "value2"}, "otel_scope_attr2", "value2",
} )
return map[uint64]*prompb.TimeSeries{ return []combinedHistogram{
timeSeriesSignature(labels): { {
Labels: labels, metricFamilyName: "test_hist_to_nhcb",
Histograms: []prompb.Histogram{ ls: lbls,
{ meta: metadata.Metadata{},
Count: &prompb.Histogram_CountInt{CountInt: 3}, t: 0,
Sum: 3, ct: 0,
Schema: -53, h: &histogram.Histogram{
PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 3}}, Count: 3,
PositiveDeltas: []int64{2, -2, 1}, Sum: 3,
CustomValues: []float64{5, 10}, Schema: -53,
}, PositiveSpans: []histogram.Span{{Offset: 0, Length: 3}},
{ PositiveBuckets: []int64{2, -2, 1},
Count: &prompb.Histogram_CountInt{CountInt: 11}, CustomValues: []float64{5, 10},
Sum: 5,
Schema: -53,
PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 3}},
PositiveDeltas: []int64{3, 5, -8},
CustomValues: []float64{0, 1},
},
}, },
Exemplars: []prompb.Exemplar{ es: []exemplar.Exemplar{{Value: 1}},
{Value: 1}, },
{Value: 2}, {
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, scope: defaultScope,
promoteScope: false, promoteScope: false,
wantSeries: func() map[uint64]*prompb.TimeSeries { wantSeries: func() []combinedHistogram {
labels := []prompb.Label{ lbls := labels.FromStrings(
{Name: model.MetricNameLabel, Value: "test_hist_to_nhcb"}, model.MetricNameLabel, "test_hist_to_nhcb",
{Name: "attr", Value: "test_attr"}, "attr", "test_attr",
} )
labelsAnother := []prompb.Label{ labelsAnother := labels.FromStrings(
{Name: model.MetricNameLabel, Value: "test_hist_to_nhcb"}, model.MetricNameLabel, "test_hist_to_nhcb",
{Name: "attr", Value: "test_attr_two"}, "attr", "test_attr_two",
} )
return map[uint64]*prompb.TimeSeries{ return []combinedHistogram{
timeSeriesSignature(labels): { {
Labels: labels, metricFamilyName: "test_hist_to_nhcb",
Histograms: []prompb.Histogram{ ls: lbls,
{ meta: metadata.Metadata{},
Count: &prompb.Histogram_CountInt{CountInt: 6}, t: 0,
Sum: 3, ct: 0,
Schema: -53, h: &histogram.Histogram{
PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 2}}, Count: 6,
PositiveDeltas: []int64{4, -2}, Sum: 3,
CustomValues: []float64{0, 1}, Schema: -53,
}, PositiveSpans: []histogram.Span{{Offset: 0, Length: 2}},
}, PositiveBuckets: []int64{4, -2},
Exemplars: []prompb.Exemplar{ CustomValues: []float64{0, 1},
{Value: 1},
}, },
es: []exemplar.Exemplar{{Value: 1}},
}, },
timeSeriesSignature(labelsAnother): { {
Labels: labelsAnother, metricFamilyName: "test_hist_to_nhcb",
Histograms: []prompb.Histogram{ ls: labelsAnother,
{ meta: metadata.Metadata{},
Count: &prompb.Histogram_CountInt{CountInt: 11}, t: 0,
Sum: 5, ct: 0,
Schema: -53, h: &histogram.Histogram{
PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 2}}, Count: 11,
PositiveDeltas: []int64{3, 5}, Sum: 5,
CustomValues: []float64{0, 1}, Schema: -53,
}, PositiveSpans: []histogram.Span{{Offset: 0, Length: 2}},
}, PositiveBuckets: []int64{3, 5},
Exemplars: []prompb.Exemplar{ CustomValues: []float64{0, 1},
{Value: 2},
}, },
es: []exemplar.Exemplar{{Value: 2}},
}, },
} }
}, },
@ -1301,7 +1328,8 @@ func TestPrometheusConverter_addCustomBucketsHistogramDataPoints(t *testing.T) {
t.Run(tt.name, func(t *testing.T) { t.Run(tt.name, func(t *testing.T) {
metric := tt.metric() metric := tt.metric()
converter := NewPrometheusConverter() mockAppender := &mockCombinedAppender{}
converter := NewPrometheusConverter(mockAppender)
namer := otlptranslator.MetricNamer{ namer := otlptranslator.MetricNamer{
WithMetricSuffixes: true, WithMetricSuffixes: true,
} }
@ -1316,15 +1344,18 @@ func TestPrometheusConverter_addCustomBucketsHistogramDataPoints(t *testing.T) {
ConvertHistogramsToNHCB: true, ConvertHistogramsToNHCB: true,
PromoteScopeMetadata: tt.promoteScope, PromoteScopeMetadata: tt.promoteScope,
}, },
prompb.MetricMetadata{MetricFamilyName: name}, name,
pmetric.AggregationTemporalityCumulative, pmetric.AggregationTemporalityCumulative,
tt.scope, tt.scope,
metadata.Metadata{},
) )
require.NoError(t, err) require.NoError(t, err)
require.Empty(t, annots) 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) require.Empty(t, converter.conflicts)
}) })
} }

View File

@ -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
}

View File

@ -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/<namespace>/deployments/<name>"},
{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())
}

View File

@ -21,7 +21,6 @@ import (
"errors" "errors"
"fmt" "fmt"
"math" "math"
"sort"
"time" "time"
"github.com/prometheus/otlptranslator" "github.com/prometheus/otlptranslator"
@ -30,7 +29,8 @@ import (
"go.uber.org/multierr" "go.uber.org/multierr"
"github.com/prometheus/prometheus/config" "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" "github.com/prometheus/prometheus/util/annotations"
) )
@ -59,16 +59,21 @@ type Settings struct {
// PrometheusConverter converts from OTel write format to Prometheus remote write format. // PrometheusConverter converts from OTel write format to Prometheus remote write format.
type PrometheusConverter struct { type PrometheusConverter struct {
unique map[uint64]*prompb.TimeSeries unique map[uint64]labels.Labels
conflicts map[uint64][]*prompb.TimeSeries conflicts map[uint64][]labels.Labels
everyN everyNTimes everyN everyNTimes
metadata []prompb.MetricMetadata scratchBuilder labels.ScratchBuilder
builder *labels.Builder
appender CombinedAppender
} }
func NewPrometheusConverter() *PrometheusConverter { func NewPrometheusConverter(appender CombinedAppender) *PrometheusConverter {
return &PrometheusConverter{ return &PrometheusConverter{
unique: map[uint64]*prompb.TimeSeries{}, unique: map[uint64]labels.Labels{},
conflicts: map[uint64][]*prompb.TimeSeries{}, 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, WithMetricSuffixes: settings.AddMetricSuffixes,
UTF8Allowed: settings.AllowUTF8, UTF8Allowed: settings.AllowUTF8,
} }
unitNamer := otlptranslator.UnitNamer{}
c.everyN = everyNTimes{n: 128} c.everyN = everyNTimes{n: 128}
resourceMetricsSlice := md.ResourceMetrics() resourceMetricsSlice := md.ResourceMetrics()
@ -131,7 +137,6 @@ func (c *PrometheusConverter) FromMetrics(ctx context.Context, md pmetric.Metric
numMetrics += scopeMetricsSlice.At(j).Metrics().Len() numMetrics += scopeMetricsSlice.At(j).Metrics().Len()
} }
} }
c.metadata = make([]prompb.MetricMetadata, 0, numMetrics)
for i := 0; i < resourceMetricsSlice.Len(); i++ { for i := 0; i < resourceMetricsSlice.Len(); i++ {
resourceMetrics := resourceMetricsSlice.At(i) resourceMetrics := resourceMetricsSlice.At(i)
@ -176,13 +181,11 @@ func (c *PrometheusConverter) FromMetrics(ctx context.Context, md pmetric.Metric
errs = multierr.Append(errs, err) errs = multierr.Append(errs, err)
continue continue
} }
metadata := prompb.MetricMetadata{ meta := metadata.Metadata{
Type: otelMetricTypeToPromMetricType(metric), Type: otelMetricTypeToPromMetricType(metric),
MetricFamilyName: promName, Unit: unitNamer.Build(metric.Unit()),
Help: metric.Description(), Help: metric.Description(),
Unit: metric.Unit(),
} }
c.metadata = append(c.metadata, metadata)
// handle individual metrics based on type // handle individual metrics based on type
//exhaustive:enforce //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())) errs = multierr.Append(errs, fmt.Errorf("empty data points. %s is dropped", metric.Name()))
break 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) errs = multierr.Append(errs, err)
if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) { if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) {
return 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())) errs = multierr.Append(errs, fmt.Errorf("empty data points. %s is dropped", metric.Name()))
break 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) errs = multierr.Append(errs, err)
if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) { if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) {
return return
@ -219,7 +222,7 @@ func (c *PrometheusConverter) FromMetrics(ctx context.Context, md pmetric.Metric
} }
if settings.ConvertHistogramsToNHCB { if settings.ConvertHistogramsToNHCB {
ws, err := c.addCustomBucketsHistogramDataPoints( ws, err := c.addCustomBucketsHistogramDataPoints(
ctx, dataPoints, resource, settings, metadata, temporality, scope, ctx, dataPoints, resource, settings, promName, temporality, scope, meta,
) )
annots.Merge(ws) annots.Merge(ws)
if err != nil { if err != nil {
@ -229,7 +232,7 @@ func (c *PrometheusConverter) FromMetrics(ctx context.Context, md pmetric.Metric
} }
} }
} else { } 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) errs = multierr.Append(errs, err)
if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) { if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) {
return return
@ -247,9 +250,10 @@ func (c *PrometheusConverter) FromMetrics(ctx context.Context, md pmetric.Metric
dataPoints, dataPoints,
resource, resource,
settings, settings,
metadata, promName,
temporality, temporality,
scope, scope,
meta,
) )
annots.Merge(ws) annots.Merge(ws)
if err != nil { 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())) errs = multierr.Append(errs, fmt.Errorf("empty data points. %s is dropped", metric.Name()))
break 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) errs = multierr.Append(errs, err)
if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) { if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) {
return return
@ -278,72 +282,16 @@ func (c *PrometheusConverter) FromMetrics(ctx context.Context, md pmetric.Metric
if earliestTimestamp < pcommon.Timestamp(math.MaxUint64) { if earliestTimestamp < pcommon.Timestamp(math.MaxUint64) {
// We have at least one metric sample for this resource. // We have at least one metric sample for this resource.
// Generate a corresponding target_info series. // Generate a corresponding target_info series.
err := addResourceTargetInfo(resource, settings, earliestTimestamp.AsTime(), latestTimestamp.AsTime(), c) if err := c.addResourceTargetInfo(resource, settings, earliestTimestamp.AsTime(), latestTimestamp.AsTime()); err != nil {
if err != nil {
errs = multierr.Append(errs, err) 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 { return
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
} }
func NewPromoteResourceAttributes(otlpCfg config.OTLPConfig) *PromoteResourceAttributes { func NewPromoteResourceAttributes(otlpCfg config.OTLPConfig) *PromoteResourceAttributes {
@ -361,30 +309,43 @@ func NewPromoteResourceAttributes(otlpCfg config.OTLPConfig) *PromoteResourceAtt
} }
} }
// promotedAttributes returns labels for promoted resourceAttributes. // addPromotedAttributes adds labels for promoted resourceAttributes to the builder.
func (s *PromoteResourceAttributes) promotedAttributes(resourceAttributes pcommon.Map) []prompb.Label { func (s *PromoteResourceAttributes) addPromotedAttributes(builder *labels.Builder, resourceAttributes pcommon.Map, allowUTF8 bool) error {
if s == nil { if s == nil {
return nil return nil
} }
var promotedAttrs []prompb.Label labelNamer := otlptranslator.LabelNamer{UTF8Allowed: allowUTF8}
if s.promoteAll { if s.promoteAll {
promotedAttrs = make([]prompb.Label, 0, resourceAttributes.Len()) var err error
resourceAttributes.Range(func(name string, value pcommon.Value) bool { resourceAttributes.Range(func(name string, value pcommon.Value) bool {
if _, exists := s.attrs[name]; !exists { if _, exists := s.attrs[name]; !exists {
promotedAttrs = append(promotedAttrs, prompb.Label{Name: name, Value: value.AsString()}) var normalized string
} normalized, err = labelNamer.Build(name)
return true if err != nil {
}) return false
} else { }
promotedAttrs = make([]prompb.Label, 0, len(s.attrs)) if builder.Get(normalized) == "" {
resourceAttributes.Range(func(name string, value pcommon.Value) bool { builder.Set(normalized, value.AsString())
if _, exists := s.attrs[name]; exists { }
promotedAttrs = append(promotedAttrs, prompb.Label{Name: name, Value: value.AsString()})
} }
return true return true
}) })
return err
} }
sort.Stable(ByLabelName(promotedAttrs)) var err error
return promotedAttrs 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
} }

View File

@ -19,19 +19,20 @@ package prometheusremotewrite
import ( import (
"context" "context"
"fmt" "fmt"
"sort"
"testing" "testing"
"time" "time"
"github.com/prometheus/common/model"
"github.com/prometheus/otlptranslator" "github.com/prometheus/otlptranslator"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
"go.opentelemetry.io/collector/pdata/pcommon" "go.opentelemetry.io/collector/pdata/pcommon"
"go.opentelemetry.io/collector/pdata/pmetric" "go.opentelemetry.io/collector/pdata/pmetric"
"go.opentelemetry.io/collector/pdata/pmetric/pmetricotlp" "go.opentelemetry.io/collector/pdata/pmetric/pmetricotlp"
"github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/model/histogram"
"github.com/prometheus/prometheus/prompb" modelLabels "github.com/prometheus/prometheus/model/labels"
"github.com/prometheus/prometheus/util/testutil" "github.com/prometheus/prometheus/model/metadata"
"github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite/labels"
) )
func TestFromMetrics(t *testing.T) { func TestFromMetrics(t *testing.T) {
@ -77,9 +78,9 @@ func TestFromMetrics(t *testing.T) {
}, },
} { } {
t.Run(tc.name, func(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) payload, wantPromMetrics := createExportRequest(5, 128, 128, 2, 0, tc.settings, tc.temporality)
var expMetadata []prompb.MetricMetadata
seenFamilyNames := map[string]struct{}{} seenFamilyNames := map[string]struct{}{}
for _, wantMetric := range wantPromMetrics { for _, wantMetric := range wantPromMetrics {
if _, exists := seenFamilyNames[wantMetric.familyName]; exists { if _, exists := seenFamilyNames[wantMetric.familyName]; exists {
@ -90,12 +91,6 @@ func TestFromMetrics(t *testing.T) {
} }
seenFamilyNames[wantMetric.familyName] = struct{}{} 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( annots, err := converter.FromMetrics(
@ -106,16 +101,13 @@ func TestFromMetrics(t *testing.T) {
require.NoError(t, err) require.NoError(t, err)
require.Empty(t, annots) require.Empty(t, annots)
testutil.RequireEqual(t, expMetadata, converter.Metadata()) ts := mockAppender.samples
ts := converter.TimeSeries()
require.Len(t, ts, 1536+1) // +1 for the target_info. require.Len(t, ts, 1536+1) // +1 for the target_info.
tgtInfoCount := 0 tgtInfoCount := 0
for _, s := range ts { for _, s := range ts {
b := labels.NewScratchBuilder(2) lbls := s.ls
lbls := s.ToLabels(&b, nil) if lbls.Get(modelLabels.MetricName) == "target_info" {
if lbls.Get(labels.MetricName) == "target_info" {
tgtInfoCount++ tgtInfoCount++
require.Equal(t, "test-namespace/test-service", lbls.Get("job")) require.Equal(t, "test-namespace/test-service", lbls.Get("job"))
require.Equal(t, "id1234", lbls.Get("instance")) require.Equal(t, "id1234", lbls.Get("instance"))
@ -156,7 +148,8 @@ func TestFromMetrics(t *testing.T) {
generateAttributes(h.Attributes(), "series", 1) generateAttributes(h.Attributes(), "series", 1)
converter := NewPrometheusConverter() mockAppender := &mockCombinedAppender{}
converter := NewPrometheusConverter(mockAppender)
annots, err := converter.FromMetrics( annots, err := converter.FromMetrics(
context.Background(), context.Background(),
request.Metrics(), request.Metrics(),
@ -165,24 +158,19 @@ func TestFromMetrics(t *testing.T) {
require.NoError(t, err) require.NoError(t, err)
require.Empty(t, annots) require.Empty(t, annots)
series := converter.TimeSeries()
if convertHistogramsToNHCB { if convertHistogramsToNHCB {
require.Len(t, series[0].Histograms, 1) require.Len(t, mockAppender.histograms, 1)
require.Empty(t, series[0].Samples) require.Empty(t, mockAppender.samples)
} else { } else {
require.Len(t, series, 3) require.Empty(t, mockAppender.histograms)
for i := range series { require.Len(t, mockAppender.samples, 3)
require.Len(t, series[i].Samples, 1)
require.Nil(t, series[i].Histograms)
}
} }
}) })
} }
t.Run("context cancellation", func(t *testing.T) { t.Run("context cancellation", func(t *testing.T) {
settings := Settings{} settings := Settings{}
converter := NewPrometheusConverter() converter := NewPrometheusConverter(&mockCombinedAppender{})
ctx, cancel := context.WithCancel(context.Background()) ctx, cancel := context.WithCancel(context.Background())
// Verify that converter.FromMetrics respects cancellation. // Verify that converter.FromMetrics respects cancellation.
cancel() cancel()
@ -195,7 +183,7 @@ func TestFromMetrics(t *testing.T) {
t.Run("context timeout", func(t *testing.T) { t.Run("context timeout", func(t *testing.T) {
settings := Settings{} settings := Settings{}
converter := NewPrometheusConverter() converter := NewPrometheusConverter(&mockCombinedAppender{})
// Verify that converter.FromMetrics respects timeout. // Verify that converter.FromMetrics respects timeout.
ctx, cancel := context.WithTimeout(context.Background(), 0) ctx, cancel := context.WithTimeout(context.Background(), 0)
t.Cleanup(cancel) t.Cleanup(cancel)
@ -228,7 +216,7 @@ func TestFromMetrics(t *testing.T) {
generateAttributes(h.Attributes(), "series", 10) generateAttributes(h.Attributes(), "series", 10)
} }
converter := NewPrometheusConverter() converter := NewPrometheusConverter(&mockCombinedAppender{})
annots, err := converter.FromMetrics(context.Background(), request.Metrics(), Settings{}) annots, err := converter.FromMetrics(context.Background(), request.Metrics(), Settings{})
require.NoError(t, err) require.NoError(t, err)
require.NotEmpty(t, annots) require.NotEmpty(t, annots)
@ -261,7 +249,7 @@ func TestFromMetrics(t *testing.T) {
generateAttributes(h.Attributes(), "series", 10) generateAttributes(h.Attributes(), "series", 10)
} }
converter := NewPrometheusConverter() converter := NewPrometheusConverter(&mockCombinedAppender{})
annots, err := converter.FromMetrics( annots, err := converter.FromMetrics(
context.Background(), context.Background(),
request.Metrics(), request.Metrics(),
@ -292,7 +280,6 @@ func TestFromMetrics(t *testing.T) {
metrics := rm.ScopeMetrics().AppendEmpty().Metrics() metrics := rm.ScopeMetrics().AppendEmpty().Metrics()
ts := pcommon.NewTimestampFromTime(time.Now()) ts := pcommon.NewTimestampFromTime(time.Now())
var expMetadata []prompb.MetricMetadata
for i := range 3 { for i := range 3 {
m := metrics.AppendEmpty() m := metrics.AppendEmpty()
m.SetEmptyGauge() m.SetEmptyGauge()
@ -308,68 +295,65 @@ func TestFromMetrics(t *testing.T) {
generateAttributes(point.Attributes(), "series", 2) generateAttributes(point.Attributes(), "series", 2)
curTs = curTs.Add(defaultLookbackDelta / 4) 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() mockAppender := &mockCombinedAppender{}
annots, err := converter.FromMetrics( converter := NewPrometheusConverter(mockAppender)
context.Background(), settings := Settings{
request.Metrics(), LookbackDelta: defaultLookbackDelta,
Settings{ }
LookbackDelta: defaultLookbackDelta,
}, annots, err := converter.FromMetrics(context.Background(), request.Metrics(), settings)
)
require.NoError(t, err) require.NoError(t, err)
require.Empty(t, annots) require.Empty(t, annots)
testutil.RequireEqual(t, expMetadata, converter.Metadata()) 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,
timeSeries := converter.TimeSeries() // then one at the latest metric timestamp.
tgtInfoCount := 0 targetInfoLabels := labels.FromStrings(
for _, s := range timeSeries { "__name__", "target_info",
b := labels.NewScratchBuilder(2) "instance", "id1234",
lbls := s.ToLabels(&b, nil) "job", "test-namespace/test-service",
if lbls.Get(labels.MetricName) != "target_info" { "resource_name_1", "value-1",
continue "resource_name_2", "value-2",
} "resource_name_3", "value-3",
"resource_name_4", "value-4",
tgtInfoCount++ "resource_name_5", "value-5",
require.Equal(t, "test-namespace/test-service", lbls.Get("job")) )
require.Equal(t, "id1234", lbls.Get("instance")) targetInfoMeta := metadata.Metadata{
require.False(t, lbls.Has("service_name")) Type: model.MetricTypeGauge,
require.False(t, lbls.Has("service_namespace")) Help: "Target metadata",
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.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) ts := time.Unix(100, 0)
tests := []struct { tests := []struct {
name string name string
allowDelta bool allowDelta bool
convertToNHCB bool convertToNHCB bool
inputSeries []pmetric.Metric inputSeries []pmetric.Metric
expectedSeries []prompb.TimeSeries expectedSamples []combinedSample
expectedError string expectedHistograms []combinedHistogram
expectedError string
}{ }{
{ {
name: "all cumulative when delta not allowed", 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_1", pmetric.AggregationTemporalityCumulative, ts),
createOtelSum("test_metric_2", pmetric.AggregationTemporalityCumulative, ts), createOtelSum("test_metric_2", pmetric.AggregationTemporalityCumulative, ts),
}, },
expectedSeries: []prompb.TimeSeries{ expectedSamples: []combinedSample{
createPromFloatSeries("test_metric_1", ts), createPromFloatSeries("test_metric_1", ts, model.MetricTypeCounter),
createPromFloatSeries("test_metric_2", ts), 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_1", pmetric.AggregationTemporalityDelta, ts),
createOtelSum("test_metric_2", pmetric.AggregationTemporalityDelta, ts), createOtelSum("test_metric_2", pmetric.AggregationTemporalityDelta, ts),
}, },
expectedSeries: []prompb.TimeSeries{ expectedSamples: []combinedSample{
createPromFloatSeries("test_metric_1", ts), createPromFloatSeries("test_metric_1", ts, model.MetricTypeUnknown),
createPromFloatSeries("test_metric_2", ts), 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_1", pmetric.AggregationTemporalityDelta, ts),
createOtelSum("test_metric_2", pmetric.AggregationTemporalityCumulative, ts), createOtelSum("test_metric_2", pmetric.AggregationTemporalityCumulative, ts),
}, },
expectedSeries: []prompb.TimeSeries{ expectedSamples: []combinedSample{
createPromFloatSeries("test_metric_1", ts), createPromFloatSeries("test_metric_1", ts, model.MetricTypeUnknown),
createPromFloatSeries("test_metric_2", ts), 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_1", pmetric.AggregationTemporalityCumulative, ts),
createOtelSum("test_metric_2", pmetric.AggregationTemporalityDelta, ts), createOtelSum("test_metric_2", pmetric.AggregationTemporalityDelta, ts),
}, },
expectedSeries: []prompb.TimeSeries{ expectedSamples: []combinedSample{
createPromFloatSeries("test_metric_1", ts), createPromFloatSeries("test_metric_1", ts, model.MetricTypeCounter),
}, },
expectedError: `invalid temporality and type combination for metric "test_metric_2"`, 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_1", pmetric.AggregationTemporalityCumulative, ts),
createOtelSum("test_metric_2", pmetric.AggregationTemporalityUnspecified, ts), createOtelSum("test_metric_2", pmetric.AggregationTemporalityUnspecified, ts),
}, },
expectedSeries: []prompb.TimeSeries{ expectedSamples: []combinedSample{
createPromFloatSeries("test_metric_1", ts), createPromFloatSeries("test_metric_1", ts, model.MetricTypeCounter),
}, },
expectedError: `invalid temporality and type combination for metric "test_metric_2"`, expectedError: `invalid temporality and type combination for metric "test_metric_2"`,
}, },
@ -450,8 +435,8 @@ func TestTemporality(t *testing.T) {
inputSeries: []pmetric.Metric{ inputSeries: []pmetric.Metric{
createOtelExponentialHistogram("test_histogram", pmetric.AggregationTemporalityCumulative, ts), createOtelExponentialHistogram("test_histogram", pmetric.AggregationTemporalityCumulative, ts),
}, },
expectedSeries: []prompb.TimeSeries{ expectedHistograms: []combinedHistogram{
createPromNativeHistogramSeries("test_histogram", prompb.Histogram_UNKNOWN, ts), 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_1", pmetric.AggregationTemporalityDelta, ts),
createOtelExponentialHistogram("test_histogram_2", pmetric.AggregationTemporalityCumulative, ts), createOtelExponentialHistogram("test_histogram_2", pmetric.AggregationTemporalityCumulative, ts),
}, },
expectedSeries: []prompb.TimeSeries{ expectedHistograms: []combinedHistogram{
createPromNativeHistogramSeries("test_histogram_1", prompb.Histogram_GAUGE, ts), createPromNativeHistogramSeries("test_histogram_1", histogram.GaugeType, ts, model.MetricTypeUnknown),
createPromNativeHistogramSeries("test_histogram_2", prompb.Histogram_UNKNOWN, ts), 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_1", pmetric.AggregationTemporalityDelta, ts),
createOtelExponentialHistogram("test_histogram_2", pmetric.AggregationTemporalityCumulative, ts), createOtelExponentialHistogram("test_histogram_2", pmetric.AggregationTemporalityCumulative, ts),
}, },
expectedSeries: []prompb.TimeSeries{ expectedHistograms: []combinedHistogram{
createPromNativeHistogramSeries("test_histogram_2", prompb.Histogram_UNKNOWN, ts), createPromNativeHistogramSeries("test_histogram_2", histogram.UnknownCounterReset, ts, model.MetricTypeHistogram),
}, },
expectedError: `invalid temporality and type combination for metric "test_histogram_1"`, expectedError: `invalid temporality and type combination for metric "test_histogram_1"`,
}, },
@ -485,8 +470,8 @@ func TestTemporality(t *testing.T) {
inputSeries: []pmetric.Metric{ inputSeries: []pmetric.Metric{
createOtelExplicitHistogram("test_histogram", pmetric.AggregationTemporalityCumulative, ts), createOtelExplicitHistogram("test_histogram", pmetric.AggregationTemporalityCumulative, ts),
}, },
expectedSeries: []prompb.TimeSeries{ expectedHistograms: []combinedHistogram{
createPromNHCBSeries("test_histogram", prompb.Histogram_UNKNOWN, ts), 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_1", pmetric.AggregationTemporalityDelta, ts),
createOtelExplicitHistogram("test_histogram_2", pmetric.AggregationTemporalityCumulative, ts), createOtelExplicitHistogram("test_histogram_2", pmetric.AggregationTemporalityCumulative, ts),
}, },
expectedSeries: []prompb.TimeSeries{ expectedHistograms: []combinedHistogram{
createPromNHCBSeries("test_histogram_1", prompb.Histogram_GAUGE, ts), createPromNHCBSeries("test_histogram_1", histogram.GaugeType, ts, model.MetricTypeUnknown),
createPromNHCBSeries("test_histogram_2", prompb.Histogram_UNKNOWN, ts), 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_1", pmetric.AggregationTemporalityDelta, ts),
createOtelExplicitHistogram("test_histogram_2", pmetric.AggregationTemporalityCumulative, ts), createOtelExplicitHistogram("test_histogram_2", pmetric.AggregationTemporalityCumulative, ts),
}, },
expectedSeries: []prompb.TimeSeries{ expectedHistograms: []combinedHistogram{
createPromNHCBSeries("test_histogram_2", prompb.Histogram_UNKNOWN, ts), createPromNHCBSeries("test_histogram_2", histogram.UnknownCounterReset, ts, model.MetricTypeHistogram),
}, },
expectedError: `invalid temporality and type combination for metric "test_histogram_1"`, 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_1", pmetric.AggregationTemporalityDelta, ts),
createOtelExplicitHistogram("test_histogram_2", pmetric.AggregationTemporalityCumulative, ts), createOtelExplicitHistogram("test_histogram_2", pmetric.AggregationTemporalityCumulative, ts),
}, },
expectedSeries: createPromClassicHistogramSeries("test_histogram_2", ts), expectedSamples: createPromClassicHistogramSeries("test_histogram_2", ts, model.MetricTypeHistogram),
expectedError: `invalid temporality and type combination for metric "test_histogram_1"`, expectedError: `invalid temporality and type combination for metric "test_histogram_1"`,
}, },
{ {
name: "delta histogram with buckets and convertToNHCB=false when allowed", 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_1", pmetric.AggregationTemporalityDelta, ts),
createOtelExplicitHistogram("test_histogram_2", pmetric.AggregationTemporalityCumulative, ts), createOtelExplicitHistogram("test_histogram_2", pmetric.AggregationTemporalityCumulative, ts),
}, },
expectedSeries: append( expectedSamples: append(
createPromClassicHistogramSeries("test_histogram_1", ts), createPromClassicHistogramSeries("test_histogram_1", ts, model.MetricTypeUnknown),
createPromClassicHistogramSeries("test_histogram_2", ts)..., createPromClassicHistogramSeries("test_histogram_2", ts, model.MetricTypeHistogram)...,
), ),
}, },
{ {
@ -544,15 +529,15 @@ func TestTemporality(t *testing.T) {
inputSeries: []pmetric.Metric{ inputSeries: []pmetric.Metric{
createOtelSummary("test_summary_1", ts), createOtelSummary("test_summary_1", ts),
}, },
expectedSeries: createPromSummarySeries("test_summary_1", ts), expectedSamples: createPromSummarySeries("test_summary_1", ts),
}, },
{ {
name: "gauge does not have temporality", name: "gauge does not have temporality",
inputSeries: []pmetric.Metric{ inputSeries: []pmetric.Metric{
createOtelGauge("test_gauge_1", ts), createOtelGauge("test_gauge_1", ts),
}, },
expectedSeries: []prompb.TimeSeries{ expectedSamples: []combinedSample{
createPromFloatSeries("test_gauge_1", ts), createPromFloatSeries("test_gauge_1", ts, model.MetricTypeGauge),
}, },
}, },
{ {
@ -560,8 +545,7 @@ func TestTemporality(t *testing.T) {
inputSeries: []pmetric.Metric{ inputSeries: []pmetric.Metric{
createOtelEmptyType("test_empty"), 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()) s.CopyTo(sm.Metrics().AppendEmpty())
} }
c := NewPrometheusConverter() mockAppender := &mockCombinedAppender{}
c := NewPrometheusConverter(mockAppender)
settings := Settings{ settings := Settings{
AllowDeltaTemporality: tc.allowDelta, AllowDeltaTemporality: tc.allowDelta,
ConvertHistogramsToNHCB: tc.convertToNHCB, ConvertHistogramsToNHCB: tc.convertToNHCB,
@ -589,10 +574,9 @@ func TestTemporality(t *testing.T) {
require.NoError(t, err) require.NoError(t, err)
} }
series := c.TimeSeries()
// Sort series to make the test deterministic. // 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) m.SetName(name)
sum := m.SetEmptySum() sum := m.SetEmptySum()
sum.SetAggregationTemporality(temporality) sum.SetAggregationTemporality(temporality)
sum.SetIsMonotonic(true)
dp := sum.DataPoints().AppendEmpty() dp := sum.DataPoints().AppendEmpty()
dp.SetDoubleValue(5) dp.SetDoubleValue(5)
dp.SetTimestamp(pcommon.NewTimestampFromTime(ts)) dp.SetTimestamp(pcommon.NewTimestampFromTime(ts))
@ -610,16 +595,15 @@ func createOtelSum(name string, temporality pmetric.AggregationTemporality, ts t
return m return m
} }
func createPromFloatSeries(name string, ts time.Time) prompb.TimeSeries { func createPromFloatSeries(name string, ts time.Time, typ model.MetricType) combinedSample {
return prompb.TimeSeries{ return combinedSample{
Labels: []prompb.Label{ metricFamilyName: name,
{Name: "__name__", Value: name}, ls: labels.FromStrings("__name__", name, "test_label", "test_value"),
{Name: "test_label", Value: "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 return m
} }
func createPromNativeHistogramSeries(name string, hint prompb.Histogram_ResetHint, ts time.Time) prompb.TimeSeries { func createPromNativeHistogramSeries(name string, hint histogram.CounterResetHint, ts time.Time, typ model.MetricType) combinedHistogram {
return prompb.TimeSeries{ return combinedHistogram{
Labels: []prompb.Label{ metricFamilyName: name,
{Name: "__name__", Value: name}, ls: labels.FromStrings("__name__", name, "test_label", "test_value"),
{Name: "test_label", Value: "test_value"}, t: ts.UnixMilli(),
meta: metadata.Metadata{
Type: typ,
}, },
Histograms: []prompb.Histogram{ h: &histogram.Histogram{
{ Count: 1,
Count: &prompb.Histogram_CountInt{CountInt: 1}, Sum: 5,
Sum: 5, Schema: 0,
Schema: 0, ZeroThreshold: 1e-128,
ZeroThreshold: 1e-128, ZeroCount: 0,
ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: 0}, CounterResetHint: hint,
Timestamp: ts.UnixMilli(),
ResetHint: hint,
},
}, },
} }
} }
@ -685,72 +668,77 @@ func createOtelExplicitHistogram(name string, temporality pmetric.AggregationTem
return m return m
} }
func createPromNHCBSeries(name string, hint prompb.Histogram_ResetHint, ts time.Time) prompb.TimeSeries { func createPromNHCBSeries(name string, hint histogram.CounterResetHint, ts time.Time, typ model.MetricType) combinedHistogram {
return prompb.TimeSeries{ return combinedHistogram{
Labels: []prompb.Label{ metricFamilyName: name,
{Name: "__name__", Value: name}, ls: labels.FromStrings("__name__", name, "test_label", "test_value"),
{Name: "test_label", Value: "test_value"}, meta: metadata.Metadata{
Type: typ,
}, },
Histograms: []prompb.Histogram{ t: ts.UnixMilli(),
{ h: &histogram.Histogram{
Count: &prompb.Histogram_CountInt{CountInt: 20}, Count: 20,
Sum: 30, Sum: 30,
Schema: -53, Schema: -53,
ZeroThreshold: 0, ZeroThreshold: 0,
ZeroCount: nil, PositiveSpans: []histogram.Span{
PositiveSpans: []prompb.BucketSpan{ {
{ Length: 3,
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 { func createPromClassicHistogramSeries(name string, ts time.Time, typ model.MetricType) []combinedSample {
return []prompb.TimeSeries{ return []combinedSample{
{ {
Labels: []prompb.Label{ metricFamilyName: name,
{Name: "__name__", Value: name + "_bucket"}, ls: labels.FromStrings("__name__", name+"_sum", "test_label", "test_value"),
{Name: "le", Value: "1"}, t: ts.UnixMilli(),
{Name: "test_label", Value: "test_value"}, v: 30,
meta: metadata.Metadata{
Type: typ,
}, },
Samples: []prompb.Sample{{Value: 10, Timestamp: ts.UnixMilli()}},
}, },
{ {
Labels: []prompb.Label{ metricFamilyName: name,
{Name: "__name__", Value: name + "_bucket"}, ls: labels.FromStrings("__name__", name+"_count", "test_label", "test_value"),
{Name: "le", Value: "2"}, t: ts.UnixMilli(),
{Name: "test_label", Value: "test_value"}, v: 20,
meta: metadata.Metadata{
Type: typ,
}, },
Samples: []prompb.Sample{{Value: 20, Timestamp: ts.UnixMilli()}},
}, },
{ {
Labels: []prompb.Label{ metricFamilyName: name,
{Name: "__name__", Value: name + "_bucket"}, ls: labels.FromStrings("__name__", name+"_bucket", "le", "1", "test_label", "test_value"),
{Name: "le", Value: "+Inf"}, t: ts.UnixMilli(),
{Name: "test_label", Value: "test_value"}, v: 10,
meta: metadata.Metadata{
Type: typ,
}, },
Samples: []prompb.Sample{{Value: 20, Timestamp: ts.UnixMilli()}},
}, },
{ {
Labels: []prompb.Label{ metricFamilyName: name,
{Name: "__name__", Value: name + "_count"}, ls: labels.FromStrings("__name__", name+"_bucket", "le", "2", "test_label", "test_value"),
{Name: "test_label", Value: "test_value"}, t: ts.UnixMilli(),
v: 20,
meta: metadata.Metadata{
Type: typ,
}, },
Samples: []prompb.Sample{{Value: 20, Timestamp: ts.UnixMilli()}},
}, },
{ {
Labels: []prompb.Label{ metricFamilyName: name,
{Name: "__name__", Value: name + "_sum"}, ls: labels.FromStrings("__name__", name+"_bucket", "le", "+Inf", "test_label", "test_value"),
{Name: "test_label", Value: "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 return m
} }
func createPromSummarySeries(name string, ts time.Time) []prompb.TimeSeries { func createPromSummarySeries(name string, ts time.Time) []combinedSample {
return []prompb.TimeSeries{ return []combinedSample{
{ {
Labels: []prompb.Label{ metricFamilyName: name,
{Name: "__name__", Value: name + "_sum"}, ls: labels.FromStrings("__name__", name+"_sum", "test_label", "test_value"),
{Name: "test_label", Value: "test_value"}, t: ts.UnixMilli(),
v: 18,
meta: metadata.Metadata{
Type: model.MetricTypeSummary,
}, },
Samples: []prompb.Sample{{
Value: 18,
Timestamp: ts.UnixMilli(),
}},
}, },
{ {
Labels: []prompb.Label{ metricFamilyName: name,
{Name: "__name__", Value: name + "_count"}, ls: labels.FromStrings("__name__", name+"_count", "test_label", "test_value"),
{Name: "test_label", Value: "test_value"}, t: ts.UnixMilli(),
v: 9,
meta: metadata.Metadata{
Type: model.MetricTypeSummary,
}, },
Samples: []prompb.Sample{{
Value: 9,
Timestamp: ts.UnixMilli(),
}},
}, },
{ {
Labels: []prompb.Label{ metricFamilyName: name,
{Name: "__name__", Value: name}, ls: labels.FromStrings("__name__", name, "quantile", "0.5", "test_label", "test_value"),
{Name: "quantile", Value: "0.5"}, t: ts.UnixMilli(),
{Name: "test_label", Value: "test_value"}, 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 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) { func TestTranslatorMetricFromOtelMetric(t *testing.T) {
tests := []struct { tests := []struct {
name string name string
@ -968,7 +938,7 @@ func BenchmarkPrometheusConverter_FromMetrics(b *testing.B) {
b.Run(fmt.Sprintf("histogram count: %v", histogramCount), func(b *testing.B) { b.Run(fmt.Sprintf("histogram count: %v", histogramCount), func(b *testing.B) {
nonHistogramCounts := []int{0, 1000} nonHistogramCounts := []int{0, 1000}
if resourceAttributeCount == 0 && histogramCount == 0 { if histogramCount == 0 {
// Don't bother running a scenario where we'll generate no series. // Don't bother running a scenario where we'll generate no series.
nonHistogramCounts = []int{1000} nonHistogramCounts = []int{1000}
} }
@ -992,12 +962,12 @@ func BenchmarkPrometheusConverter_FromMetrics(b *testing.B) {
b.ResetTimer() b.ResetTimer()
for range b.N { for range b.N {
converter := NewPrometheusConverter() mockAppender := &mockCombinedAppender{}
converter := NewPrometheusConverter(mockAppender)
annots, err := converter.FromMetrics(context.Background(), payload.Metrics(), settings) annots, err := converter.FromMetrics(context.Background(), payload.Metrics(), settings)
require.NoError(b, err) require.NoError(b, err)
require.Empty(b, annots) require.Empty(b, annots)
require.NotNil(b, converter.TimeSeries()) require.Positive(b, len(mockAppender.samples)+len(mockAppender.histograms))
require.NotNil(b, converter.Metadata())
} }
}) })
} }
@ -1014,7 +984,7 @@ func BenchmarkPrometheusConverter_FromMetrics(b *testing.B) {
type wantPrometheusMetric struct { type wantPrometheusMetric struct {
name string name string
familyName string familyName string
metricType prompb.MetricMetadata_MetricType metricType model.MetricType
description string description string
unit string unit string
} }
@ -1061,11 +1031,11 @@ func createExportRequest(resourceAttributeCount, histogramCount, nonHistogramCou
generateAttributes(h.Attributes(), "series", labelsPerMetric) generateAttributes(h.Attributes(), "series", labelsPerMetric)
generateExemplars(h.Exemplars(), exemplarsPerSeries, ts) generateExemplars(h.Exemplars(), exemplarsPerSeries, ts)
metricType := prompb.MetricMetadata_HISTOGRAM metricType := model.MetricTypeHistogram
if temporality != pmetric.AggregationTemporalityCumulative { if temporality != pmetric.AggregationTemporalityCumulative {
// We're in an early phase of implementing delta support (proposal: https://github.com/prometheus/proposals/pull/48/) // 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. // 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{ wantPromMetrics = append(wantPromMetrics, wantPrometheusMetric{
name: fmt.Sprintf("histogram_%d%s_bucket", i, suffix), name: fmt.Sprintf("histogram_%d%s_bucket", i, suffix),
@ -1103,11 +1073,11 @@ func createExportRequest(resourceAttributeCount, histogramCount, nonHistogramCou
generateAttributes(point.Attributes(), "series", labelsPerMetric) generateAttributes(point.Attributes(), "series", labelsPerMetric)
generateExemplars(point.Exemplars(), exemplarsPerSeries, ts) generateExemplars(point.Exemplars(), exemplarsPerSeries, ts)
metricType := prompb.MetricMetadata_GAUGE metricType := model.MetricTypeGauge
if temporality != pmetric.AggregationTemporalityCumulative { if temporality != pmetric.AggregationTemporalityCumulative {
// We're in an early phase of implementing delta support (proposal: https://github.com/prometheus/proposals/pull/48/) // 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. // 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{ wantPromMetrics = append(wantPromMetrics, wantPrometheusMetric{
name: fmt.Sprintf("non_monotonic_sum_%d%s", i, suffix), name: fmt.Sprintf("non_monotonic_sum_%d%s", i, suffix),
@ -1137,11 +1107,11 @@ func createExportRequest(resourceAttributeCount, histogramCount, nonHistogramCou
counterSuffix = suffix + "_total" counterSuffix = suffix + "_total"
} }
metricType := prompb.MetricMetadata_COUNTER metricType := model.MetricTypeCounter
if temporality != pmetric.AggregationTemporalityCumulative { if temporality != pmetric.AggregationTemporalityCumulative {
// We're in an early phase of implementing delta support (proposal: https://github.com/prometheus/proposals/pull/48/) // 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. // 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{ wantPromMetrics = append(wantPromMetrics, wantPrometheusMetric{
name: fmt.Sprintf("monotonic_sum_%d%s", i, counterSuffix), name: fmt.Sprintf("monotonic_sum_%d%s", i, counterSuffix),
@ -1167,7 +1137,7 @@ func createExportRequest(resourceAttributeCount, histogramCount, nonHistogramCou
wantPromMetrics = append(wantPromMetrics, wantPrometheusMetric{ wantPromMetrics = append(wantPromMetrics, wantPrometheusMetric{
name: fmt.Sprintf("gauge_%d%s", i, suffix), name: fmt.Sprintf("gauge_%d%s", i, suffix),
familyName: fmt.Sprintf("gauge_%d%s", i, suffix), familyName: fmt.Sprintf("gauge_%d%s", i, suffix),
metricType: prompb.MetricMetadata_GAUGE, metricType: model.MetricTypeGauge,
unit: "unit", unit: "unit",
description: "gauge", description: "gauge",
}) })

View File

@ -24,12 +24,12 @@ import (
"go.opentelemetry.io/collector/pdata/pcommon" "go.opentelemetry.io/collector/pdata/pcommon"
"go.opentelemetry.io/collector/pdata/pmetric" "go.opentelemetry.io/collector/pdata/pmetric"
"github.com/prometheus/prometheus/model/metadata"
"github.com/prometheus/prometheus/model/value" "github.com/prometheus/prometheus/model/value"
"github.com/prometheus/prometheus/prompb"
) )
func (c *PrometheusConverter) addGaugeNumberDataPoints(ctx context.Context, dataPoints pmetric.NumberDataPointSlice, 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 { ) error {
for x := 0; x < dataPoints.Len(); x++ { for x := 0; x < dataPoints.Len(); x++ {
if err := c.everyN.checkContext(ctx); err != nil { if err := c.everyN.checkContext(ctx); err != nil {
@ -37,42 +37,42 @@ func (c *PrometheusConverter) addGaugeNumberDataPoints(ctx context.Context, data
} }
pt := dataPoints.At(x) pt := dataPoints.At(x)
labels, err := createAttributes( labels, err := c.createAttributes(
resource, resource,
pt.Attributes(), pt.Attributes(),
scope, scope,
settings, settings,
nil, nil,
true, true,
metadata, meta,
model.MetricNameLabel, model.MetricNameLabel,
metadata.MetricFamilyName, name,
) )
if err != nil { if err != nil {
return err return err
} }
sample := &prompb.Sample{ var val float64
// convert ns to ms
Timestamp: convertTimeStamp(pt.Timestamp()),
}
switch pt.ValueType() { switch pt.ValueType() {
case pmetric.NumberDataPointValueTypeInt: case pmetric.NumberDataPointValueTypeInt:
sample.Value = float64(pt.IntValue()) val = float64(pt.IntValue())
case pmetric.NumberDataPointValueTypeDouble: case pmetric.NumberDataPointValueTypeDouble:
sample.Value = pt.DoubleValue() val = pt.DoubleValue()
} }
if pt.Flags().NoRecordedValue() { 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 return nil
} }
func (c *PrometheusConverter) addSumNumberDataPoints(ctx context.Context, dataPoints pmetric.NumberDataPointSlice, 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 { ) error {
for x := 0; x < dataPoints.Len(); x++ { for x := 0; x < dataPoints.Len(); x++ {
if err := c.everyN.checkContext(ctx); err != nil { if err := c.everyN.checkContext(ctx); err != nil {
@ -80,59 +80,51 @@ func (c *PrometheusConverter) addSumNumberDataPoints(ctx context.Context, dataPo
} }
pt := dataPoints.At(x) pt := dataPoints.At(x)
lbls, err := createAttributes( lbls, err := c.createAttributes(
resource, resource,
pt.Attributes(), pt.Attributes(),
scope, scope,
settings, settings,
nil, nil,
true, true,
metadata, meta,
model.MetricNameLabel, 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 { if err != nil {
return err return err
} }
sample := &prompb.Sample{ if err := c.appender.AppendSample(name, lbls, meta, ts, ct, val, exemplars); err != nil {
// convert ns to ms return err
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...)
} }
// add created time series if needed // add created time series if needed
if settings.ExportCreatedMetric && metric.Sum().IsMonotonic() { if settings.ExportCreatedMetric && metric.Sum().IsMonotonic() && pt.StartTimestamp() != 0 {
startTimestamp := pt.StartTimestamp() c.builder.Reset(lbls)
if startTimestamp == 0 { // Add created suffix to the metric name for CT series.
return nil c.builder.Set(model.MetricNameLabel, name+createdSuffix)
} ls := c.builder.Labels()
if c.timeSeriesIsNew(ls) {
createdLabels := make([]prompb.Label, len(lbls)) if err := c.appender.AppendSample(name, ls, meta, ts, 0, float64(ct), nil); err != nil {
copy(createdLabels, lbls) return err
for i, l := range createdLabels {
if l.Name == model.MetricNameLabel {
createdLabels[i].Value = metadata.MetricFamilyName + createdSuffix
break
} }
} }
c.addTimeSeriesIfNeeded(createdLabels, startTimestamp, pt.Timestamp())
} }
} }

View File

@ -26,7 +26,9 @@ import (
"go.opentelemetry.io/collector/pdata/pcommon" "go.opentelemetry.io/collector/pdata/pcommon"
"go.opentelemetry.io/collector/pdata/pmetric" "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) { func TestPrometheusConverter_addGaugeNumberDataPoints(t *testing.T) {
@ -47,7 +49,7 @@ func TestPrometheusConverter_addGaugeNumberDataPoints(t *testing.T) {
metric func() pmetric.Metric metric func() pmetric.Metric
scope scope scope scope
promoteScope bool promoteScope bool
want func() map[uint64]*prompb.TimeSeries want func() []combinedSample
}{ }{
{ {
name: "gauge without scope promotion", name: "gauge without scope promotion",
@ -60,19 +62,17 @@ func TestPrometheusConverter_addGaugeNumberDataPoints(t *testing.T) {
}, },
scope: defaultScope, scope: defaultScope,
promoteScope: false, promoteScope: false,
want: func() map[uint64]*prompb.TimeSeries { want: func() []combinedSample {
labels := []prompb.Label{ lbls := labels.FromStrings(
{Name: model.MetricNameLabel, Value: "test"}, model.MetricNameLabel, "test",
} )
return map[uint64]*prompb.TimeSeries{ return []combinedSample{
timeSeriesSignature(labels): { {
Labels: labels, metricFamilyName: "test",
Samples: []prompb.Sample{ ls: lbls,
{ meta: metadata.Metadata{},
Value: 1, t: convertTimeStamp(pcommon.Timestamp(ts)),
Timestamp: convertTimeStamp(pcommon.Timestamp(ts)), v: 1,
},
},
}, },
} }
}, },
@ -88,24 +88,22 @@ func TestPrometheusConverter_addGaugeNumberDataPoints(t *testing.T) {
}, },
scope: defaultScope, scope: defaultScope,
promoteScope: true, promoteScope: true,
want: func() map[uint64]*prompb.TimeSeries { want: func() []combinedSample {
labels := []prompb.Label{ lbls := labels.FromStrings(
{Name: model.MetricNameLabel, Value: "test"}, model.MetricNameLabel, "test",
{Name: "otel_scope_name", Value: defaultScope.name}, "otel_scope_name", defaultScope.name,
{Name: "otel_scope_schema_url", Value: defaultScope.schemaURL}, "otel_scope_schema_url", defaultScope.schemaURL,
{Name: "otel_scope_version", Value: defaultScope.version}, "otel_scope_version", defaultScope.version,
{Name: "otel_scope_attr1", Value: "value1"}, "otel_scope_attr1", "value1",
{Name: "otel_scope_attr2", Value: "value2"}, "otel_scope_attr2", "value2",
} )
return map[uint64]*prompb.TimeSeries{ return []combinedSample{
timeSeriesSignature(labels): { {
Labels: labels, metricFamilyName: "test",
Samples: []prompb.Sample{ ls: lbls,
{ meta: metadata.Metadata{},
Value: 1, t: convertTimeStamp(pcommon.Timestamp(ts)),
Timestamp: convertTimeStamp(pcommon.Timestamp(ts)), v: 1,
},
},
}, },
} }
}, },
@ -114,7 +112,8 @@ func TestPrometheusConverter_addGaugeNumberDataPoints(t *testing.T) {
for _, tt := range tests { for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) { t.Run(tt.name, func(t *testing.T) {
metric := tt.metric() metric := tt.metric()
converter := NewPrometheusConverter() mockAppender := &mockCombinedAppender{}
converter := NewPrometheusConverter(mockAppender)
converter.addGaugeNumberDataPoints( converter.addGaugeNumberDataPoints(
context.Background(), context.Background(),
@ -124,11 +123,13 @@ func TestPrometheusConverter_addGaugeNumberDataPoints(t *testing.T) {
ExportCreatedMetric: true, ExportCreatedMetric: true,
PromoteScopeMetadata: tt.promoteScope, PromoteScopeMetadata: tt.promoteScope,
}, },
prompb.MetricMetadata{MetricFamilyName: metric.Name()}, metric.Name(),
tt.scope, 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) require.Empty(t, converter.conflicts)
}) })
} }
@ -152,7 +153,7 @@ func TestPrometheusConverter_addSumNumberDataPoints(t *testing.T) {
metric func() pmetric.Metric metric func() pmetric.Metric
scope scope scope scope
promoteScope bool promoteScope bool
want func() map[uint64]*prompb.TimeSeries want func() []combinedSample
}{ }{
{ {
name: "sum without scope promotion", name: "sum without scope promotion",
@ -166,19 +167,17 @@ func TestPrometheusConverter_addSumNumberDataPoints(t *testing.T) {
}, },
scope: defaultScope, scope: defaultScope,
promoteScope: false, promoteScope: false,
want: func() map[uint64]*prompb.TimeSeries { want: func() []combinedSample {
labels := []prompb.Label{ lbls := labels.FromStrings(
{Name: model.MetricNameLabel, Value: "test"}, model.MetricNameLabel, "test",
} )
return map[uint64]*prompb.TimeSeries{ return []combinedSample{
timeSeriesSignature(labels): { {
Labels: labels, metricFamilyName: "test",
Samples: []prompb.Sample{ ls: lbls,
{ meta: metadata.Metadata{},
Value: 1, t: convertTimeStamp(ts),
Timestamp: convertTimeStamp(ts), v: 1,
},
},
}, },
} }
}, },
@ -195,24 +194,22 @@ func TestPrometheusConverter_addSumNumberDataPoints(t *testing.T) {
}, },
scope: defaultScope, scope: defaultScope,
promoteScope: true, promoteScope: true,
want: func() map[uint64]*prompb.TimeSeries { want: func() []combinedSample {
labels := []prompb.Label{ lbls := labels.FromStrings(
{Name: model.MetricNameLabel, Value: "test"}, model.MetricNameLabel, "test",
{Name: "otel_scope_name", Value: defaultScope.name}, "otel_scope_name", defaultScope.name,
{Name: "otel_scope_schema_url", Value: defaultScope.schemaURL}, "otel_scope_schema_url", defaultScope.schemaURL,
{Name: "otel_scope_version", Value: defaultScope.version}, "otel_scope_version", defaultScope.version,
{Name: "otel_scope_attr1", Value: "value1"}, "otel_scope_attr1", "value1",
{Name: "otel_scope_attr2", Value: "value2"}, "otel_scope_attr2", "value2",
} )
return map[uint64]*prompb.TimeSeries{ return []combinedSample{
timeSeriesSignature(labels): { {
Labels: labels, metricFamilyName: "test",
Samples: []prompb.Sample{ ls: lbls,
{ meta: metadata.Metadata{},
Value: 1, t: convertTimeStamp(ts),
Timestamp: convertTimeStamp(ts), v: 1,
},
},
}, },
} }
}, },
@ -231,18 +228,18 @@ func TestPrometheusConverter_addSumNumberDataPoints(t *testing.T) {
}, },
scope: defaultScope, scope: defaultScope,
promoteScope: false, promoteScope: false,
want: func() map[uint64]*prompb.TimeSeries { want: func() []combinedSample {
labels := []prompb.Label{ lbls := labels.FromStrings(
{Name: model.MetricNameLabel, Value: "test"}, model.MetricNameLabel, "test",
} )
return map[uint64]*prompb.TimeSeries{ return []combinedSample{
timeSeriesSignature(labels): { {
Labels: labels, metricFamilyName: "test",
Samples: []prompb.Sample{{ ls: lbls,
Value: 1, meta: metadata.Metadata{},
Timestamp: convertTimeStamp(ts), t: convertTimeStamp(ts),
}}, v: 1,
Exemplars: []prompb.Exemplar{ es: []exemplar.Exemplar{
{Value: 2}, {Value: 2},
}, },
}, },
@ -266,25 +263,28 @@ func TestPrometheusConverter_addSumNumberDataPoints(t *testing.T) {
}, },
scope: defaultScope, scope: defaultScope,
promoteScope: false, promoteScope: false,
want: func() map[uint64]*prompb.TimeSeries { want: func() []combinedSample {
labels := []prompb.Label{ lbls := labels.FromStrings(
{Name: model.MetricNameLabel, Value: "test_sum"}, model.MetricNameLabel, "test_sum",
} )
createdLabels := []prompb.Label{ createdLabels := labels.FromStrings(
{Name: model.MetricNameLabel, Value: "test_sum" + createdSuffix}, model.MetricNameLabel, "test_sum"+createdSuffix,
} )
return map[uint64]*prompb.TimeSeries{ return []combinedSample{
timeSeriesSignature(labels): { {
Labels: labels, metricFamilyName: "test_sum",
Samples: []prompb.Sample{ ls: lbls,
{Value: 1, Timestamp: convertTimeStamp(ts)}, meta: metadata.Metadata{},
}, t: convertTimeStamp(ts),
ct: convertTimeStamp(ts),
v: 1,
}, },
timeSeriesSignature(createdLabels): { {
Labels: createdLabels, metricFamilyName: "test_sum",
Samples: []prompb.Sample{ ls: createdLabels,
{Value: float64(convertTimeStamp(ts)), Timestamp: convertTimeStamp(ts)}, meta: metadata.Metadata{},
}, t: convertTimeStamp(ts),
v: float64(convertTimeStamp(ts)),
}, },
} }
}, },
@ -304,16 +304,17 @@ func TestPrometheusConverter_addSumNumberDataPoints(t *testing.T) {
}, },
scope: defaultScope, scope: defaultScope,
promoteScope: false, promoteScope: false,
want: func() map[uint64]*prompb.TimeSeries { want: func() []combinedSample {
labels := []prompb.Label{ lbls := labels.FromStrings(
{Name: model.MetricNameLabel, Value: "test_sum"}, model.MetricNameLabel, "test_sum",
} )
return map[uint64]*prompb.TimeSeries{ return []combinedSample{
timeSeriesSignature(labels): { {
Labels: labels, metricFamilyName: "test_sum",
Samples: []prompb.Sample{ ls: lbls,
{Value: 0, Timestamp: convertTimeStamp(ts)}, meta: metadata.Metadata{},
}, t: convertTimeStamp(ts),
v: 0,
}, },
} }
}, },
@ -333,16 +334,17 @@ func TestPrometheusConverter_addSumNumberDataPoints(t *testing.T) {
}, },
scope: defaultScope, scope: defaultScope,
promoteScope: false, promoteScope: false,
want: func() map[uint64]*prompb.TimeSeries { want: func() []combinedSample {
labels := []prompb.Label{ lbls := labels.FromStrings(
{Name: model.MetricNameLabel, Value: "test_sum"}, model.MetricNameLabel, "test_sum",
} )
return map[uint64]*prompb.TimeSeries{ return []combinedSample{
timeSeriesSignature(labels): { {
Labels: labels, metricFamilyName: "test_sum",
Samples: []prompb.Sample{ ls: lbls,
{Value: 0, Timestamp: convertTimeStamp(ts)}, meta: metadata.Metadata{},
}, t: convertTimeStamp(ts),
v: 0,
}, },
} }
}, },
@ -351,7 +353,8 @@ func TestPrometheusConverter_addSumNumberDataPoints(t *testing.T) {
for _, tt := range tests { for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) { t.Run(tt.name, func(t *testing.T) {
metric := tt.metric() metric := tt.metric()
converter := NewPrometheusConverter() mockAppender := &mockCombinedAppender{}
converter := NewPrometheusConverter(mockAppender)
converter.addSumNumberDataPoints( converter.addSumNumberDataPoints(
context.Background(), context.Background(),
@ -362,11 +365,13 @@ func TestPrometheusConverter_addSumNumberDataPoints(t *testing.T) {
ExportCreatedMetric: true, ExportCreatedMetric: true,
PromoteScopeMetadata: tt.promoteScope, PromoteScopeMetadata: tt.promoteScope,
}, },
prompb.MetricMetadata{MetricFamilyName: metric.Name()}, metric.Name(),
tt.scope, 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) require.Empty(t, converter.conflicts)
}) })
} }

View File

@ -17,42 +17,41 @@
package prometheusremotewrite package prometheusremotewrite
import ( import (
"github.com/prometheus/common/model"
"go.opentelemetry.io/collector/pdata/pmetric" "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() { switch otelMetric.Type() {
case pmetric.MetricTypeGauge: case pmetric.MetricTypeGauge:
return prompb.MetricMetadata_GAUGE return model.MetricTypeGauge
case pmetric.MetricTypeSum: case pmetric.MetricTypeSum:
metricType := prompb.MetricMetadata_GAUGE metricType := model.MetricTypeGauge
if otelMetric.Sum().IsMonotonic() { 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'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. // 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 { if otelMetric.Sum().AggregationTemporality() == pmetric.AggregationTemporalityDelta {
metricType = prompb.MetricMetadata_UNKNOWN metricType = model.MetricTypeUnknown
} }
return metricType return metricType
case pmetric.MetricTypeHistogram: case pmetric.MetricTypeHistogram:
// We're in an early phase of implementing delta support (proposal: https://github.com/prometheus/proposals/pull/48/) // 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. // 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 { if otelMetric.Histogram().AggregationTemporality() == pmetric.AggregationTemporalityDelta {
return prompb.MetricMetadata_UNKNOWN return model.MetricTypeUnknown
} }
return prompb.MetricMetadata_HISTOGRAM return model.MetricTypeHistogram
case pmetric.MetricTypeSummary: case pmetric.MetricTypeSummary:
return prompb.MetricMetadata_SUMMARY return model.MetricTypeSummary
case pmetric.MetricTypeExponentialHistogram: case pmetric.MetricTypeExponentialHistogram:
if otelMetric.ExponentialHistogram().AggregationTemporality() == pmetric.AggregationTemporalityDelta { 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'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. // 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
} }

View File

@ -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
}

View File

@ -535,25 +535,28 @@ type OTLPOptions struct {
LookbackDelta time.Duration LookbackDelta time.Duration
// Add type and unit labels to the metrics. // Add type and unit labels to the metrics.
EnableTypeAndUnitLabels bool 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 // NewOTLPWriteHandler creates a http.Handler that accepts OTLP write requests and
// writes them to the provided appendable. // 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 { if opts.NativeDelta && opts.ConvertDelta {
// This should be validated when iterating through feature flags, so not expected to fail here. // 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") panic("cannot enable native delta ingestion and delta2cumulative conversion at the same time")
} }
ex := &rwExporter{ ex := &rwExporter{
writeHandler: &writeHandler{ logger: logger,
logger: logger, appendable: appendable,
appendable: appendable,
},
config: configFunc, config: configFunc,
allowDeltaTemporality: opts.NativeDelta, allowDeltaTemporality: opts.NativeDelta,
lookbackDelta: opts.LookbackDelta, lookbackDelta: opts.LookbackDelta,
ingestCTZeroSample: opts.IngestCTZeroSample,
enableTypeAndUnitLabels: opts.EnableTypeAndUnitLabels, enableTypeAndUnitLabels: opts.EnableTypeAndUnitLabels,
reg: reg,
} }
wh := &otlpWriteHandler{logger: logger, defaultConsumer: ex} wh := &otlpWriteHandler{logger: logger, defaultConsumer: ex}
@ -587,18 +590,24 @@ func NewOTLPWriteHandler(logger *slog.Logger, _ prometheus.Registerer, appendabl
} }
type rwExporter struct { type rwExporter struct {
*writeHandler logger *slog.Logger
appendable storage.Appendable
config func() config.Config config func() config.Config
allowDeltaTemporality bool allowDeltaTemporality bool
lookbackDelta time.Duration lookbackDelta time.Duration
ingestCTZeroSample bool
enableTypeAndUnitLabels bool enableTypeAndUnitLabels bool
reg prometheus.Registerer
} }
func (rw *rwExporter) ConsumeMetrics(ctx context.Context, md pmetric.Metrics) error { func (rw *rwExporter) ConsumeMetrics(ctx context.Context, md pmetric.Metrics) error {
otlpCfg := rw.config().OTLPConfig otlpCfg := rw.config().OTLPConfig
app := &timeLimitAppender{
converter := otlptranslator.NewPrometheusConverter() 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{ annots, err := converter.FromMetrics(ctx, md, otlptranslator.Settings{
AddMetricSuffixes: otlpCfg.TranslationStrategy.ShouldAddSuffixes(), AddMetricSuffixes: otlpCfg.TranslationStrategy.ShouldAddSuffixes(),
AllowUTF8: !otlpCfg.TranslationStrategy.ShouldEscape(), AllowUTF8: !otlpCfg.TranslationStrategy.ShouldEscape(),
@ -610,18 +619,18 @@ func (rw *rwExporter) ConsumeMetrics(ctx context.Context, md pmetric.Metrics) er
LookbackDelta: rw.lookbackDelta, LookbackDelta: rw.lookbackDelta,
EnableTypeAndUnitLabels: rw.enableTypeAndUnitLabels, 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) ws, _ := annots.AsStrings("", 0, 0)
if len(ws) > 0 { if len(ws) > 0 {
rw.logger.Warn("Warnings translating OTLP metrics to Prometheus write request", "warnings", ws) 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 return err
} }

View File

@ -44,6 +44,7 @@ import (
"github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/config"
"github.com/prometheus/prometheus/model/histogram" "github.com/prometheus/prometheus/model/histogram"
"github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/model/labels"
"github.com/prometheus/prometheus/model/metadata"
"github.com/prometheus/prometheus/model/relabel" "github.com/prometheus/prometheus/model/relabel"
"github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/storage"
) )
@ -381,14 +382,16 @@ func TestWriteStorageApplyConfig_PartialUpdate(t *testing.T) {
require.NoError(t, s.Close()) require.NoError(t, s.Close())
} }
func TestOTLPWriteHandler(t *testing.T) { func TestOTLPWriteHandlerNaming(t *testing.T) {
timestamp := time.Now() timestamp := time.Now()
exportRequest := generateOTLPWriteRequest(timestamp) var zeroTime time.Time
exportRequest := generateOTLPWriteRequest(timestamp, zeroTime)
for _, testCase := range []struct { for _, testCase := range []struct {
name string name string
otlpCfg config.OTLPConfig otlpCfg config.OTLPConfig
typeAndUnitLabels bool typeAndUnitLabels bool
expectedSamples []mockSample expectedSamples []mockSample
expectedMetadata []mockMetadata
}{ }{
{ {
name: "NoTranslation/NoTypeAndUnitLabels", name: "NoTranslation/NoTypeAndUnitLabels",
@ -397,24 +400,70 @@ func TestOTLPWriteHandler(t *testing.T) {
}, },
expectedSamples: []mockSample{ expectedSamples: []mockSample{
{ {
l: labels.New(labels.Label{Name: "__name__", Value: "test.counter"}, l: labels.FromStrings(model.MetricNameLabel, "test.counter", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"),
labels.Label{Name: "foo.bar", Value: "baz"},
labels.Label{Name: "instance", Value: "test-instance"},
labels.Label{Name: "job", Value: "test-service"}),
t: timestamp.UnixMilli(), t: timestamp.UnixMilli(),
v: 10.0, v: 10.0,
}, },
{ {
l: labels.New( l: labels.FromStrings(model.MetricNameLabel, "target_info", "host.name", "test-host", "instance", "test-instance", "job", "test-service"),
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"},
),
t: timestamp.UnixMilli(), t: timestamp.UnixMilli(),
v: 1, 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", name: "NoTranslation/WithTypeAndUnitLabels",
@ -424,26 +473,71 @@ func TestOTLPWriteHandler(t *testing.T) {
typeAndUnitLabels: true, typeAndUnitLabels: true,
expectedSamples: []mockSample{ expectedSamples: []mockSample{
{ {
l: labels.New(labels.Label{Name: "__name__", Value: "test.counter"}, l: labels.FromStrings(model.MetricNameLabel, "test.counter", "__type__", "counter", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"),
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"}),
t: timestamp.UnixMilli(), t: timestamp.UnixMilli(),
v: 10.0, v: 10.0,
}, },
{ {
l: labels.New( l: labels.FromStrings(model.MetricNameLabel, "target_info", "host.name", "test-host", "instance", "test-instance", "job", "test-service"),
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"},
),
t: timestamp.UnixMilli(), t: timestamp.UnixMilli(),
v: 1, 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", name: "UnderscoreEscapingWithSuffixes/NoTypeAndUnitLabels",
@ -452,24 +546,71 @@ func TestOTLPWriteHandler(t *testing.T) {
}, },
expectedSamples: []mockSample{ expectedSamples: []mockSample{
{ {
l: labels.New(labels.Label{Name: "__name__", Value: "test_counter_bytes_total"}, l: labels.FromStrings(model.MetricNameLabel, "test_counter_bytes_total", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"),
labels.Label{Name: "foo_bar", Value: "baz"},
labels.Label{Name: "instance", Value: "test-instance"},
labels.Label{Name: "job", Value: "test-service"}),
t: timestamp.UnixMilli(), t: timestamp.UnixMilli(),
v: 10.0, v: 10.0,
}, },
{ {
l: labels.New( l: labels.FromStrings(model.MetricNameLabel, "target_info", "host_name", "test-host", "instance", "test-instance", "job", "test-service"),
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"},
),
t: timestamp.UnixMilli(), t: timestamp.UnixMilli(),
v: 1, 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", name: "UnderscoreEscapingWithoutSuffixes",
@ -478,24 +619,70 @@ func TestOTLPWriteHandler(t *testing.T) {
}, },
expectedSamples: []mockSample{ expectedSamples: []mockSample{
{ {
l: labels.New(labels.Label{Name: "__name__", Value: "test_counter"}, l: labels.FromStrings(model.MetricNameLabel, "test_counter", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"),
labels.Label{Name: "foo_bar", Value: "baz"},
labels.Label{Name: "instance", Value: "test-instance"},
labels.Label{Name: "job", Value: "test-service"}),
t: timestamp.UnixMilli(), t: timestamp.UnixMilli(),
v: 10.0, v: 10.0,
}, },
{ {
l: labels.New( l: labels.FromStrings(model.MetricNameLabel, "target_info", "host_name", "test-host", "instance", "test-instance", "job", "test-service"),
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"},
),
t: timestamp.UnixMilli(), t: timestamp.UnixMilli(),
v: 1, 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", name: "UnderscoreEscapingWithSuffixes/WithTypeAndUnitLabels",
@ -525,6 +712,60 @@ func TestOTLPWriteHandler(t *testing.T) {
v: 1, 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", name: "NoUTF8EscapingWithSuffixes/NoTypeAndUnitLabels",
@ -533,24 +774,71 @@ func TestOTLPWriteHandler(t *testing.T) {
}, },
expectedSamples: []mockSample{ expectedSamples: []mockSample{
{ {
l: labels.New(labels.Label{Name: "__name__", Value: "test.counter_bytes_total"}, l: labels.FromStrings(model.MetricNameLabel, "test.counter_bytes_total", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"),
labels.Label{Name: "foo.bar", Value: "baz"},
labels.Label{Name: "instance", Value: "test-instance"},
labels.Label{Name: "job", Value: "test-service"}),
t: timestamp.UnixMilli(), t: timestamp.UnixMilli(),
v: 10.0, v: 10.0,
}, },
{ {
l: labels.New( l: labels.FromStrings(model.MetricNameLabel, "target_info", "host.name", "test-host", "instance", "test-instance", "job", "test-service"),
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"},
),
t: timestamp.UnixMilli(), t: timestamp.UnixMilli(),
v: 1, 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", name: "NoUTF8EscapingWithSuffixes/WithTypeAndUnitLabels",
@ -560,40 +848,268 @@ func TestOTLPWriteHandler(t *testing.T) {
typeAndUnitLabels: true, typeAndUnitLabels: true,
expectedSamples: []mockSample{ expectedSamples: []mockSample{
{ {
l: labels.New(labels.Label{Name: "__name__", Value: "test.counter_bytes_total"}, l: labels.FromStrings(model.MetricNameLabel, "test.counter_bytes_total", "__type__", "counter", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"),
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"}),
t: timestamp.UnixMilli(), t: timestamp.UnixMilli(),
v: 10.0, v: 10.0,
}, },
{ {
l: labels.New( l: labels.FromStrings(model.MetricNameLabel, "target_info", "host.name", "test-host", "instance", "test-instance", "job", "test-service"),
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"},
),
t: timestamp.UnixMilli(), t: timestamp.UnixMilli(),
v: 1, 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) { 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 { for _, sample := range testCase.expectedSamples {
requireContainsSample(t, appendable.samples, sample) 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.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.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) 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) { func requireContainsSample(t *testing.T, actual []mockSample, expected mockSample) {
t.Helper() t.Helper()
@ -607,7 +1123,20 @@ func requireContainsSample(t *testing.T, actual []mockSample, expected mockSampl
"actual : %v", expected, actual)) "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() buf, err := exportRequest.MarshalProto()
require.NoError(t, err) require.NoError(t, err)
@ -615,12 +1144,13 @@ func handleOTLP(t *testing.T, exportRequest pmetricotlp.ExportRequest, otlpCfg c
require.NoError(t, err) require.NoError(t, err)
req.Header.Set("Content-Type", "application/x-protobuf") req.Header.Set("Content-Type", "application/x-protobuf")
log := slog.New(slog.NewTextHandler(os.Stderr, &slog.HandlerOptions{Level: slog.LevelWarn}))
appendable := &mockAppendable{} appendable := &mockAppendable{}
handler := NewOTLPWriteHandler(nil, nil, appendable, func() config.Config { handler := NewOTLPWriteHandler(log, nil, appendable, func() config.Config {
return config.Config{ return config.Config{
OTLPConfig: otlpCfg, OTLPConfig: otlpCfg,
} }
}, OTLPOptions{EnableTypeAndUnitLabels: typeAndUnitLabels}) }, otlpOpts)
recorder := httptest.NewRecorder() recorder := httptest.NewRecorder()
handler.ServeHTTP(recorder, req) handler.ServeHTTP(recorder, req)
@ -630,7 +1160,7 @@ func handleOTLP(t *testing.T, exportRequest pmetricotlp.ExportRequest, otlpCfg c
return appendable return appendable
} }
func generateOTLPWriteRequest(timestamp time.Time) pmetricotlp.ExportRequest { func generateOTLPWriteRequest(timestamp, startTime time.Time) pmetricotlp.ExportRequest {
d := pmetric.NewMetrics() d := pmetric.NewMetrics()
// Generate One Counter, One Gauge, One Histogram, One Exponential-Histogram // 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 := counterMetric.Sum().DataPoints().AppendEmpty()
counterDataPoint.SetTimestamp(pcommon.NewTimestampFromTime(timestamp)) counterDataPoint.SetTimestamp(pcommon.NewTimestampFromTime(timestamp))
counterDataPoint.SetStartTimestamp(pcommon.NewTimestampFromTime(startTime))
counterDataPoint.SetDoubleValue(10.0) counterDataPoint.SetDoubleValue(10.0)
counterDataPoint.Attributes().PutStr("foo.bar", "baz") counterDataPoint.Attributes().PutStr("foo.bar", "baz")
@ -674,6 +1205,7 @@ func generateOTLPWriteRequest(timestamp time.Time) pmetricotlp.ExportRequest {
gaugeDataPoint := gaugeMetric.Gauge().DataPoints().AppendEmpty() gaugeDataPoint := gaugeMetric.Gauge().DataPoints().AppendEmpty()
gaugeDataPoint.SetTimestamp(pcommon.NewTimestampFromTime(timestamp)) gaugeDataPoint.SetTimestamp(pcommon.NewTimestampFromTime(timestamp))
gaugeDataPoint.SetStartTimestamp(pcommon.NewTimestampFromTime(startTime))
gaugeDataPoint.SetDoubleValue(10.0) gaugeDataPoint.SetDoubleValue(10.0)
gaugeDataPoint.Attributes().PutStr("foo.bar", "baz") gaugeDataPoint.Attributes().PutStr("foo.bar", "baz")
@ -687,9 +1219,10 @@ func generateOTLPWriteRequest(timestamp time.Time) pmetricotlp.ExportRequest {
histogramDataPoint := histogramMetric.Histogram().DataPoints().AppendEmpty() histogramDataPoint := histogramMetric.Histogram().DataPoints().AppendEmpty()
histogramDataPoint.SetTimestamp(pcommon.NewTimestampFromTime(timestamp)) 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.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.BucketCounts().FromRaw([]uint64{2, 2, 2, 2, 2, 2})
histogramDataPoint.SetCount(10) histogramDataPoint.SetCount(12)
histogramDataPoint.SetSum(30.0) histogramDataPoint.SetSum(30.0)
histogramDataPoint.Attributes().PutStr("foo.bar", "baz") histogramDataPoint.Attributes().PutStr("foo.bar", "baz")
@ -703,6 +1236,7 @@ func generateOTLPWriteRequest(timestamp time.Time) pmetricotlp.ExportRequest {
exponentialHistogramDataPoint := exponentialHistogramMetric.ExponentialHistogram().DataPoints().AppendEmpty() exponentialHistogramDataPoint := exponentialHistogramMetric.ExponentialHistogram().DataPoints().AppendEmpty()
exponentialHistogramDataPoint.SetTimestamp(pcommon.NewTimestampFromTime(timestamp)) exponentialHistogramDataPoint.SetTimestamp(pcommon.NewTimestampFromTime(timestamp))
exponentialHistogramDataPoint.SetStartTimestamp(pcommon.NewTimestampFromTime(startTime))
exponentialHistogramDataPoint.SetScale(2.0) exponentialHistogramDataPoint.SetScale(2.0)
exponentialHistogramDataPoint.Positive().BucketCounts().FromRaw([]uint64{2, 2, 2, 2, 2}) exponentialHistogramDataPoint.Positive().BucketCounts().FromRaw([]uint64{2, 2, 2, 2, 2})
exponentialHistogramDataPoint.SetZeroCount(2) exponentialHistogramDataPoint.SetZeroCount(2)

View File

@ -317,6 +317,7 @@ func NewAPI(
ConvertDelta: otlpDeltaToCumulative, ConvertDelta: otlpDeltaToCumulative,
NativeDelta: otlpNativeDeltaIngestion, NativeDelta: otlpNativeDeltaIngestion,
LookbackDelta: lookbackDelta, LookbackDelta: lookbackDelta,
IngestCTZeroSample: ctZeroIngestionEnabled,
EnableTypeAndUnitLabels: enableTypeAndUnitLabels, EnableTypeAndUnitLabels: enableTypeAndUnitLabels,
}) })
} }