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().
origAdd, origDel := append(addStack[:0], b.add...), append(delStack[:0], b.del...)
b.base.Range(func(l Label) {
if !slices.Contains(origDel, l.Name) && !contains(origAdd, l.Name) {
if !slices.Contains(origDel, l.Name) && !Contains(origAdd, l.Name) {
f(l)
}
})
@ -224,7 +224,8 @@ func (b *Builder) Range(f func(l Label)) {
}
}
func contains(s []Label, n string) bool {
// Contains returns true if the label name n is present in s, otherwise false.
func Contains(s []Label, n string) bool {
for _, a := range s {
if a.Name == n {
return true

View File

@ -464,6 +464,18 @@ func New(ls ...Label) Labels {
return Labels{syms: syms.nameTable, data: yoloString(buf)}
}
// NewFromSorted returns sorted Labels from the given
// sorted labels. In case of slices it returns the
// input.
func NewFromSorted(ls []Label) Labels {
syms := NewSymbolTable()
var stackSpace [16]int
size, nums := mapLabelsToNumbers(syms, ls, stackSpace[:])
buf := make([]byte, size)
marshalNumbersToSizedBuffer(nums, buf)
return Labels{syms: syms.nameTable, data: yoloString(buf)}
}
// FromStrings creates new labels from pairs of strings.
func FromStrings(ss ...string) Labels {
if len(ss)%2 != 0 {

View File

@ -279,6 +279,13 @@ func New(ls ...Label) Labels {
return set
}
// NewFromSorted returns sorted Labels from the given
// sorted labels. In case of slices it returns the
// input. This function does NOT copy!
func NewFromSorted(ls []Label) Labels {
return ls
}
// FromStrings creates new labels from pairs of strings.
func FromStrings(ss ...string) Labels {
if len(ss)%2 != 0 {
@ -423,7 +430,7 @@ func (b *Builder) Labels() Labels {
}
res := make(Labels, 0, expectedSize)
for _, l := range b.base {
if slices.Contains(b.del, l.Name) || contains(b.add, l.Name) {
if slices.Contains(b.del, l.Name) || Contains(b.add, l.Name) {
continue
}
res = append(res, l)

View File

@ -314,6 +314,16 @@ func New(ls ...Label) Labels {
return Labels{data: yoloString(buf)}
}
// NewFromSorted returns sorted Labels from the given
// sorted labels. In case of slices it returns the
// input.
func NewFromSorted(ls []Label) Labels {
size := labelsSize(ls)
buf := make([]byte, size)
marshalLabelsToSizedBuffer(ls, buf)
return Labels{data: yoloString(buf)}
}
// FromStrings creates new labels from pairs of strings.
func FromStrings(ss ...string) Labels {
if len(ss)%2 != 0 {

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"
"math"
"slices"
"sort"
"strconv"
"strings"
"time"
"unicode/utf8"
"github.com/cespare/xxhash/v2"
"github.com/prometheus/common/model"
"github.com/prometheus/otlptranslator"
"go.opentelemetry.io/collector/pdata/pcommon"
"go.opentelemetry.io/collector/pdata/pmetric"
conventions "go.opentelemetry.io/collector/semconv/v1.6.1"
"github.com/prometheus/prometheus/model/exemplar"
modelLabels "github.com/prometheus/prometheus/model/labels"
"github.com/prometheus/prometheus/model/metadata"
"github.com/prometheus/prometheus/model/timestamp"
"github.com/prometheus/prometheus/model/value"
"github.com/prometheus/prometheus/prompb"
"github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite/labels"
)
const (
@ -62,177 +64,115 @@ const (
defaultLookbackDelta = 5 * time.Minute
)
type bucketBoundsData struct {
ts *prompb.TimeSeries
bound float64
}
// byBucketBoundsData enables the usage of sort.Sort() with a slice of bucket bounds.
type byBucketBoundsData []bucketBoundsData
func (m byBucketBoundsData) Len() int { return len(m) }
func (m byBucketBoundsData) Less(i, j int) bool { return m[i].bound < m[j].bound }
func (m byBucketBoundsData) Swap(i, j int) { m[i], m[j] = m[j], m[i] }
// ByLabelName enables the usage of sort.Sort() with a slice of labels.
type ByLabelName []prompb.Label
func (a ByLabelName) Len() int { return len(a) }
func (a ByLabelName) Less(i, j int) bool { return a[i].Name < a[j].Name }
func (a ByLabelName) Swap(i, j int) { a[i], a[j] = a[j], a[i] }
// timeSeriesSignature returns a hashed label set signature.
// The label slice should not contain duplicate label names; this method sorts the slice by label name before creating
// the signature.
// The algorithm is the same as in Prometheus' labels.StableHash function.
func timeSeriesSignature(labels []prompb.Label) uint64 {
sort.Sort(ByLabelName(labels))
// Use xxhash.Sum64(b) for fast path as it's faster.
b := make([]byte, 0, 1024)
for i, v := range labels {
if len(b)+len(v.Name)+len(v.Value)+2 >= cap(b) {
// If labels entry is 1KB+ do not allocate whole entry.
h := xxhash.New()
_, _ = h.Write(b)
for _, v := range labels[i:] {
_, _ = h.WriteString(v.Name)
_, _ = h.Write(seps)
_, _ = h.WriteString(v.Value)
_, _ = h.Write(seps)
}
return h.Sum64()
}
b = append(b, v.Name...)
b = append(b, seps[0])
b = append(b, v.Value...)
b = append(b, seps[0])
}
return xxhash.Sum64(b)
}
var seps = []byte{'\xff'}
// createAttributes creates a slice of Prometheus Labels with OTLP attributes and pairs of string values.
// Unpaired string values are ignored. String pairs overwrite OTLP labels if collisions happen and
// if logOnOverwrite is true, the overwrite is logged. Resulting label names are sanitized.
// If settings.PromoteResourceAttributes is not empty, it's a set of resource attributes that should be promoted to labels.
func createAttributes(resource pcommon.Resource, attributes pcommon.Map, scope scope, settings Settings,
ignoreAttrs []string, logOnOverwrite bool, metadata prompb.MetricMetadata, extras ...string,
) ([]prompb.Label, error) {
func (c *PrometheusConverter) createAttributes(resource pcommon.Resource, attributes pcommon.Map, scope scope, settings Settings,
ignoreAttrs []string, logOnOverwrite bool, meta metadata.Metadata, extras ...string,
) (labels.Labels, error) {
resourceAttrs := resource.Attributes()
serviceName, haveServiceName := resourceAttrs.Get(conventions.AttributeServiceName)
instance, haveInstanceID := resourceAttrs.Get(conventions.AttributeServiceInstanceID)
promotedAttrs := settings.PromoteResourceAttributes.promotedAttributes(resourceAttrs)
promoteScope := settings.PromoteScopeMetadata && scope.name != ""
scopeLabelCount := 0
if promoteScope {
// Include name, version and schema URL.
scopeLabelCount = scope.attributes.Len() + 3
}
// Calculate the maximum possible number of labels we could return so we can preallocate l.
maxLabelCount := attributes.Len() + len(settings.ExternalLabels) + len(promotedAttrs) + scopeLabelCount + len(extras)/2
if haveServiceName {
maxLabelCount++
}
if haveInstanceID {
maxLabelCount++
}
if settings.EnableTypeAndUnitLabels {
maxLabelCount += 2
}
// Ensure attributes are sorted by key for consistent merging of keys which
// collide when sanitized.
labels := make([]prompb.Label, 0, maxLabelCount)
c.scratchBuilder.Reset()
// XXX: Should we always drop service namespace/service name/service instance ID from the labels
// (as they get mapped to other Prometheus labels)?
attributes.Range(func(key string, value pcommon.Value) bool {
if !slices.Contains(ignoreAttrs, key) {
labels = append(labels, prompb.Label{Name: key, Value: value.AsString()})
c.scratchBuilder.Add(key, value.AsString())
}
return true
})
sort.Stable(ByLabelName(labels))
c.scratchBuilder.Sort()
sortedLabels := c.scratchBuilder.Labels()
// map ensures no duplicate label names.
l := make(map[string]string, maxLabelCount)
labelNamer := otlptranslator.LabelNamer{UTF8Allowed: settings.AllowUTF8}
for _, label := range labels {
finalKey, err := labelNamer.Build(label.Name)
if err != nil {
return nil, err
}
if existingValue, alreadyExists := l[finalKey]; alreadyExists {
l[finalKey] = existingValue + ";" + label.Value
} else {
l[finalKey] = label.Value
}
}
for _, lbl := range promotedAttrs {
normalized, err := labelNamer.Build(lbl.Name)
if err != nil {
return nil, err
}
if _, exists := l[normalized]; !exists {
l[normalized] = lbl.Value
}
}
if promoteScope {
var rangeErr error
scope.attributes.Range(func(k string, v pcommon.Value) bool {
name, err := labelNamer.Build("otel_scope_" + k)
if err != nil {
rangeErr = err
return false
if settings.AllowUTF8 {
// UTF8 is allowed, so conflicts aren't possible.
c.builder.Reset(sortedLabels)
} else {
// Now that we have sorted and filtered the labels, build the actual list
// of labels, and handle conflicts by appending values.
c.builder.Reset(labels.EmptyLabels())
var sortErr error
sortedLabels.Range(func(l modelLabels.Label) {
finalKey, err := labelNamer.Build(l.Name)
if err != nil && sortErr == nil {
sortErr = err
return
}
if existingValue := c.builder.Get(finalKey); existingValue != "" {
c.builder.Set(finalKey, existingValue+";"+l.Value)
} else {
c.builder.Set(finalKey, l.Value)
}
l[name] = v.AsString()
return true
})
if rangeErr != nil {
return nil, rangeErr
if sortErr != nil {
return nil, sortErr
}
// Scope Name, Version and Schema URL are added after attributes to ensure they are not overwritten by attributes.
l["otel_scope_name"] = scope.name
l["otel_scope_version"] = scope.version
l["otel_scope_schema_url"] = scope.schemaURL
}
if settings.EnableTypeAndUnitLabels {
unitNamer := otlptranslator.UnitNamer{UTF8Allowed: settings.AllowUTF8}
if metadata.Type != prompb.MetricMetadata_UNKNOWN {
l["__type__"] = strings.ToLower(metadata.Type.String())
if meta.Type != model.MetricTypeUnknown {
c.builder.Set("__type__", strings.ToLower(string(meta.Type)))
}
if metadata.Unit != "" {
l["__unit__"] = unitNamer.Build(metadata.Unit)
if meta.Unit != "" {
c.builder.Set("__unit__", unitNamer.Build(meta.Unit))
}
}
err := settings.PromoteResourceAttributes.addPromotedAttributes(c.builder, resourceAttrs, settings.AllowUTF8)
if err != nil {
return nil, err
}
if promoteScope {
// Scope Name, Version and Schema URL are added after attributes to ensure they are not overwritten by attributes.
c.builder.Set("otel_scope_name", scope.name)
c.builder.Set("otel_scope_version", scope.version)
c.builder.Set("otel_scope_schema_url", scope.schemaURL)
var scopeErr error
scope.attributes.Range(func(k string, v pcommon.Value) bool {
name := "otel_scope_" + k
var err error
name, err = labelNamer.Build(name)
if err != nil && scopeErr == nil {
scopeErr = err
return false
}
c.builder.Set(name, v.AsString())
return true
})
if scopeErr != nil {
return nil, scopeErr
}
}
// Map service.name + service.namespace to job.
if haveServiceName {
val := serviceName.AsString()
if serviceNamespace, ok := resourceAttrs.Get(conventions.AttributeServiceNamespace); ok {
val = fmt.Sprintf("%s/%s", serviceNamespace.AsString(), val)
}
l[model.JobLabel] = val
c.builder.Set(model.JobLabel, val)
}
// Map service.instance.id to instance.
if haveInstanceID {
l[model.InstanceLabel] = instance.AsString()
c.builder.Set(model.InstanceLabel, instance.AsString())
}
for key, value := range settings.ExternalLabels {
// External labels have already been sanitized.
if _, alreadyExists := l[key]; alreadyExists {
if existingValue := c.builder.Get(key); existingValue != "" {
// Skip external labels if they are overridden by metric attributes.
continue
}
l[key] = value
c.builder.Set(key, value)
}
for i := 0; i < len(extras); i += 2 {
@ -241,8 +181,7 @@ func createAttributes(resource pcommon.Resource, attributes pcommon.Map, scope s
}
name := extras[i]
_, found := l[name]
if found && logOnOverwrite {
if existingValue := c.builder.Get(name); existingValue != "" && logOnOverwrite {
log.Println("label " + name + " is overwritten. Check if Prometheus reserved labels are used.")
}
// internal labels should be maintained.
@ -253,15 +192,10 @@ func createAttributes(resource pcommon.Resource, attributes pcommon.Map, scope s
return nil, err
}
}
l[name] = extras[i+1]
c.builder.Set(name, extras[i+1])
}
labels = labels[:0]
for k, v := range l {
labels = append(labels, prompb.Label{Name: k, Value: v})
}
return labels, nil
return c.builder.Labels(), nil
}
func aggregationTemporality(metric pmetric.Metric) (pmetric.AggregationTemporality, bool, error) {
@ -287,7 +221,7 @@ func aggregationTemporality(metric pmetric.Metric) (pmetric.AggregationTemporali
// However, work is under way to resolve this shortcoming through a feature called native histograms custom buckets:
// https://github.com/prometheus/prometheus/issues/13485.
func (c *PrometheusConverter) addHistogramDataPoints(ctx context.Context, dataPoints pmetric.HistogramDataPointSlice,
resource pcommon.Resource, settings Settings, metadata prompb.MetricMetadata, scope scope,
resource pcommon.Resource, settings Settings, baseName string, scope scope, meta metadata.Metadata,
) error {
for x := 0; x < dataPoints.Len(); x++ {
if err := c.everyN.checkContext(ctx); err != nil {
@ -296,7 +230,8 @@ func (c *PrometheusConverter) addHistogramDataPoints(ctx context.Context, dataPo
pt := dataPoints.At(x)
timestamp := convertTimeStamp(pt.Timestamp())
baseLabels, err := createAttributes(resource, pt.Attributes(), scope, settings, nil, false, metadata)
startTimestamp := convertTimeStamp(pt.StartTimestamp())
baseLabels, err := c.createAttributes(resource, pt.Attributes(), scope, settings, nil, false, meta)
if err != nil {
return err
}
@ -305,35 +240,36 @@ func (c *PrometheusConverter) addHistogramDataPoints(ctx context.Context, dataPo
// omitted
if pt.HasSum() {
// treat sum as a sample in an individual TimeSeries
sum := &prompb.Sample{
Value: pt.Sum(),
Timestamp: timestamp,
}
val := pt.Sum()
if pt.Flags().NoRecordedValue() {
sum.Value = math.Float64frombits(value.StaleNaN)
val = math.Float64frombits(value.StaleNaN)
}
sumlabels := createLabels(metadata.MetricFamilyName+sumStr, baseLabels)
c.addSample(sum, sumlabels)
sumlabels := c.addLabels(baseName+sumStr, baseLabels)
if err := c.appender.AppendSample(baseName, sumlabels, meta, timestamp, startTimestamp, val, nil); err != nil {
return err
}
}
// treat count as a sample in an individual TimeSeries
count := &prompb.Sample{
Value: float64(pt.Count()),
Timestamp: timestamp,
}
val := float64(pt.Count())
if pt.Flags().NoRecordedValue() {
count.Value = math.Float64frombits(value.StaleNaN)
val = math.Float64frombits(value.StaleNaN)
}
countlabels := createLabels(metadata.MetricFamilyName+countStr, baseLabels)
c.addSample(count, countlabels)
countlabels := c.addLabels(baseName+countStr, baseLabels)
if err := c.appender.AppendSample(baseName, countlabels, meta, timestamp, startTimestamp, val, nil); err != nil {
return err
}
exemplars, err := c.getPromExemplars(ctx, pt.Exemplars())
if err != nil {
return err
}
nextExemplarIdx := 0
// cumulative count for conversion to cumulative histogram
var cumulativeCount uint64
var bucketBounds []bucketBoundsData
// process each bound, based on histograms proto definition, # of buckets = # of explicit bounds + 1
for i := 0; i < pt.ExplicitBounds().Len() && i < pt.BucketCounts().Len(); i++ {
if err := c.everyN.checkContext(ctx); err != nil {
@ -342,116 +278,110 @@ func (c *PrometheusConverter) addHistogramDataPoints(ctx context.Context, dataPo
bound := pt.ExplicitBounds().At(i)
cumulativeCount += pt.BucketCounts().At(i)
bucket := &prompb.Sample{
Value: float64(cumulativeCount),
Timestamp: timestamp,
// Find exemplars that belong to this bucket. Both exemplars and
// buckets are sorted in ascending order.
var currentBucketExemplars []exemplar.Exemplar
for ; nextExemplarIdx < len(exemplars); nextExemplarIdx++ {
ex := exemplars[nextExemplarIdx]
if ex.Value > bound {
// This exemplar belongs in a higher bucket.
break
}
currentBucketExemplars = append(currentBucketExemplars, ex)
}
val := float64(cumulativeCount)
if pt.Flags().NoRecordedValue() {
bucket.Value = math.Float64frombits(value.StaleNaN)
val = math.Float64frombits(value.StaleNaN)
}
boundStr := strconv.FormatFloat(bound, 'f', -1, 64)
labels := createLabels(metadata.MetricFamilyName+bucketStr, baseLabels, leStr, boundStr)
ts := c.addSample(bucket, labels)
bucketBounds = append(bucketBounds, bucketBoundsData{ts: ts, bound: bound})
labels := c.addLabels(baseName+bucketStr, baseLabels, leStr, boundStr)
if err := c.appender.AppendSample(baseName, labels, meta, timestamp, startTimestamp, val, currentBucketExemplars); err != nil {
return err
}
}
// add le=+Inf bucket
infBucket := &prompb.Sample{
Timestamp: timestamp,
}
val = float64(pt.Count())
if pt.Flags().NoRecordedValue() {
infBucket.Value = math.Float64frombits(value.StaleNaN)
} else {
infBucket.Value = float64(pt.Count())
val = math.Float64frombits(value.StaleNaN)
}
infLabels := createLabels(metadata.MetricFamilyName+bucketStr, baseLabels, leStr, pInfStr)
ts := c.addSample(infBucket, infLabels)
bucketBounds = append(bucketBounds, bucketBoundsData{ts: ts, bound: math.Inf(1)})
if err := c.addExemplars(ctx, pt, bucketBounds); err != nil {
infLabels := c.addLabels(baseName+bucketStr, baseLabels, leStr, pInfStr)
if err := c.appender.AppendSample(baseName, infLabels, meta, timestamp, startTimestamp, val, exemplars[nextExemplarIdx:]); err != nil {
return err
}
startTimestamp := pt.StartTimestamp()
if settings.ExportCreatedMetric && startTimestamp != 0 {
labels := createLabels(metadata.MetricFamilyName+createdSuffix, baseLabels)
c.addTimeSeriesIfNeeded(labels, startTimestamp, pt.Timestamp())
if settings.ExportCreatedMetric && pt.StartTimestamp() != 0 {
labels := c.addLabels(baseName+createdSuffix, baseLabels)
if c.timeSeriesIsNew(labels) {
if err := c.appender.AppendSample(baseName, labels, meta, timestamp, 0, float64(startTimestamp), nil); err != nil {
return err
}
}
}
}
return nil
}
type exemplarType interface {
pmetric.ExponentialHistogramDataPoint | pmetric.HistogramDataPoint | pmetric.NumberDataPoint
Exemplars() pmetric.ExemplarSlice
}
func getPromExemplars[T exemplarType](ctx context.Context, everyN *everyNTimes, pt T) ([]prompb.Exemplar, error) {
promExemplars := make([]prompb.Exemplar, 0, pt.Exemplars().Len())
for i := 0; i < pt.Exemplars().Len(); i++ {
if err := everyN.checkContext(ctx); err != nil {
func (c *PrometheusConverter) getPromExemplars(ctx context.Context, exemplars pmetric.ExemplarSlice) ([]exemplar.Exemplar, error) {
if exemplars.Len() == 0 {
return nil, nil
}
outputExemplars := make([]exemplar.Exemplar, 0, exemplars.Len())
for i := 0; i < exemplars.Len(); i++ {
if err := c.everyN.checkContext(ctx); err != nil {
return nil, err
}
exemplar := pt.Exemplars().At(i)
ex := exemplars.At(i)
exemplarRunes := 0
promExemplar := prompb.Exemplar{
Timestamp: timestamp.FromTime(exemplar.Timestamp().AsTime()),
ts := timestamp.FromTime(ex.Timestamp().AsTime())
newExemplar := exemplar.Exemplar{
Ts: ts,
HasTs: ts != 0,
}
switch exemplar.ValueType() {
c.scratchBuilder.Reset()
switch ex.ValueType() {
case pmetric.ExemplarValueTypeInt:
promExemplar.Value = float64(exemplar.IntValue())
newExemplar.Value = float64(ex.IntValue())
case pmetric.ExemplarValueTypeDouble:
promExemplar.Value = exemplar.DoubleValue()
newExemplar.Value = ex.DoubleValue()
default:
return nil, fmt.Errorf("unsupported exemplar value type: %v", exemplar.ValueType())
return nil, fmt.Errorf("unsupported exemplar value type: %v", ex.ValueType())
}
if traceID := exemplar.TraceID(); !traceID.IsEmpty() {
if traceID := ex.TraceID(); !traceID.IsEmpty() {
val := hex.EncodeToString(traceID[:])
exemplarRunes += utf8.RuneCountInString(traceIDKey) + utf8.RuneCountInString(val)
promLabel := prompb.Label{
Name: traceIDKey,
Value: val,
}
promExemplar.Labels = append(promExemplar.Labels, promLabel)
c.scratchBuilder.Add(traceIDKey, val)
}
if spanID := exemplar.SpanID(); !spanID.IsEmpty() {
if spanID := ex.SpanID(); !spanID.IsEmpty() {
val := hex.EncodeToString(spanID[:])
exemplarRunes += utf8.RuneCountInString(spanIDKey) + utf8.RuneCountInString(val)
promLabel := prompb.Label{
Name: spanIDKey,
Value: val,
}
promExemplar.Labels = append(promExemplar.Labels, promLabel)
c.scratchBuilder.Add(spanIDKey, val)
}
attrs := exemplar.FilteredAttributes()
labelsFromAttributes := make([]prompb.Label, 0, attrs.Len())
attrs := ex.FilteredAttributes()
attrs.Range(func(key string, value pcommon.Value) bool {
val := value.AsString()
exemplarRunes += utf8.RuneCountInString(key) + utf8.RuneCountInString(val)
promLabel := prompb.Label{
Name: key,
Value: val,
}
labelsFromAttributes = append(labelsFromAttributes, promLabel)
exemplarRunes += utf8.RuneCountInString(key) + utf8.RuneCountInString(value.AsString())
return true
})
if exemplarRunes <= maxExemplarRunes {
// only append filtered attributes if it does not cause exemplar
// labels to exceed the max number of runes
promExemplar.Labels = append(promExemplar.Labels, labelsFromAttributes...)
}
promExemplars = append(promExemplars, promExemplar)
// Only append filtered attributes if it does not cause exemplar
// labels to exceed the max number of runes.
if exemplarRunes <= maxExemplarRunes {
attrs.Range(func(key string, value pcommon.Value) bool {
c.scratchBuilder.Add(key, value.AsString())
return true
})
}
c.scratchBuilder.Sort()
newExemplar.Labels = modelLabels.NewFromSorted(c.scratchBuilder.Labels())
outputExemplars = append(outputExemplars, newExemplar)
}
return promExemplars, nil
return outputExemplars, nil
}
// findMinAndMaxTimestamps returns the minimum of minTimestamp and the earliest timestamp in metric and
@ -500,7 +430,7 @@ func findMinAndMaxTimestamps(metric pmetric.Metric, minTimestamp, maxTimestamp p
}
func (c *PrometheusConverter) addSummaryDataPoints(ctx context.Context, dataPoints pmetric.SummaryDataPointSlice, resource pcommon.Resource,
settings Settings, metadata prompb.MetricMetadata, scope scope,
settings Settings, baseName string, scope scope, meta metadata.Metadata,
) error {
for x := 0; x < dataPoints.Len(); x++ {
if err := c.everyN.checkContext(ctx); err != nil {
@ -509,75 +439,73 @@ func (c *PrometheusConverter) addSummaryDataPoints(ctx context.Context, dataPoin
pt := dataPoints.At(x)
timestamp := convertTimeStamp(pt.Timestamp())
baseLabels, err := createAttributes(resource, pt.Attributes(), scope, settings, nil, false, metadata)
startTimestamp := convertTimeStamp(pt.StartTimestamp())
baseLabels, err := c.createAttributes(resource, pt.Attributes(), scope, settings, nil, false, meta)
if err != nil {
return err
}
// treat sum as a sample in an individual TimeSeries
sum := &prompb.Sample{
Value: pt.Sum(),
Timestamp: timestamp,
}
val := pt.Sum()
if pt.Flags().NoRecordedValue() {
sum.Value = math.Float64frombits(value.StaleNaN)
val = math.Float64frombits(value.StaleNaN)
}
// sum and count of the summary should append suffix to baseName
sumlabels := createLabels(metadata.MetricFamilyName+sumStr, baseLabels)
c.addSample(sum, sumlabels)
sumlabels := c.addLabels(baseName+sumStr, baseLabels)
if err := c.appender.AppendSample(baseName, sumlabels, meta, timestamp, startTimestamp, val, nil); err != nil {
return err
}
// treat count as a sample in an individual TimeSeries
count := &prompb.Sample{
Value: float64(pt.Count()),
Timestamp: timestamp,
}
val = float64(pt.Count())
if pt.Flags().NoRecordedValue() {
count.Value = math.Float64frombits(value.StaleNaN)
val = math.Float64frombits(value.StaleNaN)
}
countlabels := c.addLabels(baseName+countStr, baseLabels)
if err := c.appender.AppendSample(baseName, countlabels, meta, timestamp, startTimestamp, val, nil); err != nil {
return err
}
countlabels := createLabels(metadata.MetricFamilyName+countStr, baseLabels)
c.addSample(count, countlabels)
// process each percentile/quantile
for i := 0; i < pt.QuantileValues().Len(); i++ {
qt := pt.QuantileValues().At(i)
quantile := &prompb.Sample{
Value: qt.Value(),
Timestamp: timestamp,
}
val = qt.Value()
if pt.Flags().NoRecordedValue() {
quantile.Value = math.Float64frombits(value.StaleNaN)
val = math.Float64frombits(value.StaleNaN)
}
percentileStr := strconv.FormatFloat(qt.Quantile(), 'f', -1, 64)
qtlabels := createLabels(metadata.MetricFamilyName, baseLabels, quantileStr, percentileStr)
c.addSample(quantile, qtlabels)
qtlabels := c.addLabels(baseName, baseLabels, quantileStr, percentileStr)
if err := c.appender.AppendSample(baseName, qtlabels, meta, timestamp, startTimestamp, val, nil); err != nil {
return err
}
}
startTimestamp := pt.StartTimestamp()
if settings.ExportCreatedMetric && startTimestamp != 0 {
createdLabels := createLabels(metadata.MetricFamilyName+createdSuffix, baseLabels)
c.addTimeSeriesIfNeeded(createdLabels, startTimestamp, pt.Timestamp())
if settings.ExportCreatedMetric && pt.StartTimestamp() != 0 {
createdLabels := c.addLabels(baseName+createdSuffix, baseLabels)
if c.timeSeriesIsNew(createdLabels) {
if err := c.appender.AppendSample(baseName, createdLabels, meta, timestamp, 0, float64(startTimestamp), nil); err != nil {
return err
}
}
}
}
return nil
}
// createLabels returns a copy of baseLabels, adding to it the pair model.MetricNameLabel=name.
// addLabels returns a copy of baseLabels, adding to it the pair model.MetricNameLabel=name.
// If extras are provided, corresponding label pairs are also added to the returned slice.
// If extras is uneven length, the last (unpaired) extra will be ignored.
func createLabels(name string, baseLabels []prompb.Label, extras ...string) []prompb.Label {
extraLabelCount := len(extras) / 2
labels := make([]prompb.Label, len(baseLabels), len(baseLabels)+extraLabelCount+1) // +1 for name
copy(labels, baseLabels)
func (c *PrometheusConverter) addLabels(name string, baseLabels labels.Labels, extras ...string) labels.Labels {
c.builder.Reset(baseLabels)
n := len(extras)
n -= n % 2
for extrasIdx := 0; extrasIdx < n; extrasIdx += 2 {
labels = append(labels, prompb.Label{Name: extras[extrasIdx], Value: extras[extrasIdx+1]})
c.builder.Set(extras[extrasIdx], extras[extrasIdx+1])
}
labels = append(labels, prompb.Label{Name: model.MetricNameLabel, Value: name})
return labels
c.builder.Set(model.MetricNameLabel, name)
return c.builder.Labels()
}
// addTypeAndUnitLabels appends type and unit labels to the given labels slice.
@ -596,57 +524,35 @@ func addTypeAndUnitLabels(labels []prompb.Label, metadata prompb.MetricMetadata,
// getOrCreateTimeSeries returns the time series corresponding to the label set if existent, and false.
// Otherwise it creates a new one and returns that, and true.
func (c *PrometheusConverter) getOrCreateTimeSeries(lbls []prompb.Label) (*prompb.TimeSeries, bool) {
h := timeSeriesSignature(lbls)
ts := c.unique[h]
if ts != nil {
if isSameMetric(ts, lbls) {
func (c *PrometheusConverter) timeSeriesIsNew(lbls labels.Labels) bool {
h := lbls.Hash()
uLabels, ok := c.unique[h]
if ok {
if labels.Equal(uLabels, lbls) {
// We already have this metric
return ts, false
return false
}
// Look for a matching conflict
for _, cTS := range c.conflicts[h] {
if isSameMetric(cTS, lbls) {
for _, cLabels := range c.conflicts[h] {
if labels.Equal(cLabels, lbls) {
// We already have this metric
return cTS, false
return false
}
}
// New conflict
ts = &prompb.TimeSeries{
Labels: lbls,
}
c.conflicts[h] = append(c.conflicts[h], ts)
return ts, true
c.conflicts[h] = append(c.conflicts[h], uLabels)
return true
}
// This metric is new
ts = &prompb.TimeSeries{
Labels: lbls,
}
c.unique[h] = ts
return ts, true
}
// addTimeSeriesIfNeeded adds a corresponding time series if it doesn't already exist.
// If the time series doesn't already exist, it gets added with startTimestamp for its value and timestamp for its timestamp,
// both converted to milliseconds.
func (c *PrometheusConverter) addTimeSeriesIfNeeded(lbls []prompb.Label, startTimestamp, timestamp pcommon.Timestamp) {
ts, created := c.getOrCreateTimeSeries(lbls)
if created {
ts.Samples = []prompb.Sample{
{
// convert ns to ms
Value: float64(convertTimeStamp(startTimestamp)),
Timestamp: convertTimeStamp(timestamp),
},
}
}
c.unique[h] = uLabels
return true
}
// addResourceTargetInfo converts the resource to the target info metric.
func addResourceTargetInfo(resource pcommon.Resource, settings Settings, earliestTimestamp, latestTimestamp time.Time, converter *PrometheusConverter) error {
func (c *PrometheusConverter) addResourceTargetInfo(resource pcommon.Resource, settings Settings, earliestTimestamp, latestTimestamp time.Time) error {
if settings.DisableTargetInfo {
return nil
}
@ -679,17 +585,21 @@ func addResourceTargetInfo(resource pcommon.Resource, settings Settings, earlies
// Do not pass identifying attributes as ignoreAttrs below.
identifyingAttrs = nil
}
labels, err := createAttributes(resource, attributes, scope{}, settings, identifyingAttrs, false, prompb.MetricMetadata{}, model.MetricNameLabel, name)
meta := metadata.Metadata{
Type: model.MetricTypeGauge,
Help: "Target metadata",
}
// TODO: should target info have the __type__ metadata label?
lbls, err := c.createAttributes(resource, attributes, scope{}, settings, identifyingAttrs, false, metadata.Metadata{}, model.MetricNameLabel, name)
if err != nil {
return err
}
haveIdentifier := false
for _, l := range labels {
lbls.Range(func(l modelLabels.Label) {
if l.Name == model.JobLabel || l.Name == model.InstanceLabel {
haveIdentifier = true
break
}
}
})
if !haveIdentifier {
// We need at least one identifying label to generate target_info.
@ -704,18 +614,13 @@ func addResourceTargetInfo(resource pcommon.Resource, settings Settings, earlies
settings.LookbackDelta = defaultLookbackDelta
}
interval := settings.LookbackDelta / 2
ts, _ := converter.getOrCreateTimeSeries(labels)
for timestamp := earliestTimestamp; timestamp.Before(latestTimestamp); timestamp = timestamp.Add(interval) {
ts.Samples = append(ts.Samples, prompb.Sample{
Value: float64(1),
Timestamp: timestamp.UnixMilli(),
})
timestamp := earliestTimestamp
for ; timestamp.Before(latestTimestamp); timestamp = timestamp.Add(interval) {
if err := c.appender.AppendSample(name, lbls, meta, timestamp.UnixMilli(), 0, float64(1), nil); err != nil {
return err
}
}
ts.Samples = append(ts.Samples, prompb.Sample{
Value: float64(1),
Timestamp: latestTimestamp.UnixMilli(),
})
return nil
return c.appender.AppendSample(name, lbls, meta, latestTimestamp.UnixMilli(), 0, float64(1), nil)
}
// convertTimeStamp converts OTLP timestamp in ns to timestamp in ms.

View File

@ -26,8 +26,8 @@ import (
"go.opentelemetry.io/collector/pdata/pmetric"
"github.com/prometheus/prometheus/model/histogram"
"github.com/prometheus/prometheus/model/metadata"
"github.com/prometheus/prometheus/model/value"
"github.com/prometheus/prometheus/prompb"
"github.com/prometheus/prometheus/util/annotations"
)
@ -36,7 +36,8 @@ const defaultZeroThreshold = 1e-128
// addExponentialHistogramDataPoints adds OTel exponential histogram data points to the corresponding time series
// as native histogram samples.
func (c *PrometheusConverter) addExponentialHistogramDataPoints(ctx context.Context, dataPoints pmetric.ExponentialHistogramDataPointSlice,
resource pcommon.Resource, settings Settings, metadata prompb.MetricMetadata, temporality pmetric.AggregationTemporality, scope scope,
resource pcommon.Resource, settings Settings, promName string, temporality pmetric.AggregationTemporality,
scope scope, meta metadata.Metadata,
) (annotations.Annotations, error) {
var annots annotations.Annotations
for x := 0; x < dataPoints.Len(); x++ {
@ -46,34 +47,36 @@ func (c *PrometheusConverter) addExponentialHistogramDataPoints(ctx context.Cont
pt := dataPoints.At(x)
histogram, ws, err := exponentialToNativeHistogram(pt, temporality)
hp, ws, err := exponentialToNativeHistogram(pt, temporality)
annots.Merge(ws)
if err != nil {
return annots, err
}
lbls, err := createAttributes(
lbls, err := c.createAttributes(
resource,
pt.Attributes(),
scope,
settings,
nil,
true,
metadata,
meta,
model.MetricNameLabel,
metadata.MetricFamilyName,
promName,
)
if err != nil {
return nil, err
}
ts, _ := c.getOrCreateTimeSeries(lbls)
ts.Histograms = append(ts.Histograms, histogram)
exemplars, err := getPromExemplars[pmetric.ExponentialHistogramDataPoint](ctx, &c.everyN, pt)
if err != nil {
return annots, err
}
ts.Exemplars = append(ts.Exemplars, exemplars...)
ts := convertTimeStamp(pt.Timestamp())
ct := convertTimeStamp(pt.StartTimestamp())
exemplars, err := c.getPromExemplars(ctx, pt.Exemplars())
if err != nil {
return annots, err
}
// OTel exponential histograms are always Int Histograms.
if err = c.appender.AppendHistogram(promName, lbls, meta, ts, ct, hp, exemplars); err != nil {
return annots, err
}
}
return annots, nil
@ -81,11 +84,11 @@ func (c *PrometheusConverter) addExponentialHistogramDataPoints(ctx context.Cont
// exponentialToNativeHistogram translates an OTel Exponential Histogram data point
// to a Prometheus Native Histogram.
func exponentialToNativeHistogram(p pmetric.ExponentialHistogramDataPoint, temporality pmetric.AggregationTemporality) (prompb.Histogram, annotations.Annotations, error) {
func exponentialToNativeHistogram(p pmetric.ExponentialHistogramDataPoint, temporality pmetric.AggregationTemporality) (*histogram.Histogram, annotations.Annotations, error) {
var annots annotations.Annotations
scale := p.Scale()
if scale < -4 {
return prompb.Histogram{}, annots,
return nil, annots,
fmt.Errorf("cannot convert exponential to native histogram."+
" Scale must be >= -4, was %d", scale)
}
@ -108,41 +111,36 @@ func exponentialToNativeHistogram(p pmetric.ExponentialHistogramDataPoint, tempo
// need to know here if it was used for the detection.
// Ref: https://github.com/open-telemetry/opentelemetry-collector-contrib/pull/28663#issuecomment-1810577303
// Counter reset detection in Prometheus: https://github.com/prometheus/prometheus/blob/f997c72f294c0f18ca13fa06d51889af04135195/tsdb/chunkenc/histogram.go#L232
resetHint := prompb.Histogram_UNKNOWN
resetHint := histogram.UnknownCounterReset
if temporality == pmetric.AggregationTemporalityDelta {
// If the histogram has delta temporality, set the reset hint to gauge to avoid unnecessary chunk cutting.
// We're in an early phase of implementing delta support (proposal: https://github.com/prometheus/proposals/pull/48/).
// This might be changed to a different hint name as gauge type might be misleading for samples that should be
// summed over time.
resetHint = prompb.Histogram_GAUGE
resetHint = histogram.GaugeType
}
h := prompb.Histogram{
ResetHint: resetHint,
Schema: scale,
ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: p.ZeroCount()},
h := &histogram.Histogram{
CounterResetHint: resetHint,
Schema: scale,
// TODO use zero_threshold, if set, see
// https://github.com/open-telemetry/opentelemetry-proto/pull/441
ZeroThreshold: defaultZeroThreshold,
PositiveSpans: pSpans,
PositiveDeltas: pDeltas,
NegativeSpans: nSpans,
NegativeDeltas: nDeltas,
Timestamp: convertTimeStamp(p.Timestamp()),
ZeroThreshold: defaultZeroThreshold,
ZeroCount: p.ZeroCount(),
PositiveSpans: pSpans,
PositiveBuckets: pDeltas,
NegativeSpans: nSpans,
NegativeBuckets: nDeltas,
}
if p.Flags().NoRecordedValue() {
h.Sum = math.Float64frombits(value.StaleNaN)
h.Count = &prompb.Histogram_CountInt{CountInt: value.StaleNaN}
h.Count = value.StaleNaN
} else {
if p.HasSum() {
h.Sum = p.Sum()
}
h.Count = &prompb.Histogram_CountInt{CountInt: p.Count()}
h.Count = p.Count()
if p.Count() == 0 && h.Sum != 0 {
annots.Add(fmt.Errorf("exponential histogram data point has zero count, but non-zero sum: %f", h.Sum))
}
@ -167,13 +165,13 @@ func exponentialToNativeHistogram(p pmetric.ExponentialHistogramDataPoint, tempo
//
// When converting from OTel Explicit Histograms to Native Histograms with Custom Buckets,
// the bucket indexes are not scaled, and the indices are not adjusted by 1.
func convertBucketsLayout(bucketCounts []uint64, offset, scaleDown int32, adjustOffset bool) ([]prompb.BucketSpan, []int64) {
func convertBucketsLayout(bucketCounts []uint64, offset, scaleDown int32, adjustOffset bool) ([]histogram.Span, []int64) {
if len(bucketCounts) == 0 {
return nil, nil
}
var (
spans []prompb.BucketSpan
spans []histogram.Span
deltas []int64
count int64
prevCount int64
@ -196,7 +194,7 @@ func convertBucketsLayout(bucketCounts []uint64, offset, scaleDown int32, adjust
initialOffset = initialOffset>>scaleDown + 1
}
spans = append(spans, prompb.BucketSpan{
spans = append(spans, histogram.Span{
Offset: initialOffset,
Length: 0,
})
@ -217,7 +215,7 @@ func convertBucketsLayout(bucketCounts []uint64, offset, scaleDown int32, adjust
// We have to create a new span, because we have found a gap
// of more than two buckets. The constant 2 is copied from the logic in
// https://github.com/prometheus/client_golang/blob/27f0506d6ebbb117b6b697d0552ee5be2502c5f2/prometheus/histogram.go#L1296
spans = append(spans, prompb.BucketSpan{
spans = append(spans, histogram.Span{
Offset: gap,
Length: 0,
})
@ -239,7 +237,7 @@ func convertBucketsLayout(bucketCounts []uint64, offset, scaleDown int32, adjust
// We have to create a new span, because we have found a gap
// of more than two buckets. The constant 2 is copied from the logic in
// https://github.com/prometheus/client_golang/blob/27f0506d6ebbb117b6b697d0552ee5be2502c5f2/prometheus/histogram.go#L1296
spans = append(spans, prompb.BucketSpan{
spans = append(spans, histogram.Span{
Offset: gap,
Length: 0,
})
@ -256,7 +254,8 @@ func convertBucketsLayout(bucketCounts []uint64, offset, scaleDown int32, adjust
}
func (c *PrometheusConverter) addCustomBucketsHistogramDataPoints(ctx context.Context, dataPoints pmetric.HistogramDataPointSlice,
resource pcommon.Resource, settings Settings, metadata prompb.MetricMetadata, temporality pmetric.AggregationTemporality, scope scope,
resource pcommon.Resource, settings Settings, promName string, temporality pmetric.AggregationTemporality,
scope scope, meta metadata.Metadata,
) (annotations.Annotations, error) {
var annots annotations.Annotations
@ -267,41 +266,41 @@ func (c *PrometheusConverter) addCustomBucketsHistogramDataPoints(ctx context.Co
pt := dataPoints.At(x)
histogram, ws, err := explicitHistogramToCustomBucketsHistogram(pt, temporality)
hp, ws, err := explicitHistogramToCustomBucketsHistogram(pt, temporality)
annots.Merge(ws)
if err != nil {
return annots, err
}
lbls, err := createAttributes(
lbls, err := c.createAttributes(
resource,
pt.Attributes(),
scope,
settings,
nil,
true,
metadata,
meta,
model.MetricNameLabel,
metadata.MetricFamilyName,
promName,
)
if err != nil {
return nil, err
}
ts, _ := c.getOrCreateTimeSeries(lbls)
ts.Histograms = append(ts.Histograms, histogram)
exemplars, err := getPromExemplars[pmetric.HistogramDataPoint](ctx, &c.everyN, pt)
if err != nil {
return annots, err
}
ts.Exemplars = append(ts.Exemplars, exemplars...)
ts := convertTimeStamp(pt.Timestamp())
ct := convertTimeStamp(pt.StartTimestamp())
exemplars, err := c.getPromExemplars(ctx, pt.Exemplars())
if err != nil {
return annots, err
}
if err = c.appender.AppendHistogram(promName, lbls, meta, ts, ct, hp, exemplars); err != nil {
return annots, err
}
}
return annots, nil
}
func explicitHistogramToCustomBucketsHistogram(p pmetric.HistogramDataPoint, temporality pmetric.AggregationTemporality) (prompb.Histogram, annotations.Annotations, error) {
func explicitHistogramToCustomBucketsHistogram(p pmetric.HistogramDataPoint, temporality pmetric.AggregationTemporality) (*histogram.Histogram, annotations.Annotations, error) {
var annots annotations.Annotations
buckets := p.BucketCounts().AsRaw()
@ -318,23 +317,22 @@ func explicitHistogramToCustomBucketsHistogram(p pmetric.HistogramDataPoint, tem
// need to know here if it was used for the detection.
// Ref: https://github.com/open-telemetry/opentelemetry-collector-contrib/pull/28663#issuecomment-1810577303
// Counter reset detection in Prometheus: https://github.com/prometheus/prometheus/blob/f997c72f294c0f18ca13fa06d51889af04135195/tsdb/chunkenc/histogram.go#L232
resetHint := prompb.Histogram_UNKNOWN
resetHint := histogram.UnknownCounterReset
if temporality == pmetric.AggregationTemporalityDelta {
// If the histogram has delta temporality, set the reset hint to gauge to avoid unnecessary chunk cutting.
// We're in an early phase of implementing delta support (proposal: https://github.com/prometheus/proposals/pull/48/).
// This might be changed to a different hint name as gauge type might be misleading for samples that should be
// summed over time.
resetHint = prompb.Histogram_GAUGE
resetHint = histogram.GaugeType
}
// TODO(carrieedwards): Add setting to limit maximum bucket count
h := prompb.Histogram{
ResetHint: resetHint,
Schema: histogram.CustomBucketsSchema,
PositiveSpans: positiveSpans,
PositiveDeltas: positiveDeltas,
h := &histogram.Histogram{
CounterResetHint: resetHint,
Schema: histogram.CustomBucketsSchema,
PositiveSpans: positiveSpans,
PositiveBuckets: positiveDeltas,
// Note: OTel explicit histograms have an implicit +Inf bucket, which has a lower bound
// of the last element in the explicit_bounds array.
// This is similar to the custom_values array in native histograms with custom buckets.
@ -342,18 +340,16 @@ func explicitHistogramToCustomBucketsHistogram(p pmetric.HistogramDataPoint, tem
// can be mapped directly to the custom_values array.
// See: https://github.com/open-telemetry/opentelemetry-proto/blob/d7770822d70c7bd47a6891fc9faacc66fc4af3d3/opentelemetry/proto/metrics/v1/metrics.proto#L469
CustomValues: p.ExplicitBounds().AsRaw(),
Timestamp: convertTimeStamp(p.Timestamp()),
}
if p.Flags().NoRecordedValue() {
h.Sum = math.Float64frombits(value.StaleNaN)
h.Count = &prompb.Histogram_CountInt{CountInt: value.StaleNaN}
h.Count = value.StaleNaN
} else {
if p.HasSum() {
h.Sum = p.Sum()
}
h.Count = &prompb.Histogram_CountInt{CountInt: p.Count()}
h.Count = p.Count()
if p.Count() == 0 && h.Sum != 0 {
annots.Add(fmt.Errorf("histogram data point has zero count, but non-zero sum: %f", h.Sum))
}

View File

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

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"
"fmt"
"math"
"sort"
"time"
"github.com/prometheus/otlptranslator"
@ -30,7 +29,8 @@ import (
"go.uber.org/multierr"
"github.com/prometheus/prometheus/config"
"github.com/prometheus/prometheus/prompb"
"github.com/prometheus/prometheus/model/metadata"
"github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite/labels"
"github.com/prometheus/prometheus/util/annotations"
)
@ -59,16 +59,21 @@ type Settings struct {
// PrometheusConverter converts from OTel write format to Prometheus remote write format.
type PrometheusConverter struct {
unique map[uint64]*prompb.TimeSeries
conflicts map[uint64][]*prompb.TimeSeries
everyN everyNTimes
metadata []prompb.MetricMetadata
unique map[uint64]labels.Labels
conflicts map[uint64][]labels.Labels
everyN everyNTimes
scratchBuilder labels.ScratchBuilder
builder *labels.Builder
appender CombinedAppender
}
func NewPrometheusConverter() *PrometheusConverter {
func NewPrometheusConverter(appender CombinedAppender) *PrometheusConverter {
return &PrometheusConverter{
unique: map[uint64]*prompb.TimeSeries{},
conflicts: map[uint64][]*prompb.TimeSeries{},
unique: map[uint64]labels.Labels{},
conflicts: map[uint64][]labels.Labels{},
scratchBuilder: labels.NewScratchBuilder(0),
builder: labels.NewBuilder(labels.EmptyLabels()),
appender: appender,
}
}
@ -121,6 +126,7 @@ func (c *PrometheusConverter) FromMetrics(ctx context.Context, md pmetric.Metric
WithMetricSuffixes: settings.AddMetricSuffixes,
UTF8Allowed: settings.AllowUTF8,
}
unitNamer := otlptranslator.UnitNamer{}
c.everyN = everyNTimes{n: 128}
resourceMetricsSlice := md.ResourceMetrics()
@ -131,7 +137,6 @@ func (c *PrometheusConverter) FromMetrics(ctx context.Context, md pmetric.Metric
numMetrics += scopeMetricsSlice.At(j).Metrics().Len()
}
}
c.metadata = make([]prompb.MetricMetadata, 0, numMetrics)
for i := 0; i < resourceMetricsSlice.Len(); i++ {
resourceMetrics := resourceMetricsSlice.At(i)
@ -176,13 +181,11 @@ func (c *PrometheusConverter) FromMetrics(ctx context.Context, md pmetric.Metric
errs = multierr.Append(errs, err)
continue
}
metadata := prompb.MetricMetadata{
Type: otelMetricTypeToPromMetricType(metric),
MetricFamilyName: promName,
Help: metric.Description(),
Unit: metric.Unit(),
meta := metadata.Metadata{
Type: otelMetricTypeToPromMetricType(metric),
Unit: unitNamer.Build(metric.Unit()),
Help: metric.Description(),
}
c.metadata = append(c.metadata, metadata)
// handle individual metrics based on type
//exhaustive:enforce
@ -193,7 +196,7 @@ func (c *PrometheusConverter) FromMetrics(ctx context.Context, md pmetric.Metric
errs = multierr.Append(errs, fmt.Errorf("empty data points. %s is dropped", metric.Name()))
break
}
if err := c.addGaugeNumberDataPoints(ctx, dataPoints, resource, settings, metadata, scope); err != nil {
if err := c.addGaugeNumberDataPoints(ctx, dataPoints, resource, settings, promName, scope, meta); err != nil {
errs = multierr.Append(errs, err)
if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) {
return
@ -205,7 +208,7 @@ func (c *PrometheusConverter) FromMetrics(ctx context.Context, md pmetric.Metric
errs = multierr.Append(errs, fmt.Errorf("empty data points. %s is dropped", metric.Name()))
break
}
if err := c.addSumNumberDataPoints(ctx, dataPoints, resource, metric, settings, metadata, scope); err != nil {
if err := c.addSumNumberDataPoints(ctx, dataPoints, resource, metric, settings, promName, scope, meta); err != nil {
errs = multierr.Append(errs, err)
if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) {
return
@ -219,7 +222,7 @@ func (c *PrometheusConverter) FromMetrics(ctx context.Context, md pmetric.Metric
}
if settings.ConvertHistogramsToNHCB {
ws, err := c.addCustomBucketsHistogramDataPoints(
ctx, dataPoints, resource, settings, metadata, temporality, scope,
ctx, dataPoints, resource, settings, promName, temporality, scope, meta,
)
annots.Merge(ws)
if err != nil {
@ -229,7 +232,7 @@ func (c *PrometheusConverter) FromMetrics(ctx context.Context, md pmetric.Metric
}
}
} else {
if err := c.addHistogramDataPoints(ctx, dataPoints, resource, settings, metadata, scope); err != nil {
if err := c.addHistogramDataPoints(ctx, dataPoints, resource, settings, promName, scope, meta); err != nil {
errs = multierr.Append(errs, err)
if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) {
return
@ -247,9 +250,10 @@ func (c *PrometheusConverter) FromMetrics(ctx context.Context, md pmetric.Metric
dataPoints,
resource,
settings,
metadata,
promName,
temporality,
scope,
meta,
)
annots.Merge(ws)
if err != nil {
@ -264,7 +268,7 @@ func (c *PrometheusConverter) FromMetrics(ctx context.Context, md pmetric.Metric
errs = multierr.Append(errs, fmt.Errorf("empty data points. %s is dropped", metric.Name()))
break
}
if err := c.addSummaryDataPoints(ctx, dataPoints, resource, settings, metadata, scope); err != nil {
if err := c.addSummaryDataPoints(ctx, dataPoints, resource, settings, promName, scope, meta); err != nil {
errs = multierr.Append(errs, err)
if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) {
return
@ -278,72 +282,16 @@ func (c *PrometheusConverter) FromMetrics(ctx context.Context, md pmetric.Metric
if earliestTimestamp < pcommon.Timestamp(math.MaxUint64) {
// We have at least one metric sample for this resource.
// Generate a corresponding target_info series.
err := addResourceTargetInfo(resource, settings, earliestTimestamp.AsTime(), latestTimestamp.AsTime(), c)
if err != nil {
if err := c.addResourceTargetInfo(resource, settings, earliestTimestamp.AsTime(), latestTimestamp.AsTime()); err != nil {
errs = multierr.Append(errs, err)
}
}
}
return annots, errs
}
err := c.appender.Commit()
errs = multierr.Append(errs, err)
func isSameMetric(ts *prompb.TimeSeries, lbls []prompb.Label) bool {
if len(ts.Labels) != len(lbls) {
return false
}
for i, l := range ts.Labels {
if l.Name != ts.Labels[i].Name || l.Value != ts.Labels[i].Value {
return false
}
}
return true
}
// addExemplars adds exemplars for the dataPoint. For each exemplar, if it can find a bucket bound corresponding to its value,
// the exemplar is added to the bucket bound's time series, provided that the time series' has samples.
func (c *PrometheusConverter) addExemplars(ctx context.Context, dataPoint pmetric.HistogramDataPoint, bucketBounds []bucketBoundsData) error {
if len(bucketBounds) == 0 {
return nil
}
exemplars, err := getPromExemplars(ctx, &c.everyN, dataPoint)
if err != nil {
return err
}
if len(exemplars) == 0 {
return nil
}
sort.Sort(byBucketBoundsData(bucketBounds))
for _, exemplar := range exemplars {
for _, bound := range bucketBounds {
if err := c.everyN.checkContext(ctx); err != nil {
return err
}
if len(bound.ts.Samples) > 0 && exemplar.Value <= bound.bound {
bound.ts.Exemplars = append(bound.ts.Exemplars, exemplar)
break
}
}
}
return nil
}
// addSample finds a TimeSeries that corresponds to lbls, and adds sample to it.
// If there is no corresponding TimeSeries already, it's created.
// The corresponding TimeSeries is returned.
// If either lbls is nil/empty or sample is nil, nothing is done.
func (c *PrometheusConverter) addSample(sample *prompb.Sample, lbls []prompb.Label) *prompb.TimeSeries {
if sample == nil || len(lbls) == 0 {
// This shouldn't happen
return nil
}
ts, _ := c.getOrCreateTimeSeries(lbls)
ts.Samples = append(ts.Samples, *sample)
return ts
return
}
func NewPromoteResourceAttributes(otlpCfg config.OTLPConfig) *PromoteResourceAttributes {
@ -361,30 +309,43 @@ func NewPromoteResourceAttributes(otlpCfg config.OTLPConfig) *PromoteResourceAtt
}
}
// promotedAttributes returns labels for promoted resourceAttributes.
func (s *PromoteResourceAttributes) promotedAttributes(resourceAttributes pcommon.Map) []prompb.Label {
// addPromotedAttributes adds labels for promoted resourceAttributes to the builder.
func (s *PromoteResourceAttributes) addPromotedAttributes(builder *labels.Builder, resourceAttributes pcommon.Map, allowUTF8 bool) error {
if s == nil {
return nil
}
var promotedAttrs []prompb.Label
labelNamer := otlptranslator.LabelNamer{UTF8Allowed: allowUTF8}
if s.promoteAll {
promotedAttrs = make([]prompb.Label, 0, resourceAttributes.Len())
var err error
resourceAttributes.Range(func(name string, value pcommon.Value) bool {
if _, exists := s.attrs[name]; !exists {
promotedAttrs = append(promotedAttrs, prompb.Label{Name: name, Value: value.AsString()})
}
return true
})
} else {
promotedAttrs = make([]prompb.Label, 0, len(s.attrs))
resourceAttributes.Range(func(name string, value pcommon.Value) bool {
if _, exists := s.attrs[name]; exists {
promotedAttrs = append(promotedAttrs, prompb.Label{Name: name, Value: value.AsString()})
var normalized string
normalized, err = labelNamer.Build(name)
if err != nil {
return false
}
if builder.Get(normalized) == "" {
builder.Set(normalized, value.AsString())
}
}
return true
})
return err
}
sort.Stable(ByLabelName(promotedAttrs))
return promotedAttrs
var err error
resourceAttributes.Range(func(name string, value pcommon.Value) bool {
if _, exists := s.attrs[name]; exists {
var normalized string
normalized, err = labelNamer.Build(name)
if err != nil {
return false
}
if builder.Get(normalized) == "" {
builder.Set(normalized, value.AsString())
}
}
return true
})
return err
}

View File

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

View File

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

View File

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

View File

@ -17,42 +17,41 @@
package prometheusremotewrite
import (
"github.com/prometheus/common/model"
"go.opentelemetry.io/collector/pdata/pmetric"
"github.com/prometheus/prometheus/prompb"
)
func otelMetricTypeToPromMetricType(otelMetric pmetric.Metric) prompb.MetricMetadata_MetricType {
func otelMetricTypeToPromMetricType(otelMetric pmetric.Metric) model.MetricType {
switch otelMetric.Type() {
case pmetric.MetricTypeGauge:
return prompb.MetricMetadata_GAUGE
return model.MetricTypeGauge
case pmetric.MetricTypeSum:
metricType := prompb.MetricMetadata_GAUGE
metricType := model.MetricTypeGauge
if otelMetric.Sum().IsMonotonic() {
metricType = prompb.MetricMetadata_COUNTER
metricType = model.MetricTypeCounter
}
// We're in an early phase of implementing delta support (proposal: https://github.com/prometheus/proposals/pull/48/)
// We don't have a proper way to flag delta metrics yet, therefore marking the metric type as unknown for now.
if otelMetric.Sum().AggregationTemporality() == pmetric.AggregationTemporalityDelta {
metricType = prompb.MetricMetadata_UNKNOWN
metricType = model.MetricTypeUnknown
}
return metricType
case pmetric.MetricTypeHistogram:
// We're in an early phase of implementing delta support (proposal: https://github.com/prometheus/proposals/pull/48/)
// We don't have a proper way to flag delta metrics yet, therefore marking the metric type as unknown for now.
if otelMetric.Histogram().AggregationTemporality() == pmetric.AggregationTemporalityDelta {
return prompb.MetricMetadata_UNKNOWN
return model.MetricTypeUnknown
}
return prompb.MetricMetadata_HISTOGRAM
return model.MetricTypeHistogram
case pmetric.MetricTypeSummary:
return prompb.MetricMetadata_SUMMARY
return model.MetricTypeSummary
case pmetric.MetricTypeExponentialHistogram:
if otelMetric.ExponentialHistogram().AggregationTemporality() == pmetric.AggregationTemporalityDelta {
// We're in an early phase of implementing delta support (proposal: https://github.com/prometheus/proposals/pull/48/)
// We don't have a proper way to flag delta metrics yet, therefore marking the metric type as unknown for now.
return prompb.MetricMetadata_UNKNOWN
return model.MetricTypeUnknown
}
return prompb.MetricMetadata_HISTOGRAM
return model.MetricTypeHistogram
}
return prompb.MetricMetadata_UNKNOWN
return model.MetricTypeUnknown
}

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

View File

@ -44,6 +44,7 @@ import (
"github.com/prometheus/prometheus/config"
"github.com/prometheus/prometheus/model/histogram"
"github.com/prometheus/prometheus/model/labels"
"github.com/prometheus/prometheus/model/metadata"
"github.com/prometheus/prometheus/model/relabel"
"github.com/prometheus/prometheus/storage"
)
@ -381,14 +382,16 @@ func TestWriteStorageApplyConfig_PartialUpdate(t *testing.T) {
require.NoError(t, s.Close())
}
func TestOTLPWriteHandler(t *testing.T) {
func TestOTLPWriteHandlerNaming(t *testing.T) {
timestamp := time.Now()
exportRequest := generateOTLPWriteRequest(timestamp)
var zeroTime time.Time
exportRequest := generateOTLPWriteRequest(timestamp, zeroTime)
for _, testCase := range []struct {
name string
otlpCfg config.OTLPConfig
typeAndUnitLabels bool
expectedSamples []mockSample
expectedMetadata []mockMetadata
}{
{
name: "NoTranslation/NoTypeAndUnitLabels",
@ -397,24 +400,70 @@ func TestOTLPWriteHandler(t *testing.T) {
},
expectedSamples: []mockSample{
{
l: labels.New(labels.Label{Name: "__name__", Value: "test.counter"},
labels.Label{Name: "foo.bar", Value: "baz"},
labels.Label{Name: "instance", Value: "test-instance"},
labels.Label{Name: "job", Value: "test-service"}),
l: labels.FromStrings(model.MetricNameLabel, "test.counter", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"),
t: timestamp.UnixMilli(),
v: 10.0,
},
{
l: labels.New(
labels.Label{Name: "__name__", Value: "target_info"},
labels.Label{Name: "host.name", Value: "test-host"},
labels.Label{Name: "instance", Value: "test-instance"},
labels.Label{Name: "job", Value: "test-service"},
),
l: labels.FromStrings(model.MetricNameLabel, "target_info", "host.name", "test-host", "instance", "test-instance", "job", "test-service"),
t: timestamp.UnixMilli(),
v: 1,
},
},
expectedMetadata: []mockMetadata{
{
l: labels.FromStrings(model.MetricNameLabel, "test.counter", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeCounter, Unit: "bytes", Help: "test-counter-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.gauge", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeGauge, Unit: "bytes", Help: "test-gauge-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_sum", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_count", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "0"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "1"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "2"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "3"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "4"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "5"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "+Inf"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.exponential.histogram", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-exponential-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "target_info", "host.name", "test-host", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeGauge, Unit: "", Help: "Target metadata"},
},
},
},
{
name: "NoTranslation/WithTypeAndUnitLabels",
@ -424,26 +473,71 @@ func TestOTLPWriteHandler(t *testing.T) {
typeAndUnitLabels: true,
expectedSamples: []mockSample{
{
l: labels.New(labels.Label{Name: "__name__", Value: "test.counter"},
labels.Label{Name: "__type__", Value: "counter"},
labels.Label{Name: "__unit__", Value: "bytes"},
labels.Label{Name: "foo.bar", Value: "baz"},
labels.Label{Name: "instance", Value: "test-instance"},
labels.Label{Name: "job", Value: "test-service"}),
l: labels.FromStrings(model.MetricNameLabel, "test.counter", "__type__", "counter", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"),
t: timestamp.UnixMilli(),
v: 10.0,
},
{
l: labels.New(
labels.Label{Name: "__name__", Value: "target_info"},
labels.Label{Name: "host.name", Value: "test-host"},
labels.Label{Name: "instance", Value: "test-instance"},
labels.Label{Name: "job", Value: "test-service"},
),
l: labels.FromStrings(model.MetricNameLabel, "target_info", "host.name", "test-host", "instance", "test-instance", "job", "test-service"),
t: timestamp.UnixMilli(),
v: 1,
},
},
expectedMetadata: []mockMetadata{
{
// Metadata labels follow series labels.
l: labels.FromStrings(model.MetricNameLabel, "test.counter", "__type__", "counter", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeCounter, Unit: "bytes", Help: "test-counter-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.gauge", "__type__", "gauge", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeGauge, Unit: "bytes", Help: "test-gauge-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_sum", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_count", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "0"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "1"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "2"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "3"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "4"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "5"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "+Inf"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.exponential.histogram", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-exponential-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "target_info", "host.name", "test-host", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeGauge, Unit: "", Help: "Target metadata"},
},
},
},
{
name: "UnderscoreEscapingWithSuffixes/NoTypeAndUnitLabels",
@ -452,24 +546,71 @@ func TestOTLPWriteHandler(t *testing.T) {
},
expectedSamples: []mockSample{
{
l: labels.New(labels.Label{Name: "__name__", Value: "test_counter_bytes_total"},
labels.Label{Name: "foo_bar", Value: "baz"},
labels.Label{Name: "instance", Value: "test-instance"},
labels.Label{Name: "job", Value: "test-service"}),
l: labels.FromStrings(model.MetricNameLabel, "test_counter_bytes_total", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"),
t: timestamp.UnixMilli(),
v: 10.0,
},
{
l: labels.New(
labels.Label{Name: "__name__", Value: "target_info"},
labels.Label{Name: "host_name", Value: "test-host"},
labels.Label{Name: "instance", Value: "test-instance"},
labels.Label{Name: "job", Value: "test-service"},
),
l: labels.FromStrings(model.MetricNameLabel, "target_info", "host_name", "test-host", "instance", "test-instance", "job", "test-service"),
t: timestamp.UnixMilli(),
v: 1,
},
},
expectedMetadata: []mockMetadata{
// All get _bytes unit suffix and counter also gets _total.
{
l: labels.FromStrings(model.MetricNameLabel, "test_counter_bytes_total", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeCounter, Unit: "bytes", Help: "test-counter-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test_gauge_bytes", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeGauge, Unit: "bytes", Help: "test-gauge-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_sum", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_count", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_bucket", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "0"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_bucket", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "1"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_bucket", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "2"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_bucket", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "3"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_bucket", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "4"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_bucket", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "5"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_bucket", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "+Inf"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test_exponential_histogram_bytes", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-exponential-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "target_info", "host_name", "test-host", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeGauge, Unit: "", Help: "Target metadata"},
},
},
},
{
name: "UnderscoreEscapingWithoutSuffixes",
@ -478,24 +619,70 @@ func TestOTLPWriteHandler(t *testing.T) {
},
expectedSamples: []mockSample{
{
l: labels.New(labels.Label{Name: "__name__", Value: "test_counter"},
labels.Label{Name: "foo_bar", Value: "baz"},
labels.Label{Name: "instance", Value: "test-instance"},
labels.Label{Name: "job", Value: "test-service"}),
l: labels.FromStrings(model.MetricNameLabel, "test_counter", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"),
t: timestamp.UnixMilli(),
v: 10.0,
},
{
l: labels.New(
labels.Label{Name: "__name__", Value: "target_info"},
labels.Label{Name: "host_name", Value: "test-host"},
labels.Label{Name: "instance", Value: "test-instance"},
labels.Label{Name: "job", Value: "test-service"},
),
l: labels.FromStrings(model.MetricNameLabel, "target_info", "host_name", "test-host", "instance", "test-instance", "job", "test-service"),
t: timestamp.UnixMilli(),
v: 1,
},
},
expectedMetadata: []mockMetadata{
{
l: labels.FromStrings(model.MetricNameLabel, "test_counter", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeCounter, Unit: "bytes", Help: "test-counter-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test_gauge", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeGauge, Unit: "bytes", Help: "test-gauge-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test_histogram_sum", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test_histogram_count", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bucket", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "0"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bucket", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "1"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bucket", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "2"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bucket", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "3"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bucket", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "4"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bucket", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "5"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bucket", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "+Inf"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test_exponential_histogram", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-exponential-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "target_info", "host_name", "test-host", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeGauge, Unit: "", Help: "Target metadata"},
},
},
},
{
name: "UnderscoreEscapingWithSuffixes/WithTypeAndUnitLabels",
@ -525,6 +712,60 @@ func TestOTLPWriteHandler(t *testing.T) {
v: 1,
},
},
expectedMetadata: []mockMetadata{
{
l: labels.FromStrings(model.MetricNameLabel, "test_counter_bytes_total", "__type__", "counter", "__unit__", "bytes", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeCounter, Unit: "bytes", Help: "test-counter-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test_gauge_bytes", "__type__", "gauge", "__unit__", "bytes", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeGauge, Unit: "bytes", Help: "test-gauge-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_sum", "__type__", "histogram", "__unit__", "bytes", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_count", "__type__", "histogram", "__unit__", "bytes", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_bucket", "__type__", "histogram", "__unit__", "bytes", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "0"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_bucket", "__type__", "histogram", "__unit__", "bytes", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "1"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_bucket", "__type__", "histogram", "__unit__", "bytes", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "2"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_bucket", "__type__", "histogram", "__unit__", "bytes", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "3"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_bucket", "__type__", "histogram", "__unit__", "bytes", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "4"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_bucket", "__type__", "histogram", "__unit__", "bytes", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "5"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test_histogram_bytes_bucket", "__type__", "histogram", "__unit__", "bytes", "foo_bar", "baz", "instance", "test-instance", "job", "test-service", "le", "+Inf"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test_exponential_histogram_bytes", "__type__", "histogram", "__unit__", "bytes", "foo_bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-exponential-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "target_info", "host_name", "test-host", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeGauge, Unit: "", Help: "Target metadata"},
},
},
},
{
name: "NoUTF8EscapingWithSuffixes/NoTypeAndUnitLabels",
@ -533,24 +774,71 @@ func TestOTLPWriteHandler(t *testing.T) {
},
expectedSamples: []mockSample{
{
l: labels.New(labels.Label{Name: "__name__", Value: "test.counter_bytes_total"},
labels.Label{Name: "foo.bar", Value: "baz"},
labels.Label{Name: "instance", Value: "test-instance"},
labels.Label{Name: "job", Value: "test-service"}),
l: labels.FromStrings(model.MetricNameLabel, "test.counter_bytes_total", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"),
t: timestamp.UnixMilli(),
v: 10.0,
},
{
l: labels.New(
labels.Label{Name: "__name__", Value: "target_info"},
labels.Label{Name: "host.name", Value: "test-host"},
labels.Label{Name: "instance", Value: "test-instance"},
labels.Label{Name: "job", Value: "test-service"},
),
l: labels.FromStrings(model.MetricNameLabel, "target_info", "host.name", "test-host", "instance", "test-instance", "job", "test-service"),
t: timestamp.UnixMilli(),
v: 1,
},
},
expectedMetadata: []mockMetadata{
// All get _bytes unit suffix and counter also gets _total.
{
l: labels.FromStrings(model.MetricNameLabel, "test.counter_bytes_total", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeCounter, Unit: "bytes", Help: "test-counter-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.gauge_bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeGauge, Unit: "bytes", Help: "test-gauge-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_sum", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_count", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "0"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "1"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "2"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "3"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "4"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "5"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "+Inf"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.exponential.histogram_bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-exponential-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "target_info", "host.name", "test-host", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeGauge, Unit: "", Help: "Target metadata"},
},
},
},
{
name: "NoUTF8EscapingWithSuffixes/WithTypeAndUnitLabels",
@ -560,40 +848,268 @@ func TestOTLPWriteHandler(t *testing.T) {
typeAndUnitLabels: true,
expectedSamples: []mockSample{
{
l: labels.New(labels.Label{Name: "__name__", Value: "test.counter_bytes_total"},
labels.Label{Name: "__type__", Value: "counter"},
labels.Label{Name: "__unit__", Value: "bytes"},
labels.Label{Name: "foo.bar", Value: "baz"},
labels.Label{Name: "instance", Value: "test-instance"},
labels.Label{Name: "job", Value: "test-service"}),
l: labels.FromStrings(model.MetricNameLabel, "test.counter_bytes_total", "__type__", "counter", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"),
t: timestamp.UnixMilli(),
v: 10.0,
},
{
l: labels.New(
labels.Label{Name: "__name__", Value: "target_info"},
labels.Label{Name: "host.name", Value: "test-host"},
labels.Label{Name: "instance", Value: "test-instance"},
labels.Label{Name: "job", Value: "test-service"},
),
l: labels.FromStrings(model.MetricNameLabel, "target_info", "host.name", "test-host", "instance", "test-instance", "job", "test-service"),
t: timestamp.UnixMilli(),
v: 1,
},
},
expectedMetadata: []mockMetadata{
// All get _bytes unit suffix and counter also gets _total.
{
l: labels.FromStrings(model.MetricNameLabel, "test.counter_bytes_total", "__type__", "counter", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeCounter, Unit: "bytes", Help: "test-counter-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.gauge_bytes", "__type__", "gauge", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeGauge, Unit: "bytes", Help: "test-gauge-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_sum", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_count", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_bucket", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "0"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_bucket", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "1"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_bucket", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "2"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_bucket", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "3"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_bucket", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "4"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_bucket", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "5"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bytes_bucket", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "+Inf"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.exponential.histogram_bytes", "__type__", "histogram", "__unit__", "bytes", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeHistogram, Unit: "bytes", Help: "test-exponential-histogram-description"},
},
{
l: labels.FromStrings(model.MetricNameLabel, "target_info", "host.name", "test-host", "instance", "test-instance", "job", "test-service"),
m: metadata.Metadata{Type: model.MetricTypeGauge, Unit: "", Help: "Target metadata"},
},
},
},
} {
t.Run(testCase.name, func(t *testing.T) {
appendable := handleOTLP(t, exportRequest, testCase.otlpCfg, testCase.typeAndUnitLabels)
otlpOpts := OTLPOptions{
EnableTypeAndUnitLabels: testCase.typeAndUnitLabels,
}
appendable := handleOTLP(t, exportRequest, testCase.otlpCfg, otlpOpts)
for _, sample := range testCase.expectedSamples {
requireContainsSample(t, appendable.samples, sample)
}
for _, meta := range testCase.expectedMetadata {
requireContainsMetadata(t, appendable.metadata, meta)
}
require.Len(t, appendable.samples, 12) // 1 (counter) + 1 (gauge) + 1 (target_info) + 7 (hist_bucket) + 2 (hist_sum, hist_count)
require.Len(t, appendable.histograms, 1) // 1 (exponential histogram)
require.Len(t, appendable.metadata, 13) // for each float and histogram sample
require.Len(t, appendable.exemplars, 1) // 1 (exemplar)
})
}
}
// Check that start time is ingested if ingestCTZeroSample is enabled
// and the start time is actually set (non-zero).
func TestOTLPWRiteHandlerStartTime(t *testing.T) {
timestamp := time.Now()
startTime := timestamp.Add(-1 * time.Millisecond)
var zeroTime time.Time
expectedSamples := []mockSample{
{
l: labels.FromStrings(model.MetricNameLabel, "test.counter", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"),
t: timestamp.UnixMilli(),
v: 10.0,
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.gauge", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"),
t: timestamp.UnixMilli(),
v: 10.0,
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_sum", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"),
t: timestamp.UnixMilli(),
v: 30.0,
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_count", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"),
t: timestamp.UnixMilli(),
v: 12.0,
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "0"),
t: timestamp.UnixMilli(),
v: 2.0,
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "1"),
t: timestamp.UnixMilli(),
v: 4.0,
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "2"),
t: timestamp.UnixMilli(),
v: 6.0,
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "3"),
t: timestamp.UnixMilli(),
v: 8.0,
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "4"),
t: timestamp.UnixMilli(),
v: 10.0,
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "5"),
t: timestamp.UnixMilli(),
v: 12.0,
},
{
l: labels.FromStrings(model.MetricNameLabel, "test.histogram_bucket", "foo.bar", "baz", "instance", "test-instance", "job", "test-service", "le", "+Inf"),
t: timestamp.UnixMilli(),
v: 12.0,
},
{
l: labels.FromStrings(model.MetricNameLabel, "target_info", "host.name", "test-host", "instance", "test-instance", "job", "test-service"),
t: timestamp.UnixMilli(),
v: 1.0,
},
}
expectedHistograms := []mockHistogram{
{
l: labels.FromStrings(model.MetricNameLabel, "test.exponential.histogram", "foo.bar", "baz", "instance", "test-instance", "job", "test-service"),
t: timestamp.UnixMilli(),
h: &histogram.Histogram{
Schema: 2,
ZeroThreshold: 1e-128,
ZeroCount: 2,
Count: 10,
Sum: 30,
PositiveSpans: []histogram.Span{{Offset: 1, Length: 5}},
PositiveBuckets: []int64{2, 0, 0, 0, 0},
},
},
}
expectedSamplesWithCTZero := make([]mockSample, 0, len(expectedSamples)*2-1) // All samples will get CT zero, except target_info.
for _, s := range expectedSamples {
if s.l.Get(model.MetricNameLabel) != "target_info" {
expectedSamplesWithCTZero = append(expectedSamplesWithCTZero, mockSample{
l: s.l.Copy(),
t: startTime.UnixMilli(),
v: 0,
})
}
expectedSamplesWithCTZero = append(expectedSamplesWithCTZero, s)
}
expectedHistogramsWithCTZero := make([]mockHistogram, 0, len(expectedHistograms)*2)
for _, s := range expectedHistograms {
if s.l.Get(model.MetricNameLabel) != "target_info" {
expectedHistogramsWithCTZero = append(expectedHistogramsWithCTZero, mockHistogram{
l: s.l.Copy(),
t: startTime.UnixMilli(),
h: &histogram.Histogram{},
})
}
expectedHistogramsWithCTZero = append(expectedHistogramsWithCTZero, s)
}
for _, testCase := range []struct {
name string
otlpOpts OTLPOptions
startTime time.Time
expectCTZero bool
expectedSamples []mockSample
expectedHistograms []mockHistogram
}{
{
name: "IngestCTZero=false/startTime=0",
otlpOpts: OTLPOptions{
IngestCTZeroSample: false,
},
startTime: zeroTime,
expectedSamples: expectedSamples,
expectedHistograms: expectedHistograms,
},
{
name: "IngestCTZero=true/startTime=0",
otlpOpts: OTLPOptions{
IngestCTZeroSample: true,
},
startTime: zeroTime,
expectedSamples: expectedSamples,
expectedHistograms: expectedHistograms,
},
{
name: "IngestCTZero=false/startTime=ts-1ms",
otlpOpts: OTLPOptions{
IngestCTZeroSample: false,
},
startTime: startTime,
expectedSamples: expectedSamples,
expectedHistograms: expectedHistograms,
},
{
name: "IngestCTZero=true/startTime=ts-1ms",
otlpOpts: OTLPOptions{
IngestCTZeroSample: true,
},
startTime: startTime,
expectedSamples: expectedSamplesWithCTZero,
expectedHistograms: expectedHistogramsWithCTZero,
},
} {
t.Run(testCase.name, func(t *testing.T) {
exportRequest := generateOTLPWriteRequest(timestamp, testCase.startTime)
appendable := handleOTLP(t, exportRequest, config.OTLPConfig{
TranslationStrategy: otlptranslator.NoTranslation,
}, testCase.otlpOpts)
for i, expect := range testCase.expectedSamples {
actual := appendable.samples[i]
require.True(t, labels.Equal(expect.l, actual.l), "sample labels,pos=%v", i)
require.Equal(t, expect.t, actual.t, "sample timestamp,pos=%v", i)
require.Equal(t, expect.v, actual.v, "sample value,pos=%v", i)
}
for i, expect := range testCase.expectedHistograms {
actual := appendable.histograms[i]
require.True(t, labels.Equal(expect.l, actual.l), "histogram labels,pos=%v", i)
require.Equal(t, expect.t, actual.t, "histogram timestamp,pos=%v", i)
require.True(t, expect.h.Equals(actual.h), "histogram value,pos=%v", i)
}
require.Len(t, appendable.samples, len(testCase.expectedSamples))
require.Len(t, appendable.histograms, len(testCase.expectedHistograms))
})
}
}
func requireContainsSample(t *testing.T, actual []mockSample, expected mockSample) {
t.Helper()
@ -607,7 +1123,20 @@ func requireContainsSample(t *testing.T, actual []mockSample, expected mockSampl
"actual : %v", expected, actual))
}
func handleOTLP(t *testing.T, exportRequest pmetricotlp.ExportRequest, otlpCfg config.OTLPConfig, typeAndUnitLabels bool) *mockAppendable {
func requireContainsMetadata(t *testing.T, actual []mockMetadata, expected mockMetadata) {
t.Helper()
for _, got := range actual {
if labels.Equal(expected.l, got.l) && expected.m.Type == got.m.Type && expected.m.Unit == got.m.Unit && expected.m.Help == got.m.Help {
return
}
}
require.Fail(t, fmt.Sprintf("Metadata not found: \n"+
"expected: %v\n"+
"actual : %v", expected, actual))
}
func handleOTLP(t *testing.T, exportRequest pmetricotlp.ExportRequest, otlpCfg config.OTLPConfig, otlpOpts OTLPOptions) *mockAppendable {
buf, err := exportRequest.MarshalProto()
require.NoError(t, err)
@ -615,12 +1144,13 @@ func handleOTLP(t *testing.T, exportRequest pmetricotlp.ExportRequest, otlpCfg c
require.NoError(t, err)
req.Header.Set("Content-Type", "application/x-protobuf")
log := slog.New(slog.NewTextHandler(os.Stderr, &slog.HandlerOptions{Level: slog.LevelWarn}))
appendable := &mockAppendable{}
handler := NewOTLPWriteHandler(nil, nil, appendable, func() config.Config {
handler := NewOTLPWriteHandler(log, nil, appendable, func() config.Config {
return config.Config{
OTLPConfig: otlpCfg,
}
}, OTLPOptions{EnableTypeAndUnitLabels: typeAndUnitLabels})
}, otlpOpts)
recorder := httptest.NewRecorder()
handler.ServeHTTP(recorder, req)
@ -630,7 +1160,7 @@ func handleOTLP(t *testing.T, exportRequest pmetricotlp.ExportRequest, otlpCfg c
return appendable
}
func generateOTLPWriteRequest(timestamp time.Time) pmetricotlp.ExportRequest {
func generateOTLPWriteRequest(timestamp, startTime time.Time) pmetricotlp.ExportRequest {
d := pmetric.NewMetrics()
// Generate One Counter, One Gauge, One Histogram, One Exponential-Histogram
@ -655,6 +1185,7 @@ func generateOTLPWriteRequest(timestamp time.Time) pmetricotlp.ExportRequest {
counterDataPoint := counterMetric.Sum().DataPoints().AppendEmpty()
counterDataPoint.SetTimestamp(pcommon.NewTimestampFromTime(timestamp))
counterDataPoint.SetStartTimestamp(pcommon.NewTimestampFromTime(startTime))
counterDataPoint.SetDoubleValue(10.0)
counterDataPoint.Attributes().PutStr("foo.bar", "baz")
@ -674,6 +1205,7 @@ func generateOTLPWriteRequest(timestamp time.Time) pmetricotlp.ExportRequest {
gaugeDataPoint := gaugeMetric.Gauge().DataPoints().AppendEmpty()
gaugeDataPoint.SetTimestamp(pcommon.NewTimestampFromTime(timestamp))
gaugeDataPoint.SetStartTimestamp(pcommon.NewTimestampFromTime(startTime))
gaugeDataPoint.SetDoubleValue(10.0)
gaugeDataPoint.Attributes().PutStr("foo.bar", "baz")
@ -687,9 +1219,10 @@ func generateOTLPWriteRequest(timestamp time.Time) pmetricotlp.ExportRequest {
histogramDataPoint := histogramMetric.Histogram().DataPoints().AppendEmpty()
histogramDataPoint.SetTimestamp(pcommon.NewTimestampFromTime(timestamp))
histogramDataPoint.SetStartTimestamp(pcommon.NewTimestampFromTime(startTime))
histogramDataPoint.ExplicitBounds().FromRaw([]float64{0.0, 1.0, 2.0, 3.0, 4.0, 5.0})
histogramDataPoint.BucketCounts().FromRaw([]uint64{2, 2, 2, 2, 2, 2})
histogramDataPoint.SetCount(10)
histogramDataPoint.SetCount(12)
histogramDataPoint.SetSum(30.0)
histogramDataPoint.Attributes().PutStr("foo.bar", "baz")
@ -703,6 +1236,7 @@ func generateOTLPWriteRequest(timestamp time.Time) pmetricotlp.ExportRequest {
exponentialHistogramDataPoint := exponentialHistogramMetric.ExponentialHistogram().DataPoints().AppendEmpty()
exponentialHistogramDataPoint.SetTimestamp(pcommon.NewTimestampFromTime(timestamp))
exponentialHistogramDataPoint.SetStartTimestamp(pcommon.NewTimestampFromTime(startTime))
exponentialHistogramDataPoint.SetScale(2.0)
exponentialHistogramDataPoint.Positive().BucketCounts().FromRaw([]uint64{2, 2, 2, 2, 2})
exponentialHistogramDataPoint.SetZeroCount(2)

View File

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