Do not use custom labels implementation
Signed-off-by: György Krajcsovits <gyorgy.krajcsovits@grafana.com>
This commit is contained in:
parent
eb3f40260b
commit
7268ac6d78
|
@ -23,10 +23,9 @@ import (
|
|||
|
||||
"github.com/prometheus/prometheus/model/exemplar"
|
||||
"github.com/prometheus/prometheus/model/histogram"
|
||||
modelLabels "github.com/prometheus/prometheus/model/labels"
|
||||
"github.com/prometheus/prometheus/model/labels"
|
||||
"github.com/prometheus/prometheus/model/metadata"
|
||||
"github.com/prometheus/prometheus/storage"
|
||||
"github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite/labels"
|
||||
)
|
||||
|
||||
// CombinedAppender is similar to storage.Appender, but combines updates to
|
||||
|
@ -81,7 +80,7 @@ func NewCombinedAppender(app storage.Appender, logger *slog.Logger, ingestCTZero
|
|||
type seriesRef struct {
|
||||
ref storage.SeriesRef
|
||||
ct int64
|
||||
ls modelLabels.Labels
|
||||
ls labels.Labels
|
||||
meta metadata.Metadata
|
||||
}
|
||||
|
||||
|
@ -97,26 +96,24 @@ type combinedAppender struct {
|
|||
refs map[uint64]seriesRef
|
||||
}
|
||||
|
||||
func (b *combinedAppender) AppendSample(_ string, rawls labels.Labels, meta metadata.Metadata, t, ct int64, v float64, es []exemplar.Exemplar) (err error) {
|
||||
return b.appendFloatOrHistogram(rawls, meta, t, ct, v, nil, es)
|
||||
func (b *combinedAppender) AppendSample(_ string, ls labels.Labels, meta metadata.Metadata, t, ct int64, v float64, es []exemplar.Exemplar) (err error) {
|
||||
return b.appendFloatOrHistogram(ls, meta, t, ct, v, nil, es)
|
||||
}
|
||||
|
||||
func (b *combinedAppender) AppendHistogram(_ string, rawls labels.Labels, meta metadata.Metadata, t, ct int64, h *histogram.Histogram, es []exemplar.Exemplar) (err error) {
|
||||
func (b *combinedAppender) AppendHistogram(_ string, ls labels.Labels, meta metadata.Metadata, t, ct int64, h *histogram.Histogram, es []exemplar.Exemplar) (err error) {
|
||||
if h == nil {
|
||||
// Sanity check, we should never get here with a nil histogram.
|
||||
ls := modelLabels.NewFromSorted(rawls)
|
||||
b.logger.Error("Received nil histogram in CombinedAppender.AppendHistogram", "series", ls.String())
|
||||
return nil
|
||||
}
|
||||
return b.appendFloatOrHistogram(rawls, meta, t, ct, 0, h, es)
|
||||
return b.appendFloatOrHistogram(ls, meta, t, ct, 0, h, es)
|
||||
}
|
||||
|
||||
func (b *combinedAppender) appendFloatOrHistogram(rawls labels.Labels, meta metadata.Metadata, t, ct int64, v float64, h *histogram.Histogram, es []exemplar.Exemplar) (err error) {
|
||||
ls := modelLabels.NewFromSorted(rawls)
|
||||
func (b *combinedAppender) appendFloatOrHistogram(ls labels.Labels, meta metadata.Metadata, t, ct int64, v float64, h *histogram.Histogram, es []exemplar.Exemplar) (err error) {
|
||||
hash := ls.Hash()
|
||||
series, exists := b.refs[hash]
|
||||
ref := series.ref
|
||||
if exists && !modelLabels.Equal(series.ls, ls) {
|
||||
if exists && !labels.Equal(series.ls, ls) {
|
||||
// Hash collision, this is a new series.
|
||||
exists = false
|
||||
ref = 0
|
||||
|
@ -200,7 +197,7 @@ func sampleType(h *histogram.Histogram) string {
|
|||
return "histogram"
|
||||
}
|
||||
|
||||
func (b *combinedAppender) appendExemplars(ref storage.SeriesRef, ls modelLabels.Labels, es []exemplar.Exemplar) storage.SeriesRef {
|
||||
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 {
|
||||
|
|
|
@ -28,10 +28,9 @@ import (
|
|||
|
||||
"github.com/prometheus/prometheus/model/exemplar"
|
||||
"github.com/prometheus/prometheus/model/histogram"
|
||||
modelLabels "github.com/prometheus/prometheus/model/labels"
|
||||
"github.com/prometheus/prometheus/model/labels"
|
||||
"github.com/prometheus/prometheus/model/metadata"
|
||||
"github.com/prometheus/prometheus/storage"
|
||||
"github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite/labels"
|
||||
"github.com/prometheus/prometheus/tsdb"
|
||||
"github.com/prometheus/prometheus/tsdb/chunkenc"
|
||||
"github.com/prometheus/prometheus/tsdb/tsdbutil"
|
||||
|
@ -119,17 +118,17 @@ func testCombinedAppenderOnTSDB(t *testing.T, ingestCTZeroSample bool) {
|
|||
|
||||
testExemplars := []exemplar.Exemplar{
|
||||
{
|
||||
Labels: modelLabels.FromStrings("tracid", "122"),
|
||||
Labels: labels.FromStrings("tracid", "122"),
|
||||
Value: 1337,
|
||||
},
|
||||
{
|
||||
Labels: modelLabels.FromStrings("tracid", "132"),
|
||||
Labels: labels.FromStrings("tracid", "132"),
|
||||
Value: 7777,
|
||||
},
|
||||
}
|
||||
expectedExemplars := []exemplar.QueryResult{
|
||||
{
|
||||
SeriesLabels: modelLabels.FromStrings(
|
||||
SeriesLabels: labels.FromStrings(
|
||||
model.MetricNameLabel, "test_bytes_total",
|
||||
"foo", "bar",
|
||||
),
|
||||
|
@ -460,7 +459,7 @@ func testCombinedAppenderOnTSDB(t *testing.T, ingestCTZeroSample bool) {
|
|||
ss := q.Select(ctx, false, &storage.SelectHints{
|
||||
Start: int64(math.MinInt64),
|
||||
End: int64(math.MaxInt64),
|
||||
}, modelLabels.MustNewMatcher(modelLabels.MatchEqual, model.MetricNameLabel, "test_bytes_total"))
|
||||
}, labels.MustNewMatcher(labels.MatchEqual, model.MetricNameLabel, "test_bytes_total"))
|
||||
|
||||
require.NoError(t, ss.Err())
|
||||
|
||||
|
@ -487,7 +486,7 @@ func testCombinedAppenderOnTSDB(t *testing.T, ingestCTZeroSample bool) {
|
|||
|
||||
eq, err := db.ExemplarQuerier(ctx)
|
||||
require.NoError(t, err)
|
||||
exResult, err := eq.Select(int64(math.MinInt64), int64(math.MaxInt64), []*modelLabels.Matcher{modelLabels.MustNewMatcher(modelLabels.MatchEqual, model.MetricNameLabel, "test_bytes_total")})
|
||||
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{}
|
||||
|
@ -530,7 +529,7 @@ func TestCombinedAppenderSeriesRefs(t *testing.T) {
|
|||
Help: "some help",
|
||||
}, 4, 3, 62.0, []exemplar.Exemplar{
|
||||
{
|
||||
Labels: modelLabels.FromStrings("tracid", "122"),
|
||||
Labels: labels.FromStrings("tracid", "122"),
|
||||
Value: 1337,
|
||||
},
|
||||
}))
|
||||
|
@ -715,7 +714,7 @@ func TestCombinedAppenderSeriesRefs(t *testing.T) {
|
|||
Help: "some help",
|
||||
}, 2, 0, 42.0, []exemplar.Exemplar{
|
||||
{
|
||||
Labels: modelLabels.FromStrings("tracid", "122"),
|
||||
Labels: labels.FromStrings("tracid", "122"),
|
||||
Value: 1337,
|
||||
},
|
||||
}))
|
||||
|
@ -723,7 +722,7 @@ func TestCombinedAppenderSeriesRefs(t *testing.T) {
|
|||
require.Len(t, app.records, 1)
|
||||
require.Equal(t, appenderRecord{
|
||||
op: "Append",
|
||||
ls: modelLabels.FromStrings(model.MetricNameLabel, "test_bytes_total", "foo", "bar"),
|
||||
ls: labels.FromStrings(model.MetricNameLabel, "test_bytes_total", "foo", "bar"),
|
||||
}, app.records[0])
|
||||
})
|
||||
|
||||
|
@ -748,7 +747,7 @@ func TestCombinedAppenderSeriesRefs(t *testing.T) {
|
|||
Help: "some other help",
|
||||
}, 4, 3, 62.0, []exemplar.Exemplar{
|
||||
{
|
||||
Labels: modelLabels.FromStrings("tracid", "122"),
|
||||
Labels: labels.FromStrings("tracid", "122"),
|
||||
Value: 1337,
|
||||
},
|
||||
}))
|
||||
|
@ -781,10 +780,10 @@ func TestCombinedAppenderSeriesRefs(t *testing.T) {
|
|||
Help: "some help",
|
||||
}, 2, 1, 42.0, nil))
|
||||
|
||||
hash := modelLabels.NewFromSorted(ls).Hash()
|
||||
hash := ls.Hash()
|
||||
cappImpl := capp.(*combinedAppender)
|
||||
series := cappImpl.refs[hash]
|
||||
series.ls = modelLabels.FromStrings(
|
||||
series.ls = labels.FromStrings(
|
||||
model.MetricNameLabel, "test_bytes_total",
|
||||
"foo", "club",
|
||||
)
|
||||
|
@ -798,7 +797,7 @@ func TestCombinedAppenderSeriesRefs(t *testing.T) {
|
|||
Help: "some help",
|
||||
}, 4, 3, 62.0, []exemplar.Exemplar{
|
||||
{
|
||||
Labels: modelLabels.FromStrings("tracid", "122"),
|
||||
Labels: labels.FromStrings("tracid", "122"),
|
||||
Value: 1337,
|
||||
},
|
||||
}))
|
||||
|
@ -828,7 +827,7 @@ type appenderRecord struct {
|
|||
op string
|
||||
ref storage.SeriesRef
|
||||
outRef storage.SeriesRef
|
||||
ls modelLabels.Labels
|
||||
ls labels.Labels
|
||||
}
|
||||
|
||||
type appenderRecorder struct {
|
||||
|
@ -857,7 +856,7 @@ func (a *appenderRecorder) newRef() storage.SeriesRef {
|
|||
return storage.SeriesRef(a.refcount)
|
||||
}
|
||||
|
||||
func (a *appenderRecorder) Append(ref storage.SeriesRef, ls modelLabels.Labels, _ int64, _ float64) (storage.SeriesRef, error) {
|
||||
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
|
||||
|
@ -869,7 +868,7 @@ func (a *appenderRecorder) Append(ref storage.SeriesRef, ls modelLabels.Labels,
|
|||
return ref, nil
|
||||
}
|
||||
|
||||
func (a *appenderRecorder) AppendCTZeroSample(ref storage.SeriesRef, ls modelLabels.Labels, _, _ int64) (storage.SeriesRef, error) {
|
||||
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
|
||||
|
@ -881,7 +880,7 @@ func (a *appenderRecorder) AppendCTZeroSample(ref storage.SeriesRef, ls modelLab
|
|||
return ref, nil
|
||||
}
|
||||
|
||||
func (a *appenderRecorder) AppendHistogram(ref storage.SeriesRef, ls modelLabels.Labels, _ int64, _ *histogram.Histogram, _ *histogram.FloatHistogram) (storage.SeriesRef, error) {
|
||||
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
|
||||
|
@ -893,7 +892,7 @@ func (a *appenderRecorder) AppendHistogram(ref storage.SeriesRef, ls modelLabels
|
|||
return ref, nil
|
||||
}
|
||||
|
||||
func (a *appenderRecorder) AppendHistogramCTZeroSample(ref storage.SeriesRef, ls modelLabels.Labels, _, _ int64, _ *histogram.Histogram, _ *histogram.FloatHistogram) (storage.SeriesRef, error) {
|
||||
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
|
||||
|
@ -905,7 +904,7 @@ func (a *appenderRecorder) AppendHistogramCTZeroSample(ref storage.SeriesRef, ls
|
|||
return ref, nil
|
||||
}
|
||||
|
||||
func (a *appenderRecorder) UpdateMetadata(ref storage.SeriesRef, ls modelLabels.Labels, _ metadata.Metadata) (storage.SeriesRef, error) {
|
||||
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
|
||||
|
@ -914,7 +913,7 @@ func (a *appenderRecorder) UpdateMetadata(ref storage.SeriesRef, ls modelLabels.
|
|||
return ref, nil
|
||||
}
|
||||
|
||||
func (a *appenderRecorder) AppendExemplar(ref storage.SeriesRef, ls modelLabels.Labels, _ exemplar.Exemplar) (storage.SeriesRef, error) {
|
||||
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
|
||||
|
|
|
@ -35,12 +35,11 @@ import (
|
|||
conventions "go.opentelemetry.io/collector/semconv/v1.6.1"
|
||||
|
||||
"github.com/prometheus/prometheus/model/exemplar"
|
||||
modelLabels "github.com/prometheus/prometheus/model/labels"
|
||||
"github.com/prometheus/prometheus/model/labels"
|
||||
"github.com/prometheus/prometheus/model/metadata"
|
||||
"github.com/prometheus/prometheus/model/timestamp"
|
||||
"github.com/prometheus/prometheus/model/value"
|
||||
"github.com/prometheus/prometheus/prompb"
|
||||
"github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite/labels"
|
||||
)
|
||||
|
||||
const (
|
||||
|
@ -58,7 +57,6 @@ 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
|
||||
)
|
||||
|
@ -101,7 +99,7 @@ func (c *PrometheusConverter) createAttributes(resource pcommon.Resource, attrib
|
|||
// of labels, and handle conflicts by appending values.
|
||||
c.builder.Reset(labels.EmptyLabels())
|
||||
var sortErr error
|
||||
sortedLabels.Range(func(l modelLabels.Label) {
|
||||
sortedLabels.Range(func(l labels.Label) {
|
||||
finalKey, err := labelNamer.Build(l.Name)
|
||||
if err != nil && sortErr == nil {
|
||||
sortErr = err
|
||||
|
@ -114,7 +112,7 @@ func (c *PrometheusConverter) createAttributes(resource pcommon.Resource, attrib
|
|||
}
|
||||
})
|
||||
if sortErr != nil {
|
||||
return nil, sortErr
|
||||
return labels.EmptyLabels(), sortErr
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -130,7 +128,7 @@ func (c *PrometheusConverter) createAttributes(resource pcommon.Resource, attrib
|
|||
|
||||
err := settings.PromoteResourceAttributes.addPromotedAttributes(c.builder, resourceAttrs, settings.AllowUTF8)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
return labels.EmptyLabels(), err
|
||||
}
|
||||
if promoteScope {
|
||||
// Scope Name, Version and Schema URL are added after attributes to ensure they are not overwritten by attributes.
|
||||
|
@ -148,7 +146,7 @@ func (c *PrometheusConverter) createAttributes(resource pcommon.Resource, attrib
|
|||
return true
|
||||
})
|
||||
if scopeErr != nil {
|
||||
return nil, scopeErr
|
||||
return labels.EmptyLabels(), scopeErr
|
||||
}
|
||||
}
|
||||
// Map service.name + service.namespace to job.
|
||||
|
@ -186,7 +184,7 @@ func (c *PrometheusConverter) createAttributes(resource pcommon.Resource, attrib
|
|||
var err error
|
||||
name, err = labelNamer.Build(name)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
return labels.EmptyLabels(), err
|
||||
}
|
||||
}
|
||||
c.builder.Set(name, extras[i+1])
|
||||
|
@ -365,7 +363,7 @@ func (c *PrometheusConverter) getPromExemplars(ctx context.Context, exemplars pm
|
|||
})
|
||||
}
|
||||
c.scratchBuilder.Sort()
|
||||
newExemplar.Labels = modelLabels.NewFromSorted(c.scratchBuilder.Labels())
|
||||
newExemplar.Labels = c.scratchBuilder.Labels()
|
||||
outputExemplars = append(outputExemplars, newExemplar)
|
||||
}
|
||||
|
||||
|
@ -545,7 +543,7 @@ func (c *PrometheusConverter) addResourceTargetInfo(resource pcommon.Resource, s
|
|||
return err
|
||||
}
|
||||
haveIdentifier := false
|
||||
lbls.Range(func(l modelLabels.Label) {
|
||||
lbls.Range(func(l labels.Label) {
|
||||
if l.Name == model.JobLabel || l.Name == model.InstanceLabel {
|
||||
haveIdentifier = true
|
||||
}
|
||||
|
|
|
@ -27,9 +27,9 @@ import (
|
|||
"go.opentelemetry.io/collector/pdata/pmetric"
|
||||
|
||||
"github.com/prometheus/prometheus/config"
|
||||
"github.com/prometheus/prometheus/model/labels"
|
||||
"github.com/prometheus/prometheus/model/metadata"
|
||||
"github.com/prometheus/prometheus/prompb"
|
||||
"github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite/labels"
|
||||
"github.com/prometheus/prometheus/util/testutil"
|
||||
)
|
||||
|
||||
|
|
|
@ -30,8 +30,8 @@ import (
|
|||
|
||||
"github.com/prometheus/prometheus/model/exemplar"
|
||||
"github.com/prometheus/prometheus/model/histogram"
|
||||
"github.com/prometheus/prometheus/model/labels"
|
||||
"github.com/prometheus/prometheus/model/metadata"
|
||||
"github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite/labels"
|
||||
)
|
||||
|
||||
type expectedBucketLayout struct {
|
||||
|
|
|
@ -1,336 +0,0 @@
|
|||
// Copyright 2025 The Prometheus Authors
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
// This file is copied from model/labels/labels_slicelabels.go and
|
||||
// labels_common.go. Unused functions are removed.
|
||||
|
||||
package labels
|
||||
|
||||
import (
|
||||
"slices"
|
||||
"strings"
|
||||
"unsafe"
|
||||
|
||||
"github.com/cespare/xxhash/v2"
|
||||
|
||||
common "github.com/prometheus/prometheus/model/labels"
|
||||
)
|
||||
|
||||
const sep = '\xff' // Used between labels in `Bytes` and `Hash`.
|
||||
var seps = []byte{sep} // Used with Hash, which has no WriteByte method.
|
||||
|
||||
// Labels is a sorted set of labels. Order has to be guaranteed upon
|
||||
// instantiation.
|
||||
type Labels []common.Label
|
||||
|
||||
func (ls Labels) Len() int { return len(ls) }
|
||||
func (ls Labels) Swap(i, j int) { ls[i], ls[j] = ls[j], ls[i] }
|
||||
func (ls Labels) Less(i, j int) bool { return ls[i].Name < ls[j].Name }
|
||||
|
||||
// Hash returns a hash value for the label set.
|
||||
// Note: the result is not guaranteed to be consistent across different runs of Prometheus.
|
||||
func (ls Labels) Hash() uint64 {
|
||||
// Use xxhash.Sum64(b) for fast path as it's faster.
|
||||
b := make([]byte, 0, 1024)
|
||||
for i, v := range ls {
|
||||
if len(b)+len(v.Name)+len(v.Value)+2 >= cap(b) {
|
||||
// If labels entry is 1KB+ do not allocate whole entry.
|
||||
h := xxhash.New()
|
||||
_, _ = h.Write(b)
|
||||
for _, v := range ls[i:] {
|
||||
_, _ = h.WriteString(v.Name)
|
||||
_, _ = h.Write(seps)
|
||||
_, _ = h.WriteString(v.Value)
|
||||
_, _ = h.Write(seps)
|
||||
}
|
||||
return h.Sum64()
|
||||
}
|
||||
|
||||
b = append(b, v.Name...)
|
||||
b = append(b, sep)
|
||||
b = append(b, v.Value...)
|
||||
b = append(b, sep)
|
||||
}
|
||||
return xxhash.Sum64(b)
|
||||
}
|
||||
|
||||
// Get returns the value for the label with the given name.
|
||||
// Returns an empty string if the label doesn't exist.
|
||||
func (ls Labels) Get(name string) string {
|
||||
for _, l := range ls {
|
||||
if l.Name == name {
|
||||
return l.Value
|
||||
}
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
// Has returns true if the label with the given name is present.
|
||||
func (ls Labels) Has(name string) bool {
|
||||
for _, l := range ls {
|
||||
if l.Name == name {
|
||||
return true
|
||||
}
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
func (ls Labels) IsEmpty() bool {
|
||||
return len(ls) == 0
|
||||
}
|
||||
|
||||
// HasDuplicateLabelNames returns whether ls has duplicate label names.
|
||||
// It assumes that the labelset is sorted.
|
||||
func (ls Labels) HasDuplicateLabelNames() (string, bool) {
|
||||
for i, l := range ls {
|
||||
if i == 0 {
|
||||
continue
|
||||
}
|
||||
if l.Name == ls[i-1].Name {
|
||||
return l.Name, true
|
||||
}
|
||||
}
|
||||
return "", false
|
||||
}
|
||||
|
||||
// Equal returns whether the two label sets are equal.
|
||||
func Equal(ls, o Labels) bool {
|
||||
return slices.Equal(ls, o)
|
||||
}
|
||||
|
||||
// EmptyLabels returns n empty Labels value, for convenience.
|
||||
func EmptyLabels() Labels {
|
||||
return Labels{}
|
||||
}
|
||||
|
||||
// FromStrings creates new labels from pairs of strings.
|
||||
func FromStrings(ss ...string) Labels {
|
||||
if len(ss)%2 != 0 {
|
||||
panic("invalid number of strings")
|
||||
}
|
||||
res := make(Labels, 0, len(ss)/2)
|
||||
for i := 0; i < len(ss); i += 2 {
|
||||
res = append(res, common.Label{Name: ss[i], Value: ss[i+1]})
|
||||
}
|
||||
|
||||
slices.SortFunc(res, func(a, b common.Label) int { return strings.Compare(a.Name, b.Name) })
|
||||
return res
|
||||
}
|
||||
|
||||
// Range calls f on each label.
|
||||
func (ls Labels) Range(f func(l common.Label)) {
|
||||
for _, l := range ls {
|
||||
f(l)
|
||||
}
|
||||
}
|
||||
|
||||
// Builder allows modifying Labels.
|
||||
type Builder struct {
|
||||
base Labels
|
||||
del []string
|
||||
add []common.Label
|
||||
}
|
||||
|
||||
// NewBuilder returns a new LabelsBuilder.
|
||||
func NewBuilder(base Labels) *Builder {
|
||||
b := &Builder{
|
||||
del: make([]string, 0, 5),
|
||||
add: make([]common.Label, 0, 5),
|
||||
}
|
||||
b.Reset(base)
|
||||
return b
|
||||
}
|
||||
|
||||
// Reset clears all current state for the builder.
|
||||
func (b *Builder) Reset(base Labels) {
|
||||
b.base = base
|
||||
b.del = b.del[:0]
|
||||
b.add = b.add[:0]
|
||||
b.base.Range(func(l common.Label) {
|
||||
if l.Value == "" {
|
||||
b.del = append(b.del, l.Name)
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
// Del deletes the label of the given name.
|
||||
func (b *Builder) Del(ns ...string) *Builder {
|
||||
for _, n := range ns {
|
||||
for i, a := range b.add {
|
||||
if a.Name == n {
|
||||
b.add = append(b.add[:i], b.add[i+1:]...)
|
||||
}
|
||||
}
|
||||
b.del = append(b.del, n)
|
||||
}
|
||||
return b
|
||||
}
|
||||
|
||||
// Keep removes all labels from the base except those with the given names.
|
||||
func (b *Builder) Keep(ns ...string) *Builder {
|
||||
b.base.Range(func(l common.Label) {
|
||||
if slices.Contains(ns, l.Name) {
|
||||
return
|
||||
}
|
||||
b.del = append(b.del, l.Name)
|
||||
})
|
||||
return b
|
||||
}
|
||||
|
||||
// Set the name/value pair as a label. A value of "" means delete that label.
|
||||
func (b *Builder) Set(n, v string) *Builder {
|
||||
if v == "" {
|
||||
// Empty labels are the same as missing labels.
|
||||
return b.Del(n)
|
||||
}
|
||||
for i, a := range b.add {
|
||||
if a.Name == n {
|
||||
b.add[i].Value = v
|
||||
return b
|
||||
}
|
||||
}
|
||||
b.add = append(b.add, common.Label{Name: n, Value: v})
|
||||
|
||||
return b
|
||||
}
|
||||
|
||||
func (b *Builder) Get(n string) string {
|
||||
// Del() removes entries from .add but Set() does not remove from .del, so check .add first.
|
||||
for _, a := range b.add {
|
||||
if a.Name == n {
|
||||
return a.Value
|
||||
}
|
||||
}
|
||||
if slices.Contains(b.del, n) {
|
||||
return ""
|
||||
}
|
||||
return b.base.Get(n)
|
||||
}
|
||||
|
||||
// Range calls f on each label in the Builder.
|
||||
func (b *Builder) Range(f func(l common.Label)) {
|
||||
// Stack-based arrays to avoid heap allocation in most cases.
|
||||
var addStack [128]common.Label
|
||||
var delStack [128]string
|
||||
// Take a copy of add and del, so they are unaffected by calls to Set() or Del().
|
||||
origAdd, origDel := append(addStack[:0], b.add...), append(delStack[:0], b.del...)
|
||||
b.base.Range(func(l common.Label) {
|
||||
if !slices.Contains(origDel, l.Name) && !common.Contains(origAdd, l.Name) {
|
||||
f(l)
|
||||
}
|
||||
})
|
||||
for _, a := range origAdd {
|
||||
f(a)
|
||||
}
|
||||
}
|
||||
|
||||
// Labels returns the labels from the builder.
|
||||
// If no modifications were made, the original labels are returned.
|
||||
func (b *Builder) Labels() Labels {
|
||||
if len(b.del) == 0 && len(b.add) == 0 {
|
||||
return b.base
|
||||
}
|
||||
|
||||
expectedSize := len(b.base) + len(b.add) - len(b.del)
|
||||
if expectedSize < 1 {
|
||||
expectedSize = 1
|
||||
}
|
||||
res := make(Labels, 0, expectedSize)
|
||||
for _, l := range b.base {
|
||||
if slices.Contains(b.del, l.Name) || common.Contains(b.add, l.Name) {
|
||||
continue
|
||||
}
|
||||
res = append(res, l)
|
||||
}
|
||||
if len(b.add) > 0 { // Base is already in order, so we only need to sort if we add to it.
|
||||
res = append(res, b.add...)
|
||||
slices.SortFunc(res, func(a, b common.Label) int { return strings.Compare(a.Name, b.Name) })
|
||||
}
|
||||
return res
|
||||
}
|
||||
|
||||
// ScratchBuilder allows efficient construction of a Labels from scratch.
|
||||
type ScratchBuilder struct {
|
||||
add Labels
|
||||
}
|
||||
|
||||
// SymbolTable is no-op, just for api parity with dedupelabels.
|
||||
type SymbolTable struct{}
|
||||
|
||||
func NewSymbolTable() *SymbolTable { return nil }
|
||||
|
||||
func (*SymbolTable) Len() int { return 0 }
|
||||
|
||||
// NewScratchBuilder creates a ScratchBuilder initialized for Labels with n entries.
|
||||
func NewScratchBuilder(n int) ScratchBuilder {
|
||||
return ScratchBuilder{add: make([]common.Label, 0, n)}
|
||||
}
|
||||
|
||||
// NewBuilderWithSymbolTable creates a Builder, for api parity with dedupelabels.
|
||||
func NewBuilderWithSymbolTable(_ *SymbolTable) *Builder {
|
||||
return NewBuilder(EmptyLabels())
|
||||
}
|
||||
|
||||
// NewScratchBuilderWithSymbolTable creates a ScratchBuilder, for api parity with dedupelabels.
|
||||
func NewScratchBuilderWithSymbolTable(_ *SymbolTable, n int) ScratchBuilder {
|
||||
return NewScratchBuilder(n)
|
||||
}
|
||||
|
||||
func (*ScratchBuilder) SetSymbolTable(_ *SymbolTable) {
|
||||
// no-op
|
||||
}
|
||||
|
||||
func (b *ScratchBuilder) Reset() {
|
||||
b.add = b.add[:0]
|
||||
}
|
||||
|
||||
// Add a name/value pair.
|
||||
// Note if you Add the same name twice you will get a duplicate label, which is invalid.
|
||||
func (b *ScratchBuilder) Add(name, value string) {
|
||||
b.add = append(b.add, common.Label{Name: name, Value: value})
|
||||
}
|
||||
|
||||
// UnsafeAddBytes adds a name/value pair, using []byte instead of string.
|
||||
// The '-tags stringlabels' version of this function is unsafe, hence the name.
|
||||
// This version is safe - it copies the strings immediately - but we keep the same name so everything compiles.
|
||||
func (b *ScratchBuilder) UnsafeAddBytes(name, value []byte) {
|
||||
b.add = append(b.add, common.Label{Name: string(name), Value: string(value)})
|
||||
}
|
||||
|
||||
// Sort the labels added so far by name.
|
||||
func (b *ScratchBuilder) Sort() {
|
||||
slices.SortFunc(b.add, func(a, b common.Label) int { return strings.Compare(a.Name, b.Name) })
|
||||
}
|
||||
|
||||
// Assign is for when you already have a Labels which you want this ScratchBuilder to return.
|
||||
func (b *ScratchBuilder) Assign(ls Labels) {
|
||||
b.add = append(b.add[:0], ls...) // Copy on top of our slice, so we don't retain the input slice.
|
||||
}
|
||||
|
||||
// Labels returns the name/value pairs added so far as a Labels object.
|
||||
// Note: if you want them sorted, call Sort() first.
|
||||
func (b *ScratchBuilder) Labels() Labels {
|
||||
// Copy the slice, so the next use of ScratchBuilder doesn't overwrite.
|
||||
return append([]common.Label{}, b.add...)
|
||||
}
|
||||
|
||||
// Overwrite the newly-built Labels out to ls.
|
||||
// Callers must ensure that there are no other references to ls, or any strings fetched from it.
|
||||
func (b *ScratchBuilder) Overwrite(ls *Labels) {
|
||||
*ls = append((*ls)[:0], b.add...)
|
||||
}
|
||||
|
||||
// SizeOfLabels returns the approximate space required for n copies of a label.
|
||||
func SizeOfLabels(name, value string, n uint64) uint64 {
|
||||
return (uint64(len(name)) + uint64(unsafe.Sizeof(name)) + uint64(len(value)) + uint64(unsafe.Sizeof(value))) * n
|
||||
}
|
|
@ -1,424 +0,0 @@
|
|||
// Copyright 2025 The Prometheus Authors
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package labels
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"net/http"
|
||||
"strconv"
|
||||
"strings"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
common "github.com/prometheus/prometheus/model/labels"
|
||||
)
|
||||
|
||||
func TestLabels_Equal(t *testing.T) {
|
||||
labels := FromStrings(
|
||||
"aaa", "111",
|
||||
"bbb", "222")
|
||||
|
||||
tests := []struct {
|
||||
compared Labels
|
||||
expected bool
|
||||
}{
|
||||
{
|
||||
compared: FromStrings(
|
||||
"aaa", "111",
|
||||
"bbb", "222",
|
||||
"ccc", "333"),
|
||||
expected: false,
|
||||
},
|
||||
{
|
||||
compared: FromStrings(
|
||||
"aaa", "111",
|
||||
"bar", "222"),
|
||||
expected: false,
|
||||
},
|
||||
{
|
||||
compared: FromStrings(
|
||||
"aaa", "111",
|
||||
"bbb", "233"),
|
||||
expected: false,
|
||||
},
|
||||
{
|
||||
compared: FromStrings(
|
||||
"aaa", "111",
|
||||
"bbb", "222"),
|
||||
expected: true,
|
||||
},
|
||||
}
|
||||
|
||||
for i, test := range tests {
|
||||
got := Equal(labels, test.compared)
|
||||
require.Equal(t, test.expected, got, "unexpected comparison result for test case %d", i)
|
||||
}
|
||||
}
|
||||
|
||||
func TestLabels_FromStrings(t *testing.T) {
|
||||
labels := FromStrings("aaa", "111", "bbb", "222")
|
||||
x := 0
|
||||
labels.Range(func(l common.Label) {
|
||||
switch x {
|
||||
case 0:
|
||||
require.Equal(t, common.Label{Name: "aaa", Value: "111"}, l, "unexpected value")
|
||||
case 1:
|
||||
require.Equal(t, common.Label{Name: "bbb", Value: "222"}, l, "unexpected value")
|
||||
default:
|
||||
t.Fatalf("unexpected labelset value %d: %v", x, l)
|
||||
}
|
||||
x++
|
||||
})
|
||||
|
||||
require.Panics(t, func() { FromStrings("aaa", "111", "bbb") }) //nolint:staticcheck // Ignore SA5012, error is intentional test.
|
||||
}
|
||||
|
||||
func TestLabels_Has(t *testing.T) {
|
||||
tests := []struct {
|
||||
input string
|
||||
expected bool
|
||||
}{
|
||||
{
|
||||
input: "foo",
|
||||
expected: false,
|
||||
},
|
||||
{
|
||||
input: "aaa",
|
||||
expected: true,
|
||||
},
|
||||
}
|
||||
|
||||
labelsSet := FromStrings(
|
||||
"aaa", "111",
|
||||
"bbb", "222")
|
||||
|
||||
for i, test := range tests {
|
||||
got := labelsSet.Has(test.input)
|
||||
require.Equal(t, test.expected, got, "unexpected comparison result for test case %d", i)
|
||||
}
|
||||
}
|
||||
|
||||
func TestLabels_Get(t *testing.T) {
|
||||
require.Empty(t, FromStrings("aaa", "111", "bbb", "222").Get("foo"))
|
||||
require.Equal(t, "111", FromStrings("aaaa", "111", "bbb", "222").Get("aaaa"))
|
||||
require.Equal(t, "222", FromStrings("aaaa", "111", "bbb", "222").Get("bbb"))
|
||||
}
|
||||
|
||||
func ScratchBuilderForBenchmark() ScratchBuilder {
|
||||
// (Only relevant to -tags dedupelabels: stuff the symbol table before adding the real labels, to avoid having everything fitting into 1 byte.)
|
||||
b := NewScratchBuilder(256)
|
||||
for i := 0; i < 256; i++ {
|
||||
b.Add(fmt.Sprintf("name%d", i), fmt.Sprintf("value%d", i))
|
||||
}
|
||||
b.Labels()
|
||||
b.Reset()
|
||||
return b
|
||||
}
|
||||
|
||||
func NewForBenchmark(ls ...common.Label) Labels {
|
||||
b := ScratchBuilderForBenchmark()
|
||||
for _, l := range ls {
|
||||
b.Add(l.Name, l.Value)
|
||||
}
|
||||
b.Sort()
|
||||
return b.Labels()
|
||||
}
|
||||
|
||||
func FromStringsForBenchmark(ss ...string) Labels {
|
||||
if len(ss)%2 != 0 {
|
||||
panic("invalid number of strings")
|
||||
}
|
||||
b := ScratchBuilderForBenchmark()
|
||||
for i := 0; i < len(ss); i += 2 {
|
||||
b.Add(ss[i], ss[i+1])
|
||||
}
|
||||
b.Sort()
|
||||
return b.Labels()
|
||||
}
|
||||
|
||||
// BenchmarkLabels_Get was written to check whether a binary search can improve the performance vs the linear search implementation
|
||||
// The results have shown that binary search would only be better when searching last labels in scenarios with more than 10 labels.
|
||||
// In the following list, `old` is the linear search while `new` is the binary search implementation (without calling sort.Search, which performs even worse here)
|
||||
//
|
||||
// name old time/op new time/op delta
|
||||
// Labels_Get/with_5_labels/get_first_label 5.12ns ± 0% 14.24ns ± 0% ~ (p=1.000 n=1+1)
|
||||
// Labels_Get/with_5_labels/get_middle_label 13.5ns ± 0% 18.5ns ± 0% ~ (p=1.000 n=1+1)
|
||||
// Labels_Get/with_5_labels/get_last_label 21.9ns ± 0% 18.9ns ± 0% ~ (p=1.000 n=1+1)
|
||||
// Labels_Get/with_10_labels/get_first_label 5.11ns ± 0% 19.47ns ± 0% ~ (p=1.000 n=1+1)
|
||||
// Labels_Get/with_10_labels/get_middle_label 26.2ns ± 0% 19.3ns ± 0% ~ (p=1.000 n=1+1)
|
||||
// Labels_Get/with_10_labels/get_last_label 42.8ns ± 0% 23.4ns ± 0% ~ (p=1.000 n=1+1)
|
||||
// Labels_Get/with_30_labels/get_first_label 5.10ns ± 0% 24.63ns ± 0% ~ (p=1.000 n=1+1)
|
||||
// Labels_Get/with_30_labels/get_middle_label 75.8ns ± 0% 29.7ns ± 0% ~ (p=1.000 n=1+1)
|
||||
// Labels_Get/with_30_labels/get_last_label 169ns ± 0% 29ns ± 0% ~ (p=1.000 n=1+1)
|
||||
func BenchmarkLabels_Get(b *testing.B) {
|
||||
maxLabels := 30
|
||||
allLabels := make([]common.Label, maxLabels)
|
||||
for i := 0; i < maxLabels; i++ {
|
||||
allLabels[i] = common.Label{Name: strings.Repeat(string('a'+byte(i)), 5+(i%5))}
|
||||
}
|
||||
for _, size := range []int{5, 10, maxLabels} {
|
||||
b.Run(fmt.Sprintf("with %d labels", size), func(b *testing.B) {
|
||||
labels := NewForBenchmark(allLabels[:size]...)
|
||||
for _, scenario := range []struct {
|
||||
desc, label string
|
||||
}{
|
||||
{"first label", allLabels[0].Name},
|
||||
{"middle label", allLabels[size/2].Name},
|
||||
{"last label", allLabels[size-1].Name},
|
||||
{"not-found label", "benchmark"},
|
||||
} {
|
||||
b.Run(scenario.desc, func(b *testing.B) {
|
||||
b.Run("get", func(b *testing.B) {
|
||||
for i := 0; i < b.N; i++ {
|
||||
_ = labels.Get(scenario.label)
|
||||
}
|
||||
})
|
||||
b.Run("has", func(b *testing.B) {
|
||||
for i := 0; i < b.N; i++ {
|
||||
_ = labels.Has(scenario.label)
|
||||
}
|
||||
})
|
||||
})
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
var comparisonBenchmarkScenarios = []struct {
|
||||
desc string
|
||||
base, other Labels
|
||||
}{
|
||||
{
|
||||
"equal",
|
||||
FromStringsForBenchmark("a_label_name", "a_label_value", "another_label_name", "another_label_value"),
|
||||
FromStringsForBenchmark("a_label_name", "a_label_value", "another_label_name", "another_label_value"),
|
||||
},
|
||||
{
|
||||
"not equal",
|
||||
FromStringsForBenchmark("a_label_name", "a_label_value", "another_label_name", "another_label_value"),
|
||||
FromStringsForBenchmark("a_label_name", "a_label_value", "another_label_name", "a_different_label_value"),
|
||||
},
|
||||
{
|
||||
"different sizes",
|
||||
FromStringsForBenchmark("a_label_name", "a_label_value", "another_label_name", "another_label_value"),
|
||||
FromStringsForBenchmark("a_label_name", "a_label_value"),
|
||||
},
|
||||
{
|
||||
"lots",
|
||||
FromStringsForBenchmark("aaa", "bbb", "ccc", "ddd", "eee", "fff", "ggg", "hhh", "iii", "jjj", "kkk", "lll", "mmm", "nnn", "ooo", "ppp", "qqq", "rrz"),
|
||||
FromStringsForBenchmark("aaa", "bbb", "ccc", "ddd", "eee", "fff", "ggg", "hhh", "iii", "jjj", "kkk", "lll", "mmm", "nnn", "ooo", "ppp", "qqq", "rrr"),
|
||||
},
|
||||
{
|
||||
"real long equal",
|
||||
FromStringsForBenchmark("__name__", "kube_pod_container_status_last_terminated_exitcode", "cluster", "prod-af-north-0", " container", "prometheus", "instance", "kube-state-metrics-0:kube-state-metrics:ksm", "job", "kube-state-metrics/kube-state-metrics", " namespace", "observability-prometheus", "pod", "observability-prometheus-0", "uid", "d3ec90b2-4975-4607-b45d-b9ad64bb417e"),
|
||||
FromStringsForBenchmark("__name__", "kube_pod_container_status_last_terminated_exitcode", "cluster", "prod-af-north-0", " container", "prometheus", "instance", "kube-state-metrics-0:kube-state-metrics:ksm", "job", "kube-state-metrics/kube-state-metrics", " namespace", "observability-prometheus", "pod", "observability-prometheus-0", "uid", "d3ec90b2-4975-4607-b45d-b9ad64bb417e"),
|
||||
},
|
||||
{
|
||||
"real long different end",
|
||||
FromStringsForBenchmark("__name__", "kube_pod_container_status_last_terminated_exitcode", "cluster", "prod-af-north-0", " container", "prometheus", "instance", "kube-state-metrics-0:kube-state-metrics:ksm", "job", "kube-state-metrics/kube-state-metrics", " namespace", "observability-prometheus", "pod", "observability-prometheus-0", "uid", "d3ec90b2-4975-4607-b45d-b9ad64bb417e"),
|
||||
FromStringsForBenchmark("__name__", "kube_pod_container_status_last_terminated_exitcode", "cluster", "prod-af-north-0", " container", "prometheus", "instance", "kube-state-metrics-0:kube-state-metrics:ksm", "job", "kube-state-metrics/kube-state-metrics", " namespace", "observability-prometheus", "pod", "observability-prometheus-0", "uid", "deadbeef-0000-1111-2222-b9ad64bb417e"),
|
||||
},
|
||||
}
|
||||
|
||||
func BenchmarkLabels_Equals(b *testing.B) {
|
||||
for _, scenario := range comparisonBenchmarkScenarios {
|
||||
b.Run(scenario.desc, func(b *testing.B) {
|
||||
b.ResetTimer()
|
||||
for i := 0; i < b.N; i++ {
|
||||
_ = Equal(scenario.base, scenario.other)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestBuilder(t *testing.T) {
|
||||
reuseBuilder := NewBuilderWithSymbolTable(NewSymbolTable())
|
||||
for i, tcase := range []struct {
|
||||
base Labels
|
||||
del []string
|
||||
keep []string
|
||||
set []common.Label
|
||||
want Labels
|
||||
}{
|
||||
{
|
||||
base: FromStrings("aaa", "111"),
|
||||
want: FromStrings("aaa", "111"),
|
||||
},
|
||||
{
|
||||
base: EmptyLabels(),
|
||||
set: []common.Label{{Name: "aaa", Value: "444"}, {Name: "bbb", Value: "555"}, {Name: "ccc", Value: "666"}},
|
||||
want: FromStrings("aaa", "444", "bbb", "555", "ccc", "666"),
|
||||
},
|
||||
{
|
||||
base: FromStrings("aaa", "111", "bbb", "222", "ccc", "333"),
|
||||
set: []common.Label{{Name: "aaa", Value: "444"}, {Name: "bbb", Value: "555"}, {Name: "ccc", Value: "666"}},
|
||||
want: FromStrings("aaa", "444", "bbb", "555", "ccc", "666"),
|
||||
},
|
||||
{
|
||||
base: FromStrings("aaa", "111", "bbb", "222", "ccc", "333"),
|
||||
del: []string{"bbb"},
|
||||
want: FromStrings("aaa", "111", "ccc", "333"),
|
||||
},
|
||||
{
|
||||
set: []common.Label{{Name: "aaa", Value: "111"}, {Name: "bbb", Value: "222"}, {Name: "ccc", Value: "333"}},
|
||||
del: []string{"bbb"},
|
||||
want: FromStrings("aaa", "111", "ccc", "333"),
|
||||
},
|
||||
{
|
||||
base: FromStrings("aaa", "111"),
|
||||
set: []common.Label{{Name: "bbb", Value: "222"}},
|
||||
want: FromStrings("aaa", "111", "bbb", "222"),
|
||||
},
|
||||
{
|
||||
base: FromStrings("aaa", "111"),
|
||||
set: []common.Label{{Name: "bbb", Value: "222"}, {Name: "bbb", Value: "333"}},
|
||||
want: FromStrings("aaa", "111", "bbb", "333"),
|
||||
},
|
||||
{
|
||||
base: FromStrings("aaa", "111", "bbb", "222", "ccc", "333"),
|
||||
del: []string{"bbb"},
|
||||
set: []common.Label{{Name: "ddd", Value: "444"}},
|
||||
want: FromStrings("aaa", "111", "ccc", "333", "ddd", "444"),
|
||||
},
|
||||
{ // Blank value is interpreted as delete.
|
||||
base: FromStrings("aaa", "111", "bbb", "", "ccc", "333"),
|
||||
want: FromStrings("aaa", "111", "ccc", "333"),
|
||||
},
|
||||
{
|
||||
base: FromStrings("aaa", "111", "bbb", "222", "ccc", "333"),
|
||||
set: []common.Label{{Name: "bbb", Value: ""}},
|
||||
want: FromStrings("aaa", "111", "ccc", "333"),
|
||||
},
|
||||
{
|
||||
base: FromStrings("aaa", "111", "bbb", "222", "ccc", "333"),
|
||||
keep: []string{"bbb"},
|
||||
want: FromStrings("bbb", "222"),
|
||||
},
|
||||
{
|
||||
base: FromStrings("aaa", "111", "bbb", "222", "ccc", "333"),
|
||||
keep: []string{"aaa", "ccc"},
|
||||
want: FromStrings("aaa", "111", "ccc", "333"),
|
||||
},
|
||||
{
|
||||
base: FromStrings("aaa", "111", "bbb", "222", "ccc", "333"),
|
||||
del: []string{"bbb"},
|
||||
set: []common.Label{{Name: "ddd", Value: "444"}},
|
||||
keep: []string{"aaa", "ddd"},
|
||||
want: FromStrings("aaa", "111", "ddd", "444"),
|
||||
},
|
||||
} {
|
||||
test := func(t *testing.T, b *Builder) {
|
||||
for _, lbl := range tcase.set {
|
||||
b.Set(lbl.Name, lbl.Value)
|
||||
}
|
||||
if len(tcase.keep) > 0 {
|
||||
b.Keep(tcase.keep...)
|
||||
}
|
||||
b.Del(tcase.del...)
|
||||
require.True(t, Equal(tcase.want, b.Labels()))
|
||||
|
||||
// Check what happens when we call Range and mutate the builder.
|
||||
b.Range(func(l common.Label) {
|
||||
if l.Name == "aaa" || l.Name == "bbb" {
|
||||
b.Del(l.Name)
|
||||
}
|
||||
})
|
||||
// require.Equal(t, tcase.want.BytesWithoutLabels(nil, "aaa", "bbb"), b.Labels().Bytes(nil))
|
||||
}
|
||||
t.Run(fmt.Sprintf("NewBuilder %d", i), func(t *testing.T) {
|
||||
test(t, NewBuilder(tcase.base))
|
||||
})
|
||||
t.Run(fmt.Sprintf("NewSymbolTable %d", i), func(t *testing.T) {
|
||||
b := NewBuilderWithSymbolTable(NewSymbolTable())
|
||||
b.Reset(tcase.base)
|
||||
test(t, b)
|
||||
})
|
||||
t.Run(fmt.Sprintf("reuseBuilder %d", i), func(t *testing.T) {
|
||||
reuseBuilder.Reset(tcase.base)
|
||||
test(t, reuseBuilder)
|
||||
})
|
||||
}
|
||||
t.Run("set_after_del", func(t *testing.T) {
|
||||
b := NewBuilder(FromStrings("aaa", "111"))
|
||||
b.Del("bbb")
|
||||
b.Set("bbb", "222")
|
||||
require.Equal(t, FromStrings("aaa", "111", "bbb", "222"), b.Labels())
|
||||
require.Equal(t, "222", b.Get("bbb"))
|
||||
})
|
||||
}
|
||||
|
||||
func TestScratchBuilder(t *testing.T) {
|
||||
for i, tcase := range []struct {
|
||||
add []common.Label
|
||||
want Labels
|
||||
}{
|
||||
{
|
||||
add: []common.Label{},
|
||||
want: EmptyLabels(),
|
||||
},
|
||||
{
|
||||
add: []common.Label{{Name: "aaa", Value: "111"}},
|
||||
want: FromStrings("aaa", "111"),
|
||||
},
|
||||
{
|
||||
add: []common.Label{{Name: "aaa", Value: "111"}, {Name: "bbb", Value: "222"}, {Name: "ccc", Value: "333"}},
|
||||
want: FromStrings("aaa", "111", "bbb", "222", "ccc", "333"),
|
||||
},
|
||||
{
|
||||
add: []common.Label{{Name: "bbb", Value: "222"}, {Name: "aaa", Value: "111"}, {Name: "ccc", Value: "333"}},
|
||||
want: FromStrings("aaa", "111", "bbb", "222", "ccc", "333"),
|
||||
},
|
||||
{
|
||||
add: []common.Label{{Name: "ddd", Value: "444"}},
|
||||
want: FromStrings("ddd", "444"),
|
||||
},
|
||||
} {
|
||||
t.Run(strconv.Itoa(i), func(t *testing.T) {
|
||||
b := NewScratchBuilder(len(tcase.add))
|
||||
for _, lbl := range tcase.add {
|
||||
b.Add(lbl.Name, lbl.Value)
|
||||
}
|
||||
b.Sort()
|
||||
require.True(t, Equal(tcase.want, b.Labels()))
|
||||
b.Assign(tcase.want)
|
||||
require.True(t, Equal(tcase.want, b.Labels()))
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
var benchmarkLabels = []common.Label{
|
||||
{Name: "job", Value: "node"},
|
||||
{Name: "instance", Value: "123.123.1.211:9090"},
|
||||
{Name: "path", Value: "/api/v1/namespaces/<namespace>/deployments/<name>"},
|
||||
{Name: "method", Value: http.MethodGet},
|
||||
{Name: "namespace", Value: "system"},
|
||||
{Name: "status", Value: "500"},
|
||||
{Name: "prometheus", Value: "prometheus-core-1"},
|
||||
{Name: "datacenter", Value: "eu-west-1"},
|
||||
{Name: "pod_name", Value: "abcdef-99999-defee"},
|
||||
}
|
||||
|
||||
func BenchmarkBuilder(b *testing.B) {
|
||||
var l Labels
|
||||
builder := NewBuilder(EmptyLabels())
|
||||
for i := 0; i < b.N; i++ {
|
||||
builder.Reset(EmptyLabels())
|
||||
for _, l := range benchmarkLabels {
|
||||
builder.Set(l.Name, l.Value)
|
||||
}
|
||||
l = builder.Labels()
|
||||
}
|
||||
require.Equal(b, 9, l.Len())
|
||||
}
|
|
@ -29,8 +29,8 @@ import (
|
|||
"go.uber.org/multierr"
|
||||
|
||||
"github.com/prometheus/prometheus/config"
|
||||
"github.com/prometheus/prometheus/model/labels"
|
||||
"github.com/prometheus/prometheus/model/metadata"
|
||||
"github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite/labels"
|
||||
"github.com/prometheus/prometheus/util/annotations"
|
||||
)
|
||||
|
||||
|
|
|
@ -33,10 +33,9 @@ import (
|
|||
|
||||
"github.com/prometheus/prometheus/model/exemplar"
|
||||
"github.com/prometheus/prometheus/model/histogram"
|
||||
modelLabels "github.com/prometheus/prometheus/model/labels"
|
||||
"github.com/prometheus/prometheus/model/labels"
|
||||
"github.com/prometheus/prometheus/model/metadata"
|
||||
"github.com/prometheus/prometheus/storage"
|
||||
"github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite/labels"
|
||||
)
|
||||
|
||||
func TestFromMetrics(t *testing.T) {
|
||||
|
@ -113,7 +112,7 @@ func TestFromMetrics(t *testing.T) {
|
|||
tgtInfoCount := 0
|
||||
for _, s := range ts {
|
||||
lbls := s.ls
|
||||
if lbls.Get(modelLabels.MetricName) == "target_info" {
|
||||
if lbls.Get(labels.MetricName) == "target_info" {
|
||||
tgtInfoCount++
|
||||
require.Equal(t, "test-namespace/test-service", lbls.Get("job"))
|
||||
require.Equal(t, "id1234", lbls.Get("instance"))
|
||||
|
@ -1006,30 +1005,30 @@ type noOpAppender struct {
|
|||
|
||||
var _ storage.Appender = &noOpAppender{}
|
||||
|
||||
func (a *noOpAppender) Append(_ storage.SeriesRef, _ modelLabels.Labels, _ int64, _ float64) (storage.SeriesRef, error) {
|
||||
func (a *noOpAppender) Append(_ storage.SeriesRef, _ labels.Labels, _ int64, _ float64) (storage.SeriesRef, error) {
|
||||
a.samples++
|
||||
return 1, nil
|
||||
}
|
||||
|
||||
func (*noOpAppender) AppendCTZeroSample(_ storage.SeriesRef, _ modelLabels.Labels, _, _ int64) (storage.SeriesRef, error) {
|
||||
func (*noOpAppender) AppendCTZeroSample(_ storage.SeriesRef, _ labels.Labels, _, _ int64) (storage.SeriesRef, error) {
|
||||
return 1, nil
|
||||
}
|
||||
|
||||
func (a *noOpAppender) AppendHistogram(_ storage.SeriesRef, _ modelLabels.Labels, _ int64, _ *histogram.Histogram, _ *histogram.FloatHistogram) (storage.SeriesRef, error) {
|
||||
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, _ modelLabels.Labels, _, _ int64, _ *histogram.Histogram, _ *histogram.FloatHistogram) (storage.SeriesRef, error) {
|
||||
func (*noOpAppender) AppendHistogramCTZeroSample(_ storage.SeriesRef, _ labels.Labels, _, _ int64, _ *histogram.Histogram, _ *histogram.FloatHistogram) (storage.SeriesRef, error) {
|
||||
return 1, nil
|
||||
}
|
||||
|
||||
func (a *noOpAppender) UpdateMetadata(_ storage.SeriesRef, _ modelLabels.Labels, _ metadata.Metadata) (storage.SeriesRef, error) {
|
||||
func (a *noOpAppender) UpdateMetadata(_ storage.SeriesRef, _ labels.Labels, _ metadata.Metadata) (storage.SeriesRef, error) {
|
||||
a.metadata++
|
||||
return 1, nil
|
||||
}
|
||||
|
||||
func (*noOpAppender) AppendExemplar(_ storage.SeriesRef, _ modelLabels.Labels, _ exemplar.Exemplar) (storage.SeriesRef, error) {
|
||||
func (*noOpAppender) AppendExemplar(_ storage.SeriesRef, _ labels.Labels, _ exemplar.Exemplar) (storage.SeriesRef, error) {
|
||||
return 1, nil
|
||||
}
|
||||
|
||||
|
|
|
@ -27,8 +27,8 @@ import (
|
|||
"go.opentelemetry.io/collector/pdata/pmetric"
|
||||
|
||||
"github.com/prometheus/prometheus/model/exemplar"
|
||||
"github.com/prometheus/prometheus/model/labels"
|
||||
"github.com/prometheus/prometheus/model/metadata"
|
||||
"github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite/labels"
|
||||
)
|
||||
|
||||
func TestPrometheusConverter_addGaugeNumberDataPoints(t *testing.T) {
|
||||
|
|
Loading…
Reference in New Issue