lib/promscrape: optimize service discovery speed

- Return meta-labels for the discovered targets via promutils.Labels
  instead of map[string]string. This improves the speed of generating
  meta-labels for discovered targets by up to 5x.

- Remove memory allocations in hot paths during ScrapeWork generation.
  The ScrapeWork contains scrape settings for a single discovered target.
  This improves the service discovery speed by up to 2x.
This commit is contained in:
Aliaksandr Valialkin 2022-11-29 21:22:12 -08:00
parent c7ce4979ec
commit f325410c26
No known key found for this signature in database
GPG key ID: A72BEC6CD3D0DED1
88 changed files with 2295 additions and 2425 deletions

View file

@ -12,7 +12,6 @@ import (
"time" "time"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promrelabel" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promrelabel"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/consul" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/consul"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/dns" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/dns"
@ -130,24 +129,24 @@ func parseConfig(path string) (*Config, error) {
return cfg, nil return cfg, nil
} }
func parseLabels(target string, metaLabels map[string]string, cfg *Config) (string, []prompbmarshal.Label, error) { func parseLabels(target string, metaLabels *promutils.Labels, cfg *Config) (string, *promutils.Labels, error) {
labels := mergeLabels(target, metaLabels, cfg) labels := mergeLabels(target, metaLabels, cfg)
labels = cfg.parsedRelabelConfigs.Apply(labels, 0) labels.Labels = cfg.parsedRelabelConfigs.Apply(labels.Labels, 0)
labels = promrelabel.RemoveMetaLabels(labels[:0], labels) labels.RemoveMetaLabels()
promrelabel.SortLabels(labels) labels.Sort()
// Remove references to already deleted labels, so GC could clean strings for label name and label value past len(labels). // Remove references to already deleted labels, so GC could clean strings for label name and label value past len(labels).
// This should reduce memory usage when relabeling creates big number of temporary labels with long names and/or values. // This should reduce memory usage when relabeling creates big number of temporary labels with long names and/or values.
// See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/825 for details. // See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/825 for details.
labels = append([]prompbmarshal.Label{}, labels...) labels = labels.Clone()
if len(labels) == 0 { if labels.Len() == 0 {
return "", nil, nil return "", nil, nil
} }
schemeRelabeled := promrelabel.GetLabelValueByName(labels, "__scheme__") schemeRelabeled := labels.Get("__scheme__")
if len(schemeRelabeled) == 0 { if len(schemeRelabeled) == 0 {
schemeRelabeled = "http" schemeRelabeled = "http"
} }
addressRelabeled := promrelabel.GetLabelValueByName(labels, "__address__") addressRelabeled := labels.Get("__address__")
if len(addressRelabeled) == 0 { if len(addressRelabeled) == 0 {
return "", nil, nil return "", nil, nil
} }
@ -155,7 +154,7 @@ func parseLabels(target string, metaLabels map[string]string, cfg *Config) (stri
return "", nil, nil return "", nil, nil
} }
addressRelabeled = addMissingPort(schemeRelabeled, addressRelabeled) addressRelabeled = addMissingPort(schemeRelabeled, addressRelabeled)
alertsPathRelabeled := promrelabel.GetLabelValueByName(labels, "__alerts_path__") alertsPathRelabeled := labels.Get("__alerts_path__")
if !strings.HasPrefix(alertsPathRelabeled, "/") { if !strings.HasPrefix(alertsPathRelabeled, "/") {
alertsPathRelabeled = "/" + alertsPathRelabeled alertsPathRelabeled = "/" + alertsPathRelabeled
} }
@ -179,21 +178,12 @@ func addMissingPort(scheme, target string) string {
return target return target
} }
func mergeLabels(target string, metaLabels map[string]string, cfg *Config) []prompbmarshal.Label { func mergeLabels(target string, metaLabels *promutils.Labels, cfg *Config) *promutils.Labels {
// See https://prometheus.io/docs/prometheus/latest/configuration/configuration/#relabel_config // See https://prometheus.io/docs/prometheus/latest/configuration/configuration/#relabel_config
m := make(map[string]string) m := promutils.NewLabels(3 + metaLabels.Len())
m["__address__"] = target m.Add("__address__", target)
m["__scheme__"] = cfg.Scheme m.Add("__scheme__", cfg.Scheme)
m["__alerts_path__"] = path.Join("/", cfg.PathPrefix, alertManagerPath) m.Add("__alerts_path__", path.Join("/", cfg.PathPrefix, alertManagerPath))
for k, v := range metaLabels { m.AddFrom(metaLabels)
m[k] = v return m
}
result := make([]prompbmarshal.Label, 0, len(m))
for k, v := range m {
result = append(result, prompbmarshal.Label{
Name: k,
Value: v,
})
}
return result
} }

View file

