prometheus/scrape/manager.go

325 lines
8.5 KiB
Go
Raw Normal View History

2017-11-25 05:13:54 -08:00
// Copyright 2013 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
2017-11-25 05:13:54 -08:00
import (
"fmt"
"hash/fnv"
2018-01-18 03:49:42 -08:00
"reflect"
2018-01-17 03:46:17 -08:00
"sync"
"time"
2017-11-25 05:13:54 -08:00
"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"
2017-11-25 05:13:54 -08:00
"github.com/prometheus/prometheus/config"
Refactor SD configuration to remove `config` dependency (#3629) * refactor: move targetGroup struct and CheckOverflow() to their own package * refactor: move auth and security related structs to a utility package, fix import error in utility package * refactor: Azure SD, remove SD struct from config * refactor: DNS SD, remove SD struct from config into dns package * refactor: ec2 SD, move SD struct from config into the ec2 package * refactor: file SD, move SD struct from config to file discovery package * refactor: gce, move SD struct from config to gce discovery package * refactor: move HTTPClientConfig and URL into util/config, fix import error in httputil * refactor: consul, move SD struct from config into consul discovery package * refactor: marathon, move SD struct from config into marathon discovery package * refactor: triton, move SD struct from config to triton discovery package, fix test * refactor: zookeeper, move SD structs from config to zookeeper discovery package * refactor: openstack, remove SD struct from config, move into openstack discovery package * refactor: kubernetes, move SD struct from config into kubernetes discovery package * refactor: notifier, use targetgroup package instead of config * refactor: tests for file, marathon, triton SD - use targetgroup package instead of config.TargetGroup * refactor: retrieval, use targetgroup package instead of config.TargetGroup * refactor: storage, use config util package * refactor: discovery manager, use targetgroup package instead of config.TargetGroup * refactor: use HTTPClient and TLS config from configUtil instead of config * refactor: tests, use targetgroup package instead of config.TargetGroup * refactor: fix tagetgroup.Group pointers that were removed by mistake * refactor: openstack, kubernetes: drop prefixes * refactor: remove import aliases forced due to vscode bug * refactor: move main SD struct out of config into discovery/config * refactor: rename configUtil to config_util * refactor: rename yamlUtil to yaml_config * refactor: kubernetes, remove prefixes * refactor: move the TargetGroup package to discovery/ * refactor: fix order of imports
2017-12-29 12:01:34 -08:00
"github.com/prometheus/prometheus/discovery/targetgroup"
"github.com/prometheus/prometheus/model/labels"
2017-11-25 05:13:54 -08:00
"github.com/prometheus/prometheus/storage"
"github.com/prometheus/prometheus/util/osutil"
2017-11-25 05:13:54 -08:00
)
// NewManager is the Manager constructor.
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{
2017-11-25 05:13:54 -08:00
append: app,
opts: o,
2017-11-25 05:13:54 -08:00
logger: logger,
scrapeConfigs: make(map[string]*config.ScrapeConfig),
scrapePools: make(map[string]*scrapePool),
graceShut: make(chan struct{}),
triggerReload: make(chan struct{}, 1),
metrics: sm,
2017-11-25 05:13:54 -08:00
}
m.metrics.setTargetMetadataCacheGatherer(m)
return m, nil
2017-11-25 05:13:54 -08:00
}
// Options are the configuration parameters to the scrape manager.
type Options struct {
ExtraMetrics bool
NoDefaultPort bool
// Option used by downstream scraper users like OpenTelemetry Collector
// to help lookup metric metadata. Should be false for Prometheus.
PassMetadataInContext bool
Append metadata to the WAL in the scrape loop (#10312) * Append metadata to the WAL Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Remove extra whitespace; Reword some docstrings and comments Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Use RLock() for hasNewMetadata check Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Use single byte for metric type in RefMetadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Update proposed WAL format for single-byte type metadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Address first round of review comments Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Amend description of metadata in wal.md Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Correct key used to retrieve metadata from cache When we're setting metadata entries in the scrapeCace, we're using the p.Help(), p.Unit(), p.Type() helpers, which retrieve the series name and use it as the cache key. When checking for cache entries though, we used p.Series() as the key, which included the metric name _with_ its labels. That meant that we were never actually hitting the cache. We're fixing this by utiling the __name__ internal label for correctly getting the cache entries after they've been set by setHelp(), setType() or setUnit(). Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Put feature behind a feature flag Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Reorder WAL format document Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix CR comments Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Extract logic about changing metadata in an anonymous function Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Implement new proposed WAL format and amend relevant tests Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Use 'const' for metadata field names Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Apply metadata to head memSeries in Commit, not in AppendMetadata Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Add docstring and rename extracted helper in scrape.go Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Fix review comments around TestMetadata* tests Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Rebase with merged TSDB changes; fix duplicate definitions after rebase Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Remove leftover changes on db_test.go Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Rename feature flag Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Simplify updateMetadata helper function Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> * Remove extra newline Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com> Signed-off-by: Paschalis Tsilias <paschalist0@gmail.com>
2022-08-31 06:50:05 -07:00
// Option to enable the experimental in-memory metadata storage and append
// metadata to the WAL.
EnableMetadataStorage bool
// Option to increase the interval used by scrape manager to throttle target groups updates.
DiscoveryReloadInterval model.Duration
// Optional HTTP client options to use when scraping.
HTTPClientOptions []config_util.HTTPClientOption
}
// Manager maintains a set of scrape pools and manages start/stop cycles
// when receiving new target groups from the discovery manager.
type Manager struct {
opts *Options
logger log.Logger
append storage.Appendable
graceShut chan struct{}
offsetSeed uint64 // Global offsetSeed seed is used to spread scrape workload across HA setup.
mtxScrape sync.Mutex // Guards the fields below.
2017-11-25 05:13:54 -08:00
scrapeConfigs map[string]*config.ScrapeConfig
scrapePools map[string]*scrapePool
targetSets map[string][]*targetgroup.Group
triggerReload chan struct{}
metrics *scrapeMetrics
2017-11-25 05:13:54 -08:00
}
// Run receives and saves target set updates and triggers the scraping loops reloading.
// Reloading happens in the background so that it doesn't block receiving targets updates.
func (m *Manager) Run(tsets <-chan map[string][]*targetgroup.Group) error {
go m.reloader()
2017-11-25 05:13:54 -08:00
for {
select {
case ts := <-tsets:
m.updateTsets(ts)
select {
case m.triggerReload <- struct{}{}:
default:
}
case <-m.graceShut:
return nil
2017-11-25 05:13:54 -08:00
}
}
}
func (m *Manager) reloader() {
reloadIntervalDuration := m.opts.DiscoveryReloadInterval
if reloadIntervalDuration < model.Duration(5*time.Second) {
reloadIntervalDuration = model.Duration(5 * time.Second)
}
ticker := time.NewTicker(time.Duration(reloadIntervalDuration))
defer ticker.Stop()
for {
select {
case <-m.graceShut:
return
case <-ticker.C:
select {
case <-m.triggerReload:
m.reload()
case <-m.graceShut:
return
}
}
}
}
func (m *Manager) reload() {
m.mtxScrape.Lock()
var wg sync.WaitGroup
for setName, groups := range m.targetSets {
if _, ok := m.scrapePools[setName]; !ok {
scrapeConfig, ok := m.scrapeConfigs[setName]
if !ok {
level.Error(m.logger).Log("msg", "error reloading target set", "err", "invalid config id:"+setName)
continue
}
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
}
m.scrapePools[setName] = sp
}
wg.Add(1)
// Run the sync in parallel as these take a while and at high load can't catch up.
go func(sp *scrapePool, groups []*targetgroup.Group) {
sp.Sync(groups)
wg.Done()
}(m.scrapePools[setName], groups)
}
m.mtxScrape.Unlock()
wg.Wait()
}
// setOffsetSeed calculates a global offsetSeed per server relying on extra label set.
func (m *Manager) setOffsetSeed(labels labels.Labels) error {
h := fnv.New64a()
hostname, err := osutil.GetFQDN()
if err != nil {
return err
}
if _, err := fmt.Fprintf(h, "%s%s", hostname, labels.String()); err != nil {
return err
}
m.offsetSeed = h.Sum64()
return nil
}
// Stop cancels all running scrape pools and blocks until all have exited.
func (m *Manager) Stop() {
m.mtxScrape.Lock()
defer m.mtxScrape.Unlock()
for _, sp := range m.scrapePools {
sp.stop()
}
close(m.graceShut)
}
func (m *Manager) updateTsets(tsets map[string][]*targetgroup.Group) {
m.mtxScrape.Lock()
m.targetSets = tsets
m.mtxScrape.Unlock()
}
2017-11-25 05:13:54 -08:00
// ApplyConfig resets the manager's target providers and job configurations as defined by the new cfg.
func (m *Manager) ApplyConfig(cfg *config.Config) error {
m.mtxScrape.Lock()
defer m.mtxScrape.Unlock()
scfgs, err := cfg.GetScrapeConfigs()
if err != nil {
return err
}
2018-01-17 03:46:17 -08:00
c := make(map[string]*config.ScrapeConfig)
for _, scfg := range scfgs {
2018-01-17 03:46:17 -08:00
c[scfg.JobName] = scfg
2017-11-25 05:13:54 -08:00
}
2018-01-17 03:46:17 -08:00
m.scrapeConfigs = c
2018-01-18 03:49:42 -08:00
if err := m.setOffsetSeed(cfg.GlobalConfig.ExternalLabels); err != nil {
return err
}
// Cleanup and reload pool if the configuration has changed.
var failed bool
2018-01-18 03:49:42 -08:00
for name, sp := range m.scrapePools {
switch cfg, ok := m.scrapeConfigs[name]; {
case !ok:
2018-01-18 03:49:42 -08:00
sp.stop()
delete(m.scrapePools, name)
case !reflect.DeepEqual(sp.config, cfg):
err := sp.reload(cfg)
if err != nil {
level.Error(m.logger).Log("msg", "error reloading scrape pool", "err", err, "scrape_pool", name)
failed = true
}
2018-01-18 03:49:42 -08:00
}
}
if failed {
return errors.New("failed to apply the new configuration")
}
2017-11-25 05:13:54 -08:00
return nil
}
// TargetsAll returns active and dropped targets grouped by job_name.
func (m *Manager) TargetsAll() map[string][]*Target {
m.mtxScrape.Lock()
defer m.mtxScrape.Unlock()
2018-01-17 03:46:17 -08:00
targets := make(map[string][]*Target, len(m.scrapePools))
for tset, sp := range m.scrapePools {
targets[tset] = append(sp.ActiveTargets(), sp.DroppedTargets()...)
}
return targets
}
2018-01-17 03:46:17 -08:00
2022-12-23 02:55:08 -08:00
// ScrapePools returns the list of all scrape pool names.
func (m *Manager) ScrapePools() []string {
m.mtxScrape.Lock()
defer m.mtxScrape.Unlock()
names := make([]string, 0, len(m.scrapePools))
for name := range m.scrapePools {
names = append(names, name)
}
return names
}
// TargetsActive returns the active targets currently being scraped.
func (m *Manager) TargetsActive() map[string][]*Target {
m.mtxScrape.Lock()
defer m.mtxScrape.Unlock()
2017-11-25 05:13:54 -08:00
var (
wg sync.WaitGroup
mtx sync.Mutex
)
targets := make(map[string][]*Target, len(m.scrapePools))
wg.Add(len(m.scrapePools))
for tset, sp := range m.scrapePools {
// Running in parallel limits the blocking time of scrapePool to scrape
// interval when there's an update from SD.
go func(tset string, sp *scrapePool) {
mtx.Lock()
targets[tset] = sp.ActiveTargets()
mtx.Unlock()
wg.Done()
}(tset, sp)
}
wg.Wait()
return targets
}
// TargetsDropped returns the dropped targets during relabelling, subject to KeepDroppedTargets limit.
func (m *Manager) TargetsDropped() map[string][]*Target {
m.mtxScrape.Lock()
defer m.mtxScrape.Unlock()
2017-11-25 05:13:54 -08:00
targets := make(map[string][]*Target, len(m.scrapePools))
for tset, sp := range m.scrapePools {
targets[tset] = sp.DroppedTargets()
2018-01-14 11:42:31 -08:00
}
return targets
2017-11-25 05:13:54 -08:00
}
func (m *Manager) TargetsDroppedCounts() map[string]int {
m.mtxScrape.Lock()
defer m.mtxScrape.Unlock()
counts := make(map[string]int, len(m.scrapePools))
for tset, sp := range m.scrapePools {
counts[tset] = sp.droppedTargetsCount
}
return counts
}