mirror of
https://github.com/prometheus/prometheus
synced 2024-12-27 00:53:12 +00:00
Merge pull request #12958 from ptodev/prefer-to-not-register-metrics-globally-scrape
Metrics in the "scrape" package can now be registered with a non-default registry
This commit is contained in:
commit
f2e02c52db
@ -621,8 +621,18 @@ func main() {
|
||||
discoveryManagerNotify = legacymanager.NewManager(ctxNotify, log.With(logger, "component", "discovery manager notify"), legacymanager.Name("notify"))
|
||||
}
|
||||
|
||||
scrapeManager, err := scrape.NewManager(
|
||||
&cfg.scrape,
|
||||
log.With(logger, "component", "scrape manager"),
|
||||
fanoutStorage,
|
||||
prometheus.DefaultRegisterer,
|
||||
)
|
||||
if err != nil {
|
||||
level.Error(logger).Log("msg", "failed to create a scrape manager", "err", err)
|
||||
os.Exit(1)
|
||||
}
|
||||
|
||||
var (
|
||||
scrapeManager = scrape.NewManager(&cfg.scrape, log.With(logger, "component", "scrape manager"), fanoutStorage)
|
||||
tracingManager = tracing.NewManager(logger)
|
||||
|
||||
queryEngine *promql.Engine
|
||||
|
@ -34,80 +34,20 @@ import (
|
||||
"github.com/prometheus/prometheus/util/osutil"
|
||||
)
|
||||
|
||||
var targetMetadataCache = newMetadataMetricsCollector()
|
||||
|
||||
// MetadataMetricsCollector is a Custom Collector for the metadata cache metrics.
|
||||
type MetadataMetricsCollector struct {
|
||||
CacheEntries *prometheus.Desc
|
||||
CacheBytes *prometheus.Desc
|
||||
|
||||
scrapeManager *Manager
|
||||
}
|
||||
|
||||
func newMetadataMetricsCollector() *MetadataMetricsCollector {
|
||||
return &MetadataMetricsCollector{
|
||||
CacheEntries: prometheus.NewDesc(
|
||||
"prometheus_target_metadata_cache_entries",
|
||||
"Total number of metric metadata entries in the cache",
|
||||
[]string{"scrape_job"},
|
||||
nil,
|
||||
),
|
||||
CacheBytes: prometheus.NewDesc(
|
||||
"prometheus_target_metadata_cache_bytes",
|
||||
"The number of bytes that are currently used for storing metric metadata in the cache",
|
||||
[]string{"scrape_job"},
|
||||
nil,
|
||||
),
|
||||
}
|
||||
}
|
||||
|
||||
func (mc *MetadataMetricsCollector) registerManager(m *Manager) {
|
||||
mc.scrapeManager = m
|
||||
}
|
||||
|
||||
// Describe sends the metrics descriptions to the channel.
|
||||
func (mc *MetadataMetricsCollector) Describe(ch chan<- *prometheus.Desc) {
|
||||
ch <- mc.CacheEntries
|
||||
ch <- mc.CacheBytes
|
||||
}
|
||||
|
||||
// Collect creates and sends the metrics for the metadata cache.
|
||||
func (mc *MetadataMetricsCollector) Collect(ch chan<- prometheus.Metric) {
|
||||
if mc.scrapeManager == nil {
|
||||
return
|
||||
}
|
||||
|
||||
for tset, targets := range mc.scrapeManager.TargetsActive() {
|
||||
var size, length int
|
||||
for _, t := range targets {
|
||||
size += t.MetadataSize()
|
||||
length += t.MetadataLength()
|
||||
}
|
||||
|
||||
ch <- prometheus.MustNewConstMetric(
|
||||
mc.CacheEntries,
|
||||
prometheus.GaugeValue,
|
||||
float64(length),
|
||||
tset,
|
||||
)
|
||||
|
||||
ch <- prometheus.MustNewConstMetric(
|
||||
mc.CacheBytes,
|
||||
prometheus.GaugeValue,
|
||||
float64(size),
|
||||
tset,
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
// NewManager is the Manager constructor
|
||||
func NewManager(o *Options, logger log.Logger, app storage.Appendable) *Manager {
|
||||
func NewManager(o *Options, logger log.Logger, app storage.Appendable, registerer prometheus.Registerer) (*Manager, error) {
|
||||
if o == nil {
|
||||
o = &Options{}
|
||||
}
|
||||
if logger == nil {
|
||||
logger = log.NewNopLogger()
|
||||
}
|
||||
|
||||
sm, err := newScrapeMetrics(registerer)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to create scrape manager due to error: %w", err)
|
||||
}
|
||||
|
||||
m := &Manager{
|
||||
append: app,
|
||||
opts: o,
|
||||
@ -116,10 +56,12 @@ func NewManager(o *Options, logger log.Logger, app storage.Appendable) *Manager
|
||||
scrapePools: make(map[string]*scrapePool),
|
||||
graceShut: make(chan struct{}),
|
||||
triggerReload: make(chan struct{}, 1),
|
||||
metrics: sm,
|
||||
}
|
||||
targetMetadataCache.registerManager(m)
|
||||
|
||||
return m
|
||||
m.metrics.setTargetMetadataCacheGatherer(m)
|
||||
|
||||
return m, nil
|
||||
}
|
||||
|
||||
// Options are the configuration parameters to the scrape manager.
|
||||
@ -154,6 +96,8 @@ type Manager struct {
|
||||
targetSets map[string][]*targetgroup.Group
|
||||
|
||||
triggerReload chan struct{}
|
||||
|
||||
metrics *scrapeMetrics
|
||||
}
|
||||
|
||||
// Run receives and saves target set updates and triggers the scraping loops reloading.
|
||||
@ -211,8 +155,10 @@ func (m *Manager) reload() {
|
||||
level.Error(m.logger).Log("msg", "error reloading target set", "err", "invalid config id:"+setName)
|
||||
continue
|
||||
}
|
||||
sp, err := newScrapePool(scrapeConfig, m.append, m.offsetSeed, log.With(m.logger, "scrape_pool", setName), m.opts)
|
||||
m.metrics.targetScrapePools.Inc()
|
||||
sp, err := newScrapePool(scrapeConfig, m.append, m.offsetSeed, log.With(m.logger, "scrape_pool", setName), m.opts, m.metrics)
|
||||
if err != nil {
|
||||
m.metrics.targetScrapePoolsFailed.Inc()
|
||||
level.Error(m.logger).Log("msg", "error creating new scrape pool", "err", err, "scrape_pool", setName)
|
||||
continue
|
||||
}
|
||||
|
@ -20,6 +20,7 @@ import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
"github.com/prometheus/common/model"
|
||||
"github.com/stretchr/testify/require"
|
||||
"gopkg.in/yaml.v2"
|
||||
@ -492,10 +493,13 @@ scrape_configs:
|
||||
cfg3 = loadConfiguration(t, cfgText3)
|
||||
|
||||
ch = make(chan struct{}, 1)
|
||||
|
||||
testRegistry = prometheus.NewRegistry()
|
||||
)
|
||||
|
||||
opts := Options{}
|
||||
scrapeManager := NewManager(&opts, nil, nil)
|
||||
scrapeManager, err := NewManager(&opts, nil, nil, testRegistry)
|
||||
require.NoError(t, err)
|
||||
newLoop := func(scrapeLoopOptions) loop {
|
||||
ch <- struct{}{}
|
||||
return noopLoop()
|
||||
@ -512,6 +516,7 @@ scrape_configs:
|
||||
logger: nil,
|
||||
config: cfg1.ScrapeConfigs[0],
|
||||
client: http.DefaultClient,
|
||||
metrics: scrapeManager.metrics,
|
||||
}
|
||||
scrapeManager.scrapePools = map[string]*scrapePool{
|
||||
"job1": sp,
|
||||
@ -560,7 +565,9 @@ scrape_configs:
|
||||
|
||||
func TestManagerTargetsUpdates(t *testing.T) {
|
||||
opts := Options{}
|
||||
m := NewManager(&opts, nil, nil)
|
||||
testRegistry := prometheus.NewRegistry()
|
||||
m, err := NewManager(&opts, nil, nil, testRegistry)
|
||||
require.NoError(t, err)
|
||||
|
||||
ts := make(chan map[string][]*targetgroup.Group)
|
||||
go m.Run(ts)
|
||||
@ -613,7 +620,9 @@ global:
|
||||
}
|
||||
|
||||
opts := Options{}
|
||||
scrapeManager := NewManager(&opts, nil, nil)
|
||||
testRegistry := prometheus.NewRegistry()
|
||||
scrapeManager, err := NewManager(&opts, nil, nil, testRegistry)
|
||||
require.NoError(t, err)
|
||||
|
||||
// Load the first config.
|
||||
cfg1 := getConfig("ha1")
|
||||
@ -658,8 +667,9 @@ scrape_configs:
|
||||
- targets: ["foo:9093"]
|
||||
`
|
||||
var (
|
||||
cfg1 = loadConfiguration(t, cfgText1)
|
||||
cfg2 = loadConfiguration(t, cfgText2)
|
||||
cfg1 = loadConfiguration(t, cfgText1)
|
||||
cfg2 = loadConfiguration(t, cfgText2)
|
||||
testRegistry = prometheus.NewRegistry()
|
||||
)
|
||||
|
||||
reload := func(scrapeManager *Manager, cfg *config.Config) {
|
||||
@ -695,7 +705,8 @@ scrape_configs:
|
||||
}
|
||||
|
||||
opts := Options{}
|
||||
scrapeManager := NewManager(&opts, nil, nil)
|
||||
scrapeManager, err := NewManager(&opts, nil, nil, testRegistry)
|
||||
require.NoError(t, err)
|
||||
|
||||
reload(scrapeManager, cfg1)
|
||||
require.ElementsMatch(t, []string{"job1", "job2"}, scrapeManager.ScrapePools())
|
||||
|
307
scrape/metrics.go
Normal file
307
scrape/metrics.go
Normal file
@ -0,0 +1,307 @@
|
||||
// Copyright 2016 The Prometheus Authors
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package scrape
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
)
|
||||
|
||||
type scrapeMetrics struct {
|
||||
// Used by Manager.
|
||||
targetMetadataCache *MetadataMetricsCollector
|
||||
targetScrapePools prometheus.Counter
|
||||
targetScrapePoolsFailed prometheus.Counter
|
||||
|
||||
// Used by scrapePool.
|
||||
targetReloadIntervalLength *prometheus.SummaryVec
|
||||
targetScrapePoolReloads prometheus.Counter
|
||||
targetScrapePoolReloadsFailed prometheus.Counter
|
||||
targetScrapePoolSyncsCounter *prometheus.CounterVec
|
||||
targetScrapePoolExceededTargetLimit prometheus.Counter
|
||||
targetScrapePoolTargetLimit *prometheus.GaugeVec
|
||||
targetScrapePoolTargetsAdded *prometheus.GaugeVec
|
||||
targetSyncIntervalLength *prometheus.SummaryVec
|
||||
targetSyncFailed *prometheus.CounterVec
|
||||
|
||||
// Used by targetScraper.
|
||||
targetScrapeExceededBodySizeLimit prometheus.Counter
|
||||
|
||||
// Used by scrapeCache.
|
||||
targetScrapeCacheFlushForced prometheus.Counter
|
||||
|
||||
// Used by scrapeLoop.
|
||||
targetIntervalLength *prometheus.SummaryVec
|
||||
targetScrapeSampleLimit prometheus.Counter
|
||||
targetScrapeSampleDuplicate prometheus.Counter
|
||||
targetScrapeSampleOutOfOrder prometheus.Counter
|
||||
targetScrapeSampleOutOfBounds prometheus.Counter
|
||||
targetScrapeExemplarOutOfOrder prometheus.Counter
|
||||
targetScrapePoolExceededLabelLimits prometheus.Counter
|
||||
targetScrapeNativeHistogramBucketLimit prometheus.Counter
|
||||
}
|
||||
|
||||
func newScrapeMetrics(reg prometheus.Registerer) (*scrapeMetrics, error) {
|
||||
sm := &scrapeMetrics{}
|
||||
|
||||
// Manager metrics.
|
||||
sm.targetMetadataCache = &MetadataMetricsCollector{
|
||||
CacheEntries: prometheus.NewDesc(
|
||||
"prometheus_target_metadata_cache_entries",
|
||||
"Total number of metric metadata entries in the cache",
|
||||
[]string{"scrape_job"},
|
||||
nil,
|
||||
),
|
||||
CacheBytes: prometheus.NewDesc(
|
||||
"prometheus_target_metadata_cache_bytes",
|
||||
"The number of bytes that are currently used for storing metric metadata in the cache",
|
||||
[]string{"scrape_job"},
|
||||
nil,
|
||||
),
|
||||
// TargetsGatherer should be set later, because it's a circular dependency.
|
||||
// newScrapeMetrics() is called by NewManager(), while also TargetsGatherer is the new Manager.
|
||||
}
|
||||
|
||||
sm.targetScrapePools = prometheus.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Name: "prometheus_target_scrape_pools_total",
|
||||
Help: "Total number of scrape pool creation attempts.",
|
||||
},
|
||||
)
|
||||
sm.targetScrapePoolsFailed = prometheus.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Name: "prometheus_target_scrape_pools_failed_total",
|
||||
Help: "Total number of scrape pool creations that failed.",
|
||||
},
|
||||
)
|
||||
|
||||
// Used by scrapePool.
|
||||
sm.targetReloadIntervalLength = prometheus.NewSummaryVec(
|
||||
prometheus.SummaryOpts{
|
||||
Name: "prometheus_target_reload_length_seconds",
|
||||
Help: "Actual interval to reload the scrape pool with a given configuration.",
|
||||
Objectives: map[float64]float64{0.01: 0.001, 0.05: 0.005, 0.5: 0.05, 0.90: 0.01, 0.99: 0.001},
|
||||
},
|
||||
[]string{"interval"},
|
||||
)
|
||||
sm.targetScrapePoolReloads = prometheus.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Name: "prometheus_target_scrape_pool_reloads_total",
|
||||
Help: "Total number of scrape pool reloads.",
|
||||
},
|
||||
)
|
||||
sm.targetScrapePoolReloadsFailed = prometheus.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Name: "prometheus_target_scrape_pool_reloads_failed_total",
|
||||
Help: "Total number of failed scrape pool reloads.",
|
||||
},
|
||||
)
|
||||
sm.targetScrapePoolExceededTargetLimit = prometheus.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Name: "prometheus_target_scrape_pool_exceeded_target_limit_total",
|
||||
Help: "Total number of times scrape pools hit the target limit, during sync or config reload.",
|
||||
},
|
||||
)
|
||||
sm.targetScrapePoolTargetLimit = prometheus.NewGaugeVec(
|
||||
prometheus.GaugeOpts{
|
||||
Name: "prometheus_target_scrape_pool_target_limit",
|
||||
Help: "Maximum number of targets allowed in this scrape pool.",
|
||||
},
|
||||
[]string{"scrape_job"},
|
||||
)
|
||||
sm.targetScrapePoolTargetsAdded = prometheus.NewGaugeVec(
|
||||
prometheus.GaugeOpts{
|
||||
Name: "prometheus_target_scrape_pool_targets",
|
||||
Help: "Current number of targets in this scrape pool.",
|
||||
},
|
||||
[]string{"scrape_job"},
|
||||
)
|
||||
sm.targetScrapePoolSyncsCounter = prometheus.NewCounterVec(
|
||||
prometheus.CounterOpts{
|
||||
Name: "prometheus_target_scrape_pool_sync_total",
|
||||
Help: "Total number of syncs that were executed on a scrape pool.",
|
||||
},
|
||||
[]string{"scrape_job"},
|
||||
)
|
||||
sm.targetSyncIntervalLength = prometheus.NewSummaryVec(
|
||||
prometheus.SummaryOpts{
|
||||
Name: "prometheus_target_sync_length_seconds",
|
||||
Help: "Actual interval to sync the scrape pool.",
|
||||
Objectives: map[float64]float64{0.01: 0.001, 0.05: 0.005, 0.5: 0.05, 0.90: 0.01, 0.99: 0.001},
|
||||
},
|
||||
[]string{"scrape_job"},
|
||||
)
|
||||
sm.targetSyncFailed = prometheus.NewCounterVec(
|
||||
prometheus.CounterOpts{
|
||||
Name: "prometheus_target_sync_failed_total",
|
||||
Help: "Total number of target sync failures.",
|
||||
},
|
||||
[]string{"scrape_job"},
|
||||
)
|
||||
|
||||
// Used by targetScraper.
|
||||
sm.targetScrapeExceededBodySizeLimit = prometheus.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Name: "prometheus_target_scrapes_exceeded_body_size_limit_total",
|
||||
Help: "Total number of scrapes that hit the body size limit",
|
||||
},
|
||||
)
|
||||
|
||||
// Used by scrapeCache.
|
||||
sm.targetScrapeCacheFlushForced = prometheus.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Name: "prometheus_target_scrapes_cache_flush_forced_total",
|
||||
Help: "How many times a scrape cache was flushed due to getting big while scrapes are failing.",
|
||||
},
|
||||
)
|
||||
|
||||
// Used by scrapeLoop.
|
||||
sm.targetIntervalLength = prometheus.NewSummaryVec(
|
||||
prometheus.SummaryOpts{
|
||||
Name: "prometheus_target_interval_length_seconds",
|
||||
Help: "Actual intervals between scrapes.",
|
||||
Objectives: map[float64]float64{0.01: 0.001, 0.05: 0.005, 0.5: 0.05, 0.90: 0.01, 0.99: 0.001},
|
||||
},
|
||||
[]string{"interval"},
|
||||
)
|
||||
sm.targetScrapeSampleLimit = prometheus.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Name: "prometheus_target_scrapes_exceeded_sample_limit_total",
|
||||
Help: "Total number of scrapes that hit the sample limit and were rejected.",
|
||||
},
|
||||
)
|
||||
sm.targetScrapeSampleDuplicate = prometheus.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Name: "prometheus_target_scrapes_sample_duplicate_timestamp_total",
|
||||
Help: "Total number of samples rejected due to duplicate timestamps but different values.",
|
||||
},
|
||||
)
|
||||
sm.targetScrapeSampleOutOfOrder = prometheus.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Name: "prometheus_target_scrapes_sample_out_of_order_total",
|
||||
Help: "Total number of samples rejected due to not being out of the expected order.",
|
||||
},
|
||||
)
|
||||
sm.targetScrapeSampleOutOfBounds = prometheus.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Name: "prometheus_target_scrapes_sample_out_of_bounds_total",
|
||||
Help: "Total number of samples rejected due to timestamp falling outside of the time bounds.",
|
||||
},
|
||||
)
|
||||
sm.targetScrapePoolExceededLabelLimits = prometheus.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Name: "prometheus_target_scrape_pool_exceeded_label_limits_total",
|
||||
Help: "Total number of times scrape pools hit the label limits, during sync or config reload.",
|
||||
},
|
||||
)
|
||||
sm.targetScrapeNativeHistogramBucketLimit = prometheus.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Name: "prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total",
|
||||
Help: "Total number of scrapes that hit the native histogram bucket limit and were rejected.",
|
||||
},
|
||||
)
|
||||
sm.targetScrapeExemplarOutOfOrder = prometheus.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Name: "prometheus_target_scrapes_exemplar_out_of_order_total",
|
||||
Help: "Total number of exemplar rejected due to not being out of the expected order.",
|
||||
},
|
||||
)
|
||||
|
||||
for _, collector := range []prometheus.Collector{
|
||||
// Used by Manager.
|
||||
sm.targetMetadataCache,
|
||||
sm.targetScrapePools,
|
||||
sm.targetScrapePoolsFailed,
|
||||
// Used by scrapePool.
|
||||
sm.targetReloadIntervalLength,
|
||||
sm.targetScrapePoolReloads,
|
||||
sm.targetScrapePoolReloadsFailed,
|
||||
sm.targetSyncIntervalLength,
|
||||
sm.targetScrapePoolSyncsCounter,
|
||||
sm.targetScrapePoolExceededTargetLimit,
|
||||
sm.targetScrapePoolTargetLimit,
|
||||
sm.targetScrapePoolTargetsAdded,
|
||||
sm.targetSyncFailed,
|
||||
// Used by targetScraper.
|
||||
sm.targetScrapeExceededBodySizeLimit,
|
||||
// Used by scrapeCache.
|
||||
sm.targetScrapeCacheFlushForced,
|
||||
// Used by scrapeLoop.
|
||||
sm.targetIntervalLength,
|
||||
sm.targetScrapeSampleLimit,
|
||||
sm.targetScrapeSampleDuplicate,
|
||||
sm.targetScrapeSampleOutOfOrder,
|
||||
sm.targetScrapeSampleOutOfBounds,
|
||||
sm.targetScrapeExemplarOutOfOrder,
|
||||
sm.targetScrapePoolExceededLabelLimits,
|
||||
sm.targetScrapeNativeHistogramBucketLimit,
|
||||
} {
|
||||
err := reg.Register(collector)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to register scrape metrics: %w", err)
|
||||
}
|
||||
}
|
||||
return sm, nil
|
||||
}
|
||||
|
||||
func (sm *scrapeMetrics) setTargetMetadataCacheGatherer(gatherer TargetsGatherer) {
|
||||
sm.targetMetadataCache.TargetsGatherer = gatherer
|
||||
}
|
||||
|
||||
type TargetsGatherer interface {
|
||||
TargetsActive() map[string][]*Target
|
||||
}
|
||||
|
||||
// MetadataMetricsCollector is a Custom Collector for the metadata cache metrics.
|
||||
type MetadataMetricsCollector struct {
|
||||
CacheEntries *prometheus.Desc
|
||||
CacheBytes *prometheus.Desc
|
||||
TargetsGatherer TargetsGatherer
|
||||
}
|
||||
|
||||
// Describe sends the metrics descriptions to the channel.
|
||||
func (mc *MetadataMetricsCollector) Describe(ch chan<- *prometheus.Desc) {
|
||||
ch <- mc.CacheEntries
|
||||
ch <- mc.CacheBytes
|
||||
}
|
||||
|
||||
// Collect creates and sends the metrics for the metadata cache.
|
||||
func (mc *MetadataMetricsCollector) Collect(ch chan<- prometheus.Metric) {
|
||||
if mc.TargetsGatherer == nil {
|
||||
return
|
||||
}
|
||||
|
||||
for tset, targets := range mc.TargetsGatherer.TargetsActive() {
|
||||
var size, length int
|
||||
for _, t := range targets {
|
||||
size += t.MetadataSize()
|
||||
length += t.MetadataLength()
|
||||
}
|
||||
|
||||
ch <- prometheus.MustNewConstMetric(
|
||||
mc.CacheEntries,
|
||||
prometheus.GaugeValue,
|
||||
float64(length),
|
||||
tset,
|
||||
)
|
||||
|
||||
ch <- prometheus.MustNewConstMetric(
|
||||
mc.CacheBytes,
|
||||
prometheus.GaugeValue,
|
||||
float64(size),
|
||||
tset,
|
||||
)
|
||||
}
|
||||
}
|
245
scrape/scrape.go
245
scrape/scrape.go
@ -31,7 +31,6 @@ import (
|
||||
"github.com/go-kit/log"
|
||||
"github.com/go-kit/log/level"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
config_util "github.com/prometheus/common/config"
|
||||
"github.com/prometheus/common/model"
|
||||
"github.com/prometheus/common/version"
|
||||
@ -61,172 +60,6 @@ var AlignScrapeTimestamps = true
|
||||
|
||||
var errNameLabelMandatory = fmt.Errorf("missing metric name (%s label)", labels.MetricName)
|
||||
|
||||
var (
|
||||
targetIntervalLength = prometheus.NewSummaryVec(
|
||||
prometheus.SummaryOpts{
|
||||
Name: "prometheus_target_interval_length_seconds",
|
||||
Help: "Actual intervals between scrapes.",
|
||||
Objectives: map[float64]float64{0.01: 0.001, 0.05: 0.005, 0.5: 0.05, 0.90: 0.01, 0.99: 0.001},
|
||||
},
|
||||
[]string{"interval"},
|
||||
)
|
||||
targetReloadIntervalLength = prometheus.NewSummaryVec(
|
||||
prometheus.SummaryOpts{
|
||||
Name: "prometheus_target_reload_length_seconds",
|
||||
Help: "Actual interval to reload the scrape pool with a given configuration.",
|
||||
Objectives: map[float64]float64{0.01: 0.001, 0.05: 0.005, 0.5: 0.05, 0.90: 0.01, 0.99: 0.001},
|
||||
},
|
||||
[]string{"interval"},
|
||||
)
|
||||
targetScrapePools = prometheus.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Name: "prometheus_target_scrape_pools_total",
|
||||
Help: "Total number of scrape pool creation attempts.",
|
||||
},
|
||||
)
|
||||
targetScrapePoolsFailed = prometheus.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Name: "prometheus_target_scrape_pools_failed_total",
|
||||
Help: "Total number of scrape pool creations that failed.",
|
||||
},
|
||||
)
|
||||
targetScrapePoolReloads = prometheus.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Name: "prometheus_target_scrape_pool_reloads_total",
|
||||
Help: "Total number of scrape pool reloads.",
|
||||
},
|
||||
)
|
||||
targetScrapePoolReloadsFailed = prometheus.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Name: "prometheus_target_scrape_pool_reloads_failed_total",
|
||||
Help: "Total number of failed scrape pool reloads.",
|
||||
},
|
||||
)
|
||||
targetScrapePoolExceededTargetLimit = prometheus.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Name: "prometheus_target_scrape_pool_exceeded_target_limit_total",
|
||||
Help: "Total number of times scrape pools hit the target limit, during sync or config reload.",
|
||||
},
|
||||
)
|
||||
targetScrapePoolTargetLimit = prometheus.NewGaugeVec(
|
||||
prometheus.GaugeOpts{
|
||||
Name: "prometheus_target_scrape_pool_target_limit",
|
||||
Help: "Maximum number of targets allowed in this scrape pool.",
|
||||
},
|
||||
[]string{"scrape_job"},
|
||||
)
|
||||
targetScrapePoolTargetsAdded = prometheus.NewGaugeVec(
|
||||
prometheus.GaugeOpts{
|
||||
Name: "prometheus_target_scrape_pool_targets",
|
||||
Help: "Current number of targets in this scrape pool.",
|
||||
},
|
||||
[]string{"scrape_job"},
|
||||
)
|
||||
targetSyncIntervalLength = prometheus.NewSummaryVec(
|
||||
prometheus.SummaryOpts{
|
||||
Name: "prometheus_target_sync_length_seconds",
|
||||
Help: "Actual interval to sync the scrape pool.",
|
||||
Objectives: map[float64]float64{0.01: 0.001, 0.05: 0.005, 0.5: 0.05, 0.90: 0.01, 0.99: 0.001},
|
||||
},
|
||||
[]string{"scrape_job"},
|
||||
)
|
||||
targetScrapePoolSyncsCounter = prometheus.NewCounterVec(
|
||||
prometheus.CounterOpts{
|
||||
Name: "prometheus_target_scrape_pool_sync_total",
|
||||
Help: "Total number of syncs that were executed on a scrape pool.",
|
||||
},
|
||||
[]string{"scrape_job"},
|
||||
)
|
||||
targetScrapeExceededBodySizeLimit = prometheus.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Name: "prometheus_target_scrapes_exceeded_body_size_limit_total",
|
||||
Help: "Total number of scrapes that hit the body size limit",
|
||||
},
|
||||
)
|
||||
targetScrapeSampleLimit = prometheus.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Name: "prometheus_target_scrapes_exceeded_sample_limit_total",
|
||||
Help: "Total number of scrapes that hit the sample limit and were rejected.",
|
||||
},
|
||||
)
|
||||
targetScrapeSampleDuplicate = prometheus.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Name: "prometheus_target_scrapes_sample_duplicate_timestamp_total",
|
||||
Help: "Total number of samples rejected due to duplicate timestamps but different values.",
|
||||
},
|
||||
)
|
||||
targetScrapeSampleOutOfOrder = prometheus.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Name: "prometheus_target_scrapes_sample_out_of_order_total",
|
||||
Help: "Total number of samples rejected due to not being out of the expected order.",
|
||||
},
|
||||
)
|
||||
targetScrapeSampleOutOfBounds = prometheus.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Name: "prometheus_target_scrapes_sample_out_of_bounds_total",
|
||||
Help: "Total number of samples rejected due to timestamp falling outside of the time bounds.",
|
||||
},
|
||||
)
|
||||
targetScrapeCacheFlushForced = prometheus.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Name: "prometheus_target_scrapes_cache_flush_forced_total",
|
||||
Help: "How many times a scrape cache was flushed due to getting big while scrapes are failing.",
|
||||
},
|
||||
)
|
||||
targetScrapeExemplarOutOfOrder = prometheus.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Name: "prometheus_target_scrapes_exemplar_out_of_order_total",
|
||||
Help: "Total number of exemplar rejected due to not being out of the expected order.",
|
||||
},
|
||||
)
|
||||
targetScrapePoolExceededLabelLimits = prometheus.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Name: "prometheus_target_scrape_pool_exceeded_label_limits_total",
|
||||
Help: "Total number of times scrape pools hit the label limits, during sync or config reload.",
|
||||
},
|
||||
)
|
||||
targetSyncFailed = prometheus.NewCounterVec(
|
||||
prometheus.CounterOpts{
|
||||
Name: "prometheus_target_sync_failed_total",
|
||||
Help: "Total number of target sync failures.",
|
||||
},
|
||||
[]string{"scrape_job"},
|
||||
)
|
||||
targetScrapeNativeHistogramBucketLimit = prometheus.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Name: "prometheus_target_scrapes_exceeded_native_histogram_bucket_limit_total",
|
||||
Help: "Total number of scrapes that hit the native histogram bucket limit and were rejected.",
|
||||
},
|
||||
)
|
||||
)
|
||||
|
||||
func init() {
|
||||
prometheus.MustRegister(
|
||||
targetIntervalLength,
|
||||
targetReloadIntervalLength,
|
||||
targetScrapePools,
|
||||
targetScrapePoolsFailed,
|
||||
targetScrapePoolReloads,
|
||||
targetScrapePoolReloadsFailed,
|
||||
targetSyncIntervalLength,
|
||||
targetScrapePoolSyncsCounter,
|
||||
targetScrapeExceededBodySizeLimit,
|
||||
targetScrapeSampleLimit,
|
||||
targetScrapeSampleDuplicate,
|
||||
targetScrapeSampleOutOfOrder,
|
||||
targetScrapeSampleOutOfBounds,
|
||||
targetScrapePoolExceededTargetLimit,
|
||||
targetScrapePoolTargetLimit,
|
||||
targetScrapePoolTargetsAdded,
|
||||
targetScrapeCacheFlushForced,
|
||||
targetMetadataCache,
|
||||
targetScrapeExemplarOutOfOrder,
|
||||
targetScrapePoolExceededLabelLimits,
|
||||
targetSyncFailed,
|
||||
targetScrapeNativeHistogramBucketLimit,
|
||||
)
|
||||
}
|
||||
|
||||
// scrapePool manages scrapes for sets of targets.
|
||||
type scrapePool struct {
|
||||
appendable storage.Appendable
|
||||
@ -251,6 +84,8 @@ type scrapePool struct {
|
||||
newLoop func(scrapeLoopOptions) loop
|
||||
|
||||
noDefaultPort bool
|
||||
|
||||
metrics *scrapeMetrics
|
||||
}
|
||||
|
||||
type labelLimits struct {
|
||||
@ -279,15 +114,13 @@ const maxAheadTime = 10 * time.Minute
|
||||
// returning an empty label set is interpreted as "drop"
|
||||
type labelsMutator func(labels.Labels) labels.Labels
|
||||
|
||||
func newScrapePool(cfg *config.ScrapeConfig, app storage.Appendable, offsetSeed uint64, logger log.Logger, options *Options) (*scrapePool, error) {
|
||||
targetScrapePools.Inc()
|
||||
func newScrapePool(cfg *config.ScrapeConfig, app storage.Appendable, offsetSeed uint64, logger log.Logger, options *Options, metrics *scrapeMetrics) (*scrapePool, error) {
|
||||
if logger == nil {
|
||||
logger = log.NewNopLogger()
|
||||
}
|
||||
|
||||
client, err := config_util.NewClientFromConfig(cfg.HTTPClientConfig, cfg.JobName, options.HTTPClientOptions...)
|
||||
if err != nil {
|
||||
targetScrapePoolsFailed.Inc()
|
||||
return nil, errors.Wrap(err, "error creating HTTP client")
|
||||
}
|
||||
|
||||
@ -302,6 +135,7 @@ func newScrapePool(cfg *config.ScrapeConfig, app storage.Appendable, offsetSeed
|
||||
activeTargets: map[uint64]*Target{},
|
||||
loops: map[uint64]loop{},
|
||||
logger: logger,
|
||||
metrics: metrics,
|
||||
httpOpts: options.HTTPClientOptions,
|
||||
noDefaultPort: options.NoDefaultPort,
|
||||
}
|
||||
@ -309,7 +143,7 @@ func newScrapePool(cfg *config.ScrapeConfig, app storage.Appendable, offsetSeed
|
||||
// Update the targets retrieval function for metadata to a new scrape cache.
|
||||
cache := opts.cache
|
||||
if cache == nil {
|
||||
cache = newScrapeCache()
|
||||
cache = newScrapeCache(metrics)
|
||||
}
|
||||
opts.target.SetMetadataStore(cache)
|
||||
|
||||
@ -336,9 +170,10 @@ func newScrapePool(cfg *config.ScrapeConfig, app storage.Appendable, offsetSeed
|
||||
options.EnableMetadataStorage,
|
||||
opts.target,
|
||||
options.PassMetadataInContext,
|
||||
metrics,
|
||||
)
|
||||
}
|
||||
targetScrapePoolTargetLimit.WithLabelValues(sp.config.JobName).Set(float64(sp.config.TargetLimit))
|
||||
sp.metrics.targetScrapePoolTargetLimit.WithLabelValues(sp.config.JobName).Set(float64(sp.config.TargetLimit))
|
||||
return sp, nil
|
||||
}
|
||||
|
||||
@ -393,11 +228,11 @@ func (sp *scrapePool) stop() {
|
||||
sp.client.CloseIdleConnections()
|
||||
|
||||
if sp.config != nil {
|
||||
targetScrapePoolSyncsCounter.DeleteLabelValues(sp.config.JobName)
|
||||
targetScrapePoolTargetLimit.DeleteLabelValues(sp.config.JobName)
|
||||
targetScrapePoolTargetsAdded.DeleteLabelValues(sp.config.JobName)
|
||||
targetSyncIntervalLength.DeleteLabelValues(sp.config.JobName)
|
||||
targetSyncFailed.DeleteLabelValues(sp.config.JobName)
|
||||
sp.metrics.targetScrapePoolSyncsCounter.DeleteLabelValues(sp.config.JobName)
|
||||
sp.metrics.targetScrapePoolTargetLimit.DeleteLabelValues(sp.config.JobName)
|
||||
sp.metrics.targetScrapePoolTargetsAdded.DeleteLabelValues(sp.config.JobName)
|
||||
sp.metrics.targetSyncIntervalLength.DeleteLabelValues(sp.config.JobName)
|
||||
sp.metrics.targetSyncFailed.DeleteLabelValues(sp.config.JobName)
|
||||
}
|
||||
}
|
||||
|
||||
@ -407,12 +242,12 @@ func (sp *scrapePool) stop() {
|
||||
func (sp *scrapePool) reload(cfg *config.ScrapeConfig) error {
|
||||
sp.mtx.Lock()
|
||||
defer sp.mtx.Unlock()
|
||||
targetScrapePoolReloads.Inc()
|
||||
sp.metrics.targetScrapePoolReloads.Inc()
|
||||
start := time.Now()
|
||||
|
||||
client, err := config_util.NewClientFromConfig(cfg.HTTPClientConfig, cfg.JobName, sp.httpOpts...)
|
||||
if err != nil {
|
||||
targetScrapePoolReloadsFailed.Inc()
|
||||
sp.metrics.targetScrapePoolReloadsFailed.Inc()
|
||||
return errors.Wrap(err, "error creating HTTP client")
|
||||
}
|
||||
|
||||
@ -421,7 +256,7 @@ func (sp *scrapePool) reload(cfg *config.ScrapeConfig) error {
|
||||
oldClient := sp.client
|
||||
sp.client = client
|
||||
|
||||
targetScrapePoolTargetLimit.WithLabelValues(sp.config.JobName).Set(float64(sp.config.TargetLimit))
|
||||
sp.metrics.targetScrapePoolTargetLimit.WithLabelValues(sp.config.JobName).Set(float64(sp.config.TargetLimit))
|
||||
|
||||
var (
|
||||
wg sync.WaitGroup
|
||||
@ -449,7 +284,7 @@ func (sp *scrapePool) reload(cfg *config.ScrapeConfig) error {
|
||||
oldLoop.disableEndOfRunStalenessMarkers()
|
||||
cache = oc
|
||||
} else {
|
||||
cache = newScrapeCache()
|
||||
cache = newScrapeCache(sp.metrics)
|
||||
}
|
||||
|
||||
t := sp.activeTargets[fp]
|
||||
@ -496,7 +331,7 @@ func (sp *scrapePool) reload(cfg *config.ScrapeConfig) error {
|
||||
|
||||
wg.Wait()
|
||||
oldClient.CloseIdleConnections()
|
||||
targetReloadIntervalLength.WithLabelValues(interval.String()).Observe(
|
||||
sp.metrics.targetReloadIntervalLength.WithLabelValues(interval.String()).Observe(
|
||||
time.Since(start).Seconds(),
|
||||
)
|
||||
return nil
|
||||
@ -520,7 +355,7 @@ func (sp *scrapePool) Sync(tgs []*targetgroup.Group) {
|
||||
for _, err := range failures {
|
||||
level.Error(sp.logger).Log("msg", "Creating target failed", "err", err)
|
||||
}
|
||||
targetSyncFailed.WithLabelValues(sp.config.JobName).Add(float64(len(failures)))
|
||||
sp.metrics.targetSyncFailed.WithLabelValues(sp.config.JobName).Add(float64(len(failures)))
|
||||
for _, t := range targets {
|
||||
// Replicate .Labels().IsEmpty() with a loop here to avoid generating garbage.
|
||||
nonEmpty := false
|
||||
@ -539,10 +374,10 @@ func (sp *scrapePool) Sync(tgs []*targetgroup.Group) {
|
||||
sp.targetMtx.Unlock()
|
||||
sp.sync(all)
|
||||
|
||||
targetSyncIntervalLength.WithLabelValues(sp.config.JobName).Observe(
|
||||
sp.metrics.targetSyncIntervalLength.WithLabelValues(sp.config.JobName).Observe(
|
||||
time.Since(start).Seconds(),
|
||||
)
|
||||
targetScrapePoolSyncsCounter.WithLabelValues(sp.config.JobName).Inc()
|
||||
sp.metrics.targetScrapePoolSyncsCounter.WithLabelValues(sp.config.JobName).Inc()
|
||||
}
|
||||
|
||||
// sync takes a list of potentially duplicated targets, deduplicates them, starts
|
||||
@ -583,6 +418,7 @@ func (sp *scrapePool) sync(targets []*Target) {
|
||||
timeout: timeout,
|
||||
bodySizeLimit: bodySizeLimit,
|
||||
acceptHeader: acceptHeader(sp.config.ScrapeProtocols),
|
||||
metrics: sp.metrics,
|
||||
}
|
||||
l := sp.newLoop(scrapeLoopOptions{
|
||||
target: t,
|
||||
@ -634,7 +470,7 @@ func (sp *scrapePool) sync(targets []*Target) {
|
||||
|
||||
sp.targetMtx.Unlock()
|
||||
|
||||
targetScrapePoolTargetsAdded.WithLabelValues(sp.config.JobName).Set(float64(len(uniqueLoops)))
|
||||
sp.metrics.targetScrapePoolTargetsAdded.WithLabelValues(sp.config.JobName).Set(float64(len(uniqueLoops)))
|
||||
forcedErr := sp.refreshTargetLimitErr()
|
||||
for _, l := range sp.loops {
|
||||
l.setForcedError(forcedErr)
|
||||
@ -658,7 +494,7 @@ func (sp *scrapePool) refreshTargetLimitErr() error {
|
||||
return nil
|
||||
}
|
||||
if l := len(sp.activeTargets); l > int(sp.config.TargetLimit) {
|
||||
targetScrapePoolExceededTargetLimit.Inc()
|
||||
sp.metrics.targetScrapePoolExceededTargetLimit.Inc()
|
||||
return fmt.Errorf("target_limit exceeded (number of targets: %d, limit: %d)", l, sp.config.TargetLimit)
|
||||
}
|
||||
return nil
|
||||
@ -806,6 +642,8 @@ type targetScraper struct {
|
||||
|
||||
bodySizeLimit int64
|
||||
acceptHeader string
|
||||
|
||||
metrics *scrapeMetrics
|
||||
}
|
||||
|
||||
var errBodySizeLimit = errors.New("body size limit exceeded")
|
||||
@ -863,7 +701,7 @@ func (s *targetScraper) readResponse(ctx context.Context, resp *http.Response, w
|
||||
return "", err
|
||||
}
|
||||
if n >= s.bodySizeLimit {
|
||||
targetScrapeExceededBodySizeLimit.Inc()
|
||||
s.metrics.targetScrapeExceededBodySizeLimit.Inc()
|
||||
return "", errBodySizeLimit
|
||||
}
|
||||
return resp.Header.Get("Content-Type"), nil
|
||||
@ -889,7 +727,7 @@ func (s *targetScraper) readResponse(ctx context.Context, resp *http.Response, w
|
||||
return "", err
|
||||
}
|
||||
if n >= s.bodySizeLimit {
|
||||
targetScrapeExceededBodySizeLimit.Inc()
|
||||
s.metrics.targetScrapeExceededBodySizeLimit.Inc()
|
||||
return "", errBodySizeLimit
|
||||
}
|
||||
return resp.Header.Get("Content-Type"), nil
|
||||
@ -942,6 +780,8 @@ type scrapeLoop struct {
|
||||
|
||||
reportExtraMetrics bool
|
||||
appendMetadataToWAL bool
|
||||
|
||||
metrics *scrapeMetrics
|
||||
}
|
||||
|
||||
// scrapeCache tracks mappings of exposed metric strings to label sets and
|
||||
@ -969,6 +809,8 @@ type scrapeCache struct {
|
||||
|
||||
metaMtx sync.Mutex
|
||||
metadata map[string]*metaEntry
|
||||
|
||||
metrics *scrapeMetrics
|
||||
}
|
||||
|
||||
// metaEntry holds meta information about a metric.
|
||||
@ -984,13 +826,14 @@ func (m *metaEntry) size() int {
|
||||
return len(m.Help) + len(m.Unit) + len(m.Type)
|
||||
}
|
||||
|
||||
func newScrapeCache() *scrapeCache {
|
||||
func newScrapeCache(metrics *scrapeMetrics) *scrapeCache {
|
||||
return &scrapeCache{
|
||||
series: map[string]*cacheEntry{},
|
||||
droppedSeries: map[string]*uint64{},
|
||||
seriesCur: map[uint64]labels.Labels{},
|
||||
seriesPrev: map[uint64]labels.Labels{},
|
||||
metadata: map[string]*metaEntry{},
|
||||
metrics: metrics,
|
||||
}
|
||||
}
|
||||
|
||||
@ -1009,7 +852,7 @@ func (c *scrapeCache) iterDone(flushCache bool) {
|
||||
// since the last scrape, and allow an additional 1000 in case
|
||||
// initial scrapes all fail.
|
||||
flushCache = true
|
||||
targetScrapeCacheFlushForced.Inc()
|
||||
c.metrics.targetScrapeCacheFlushForced.Inc()
|
||||
}
|
||||
|
||||
if flushCache {
|
||||
@ -1213,6 +1056,7 @@ func newScrapeLoop(ctx context.Context,
|
||||
appendMetadataToWAL bool,
|
||||
target *Target,
|
||||
passMetadataInContext bool,
|
||||
metrics *scrapeMetrics,
|
||||
) *scrapeLoop {
|
||||
if l == nil {
|
||||
l = log.NewNopLogger()
|
||||
@ -1221,7 +1065,7 @@ func newScrapeLoop(ctx context.Context,
|
||||
buffers = pool.New(1e3, 1e6, 3, func(sz int) interface{} { return make([]byte, 0, sz) })
|
||||
}
|
||||
if cache == nil {
|
||||
cache = newScrapeCache()
|
||||
cache = newScrapeCache(metrics)
|
||||
}
|
||||
|
||||
appenderCtx := ctx
|
||||
@ -1256,6 +1100,7 @@ func newScrapeLoop(ctx context.Context,
|
||||
scrapeClassicHistograms: scrapeClassicHistograms,
|
||||
reportExtraMetrics: reportExtraMetrics,
|
||||
appendMetadataToWAL: appendMetadataToWAL,
|
||||
metrics: metrics,
|
||||
}
|
||||
sl.ctx, sl.cancel = context.WithCancel(ctx)
|
||||
|
||||
@ -1335,7 +1180,7 @@ func (sl *scrapeLoop) scrapeAndReport(last, appendTime time.Time, errc chan<- er
|
||||
|
||||
// Only record after the first scrape.
|
||||
if !last.IsZero() {
|
||||
targetIntervalLength.WithLabelValues(sl.interval.String()).Observe(
|
||||
sl.metrics.targetIntervalLength.WithLabelValues(sl.interval.String()).Observe(
|
||||
time.Since(last).Seconds(),
|
||||
)
|
||||
}
|
||||
@ -1676,7 +1521,7 @@ loop:
|
||||
|
||||
// If any label limits is exceeded the scrape should fail.
|
||||
if err = verifyLabelLimits(lset, sl.labelLimits); err != nil {
|
||||
targetScrapePoolExceededLabelLimits.Inc()
|
||||
sl.metrics.targetScrapePoolExceededLabelLimits.Inc()
|
||||
break loop
|
||||
}
|
||||
|
||||
@ -1741,14 +1586,14 @@ loop:
|
||||
err = sampleLimitErr
|
||||
}
|
||||
// We only want to increment this once per scrape, so this is Inc'd outside the loop.
|
||||
targetScrapeSampleLimit.Inc()
|
||||
sl.metrics.targetScrapeSampleLimit.Inc()
|
||||
}
|
||||
if bucketLimitErr != nil {
|
||||
if err == nil {
|
||||
err = bucketLimitErr // If sample limit is hit, that error takes precedence.
|
||||
}
|
||||
// We only want to increment this once per scrape, so this is Inc'd outside the loop.
|
||||
targetScrapeNativeHistogramBucketLimit.Inc()
|
||||
sl.metrics.targetScrapeNativeHistogramBucketLimit.Inc()
|
||||
}
|
||||
if appErrs.numOutOfOrder > 0 {
|
||||
level.Warn(sl.l).Log("msg", "Error on ingesting out-of-order samples", "num_dropped", appErrs.numOutOfOrder)
|
||||
@ -1792,17 +1637,17 @@ func (sl *scrapeLoop) checkAddError(ce *cacheEntry, met []byte, tp *int64, err e
|
||||
case storage.ErrOutOfOrderSample:
|
||||
appErrs.numOutOfOrder++
|
||||
level.Debug(sl.l).Log("msg", "Out of order sample", "series", string(met))
|
||||
targetScrapeSampleOutOfOrder.Inc()
|
||||
sl.metrics.targetScrapeSampleOutOfOrder.Inc()
|
||||
return false, nil
|
||||
case storage.ErrDuplicateSampleForTimestamp:
|
||||
appErrs.numDuplicates++
|
||||
level.Debug(sl.l).Log("msg", "Duplicate sample for timestamp", "series", string(met))
|
||||
targetScrapeSampleDuplicate.Inc()
|
||||
sl.metrics.targetScrapeSampleDuplicate.Inc()
|
||||
return false, nil
|
||||
case storage.ErrOutOfBounds:
|
||||
appErrs.numOutOfBounds++
|
||||
level.Debug(sl.l).Log("msg", "Out of bounds metric", "series", string(met))
|
||||
targetScrapeSampleOutOfBounds.Inc()
|
||||
sl.metrics.targetScrapeSampleOutOfBounds.Inc()
|
||||
return false, nil
|
||||
case errSampleLimit:
|
||||
// Keep on parsing output if we hit the limit, so we report the correct
|
||||
@ -1826,7 +1671,7 @@ func (sl *scrapeLoop) checkAddExemplarError(err error, e exemplar.Exemplar, appE
|
||||
case storage.ErrOutOfOrderExemplar:
|
||||
appErrs.numExemplarOutOfOrder++
|
||||
level.Debug(sl.l).Log("msg", "Out of order exemplar", "exemplar", fmt.Sprintf("%+v", e))
|
||||
targetScrapeExemplarOutOfOrder.Inc()
|
||||
sl.metrics.targetScrapeExemplarOutOfOrder.Inc()
|
||||
return nil
|
||||
default:
|
||||
return err
|
||||
|
@ -57,11 +57,18 @@ func TestMain(m *testing.M) {
|
||||
testutil.TolerantVerifyLeak(m)
|
||||
}
|
||||
|
||||
func newTestScrapeMetrics(t testing.TB) *scrapeMetrics {
|
||||
reg := prometheus.NewRegistry()
|
||||
metrics, err := newScrapeMetrics(reg)
|
||||
require.NoError(t, err)
|
||||
return metrics
|
||||
}
|
||||
|
||||
func TestNewScrapePool(t *testing.T) {
|
||||
var (
|
||||
app = &nopAppendable{}
|
||||
cfg = &config.ScrapeConfig{}
|
||||
sp, _ = newScrapePool(cfg, app, 0, nil, &Options{})
|
||||
sp, _ = newScrapePool(cfg, app, 0, nil, &Options{}, newTestScrapeMetrics(t))
|
||||
)
|
||||
|
||||
if a, ok := sp.appendable.(*nopAppendable); !ok || a != app {
|
||||
@ -97,7 +104,7 @@ func TestDroppedTargetsList(t *testing.T) {
|
||||
},
|
||||
},
|
||||
}
|
||||
sp, _ = newScrapePool(cfg, app, 0, nil, &Options{})
|
||||
sp, _ = newScrapePool(cfg, app, 0, nil, &Options{}, newTestScrapeMetrics(t))
|
||||
expectedLabelSetString = "{__address__=\"127.0.0.1:9090\", __scrape_interval__=\"0s\", __scrape_timeout__=\"0s\", job=\"dropMe\"}"
|
||||
expectedLength = 2
|
||||
)
|
||||
@ -117,7 +124,10 @@ func TestDroppedTargetsList(t *testing.T) {
|
||||
// TestDiscoveredLabelsUpdate checks that DiscoveredLabels are updated
|
||||
// even when new labels don't affect the target `hash`.
|
||||
func TestDiscoveredLabelsUpdate(t *testing.T) {
|
||||
sp := &scrapePool{}
|
||||
sp := &scrapePool{
|
||||
metrics: newTestScrapeMetrics(t),
|
||||
}
|
||||
|
||||
// These are used when syncing so need this to avoid a panic.
|
||||
sp.config = &config.ScrapeConfig{
|
||||
ScrapeInterval: model.Duration(1),
|
||||
@ -184,6 +194,7 @@ func TestScrapePoolStop(t *testing.T) {
|
||||
loops: map[uint64]loop{},
|
||||
cancel: func() {},
|
||||
client: http.DefaultClient,
|
||||
metrics: newTestScrapeMetrics(t),
|
||||
}
|
||||
var mtx sync.Mutex
|
||||
stopped := map[uint64]bool{}
|
||||
@ -262,6 +273,7 @@ func TestScrapePoolReload(t *testing.T) {
|
||||
}
|
||||
return l
|
||||
}
|
||||
|
||||
sp := &scrapePool{
|
||||
appendable: &nopAppendable{},
|
||||
activeTargets: map[uint64]*Target{},
|
||||
@ -269,6 +281,7 @@ func TestScrapePoolReload(t *testing.T) {
|
||||
newLoop: newLoop,
|
||||
logger: nil,
|
||||
client: http.DefaultClient,
|
||||
metrics: newTestScrapeMetrics(t),
|
||||
}
|
||||
|
||||
// Reloading a scrape pool with a new scrape configuration must stop all scrape
|
||||
@ -352,6 +365,7 @@ func TestScrapePoolReloadPreserveRelabeledIntervalTimeout(t *testing.T) {
|
||||
newLoop: newLoop,
|
||||
logger: nil,
|
||||
client: http.DefaultClient,
|
||||
metrics: newTestScrapeMetrics(t),
|
||||
}
|
||||
|
||||
err := sp.reload(reloadCfg)
|
||||
@ -381,6 +395,7 @@ func TestScrapePoolTargetLimit(t *testing.T) {
|
||||
newLoop: newLoop,
|
||||
logger: log.NewNopLogger(),
|
||||
client: http.DefaultClient,
|
||||
metrics: newTestScrapeMetrics(t),
|
||||
}
|
||||
|
||||
tgs := []*targetgroup.Group{}
|
||||
@ -489,7 +504,7 @@ func TestScrapePoolTargetLimit(t *testing.T) {
|
||||
func TestScrapePoolAppender(t *testing.T) {
|
||||
cfg := &config.ScrapeConfig{}
|
||||
app := &nopAppendable{}
|
||||
sp, _ := newScrapePool(cfg, app, 0, nil, &Options{})
|
||||
sp, _ := newScrapePool(cfg, app, 0, nil, &Options{}, newTestScrapeMetrics(t))
|
||||
|
||||
loop := sp.newLoop(scrapeLoopOptions{
|
||||
target: &Target{},
|
||||
@ -545,7 +560,7 @@ func TestScrapePoolRaces(t *testing.T) {
|
||||
newConfig := func() *config.ScrapeConfig {
|
||||
return &config.ScrapeConfig{ScrapeInterval: interval, ScrapeTimeout: timeout}
|
||||
}
|
||||
sp, _ := newScrapePool(newConfig(), &nopAppendable{}, 0, nil, &Options{})
|
||||
sp, _ := newScrapePool(newConfig(), &nopAppendable{}, 0, nil, &Options{}, newTestScrapeMetrics(t))
|
||||
tgts := []*targetgroup.Group{
|
||||
{
|
||||
Targets: []model.LabelSet{
|
||||
@ -595,6 +610,7 @@ func TestScrapePoolScrapeLoopsStarted(t *testing.T) {
|
||||
newLoop: newLoop,
|
||||
logger: nil,
|
||||
client: http.DefaultClient,
|
||||
metrics: newTestScrapeMetrics(t),
|
||||
}
|
||||
|
||||
tgs := []*targetgroup.Group{
|
||||
@ -643,6 +659,7 @@ func TestScrapeLoopStopBeforeRun(t *testing.T) {
|
||||
false,
|
||||
nil,
|
||||
false,
|
||||
newTestScrapeMetrics(t),
|
||||
)
|
||||
|
||||
// The scrape pool synchronizes on stopping scrape loops. However, new scrape
|
||||
@ -716,6 +733,7 @@ func TestScrapeLoopStop(t *testing.T) {
|
||||
false,
|
||||
nil,
|
||||
false,
|
||||
newTestScrapeMetrics(t),
|
||||
)
|
||||
|
||||
// Terminate loop after 2 scrapes.
|
||||
@ -793,6 +811,7 @@ func TestScrapeLoopRun(t *testing.T) {
|
||||
false,
|
||||
nil,
|
||||
false,
|
||||
newTestScrapeMetrics(t),
|
||||
)
|
||||
|
||||
// The loop must terminate during the initial offset if the context
|
||||
@ -849,6 +868,7 @@ func TestScrapeLoopRun(t *testing.T) {
|
||||
false,
|
||||
nil,
|
||||
false,
|
||||
newTestScrapeMetrics(t),
|
||||
)
|
||||
|
||||
go func() {
|
||||
@ -909,6 +929,7 @@ func TestScrapeLoopForcedErr(t *testing.T) {
|
||||
false,
|
||||
nil,
|
||||
false,
|
||||
newTestScrapeMetrics(t),
|
||||
)
|
||||
|
||||
forcedErr := fmt.Errorf("forced err")
|
||||
@ -945,7 +966,7 @@ func TestScrapeLoopMetadata(t *testing.T) {
|
||||
var (
|
||||
signal = make(chan struct{})
|
||||
scraper = &testScraper{}
|
||||
cache = newScrapeCache()
|
||||
cache = newScrapeCache(newTestScrapeMetrics(t))
|
||||
)
|
||||
defer close(signal)
|
||||
|
||||
@ -968,6 +989,7 @@ func TestScrapeLoopMetadata(t *testing.T) {
|
||||
false,
|
||||
nil,
|
||||
false,
|
||||
newTestScrapeMetrics(t),
|
||||
)
|
||||
defer cancel()
|
||||
|
||||
@ -1026,6 +1048,7 @@ func simpleTestScrapeLoop(t testing.TB) (context.Context, *scrapeLoop) {
|
||||
false,
|
||||
nil,
|
||||
false,
|
||||
newTestScrapeMetrics(t),
|
||||
)
|
||||
t.Cleanup(func() { cancel() })
|
||||
|
||||
@ -1087,6 +1110,7 @@ func TestScrapeLoopFailWithInvalidLabelsAfterRelabel(t *testing.T) {
|
||||
false,
|
||||
nil,
|
||||
false,
|
||||
newTestScrapeMetrics(t),
|
||||
)
|
||||
|
||||
slApp := sl.appender(ctx)
|
||||
@ -1166,6 +1190,7 @@ func TestScrapeLoopRunCreatesStaleMarkersOnFailedScrape(t *testing.T) {
|
||||
false,
|
||||
nil,
|
||||
false,
|
||||
newTestScrapeMetrics(t),
|
||||
)
|
||||
// Succeed once, several failures, then stop.
|
||||
numScrapes := 0
|
||||
@ -1230,6 +1255,7 @@ func TestScrapeLoopRunCreatesStaleMarkersOnParseFailure(t *testing.T) {
|
||||
false,
|
||||
nil,
|
||||
false,
|
||||
newTestScrapeMetrics(t),
|
||||
)
|
||||
|
||||
// Succeed once, several failures, then stop.
|
||||
@ -1297,6 +1323,7 @@ func TestScrapeLoopCache(t *testing.T) {
|
||||
false,
|
||||
nil,
|
||||
false,
|
||||
newTestScrapeMetrics(t),
|
||||
)
|
||||
|
||||
numScrapes := 0
|
||||
@ -1381,6 +1408,7 @@ func TestScrapeLoopCacheMemoryExhaustionProtection(t *testing.T) {
|
||||
false,
|
||||
nil,
|
||||
false,
|
||||
newTestScrapeMetrics(t),
|
||||
)
|
||||
|
||||
numScrapes := 0
|
||||
@ -1496,6 +1524,7 @@ func TestScrapeLoopAppend(t *testing.T) {
|
||||
false,
|
||||
nil,
|
||||
false,
|
||||
newTestScrapeMetrics(t),
|
||||
)
|
||||
|
||||
now := time.Now()
|
||||
@ -1583,7 +1612,8 @@ func TestScrapeLoopAppendForConflictingPrefixedLabels(t *testing.T) {
|
||||
return mutateSampleLabels(l, &Target{labels: labels.FromStrings(tc.targetLabels...)}, false, nil)
|
||||
},
|
||||
nil,
|
||||
func(ctx context.Context) storage.Appender { return app }, nil, 0, true, 0, 0, nil, 0, 0, false, false, false, nil, false,
|
||||
func(ctx context.Context) storage.Appender { return app },
|
||||
nil, 0, true, 0, 0, nil, 0, 0, false, false, false, nil, false, newTestScrapeMetrics(t),
|
||||
)
|
||||
slApp := sl.appender(context.Background())
|
||||
_, _, _, err := sl.append(slApp, []byte(tc.exposedLabels), "", time.Date(2000, 1, 1, 1, 0, 0, 0, time.UTC))
|
||||
@ -1623,6 +1653,7 @@ func TestScrapeLoopAppendCacheEntryButErrNotFound(t *testing.T) {
|
||||
false,
|
||||
nil,
|
||||
false,
|
||||
newTestScrapeMetrics(t),
|
||||
)
|
||||
|
||||
fakeRef := storage.SeriesRef(1)
|
||||
@ -1682,11 +1713,12 @@ func TestScrapeLoopAppendSampleLimit(t *testing.T) {
|
||||
false,
|
||||
nil,
|
||||
false,
|
||||
newTestScrapeMetrics(t),
|
||||
)
|
||||
|
||||
// Get the value of the Counter before performing the append.
|
||||
beforeMetric := dto.Metric{}
|
||||
err := targetScrapeSampleLimit.Write(&beforeMetric)
|
||||
err := sl.metrics.targetScrapeSampleLimit.Write(&beforeMetric)
|
||||
require.NoError(t, err)
|
||||
|
||||
beforeMetricValue := beforeMetric.GetCounter().GetValue()
|
||||
@ -1705,7 +1737,7 @@ func TestScrapeLoopAppendSampleLimit(t *testing.T) {
|
||||
// Check that the Counter has been incremented a single time for the scrape,
|
||||
// not multiple times for each sample.
|
||||
metric := dto.Metric{}
|
||||
err = targetScrapeSampleLimit.Write(&metric)
|
||||
err = sl.metrics.targetScrapeSampleLimit.Write(&metric)
|
||||
require.NoError(t, err)
|
||||
|
||||
value := metric.GetCounter().GetValue()
|
||||
@ -1760,10 +1792,11 @@ func TestScrapeLoop_HistogramBucketLimit(t *testing.T) {
|
||||
false,
|
||||
nil,
|
||||
false,
|
||||
newTestScrapeMetrics(t),
|
||||
)
|
||||
|
||||
metric := dto.Metric{}
|
||||
err := targetScrapeNativeHistogramBucketLimit.Write(&metric)
|
||||
err := sl.metrics.targetScrapeNativeHistogramBucketLimit.Write(&metric)
|
||||
require.NoError(t, err)
|
||||
beforeMetricValue := metric.GetCounter().GetValue()
|
||||
|
||||
@ -1801,7 +1834,7 @@ func TestScrapeLoop_HistogramBucketLimit(t *testing.T) {
|
||||
require.Equal(t, 3, added)
|
||||
require.Equal(t, 3, seriesAdded)
|
||||
|
||||
err = targetScrapeNativeHistogramBucketLimit.Write(&metric)
|
||||
err = sl.metrics.targetScrapeNativeHistogramBucketLimit.Write(&metric)
|
||||
require.NoError(t, err)
|
||||
metricValue := metric.GetCounter().GetValue()
|
||||
require.Equal(t, beforeMetricValue, metricValue)
|
||||
@ -1827,7 +1860,7 @@ func TestScrapeLoop_HistogramBucketLimit(t *testing.T) {
|
||||
require.Equal(t, 3, added)
|
||||
require.Equal(t, 0, seriesAdded)
|
||||
|
||||
err = targetScrapeNativeHistogramBucketLimit.Write(&metric)
|
||||
err = sl.metrics.targetScrapeNativeHistogramBucketLimit.Write(&metric)
|
||||
require.NoError(t, err)
|
||||
metricValue = metric.GetCounter().GetValue()
|
||||
require.Equal(t, beforeMetricValue+1, metricValue)
|
||||
@ -1859,6 +1892,7 @@ func TestScrapeLoop_ChangingMetricString(t *testing.T) {
|
||||
false,
|
||||
nil,
|
||||
false,
|
||||
newTestScrapeMetrics(t),
|
||||
)
|
||||
|
||||
now := time.Now()
|
||||
@ -1908,6 +1942,7 @@ func TestScrapeLoopAppendStaleness(t *testing.T) {
|
||||
false,
|
||||
nil,
|
||||
false,
|
||||
newTestScrapeMetrics(t),
|
||||
)
|
||||
|
||||
now := time.Now()
|
||||
@ -1960,6 +1995,7 @@ func TestScrapeLoopAppendNoStalenessIfTimestamp(t *testing.T) {
|
||||
false,
|
||||
nil,
|
||||
false,
|
||||
newTestScrapeMetrics(t),
|
||||
)
|
||||
|
||||
now := time.Now()
|
||||
@ -2286,6 +2322,7 @@ metric: <
|
||||
false,
|
||||
nil,
|
||||
false,
|
||||
newTestScrapeMetrics(t),
|
||||
)
|
||||
|
||||
now := time.Now()
|
||||
@ -2374,6 +2411,7 @@ func TestScrapeLoopAppendExemplarSeries(t *testing.T) {
|
||||
false,
|
||||
nil,
|
||||
false,
|
||||
newTestScrapeMetrics(t),
|
||||
)
|
||||
|
||||
now := time.Now()
|
||||
@ -2427,6 +2465,7 @@ func TestScrapeLoopRunReportsTargetDownOnScrapeError(t *testing.T) {
|
||||
false,
|
||||
nil,
|
||||
false,
|
||||
newTestScrapeMetrics(t),
|
||||
)
|
||||
|
||||
scraper.scrapeFunc = func(ctx context.Context, w io.Writer) error {
|
||||
@ -2464,6 +2503,7 @@ func TestScrapeLoopRunReportsTargetDownOnInvalidUTF8(t *testing.T) {
|
||||
false,
|
||||
nil,
|
||||
false,
|
||||
newTestScrapeMetrics(t),
|
||||
)
|
||||
|
||||
scraper.scrapeFunc = func(ctx context.Context, w io.Writer) error {
|
||||
@ -2514,6 +2554,7 @@ func TestScrapeLoopAppendGracefullyIfAmendOrOutOfOrderOrOutOfBounds(t *testing.T
|
||||
false,
|
||||
nil,
|
||||
false,
|
||||
newTestScrapeMetrics(t),
|
||||
)
|
||||
|
||||
now := time.Unix(1, 0)
|
||||
@ -2560,6 +2601,7 @@ func TestScrapeLoopOutOfBoundsTimeError(t *testing.T) {
|
||||
false,
|
||||
nil,
|
||||
false,
|
||||
newTestScrapeMetrics(t),
|
||||
)
|
||||
|
||||
now := time.Now().Add(20 * time.Minute)
|
||||
@ -2755,6 +2797,7 @@ func TestTargetScraperBodySizeLimit(t *testing.T) {
|
||||
client: http.DefaultClient,
|
||||
bodySizeLimit: bodySizeLimit,
|
||||
acceptHeader: acceptHeader(config.DefaultGlobalConfig.ScrapeProtocols),
|
||||
metrics: newTestScrapeMetrics(t),
|
||||
}
|
||||
var buf bytes.Buffer
|
||||
|
||||
@ -2849,6 +2892,7 @@ func TestScrapeLoop_RespectTimestamps(t *testing.T) {
|
||||
false,
|
||||
nil,
|
||||
false,
|
||||
newTestScrapeMetrics(t),
|
||||
)
|
||||
|
||||
now := time.Now()
|
||||
@ -2891,6 +2935,7 @@ func TestScrapeLoop_DiscardTimestamps(t *testing.T) {
|
||||
false,
|
||||
nil,
|
||||
false,
|
||||
newTestScrapeMetrics(t),
|
||||
)
|
||||
|
||||
now := time.Now()
|
||||
@ -2932,6 +2977,7 @@ func TestScrapeLoopDiscardDuplicateLabels(t *testing.T) {
|
||||
false,
|
||||
nil,
|
||||
false,
|
||||
newTestScrapeMetrics(t),
|
||||
)
|
||||
defer cancel()
|
||||
|
||||
@ -2991,6 +3037,7 @@ func TestScrapeLoopDiscardUnnamedMetrics(t *testing.T) {
|
||||
false,
|
||||
nil,
|
||||
false,
|
||||
newTestScrapeMetrics(t),
|
||||
)
|
||||
defer cancel()
|
||||
|
||||
@ -3083,7 +3130,7 @@ func TestReuseScrapeCache(t *testing.T) {
|
||||
ScrapeInterval: model.Duration(5 * time.Second),
|
||||
MetricsPath: "/metrics",
|
||||
}
|
||||
sp, _ = newScrapePool(cfg, app, 0, nil, &Options{})
|
||||
sp, _ = newScrapePool(cfg, app, 0, nil, &Options{}, newTestScrapeMetrics(t))
|
||||
t1 = &Target{
|
||||
discoveredLabels: labels.FromStrings("labelNew", "nameNew", "labelNew1", "nameNew1", "labelNew2", "nameNew2"),
|
||||
}
|
||||
@ -3255,6 +3302,7 @@ func TestScrapeAddFast(t *testing.T) {
|
||||
false,
|
||||
nil,
|
||||
false,
|
||||
newTestScrapeMetrics(t),
|
||||
)
|
||||
defer cancel()
|
||||
|
||||
@ -3275,7 +3323,7 @@ func TestScrapeAddFast(t *testing.T) {
|
||||
require.NoError(t, slApp.Commit())
|
||||
}
|
||||
|
||||
func TestReuseCacheRace(*testing.T) {
|
||||
func TestReuseCacheRace(t *testing.T) {
|
||||
var (
|
||||
app = &nopAppendable{}
|
||||
cfg = &config.ScrapeConfig{
|
||||
@ -3284,7 +3332,7 @@ func TestReuseCacheRace(*testing.T) {
|
||||
ScrapeInterval: model.Duration(5 * time.Second),
|
||||
MetricsPath: "/metrics",
|
||||
}
|
||||
sp, _ = newScrapePool(cfg, app, 0, nil, &Options{})
|
||||
sp, _ = newScrapePool(cfg, app, 0, nil, &Options{}, newTestScrapeMetrics(t))
|
||||
t1 = &Target{
|
||||
discoveredLabels: labels.FromStrings("labelNew", "nameNew"),
|
||||
}
|
||||
@ -3309,7 +3357,7 @@ func TestReuseCacheRace(*testing.T) {
|
||||
|
||||
func TestCheckAddError(t *testing.T) {
|
||||
var appErrs appendErrors
|
||||
sl := scrapeLoop{l: log.NewNopLogger()}
|
||||
sl := scrapeLoop{l: log.NewNopLogger(), metrics: newTestScrapeMetrics(t)}
|
||||
sl.checkAddError(nil, nil, nil, storage.ErrOutOfOrderSample, nil, nil, &appErrs)
|
||||
require.Equal(t, 1, appErrs.numOutOfOrder)
|
||||
}
|
||||
@ -3342,6 +3390,7 @@ func TestScrapeReportSingleAppender(t *testing.T) {
|
||||
false,
|
||||
nil,
|
||||
false,
|
||||
newTestScrapeMetrics(t),
|
||||
)
|
||||
|
||||
numScrapes := 0
|
||||
@ -3412,7 +3461,7 @@ func TestScrapeReportLimit(t *testing.T) {
|
||||
}))
|
||||
defer ts.Close()
|
||||
|
||||
sp, err := newScrapePool(cfg, s, 0, nil, &Options{})
|
||||
sp, err := newScrapePool(cfg, s, 0, nil, &Options{}, newTestScrapeMetrics(t))
|
||||
require.NoError(t, err)
|
||||
defer sp.stop()
|
||||
|
||||
@ -3545,6 +3594,7 @@ func TestScrapeLoopLabelLimit(t *testing.T) {
|
||||
false,
|
||||
nil,
|
||||
false,
|
||||
newTestScrapeMetrics(t),
|
||||
)
|
||||
|
||||
slApp := sl.appender(context.Background())
|
||||
@ -3583,7 +3633,7 @@ func TestTargetScrapeIntervalAndTimeoutRelabel(t *testing.T) {
|
||||
},
|
||||
},
|
||||
}
|
||||
sp, _ := newScrapePool(config, &nopAppendable{}, 0, nil, &Options{})
|
||||
sp, _ := newScrapePool(config, &nopAppendable{}, 0, nil, &Options{}, newTestScrapeMetrics(t))
|
||||
tgts := []*targetgroup.Group{
|
||||
{
|
||||
Targets: []model.LabelSet{{model.AddressLabel: "127.0.0.1:9090"}},
|
||||
|
Loading…
Reference in New Issue
Block a user