Do not use custom labels implementation

Signed-off-by: György Krajcsovits <gyorgy.krajcsovits@grafana.com>
This commit is contained in:
György Krajcsovits 2025-08-08 00:24:46 +02:00
parent eb3f40260b
commit 7268ac6d78
No known key found for this signature in database
GPG Key ID: 47A8F9CE80FD7C7F
10 changed files with 49 additions and 816 deletions

View File

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

View File

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

View File

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

View File

@ -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"
)

View File

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

View File

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

View File

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

View File

@ -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"
)

View File

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

View File

@ -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) {