@ -9,6 +9,7 @@ import (
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/consul" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/consul"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/dns" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/dns"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
// configWatcher supports dynamic reload of Notifier objects // configWatcher supports dynamic reload of Notifier objects
@ -123,7 +124,7 @@ func targetsFromLabels(labelsFn getLabels, cfg *Config, genFn AlertURLGenerator)
var errors []error var errors []error
duplicates := make(map[string]struct{}) duplicates := make(map[string]struct{})
for _, labels := range metaLabels { for _, labels := range metaLabels {
target := labels["__address__"] target := labels.Get("__address__")
u, processedLabels, err := parseLabels(target, labels, cfg) u, processedLabels, err := parseLabels(target, labels, cfg)
if err != nil { if err != nil {
errors = append(errors, err) errors = append(errors, err)
@ -156,7 +157,7 @@ func targetsFromLabels(labelsFn getLabels, cfg *Config, genFn AlertURLGenerator)
return targets, errors return targets, errors
} }
type getLabels func() ([]map[string]string, error) type getLabels func() ([]*promutils.Labels, error)
func (cw *configWatcher) start() error { func (cw *configWatcher) start() error {
if len(cw.cfg.StaticConfigs) > 0 { if len(cw.cfg.StaticConfigs) > 0 {
@ -182,8 +183,8 @@ func (cw *configWatcher) start() error {
} }
if len(cw.cfg.ConsulSDConfigs) > 0 { if len(cw.cfg.ConsulSDConfigs) > 0 {
err := cw.add(TargetConsul, *consul.SDCheckInterval, func() ([]map[string]string, error) { err := cw.add(TargetConsul, *consul.SDCheckInterval, func() ([]*promutils.Labels, error) {
var labels []map[string]string var labels []*promutils.Labels
for i := range cw.cfg.ConsulSDConfigs { for i := range cw.cfg.ConsulSDConfigs {
sdc := &cw.cfg.ConsulSDConfigs[i] sdc := &cw.cfg.ConsulSDConfigs[i]
targetLabels, err := sdc.GetLabels(cw.cfg.baseDir) targetLabels, err := sdc.GetLabels(cw.cfg.baseDir)
@ -200,8 +201,8 @@ func (cw *configWatcher) start() error {
} }
if len(cw.cfg.DNSSDConfigs) > 0 { if len(cw.cfg.DNSSDConfigs) > 0 {
err := cw.add(TargetDNS, *dns.SDCheckInterval, func() ([]map[string]string, error) { err := cw.add(TargetDNS, *dns.SDCheckInterval, func() ([]*promutils.Labels, error) {
var labels []map[string]string var labels []*promutils.Labels
for i := range cw.cfg.DNSSDConfigs { for i := range cw.cfg.DNSSDConfigs {
sdc := &cw.cfg.DNSSDConfigs[i] sdc := &cw.cfg.DNSSDConfigs[i]
targetLabels, err := sdc.GetLabels(cw.cfg.baseDir) targetLabels, err := sdc.GetLabels(cw.cfg.baseDir)

View file

@ -10,7 +10,7 @@ import (
"github.com/VictoriaMetrics/VictoriaMetrics/app/vmalert/templates" "github.com/VictoriaMetrics/VictoriaMetrics/app/vmalert/templates"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/flagutil" "github.com/VictoriaMetrics/VictoriaMetrics/lib/flagutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
var ( var (
@ -159,7 +159,7 @@ func notifiersFromFlags(gen AlertURLGenerator) ([]Notifier, error) {
// list of labels added during discovery. // list of labels added during discovery.
type Target struct { type Target struct {
Notifier Notifier
Labels []prompbmarshal.Label Labels *promutils.Labels
} }
// TargetType defines how the Target was discovered // TargetType defines how the Target was discovered

View file

@ -248,7 +248,7 @@
{% for _, n := range ns %} {% for _, n := range ns %}
<tr> <tr>
<td> <td>
{% for _, l := range n.Labels %} {% for _, l := range n.Labels.GetLabels() %}
<span class="ms-1 badge bg-primary">{%s l.Name %}={%s l.Value %}</span> <span class="ms-1 badge bg-primary">{%s l.Name %}={%s l.Value %}</span>
{% endfor %} {% endfor %}
</td> </td>

View file

@ -824,7 +824,7 @@ func StreamListTargets(qw422016 *qt422016.Writer, r *http.Request, targets map[n
<td> <td>
`) `)
//line app/vmalert/web.qtpl:251 //line app/vmalert/web.qtpl:251
for _, l := range n.Labels { for _, l := range n.Labels.GetLabels() {
//line app/vmalert/web.qtpl:251 //line app/vmalert/web.qtpl:251
qw422016.N().S(` qw422016.N().S(`
<span class="ms-1 badge bg-primary">`) <span class="ms-1 badge bg-primary">`)

View file

@ -15,6 +15,7 @@ The following tip changes can be tested by building VictoriaMetrics components f
## tip ## tip
* FEATURE: [vmagent](https://docs.victoriametrics.com/vmagent.html): improve [service discovery](https://docs.victoriametrics.com/sd_configs.html) performance when discovering big number of targets (10K and more).
* FEATURE: [vmagent](https://docs.victoriametrics.com/vmagent.html): add `exported_` prefix to metric names exported by scrape targets if these metric names clash with [automatically generated metrics](https://docs.victoriametrics.com/vmagent.html#automatically-generated-metrics) such as `up`, `scrape_samples_scraped`, etc. This prevents from corruption of automatically generated metrics. See [this issue](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/3406). * FEATURE: [vmagent](https://docs.victoriametrics.com/vmagent.html): add `exported_` prefix to metric names exported by scrape targets if these metric names clash with [automatically generated metrics](https://docs.victoriametrics.com/vmagent.html#automatically-generated-metrics) such as `up`, `scrape_samples_scraped`, etc. This prevents from corruption of automatically generated metrics. See [this issue](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/3406).
* FEATURE: [VictoriaMetrics cluster](https://docs.victoriametrics.com/Cluster-VictoriaMetrics.html): improve error message when the requested path cannot be properly parsed, so users could identify the issue and properly fix the path. Now the error message links to [url format docs](https://docs.victoriametrics.com/Cluster-VictoriaMetrics.html#url-format). See [this issue](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/3402). * FEATURE: [VictoriaMetrics cluster](https://docs.victoriametrics.com/Cluster-VictoriaMetrics.html): improve error message when the requested path cannot be properly parsed, so users could identify the issue and properly fix the path. Now the error message links to [url format docs](https://docs.victoriametrics.com/Cluster-VictoriaMetrics.html#url-format). See [this issue](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/3402).
* FEATURE: [vmctl](https://docs.victoriametrics.com/vmctl.html): add ability to copy data from sources via Prometheus `remote_read` protocol. See [these docs](https://docs.victoriametrics.com/vmctl.html#migrating-data-by-remote-read-protocol). The related issues: [one](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/3132) and [two](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/1101). * FEATURE: [vmctl](https://docs.victoriametrics.com/vmctl.html): add ability to copy data from sources via Prometheus `remote_read` protocol. See [these docs](https://docs.victoriametrics.com/vmctl.html#migrating-data-by-remote-read-protocol). The related issues: [one](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/3132) and [two](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/1101).

20
lib/bytesutil/itoa.go Normal file
View file

@ -0,0 +1,20 @@
package bytesutil
import (
"strconv"
)
// Itoa returns string representation of n.
//
// This function doesn't allocate memory on repeated calls for the same n.
func Itoa(n int) string {
bb := bbPool.Get()
b := bb.B[:0]
b = strconv.AppendInt(b, int64(n), 10)
s := InternString(ToUnsafeString(b))
bb.B = b
bbPool.Put(bb)
return s
}
var bbPool ByteBufferPool

View file

@ -0,0 +1,21 @@
package bytesutil
import (
"testing"
)
func TestItoa(t *testing.T) {
f := func(n int, resultExpected string) {
t.Helper()
for i := 0; i < 5; i++ {
result := Itoa(n)
if result != resultExpected {
t.Fatalf("unexpected result for Itoa(%d); got %q; want %q", n, result, resultExpected)
}
}
}
f(0, "0")
f(1, "1")
f(-123, "-123")
f(343432, "343432")
}

View file

@ -5,6 +5,7 @@ import (
"encoding/json" "encoding/json"
"testing" "testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
"gopkg.in/yaml.v2" "gopkg.in/yaml.v2"
) )
@ -121,8 +122,8 @@ func TestIfExpressionMatch(t *testing.T) {
if err := yaml.UnmarshalStrict([]byte(ifExpr), &ie); err != nil { if err := yaml.UnmarshalStrict([]byte(ifExpr), &ie); err != nil {
t.Fatalf("unexpected error during unmarshal: %s", err) t.Fatalf("unexpected error during unmarshal: %s", err)
} }
labels := MustParseMetricWithLabels(metricWithLabels) labels := promutils.NewLabelsFromString(metricWithLabels)
if !ie.Match(labels) { if !ie.Match(labels.GetLabels()) {
t.Fatalf("unexpected mismatch of ifExpr=%s for %s", ifExpr, metricWithLabels) t.Fatalf("unexpected mismatch of ifExpr=%s for %s", ifExpr, metricWithLabels)
} }
} }
@ -155,8 +156,8 @@ func TestIfExpressionMismatch(t *testing.T) {
if err := yaml.UnmarshalStrict([]byte(ifExpr), &ie); err != nil { if err := yaml.UnmarshalStrict([]byte(ifExpr), &ie); err != nil {
t.Fatalf("unexpected error during unmarshal: %s", err) t.Fatalf("unexpected error during unmarshal: %s", err)
} }
labels := MustParseMetricWithLabels(metricWithLabels) labels := promutils.NewLabelsFromString(metricWithLabels)
if ie.Match(labels) { if ie.Match(labels.GetLabels()) {
t.Fatalf("unexpected match of ifExpr=%s for %s", ifExpr, metricWithLabels) t.Fatalf("unexpected match of ifExpr=%s for %s", ifExpr, metricWithLabels)
} }
} }

View file

@ -9,6 +9,7 @@ import (
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil" "github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger" "github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal" "github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/regexutil" "github.com/VictoriaMetrics/VictoriaMetrics/lib/regexutil"
"github.com/cespare/xxhash/v2" "github.com/cespare/xxhash/v2"
) )
@ -48,8 +49,6 @@ func (prc *parsedRelabelConfig) String() string {
} }
// Apply applies pcs to labels starting from the labelsOffset. // Apply applies pcs to labels starting from the labelsOffset.
//
// If isFinalize is set, then FinalizeLabels is called on the labels[labelsOffset:].
func (pcs *ParsedConfigs) Apply(labels []prompbmarshal.Label, labelsOffset int) []prompbmarshal.Label { func (pcs *ParsedConfigs) Apply(labels []prompbmarshal.Label, labelsOffset int) []prompbmarshal.Label {
var inStr string var inStr string
relabelDebug := false relabelDebug := false
@ -111,32 +110,6 @@ func removeEmptyLabels(labels []prompbmarshal.Label, labelsOffset int) []prompbm
return dst return dst
} }
// RemoveMetaLabels removes all the `__meta_` labels from src and puts the rest of labels to dst.
//
// See https://www.robustperception.io/life-of-a-label fo details.
func RemoveMetaLabels(dst, src []prompbmarshal.Label) []prompbmarshal.Label {
for _, label := range src {
if strings.HasPrefix(label.Name, "__meta_") {
continue
}
dst = append(dst, label)
}
return dst
}
// RemoveLabelsWithDoubleDashPrefix removes labels with "__" prefix from src, appends the remaining lables to dst and returns the result.
func RemoveLabelsWithDoubleDashPrefix(dst, src []prompbmarshal.Label) []prompbmarshal.Label {
for _, label := range src {
name := label.Name
// A hack: do not delete __vm_filepath label, since it is used by internal logic for FileSDConfig.
if strings.HasPrefix(name, "__") && name != "__vm_filepath" {
continue
}
dst = append(dst, label)
}
return dst
}
// FinalizeLabels removes labels with "__" in the beginning (except of "__name__"). // FinalizeLabels removes labels with "__" in the beginning (except of "__name__").
func FinalizeLabels(dst, src []prompbmarshal.Label) []prompbmarshal.Label { func FinalizeLabels(dst, src []prompbmarshal.Label) []prompbmarshal.Label {
for _, label := range src { for _, label := range src {
@ -164,7 +137,7 @@ func (prc *parsedRelabelConfig) apply(labels []prompbmarshal.Label, labelsOffset
} }
switch prc.Action { switch prc.Action {
case "graphite": case "graphite":
metricName := GetLabelValueByName(src, "__name__") metricName := getLabelValue(src, "__name__")
gm := graphiteMatchesPool.Get().(*graphiteMatches) gm := graphiteMatchesPool.Get().(*graphiteMatches)
var ok bool var ok bool
gm.a, ok = prc.graphiteMatchTemplate.Match(gm.a[:0], metricName) gm.a, ok = prc.graphiteMatchTemplate.Match(gm.a[:0], metricName)
@ -464,9 +437,9 @@ func areEqualLabelValues(labels []prompbmarshal.Label, labelNames []string) bool
logger.Panicf("BUG: expecting at least 2 labelNames; got %d", len(labelNames)) logger.Panicf("BUG: expecting at least 2 labelNames; got %d", len(labelNames))
return false return false
} }
labelValue := GetLabelValueByName(labels, labelNames[0]) labelValue := getLabelValue(labels, labelNames[0])
for _, labelName := range labelNames[1:] { for _, labelName := range labelNames[1:] {
v := GetLabelValueByName(labels, labelName) v := getLabelValue(labels, labelName)
if v != labelValue { if v != labelValue {
return false return false
} }
@ -500,6 +473,15 @@ func setLabelValue(labels []prompbmarshal.Label, labelsOffset int, name, value s
return labels return labels
} }
func getLabelValue(labels []prompbmarshal.Label, name string) string {
for _, label := range labels {
if label.Name == name {
return label.Value
}
}
return ""
}
// GetLabelByName returns label with the given name from labels. // GetLabelByName returns label with the given name from labels.
func GetLabelByName(labels []prompbmarshal.Label, name string) *prompbmarshal.Label { func GetLabelByName(labels []prompbmarshal.Label, name string) *prompbmarshal.Label {
for i := range labels { for i := range labels {
@ -511,17 +493,6 @@ func GetLabelByName(labels []prompbmarshal.Label, name string) *prompbmarshal.La
return nil return nil
} }
// GetLabelValueByName returns value for label with the given name from labels.
//
// It returns empty string for non-existing label.
func GetLabelValueByName(labels []prompbmarshal.Label, name string) string {
label := GetLabelByName(labels, name)
if label == nil {
return ""
}
return label.Value
}
// CleanLabels sets label.Name and label.Value to an empty string for all the labels. // CleanLabels sets label.Name and label.Value to an empty string for all the labels.
// //
// This should help GC cleaning up label.Name and label.Value strings. // This should help GC cleaning up label.Name and label.Value strings.
@ -563,6 +534,14 @@ func labelsToString(labels []prompbmarshal.Label) string {
return string(b) return string(b)
} }
// SortLabels sorts labels in alphabetical order.
func SortLabels(labels []prompbmarshal.Label) {
x := &promutils.Labels{
Labels: labels,
}
x.Sort()
}
func fillLabelReferences(dst []byte, replacement string, labels []prompbmarshal.Label) []byte { func fillLabelReferences(dst []byte, replacement string, labels []prompbmarshal.Label) []byte {
s := replacement s := replacement
for len(s) > 0 { for len(s) > 0 {
@ -579,7 +558,7 @@ func fillLabelReferences(dst []byte, replacement string, labels []prompbmarshal.
} }
labelName := s[:n] labelName := s[:n]
s = s[n+2:] s = s[n+2:]
labelValue := GetLabelValueByName(labels, labelName) labelValue := getLabelValue(labels, labelName)
dst = append(dst, labelValue...) dst = append(dst, labelValue...)
} }
return dst return dst

View file

@ -5,6 +5,7 @@ import (
"testing" "testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal" "github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
func TestSanitizeName(t *testing.T) { func TestSanitizeName(t *testing.T) {
@ -77,8 +78,8 @@ func TestApplyRelabelConfigs(t *testing.T) {
if err != nil { if err != nil {
t.Fatalf("cannot parse %q: %s", config, err) t.Fatalf("cannot parse %q: %s", config, err)
} }
labels := MustParseMetricWithLabels(metric) labels := promutils.NewLabelsFromString(metric)
resultLabels := pcs.Apply(labels, 0) resultLabels := pcs.Apply(labels.GetLabels(), 0)
if isFinalize { if isFinalize {
resultLabels = FinalizeLabels(resultLabels[:0], resultLabels) resultLabels = FinalizeLabels(resultLabels[:0], resultLabels)
} }
@ -725,8 +726,8 @@ func TestApplyRelabelConfigs(t *testing.T) {
func TestFinalizeLabels(t *testing.T) { func TestFinalizeLabels(t *testing.T) {
f := func(metric, resultExpected string) { f := func(metric, resultExpected string) {
t.Helper() t.Helper()
labels := MustParseMetricWithLabels(metric) labels := promutils.NewLabelsFromString(metric)
resultLabels := FinalizeLabels(nil, labels) resultLabels := FinalizeLabels(nil, labels.GetLabels())
result := labelsToString(resultLabels) result := labelsToString(resultLabels)
if result != resultExpected { if result != resultExpected {
t.Fatalf("unexpected result; got\n%s\nwant\n%s", result, resultExpected) t.Fatalf("unexpected result; got\n%s\nwant\n%s", result, resultExpected)
@ -738,27 +739,11 @@ func TestFinalizeLabels(t *testing.T) {
f(`{foo="bar",abc="def",__address__="foo.com"}`, `{abc="def",foo="bar"}`) f(`{foo="bar",abc="def",__address__="foo.com"}`, `{abc="def",foo="bar"}`)
} }
func TestRemoveMetaLabels(t *testing.T) {
f := func(metric, resultExpected string) {
t.Helper()
labels := MustParseMetricWithLabels(metric)
resultLabels := RemoveMetaLabels(nil, labels)
result := labelsToString(resultLabels)
if result != resultExpected {
t.Fatalf("unexpected result of RemoveMetaLabels;\ngot\n%s\nwant\n%s", result, resultExpected)
}
}
f(`{}`, `{}`)
f(`{foo="bar"}`, `{foo="bar"}`)
f(`{__meta_foo="bar"}`, `{}`)
f(`{__meta_foo="bdffr",foo="bar",__meta_xxx="basd"}`, `{foo="bar"}`)
}
func TestFillLabelReferences(t *testing.T) { func TestFillLabelReferences(t *testing.T) {
f := func(replacement, metric, resultExpected string) { f := func(replacement, metric, resultExpected string) {
t.Helper() t.Helper()
labels := MustParseMetricWithLabels(metric) labels := promutils.NewLabelsFromString(metric)
result := fillLabelReferences(nil, replacement, labels) result := fillLabelReferences(nil, replacement, labels.GetLabels())
if string(result) != resultExpected { if string(result) != resultExpected {
t.Fatalf("unexpected result; got\n%q\nwant\n%q", result, resultExpected) t.Fatalf("unexpected result; got\n%q\nwant\n%q", result, resultExpected)
} }

View file

@ -1,54 +0,0 @@
package promrelabel
import (
"sort"
"sync"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
)
// SortLabels sorts labels.
func SortLabels(labels []prompbmarshal.Label) {
if len(labels) < 2 {
return
}
ls := labelsSorterPool.Get().(*labelsSorter)
*ls = labels
if !sort.IsSorted(ls) {
sort.Sort(ls)
}
*ls = nil
labelsSorterPool.Put(ls)
}
// SortLabelsStable sorts labels using stable sort.
func SortLabelsStable(labels []prompbmarshal.Label) {
if len(labels) < 2 {
return
}
ls := labelsSorterPool.Get().(*labelsSorter)
*ls = labels
if !sort.IsSorted(ls) {
sort.Stable(ls)
}
*ls = nil
labelsSorterPool.Put(ls)
}
var labelsSorterPool = &sync.Pool{
New: func() interface{} {
return &labelsSorter{}
},
}
type labelsSorter []prompbmarshal.Label
func (ls *labelsSorter) Len() int { return len(*ls) }
func (ls *labelsSorter) Swap(i, j int) {
a := *ls
a[i], a[j] = a[j], a[i]
}
func (ls *labelsSorter) Less(i, j int) bool {
a := *ls
return a[i].Name < a[j].Name
}

View file

@ -1,43 +0,0 @@
package promrelabel
import (
"reflect"
"testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
)
func TestSortLabels(t *testing.T) {
labels := []prompbmarshal.Label{
{
Name: "foo",
Value: "bar",
},
{
Name: "aa",
Value: "bb",
},
{
Name: "ba",
Value: "zz",
},
}
labelsExpected := []prompbmarshal.Label{
{
Name: "aa",
Value: "bb",
},
{
Name: "ba",
Value: "zz",
},
{
Name: "foo",
Value: "bar",
},
}
SortLabels(labels)
if !reflect.DeepEqual(labels, labelsExpected) {
t.Fatalf("unexpected sorted labels; got\n%v\nwant\n%v", labels, labelsExpected)
}
}

View file

@ -1,50 +0,0 @@
package promrelabel
import (
"fmt"
"strings"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/prometheus"
)
// MustParseMetricWithLabels parses s, which can have the form `metric{labels}`.
//
// This function is indended mostly for tests.
func MustParseMetricWithLabels(metricWithLabels string) []prompbmarshal.Label {
stripDummyMetric := false
if strings.HasPrefix(metricWithLabels, "{") {
// Add a dummy metric name, since the parser needs it
metricWithLabels = "dummy_metric" + metricWithLabels
stripDummyMetric = true
}
// add a value to metricWithLabels, so it could be parsed by prometheus protocol parser.
s := metricWithLabels + " 123"
var rows prometheus.Rows
var err error
rows.UnmarshalWithErrLogger(s, func(s string) {
err = fmt.Errorf("error during metric parse: %s", s)
})
if err != nil {
logger.Panicf("BUG: cannot parse %q: %s", metricWithLabels, err)
}
if len(rows.Rows) != 1 {
logger.Panicf("BUG: unexpected number of rows parsed; got %d; want 1", len(rows.Rows))
}
r := rows.Rows[0]
var lfs []prompbmarshal.Label
if !stripDummyMetric {
lfs = append(lfs, prompbmarshal.Label{
Name: "__name__",
Value: r.Metric,
})
}
for _, tag := range r.Tags {
lfs = append(lfs, prompbmarshal.Label{
Name: tag.Key,
Value: tag.Value,
})
}
return lfs
}

View file

@ -62,9 +62,21 @@ func addMissingPort(addr string, isTLS bool) string {
return addr return addr
} }
if isTLS { if isTLS {
return addr + ":443" return concatTwoStrings(addr, ":443")
} }
return addr + ":80" return concatTwoStrings(addr, ":80")
}
func concatTwoStrings(x, y string) string {
bb := bbPool.Get()
b := bb.B[:0]
b = append(b, x...)
b = append(b, y...)
s := bytesutil.ToUnsafeString(b)
s = bytesutil.InternString(s)
bb.B = b
bbPool.Put(bb)
return s
} }
func newClient(sw *ScrapeWork) *client { func newClient(sw *ScrapeWork) *client {

View file

@ -9,7 +9,6 @@ import (
"sort" "sort"
"strconv" "strconv"
"strings" "strings"
"sync"
"time" "time"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/auth" "github.com/VictoriaMetrics/VictoriaMetrics/lib/auth"
@ -19,7 +18,6 @@ import (
"github.com/VictoriaMetrics/VictoriaMetrics/lib/fs" "github.com/VictoriaMetrics/VictoriaMetrics/lib/fs"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger" "github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promrelabel" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promrelabel"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/azure" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/azure"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/consul" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/consul"
@ -230,23 +228,7 @@ func (cfg *Config) getJobNames() []string {
type GlobalConfig struct { type GlobalConfig struct {
ScrapeInterval *promutils.Duration `yaml:"scrape_interval,omitempty"` ScrapeInterval *promutils.Duration `yaml:"scrape_interval,omitempty"`
ScrapeTimeout *promutils.Duration `yaml:"scrape_timeout,omitempty"` ScrapeTimeout *promutils.Duration `yaml:"scrape_timeout,omitempty"`
ExternalLabels map[string]string `yaml:"external_labels,omitempty"` ExternalLabels *promutils.Labels `yaml:"external_labels,omitempty"`
}
func (gc *GlobalConfig) getExternalLabels() []prompbmarshal.Label {
externalLabels := gc.ExternalLabels
if len(externalLabels) == 0 {
return nil
}
labels := make([]prompbmarshal.Label, 0, len(externalLabels))
for name, value := range externalLabels {
labels = append(labels, prompbmarshal.Label{
Name: name,
Value: value,
})
}
promrelabel.SortLabels(labels)
return labels
} }
// ScrapeConfig represents essential parts for `scrape_config` section of Prometheus config. // ScrapeConfig represents essential parts for `scrape_config` section of Prometheus config.
@ -301,8 +283,8 @@ type ScrapeConfig struct {
} }
func (sc *ScrapeConfig) mustStart(baseDir string) { func (sc *ScrapeConfig) mustStart(baseDir string) {
swosFunc := func(metaLabels map[string]string) interface{} { swosFunc := func(metaLabels *promutils.Labels) interface{} {
target := metaLabels["__address__"] target := metaLabels.Get("__address__")
sw, err := sc.swc.getScrapeWork(target, nil, metaLabels) sw, err := sc.swc.getScrapeWork(target, nil, metaLabels)
if err != nil { if err != nil {
logger.Errorf("cannot create kubernetes_sd_config target %q for job_name %q: %s", target, sc.swc.jobName, err) logger.Errorf("cannot create kubernetes_sd_config target %q for job_name %q: %s", target, sc.swc.jobName, err)
@ -367,7 +349,7 @@ type FileSDConfig struct {
// See https://prometheus.io/docs/prometheus/latest/configuration/configuration/#static_config // See https://prometheus.io/docs/prometheus/latest/configuration/configuration/#static_config
type StaticConfig struct { type StaticConfig struct {
Targets []string `yaml:"targets"` Targets []string `yaml:"targets"`
Labels map[string]string `yaml:"labels,omitempty"` Labels *promutils.Labels `yaml:"labels,omitempty"`
} }
func loadStaticConfigs(path string) ([]StaticConfig, error) { func loadStaticConfigs(path string) ([]StaticConfig, error) {
@ -723,7 +705,7 @@ func (cfg *Config) getFileSDScrapeWork(prev []*ScrapeWork) []*ScrapeWork {
// Create a map for the previous scrape work. // Create a map for the previous scrape work.
swsMapPrev := make(map[string][]*ScrapeWork) swsMapPrev := make(map[string][]*ScrapeWork)
for _, sw := range prev { for _, sw := range prev {
filepath := promrelabel.GetLabelValueByName(sw.Labels, "__vm_filepath") filepath := sw.Labels.Get("__vm_filepath")
if len(filepath) == 0 { if len(filepath) == 0 {
logger.Panicf("BUG: missing `__vm_filepath` label") logger.Panicf("BUG: missing `__vm_filepath` label")
} else { } else {
@ -960,7 +942,7 @@ func getScrapeWorkConfig(sc *ScrapeConfig, baseDir string, globalCfg *GlobalConf
if (*streamParse || sc.StreamParse) && sc.SeriesLimit > 0 { if (*streamParse || sc.StreamParse) && sc.SeriesLimit > 0 {
return nil, fmt.Errorf("cannot use stream parsing mode when `series_limit` is set for `job_name` %q", jobName) return nil, fmt.Errorf("cannot use stream parsing mode when `series_limit` is set for `job_name` %q", jobName)
} }
externalLabels := globalCfg.getExternalLabels() externalLabels := globalCfg.ExternalLabels
noStaleTracking := *noStaleMarkers noStaleTracking := *noStaleMarkers
if sc.NoStaleMarkers != nil { if sc.NoStaleMarkers != nil {
noStaleTracking = *sc.NoStaleMarkers noStaleTracking = *sc.NoStaleMarkers
@ -1010,7 +992,7 @@ type scrapeWorkConfig struct {
honorLabels bool honorLabels bool
honorTimestamps bool honorTimestamps bool
denyRedirects bool denyRedirects bool
externalLabels []prompbmarshal.Label externalLabels *promutils.Labels
relabelConfigs *promrelabel.ParsedConfigs relabelConfigs *promrelabel.ParsedConfigs
metricRelabelConfigs *promrelabel.ParsedConfigs metricRelabelConfigs *promrelabel.ParsedConfigs
sampleLimit int sampleLimit int
@ -1024,7 +1006,7 @@ type scrapeWorkConfig struct {
} }
type targetLabelsGetter interface { type targetLabelsGetter interface {
GetLabels(baseDir string) ([]map[string]string, error) GetLabels(baseDir string) ([]*promutils.Labels, error)
} }
func appendSDScrapeWork(dst []*ScrapeWork, sdc targetLabelsGetter, baseDir string, swc *scrapeWorkConfig, discoveryType string) ([]*ScrapeWork, bool) { func appendSDScrapeWork(dst []*ScrapeWork, sdc targetLabelsGetter, baseDir string, swc *scrapeWorkConfig, discoveryType string) ([]*ScrapeWork, bool) {
@ -1036,7 +1018,7 @@ func appendSDScrapeWork(dst []*ScrapeWork, sdc targetLabelsGetter, baseDir strin
return appendScrapeWorkForTargetLabels(dst, swc, targetLabels, discoveryType), true return appendScrapeWorkForTargetLabels(dst, swc, targetLabels, discoveryType), true
} }
func appendScrapeWorkForTargetLabels(dst []*ScrapeWork, swc *scrapeWorkConfig, targetLabels []map[string]string, discoveryType string) []*ScrapeWork { func appendScrapeWorkForTargetLabels(dst []*ScrapeWork, swc *scrapeWorkConfig, targetLabels []*promutils.Labels, discoveryType string) []*ScrapeWork {
startTime := time.Now() startTime := time.Now()
// Process targetLabels in parallel in order to reduce processing time for big number of targetLabels. // Process targetLabels in parallel in order to reduce processing time for big number of targetLabels.
type result struct { type result struct {
@ -1045,11 +1027,11 @@ func appendScrapeWorkForTargetLabels(dst []*ScrapeWork, swc *scrapeWorkConfig, t
} }
goroutines := cgroup.AvailableCPUs() goroutines := cgroup.AvailableCPUs()
resultCh := make(chan result, len(targetLabels)) resultCh := make(chan result, len(targetLabels))
workCh := make(chan map[string]string, goroutines) workCh := make(chan *promutils.Labels, goroutines)
for i := 0; i < goroutines; i++ { for i := 0; i < goroutines; i++ {
go func() { go func() {
for metaLabels := range workCh { for metaLabels := range workCh {
target := metaLabels["__address__"] target := metaLabels.Get("__address__")
sw, err := swc.getScrapeWork(target, nil, metaLabels) sw, err := swc.getScrapeWork(target, nil, metaLabels)
if err != nil { if err != nil {
err = fmt.Errorf("skipping %s target %q for job_name %q because of error: %w", discoveryType, target, swc.jobName, err) err = fmt.Errorf("skipping %s target %q for job_name %q because of error: %w", discoveryType, target, swc.jobName, err)
@ -1080,6 +1062,8 @@ func appendScrapeWorkForTargetLabels(dst []*ScrapeWork, swc *scrapeWorkConfig, t
} }
func (sdc *FileSDConfig) appendScrapeWork(dst []*ScrapeWork, swsMapPrev map[string][]*ScrapeWork, baseDir string, swc *scrapeWorkConfig) []*ScrapeWork { func (sdc *FileSDConfig) appendScrapeWork(dst []*ScrapeWork, swsMapPrev map[string][]*ScrapeWork, baseDir string, swc *scrapeWorkConfig) []*ScrapeWork {
metaLabels := promutils.GetLabels()
defer promutils.PutLabels(metaLabels)
for _, file := range sdc.Files { for _, file := range sdc.Files {
pathPattern := fs.GetFilepath(baseDir, file) pathPattern := fs.GetFilepath(baseDir, file)
paths := []string{pathPattern} paths := []string{pathPattern}
@ -1112,10 +1096,9 @@ func (sdc *FileSDConfig) appendScrapeWork(dst []*ScrapeWork, swsMapPrev map[stri
pathShort = pathShort[1:] pathShort = pathShort[1:]
} }
} }
metaLabels := map[string]string{ metaLabels.Reset()
"__meta_filepath": pathShort, metaLabels.Add("__meta_filepath", pathShort)
"__vm_filepath": path, // This label is needed for internal promscrape logic metaLabels.Add("__vm_filepath", path) // This label is needed for internal promscrape logic
}
for i := range stcs { for i := range stcs {
dst = stcs[i].appendScrapeWork(dst, swc, metaLabels) dst = stcs[i].appendScrapeWork(dst, swc, metaLabels)
} }
@ -1124,7 +1107,7 @@ func (sdc *FileSDConfig) appendScrapeWork(dst []*ScrapeWork, swsMapPrev map[stri
return dst return dst
} }
func (stc *StaticConfig) appendScrapeWork(dst []*ScrapeWork, swc *scrapeWorkConfig, metaLabels map[string]string) []*ScrapeWork { func (stc *StaticConfig) appendScrapeWork(dst []*ScrapeWork, swc *scrapeWorkConfig, metaLabels *promutils.Labels) []*ScrapeWork {
for _, target := range stc.Targets { for _, target := range stc.Targets {
if target == "" { if target == "" {
// Do not return this error, since other targets may be valid // Do not return this error, since other targets may be valid
@ -1144,8 +1127,8 @@ func (stc *StaticConfig) appendScrapeWork(dst []*ScrapeWork, swc *scrapeWorkConf
return dst return dst
} }
func appendScrapeWorkKey(dst []byte, labels []prompbmarshal.Label) []byte { func appendScrapeWorkKey(dst []byte, labels *promutils.Labels) []byte {
for _, label := range labels { for _, label := range labels.GetLabels() {
// Do not use strconv.AppendQuote, since it is slow according to CPU profile. // Do not use strconv.AppendQuote, since it is slow according to CPU profile.
dst = append(dst, label.Name...) dst = append(dst, label.Name...)
dst = append(dst, '=') dst = append(dst, '=')
@ -1176,45 +1159,20 @@ func needSkipScrapeWork(key string, membersCount, replicasCount, memberNum int)
return true return true
} }
type labelsContext struct {
labels []prompbmarshal.Label
}
func getLabelsContext() *labelsContext {
v := labelsContextPool.Get()
if v == nil {
return &labelsContext{}
}
return v.(*labelsContext)
}
func putLabelsContext(lctx *labelsContext) {
labels := lctx.labels
for i := range labels {
labels[i].Name = ""
labels[i].Value = ""
}
lctx.labels = lctx.labels[:0]
labelsContextPool.Put(lctx)
}
var labelsContextPool sync.Pool
var scrapeWorkKeyBufPool bytesutil.ByteBufferPool var scrapeWorkKeyBufPool bytesutil.ByteBufferPool
func (swc *scrapeWorkConfig) getScrapeWork(target string, extraLabels, metaLabels map[string]string) (*ScrapeWork, error) { func (swc *scrapeWorkConfig) getScrapeWork(target string, extraLabels, metaLabels *promutils.Labels) (*ScrapeWork, error) {
lctx := getLabelsContext() labels := promutils.GetLabels()
defer putLabelsContext(lctx) defer promutils.PutLabels(labels)
labels := mergeLabels(lctx.labels[:0], swc, target, extraLabels, metaLabels) mergeLabels(labels, swc, target, extraLabels, metaLabels)
var originalLabels []prompbmarshal.Label var originalLabels *promutils.Labels
if !*dropOriginalLabels { if !*dropOriginalLabels {
originalLabels = append([]prompbmarshal.Label{}, labels...) originalLabels = labels.Clone()
} }
labels = swc.relabelConfigs.Apply(labels, 0) labels.Labels = swc.relabelConfigs.Apply(labels.Labels, 0)
// Remove labels starting from "__meta_" prefix according to https://www.robustperception.io/life-of-a-label/ // Remove labels starting from "__meta_" prefix according to https://www.robustperception.io/life-of-a-label/
labels = promrelabel.RemoveMetaLabels(labels[:0], labels) labels.RemoveMetaLabels()
lctx.labels = labels
// Verify whether the scrape work must be skipped because of `-promscrape.cluster.*` configs. // Verify whether the scrape work must be skipped because of `-promscrape.cluster.*` configs.
// Perform the verification on labels after the relabeling in order to guarantee that targets with the same set of labels // Perform the verification on labels after the relabeling in order to guarantee that targets with the same set of labels
@ -1230,25 +1188,25 @@ func (swc *scrapeWorkConfig) getScrapeWork(target string, extraLabels, metaLabel
} }
} }
if !*dropOriginalLabels { if !*dropOriginalLabels {
promrelabel.SortLabels(originalLabels) originalLabels.Sort()
// Reduce memory usage by interning all the strings in originalLabels. // Reduce memory usage by interning all the strings in originalLabels.
internLabelStrings(originalLabels) originalLabels.InternStrings()
} }
if len(labels) == 0 { if labels.Len() == 0 {
// Drop target without labels. // Drop target without labels.
droppedTargetsMap.Register(originalLabels) droppedTargetsMap.Register(originalLabels)
return nil, nil return nil, nil
} }
// See https://www.robustperception.io/life-of-a-label // See https://www.robustperception.io/life-of-a-label
scheme := promrelabel.GetLabelValueByName(labels, "__scheme__") scheme := labels.Get("__scheme__")
if len(scheme) == 0 { if len(scheme) == 0 {
scheme = "http" scheme = "http"
} }
metricsPath := promrelabel.GetLabelValueByName(labels, "__metrics_path__") metricsPath := labels.Get("__metrics_path__")
if len(metricsPath) == 0 { if len(metricsPath) == 0 {
metricsPath = "/metrics" metricsPath = "/metrics"
} }
address := promrelabel.GetLabelValueByName(labels, "__address__") address := labels.Get("__address__")
if len(address) == 0 { if len(address) == 0 {
// Drop target without scrape address. // Drop target without scrape address.
droppedTargetsMap.Register(originalLabels) droppedTargetsMap.Register(originalLabels)
@ -1271,7 +1229,7 @@ func (swc *scrapeWorkConfig) getScrapeWork(target string, extraLabels, metaLabel
address = addMissingPort(address, scheme == "https") address = addMissingPort(address, scheme == "https")
var at *auth.Token var at *auth.Token
tenantID := promrelabel.GetLabelValueByName(labels, "__tenant_id__") tenantID := labels.Get("__tenant_id__")
if len(tenantID) > 0 { if len(tenantID) > 0 {
newToken, err := auth.NewToken(tenantID) newToken, err := auth.NewToken(tenantID)
if err != nil { if err != nil {
@ -1292,14 +1250,14 @@ func (swc *scrapeWorkConfig) getScrapeWork(target string, extraLabels, metaLabel
} }
} }
paramsStr := url.Values(params).Encode() paramsStr := url.Values(params).Encode()
scrapeURL := fmt.Sprintf("%s://%s%s%s%s", scheme, address, metricsPath, optionalQuestion, paramsStr) scrapeURL := getScrapeURL(scheme, address, metricsPath, optionalQuestion, paramsStr)
if _, err := url.Parse(scrapeURL); err != nil { if _, err := url.Parse(scrapeURL); err != nil {
return nil, fmt.Errorf("invalid url %q for scheme=%q, target=%q, address=%q, metrics_path=%q for job=%q: %w", return nil, fmt.Errorf("invalid url %q for scheme=%q, target=%q, address=%q, metrics_path=%q for job=%q: %w",
scrapeURL, scheme, target, address, metricsPath, swc.jobName, err) scrapeURL, scheme, target, address, metricsPath, swc.jobName, err)
} }
// Read __scrape_interval__ and __scrape_timeout__ from labels. // Read __scrape_interval__ and __scrape_timeout__ from labels.
scrapeInterval := swc.scrapeInterval scrapeInterval := swc.scrapeInterval
if s := promrelabel.GetLabelValueByName(labels, "__scrape_interval__"); len(s) > 0 { if s := labels.Get("__scrape_interval__"); len(s) > 0 {
d, err := promutils.ParseDuration(s) d, err := promutils.ParseDuration(s)
if err != nil { if err != nil {
return nil, fmt.Errorf("cannot parse __scrape_interval__=%q: %w", s, err) return nil, fmt.Errorf("cannot parse __scrape_interval__=%q: %w", s, err)
@ -1307,7 +1265,7 @@ func (swc *scrapeWorkConfig) getScrapeWork(target string, extraLabels, metaLabel
scrapeInterval = d scrapeInterval = d
} }
scrapeTimeout := swc.scrapeTimeout scrapeTimeout := swc.scrapeTimeout
if s := promrelabel.GetLabelValueByName(labels, "__scrape_timeout__"); len(s) > 0 { if s := labels.Get("__scrape_timeout__"); len(s) > 0 {
d, err := promutils.ParseDuration(s) d, err := promutils.ParseDuration(s)
if err != nil { if err != nil {
return nil, fmt.Errorf("cannot parse __scrape_timeout__=%q: %w", s, err) return nil, fmt.Errorf("cannot parse __scrape_timeout__=%q: %w", s, err)
@ -1317,7 +1275,7 @@ func (swc *scrapeWorkConfig) getScrapeWork(target string, extraLabels, metaLabel
// Read series_limit option from __series_limit__ label. // Read series_limit option from __series_limit__ label.
// See https://docs.victoriametrics.com/vmagent.html#cardinality-limiter // See https://docs.victoriametrics.com/vmagent.html#cardinality-limiter
seriesLimit := swc.seriesLimit seriesLimit := swc.seriesLimit
if s := promrelabel.GetLabelValueByName(labels, "__series_limit__"); len(s) > 0 { if s := labels.Get("__series_limit__"); len(s) > 0 {
n, err := strconv.Atoi(s) n, err := strconv.Atoi(s)
if err != nil { if err != nil {
return nil, fmt.Errorf("cannot parse __series_limit__=%q: %w", s, err) return nil, fmt.Errorf("cannot parse __series_limit__=%q: %w", s, err)
@ -1327,7 +1285,7 @@ func (swc *scrapeWorkConfig) getScrapeWork(target string, extraLabels, metaLabel
// Read stream_parse option from __stream_parse__ label. // Read stream_parse option from __stream_parse__ label.
// See https://docs.victoriametrics.com/vmagent.html#stream-parsing-mode // See https://docs.victoriametrics.com/vmagent.html#stream-parsing-mode
streamParse := swc.streamParse streamParse := swc.streamParse
if s := promrelabel.GetLabelValueByName(labels, "__stream_parse__"); len(s) > 0 { if s := labels.Get("__stream_parse__"); len(s) > 0 {
b, err := strconv.ParseBool(s) b, err := strconv.ParseBool(s)
if err != nil { if err != nil {
return nil, fmt.Errorf("cannot parse __stream_parse__=%q: %w", s, err) return nil, fmt.Errorf("cannot parse __stream_parse__=%q: %w", s, err)
@ -1335,22 +1293,19 @@ func (swc *scrapeWorkConfig) getScrapeWork(target string, extraLabels, metaLabel
streamParse = b streamParse = b
} }
// Remove labels with "__" prefix according to https://www.robustperception.io/life-of-a-label/ // Remove labels with "__" prefix according to https://www.robustperception.io/life-of-a-label/
labels = promrelabel.RemoveLabelsWithDoubleDashPrefix(labels[:0], labels) labels.RemoveLabelsWithDoubleUnderscorePrefix()
// Remove references to deleted labels, so GC could clean strings for label name and label value past len(labels). // Add missing "instance" label according to https://www.robustperception.io/life-of-a-label
if labels.Get("instance") == "" {
labels.Add("instance", address)
}
// Remove references to deleted labels, so GC could clean strings for label name and label value past len(labels.Labels).
// This should reduce memory usage when relabeling creates big number of temporary labels with long names and/or values. // This should reduce memory usage when relabeling creates big number of temporary labels with long names and/or values.
// See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/825 for details. // See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/825 for details.
labelsCopy := make([]prompbmarshal.Label, len(labels)+1) labelsCopy := labels.Clone()
labels = append(labelsCopy[:0], labels...) // Sort labels in alphabetical order of their names.
// Add missing "instance" label according to https://www.robustperception.io/life-of-a-label labelsCopy.Sort()
if promrelabel.GetLabelByName(labels, "instance") == nil {
labels = append(labels, prompbmarshal.Label{
Name: "instance",
Value: address,
})
}
promrelabel.SortLabels(labels)
// Reduce memory usage by interning all the strings in labels. // Reduce memory usage by interning all the strings in labels.
internLabelStrings(labels) labelsCopy.InternStrings()
sw := &ScrapeWork{ sw := &ScrapeWork{
ScrapeURL: scrapeURL, ScrapeURL: scrapeURL,
@ -1360,7 +1315,7 @@ func (swc *scrapeWorkConfig) getScrapeWork(target string, extraLabels, metaLabel
HonorTimestamps: swc.honorTimestamps, HonorTimestamps: swc.honorTimestamps,
DenyRedirects: swc.denyRedirects, DenyRedirects: swc.denyRedirects,
OriginalLabels: originalLabels, OriginalLabels: originalLabels,
Labels: labels, Labels: labelsCopy,
ExternalLabels: swc.externalLabels, ExternalLabels: swc.externalLabels,
ProxyURL: swc.proxyURL, ProxyURL: swc.proxyURL,
ProxyAuthConfig: swc.proxyAuthConfig, ProxyAuthConfig: swc.proxyAuthConfig,
@ -1381,19 +1336,26 @@ func (swc *scrapeWorkConfig) getScrapeWork(target string, extraLabels, metaLabel
return sw, nil return sw, nil
} }
func internLabelStrings(labels []prompbmarshal.Label) { func getScrapeURL(scheme, address, metricsPath, optionalQuestion, paramsStr string) string {
for i := range labels { bb := bbPool.Get()
label := &labels[i] b := bb.B[:0]
label.Name = bytesutil.InternString(label.Name) b = append(b, scheme...)
label.Value = bytesutil.InternString(label.Value) b = append(b, "://"...)
} b = append(b, address...)
b = append(b, metricsPath...)
b = append(b, optionalQuestion...)
b = append(b, paramsStr...)
s := bytesutil.ToUnsafeString(b)
s = bytesutil.InternString(s)
bb.B = b
bbPool.Put(bb)
return s
} }
func getParamsFromLabels(labels []prompbmarshal.Label, paramsOrig map[string][]string) map[string][]string { func getParamsFromLabels(labels *promutils.Labels, paramsOrig map[string][]string) map[string][]string {
// See https://www.robustperception.io/life-of-a-label // See https://www.robustperception.io/life-of-a-label
m := make(map[string][]string) var m map[string][]string
for i := range labels { for _, label := range labels.GetLabels() {
label := &labels[i]
if !strings.HasPrefix(label.Name, "__param_") { if !strings.HasPrefix(label.Name, "__param_") {
continue continue
} }
@ -1402,79 +1364,36 @@ func getParamsFromLabels(labels []prompbmarshal.Label, paramsOrig map[string][]s
if p := paramsOrig[name]; len(p) > 1 { if p := paramsOrig[name]; len(p) > 1 {
values = append(values, p[1:]...) values = append(values, p[1:]...)
} }
if m == nil {
m = make(map[string][]string)
}
m[name] = values m[name] = values
} }
return m return m
} }
func mergeLabels(dst []prompbmarshal.Label, swc *scrapeWorkConfig, target string, extraLabels, metaLabels map[string]string) []prompbmarshal.Label { func mergeLabels(dst *promutils.Labels, swc *scrapeWorkConfig, target string, extraLabels, metaLabels *promutils.Labels) {
if len(dst) > 0 { if n := dst.Len(); n > 0 {
logger.Panicf("BUG: len(dst) must be 0; got %d", len(dst)) logger.Panicf("BUG: len(dst.Labels) must be 0; got %d", n)
} }
// See https://prometheus.io/docs/prometheus/latest/configuration/configuration/#relabel_config // See https://prometheus.io/docs/prometheus/latest/configuration/configuration/#relabel_config
dst = appendLabel(dst, "job", swc.jobName) dst.Add("job", swc.jobName)
dst = appendLabel(dst, "__address__", target) dst.Add("__address__", target)
dst = appendLabel(dst, "__scheme__", swc.scheme) dst.Add("__scheme__", swc.scheme)
dst = appendLabel(dst, "__metrics_path__", swc.metricsPath) dst.Add("__metrics_path__", swc.metricsPath)
dst = appendLabel(dst, "__scrape_interval__", swc.scrapeIntervalString) dst.Add("__scrape_interval__", swc.scrapeIntervalString)
dst = appendLabel(dst, "__scrape_timeout__", swc.scrapeTimeoutString) dst.Add("__scrape_timeout__", swc.scrapeTimeoutString)
for k, args := range swc.params { for k, args := range swc.params {
if len(args) == 0 { if len(args) == 0 {
continue continue
} }
k = "__param_" + k k = "__param_" + k
v := args[0] v := args[0]
dst = appendLabel(dst, k, v) dst.Add(k, v)
} }
for k, v := range extraLabels { dst.AddFrom(extraLabels)
dst = appendLabel(dst, k, v) dst.AddFrom(metaLabels)
} dst.RemoveDuplicates()
for k, v := range metaLabels {
dst = appendLabel(dst, k, v)
}
if len(dst) < 2 {
return dst
}
// Remove duplicate labels if any.
// Stable sorting is needed in order to preserve the order for labels with identical names.
// This is needed in order to remove labels with duplicate names other than the last one.
promrelabel.SortLabelsStable(dst)
prevName := dst[0].Name
hasDuplicateLabels := false
for _, label := range dst[1:] {
if label.Name == prevName {
hasDuplicateLabels = true
break
}
prevName = label.Name
}
if !hasDuplicateLabels {
return dst
}
prevName = dst[0].Name
tmp := dst[:1]
for _, label := range dst[1:] {
if label.Name == prevName {
tmp[len(tmp)-1] = label
} else {
tmp = append(tmp, label)
prevName = label.Name
}
}
tail := dst[len(tmp):]
for i := range tail {
label := &tail[i]
label.Name = ""
label.Value = ""
}
return tmp
}
func appendLabel(dst []prompbmarshal.Label, name, value string) []prompbmarshal.Label {
return append(dst, prompbmarshal.Label{
Name: name,
Value: value,
})
} }
const ( const (

View file

@ -10,7 +10,6 @@ import (
"time" "time"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promrelabel" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promrelabel"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/gce" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/gce"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
@ -18,11 +17,13 @@ import (
) )
func TestMergeLabels(t *testing.T) { func TestMergeLabels(t *testing.T) {
f := func(swc *scrapeWorkConfig, target string, extraLabels, metaLabels map[string]string, resultExpected string) { f := func(swc *scrapeWorkConfig, target string, extraLabelsMap, metaLabelsMap map[string]string, resultExpected string) {
t.Helper() t.Helper()
var labels []prompbmarshal.Label extraLabels := promutils.NewLabelsFromMap(extraLabelsMap)
labels = mergeLabels(labels[:0], swc, target, extraLabels, metaLabels) metaLabels := promutils.NewLabelsFromMap(metaLabelsMap)
result := promLabelsString(labels) labels := promutils.NewLabels(0)
mergeLabels(labels, swc, target, extraLabels, metaLabels)
result := labels.String()
if result != resultExpected { if result != resultExpected {
t.Fatalf("unexpected result;\ngot\n%s\nwant\n%s", result, resultExpected) t.Fatalf("unexpected result;\ngot\n%s\nwant\n%s", result, resultExpected)
} }
@ -247,16 +248,10 @@ scrape_configs:
ScrapeInterval: defaultScrapeInterval, ScrapeInterval: defaultScrapeInterval,
ScrapeTimeout: defaultScrapeTimeout, ScrapeTimeout: defaultScrapeTimeout,
HonorTimestamps: true, HonorTimestamps: true,
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "instance": "host1:80",
Name: "instance", "job": "abc",
Value: "host1:80", }),
},
{
Name: "job",
Value: "abc",
},
},
AuthConfig: &promauth.Config{}, AuthConfig: &promauth.Config{},
ProxyAuthConfig: &promauth.Config{}, ProxyAuthConfig: &promauth.Config{},
jobNameOriginal: "abc", jobNameOriginal: "abc",
@ -266,16 +261,10 @@ scrape_configs:
ScrapeInterval: defaultScrapeInterval, ScrapeInterval: defaultScrapeInterval,
ScrapeTimeout: defaultScrapeTimeout, ScrapeTimeout: defaultScrapeTimeout,
HonorTimestamps: true, HonorTimestamps: true,
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "instance": "host2:443",
Name: "instance", "job": "abc",
Value: "host2:443", }),
},
{
Name: "job",
Value: "abc",
},
},
AuthConfig: &promauth.Config{}, AuthConfig: &promauth.Config{},
ProxyAuthConfig: &promauth.Config{}, ProxyAuthConfig: &promauth.Config{},
jobNameOriginal: "abc", jobNameOriginal: "abc",
@ -285,16 +274,10 @@ scrape_configs:
ScrapeInterval: defaultScrapeInterval, ScrapeInterval: defaultScrapeInterval,
ScrapeTimeout: defaultScrapeTimeout, ScrapeTimeout: defaultScrapeTimeout,
HonorTimestamps: true, HonorTimestamps: true,
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "instance": "host3:1234",
Name: "instance", "job": "abc",
Value: "host3:1234", }),
},
{
Name: "job",
Value: "abc",
},
},
AuthConfig: &promauth.Config{}, AuthConfig: &promauth.Config{},
ProxyAuthConfig: &promauth.Config{}, ProxyAuthConfig: &promauth.Config{},
jobNameOriginal: "abc", jobNameOriginal: "abc",
@ -304,16 +287,10 @@ scrape_configs:
ScrapeInterval: defaultScrapeInterval, ScrapeInterval: defaultScrapeInterval,
ScrapeTimeout: defaultScrapeTimeout, ScrapeTimeout: defaultScrapeTimeout,
HonorTimestamps: true, HonorTimestamps: true,
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "instance": "host4:1234",
Name: "instance", "job": "abc",
Value: "host4:1234", }),
},
{
Name: "job",
Value: "abc",
},
},
AuthConfig: &promauth.Config{}, AuthConfig: &promauth.Config{},
ProxyAuthConfig: &promauth.Config{}, ProxyAuthConfig: &promauth.Config{},
jobNameOriginal: "abc", jobNameOriginal: "abc",
@ -358,16 +335,10 @@ scrape_configs:
ScrapeInterval: defaultScrapeInterval, ScrapeInterval: defaultScrapeInterval,
ScrapeTimeout: defaultScrapeTimeout, ScrapeTimeout: defaultScrapeTimeout,
HonorTimestamps: true, HonorTimestamps: true,
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "instance": "8.8.8.8",
Name: "instance", "job": "blackbox",
Value: "8.8.8.8", }),
},
{
Name: "job",
Value: "blackbox",
},
},
AuthConfig: &promauth.Config{}, AuthConfig: &promauth.Config{},
ProxyAuthConfig: &promauth.Config{}, ProxyAuthConfig: &promauth.Config{},
jobNameOriginal: "blackbox", jobNameOriginal: "blackbox",
@ -770,8 +741,9 @@ func TestGetFileSDScrapeWorkSuccess(t *testing.T) {
// Remove `__vm_filepath` label, since its value depends on the current working dir. // Remove `__vm_filepath` label, since its value depends on the current working dir.
for _, sw := range sws { for _, sw := range sws {
for j := range sw.Labels { labels := sw.Labels.GetLabels()
label := &sw.Labels[j] for j := range labels {
label := &labels[j]
if label.Name == "__vm_filepath" { if label.Name == "__vm_filepath" {
label.Value = "" label.Value = ""
} }
@ -799,24 +771,12 @@ scrape_configs:
ScrapeInterval: defaultScrapeInterval, ScrapeInterval: defaultScrapeInterval,
ScrapeTimeout: defaultScrapeTimeout, ScrapeTimeout: defaultScrapeTimeout,
HonorTimestamps: true, HonorTimestamps: true,
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "__vm_filepath": "",
Name: "__vm_filepath", "instance": "host1:80",
Value: "", "job": "foo",
}, "qwe": "rty",
{ }),
Name: "instance",
Value: "host1:80",
},
{
Name: "job",
Value: "foo",
},
{
Name: "qwe",
Value: "rty",
},
},
AuthConfig: &promauth.Config{}, AuthConfig: &promauth.Config{},
ProxyAuthConfig: &promauth.Config{}, ProxyAuthConfig: &promauth.Config{},
jobNameOriginal: "foo", jobNameOriginal: "foo",
@ -826,24 +786,12 @@ scrape_configs:
ScrapeInterval: defaultScrapeInterval, ScrapeInterval: defaultScrapeInterval,
ScrapeTimeout: defaultScrapeTimeout, ScrapeTimeout: defaultScrapeTimeout,
HonorTimestamps: true, HonorTimestamps: true,
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "__vm_filepath": "",
Name: "__vm_filepath", "instance": "host2:80",
Value: "", "job": "foo",
}, "qwe": "rty",
{ }),
Name: "instance",
Value: "host2:80",
},
{
Name: "job",
Value: "foo",
},
{
Name: "qwe",
Value: "rty",
},
},
AuthConfig: &promauth.Config{}, AuthConfig: &promauth.Config{},
ProxyAuthConfig: &promauth.Config{}, ProxyAuthConfig: &promauth.Config{},
jobNameOriginal: "foo", jobNameOriginal: "foo",
@ -853,24 +801,12 @@ scrape_configs:
ScrapeInterval: defaultScrapeInterval, ScrapeInterval: defaultScrapeInterval,
ScrapeTimeout: defaultScrapeTimeout, ScrapeTimeout: defaultScrapeTimeout,
HonorTimestamps: true, HonorTimestamps: true,
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "__vm_filepath": "",
Name: "__vm_filepath", "instance": "localhost:9090",
Value: "", "job": "foo",
}, "yml": "test",
{ }),
Name: "instance",
Value: "localhost:9090",
},
{
Name: "job",
Value: "foo",
},
{
Name: "yml",
Value: "test",
},
},
AuthConfig: &promauth.Config{}, AuthConfig: &promauth.Config{},
ProxyAuthConfig: &promauth.Config{}, ProxyAuthConfig: &promauth.Config{},
jobNameOriginal: "foo", jobNameOriginal: "foo",
@ -902,16 +838,10 @@ scrape_configs:
ScrapeInterval: defaultScrapeInterval, ScrapeInterval: defaultScrapeInterval,
ScrapeTimeout: defaultScrapeTimeout, ScrapeTimeout: defaultScrapeTimeout,
HonorTimestamps: true, HonorTimestamps: true,
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "instance": "foo.bar:1234",
Name: "instance", "job": "foo",
Value: "foo.bar:1234", }),
},
{
Name: "job",
Value: "foo",
},
},
AuthConfig: &promauth.Config{}, AuthConfig: &promauth.Config{},
ProxyAuthConfig: &promauth.Config{}, ProxyAuthConfig: &promauth.Config{},
jobNameOriginal: "foo", jobNameOriginal: "foo",
@ -932,26 +862,14 @@ scrape_configs:
ScrapeInterval: defaultScrapeInterval, ScrapeInterval: defaultScrapeInterval,
ScrapeTimeout: defaultScrapeTimeout, ScrapeTimeout: defaultScrapeTimeout,
HonorTimestamps: true, HonorTimestamps: true,
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "instance": "foo.bar:1234",
Name: "instance", "job": "foo",
Value: "foo.bar:1234", }),
}, ExternalLabels: promutils.NewLabelsFromMap(map[string]string{
{ "datacenter": "foobar",
Name: "job", "jobs": "xxx",
Value: "foo", }),
},
},
ExternalLabels: []prompbmarshal.Label{
{
Name: "datacenter",
Value: "foobar",
},
{
Name: "jobs",
Value: "xxx",
},
},
AuthConfig: &promauth.Config{}, AuthConfig: &promauth.Config{},
ProxyAuthConfig: &promauth.Config{}, ProxyAuthConfig: &promauth.Config{},
jobNameOriginal: "foo", jobNameOriginal: "foo",
@ -996,20 +914,11 @@ scrape_configs:
HonorLabels: true, HonorLabels: true,
HonorTimestamps: false, HonorTimestamps: false,
DenyRedirects: true, DenyRedirects: true,
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "instance": "foo.bar:443",
Name: "instance", "job": "foo",
Value: "foo.bar:443", "x": "y",
}, }),
{
Name: "job",
Value: "foo",
},
{
Name: "x",
Value: "y",
},
},
AuthConfig: &promauth.Config{}, AuthConfig: &promauth.Config{},
ProxyAuthConfig: &promauth.Config{}, ProxyAuthConfig: &promauth.Config{},
ProxyURL: proxy.MustNewURL("http://foo.bar"), ProxyURL: proxy.MustNewURL("http://foo.bar"),
@ -1022,20 +931,11 @@ scrape_configs:
HonorLabels: true, HonorLabels: true,
HonorTimestamps: false, HonorTimestamps: false,
DenyRedirects: true, DenyRedirects: true,
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "instance": "aaa:443",
Name: "instance", "job": "foo",
Value: "aaa:443", "x": "y",
}, }),
{
Name: "job",
Value: "foo",
},
{
Name: "x",
Value: "y",
},
},
AuthConfig: &promauth.Config{}, AuthConfig: &promauth.Config{},
ProxyAuthConfig: &promauth.Config{}, ProxyAuthConfig: &promauth.Config{},
ProxyURL: proxy.MustNewURL("http://foo.bar"), ProxyURL: proxy.MustNewURL("http://foo.bar"),
@ -1046,16 +946,10 @@ scrape_configs:
ScrapeInterval: 8 * time.Second, ScrapeInterval: 8 * time.Second,
ScrapeTimeout: 8 * time.Second, ScrapeTimeout: 8 * time.Second,
HonorTimestamps: true, HonorTimestamps: true,
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "instance": "1.2.3.4:80",
Name: "instance", "job": "qwer",
Value: "1.2.3.4:80", }),
},
{
Name: "job",
Value: "qwer",
},
},
AuthConfig: &promauth.Config{ AuthConfig: &promauth.Config{
TLSServerName: "foobar", TLSServerName: "foobar",
TLSInsecureSkipVerify: true, TLSInsecureSkipVerify: true,
@ -1068,16 +962,10 @@ scrape_configs:
ScrapeInterval: 8 * time.Second, ScrapeInterval: 8 * time.Second,
ScrapeTimeout: 8 * time.Second, ScrapeTimeout: 8 * time.Second,
HonorTimestamps: true, HonorTimestamps: true,
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "instance": "foobar:80",
Name: "instance", "job": "asdf",
Value: "foobar:80", }),
},
{
Name: "job",
Value: "asdf",
},
},
AuthConfig: &promauth.Config{}, AuthConfig: &promauth.Config{},
ProxyAuthConfig: &promauth.Config{}, ProxyAuthConfig: &promauth.Config{},
jobNameOriginal: "asdf", jobNameOriginal: "asdf",
@ -1124,24 +1012,12 @@ scrape_configs:
ScrapeInterval: defaultScrapeInterval, ScrapeInterval: defaultScrapeInterval,
ScrapeTimeout: defaultScrapeTimeout, ScrapeTimeout: defaultScrapeTimeout,
HonorTimestamps: true, HonorTimestamps: true,
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "hash": "82",
Name: "hash", "instance": "foo.bar:1234",
Value: "82", "prefix:url": "http://foo.bar:1234/metrics",
}, "url": "http://foo.bar:1234/metrics",
{ }),
Name: "instance",
Value: "foo.bar:1234",
},
{
Name: "prefix:url",
Value: "http://foo.bar:1234/metrics",
},
{
Name: "url",
Value: "http://foo.bar:1234/metrics",
},
},
AuthConfig: &promauth.Config{}, AuthConfig: &promauth.Config{},
ProxyAuthConfig: &promauth.Config{}, ProxyAuthConfig: &promauth.Config{},
jobNameOriginal: "foo", jobNameOriginal: "foo",
@ -1180,16 +1056,10 @@ scrape_configs:
ScrapeInterval: defaultScrapeInterval, ScrapeInterval: defaultScrapeInterval,
ScrapeTimeout: defaultScrapeTimeout, ScrapeTimeout: defaultScrapeTimeout,
HonorTimestamps: true, HonorTimestamps: true,
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "instance": "fake.addr",
Name: "instance", "job": "https",
Value: "fake.addr", }),
},
{
Name: "job",
Value: "https",
},
},
AuthConfig: &promauth.Config{}, AuthConfig: &promauth.Config{},
ProxyAuthConfig: &promauth.Config{}, ProxyAuthConfig: &promauth.Config{},
jobNameOriginal: "foo", jobNameOriginal: "foo",
@ -1221,16 +1091,10 @@ scrape_configs:
ScrapeInterval: defaultScrapeInterval, ScrapeInterval: defaultScrapeInterval,
ScrapeTimeout: defaultScrapeTimeout, ScrapeTimeout: defaultScrapeTimeout,
HonorTimestamps: true, HonorTimestamps: true,
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "instance": "foo.bar:1234",
Name: "instance", "job": "3",
Value: "foo.bar:1234", }),
},
{
Name: "job",
Value: "3",
},
},
AuthConfig: &promauth.Config{}, AuthConfig: &promauth.Config{},
ProxyAuthConfig: &promauth.Config{}, ProxyAuthConfig: &promauth.Config{},
jobNameOriginal: "foo", jobNameOriginal: "foo",
@ -1251,16 +1115,10 @@ scrape_configs:
ScrapeInterval: defaultScrapeInterval, ScrapeInterval: defaultScrapeInterval,
ScrapeTimeout: defaultScrapeTimeout, ScrapeTimeout: defaultScrapeTimeout,
HonorTimestamps: true, HonorTimestamps: true,
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "instance": "foo.bar:1234",
Name: "instance", "job": "foo",
Value: "foo.bar:1234", }),
},
{
Name: "job",
Value: "foo",
},
},
AuthConfig: &promauth.Config{}, AuthConfig: &promauth.Config{},
ProxyAuthConfig: &promauth.Config{}, ProxyAuthConfig: &promauth.Config{},
jobNameOriginal: "foo", jobNameOriginal: "foo",
@ -1277,16 +1135,10 @@ scrape_configs:
ScrapeInterval: defaultScrapeInterval, ScrapeInterval: defaultScrapeInterval,
ScrapeTimeout: defaultScrapeTimeout, ScrapeTimeout: defaultScrapeTimeout,
HonorTimestamps: true, HonorTimestamps: true,
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "instance": "foo.bar:1234",
Name: "instance", "job": "foo",
Value: "foo.bar:1234", }),
},
{
Name: "job",
Value: "foo",
},
},
AuthConfig: &promauth.Config{}, AuthConfig: &promauth.Config{},
ProxyAuthConfig: &promauth.Config{}, ProxyAuthConfig: &promauth.Config{},
jobNameOriginal: "foo", jobNameOriginal: "foo",
@ -1303,16 +1155,10 @@ scrape_configs:
ScrapeInterval: defaultScrapeInterval, ScrapeInterval: defaultScrapeInterval,
ScrapeTimeout: defaultScrapeTimeout, ScrapeTimeout: defaultScrapeTimeout,
HonorTimestamps: true, HonorTimestamps: true,
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "instance": "foo.bar:1234",
Name: "instance", "job": "foo",
Value: "foo.bar:1234", }),
},
{
Name: "job",
Value: "foo",
},
},
AuthConfig: &promauth.Config{}, AuthConfig: &promauth.Config{},
ProxyAuthConfig: &promauth.Config{}, ProxyAuthConfig: &promauth.Config{},
jobNameOriginal: "foo", jobNameOriginal: "foo",
@ -1343,42 +1189,18 @@ scrape_configs:
ScrapeInterval: defaultScrapeInterval, ScrapeInterval: defaultScrapeInterval,
ScrapeTimeout: defaultScrapeTimeout, ScrapeTimeout: defaultScrapeTimeout,
HonorTimestamps: true, HonorTimestamps: true,
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "foo": "bar",
Name: "foo", "instance": "pp:80",
Value: "bar", "job": "yyy",
}, }),
{ ExternalLabels: promutils.NewLabelsFromMap(map[string]string{
Name: "instance", "__address__": "aaasdf",
Value: "pp:80", "__param_a": "jlfd",
}, "foo": "xx",
{ "job": "foobar",
Name: "job", "q": "qwe",
Value: "yyy", }),
},
},
ExternalLabels: []prompbmarshal.Label{
{
Name: "__address__",
Value: "aaasdf",
},
{
Name: "__param_a",
Value: "jlfd",
},
{
Name: "foo",
Value: "xx",
},
{
Name: "job",
Value: "foobar",
},
{
Name: "q",
Value: "qwe",
},
},
AuthConfig: &promauth.Config{}, AuthConfig: &promauth.Config{},
ProxyAuthConfig: &promauth.Config{}, ProxyAuthConfig: &promauth.Config{},
jobNameOriginal: "aaa", jobNameOriginal: "aaa",
@ -1434,16 +1256,10 @@ scrape_configs:
ScrapeInterval: defaultScrapeInterval, ScrapeInterval: defaultScrapeInterval,
ScrapeTimeout: defaultScrapeTimeout, ScrapeTimeout: defaultScrapeTimeout,
HonorTimestamps: true, HonorTimestamps: true,
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "instance": "192.168.1.2",
Name: "instance", "job": "snmp",
Value: "192.168.1.2", }),
},
{
Name: "job",
Value: "snmp",
},
},
AuthConfig: ac, AuthConfig: ac,
ProxyAuthConfig: proxyAC, ProxyAuthConfig: proxyAC,
SampleLimit: 100, SampleLimit: 100,
@ -1470,16 +1286,10 @@ scrape_configs:
ScrapeInterval: defaultScrapeInterval, ScrapeInterval: defaultScrapeInterval,
ScrapeTimeout: defaultScrapeTimeout, ScrapeTimeout: defaultScrapeTimeout,
HonorTimestamps: true, HonorTimestamps: true,
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "instance": "foo.bar:1234",
Name: "instance", "job": "path wo slash",
Value: "foo.bar:1234", }),
},
{
Name: "job",
Value: "path wo slash",
},
},
jobNameOriginal: "path wo slash", jobNameOriginal: "path wo slash",
AuthConfig: &promauth.Config{}, AuthConfig: &promauth.Config{},
ProxyAuthConfig: &promauth.Config{}, ProxyAuthConfig: &promauth.Config{},
@ -1505,16 +1315,10 @@ scrape_configs:
ScrapeOffset: time.Hour * 24 * 2, ScrapeOffset: time.Hour * 24 * 2,
HonorTimestamps: true, HonorTimestamps: true,
NoStaleMarkers: true, NoStaleMarkers: true,
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "instance": "foo.bar:1234",
Name: "instance", "job": "foo",
Value: "foo.bar:1234", }),
},
{
Name: "job",
Value: "foo",
},
},
AuthConfig: &promauth.Config{}, AuthConfig: &promauth.Config{},
ProxyAuthConfig: &promauth.Config{}, ProxyAuthConfig: &promauth.Config{},
jobNameOriginal: "foo", jobNameOriginal: "foo",

View file

@ -1,31 +1,42 @@
package promscrape package promscrape
import ( import (
"fmt"
"testing" "testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
func BenchmarkInternLabelStrings(b *testing.B) { func BenchmarkGetScrapeWork(b *testing.B) {
swc := &scrapeWorkConfig{
jobName: "job-1",
scheme: "http",
metricsPath: "/metrics",
scrapeIntervalString: "30s",
scrapeTimeoutString: "10s",
}
target := "host1.com:1234"
extraLabels := promutils.NewLabelsFromMap(map[string]string{
"env": "prod",
"datacenter": "dc-foo",
})
metaLabels := promutils.NewLabelsFromMap(map[string]string{
"__meta_foo": "bar",
"__meta_kubernetes_namespace": "default",
"__address__": "foobar.com",
"__meta_sfdfdf_dsfds_fdfdfds_fdfdfd": "true",
})
b.ReportAllocs() b.ReportAllocs()
b.SetBytes(1) b.SetBytes(1)
b.RunParallel(func(pb *testing.PB) { b.RunParallel(func(pb *testing.PB) {
labels := []prompbmarshal.Label{
{
Name: "job",
Value: "node-exporter",
},
{
Name: "instance",
Value: "foo.bar.baz:1234",
},
{
Name: "__meta_kubernetes_namespace",
Value: "default",
},
}
for pb.Next() { for pb.Next() {
internLabelStrings(labels) sw, err := swc.getScrapeWork(target, extraLabels, metaLabels)
if err != nil {
panic(fmt.Errorf("BUG: getScrapeWork returned non-nil error: %w", err))
}
if sw == nil {
panic(fmt.Errorf("BUG: getScrapeWork returned nil ScrapeWork"))
}
} }
}) })
} }

View file

@ -8,6 +8,7 @@ import (
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/proxy" "github.com/VictoriaMetrics/VictoriaMetrics/lib/proxy"
) )
@ -43,7 +44,7 @@ type SDConfig struct {
} }
// GetLabels returns Consul labels according to sdc. // GetLabels returns Consul labels according to sdc.
func (sdc *SDConfig) GetLabels(baseDir string) ([]map[string]string, error) { func (sdc *SDConfig) GetLabels(baseDir string) ([]*promutils.Labels, error) {
ac, err := getAPIConfig(sdc, baseDir) ac, err := getAPIConfig(sdc, baseDir)
if err != nil { if err != nil {
return nil, fmt.Errorf("cannot get API config: %w", err) return nil, fmt.Errorf("cannot get API config: %w", err)
@ -60,8 +61,8 @@ func (sdc *SDConfig) MustStop() {
configMap.Delete(sdc) configMap.Delete(sdc)
} }
func appendMachineLabels(vms []virtualMachine, port int, sdc *SDConfig) []map[string]string { func appendMachineLabels(vms []virtualMachine, port int, sdc *SDConfig) []*promutils.Labels {
ms := make([]map[string]string, 0, len(vms)) ms := make([]*promutils.Labels, 0, len(vms))
for i := range vms { for i := range vms {
vm := &vms[i] vm := &vms[i]
for _, ips := range vm.ipAddresses { for _, ips := range vm.ipAddresses {
@ -69,36 +70,35 @@ func appendMachineLabels(vms []virtualMachine, port int, sdc *SDConfig) []map[st
continue continue
} }
addr := discoveryutils.JoinHostPort(ips.privateIP, port) addr := discoveryutils.JoinHostPort(ips.privateIP, port)
m := map[string]string{ m := promutils.NewLabels(16)
"__address__": addr, m.Add("__address__", addr)
"__meta_azure_subscription_id": sdc.SubscriptionID, m.Add("__meta_azure_subscription_id", sdc.SubscriptionID)
"__meta_azure_machine_id": vm.ID, m.Add("__meta_azure_machine_id", vm.ID)
"__meta_azure_machine_name": vm.Name, m.Add("__meta_azure_machine_name", vm.Name)
"__meta_azure_machine_location": vm.Location, m.Add("__meta_azure_machine_location", vm.Location)
"__meta_azure_machine_private_ip": ips.privateIP, m.Add("__meta_azure_machine_private_ip", ips.privateIP)
}
if sdc.TenantID != "" { if sdc.TenantID != "" {
m["__meta_azure_tenant_id"] = sdc.TenantID m.Add("__meta_azure_tenant_id", sdc.TenantID)
} }
// /subscriptions/SUBSCRIPTION_ID/resourceGroups/RESOURCE_GROUP/providers/PROVIDER/TYPE/NAME // /subscriptions/SUBSCRIPTION_ID/resourceGroups/RESOURCE_GROUP/providers/PROVIDER/TYPE/NAME
idPath := strings.Split(vm.ID, "/") idPath := strings.Split(vm.ID, "/")
if len(idPath) > 4 { if len(idPath) > 4 {
m["__meta_azure_machine_resource_group"] = idPath[4] m.Add("__meta_azure_machine_resource_group", idPath[4])
} }
if vm.Properties.StorageProfile.OsDisk.OsType != "" { if vm.Properties.StorageProfile.OsDisk.OsType != "" {
m["__meta_azure_machine_os_type"] = vm.Properties.StorageProfile.OsDisk.OsType m.Add("__meta_azure_machine_os_type", vm.Properties.StorageProfile.OsDisk.OsType)
} }
if vm.Properties.OsProfile.ComputerName != "" { if vm.Properties.OsProfile.ComputerName != "" {
m["__meta_azure_machine_computer_name"] = vm.Properties.OsProfile.ComputerName m.Add("__meta_azure_machine_computer_name", vm.Properties.OsProfile.ComputerName)
} }
if ips.publicIP != "" { if ips.publicIP != "" {
m["__meta_azure_machine_public_ip"] = ips.publicIP m.Add("__meta_azure_machine_public_ip", ips.publicIP)
} }
if vm.scaleSet != "" { if vm.scaleSet != "" {
m["__meta_azure_machine_scale_set"] = vm.scaleSet m.Add("__meta_azure_machine_scale_set", vm.scaleSet)
} }
for k, v := range vm.Tags { for k, v := range vm.Tags {
m[discoveryutils.SanitizeLabelName("__meta_azure_machine_tag_"+k)] = v m.Add(discoveryutils.SanitizeLabelName("__meta_azure_machine_tag_"+k), v)
} }
ms = append(ms, m) ms = append(ms, m)
} }

View file

@ -1,24 +1,17 @@
package azure package azure
import ( import (
"reflect"
"testing" "testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
func TestAppendMachineLabels(t *testing.T) { func TestAppendMachineLabels(t *testing.T) {
f := func(name string, vms []virtualMachine, expectedLabels [][]prompbmarshal.Label) { f := func(name string, vms []virtualMachine, expectedLabels []*promutils.Labels) {
t.Run(name, func(t *testing.T) { t.Run(name, func(t *testing.T) {
labelss := appendMachineLabels(vms, 80, &SDConfig{SubscriptionID: "some-id"}) labelss := appendMachineLabels(vms, 80, &SDConfig{SubscriptionID: "some-id"})
var sortedLabelss [][]prompbmarshal.Label discoveryutils.TestEqualLabelss(t, labelss, expectedLabels)
for _, labels := range labelss {
sortedLabelss = append(sortedLabelss, discoveryutils.GetSortedLabels(labels))
}
if !reflect.DeepEqual(sortedLabelss, expectedLabels) {
t.Fatalf("unexpected labels:\ngot\n%v\nwant\n%v", sortedLabelss, expectedLabels)
}
}) })
} }
f("single vm", []virtualMachine{ f("single vm", []virtualMachine{
@ -33,8 +26,8 @@ func TestAppendMachineLabels(t *testing.T) {
{privateIP: "10.10.10.1"}, {privateIP: "10.10.10.1"},
}, },
}, },
}, [][]prompbmarshal.Label{ }, []*promutils.Labels{
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "10.10.10.1:80", "__address__": "10.10.10.1:80",
"__meta_azure_machine_id": "id-2", "__meta_azure_machine_id": "id-2",
"__meta_azure_subscription_id": "some-id", "__meta_azure_subscription_id": "some-id",

View file

@ -4,6 +4,7 @@ import (
"fmt" "fmt"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/proxy" "github.com/VictoriaMetrics/VictoriaMetrics/lib/proxy"
) )
@ -38,7 +39,7 @@ type SDConfig struct {
} }
// GetLabels returns Consul labels according to sdc. // GetLabels returns Consul labels according to sdc.
func (sdc *SDConfig) GetLabels(baseDir string) ([]map[string]string, error) { func (sdc *SDConfig) GetLabels(baseDir string) ([]*promutils.Labels, error) {
cfg, err := getAPIConfig(sdc, baseDir) cfg, err := getAPIConfig(sdc, baseDir)
if err != nil { if err != nil {
return nil, fmt.Errorf("cannot get API config: %w", err) return nil, fmt.Errorf("cannot get API config: %w", err)

View file

@ -7,12 +7,13 @@ import (
"strings" "strings"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
// getServiceNodesLabels returns labels for Consul service nodes with given cfg. // getServiceNodesLabels returns labels for Consul service nodes with given cfg.
func getServiceNodesLabels(cfg *apiConfig) []map[string]string { func getServiceNodesLabels(cfg *apiConfig) []*promutils.Labels {
sns := cfg.consulWatcher.getServiceNodesSnapshot() sns := cfg.consulWatcher.getServiceNodesSnapshot()
var ms []map[string]string var ms []*promutils.Labels
for svc, sn := range sns { for svc, sn := range sns {
for i := range sn { for i := range sn {
ms = sn[i].appendTargetLabels(ms, svc, cfg.tagSeparator) ms = sn[i].appendTargetLabels(ms, svc, cfg.tagSeparator)
@ -71,38 +72,37 @@ func parseServiceNodes(data []byte) ([]ServiceNode, error) {
return sns, nil return sns, nil
} }
func (sn *ServiceNode) appendTargetLabels(ms []map[string]string, serviceName, tagSeparator string) []map[string]string { func (sn *ServiceNode) appendTargetLabels(ms []*promutils.Labels, serviceName, tagSeparator string) []*promutils.Labels {
var addr string var addr string
if sn.Service.Address != "" { if sn.Service.Address != "" {
addr = discoveryutils.JoinHostPort(sn.Service.Address, sn.Service.Port) addr = discoveryutils.JoinHostPort(sn.Service.Address, sn.Service.Port)
} else { } else {
addr = discoveryutils.JoinHostPort(sn.Node.Address, sn.Service.Port) addr = discoveryutils.JoinHostPort(sn.Node.Address, sn.Service.Port)
} }
m := map[string]string{ m := promutils.NewLabels(16)
"__address__": addr, m.Add("__address__", addr)
"__meta_consul_address": sn.Node.Address, m.Add("__meta_consul_address", sn.Node.Address)
"__meta_consul_dc": sn.Node.Datacenter, m.Add("__meta_consul_dc", sn.Node.Datacenter)
"__meta_consul_health": aggregatedStatus(sn.Checks), m.Add("__meta_consul_health", aggregatedStatus(sn.Checks))
"__meta_consul_namespace": sn.Service.Namespace, m.Add("__meta_consul_namespace", sn.Service.Namespace)
"__meta_consul_partition": sn.Service.Partition, m.Add("__meta_consul_partition", sn.Service.Partition)
"__meta_consul_node": sn.Node.Node, m.Add("__meta_consul_node", sn.Node.Node)
"__meta_consul_service": serviceName, m.Add("__meta_consul_service", serviceName)
"__meta_consul_service_address": sn.Service.Address, m.Add("__meta_consul_service_address", sn.Service.Address)
"__meta_consul_service_id": sn.Service.ID, m.Add("__meta_consul_service_id", sn.Service.ID)
"__meta_consul_service_port": strconv.Itoa(sn.Service.Port), m.Add("__meta_consul_service_port", strconv.Itoa(sn.Service.Port))
}
// We surround the separated list with the separator as well. This way regular expressions // We surround the separated list with the separator as well. This way regular expressions
// in relabeling rules don't have to consider tag positions. // in relabeling rules don't have to consider tag positions.
m["__meta_consul_tags"] = tagSeparator + strings.Join(sn.Service.Tags, tagSeparator) + tagSeparator m.Add("__meta_consul_tags", tagSeparator+strings.Join(sn.Service.Tags, tagSeparator)+tagSeparator)
for k, v := range sn.Node.Meta { for k, v := range sn.Node.Meta {
m[discoveryutils.SanitizeLabelName("__meta_consul_metadata_"+k)] = v m.Add(discoveryutils.SanitizeLabelName("__meta_consul_metadata_"+k), v)
} }
for k, v := range sn.Service.Meta { for k, v := range sn.Service.Meta {
m[discoveryutils.SanitizeLabelName("__meta_consul_service_metadata_"+k)] = v m.Add(discoveryutils.SanitizeLabelName("__meta_consul_service_metadata_"+k), v)
} }
for k, v := range sn.Node.TaggedAddresses { for k, v := range sn.Node.TaggedAddresses {
m[discoveryutils.SanitizeLabelName("__meta_consul_tagged_address_"+k)] = v m.Add(discoveryutils.SanitizeLabelName("__meta_consul_tagged_address_"+k), v)
} }
ms = append(ms, m) ms = append(ms, m)
return ms return ms

View file

@ -1,11 +1,10 @@
package consul package consul
import ( import (
"reflect"
"testing" "testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
func TestParseServiceNodesFailure(t *testing.T) { func TestParseServiceNodesFailure(t *testing.T) {
@ -108,12 +107,8 @@ func TestParseServiceNodesSuccess(t *testing.T) {
// Check sn.appendTargetLabels() // Check sn.appendTargetLabels()
tagSeparator := "," tagSeparator := ","
labelss := sn.appendTargetLabels(nil, "redis", tagSeparator) labelss := sn.appendTargetLabels(nil, "redis", tagSeparator)
var sortedLabelss [][]prompbmarshal.Label expectedLabelss := []*promutils.Labels{
for _, labels := range labelss { promutils.NewLabelsFromMap(map[string]string{
sortedLabelss = append(sortedLabelss, discoveryutils.GetSortedLabels(labels))
}
expectedLabelss := [][]prompbmarshal.Label{
discoveryutils.GetSortedLabels(map[string]string{
"__address__": "10.1.10.12:8000", "__address__": "10.1.10.12:8000",
"__meta_consul_address": "10.1.10.12", "__meta_consul_address": "10.1.10.12",
"__meta_consul_dc": "dc1", "__meta_consul_dc": "dc1",
@ -132,7 +127,5 @@ func TestParseServiceNodesSuccess(t *testing.T) {
"__meta_consul_tags": ",primary,", "__meta_consul_tags": ",primary,",
}), }),
} }
if !reflect.DeepEqual(sortedLabelss, expectedLabelss) { discoveryutils.TestEqualLabelss(t, labelss, expectedLabelss)
t.Fatalf("unexpected labels:\ngot\n%v\nwant\n%v", sortedLabelss, expectedLabelss)
}
} }

View file

@ -10,6 +10,7 @@ import (
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger" "github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/proxy" "github.com/VictoriaMetrics/VictoriaMetrics/lib/proxy"
) )
@ -30,7 +31,7 @@ type SDConfig struct {
} }
// GetLabels returns Digital Ocean droplet labels according to sdc. // GetLabels returns Digital Ocean droplet labels according to sdc.
func (sdc *SDConfig) GetLabels(baseDir string) ([]map[string]string, error) { func (sdc *SDConfig) GetLabels(baseDir string) ([]*promutils.Labels, error) {
cfg, err := getAPIConfig(sdc, baseDir) cfg, err := getAPIConfig(sdc, baseDir)
if err != nil { if err != nil {
return nil, fmt.Errorf("cannot get API config: %w", err) return nil, fmt.Errorf("cannot get API config: %w", err)
@ -39,7 +40,6 @@ func (sdc *SDConfig) GetLabels(baseDir string) ([]map[string]string, error) {
if err != nil { if err != nil {
return nil, err return nil, err
} }
return addDropletLabels(droplets, cfg.port), nil return addDropletLabels(droplets, cfg.port), nil
} }
@ -115,8 +115,8 @@ func (r *listDropletResponse) nextURLPath() (string, error) {
return u.RequestURI(), nil return u.RequestURI(), nil
} }
func addDropletLabels(droplets []droplet, defaultPort int) []map[string]string { func addDropletLabels(droplets []droplet, defaultPort int) []*promutils.Labels {
var ms []map[string]string var ms []*promutils.Labels
for _, droplet := range droplets { for _, droplet := range droplets {
if len(droplet.Networks.V4) == 0 { if len(droplet.Networks.V4) == 0 {
continue continue
@ -127,27 +127,26 @@ func addDropletLabels(droplets []droplet, defaultPort int) []map[string]string {
publicIPv6 := droplet.getIPByNet("v6", "public") publicIPv6 := droplet.getIPByNet("v6", "public")
addr := discoveryutils.JoinHostPort(publicIPv4, defaultPort) addr := discoveryutils.JoinHostPort(publicIPv4, defaultPort)
m := map[string]string{ m := promutils.NewLabels(16)
"__address__": addr, m.Add("__address__", addr)
"__meta_digitalocean_droplet_id": fmt.Sprintf("%d", droplet.ID), m.Add("__meta_digitalocean_droplet_id", fmt.Sprintf("%d", droplet.ID))
"__meta_digitalocean_droplet_name": droplet.Name, m.Add("__meta_digitalocean_droplet_name", droplet.Name)
"__meta_digitalocean_image": droplet.Image.Slug, m.Add("__meta_digitalocean_image", droplet.Image.Slug)
"__meta_digitalocean_image_name": droplet.Image.Name, m.Add("__meta_digitalocean_image_name", droplet.Image.Name)
"__meta_digitalocean_private_ipv4": privateIPv4, m.Add("__meta_digitalocean_private_ipv4", privateIPv4)
"__meta_digitalocean_public_ipv4": publicIPv4, m.Add("__meta_digitalocean_public_ipv4", publicIPv4)
"__meta_digitalocean_public_ipv6": publicIPv6, m.Add("__meta_digitalocean_public_ipv6", publicIPv6)
"__meta_digitalocean_region": droplet.Region.Slug, m.Add("__meta_digitalocean_region", droplet.Region.Slug)
"__meta_digitalocean_size": droplet.SizeSlug, m.Add("__meta_digitalocean_size", droplet.SizeSlug)
"__meta_digitalocean_status": droplet.Status, m.Add("__meta_digitalocean_status", droplet.Status)
"__meta_digitalocean_vpc": droplet.VpcUUID, m.Add("__meta_digitalocean_vpc", droplet.VpcUUID)
}
if len(droplet.Features) > 0 { if len(droplet.Features) > 0 {
features := fmt.Sprintf(",%s,", strings.Join(droplet.Features, ",")) features := fmt.Sprintf(",%s,", strings.Join(droplet.Features, ","))
m["__meta_digitalocean_features"] = features m.Add("__meta_digitalocean_features", features)
} }
if len(droplet.Tags) > 0 { if len(droplet.Tags) > 0 {
tags := fmt.Sprintf(",%s,", strings.Join(droplet.Tags, ",")) tags := fmt.Sprintf(",%s,", strings.Join(droplet.Tags, ","))
m["__meta_digitalocean_tags"] = tags m.Add("__meta_digitalocean_tags", tags)
} }
ms = append(ms, m) ms = append(ms, m)
} }

View file

@ -1,11 +1,10 @@
package digitalocean package digitalocean
import ( import (
"reflect"
"testing" "testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
func Test_addDropletLabels(t *testing.T) { func Test_addDropletLabels(t *testing.T) {
@ -16,7 +15,7 @@ func Test_addDropletLabels(t *testing.T) {
tests := []struct { tests := []struct {
name string name string
args args args args
want [][]prompbmarshal.Label want []*promutils.Labels
}{ }{
{ {
name: "base labels add test", name: "base labels add test",
@ -62,8 +61,8 @@ func Test_addDropletLabels(t *testing.T) {
}, },
defaultPort: 9100, defaultPort: 9100,
}, },
want: [][]prompbmarshal.Label{ want: []*promutils.Labels{
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "100.100.100.100:9100", "__address__": "100.100.100.100:9100",
"__meta_digitalocean_droplet_id": "15", "__meta_digitalocean_droplet_id": "15",
"__meta_digitalocean_droplet_name": "ubuntu-1", "__meta_digitalocean_droplet_name": "ubuntu-1",
@ -85,14 +84,7 @@ func Test_addDropletLabels(t *testing.T) {
for _, tt := range tests { for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) { t.Run(tt.name, func(t *testing.T) {
got := addDropletLabels(tt.args.droplets, tt.args.defaultPort) got := addDropletLabels(tt.args.droplets, tt.args.defaultPort)
var sortedLabelss [][]prompbmarshal.Label discoveryutils.TestEqualLabelss(t, got, tt.want)
for _, labels := range got {
sortedLabelss = append(sortedLabelss, discoveryutils.GetSortedLabels(labels))
}
if !reflect.DeepEqual(sortedLabelss, tt.want) {
t.Errorf("addTasksLabels() \ngot \n%v\n, \nwant \n%v\n", sortedLabelss, tt.want)
}
}) })
} }
} }

View file

@ -11,6 +11,7 @@ import (
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger" "github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
// SDCheckInterval defines interval for targets refresh. // SDCheckInterval defines interval for targets refresh.
@ -30,7 +31,7 @@ type SDConfig struct {
} }
// GetLabels returns DNS labels according to sdc. // GetLabels returns DNS labels according to sdc.
func (sdc *SDConfig) GetLabels(baseDir string) ([]map[string]string, error) { func (sdc *SDConfig) GetLabels(baseDir string) ([]*promutils.Labels, error) {
if len(sdc.Names) == 0 { if len(sdc.Names) == 0 {
return nil, fmt.Errorf("`names` cannot be empty in `dns_sd_config`") return nil, fmt.Errorf("`names` cannot be empty in `dns_sd_config`")
} }
@ -60,7 +61,7 @@ func (sdc *SDConfig) MustStop() {
// nothing to do // nothing to do
} }
func getMXAddrLabels(ctx context.Context, sdc *SDConfig) []map[string]string { func getMXAddrLabels(ctx context.Context, sdc *SDConfig) []*promutils.Labels {
port := 25 port := 25
if sdc.Port != nil { if sdc.Port != nil {
port = *sdc.Port port = *sdc.Port
@ -81,7 +82,7 @@ func getMXAddrLabels(ctx context.Context, sdc *SDConfig) []map[string]string {
} }
}(name) }(name)
} }
var ms []map[string]string var ms []*promutils.Labels
for range sdc.Names { for range sdc.Names {
r := <-ch r := <-ch
if r.err != nil { if r.err != nil {
@ -99,7 +100,7 @@ func getMXAddrLabels(ctx context.Context, sdc *SDConfig) []map[string]string {
return ms return ms
} }
func getSRVAddrLabels(ctx context.Context, sdc *SDConfig) []map[string]string { func getSRVAddrLabels(ctx context.Context, sdc *SDConfig) []*promutils.Labels {
type result struct { type result struct {
name string name string
as []*net.SRV as []*net.SRV
@ -116,7 +117,7 @@ func getSRVAddrLabels(ctx context.Context, sdc *SDConfig) []map[string]string {
} }
}(name) }(name)
} }
var ms []map[string]string var ms []*promutils.Labels
for range sdc.Names { for range sdc.Names {
r := <-ch r := <-ch
if r.err != nil { if r.err != nil {
@ -134,7 +135,7 @@ func getSRVAddrLabels(ctx context.Context, sdc *SDConfig) []map[string]string {
return ms return ms
} }
func getAAddrLabels(ctx context.Context, sdc *SDConfig, lookupType string) ([]map[string]string, error) { func getAAddrLabels(ctx context.Context, sdc *SDConfig, lookupType string) ([]*promutils.Labels, error) {
if sdc.Port == nil { if sdc.Port == nil {
return nil, fmt.Errorf("missing `port` in `dns_sd_config` for `type: %s`", lookupType) return nil, fmt.Errorf("missing `port` in `dns_sd_config` for `type: %s`", lookupType)
} }
@ -155,7 +156,7 @@ func getAAddrLabels(ctx context.Context, sdc *SDConfig, lookupType string) ([]ma
} }
}(name) }(name)
} }
var ms []map[string]string var ms []*promutils.Labels
for range sdc.Names { for range sdc.Names {
r := <-ch r := <-ch
if r.err != nil { if r.err != nil {
@ -173,24 +174,22 @@ func getAAddrLabels(ctx context.Context, sdc *SDConfig, lookupType string) ([]ma
return ms, nil return ms, nil
} }
func appendMXLabels(ms []map[string]string, name, target string, port int) []map[string]string { func appendMXLabels(ms []*promutils.Labels, name, target string, port int) []*promutils.Labels {
addr := discoveryutils.JoinHostPort(target, port) addr := discoveryutils.JoinHostPort(target, port)
m := map[string]string{ m := promutils.NewLabels(3)
"__address__": addr, m.Add("__address__", addr)
"__meta_dns_name": name, m.Add("__meta_dns_name", name)
"__meta_dns_mx_record_target": target, m.Add("__meta_dns_mx_record_target", target)
}
return append(ms, m) return append(ms, m)
} }
func appendAddrLabels(ms []map[string]string, name, target string, port int) []map[string]string { func appendAddrLabels(ms []*promutils.Labels, name, target string, port int) []*promutils.Labels {
addr := discoveryutils.JoinHostPort(target, port) addr := discoveryutils.JoinHostPort(target, port)
m := map[string]string{ m := promutils.NewLabels(4)
"__address__": addr, m.Add("__address__", addr)
"__meta_dns_name": name, m.Add("__meta_dns_name", name)
"__meta_dns_srv_record_target": target, m.Add("__meta_dns_srv_record_target", target)
"__meta_dns_srv_record_port": strconv.Itoa(port), m.Add("__meta_dns_srv_record_port", strconv.Itoa(port))
}
return append(ms, m) return append(ms, m)
} }

View file

@ -6,6 +6,7 @@ import (
"strconv" "strconv"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
// See https://github.com/moby/moby/blob/314759dc2f4745925d8dec6d15acc7761c6e5c92/docs/api/v1.41.yaml#L4024 // See https://github.com/moby/moby/blob/314759dc2f4745925d8dec6d15acc7761c6e5c92/docs/api/v1.41.yaml#L4024
@ -30,7 +31,7 @@ type container struct {
} }
} }
func getContainersLabels(cfg *apiConfig) ([]map[string]string, error) { func getContainersLabels(cfg *apiConfig) ([]*promutils.Labels, error) {
networkLabels, err := getNetworksLabelsByNetworkID(cfg) networkLabels, err := getNetworksLabelsByNetworkID(cfg)
if err != nil { if err != nil {
return nil, err return nil, err
@ -58,8 +59,8 @@ func parseContainers(data []byte) ([]container, error) {
return containers, nil return containers, nil
} }
func addContainersLabels(containers []container, networkLabels map[string]map[string]string, defaultPort int, hostNetworkingHost string) []map[string]string { func addContainersLabels(containers []container, networkLabels map[string]*promutils.Labels, defaultPort int, hostNetworkingHost string) []*promutils.Labels {
var ms []map[string]string var ms []*promutils.Labels
for i := range containers { for i := range containers {
c := &containers[i] c := &containers[i]
if len(c.Names) == 0 { if len(c.Names) == 0 {
@ -71,16 +72,16 @@ func addContainersLabels(containers []container, networkLabels map[string]map[st
if p.Type != "tcp" { if p.Type != "tcp" {
continue continue
} }
m := map[string]string{ m := promutils.NewLabels(16)
"__address__": discoveryutils.JoinHostPort(n.IPAddress, p.PrivatePort), m.Add("__address__", discoveryutils.JoinHostPort(n.IPAddress, p.PrivatePort))
"__meta_docker_network_ip": n.IPAddress, m.Add("__meta_docker_network_ip", n.IPAddress)
"__meta_docker_port_private": strconv.Itoa(p.PrivatePort), m.Add("__meta_docker_port_private", strconv.Itoa(p.PrivatePort))
}
if p.PublicPort > 0 { if p.PublicPort > 0 {
m["__meta_docker_port_public"] = strconv.Itoa(p.PublicPort) m.Add("__meta_docker_port_public", strconv.Itoa(p.PublicPort))
m["__meta_docker_port_public_ip"] = p.IP m.Add("__meta_docker_port_public_ip", p.IP)
} }
addCommonLabels(m, c, networkLabels[n.NetworkID]) addCommonLabels(m, c, networkLabels[n.NetworkID])
m.RemoveDuplicates()
ms = append(ms, m) ms = append(ms, m)
added = true added = true
} }
@ -90,11 +91,11 @@ func addContainersLabels(containers []container, networkLabels map[string]map[st
if c.HostConfig.NetworkMode != "host" { if c.HostConfig.NetworkMode != "host" {
addr = discoveryutils.JoinHostPort(n.IPAddress, defaultPort) addr = discoveryutils.JoinHostPort(n.IPAddress, defaultPort)
} }
m := map[string]string{ m := promutils.NewLabels(16)
"__address__": addr, m.Add("__address__", addr)
"__meta_docker_network_ip": n.IPAddress, m.Add("__meta_docker_network_ip", n.IPAddress)
}
addCommonLabels(m, c, networkLabels[n.NetworkID]) addCommonLabels(m, c, networkLabels[n.NetworkID])
m.RemoveDuplicates()
ms = append(ms, m) ms = append(ms, m)
} }
} }
@ -102,14 +103,12 @@ func addContainersLabels(containers []container, networkLabels map[string]map[st
return ms return ms
} }
func addCommonLabels(m map[string]string, c *container, networkLabels map[string]string) { func addCommonLabels(m *promutils.Labels, c *container, networkLabels *promutils.Labels) {
m["__meta_docker_container_id"] = c.ID m.Add("__meta_docker_container_id", c.ID)
m["__meta_docker_container_name"] = c.Names[0] m.Add("__meta_docker_container_name", c.Names[0])
m["__meta_docker_container_network_mode"] = c.HostConfig.NetworkMode m.Add("__meta_docker_container_network_mode", c.HostConfig.NetworkMode)
for k, v := range c.Labels { for k, v := range c.Labels {
m[discoveryutils.SanitizeLabelName("__meta_docker_container_label_"+k)] = v m.Add(discoveryutils.SanitizeLabelName("__meta_docker_container_label_"+k), v)
}
for k, v := range networkLabels {
m[k] = v
} }
m.AddFrom(networkLabels)
} }

View file

@ -3,6 +3,9 @@ package docker
import ( import (
"reflect" "reflect"
"testing" "testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
func Test_parseContainers(t *testing.T) { func Test_parseContainers(t *testing.T) {
@ -314,7 +317,7 @@ func Test_addContainerLabels(t *testing.T) {
tests := []struct { tests := []struct {
name string name string
c container c container
want []map[string]string want []*promutils.Labels
wantErr bool wantErr bool
}{ }{
{ {
@ -352,8 +355,8 @@ func Test_addContainerLabels(t *testing.T) {
}, },
}, },
}, },
want: []map[string]string{ want: []*promutils.Labels{
{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "172.17.0.2:8012", "__address__": "172.17.0.2:8012",
"__meta_docker_container_id": "90bc3b31aa13da5c0b11af2e228d54b38428a84e25d4e249ae9e9c95e51a0700", "__meta_docker_container_id": "90bc3b31aa13da5c0b11af2e228d54b38428a84e25d4e249ae9e9c95e51a0700",
"__meta_docker_container_label_com_docker_compose_config_hash": "c9f0bd5bb31921f94cff367d819a30a0cc08d4399080897a6c5cd74b983156ec", "__meta_docker_container_label_com_docker_compose_config_hash": "c9f0bd5bb31921f94cff367d819a30a0cc08d4399080897a6c5cd74b983156ec",
@ -370,7 +373,7 @@ func Test_addContainerLabels(t *testing.T) {
"__meta_docker_network_ip": "172.17.0.2", "__meta_docker_network_ip": "172.17.0.2",
"__meta_docker_network_name": "bridge", "__meta_docker_network_name": "bridge",
"__meta_docker_network_scope": "local", "__meta_docker_network_scope": "local",
}, }),
}, },
}, },
{ {
@ -408,8 +411,8 @@ func Test_addContainerLabels(t *testing.T) {
}, },
}, },
}, },
want: []map[string]string{ want: []*promutils.Labels{
{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "foobar", "__address__": "foobar",
"__meta_docker_container_id": "90bc3b31aa13da5c0b11af2e228d54b38428a84e25d4e249ae9e9c95e51a0700", "__meta_docker_container_id": "90bc3b31aa13da5c0b11af2e228d54b38428a84e25d4e249ae9e9c95e51a0700",
"__meta_docker_container_label_com_docker_compose_config_hash": "c9f0bd5bb31921f94cff367d819a30a0cc08d4399080897a6c5cd74b983156ec", "__meta_docker_container_label_com_docker_compose_config_hash": "c9f0bd5bb31921f94cff367d819a30a0cc08d4399080897a6c5cd74b983156ec",
@ -426,7 +429,7 @@ func Test_addContainerLabels(t *testing.T) {
"__meta_docker_network_ip": "172.17.0.2", "__meta_docker_network_ip": "172.17.0.2",
"__meta_docker_network_name": "bridge", "__meta_docker_network_name": "bridge",
"__meta_docker_network_scope": "local", "__meta_docker_network_scope": "local",
}, }),
}, },
}, },
{ {
@ -475,8 +478,8 @@ func Test_addContainerLabels(t *testing.T) {
}, },
}, },
}, },
want: []map[string]string{ want: []*promutils.Labels{
{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "172.17.0.2:8080", "__address__": "172.17.0.2:8080",
"__meta_docker_container_id": "90bc3b31aa13da5c0b11af2e228d54b38428a84e25d4e249ae9e9c95e51a0700", "__meta_docker_container_id": "90bc3b31aa13da5c0b11af2e228d54b38428a84e25d4e249ae9e9c95e51a0700",
"__meta_docker_container_label_com_docker_compose_config_hash": "c9f0bd5bb31921f94cff367d819a30a0cc08d4399080897a6c5cd74b983156ec", "__meta_docker_container_label_com_docker_compose_config_hash": "c9f0bd5bb31921f94cff367d819a30a0cc08d4399080897a6c5cd74b983156ec",
@ -496,21 +499,19 @@ func Test_addContainerLabels(t *testing.T) {
"__meta_docker_port_private": "8080", "__meta_docker_port_private": "8080",
"__meta_docker_port_public": "18081", "__meta_docker_port_public": "18081",
"__meta_docker_port_public_ip": "0.0.0.0", "__meta_docker_port_public_ip": "0.0.0.0",
}, }),
}, },
}, },
} }
for _, tt := range tests { for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) { t.Run(tt.name, func(t *testing.T) {
labelsMap := addContainersLabels([]container{tt.c}, networkLabels, 8012, "foobar") labelss := addContainersLabels([]container{tt.c}, networkLabels, 8012, "foobar")
if (err != nil) != tt.wantErr { if (err != nil) != tt.wantErr {
t.Errorf("addContainersLabels() error = %v, wantErr %v", err, tt.wantErr) t.Errorf("addContainersLabels() error = %v, wantErr %v", err, tt.wantErr)
return return
} }
if !reflect.DeepEqual(labelsMap, tt.want) { discoveryutils.TestEqualLabelss(t, labelss, tt.want)
t.Errorf("addContainersLabels() \ngot %v, \nwant %v", labelsMap, tt.want)
}
}) })
} }
} }

View file

@ -6,6 +6,7 @@ import (
"time" "time"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/proxy" "github.com/VictoriaMetrics/VictoriaMetrics/lib/proxy"
) )
@ -36,7 +37,7 @@ type Filter struct {
} }
// GetLabels returns docker labels according to sdc. // GetLabels returns docker labels according to sdc.
func (sdc *SDConfig) GetLabels(baseDir string) ([]map[string]string, error) { func (sdc *SDConfig) GetLabels(baseDir string) ([]*promutils.Labels, error) {
cfg, err := getAPIConfig(sdc, baseDir) cfg, err := getAPIConfig(sdc, baseDir)
if err != nil { if err != nil {
return nil, fmt.Errorf("cannot get API config: %w", err) return nil, fmt.Errorf("cannot get API config: %w", err)

View file

@ -6,6 +6,7 @@ import (
"strconv" "strconv"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
// See https://docs.docker.com/engine/api/v1.40/#tag/Network // See https://docs.docker.com/engine/api/v1.40/#tag/Network
@ -18,7 +19,7 @@ type network struct {
Labels map[string]string Labels map[string]string
} }
func getNetworksLabelsByNetworkID(cfg *apiConfig) (map[string]map[string]string, error) { func getNetworksLabelsByNetworkID(cfg *apiConfig) (map[string]*promutils.Labels, error) {
networks, err := getNetworks(cfg) networks, err := getNetworks(cfg)
if err != nil { if err != nil {
return nil, err return nil, err
@ -42,18 +43,17 @@ func parseNetworks(data []byte) ([]network, error) {
return networks, nil return networks, nil
} }
func getNetworkLabelsByNetworkID(networks []network) map[string]map[string]string { func getNetworkLabelsByNetworkID(networks []network) map[string]*promutils.Labels {
ms := make(map[string]map[string]string) ms := make(map[string]*promutils.Labels)
for _, network := range networks { for _, network := range networks {
m := map[string]string{ m := promutils.NewLabels(8)
"__meta_docker_network_id": network.ID, m.Add("__meta_docker_network_id", network.ID)
"__meta_docker_network_name": network.Name, m.Add("__meta_docker_network_name", network.Name)
"__meta_docker_network_internal": strconv.FormatBool(network.Internal), m.Add("__meta_docker_network_internal", strconv.FormatBool(network.Internal))
"__meta_docker_network_ingress": strconv.FormatBool(network.Ingress), m.Add("__meta_docker_network_ingress", strconv.FormatBool(network.Ingress))
"__meta_docker_network_scope": network.Scope, m.Add("__meta_docker_network_scope", network.Scope)
}
for k, v := range network.Labels { for k, v := range network.Labels {
m[discoveryutils.SanitizeLabelName("__meta_docker_network_label_"+k)] = v m.Add(discoveryutils.SanitizeLabelName("__meta_docker_network_label_"+k), v)
} }
ms[network.ID] = m ms[network.ID] = m
} }

View file

@ -5,8 +5,8 @@ import (
"sort" "sort"
"testing" "testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
func Test_addNetworkLabels(t *testing.T) { func Test_addNetworkLabels(t *testing.T) {
@ -16,7 +16,7 @@ func Test_addNetworkLabels(t *testing.T) {
tests := []struct { tests := []struct {
name string name string
args args args args
want [][]prompbmarshal.Label want []*promutils.Labels
}{ }{
{ {
name: "ingress network", name: "ingress network",
@ -33,8 +33,8 @@ func Test_addNetworkLabels(t *testing.T) {
}, },
}, },
}, },
want: [][]prompbmarshal.Label{ want: []*promutils.Labels{
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__meta_docker_network_id": "qs0hog6ldlei9ct11pr3c77v1", "__meta_docker_network_id": "qs0hog6ldlei9ct11pr3c77v1",
"__meta_docker_network_ingress": "true", "__meta_docker_network_ingress": "true",
"__meta_docker_network_internal": "false", "__meta_docker_network_internal": "false",
@ -52,14 +52,11 @@ func Test_addNetworkLabels(t *testing.T) {
networkIDs = append(networkIDs, networkID) networkIDs = append(networkIDs, networkID)
} }
sort.Strings(networkIDs) sort.Strings(networkIDs)
var sortedLabelss [][]prompbmarshal.Label var labelss []*promutils.Labels
for _, networkID := range networkIDs { for _, networkID := range networkIDs {
labels := got[networkID] labelss = append(labelss, got[networkID])
sortedLabelss = append(sortedLabelss, discoveryutils.GetSortedLabels(labels))
}
if !reflect.DeepEqual(sortedLabelss, tt.want) {
t.Errorf("addNetworkLabels() \ngot %v, \nwant %v", sortedLabelss, tt.want)
} }
discoveryutils.TestEqualLabelss(t, labelss, tt.want)
}) })
} }
} }

View file

@ -6,6 +6,7 @@ import (
"time" "time"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/proxy" "github.com/VictoriaMetrics/VictoriaMetrics/lib/proxy"
) )
@ -36,7 +37,7 @@ type Filter struct {
} }
// GetLabels returns dockerswarm labels according to sdc. // GetLabels returns dockerswarm labels according to sdc.
func (sdc *SDConfig) GetLabels(baseDir string) ([]map[string]string, error) { func (sdc *SDConfig) GetLabels(baseDir string) ([]*promutils.Labels, error) {
cfg, err := getAPIConfig(sdc, baseDir) cfg, err := getAPIConfig(sdc, baseDir)
if err != nil { if err != nil {
return nil, fmt.Errorf("cannot get API config: %w", err) return nil, fmt.Errorf("cannot get API config: %w", err)

View file

@ -6,6 +6,7 @@ import (
"strconv" "strconv"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
// See https://docs.docker.com/engine/api/v1.40/#tag/Network // See https://docs.docker.com/engine/api/v1.40/#tag/Network
@ -18,7 +19,7 @@ type network struct {
Labels map[string]string Labels map[string]string
} }
func getNetworksLabelsByNetworkID(cfg *apiConfig) (map[string]map[string]string, error) { func getNetworksLabelsByNetworkID(cfg *apiConfig) (map[string]*promutils.Labels, error) {
networks, err := getNetworks(cfg) networks, err := getNetworks(cfg)
if err != nil { if err != nil {
return nil, err return nil, err
@ -42,18 +43,17 @@ func parseNetworks(data []byte) ([]network, error) {
return networks, nil return networks, nil
} }
func getNetworkLabelsByNetworkID(networks []network) map[string]map[string]string { func getNetworkLabelsByNetworkID(networks []network) map[string]*promutils.Labels {
ms := make(map[string]map[string]string) ms := make(map[string]*promutils.Labels)
for _, network := range networks { for _, network := range networks {
m := map[string]string{ m := promutils.NewLabels(8)
"__meta_dockerswarm_network_id": network.ID, m.Add("__meta_dockerswarm_network_id", network.ID)
"__meta_dockerswarm_network_name": network.Name, m.Add("__meta_dockerswarm_network_name", network.Name)
"__meta_dockerswarm_network_internal": strconv.FormatBool(network.Internal), m.Add("__meta_dockerswarm_network_internal", strconv.FormatBool(network.Internal))
"__meta_dockerswarm_network_ingress": strconv.FormatBool(network.Ingress), m.Add("__meta_dockerswarm_network_ingress", strconv.FormatBool(network.Ingress))
"__meta_dockerswarm_network_scope": network.Scope, m.Add("__meta_dockerswarm_network_scope", network.Scope)
}
for k, v := range network.Labels { for k, v := range network.Labels {
m[discoveryutils.SanitizeLabelName("__meta_dockerswarm_network_label_"+k)] = v m.Add(discoveryutils.SanitizeLabelName("__meta_dockerswarm_network_label_"+k), v)
} }
ms[network.ID] = m ms[network.ID] = m
} }

View file

@ -5,8 +5,8 @@ import (
"sort" "sort"
"testing" "testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
func Test_addNetworkLabels(t *testing.T) { func Test_addNetworkLabels(t *testing.T) {
@ -16,7 +16,7 @@ func Test_addNetworkLabels(t *testing.T) {
tests := []struct { tests := []struct {
name string name string
args args args args
want [][]prompbmarshal.Label want []*promutils.Labels
}{ }{
{ {
name: "ingress network", name: "ingress network",
@ -33,8 +33,8 @@ func Test_addNetworkLabels(t *testing.T) {
}, },
}, },
}, },
want: [][]prompbmarshal.Label{ want: []*promutils.Labels{
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__meta_dockerswarm_network_id": "qs0hog6ldlei9ct11pr3c77v1", "__meta_dockerswarm_network_id": "qs0hog6ldlei9ct11pr3c77v1",
"__meta_dockerswarm_network_ingress": "true", "__meta_dockerswarm_network_ingress": "true",
"__meta_dockerswarm_network_internal": "false", "__meta_dockerswarm_network_internal": "false",
@ -52,14 +52,11 @@ func Test_addNetworkLabels(t *testing.T) {
networkIDs = append(networkIDs, networkID) networkIDs = append(networkIDs, networkID)
} }
sort.Strings(networkIDs) sort.Strings(networkIDs)
var sortedLabelss [][]prompbmarshal.Label var labelss []*promutils.Labels
for _, networkID := range networkIDs { for _, networkID := range networkIDs {
labels := got[networkID] labelss = append(labelss, got[networkID])
sortedLabelss = append(sortedLabelss, discoveryutils.GetSortedLabels(labels))
}
if !reflect.DeepEqual(sortedLabelss, tt.want) {
t.Errorf("addNetworkLabels() \ngot %v, \nwant %v", sortedLabelss, tt.want)
} }
discoveryutils.TestEqualLabelss(t, labelss, tt.want)
}) })
} }
} }

View file

@ -5,6 +5,7 @@ import (
"fmt" "fmt"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
// See https://docs.docker.com/engine/api/v1.40/#tag/Node // See https://docs.docker.com/engine/api/v1.40/#tag/Node
@ -37,7 +38,7 @@ type node struct {
} }
} }
func getNodesLabels(cfg *apiConfig) ([]map[string]string, error) { func getNodesLabels(cfg *apiConfig) ([]*promutils.Labels, error) {
nodes, err := getNodes(cfg) nodes, err := getNodes(cfg)
if err != nil { if err != nil {
return nil, err return nil, err
@ -61,26 +62,25 @@ func parseNodes(data []byte) ([]node, error) {
return nodes, nil return nodes, nil
} }
func addNodeLabels(nodes []node, port int) []map[string]string { func addNodeLabels(nodes []node, port int) []*promutils.Labels {
var ms []map[string]string var ms []*promutils.Labels
for _, node := range nodes { for _, node := range nodes {
m := map[string]string{ m := promutils.NewLabels(16)
"__address__": discoveryutils.JoinHostPort(node.Status.Addr, port), m.Add("__address__", discoveryutils.JoinHostPort(node.Status.Addr, port))
"__meta_dockerswarm_node_address": node.Status.Addr, m.Add("__meta_dockerswarm_node_address", node.Status.Addr)
"__meta_dockerswarm_node_availability": node.Spec.Availability, m.Add("__meta_dockerswarm_node_availability", node.Spec.Availability)
"__meta_dockerswarm_node_engine_version": node.Description.Engine.EngineVersion, m.Add("__meta_dockerswarm_node_engine_version", node.Description.Engine.EngineVersion)
"__meta_dockerswarm_node_hostname": node.Description.Hostname, m.Add("__meta_dockerswarm_node_hostname", node.Description.Hostname)
"__meta_dockerswarm_node_id": node.ID, m.Add("__meta_dockerswarm_node_id", node.ID)
"__meta_dockerswarm_node_manager_address": node.ManagerStatus.Addr, m.Add("__meta_dockerswarm_node_manager_address", node.ManagerStatus.Addr)
"__meta_dockerswarm_node_manager_leader": fmt.Sprintf("%t", node.ManagerStatus.Leader), m.Add("__meta_dockerswarm_node_manager_leader", fmt.Sprintf("%t", node.ManagerStatus.Leader))
"__meta_dockerswarm_node_manager_reachability": node.ManagerStatus.Reachability, m.Add("__meta_dockerswarm_node_manager_reachability", node.ManagerStatus.Reachability)
"__meta_dockerswarm_node_platform_architecture": node.Description.Platform.Architecture, m.Add("__meta_dockerswarm_node_platform_architecture", node.Description.Platform.Architecture)
"__meta_dockerswarm_node_platform_os": node.Description.Platform.OS, m.Add("__meta_dockerswarm_node_platform_os", node.Description.Platform.OS)
"__meta_dockerswarm_node_role": node.Spec.Role, m.Add("__meta_dockerswarm_node_role", node.Spec.Role)
"__meta_dockerswarm_node_status": node.Status.State, m.Add("__meta_dockerswarm_node_status", node.Status.State)
}
for k, v := range node.Spec.Labels { for k, v := range node.Spec.Labels {
m[discoveryutils.SanitizeLabelName("__meta_dockerswarm_node_label_"+k)] = v m.Add(discoveryutils.SanitizeLabelName("__meta_dockerswarm_node_label_"+k), v)
} }
ms = append(ms, m) ms = append(ms, m)
} }

View file

@ -4,8 +4,8 @@ import (
"reflect" "reflect"
"testing" "testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
func Test_parseNodes(t *testing.T) { func Test_parseNodes(t *testing.T) {
@ -112,7 +112,7 @@ func Test_addNodeLabels(t *testing.T) {
tests := []struct { tests := []struct {
name string name string
args args args args
want [][]prompbmarshal.Label want []*promutils.Labels
}{ }{
{ {
name: "add labels to one node", name: "add labels to one node",
@ -154,8 +154,8 @@ func Test_addNodeLabels(t *testing.T) {
}, },
port: 9100, port: 9100,
}, },
want: [][]prompbmarshal.Label{ want: []*promutils.Labels{
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "172.31.40.97:9100", "__address__": "172.31.40.97:9100",
"__meta_dockerswarm_node_address": "172.31.40.97", "__meta_dockerswarm_node_address": "172.31.40.97",
"__meta_dockerswarm_node_availability": "active", "__meta_dockerswarm_node_availability": "active",
@ -175,14 +175,7 @@ func Test_addNodeLabels(t *testing.T) {
for _, tt := range tests { for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) { t.Run(tt.name, func(t *testing.T) {
got := addNodeLabels(tt.args.nodes, tt.args.port) got := addNodeLabels(tt.args.nodes, tt.args.port)
discoveryutils.TestEqualLabelss(t, got, tt.want)
var sortedLabelss [][]prompbmarshal.Label
for _, labels := range got {
sortedLabelss = append(sortedLabelss, discoveryutils.GetSortedLabels(labels))
}
if !reflect.DeepEqual(sortedLabelss, tt.want) {
t.Errorf("addNodeLabels() \ngot %v, \nwant %v", sortedLabelss, tt.want)
}
}) })
} }
} }

View file

@ -6,8 +6,8 @@ import (
"net" "net"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger" "github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
// https://docs.docker.com/engine/api/v1.40/#tag/Service // https://docs.docker.com/engine/api/v1.40/#tag/Service
@ -46,7 +46,7 @@ type portConfig struct {
PublishedPort int PublishedPort int
} }
func getServicesLabels(cfg *apiConfig) ([]map[string]string, error) { func getServicesLabels(cfg *apiConfig) ([]*promutils.Labels, error) {
services, err := getServices(cfg) services, err := getServices(cfg)
if err != nil { if err != nil {
return nil, err return nil, err
@ -84,19 +84,18 @@ func getServiceMode(svc service) string {
return "" return ""
} }
func addServicesLabels(services []service, networksLabels map[string]map[string]string, port int) []map[string]string { func addServicesLabels(services []service, networksLabels map[string]*promutils.Labels, port int) []*promutils.Labels {
var ms []map[string]string var ms []*promutils.Labels
for _, service := range services { for _, service := range services {
commonLabels := map[string]string{ commonLabels := promutils.NewLabels(10)
"__meta_dockerswarm_service_id": service.ID, commonLabels.Add("__meta_dockerswarm_service_id", service.ID)
"__meta_dockerswarm_service_name": service.Spec.Name, commonLabels.Add("__meta_dockerswarm_service_name", service.Spec.Name)
"__meta_dockerswarm_service_mode": getServiceMode(service), commonLabels.Add("__meta_dockerswarm_service_mode", getServiceMode(service))
"__meta_dockerswarm_service_task_container_hostname": service.Spec.TaskTemplate.ContainerSpec.Hostname, commonLabels.Add("__meta_dockerswarm_service_task_container_hostname", service.Spec.TaskTemplate.ContainerSpec.Hostname)
"__meta_dockerswarm_service_task_container_image": service.Spec.TaskTemplate.ContainerSpec.Image, commonLabels.Add("__meta_dockerswarm_service_task_container_image", service.Spec.TaskTemplate.ContainerSpec.Image)
"__meta_dockerswarm_service_updating_status": service.UpdateStatus.State, commonLabels.Add("__meta_dockerswarm_service_updating_status", service.UpdateStatus.State)
}
for k, v := range service.Spec.Labels { for k, v := range service.Spec.Labels {
commonLabels[discoveryutils.SanitizeLabelName("__meta_dockerswarm_service_label_"+k)] = v commonLabels.Add(discoveryutils.SanitizeLabelName("__meta_dockerswarm_service_label_"+k), v)
} }
for _, vip := range service.Endpoint.VirtualIPs { for _, vip := range service.Endpoint.VirtualIPs {
// skip services without virtual address. // skip services without virtual address.
@ -114,30 +113,22 @@ func addServicesLabels(services []service, networksLabels map[string]map[string]
if ep.Protocol != "tcp" { if ep.Protocol != "tcp" {
continue continue
} }
m := map[string]string{ m := promutils.NewLabels(24)
"__address__": discoveryutils.JoinHostPort(ip.String(), ep.PublishedPort), m.Add("__address__", discoveryutils.JoinHostPort(ip.String(), ep.PublishedPort))
"__meta_dockerswarm_service_endpoint_port_name": ep.Name, m.Add("__meta_dockerswarm_service_endpoint_port_name", ep.Name)
"__meta_dockerswarm_service_endpoint_port_publish_mode": ep.PublishMode, m.Add("__meta_dockerswarm_service_endpoint_port_publish_mode", ep.PublishMode)
} m.AddFrom(commonLabels)
for k, v := range commonLabels { m.AddFrom(networksLabels[vip.NetworkID])
m[k] = v m.RemoveDuplicates()
}
for k, v := range networksLabels[vip.NetworkID] {
m[k] = v
}
added = true added = true
ms = append(ms, m) ms = append(ms, m)
} }
if !added { if !added {
m := map[string]string{ m := promutils.NewLabels(24)
"__address__": discoveryutils.JoinHostPort(ip.String(), port), m.Add("__address__", discoveryutils.JoinHostPort(ip.String(), port))
} m.AddFrom(commonLabels)
for k, v := range commonLabels { m.AddFrom(networksLabels[vip.NetworkID])
m[k] = v m.RemoveDuplicates()
}
for k, v := range networksLabels[vip.NetworkID] {
m[k] = v
}
ms = append(ms, m) ms = append(ms, m)
} }
} }

View file

@ -4,8 +4,8 @@ import (
"reflect" "reflect"
"testing" "testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
func Test_parseServicesResponse(t *testing.T) { func Test_parseServicesResponse(t *testing.T) {
@ -172,27 +172,27 @@ func Test_parseServicesResponse(t *testing.T) {
func Test_addServicesLabels(t *testing.T) { func Test_addServicesLabels(t *testing.T) {
type args struct { type args struct {
services []service services []service
networksLabels map[string]map[string]string networksLabels map[string]*promutils.Labels
port int port int
} }
tests := []struct { tests := []struct {
name string name string
args args args args
want [][]prompbmarshal.Label want []*promutils.Labels
}{ }{
{ {
name: "add 2 services with network labels join", name: "add 2 services with network labels join",
args: args{ args: args{
port: 9100, port: 9100,
networksLabels: map[string]map[string]string{ networksLabels: map[string]*promutils.Labels{
"qs0hog6ldlei9ct11pr3c77v1": { "qs0hog6ldlei9ct11pr3c77v1": promutils.NewLabelsFromMap(map[string]string{
"__meta_dockerswarm_network_id": "qs0hog6ldlei9ct11pr3c77v1", "__meta_dockerswarm_network_id": "qs0hog6ldlei9ct11pr3c77v1",
"__meta_dockerswarm_network_ingress": "true", "__meta_dockerswarm_network_ingress": "true",
"__meta_dockerswarm_network_internal": "false", "__meta_dockerswarm_network_internal": "false",
"__meta_dockerswarm_network_label_key1": "value1", "__meta_dockerswarm_network_label_key1": "value1",
"__meta_dockerswarm_network_name": "ingress", "__meta_dockerswarm_network_name": "ingress",
"__meta_dockerswarm_network_scope": "swarm", "__meta_dockerswarm_network_scope": "swarm",
}, }),
}, },
services: []service{ services: []service{
{ {
@ -259,8 +259,8 @@ func Test_addServicesLabels(t *testing.T) {
}, },
}, },
}, },
want: [][]prompbmarshal.Label{ want: []*promutils.Labels{
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "10.0.0.3:0", "__address__": "10.0.0.3:0",
"__meta_dockerswarm_network_id": "qs0hog6ldlei9ct11pr3c77v1", "__meta_dockerswarm_network_id": "qs0hog6ldlei9ct11pr3c77v1",
"__meta_dockerswarm_network_ingress": "true", "__meta_dockerswarm_network_ingress": "true",
@ -282,13 +282,7 @@ func Test_addServicesLabels(t *testing.T) {
for _, tt := range tests { for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) { t.Run(tt.name, func(t *testing.T) {
got := addServicesLabels(tt.args.services, tt.args.networksLabels, tt.args.port) got := addServicesLabels(tt.args.services, tt.args.networksLabels, tt.args.port)
var sortedLabelss [][]prompbmarshal.Label discoveryutils.TestEqualLabelss(t, got, tt.want)
for _, labels := range got {
sortedLabelss = append(sortedLabelss, discoveryutils.GetSortedLabels(labels))
}
if !reflect.DeepEqual(sortedLabelss, tt.want) {
t.Errorf("addServicesLabels() \ngot %v, \nwant %v", sortedLabelss, tt.want)
}
}) })
} }
} }

View file

@ -8,6 +8,7 @@ import (
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger" "github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
// See https://docs.docker.com/engine/api/v1.40/#tag/Task // See https://docs.docker.com/engine/api/v1.40/#tag/Task
@ -39,7 +40,7 @@ type task struct {
Slot int Slot int
} }
func getTasksLabels(cfg *apiConfig) ([]map[string]string, error) { func getTasksLabels(cfg *apiConfig) ([]*promutils.Labels, error) {
tasks, err := getTasks(cfg) tasks, err := getTasks(cfg)
if err != nil { if err != nil {
return nil, err return nil, err
@ -76,18 +77,17 @@ func parseTasks(data []byte) ([]task, error) {
return tasks, nil return tasks, nil
} }
func addTasksLabels(tasks []task, nodesLabels, servicesLabels []map[string]string, networksLabels map[string]map[string]string, services []service, port int) []map[string]string { func addTasksLabels(tasks []task, nodesLabels, servicesLabels []*promutils.Labels, networksLabels map[string]*promutils.Labels, services []service, port int) []*promutils.Labels {
var ms []map[string]string var ms []*promutils.Labels
for _, task := range tasks { for _, task := range tasks {
commonLabels := map[string]string{ commonLabels := promutils.NewLabels(8)
"__meta_dockerswarm_task_id": task.ID, commonLabels.Add("__meta_dockerswarm_task_id", task.ID)
"__meta_dockerswarm_task_container_id": task.Status.ContainerStatus.ContainerID, commonLabels.Add("__meta_dockerswarm_task_container_id", task.Status.ContainerStatus.ContainerID)
"__meta_dockerswarm_task_desired_state": task.DesiredState, commonLabels.Add("__meta_dockerswarm_task_desired_state", task.DesiredState)
"__meta_dockerswarm_task_slot": strconv.Itoa(task.Slot), commonLabels.Add("__meta_dockerswarm_task_slot", strconv.Itoa(task.Slot))
"__meta_dockerswarm_task_state": task.Status.State, commonLabels.Add("__meta_dockerswarm_task_state", task.Status.State)
}
for k, v := range task.Spec.ContainerSpec.Labels { for k, v := range task.Spec.ContainerSpec.Labels {
commonLabels[discoveryutils.SanitizeLabelName("__meta_dockerswarm_container_label_"+k)] = v commonLabels.Add(discoveryutils.SanitizeLabelName("__meta_dockerswarm_container_label_"+k), v)
} }
var svcPorts []portConfig var svcPorts []portConfig
for i, v := range services { for i, v := range services {
@ -103,12 +103,11 @@ func addTasksLabels(tasks []task, nodesLabels, servicesLabels []map[string]strin
if port.Protocol != "tcp" { if port.Protocol != "tcp" {
continue continue
} }
m := make(map[string]string, len(commonLabels)+2) m := promutils.NewLabels(10)
for k, v := range commonLabels { m.AddFrom(commonLabels)
m[k] = v m.Add("__address__", discoveryutils.JoinHostPort(commonLabels.Get("__meta_dockerswarm_node_address"), port.PublishedPort))
} m.Add("__meta_dockerswarm_task_port_publish_mode", port.PublishMode)
m["__address__"] = discoveryutils.JoinHostPort(commonLabels["__meta_dockerswarm_node_address"], port.PublishedPort) m.RemoveDuplicates()
m["__meta_dockerswarm_task_port_publish_mode"] = port.PublishMode
ms = append(ms, m) ms = append(ms, m)
} }
for _, na := range task.NetworksAttachments { for _, na := range task.NetworksAttachments {
@ -124,27 +123,21 @@ func addTasksLabels(tasks []task, nodesLabels, servicesLabels []map[string]strin
if ep.Protocol != "tcp" { if ep.Protocol != "tcp" {
continue continue
} }
m := make(map[string]string, len(commonLabels)+len(networkLabels)+2) m := promutils.NewLabels(20)
for k, v := range commonLabels { m.AddFrom(commonLabels)
m[k] = v m.AddFrom(networkLabels)
} m.Add("__address__", discoveryutils.JoinHostPort(ip.String(), ep.PublishedPort))
for k, v := range networkLabels { m.Add("__meta_dockerswarm_task_port_publish_mode", ep.PublishMode)
m[k] = v m.RemoveDuplicates()
}
m["__address__"] = discoveryutils.JoinHostPort(ip.String(), ep.PublishedPort)
m["__meta_dockerswarm_task_port_publish_mode"] = ep.PublishMode
ms = append(ms, m) ms = append(ms, m)
added = true added = true
} }
if !added { if !added {
m := make(map[string]string, len(commonLabels)+len(networkLabels)+1) m := promutils.NewLabels(20)
for k, v := range commonLabels { m.AddFrom(commonLabels)
m[k] = v m.AddFrom(networkLabels)
} m.Add("__address__", discoveryutils.JoinHostPort(ip.String(), port))
for k, v := range networkLabels { m.RemoveDuplicates()
m[k] = v
}
m["__address__"] = discoveryutils.JoinHostPort(ip.String(), port)
ms = append(ms, m) ms = append(ms, m)
} }
} }
@ -154,13 +147,13 @@ func addTasksLabels(tasks []task, nodesLabels, servicesLabels []map[string]strin
} }
// addLabels adds lables from src to dst if they contain the given `key: value` pair. // addLabels adds lables from src to dst if they contain the given `key: value` pair.
func addLabels(dst map[string]string, src []map[string]string, key, value string) { func addLabels(dst *promutils.Labels, src []*promutils.Labels, key, value string) {
for _, m := range src { for _, m := range src {
if m[key] != value { if m.Get(key) != value {
continue continue
} }
for k, v := range m { for _, label := range m.GetLabels() {
dst[k] = v dst.Add(label.Name, label.Value)
} }
return return
} }

View file

@ -4,8 +4,8 @@ import (
"reflect" "reflect"
"testing" "testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
func Test_parseTasks(t *testing.T) { func Test_parseTasks(t *testing.T) {
@ -116,16 +116,16 @@ func Test_parseTasks(t *testing.T) {
func Test_addTasksLabels(t *testing.T) { func Test_addTasksLabels(t *testing.T) {
type args struct { type args struct {
tasks []task tasks []task
nodesLabels []map[string]string nodesLabels []*promutils.Labels
servicesLabels []map[string]string servicesLabels []*promutils.Labels
networksLabels map[string]map[string]string networksLabels map[string]*promutils.Labels
services []service services []service
port int port int
} }
tests := []struct { tests := []struct {
name string name string
args args args args
want [][]prompbmarshal.Label want []*promutils.Labels
}{ }{
{ {
name: "adds 1 task with nodes labels", name: "adds 1 task with nodes labels",
@ -159,8 +159,8 @@ func Test_addTasksLabels(t *testing.T) {
}}, }},
}, },
}, },
nodesLabels: []map[string]string{ nodesLabels: []*promutils.Labels{
{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "172.31.40.97:9100", "__address__": "172.31.40.97:9100",
"__meta_dockerswarm_node_address": "172.31.40.97", "__meta_dockerswarm_node_address": "172.31.40.97",
"__meta_dockerswarm_node_availability": "active", "__meta_dockerswarm_node_availability": "active",
@ -171,11 +171,11 @@ func Test_addTasksLabels(t *testing.T) {
"__meta_dockerswarm_node_platform_os": "linux", "__meta_dockerswarm_node_platform_os": "linux",
"__meta_dockerswarm_node_role": "manager", "__meta_dockerswarm_node_role": "manager",
"__meta_dockerswarm_node_status": "ready", "__meta_dockerswarm_node_status": "ready",
}),
}, },
}, },
}, want: []*promutils.Labels{
want: [][]prompbmarshal.Label{ promutils.NewLabelsFromMap(map[string]string{
discoveryutils.GetSortedLabels(map[string]string{
"__address__": "172.31.40.97:6379", "__address__": "172.31.40.97:6379",
"__meta_dockerswarm_node_address": "172.31.40.97", "__meta_dockerswarm_node_address": "172.31.40.97",
"__meta_dockerswarm_node_availability": "active", "__meta_dockerswarm_node_availability": "active",
@ -230,18 +230,18 @@ func Test_addTasksLabels(t *testing.T) {
PortStatus: struct{ Ports []portConfig }{}}, PortStatus: struct{ Ports []portConfig }{}},
}, },
}, },
networksLabels: map[string]map[string]string{ networksLabels: map[string]*promutils.Labels{
"qs0hog6ldlei9ct11pr3c77v1": { "qs0hog6ldlei9ct11pr3c77v1": promutils.NewLabelsFromMap(map[string]string{
"__meta_dockerswarm_network_id": "qs0hog6ldlei9ct11pr3c77v1", "__meta_dockerswarm_network_id": "qs0hog6ldlei9ct11pr3c77v1",
"__meta_dockerswarm_network_ingress": "true", "__meta_dockerswarm_network_ingress": "true",
"__meta_dockerswarm_network_internal": "false", "__meta_dockerswarm_network_internal": "false",
"__meta_dockerswarm_network_label_key1": "value1", "__meta_dockerswarm_network_label_key1": "value1",
"__meta_dockerswarm_network_name": "ingress", "__meta_dockerswarm_network_name": "ingress",
"__meta_dockerswarm_network_scope": "swarm", "__meta_dockerswarm_network_scope": "swarm",
}),
}, },
}, nodesLabels: []*promutils.Labels{
nodesLabels: []map[string]string{ promutils.NewLabelsFromMap(map[string]string{
{
"__address__": "172.31.40.97:9100", "__address__": "172.31.40.97:9100",
"__meta_dockerswarm_node_address": "172.31.40.97", "__meta_dockerswarm_node_address": "172.31.40.97",
"__meta_dockerswarm_node_availability": "active", "__meta_dockerswarm_node_availability": "active",
@ -252,7 +252,7 @@ func Test_addTasksLabels(t *testing.T) {
"__meta_dockerswarm_node_platform_os": "linux", "__meta_dockerswarm_node_platform_os": "linux",
"__meta_dockerswarm_node_role": "manager", "__meta_dockerswarm_node_role": "manager",
"__meta_dockerswarm_node_status": "ready", "__meta_dockerswarm_node_status": "ready",
}, }),
}, },
services: []service{ services: []service{
{ {
@ -320,10 +320,10 @@ func Test_addTasksLabels(t *testing.T) {
}, },
}, },
}, },
servicesLabels: []map[string]string{}, servicesLabels: []*promutils.Labels{},
}, },
want: [][]prompbmarshal.Label{ want: []*promutils.Labels{
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "10.10.15.15:6379", "__address__": "10.10.15.15:6379",
"__meta_dockerswarm_network_id": "qs0hog6ldlei9ct11pr3c77v1", "__meta_dockerswarm_network_id": "qs0hog6ldlei9ct11pr3c77v1",
"__meta_dockerswarm_network_ingress": "true", "__meta_dockerswarm_network_ingress": "true",
@ -353,13 +353,7 @@ func Test_addTasksLabels(t *testing.T) {
for _, tt := range tests { for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) { t.Run(tt.name, func(t *testing.T) {
got := addTasksLabels(tt.args.tasks, tt.args.nodesLabels, tt.args.servicesLabels, tt.args.networksLabels, tt.args.services, tt.args.port) got := addTasksLabels(tt.args.tasks, tt.args.nodesLabels, tt.args.servicesLabels, tt.args.networksLabels, tt.args.services, tt.args.port)
var sortedLabelss [][]prompbmarshal.Label discoveryutils.TestEqualLabelss(t, got, tt.want)
for _, labels := range got {
sortedLabelss = append(sortedLabelss, discoveryutils.GetSortedLabels(labels))
}
if !reflect.DeepEqual(sortedLabelss, tt.want) {
t.Errorf("addTasksLabels() \ngot %v, \nwant %v", sortedLabelss, tt.want)
}
}) })
} }
} }

View file

@ -7,6 +7,7 @@ import (
"github.com/VictoriaMetrics/VictoriaMetrics/lib/awsapi" "github.com/VictoriaMetrics/VictoriaMetrics/lib/awsapi"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
// SDCheckInterval defines interval for targets refresh. // SDCheckInterval defines interval for targets refresh.
@ -34,7 +35,7 @@ type SDConfig struct {
} }
// GetLabels returns ec2 labels according to sdc. // GetLabels returns ec2 labels according to sdc.
func (sdc *SDConfig) GetLabels(baseDir string) ([]map[string]string, error) { func (sdc *SDConfig) GetLabels(baseDir string) ([]*promutils.Labels, error) {
cfg, err := getAPIConfig(sdc) cfg, err := getAPIConfig(sdc)
if err != nil { if err != nil {
return nil, fmt.Errorf("cannot get API config: %w", err) return nil, fmt.Errorf("cannot get API config: %w", err)

View file

@ -7,17 +7,18 @@ import (
"github.com/VictoriaMetrics/VictoriaMetrics/lib/awsapi" "github.com/VictoriaMetrics/VictoriaMetrics/lib/awsapi"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
// getInstancesLabels returns labels for ec2 instances obtained from the given cfg // getInstancesLabels returns labels for ec2 instances obtained from the given cfg
func getInstancesLabels(cfg *apiConfig) ([]map[string]string, error) { func getInstancesLabels(cfg *apiConfig) ([]*promutils.Labels, error) {
rs, err := getReservations(cfg) rs, err := getReservations(cfg)
if err != nil { if err != nil {
return nil, err return nil, err
} }
azMap := getAZMap(cfg) azMap := getAZMap(cfg)
region := cfg.awsConfig.GetRegion() region := cfg.awsConfig.GetRegion()
var ms []map[string]string var ms []*promutils.Labels
for _, r := range rs { for _, r := range rs {
for _, inst := range r.InstanceSet.Items { for _, inst := range r.InstanceSet.Items {
ms = inst.appendTargetLabels(ms, r.OwnerID, region, cfg.port, azMap) ms = inst.appendTargetLabels(ms, r.OwnerID, region, cfg.port, azMap)
@ -135,32 +136,31 @@ func parseInstancesResponse(data []byte) (*InstancesResponse, error) {
return &v, nil return &v, nil
} }
func (inst *Instance) appendTargetLabels(ms []map[string]string, ownerID, region string, port int, azMap map[string]string) []map[string]string { func (inst *Instance) appendTargetLabels(ms []*promutils.Labels, ownerID, region string, port int, azMap map[string]string) []*promutils.Labels {
if len(inst.PrivateIPAddress) == 0 { if len(inst.PrivateIPAddress) == 0 {
// Cannot scrape instance without private IP address // Cannot scrape instance without private IP address
return ms return ms
} }
addr := discoveryutils.JoinHostPort(inst.PrivateIPAddress, port) addr := discoveryutils.JoinHostPort(inst.PrivateIPAddress, port)
m := map[string]string{ m := promutils.NewLabels(24)
"__address__": addr, m.Add("__address__", addr)
"__meta_ec2_architecture": inst.Architecture, m.Add("__meta_ec2_architecture", inst.Architecture)
"__meta_ec2_ami": inst.ImageID, m.Add("__meta_ec2_ami", inst.ImageID)
"__meta_ec2_availability_zone": inst.Placement.AvailabilityZone, m.Add("__meta_ec2_availability_zone", inst.Placement.AvailabilityZone)
"__meta_ec2_availability_zone_id": azMap[inst.Placement.AvailabilityZone], m.Add("__meta_ec2_availability_zone_id", azMap[inst.Placement.AvailabilityZone])
"__meta_ec2_instance_id": inst.ID, m.Add("__meta_ec2_instance_id", inst.ID)
"__meta_ec2_instance_lifecycle": inst.Lifecycle, m.Add("__meta_ec2_instance_lifecycle", inst.Lifecycle)
"__meta_ec2_instance_state": inst.State.Name, m.Add("__meta_ec2_instance_state", inst.State.Name)
"__meta_ec2_instance_type": inst.Type, m.Add("__meta_ec2_instance_type", inst.Type)
"__meta_ec2_owner_id": ownerID, m.Add("__meta_ec2_owner_id", ownerID)
"__meta_ec2_platform": inst.Platform, m.Add("__meta_ec2_platform", inst.Platform)
"__meta_ec2_primary_subnet_id": inst.SubnetID, m.Add("__meta_ec2_primary_subnet_id", inst.SubnetID)
"__meta_ec2_private_dns_name": inst.PrivateDNSName, m.Add("__meta_ec2_private_dns_name", inst.PrivateDNSName)
"__meta_ec2_private_ip": inst.PrivateIPAddress, m.Add("__meta_ec2_private_ip", inst.PrivateIPAddress)
"__meta_ec2_public_dns_name": inst.PublicDNSName, m.Add("__meta_ec2_public_dns_name", inst.PublicDNSName)
"__meta_ec2_public_ip": inst.PublicIPAddress, m.Add("__meta_ec2_public_ip", inst.PublicIPAddress)
"__meta_ec2_region": region, m.Add("__meta_ec2_region", region)
"__meta_ec2_vpc_id": inst.VPCID, m.Add("__meta_ec2_vpc_id", inst.VPCID)
}
if len(inst.VPCID) > 0 { if len(inst.VPCID) > 0 {
subnets := make([]string, 0, len(inst.NetworkInterfaceSet.Items)) subnets := make([]string, 0, len(inst.NetworkInterfaceSet.Items))
seenSubnets := make(map[string]bool, len(inst.NetworkInterfaceSet.Items)) seenSubnets := make(map[string]bool, len(inst.NetworkInterfaceSet.Items))
@ -179,16 +179,16 @@ func (inst *Instance) appendTargetLabels(ms []map[string]string, ownerID, region
} }
// We surround the separated list with the separator as well. This way regular expressions // We surround the separated list with the separator as well. This way regular expressions
// in relabeling rules don't have to consider tag positions. // in relabeling rules don't have to consider tag positions.
m["__meta_ec2_subnet_id"] = "," + strings.Join(subnets, ",") + "," m.Add("__meta_ec2_subnet_id", ","+strings.Join(subnets, ",")+",")
if len(ipv6Addrs) > 0 { if len(ipv6Addrs) > 0 {
m["__meta_ec2_ipv6_addresses"] = "," + strings.Join(ipv6Addrs, ",") + "," m.Add("__meta_ec2_ipv6_addresses", ","+strings.Join(ipv6Addrs, ",")+",")
} }
} }
for _, t := range inst.TagSet.Items { for _, t := range inst.TagSet.Items {
if len(t.Key) == 0 || len(t.Value) == 0 { if len(t.Key) == 0 || len(t.Value) == 0 {
continue continue
} }
m[discoveryutils.SanitizeLabelName("__meta_ec2_tag_"+t.Key)] = t.Value m.Add(discoveryutils.SanitizeLabelName("__meta_ec2_tag_"+t.Key), t.Value)
} }
ms = append(ms, m) ms = append(ms, m)
return ms return ms

View file

@ -4,8 +4,8 @@ import (
"reflect" "reflect"
"testing" "testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
func TestDescribeAvailabilityZonesResponse(t *testing.T) { func TestDescribeAvailabilityZonesResponse(t *testing.T) {
@ -241,12 +241,8 @@ func TestParseInstancesResponse(t *testing.T) {
labelss := inst.appendTargetLabels(nil, ownerID, "region-a", port, map[string]string{ labelss := inst.appendTargetLabels(nil, ownerID, "region-a", port, map[string]string{
"eu-west-2c": "foobar-zone", "eu-west-2c": "foobar-zone",
}) })
var sortedLabelss [][]prompbmarshal.Label expectedLabels := []*promutils.Labels{
for _, labels := range labelss { promutils.NewLabelsFromMap(map[string]string{
sortedLabelss = append(sortedLabelss, discoveryutils.GetSortedLabels(labels))
}
expectedLabels := [][]prompbmarshal.Label{
discoveryutils.GetSortedLabels(map[string]string{
"__address__": "172.31.11.152:423", "__address__": "172.31.11.152:423",
"__meta_ec2_architecture": "x86_64", "__meta_ec2_architecture": "x86_64",
"__meta_ec2_availability_zone": "eu-west-2c", "__meta_ec2_availability_zone": "eu-west-2c",
@ -269,7 +265,5 @@ func TestParseInstancesResponse(t *testing.T) {
"__meta_ec2_vpc_id": "vpc-f1eaad99", "__meta_ec2_vpc_id": "vpc-f1eaad99",
}), }),
} }
if !reflect.DeepEqual(sortedLabelss, expectedLabels) { discoveryutils.TestEqualLabelss(t, labelss, expectedLabels)
t.Fatalf("unexpected labels:\ngot\n%v\nwant\n%v", sortedLabelss, expectedLabels)
}
} }

View file

@ -9,6 +9,7 @@ import (
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/proxy" "github.com/VictoriaMetrics/VictoriaMetrics/lib/proxy"
) )
@ -82,7 +83,7 @@ type DataCenterInfo struct {
} }
// GetLabels returns Eureka labels according to sdc. // GetLabels returns Eureka labels according to sdc.
func (sdc *SDConfig) GetLabels(baseDir string) ([]map[string]string, error) { func (sdc *SDConfig) GetLabels(baseDir string) ([]*promutils.Labels, error) {
cfg, err := getAPIConfig(sdc, baseDir) cfg, err := getAPIConfig(sdc, baseDir)
if err != nil { if err != nil {
return nil, fmt.Errorf("cannot get API config: %w", err) return nil, fmt.Errorf("cannot get API config: %w", err)
@ -103,8 +104,8 @@ func (sdc *SDConfig) MustStop() {
configMap.Delete(sdc) configMap.Delete(sdc)
} }
func addInstanceLabels(apps *applications) []map[string]string { func addInstanceLabels(apps *applications) []*promutils.Labels {
var ms []map[string]string var ms []*promutils.Labels
for _, app := range apps.Applications { for _, app := range apps.Applications {
for _, instance := range app.Instances { for _, instance := range app.Instances {
instancePort := 80 instancePort := 80
@ -112,38 +113,37 @@ func addInstanceLabels(apps *applications) []map[string]string {
instancePort = instance.Port.Port instancePort = instance.Port.Port
} }
targetAddress := discoveryutils.JoinHostPort(instance.HostName, instancePort) targetAddress := discoveryutils.JoinHostPort(instance.HostName, instancePort)
m := map[string]string{ m := promutils.NewLabels(24)
"__address__": targetAddress, m.Add("__address__", targetAddress)
"instance": instance.InstanceID, m.Add("instance", instance.InstanceID)
"__meta_eureka_app_name": app.Name, m.Add("__meta_eureka_app_name", app.Name)
"__meta_eureka_app_instance_hostname": instance.HostName, m.Add("__meta_eureka_app_instance_hostname", instance.HostName)
"__meta_eureka_app_instance_homepage_url": instance.HomePageURL, m.Add("__meta_eureka_app_instance_homepage_url", instance.HomePageURL)
"__meta_eureka_app_instance_statuspage_url": instance.StatusPageURL, m.Add("__meta_eureka_app_instance_statuspage_url", instance.StatusPageURL)
"__meta_eureka_app_instance_healthcheck_url": instance.HealthCheckURL, m.Add("__meta_eureka_app_instance_healthcheck_url", instance.HealthCheckURL)
"__meta_eureka_app_instance_ip_addr": instance.IPAddr, m.Add("__meta_eureka_app_instance_ip_addr", instance.IPAddr)
"__meta_eureka_app_instance_vip_address": instance.VipAddress, m.Add("__meta_eureka_app_instance_vip_address", instance.VipAddress)
"__meta_eureka_app_instance_secure_vip_address": instance.SecureVipAddress, m.Add("__meta_eureka_app_instance_secure_vip_address", instance.SecureVipAddress)
"__meta_eureka_app_instance_status": instance.Status, m.Add("__meta_eureka_app_instance_status", instance.Status)
"__meta_eureka_app_instance_country_id": strconv.Itoa(instance.CountryID), m.Add("__meta_eureka_app_instance_country_id", strconv.Itoa(instance.CountryID))
"__meta_eureka_app_instance_id": instance.InstanceID, m.Add("__meta_eureka_app_instance_id", instance.InstanceID)
}
if instance.Port.Port != 0 { if instance.Port.Port != 0 {
m["__meta_eureka_app_instance_port"] = strconv.Itoa(instance.Port.Port) m.Add("__meta_eureka_app_instance_port", strconv.Itoa(instance.Port.Port))
m["__meta_eureka_app_instance_port_enabled"] = strconv.FormatBool(instance.Port.Enabled) m.Add("__meta_eureka_app_instance_port_enabled", strconv.FormatBool(instance.Port.Enabled))
} }
if instance.SecurePort.Port != 0 { if instance.SecurePort.Port != 0 {
m["__meta_eureka_app_instance_secure_port"] = strconv.Itoa(instance.SecurePort.Port) m.Add("__meta_eureka_app_instance_secure_port", strconv.Itoa(instance.SecurePort.Port))
m["__meta_eureka_app_instance_secure_port_enabled"] = strconv.FormatBool(instance.SecurePort.Enabled) m.Add("__meta_eureka_app_instance_secure_port_enabled", strconv.FormatBool(instance.SecurePort.Enabled))
} }
if len(instance.DataCenterInfo.Name) > 0 { if len(instance.DataCenterInfo.Name) > 0 {
m["__meta_eureka_app_instance_datacenterinfo_name"] = instance.DataCenterInfo.Name m.Add("__meta_eureka_app_instance_datacenterinfo_name", instance.DataCenterInfo.Name)
for _, tag := range instance.DataCenterInfo.Metadata.Items { for _, tag := range instance.DataCenterInfo.Metadata.Items {
m[discoveryutils.SanitizeLabelName("__meta_eureka_app_instance_datacenterinfo_metadata_"+tag.XMLName.Local)] = tag.Content m.Add(discoveryutils.SanitizeLabelName("__meta_eureka_app_instance_datacenterinfo_metadata_"+tag.XMLName.Local), tag.Content)
} }
} }
for _, tag := range instance.Metadata.Items { for _, tag := range instance.Metadata.Items {
m[discoveryutils.SanitizeLabelName("__meta_eureka_app_instance_metadata_"+tag.XMLName.Local)] = tag.Content m.Add(discoveryutils.SanitizeLabelName("__meta_eureka_app_instance_metadata_"+tag.XMLName.Local), tag.Content)
} }
ms = append(ms, m) ms = append(ms, m)
} }

View file

@ -1,11 +1,10 @@
package eureka package eureka
import ( import (
"reflect"
"testing" "testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
func Test_addInstanceLabels(t *testing.T) { func Test_addInstanceLabels(t *testing.T) {
@ -15,7 +14,7 @@ func Test_addInstanceLabels(t *testing.T) {
tests := []struct { tests := []struct {
name string name string
args args args args
want [][]prompbmarshal.Label want []*promutils.Labels
}{ }{
{ {
name: "1 application", name: "1 application",
@ -50,8 +49,8 @@ func Test_addInstanceLabels(t *testing.T) {
}, },
}, },
}, },
want: [][]prompbmarshal.Label{ want: []*promutils.Labels{
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "host-1:9100", "__address__": "host-1:9100",
"instance": "some-id", "instance": "some-id",
"__meta_eureka_app_instance_hostname": "host-1", "__meta_eureka_app_instance_hostname": "host-1",
@ -75,13 +74,7 @@ func Test_addInstanceLabels(t *testing.T) {
for _, tt := range tests { for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) { t.Run(tt.name, func(t *testing.T) {
got := addInstanceLabels(tt.args.applications) got := addInstanceLabels(tt.args.applications)
var sortedLabelss [][]prompbmarshal.Label discoveryutils.TestEqualLabelss(t, got, tt.want)
for _, labels := range got {
sortedLabelss = append(sortedLabelss, discoveryutils.GetSortedLabels(labels))
}
if !reflect.DeepEqual(sortedLabelss, tt.want) {
t.Fatalf("unexpected labels \ngot : %v, \nwant: %v", got, tt.want)
}
}) })
} }
} }

View file

@ -4,6 +4,8 @@ import (
"flag" "flag"
"fmt" "fmt"
"time" "time"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
// SDCheckInterval defines interval for targets refresh. // SDCheckInterval defines interval for targets refresh.
@ -60,7 +62,7 @@ func (z ZoneYAML) MarshalYAML() (interface{}, error) {
} }
// GetLabels returns gce labels according to sdc. // GetLabels returns gce labels according to sdc.
func (sdc *SDConfig) GetLabels(baseDir string) ([]map[string]string, error) { func (sdc *SDConfig) GetLabels(baseDir string) ([]*promutils.Labels, error) {
cfg, err := getAPIConfig(sdc) cfg, err := getAPIConfig(sdc)
if err != nil { if err != nil {
return nil, fmt.Errorf("cannot get API config: %w", err) return nil, fmt.Errorf("cannot get API config: %w", err)

View file

@ -8,12 +8,13 @@ import (
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger" "github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
// getInstancesLabels returns labels for gce instances obtained from the given cfg // getInstancesLabels returns labels for gce instances obtained from the given cfg
func getInstancesLabels(cfg *apiConfig) []map[string]string { func getInstancesLabels(cfg *apiConfig) []*promutils.Labels {
insts := getInstances(cfg) insts := getInstances(cfg)
var ms []map[string]string var ms []*promutils.Labels
for _, inst := range insts { for _, inst := range insts {
ms = inst.appendTargetLabels(ms, cfg.project, cfg.tagSeparator, cfg.port) ms = inst.appendTargetLabels(ms, cfg.project, cfg.tagSeparator, cfg.port)
} }
@ -88,7 +89,7 @@ type Instance struct {
NetworkInterfaces []NetworkInterface NetworkInterfaces []NetworkInterface
Tags TagList Tags TagList
Metadata MetadataList Metadata MetadataList
Labels discoveryutils.SortedLabels Labels *promutils.Labels
} }
// NetworkInterface is network interface from https://cloud.google.com/compute/docs/reference/rest/v1/instances/list // NetworkInterface is network interface from https://cloud.google.com/compute/docs/reference/rest/v1/instances/list
@ -131,42 +132,41 @@ func parseInstanceList(data []byte) (*InstanceList, error) {
return &il, nil return &il, nil
} }
func (inst *Instance) appendTargetLabels(ms []map[string]string, project, tagSeparator string, port int) []map[string]string { func (inst *Instance) appendTargetLabels(ms []*promutils.Labels, project, tagSeparator string, port int) []*promutils.Labels {
if len(inst.NetworkInterfaces) == 0 { if len(inst.NetworkInterfaces) == 0 {
return ms return ms
} }
iface := inst.NetworkInterfaces[0] iface := inst.NetworkInterfaces[0]
addr := discoveryutils.JoinHostPort(iface.NetworkIP, port) addr := discoveryutils.JoinHostPort(iface.NetworkIP, port)
m := map[string]string{ m := promutils.NewLabels(24)
"__address__": addr, m.Add("__address__", addr)
"__meta_gce_instance_id": inst.ID, m.Add("__meta_gce_instance_id", inst.ID)
"__meta_gce_instance_status": inst.Status, m.Add("__meta_gce_instance_status", inst.Status)
"__meta_gce_instance_name": inst.Name, m.Add("__meta_gce_instance_name", inst.Name)
"__meta_gce_machine_type": inst.MachineType, m.Add("__meta_gce_machine_type", inst.MachineType)
"__meta_gce_network": iface.Network, m.Add("__meta_gce_network", iface.Network)
"__meta_gce_private_ip": iface.NetworkIP, m.Add("__meta_gce_private_ip", iface.NetworkIP)
"__meta_gce_project": project, m.Add("__meta_gce_project", project)
"__meta_gce_subnetwork": iface.Subnetwork, m.Add("__meta_gce_subnetwork", iface.Subnetwork)
"__meta_gce_zone": inst.Zone, m.Add("__meta_gce_zone", inst.Zone)
}
for _, iface := range inst.NetworkInterfaces { for _, iface := range inst.NetworkInterfaces {
m[discoveryutils.SanitizeLabelName("__meta_gce_interface_ipv4_"+iface.Name)] = iface.NetworkIP m.Add(discoveryutils.SanitizeLabelName("__meta_gce_interface_ipv4_"+iface.Name), iface.NetworkIP)
} }
if len(inst.Tags.Items) > 0 { if len(inst.Tags.Items) > 0 {
// We surround the separated list with the separator as well. This way regular expressions // We surround the separated list with the separator as well. This way regular expressions
// in relabeling rules don't have to consider tag positions. // in relabeling rules don't have to consider tag positions.
m["__meta_gce_tags"] = tagSeparator + strings.Join(inst.Tags.Items, tagSeparator) + tagSeparator m.Add("__meta_gce_tags", tagSeparator+strings.Join(inst.Tags.Items, tagSeparator)+tagSeparator)
} }
for _, item := range inst.Metadata.Items { for _, item := range inst.Metadata.Items {
m[discoveryutils.SanitizeLabelName("__meta_gce_metadata_"+item.Key)] = item.Value m.Add(discoveryutils.SanitizeLabelName("__meta_gce_metadata_"+item.Key), item.Value)
} }
for _, label := range inst.Labels { for _, label := range inst.Labels.Labels {
m[discoveryutils.SanitizeLabelName("__meta_gce_label_"+label.Name)] = label.Value m.Add(discoveryutils.SanitizeLabelName("__meta_gce_label_"+label.Name), label.Value)
} }
if len(iface.AccessConfigs) > 0 { if len(iface.AccessConfigs) > 0 {
ac := iface.AccessConfigs[0] ac := iface.AccessConfigs[0]
if ac.Type == "ONE_TO_ONE_NAT" { if ac.Type == "ONE_TO_ONE_NAT" {
m["__meta_gce_public_ip"] = ac.NatIP m.Add("__meta_gce_public_ip", ac.NatIP)
} }
} }
ms = append(ms, m) ms = append(ms, m)

View file

@ -1,11 +1,10 @@
package gce package gce
import ( import (
"reflect"
"testing" "testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
func TestParseInstanceListFailure(t *testing.T) { func TestParseInstanceListFailure(t *testing.T) {
@ -148,12 +147,8 @@ func TestParseInstanceListSuccess(t *testing.T) {
tagSeparator := "," tagSeparator := ","
port := 80 port := 80
labelss := inst.appendTargetLabels(nil, project, tagSeparator, port) labelss := inst.appendTargetLabels(nil, project, tagSeparator, port)
var sortedLabelss [][]prompbmarshal.Label expectedLabelss := []*promutils.Labels{
for _, labels := range labelss { promutils.NewLabelsFromMap(map[string]string{
sortedLabelss = append(sortedLabelss, discoveryutils.GetSortedLabels(labels))
}
expectedLabelss := [][]prompbmarshal.Label{
discoveryutils.GetSortedLabels(map[string]string{
"__address__": "10.11.2.7:80", "__address__": "10.11.2.7:80",
"__meta_gce_instance_id": "7897352091592122", "__meta_gce_instance_id": "7897352091592122",
"__meta_gce_instance_name": "play-1m-1-vmagent", "__meta_gce_instance_name": "play-1m-1-vmagent",
@ -174,7 +169,5 @@ func TestParseInstanceListSuccess(t *testing.T) {
"__meta_gce_zone": "https://www.googleapis.com/compute/v1/projects/victoriametrics-test/zones/us-east1-b", "__meta_gce_zone": "https://www.googleapis.com/compute/v1/projects/victoriametrics-test/zones/us-east1-b",
}), }),
} }
if !reflect.DeepEqual(sortedLabelss, expectedLabelss) { discoveryutils.TestEqualLabelss(t, labelss, expectedLabelss)
t.Fatalf("unexpected labels:\ngot\n%v\nwant\n%v", sortedLabelss, expectedLabelss)
}
} }

View file

@ -7,6 +7,7 @@ import (
"strconv" "strconv"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
"github.com/VictoriaMetrics/fasthttp" "github.com/VictoriaMetrics/fasthttp"
"github.com/VictoriaMetrics/metrics" "github.com/VictoriaMetrics/metrics"
) )
@ -25,7 +26,7 @@ type apiConfig struct {
// https://prometheus.io/docs/prometheus/latest/http_sd/ // https://prometheus.io/docs/prometheus/latest/http_sd/
type httpGroupTarget struct { type httpGroupTarget struct {
Targets []string `json:"targets"` Targets []string `json:"targets"`
Labels map[string]string `json:"labels"` Labels *promutils.Labels `json:"labels"`
} }
func newAPIConfig(sdc *SDConfig, baseDir string) (*apiConfig, error) { func newAPIConfig(sdc *SDConfig, baseDir string) (*apiConfig, error) {

View file

@ -3,6 +3,8 @@ package http
import ( import (
"reflect" "reflect"
"testing" "testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
func Test_parseAPIResponse(t *testing.T) { func Test_parseAPIResponse(t *testing.T) {
@ -28,7 +30,7 @@ func Test_parseAPIResponse(t *testing.T) {
}, },
want: []httpGroupTarget{ want: []httpGroupTarget{
{ {
Labels: map[string]string{"label-1": "value-1"}, Labels: promutils.NewLabelsFromMap(map[string]string{"label-1": "value-1"}),
Targets: []string{"http://target-1:9100", "http://target-2:9150"}, Targets: []string{"http://target-1:9100", "http://target-2:9150"},
}, },
}, },

View file

@ -6,6 +6,7 @@ import (
"time" "time"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/proxy" "github.com/VictoriaMetrics/VictoriaMetrics/lib/proxy"
) )
@ -25,7 +26,7 @@ type SDConfig struct {
} }
// GetLabels returns http service discovery labels according to sdc. // GetLabels returns http service discovery labels according to sdc.
func (sdc *SDConfig) GetLabels(baseDir string) ([]map[string]string, error) { func (sdc *SDConfig) GetLabels(baseDir string) ([]*promutils.Labels, error) {
cfg, err := getAPIConfig(sdc, baseDir) cfg, err := getAPIConfig(sdc, baseDir)
if err != nil { if err != nil {
return nil, fmt.Errorf("cannot get API config: %w", err) return nil, fmt.Errorf("cannot get API config: %w", err)
@ -37,17 +38,16 @@ func (sdc *SDConfig) GetLabels(baseDir string) ([]map[string]string, error) {
return addHTTPTargetLabels(hts, sdc.URL), nil return addHTTPTargetLabels(hts, sdc.URL), nil
} }
func addHTTPTargetLabels(src []httpGroupTarget, sourceURL string) []map[string]string { func addHTTPTargetLabels(src []httpGroupTarget, sourceURL string) []*promutils.Labels {
ms := make([]map[string]string, 0, len(src)) ms := make([]*promutils.Labels, 0, len(src))
for _, targetGroup := range src { for _, targetGroup := range src {
labels := targetGroup.Labels labels := targetGroup.Labels
for _, target := range targetGroup.Targets { for _, target := range targetGroup.Targets {
m := make(map[string]string, len(labels)) m := promutils.NewLabels(2 + labels.Len())
for k, v := range labels { m.AddFrom(labels)
m[k] = v m.Add("__address__", target)
} m.Add("__meta_url", sourceURL)
m["__address__"] = target m.RemoveDuplicates()
m["__meta_url"] = sourceURL
ms = append(ms, m) ms = append(ms, m)
} }
} }

View file

@ -1,11 +1,10 @@
package http package http
import ( import (
"reflect"
"testing" "testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
func Test_addHTTPTargetLabels(t *testing.T) { func Test_addHTTPTargetLabels(t *testing.T) {
@ -15,7 +14,7 @@ func Test_addHTTPTargetLabels(t *testing.T) {
tests := []struct { tests := []struct {
name string name string
args args args args
want [][]prompbmarshal.Label want []*promutils.Labels
}{ }{
{ {
name: "add ok", name: "add ok",
@ -23,18 +22,18 @@ func Test_addHTTPTargetLabels(t *testing.T) {
src: []httpGroupTarget{ src: []httpGroupTarget{
{ {
Targets: []string{"127.0.0.1:9100", "127.0.0.2:91001"}, Targets: []string{"127.0.0.1:9100", "127.0.0.2:91001"},
Labels: map[string]string{"__meta_kubernetes_pod": "pod-1", "__meta_consul_dc": "dc-2"}, Labels: promutils.NewLabelsFromMap(map[string]string{"__meta_kubernetes_pod": "pod-1", "__meta_consul_dc": "dc-2"}),
}, },
}, },
}, },
want: [][]prompbmarshal.Label{ want: []*promutils.Labels{
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "127.0.0.1:9100", "__address__": "127.0.0.1:9100",
"__meta_kubernetes_pod": "pod-1", "__meta_kubernetes_pod": "pod-1",
"__meta_consul_dc": "dc-2", "__meta_consul_dc": "dc-2",
"__meta_url": "http://foo.bar/baz?aaa=bb", "__meta_url": "http://foo.bar/baz?aaa=bb",
}), }),
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "127.0.0.2:91001", "__address__": "127.0.0.2:91001",
"__meta_kubernetes_pod": "pod-1", "__meta_kubernetes_pod": "pod-1",
"__meta_consul_dc": "dc-2", "__meta_consul_dc": "dc-2",
@ -46,13 +45,7 @@ func Test_addHTTPTargetLabels(t *testing.T) {
for _, tt := range tests { for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) { t.Run(tt.name, func(t *testing.T) {
got := addHTTPTargetLabels(tt.args.src, "http://foo.bar/baz?aaa=bb") got := addHTTPTargetLabels(tt.args.src, "http://foo.bar/baz?aaa=bb")
var sortedLabelss [][]prompbmarshal.Label discoveryutils.TestEqualLabelss(t, got, tt.want)
for _, labels := range got {
sortedLabelss = append(sortedLabelss, discoveryutils.GetSortedLabels(labels))
}
if !reflect.DeepEqual(sortedLabelss, tt.want) {
t.Errorf("addHTTPTargetLabels() \ngot \n%v\n, \nwant \n%v\n", sortedLabelss, tt.want)
}
}) })
} }
} }

View file

@ -19,6 +19,7 @@ import (
"github.com/VictoriaMetrics/VictoriaMetrics/lib/cgroup" "github.com/VictoriaMetrics/VictoriaMetrics/lib/cgroup"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger" "github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
"github.com/VictoriaMetrics/metrics" "github.com/VictoriaMetrics/metrics"
) )
@ -37,7 +38,7 @@ type object interface {
key() string key() string
// getTargetLabels must be called under gw.mu lock. // getTargetLabels must be called under gw.mu lock.
getTargetLabels(gw *groupWatcher) []map[string]string getTargetLabels(gw *groupWatcher) []*promutils.Labels
} }
// parseObjectFunc must parse object from the given data. // parseObjectFunc must parse object from the given data.
@ -136,8 +137,8 @@ func (aw *apiWatcher) updateScrapeWorks(uw *urlWatcher, swosByKey map[string][]i
aw.swosByURLWatcherLock.Unlock() aw.swosByURLWatcherLock.Unlock()
} }
func (aw *apiWatcher) setScrapeWorks(uw *urlWatcher, key string, labels []map[string]string) { func (aw *apiWatcher) setScrapeWorks(uw *urlWatcher, key string, labelss []*promutils.Labels) {
swos := getScrapeWorkObjectsForLabels(aw.swcFunc, labels) swos := getScrapeWorkObjectsForLabels(aw.swcFunc, labelss)
aw.swosByURLWatcherLock.Lock() aw.swosByURLWatcherLock.Lock()
swosByKey := aw.swosByURLWatcher[uw] swosByKey := aw.swosByURLWatcher[uw]
if swosByKey == nil { if swosByKey == nil {
@ -163,7 +164,7 @@ func (aw *apiWatcher) removeScrapeWorks(uw *urlWatcher, key string) {
aw.swosByURLWatcherLock.Unlock() aw.swosByURLWatcherLock.Unlock()
} }
func getScrapeWorkObjectsForLabels(swcFunc ScrapeWorkConstructorFunc, labelss []map[string]string) []interface{} { func getScrapeWorkObjectsForLabels(swcFunc ScrapeWorkConstructorFunc, labelss []*promutils.Labels) []interface{} {
// Do not pre-allocate swos, since it is likely the swos will be empty because of relabeling // Do not pre-allocate swos, since it is likely the swos will be empty because of relabeling
var swos []interface{} var swos []interface{}
for _, labels := range labelss { for _, labels := range labelss {
@ -299,24 +300,31 @@ func (gw *groupWatcher) getScrapeWorkObjectsByAPIWatcherLocked(objectsByKey map[
var wg sync.WaitGroup var wg sync.WaitGroup
limiterCh := make(chan struct{}, cgroup.AvailableCPUs()) limiterCh := make(chan struct{}, cgroup.AvailableCPUs())
for key, o := range objectsByKey { for key, o := range objectsByKey {
labels := o.getTargetLabels(gw) labelss := o.getTargetLabels(gw)
wg.Add(1) wg.Add(1)
limiterCh <- struct{}{} limiterCh <- struct{}{}
go func(key string, labels []map[string]string) { go func(key string, labelss []*promutils.Labels) {
for aw, e := range swosByAPIWatcher { for aw, e := range swosByAPIWatcher {
swos := getScrapeWorkObjectsForLabels(aw.swcFunc, labels) swos := getScrapeWorkObjectsForLabels(aw.swcFunc, labelss)
e.mu.Lock() e.mu.Lock()
e.swosByKey[key] = swos e.swosByKey[key] = swos
e.mu.Unlock() e.mu.Unlock()
} }
putLabelssToPool(labelss)
wg.Done() wg.Done()
<-limiterCh <-limiterCh
}(key, labels) }(key, labelss)
} }
wg.Wait() wg.Wait()
return swosByAPIWatcher return swosByAPIWatcher
} }
func putLabelssToPool(labelss []*promutils.Labels) {
for _, labels := range labelss {
promutils.PutLabels(labels)
}
}
func (gw *groupWatcher) getObjectByRoleLocked(role, namespace, name string) object { func (gw *groupWatcher) getObjectByRoleLocked(role, namespace, name string) object {
if role == "node" { if role == "node" {
// Node objects have no namespace // Node objects have no namespace
@ -764,10 +772,11 @@ func (uw *urlWatcher) updateObjectLocked(key string, o object) {
uw.objectsUpdated.Inc() uw.objectsUpdated.Inc()
} }
if len(uw.aws) > 0 { if len(uw.aws) > 0 {
labels := o.getTargetLabels(uw.gw) labelss := o.getTargetLabels(uw.gw)
for aw := range uw.aws { for aw := range uw.aws {
aw.setScrapeWorks(uw, key, labels) aw.setScrapeWorks(uw, key, labelss)
} }
putLabelssToPool(labelss)
} }
uw.maybeUpdateDependedScrapeWorksLocked() uw.maybeUpdateDependedScrapeWorksLocked()
} }

View file

@ -8,6 +8,8 @@ import (
"sync" "sync"
"testing" "testing"
"time" "time"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
func TestGetAPIPathsWithNamespaces(t *testing.T) { func TestGetAPIPathsWithNamespaces(t *testing.T) {
@ -919,10 +921,10 @@ func TestGetScrapeWorkObjects(t *testing.T) {
} }
testAPIServer := httptest.NewServer(mux) testAPIServer := httptest.NewServer(mux)
tc.sdc.APIServer = testAPIServer.URL tc.sdc.APIServer = testAPIServer.URL
ac, err := newAPIConfig(tc.sdc, "", func(metaLabels map[string]string) interface{} { ac, err := newAPIConfig(tc.sdc, "", func(metaLabels *promutils.Labels) interface{} {
var res []interface{} var res []interface{}
for k := range metaLabels { for _, label := range metaLabels.Labels {
res = append(res, k) res = append(res, label.Name)
} }
return res return res
}) })

View file

@ -1,7 +1,9 @@
package kubernetes package kubernetes
import ( import (
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
// ObjectMeta represents ObjectMeta from k8s API. // ObjectMeta represents ObjectMeta from k8s API.
@ -11,8 +13,8 @@ type ObjectMeta struct {
Name string Name string
Namespace string Namespace string
UID string UID string
Labels discoveryutils.SortedLabels Labels *promutils.Labels
Annotations discoveryutils.SortedLabels Annotations *promutils.Labels
OwnerReferences []OwnerReference OwnerReferences []OwnerReference
} }
@ -26,15 +28,38 @@ type ListMeta struct {
ResourceVersion string ResourceVersion string
} }
func (om *ObjectMeta) registerLabelsAndAnnotations(prefix string, m map[string]string) { func (om *ObjectMeta) registerLabelsAndAnnotations(prefix string, m *promutils.Labels) {
for _, lb := range om.Labels { bb := bbPool.Get()
m[discoveryutils.SanitizeLabelName(prefix+"_label_"+lb.Name)] = lb.Value b := bb.B
m[discoveryutils.SanitizeLabelName(prefix+"_labelpresent_"+lb.Name)] = "true" for _, lb := range om.Labels.GetLabels() {
b = appendThreeStrings(b[:0], prefix, "_label_", lb.Name)
labelName := bytesutil.ToUnsafeString(b)
m.Add(discoveryutils.SanitizeLabelName(labelName), lb.Value)
b = appendThreeStrings(b[:0], prefix, "_labelpresent_", lb.Name)
labelName = bytesutil.ToUnsafeString(b)
m.Add(discoveryutils.SanitizeLabelName(labelName), "true")
} }
for _, a := range om.Annotations { for _, a := range om.Annotations.GetLabels() {
m[discoveryutils.SanitizeLabelName(prefix+"_annotation_"+a.Name)] = a.Value b = appendThreeStrings(b[:0], prefix, "_annotation_", a.Name)
m[discoveryutils.SanitizeLabelName(prefix+"_annotationpresent_"+a.Name)] = "true" labelName := bytesutil.ToUnsafeString(b)
m.Add(discoveryutils.SanitizeLabelName(labelName), a.Value)
b = appendThreeStrings(b[:0], prefix, "_annotationpresent_", a.Name)
labelName = bytesutil.ToUnsafeString(b)
m.Add(discoveryutils.SanitizeLabelName(labelName), "true")
} }
bb.B = b
bbPool.Put(bb)
}
var bbPool bytesutil.ByteBufferPool
func appendThreeStrings(dst []byte, a, b, c string) []byte {
dst = append(dst, a...)
dst = append(dst, b...)
dst = append(dst, c...)
return dst
} }
// OwnerReference represents OwnerReferense from k8s API. // OwnerReference represents OwnerReferense from k8s API.

View file

@ -7,6 +7,7 @@ import (
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger" "github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
func (eps *Endpoints) key() string { func (eps *Endpoints) key() string {
@ -91,13 +92,13 @@ type EndpointPort struct {
// getTargetLabels returns labels for each endpoint in eps. // getTargetLabels returns labels for each endpoint in eps.
// //
// See https://prometheus.io/docs/prometheus/latest/configuration/configuration/#endpoints // See https://prometheus.io/docs/prometheus/latest/configuration/configuration/#endpoints
func (eps *Endpoints) getTargetLabels(gw *groupWatcher) []map[string]string { func (eps *Endpoints) getTargetLabels(gw *groupWatcher) []*promutils.Labels {
var svc *Service var svc *Service
if o := gw.getObjectByRoleLocked("service", eps.Metadata.Namespace, eps.Metadata.Name); o != nil { if o := gw.getObjectByRoleLocked("service", eps.Metadata.Namespace, eps.Metadata.Name); o != nil {
svc = o.(*Service) svc = o.(*Service)
} }
podPortsSeen := make(map[*Pod][]int) podPortsSeen := make(map[*Pod][]int)
var ms []map[string]string var ms []*promutils.Labels
for _, ess := range eps.Subsets { for _, ess := range eps.Subsets {
for _, epp := range ess.Ports { for _, epp := range ess.Ports {
ms = appendEndpointLabelsForAddresses(ms, gw, podPortsSeen, eps, ess.Addresses, epp, svc, "true") ms = appendEndpointLabelsForAddresses(ms, gw, podPortsSeen, eps, ess.Addresses, epp, svc, "true")
@ -106,7 +107,7 @@ func (eps *Endpoints) getTargetLabels(gw *groupWatcher) []map[string]string {
} }
// See https://kubernetes.io/docs/reference/labels-annotations-taints/#endpoints-kubernetes-io-over-capacity // See https://kubernetes.io/docs/reference/labels-annotations-taints/#endpoints-kubernetes-io-over-capacity
// and https://github.com/kubernetes/kubernetes/pull/99975 // and https://github.com/kubernetes/kubernetes/pull/99975
switch eps.Metadata.Annotations.GetByName("endpoints.kubernetes.io/over-capacity") { switch eps.Metadata.Annotations.Get("endpoints.kubernetes.io/over-capacity") {
case "truncated": case "truncated":
logger.Warnf(`the number of targets for "role: endpoints" %q exceeds 1000 and has been truncated; please use "role: endpointslice" instead`, eps.Metadata.key()) logger.Warnf(`the number of targets for "role: endpoints" %q exceeds 1000 and has been truncated; please use "role: endpointslice" instead`, eps.Metadata.key())
case "warning": case "warning":
@ -129,14 +130,14 @@ func (eps *Endpoints) getTargetLabels(gw *groupWatcher) []map[string]string {
continue continue
} }
addr := discoveryutils.JoinHostPort(p.Status.PodIP, cp.ContainerPort) addr := discoveryutils.JoinHostPort(p.Status.PodIP, cp.ContainerPort)
m := map[string]string{ m := promutils.GetLabels()
"__address__": addr, m.Add("__address__", addr)
}
p.appendCommonLabels(m, gw) p.appendCommonLabels(m, gw)
p.appendContainerLabels(m, c, &cp) p.appendContainerLabels(m, c, &cp)
if svc != nil { if svc != nil {
svc.appendCommonLabels(m) svc.appendCommonLabels(m)
} }
m.RemoveDuplicates()
ms = append(ms, m) ms = append(ms, m)
} }
} }
@ -144,8 +145,8 @@ func (eps *Endpoints) getTargetLabels(gw *groupWatcher) []map[string]string {
return ms return ms
} }
func appendEndpointLabelsForAddresses(ms []map[string]string, gw *groupWatcher, podPortsSeen map[*Pod][]int, eps *Endpoints, func appendEndpointLabelsForAddresses(ms []*promutils.Labels, gw *groupWatcher, podPortsSeen map[*Pod][]int, eps *Endpoints,
eas []EndpointAddress, epp EndpointPort, svc *Service, ready string) []map[string]string { eas []EndpointAddress, epp EndpointPort, svc *Service, ready string) []*promutils.Labels {
for _, ea := range eas { for _, ea := range eas {
var p *Pod var p *Pod
if ea.TargetRef.Name != "" { if ea.TargetRef.Name != "" {
@ -154,13 +155,14 @@ func appendEndpointLabelsForAddresses(ms []map[string]string, gw *groupWatcher,
} }
} }
m := getEndpointLabelsForAddressAndPort(gw, podPortsSeen, eps, ea, epp, p, svc, ready) m := getEndpointLabelsForAddressAndPort(gw, podPortsSeen, eps, ea, epp, p, svc, ready)
m.RemoveDuplicates()
ms = append(ms, m) ms = append(ms, m)
} }
return ms return ms
} }
func getEndpointLabelsForAddressAndPort(gw *groupWatcher, podPortsSeen map[*Pod][]int, eps *Endpoints, ea EndpointAddress, epp EndpointPort, func getEndpointLabelsForAddressAndPort(gw *groupWatcher, podPortsSeen map[*Pod][]int, eps *Endpoints, ea EndpointAddress, epp EndpointPort,
p *Pod, svc *Service, ready string) map[string]string { p *Pod, svc *Service, ready string) *promutils.Labels {
m := getEndpointLabels(eps.Metadata, ea, epp, ready) m := getEndpointLabels(eps.Metadata, ea, epp, ready)
if svc != nil { if svc != nil {
svc.appendCommonLabels(m) svc.appendCommonLabels(m)
@ -188,26 +190,24 @@ func getEndpointLabelsForAddressAndPort(gw *groupWatcher, podPortsSeen map[*Pod]
return m return m
} }
func getEndpointLabels(om ObjectMeta, ea EndpointAddress, epp EndpointPort, ready string) map[string]string { func getEndpointLabels(om ObjectMeta, ea EndpointAddress, epp EndpointPort, ready string) *promutils.Labels {
addr := discoveryutils.JoinHostPort(ea.IP, epp.Port) addr := discoveryutils.JoinHostPort(ea.IP, epp.Port)
m := map[string]string{ m := promutils.GetLabels()
"__address__": addr, m.Add("__address__", addr)
"__meta_kubernetes_namespace": om.Namespace, m.Add("__meta_kubernetes_namespace", om.Namespace)
"__meta_kubernetes_endpoints_name": om.Name, m.Add("__meta_kubernetes_endpoints_name", om.Name)
m.Add("__meta_kubernetes_endpoint_ready", ready)
"__meta_kubernetes_endpoint_ready": ready, m.Add("__meta_kubernetes_endpoint_port_name", epp.Name)
"__meta_kubernetes_endpoint_port_name": epp.Name, m.Add("__meta_kubernetes_endpoint_port_protocol", epp.Protocol)
"__meta_kubernetes_endpoint_port_protocol": epp.Protocol,
}
if ea.TargetRef.Kind != "" { if ea.TargetRef.Kind != "" {
m["__meta_kubernetes_endpoint_address_target_kind"] = ea.TargetRef.Kind m.Add("__meta_kubernetes_endpoint_address_target_kind", ea.TargetRef.Kind)
m["__meta_kubernetes_endpoint_address_target_name"] = ea.TargetRef.Name m.Add("__meta_kubernetes_endpoint_address_target_name", ea.TargetRef.Name)
} }
if ea.NodeName != "" { if ea.NodeName != "" {
m["__meta_kubernetes_endpoint_node_name"] = ea.NodeName m.Add("__meta_kubernetes_endpoint_node_name", ea.NodeName)
} }
if ea.Hostname != "" { if ea.Hostname != "" {
m["__meta_kubernetes_endpoint_hostname"] = ea.Hostname m.Add("__meta_kubernetes_endpoint_hostname", ea.Hostname)
} }
return m return m
} }

View file

@ -4,8 +4,7 @@ import (
"bytes" "bytes"
"testing" "testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
) )
func TestParseEndpointsListFailure(t *testing.T) { func TestParseEndpointsListFailure(t *testing.T) {
@ -91,8 +90,8 @@ func TestParseEndpointsListSuccess(t *testing.T) {
} }
sortedLabelss := getSortedLabelss(objectsByKey) sortedLabelss := getSortedLabelss(objectsByKey)
expectedLabelss := [][]prompbmarshal.Label{ expectedLabelss := []*promutils.Labels{
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "172.17.0.2:8443", "__address__": "172.17.0.2:8443",
"__meta_kubernetes_endpoint_address_target_kind": "Pod", "__meta_kubernetes_endpoint_address_target_kind": "Pod",
"__meta_kubernetes_endpoint_address_target_name": "coredns-6955765f44-lnp6t", "__meta_kubernetes_endpoint_address_target_name": "coredns-6955765f44-lnp6t",
@ -119,7 +118,7 @@ func TestGetEndpointsLabels(t *testing.T) {
containerPorts map[string][]ContainerPort containerPorts map[string][]ContainerPort
endpointPorts []EndpointPort endpointPorts []EndpointPort
} }
f := func(t *testing.T, args testArgs, wantLabels [][]prompbmarshal.Label) { f := func(t *testing.T, args testArgs, wantLabels []*promutils.Labels) {
t.Helper() t.Helper()
eps := Endpoints{ eps := Endpoints{
Metadata: ObjectMeta{ Metadata: ObjectMeta{
@ -175,12 +174,7 @@ func TestGetEndpointsLabels(t *testing.T) {
} }
node := Node{ node := Node{
Metadata: ObjectMeta{ Metadata: ObjectMeta{
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{"node-label": "xyz"}),
{
Name: "node-label",
Value: "xyz",
},
},
}, },
} }
for cn, ports := range args.containerPorts { for cn, ports := range args.containerPorts {
@ -212,10 +206,11 @@ func TestGetEndpointsLabels(t *testing.T) {
}, },
} }
gw.attachNodeMetadata = true gw.attachNodeMetadata = true
var sortedLabelss [][]prompbmarshal.Label var sortedLabelss []*promutils.Labels
gotLabels := eps.getTargetLabels(&gw) gotLabels := eps.getTargetLabels(&gw)
for _, lbs := range gotLabels { for _, lbs := range gotLabels {
sortedLabelss = append(sortedLabelss, discoveryutils.GetSortedLabels(lbs)) lbs.Sort()
sortedLabelss = append(sortedLabelss, lbs)
} }
if !areEqualLabelss(sortedLabelss, wantLabels) { if !areEqualLabelss(sortedLabelss, wantLabels) {
t.Fatalf("unexpected labels:\ngot\n%v\nwant\n%v", sortedLabelss, wantLabels) t.Fatalf("unexpected labels:\ngot\n%v\nwant\n%v", sortedLabelss, wantLabels)
@ -231,8 +226,8 @@ func TestGetEndpointsLabels(t *testing.T) {
Protocol: "foobar", Protocol: "foobar",
}, },
}, },
}, [][]prompbmarshal.Label{ }, []*promutils.Labels{
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "10.13.15.15:8081", "__address__": "10.13.15.15:8081",
"__meta_kubernetes_endpoint_address_target_kind": "Pod", "__meta_kubernetes_endpoint_address_target_kind": "Pod",
"__meta_kubernetes_endpoint_address_target_name": "test-pod", "__meta_kubernetes_endpoint_address_target_name": "test-pod",
@ -272,8 +267,8 @@ func TestGetEndpointsLabels(t *testing.T) {
Protocol: "https", Protocol: "https",
}, },
}, },
}, [][]prompbmarshal.Label{ }, []*promutils.Labels{
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "10.13.15.15:8081", "__address__": "10.13.15.15:8081",
"__meta_kubernetes_endpoint_address_target_kind": "Pod", "__meta_kubernetes_endpoint_address_target_kind": "Pod",
"__meta_kubernetes_endpoint_address_target_name": "test-pod", "__meta_kubernetes_endpoint_address_target_name": "test-pod",
@ -296,7 +291,7 @@ func TestGetEndpointsLabels(t *testing.T) {
"__meta_kubernetes_service_name": "test-eps", "__meta_kubernetes_service_name": "test-eps",
"__meta_kubernetes_service_type": "service-type", "__meta_kubernetes_service_type": "service-type",
}), }),
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "192.168.15.1:8428", "__address__": "192.168.15.1:8428",
"__meta_kubernetes_namespace": "default", "__meta_kubernetes_namespace": "default",
"__meta_kubernetes_node_label_node_label": "xyz", "__meta_kubernetes_node_label_node_label": "xyz",
@ -335,8 +330,8 @@ func TestGetEndpointsLabels(t *testing.T) {
Protocol: "xabc", Protocol: "xabc",
}, },
}, },
}, [][]prompbmarshal.Label{ }, []*promutils.Labels{
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "10.13.15.15:8428", "__address__": "10.13.15.15:8428",
"__meta_kubernetes_endpoint_address_target_kind": "Pod", "__meta_kubernetes_endpoint_address_target_kind": "Pod",
"__meta_kubernetes_endpoint_address_target_name": "test-pod", "__meta_kubernetes_endpoint_address_target_name": "test-pod",

View file

@ -7,6 +7,7 @@ import (
"strconv" "strconv"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
func (eps *EndpointSlice) key() string { func (eps *EndpointSlice) key() string {
@ -37,16 +38,16 @@ func parseEndpointSlice(data []byte) (object, error) {
// getTargetLabels returns labels for eps. // getTargetLabels returns labels for eps.
// //
// See https://prometheus.io/docs/prometheus/latest/configuration/configuration/#endpointslices // See https://prometheus.io/docs/prometheus/latest/configuration/configuration/#endpointslices
func (eps *EndpointSlice) getTargetLabels(gw *groupWatcher) []map[string]string { func (eps *EndpointSlice) getTargetLabels(gw *groupWatcher) []*promutils.Labels {
// The associated service name is stored in kubernetes.io/service-name label. // The associated service name is stored in kubernetes.io/service-name label.
// See https://kubernetes.io/docs/reference/labels-annotations-taints/#kubernetesioservice-name // See https://kubernetes.io/docs/reference/labels-annotations-taints/#kubernetesioservice-name
svcName := eps.Metadata.Labels.GetByName("kubernetes.io/service-name") svcName := eps.Metadata.Labels.Get("kubernetes.io/service-name")
var svc *Service var svc *Service
if o := gw.getObjectByRoleLocked("service", eps.Metadata.Namespace, svcName); o != nil { if o := gw.getObjectByRoleLocked("service", eps.Metadata.Namespace, svcName); o != nil {
svc = o.(*Service) svc = o.(*Service)
} }
podPortsSeen := make(map[*Pod][]int) podPortsSeen := make(map[*Pod][]int)
var ms []map[string]string var ms []*promutils.Labels
for _, ess := range eps.Endpoints { for _, ess := range eps.Endpoints {
var p *Pod var p *Pod
if o := gw.getObjectByRoleLocked("pod", ess.TargetRef.Namespace, ess.TargetRef.Name); o != nil { if o := gw.getObjectByRoleLocked("pod", ess.TargetRef.Namespace, ess.TargetRef.Name); o != nil {
@ -55,6 +56,7 @@ func (eps *EndpointSlice) getTargetLabels(gw *groupWatcher) []map[string]string
for _, epp := range eps.Ports { for _, epp := range eps.Ports {
for _, addr := range ess.Addresses { for _, addr := range ess.Addresses {
m := getEndpointSliceLabelsForAddressAndPort(gw, podPortsSeen, addr, eps, ess, epp, p, svc) m := getEndpointSliceLabelsForAddressAndPort(gw, podPortsSeen, addr, eps, ess, epp, p, svc)
m.RemoveDuplicates()
ms = append(ms, m) ms = append(ms, m)
} }
@ -77,14 +79,14 @@ func (eps *EndpointSlice) getTargetLabels(gw *groupWatcher) []map[string]string
continue continue
} }
addr := discoveryutils.JoinHostPort(p.Status.PodIP, cp.ContainerPort) addr := discoveryutils.JoinHostPort(p.Status.PodIP, cp.ContainerPort)
m := map[string]string{ m := promutils.GetLabels()
"__address__": addr, m.Add("__address__", addr)
}
p.appendCommonLabels(m, gw) p.appendCommonLabels(m, gw)
p.appendContainerLabels(m, c, &cp) p.appendContainerLabels(m, c, &cp)
if svc != nil { if svc != nil {
svc.appendCommonLabels(m) svc.appendCommonLabels(m)
} }
m.RemoveDuplicates()
ms = append(ms, m) ms = append(ms, m)
} }
} }
@ -98,7 +100,7 @@ func (eps *EndpointSlice) getTargetLabels(gw *groupWatcher) []map[string]string
// p appended to seen Ports // p appended to seen Ports
// if TargetRef matches // if TargetRef matches
func getEndpointSliceLabelsForAddressAndPort(gw *groupWatcher, podPortsSeen map[*Pod][]int, addr string, eps *EndpointSlice, ea Endpoint, epp EndpointPort, func getEndpointSliceLabelsForAddressAndPort(gw *groupWatcher, podPortsSeen map[*Pod][]int, addr string, eps *EndpointSlice, ea Endpoint, epp EndpointPort,
p *Pod, svc *Service) map[string]string { p *Pod, svc *Service) *promutils.Labels {
m := getEndpointSliceLabels(eps, addr, ea, epp) m := getEndpointSliceLabels(eps, addr, ea, epp)
if svc != nil { if svc != nil {
svc.appendCommonLabels(m) svc.appendCommonLabels(m)
@ -128,31 +130,30 @@ func getEndpointSliceLabelsForAddressAndPort(gw *groupWatcher, podPortsSeen map[
} }
// //getEndpointSliceLabels builds labels for given EndpointSlice // //getEndpointSliceLabels builds labels for given EndpointSlice
func getEndpointSliceLabels(eps *EndpointSlice, addr string, ea Endpoint, epp EndpointPort) map[string]string { func getEndpointSliceLabels(eps *EndpointSlice, addr string, ea Endpoint, epp EndpointPort) *promutils.Labels {
addr = discoveryutils.JoinHostPort(addr, epp.Port) addr = discoveryutils.JoinHostPort(addr, epp.Port)
m := map[string]string{ m := promutils.GetLabels()
"__address__": addr, m.Add("__address__", addr)
"__meta_kubernetes_namespace": eps.Metadata.Namespace, m.Add("__meta_kubernetes_namespace", eps.Metadata.Namespace)
"__meta_kubernetes_endpointslice_name": eps.Metadata.Name, m.Add("__meta_kubernetes_endpointslice_name", eps.Metadata.Name)
"__meta_kubernetes_endpointslice_address_type": eps.AddressType, m.Add("__meta_kubernetes_endpointslice_address_type", eps.AddressType)
"__meta_kubernetes_endpointslice_endpoint_conditions_ready": strconv.FormatBool(ea.Conditions.Ready), m.Add("__meta_kubernetes_endpointslice_endpoint_conditions_ready", strconv.FormatBool(ea.Conditions.Ready))
"__meta_kubernetes_endpointslice_port_name": epp.Name, m.Add("__meta_kubernetes_endpointslice_port_name", epp.Name)
"__meta_kubernetes_endpointslice_port_protocol": epp.Protocol, m.Add("__meta_kubernetes_endpointslice_port_protocol", epp.Protocol)
"__meta_kubernetes_endpointslice_port": strconv.Itoa(epp.Port), m.Add("__meta_kubernetes_endpointslice_port", strconv.Itoa(epp.Port))
}
if epp.AppProtocol != "" { if epp.AppProtocol != "" {
m["__meta_kubernetes_endpointslice_port_app_protocol"] = epp.AppProtocol m.Add("__meta_kubernetes_endpointslice_port_app_protocol", epp.AppProtocol)
} }
if ea.TargetRef.Kind != "" { if ea.TargetRef.Kind != "" {
m["__meta_kubernetes_endpointslice_address_target_kind"] = ea.TargetRef.Kind m.Add("__meta_kubernetes_endpointslice_address_target_kind", ea.TargetRef.Kind)
m["__meta_kubernetes_endpointslice_address_target_name"] = ea.TargetRef.Name m.Add("__meta_kubernetes_endpointslice_address_target_name", ea.TargetRef.Name)
} }
if ea.Hostname != "" { if ea.Hostname != "" {
m["__meta_kubernetes_endpointslice_endpoint_hostname"] = ea.Hostname m.Add("__meta_kubernetes_endpointslice_endpoint_hostname", ea.Hostname)
} }
for k, v := range ea.Topology { for k, v := range ea.Topology {
m[discoveryutils.SanitizeLabelName("__meta_kubernetes_endpointslice_endpoint_topology_"+k)] = v m.Add(discoveryutils.SanitizeLabelName("__meta_kubernetes_endpointslice_endpoint_topology_"+k), v)
m[discoveryutils.SanitizeLabelName("__meta_kubernetes_endpointslice_endpoint_topology_present_"+k)] = "true" m.Add(discoveryutils.SanitizeLabelName("__meta_kubernetes_endpointslice_endpoint_topology_present_"+k), "true")
} }
return m return m
} }

View file

@ -4,8 +4,7 @@ import (
"bytes" "bytes"
"testing" "testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
) )
func TestParseEndpointSliceListFail(t *testing.T) { func TestParseEndpointSliceListFail(t *testing.T) {
@ -165,8 +164,8 @@ func TestParseEndpointSliceListSuccess(t *testing.T) {
t.Fatalf("unexpected resource version; got %s; want %s", meta.ResourceVersion, expectedResourceVersion) t.Fatalf("unexpected resource version; got %s; want %s", meta.ResourceVersion, expectedResourceVersion)
} }
sortedLabelss := getSortedLabelss(objectsByKey) sortedLabelss := getSortedLabelss(objectsByKey)
expectedLabelss := [][]prompbmarshal.Label{ expectedLabelss := []*promutils.Labels{
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "10.244.0.3:53", "__address__": "10.244.0.3:53",
"__meta_kubernetes_endpointslice_address_target_kind": "Pod", "__meta_kubernetes_endpointslice_address_target_kind": "Pod",
"__meta_kubernetes_endpointslice_address_target_name": "coredns-66bff467f8-z8czk", "__meta_kubernetes_endpointslice_address_target_name": "coredns-66bff467f8-z8czk",
@ -186,7 +185,7 @@ func TestParseEndpointSliceListSuccess(t *testing.T) {
"__meta_kubernetes_endpointslice_port_protocol": "UDP", "__meta_kubernetes_endpointslice_port_protocol": "UDP",
"__meta_kubernetes_namespace": "kube-system", "__meta_kubernetes_namespace": "kube-system",
}), }),
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "10.244.0.3:9153", "__address__": "10.244.0.3:9153",
"__meta_kubernetes_endpointslice_address_target_kind": "Pod", "__meta_kubernetes_endpointslice_address_target_kind": "Pod",
"__meta_kubernetes_endpointslice_address_target_name": "coredns-66bff467f8-z8czk", "__meta_kubernetes_endpointslice_address_target_name": "coredns-66bff467f8-z8czk",
@ -206,7 +205,7 @@ func TestParseEndpointSliceListSuccess(t *testing.T) {
"__meta_kubernetes_endpointslice_port_protocol": "TCP", "__meta_kubernetes_endpointslice_port_protocol": "TCP",
"__meta_kubernetes_namespace": "kube-system", "__meta_kubernetes_namespace": "kube-system",
}), }),
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "172.18.0.2:6443", "__address__": "172.18.0.2:6443",
"__meta_kubernetes_endpointslice_address_type": "IPv4", "__meta_kubernetes_endpointslice_address_type": "IPv4",
"__meta_kubernetes_endpointslice_endpoint_conditions_ready": "true", "__meta_kubernetes_endpointslice_endpoint_conditions_ready": "true",
@ -230,13 +229,13 @@ func TestGetEndpointsliceLabels(t *testing.T) {
containerPorts map[string][]ContainerPort containerPorts map[string][]ContainerPort
endpointPorts []EndpointPort endpointPorts []EndpointPort
} }
f := func(t *testing.T, args testArgs, wantLabels [][]prompbmarshal.Label) { f := func(t *testing.T, args testArgs, wantLabels []*promutils.Labels) {
t.Helper() t.Helper()
eps := EndpointSlice{ eps := EndpointSlice{
Metadata: ObjectMeta{ Metadata: ObjectMeta{
Name: "test-eps", Name: "test-eps",
Namespace: "default", Namespace: "default",
Labels: discoveryutils.GetSortedLabels(map[string]string{ Labels: promutils.NewLabelsFromMap(map[string]string{
"kubernetes.io/service-name": "test-svc", "kubernetes.io/service-name": "test-svc",
}), }),
}, },
@ -295,12 +294,7 @@ func TestGetEndpointsliceLabels(t *testing.T) {
} }
node := Node{ node := Node{
Metadata: ObjectMeta{ Metadata: ObjectMeta{
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{"node-label": "xyz"}),
{
Name: "node-label",
Value: "xyz",
},
},
}, },
} }
for cn, ports := range args.containerPorts { for cn, ports := range args.containerPorts {
@ -332,10 +326,11 @@ func TestGetEndpointsliceLabels(t *testing.T) {
}, },
} }
gw.attachNodeMetadata = true gw.attachNodeMetadata = true
var sortedLabelss [][]prompbmarshal.Label var sortedLabelss []*promutils.Labels
gotLabels := eps.getTargetLabels(&gw) gotLabels := eps.getTargetLabels(&gw)
for _, lbs := range gotLabels { for _, lbs := range gotLabels {
sortedLabelss = append(sortedLabelss, discoveryutils.GetSortedLabels(lbs)) lbs.Sort()
sortedLabelss = append(sortedLabelss, lbs)
} }
if !areEqualLabelss(sortedLabelss, wantLabels) { if !areEqualLabelss(sortedLabelss, wantLabels) {
t.Fatalf("unexpected labels:\ngot\n%v\nwant\n%v", sortedLabelss, wantLabels) t.Fatalf("unexpected labels:\ngot\n%v\nwant\n%v", sortedLabelss, wantLabels)
@ -351,8 +346,8 @@ func TestGetEndpointsliceLabels(t *testing.T) {
Protocol: "foobar", Protocol: "foobar",
}, },
}, },
}, [][]prompbmarshal.Label{ }, []*promutils.Labels{
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "10.13.15.15:8081", "__address__": "10.13.15.15:8081",
"__meta_kubernetes_endpointslice_address_target_kind": "Pod", "__meta_kubernetes_endpointslice_address_target_kind": "Pod",
"__meta_kubernetes_endpointslice_address_target_name": "test-pod", "__meta_kubernetes_endpointslice_address_target_name": "test-pod",
@ -399,8 +394,8 @@ func TestGetEndpointsliceLabels(t *testing.T) {
Protocol: "https", Protocol: "https",
}, },
}, },
}, [][]prompbmarshal.Label{ }, []*promutils.Labels{
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "10.13.15.15:8081", "__address__": "10.13.15.15:8081",
"__meta_kubernetes_endpointslice_address_target_kind": "Pod", "__meta_kubernetes_endpointslice_address_target_kind": "Pod",
"__meta_kubernetes_endpointslice_address_target_name": "test-pod", "__meta_kubernetes_endpointslice_address_target_name": "test-pod",
@ -430,7 +425,7 @@ func TestGetEndpointsliceLabels(t *testing.T) {
"__meta_kubernetes_service_name": "test-svc", "__meta_kubernetes_service_name": "test-svc",
"__meta_kubernetes_service_type": "service-type", "__meta_kubernetes_service_type": "service-type",
}), }),
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "192.168.15.1:8428", "__address__": "192.168.15.1:8428",
"__meta_kubernetes_namespace": "default", "__meta_kubernetes_namespace": "default",
"__meta_kubernetes_node_label_node_label": "xyz", "__meta_kubernetes_node_label_node_label": "xyz",
@ -469,8 +464,8 @@ func TestGetEndpointsliceLabels(t *testing.T) {
Protocol: "xabc", Protocol: "xabc",
}, },
}, },
}, [][]prompbmarshal.Label{ }, []*promutils.Labels{
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "10.13.15.15:8428", "__address__": "10.13.15.15:8428",
"__meta_kubernetes_endpointslice_address_target_kind": "Pod", "__meta_kubernetes_endpointslice_address_target_kind": "Pod",
"__meta_kubernetes_endpointslice_address_target_name": "test-pod", "__meta_kubernetes_endpointslice_address_target_name": "test-pod",

View file

@ -5,6 +5,8 @@ import (
"fmt" "fmt"
"io" "io"
"strings" "strings"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
func (ig *Ingress) key() string { func (ig *Ingress) key() string {
@ -89,8 +91,8 @@ type HTTPIngressPath struct {
// getTargetLabels returns labels for ig. // getTargetLabels returns labels for ig.
// //
// See https://prometheus.io/docs/prometheus/latest/configuration/configuration/#ingress // See https://prometheus.io/docs/prometheus/latest/configuration/configuration/#ingress
func (ig *Ingress) getTargetLabels(gw *groupWatcher) []map[string]string { func (ig *Ingress) getTargetLabels(gw *groupWatcher) []*promutils.Labels {
var ms []map[string]string var ms []*promutils.Labels
for _, r := range ig.Spec.Rules { for _, r := range ig.Spec.Rules {
paths := getIngressRulePaths(r.HTTP.Paths) paths := getIngressRulePaths(r.HTTP.Paths)
scheme := getSchemeForHost(r.Host, ig.Spec.TLS) scheme := getSchemeForHost(r.Host, ig.Spec.TLS)
@ -129,16 +131,15 @@ func matchesHostPattern(pattern, host string) bool {
return pattern == host return pattern == host
} }
func getLabelsForIngressPath(ig *Ingress, scheme, host, path string) map[string]string { func getLabelsForIngressPath(ig *Ingress, scheme, host, path string) *promutils.Labels {
m := map[string]string{ m := promutils.GetLabels()
"__address__": host, m.Add("__address__", host)
"__meta_kubernetes_namespace": ig.Metadata.Namespace, m.Add("__meta_kubernetes_namespace", ig.Metadata.Namespace)
"__meta_kubernetes_ingress_name": ig.Metadata.Name, m.Add("__meta_kubernetes_ingress_name", ig.Metadata.Name)
"__meta_kubernetes_ingress_scheme": scheme, m.Add("__meta_kubernetes_ingress_scheme", scheme)
"__meta_kubernetes_ingress_host": host, m.Add("__meta_kubernetes_ingress_host", host)
"__meta_kubernetes_ingress_path": path, m.Add("__meta_kubernetes_ingress_path", path)
"__meta_kubernetes_ingress_class_name": ig.Spec.IngressClassName, m.Add("__meta_kubernetes_ingress_class_name", ig.Spec.IngressClassName)
}
ig.Metadata.registerLabelsAndAnnotations("__meta_kubernetes_ingress", m) ig.Metadata.registerLabelsAndAnnotations("__meta_kubernetes_ingress", m)
return m return m
} }

View file

@ -4,8 +4,7 @@ import (
"bytes" "bytes"
"testing" "testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
) )
func TestMatchesHostPattern(t *testing.T) { func TestMatchesHostPattern(t *testing.T) {
@ -103,8 +102,8 @@ func TestParseIngressListSuccess(t *testing.T) {
t.Fatalf("unexpected resource version; got %s; want %s", meta.ResourceVersion, expectedResourceVersion) t.Fatalf("unexpected resource version; got %s; want %s", meta.ResourceVersion, expectedResourceVersion)
} }
sortedLabelss := getSortedLabelss(objectsByKey) sortedLabelss := getSortedLabelss(objectsByKey)
expectedLabelss := [][]prompbmarshal.Label{ expectedLabelss := []*promutils.Labels{
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "foobar", "__address__": "foobar",
"__meta_kubernetes_ingress_annotation_kubectl_kubernetes_io_last_applied_configuration": `{"apiVersion":"networking.k8s.io/v1","kind":"Ingress","metadata":{"annotations":{},"name":"test-ingress","namespace":"default"},"spec":{"backend":{"serviceName":"testsvc","servicePort":80}}}` + "\n", "__meta_kubernetes_ingress_annotation_kubectl_kubernetes_io_last_applied_configuration": `{"apiVersion":"networking.k8s.io/v1","kind":"Ingress","metadata":{"annotations":{},"name":"test-ingress","namespace":"default"},"spec":{"backend":{"serviceName":"testsvc","servicePort":80}}}` + "\n",
"__meta_kubernetes_ingress_annotationpresent_kubectl_kubernetes_io_last_applied_configuration": "true", "__meta_kubernetes_ingress_annotationpresent_kubectl_kubernetes_io_last_applied_configuration": "true",

View file

@ -6,6 +6,7 @@ import (
"time" "time"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/proxy" "github.com/VictoriaMetrics/VictoriaMetrics/lib/proxy"
) )
@ -69,7 +70,7 @@ type Selector struct {
} }
// ScrapeWorkConstructorFunc must construct ScrapeWork object for the given metaLabels. // ScrapeWorkConstructorFunc must construct ScrapeWork object for the given metaLabels.
type ScrapeWorkConstructorFunc func(metaLabels map[string]string) interface{} type ScrapeWorkConstructorFunc func(metaLabels *promutils.Labels) interface{}
// GetScrapeWorkObjects returns ScrapeWork objects for the given sdc. // GetScrapeWorkObjects returns ScrapeWork objects for the given sdc.
// //

View file

@ -6,6 +6,7 @@ import (
"io" "io"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
// getNodesLabels returns labels for k8s nodes obtained from the given cfg // getNodesLabels returns labels for k8s nodes obtained from the given cfg
@ -84,19 +85,18 @@ type NodeDaemonEndpoints struct {
// getTargetLabels returs labels for the given n. // getTargetLabels returs labels for the given n.
// //
// See https://prometheus.io/docs/prometheus/latest/configuration/configuration/#node // See https://prometheus.io/docs/prometheus/latest/configuration/configuration/#node
func (n *Node) getTargetLabels(gw *groupWatcher) []map[string]string { func (n *Node) getTargetLabels(gw *groupWatcher) []*promutils.Labels {
addr := getNodeAddr(n.Status.Addresses) addr := getNodeAddr(n.Status.Addresses)
if len(addr) == 0 { if len(addr) == 0 {
// Skip node without address // Skip node without address
return nil return nil
} }
addr = discoveryutils.JoinHostPort(addr, n.Status.DaemonEndpoints.KubeletEndpoint.Port) addr = discoveryutils.JoinHostPort(addr, n.Status.DaemonEndpoints.KubeletEndpoint.Port)
m := map[string]string{ m := promutils.GetLabels()
"__address__": addr, m.Add("__address__", addr)
"instance": n.Metadata.Name, m.Add("instance", n.Metadata.Name)
"__meta_kubernetes_node_name": n.Metadata.Name, m.Add("__meta_kubernetes_node_name", n.Metadata.Name)
"__meta_kubernetes_node_provider_id": n.Spec.ProviderID, m.Add("__meta_kubernetes_node_provider_id", n.Spec.ProviderID)
}
n.Metadata.registerLabelsAndAnnotations("__meta_kubernetes_node", m) n.Metadata.registerLabelsAndAnnotations("__meta_kubernetes_node", m)
addrTypesUsed := make(map[string]bool, len(n.Status.Addresses)) addrTypesUsed := make(map[string]bool, len(n.Status.Addresses))
for _, a := range n.Status.Addresses { for _, a := range n.Status.Addresses {
@ -104,9 +104,9 @@ func (n *Node) getTargetLabels(gw *groupWatcher) []map[string]string {
continue continue
} }
addrTypesUsed[a.Type] = true addrTypesUsed[a.Type] = true
m[discoveryutils.SanitizeLabelName("__meta_kubernetes_node_address_"+a.Type)] = a.Address m.Add(discoveryutils.SanitizeLabelName("__meta_kubernetes_node_address_"+a.Type), a.Address)
} }
return []map[string]string{m} return []*promutils.Labels{m}
} }
func getNodeAddr(nas []NodeAddress) string { func getNodeAddr(nas []NodeAddress) string {

View file

@ -7,8 +7,7 @@ import (
"strconv" "strconv"
"testing" "testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
) )
func TestParseNodeListFailure(t *testing.T) { func TestParseNodeListFailure(t *testing.T) {
@ -242,8 +241,8 @@ func TestParseNodeListSuccess(t *testing.T) {
t.Fatalf("unexpected resource version; got %s; want %s", meta.ResourceVersion, expectedResourceVersion) t.Fatalf("unexpected resource version; got %s; want %s", meta.ResourceVersion, expectedResourceVersion)
} }
sortedLabelss := getSortedLabelss(objectsByKey) sortedLabelss := getSortedLabelss(objectsByKey)
expectedLabelss := [][]prompbmarshal.Label{ expectedLabelss := []*promutils.Labels{
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"instance": "m01", "instance": "m01",
"__address__": "172.17.0.2:10250", "__address__": "172.17.0.2:10250",
"__meta_kubernetes_node_name": "m01", "__meta_kubernetes_node_name": "m01",
@ -288,13 +287,14 @@ func TestParseNodeListSuccess(t *testing.T) {
} }
} }
func getSortedLabelss(objectsByKey map[string]object) [][]prompbmarshal.Label { func getSortedLabelss(objectsByKey map[string]object) []*promutils.Labels {
gw := newTestGroupWatcher() gw := newTestGroupWatcher()
var result [][]prompbmarshal.Label var result []*promutils.Labels
for _, o := range objectsByKey { for _, o := range objectsByKey {
labelss := o.getTargetLabels(gw) labelss := o.getTargetLabels(gw)
for _, labels := range labelss { for _, labels := range labelss {
result = append(result, discoveryutils.GetSortedLabels(labels)) labels.Sort()
result = append(result, labels)
} }
} }
return result return result
@ -308,12 +308,7 @@ func newTestGroupWatcher() *groupWatcher {
objectsByKey: map[string]object{ objectsByKey: map[string]object{
"/test-node": &Node{ "/test-node": &Node{
Metadata: ObjectMeta{ Metadata: ObjectMeta{
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{"node-label": "xyz"}),
{
Name: "node-label",
Value: "xyz",
},
},
}, },
}, },
}, },
@ -323,21 +318,21 @@ func newTestGroupWatcher() *groupWatcher {
return &gw return &gw
} }
func areEqualLabelss(a, b [][]prompbmarshal.Label) bool { func areEqualLabelss(a, b []*promutils.Labels) bool {
sortLabelss(a) sortLabelss(a)
sortLabelss(b) sortLabelss(b)
return reflect.DeepEqual(a, b) return reflect.DeepEqual(a, b)
} }
func sortLabelss(a [][]prompbmarshal.Label) { func sortLabelss(a []*promutils.Labels) {
sort.Slice(a, func(i, j int) bool { sort.Slice(a, func(i, j int) bool {
return marshalLabels(a[i]) < marshalLabels(a[j]) return marshalLabels(a[i]) < marshalLabels(a[j])
}) })
} }
func marshalLabels(a []prompbmarshal.Label) string { func marshalLabels(a *promutils.Labels) string {
var b []byte var b []byte
for _, label := range a { for _, label := range a.Labels {
b = strconv.AppendQuote(b, label.Name) b = strconv.AppendQuote(b, label.Name)
b = append(b, ':') b = append(b, ':')
b = strconv.AppendQuote(b, label.Value) b = strconv.AppendQuote(b, label.Value)

View file

@ -4,10 +4,11 @@ import (
"encoding/json" "encoding/json"
"fmt" "fmt"
"io" "io"
"strconv"
"strings" "strings"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
func (p *Pod) key() string { func (p *Pod) key() string {
@ -98,18 +99,18 @@ type PodCondition struct {
// getTargetLabels returns labels for each port of the given p. // getTargetLabels returns labels for each port of the given p.
// //
// See https://prometheus.io/docs/prometheus/latest/configuration/configuration/#pod // See https://prometheus.io/docs/prometheus/latest/configuration/configuration/#pod
func (p *Pod) getTargetLabels(gw *groupWatcher) []map[string]string { func (p *Pod) getTargetLabels(gw *groupWatcher) []*promutils.Labels {
if len(p.Status.PodIP) == 0 { if len(p.Status.PodIP) == 0 {
// Skip pod without IP // Skip pod without IP
return nil return nil
} }
var ms []map[string]string var ms []*promutils.Labels
ms = appendPodLabels(ms, gw, p, p.Spec.Containers, "false") ms = appendPodLabels(ms, gw, p, p.Spec.Containers, "false")
ms = appendPodLabels(ms, gw, p, p.Spec.InitContainers, "true") ms = appendPodLabels(ms, gw, p, p.Spec.InitContainers, "true")
return ms return ms
} }
func appendPodLabels(ms []map[string]string, gw *groupWatcher, p *Pod, cs []Container, isInit string) []map[string]string { func appendPodLabels(ms []*promutils.Labels, gw *groupWatcher, p *Pod, cs []Container, isInit string) []*promutils.Labels {
for _, c := range cs { for _, c := range cs {
for _, cp := range c.Ports { for _, cp := range c.Ports {
ms = appendPodLabelsInternal(ms, gw, p, c, &cp, isInit) ms = appendPodLabelsInternal(ms, gw, p, c, &cp, isInit)
@ -121,53 +122,52 @@ func appendPodLabels(ms []map[string]string, gw *groupWatcher, p *Pod, cs []Cont
return ms return ms
} }
func appendPodLabelsInternal(ms []map[string]string, gw *groupWatcher, p *Pod, c Container, cp *ContainerPort, isInit string) []map[string]string { func appendPodLabelsInternal(ms []*promutils.Labels, gw *groupWatcher, p *Pod, c Container, cp *ContainerPort, isInit string) []*promutils.Labels {
addr := p.Status.PodIP addr := p.Status.PodIP
if cp != nil { if cp != nil {
addr = discoveryutils.JoinHostPort(addr, cp.ContainerPort) addr = discoveryutils.JoinHostPort(addr, cp.ContainerPort)
} }
m := map[string]string{ m := promutils.GetLabels()
"__address__": addr, m.Add("__address__", addr)
"__meta_kubernetes_pod_container_init": isInit, m.Add("__meta_kubernetes_pod_container_init", isInit)
}
p.appendCommonLabels(m, gw) p.appendCommonLabels(m, gw)
p.appendContainerLabels(m, c, cp) p.appendContainerLabels(m, c, cp)
return append(ms, m) return append(ms, m)
} }
func (p *Pod) appendContainerLabels(m map[string]string, c Container, cp *ContainerPort) { func (p *Pod) appendContainerLabels(m *promutils.Labels, c Container, cp *ContainerPort) {
m["__meta_kubernetes_pod_container_image"] = c.Image m.Add("__meta_kubernetes_pod_container_image", c.Image)
m["__meta_kubernetes_pod_container_name"] = c.Name m.Add("__meta_kubernetes_pod_container_name", c.Name)
if cp != nil { if cp != nil {
m["__meta_kubernetes_pod_container_port_name"] = cp.Name m.Add("__meta_kubernetes_pod_container_port_name", cp.Name)
m["__meta_kubernetes_pod_container_port_number"] = strconv.Itoa(cp.ContainerPort) m.Add("__meta_kubernetes_pod_container_port_number", bytesutil.Itoa(cp.ContainerPort))
m["__meta_kubernetes_pod_container_port_protocol"] = cp.Protocol m.Add("__meta_kubernetes_pod_container_port_protocol", cp.Protocol)
} }
} }
func (p *Pod) appendCommonLabels(m map[string]string, gw *groupWatcher) { func (p *Pod) appendCommonLabels(m *promutils.Labels, gw *groupWatcher) {
if gw.attachNodeMetadata { if gw.attachNodeMetadata {
m["__meta_kubernetes_node_name"] = p.Spec.NodeName m.Add("__meta_kubernetes_node_name", p.Spec.NodeName)
o := gw.getObjectByRoleLocked("node", p.Metadata.Namespace, p.Spec.NodeName) o := gw.getObjectByRoleLocked("node", p.Metadata.Namespace, p.Spec.NodeName)
if o != nil { if o != nil {
n := o.(*Node) n := o.(*Node)
n.Metadata.registerLabelsAndAnnotations("__meta_kubernetes_node", m) n.Metadata.registerLabelsAndAnnotations("__meta_kubernetes_node", m)
} }
} }
m["__meta_kubernetes_pod_name"] = p.Metadata.Name m.Add("__meta_kubernetes_pod_name", p.Metadata.Name)
m["__meta_kubernetes_pod_ip"] = p.Status.PodIP m.Add("__meta_kubernetes_pod_ip", p.Status.PodIP)
m["__meta_kubernetes_pod_ready"] = getPodReadyStatus(p.Status.Conditions) m.Add("__meta_kubernetes_pod_ready", getPodReadyStatus(p.Status.Conditions))
m["__meta_kubernetes_pod_phase"] = p.Status.Phase m.Add("__meta_kubernetes_pod_phase", p.Status.Phase)
m["__meta_kubernetes_pod_node_name"] = p.Spec.NodeName m.Add("__meta_kubernetes_pod_node_name", p.Spec.NodeName)
m["__meta_kubernetes_pod_host_ip"] = p.Status.HostIP m.Add("__meta_kubernetes_pod_host_ip", p.Status.HostIP)
m["__meta_kubernetes_pod_uid"] = p.Metadata.UID m.Add("__meta_kubernetes_pod_uid", p.Metadata.UID)
m["__meta_kubernetes_namespace"] = p.Metadata.Namespace m.Add("__meta_kubernetes_namespace", p.Metadata.Namespace)
if pc := getPodController(p.Metadata.OwnerReferences); pc != nil { if pc := getPodController(p.Metadata.OwnerReferences); pc != nil {
if pc.Kind != "" { if pc.Kind != "" {
m["__meta_kubernetes_pod_controller_kind"] = pc.Kind m.Add("__meta_kubernetes_pod_controller_kind", pc.Kind)
} }
if pc.Name != "" { if pc.Name != "" {
m["__meta_kubernetes_pod_controller_name"] = pc.Name m.Add("__meta_kubernetes_pod_controller_name", pc.Name)
} }
} }
p.Metadata.registerLabelsAndAnnotations("__meta_kubernetes_pod", m) p.Metadata.registerLabelsAndAnnotations("__meta_kubernetes_pod", m)
@ -185,8 +185,10 @@ func getPodController(ors []OwnerReference) *OwnerReference {
func getPodReadyStatus(conds []PodCondition) string { func getPodReadyStatus(conds []PodCondition) string {
for _, c := range conds { for _, c := range conds {
if c.Type == "Ready" { if c.Type == "Ready" {
return strings.ToLower(c.Status) return toLowerConverter.Transform(c.Status)
} }
} }
return "unknown" return "unknown"
} }
var toLowerConverter = bytesutil.NewFastStringTransformer(strings.ToLower)

View file

@ -4,8 +4,7 @@ import (
"bytes" "bytes"
"testing" "testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
) )
func TestParsePodListFailure(t *testing.T) { func TestParsePodListFailure(t *testing.T) {
@ -240,8 +239,8 @@ func TestParsePodListSuccess(t *testing.T) {
t.Fatalf("unexpected resource version; got %s; want %s", meta.ResourceVersion, expectedResourceVersion) t.Fatalf("unexpected resource version; got %s; want %s", meta.ResourceVersion, expectedResourceVersion)
} }
sortedLabelss := getSortedLabelss(objectsByKey) sortedLabelss := getSortedLabelss(objectsByKey)
expectedLabelss := [][]prompbmarshal.Label{ expectedLabelss := []*promutils.Labels{
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "172.17.0.2:1234", "__address__": "172.17.0.2:1234",
"__meta_kubernetes_namespace": "kube-system", "__meta_kubernetes_namespace": "kube-system",

View file

@ -29,6 +29,7 @@ func BenchmarkPodGetTargetLabels(b *testing.B) {
if len(labelss) != 1 { if len(labelss) != 1 {
panic(fmt.Errorf("BUG: unexpected number of labelss returned: %d; want 1", len(labelss))) panic(fmt.Errorf("BUG: unexpected number of labelss returned: %d; want 1", len(labelss)))
} }
putLabelssToPool(labelss)
} }
}) })
} }

View file

@ -7,6 +7,7 @@ import (
"strconv" "strconv"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
func (s *Service) key() string { func (s *Service) key() string {
@ -72,31 +73,30 @@ type ServicePort struct {
// getTargetLabels returns labels for each port of the given s. // getTargetLabels returns labels for each port of the given s.
// //
// See https://prometheus.io/docs/prometheus/latest/configuration/configuration/#service // See https://prometheus.io/docs/prometheus/latest/configuration/configuration/#service
func (s *Service) getTargetLabels(gw *groupWatcher) []map[string]string { func (s *Service) getTargetLabels(gw *groupWatcher) []*promutils.Labels {
host := fmt.Sprintf("%s.%s.svc", s.Metadata.Name, s.Metadata.Namespace) host := fmt.Sprintf("%s.%s.svc", s.Metadata.Name, s.Metadata.Namespace)
var ms []map[string]string var ms []*promutils.Labels
for _, sp := range s.Spec.Ports { for _, sp := range s.Spec.Ports {
addr := discoveryutils.JoinHostPort(host, sp.Port) addr := discoveryutils.JoinHostPort(host, sp.Port)
m := map[string]string{ m := promutils.GetLabels()
"__address__": addr, m.Add("__address__", addr)
"__meta_kubernetes_service_port_name": sp.Name, m.Add("__meta_kubernetes_service_port_name", sp.Name)
"__meta_kubernetes_service_port_number": strconv.Itoa(sp.Port), m.Add("__meta_kubernetes_service_port_number", strconv.Itoa(sp.Port))
"__meta_kubernetes_service_port_protocol": sp.Protocol, m.Add("__meta_kubernetes_service_port_protocol", sp.Protocol)
}
s.appendCommonLabels(m) s.appendCommonLabels(m)
ms = append(ms, m) ms = append(ms, m)
} }
return ms return ms
} }
func (s *Service) appendCommonLabels(m map[string]string) { func (s *Service) appendCommonLabels(m *promutils.Labels) {
m["__meta_kubernetes_namespace"] = s.Metadata.Namespace m.Add("__meta_kubernetes_namespace", s.Metadata.Namespace)
m["__meta_kubernetes_service_name"] = s.Metadata.Name m.Add("__meta_kubernetes_service_name", s.Metadata.Name)
m["__meta_kubernetes_service_type"] = s.Spec.Type m.Add("__meta_kubernetes_service_type", s.Spec.Type)
if s.Spec.Type != "ExternalName" { if s.Spec.Type != "ExternalName" {
m["__meta_kubernetes_service_cluster_ip"] = s.Spec.ClusterIP m.Add("__meta_kubernetes_service_cluster_ip", s.Spec.ClusterIP)
} else { } else {
m["__meta_kubernetes_service_external_name"] = s.Spec.ExternalName m.Add("__meta_kubernetes_service_external_name", s.Spec.ExternalName)
} }
s.Metadata.registerLabelsAndAnnotations("__meta_kubernetes_service", m) s.Metadata.registerLabelsAndAnnotations("__meta_kubernetes_service", m)
} }

View file

@ -4,8 +4,7 @@ import (
"bytes" "bytes"
"testing" "testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
) )
func TestParseServiceListFailure(t *testing.T) { func TestParseServiceListFailure(t *testing.T) {
@ -100,8 +99,8 @@ func TestParseServiceListSuccess(t *testing.T) {
t.Fatalf("unexpected resource version; got %s; want %s", meta.ResourceVersion, expectedResourceVersion) t.Fatalf("unexpected resource version; got %s; want %s", meta.ResourceVersion, expectedResourceVersion)
} }
sortedLabelss := getSortedLabelss(objectsByKey) sortedLabelss := getSortedLabelss(objectsByKey)
expectedLabelss := [][]prompbmarshal.Label{ expectedLabelss := []*promutils.Labels{
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "kube-dns.kube-system.svc:53", "__address__": "kube-dns.kube-system.svc:53",
"__meta_kubernetes_namespace": "kube-system", "__meta_kubernetes_namespace": "kube-system",
"__meta_kubernetes_service_name": "kube-dns", "__meta_kubernetes_service_name": "kube-dns",
@ -125,7 +124,7 @@ func TestParseServiceListSuccess(t *testing.T) {
"__meta_kubernetes_service_annotationpresent_prometheus_io_port": "true", "__meta_kubernetes_service_annotationpresent_prometheus_io_port": "true",
"__meta_kubernetes_service_annotationpresent_prometheus_io_scrape": "true", "__meta_kubernetes_service_annotationpresent_prometheus_io_scrape": "true",
}), }),
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "kube-dns.kube-system.svc:53", "__address__": "kube-dns.kube-system.svc:53",
"__meta_kubernetes_namespace": "kube-system", "__meta_kubernetes_namespace": "kube-system",
"__meta_kubernetes_service_name": "kube-dns", "__meta_kubernetes_service_name": "kube-dns",
@ -149,7 +148,7 @@ func TestParseServiceListSuccess(t *testing.T) {
"__meta_kubernetes_service_annotationpresent_prometheus_io_port": "true", "__meta_kubernetes_service_annotationpresent_prometheus_io_port": "true",
"__meta_kubernetes_service_annotationpresent_prometheus_io_scrape": "true", "__meta_kubernetes_service_annotationpresent_prometheus_io_scrape": "true",
}), }),
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "kube-dns.kube-system.svc:9153", "__address__": "kube-dns.kube-system.svc:9153",
"__meta_kubernetes_namespace": "kube-system", "__meta_kubernetes_namespace": "kube-system",
"__meta_kubernetes_service_name": "kube-dns", "__meta_kubernetes_service_name": "kube-dns",

View file

@ -7,6 +7,7 @@ import (
"strconv" "strconv"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
// See https://docs.openstack.org/api-ref/compute/#list-hypervisors-details // See https://docs.openstack.org/api-ref/compute/#list-hypervisors-details
@ -61,25 +62,24 @@ func (cfg *apiConfig) getHypervisors() ([]hypervisor, error) {
} }
} }
func addHypervisorLabels(hvs []hypervisor, port int) []map[string]string { func addHypervisorLabels(hvs []hypervisor, port int) []*promutils.Labels {
var ms []map[string]string var ms []*promutils.Labels
for _, hv := range hvs { for _, hv := range hvs {
addr := discoveryutils.JoinHostPort(hv.HostIP, port) addr := discoveryutils.JoinHostPort(hv.HostIP, port)
m := map[string]string{ m := promutils.NewLabels(8)
"__address__": addr, m.Add("__address__", addr)
"__meta_openstack_hypervisor_type": hv.Type, m.Add("__meta_openstack_hypervisor_type", hv.Type)
"__meta_openstack_hypervisor_status": hv.Status, m.Add("__meta_openstack_hypervisor_status", hv.Status)
"__meta_openstack_hypervisor_hostname": hv.Hostname, m.Add("__meta_openstack_hypervisor_hostname", hv.Hostname)
"__meta_openstack_hypervisor_state": hv.State, m.Add("__meta_openstack_hypervisor_state", hv.State)
"__meta_openstack_hypervisor_host_ip": hv.HostIP, m.Add("__meta_openstack_hypervisor_host_ip", hv.HostIP)
"__meta_openstack_hypervisor_id": strconv.Itoa(hv.ID), m.Add("__meta_openstack_hypervisor_id", strconv.Itoa(hv.ID))
}
ms = append(ms, m) ms = append(ms, m)
} }
return ms return ms
} }
func getHypervisorLabels(cfg *apiConfig) ([]map[string]string, error) { func getHypervisorLabels(cfg *apiConfig) ([]*promutils.Labels, error) {
hvs, err := cfg.getHypervisors() hvs, err := cfg.getHypervisors()
if err != nil { if err != nil {
return nil, fmt.Errorf("cannot get hypervisors: %w", err) return nil, fmt.Errorf("cannot get hypervisors: %w", err)

View file

@ -4,8 +4,8 @@ import (
"reflect" "reflect"
"testing" "testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
func Test_parseHypervisorDetail(t *testing.T) { func Test_parseHypervisorDetail(t *testing.T) {
@ -107,7 +107,7 @@ func Test_addHypervisorLabels(t *testing.T) {
tests := []struct { tests := []struct {
name string name string
args args args args
want [][]prompbmarshal.Label want []*promutils.Labels
}{ }{
{ {
name: "", name: "",
@ -124,8 +124,8 @@ func Test_addHypervisorLabels(t *testing.T) {
}, },
}, },
}, },
want: [][]prompbmarshal.Label{ want: []*promutils.Labels{
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "1.2.2.2:9100", "__address__": "1.2.2.2:9100",
"__meta_openstack_hypervisor_host_ip": "1.2.2.2", "__meta_openstack_hypervisor_host_ip": "1.2.2.2",
"__meta_openstack_hypervisor_hostname": "fakehost", "__meta_openstack_hypervisor_hostname": "fakehost",
@ -140,13 +140,7 @@ func Test_addHypervisorLabels(t *testing.T) {
for _, tt := range tests { for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) { t.Run(tt.name, func(t *testing.T) {
got := addHypervisorLabels(tt.args.hvs, tt.args.port) got := addHypervisorLabels(tt.args.hvs, tt.args.port)
var sortedLabelss [][]prompbmarshal.Label discoveryutils.TestEqualLabelss(t, got, tt.want)
for _, labels := range got {
sortedLabelss = append(sortedLabelss, discoveryutils.GetSortedLabels(labels))
}
if !reflect.DeepEqual(sortedLabelss, tt.want) {
t.Errorf("addHypervisorLabels() = %v, want %v", sortedLabelss, tt.want)
}
}) })
} }
} }

View file

@ -8,6 +8,7 @@ import (
"strconv" "strconv"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
// See https://docs.openstack.org/api-ref/compute/#list-servers // See https://docs.openstack.org/api-ref/compute/#list-servers
@ -45,19 +46,18 @@ func parseServersDetail(data []byte) (*serversDetail, error) {
return &srvd, nil return &srvd, nil
} }
func addInstanceLabels(servers []server, port int) []map[string]string { func addInstanceLabels(servers []server, port int) []*promutils.Labels {
var ms []map[string]string var ms []*promutils.Labels
for _, server := range servers { for _, server := range servers {
m := map[string]string{ commonLabels := promutils.NewLabels(16)
"__meta_openstack_instance_id": server.ID, commonLabels.Add("__meta_openstack_instance_id", server.ID)
"__meta_openstack_instance_status": server.Status, commonLabels.Add("__meta_openstack_instance_status", server.Status)
"__meta_openstack_instance_name": server.Name, commonLabels.Add("__meta_openstack_instance_name", server.Name)
"__meta_openstack_project_id": server.TenantID, commonLabels.Add("__meta_openstack_project_id", server.TenantID)
"__meta_openstack_user_id": server.UserID, commonLabels.Add("__meta_openstack_user_id", server.UserID)
"__meta_openstack_instance_flavor": server.Flavor.ID, commonLabels.Add("__meta_openstack_instance_flavor", server.Flavor.ID)
}
for k, v := range server.Metadata { for k, v := range server.Metadata {
m[discoveryutils.SanitizeLabelName("__meta_openstack_tag_"+k)] = v commonLabels.Add(discoveryutils.SanitizeLabelName("__meta_openstack_tag_"+k), v)
} }
// Traverse server.Addresses in alphabetical order of pool name // Traverse server.Addresses in alphabetical order of pool name
// in order to return targets in deterministic order. // in order to return targets in deterministic order.
@ -87,17 +87,15 @@ func addInstanceLabels(servers []server, port int) []map[string]string {
continue continue
} }
// copy labels // copy labels
lbls := make(map[string]string, len(m)) m := promutils.NewLabels(20)
for k, v := range m { m.AddFrom(commonLabels)
lbls[k] = v m.Add("__meta_openstack_address_pool", pool)
} m.Add("__meta_openstack_private_ip", ip.Address)
lbls["__meta_openstack_address_pool"] = pool
lbls["__meta_openstack_private_ip"] = ip.Address
if len(publicIP) > 0 { if len(publicIP) > 0 {
lbls["__meta_openstack_public_ip"] = publicIP m.Add("__meta_openstack_public_ip", publicIP)
} }
lbls["__address__"] = discoveryutils.JoinHostPort(ip.Address, port) m.Add("__address__", discoveryutils.JoinHostPort(ip.Address, port))
ms = append(ms, lbls) ms = append(ms, m)
} }
} }
} }
@ -133,7 +131,7 @@ func (cfg *apiConfig) getServers() ([]server, error) {
} }
} }
func getInstancesLabels(cfg *apiConfig) ([]map[string]string, error) { func getInstancesLabels(cfg *apiConfig) ([]*promutils.Labels, error) {
srv, err := cfg.getServers() srv, err := cfg.getServers()
if err != nil { if err != nil {
return nil, err return nil, err

View file

@ -4,8 +4,8 @@ import (
"reflect" "reflect"
"testing" "testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
func Test_addInstanceLabels(t *testing.T) { func Test_addInstanceLabels(t *testing.T) {
@ -16,7 +16,7 @@ func Test_addInstanceLabels(t *testing.T) {
tests := []struct { tests := []struct {
name string name string
args args args args
want [][]prompbmarshal.Label want []*promutils.Labels
}{ }{
{ {
name: "empty_response", name: "empty_response",
@ -55,8 +55,8 @@ func Test_addInstanceLabels(t *testing.T) {
}, },
}, },
}, },
want: [][]prompbmarshal.Label{ want: []*promutils.Labels{
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "192.168.0.1:9100", "__address__": "192.168.0.1:9100",
"__meta_openstack_address_pool": "test", "__meta_openstack_address_pool": "test",
"__meta_openstack_instance_flavor": "5", "__meta_openstack_instance_flavor": "5",
@ -112,8 +112,8 @@ func Test_addInstanceLabels(t *testing.T) {
}, },
}, },
}, },
want: [][]prompbmarshal.Label{ want: []*promutils.Labels{
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "10.10.0.1:9100", "__address__": "10.10.0.1:9100",
"__meta_openstack_address_pool": "internal", "__meta_openstack_address_pool": "internal",
"__meta_openstack_instance_flavor": "5", "__meta_openstack_instance_flavor": "5",
@ -124,7 +124,7 @@ func Test_addInstanceLabels(t *testing.T) {
"__meta_openstack_project_id": "some-tenant-id", "__meta_openstack_project_id": "some-tenant-id",
"__meta_openstack_user_id": "some-user-id", "__meta_openstack_user_id": "some-user-id",
}), }),
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "192.168.0.1:9100", "__address__": "192.168.0.1:9100",
"__meta_openstack_address_pool": "test", "__meta_openstack_address_pool": "test",
"__meta_openstack_instance_flavor": "5", "__meta_openstack_instance_flavor": "5",
@ -142,13 +142,7 @@ func Test_addInstanceLabels(t *testing.T) {
for _, tt := range tests { for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) { t.Run(tt.name, func(t *testing.T) {
got := addInstanceLabels(tt.args.servers, tt.args.port) got := addInstanceLabels(tt.args.servers, tt.args.port)
var sortedLabelss [][]prompbmarshal.Label discoveryutils.TestEqualLabelss(t, got, tt.want)
for _, labels := range got {
sortedLabelss = append(sortedLabelss, discoveryutils.GetSortedLabels(labels))
}
if !reflect.DeepEqual(sortedLabelss, tt.want) {
t.Errorf("addInstanceLabels() = \n got: %v,\nwant: %v", sortedLabelss, tt.want)
}
}) })
} }
} }

View file

@ -6,6 +6,7 @@ import (
"time" "time"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
// SDCheckInterval defines interval for targets refresh. // SDCheckInterval defines interval for targets refresh.
@ -39,7 +40,7 @@ type SDConfig struct {
} }
// GetLabels returns OpenStack labels according to sdc. // GetLabels returns OpenStack labels according to sdc.
func (sdc *SDConfig) GetLabels(baseDir string) ([]map[string]string, error) { func (sdc *SDConfig) GetLabels(baseDir string) ([]*promutils.Labels, error) {
cfg, err := getAPIConfig(sdc, baseDir) cfg, err := getAPIConfig(sdc, baseDir)
if err != nil { if err != nil {
return nil, fmt.Errorf("cannot get API config: %w", err) return nil, fmt.Errorf("cannot get API config: %w", err)

View file

@ -5,9 +5,10 @@ import (
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger" "github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
func getInstancesLabels(cfg *apiConfig) ([]map[string]string, error) { func getInstancesLabels(cfg *apiConfig) ([]*promutils.Labels, error) {
organizations, err := cfg.getOrganizations() organizations, err := cfg.getOrganizations()
if err != nil { if err != nil {
return nil, err return nil, err
@ -35,46 +36,40 @@ func getInstancesLabels(cfg *apiConfig) ([]map[string]string, error) {
return addInstanceLabels(instances), nil return addInstanceLabels(instances), nil
} }
func addInstanceLabels(instances []instance) []map[string]string { func addInstanceLabels(instances []instance) []*promutils.Labels {
var ms []map[string]string var ms []*promutils.Labels
for _, server := range instances { for _, server := range instances {
m := map[string]string{ m := promutils.NewLabels(24)
"__address__": server.FQDN, m.Add("__address__", server.FQDN)
"__meta_yandexcloud_instance_name": server.Name, m.Add("__meta_yandexcloud_instance_name", server.Name)
"__meta_yandexcloud_instance_fqdn": server.FQDN, m.Add("__meta_yandexcloud_instance_fqdn", server.FQDN)
"__meta_yandexcloud_instance_id": server.ID, m.Add("__meta_yandexcloud_instance_id", server.ID)
"__meta_yandexcloud_instance_status": server.Status, m.Add("__meta_yandexcloud_instance_status", server.Status)
"__meta_yandexcloud_instance_platform_id": server.PlatformID, m.Add("__meta_yandexcloud_instance_platform_id", server.PlatformID)
"__meta_yandexcloud_instance_resources_cores": server.Resources.Cores, m.Add("__meta_yandexcloud_instance_resources_cores", server.Resources.Cores)
"__meta_yandexcloud_instance_resources_core_fraction": server.Resources.CoreFraction, m.Add("__meta_yandexcloud_instance_resources_core_fraction", server.Resources.CoreFraction)
"__meta_yandexcloud_instance_resources_memory": server.Resources.Memory, m.Add("__meta_yandexcloud_instance_resources_memory", server.Resources.Memory)
"__meta_yandexcloud_folder_id": server.FolderID, m.Add("__meta_yandexcloud_folder_id", server.FolderID)
}
for k, v := range server.Labels { for k, v := range server.Labels {
m[discoveryutils.SanitizeLabelName("__meta_yandexcloud_instance_label_"+k)] = v m.Add(discoveryutils.SanitizeLabelName("__meta_yandexcloud_instance_label_"+k), v)
} }
for _, ni := range server.NetworkInterfaces { for _, ni := range server.NetworkInterfaces {
privateIPLabel := fmt.Sprintf("__meta_yandexcloud_instance_private_ip_%s", ni.Index) privateIPLabel := fmt.Sprintf("__meta_yandexcloud_instance_private_ip_%s", ni.Index)
m[privateIPLabel] = ni.PrimaryV4Address.Address m.Add(privateIPLabel, ni.PrimaryV4Address.Address)
if len(ni.PrimaryV4Address.OneToOneNat.Address) > 0 { if len(ni.PrimaryV4Address.OneToOneNat.Address) > 0 {
publicIPLabel := fmt.Sprintf("__meta_yandexcloud_instance_public_ip_%s", ni.Index) publicIPLabel := fmt.Sprintf("__meta_yandexcloud_instance_public_ip_%s", ni.Index)
m[publicIPLabel] = ni.PrimaryV4Address.OneToOneNat.Address m.Add(publicIPLabel, ni.PrimaryV4Address.OneToOneNat.Address)
} }
for j, dnsRecord := range ni.PrimaryV4Address.DNSRecords { for j, dnsRecord := range ni.PrimaryV4Address.DNSRecords {
dnsRecordLabel := fmt.Sprintf("__meta_yandexcloud_instance_private_dns_%d", j) dnsRecordLabel := fmt.Sprintf("__meta_yandexcloud_instance_private_dns_%d", j)
m[dnsRecordLabel] = dnsRecord.FQDN m.Add(dnsRecordLabel, dnsRecord.FQDN)
} }
for j, dnsRecord := range ni.PrimaryV4Address.OneToOneNat.DNSRecords { for j, dnsRecord := range ni.PrimaryV4Address.OneToOneNat.DNSRecords {
dnsRecordLabel := fmt.Sprintf("__meta_yandexcloud_instance_public_dns_%d", j) dnsRecordLabel := fmt.Sprintf("__meta_yandexcloud_instance_public_dns_%d", j)
m[dnsRecordLabel] = dnsRecord.FQDN m.Add(dnsRecordLabel, dnsRecord.FQDN)
} }
} }
ms = append(ms, m) ms = append(ms, m)
} }
return ms return ms
} }

View file

@ -1,11 +1,10 @@
package yandexcloud package yandexcloud
import ( import (
"reflect"
"testing" "testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discoveryutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
func Test_addInstanceLabels(t *testing.T) { func Test_addInstanceLabels(t *testing.T) {
@ -15,7 +14,7 @@ func Test_addInstanceLabels(t *testing.T) {
tests := []struct { tests := []struct {
name string name string
args args args args
want [][]prompbmarshal.Label want []*promutils.Labels
}{ }{
{ {
name: "empty_response", name: "empty_response",
@ -48,8 +47,8 @@ func Test_addInstanceLabels(t *testing.T) {
}, },
}, },
}, },
want: [][]prompbmarshal.Label{ want: []*promutils.Labels{
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "server-1.ru-central1.internal", "__address__": "server-1.ru-central1.internal",
"__meta_yandexcloud_instance_name": "server-1", "__meta_yandexcloud_instance_name": "server-1",
"__meta_yandexcloud_instance_fqdn": "server-1.ru-central1.internal", "__meta_yandexcloud_instance_fqdn": "server-1.ru-central1.internal",
@ -94,8 +93,8 @@ func Test_addInstanceLabels(t *testing.T) {
}, },
}, },
}, },
want: [][]prompbmarshal.Label{ want: []*promutils.Labels{
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "server-1.ru-central1.internal", "__address__": "server-1.ru-central1.internal",
"__meta_yandexcloud_instance_fqdn": "server-1.ru-central1.internal", "__meta_yandexcloud_instance_fqdn": "server-1.ru-central1.internal",
"__meta_yandexcloud_instance_name": "server-1", "__meta_yandexcloud_instance_name": "server-1",
@ -147,8 +146,8 @@ func Test_addInstanceLabels(t *testing.T) {
}, },
}, },
}, },
want: [][]prompbmarshal.Label{ want: []*promutils.Labels{
discoveryutils.GetSortedLabels(map[string]string{ promutils.NewLabelsFromMap(map[string]string{
"__address__": "server-1.ru-central1.internal", "__address__": "server-1.ru-central1.internal",
"__meta_yandexcloud_instance_name": "server-1", "__meta_yandexcloud_instance_name": "server-1",
"__meta_yandexcloud_instance_fqdn": "server-1.ru-central1.internal", "__meta_yandexcloud_instance_fqdn": "server-1.ru-central1.internal",
@ -170,13 +169,7 @@ func Test_addInstanceLabels(t *testing.T) {
for _, tt := range tests { for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) { t.Run(tt.name, func(t *testing.T) {
got := addInstanceLabels(tt.args.instances) got := addInstanceLabels(tt.args.instances)
var sortedLabelss [][]prompbmarshal.Label discoveryutils.TestEqualLabelss(t, got, tt.want)
for _, labels := range got {
sortedLabelss = append(sortedLabelss, discoveryutils.GetSortedLabels(labels))
}
if !reflect.DeepEqual(sortedLabelss, tt.want) {
t.Errorf("addInstanceLabels() = \n got: %v,\nwant: %v", sortedLabelss, tt.want)
}
}) })
} }
} }

View file

@ -8,6 +8,7 @@ import (
"time" "time"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
// SDCheckInterval defines interval for targets refresh. // SDCheckInterval defines interval for targets refresh.
@ -24,7 +25,7 @@ type SDConfig struct {
} }
// GetLabels returns labels for Yandex Cloud according to service discover config. // GetLabels returns labels for Yandex Cloud according to service discover config.
func (sdc *SDConfig) GetLabels(baseDir string) ([]map[string]string, error) { func (sdc *SDConfig) GetLabels(baseDir string) ([]*promutils.Labels, error) {
cfg, err := getAPIConfig(sdc, baseDir) cfg, err := getAPIConfig(sdc, baseDir)
if err != nil { if err != nil {
return nil, fmt.Errorf("cannot get API config: %w", err) return nil, fmt.Errorf("cannot get API config: %w", err)

View file

@ -1,14 +1,14 @@
package discoveryutils package discoveryutils
import ( import (
"encoding/json" "reflect"
"net"
"regexp" "regexp"
"sort"
"strconv" "strconv"
"strings"
"testing"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil" "github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
) )
// SanitizeLabelName replaces anything that doesn't match // SanitizeLabelName replaces anything that doesn't match
@ -29,44 +29,37 @@ var invalidLabelCharRE = regexp.MustCompile(`[^a-zA-Z0-9_]`)
// //
// Host may be dns name, ipv4 or ipv6 address. // Host may be dns name, ipv4 or ipv6 address.
func JoinHostPort(host string, port int) string { func JoinHostPort(host string, port int) string {
portStr := strconv.Itoa(port) bb := bbPool.Get()
return net.JoinHostPort(host, portStr) b := bb.B[:0]
isIPv6 := strings.IndexByte(host, ':') >= 0
if isIPv6 {
b = append(b, '[')
}
b = append(b, host...)
if isIPv6 {
b = append(b, ']')
}
b = append(b, ':')
b = strconv.AppendInt(b, int64(port), 10)
s := bytesutil.ToUnsafeString(b)
s = bytesutil.InternString(s)
bb.B = b
bbPool.Put(bb)
return s
} }
// SortedLabels represents sorted labels. var bbPool bytesutil.ByteBufferPool
type SortedLabels []prompbmarshal.Label
// GetByName returns the label with the given name from sls. // TestEqualLabelss tests whether got are equal to want.
func (sls *SortedLabels) GetByName(name string) string { func TestEqualLabelss(t *testing.T, got, want []*promutils.Labels) {
for _, lb := range *sls { t.Helper()
if lb.Name == name { var gotCopy []*promutils.Labels
return lb.Value for _, labels := range got {
labels = labels.Clone()
labels.Sort()
gotCopy = append(gotCopy, labels)
}
if !reflect.DeepEqual(gotCopy, want) {
t.Fatalf("unexpected labels:\ngot\n%v\nwant\n%v", gotCopy, want)
} }
} }
return ""
}
// UnmarshalJSON unmarshals JSON from data.
func (sls *SortedLabels) UnmarshalJSON(data []byte) error {
var m map[string]string
if err := json.Unmarshal(data, &m); err != nil {
return err
}
*sls = GetSortedLabels(m)
return nil
}
// GetSortedLabels returns SortedLabels built from m.
func GetSortedLabels(m map[string]string) SortedLabels {
a := make([]prompbmarshal.Label, 0, len(m))
for k, v := range m {
a = append(a, prompbmarshal.Label{
Name: k,
Value: v,
})
}
sort.Slice(a, func(i, j int) bool {
return a[i].Name < a[j].Name
})
return a
}

View file

@ -6,6 +6,20 @@ import (
"time" "time"
) )
func TestJoinHostPort(t *testing.T) {
f := func(host string, port int, resultExpected string) {
t.Helper()
for i := 0; i < 5; i++ {
result := JoinHostPort(host, port)
if result != resultExpected {
t.Fatalf("unexpected result for JoinHostPort(%q, %d); got %q; want %q", host, port, result, resultExpected)
}
}
}
f("foo", 123, "foo:123")
f("1:32::43", 80, "[1:32::43]:80")
}
func TestSanitizeLabelNameSerial(t *testing.T) { func TestSanitizeLabelNameSerial(t *testing.T) {
if err := testSanitizeLabelName(); err != nil { if err := testSanitizeLabelName(); err != nil {
t.Fatalf("unexpected error: %s", err) t.Fatalf("unexpected error: %s", err)

View file

@ -27,6 +27,7 @@ import (
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/kubernetes" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/kubernetes"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/openstack" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/openstack"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/yandexcloud" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promscrape/discovery/yandexcloud"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
"github.com/VictoriaMetrics/metrics" "github.com/VictoriaMetrics/metrics"
) )
@ -351,7 +352,7 @@ func (sg *scraperGroup) update(sws []*ScrapeWork) {
additionsCount := 0 additionsCount := 0
deletionsCount := 0 deletionsCount := 0
swsMap := make(map[string][]prompbmarshal.Label, len(sws)) swsMap := make(map[string]*promutils.Labels, len(sws))
var swsToStart []*ScrapeWork var swsToStart []*ScrapeWork
for _, sw := range sws { for _, sw := range sws {
key := sw.key() key := sw.key()
@ -362,7 +363,7 @@ func (sg *scraperGroup) update(sws []*ScrapeWork) {
"make sure service discovery and relabeling is set up properly; "+ "make sure service discovery and relabeling is set up properly; "+
"see also https://docs.victoriametrics.com/vmagent.html#troubleshooting; "+ "see also https://docs.victoriametrics.com/vmagent.html#troubleshooting; "+
"original labels for target1: %s; original labels for target2: %s", "original labels for target1: %s; original labels for target2: %s",
sw.ScrapeURL, sw.LabelsString(), promLabelsString(originalLabels), promLabelsString(sw.OriginalLabels)) sw.ScrapeURL, sw.LabelsString(), originalLabels.String(), sw.OriginalLabels.String())
} }
droppedTargetsMap.Register(sw.OriginalLabels) droppedTargetsMap.Register(sw.OriginalLabels)
continue continue

View file

@ -7,7 +7,6 @@ import (
"io" "io"
"math" "math"
"math/bits" "math/bits"
"strconv"
"strings" "strings"
"sync" "sync"
"time" "time"
@ -23,6 +22,7 @@ import (
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promauth"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal" "github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promrelabel" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promrelabel"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
parser "github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/prometheus" parser "github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/prometheus"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/proxy" "github.com/VictoriaMetrics/VictoriaMetrics/lib/proxy"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/timerpool" "github.com/VictoriaMetrics/VictoriaMetrics/lib/timerpool"
@ -69,7 +69,7 @@ type ScrapeWork struct {
// These labels are needed for relabeling troubleshooting at /targets page. // These labels are needed for relabeling troubleshooting at /targets page.
// //
// OriginalLabels are sorted by name. // OriginalLabels are sorted by name.
OriginalLabels []prompbmarshal.Label OriginalLabels *promutils.Labels
// Labels to add to the scraped metrics. // Labels to add to the scraped metrics.
// //
@ -82,7 +82,7 @@ type ScrapeWork struct {
// See also https://prometheus.io/docs/concepts/jobs_instances/ // See also https://prometheus.io/docs/concepts/jobs_instances/
// //
// Labels are sorted by name. // Labels are sorted by name.
Labels []prompbmarshal.Label Labels *promutils.Labels
// ExternalLabels contains labels from global->external_labels section of -promscrape.config // ExternalLabels contains labels from global->external_labels section of -promscrape.config
// //
@ -90,7 +90,7 @@ type ScrapeWork struct {
// See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/3137 // See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/3137
// //
// ExternalLabels are sorted by name. // ExternalLabels are sorted by name.
ExternalLabels []prompbmarshal.Label ExternalLabels *promutils.Labels
// ProxyURL HTTP proxy url // ProxyURL HTTP proxy url
ProxyURL *proxy.URL ProxyURL *proxy.URL
@ -153,7 +153,7 @@ func (sw *ScrapeWork) key() string {
"ProxyURL=%s, ProxyAuthConfig=%s, AuthConfig=%s, MetricRelabelConfigs=%s, SampleLimit=%d, DisableCompression=%v, DisableKeepAlive=%v, StreamParse=%v, "+ "ProxyURL=%s, ProxyAuthConfig=%s, AuthConfig=%s, MetricRelabelConfigs=%s, SampleLimit=%d, DisableCompression=%v, DisableKeepAlive=%v, StreamParse=%v, "+
"ScrapeAlignInterval=%s, ScrapeOffset=%s, SeriesLimit=%d, NoStaleMarkers=%v", "ScrapeAlignInterval=%s, ScrapeOffset=%s, SeriesLimit=%d, NoStaleMarkers=%v",
sw.jobNameOriginal, sw.ScrapeURL, sw.ScrapeInterval, sw.ScrapeTimeout, sw.HonorLabels, sw.HonorTimestamps, sw.DenyRedirects, sw.LabelsString(), sw.jobNameOriginal, sw.ScrapeURL, sw.ScrapeInterval, sw.ScrapeTimeout, sw.HonorLabels, sw.HonorTimestamps, sw.DenyRedirects, sw.LabelsString(),
promLabelsString(sw.ExternalLabels), sw.ExternalLabels.String(),
sw.ProxyURL.String(), sw.ProxyAuthConfig.String(), sw.ProxyURL.String(), sw.ProxyAuthConfig.String(),
sw.AuthConfig.String(), sw.MetricRelabelConfigs.String(), sw.SampleLimit, sw.DisableCompression, sw.DisableKeepAlive, sw.StreamParse, sw.AuthConfig.String(), sw.MetricRelabelConfigs.String(), sw.SampleLimit, sw.DisableCompression, sw.DisableKeepAlive, sw.StreamParse,
sw.ScrapeAlignInterval, sw.ScrapeOffset, sw.SeriesLimit, sw.NoStaleMarkers) sw.ScrapeAlignInterval, sw.ScrapeOffset, sw.SeriesLimit, sw.NoStaleMarkers)
@ -162,33 +162,12 @@ func (sw *ScrapeWork) key() string {
// Job returns job for the ScrapeWork // Job returns job for the ScrapeWork
func (sw *ScrapeWork) Job() string { func (sw *ScrapeWork) Job() string {
return promrelabel.GetLabelValueByName(sw.Labels, "job") return sw.Labels.Get("job")
} }
// LabelsString returns labels in Prometheus format for the given sw. // LabelsString returns labels in Prometheus format for the given sw.
func (sw *ScrapeWork) LabelsString() string { func (sw *ScrapeWork) LabelsString() string {
return promLabelsString(sw.Labels) return sw.Labels.String()
}
func promLabelsString(labels []prompbmarshal.Label) string {
// Calculate the required memory for storing serialized labels.
n := 2 // for `{...}`
for _, label := range labels {
n += len(label.Name) + len(label.Value)
n += 4 // for `="...",`
}
b := make([]byte, 0, n)
b = append(b, '{')
for i, label := range labels {
b = append(b, label.Name...)
b = append(b, '=')
b = strconv.AppendQuote(b, label.Value)
if i+1 < len(labels) {
b = append(b, ',')
}
}
b = append(b, '}')
return bytesutil.ToUnsafeString(b)
} }
type scrapeWork struct { type scrapeWork struct {
@ -863,7 +842,8 @@ func (sw *scrapeWork) addRowToTimeseries(wc *writeRequestCtx, r *parser.Row, tim
bbPool.Put(bb) bbPool.Put(bb)
} }
labelsLen := len(wc.labels) labelsLen := len(wc.labels)
wc.labels = appendLabels(wc.labels, metric, r.Tags, sw.Config.Labels, sw.Config.HonorLabels) targetLabels := sw.Config.Labels.GetLabels()
wc.labels = appendLabels(wc.labels, metric, r.Tags, targetLabels, sw.Config.HonorLabels)
if needRelabel { if needRelabel {
wc.labels = sw.Config.MetricRelabelConfigs.Apply(wc.labels, labelsLen) wc.labels = sw.Config.MetricRelabelConfigs.Apply(wc.labels, labelsLen)
} }
@ -874,7 +854,8 @@ func (sw *scrapeWork) addRowToTimeseries(wc *writeRequestCtx, r *parser.Row, tim
} }
// Add labels from `global->external_labels` section after the relabeling like Prometheus does. // Add labels from `global->external_labels` section after the relabeling like Prometheus does.
// See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/3137 // See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/3137
wc.labels = appendExtraLabels(wc.labels, sw.Config.ExternalLabels, labelsLen, sw.Config.HonorLabels) externalLabels := sw.Config.ExternalLabels.GetLabels()
wc.labels = appendExtraLabels(wc.labels, externalLabels, labelsLen, sw.Config.HonorLabels)
sampleTimestamp := r.Timestamp sampleTimestamp := r.Timestamp
if !sw.Config.HonorTimestamps || sampleTimestamp == 0 { if !sw.Config.HonorTimestamps || sampleTimestamp == 0 {
sampleTimestamp = timestamp sampleTimestamp = timestamp

View file

@ -9,6 +9,7 @@ import (
"github.com/VictoriaMetrics/VictoriaMetrics/lib/auth" "github.com/VictoriaMetrics/VictoriaMetrics/lib/auth"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal" "github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promrelabel" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promrelabel"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
parser "github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/prometheus" parser "github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/prometheus"
) )
@ -39,10 +40,11 @@ func TestIsAutoMetric(t *testing.T) {
func TestAppendExtraLabels(t *testing.T) { func TestAppendExtraLabels(t *testing.T) {
f := func(sourceLabels, extraLabels string, honorLabels bool, resultExpected string) { f := func(sourceLabels, extraLabels string, honorLabels bool, resultExpected string) {
t.Helper() t.Helper()
src := promrelabel.MustParseMetricWithLabels(sourceLabels) src := promutils.NewLabelsFromString(sourceLabels)
extra := promrelabel.MustParseMetricWithLabels(extraLabels) extra := promutils.NewLabelsFromString(extraLabels)
labels := appendExtraLabels(src, extra, 0, honorLabels) var labels promutils.Labels
result := promLabelsString(labels) labels.Labels = appendExtraLabels(src.GetLabels(), extra.GetLabels(), 0, honorLabels)
result := labels.String()
if result != resultExpected { if result != resultExpected {
t.Fatalf("unexpected result; got\n%s\nwant\n%s", result, resultExpected) t.Fatalf("unexpected result; got\n%s\nwant\n%s", result, resultExpected)
} }
@ -69,33 +71,6 @@ func TestAppendExtraLabels(t *testing.T) {
f(`{foo="a",exported_foo="b"}`, `{exported_foo="c",foo="d"}`, false, `{exported_foo="a",exported_exported_foo="b",exported_foo="c",foo="d"}`) f(`{foo="a",exported_foo="b"}`, `{exported_foo="c",foo="d"}`, false, `{exported_foo="a",exported_exported_foo="b",exported_foo="c",foo="d"}`)
} }
func TestPromLabelsString(t *testing.T) {
f := func(labels []prompbmarshal.Label, resultExpected string) {
t.Helper()
result := promLabelsString(labels)
if result != resultExpected {
t.Fatalf("unexpected result; got\n%s\nwant\n%s", result, resultExpected)
}
}
f([]prompbmarshal.Label{}, "{}")
f([]prompbmarshal.Label{
{
Name: "foo",
Value: "bar",
},
}, `{foo="bar"}`)
f([]prompbmarshal.Label{
{
Name: "foo",
Value: "bar",
},
{
Name: "a",
Value: `"b"`,
},
}, `{foo="bar",a="\"b\""}`)
}
func TestScrapeWorkScrapeInternalFailure(t *testing.T) { func TestScrapeWorkScrapeInternalFailure(t *testing.T) {
dataExpected := ` dataExpected := `
up 0 123 up 0 123
@ -226,12 +201,9 @@ func TestScrapeWorkScrapeInternalSuccess(t *testing.T) {
`, &ScrapeWork{ `, &ScrapeWork{
ScrapeTimeout: time.Second * 42, ScrapeTimeout: time.Second * 42,
HonorTimestamps: true, HonorTimestamps: true,
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "foo": "x",
Name: "foo", }),
Value: "x",
},
},
}, ` }, `
foo{bar="baz",foo="x"} 34.45 3 foo{bar="baz",foo="x"} 34.45 3
abc{foo="x"} -2 123 abc{foo="x"} -2 123
@ -248,12 +220,9 @@ func TestScrapeWorkScrapeInternalSuccess(t *testing.T) {
`, &ScrapeWork{ `, &ScrapeWork{
ScrapeTimeout: time.Second * 42, ScrapeTimeout: time.Second * 42,
HonorLabels: false, HonorLabels: false,
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "job": "override",
Name: "job", }),
Value: "override",
},
},
}, ` }, `
foo{exported_job="orig",job="override",bar="baz"} 34.45 123 foo{exported_job="orig",job="override",bar="baz"} 34.45 123
bar{exported_job="aa",job="override",x="1",a="b",y="2"} -3e4 123 bar{exported_job="aa",job="override",x="1",a="b",y="2"} -3e4 123
@ -271,16 +240,10 @@ func TestScrapeWorkScrapeInternalSuccess(t *testing.T) {
`, &ScrapeWork{ `, &ScrapeWork{
ScrapeTimeout: time.Second * 42, ScrapeTimeout: time.Second * 42,
HonorLabels: true, HonorLabels: true,
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "instance": "foobar",
Name: "instance", "job": "xxx",
Value: "foobar", }),
},
{
Name: "job",
Value: "xxx",
},
},
}, ` }, `
no_instance{job="some_job",label="val1"} 5555 123 no_instance{job="some_job",label="val1"} 5555 123
test_with_instance{instance="some_instance",job="some_job",label="val2"} 1555 123 test_with_instance{instance="some_instance",job="some_job",label="val2"} 1555 123
@ -297,16 +260,10 @@ func TestScrapeWorkScrapeInternalSuccess(t *testing.T) {
`, &ScrapeWork{ `, &ScrapeWork{
ScrapeTimeout: time.Second * 42, ScrapeTimeout: time.Second * 42,
HonorLabels: false, HonorLabels: false,
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "instance": "foobar",
Name: "instance", "job": "xxx",
Value: "foobar", }),
},
{
Name: "job",
Value: "xxx",
},
},
}, ` }, `
no_instance{exported_job="some_job",instance="foobar",job="xxx",label="val1"} 5555 123 no_instance{exported_job="some_job",instance="foobar",job="xxx",label="val1"} 5555 123
test_with_instance{exported_instance="some_instance",exported_job="some_job",instance="foobar",job="xxx",label="val2"} 1555 123 test_with_instance{exported_instance="some_instance",exported_job="some_job",instance="foobar",job="xxx",label="val2"} 1555 123
@ -323,12 +280,9 @@ func TestScrapeWorkScrapeInternalSuccess(t *testing.T) {
`, &ScrapeWork{ `, &ScrapeWork{
ScrapeTimeout: time.Second * 42, ScrapeTimeout: time.Second * 42,
HonorLabels: true, HonorLabels: true,
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "job": "override",
Name: "job", }),
Value: "override",
},
},
}, ` }, `
foo{job="orig",bar="baz"} 34.45 123 foo{job="orig",bar="baz"} 34.45 123
bar{job="aa",a="b"} -3e4 123 bar{job="aa",a="b"} -3e4 123
@ -345,16 +299,10 @@ func TestScrapeWorkScrapeInternalSuccess(t *testing.T) {
`, &ScrapeWork{ `, &ScrapeWork{
ScrapeTimeout: time.Second * 42, ScrapeTimeout: time.Second * 42,
HonorLabels: true, HonorLabels: true,
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "job": "xx",
Name: "job", "__address__": "foo.com",
Value: "xx", }),
},
{
Name: "__address__",
Value: "foo.com",
},
},
MetricRelabelConfigs: mustParseRelabelConfigs(` MetricRelabelConfigs: mustParseRelabelConfigs(`
- action: replace - action: replace
source_labels: ["__address__", "job"] source_labels: ["__address__", "job"]
@ -381,16 +329,10 @@ func TestScrapeWorkScrapeInternalSuccess(t *testing.T) {
`, &ScrapeWork{ `, &ScrapeWork{
ScrapeTimeout: time.Second * 42, ScrapeTimeout: time.Second * 42,
HonorLabels: true, HonorLabels: true,
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "job": "xx",
Name: "job", "instance": "foo.com",
Value: "xx", }),
},
{
Name: "instance",
Value: "foo.com",
},
},
MetricRelabelConfigs: mustParseRelabelConfigs(` MetricRelabelConfigs: mustParseRelabelConfigs(`
- action: drop - action: drop
separator: "" separator: ""
@ -527,322 +469,218 @@ func TestAddRowToTimeseriesNoRelabeling(t *testing.T) {
// HonorLabels=false, empty Labels and ExternalLabels // HonorLabels=false, empty Labels and ExternalLabels
f(`metric 0 123`, f(`metric 0 123`,
&ScrapeWork{ &ScrapeWork{
Labels: []prompbmarshal.Label{},
ExternalLabels: []prompbmarshal.Label{},
HonorLabels: false, HonorLabels: false,
}, },
`metric 0 123`) `metric 0 123`)
f(`metric{a="f"} 0 123`, f(`metric{a="f"} 0 123`,
&ScrapeWork{ &ScrapeWork{
Labels: []prompbmarshal.Label{},
ExternalLabels: []prompbmarshal.Label{},
HonorLabels: false, HonorLabels: false,
}, },
`metric{a="f"} 0 123`) `metric{a="f"} 0 123`)
// HonorLabels=true, empty Labels and ExternalLabels // HonorLabels=true, empty Labels and ExternalLabels
f(`metric 0 123`, f(`metric 0 123`,
&ScrapeWork{ &ScrapeWork{
Labels: []prompbmarshal.Label{},
ExternalLabels: []prompbmarshal.Label{},
HonorLabels: true, HonorLabels: true,
}, },
`metric 0 123`) `metric 0 123`)
f(`metric{a="f"} 0 123`, f(`metric{a="f"} 0 123`,
&ScrapeWork{ &ScrapeWork{
Labels: []prompbmarshal.Label{},
ExternalLabels: []prompbmarshal.Label{},
HonorLabels: true, HonorLabels: true,
}, },
`metric{a="f"} 0 123`) `metric{a="f"} 0 123`)
// HonorLabels=false, non-empty Labels // HonorLabels=false, non-empty Labels
f(`metric 0 123`, f(`metric 0 123`,
&ScrapeWork{ &ScrapeWork{
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "a": "f",
Name: "a", }),
Value: "f",
},
},
ExternalLabels: []prompbmarshal.Label{},
HonorLabels: false, HonorLabels: false,
}, },
`metric{a="f"} 0 123`) `metric{a="f"} 0 123`)
f(`metric{foo="bar"} 0 123`, f(`metric{foo="bar"} 0 123`,
&ScrapeWork{ &ScrapeWork{
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "a": "f",
Name: "a", }),
Value: "f",
},
},
ExternalLabels: []prompbmarshal.Label{},
HonorLabels: false, HonorLabels: false,
}, },
`metric{a="f",foo="bar"} 0 123`) `metric{a="f",foo="bar"} 0 123`)
// HonorLabels=true, non-empty Labels // HonorLabels=true, non-empty Labels
f(`metric 0 123`, f(`metric 0 123`,
&ScrapeWork{ &ScrapeWork{
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "a": "f",
Name: "a", }),
Value: "f",
},
},
ExternalLabels: []prompbmarshal.Label{},
HonorLabels: true, HonorLabels: true,
}, },
`metric{a="f"} 0 123`) `metric{a="f"} 0 123`)
f(`metric{foo="bar"} 0 123`, f(`metric{foo="bar"} 0 123`,
&ScrapeWork{ &ScrapeWork{
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "a": "f",
Name: "a", }),
Value: "f",
},
},
ExternalLabels: []prompbmarshal.Label{},
HonorLabels: true, HonorLabels: true,
}, },
`metric{a="f",foo="bar"} 0 123`) `metric{a="f",foo="bar"} 0 123`)
// HonorLabels=false, non-empty ExternalLabels // HonorLabels=false, non-empty ExternalLabels
f(`metric 0 123`, f(`metric 0 123`,
&ScrapeWork{ &ScrapeWork{
Labels: []prompbmarshal.Label{}, ExternalLabels: promutils.NewLabelsFromMap(map[string]string{
ExternalLabels: []prompbmarshal.Label{ "a": "f",
{ }),
Name: "a",
Value: "f",
},
},
HonorLabels: false, HonorLabels: false,
}, },
`metric{a="f"} 0 123`) `metric{a="f"} 0 123`)
f(`metric{foo="bar"} 0 123`, f(`metric{foo="bar"} 0 123`,
&ScrapeWork{ &ScrapeWork{
Labels: []prompbmarshal.Label{}, ExternalLabels: promutils.NewLabelsFromMap(map[string]string{
ExternalLabels: []prompbmarshal.Label{ "a": "f",
{ }),
Name: "a",
Value: "f",
},
},
HonorLabels: false, HonorLabels: false,
}, },
`metric{a="f",foo="bar"} 0 123`) `metric{a="f",foo="bar"} 0 123`)
// HonorLabels=true, non-empty ExternalLabels // HonorLabels=true, non-empty ExternalLabels
f(`metric 0 123`, f(`metric 0 123`,
&ScrapeWork{ &ScrapeWork{
Labels: []prompbmarshal.Label{}, ExternalLabels: promutils.NewLabelsFromMap(map[string]string{
ExternalLabels: []prompbmarshal.Label{ "a": "f",
{ }),
Name: "a",
Value: "f",
},
},
HonorLabels: true, HonorLabels: true,
}, },
`metric{a="f"} 0 123`) `metric{a="f"} 0 123`)
f(`metric{foo="bar"} 0 123`, f(`metric{foo="bar"} 0 123`,
&ScrapeWork{ &ScrapeWork{
Labels: []prompbmarshal.Label{}, ExternalLabels: promutils.NewLabelsFromMap(map[string]string{
ExternalLabels: []prompbmarshal.Label{ "a": "f",
{ }),
Name: "a",
Value: "f",
},
},
HonorLabels: true, HonorLabels: true,
}, },
`metric{a="f",foo="bar"} 0 123`) `metric{a="f",foo="bar"} 0 123`)
// HonorLabels=false, non-empty Labels and ExternalLabels // HonorLabels=false, non-empty Labels and ExternalLabels
f(`metric 0 123`, f(`metric 0 123`,
&ScrapeWork{ &ScrapeWork{
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "x": "y",
Name: "x", }),
Value: "y", ExternalLabels: promutils.NewLabelsFromMap(map[string]string{
}, "a": "f",
}, }),
ExternalLabels: []prompbmarshal.Label{
{
Name: "a",
Value: "f",
},
},
HonorLabels: false, HonorLabels: false,
}, },
`metric{a="f",x="y"} 0 123`) `metric{a="f",x="y"} 0 123`)
f(`metric{foo="bar"} 0 123`, f(`metric{foo="bar"} 0 123`,
&ScrapeWork{ &ScrapeWork{
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "x": "y",
Name: "x", }),
Value: "y", ExternalLabels: promutils.NewLabelsFromMap(map[string]string{
}, "a": "f",
}, }),
ExternalLabels: []prompbmarshal.Label{
{
Name: "a",
Value: "f",
},
},
HonorLabels: false, HonorLabels: false,
}, },
`metric{a="f",foo="bar",x="y"} 0 123`) `metric{a="f",foo="bar",x="y"} 0 123`)
// HonorLabels=true, non-empty Labels and ExternalLabels // HonorLabels=true, non-empty Labels and ExternalLabels
f(`metric 0 123`, f(`metric 0 123`,
&ScrapeWork{ &ScrapeWork{
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "x": "y",
Name: "x", }),
Value: "y", ExternalLabels: promutils.NewLabelsFromMap(map[string]string{
}, "a": "f",
}, }),
ExternalLabels: []prompbmarshal.Label{
{
Name: "a",
Value: "f",
},
},
HonorLabels: true, HonorLabels: true,
}, },
`metric{a="f",x="y"} 0 123`) `metric{a="f",x="y"} 0 123`)
f(`metric{foo="bar"} 0 123`, f(`metric{foo="bar"} 0 123`,
&ScrapeWork{ &ScrapeWork{
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "x": "y",
Name: "x", }),
Value: "y", ExternalLabels: promutils.NewLabelsFromMap(map[string]string{
}, "a": "f",
}, }),
ExternalLabels: []prompbmarshal.Label{
{
Name: "a",
Value: "f",
},
},
HonorLabels: true, HonorLabels: true,
}, },
`metric{a="f",foo="bar",x="y"} 0 123`) `metric{a="f",foo="bar",x="y"} 0 123`)
// HonorLabels=false, clashing Labels and metric label // HonorLabels=false, clashing Labels and metric label
f(`metric{a="b"} 0 123`, f(`metric{a="b"} 0 123`,
&ScrapeWork{ &ScrapeWork{
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "a": "f",
Name: "a", }),
Value: "f",
},
},
ExternalLabels: []prompbmarshal.Label{},
HonorLabels: false, HonorLabels: false,
}, },
`metric{a="f",exported_a="b"} 0 123`) `metric{a="f",exported_a="b"} 0 123`)
// HonorLabels=true, clashing Labels and metric label // HonorLabels=true, clashing Labels and metric label
f(`metric{a="b"} 0 123`, f(`metric{a="b"} 0 123`,
&ScrapeWork{ &ScrapeWork{
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "a": "f",
Name: "a", }),
Value: "f",
},
},
ExternalLabels: []prompbmarshal.Label{},
HonorLabels: true, HonorLabels: true,
}, },
`metric{a="b"} 0 123`) `metric{a="b"} 0 123`)
// HonorLabels=false, clashing ExternalLabels and metric label // HonorLabels=false, clashing ExternalLabels and metric label
f(`metric{a="b"} 0 123`, f(`metric{a="b"} 0 123`,
&ScrapeWork{ &ScrapeWork{
Labels: []prompbmarshal.Label{}, ExternalLabels: promutils.NewLabelsFromMap(map[string]string{
ExternalLabels: []prompbmarshal.Label{ "a": "f",
{ }),
Name: "a",
Value: "f",
},
},
HonorLabels: false, HonorLabels: false,
}, },
`metric{a="f",exported_a="b"} 0 123`) `metric{a="f",exported_a="b"} 0 123`)
// HonorLabels=true, clashing ExternalLabels and metric label // HonorLabels=true, clashing ExternalLabels and metric label
f(`metric{a="b"} 0 123`, f(`metric{a="b"} 0 123`,
&ScrapeWork{ &ScrapeWork{
Labels: []prompbmarshal.Label{}, ExternalLabels: promutils.NewLabelsFromMap(map[string]string{
ExternalLabels: []prompbmarshal.Label{ "a": "f",
{ }),
Name: "a",
Value: "f",
},
},
HonorLabels: true, HonorLabels: true,
}, },
`metric{a="b"} 0 123`) `metric{a="b"} 0 123`)
// HonorLabels=false, clashing Labels and ExternalLAbels // HonorLabels=false, clashing Labels and ExternalLAbels
f(`metric 0 123`, f(`metric 0 123`,
&ScrapeWork{ &ScrapeWork{
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "a": "e",
Name: "a", }),
Value: "e", ExternalLabels: promutils.NewLabelsFromMap(map[string]string{
}, "a": "f",
}, }),
ExternalLabels: []prompbmarshal.Label{
{
Name: "a",
Value: "f",
},
},
HonorLabels: false, HonorLabels: false,
}, },
`metric{a="f",exported_a="e"} 0 123`) `metric{a="f",exported_a="e"} 0 123`)
f(`metric{foo="bar"} 0 123`, f(`metric{foo="bar"} 0 123`,
&ScrapeWork{ &ScrapeWork{
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "a": "e",
Name: "a", }),
Value: "e", ExternalLabels: promutils.NewLabelsFromMap(map[string]string{
}, "a": "f",
}, }),
ExternalLabels: []prompbmarshal.Label{
{
Name: "a",
Value: "f",
},
},
HonorLabels: false, HonorLabels: false,
}, },
`metric{a="f",foo="bar",exported_a="e"} 0 123`) `metric{a="f",foo="bar",exported_a="e"} 0 123`)
// HonorLabels=true, clashing Labels and ExternalLAbels // HonorLabels=true, clashing Labels and ExternalLAbels
f(`metric 0 123`, f(`metric 0 123`,
&ScrapeWork{ &ScrapeWork{
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "a": "e",
Name: "a", }),
Value: "e", ExternalLabels: promutils.NewLabelsFromMap(map[string]string{
}, "a": "f",
}, }),
ExternalLabels: []prompbmarshal.Label{
{
Name: "a",
Value: "f",
},
},
HonorLabels: true, HonorLabels: true,
}, },
`metric{a="e"} 0 123`) `metric{a="e"} 0 123`)
f(`metric{foo="bar"} 0 123`, f(`metric{foo="bar"} 0 123`,
&ScrapeWork{ &ScrapeWork{
Labels: []prompbmarshal.Label{ Labels: promutils.NewLabelsFromMap(map[string]string{
{ "a": "e",
Name: "a", }),
Value: "e", ExternalLabels: promutils.NewLabelsFromMap(map[string]string{
}, "a": "f",
}, }),
ExternalLabels: []prompbmarshal.Label{
{
Name: "a",
Value: "f",
},
},
HonorLabels: true, HonorLabels: true,
}, },
`metric{a="e",foo="bar"} 0 123`) `metric{a="e",foo="bar"} 0 123`)

View file

@ -14,8 +14,8 @@ import (
"unsafe" "unsafe"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/fasttime" "github.com/VictoriaMetrics/VictoriaMetrics/lib/fasttime"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promrelabel" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promrelabel"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
"github.com/cespare/xxhash/v2" "github.com/cespare/xxhash/v2"
) )
@ -181,8 +181,8 @@ func (tsm *targetStatusMap) getActiveTargetStatuses() []targetStatus {
tsm.mu.Unlock() tsm.mu.Unlock()
// Sort discovered targets by __address__ label, so they stay in consistent order across calls // Sort discovered targets by __address__ label, so they stay in consistent order across calls
sort.Slice(tss, func(i, j int) bool { sort.Slice(tss, func(i, j int) bool {
addr1 := promrelabel.GetLabelValueByName(tss[i].sw.Config.OriginalLabels, "__address__") addr1 := tss[i].sw.Config.OriginalLabels.Get("__address__")
addr2 := promrelabel.GetLabelValueByName(tss[j].sw.Config.OriginalLabels, "__address__") addr2 := tss[j].sw.Config.OriginalLabels.Get("__address__")
return addr1 < addr2 return addr1 < addr2
}) })
return tss return tss
@ -219,11 +219,12 @@ func (tsm *targetStatusMap) WriteActiveTargetsJSON(w io.Writer) {
fmt.Fprintf(w, `]`) fmt.Fprintf(w, `]`)
} }
func writeLabelsJSON(w io.Writer, labels []prompbmarshal.Label) { func writeLabelsJSON(w io.Writer, labels *promutils.Labels) {
fmt.Fprintf(w, `{`) fmt.Fprintf(w, `{`)
for i, label := range labels { labelsList := labels.GetLabels()
for i, label := range labelsList {
fmt.Fprintf(w, "%q:%q", label.Name, label.Value) fmt.Fprintf(w, "%q:%q", label.Name, label.Value)
if i+1 < len(labels) { if i+1 < len(labelsList) {
fmt.Fprintf(w, `,`) fmt.Fprintf(w, `,`)
} }
} }
@ -252,27 +253,27 @@ type droppedTargets struct {
} }
type droppedTarget struct { type droppedTarget struct {
originalLabels []prompbmarshal.Label originalLabels *promutils.Labels
deadline uint64 deadline uint64
} }
func (dt *droppedTargets) getTargetsLabels() [][]prompbmarshal.Label { func (dt *droppedTargets) getTargetsLabels() []*promutils.Labels {
dt.mu.Lock() dt.mu.Lock()
dtls := make([][]prompbmarshal.Label, 0, len(dt.m)) dtls := make([]*promutils.Labels, 0, len(dt.m))
for _, v := range dt.m { for _, v := range dt.m {
dtls = append(dtls, v.originalLabels) dtls = append(dtls, v.originalLabels)
} }
dt.mu.Unlock() dt.mu.Unlock()
// Sort discovered targets by __address__ label, so they stay in consistent order across calls // Sort discovered targets by __address__ label, so they stay in consistent order across calls
sort.Slice(dtls, func(i, j int) bool { sort.Slice(dtls, func(i, j int) bool {
addr1 := promrelabel.GetLabelValueByName(dtls[i], "__address__") addr1 := dtls[i].Get("__address__")
addr2 := promrelabel.GetLabelValueByName(dtls[j], "__address__") addr2 := dtls[j].Get("__address__")
return addr1 < addr2 return addr1 < addr2
}) })
return dtls return dtls
} }
func (dt *droppedTargets) Register(originalLabels []prompbmarshal.Label) { func (dt *droppedTargets) Register(originalLabels *promutils.Labels) {
// It is better to have hash collisions instead of spending additional CPU on promLabelsString() call. // It is better to have hash collisions instead of spending additional CPU on promLabelsString() call.
key := labelsHash(originalLabels) key := labelsHash(originalLabels)
currentTime := fasttime.UnixTimestamp() currentTime := fasttime.UnixTimestamp()
@ -297,9 +298,9 @@ func (dt *droppedTargets) Register(originalLabels []prompbmarshal.Label) {
dt.mu.Unlock() dt.mu.Unlock()
} }
func labelsHash(labels []prompbmarshal.Label) uint64 { func labelsHash(labels *promutils.Labels) uint64 {
d := xxhashPool.Get().(*xxhash.Digest) d := xxhashPool.Get().(*xxhash.Digest)
for _, label := range labels { for _, label := range labels.Labels {
_, _ = d.WriteString(label.Name) _, _ = d.WriteString(label.Name)
_, _ = d.WriteString(label.Value) _, _ = d.WriteString(label.Value)
} }
@ -431,7 +432,8 @@ func filterTargetsByLabels(jts []*jobTargetsStatuses, searchQuery string) ([]*jo
for _, job := range jts { for _, job := range jts {
var tss []targetStatus var tss []targetStatus
for _, ts := range job.targetsStatus { for _, ts := range job.targetsStatus {
if ie.Match(ts.sw.Config.Labels) { labels := ts.sw.Config.Labels.GetLabels()
if ie.Match(labels) {
tss = append(tss, ts) tss = append(tss, ts)
} }
} }
@ -496,15 +498,15 @@ func getRequestFilter(r *http.Request) *requestFilter {
type targetsStatusResult struct { type targetsStatusResult struct {
jobTargetsStatuses []*jobTargetsStatuses jobTargetsStatuses []*jobTargetsStatuses
droppedTargetsLabels [][]prompbmarshal.Label droppedTargetsLabels []*promutils.Labels
emptyJobs []string emptyJobs []string
err error err error
} }
type targetLabels struct { type targetLabels struct {
up bool up bool
discoveredLabels []prompbmarshal.Label discoveredLabels *promutils.Labels
labels []prompbmarshal.Label labels *promutils.Labels
} }
type targetLabelsByJob struct { type targetLabelsByJob struct {
jobName string jobName string
@ -534,7 +536,7 @@ func (tsr *targetsStatusResult) getTargetLabelsByJob() []*targetLabelsByJob {
} }
} }
for _, labels := range tsr.droppedTargetsLabels { for _, labels := range tsr.droppedTargetsLabels {
jobName := promrelabel.GetLabelValueByName(labels, "job") jobName := labels.Get("job")
m := byJob[jobName] m := byJob[jobName]
if m == nil { if m == nil {
m = &targetLabelsByJob{ m = &targetLabelsByJob{

View file

@ -2,8 +2,7 @@
"net/url" "net/url"
"time" "time"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal" "github.com/VictoriaMetrics/VictoriaMetrics/lib/promutils"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/promrelabel"
) %} ) %}
{% stripspace %} {% stripspace %}
@ -22,8 +21,8 @@
{%s= "\t" %} {%s= "\t" %}
state={% if ts.up %}up{% else %}down{% endif %},{% space %} state={% if ts.up %}up{% else %}down{% endif %},{% space %}
endpoint={%s= ts.sw.Config.ScrapeURL %},{% space %} endpoint={%s= ts.sw.Config.ScrapeURL %},{% space %}
labels={%s= promLabelsString(promrelabel.FinalizeLabels(nil, ts.sw.Config.Labels)) %},{% space %} labels={%s= ts.sw.Config.Labels.String() %},{% space %}
{% if filter.showOriginalLabels %}originalLabels={%s= promLabelsString(ts.sw.Config.OriginalLabels) %},{% space %}{% endif %} {% if filter.showOriginalLabels %}originalLabels={%s= ts.sw.Config.OriginalLabels.String() %},{% space %}{% endif %}
scrapes_total={%d ts.scrapesTotal %},{% space %} scrapes_total={%d ts.scrapesTotal %},{% space %}
scrapes_failed={%d ts.scrapesFailed %},{% space %} scrapes_failed={%d ts.scrapesFailed %},{% space %}
last_scrape={%d int(ts.getDurationFromLastScrape().Milliseconds()) %}ms ago,{% space %} last_scrape={%d int(ts.getDurationFromLastScrape().Milliseconds()) %}ms ago,{% space %}
@ -258,10 +257,10 @@
<td class="labels"> <td class="labels">
<div title="click to show original labels" <div title="click to show original labels"
onclick="document.getElementById('original-labels-{%s targetID %}').style.display='block'"> onclick="document.getElementById('original-labels-{%s targetID %}').style.display='block'">
{%= formatLabel(promrelabel.FinalizeLabels(nil, ts.sw.Config.Labels)) %} {%= formatLabels(ts.sw.Config.Labels) %}
</div> </div>
<div style="display:none" id="original-labels-{%s targetID %}"> <div style="display:none" id="original-labels-{%s targetID %}">
{%= formatLabel(ts.sw.Config.OriginalLabels) %} {%= formatLabels(ts.sw.Config.OriginalLabels) %}
</div> </div>
</td> </td>
<td>{%d ts.scrapesTotal %}</td> <td>{%d ts.scrapesTotal %}</td>
@ -314,7 +313,7 @@
<tr <tr
{% if !t.up %} {% if !t.up %}
{% space %}role="alert"{% space %} {% space %}role="alert"{% space %}
{% if len(t.labels) > 0 %} {% if t.labels.Len() > 0 %}
class="alert alert-danger" class="alert alert-danger"
{% else %} {% else %}
class="alert alert-warning" class="alert alert-warning"
@ -324,17 +323,17 @@
<td> <td>
{% if t.up %} {% if t.up %}
<span class="badge bg-success">UP</span> <span class="badge bg-success">UP</span>
{% elseif len(t.labels) > 0 %} {% elseif t.labels.Len() > 0 %}
<span class="badge bg-danger">DOWN</span> <span class="badge bg-danger">DOWN</span>
{% else %} {% else %}
<span class="badge bg-warning">DROPPED</span> <span class="badge bg-warning">DROPPED</span>
{% endif %} {% endif %}
</td> </td>
<td class="labels"> <td class="labels">
{%= formatLabel(t.discoveredLabels) %} {%= formatLabels(t.discoveredLabels) %}
</td> </td>
<td class="labels"> <td class="labels">
{%= formatLabel(promrelabel.FinalizeLabels(nil, t.labels)) %} {%= formatLabels(t.labels) %}
</td> </td>
</tr> </tr>
{% endfor %} {% endfor %}
@ -376,11 +375,12 @@
{%s qa.Encode() %} {%s qa.Encode() %}
{% endfunc %} {% endfunc %}
{% func formatLabel(labels []prompbmarshal.Label) %} {% func formatLabels(labels *promutils.Labels) %}
{% code labelsList := labels.GetLabels() %}
{ {
{% for i, label := range labels %} {% for i, label := range labelsList %}
{%s label.Name %}={%q label.Value %} {%s label.Name %}={%q label.Value %}
{% if i+1 < len(labels) %},{% space %}{% endif %} {% if i+1 < len(labelsList) %},{% space %}{% endif %}
{% endfor %} {% endfor %}
} }
{% endfunc %} {% endfunc %}

File diff suppressed because it is too large Load diff

328
lib/promutils/labels.go Normal file
View file

@ -0,0 +1,328 @@
package promutils
import (
"encoding/json"
"fmt"
"sort"
"strconv"
"strings"
"sync"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/prompbmarshal"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/protoparser/prometheus"
)
// Labels contains Prometheus labels.
type Labels struct {
Labels []prompbmarshal.Label
}
// NewLabels returns Labels with the given capacity.
func NewLabels(capacity int) *Labels {
return &Labels{
Labels: make([]prompbmarshal.Label, 0, capacity),
}
}
// NewLabelsFromMap returns Labels generated from m.
func NewLabelsFromMap(m map[string]string) *Labels {
var x Labels
x.InitFromMap(m)
return &x
}
// MarshalYAML implements yaml.Marshaler interface.
func (x *Labels) MarshalYAML() (interface{}, error) {
m := x.toMap()
return m, nil
}
// UnmarshalYAML implements yaml.Unmarshaler interface.
func (x *Labels) UnmarshalYAML(unmarshal func(interface{}) error) error {
var m map[string]string
if err := unmarshal(&m); err != nil {
return err
}
x.InitFromMap(m)
return nil
}
// MarshalJSON returns JSON respresentation for x.
func (x *Labels) MarshalJSON() ([]byte, error) {
m := x.toMap()
return json.Marshal(m)
}
// UnmarshalJSON unmarshals JSON from data.
func (x *Labels) UnmarshalJSON(data []byte) error {
var m map[string]string
if err := json.Unmarshal(data, &m); err != nil {
return err
}
x.InitFromMap(m)
return nil
}
// InitFromMap initializes x from m.
func (x *Labels) InitFromMap(m map[string]string) {
x.Reset()
for name, value := range m {
x.Add(name, value)
}
x.Sort()
}
func (x *Labels) toMap() map[string]string {
labels := x.GetLabels()
m := make(map[string]string, len(labels))
for _, label := range labels {
m[label.Name] = label.Value
}
return m
}
// GetLabels returns the list of labels from x.
func (x *Labels) GetLabels() []prompbmarshal.Label {
if x == nil {
return nil
}
return x.Labels
}
// String returns string representation of x.
func (x *Labels) String() string {
labels := x.GetLabels()
// Calculate the required memory for storing serialized labels.
n := 2 // for `{...}`
for _, label := range labels {
n += len(label.Name) + len(label.Value)
n += 4 // for `="...",`
}
b := make([]byte, 0, n)
b = append(b, '{')
for i, label := range labels {
b = append(b, label.Name...)
b = append(b, '=')
b = strconv.AppendQuote(b, label.Value)
if i+1 < len(labels) {
b = append(b, ',')
}
}
b = append(b, '}')
return bytesutil.ToUnsafeString(b)
}
// Reset resets x.
func (x *Labels) Reset() {
cleanLabels(x.Labels)
x.Labels = x.Labels[:0]
}
// Clone returns a clone of x.
func (x *Labels) Clone() *Labels {
srcLabels := x.GetLabels()
labels := append([]prompbmarshal.Label{}, srcLabels...)
return &Labels{
Labels: labels,
}
}
// Sort sorts x labels in alphabetical order of their names.
func (x *Labels) Sort() {
if !sort.IsSorted(x) {
sort.Sort(x)
}
}
// SortStable sorts x labels in alphabetical order of their name using stable sort.
func (x *Labels) SortStable() {
if !sort.IsSorted(x) {
sort.Stable(x)
}
}
// Len returns the number of labels in x.
func (x *Labels) Len() int {
labels := x.GetLabels()
return len(labels)
}
// Less compares label names at i and j index.
func (x *Labels) Less(i, j int) bool {
labels := x.Labels
return labels[i].Name < labels[j].Name
}
// Swap swaps labels at i and j index.
func (x *Labels) Swap(i, j int) {
labels := x.Labels
labels[i], labels[j] = labels[j], labels[i]
}
// Add adds name=value label to x.
func (x *Labels) Add(name, value string) {
x.Labels = append(x.Labels, prompbmarshal.Label{
Name: name,
Value: value,
})
}
// AddFrom adds src labels to x.
func (x *Labels) AddFrom(src *Labels) {
for _, label := range src.GetLabels() {
x.Add(label.Name, label.Value)
}
}
// Get returns value for label with the given name.
func (x *Labels) Get(name string) string {
labels := x.GetLabels()
for _, label := range labels {
if label.Name == name {
return label.Value
}
}
return ""
}
// InternStrings interns all the strings used in x labels.
func (x *Labels) InternStrings() {
labels := x.GetLabels()
for _, label := range labels {
label.Name = bytesutil.InternString(label.Name)
label.Value = bytesutil.InternString(label.Value)
}
}
// RemoveDuplicates removes labels with duplicate names.
func (x *Labels) RemoveDuplicates() {
if x.Len() < 2 {
return
}
// Remove duplicate labels if any.
// Stable sorting is needed in order to preserve the order for labels with identical names.
// This is needed in order to remove labels with duplicate names other than the last one.
x.SortStable()
labels := x.Labels
prevName := labels[0].Name
hasDuplicateLabels := false
for _, label := range labels[1:] {
if label.Name == prevName {
hasDuplicateLabels = true
break
}
prevName = label.Name
}
if !hasDuplicateLabels {
return
}
prevName = labels[0].Name
tmp := labels[:1]
for _, label := range labels[1:] {
if label.Name == prevName {
tmp[len(tmp)-1] = label
} else {
tmp = append(tmp, label)
prevName = label.Name
}
}
cleanLabels(labels[len(tmp):])
x.Labels = tmp
}
// RemoveMetaLabels removes all the `__meta_` labels from x.
//
// See https://www.robustperception.io/life-of-a-label fo details.
func (x *Labels) RemoveMetaLabels() {
src := x.Labels
dst := x.Labels[:0]
for _, label := range src {
if strings.HasPrefix(label.Name, "__meta_") {
continue
}
dst = append(dst, label)
}
cleanLabels(src[len(dst):])
x.Labels = dst
}
// RemoveLabelsWithDoubleUnderscorePrefix removes labels with "__" prefix from x.
func (x *Labels) RemoveLabelsWithDoubleUnderscorePrefix() {
src := x.Labels
dst := x.Labels[:0]
for _, label := range src {
name := label.Name
// A hack: do not delete __vm_filepath label, since it is used by internal logic for FileSDConfig.
if strings.HasPrefix(name, "__") && name != "__vm_filepath" {
continue
}
dst = append(dst, label)
}
cleanLabels(src[len(dst):])
x.Labels = dst
}
func cleanLabels(labels []prompbmarshal.Label) {
for i := range labels {
label := &labels[i]
label.Name = ""
label.Value = ""
}
}
// GetLabels returns and empty Labels instance from the pool.
//
// The returned Labels instance must be returned to pool via PutLabels() when no longer needed.
func GetLabels() *Labels {
v := labelsPool.Get()
if v == nil {
return &Labels{}
}
return v.(*Labels)
}
// PutLabels returns x, which has been obtained via GetLabels(), to the pool.
//
// The x mustn't be used after returning to the pool.
func PutLabels(x *Labels) {
x.Reset()
labelsPool.Put(x)
}
var labelsPool sync.Pool
// NewLabelsFromString creates labels from s, which can have the form `metric{labels}`.
//
// This function must be used only in tests
func NewLabelsFromString(metricWithLabels string) *Labels {
stripDummyMetric := false
if strings.HasPrefix(metricWithLabels, "{") {
// Add a dummy metric name, since the parser needs it
metricWithLabels = "dummy_metric" + metricWithLabels
stripDummyMetric = true
}
// add a value to metricWithLabels, so it could be parsed by prometheus protocol parser.
s := metricWithLabels + " 123"
var rows prometheus.Rows
var err error
rows.UnmarshalWithErrLogger(s, func(s string) {
err = fmt.Errorf("error during metric parse: %s", s)
})
if err != nil {
logger.Panicf("BUG: cannot parse %q: %s", metricWithLabels, err)
}
if len(rows.Rows) != 1 {
logger.Panicf("BUG: unexpected number of rows parsed; got %d; want 1", len(rows.Rows))
}
r := rows.Rows[0]
var x Labels
if !stripDummyMetric {
x.Add("__name__", r.Metric)
}
for _, tag := range r.Tags {
x.Add(tag.Key, tag.Value)
}
return &x
}

View file

@ -0,0 +1,177 @@
package promutils
import (
"encoding/json"
"testing"
"gopkg.in/yaml.v2"
)
func TestLabels(t *testing.T) {
x := NewLabels(2)
x.Add("job", "bar")
x.Add("instance", "foo")
v := x.Get("instance")
if v != "foo" {
t.Fatalf("unexpected value obtained; got %q; want %q", v, "foo")
}
v = x.Get("non-existing-label")
if v != "" {
t.Fatalf("unexpected non-empty value obtained for non-existing label: %q", v)
}
n := x.Len()
if n != 2 {
t.Fatalf("unexpected labels len; got %d; want 2", n)
}
x.Sort()
x.SortStable()
s, err := yaml.Marshal(x)
if err != nil {
t.Fatalf("unexpected error in yaml.Marshal: %s", err)
}
sExpected := "instance: foo\njob: bar\n"
if string(s) != sExpected {
t.Fatalf("unexpected marshaled value;\ngot\n%s\nwant\n%q", s, sExpected)
}
x1 := GetLabels()
if err := yaml.Unmarshal(s, &x1); err != nil {
t.Fatalf("unexpected error in yaml.Unmarshal: %s", err)
}
x1.InternStrings()
s, err = yaml.Marshal(&x1)
if err != nil {
t.Fatalf("unexpected error in yaml.Marshal: %s", err)
}
if string(s) != sExpected {
t.Fatalf("unexpected marshaled value;\ngot\n%s\nwant\n%q", s, sExpected)
}
PutLabels(x1)
x1 = nil
if n = x1.Len(); n != 0 {
t.Fatalf("unexpected len for empty labels: %d", n)
}
x1 = GetLabels()
x2 := x.Clone()
s, err = yaml.Marshal(x2)
if err != nil {
t.Fatalf("cannot marshal cloned labels")
}
if string(s) != sExpected {
t.Fatalf("unexpected marshaled value;\ngot\n%s\nwant\n%q", s, sExpected)
}
s2 := x2.String()
s2Expected := `{instance="foo",job="bar"}`
if s2 != s2Expected {
t.Fatalf("unexpected string representation for labels;\ngot\n%s\nwant\n%s", s2, s2Expected)
}
}
func TestLabelsUnmarshalYAMLFailure(t *testing.T) {
f := func(s string) {
t.Helper()
var x Labels
if err := yaml.Unmarshal([]byte(s), &x); err == nil {
t.Fatalf("expecting non-nil error")
}
}
f("foobar")
f("[foo,bar]")
f("{foo:[bar]}")
f("[aa")
}
func TestLabelsUnmarshalJSONSuccess(t *testing.T) {
f := func(s string) {
t.Helper()
var x Labels
if err := json.Unmarshal([]byte(s), &x); err != nil {
t.Fatalf("unexpected error in json.Unmarshal: %s", err)
}
data, err := json.Marshal(&x)
if err != nil {
t.Fatalf("json.Marshal error: %s", err)
}
if string(data) != s {
t.Fatalf("unexpected marshaled JSON;\ngot\n%s\nwant\n%s", data, s)
}
}
f(`{}`)
f(`{"foo":"bar"}`)
f(`{"a":"y","x":"b"}`)
}
func TestLabelsUnmarshalJSONFailure(t *testing.T) {
f := func(s string) {
t.Helper()
var x Labels
if err := json.Unmarshal([]byte(s), &x); err == nil {
t.Fatalf("expecting non-nil error")
}
}
f("foobar")
f("[1,2]")
f(`{"foo":123}`)
f(`{"foo"`)
f(`"ff`)
}
func TestLabelsRemoveDuplicates(t *testing.T) {
var x Labels
x.Add("foo", "bar")
x.Add("foo", "baz")
x.Add("foo", "123")
x.Add("bar", "a")
x.RemoveDuplicates()
s := x.String()
sExpected := `{bar="a",foo="123"}`
if s != sExpected {
t.Fatalf("unexpected result;\ngot\n%s\nwant\n%s", s, sExpected)
}
}
func TestLabelsAddFrom(t *testing.T) {
var a, b Labels
a.Add("z", "x")
a.Add("foo", "bar")
b.Add("foo", "baz")
b.Add("x", "y")
a.AddFrom(&b)
a.RemoveDuplicates()
s := a.String()
sExpected := `{foo="baz",x="y",z="x"}`
if s != sExpected {
t.Fatalf("unexpected result;\ngot\n%s\nwant\n%s", s, sExpected)
}
}
func TestLabelsRemoveMetaLabels(t *testing.T) {
f := func(metric, resultExpected string) {
t.Helper()
labels := NewLabelsFromString(metric)
labels.RemoveMetaLabels()
result := labels.String()
if result != resultExpected {
t.Fatalf("unexpected result of RemoveMetaLabels;\ngot\n%s\nwant\n%s", result, resultExpected)
}
}
f(`{}`, `{}`)
f(`{foo="bar"}`, `{foo="bar"}`)
f(`{__meta_foo="bar"}`, `{}`)
f(`{__meta_foo="bdffr",foo="bar",__meta_xxx="basd"}`, `{foo="bar"}`)
}
func TestLabelsRemoveLabelsWithDoubleUnderscorePrefix(t *testing.T) {
f := func(metric, resultExpected string) {
t.Helper()
labels := NewLabelsFromString(metric)
labels.RemoveLabelsWithDoubleUnderscorePrefix()
result := labels.String()
if result != resultExpected {
t.Fatalf("unexpected result of RemoveLabelsWithDoubleUnderscorePrefix;\ngot\n%s\nwant\n%s", result, resultExpected)
}
}
f(`{}`, `{}`)
f(`{foo="bar"}`, `{foo="bar"}`)
f(`{__meta_foo="bar",a="b",__name__="foo",__vm_filepath="aa"}`, `{a="b",__vm_filepath="aa"}`)
f(`{__meta_foo="bdffr",foo="bar",__meta_xxx="basd"}`, `{foo="bar"}`)
}

View file

@ -0,0 +1,20 @@
package promutils
import (
"testing"
)
func BenchmarkLabelsInternStrings(b *testing.B) {
b.ReportAllocs()
b.SetBytes(1)
b.RunParallel(func(pb *testing.PB) {
labels := NewLabelsFromMap(map[string]string{
"job": "node-exporter",
"instance": "foo.bar.baz:1234",
"__meta_kubernetes_namespace": "default",
})
for pb.Next() {
labels.InternStrings()
}
})
}