mirror of
https://github.com/prometheus/prometheus.git
synced 2024-12-28 15:09:39 -08:00
Preserve target state across reloads.
This commit moves Scraper handling into a separate scrapePool type. TargetSets only manage TargetProvider lifecycles and sync the retrieved updates to the scrapePool. TargetProviders are now expected to send a full initial target set within 5 seconds. The scrapePools preserve target state across reloads and only drop targets after the initial set was synced.
This commit is contained in:
parent
5b30bdb610
commit
d15adfc917
|
@ -113,6 +113,7 @@ func (dd *DNSDiscovery) Run(ctx context.Context, ch chan<- []*config.TargetGroup
|
||||||
|
|
||||||
func (dd *DNSDiscovery) refreshAll(ch chan<- []*config.TargetGroup) {
|
func (dd *DNSDiscovery) refreshAll(ch chan<- []*config.TargetGroup) {
|
||||||
var wg sync.WaitGroup
|
var wg sync.WaitGroup
|
||||||
|
|
||||||
wg.Add(len(dd.names))
|
wg.Add(len(dd.names))
|
||||||
for _, name := range dd.names {
|
for _, name := range dd.names {
|
||||||
go func(n string) {
|
go func(n string) {
|
||||||
|
@ -122,6 +123,7 @@ func (dd *DNSDiscovery) refreshAll(ch chan<- []*config.TargetGroup) {
|
||||||
wg.Done()
|
wg.Done()
|
||||||
}(name)
|
}(name)
|
||||||
}
|
}
|
||||||
|
|
||||||
wg.Wait()
|
wg.Wait()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -167,6 +167,7 @@ type Target struct {
|
||||||
scraperStopping chan struct{}
|
scraperStopping chan struct{}
|
||||||
// Closing scraperStopped signals that scraping has been stopped.
|
// Closing scraperStopped signals that scraping has been stopped.
|
||||||
scraperStopped chan struct{}
|
scraperStopped chan struct{}
|
||||||
|
running bool
|
||||||
|
|
||||||
// Mutex protects the members below.
|
// Mutex protects the members below.
|
||||||
sync.RWMutex
|
sync.RWMutex
|
||||||
|
@ -411,9 +412,11 @@ func (t *Target) InstanceIdentifier() string {
|
||||||
func (t *Target) RunScraper(sampleAppender storage.SampleAppender) {
|
func (t *Target) RunScraper(sampleAppender storage.SampleAppender) {
|
||||||
defer close(t.scraperStopped)
|
defer close(t.scraperStopped)
|
||||||
|
|
||||||
lastScrapeInterval := t.interval()
|
t.Lock()
|
||||||
|
t.running = true
|
||||||
|
t.Unlock()
|
||||||
|
|
||||||
log.Debugf("Starting scraper for target %v...", t)
|
lastScrapeInterval := t.interval()
|
||||||
|
|
||||||
select {
|
select {
|
||||||
case <-time.After(t.offset(lastScrapeInterval)):
|
case <-time.After(t.offset(lastScrapeInterval)):
|
||||||
|
@ -471,6 +474,14 @@ func (t *Target) RunScraper(sampleAppender storage.SampleAppender) {
|
||||||
|
|
||||||
// StopScraper implements Target.
|
// StopScraper implements Target.
|
||||||
func (t *Target) StopScraper() {
|
func (t *Target) StopScraper() {
|
||||||
|
t.Lock()
|
||||||
|
if !t.running {
|
||||||
|
t.Unlock()
|
||||||
|
return
|
||||||
|
}
|
||||||
|
t.running = false
|
||||||
|
t.Unlock()
|
||||||
|
|
||||||
log.Debugf("Stopping scraper for target %v...", t)
|
log.Debugf("Stopping scraper for target %v...", t)
|
||||||
|
|
||||||
close(t.scraperStopping)
|
close(t.scraperStopping)
|
||||||
|
|
|
@ -17,6 +17,7 @@ import (
|
||||||
"fmt"
|
"fmt"
|
||||||
"strings"
|
"strings"
|
||||||
"sync"
|
"sync"
|
||||||
|
"time"
|
||||||
|
|
||||||
"github.com/prometheus/common/log"
|
"github.com/prometheus/common/log"
|
||||||
"github.com/prometheus/common/model"
|
"github.com/prometheus/common/model"
|
||||||
|
@ -48,20 +49,22 @@ type TargetProvider interface {
|
||||||
// target providers.
|
// target providers.
|
||||||
type TargetManager struct {
|
type TargetManager struct {
|
||||||
appender storage.SampleAppender
|
appender storage.SampleAppender
|
||||||
|
scrapeConfigs []*config.ScrapeConfig
|
||||||
|
|
||||||
mtx sync.RWMutex
|
mtx sync.RWMutex
|
||||||
ctx context.Context
|
ctx context.Context
|
||||||
cancel func()
|
cancel func()
|
||||||
wg sync.WaitGroup
|
wg sync.WaitGroup
|
||||||
|
|
||||||
// Providers by the scrape configs they are derived from.
|
// Set of unqiue targets by scrape configuration.
|
||||||
scrapeSets []*scrapeSet
|
targetSets map[string]*targetSet
|
||||||
}
|
}
|
||||||
|
|
||||||
// NewTargetManager creates a new TargetManager.
|
// NewTargetManager creates a new TargetManager.
|
||||||
func NewTargetManager(app storage.SampleAppender) *TargetManager {
|
func NewTargetManager(app storage.SampleAppender) *TargetManager {
|
||||||
return &TargetManager{
|
return &TargetManager{
|
||||||
appender: app,
|
appender: app,
|
||||||
|
targetSets: map[string]*targetSet{},
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -72,16 +75,39 @@ func (tm *TargetManager) Run() {
|
||||||
tm.mtx.Lock()
|
tm.mtx.Lock()
|
||||||
tm.ctx, tm.cancel = context.WithCancel(context.Background())
|
tm.ctx, tm.cancel = context.WithCancel(context.Background())
|
||||||
|
|
||||||
for _, ss := range tm.scrapeSets {
|
jobs := map[string]struct{}{}
|
||||||
|
|
||||||
|
// Start new target sets and update existing ones.
|
||||||
|
for _, scfg := range tm.scrapeConfigs {
|
||||||
|
jobs[scfg.JobName] = struct{}{}
|
||||||
|
|
||||||
|
ts, ok := tm.targetSets[scfg.JobName]
|
||||||
|
if !ok {
|
||||||
|
ts = newTargetSet(scfg, tm.appender)
|
||||||
|
tm.targetSets[scfg.JobName] = ts
|
||||||
|
}
|
||||||
|
ts.runProviders(tm.ctx, providersFromConfig(scfg))
|
||||||
|
}
|
||||||
|
|
||||||
|
// Stop old target sets.
|
||||||
|
for name := range tm.targetSets {
|
||||||
|
if _, ok := jobs[name]; !ok {
|
||||||
|
delete(tm.targetSets, name)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Run target sets.
|
||||||
|
for _, ts := range tm.targetSets {
|
||||||
tm.wg.Add(1)
|
tm.wg.Add(1)
|
||||||
|
|
||||||
go func(ss *scrapeSet) {
|
go func(ts *targetSet) {
|
||||||
ss.run(tm.ctx)
|
ts.run(tm.ctx)
|
||||||
tm.wg.Done()
|
tm.wg.Done()
|
||||||
}(ss)
|
}(ts)
|
||||||
}
|
}
|
||||||
|
|
||||||
tm.mtx.Unlock()
|
tm.mtx.Unlock()
|
||||||
|
|
||||||
tm.wg.Wait()
|
tm.wg.Wait()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -90,14 +116,16 @@ func (tm *TargetManager) Stop() {
|
||||||
log.Infoln("Stopping target manager...")
|
log.Infoln("Stopping target manager...")
|
||||||
|
|
||||||
tm.mtx.Lock()
|
tm.mtx.Lock()
|
||||||
|
// Cancel the base context, this will cause all target providers to shut down
|
||||||
// Cancel the base context, this will cause all in-flight scrapes to abort immmediately.
|
// and all in-flight scrapes to abort immmediately.
|
||||||
// Started inserts will be finished before terminating.
|
// Started inserts will be finished before terminating.
|
||||||
tm.cancel()
|
tm.cancel()
|
||||||
tm.mtx.Unlock()
|
tm.mtx.Unlock()
|
||||||
|
|
||||||
// Wait for all provider sets to terminate.
|
// Wait for all scrape inserts to complete.
|
||||||
tm.wg.Wait()
|
tm.wg.Wait()
|
||||||
|
|
||||||
|
log.Debugln("Target manager stopped")
|
||||||
}
|
}
|
||||||
|
|
||||||
// Pools returns the targets currently being scraped bucketed by their job name.
|
// Pools returns the targets currently being scraped bucketed by their job name.
|
||||||
|
@ -108,8 +136,8 @@ func (tm *TargetManager) Pools() map[string][]*Target {
|
||||||
pools := map[string][]*Target{}
|
pools := map[string][]*Target{}
|
||||||
|
|
||||||
// TODO(fabxc): this is just a hack to maintain compatibility for now.
|
// TODO(fabxc): this is just a hack to maintain compatibility for now.
|
||||||
for _, ps := range tm.scrapeSets {
|
for _, ps := range tm.targetSets {
|
||||||
for _, ts := range ps.tgroups {
|
for _, ts := range ps.scrapePool.tgroups {
|
||||||
for _, t := range ts {
|
for _, t := range ts {
|
||||||
job := string(t.Labels()[model.JobLabel])
|
job := string(t.Labels()[model.JobLabel])
|
||||||
pools[job] = append(pools[job], t)
|
pools[job] = append(pools[job], t)
|
||||||
|
@ -135,66 +163,120 @@ func (tm *TargetManager) ApplyConfig(cfg *config.Config) bool {
|
||||||
}
|
}
|
||||||
|
|
||||||
tm.mtx.Lock()
|
tm.mtx.Lock()
|
||||||
|
tm.scrapeConfigs = cfg.ScrapeConfigs
|
||||||
tm.scrapeSets = tm.scrapeSets[:0]
|
|
||||||
|
|
||||||
for _, scfg := range cfg.ScrapeConfigs {
|
|
||||||
tm.scrapeSets = append(tm.scrapeSets, newScrapeSet(tm.appender, scfg))
|
|
||||||
}
|
|
||||||
|
|
||||||
tm.mtx.Unlock()
|
tm.mtx.Unlock()
|
||||||
|
|
||||||
return true
|
return true
|
||||||
}
|
}
|
||||||
|
|
||||||
// scrapeSet holds several TargetProviders for which the same scrape configuration
|
// targetSet holds several TargetProviders for which the same scrape configuration
|
||||||
// is used. It runs the target providers and starts and stops scrapers as it
|
// is used. It runs the target providers and starts and stops scrapers as it
|
||||||
// receives target updates.
|
// receives target updates.
|
||||||
type scrapeSet struct {
|
type targetSet struct {
|
||||||
appender storage.SampleAppender
|
|
||||||
|
|
||||||
config *config.ScrapeConfig
|
|
||||||
tgroups map[string]map[model.Fingerprint]*Target
|
|
||||||
|
|
||||||
mtx sync.RWMutex
|
mtx sync.RWMutex
|
||||||
|
tgroups map[string]map[model.Fingerprint]*Target
|
||||||
|
providers map[string]TargetProvider
|
||||||
|
|
||||||
|
scrapePool *scrapePool
|
||||||
|
config *config.ScrapeConfig
|
||||||
|
|
||||||
|
stopProviders func()
|
||||||
|
syncCh chan struct{}
|
||||||
}
|
}
|
||||||
|
|
||||||
func newScrapeSet(app storage.SampleAppender, cfg *config.ScrapeConfig) *scrapeSet {
|
func newTargetSet(cfg *config.ScrapeConfig, app storage.SampleAppender) *targetSet {
|
||||||
return &scrapeSet{
|
ts := &targetSet{
|
||||||
appender: app,
|
|
||||||
config: cfg,
|
|
||||||
tgroups: map[string]map[model.Fingerprint]*Target{},
|
tgroups: map[string]map[model.Fingerprint]*Target{},
|
||||||
|
scrapePool: newScrapePool(app),
|
||||||
|
syncCh: make(chan struct{}, 1),
|
||||||
|
config: cfg,
|
||||||
}
|
}
|
||||||
|
return ts
|
||||||
}
|
}
|
||||||
|
|
||||||
// run starts the target providers with the given context and consumes
|
func (ts *targetSet) run(ctx context.Context) {
|
||||||
// and handles their updates. If the context is done, it blocks until the
|
ts.scrapePool.ctx = ctx
|
||||||
// target scrapers have terminated.
|
|
||||||
func (ss *scrapeSet) run(ctx context.Context) {
|
Loop:
|
||||||
var (
|
for {
|
||||||
providers = providersFromConfig(ss.config)
|
// Throttle syncing to once per five seconds.
|
||||||
wg sync.WaitGroup
|
select {
|
||||||
)
|
case <-ctx.Done():
|
||||||
|
break Loop
|
||||||
|
case <-time.After(5 * time.Second):
|
||||||
|
}
|
||||||
|
|
||||||
|
select {
|
||||||
|
case <-ctx.Done():
|
||||||
|
break Loop
|
||||||
|
case <-ts.syncCh:
|
||||||
|
ts.sync()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// We want to wait for all pending target scrapes to complete though to ensure there'll
|
||||||
|
// be no more storage writes after this point.
|
||||||
|
ts.scrapePool.stop()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (ts *targetSet) sync() {
|
||||||
|
// TODO(fabxc): temporary simple version. For a deduplicating scrape pool we will
|
||||||
|
// submit a list of all targets.
|
||||||
|
ts.scrapePool.sync(ts.tgroups)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (ts *targetSet) runProviders(ctx context.Context, providers map[string]TargetProvider) {
|
||||||
|
// Lock for the entire time. This may mean up to 5 seconds until the full initial set
|
||||||
|
// is retrieved and applied.
|
||||||
|
// We could release earlier with some tweaks, but this is easier to reason about.
|
||||||
|
ts.mtx.Lock()
|
||||||
|
defer ts.mtx.Unlock()
|
||||||
|
|
||||||
|
var wg sync.WaitGroup
|
||||||
|
|
||||||
|
if ts.stopProviders != nil {
|
||||||
|
ts.stopProviders()
|
||||||
|
}
|
||||||
|
ctx, ts.stopProviders = context.WithCancel(ctx)
|
||||||
|
|
||||||
for name, prov := range providers {
|
for name, prov := range providers {
|
||||||
var (
|
|
||||||
updates = make(chan []*config.TargetGroup)
|
|
||||||
)
|
|
||||||
|
|
||||||
wg.Add(1)
|
wg.Add(1)
|
||||||
// The update and stopping operations for the target provider handling are blocking.
|
|
||||||
// Thus the run method only returns if all background processing is complete.
|
|
||||||
go func(name string, prov TargetProvider) {
|
|
||||||
defer wg.Done()
|
|
||||||
|
|
||||||
|
updates := make(chan []*config.TargetGroup)
|
||||||
|
|
||||||
|
go func(name string, prov TargetProvider) {
|
||||||
|
var initial []*config.TargetGroup
|
||||||
|
|
||||||
|
select {
|
||||||
|
case <-ctx.Done():
|
||||||
|
wg.Done()
|
||||||
|
return
|
||||||
|
case initial = <-updates:
|
||||||
|
// First set of all targets the provider knows.
|
||||||
|
case <-time.After(5 * time.Second):
|
||||||
|
// Initial set didn't arrive. Act as if it was empty
|
||||||
|
// and wait for updates later on.
|
||||||
|
}
|
||||||
|
|
||||||
|
for _, tgroup := range initial {
|
||||||
|
targets, err := targetsFromGroup(tgroup, ts.config)
|
||||||
|
if err != nil {
|
||||||
|
log.With("target_group", tgroup).Errorf("Target update failed: %s", err)
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
ts.tgroups[name+"/"+tgroup.Source] = targets
|
||||||
|
}
|
||||||
|
|
||||||
|
wg.Done()
|
||||||
|
|
||||||
|
// Start listening for further updates.
|
||||||
for {
|
for {
|
||||||
select {
|
select {
|
||||||
case <-ctx.Done():
|
case <-ctx.Done():
|
||||||
ss.stopScrapers(name)
|
|
||||||
return
|
return
|
||||||
case tgs := <-updates:
|
case tgs := <-updates:
|
||||||
for _, tg := range tgs {
|
for _, tg := range tgs {
|
||||||
if err := ss.update(name, tg); err != nil {
|
if err := ts.update(name, tg); err != nil {
|
||||||
log.With("target_group", tg).Errorf("Target update failed: %s", err)
|
log.With("target_group", tg).Errorf("Target update failed: %s", err)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -206,18 +288,52 @@ func (ss *scrapeSet) run(ctx context.Context) {
|
||||||
}
|
}
|
||||||
|
|
||||||
wg.Wait()
|
wg.Wait()
|
||||||
|
|
||||||
|
ts.sync()
|
||||||
}
|
}
|
||||||
|
|
||||||
// stopScrapers shuts down all active scrapers for a provider.
|
// update handles a target group update from a target provider identified by the name.
|
||||||
func (ss *scrapeSet) stopScrapers(name string) {
|
func (ts *targetSet) update(name string, tgroup *config.TargetGroup) error {
|
||||||
|
targets, err := targetsFromGroup(tgroup, ts.config)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
ts.mtx.Lock()
|
||||||
|
defer ts.mtx.Unlock()
|
||||||
|
|
||||||
|
ts.tgroups[name+"/"+tgroup.Source] = targets
|
||||||
|
|
||||||
|
select {
|
||||||
|
case ts.syncCh <- struct{}{}:
|
||||||
|
default:
|
||||||
|
}
|
||||||
|
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
// scrapePool manages scrapes for sets of targets.
|
||||||
|
type scrapePool struct {
|
||||||
|
appender storage.SampleAppender
|
||||||
|
|
||||||
|
ctx context.Context
|
||||||
|
mtx sync.RWMutex
|
||||||
|
tgroups map[string]map[model.Fingerprint]*Target
|
||||||
|
}
|
||||||
|
|
||||||
|
func newScrapePool(app storage.SampleAppender) *scrapePool {
|
||||||
|
return &scrapePool{
|
||||||
|
appender: app,
|
||||||
|
tgroups: map[string]map[model.Fingerprint]*Target{},
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (sp *scrapePool) stop() {
|
||||||
var wg sync.WaitGroup
|
var wg sync.WaitGroup
|
||||||
|
|
||||||
ss.mtx.RLock()
|
sp.mtx.RLock()
|
||||||
// TODO(fabxc): the prefixing is slightly hacky but this will be gone with subsequent changes.
|
|
||||||
for source, tgroup := range ss.tgroups {
|
for _, tgroup := range sp.tgroups {
|
||||||
if !strings.HasPrefix(source, name) {
|
|
||||||
continue
|
|
||||||
}
|
|
||||||
for _, t := range tgroup {
|
for _, t := range tgroup {
|
||||||
wg.Add(1)
|
wg.Add(1)
|
||||||
|
|
||||||
|
@ -227,39 +343,37 @@ func (ss *scrapeSet) stopScrapers(name string) {
|
||||||
}(t)
|
}(t)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
ss.mtx.RUnlock()
|
sp.mtx.RUnlock()
|
||||||
|
|
||||||
wg.Wait()
|
wg.Wait()
|
||||||
}
|
}
|
||||||
|
|
||||||
// update handles a target group update from a target provider identified by the name.
|
func (sp *scrapePool) sync(tgroups map[string]map[model.Fingerprint]*Target) {
|
||||||
func (ss *scrapeSet) update(name string, tgroup *config.TargetGroup) error {
|
sp.mtx.Lock()
|
||||||
|
|
||||||
var (
|
var (
|
||||||
source = name + "/" + tgroup.Source
|
wg sync.WaitGroup
|
||||||
prevTargets = ss.tgroups[source]
|
newTgroups = map[string]map[model.Fingerprint]*Target{}
|
||||||
)
|
)
|
||||||
|
|
||||||
targets, err := targetsFromGroup(tgroup, ss.config)
|
for source, targets := range tgroups {
|
||||||
if err != nil {
|
var (
|
||||||
return err
|
prevTargets = sp.tgroups[source]
|
||||||
}
|
newTargets = map[model.Fingerprint]*Target{}
|
||||||
|
)
|
||||||
ss.mtx.Lock()
|
newTgroups[source] = newTargets
|
||||||
ss.tgroups[source] = targets
|
|
||||||
|
|
||||||
for fp, tnew := range targets {
|
for fp, tnew := range targets {
|
||||||
// If the same target existed before, we let it run and replace
|
// If the same target existed before, we let it run and replace
|
||||||
// the new one with it.
|
// the new one with it.
|
||||||
if told, ok := prevTargets[fp]; ok {
|
if told, ok := prevTargets[fp]; ok {
|
||||||
targets[fp] = told
|
newTargets[fp] = told
|
||||||
} else {
|
} else {
|
||||||
go tnew.RunScraper(ss.appender)
|
newTargets[fp] = tnew
|
||||||
|
go tnew.RunScraper(sp.appender)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
ss.mtx.Unlock()
|
for fp, told := range targets {
|
||||||
|
|
||||||
var wg sync.WaitGroup
|
|
||||||
for fp, told := range prevTargets {
|
|
||||||
// A previous target is no longer in the group.
|
// A previous target is no longer in the group.
|
||||||
if _, ok := targets[fp]; !ok {
|
if _, ok := targets[fp]; !ok {
|
||||||
wg.Add(1)
|
wg.Add(1)
|
||||||
|
@ -270,13 +384,33 @@ func (ss *scrapeSet) update(name string, tgroup *config.TargetGroup) error {
|
||||||
}(told)
|
}(told)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Stop scrapers for target groups that disappeared completely.
|
||||||
|
for source, targets := range sp.tgroups {
|
||||||
|
if _, ok := tgroups[source]; !ok {
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
for _, told := range targets {
|
||||||
|
wg.Add(1)
|
||||||
|
|
||||||
|
go func(told *Target) {
|
||||||
|
told.StopScraper()
|
||||||
|
wg.Done()
|
||||||
|
}(told)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
sp.tgroups = newTgroups
|
||||||
|
|
||||||
// Wait for all potentially stopped scrapers to terminate.
|
// Wait for all potentially stopped scrapers to terminate.
|
||||||
// This covers the case of flapping targets. If the server is under high load, a new scraper
|
// This covers the case of flapping targets. If the server is under high load, a new scraper
|
||||||
// may be active and tries to insert. The old scraper that didn't terminate yet could still
|
// may be active and tries to insert. The old scraper that didn't terminate yet could still
|
||||||
// be inserting a previous sample set.
|
// be inserting a previous sample set.
|
||||||
wg.Wait()
|
wg.Wait()
|
||||||
|
|
||||||
return nil
|
// TODO(fabxc): maybe this can be released earlier with subsequent refactoring.
|
||||||
|
sp.mtx.Unlock()
|
||||||
}
|
}
|
||||||
|
|
||||||
// providersFromConfig returns all TargetProviders configured in cfg.
|
// providersFromConfig returns all TargetProviders configured in cfg.
|
||||||
|
@ -331,6 +465,7 @@ func providersFromConfig(cfg *config.ScrapeConfig) map[string]TargetProvider {
|
||||||
// targetsFromGroup builds targets based on the given TargetGroup and config.
|
// targetsFromGroup builds targets based on the given TargetGroup and config.
|
||||||
func targetsFromGroup(tg *config.TargetGroup, cfg *config.ScrapeConfig) (map[model.Fingerprint]*Target, error) {
|
func targetsFromGroup(tg *config.TargetGroup, cfg *config.ScrapeConfig) (map[model.Fingerprint]*Target, error) {
|
||||||
targets := make(map[model.Fingerprint]*Target, len(tg.Targets))
|
targets := make(map[model.Fingerprint]*Target, len(tg.Targets))
|
||||||
|
|
||||||
for i, labels := range tg.Targets {
|
for i, labels := range tg.Targets {
|
||||||
for k, v := range cfg.Params {
|
for k, v := range cfg.Params {
|
||||||
if len(v) > 0 {
|
if len(v) > 0 {
|
||||||
|
|
Loading…
Reference in a new issue