mirror of
https://github.com/prometheus/prometheus.git
synced 2025-09-21 13:51:00 +02:00
OTLP to directly write to an interface which can hide storage details (#16951)
* OTLP writer writes directly to appender Do not convert to Remote-Write 1.0 protocol. Convert to TSDB Appender interface instead. For downstream projects that still convert OTLP to something else (e.g. Mimir using its own RW 1.0+2.0 compatible protocol), introduce a compatibility layer between OTLP decoding and TSDB Appender. This is the CombinedAppender that hides the implementation. Name is subject to change. --------- Signed-off-by: David Ashpole <dashpole@google.com> Signed-off-by: György Krajcsovits <gyorgy.krajcsovits@grafana.com> Signed-off-by: George Krajcsovits <krajorama@users.noreply.github.com> Co-authored-by: David Ashpole <dashpole@google.com> Co-authored-by: Jesus Vazquez <jesusvazquez@users.noreply.github.com> Co-authored-by: Arve Knudsen <arve.knudsen@gmail.com>
This commit is contained in:
parent
913cc8f72b
commit
979aea1d49
@ -0,0 +1,234 @@
|
||||
// Copyright The Prometheus Authors
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
// TODO(krajorama): rename this package to otlpappender or similar, as it is
|
||||
// not specific to Prometheus remote write anymore.
|
||||
// Note otlptranslator is already used by prometheus/otlptranslator repo.
|
||||
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"
|
||||
"github.com/prometheus/prometheus/model/labels"
|
||||
"github.com/prometheus/prometheus/model/metadata"
|
||||
"github.com/prometheus/prometheus/storage"
|
||||
)
|
||||
|
||||
// Metadata extends metadata.Metadata with the metric family name.
|
||||
// OTLP calculates the metric family name for all metrics and uses
|
||||
// it for generating summary, histogram series by adding the magic
|
||||
// suffixes. The metric family name is passed down to the appender
|
||||
// in case the storage needs it for metadata updates.
|
||||
// Known user is Mimir that implements /api/v1/metadata and uses
|
||||
// Remote-Write 1.0 for this. Might be removed later if no longer
|
||||
// needed by any downstream project.
|
||||
type Metadata struct {
|
||||
metadata.Metadata
|
||||
MetricFamilyName string
|
||||
}
|
||||
|
||||
// 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(ls labels.Labels, meta Metadata, ct, t int64, v float64, es []exemplar.Exemplar) error
|
||||
// AppendHistogram appends a histogram and related exemplars, metadata, and
|
||||
// created timestamp to the storage.
|
||||
AppendHistogram(ls labels.Labels, meta Metadata, ct, t int64, h *histogram.Histogram, es []exemplar.Exemplar) error
|
||||
}
|
||||
|
||||
// CombinedAppenderMetrics is for the metrics observed by the
|
||||
// combinedAppender implementation.
|
||||
type CombinedAppenderMetrics struct {
|
||||
samplesAppendedWithoutMetadata prometheus.Counter
|
||||
outOfOrderExemplars prometheus.Counter
|
||||
}
|
||||
|
||||
func NewCombinedAppenderMetrics(reg prometheus.Registerer) CombinedAppenderMetrics {
|
||||
return CombinedAppenderMetrics{
|
||||
samplesAppendedWithoutMetadata: promauto.With(reg).NewCounter(prometheus.CounterOpts{
|
||||
Namespace: "prometheus",
|
||||
Subsystem: "api",
|
||||
Name: "otlp_appended_samples_without_metadata_total",
|
||||
Help: "The total number of samples ingested from OTLP 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.",
|
||||
}),
|
||||
}
|
||||
}
|
||||
|
||||
// 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, ingestCTZeroSample bool, metrics CombinedAppenderMetrics) CombinedAppender {
|
||||
return &combinedAppender{
|
||||
app: app,
|
||||
logger: logger,
|
||||
ingestCTZeroSample: ingestCTZeroSample,
|
||||
refs: make(map[uint64]seriesRef),
|
||||
samplesAppendedWithoutMetadata: metrics.samplesAppendedWithoutMetadata,
|
||||
outOfOrderExemplars: metrics.outOfOrderExemplars,
|
||||
}
|
||||
}
|
||||
|
||||
type seriesRef struct {
|
||||
ref storage.SeriesRef
|
||||
ct int64
|
||||
ls labels.Labels
|
||||
meta metadata.Metadata
|
||||
}
|
||||
|
||||
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]seriesRef
|
||||
}
|
||||
|
||||
func (b *combinedAppender) AppendSample(ls labels.Labels, meta Metadata, ct, t int64, v float64, es []exemplar.Exemplar) (err error) {
|
||||
return b.appendFloatOrHistogram(ls, meta.Metadata, ct, t, v, nil, es)
|
||||
}
|
||||
|
||||
func (b *combinedAppender) AppendHistogram(ls labels.Labels, meta Metadata, ct, t int64, h *histogram.Histogram, es []exemplar.Exemplar) (err error) {
|
||||
if h == nil {
|
||||
// Sanity check, we should never get here with a nil histogram.
|
||||
b.logger.Error("Received nil histogram in CombinedAppender.AppendHistogram", "series", ls.String())
|
||||
return errors.New("internal error, attempted to append nil histogram")
|
||||
}
|
||||
return b.appendFloatOrHistogram(ls, meta.Metadata, ct, t, 0, h, es)
|
||||
}
|
||||
|
||||
func (b *combinedAppender) appendFloatOrHistogram(ls labels.Labels, meta metadata.Metadata, ct, t int64, v float64, h *histogram.Histogram, es []exemplar.Exemplar) (err error) {
|
||||
hash := ls.Hash()
|
||||
series, exists := b.refs[hash]
|
||||
ref := series.ref
|
||||
if exists && !labels.Equal(series.ls, ls) {
|
||||
// Hash collision. The series reference we stored is pointing to a
|
||||
// different series so we cannot use it, we need to reset the
|
||||
// reference and cache.
|
||||
// Note: we don't need to keep track of conflicts here,
|
||||
// the TSDB will handle that part when we pass 0 reference.
|
||||
exists = false
|
||||
ref = 0
|
||||
}
|
||||
updateRefs := !exists || series.ct != ct
|
||||
if updateRefs && ct != 0 && b.ingestCTZeroSample {
|
||||
var newRef storage.SeriesRef
|
||||
if h != nil {
|
||||
newRef, err = b.app.AppendHistogramCTZeroSample(ref, ls, t, ct, h, nil)
|
||||
} else {
|
||||
newRef, err = b.app.AppendCTZeroSample(ref, ls, t, ct)
|
||||
}
|
||||
if err != nil {
|
||||
if !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.Warn("Error when appending CT from OTLP", "err", err, "series", ls.String(), "created_timestamp", ct, "timestamp", t, "sample_type", sampleType(h))
|
||||
}
|
||||
} else {
|
||||
// We only use the returned reference on success as otherwise an
|
||||
// error of CT append could invalidate the series reference.
|
||||
ref = newRef
|
||||
}
|
||||
}
|
||||
{
|
||||
var newRef storage.SeriesRef
|
||||
if h != nil {
|
||||
newRef, err = b.app.AppendHistogram(ref, ls, t, h, nil)
|
||||
} else {
|
||||
newRef, err = b.app.Append(ref, ls, t, v)
|
||||
}
|
||||
if err != nil {
|
||||
// Although Append 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 sample from OTLP", "err", err.Error(), "series", ls.String(), "timestamp", t, "sample_type", sampleType(h))
|
||||
}
|
||||
} else {
|
||||
// If the append was successful, we can use the returned reference.
|
||||
ref = newRef
|
||||
}
|
||||
}
|
||||
|
||||
if ref == 0 {
|
||||
// We cannot update metadata or add exemplars on non existent series.
|
||||
return
|
||||
}
|
||||
|
||||
if !exists || series.meta.Help != meta.Help || series.meta.Type != meta.Type || series.meta.Unit != meta.Unit {
|
||||
updateRefs = true
|
||||
// If this is the first time we see this series, set the metadata.
|
||||
_, err := b.app.UpdateMetadata(ref, ls, meta)
|
||||
if err != nil {
|
||||
b.samplesAppendedWithoutMetadata.Add(1)
|
||||
b.logger.Warn("Error while updating metadata from OTLP", "err", err)
|
||||
}
|
||||
}
|
||||
|
||||
if updateRefs {
|
||||
b.refs[hash] = seriesRef{
|
||||
ref: ref,
|
||||
ct: ct,
|
||||
ls: ls,
|
||||
meta: meta,
|
||||
}
|
||||
}
|
||||
|
||||
b.appendExemplars(ref, ls, es)
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
func sampleType(h *histogram.Histogram) string {
|
||||
if h == nil {
|
||||
return "float"
|
||||
}
|
||||
return "histogram"
|
||||
}
|
||||
|
||||
func (b *combinedAppender) appendExemplars(ref storage.SeriesRef, ls labels.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
|
||||
}
|
@ -0,0 +1,767 @@
|
||||
// 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"
|
||||
"errors"
|
||||
"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"
|
||||
"github.com/prometheus/prometheus/model/labels"
|
||||
"github.com/prometheus/prometheus/model/metadata"
|
||||
"github.com/prometheus/prometheus/storage"
|
||||
"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(ls labels.Labels, meta Metadata, ct, t int64, v float64, es []exemplar.Exemplar) error {
|
||||
m.pendingSamples = append(m.pendingSamples, combinedSample{
|
||||
metricFamilyName: meta.MetricFamilyName,
|
||||
ls: ls,
|
||||
meta: meta.Metadata,
|
||||
t: t,
|
||||
ct: ct,
|
||||
v: v,
|
||||
es: es,
|
||||
})
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *mockCombinedAppender) AppendHistogram(ls labels.Labels, meta Metadata, ct, t int64, h *histogram.Histogram, es []exemplar.Exemplar) error {
|
||||
m.pendingHistograms = append(m.pendingHistograms, combinedHistogram{
|
||||
metricFamilyName: meta.MetricFamilyName,
|
||||
ls: ls,
|
||||
meta: meta.Metadata,
|
||||
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: labels.FromStrings("tracid", "122"),
|
||||
Value: 1337,
|
||||
},
|
||||
{
|
||||
Labels: labels.FromStrings("tracid", "132"),
|
||||
Value: 7777,
|
||||
},
|
||||
}
|
||||
expectedExemplars := []exemplar.QueryResult{
|
||||
{
|
||||
SeriesLabels: labels.FromStrings(
|
||||
model.MetricNameLabel, "test_bytes_total",
|
||||
"foo", "bar",
|
||||
),
|
||||
Exemplars: testExemplars,
|
||||
},
|
||||
}
|
||||
|
||||
seriesLabels := labels.FromStrings(
|
||||
model.MetricNameLabel, "test_bytes_total",
|
||||
"foo", "bar",
|
||||
)
|
||||
floatMetadata := Metadata{
|
||||
Metadata: metadata.Metadata{
|
||||
Type: model.MetricTypeCounter,
|
||||
Unit: "bytes",
|
||||
Help: "some help",
|
||||
},
|
||||
MetricFamilyName: "test_bytes_total",
|
||||
}
|
||||
|
||||
histogramMetadata := Metadata{
|
||||
Metadata: metadata.Metadata{
|
||||
Type: model.MetricTypeHistogram,
|
||||
Unit: "bytes",
|
||||
Help: "some help",
|
||||
},
|
||||
MetricFamilyName: "test_bytes",
|
||||
}
|
||||
|
||||
testCases := map[string]struct {
|
||||
appendFunc func(*testing.T, CombinedAppender)
|
||||
expectedSamples []sample
|
||||
expectedExemplars []exemplar.QueryResult
|
||||
}{
|
||||
"single float sample, zero CT": {
|
||||
appendFunc: func(t *testing.T, app CombinedAppender) {
|
||||
require.NoError(t, app.AppendSample(seriesLabels.Copy(), floatMetadata, 0, now.UnixMilli(), 42.0, testExemplars))
|
||||
},
|
||||
expectedSamples: []sample{
|
||||
{
|
||||
t: now.UnixMilli(),
|
||||
f: 42.0,
|
||||
},
|
||||
},
|
||||
expectedExemplars: expectedExemplars,
|
||||
},
|
||||
"single float sample, very old CT": {
|
||||
appendFunc: func(t *testing.T, app CombinedAppender) {
|
||||
require.NoError(t, app.AppendSample(seriesLabels.Copy(), floatMetadata, 1, now.UnixMilli(), 42.0, nil))
|
||||
},
|
||||
expectedSamples: []sample{
|
||||
{
|
||||
t: now.UnixMilli(),
|
||||
f: 42.0,
|
||||
},
|
||||
},
|
||||
},
|
||||
"single float sample, normal CT": {
|
||||
appendFunc: func(t *testing.T, app CombinedAppender) {
|
||||
require.NoError(t, app.AppendSample(seriesLabels.Copy(), floatMetadata, now.Add(-2*time.Minute).UnixMilli(), now.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 same time as sample": {
|
||||
appendFunc: func(t *testing.T, app CombinedAppender) {
|
||||
require.NoError(t, app.AppendSample(seriesLabels.Copy(), floatMetadata, 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(t *testing.T, app CombinedAppender) {
|
||||
require.NoError(t, app.AppendSample(seriesLabels.Copy(), floatMetadata, now.Add(time.Minute).UnixMilli(), now.UnixMilli(), 42.0, nil))
|
||||
},
|
||||
expectedSamples: []sample{
|
||||
{
|
||||
t: now.UnixMilli(),
|
||||
f: 42.0,
|
||||
},
|
||||
},
|
||||
},
|
||||
"single histogram sample, zero CT": {
|
||||
appendFunc: func(t *testing.T, app CombinedAppender) {
|
||||
require.NoError(t, app.AppendHistogram(seriesLabels.Copy(), histogramMetadata, 0, now.UnixMilli(), tsdbutil.GenerateTestHistogram(42), testExemplars))
|
||||
},
|
||||
expectedSamples: []sample{
|
||||
{
|
||||
t: now.UnixMilli(),
|
||||
h: tsdbutil.GenerateTestHistogram(42),
|
||||
},
|
||||
},
|
||||
expectedExemplars: expectedExemplars,
|
||||
},
|
||||
"single histogram sample, very old CT": {
|
||||
appendFunc: func(t *testing.T, app CombinedAppender) {
|
||||
require.NoError(t, app.AppendHistogram(seriesLabels.Copy(), histogramMetadata, 1, now.UnixMilli(), tsdbutil.GenerateTestHistogram(42), nil))
|
||||
},
|
||||
expectedSamples: []sample{
|
||||
{
|
||||
t: now.UnixMilli(),
|
||||
h: tsdbutil.GenerateTestHistogram(42),
|
||||
},
|
||||
},
|
||||
},
|
||||
"single histogram sample, normal CT": {
|
||||
appendFunc: func(t *testing.T, app CombinedAppender) {
|
||||
require.NoError(t, app.AppendHistogram(seriesLabels.Copy(), histogramMetadata, now.Add(-2*time.Minute).UnixMilli(), now.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 same time as sample": {
|
||||
appendFunc: func(t *testing.T, app CombinedAppender) {
|
||||
require.NoError(t, app.AppendHistogram(seriesLabels.Copy(), histogramMetadata, 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(t *testing.T, app CombinedAppender) {
|
||||
require.NoError(t, app.AppendHistogram(seriesLabels.Copy(), histogramMetadata, now.Add(time.Minute).UnixMilli(), now.UnixMilli(), tsdbutil.GenerateTestHistogram(42), nil))
|
||||
},
|
||||
expectedSamples: []sample{
|
||||
{
|
||||
t: now.UnixMilli(),
|
||||
h: tsdbutil.GenerateTestHistogram(42),
|
||||
},
|
||||
},
|
||||
},
|
||||
"multiple float samples": {
|
||||
appendFunc: func(t *testing.T, app CombinedAppender) {
|
||||
require.NoError(t, app.AppendSample(seriesLabels.Copy(), floatMetadata, 0, now.UnixMilli(), 42.0, nil))
|
||||
require.NoError(t, app.AppendSample(seriesLabels.Copy(), floatMetadata, 0, now.Add(15*time.Second).UnixMilli(), 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(t *testing.T, app CombinedAppender) {
|
||||
require.NoError(t, app.AppendHistogram(seriesLabels.Copy(), histogramMetadata, 0, now.UnixMilli(), tsdbutil.GenerateTestHistogram(42), nil))
|
||||
require.NoError(t, app.AppendHistogram(seriesLabels.Copy(), histogramMetadata, 0, now.Add(15*time.Second).UnixMilli(), tsdbutil.GenerateTestHistogram(62), nil))
|
||||
},
|
||||
expectedSamples: []sample{
|
||||
{
|
||||
t: now.UnixMilli(),
|
||||
h: tsdbutil.GenerateTestHistogram(42),
|
||||
},
|
||||
{
|
||||
t: now.Add(15 * time.Second).UnixMilli(),
|
||||
h: tsdbutil.GenerateTestHistogram(62),
|
||||
},
|
||||
},
|
||||
},
|
||||
"float samples with CT changing": {
|
||||
appendFunc: func(t *testing.T, app CombinedAppender) {
|
||||
require.NoError(t, app.AppendSample(seriesLabels.Copy(), floatMetadata, now.Add(-4*time.Second).UnixMilli(), now.Add(-3*time.Second).UnixMilli(), 42.0, nil))
|
||||
require.NoError(t, app.AppendSample(seriesLabels.Copy(), floatMetadata, now.Add(-1*time.Second).UnixMilli(), now.UnixMilli(), 62.0, nil))
|
||||
},
|
||||
expectedSamples: []sample{
|
||||
{
|
||||
ctZero: true,
|
||||
t: now.Add(-4 * time.Second).UnixMilli(),
|
||||
},
|
||||
{
|
||||
t: now.Add(-3 * time.Second).UnixMilli(),
|
||||
f: 42.0,
|
||||
},
|
||||
{
|
||||
ctZero: true,
|
||||
t: now.Add(-1 * time.Second).UnixMilli(),
|
||||
},
|
||||
{
|
||||
t: now.UnixMilli(),
|
||||
f: 62.0,
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
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()
|
||||
reg := prometheus.NewRegistry()
|
||||
app := db.Appender(ctx)
|
||||
capp := NewCombinedAppender(app, promslog.NewNopLogger(), ingestCTZeroSample, NewCombinedAppenderMetrics(reg))
|
||||
|
||||
tc.appendFunc(t, capp)
|
||||
|
||||
require.NoError(t, app.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),
|
||||
}, labels.MustNewMatcher(labels.MatchEqual, model.MetricNameLabel, "test_bytes_total"))
|
||||
|
||||
require.NoError(t, ss.Err())
|
||||
|
||||
require.True(t, ss.Next())
|
||||
series := ss.At()
|
||||
it := series.Iterator(nil)
|
||||
for i, 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, "sample ts %d", i)
|
||||
require.Equal(t, sample.f, v, "sample v %d", i)
|
||||
} else {
|
||||
require.Equal(t, chunkenc.ValHistogram, it.Next())
|
||||
ts, h := it.AtHistogram(nil)
|
||||
require.Equal(t, sample.t, ts, "sample ts %d", i)
|
||||
require.Equal(t, sample.h.Count, h.Count, "sample v %d", i)
|
||||
}
|
||||
}
|
||||
require.False(t, ss.Next())
|
||||
|
||||
eq, err := db.ExemplarQuerier(ctx)
|
||||
require.NoError(t, err)
|
||||
exResult, err := eq.Select(int64(math.MinInt64), int64(math.MaxInt64), []*labels.Matcher{labels.MustNewMatcher(labels.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
|
||||
}
|
||||
|
||||
// TestCombinedAppenderSeriesRefs checks that the combined appender
|
||||
// correctly uses and updates the series references in the internal map.
|
||||
func TestCombinedAppenderSeriesRefs(t *testing.T) {
|
||||
seriesLabels := labels.FromStrings(
|
||||
model.MetricNameLabel, "test_bytes_total",
|
||||
"foo", "bar",
|
||||
)
|
||||
|
||||
floatMetadata := Metadata{
|
||||
Metadata: metadata.Metadata{
|
||||
Type: model.MetricTypeCounter,
|
||||
Unit: "bytes",
|
||||
Help: "some help",
|
||||
},
|
||||
MetricFamilyName: "test_bytes_total",
|
||||
}
|
||||
|
||||
t.Run("happy case with CT zero, reference is passed and reused", func(t *testing.T) {
|
||||
app := &appenderRecorder{}
|
||||
capp := NewCombinedAppender(app, promslog.NewNopLogger(), true, NewCombinedAppenderMetrics(prometheus.NewRegistry()))
|
||||
|
||||
require.NoError(t, capp.AppendSample(seriesLabels.Copy(), floatMetadata, 1, 2, 42.0, nil))
|
||||
|
||||
require.NoError(t, capp.AppendSample(seriesLabels.Copy(), floatMetadata, 3, 4, 62.0, []exemplar.Exemplar{
|
||||
{
|
||||
Labels: labels.FromStrings("tracid", "122"),
|
||||
Value: 1337,
|
||||
},
|
||||
}))
|
||||
|
||||
require.Len(t, app.records, 6)
|
||||
requireEqualOpAndRef(t, "AppendCTZeroSample", 0, app.records[0])
|
||||
ref := app.records[0].outRef
|
||||
require.NotZero(t, ref)
|
||||
requireEqualOpAndRef(t, "Append", ref, app.records[1])
|
||||
requireEqualOpAndRef(t, "UpdateMetadata", ref, app.records[2])
|
||||
requireEqualOpAndRef(t, "AppendCTZeroSample", ref, app.records[3])
|
||||
requireEqualOpAndRef(t, "Append", ref, app.records[4])
|
||||
requireEqualOpAndRef(t, "AppendExemplar", ref, app.records[5])
|
||||
})
|
||||
|
||||
t.Run("error on second CT ingest doesn't update the reference", func(t *testing.T) {
|
||||
app := &appenderRecorder{}
|
||||
capp := NewCombinedAppender(app, promslog.NewNopLogger(), true, NewCombinedAppenderMetrics(prometheus.NewRegistry()))
|
||||
|
||||
require.NoError(t, capp.AppendSample(seriesLabels.Copy(), floatMetadata, 1, 2, 42.0, nil))
|
||||
|
||||
app.appendCTZeroSampleError = errors.New("test error")
|
||||
require.NoError(t, capp.AppendSample(seriesLabels.Copy(), floatMetadata, 3, 4, 62.0, nil))
|
||||
|
||||
require.Len(t, app.records, 5)
|
||||
requireEqualOpAndRef(t, "AppendCTZeroSample", 0, app.records[0])
|
||||
ref := app.records[0].outRef
|
||||
require.NotZero(t, ref)
|
||||
requireEqualOpAndRef(t, "Append", ref, app.records[1])
|
||||
requireEqualOpAndRef(t, "UpdateMetadata", ref, app.records[2])
|
||||
requireEqualOpAndRef(t, "AppendCTZeroSample", ref, app.records[3])
|
||||
require.Zero(t, app.records[3].outRef, "the second AppendCTZeroSample returned 0")
|
||||
requireEqualOpAndRef(t, "Append", ref, app.records[4])
|
||||
})
|
||||
|
||||
t.Run("updateMetadata called when meta help changes", func(t *testing.T) {
|
||||
app := &appenderRecorder{}
|
||||
capp := NewCombinedAppender(app, promslog.NewNopLogger(), true, NewCombinedAppenderMetrics(prometheus.NewRegistry()))
|
||||
|
||||
newMetadata := floatMetadata
|
||||
newMetadata.Help = "some other help"
|
||||
|
||||
require.NoError(t, capp.AppendSample(seriesLabels.Copy(), floatMetadata, 1, 2, 42.0, nil))
|
||||
require.NoError(t, capp.AppendSample(seriesLabels.Copy(), newMetadata, 3, 4, 62.0, nil))
|
||||
require.NoError(t, capp.AppendSample(seriesLabels.Copy(), newMetadata, 3, 5, 162.0, nil))
|
||||
|
||||
require.Len(t, app.records, 7)
|
||||
requireEqualOpAndRef(t, "AppendCTZeroSample", 0, app.records[0])
|
||||
ref := app.records[0].outRef
|
||||
require.NotZero(t, ref)
|
||||
requireEqualOpAndRef(t, "Append", ref, app.records[1])
|
||||
requireEqualOpAndRef(t, "UpdateMetadata", ref, app.records[2])
|
||||
requireEqualOpAndRef(t, "AppendCTZeroSample", ref, app.records[3])
|
||||
requireEqualOpAndRef(t, "Append", ref, app.records[4])
|
||||
requireEqualOpAndRef(t, "UpdateMetadata", ref, app.records[5])
|
||||
requireEqualOpAndRef(t, "Append", ref, app.records[6])
|
||||
})
|
||||
|
||||
t.Run("updateMetadata called when meta unit changes", func(t *testing.T) {
|
||||
app := &appenderRecorder{}
|
||||
capp := NewCombinedAppender(app, promslog.NewNopLogger(), true, NewCombinedAppenderMetrics(prometheus.NewRegistry()))
|
||||
|
||||
newMetadata := floatMetadata
|
||||
newMetadata.Unit = "seconds"
|
||||
|
||||
require.NoError(t, capp.AppendSample(seriesLabels.Copy(), floatMetadata, 1, 2, 42.0, nil))
|
||||
require.NoError(t, capp.AppendSample(seriesLabels.Copy(), newMetadata, 3, 4, 62.0, nil))
|
||||
require.NoError(t, capp.AppendSample(seriesLabels.Copy(), newMetadata, 3, 5, 162.0, nil))
|
||||
|
||||
require.Len(t, app.records, 7)
|
||||
requireEqualOpAndRef(t, "AppendCTZeroSample", 0, app.records[0])
|
||||
ref := app.records[0].outRef
|
||||
require.NotZero(t, ref)
|
||||
requireEqualOpAndRef(t, "Append", ref, app.records[1])
|
||||
requireEqualOpAndRef(t, "UpdateMetadata", ref, app.records[2])
|
||||
requireEqualOpAndRef(t, "AppendCTZeroSample", ref, app.records[3])
|
||||
requireEqualOpAndRef(t, "Append", ref, app.records[4])
|
||||
requireEqualOpAndRef(t, "UpdateMetadata", ref, app.records[5])
|
||||
requireEqualOpAndRef(t, "Append", ref, app.records[6])
|
||||
})
|
||||
|
||||
t.Run("updateMetadata called when meta type changes", func(t *testing.T) {
|
||||
app := &appenderRecorder{}
|
||||
capp := NewCombinedAppender(app, promslog.NewNopLogger(), true, NewCombinedAppenderMetrics(prometheus.NewRegistry()))
|
||||
|
||||
newMetadata := floatMetadata
|
||||
newMetadata.Type = model.MetricTypeGauge
|
||||
|
||||
require.NoError(t, capp.AppendSample(seriesLabels.Copy(), floatMetadata, 1, 2, 42.0, nil))
|
||||
require.NoError(t, capp.AppendSample(seriesLabels.Copy(), newMetadata, 3, 4, 62.0, nil))
|
||||
require.NoError(t, capp.AppendSample(seriesLabels.Copy(), newMetadata, 3, 5, 162.0, nil))
|
||||
|
||||
require.Len(t, app.records, 7)
|
||||
requireEqualOpAndRef(t, "AppendCTZeroSample", 0, app.records[0])
|
||||
ref := app.records[0].outRef
|
||||
require.NotZero(t, ref)
|
||||
requireEqualOpAndRef(t, "Append", ref, app.records[1])
|
||||
requireEqualOpAndRef(t, "UpdateMetadata", ref, app.records[2])
|
||||
requireEqualOpAndRef(t, "AppendCTZeroSample", ref, app.records[3])
|
||||
requireEqualOpAndRef(t, "Append", ref, app.records[4])
|
||||
requireEqualOpAndRef(t, "UpdateMetadata", ref, app.records[5])
|
||||
requireEqualOpAndRef(t, "Append", ref, app.records[6])
|
||||
})
|
||||
|
||||
t.Run("metadata, exemplars are not updated if append failed", func(t *testing.T) {
|
||||
app := &appenderRecorder{}
|
||||
capp := NewCombinedAppender(app, promslog.NewNopLogger(), true, NewCombinedAppenderMetrics(prometheus.NewRegistry()))
|
||||
app.appendError = errors.New("test error")
|
||||
require.Error(t, capp.AppendSample(seriesLabels.Copy(), floatMetadata, 0, 1, 42.0, []exemplar.Exemplar{
|
||||
{
|
||||
Labels: labels.FromStrings("tracid", "122"),
|
||||
Value: 1337,
|
||||
},
|
||||
}))
|
||||
|
||||
require.Len(t, app.records, 1)
|
||||
require.Equal(t, appenderRecord{
|
||||
op: "Append",
|
||||
ls: labels.FromStrings(model.MetricNameLabel, "test_bytes_total", "foo", "bar"),
|
||||
}, app.records[0])
|
||||
})
|
||||
|
||||
t.Run("metadata, exemplars are updated if append failed but reference is valid", func(t *testing.T) {
|
||||
app := &appenderRecorder{}
|
||||
capp := NewCombinedAppender(app, promslog.NewNopLogger(), true, NewCombinedAppenderMetrics(prometheus.NewRegistry()))
|
||||
|
||||
newMetadata := floatMetadata
|
||||
newMetadata.Help = "some other help"
|
||||
|
||||
require.NoError(t, capp.AppendSample(seriesLabels.Copy(), floatMetadata, 1, 2, 42.0, nil))
|
||||
app.appendError = errors.New("test error")
|
||||
require.Error(t, capp.AppendSample(seriesLabels.Copy(), newMetadata, 3, 4, 62.0, []exemplar.Exemplar{
|
||||
{
|
||||
Labels: labels.FromStrings("tracid", "122"),
|
||||
Value: 1337,
|
||||
},
|
||||
}))
|
||||
|
||||
require.Len(t, app.records, 7)
|
||||
requireEqualOpAndRef(t, "AppendCTZeroSample", 0, app.records[0])
|
||||
ref := app.records[0].outRef
|
||||
require.NotZero(t, ref)
|
||||
requireEqualOpAndRef(t, "Append", ref, app.records[1])
|
||||
requireEqualOpAndRef(t, "UpdateMetadata", ref, app.records[2])
|
||||
requireEqualOpAndRef(t, "AppendCTZeroSample", ref, app.records[3])
|
||||
requireEqualOpAndRef(t, "Append", ref, app.records[4])
|
||||
require.Zero(t, app.records[4].outRef, "the second Append returned 0")
|
||||
requireEqualOpAndRef(t, "UpdateMetadata", ref, app.records[5])
|
||||
requireEqualOpAndRef(t, "AppendExemplar", ref, app.records[6])
|
||||
})
|
||||
|
||||
t.Run("simulate conflict with existing series", func(t *testing.T) {
|
||||
app := &appenderRecorder{}
|
||||
capp := NewCombinedAppender(app, promslog.NewNopLogger(), true, NewCombinedAppenderMetrics(prometheus.NewRegistry()))
|
||||
|
||||
ls := labels.FromStrings(
|
||||
model.MetricNameLabel, "test_bytes_total",
|
||||
"foo", "bar",
|
||||
)
|
||||
|
||||
require.NoError(t, capp.AppendSample(ls, floatMetadata, 1, 2, 42.0, nil))
|
||||
|
||||
hash := ls.Hash()
|
||||
cappImpl := capp.(*combinedAppender)
|
||||
series := cappImpl.refs[hash]
|
||||
series.ls = labels.FromStrings(
|
||||
model.MetricNameLabel, "test_bytes_total",
|
||||
"foo", "club",
|
||||
)
|
||||
// The hash and ref remain the same, but we altered the labels.
|
||||
// This simulates a conflict with an existing series.
|
||||
cappImpl.refs[hash] = series
|
||||
|
||||
require.NoError(t, capp.AppendSample(ls, floatMetadata, 3, 4, 62.0, []exemplar.Exemplar{
|
||||
{
|
||||
Labels: labels.FromStrings("tracid", "122"),
|
||||
Value: 1337,
|
||||
},
|
||||
}))
|
||||
|
||||
require.Len(t, app.records, 7)
|
||||
requireEqualOpAndRef(t, "AppendCTZeroSample", 0, app.records[0])
|
||||
ref := app.records[0].outRef
|
||||
require.NotZero(t, ref)
|
||||
requireEqualOpAndRef(t, "Append", ref, app.records[1])
|
||||
requireEqualOpAndRef(t, "UpdateMetadata", ref, app.records[2])
|
||||
requireEqualOpAndRef(t, "AppendCTZeroSample", 0, app.records[3])
|
||||
newRef := app.records[3].outRef
|
||||
require.NotEqual(t, ref, newRef, "the second AppendCTZeroSample returned a different reference")
|
||||
requireEqualOpAndRef(t, "Append", newRef, app.records[4])
|
||||
requireEqualOpAndRef(t, "UpdateMetadata", newRef, app.records[5])
|
||||
requireEqualOpAndRef(t, "AppendExemplar", newRef, app.records[6])
|
||||
})
|
||||
|
||||
t.Run("check that invoking AppendHistogram returns an error for nil histogram", func(t *testing.T) {
|
||||
app := &appenderRecorder{}
|
||||
capp := NewCombinedAppender(app, promslog.NewNopLogger(), true, NewCombinedAppenderMetrics(prometheus.NewRegistry()))
|
||||
|
||||
ls := labels.FromStrings(
|
||||
model.MetricNameLabel, "test_bytes_total",
|
||||
"foo", "bar",
|
||||
)
|
||||
err := capp.AppendHistogram(ls, Metadata{}, 4, 2, nil, nil)
|
||||
require.Error(t, err)
|
||||
})
|
||||
}
|
||||
|
||||
func requireEqualOpAndRef(t *testing.T, expectedOp string, expectedRef storage.SeriesRef, actual appenderRecord) {
|
||||
t.Helper()
|
||||
require.Equal(t, expectedOp, actual.op)
|
||||
require.Equal(t, expectedRef, actual.ref)
|
||||
}
|
||||
|
||||
type appenderRecord struct {
|
||||
op string
|
||||
ref storage.SeriesRef
|
||||
outRef storage.SeriesRef
|
||||
ls labels.Labels
|
||||
}
|
||||
|
||||
type appenderRecorder struct {
|
||||
refcount uint64
|
||||
records []appenderRecord
|
||||
|
||||
appendError error
|
||||
appendCTZeroSampleError error
|
||||
appendHistogramError error
|
||||
appendHistogramCTZeroSampleError error
|
||||
updateMetadataError error
|
||||
appendExemplarError error
|
||||
}
|
||||
|
||||
var _ storage.Appender = &appenderRecorder{}
|
||||
|
||||
func (a *appenderRecorder) setOutRef(ref storage.SeriesRef) {
|
||||
if len(a.records) == 0 {
|
||||
return
|
||||
}
|
||||
a.records[len(a.records)-1].outRef = ref
|
||||
}
|
||||
|
||||
func (a *appenderRecorder) newRef() storage.SeriesRef {
|
||||
a.refcount++
|
||||
return storage.SeriesRef(a.refcount)
|
||||
}
|
||||
|
||||
func (a *appenderRecorder) Append(ref storage.SeriesRef, ls labels.Labels, _ int64, _ float64) (storage.SeriesRef, error) {
|
||||
a.records = append(a.records, appenderRecord{op: "Append", ref: ref, ls: ls})
|
||||
if a.appendError != nil {
|
||||
return 0, a.appendError
|
||||
}
|
||||
if ref == 0 {
|
||||
ref = a.newRef()
|
||||
}
|
||||
a.setOutRef(ref)
|
||||
return ref, nil
|
||||
}
|
||||
|
||||
func (a *appenderRecorder) AppendCTZeroSample(ref storage.SeriesRef, ls labels.Labels, _, _ int64) (storage.SeriesRef, error) {
|
||||
a.records = append(a.records, appenderRecord{op: "AppendCTZeroSample", ref: ref, ls: ls})
|
||||
if a.appendCTZeroSampleError != nil {
|
||||
return 0, a.appendCTZeroSampleError
|
||||
}
|
||||
if ref == 0 {
|
||||
ref = a.newRef()
|
||||
}
|
||||
a.setOutRef(ref)
|
||||
return ref, nil
|
||||
}
|
||||
|
||||
func (a *appenderRecorder) AppendHistogram(ref storage.SeriesRef, ls labels.Labels, _ int64, _ *histogram.Histogram, _ *histogram.FloatHistogram) (storage.SeriesRef, error) {
|
||||
a.records = append(a.records, appenderRecord{op: "AppendHistogram", ref: ref, ls: ls})
|
||||
if a.appendHistogramError != nil {
|
||||
return 0, a.appendHistogramError
|
||||
}
|
||||
if ref == 0 {
|
||||
ref = a.newRef()
|
||||
}
|
||||
a.setOutRef(ref)
|
||||
return ref, nil
|
||||
}
|
||||
|
||||
func (a *appenderRecorder) AppendHistogramCTZeroSample(ref storage.SeriesRef, ls labels.Labels, _, _ int64, _ *histogram.Histogram, _ *histogram.FloatHistogram) (storage.SeriesRef, error) {
|
||||
a.records = append(a.records, appenderRecord{op: "AppendHistogramCTZeroSample", ref: ref, ls: ls})
|
||||
if a.appendHistogramCTZeroSampleError != nil {
|
||||
return 0, a.appendHistogramCTZeroSampleError
|
||||
}
|
||||
if ref == 0 {
|
||||
ref = a.newRef()
|
||||
}
|
||||
a.setOutRef(ref)
|
||||
return ref, nil
|
||||
}
|
||||
|
||||
func (a *appenderRecorder) UpdateMetadata(ref storage.SeriesRef, ls labels.Labels, _ metadata.Metadata) (storage.SeriesRef, error) {
|
||||
a.records = append(a.records, appenderRecord{op: "UpdateMetadata", ref: ref, ls: ls})
|
||||
if a.updateMetadataError != nil {
|
||||
return 0, a.updateMetadataError
|
||||
}
|
||||
a.setOutRef(ref)
|
||||
return ref, nil
|
||||
}
|
||||
|
||||
func (a *appenderRecorder) AppendExemplar(ref storage.SeriesRef, ls labels.Labels, _ exemplar.Exemplar) (storage.SeriesRef, error) {
|
||||
a.records = append(a.records, appenderRecord{op: "AppendExemplar", ref: ref, ls: ls})
|
||||
if a.appendExemplarError != nil {
|
||||
return 0, a.appendExemplarError
|
||||
}
|
||||
a.setOutRef(ref)
|
||||
return ref, nil
|
||||
}
|
||||
|
||||
func (a *appenderRecorder) Commit() error {
|
||||
a.records = append(a.records, appenderRecord{op: "Commit"})
|
||||
return nil
|
||||
}
|
||||
|
||||
func (a *appenderRecorder) Rollback() error {
|
||||
a.records = append(a.records, appenderRecord{op: "Rollback"})
|
||||
return nil
|
||||
}
|
||||
|
||||
func (*appenderRecorder) SetOptions(_ *storage.AppendOptions) {
|
||||
panic("not implemented")
|
||||
}
|
@ -23,22 +23,22 @@ 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"
|
||||
"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"
|
||||
)
|
||||
|
||||
const (
|
||||
@ -56,132 +56,71 @@ const (
|
||||
// https://github.com/open-telemetry/opentelemetry-specification/blob/main/specification%2Fmetrics%2Fdatamodel.md#exemplars-2
|
||||
traceIDKey = "trace_id"
|
||||
spanIDKey = "span_id"
|
||||
infoType = "info"
|
||||
targetMetricName = "target_info"
|
||||
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, 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
|
||||
|
||||
if settings.AllowUTF8 {
|
||||
// UTF8 is allowed, so conflicts aren't possible.
|
||||
c.builder.Reset(sortedLabels)
|
||||
} else {
|
||||
l[finalKey] = label.Value
|
||||
// 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 labels.Label) {
|
||||
if sortErr != nil {
|
||||
return
|
||||
}
|
||||
finalKey, err := labelNamer.Build(l.Name)
|
||||
if err != 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)
|
||||
}
|
||||
})
|
||||
if sortErr != nil {
|
||||
return labels.EmptyLabels(), sortErr
|
||||
}
|
||||
}
|
||||
|
||||
for _, lbl := range promotedAttrs {
|
||||
normalized, err := labelNamer.Build(lbl.Name)
|
||||
err := settings.PromoteResourceAttributes.addPromotedAttributes(c.builder, resourceAttrs, settings.AllowUTF8)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if _, exists := l[normalized]; !exists {
|
||||
l[normalized] = lbl.Value
|
||||
}
|
||||
return labels.EmptyLabels(), err
|
||||
}
|
||||
if promoteScope {
|
||||
var rangeErr error
|
||||
@ -191,25 +130,25 @@ func createAttributes(resource pcommon.Resource, attributes pcommon.Map, scope s
|
||||
rangeErr = err
|
||||
return false
|
||||
}
|
||||
l[name] = v.AsString()
|
||||
c.builder.Set(name, v.AsString())
|
||||
return true
|
||||
})
|
||||
if rangeErr != nil {
|
||||
return nil, rangeErr
|
||||
return labels.EmptyLabels(), rangeErr
|
||||
}
|
||||
// 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
|
||||
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)
|
||||
}
|
||||
|
||||
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(model.MetricTypeLabel, strings.ToLower(string(meta.Type)))
|
||||
}
|
||||
if metadata.Unit != "" {
|
||||
l["__unit__"] = unitNamer.Build(metadata.Unit)
|
||||
if meta.Unit != "" {
|
||||
c.builder.Set(model.MetricUnitLabel, unitNamer.Build(meta.Unit))
|
||||
}
|
||||
}
|
||||
|
||||
@ -219,19 +158,19 @@ func createAttributes(resource pcommon.Resource, attributes pcommon.Map, scope s
|
||||
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 {
|
||||
@ -240,8 +179,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.
|
||||
@ -249,18 +187,13 @@ func createAttributes(resource pcommon.Resource, attributes pcommon.Map, scope s
|
||||
var err error
|
||||
name, err = labelNamer.Build(name)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
return labels.EmptyLabels(), 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) {
|
||||
@ -286,7 +219,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, scope scope, meta Metadata,
|
||||
) error {
|
||||
for x := 0; x < dataPoints.Len(); x++ {
|
||||
if err := c.everyN.checkContext(ctx); err != nil {
|
||||
@ -295,44 +228,48 @@ 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
|
||||
}
|
||||
|
||||
baseName := meta.MetricFamilyName
|
||||
|
||||
// If the sum is unset, it indicates the _sum metric point should be
|
||||
// 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(sumlabels, meta, startTimestamp, timestamp, 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(countlabels, meta, startTimestamp, timestamp, 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 {
|
||||
@ -341,33 +278,35 @@ 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(labels, meta, startTimestamp, timestamp, 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(infLabels, meta, startTimestamp, timestamp, val, exemplars[nextExemplarIdx:]); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
@ -375,76 +314,65 @@ func (c *PrometheusConverter) addHistogramDataPoints(ctx context.Context, dataPo
|
||||
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,
|
||||
c.scratchBuilder.Add(traceIDKey, val)
|
||||
}
|
||||
promExemplar.Labels = append(promExemplar.Labels, promLabel)
|
||||
}
|
||||
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
|
||||
})
|
||||
|
||||
// Only append filtered attributes if it does not cause exemplar
|
||||
// labels to exceed the max number of runes.
|
||||
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...)
|
||||
attrs.Range(func(key string, value pcommon.Value) bool {
|
||||
c.scratchBuilder.Add(key, value.AsString())
|
||||
return true
|
||||
})
|
||||
}
|
||||
c.scratchBuilder.Sort()
|
||||
newExemplar.Labels = c.scratchBuilder.Labels()
|
||||
outputExemplars = append(outputExemplars, newExemplar)
|
||||
}
|
||||
|
||||
promExemplars = append(promExemplars, promExemplar)
|
||||
}
|
||||
|
||||
return promExemplars, nil
|
||||
return outputExemplars, nil
|
||||
}
|
||||
|
||||
// findMinAndMaxTimestamps returns the minimum of minTimestamp and the earliest timestamp in metric and
|
||||
@ -493,7 +421,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, scope scope, meta Metadata,
|
||||
) error {
|
||||
for x := 0; x < dataPoints.Len(); x++ {
|
||||
if err := c.everyN.checkContext(ctx); err != nil {
|
||||
@ -502,122 +430,70 @@ 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
|
||||
}
|
||||
|
||||
baseName := meta.MetricFamilyName
|
||||
|
||||
// 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(sumlabels, meta, startTimestamp, timestamp, 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(countlabels, meta, startTimestamp, timestamp, 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(qtlabels, meta, startTimestamp, timestamp, val, 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
|
||||
}
|
||||
|
||||
// addTypeAndUnitLabels appends type and unit labels to the given labels slice.
|
||||
func addTypeAndUnitLabels(labels []prompb.Label, metadata prompb.MetricMetadata, settings Settings) []prompb.Label {
|
||||
unitNamer := otlptranslator.UnitNamer{UTF8Allowed: settings.AllowUTF8}
|
||||
|
||||
labels = slices.DeleteFunc(labels, func(l prompb.Label) bool {
|
||||
return l.Name == "__type__" || l.Name == "__unit__"
|
||||
})
|
||||
|
||||
labels = append(labels, prompb.Label{Name: "__type__", Value: strings.ToLower(metadata.Type.String())})
|
||||
labels = append(labels, prompb.Label{Name: "__unit__", Value: unitNamer.Build(metadata.Unit)})
|
||||
|
||||
return labels
|
||||
}
|
||||
|
||||
// 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) {
|
||||
// We already have this metric
|
||||
return ts, false
|
||||
}
|
||||
|
||||
// Look for a matching conflict
|
||||
for _, cTS := range c.conflicts[h] {
|
||||
if isSameMetric(cTS, lbls) {
|
||||
// We already have this metric
|
||||
return cTS, false
|
||||
}
|
||||
}
|
||||
|
||||
// New conflict
|
||||
ts = &prompb.TimeSeries{
|
||||
Labels: lbls,
|
||||
}
|
||||
c.conflicts[h] = append(c.conflicts[h], ts)
|
||||
return ts, true
|
||||
}
|
||||
|
||||
// This metric is new
|
||||
ts = &prompb.TimeSeries{
|
||||
Labels: lbls,
|
||||
}
|
||||
c.unique[h] = ts
|
||||
return ts, true
|
||||
c.builder.Set(model.MetricNameLabel, name)
|
||||
return c.builder.Labels()
|
||||
}
|
||||
|
||||
// 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
|
||||
}
|
||||
@ -650,17 +526,24 @@ 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: metadata.Metadata{
|
||||
Type: model.MetricTypeGauge,
|
||||
Help: "Target metadata",
|
||||
},
|
||||
MetricFamilyName: name,
|
||||
}
|
||||
// TODO: should target info have the __type__ metadata label?
|
||||
lbls, err := c.createAttributes(resource, attributes, scope{}, settings, identifyingAttrs, false, Metadata{}, model.MetricNameLabel, name)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
haveIdentifier := false
|
||||
for _, l := range labels {
|
||||
lbls.Range(func(l labels.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.
|
||||
@ -675,18 +558,12 @@ 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(),
|
||||
})
|
||||
if err := c.appender.AppendSample(lbls, meta, 0, timestamp.UnixMilli(), 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(lbls, meta, 0, latestTimestamp.UnixMilli(), float64(1), nil)
|
||||
}
|
||||
|
||||
// convertTimeStamp converts OTLP timestamp in ns to timestamp in ms.
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -27,7 +27,6 @@ import (
|
||||
|
||||
"github.com/prometheus/prometheus/model/histogram"
|
||||
"github.com/prometheus/prometheus/model/value"
|
||||
"github.com/prometheus/prometheus/prompb"
|
||||
"github.com/prometheus/prometheus/util/annotations"
|
||||
)
|
||||
|
||||
@ -36,7 +35,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, temporality pmetric.AggregationTemporality,
|
||||
scope scope, meta Metadata,
|
||||
) (annotations.Annotations, error) {
|
||||
var annots annotations.Annotations
|
||||
for x := 0; x < dataPoints.Len(); x++ {
|
||||
@ -46,34 +46,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,
|
||||
meta.MetricFamilyName,
|
||||
)
|
||||
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(lbls, meta, ct, ts, hp, exemplars); err != nil {
|
||||
return annots, err
|
||||
}
|
||||
}
|
||||
|
||||
return annots, nil
|
||||
@ -81,11 +83,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 +110,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,
|
||||
h := &histogram.Histogram{
|
||||
CounterResetHint: resetHint,
|
||||
Schema: scale,
|
||||
|
||||
ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: p.ZeroCount()},
|
||||
// TODO use zero_threshold, if set, see
|
||||
// https://github.com/open-telemetry/opentelemetry-proto/pull/441
|
||||
ZeroThreshold: defaultZeroThreshold,
|
||||
|
||||
ZeroCount: p.ZeroCount(),
|
||||
PositiveSpans: pSpans,
|
||||
PositiveDeltas: pDeltas,
|
||||
PositiveBuckets: pDeltas,
|
||||
NegativeSpans: nSpans,
|
||||
NegativeDeltas: nDeltas,
|
||||
|
||||
Timestamp: convertTimeStamp(p.Timestamp()),
|
||||
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 +164,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 +193,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 +214,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 +236,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 +253,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, temporality pmetric.AggregationTemporality,
|
||||
scope scope, meta Metadata,
|
||||
) (annotations.Annotations, error) {
|
||||
var annots annotations.Annotations
|
||||
|
||||
@ -267,41 +265,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,
|
||||
meta.MetricFamilyName,
|
||||
)
|
||||
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(lbls, meta, ct, ts, 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 +316,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,
|
||||
h := &histogram.Histogram{
|
||||
CounterResetHint: resetHint,
|
||||
Schema: histogram.CustomBucketsSchema,
|
||||
|
||||
PositiveSpans: positiveSpans,
|
||||
PositiveDeltas: positiveDeltas,
|
||||
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 +339,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))
|
||||
}
|
||||
|
@ -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/labels"
|
||||
"github.com/prometheus/prometheus/model/metadata"
|
||||
)
|
||||
|
||||
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},
|
||||
wantNativeHist: func() *histogram.Histogram {
|
||||
return &histogram.Histogram{
|
||||
Count: 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,
|
||||
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},
|
||||
wantNativeHist: func() *histogram.Histogram {
|
||||
return &histogram.Histogram{
|
||||
Count: 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,
|
||||
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},
|
||||
wantNativeHist: func() *histogram.Histogram {
|
||||
return &histogram.Histogram{
|
||||
Count: 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,
|
||||
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},
|
||||
wantNativeHist: func() *histogram.Histogram {
|
||||
return &histogram.Histogram{
|
||||
Count: 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,
|
||||
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{
|
||||
wantSeries: func() []combinedHistogram {
|
||||
lbls := labels.FromStrings(
|
||||
model.MetricNameLabel, "test_hist",
|
||||
"attr", "test_attr",
|
||||
)
|
||||
return []combinedHistogram{
|
||||
{
|
||||
Count: &prompb.Histogram_CountInt{CountInt: 7},
|
||||
metricFamilyName: "test_hist",
|
||||
ls: lbls,
|
||||
meta: metadata.Metadata{},
|
||||
t: 0,
|
||||
ct: 0,
|
||||
h: &histogram.Histogram{
|
||||
Count: 7,
|
||||
Schema: 1,
|
||||
ZeroThreshold: defaultZeroThreshold,
|
||||
ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: 0},
|
||||
PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 2}},
|
||||
PositiveDeltas: []int64{4, -2},
|
||||
ZeroCount: 0,
|
||||
PositiveSpans: []histogram.Span{{Offset: 0, Length: 2}},
|
||||
PositiveBuckets: []int64{4, -2},
|
||||
},
|
||||
es: []exemplar.Exemplar{{Value: 1}},
|
||||
},
|
||||
{
|
||||
Count: &prompb.Histogram_CountInt{CountInt: 4},
|
||||
metricFamilyName: "test_hist",
|
||||
ls: lbls,
|
||||
meta: metadata.Metadata{},
|
||||
t: 0,
|
||||
ct: 0,
|
||||
h: &histogram.Histogram{
|
||||
Count: 4,
|
||||
Schema: 1,
|
||||
ZeroThreshold: defaultZeroThreshold,
|
||||
ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: 0},
|
||||
PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 3}},
|
||||
PositiveDeltas: []int64{4, -2, -1},
|
||||
},
|
||||
},
|
||||
Exemplars: []prompb.Exemplar{
|
||||
{Value: 1},
|
||||
{Value: 2},
|
||||
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{
|
||||
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{
|
||||
{
|
||||
Count: &prompb.Histogram_CountInt{CountInt: 7},
|
||||
metricFamilyName: "test_hist",
|
||||
ls: lbls,
|
||||
meta: metadata.Metadata{},
|
||||
t: 0,
|
||||
ct: 0,
|
||||
h: &histogram.Histogram{
|
||||
Count: 7,
|
||||
Schema: 1,
|
||||
ZeroThreshold: defaultZeroThreshold,
|
||||
ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: 0},
|
||||
PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 2}},
|
||||
PositiveDeltas: []int64{4, -2},
|
||||
ZeroCount: 0,
|
||||
PositiveSpans: []histogram.Span{{Offset: 0, Length: 2}},
|
||||
PositiveBuckets: []int64{4, -2},
|
||||
},
|
||||
es: []exemplar.Exemplar{{Value: 1}},
|
||||
},
|
||||
{
|
||||
Count: &prompb.Histogram_CountInt{CountInt: 4},
|
||||
metricFamilyName: "test_hist",
|
||||
ls: lbls,
|
||||
meta: metadata.Metadata{},
|
||||
t: 0,
|
||||
ct: 0,
|
||||
h: &histogram.Histogram{
|
||||
Count: 4,
|
||||
Schema: 1,
|
||||
ZeroThreshold: defaultZeroThreshold,
|
||||
ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: 0},
|
||||
PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 3}},
|
||||
PositiveDeltas: []int64{4, -2, -1},
|
||||
},
|
||||
},
|
||||
Exemplars: []prompb.Exemplar{
|
||||
{Value: 1},
|
||||
{Value: 2},
|
||||
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{
|
||||
return []combinedHistogram{
|
||||
{
|
||||
Count: &prompb.Histogram_CountInt{CountInt: 7},
|
||||
metricFamilyName: "test_hist",
|
||||
ls: lbls,
|
||||
meta: metadata.Metadata{},
|
||||
t: 0,
|
||||
ct: 0,
|
||||
h: &histogram.Histogram{
|
||||
Count: 7,
|
||||
Schema: 1,
|
||||
ZeroThreshold: defaultZeroThreshold,
|
||||
ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: 0},
|
||||
PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 2}},
|
||||
PositiveDeltas: []int64{4, -2},
|
||||
ZeroCount: 0,
|
||||
PositiveSpans: []histogram.Span{{Offset: 0, Length: 2}},
|
||||
PositiveBuckets: []int64{4, -2},
|
||||
},
|
||||
es: []exemplar.Exemplar{{Value: 1}},
|
||||
},
|
||||
Exemplars: []prompb.Exemplar{
|
||||
{Value: 1},
|
||||
},
|
||||
},
|
||||
timeSeriesSignature(labelsAnother): {
|
||||
Labels: labelsAnother,
|
||||
Histograms: []prompb.Histogram{
|
||||
{
|
||||
Count: &prompb.Histogram_CountInt{CountInt: 4},
|
||||
metricFamilyName: "test_hist",
|
||||
ls: labelsAnother,
|
||||
meta: metadata.Metadata{},
|
||||
t: 0,
|
||||
ct: 0,
|
||||
h: &histogram.Histogram{
|
||||
Count: 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},
|
||||
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,
|
||||
}
|
||||
@ -856,15 +868,18 @@ func TestPrometheusConverter_addExponentialHistogramDataPoints(t *testing.T) {
|
||||
Settings{
|
||||
PromoteScopeMetadata: tt.promoteScope,
|
||||
},
|
||||
prompb.MetricMetadata{MetricFamilyName: name},
|
||||
pmetric.AggregationTemporalityCumulative,
|
||||
tt.scope,
|
||||
Metadata{
|
||||
MetricFamilyName: name,
|
||||
},
|
||||
)
|
||||
require.NoError(t, err)
|
||||
require.Empty(t, annots)
|
||||
|
||||
require.Equal(t, tt.wantSeries(), converter.unique)
|
||||
require.Empty(t, converter.conflicts)
|
||||
require.NoError(t, mockAppender.Commit())
|
||||
|
||||
requireEqual(t, tt.wantSeries(), mockAppender.histograms)
|
||||
})
|
||||
}
|
||||
}
|
||||
@ -879,7 +894,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,
|
||||
@ -892,7 +907,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,
|
||||
@ -905,7 +920,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,
|
||||
@ -918,7 +933,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,
|
||||
@ -931,7 +946,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,
|
||||
@ -948,7 +963,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,
|
||||
@ -1007,7 +1022,7 @@ func TestHistogramToCustomBucketsHistogram(t *testing.T) {
|
||||
tests := []struct {
|
||||
name string
|
||||
hist func() pmetric.HistogramDataPoint
|
||||
wantNativeHist func() prompb.Histogram
|
||||
wantNativeHist func() *histogram.Histogram
|
||||
wantErrMessage string
|
||||
}{
|
||||
{
|
||||
@ -1023,15 +1038,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},
|
||||
wantNativeHist: func() *histogram.Histogram {
|
||||
return &histogram.Histogram{
|
||||
Count: 2,
|
||||
Sum: 10.1,
|
||||
Schema: -53,
|
||||
PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 2}},
|
||||
PositiveDeltas: []int64{1, 0},
|
||||
PositiveSpans: []histogram.Span{{Offset: 0, Length: 2}},
|
||||
PositiveBuckets: []int64{1, 0},
|
||||
CustomValues: []float64{0, 1},
|
||||
Timestamp: 500,
|
||||
}
|
||||
},
|
||||
},
|
||||
@ -1047,14 +1061,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},
|
||||
wantNativeHist: func() *histogram.Histogram {
|
||||
return &histogram.Histogram{
|
||||
Count: 4,
|
||||
Schema: -53,
|
||||
PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 2}},
|
||||
PositiveDeltas: []int64{2, 0},
|
||||
PositiveSpans: []histogram.Span{{Offset: 0, Length: 2}},
|
||||
PositiveBuckets: []int64{2, 0},
|
||||
CustomValues: []float64{0, 1},
|
||||
Timestamp: 500,
|
||||
}
|
||||
},
|
||||
},
|
||||
@ -1093,7 +1106,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",
|
||||
@ -1122,36 +1135,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{
|
||||
wantSeries: func() []combinedHistogram {
|
||||
lbls := labels.FromStrings(
|
||||
model.MetricNameLabel, "test_hist_to_nhcb",
|
||||
"attr", "test_attr",
|
||||
)
|
||||
return []combinedHistogram{
|
||||
{
|
||||
Count: &prompb.Histogram_CountInt{CountInt: 3},
|
||||
metricFamilyName: "test_hist_to_nhcb",
|
||||
ls: lbls,
|
||||
meta: metadata.Metadata{},
|
||||
t: 0,
|
||||
ct: 0,
|
||||
h: &histogram.Histogram{
|
||||
Count: 3,
|
||||
Sum: 3,
|
||||
Schema: -53,
|
||||
PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 3}},
|
||||
PositiveDeltas: []int64{2, -2, 1},
|
||||
PositiveSpans: []histogram.Span{{Offset: 0, Length: 3}},
|
||||
PositiveBuckets: []int64{2, -2, 1},
|
||||
CustomValues: []float64{5, 10},
|
||||
},
|
||||
es: []exemplar.Exemplar{{Value: 1}},
|
||||
},
|
||||
{
|
||||
Count: &prompb.Histogram_CountInt{CountInt: 11},
|
||||
metricFamilyName: "test_hist_to_nhcb",
|
||||
ls: lbls,
|
||||
meta: metadata.Metadata{},
|
||||
t: 0,
|
||||
ct: 0,
|
||||
h: &histogram.Histogram{
|
||||
Count: 11,
|
||||
Sum: 5,
|
||||
Schema: -53,
|
||||
PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 3}},
|
||||
PositiveDeltas: []int64{3, 5, -8},
|
||||
PositiveSpans: []histogram.Span{{Offset: 0, Length: 3}},
|
||||
PositiveBuckets: []int64{3, 5, -8},
|
||||
CustomValues: []float64{0, 1},
|
||||
},
|
||||
},
|
||||
Exemplars: []prompb.Exemplar{
|
||||
{Value: 1},
|
||||
{Value: 2},
|
||||
},
|
||||
es: []exemplar.Exemplar{{Value: 2}},
|
||||
},
|
||||
}
|
||||
},
|
||||
@ -1183,41 +1203,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{
|
||||
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{
|
||||
{
|
||||
Count: &prompb.Histogram_CountInt{CountInt: 3},
|
||||
metricFamilyName: "test_hist_to_nhcb",
|
||||
ls: lbls,
|
||||
meta: metadata.Metadata{},
|
||||
t: 0,
|
||||
ct: 0,
|
||||
h: &histogram.Histogram{
|
||||
Count: 3,
|
||||
Sum: 3,
|
||||
Schema: -53,
|
||||
PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 3}},
|
||||
PositiveDeltas: []int64{2, -2, 1},
|
||||
PositiveSpans: []histogram.Span{{Offset: 0, Length: 3}},
|
||||
PositiveBuckets: []int64{2, -2, 1},
|
||||
CustomValues: []float64{5, 10},
|
||||
},
|
||||
es: []exemplar.Exemplar{{Value: 1}},
|
||||
},
|
||||
{
|
||||
Count: &prompb.Histogram_CountInt{CountInt: 11},
|
||||
metricFamilyName: "test_hist_to_nhcb",
|
||||
ls: lbls,
|
||||
meta: metadata.Metadata{},
|
||||
t: 0,
|
||||
ct: 0,
|
||||
h: &histogram.Histogram{
|
||||
Count: 11,
|
||||
Sum: 5,
|
||||
Schema: -53,
|
||||
PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 3}},
|
||||
PositiveDeltas: []int64{3, 5, -8},
|
||||
PositiveSpans: []histogram.Span{{Offset: 0, Length: 3}},
|
||||
PositiveBuckets: []int64{3, 5, -8},
|
||||
CustomValues: []float64{0, 1},
|
||||
},
|
||||
},
|
||||
Exemplars: []prompb.Exemplar{
|
||||
{Value: 1},
|
||||
{Value: 2},
|
||||
},
|
||||
es: []exemplar.Exemplar{{Value: 2}},
|
||||
},
|
||||
}
|
||||
},
|
||||
@ -1249,48 +1276,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{
|
||||
return []combinedHistogram{
|
||||
{
|
||||
Count: &prompb.Histogram_CountInt{CountInt: 6},
|
||||
metricFamilyName: "test_hist_to_nhcb",
|
||||
ls: lbls,
|
||||
meta: metadata.Metadata{},
|
||||
t: 0,
|
||||
ct: 0,
|
||||
h: &histogram.Histogram{
|
||||
Count: 6,
|
||||
Sum: 3,
|
||||
Schema: -53,
|
||||
PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 2}},
|
||||
PositiveDeltas: []int64{4, -2},
|
||||
PositiveSpans: []histogram.Span{{Offset: 0, Length: 2}},
|
||||
PositiveBuckets: []int64{4, -2},
|
||||
CustomValues: []float64{0, 1},
|
||||
},
|
||||
es: []exemplar.Exemplar{{Value: 1}},
|
||||
},
|
||||
Exemplars: []prompb.Exemplar{
|
||||
{Value: 1},
|
||||
},
|
||||
},
|
||||
timeSeriesSignature(labelsAnother): {
|
||||
Labels: labelsAnother,
|
||||
Histograms: []prompb.Histogram{
|
||||
{
|
||||
Count: &prompb.Histogram_CountInt{CountInt: 11},
|
||||
metricFamilyName: "test_hist_to_nhcb",
|
||||
ls: labelsAnother,
|
||||
meta: metadata.Metadata{},
|
||||
t: 0,
|
||||
ct: 0,
|
||||
h: &histogram.Histogram{
|
||||
Count: 11,
|
||||
Sum: 5,
|
||||
Schema: -53,
|
||||
PositiveSpans: []prompb.BucketSpan{{Offset: 0, Length: 2}},
|
||||
PositiveDeltas: []int64{3, 5},
|
||||
PositiveSpans: []histogram.Span{{Offset: 0, Length: 2}},
|
||||
PositiveBuckets: []int64{3, 5},
|
||||
CustomValues: []float64{0, 1},
|
||||
},
|
||||
},
|
||||
Exemplars: []prompb.Exemplar{
|
||||
{Value: 2},
|
||||
},
|
||||
es: []exemplar.Exemplar{{Value: 2}},
|
||||
},
|
||||
}
|
||||
},
|
||||
@ -1300,7 +1327,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,
|
||||
}
|
||||
@ -1314,16 +1342,19 @@ func TestPrometheusConverter_addCustomBucketsHistogramDataPoints(t *testing.T) {
|
||||
ConvertHistogramsToNHCB: true,
|
||||
PromoteScopeMetadata: tt.promoteScope,
|
||||
},
|
||||
prompb.MetricMetadata{MetricFamilyName: name},
|
||||
pmetric.AggregationTemporalityCumulative,
|
||||
tt.scope,
|
||||
Metadata{
|
||||
MetricFamilyName: name,
|
||||
},
|
||||
)
|
||||
|
||||
require.NoError(t, err)
|
||||
require.Empty(t, annots)
|
||||
|
||||
require.Equal(t, tt.wantSeries(), converter.unique)
|
||||
require.Empty(t, converter.conflicts)
|
||||
require.NoError(t, mockAppender.Commit())
|
||||
|
||||
requireEqual(t, tt.wantSeries(), mockAppender.histograms)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
@ -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/labels"
|
||||
"github.com/prometheus/prometheus/model/metadata"
|
||||
"github.com/prometheus/prometheus/util/annotations"
|
||||
)
|
||||
|
||||
@ -58,16 +58,17 @@ 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
|
||||
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{},
|
||||
scratchBuilder: labels.NewScratchBuilder(0),
|
||||
builder: labels.NewBuilder(labels.EmptyLabels()),
|
||||
appender: appender,
|
||||
}
|
||||
}
|
||||
|
||||
@ -120,6 +121,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()
|
||||
|
||||
@ -130,7 +132,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)
|
||||
@ -175,13 +176,14 @@ func (c *PrometheusConverter) FromMetrics(ctx context.Context, md pmetric.Metric
|
||||
errs = multierr.Append(errs, err)
|
||||
continue
|
||||
}
|
||||
metadata := prompb.MetricMetadata{
|
||||
meta := Metadata{
|
||||
Metadata: metadata.Metadata{
|
||||
Type: otelMetricTypeToPromMetricType(metric),
|
||||
MetricFamilyName: promName,
|
||||
Unit: unitNamer.Build(metric.Unit()),
|
||||
Help: metric.Description(),
|
||||
Unit: metric.Unit(),
|
||||
},
|
||||
MetricFamilyName: promName,
|
||||
}
|
||||
c.metadata = append(c.metadata, metadata)
|
||||
|
||||
// handle individual metrics based on type
|
||||
//exhaustive:enforce
|
||||
@ -192,7 +194,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, scope, meta); err != nil {
|
||||
errs = multierr.Append(errs, err)
|
||||
if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) {
|
||||
return
|
||||
@ -204,7 +206,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, settings, metadata, scope); err != nil {
|
||||
if err := c.addSumNumberDataPoints(ctx, dataPoints, resource, settings, scope, meta); err != nil {
|
||||
errs = multierr.Append(errs, err)
|
||||
if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) {
|
||||
return
|
||||
@ -218,7 +220,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, temporality, scope, meta,
|
||||
)
|
||||
annots.Merge(ws)
|
||||
if err != nil {
|
||||
@ -228,7 +230,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, scope, meta); err != nil {
|
||||
errs = multierr.Append(errs, err)
|
||||
if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) {
|
||||
return
|
||||
@ -246,9 +248,9 @@ func (c *PrometheusConverter) FromMetrics(ctx context.Context, md pmetric.Metric
|
||||
dataPoints,
|
||||
resource,
|
||||
settings,
|
||||
metadata,
|
||||
temporality,
|
||||
scope,
|
||||
meta,
|
||||
)
|
||||
annots.Merge(ws)
|
||||
if err != nil {
|
||||
@ -263,7 +265,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, scope, meta); err != nil {
|
||||
errs = multierr.Append(errs, err)
|
||||
if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) {
|
||||
return
|
||||
@ -277,72 +279,13 @@ 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
|
||||
}
|
||||
|
||||
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 {
|
||||
@ -360,30 +303,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()})
|
||||
var normalized string
|
||||
normalized, err = labelNamer.Build(name)
|
||||
if err != nil {
|
||||
return false
|
||||
}
|
||||
if builder.Get(normalized) == "" {
|
||||
builder.Set(normalized, value.AsString())
|
||||
}
|
||||
}
|
||||
return true
|
||||
})
|
||||
} else {
|
||||
promotedAttrs = make([]prompb.Label, 0, len(s.attrs))
|
||||
return err
|
||||
}
|
||||
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()})
|
||||
var normalized string
|
||||
normalized, err = labelNamer.Build(name)
|
||||
if err != nil {
|
||||
return false
|
||||
}
|
||||
if builder.Get(normalized) == "" {
|
||||
builder.Set(normalized, value.AsString())
|
||||
}
|
||||
}
|
||||
return true
|
||||
})
|
||||
}
|
||||
sort.Stable(ByLabelName(promotedAttrs))
|
||||
return promotedAttrs
|
||||
return err
|
||||
}
|
||||
|
@ -19,19 +19,23 @@ package prometheusremotewrite
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"sort"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
"github.com/prometheus/common/model"
|
||||
"github.com/prometheus/common/promslog"
|
||||
"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/exemplar"
|
||||
"github.com/prometheus/prometheus/model/histogram"
|
||||
"github.com/prometheus/prometheus/model/labels"
|
||||
"github.com/prometheus/prometheus/prompb"
|
||||
"github.com/prometheus/prometheus/util/testutil"
|
||||
"github.com/prometheus/prometheus/model/metadata"
|
||||
"github.com/prometheus/prometheus/storage"
|
||||
)
|
||||
|
||||
func TestFromMetrics(t *testing.T) {
|
||||
@ -77,9 +81,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 +94,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,15 +104,14 @@ func TestFromMetrics(t *testing.T) {
|
||||
require.NoError(t, err)
|
||||
require.Empty(t, annots)
|
||||
|
||||
testutil.RequireEqual(t, expMetadata, converter.Metadata())
|
||||
require.NoError(t, mockAppender.Commit())
|
||||
|
||||
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)
|
||||
lbls := s.ls
|
||||
if lbls.Get(labels.MetricName) == "target_info" {
|
||||
tgtInfoCount++
|
||||
require.Equal(t, "test-namespace/test-service", lbls.Get("job"))
|
||||
@ -156,7 +153,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(),
|
||||
@ -164,25 +162,21 @@ func TestFromMetrics(t *testing.T) {
|
||||
)
|
||||
require.NoError(t, err)
|
||||
require.Empty(t, annots)
|
||||
|
||||
series := converter.TimeSeries()
|
||||
require.NoError(t, mockAppender.Commit())
|
||||
|
||||
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 +189,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 +222,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 +255,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 +286,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,19 +301,10 @@ 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()
|
||||
mockAppender := &mockCombinedAppender{}
|
||||
converter := NewPrometheusConverter(mockAppender)
|
||||
annots, err := converter.FromMetrics(
|
||||
context.Background(),
|
||||
request.Metrics(),
|
||||
@ -330,46 +314,54 @@ func TestFromMetrics(t *testing.T) {
|
||||
)
|
||||
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"))
|
||||
require.NoError(t, mockAppender.Commit())
|
||||
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.
|
||||
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)
|
||||
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:])
|
||||
})
|
||||
}
|
||||
|
||||
@ -381,7 +373,8 @@ func TestTemporality(t *testing.T) {
|
||||
allowDelta bool
|
||||
convertToNHCB bool
|
||||
inputSeries []pmetric.Metric
|
||||
expectedSeries []prompb.TimeSeries
|
||||
expectedSamples []combinedSample
|
||||
expectedHistograms []combinedHistogram
|
||||
expectedError string
|
||||
}{
|
||||
{
|
||||
@ -391,9 +384,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 +396,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 +408,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 +420,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 +432,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 +443,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 +454,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 +466,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 +478,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 +490,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 +503,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,7 +516,7 @@ 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),
|
||||
expectedSamples: createPromClassicHistogramSeries("test_histogram_2", ts, model.MetricTypeHistogram),
|
||||
expectedError: `invalid temporality and type combination for metric "test_histogram_1"`,
|
||||
},
|
||||
{
|
||||
@ -534,9 +527,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 +537,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,7 +553,6 @@ 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`,
|
||||
},
|
||||
}
|
||||
@ -575,7 +567,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,
|
||||
@ -588,11 +581,11 @@ func TestTemporality(t *testing.T) {
|
||||
} else {
|
||||
require.NoError(t, err)
|
||||
}
|
||||
|
||||
series := c.TimeSeries()
|
||||
require.NoError(t, mockAppender.Commit())
|
||||
|
||||
// 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 +596,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 +604,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 +642,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},
|
||||
h: &histogram.Histogram{
|
||||
Count: 1,
|
||||
Sum: 5,
|
||||
Schema: 0,
|
||||
ZeroThreshold: 1e-128,
|
||||
ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: 0},
|
||||
Timestamp: ts.UnixMilli(),
|
||||
ResetHint: hint,
|
||||
},
|
||||
ZeroCount: 0,
|
||||
CounterResetHint: hint,
|
||||
},
|
||||
}
|
||||
}
|
||||
@ -685,72 +677,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},
|
||||
t: ts.UnixMilli(),
|
||||
h: &histogram.Histogram{
|
||||
Count: 20,
|
||||
Sum: 30,
|
||||
Schema: -53,
|
||||
ZeroThreshold: 0,
|
||||
ZeroCount: nil,
|
||||
PositiveSpans: []prompb.BucketSpan{
|
||||
PositiveSpans: []histogram.Span{
|
||||
{
|
||||
Length: 3,
|
||||
},
|
||||
},
|
||||
PositiveDeltas: []int64{10, 0, -10},
|
||||
PositiveBuckets: []int64{10, 0, -10},
|
||||
CustomValues: []float64{1, 2},
|
||||
Timestamp: ts.UnixMilli(),
|
||||
ResetHint: hint,
|
||||
},
|
||||
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 +768,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 +807,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
|
||||
@ -989,19 +968,23 @@ func BenchmarkPrometheusConverter_FromMetrics(b *testing.B) {
|
||||
settings,
|
||||
pmetric.AggregationTemporalityCumulative,
|
||||
)
|
||||
appMetrics := NewCombinedAppenderMetrics(prometheus.NewRegistry())
|
||||
noOpLogger := promslog.NewNopLogger()
|
||||
b.ResetTimer()
|
||||
|
||||
for range b.N {
|
||||
converter := NewPrometheusConverter()
|
||||
app := &noOpAppender{}
|
||||
mockAppender := NewCombinedAppender(app, noOpLogger, false, appMetrics)
|
||||
converter := NewPrometheusConverter(mockAppender)
|
||||
annots, err := converter.FromMetrics(context.Background(), payload.Metrics(), settings)
|
||||
require.NoError(b, err)
|
||||
require.Empty(b, annots)
|
||||
if histogramCount+nonHistogramCount > 0 {
|
||||
require.NotEmpty(b, converter.TimeSeries())
|
||||
require.NotEmpty(b, converter.Metadata())
|
||||
require.Positive(b, app.samples+app.histograms)
|
||||
require.Positive(b, app.metadata)
|
||||
} else {
|
||||
require.Empty(b, converter.TimeSeries())
|
||||
require.Empty(b, converter.Metadata())
|
||||
require.Zero(b, app.samples+app.histograms)
|
||||
require.Zero(b, app.metadata)
|
||||
}
|
||||
}
|
||||
})
|
||||
@ -1016,10 +999,57 @@ func BenchmarkPrometheusConverter_FromMetrics(b *testing.B) {
|
||||
}
|
||||
}
|
||||
|
||||
type noOpAppender struct {
|
||||
samples int
|
||||
histograms int
|
||||
metadata int
|
||||
}
|
||||
|
||||
var _ storage.Appender = &noOpAppender{}
|
||||
|
||||
func (a *noOpAppender) Append(_ storage.SeriesRef, _ labels.Labels, _ int64, _ float64) (storage.SeriesRef, error) {
|
||||
a.samples++
|
||||
return 1, nil
|
||||
}
|
||||
|
||||
func (*noOpAppender) AppendCTZeroSample(_ storage.SeriesRef, _ labels.Labels, _, _ int64) (storage.SeriesRef, error) {
|
||||
return 1, nil
|
||||
}
|
||||
|
||||
func (a *noOpAppender) AppendHistogram(_ storage.SeriesRef, _ labels.Labels, _ int64, _ *histogram.Histogram, _ *histogram.FloatHistogram) (storage.SeriesRef, error) {
|
||||
a.histograms++
|
||||
return 1, nil
|
||||
}
|
||||
|
||||
func (*noOpAppender) AppendHistogramCTZeroSample(_ storage.SeriesRef, _ labels.Labels, _, _ int64, _ *histogram.Histogram, _ *histogram.FloatHistogram) (storage.SeriesRef, error) {
|
||||
return 1, nil
|
||||
}
|
||||
|
||||
func (a *noOpAppender) UpdateMetadata(_ storage.SeriesRef, _ labels.Labels, _ metadata.Metadata) (storage.SeriesRef, error) {
|
||||
a.metadata++
|
||||
return 1, nil
|
||||
}
|
||||
|
||||
func (*noOpAppender) AppendExemplar(_ storage.SeriesRef, _ labels.Labels, _ exemplar.Exemplar) (storage.SeriesRef, error) {
|
||||
return 1, nil
|
||||
}
|
||||
|
||||
func (*noOpAppender) Commit() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (*noOpAppender) Rollback() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (*noOpAppender) SetOptions(_ *storage.AppendOptions) {
|
||||
panic("not implemented")
|
||||
}
|
||||
|
||||
type wantPrometheusMetric struct {
|
||||
name string
|
||||
familyName string
|
||||
metricType prompb.MetricMetadata_MetricType
|
||||
metricType model.MetricType
|
||||
description string
|
||||
unit string
|
||||
}
|
||||
@ -1066,11 +1096,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),
|
||||
@ -1108,11 +1138,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),
|
||||
@ -1142,11 +1172,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),
|
||||
@ -1172,7 +1202,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",
|
||||
})
|
||||
|
@ -25,11 +25,10 @@ import (
|
||||
"go.opentelemetry.io/collector/pdata/pmetric"
|
||||
|
||||
"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, scope scope, meta Metadata,
|
||||
) error {
|
||||
for x := 0; x < dataPoints.Len(); x++ {
|
||||
if err := c.everyN.checkContext(ctx); err != nil {
|
||||
@ -37,42 +36,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,
|
||||
meta.MetricFamilyName,
|
||||
)
|
||||
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(labels, meta, ct, ts, 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, settings Settings, metadata prompb.MetricMetadata, scope scope,
|
||||
resource pcommon.Resource, settings Settings, scope scope, meta Metadata,
|
||||
) error {
|
||||
for x := 0; x < dataPoints.Len(); x++ {
|
||||
if err := c.everyN.checkContext(ctx); err != nil {
|
||||
@ -80,41 +79,38 @@ 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,
|
||||
meta.MetricFamilyName,
|
||||
)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
sample := &prompb.Sample{
|
||||
// convert ns to ms
|
||||
Timestamp: convertTimeStamp(pt.Timestamp()),
|
||||
return nil
|
||||
}
|
||||
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 := c.addSample(sample, lbls)
|
||||
if ts != nil {
|
||||
exemplars, err := getPromExemplars[pmetric.NumberDataPoint](ctx, &c.everyN, pt)
|
||||
ts := convertTimeStamp(pt.Timestamp())
|
||||
ct := convertTimeStamp(pt.StartTimestamp())
|
||||
exemplars, err := c.getPromExemplars(ctx, pt.Exemplars())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
ts.Exemplars = append(ts.Exemplars, exemplars...)
|
||||
if err := c.appender.AppendSample(lbls, meta, ct, ts, val, exemplars); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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/labels"
|
||||
"github.com/prometheus/prometheus/model/metadata"
|
||||
)
|
||||
|
||||
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{
|
||||
want: func() []combinedSample {
|
||||
lbls := labels.FromStrings(
|
||||
model.MetricNameLabel, "test",
|
||||
)
|
||||
return []combinedSample{
|
||||
{
|
||||
Value: 1,
|
||||
Timestamp: convertTimeStamp(pcommon.Timestamp(ts)),
|
||||
},
|
||||
},
|
||||
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{
|
||||
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{
|
||||
{
|
||||
Value: 1,
|
||||
Timestamp: convertTimeStamp(pcommon.Timestamp(ts)),
|
||||
},
|
||||
},
|
||||
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(),
|
||||
@ -123,12 +122,14 @@ func TestPrometheusConverter_addGaugeNumberDataPoints(t *testing.T) {
|
||||
Settings{
|
||||
PromoteScopeMetadata: tt.promoteScope,
|
||||
},
|
||||
prompb.MetricMetadata{MetricFamilyName: metric.Name()},
|
||||
tt.scope,
|
||||
Metadata{
|
||||
MetricFamilyName: metric.Name(),
|
||||
},
|
||||
)
|
||||
require.NoError(t, mockAppender.Commit())
|
||||
|
||||
require.Equal(t, tt.want(), converter.unique)
|
||||
require.Empty(t, converter.conflicts)
|
||||
requireEqual(t, tt.want(), mockAppender.samples)
|
||||
})
|
||||
}
|
||||
}
|
||||
@ -151,7 +152,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",
|
||||
@ -165,19 +166,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{
|
||||
want: func() []combinedSample {
|
||||
lbls := labels.FromStrings(
|
||||
model.MetricNameLabel, "test",
|
||||
)
|
||||
return []combinedSample{
|
||||
{
|
||||
Value: 1,
|
||||
Timestamp: convertTimeStamp(ts),
|
||||
},
|
||||
},
|
||||
metricFamilyName: "test",
|
||||
ls: lbls,
|
||||
meta: metadata.Metadata{},
|
||||
t: convertTimeStamp(ts),
|
||||
v: 1,
|
||||
},
|
||||
}
|
||||
},
|
||||
@ -194,24 +193,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{
|
||||
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{
|
||||
{
|
||||
Value: 1,
|
||||
Timestamp: convertTimeStamp(ts),
|
||||
},
|
||||
},
|
||||
metricFamilyName: "test",
|
||||
ls: lbls,
|
||||
meta: metadata.Metadata{},
|
||||
t: convertTimeStamp(ts),
|
||||
v: 1,
|
||||
},
|
||||
}
|
||||
},
|
||||
@ -230,18 +227,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},
|
||||
},
|
||||
},
|
||||
@ -265,16 +262,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_sum"},
|
||||
}
|
||||
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",
|
||||
)
|
||||
return []combinedSample{
|
||||
{
|
||||
metricFamilyName: "test_sum",
|
||||
ls: lbls,
|
||||
meta: metadata.Metadata{},
|
||||
t: convertTimeStamp(ts),
|
||||
ct: convertTimeStamp(ts),
|
||||
v: 1,
|
||||
},
|
||||
}
|
||||
},
|
||||
@ -294,16 +293,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,
|
||||
},
|
||||
}
|
||||
},
|
||||
@ -323,16 +323,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,
|
||||
},
|
||||
}
|
||||
},
|
||||
@ -341,7 +342,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(),
|
||||
@ -350,12 +352,14 @@ func TestPrometheusConverter_addSumNumberDataPoints(t *testing.T) {
|
||||
Settings{
|
||||
PromoteScopeMetadata: tt.promoteScope,
|
||||
},
|
||||
prompb.MetricMetadata{MetricFamilyName: metric.Name()},
|
||||
tt.scope,
|
||||
Metadata{
|
||||
MetricFamilyName: metric.Name(),
|
||||
},
|
||||
)
|
||||
require.NoError(t, mockAppender.Commit())
|
||||
|
||||
require.Equal(t, tt.want(), converter.unique)
|
||||
require.Empty(t, converter.conflicts)
|
||||
requireEqual(t, tt.want(), mockAppender.samples)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
@ -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
|
||||
}
|
||||
|
@ -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
|
||||
}
|
@ -537,25 +537,29 @@ 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,
|
||||
},
|
||||
config: configFunc,
|
||||
allowDeltaTemporality: opts.NativeDelta,
|
||||
lookbackDelta: opts.LookbackDelta,
|
||||
ingestCTZeroSample: opts.IngestCTZeroSample,
|
||||
enableTypeAndUnitLabels: opts.EnableTypeAndUnitLabels,
|
||||
// Register metrics.
|
||||
metrics: otlptranslator.NewCombinedAppenderMetrics(reg),
|
||||
}
|
||||
|
||||
wh := &otlpWriteHandler{logger: logger, defaultConsumer: ex}
|
||||
@ -589,18 +593,26 @@ 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
|
||||
|
||||
// Metrics.
|
||||
metrics otlptranslator.CombinedAppenderMetrics
|
||||
}
|
||||
|
||||
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.ingestCTZeroSample, rw.metrics)
|
||||
converter := otlptranslator.NewPrometheusConverter(combinedAppender)
|
||||
annots, err := converter.FromMetrics(ctx, md, otlptranslator.Settings{
|
||||
AddMetricSuffixes: otlpCfg.TranslationStrategy.ShouldAddSuffixes(),
|
||||
AllowUTF8: !otlpCfg.TranslationStrategy.ShouldEscape(),
|
||||
@ -612,18 +624,18 @@ func (rw *rwExporter) ConsumeMetrics(ctx context.Context, md pmetric.Metrics) er
|
||||
LookbackDelta: rw.lookbackDelta,
|
||||
EnableTypeAndUnitLabels: rw.enableTypeAndUnitLabels,
|
||||
})
|
||||
|
||||
defer func() {
|
||||
if err != nil {
|
||||
rw.logger.Warn("Error translating OTLP metrics to Prometheus write request", "err", err)
|
||||
_ = 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
|
||||
}
|
||||
|
||||
|
@ -948,8 +948,8 @@ func (m *mockAppendable) Append(_ storage.SeriesRef, l labels.Labels, t int64, v
|
||||
if m.appendSampleErr != nil {
|
||||
return 0, m.appendSampleErr
|
||||
}
|
||||
|
||||
latestTs := m.latestSample[l.Hash()]
|
||||
hash := l.Hash()
|
||||
latestTs := m.latestSample[hash]
|
||||
if t < latestTs {
|
||||
return 0, storage.ErrOutOfOrderSample
|
||||
}
|
||||
@ -964,9 +964,9 @@ func (m *mockAppendable) Append(_ storage.SeriesRef, l labels.Labels, t int64, v
|
||||
return 0, tsdb.ErrInvalidSample
|
||||
}
|
||||
|
||||
m.latestSample[l.Hash()] = t
|
||||
m.latestSample[hash] = t
|
||||
m.samples = append(m.samples, mockSample{l, t, v})
|
||||
return 0, nil
|
||||
return storage.SeriesRef(hash), nil
|
||||
}
|
||||
|
||||
func (m *mockAppendable) Commit() error {
|
||||
@ -984,12 +984,12 @@ func (m *mockAppendable) Rollback() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *mockAppendable) AppendExemplar(_ storage.SeriesRef, l labels.Labels, e exemplar.Exemplar) (storage.SeriesRef, error) {
|
||||
func (m *mockAppendable) AppendExemplar(ref storage.SeriesRef, l labels.Labels, e exemplar.Exemplar) (storage.SeriesRef, error) {
|
||||
if m.appendExemplarErr != nil {
|
||||
return 0, m.appendExemplarErr
|
||||
}
|
||||
|
||||
latestTs := m.latestExemplar[l.Hash()]
|
||||
latestTs := m.latestExemplar[uint64(ref)]
|
||||
if e.Ts < latestTs {
|
||||
return 0, storage.ErrOutOfOrderExemplar
|
||||
}
|
||||
@ -997,21 +997,21 @@ func (m *mockAppendable) AppendExemplar(_ storage.SeriesRef, l labels.Labels, e
|
||||
return 0, storage.ErrDuplicateExemplar
|
||||
}
|
||||
|
||||
m.latestExemplar[l.Hash()] = e.Ts
|
||||
m.latestExemplar[uint64(ref)] = e.Ts
|
||||
m.exemplars = append(m.exemplars, mockExemplar{l, e.Labels, e.Ts, e.Value})
|
||||
return 0, nil
|
||||
return ref, nil
|
||||
}
|
||||
|
||||
func (m *mockAppendable) AppendHistogram(_ storage.SeriesRef, l labels.Labels, t int64, h *histogram.Histogram, fh *histogram.FloatHistogram) (storage.SeriesRef, error) {
|
||||
if m.appendHistogramErr != nil {
|
||||
return 0, m.appendHistogramErr
|
||||
}
|
||||
|
||||
hash := l.Hash()
|
||||
var latestTs int64
|
||||
if h != nil {
|
||||
latestTs = m.latestHistogram[l.Hash()]
|
||||
latestTs = m.latestHistogram[hash]
|
||||
} else {
|
||||
latestTs = m.latestFloatHist[l.Hash()]
|
||||
latestTs = m.latestFloatHist[hash]
|
||||
}
|
||||
if t < latestTs {
|
||||
return 0, storage.ErrOutOfOrderSample
|
||||
@ -1028,12 +1028,12 @@ func (m *mockAppendable) AppendHistogram(_ storage.SeriesRef, l labels.Labels, t
|
||||
}
|
||||
|
||||
if h != nil {
|
||||
m.latestHistogram[l.Hash()] = t
|
||||
m.latestHistogram[hash] = t
|
||||
} else {
|
||||
m.latestFloatHist[l.Hash()] = t
|
||||
m.latestFloatHist[hash] = t
|
||||
}
|
||||
m.histograms = append(m.histograms, mockHistogram{l, t, h, fh})
|
||||
return 0, nil
|
||||
return storage.SeriesRef(hash), nil
|
||||
}
|
||||
|
||||
func (m *mockAppendable) AppendHistogramCTZeroSample(_ storage.SeriesRef, l labels.Labels, t, ct int64, h *histogram.Histogram, _ *histogram.FloatHistogram) (storage.SeriesRef, error) {
|
||||
@ -1045,12 +1045,12 @@ func (m *mockAppendable) AppendHistogramCTZeroSample(_ storage.SeriesRef, l labe
|
||||
if ct > t {
|
||||
return 0, storage.ErrOutOfOrderSample
|
||||
}
|
||||
|
||||
hash := l.Hash()
|
||||
var latestTs int64
|
||||
if h != nil {
|
||||
latestTs = m.latestHistogram[l.Hash()]
|
||||
latestTs = m.latestHistogram[hash]
|
||||
} else {
|
||||
latestTs = m.latestFloatHist[l.Hash()]
|
||||
latestTs = m.latestFloatHist[hash]
|
||||
}
|
||||
if ct < latestTs {
|
||||
return 0, storage.ErrOutOfOrderSample
|
||||
@ -1068,22 +1068,22 @@ func (m *mockAppendable) AppendHistogramCTZeroSample(_ storage.SeriesRef, l labe
|
||||
}
|
||||
|
||||
if h != nil {
|
||||
m.latestHistogram[l.Hash()] = ct
|
||||
m.latestHistogram[hash] = ct
|
||||
m.histograms = append(m.histograms, mockHistogram{l, ct, &histogram.Histogram{}, nil})
|
||||
} else {
|
||||
m.latestFloatHist[l.Hash()] = ct
|
||||
m.latestFloatHist[hash] = ct
|
||||
m.histograms = append(m.histograms, mockHistogram{l, ct, nil, &histogram.FloatHistogram{}})
|
||||
}
|
||||
return 0, nil
|
||||
return storage.SeriesRef(hash), nil
|
||||
}
|
||||
|
||||
func (m *mockAppendable) UpdateMetadata(_ storage.SeriesRef, l labels.Labels, mp metadata.Metadata) (storage.SeriesRef, error) {
|
||||
func (m *mockAppendable) UpdateMetadata(ref storage.SeriesRef, l labels.Labels, mp metadata.Metadata) (storage.SeriesRef, error) {
|
||||
if m.updateMetadataErr != nil {
|
||||
return 0, m.updateMetadataErr
|
||||
}
|
||||
|
||||
m.metadata = append(m.metadata, mockMetadata{l: l, m: mp})
|
||||
return 0, nil
|
||||
return ref, nil
|
||||
}
|
||||
|
||||
func (m *mockAppendable) AppendCTZeroSample(_ storage.SeriesRef, l labels.Labels, t, ct int64) (storage.SeriesRef, error) {
|
||||
@ -1095,8 +1095,8 @@ func (m *mockAppendable) AppendCTZeroSample(_ storage.SeriesRef, l labels.Labels
|
||||
if ct > t {
|
||||
return 0, storage.ErrOutOfOrderSample
|
||||
}
|
||||
|
||||
latestTs := m.latestSample[l.Hash()]
|
||||
hash := l.Hash()
|
||||
latestTs := m.latestSample[hash]
|
||||
if ct < latestTs {
|
||||
return 0, storage.ErrOutOfOrderSample
|
||||
}
|
||||
@ -1111,7 +1111,7 @@ func (m *mockAppendable) AppendCTZeroSample(_ storage.SeriesRef, l labels.Labels
|
||||
return 0, tsdb.ErrInvalidSample
|
||||
}
|
||||
|
||||
m.latestSample[l.Hash()] = ct
|
||||
m.latestSample[hash] = ct
|
||||
m.samples = append(m.samples, mockSample{l, ct, 0})
|
||||
return 0, nil
|
||||
return storage.SeriesRef(hash), nil
|
||||
}
|
||||
|
@ -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"
|
||||
)
|
||||
@ -387,12 +388,14 @@ func TestWriteStorageApplyConfig_PartialUpdate(t *testing.T) {
|
||||
|
||||
func TestOTLPWriteHandler(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",
|
||||
@ -401,24 +404,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",
|
||||
@ -428,26 +477,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",
|
||||
@ -456,24 +550,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",
|
||||
@ -482,24 +623,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",
|
||||
@ -529,6 +716,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",
|
||||
@ -537,24 +778,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",
|
||||
@ -564,40 +852,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 TestOTLPWriteHandler_StartTime(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()
|
||||
|
||||
@ -611,7 +1127,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)
|
||||
|
||||
@ -619,12 +1148,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)
|
||||
|
||||
@ -634,7 +1164,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
|
||||
@ -659,6 +1189,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")
|
||||
|
||||
@ -678,6 +1209,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")
|
||||
|
||||
@ -691,9 +1223,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")
|
||||
|
||||
@ -707,6 +1240,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)
|
||||
|
@ -342,6 +342,7 @@ func NewAPI(
|
||||
ConvertDelta: otlpDeltaToCumulative,
|
||||
NativeDelta: otlpNativeDeltaIngestion,
|
||||
LookbackDelta: lookbackDelta,
|
||||
IngestCTZeroSample: ctZeroIngestionEnabled,
|
||||
EnableTypeAndUnitLabels: enableTypeAndUnitLabels,
|
||||
})
|
||||
}
|
||||
|
Loading…
x
Reference in New Issue
Block a user