1
0
Fork 0

Improve performances when Prometheus metrics are enabled

This commit is contained in:
Julien Salleyron 2022-07-07 18:00:09 +02:00 committed by GitHub
parent 8f6463ba7a
commit 0231db05b4
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
5 changed files with 270 additions and 251 deletions

View file

@ -4,8 +4,6 @@ import (
"context"
"errors"
"net/http"
"sort"
"strings"
"sync"
"time"
@ -15,7 +13,6 @@ import (
"github.com/prometheus/client_golang/prometheus/promhttp"
"github.com/traefik/traefik/v2/pkg/config/dynamic"
"github.com/traefik/traefik/v2/pkg/log"
"github.com/traefik/traefik/v2/pkg/safe"
"github.com/traefik/traefik/v2/pkg/types"
)
@ -111,37 +108,33 @@ func initStandardRegistry(config *types.Prometheus) Registry {
buckets = config.Buckets
}
safe.Go(func() {
promState.ListenValueUpdates()
})
configReloads := newCounterFrom(promState.collectors, stdprometheus.CounterOpts{
configReloads := newCounterFrom(stdprometheus.CounterOpts{
Name: configReloadsTotalName,
Help: "Config reloads",
}, []string{})
configReloadsFailures := newCounterFrom(promState.collectors, stdprometheus.CounterOpts{
configReloadsFailures := newCounterFrom(stdprometheus.CounterOpts{
Name: configReloadsFailuresTotalName,
Help: "Config failure reloads",
}, []string{})
lastConfigReloadSuccess := newGaugeFrom(promState.collectors, stdprometheus.GaugeOpts{
lastConfigReloadSuccess := newGaugeFrom(stdprometheus.GaugeOpts{
Name: configLastReloadSuccessName,
Help: "Last config reload success",
}, []string{})
lastConfigReloadFailure := newGaugeFrom(promState.collectors, stdprometheus.GaugeOpts{
lastConfigReloadFailure := newGaugeFrom(stdprometheus.GaugeOpts{
Name: configLastReloadFailureName,
Help: "Last config reload failure",
}, []string{})
tlsCertsNotAfterTimestamp := newGaugeFrom(promState.collectors, stdprometheus.GaugeOpts{
tlsCertsNotAfterTimestamp := newGaugeFrom(stdprometheus.GaugeOpts{
Name: tlsCertsNotAfterTimestamp,
Help: "Certificate expiration timestamp",
}, []string{"cn", "serial", "sans"})
promState.describers = []func(chan<- *stdprometheus.Desc){
configReloads.cv.Describe,
configReloadsFailures.cv.Describe,
lastConfigReloadSuccess.gv.Describe,
lastConfigReloadFailure.gv.Describe,
tlsCertsNotAfterTimestamp.gv.Describe,
promState.vectors = []vector{
configReloads.cv,
configReloadsFailures.cv,
lastConfigReloadSuccess.gv,
lastConfigReloadFailure.gv,
tlsCertsNotAfterTimestamp.gv,
}
reg := &standardRegistry{
@ -156,30 +149,30 @@ func initStandardRegistry(config *types.Prometheus) Registry {
}
if config.AddEntryPointsLabels {
entryPointReqs := newCounterFrom(promState.collectors, stdprometheus.CounterOpts{
entryPointReqs := newCounterFrom(stdprometheus.CounterOpts{
Name: entryPointReqsTotalName,
Help: "How many HTTP requests processed on an entrypoint, partitioned by status code, protocol, and method.",
}, []string{"code", "method", "protocol", "entrypoint"})
entryPointReqsTLS := newCounterFrom(promState.collectors, stdprometheus.CounterOpts{
entryPointReqsTLS := newCounterFrom(stdprometheus.CounterOpts{
Name: entryPointReqsTLSTotalName,
Help: "How many HTTP requests with TLS processed on an entrypoint, partitioned by TLS Version and TLS cipher Used.",
}, []string{"tls_version", "tls_cipher", "entrypoint"})
entryPointReqDurations := newHistogramFrom(promState.collectors, stdprometheus.HistogramOpts{
entryPointReqDurations := newHistogramFrom(stdprometheus.HistogramOpts{
Name: entryPointReqDurationName,
Help: "How long it took to process the request on an entrypoint, partitioned by status code, protocol, and method.",
Buckets: buckets,
}, []string{"code", "method", "protocol", "entrypoint"})
entryPointOpenConns := newGaugeFrom(promState.collectors, stdprometheus.GaugeOpts{
entryPointOpenConns := newGaugeFrom(stdprometheus.GaugeOpts{
Name: entryPointOpenConnsName,
Help: "How many open connections exist on an entrypoint, partitioned by method and protocol.",
}, []string{"method", "protocol", "entrypoint"})
promState.describers = append(promState.describers, []func(chan<- *stdprometheus.Desc){
entryPointReqs.cv.Describe,
entryPointReqsTLS.cv.Describe,
entryPointReqDurations.hv.Describe,
entryPointOpenConns.gv.Describe,
}...)
promState.vectors = append(promState.vectors,
entryPointReqs.cv,
entryPointReqsTLS.cv,
entryPointReqDurations.hv,
entryPointOpenConns.gv,
)
reg.entryPointReqsCounter = entryPointReqs
reg.entryPointReqsTLSCounter = entryPointReqsTLS
@ -188,30 +181,30 @@ func initStandardRegistry(config *types.Prometheus) Registry {
}
if config.AddRoutersLabels {
routerReqs := newCounterFrom(promState.collectors, stdprometheus.CounterOpts{
routerReqs := newCounterFrom(stdprometheus.CounterOpts{
Name: routerReqsTotalName,
Help: "How many HTTP requests are processed on a router, partitioned by service, status code, protocol, and method.",
}, []string{"code", "method", "protocol", "router", "service"})
routerReqsTLS := newCounterFrom(promState.collectors, stdprometheus.CounterOpts{
routerReqsTLS := newCounterFrom(stdprometheus.CounterOpts{
Name: routerReqsTLSTotalName,
Help: "How many HTTP requests with TLS are processed on a router, partitioned by service, TLS Version, and TLS cipher Used.",
}, []string{"tls_version", "tls_cipher", "router", "service"})
routerReqDurations := newHistogramFrom(promState.collectors, stdprometheus.HistogramOpts{
routerReqDurations := newHistogramFrom(stdprometheus.HistogramOpts{
Name: routerReqDurationName,
Help: "How long it took to process the request on a router, partitioned by service, status code, protocol, and method.",
Buckets: buckets,
}, []string{"code", "method", "protocol", "router", "service"})
routerOpenConns := newGaugeFrom(promState.collectors, stdprometheus.GaugeOpts{
routerOpenConns := newGaugeFrom(stdprometheus.GaugeOpts{
Name: routerOpenConnsName,
Help: "How many open connections exist on a router, partitioned by service, method, and protocol.",
}, []string{"method", "protocol", "router", "service"})
promState.describers = append(promState.describers, []func(chan<- *stdprometheus.Desc){
routerReqs.cv.Describe,
routerReqsTLS.cv.Describe,
routerReqDurations.hv.Describe,
routerOpenConns.gv.Describe,
}...)
promState.vectors = append(promState.vectors,
routerReqs.cv,
routerReqsTLS.cv,
routerReqDurations.hv,
routerOpenConns.gv,
)
reg.routerReqsCounter = routerReqs
reg.routerReqsTLSCounter = routerReqsTLS
reg.routerReqDurationHistogram, _ = NewHistogramWithScale(routerReqDurations, time.Second)
@ -219,40 +212,40 @@ func initStandardRegistry(config *types.Prometheus) Registry {
}
if config.AddServicesLabels {
serviceReqs := newCounterFrom(promState.collectors, stdprometheus.CounterOpts{
serviceReqs := newCounterFrom(stdprometheus.CounterOpts{
Name: serviceReqsTotalName,
Help: "How many HTTP requests processed on a service, partitioned by status code, protocol, and method.",
}, []string{"code", "method", "protocol", "service"})
serviceReqsTLS := newCounterFrom(promState.collectors, stdprometheus.CounterOpts{
serviceReqsTLS := newCounterFrom(stdprometheus.CounterOpts{
Name: serviceReqsTLSTotalName,
Help: "How many HTTP requests with TLS processed on a service, partitioned by TLS version and TLS cipher.",
}, []string{"tls_version", "tls_cipher", "service"})
serviceReqDurations := newHistogramFrom(promState.collectors, stdprometheus.HistogramOpts{
serviceReqDurations := newHistogramFrom(stdprometheus.HistogramOpts{
Name: serviceReqDurationName,
Help: "How long it took to process the request on a service, partitioned by status code, protocol, and method.",
Buckets: buckets,
}, []string{"code", "method", "protocol", "service"})
serviceOpenConns := newGaugeFrom(promState.collectors, stdprometheus.GaugeOpts{
serviceOpenConns := newGaugeFrom(stdprometheus.GaugeOpts{
Name: serviceOpenConnsName,
Help: "How many open connections exist on a service, partitioned by method and protocol.",
}, []string{"method", "protocol", "service"})
serviceRetries := newCounterFrom(promState.collectors, stdprometheus.CounterOpts{
serviceRetries := newCounterFrom(stdprometheus.CounterOpts{
Name: serviceRetriesTotalName,
Help: "How many request retries happened on a service.",
}, []string{"service"})
serviceServerUp := newGaugeFrom(promState.collectors, stdprometheus.GaugeOpts{
serviceServerUp := newGaugeFrom(stdprometheus.GaugeOpts{
Name: serviceServerUpName,
Help: "service server is up, described by gauge value of 0 or 1.",
}, []string{"service", "url"})
promState.describers = append(promState.describers, []func(chan<- *stdprometheus.Desc){
serviceReqs.cv.Describe,
serviceReqsTLS.cv.Describe,
serviceReqDurations.hv.Describe,
serviceOpenConns.gv.Describe,
serviceRetries.cv.Describe,
serviceServerUp.gv.Describe,
}...)
promState.vectors = append(promState.vectors,
serviceReqs.cv,
serviceReqsTLS.cv,
serviceReqDurations.hv,
serviceOpenConns.gv,
serviceRetries.cv,
serviceServerUp.gv,
)
reg.serviceReqsCounter = serviceReqs
reg.serviceReqsTLSCounter = serviceReqsTLS
@ -287,64 +280,93 @@ func registerPromState(ctx context.Context) bool {
// It then converts the configuration to the optimized package internal format
// and sets it to the promState.
func OnConfigurationUpdate(conf dynamic.Configuration, entryPoints []string) {
dynamicConfig := newDynamicConfig()
dynCfg := newDynamicConfig()
for _, value := range entryPoints {
dynamicConfig.entryPoints[value] = true
dynCfg.entryPoints[value] = true
}
if conf.HTTP == nil {
promState.SetDynamicConfig(dynCfg)
return
}
for name := range conf.HTTP.Routers {
dynamicConfig.routers[name] = true
dynCfg.routers[name] = true
}
for serviceName, service := range conf.HTTP.Services {
dynamicConfig.services[serviceName] = make(map[string]bool)
dynCfg.services[serviceName] = make(map[string]bool)
if service.LoadBalancer != nil {
for _, server := range service.LoadBalancer.Servers {
dynamicConfig.services[serviceName][server.URL] = true
dynCfg.services[serviceName][server.URL] = true
}
}
}
promState.SetDynamicConfig(dynamicConfig)
promState.SetDynamicConfig(dynCfg)
}
func newPrometheusState() *prometheusState {
return &prometheusState{
collectors: make(chan *collector),
dynamicConfig: newDynamicConfig(),
state: make(map[string]*collector),
deletedURLs: make(map[string]string),
}
}
type prometheusState struct {
collectors chan *collector
describers []func(ch chan<- *stdprometheus.Desc)
type vector interface {
stdprometheus.Collector
DeletePartialMatch(labels stdprometheus.Labels) int
}
mtx sync.Mutex
dynamicConfig *dynamicConfig
state map[string]*collector
type prometheusState struct {
vectors []vector
mtx sync.Mutex
dynamicConfig *dynamicConfig
deletedEP []string
deletedRouters []string
deletedServices []string
deletedURLs map[string]string
}
func (ps *prometheusState) SetDynamicConfig(dynamicConfig *dynamicConfig) {
ps.mtx.Lock()
defer ps.mtx.Unlock()
ps.dynamicConfig = dynamicConfig
}
func (ps *prometheusState) ListenValueUpdates() {
for collector := range ps.collectors {
ps.mtx.Lock()
ps.state[collector.id] = collector
ps.mtx.Unlock()
for ep := range ps.dynamicConfig.entryPoints {
if _, ok := dynamicConfig.entryPoints[ep]; !ok {
ps.deletedEP = append(ps.deletedEP, ep)
}
}
for router := range ps.dynamicConfig.routers {
if _, ok := dynamicConfig.routers[router]; !ok {
ps.deletedRouters = append(ps.deletedRouters, router)
}
}
for service, serV := range ps.dynamicConfig.services {
actualService, ok := dynamicConfig.services[service]
if !ok {
ps.deletedServices = append(ps.deletedServices, service)
continue
}
for url := range serV {
if _, ok := actualService[url]; !ok {
ps.deletedURLs[service] = url
}
}
}
ps.dynamicConfig = dynamicConfig
}
// Describe implements prometheus.Collector and simply calls
// the registered describer functions.
func (ps *prometheusState) Describe(ch chan<- *stdprometheus.Desc) {
for _, desc := range ps.describers {
desc(ch)
for _, v := range ps.vectors {
v.Describe(ch)
}
}
@ -354,49 +376,52 @@ func (ps *prometheusState) Describe(ch chan<- *stdprometheus.Desc) {
// The removal happens only after their Collect method was called to ensure that
// also those metrics will be exported on the current scrape.
func (ps *prometheusState) Collect(ch chan<- stdprometheus.Metric) {
for _, v := range ps.vectors {
v.Collect(ch)
}
ps.mtx.Lock()
defer ps.mtx.Unlock()
var outdatedKeys []string
for key, cs := range ps.state {
cs.collector.Collect(ch)
if ps.isOutdated(cs) {
outdatedKeys = append(outdatedKeys, key)
for _, ep := range ps.deletedEP {
if !ps.dynamicConfig.hasEntryPoint(ep) {
ps.DeletePartialMatch(map[string]string{"entrypoint": ep})
}
}
for _, key := range outdatedKeys {
ps.state[key].delete()
delete(ps.state, key)
for _, router := range ps.deletedRouters {
if !ps.dynamicConfig.hasRouter(router) {
ps.DeletePartialMatch(map[string]string{"router": router})
}
}
for _, service := range ps.deletedServices {
if !ps.dynamicConfig.hasService(service) {
ps.DeletePartialMatch(map[string]string{"service": service})
}
}
for service, url := range ps.deletedURLs {
if !ps.dynamicConfig.hasServerURL(service, url) {
ps.DeletePartialMatch(map[string]string{"service": service, "url": url})
}
}
ps.deletedEP = nil
ps.deletedRouters = nil
ps.deletedServices = nil
ps.deletedURLs = make(map[string]string)
}
// isOutdated checks whether the passed collector has labels that mark
// it as belonging to an outdated configuration of Traefik.
func (ps *prometheusState) isOutdated(collector *collector) bool {
labels := collector.labels
if entrypointName, ok := labels["entrypoint"]; ok && !ps.dynamicConfig.hasEntryPoint(entrypointName) {
return true
// DeletePartialMatch deletes all metrics where the variable labels contain all of those passed in as labels.
// The order of the labels does not matter.
// It returns the number of metrics deleted.
func (ps *prometheusState) DeletePartialMatch(labels stdprometheus.Labels) int {
var count int
for _, elem := range ps.vectors {
count += elem.DeletePartialMatch(labels)
}
if routerName, ok := labels["router"]; ok {
if !ps.dynamicConfig.hasRouter(routerName) {
return true
}
}
if serviceName, ok := labels["service"]; ok {
if !ps.dynamicConfig.hasService(serviceName) {
return true
}
if url, ok := labels["url"]; ok && !ps.dynamicConfig.hasServerURL(serviceName, url) {
return true
}
}
return false
return count
}
func newDynamicConfig() *dynamicConfig {
@ -440,42 +465,15 @@ func (d *dynamicConfig) hasServerURL(serviceName, serverURL string) bool {
return false
}
func newCollector(metricName string, labels stdprometheus.Labels, c stdprometheus.Collector, deleteFn func()) *collector {
return &collector{
id: buildMetricID(metricName, labels),
labels: labels,
collector: c,
delete: deleteFn,
}
}
// collector wraps a Collector object from the Prometheus client library.
// It adds information on how many generations this metric should be present
// in the /metrics output, relative to the time it was last tracked.
type collector struct {
id string
labels stdprometheus.Labels
collector stdprometheus.Collector
delete func()
}
func buildMetricID(metricName string, labels stdprometheus.Labels) string {
var labelNamesValues []string
for name, value := range labels {
labelNamesValues = append(labelNamesValues, name, value)
}
sort.Strings(labelNamesValues)
return metricName + ":" + strings.Join(labelNamesValues, "|")
}
func newCounterFrom(collectors chan<- *collector, opts stdprometheus.CounterOpts, labelNames []string) *counter {
func newCounterFrom(opts stdprometheus.CounterOpts, labelNames []string) *counter {
cv := stdprometheus.NewCounterVec(opts, labelNames)
c := &counter{
name: opts.Name,
cv: cv,
collectors: collectors,
name: opts.Name,
cv: cv,
labelNamesValues: make([]string, 0, 16),
}
if len(labelNames) == 0 {
c.collector = cv.WithLabelValues()
c.Add(0)
}
return c
@ -485,39 +483,37 @@ type counter struct {
name string
cv *stdprometheus.CounterVec
labelNamesValues labelNamesValues
collectors chan<- *collector
collector stdprometheus.Counter
}
func (c *counter) With(labelValues ...string) metrics.Counter {
lnv := c.labelNamesValues.With(labelValues...)
return &counter{
name: c.name,
cv: c.cv,
labelNamesValues: c.labelNamesValues.With(labelValues...),
collectors: c.collectors,
labelNamesValues: lnv,
collector: c.cv.With(lnv.ToLabels()),
}
}
func (c *counter) Add(delta float64) {
labels := c.labelNamesValues.ToLabels()
collector := c.cv.With(labels)
collector.Add(delta)
c.collectors <- newCollector(c.name, labels, collector, func() {
c.cv.Delete(labels)
})
c.collector.Add(delta)
}
func (c *counter) Describe(ch chan<- *stdprometheus.Desc) {
c.cv.Describe(ch)
}
func newGaugeFrom(collectors chan<- *collector, opts stdprometheus.GaugeOpts, labelNames []string) *gauge {
func newGaugeFrom(opts stdprometheus.GaugeOpts, labelNames []string) *gauge {
gv := stdprometheus.NewGaugeVec(opts, labelNames)
g := &gauge{
name: opts.Name,
gv: gv,
collectors: collectors,
name: opts.Name,
gv: gv,
labelNamesValues: make([]string, 0, 16),
}
if len(labelNames) == 0 {
g.collector = gv.WithLabelValues()
g.Set(0)
}
return g
@ -527,46 +523,37 @@ type gauge struct {
name string
gv *stdprometheus.GaugeVec
labelNamesValues labelNamesValues
collectors chan<- *collector
collector stdprometheus.Gauge
}
func (g *gauge) With(labelValues ...string) metrics.Gauge {
lnv := g.labelNamesValues.With(labelValues...)
return &gauge{
name: g.name,
gv: g.gv,
labelNamesValues: g.labelNamesValues.With(labelValues...),
collectors: g.collectors,
labelNamesValues: lnv,
collector: g.gv.With(lnv.ToLabels()),
}
}
func (g *gauge) Add(delta float64) {
labels := g.labelNamesValues.ToLabels()
collector := g.gv.With(labels)
collector.Add(delta)
g.collectors <- newCollector(g.name, labels, collector, func() {
g.gv.Delete(labels)
})
g.collector.Add(delta)
}
func (g *gauge) Set(value float64) {
labels := g.labelNamesValues.ToLabels()
collector := g.gv.With(labels)
collector.Set(value)
g.collectors <- newCollector(g.name, labels, collector, func() {
g.gv.Delete(labels)
})
g.collector.Set(value)
}
func (g *gauge) Describe(ch chan<- *stdprometheus.Desc) {
g.gv.Describe(ch)
}
func newHistogramFrom(collectors chan<- *collector, opts stdprometheus.HistogramOpts, labelNames []string) *histogram {
func newHistogramFrom(opts stdprometheus.HistogramOpts, labelNames []string) *histogram {
hv := stdprometheus.NewHistogramVec(opts, labelNames)
return &histogram{
name: opts.Name,
hv: hv,
collectors: collectors,
name: opts.Name,
hv: hv,
labelNamesValues: make([]string, 0, 16),
}
}
@ -574,28 +561,21 @@ type histogram struct {
name string
hv *stdprometheus.HistogramVec
labelNamesValues labelNamesValues
collectors chan<- *collector
collector stdprometheus.Observer
}
func (h *histogram) With(labelValues ...string) metrics.Histogram {
lnv := h.labelNamesValues.With(labelValues...)
return &histogram{
name: h.name,
hv: h.hv,
labelNamesValues: h.labelNamesValues.With(labelValues...),
collectors: h.collectors,
labelNamesValues: lnv,
collector: h.hv.With(lnv.ToLabels()),
}
}
func (h *histogram) Observe(value float64) {
labels := h.labelNamesValues.ToLabels()
observer := h.hv.With(labels)
observer.Observe(value)
// Do a type assertion to be sure that prometheus will be able to call the Collect method.
if collector, ok := observer.(stdprometheus.Histogram); ok {
h.collectors <- newCollector(h.name, labels, collector, func() {
h.hv.Delete(labels)
})
}
h.collector.Observe(value)
}
func (h *histogram) Describe(ch chan<- *stdprometheus.Desc) {
@ -618,7 +598,7 @@ func (lvs labelNamesValues) With(labelValues ...string) labelNamesValues {
// ToLabels is a convenience method to convert a labelNamesValues
// to the native prometheus.Labels.
func (lvs labelNamesValues) ToLabels() stdprometheus.Labels {
labels := stdprometheus.Labels{}
labels := make(map[string]string, len(lvs)/2)
for i := 0; i < len(lvs); i += 2 {
labels[lvs[i]] = lvs[i+1]
}

View file

@ -17,8 +17,7 @@ import (
)
func TestRegisterPromState(t *testing.T) {
// Reset state of global promState.
defer promState.reset()
t.Cleanup(promState.reset)
testCases := []struct {
desc string
@ -88,21 +87,10 @@ func TestRegisterPromState(t *testing.T) {
}
}
// reset is a utility method for unit testing. It should be called after each
// test run that changes promState internally in order to avoid dependencies
// between unit tests.
func (ps *prometheusState) reset() {
ps.collectors = make(chan *collector)
ps.describers = []func(ch chan<- *prometheus.Desc){}
ps.dynamicConfig = newDynamicConfig()
ps.state = make(map[string]*collector)
}
func TestPrometheus(t *testing.T) {
promState = newPrometheusState()
promRegistry = prometheus.NewRegistry()
// Reset state of global promState.
defer promState.reset()
t.Cleanup(promState.reset)
prometheusRegistry := RegisterPrometheus(context.Background(), &types.Prometheus{AddEntryPointsLabels: true, AddRoutersLabels: true, AddServicesLabels: true})
defer promRegistry.Unregister(promState)
@ -361,30 +349,40 @@ func TestPrometheus(t *testing.T) {
func TestPrometheusMetricRemoval(t *testing.T) {
promState = newPrometheusState()
promRegistry = prometheus.NewRegistry()
// Reset state of global promState.
defer promState.reset()
t.Cleanup(promState.reset)
prometheusRegistry := RegisterPrometheus(context.Background(), &types.Prometheus{AddEntryPointsLabels: true, AddServicesLabels: true, AddRoutersLabels: true})
defer promRegistry.Unregister(promState)
conf := dynamic.Configuration{
conf1 := dynamic.Configuration{
HTTP: th.BuildConfiguration(
th.WithRouters(
th.WithRouter("foo@providerName",
th.WithServiceName("bar")),
th.WithRouter("foo@providerName", th.WithServiceName("bar")),
th.WithRouter("router2", th.WithServiceName("bar@providerName")),
),
th.WithLoadBalancerServices(th.WithService("bar@providerName",
th.WithServers(th.WithServer("http://localhost:9000"))),
th.WithLoadBalancerServices(
th.WithService("bar@providerName", th.WithServers(
th.WithServer("http://localhost:9000"),
th.WithServer("http://localhost:9999"),
)),
th.WithService("service1", th.WithServers(th.WithServer("http://localhost:9000"))),
),
func(cfg *dynamic.HTTPConfiguration) {
cfg.Services["fii"] = &dynamic.Service{
Weighted: &dynamic.WeightedRoundRobin{},
}
},
),
}
OnConfigurationUpdate(conf, []string{"entrypoint1"})
conf2 := dynamic.Configuration{
HTTP: th.BuildConfiguration(
th.WithRouters(
th.WithRouter("foo@providerName", th.WithServiceName("bar")),
),
th.WithLoadBalancerServices(
th.WithService("bar@providerName", th.WithServers(th.WithServer("http://localhost:9000"))),
),
),
}
OnConfigurationUpdate(conf1, []string{"entrypoint1", "entrypoint2"})
OnConfigurationUpdate(conf2, []string{"entrypoint1"})
// Register some metrics manually that are not part of the active configuration.
// Those metrics should be part of the /metrics output on the first scrape but
@ -393,22 +391,21 @@ func TestPrometheusMetricRemoval(t *testing.T) {
EntryPointReqsCounter().
With("entrypoint", "entrypoint2", "code", strconv.Itoa(http.StatusOK), "method", http.MethodGet, "protocol", "http").
Add(1)
prometheusRegistry.
RouterReqsCounter().
With("router", "router2", "service", "bar@providerName", "code", strconv.Itoa(http.StatusOK), "method", http.MethodGet, "protocol", "http").
Add(1)
prometheusRegistry.
ServiceReqsCounter().
With("service", "service2", "code", strconv.Itoa(http.StatusOK), "method", http.MethodGet, "protocol", "http").
With("service", "service1", "code", strconv.Itoa(http.StatusOK), "method", http.MethodGet, "protocol", "http").
Add(1)
prometheusRegistry.
ServiceServerUpGauge().
With("service", "service1", "url", "http://localhost:9999").
With("service", "bar@providerName", "url", "http://localhost:9999").
Set(1)
prometheusRegistry.
RouterReqsCounter().
With("router", "router2", "service", "service2", "code", strconv.Itoa(http.StatusOK), "method", http.MethodGet, "protocol", "http").
Add(1)
assertMetricsExist(t, mustScrape(), entryPointReqsTotalName, serviceReqsTotalName, serviceServerUpName)
assertMetricsAbsent(t, mustScrape(), entryPointReqsTotalName, serviceReqsTotalName, serviceServerUpName)
assertMetricsAbsent(t, mustScrape(), routerReqsTotalName, routerReqDurationName, routerOpenConnsName)
assertMetricsExist(t, mustScrape(), entryPointReqsTotalName, routerReqsTotalName, serviceReqsTotalName, serviceServerUpName)
assertMetricsAbsent(t, mustScrape(), entryPointReqsTotalName, routerReqsTotalName, serviceReqsTotalName, serviceServerUpName)
// To verify that metrics belonging to active configurations are not removed
// here the counter examples.
@ -418,24 +415,39 @@ func TestPrometheusMetricRemoval(t *testing.T) {
Add(1)
prometheusRegistry.
RouterReqsCounter().
With("router", "foo@providerName", "service", "bar@providerName", "code", strconv.Itoa(http.StatusOK), "method", http.MethodGet, "protocol", "http").
With("router", "foo@providerName", "service", "bar", "code", strconv.Itoa(http.StatusOK), "method", http.MethodGet, "protocol", "http").
Add(1)
prometheusRegistry.
ServiceReqsCounter().
With("service", "bar@providerName", "code", strconv.Itoa(http.StatusOK), "method", http.MethodGet, "protocol", "http").
Add(1)
prometheusRegistry.
ServiceServerUpGauge().
With("service", "bar@providerName", "url", "http://localhost:9000").
Set(1)
delayForTrackingCompletion()
assertMetricsExist(t, mustScrape(), entryPointReqsTotalName)
assertMetricsExist(t, mustScrape(), entryPointReqsTotalName)
assertMetricsExist(t, mustScrape(), routerReqsTotalName)
assertMetricsExist(t, mustScrape(), routerReqsTotalName)
assertMetricsExist(t, mustScrape(), entryPointReqsTotalName, serviceReqsTotalName, serviceServerUpName, routerReqsTotalName)
assertMetricsExist(t, mustScrape(), entryPointReqsTotalName, serviceReqsTotalName, serviceServerUpName, routerReqsTotalName)
}
func TestPrometheusRemovedMetricsReset(t *testing.T) {
// Reset state of global promState.
defer promState.reset()
t.Cleanup(promState.reset)
prometheusRegistry := RegisterPrometheus(context.Background(), &types.Prometheus{AddEntryPointsLabels: true, AddServicesLabels: true})
defer promRegistry.Unregister(promState)
conf1 := dynamic.Configuration{
HTTP: th.BuildConfiguration(
th.WithLoadBalancerServices(th.WithService("service",
th.WithServers(th.WithServer("http://localhost:9000"))),
),
),
}
OnConfigurationUpdate(conf1, []string{"entrypoint1", "entrypoint2"})
OnConfigurationUpdate(dynamic.Configuration{}, nil)
labelNamesValues := []string{
"service", "service",
"code", strconv.Itoa(http.StatusOK),
@ -467,12 +479,24 @@ func TestPrometheusRemovedMetricsReset(t *testing.T) {
assertCounterValue(t, 1, findMetricFamily(serviceReqsTotalName, metricsFamilies), labelNamesValues...)
}
// reset is a utility method for unit testing.
// It should be called after each test run that changes promState internally
// in order to avoid dependencies between unit tests.
func (ps *prometheusState) reset() {
ps.dynamicConfig = newDynamicConfig()
ps.vectors = nil
ps.deletedEP = nil
ps.deletedRouters = nil
ps.deletedServices = nil
ps.deletedURLs = make(map[string]string)
}
// Tracking and gathering the metrics happens concurrently.
// In practice this is no problem, because in case a tracked metric would miss
// the current scrape, it would just be there in the next one.
// That we can test reliably the tracking of all metrics here, we sleep
// for a short amount of time, to make sure the metric will be present
// in the next scrape.
// In practice this is no problem, because in case a tracked metric would miss the current scrape,
// it would just be there in the next one.
// That we can test reliably the tracking of all metrics here,
// we sleep for a short amount of time,
// to make sure the metric will be present in the next scrape.
func delayForTrackingCompletion() {
time.Sleep(250 * time.Millisecond)
}