From 5bfa4cdd46b9d2ff843f6ffe85e15fc87c614737 Mon Sep 17 00:00:00 2001 From: Fabian Reinartz Date: Wed, 17 Feb 2016 19:33:17 +0100 Subject: [PATCH 01/23] Simplify target update handling. We group providers by their scrape configuration. Each provider produces target groups with an unique identifier. On stopping a set of target providers we cancel the target providers, stop scraping the targets and wait for the scrapers to finish. On configuration reload all provider sets are stopped and new ones are created. This will make targets disappear briefly on configuration reload. Potentially scrapes are missed but due to the consistent scrape intervals implemented recently, the impact is minor. --- retrieval/scrape.go | 77 ++++++ retrieval/targetmanager.go | 484 ++++++++++++++----------------------- 2 files changed, 256 insertions(+), 305 deletions(-) create mode 100644 retrieval/scrape.go diff --git a/retrieval/scrape.go b/retrieval/scrape.go new file mode 100644 index 000000000..40d398ded --- /dev/null +++ b/retrieval/scrape.go @@ -0,0 +1,77 @@ +// 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 retrieval + +// import ( +// "sync" +// "time" + +// "github.com/prometheus/common/log" +// "github.com/prometheus/common/model" +// "golang.org/x/net/context" + +// "github.com/prometheus/prometheus/config" +// "github.com/prometheus/prometheus/storage" +// ) + +// type scraper interface { +// scrape(context.Context) error +// report(start time.Time, dur time.Duration, err error) error +// } + +// type scrapePool struct { +// mtx sync.RWMutex +// targets map[model.Fingerprint]*Target +// loops map[model.Fingerprint]loop + +// config *config.ScrapeConfig + +// newLoop func(context.Context) +// } + +// func newScrapePool(c *config.ScrapeConfig) *scrapePool { +// return &scrapePool{config: c} +// } + +// func (sp *scrapePool) sync(targets []*Target) { +// sp.mtx.Lock() +// defer sp.mtx.Unlock() + +// uniqueTargets := make(map[string]*Target{}, len(targets)) + +// for _, t := range targets { +// uniqueTargets[t.fingerprint()] = t +// } + +// sp.targets = uniqueTargets +// } + +// type scrapeLoop struct { +// scraper scraper +// mtx sync.RWMutex +// } + +// func newScrapeLoop(ctx context.Context) + +// func (sl *scrapeLoop) update() {} + +// func (sl *scrapeLoop) run(ctx context.Context) { +// var wg sync.WaitGroup + +// wg.Wait() +// } + +// func (sl *scrapeLoop) stop() { + +// } diff --git a/retrieval/targetmanager.go b/retrieval/targetmanager.go index 6f06438b4..e223cc332 100644 --- a/retrieval/targetmanager.go +++ b/retrieval/targetmanager.go @@ -20,6 +20,7 @@ import ( "github.com/prometheus/common/log" "github.com/prometheus/common/model" + "golang.org/x/net/context" "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/retrieval/discovery" @@ -33,12 +34,8 @@ import ( // The TargetProvider does not have to guarantee that an actual change happened. // It does guarantee that it sends the new TargetGroup whenever a change happens. // -// Sources() is guaranteed to be called exactly once before each call to Run(). -// On a call to Run() implementing types must send a valid target group for each of -// the sources they declared in the last call to Sources(). +// Providers must initially send all known target groups as soon as it can. type TargetProvider interface { - // Sources returns the source identifiers the provider is currently aware of. - Sources() []string // Run hands a channel to the target provider through which it can send // updated target groups. The channel must be closed by the target provider // if no more updates will be sent. @@ -50,268 +47,57 @@ type TargetProvider interface { // creates the new targets based on the target groups it receives from various // target providers. type TargetManager struct { - mtx sync.RWMutex - sampleAppender storage.SampleAppender - running bool - done chan struct{} + appender storage.SampleAppender + + mtx sync.RWMutex + ctx context.Context + cancel func() + wg sync.WaitGroup - // Targets by their source ID. - targets map[string][]*Target // Providers by the scrape configs they are derived from. - providers map[*config.ScrapeConfig][]TargetProvider + scrapeSets []*scrapeSet } // NewTargetManager creates a new TargetManager. -func NewTargetManager(sampleAppender storage.SampleAppender) *TargetManager { - tm := &TargetManager{ - sampleAppender: sampleAppender, - targets: map[string][]*Target{}, +func NewTargetManager(app storage.SampleAppender) *TargetManager { + return &TargetManager{ + appender: app, } - return tm -} - -// merge multiple target group channels into a single output channel. -func merge(done <-chan struct{}, cs ...<-chan targetGroupUpdate) <-chan targetGroupUpdate { - var wg sync.WaitGroup - out := make(chan targetGroupUpdate) - - // Start an output goroutine for each input channel in cs. output - // copies values from c to out until c or done is closed, then calls - // wg.Done. - redir := func(c <-chan targetGroupUpdate) { - defer wg.Done() - for n := range c { - select { - case out <- n: - case <-done: - return - } - } - } - - wg.Add(len(cs)) - for _, c := range cs { - go redir(c) - } - - // Close the out channel if all inbound channels are closed. - go func() { - wg.Wait() - close(out) - }() - return out -} - -// targetGroupUpdate is a potentially changed/new target group -// for the given scrape configuration. -type targetGroupUpdate struct { - tg config.TargetGroup - scfg *config.ScrapeConfig } // Run starts background processing to handle target updates. func (tm *TargetManager) Run() { log.Info("Starting target manager...") - tm.done = make(chan struct{}) - - sources := map[string]struct{}{} - updates := []<-chan targetGroupUpdate{} - - for scfg, provs := range tm.providers { - for _, prov := range provs { - // Get an initial set of available sources so we don't remove - // target groups from the last run that are still available. - for _, src := range prov.Sources() { - sources[src] = struct{}{} - } - - tgc := make(chan config.TargetGroup) - // Run the target provider after cleanup of the stale targets is done. - defer func(prov TargetProvider, tgc chan<- config.TargetGroup, done <-chan struct{}) { - go prov.Run(tgc, done) - }(prov, tgc, tm.done) - - tgupc := make(chan targetGroupUpdate) - updates = append(updates, tgupc) - - go func(scfg *config.ScrapeConfig, done <-chan struct{}) { - defer close(tgupc) - for { - select { - case tg := <-tgc: - tgupc <- targetGroupUpdate{tg: tg, scfg: scfg} - case <-done: - return - } - } - }(scfg, tm.done) - } - } - - // Merge all channels of incoming target group updates into a single - // one and keep applying the updates. - go tm.handleUpdates(merge(tm.done, updates...), tm.done) - tm.mtx.Lock() - defer tm.mtx.Unlock() + tm.ctx, tm.cancel = context.WithCancel(context.Background()) - // Remove old target groups that are no longer in the set of sources. - tm.removeTargets(func(src string) bool { - if _, ok := sources[src]; ok { - return false - } - return true - }) + for _, ss := range tm.scrapeSets { + tm.wg.Add(1) - tm.running = true - log.Info("Target manager started.") -} - -// handleUpdates receives target group updates and handles them in the -// context of the given job config. -func (tm *TargetManager) handleUpdates(ch <-chan targetGroupUpdate, done <-chan struct{}) { - for { - select { - case update, ok := <-ch: - if !ok { - return - } - log.Debugf("Received potential update for target group %q", update.tg.Source) - - if err := tm.updateTargetGroup(&update.tg, update.scfg); err != nil { - log.Errorf("Error updating targets: %s", err) - } - case <-done: - return - } + go func(ss *scrapeSet) { + ss.run(tm.ctx) + tm.wg.Done() + }(ss) } + + tm.mtx.Unlock() + tm.wg.Wait() } // Stop all background processing. func (tm *TargetManager) Stop() { - tm.mtx.RLock() - if tm.running { - defer tm.stop(true) - } - // Return the lock before calling tm.stop(). - defer tm.mtx.RUnlock() -} - -// stop background processing of the target manager. If removeTargets is true, -// existing targets will be stopped and removed. -func (tm *TargetManager) stop(removeTargets bool) { - log.Info("Stopping target manager...") - defer log.Info("Target manager stopped.") - - close(tm.done) + log.Infoln("Stopping target manager...") tm.mtx.Lock() - defer tm.mtx.Unlock() - if removeTargets { - tm.removeTargets(nil) - } + // Cancel the base context, this will cause all in-flight scrapes to abort immmediately. + // Started inserts will be finished before terminating. + tm.cancel() + tm.mtx.Unlock() - tm.running = false -} - -// removeTargets stops and removes targets for sources where f(source) is true -// or if f is nil. This method is not thread-safe. -func (tm *TargetManager) removeTargets(f func(string) bool) { - if f == nil { - f = func(string) bool { return true } - } - var wg sync.WaitGroup - for src, targets := range tm.targets { - if !f(src) { - continue - } - wg.Add(len(targets)) - for _, target := range targets { - go func(t *Target) { - t.StopScraper() - wg.Done() - }(target) - } - delete(tm.targets, src) - } - wg.Wait() -} - -// updateTargetGroup creates new targets for the group and replaces the old targets -// for the source ID. -func (tm *TargetManager) updateTargetGroup(tgroup *config.TargetGroup, cfg *config.ScrapeConfig) error { - newTargets, err := tm.targetsFromGroup(tgroup, cfg) - if err != nil { - return err - } - - tm.mtx.Lock() - defer tm.mtx.Unlock() - - if !tm.running { - return nil - } - - oldTargets, ok := tm.targets[tgroup.Source] - if ok { - var wg sync.WaitGroup - // Replace the old targets with the new ones while keeping the state - // of intersecting targets. - for i, tnew := range newTargets { - var match *Target - for j, told := range oldTargets { - if told == nil { - continue - } - if tnew.fingerprint() == told.fingerprint() { - match = told - oldTargets[j] = nil - break - } - } - // Update the existing target and discard the new equivalent. - // Otherwise start scraping the new target. - if match != nil { - // Updating is blocked during a scrape. We don't want those wait times - // to build up. - wg.Add(1) - go func(t *Target) { - if err := match.Update(cfg, t.labels, t.metaLabels); err != nil { - log.Errorf("Error updating target %v: %v", t, err) - } - wg.Done() - }(tnew) - newTargets[i] = match - } else { - go tnew.RunScraper(tm.sampleAppender) - } - } - // Remove all old targets that disappeared. - for _, told := range oldTargets { - if told != nil { - wg.Add(1) - go func(t *Target) { - t.StopScraper() - wg.Done() - }(told) - } - } - wg.Wait() - } else { - // The source ID is new, start all target scrapers. - for _, tnew := range newTargets { - go tnew.RunScraper(tm.sampleAppender) - } - } - - if len(newTargets) > 0 { - tm.targets[tgroup.Source] = newTargets - } else { - delete(tm.targets, tgroup.Source) - } - return nil + // Wait for all provider sets to terminate. + tm.wg.Wait() } // Pools returns the targets currently being scraped bucketed by their job name. @@ -321,10 +107,13 @@ func (tm *TargetManager) Pools() map[string][]*Target { pools := map[string][]*Target{} - for _, ts := range tm.targets { - for _, t := range ts { - job := string(t.Labels()[model.JobLabel]) - pools[job] = append(pools[job], t) + // TODO(fabxc): this is just a hack to maintain compatibility for now. + for _, ps := range tm.scrapeSets { + for _, ts := range ps.tgroups { + for _, t := range ts { + job := string(t.Labels()[model.JobLabel]) + pools[job] = append(pools[job], t) + } } } return pools @@ -335,78 +124,173 @@ func (tm *TargetManager) Pools() map[string][]*Target { // Returns true on success. func (tm *TargetManager) ApplyConfig(cfg *config.Config) bool { tm.mtx.RLock() - running := tm.running + running := tm.ctx != nil tm.mtx.RUnlock() if running { - tm.stop(false) - // Even if updating the config failed, we want to continue rather than stop scraping anything. - defer tm.Run() - } - providers := map[*config.ScrapeConfig][]TargetProvider{} - - for _, scfg := range cfg.ScrapeConfigs { - providers[scfg] = providersFromConfig(scfg) + tm.Stop() + defer func() { + go tm.Run() + }() } tm.mtx.Lock() - defer tm.mtx.Unlock() - tm.providers = providers + tm.scrapeSets = tm.scrapeSets[:0] + + for _, scfg := range cfg.ScrapeConfigs { + tm.scrapeSets = append(tm.scrapeSets, newScrapeSet(tm.appender, scfg)) + } + + tm.mtx.Unlock() + return true } -// prefixedTargetProvider wraps TargetProvider and prefixes source strings -// to make the sources unique across a configuration. -type prefixedTargetProvider struct { - TargetProvider +// scrapeSet holds several TargetProviders for which the same scrape configuration +// is used. It runs the target providers and starts and stops scrapers as it +// receives target updates. +type scrapeSet struct { + appender storage.SampleAppender - job string - mechanism string - idx int + config *config.ScrapeConfig + tgroups map[string]map[model.Fingerprint]*Target + + mtx sync.RWMutex } -func (tp *prefixedTargetProvider) prefix(src string) string { - return fmt.Sprintf("%s:%s:%d:%s", tp.job, tp.mechanism, tp.idx, src) +func newScrapeSet(app storage.SampleAppender, cfg *config.ScrapeConfig) *scrapeSet { + return &scrapeSet{ + appender: app, + config: cfg, + tgroups: map[string]map[model.Fingerprint]*Target{}, + } } -func (tp *prefixedTargetProvider) Sources() []string { - srcs := tp.TargetProvider.Sources() - for i, src := range srcs { - srcs[i] = tp.prefix(src) +// run starts the target providers with the given context and consumes +// and handles their updates. If the context is done, it blocks until the +// target scrapers have terminated. +func (ss *scrapeSet) run(ctx context.Context) { + var ( + providers = providersFromConfig(ss.config) + wg sync.WaitGroup + ) + + for name, prov := range providers { + var ( + updates = make(chan config.TargetGroup) + ) + + 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() + + for { + select { + case <-ctx.Done(): + ss.stopScrapers(name) + return + case update := <-updates: + if err := ss.update(name, &update); err != nil { + log.With("target_group", update).Errorf("Target update failed: %s", err) + } + } + } + }(name, prov) + + done := make(chan struct{}) + + // TODO(fabxc): Adjust the TargetProvider interface so we can remove this + // redirection of the termination signal. + go func() { + <-ctx.Done() + close(done) + }() + go prov.Run(updates, done) } - return srcs + wg.Wait() } -func (tp *prefixedTargetProvider) Run(ch chan<- config.TargetGroup, done <-chan struct{}) { - defer close(ch) +// stopScrapers shuts down all active scrapers for a provider. +func (ss *scrapeSet) stopScrapers(name string) { + var wg sync.WaitGroup - ch2 := make(chan config.TargetGroup) - go tp.TargetProvider.Run(ch2, done) + ss.mtx.RLock() + // TODO(fabxc): the prefixing is slightly hacky but this will be gone with subsequent changes. + for source, tgroup := range ss.tgroups { + if !strings.HasPrefix(source, name) { + continue + } + for _, t := range tgroup { + wg.Add(1) - for { - select { - case <-done: - return - case tg := <-ch2: - tg.Source = tp.prefix(tg.Source) - ch <- tg + go func(t *Target) { + t.StopScraper() + wg.Done() + }(t) } } + ss.mtx.RUnlock() + + wg.Wait() +} + +// update handles a target group update from a target provider identified by the name. +func (ss *scrapeSet) update(name string, tgroup *config.TargetGroup) error { + var ( + source = name + "/" + tgroup.Source + prevTargets = ss.tgroups[source] + ) + + targets, err := targetsFromGroup(tgroup, ss.config) + if err != nil { + return err + } + + ss.mtx.Lock() + ss.tgroups[source] = targets + + for fp, tnew := range targets { + // If the same target existed before, we let it run and replace + // the new one with it. + if told, ok := prevTargets[fp]; ok { + targets[fp] = told + } else { + go tnew.RunScraper(ss.appender) + } + } + ss.mtx.Unlock() + + var wg sync.WaitGroup + for fp, told := range prevTargets { + // A previous target is no longer in the group. + if _, ok := targets[fp]; !ok { + wg.Add(1) + + go func(told *Target) { + told.StopScraper() + wg.Done() + }(told) + } + } + // 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 + // may be active and tries to insert. The old scraper that didn't terminate yet could still + // be inserting a previous sample set. + wg.Wait() + + return nil } // providersFromConfig returns all TargetProviders configured in cfg. -func providersFromConfig(cfg *config.ScrapeConfig) []TargetProvider { - var providers []TargetProvider +func providersFromConfig(cfg *config.ScrapeConfig) map[string]TargetProvider { + providers := map[string]TargetProvider{} app := func(mech string, i int, tp TargetProvider) { - providers = append(providers, &prefixedTargetProvider{ - job: cfg.JobName, - mechanism: mech, - idx: i, - TargetProvider: tp, - }) + providers[fmt.Sprintf("%s/%d", mech, i)] = tp } for i, c := range cfg.DNSSDConfigs { @@ -451,11 +335,8 @@ func providersFromConfig(cfg *config.ScrapeConfig) []TargetProvider { } // targetsFromGroup builds targets based on the given TargetGroup and config. -func (tm *TargetManager) targetsFromGroup(tg *config.TargetGroup, cfg *config.ScrapeConfig) ([]*Target, error) { - tm.mtx.RLock() - defer tm.mtx.RUnlock() - - targets := make([]*Target, 0, len(tg.Targets)) +func targetsFromGroup(tg *config.TargetGroup, cfg *config.ScrapeConfig) (map[model.Fingerprint]*Target, error) { + targets := make(map[model.Fingerprint]*Target, len(tg.Targets)) for i, labels := range tg.Targets { for k, v := range cfg.Params { if len(v) > 0 { @@ -522,7 +403,8 @@ func (tm *TargetManager) targetsFromGroup(tg *config.TargetGroup, cfg *config.Sc if err != nil { return nil, fmt.Errorf("error while creating instance %d in target group %s: %s", i, tg, err) } - targets = append(targets, tr) + + targets[tr.fingerprint()] = tr } return targets, nil @@ -557,11 +439,3 @@ func (sd *StaticProvider) Run(ch chan<- config.TargetGroup, done <-chan struct{} } <-done } - -// Sources returns the provider's sources. -func (sd *StaticProvider) Sources() (srcs []string) { - for _, tg := range sd.TargetGroups { - srcs = append(srcs, tg.Source) - } - return srcs -} From bb6dc3ff783463e0f9fdc915232b546cc0a116b3 Mon Sep 17 00:00:00 2001 From: Fabian Reinartz Date: Thu, 18 Feb 2016 12:32:36 +0100 Subject: [PATCH 02/23] Remove old tests --- retrieval/targetmanager_test.go | 489 -------------------------------- 1 file changed, 489 deletions(-) diff --git a/retrieval/targetmanager_test.go b/retrieval/targetmanager_test.go index 2e7d95db2..074fe51ec 100644 --- a/retrieval/targetmanager_test.go +++ b/retrieval/targetmanager_test.go @@ -12,492 +12,3 @@ // limitations under the License. package retrieval - -import ( - "net/url" - "reflect" - "testing" - "time" - - "github.com/prometheus/common/model" - - "github.com/prometheus/prometheus/config" -) - -func TestPrefixedTargetProvider(t *testing.T) { - targetGroups := []*config.TargetGroup{ - { - Targets: []model.LabelSet{ - {model.AddressLabel: "test-1:1234"}, - }, - }, { - Targets: []model.LabelSet{ - {model.AddressLabel: "test-1:1235"}, - }, - }, - } - - tp := &prefixedTargetProvider{ - job: "job-x", - mechanism: "static", - idx: 123, - TargetProvider: NewStaticProvider(targetGroups), - } - - expSources := []string{ - "job-x:static:123:0", - "job-x:static:123:1", - } - if !reflect.DeepEqual(tp.Sources(), expSources) { - t.Fatalf("expected sources %v, got %v", expSources, tp.Sources()) - } - - ch := make(chan config.TargetGroup) - done := make(chan struct{}) - - defer close(done) - go tp.Run(ch, done) - - expGroup1 := *targetGroups[0] - expGroup2 := *targetGroups[1] - expGroup1.Source = "job-x:static:123:0" - expGroup2.Source = "job-x:static:123:1" - - // The static target provider sends on the channel once per target group. - if tg := <-ch; !reflect.DeepEqual(tg, expGroup1) { - t.Fatalf("expected target group %v, got %v", expGroup1, tg) - } - if tg := <-ch; !reflect.DeepEqual(tg, expGroup2) { - t.Fatalf("expected target group %v, got %v", expGroup2, tg) - } -} - -func TestTargetManagerChan(t *testing.T) { - testJob1 := &config.ScrapeConfig{ - JobName: "test_job1", - ScrapeInterval: model.Duration(1 * time.Minute), - TargetGroups: []*config.TargetGroup{{ - Targets: []model.LabelSet{ - {model.AddressLabel: "example.org:80"}, - {model.AddressLabel: "example.com:80"}, - }, - }}, - } - prov1 := &fakeTargetProvider{ - sources: []string{"src1", "src2"}, - update: make(chan *config.TargetGroup), - } - - targetManager := &TargetManager{ - sampleAppender: nopAppender{}, - providers: map[*config.ScrapeConfig][]TargetProvider{ - testJob1: {prov1}, - }, - targets: make(map[string][]*Target), - } - go targetManager.Run() - defer targetManager.Stop() - - sequence := []struct { - tgroup *config.TargetGroup - expected map[string][]model.LabelSet - }{ - { - tgroup: &config.TargetGroup{ - Source: "src1", - Targets: []model.LabelSet{ - {model.AddressLabel: "test-1:1234"}, - {model.AddressLabel: "test-2:1234", "label": "set"}, - {model.AddressLabel: "test-3:1234"}, - }, - }, - expected: map[string][]model.LabelSet{ - "src1": { - {model.JobLabel: "test_job1", model.InstanceLabel: "test-1:1234"}, - {model.JobLabel: "test_job1", model.InstanceLabel: "test-2:1234", "label": "set"}, - {model.JobLabel: "test_job1", model.InstanceLabel: "test-3:1234"}, - }, - }, - }, { - tgroup: &config.TargetGroup{ - Source: "src2", - Targets: []model.LabelSet{ - {model.AddressLabel: "test-1:1235"}, - {model.AddressLabel: "test-2:1235"}, - {model.AddressLabel: "test-3:1235"}, - }, - Labels: model.LabelSet{"group": "label"}, - }, - expected: map[string][]model.LabelSet{ - "src1": { - {model.JobLabel: "test_job1", model.InstanceLabel: "test-1:1234"}, - {model.JobLabel: "test_job1", model.InstanceLabel: "test-2:1234", "label": "set"}, - {model.JobLabel: "test_job1", model.InstanceLabel: "test-3:1234"}, - }, - "src2": { - {model.JobLabel: "test_job1", model.InstanceLabel: "test-1:1235", "group": "label"}, - {model.JobLabel: "test_job1", model.InstanceLabel: "test-2:1235", "group": "label"}, - {model.JobLabel: "test_job1", model.InstanceLabel: "test-3:1235", "group": "label"}, - }, - }, - }, { - tgroup: &config.TargetGroup{ - Source: "src2", - Targets: []model.LabelSet{}, - }, - expected: map[string][]model.LabelSet{ - "src1": { - {model.JobLabel: "test_job1", model.InstanceLabel: "test-1:1234"}, - {model.JobLabel: "test_job1", model.InstanceLabel: "test-2:1234", "label": "set"}, - {model.JobLabel: "test_job1", model.InstanceLabel: "test-3:1234"}, - }, - }, - }, { - tgroup: &config.TargetGroup{ - Source: "src1", - Targets: []model.LabelSet{ - {model.AddressLabel: "test-1:1234", "added": "label"}, - {model.AddressLabel: "test-3:1234"}, - {model.AddressLabel: "test-4:1234", "fancy": "label"}, - }, - }, - expected: map[string][]model.LabelSet{ - "src1": { - {model.JobLabel: "test_job1", model.InstanceLabel: "test-1:1234", "added": "label"}, - {model.JobLabel: "test_job1", model.InstanceLabel: "test-3:1234"}, - {model.JobLabel: "test_job1", model.InstanceLabel: "test-4:1234", "fancy": "label"}, - }, - }, - }, - } - - for i, step := range sequence { - prov1.update <- step.tgroup - - time.Sleep(20 * time.Millisecond) - - if len(targetManager.targets) != len(step.expected) { - t.Fatalf("step %d: sources mismatch %v, %v", i, targetManager.targets, step.expected) - } - - for source, actTargets := range targetManager.targets { - expTargets, ok := step.expected[source] - if !ok { - t.Fatalf("step %d: unexpected source %q: %v", i, source, actTargets) - } - for _, expt := range expTargets { - found := false - for _, actt := range actTargets { - if reflect.DeepEqual(expt, actt.Labels()) { - found = true - break - } - } - if !found { - t.Errorf("step %d: expected target %v not found in actual targets", i, expt) - } - } - } - } -} - -func TestTargetManagerConfigUpdate(t *testing.T) { - testJob1 := &config.ScrapeConfig{ - JobName: "test_job1", - ScrapeInterval: model.Duration(1 * time.Minute), - Params: url.Values{ - "testParam": []string{"paramValue", "secondValue"}, - }, - TargetGroups: []*config.TargetGroup{{ - Targets: []model.LabelSet{ - {model.AddressLabel: "example.org:80"}, - {model.AddressLabel: "example.com"}, - }, - }}, - RelabelConfigs: []*config.RelabelConfig{ - { - // Copy out the URL parameter. - SourceLabels: model.LabelNames{"__param_testParam"}, - Regex: config.MustNewRegexp("(.*)"), - TargetLabel: "testParam", - Replacement: "$1", - Action: config.RelabelReplace, - }, - { - // The port number is added after relabeling, so - // this relabel rule should have no effect. - SourceLabels: model.LabelNames{model.AddressLabel}, - Regex: config.MustNewRegexp("example.com:80"), - Action: config.RelabelDrop, - }, - }, - } - testJob2 := &config.ScrapeConfig{ - JobName: "test_job2", - ScrapeInterval: model.Duration(1 * time.Minute), - TargetGroups: []*config.TargetGroup{ - { - Targets: []model.LabelSet{ - {model.AddressLabel: "example.org:8080"}, - {model.AddressLabel: "example.com:8081"}, - }, - Labels: model.LabelSet{ - "foo": "bar", - "boom": "box", - }, - }, - { - Targets: []model.LabelSet{ - {model.AddressLabel: "test.com:1234"}, - }, - }, - { - Targets: []model.LabelSet{ - {model.AddressLabel: "test.com:1235"}, - }, - Labels: model.LabelSet{"instance": "fixed"}, - }, - }, - RelabelConfigs: []*config.RelabelConfig{ - { - SourceLabels: model.LabelNames{model.AddressLabel}, - Regex: config.MustNewRegexp(`test\.(.*?):(.*)`), - Replacement: "foo.${1}:${2}", - TargetLabel: model.AddressLabel, - Action: config.RelabelReplace, - }, - { - // Add a new label for example.* targets. - SourceLabels: model.LabelNames{model.AddressLabel, "boom", "foo"}, - Regex: config.MustNewRegexp("example.*?-b([a-z-]+)r"), - TargetLabel: "new", - Replacement: "$1", - Separator: "-", - Action: config.RelabelReplace, - }, - { - // Drop an existing label. - SourceLabels: model.LabelNames{"boom"}, - Regex: config.MustNewRegexp(".*"), - TargetLabel: "boom", - Replacement: "", - Action: config.RelabelReplace, - }, - }, - } - // Test that targets without host:port addresses are dropped. - testJob3 := &config.ScrapeConfig{ - JobName: "test_job1", - ScrapeInterval: model.Duration(1 * time.Minute), - TargetGroups: []*config.TargetGroup{{ - Targets: []model.LabelSet{ - {model.AddressLabel: "example.net:80"}, - }, - }}, - RelabelConfigs: []*config.RelabelConfig{ - { - SourceLabels: model.LabelNames{model.AddressLabel}, - Regex: config.MustNewRegexp("(.*)"), - TargetLabel: "__address__", - Replacement: "http://$1", - Action: config.RelabelReplace, - }, - }, - } - - sequence := []struct { - scrapeConfigs []*config.ScrapeConfig - expected map[string][]model.LabelSet - }{ - { - scrapeConfigs: []*config.ScrapeConfig{testJob1}, - expected: map[string][]model.LabelSet{ - "test_job1:static:0:0": { - { - model.JobLabel: "test_job1", - "testParam": "paramValue", - model.SchemeLabel: "", - model.MetricsPathLabel: "", - model.AddressLabel: "example.org:80", - model.ParamLabelPrefix + "testParam": "paramValue", - }, - { - model.JobLabel: "test_job1", - "testParam": "paramValue", - model.SchemeLabel: "", - model.MetricsPathLabel: "", - model.AddressLabel: "example.com:80", - model.ParamLabelPrefix + "testParam": "paramValue"}, - }, - }, - }, { - scrapeConfigs: []*config.ScrapeConfig{testJob1}, - expected: map[string][]model.LabelSet{ - "test_job1:static:0:0": { - { - model.JobLabel: "test_job1", - "testParam": "paramValue", - model.SchemeLabel: "", - model.MetricsPathLabel: "", - model.AddressLabel: "example.org:80", - model.ParamLabelPrefix + "testParam": "paramValue", - }, - { - model.JobLabel: "test_job1", - "testParam": "paramValue", - model.SchemeLabel: "", - model.MetricsPathLabel: "", - model.AddressLabel: "example.com:80", - model.ParamLabelPrefix + "testParam": "paramValue", - }, - }, - }, - }, { - scrapeConfigs: []*config.ScrapeConfig{testJob1, testJob2}, - expected: map[string][]model.LabelSet{ - "test_job1:static:0:0": { - { - model.JobLabel: "test_job1", - "testParam": "paramValue", - model.SchemeLabel: "", - model.MetricsPathLabel: "", - model.AddressLabel: "example.org:80", - model.ParamLabelPrefix + "testParam": "paramValue", - }, - { - model.JobLabel: "test_job1", - "testParam": "paramValue", - model.SchemeLabel: "", - model.MetricsPathLabel: "", - model.AddressLabel: "example.com:80", - model.ParamLabelPrefix + "testParam": "paramValue", - }, - }, - "test_job2:static:0:0": { - { - model.JobLabel: "test_job2", - "foo": "bar", - "new": "ox-ba", - model.SchemeLabel: "", - model.MetricsPathLabel: "", - model.AddressLabel: "example.org:8080", - }, - { - model.JobLabel: "test_job2", - "foo": "bar", - "new": "ox-ba", - model.SchemeLabel: "", - model.MetricsPathLabel: "", - model.AddressLabel: "example.com:8081", - }, - }, - "test_job2:static:0:1": { - { - model.JobLabel: "test_job2", - model.SchemeLabel: "", - model.MetricsPathLabel: "", - model.AddressLabel: "foo.com:1234", - }, - }, - "test_job2:static:0:2": { - { - model.JobLabel: "test_job2", - model.InstanceLabel: "fixed", - model.SchemeLabel: "", - model.MetricsPathLabel: "", - model.AddressLabel: "foo.com:1235", - }, - }, - }, - }, { - scrapeConfigs: []*config.ScrapeConfig{}, - expected: map[string][]model.LabelSet{}, - }, { - scrapeConfigs: []*config.ScrapeConfig{testJob2}, - expected: map[string][]model.LabelSet{ - "test_job2:static:0:0": { - { - model.JobLabel: "test_job2", - "foo": "bar", - "new": "ox-ba", - model.SchemeLabel: "", - model.MetricsPathLabel: "", - model.AddressLabel: "example.org:8080"}, - { - model.JobLabel: "test_job2", - "foo": "bar", - "new": "ox-ba", - model.SchemeLabel: "", - model.MetricsPathLabel: "", - model.AddressLabel: "example.com:8081", - }, - }, - "test_job2:static:0:1": { - { - model.JobLabel: "test_job2", - model.SchemeLabel: "", - model.MetricsPathLabel: "", - model.AddressLabel: "foo.com:1234", - }, - }, - "test_job2:static:0:2": { - { - model.JobLabel: "test_job2", - model.InstanceLabel: "fixed", - model.SchemeLabel: "", - model.MetricsPathLabel: "", - model.AddressLabel: "foo.com:1235", - }, - }, - }, - }, { - scrapeConfigs: []*config.ScrapeConfig{testJob3}, - expected: map[string][]model.LabelSet{}, - }, - } - conf := &config.Config{} - *conf = config.DefaultConfig - - targetManager := NewTargetManager(nopAppender{}) - targetManager.ApplyConfig(conf) - - targetManager.Run() - defer targetManager.Stop() - - for i, step := range sequence { - conf.ScrapeConfigs = step.scrapeConfigs - targetManager.ApplyConfig(conf) - - time.Sleep(50 * time.Millisecond) - - if len(targetManager.targets) != len(step.expected) { - t.Fatalf("step %d: sources mismatch: expected %v, got %v", i, step.expected, targetManager.targets) - } - - for source, actTargets := range targetManager.targets { - expTargets, ok := step.expected[source] - if !ok { - t.Fatalf("step %d: unexpected source %q: %v", i, source, actTargets) - } - for _, expt := range expTargets { - found := false - for _, actt := range actTargets { - if reflect.DeepEqual(expt, actt.labels) { - found = true - break - } - } - if !found { - t.Errorf("step %d: expected target %v for %q not found in actual targets", i, expt, source) - } - } - } - } -} - -func TestHandleUpdatesReturnsWhenUpdateChanIsClosed(t *testing.T) { - tm := NewTargetManager(nopAppender{}) - ch := make(chan targetGroupUpdate) - close(ch) - tm.handleUpdates(ch, make(chan struct{})) -} From 5b30bdb610a8fe7019edbf23bfc10412309ca5b8 Mon Sep 17 00:00:00 2001 From: Fabian Reinartz Date: Thu, 18 Feb 2016 17:26:27 +0100 Subject: [PATCH 03/23] Change TargetProvider interface. This commit changes the TargetProvider interface to use a context.Context and send lists of TargetGroups, rather than single ones. --- retrieval/discovery/consul.go | 46 +++--------- retrieval/discovery/dns.go | 23 +++--- retrieval/discovery/ec2.go | 16 ++--- retrieval/discovery/file.go | 35 +++------- retrieval/discovery/file_test.go | 37 +++++----- retrieval/discovery/kubernetes/discovery.go | 77 +++++---------------- retrieval/discovery/marathon.go | 27 +++----- retrieval/discovery/marathon_test.go | 38 +++++----- retrieval/discovery/nerve.go | 29 ++++---- retrieval/discovery/serverset.go | 33 ++++----- retrieval/targetmanager.go | 43 +++++------- 11 files changed, 141 insertions(+), 263 deletions(-) diff --git a/retrieval/discovery/consul.go b/retrieval/discovery/consul.go index 84ed6d7ac..512008170 100644 --- a/retrieval/discovery/consul.go +++ b/retrieval/discovery/consul.go @@ -15,7 +15,6 @@ package discovery import ( "fmt" - "net/http" "strconv" "strings" "sync" @@ -24,6 +23,7 @@ import ( consul "github.com/hashicorp/consul/api" "github.com/prometheus/common/log" "github.com/prometheus/common/model" + "golang.org/x/net/context" "github.com/prometheus/prometheus/config" ) @@ -113,52 +113,24 @@ func NewConsulDiscovery(conf *config.ConsulSDConfig) (*ConsulDiscovery, error) { return cd, nil } -// Sources implements the TargetProvider interface. -func (cd *ConsulDiscovery) Sources() []string { - clientConf := *cd.clientConf - clientConf.HttpClient = &http.Client{Timeout: 5 * time.Second} - - client, err := consul.NewClient(&clientConf) - if err != nil { - // NewClient always returns a nil error. - panic(fmt.Errorf("discovery.ConsulDiscovery.Sources: %s", err)) - } - - srvs, _, err := client.Catalog().Services(nil) - if err != nil { - log.Errorf("Error refreshing service list: %s", err) - return nil - } - cd.mu.Lock() - defer cd.mu.Unlock() - - srcs := make([]string, 0, len(srvs)) - for name := range srvs { - if _, ok := cd.scrapedServices[name]; len(cd.scrapedServices) == 0 || ok { - srcs = append(srcs, name) - } - } - return srcs -} - // Run implements the TargetProvider interface. -func (cd *ConsulDiscovery) Run(ch chan<- config.TargetGroup, done <-chan struct{}) { +func (cd *ConsulDiscovery) Run(ctx context.Context, ch chan<- []*config.TargetGroup) { defer close(ch) defer cd.stop() update := make(chan *consulService, 10) - go cd.watchServices(update, done) + go cd.watchServices(update, ctx.Done()) for { select { - case <-done: + case <-ctx.Done(): return case srv := <-update: if srv.removed { close(srv.done) // Send clearing update. - ch <- config.TargetGroup{Source: srv.name} + ch <- []*config.TargetGroup{{Source: srv.name}} break } // Launch watcher for the service. @@ -244,7 +216,7 @@ func (cd *ConsulDiscovery) watchServices(update chan<- *consulService, done <-ch // watchService retrieves updates about srv from Consul's service endpoint. // On a potential update the resulting target group is sent to ch. -func (cd *ConsulDiscovery) watchService(srv *consulService, ch chan<- config.TargetGroup) { +func (cd *ConsulDiscovery) watchService(srv *consulService, ch chan<- []*config.TargetGroup) { catalog := cd.client.Catalog() for { nodes, meta, err := catalog.Service(srv.name, "", &consul.QueryOptions{ @@ -288,7 +260,11 @@ func (cd *ConsulDiscovery) watchService(srv *consulService, ch chan<- config.Tar default: // Continue. } - ch <- srv.tgroup + // TODO(fabxc): do a copy for now to avoid races. The integration + // needs needs some general cleanup. + tg := srv.tgroup + ch <- []*config.TargetGroup{&tg} + cd.mu.Unlock() } } diff --git a/retrieval/discovery/dns.go b/retrieval/discovery/dns.go index 40a8841e1..3c75f07fc 100644 --- a/retrieval/discovery/dns.go +++ b/retrieval/discovery/dns.go @@ -24,6 +24,7 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/common/log" "github.com/prometheus/common/model" + "golang.org/x/net/context" "github.com/prometheus/prometheus/config" ) @@ -91,7 +92,7 @@ func NewDNSDiscovery(conf *config.DNSSDConfig) *DNSDiscovery { } // Run implements the TargetProvider interface. -func (dd *DNSDiscovery) Run(ch chan<- config.TargetGroup, done <-chan struct{}) { +func (dd *DNSDiscovery) Run(ctx context.Context, ch chan<- []*config.TargetGroup) { defer close(ch) ticker := time.NewTicker(dd.interval) @@ -104,22 +105,13 @@ func (dd *DNSDiscovery) Run(ch chan<- config.TargetGroup, done <-chan struct{}) select { case <-ticker.C: dd.refreshAll(ch) - case <-done: + case <-ctx.Done(): return } } } -// Sources implements the TargetProvider interface. -func (dd *DNSDiscovery) Sources() []string { - var srcs []string - for _, name := range dd.names { - srcs = append(srcs, name) - } - return srcs -} - -func (dd *DNSDiscovery) refreshAll(ch chan<- config.TargetGroup) { +func (dd *DNSDiscovery) refreshAll(ch chan<- []*config.TargetGroup) { var wg sync.WaitGroup wg.Add(len(dd.names)) for _, name := range dd.names { @@ -133,7 +125,7 @@ func (dd *DNSDiscovery) refreshAll(ch chan<- config.TargetGroup) { wg.Wait() } -func (dd *DNSDiscovery) refresh(name string, ch chan<- config.TargetGroup) error { +func (dd *DNSDiscovery) refresh(name string, ch chan<- []*config.TargetGroup) error { response, err := lookupAll(name, dd.qtype) dnsSDLookupsCount.Inc() if err != nil { @@ -141,7 +133,8 @@ func (dd *DNSDiscovery) refresh(name string, ch chan<- config.TargetGroup) error return err } - var tg config.TargetGroup + tg := &config.TargetGroup{} + for _, record := range response.Answer { target := model.LabelValue("") switch addr := record.(type) { @@ -166,7 +159,7 @@ func (dd *DNSDiscovery) refresh(name string, ch chan<- config.TargetGroup) error } tg.Source = name - ch <- tg + ch <- []*config.TargetGroup{tg} return nil } diff --git a/retrieval/discovery/ec2.go b/retrieval/discovery/ec2.go index 46b3d371f..e39016948 100644 --- a/retrieval/discovery/ec2.go +++ b/retrieval/discovery/ec2.go @@ -23,6 +23,7 @@ import ( "github.com/aws/aws-sdk-go/aws/defaults" "github.com/prometheus/common/log" "github.com/prometheus/common/model" + "golang.org/x/net/context" "github.com/aws/aws-sdk-go/service/ec2" "github.com/prometheus/prometheus/config" @@ -46,7 +47,6 @@ const ( // the TargetProvider interface. type EC2Discovery struct { aws *aws.Config - done chan struct{} interval time.Duration port int } @@ -62,14 +62,13 @@ func NewEC2Discovery(conf *config.EC2SDConfig) *EC2Discovery { Region: &conf.Region, Credentials: creds, }, - done: make(chan struct{}), interval: time.Duration(conf.RefreshInterval), port: conf.Port, } } // Run implements the TargetProvider interface. -func (ed *EC2Discovery) Run(ch chan<- config.TargetGroup, done <-chan struct{}) { +func (ed *EC2Discovery) Run(ctx context.Context, ch chan<- []*config.TargetGroup) { defer close(ch) ticker := time.NewTicker(ed.interval) @@ -80,7 +79,7 @@ func (ed *EC2Discovery) Run(ch chan<- config.TargetGroup, done <-chan struct{}) if err != nil { log.Error(err) } else { - ch <- *tg + ch <- []*config.TargetGroup{tg} } for { @@ -90,19 +89,14 @@ func (ed *EC2Discovery) Run(ch chan<- config.TargetGroup, done <-chan struct{}) if err != nil { log.Error(err) } else { - ch <- *tg + ch <- []*config.TargetGroup{tg} } - case <-done: + case <-ctx.Done(): return } } } -// Sources implements the TargetProvider interface. -func (ed *EC2Discovery) Sources() []string { - return []string{*ed.aws.Region} -} - func (ed *EC2Discovery) refresh() (*config.TargetGroup, error) { ec2s := ec2.New(ed.aws) tg := &config.TargetGroup{ diff --git a/retrieval/discovery/file.go b/retrieval/discovery/file.go index eb0411b30..9aedb1605 100644 --- a/retrieval/discovery/file.go +++ b/retrieval/discovery/file.go @@ -23,6 +23,7 @@ import ( "github.com/prometheus/common/log" "github.com/prometheus/common/model" + "golang.org/x/net/context" "gopkg.in/fsnotify.v1" "gopkg.in/yaml.v2" @@ -53,23 +54,6 @@ func NewFileDiscovery(conf *config.FileSDConfig) *FileDiscovery { } } -// Sources implements the TargetProvider interface. -func (fd *FileDiscovery) Sources() []string { - var srcs []string - // As we allow multiple target groups per file we have no choice - // but to parse them all. - for _, p := range fd.listFiles() { - tgroups, err := readFile(p) - if err != nil { - log.Errorf("Error reading file %q: %s", p, err) - } - for _, tg := range tgroups { - srcs = append(srcs, tg.Source) - } - } - return srcs -} - // listFiles returns a list of all files that match the configured patterns. func (fd *FileDiscovery) listFiles() []string { var paths []string @@ -103,7 +87,7 @@ func (fd *FileDiscovery) watchFiles() { } // Run implements the TargetProvider interface. -func (fd *FileDiscovery) Run(ch chan<- config.TargetGroup, done <-chan struct{}) { +func (fd *FileDiscovery) Run(ctx context.Context, ch chan<- []*config.TargetGroup) { defer close(ch) defer fd.stop() @@ -123,11 +107,11 @@ func (fd *FileDiscovery) Run(ch chan<- config.TargetGroup, done <-chan struct{}) // Stopping has priority over refreshing. Thus we wrap the actual select // clause to always catch done signals. select { - case <-done: + case <-ctx.Done(): return default: select { - case <-done: + case <-ctx.Done(): return case event := <-fd.watcher.Events: @@ -188,7 +172,7 @@ func (fd *FileDiscovery) stop() { // refresh reads all files matching the discovery's patterns and sends the respective // updated target groups through the channel. -func (fd *FileDiscovery) refresh(ch chan<- config.TargetGroup) { +func (fd *FileDiscovery) refresh(ch chan<- []*config.TargetGroup) { ref := map[string]int{} for _, p := range fd.listFiles() { tgroups, err := readFile(p) @@ -198,9 +182,8 @@ func (fd *FileDiscovery) refresh(ch chan<- config.TargetGroup) { ref[p] = fd.lastRefresh[p] continue } - for _, tg := range tgroups { - ch <- *tg - } + ch <- tgroups + ref[p] = len(tgroups) } // Send empty updates for sources that disappeared. @@ -208,7 +191,9 @@ func (fd *FileDiscovery) refresh(ch chan<- config.TargetGroup) { m, ok := ref[f] if !ok || n > m { for i := m; i < n; i++ { - ch <- config.TargetGroup{Source: fileSource(f, i)} + ch <- []*config.TargetGroup{ + {Source: fileSource(f, i)}, + } } } } diff --git a/retrieval/discovery/file_test.go b/retrieval/discovery/file_test.go index 4c1407666..2e1babce1 100644 --- a/retrieval/discovery/file_test.go +++ b/retrieval/discovery/file_test.go @@ -8,6 +8,7 @@ import ( "time" "github.com/prometheus/common/model" + "golang.org/x/net/context" "github.com/prometheus/prometheus/config" ) @@ -27,17 +28,17 @@ func testFileSD(t *testing.T, ext string) { conf.RefreshInterval = model.Duration(1 * time.Hour) var ( - fsd = NewFileDiscovery(&conf) - ch = make(chan config.TargetGroup) - done = make(chan struct{}) + fsd = NewFileDiscovery(&conf) + ch = make(chan []*config.TargetGroup) + ctx, cancel = context.WithCancel(context.Background()) ) - go fsd.Run(ch, done) + go fsd.Run(ctx, ch) select { case <-time.After(25 * time.Millisecond): // Expected. - case tg := <-ch: - t.Fatalf("Unexpected target group in file discovery: %s", tg) + case tgs := <-ch: + t.Fatalf("Unexpected target groups in file discovery: %s", tgs) } newf, err := os.Create("fixtures/_test" + ext) @@ -58,37 +59,37 @@ func testFileSD(t *testing.T, ext string) { } newf.Close() - // The files contain two target groups which are read and sent in order. + // The files contain two target groups. select { case <-time.After(15 * time.Second): t.Fatalf("Expected new target group but got none") - case tg := <-ch: + case tgs := <-ch: + tg := tgs[0] + if _, ok := tg.Labels["foo"]; !ok { t.Fatalf("Label not parsed") } if tg.String() != fmt.Sprintf("fixtures/_test%s:0", ext) { t.Fatalf("Unexpected target group %s", tg) } - } - select { - case <-time.After(15 * time.Second): - t.Fatalf("Expected new target group but got none") - case tg := <-ch: + + tg = tgs[1] if tg.String() != fmt.Sprintf("fixtures/_test%s:1", ext) { - t.Fatalf("Unexpected target group %s", tg) + t.Fatalf("Unexpected target groups %s", tg) } } + // Based on unknown circumstances, sometimes fsnotify will trigger more events in // some runs (which might be empty, chains of different operations etc.). // We have to drain those (as the target manager would) to avoid deadlocking and must // not try to make sense of it all... drained := make(chan struct{}) go func() { - for tg := range ch { + for tgs := range ch { // Below we will change the file to a bad syntax. Previously extracted target // groups must not be deleted via sending an empty target group. - if len(tg.Targets) == 0 { - t.Errorf("Unexpected empty target group received: %s", tg) + if len(tgs[0].Targets) == 0 { + t.Errorf("Unexpected empty target groups received: %s", tgs) } } close(drained) @@ -107,6 +108,6 @@ func testFileSD(t *testing.T, ext string) { os.Rename(newf.Name(), "fixtures/_test"+ext) - close(done) + cancel() <-drained } diff --git a/retrieval/discovery/kubernetes/discovery.go b/retrieval/discovery/kubernetes/discovery.go index a23a3e08e..fd7974f8f 100644 --- a/retrieval/discovery/kubernetes/discovery.go +++ b/retrieval/discovery/kubernetes/discovery.go @@ -25,6 +25,7 @@ import ( "github.com/prometheus/common/log" "github.com/prometheus/common/model" + "golang.org/x/net/context" "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/util/httputil" @@ -94,75 +95,35 @@ func (kd *Discovery) Initialize() error { return nil } -// Sources implements the TargetProvider interface. -func (kd *Discovery) Sources() []string { - sourceNames := make([]string, 0, len(kd.apiServers)) - for _, apiServer := range kd.apiServers { - sourceNames = append(sourceNames, apiServersTargetGroupName+":"+apiServer.Host) - } - - nodes, _, err := kd.getNodes() - if err != nil { - // If we can't list nodes then we can't watch them. Assume this is a misconfiguration - // & log & return empty. - log.Errorf("Unable to initialize Kubernetes nodes: %s", err) - return []string{} - } - sourceNames = append(sourceNames, kd.nodeSources(nodes)...) - - services, _, err := kd.getServices() - if err != nil { - // If we can't list services then we can't watch them. Assume this is a misconfiguration - // & log & return empty. - log.Errorf("Unable to initialize Kubernetes services: %s", err) - return []string{} - } - sourceNames = append(sourceNames, kd.serviceSources(services)...) - - return sourceNames -} - -func (kd *Discovery) nodeSources(nodes map[string]*Node) []string { - var sourceNames []string - for name := range nodes { - sourceNames = append(sourceNames, nodesTargetGroupName+":"+name) - } - return sourceNames -} - -func (kd *Discovery) serviceSources(services map[string]map[string]*Service) []string { - var sourceNames []string - for _, ns := range services { - for _, service := range ns { - sourceNames = append(sourceNames, serviceSource(service)) - } - } - return sourceNames -} - // Run implements the TargetProvider interface. -func (kd *Discovery) Run(ch chan<- config.TargetGroup, done <-chan struct{}) { +func (kd *Discovery) Run(ctx context.Context, ch chan<- []*config.TargetGroup) { defer close(ch) - if tg := kd.updateAPIServersTargetGroup(); tg != nil { - select { - case ch <- *tg: - case <-done: - return - } + // Send an initial full view. + // TODO(fabxc): this does not include all available services and service + // endpoints yet. Service endpoints were also missing in the previous Sources() method. + var all []*config.TargetGroup + + all = append(all, kd.updateAPIServersTargetGroup()) + all = append(all, kd.updateNodesTargetGroup()) + + select { + case ch <- all: + case <-ctx.Done(): + return } retryInterval := time.Duration(kd.Conf.RetryInterval) update := make(chan interface{}, 10) - go kd.watchNodes(update, done, retryInterval) - go kd.startServiceWatch(update, done, retryInterval) + go kd.watchNodes(update, ctx.Done(), retryInterval) + go kd.startServiceWatch(update, ctx.Done(), retryInterval) var tg *config.TargetGroup for { select { - case <-done: + case <-ctx.Done(): return case event := <-update: switch obj := event.(type) { @@ -181,8 +142,8 @@ func (kd *Discovery) Run(ch chan<- config.TargetGroup, done <-chan struct{}) { } select { - case ch <- *tg: - case <-done: + case ch <- []*config.TargetGroup{tg}: + case <-ctx.Done(): return } } diff --git a/retrieval/discovery/marathon.go b/retrieval/discovery/marathon.go index 181ea440f..6361a7873 100644 --- a/retrieval/discovery/marathon.go +++ b/retrieval/discovery/marathon.go @@ -17,6 +17,8 @@ import ( "time" "github.com/prometheus/common/log" + "golang.org/x/net/context" + "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/retrieval/discovery/marathon" ) @@ -40,25 +42,13 @@ func NewMarathonDiscovery(conf *config.MarathonSDConfig) *MarathonDiscovery { } } -// Sources implements the TargetProvider interface. -func (md *MarathonDiscovery) Sources() []string { - var sources []string - tgroups, err := md.fetchTargetGroups() - if err == nil { - for source := range tgroups { - sources = append(sources, source) - } - } - return sources -} - // Run implements the TargetProvider interface. -func (md *MarathonDiscovery) Run(ch chan<- config.TargetGroup, done <-chan struct{}) { +func (md *MarathonDiscovery) Run(ctx context.Context, ch chan<- []*config.TargetGroup) { defer close(ch) for { select { - case <-done: + case <-ctx.Done(): return case <-time.After(md.refreshInterval): err := md.updateServices(ch) @@ -69,23 +59,24 @@ func (md *MarathonDiscovery) Run(ch chan<- config.TargetGroup, done <-chan struc } } -func (md *MarathonDiscovery) updateServices(ch chan<- config.TargetGroup) error { +func (md *MarathonDiscovery) updateServices(ch chan<- []*config.TargetGroup) error { targetMap, err := md.fetchTargetGroups() if err != nil { return err } - // Update services which are still present + all := make([]*config.TargetGroup, 0, len(targetMap)) for _, tg := range targetMap { - ch <- *tg + all = append(all, tg) } + ch <- all // Remove services which did disappear for source := range md.lastRefresh { _, ok := targetMap[source] if !ok { log.Debugf("Removing group for %s", source) - ch <- config.TargetGroup{Source: source} + ch <- []*config.TargetGroup{{Source: source}} } } diff --git a/retrieval/discovery/marathon_test.go b/retrieval/discovery/marathon_test.go index f54d82807..cc58a1380 100644 --- a/retrieval/discovery/marathon_test.go +++ b/retrieval/discovery/marathon_test.go @@ -19,6 +19,7 @@ import ( "time" "github.com/prometheus/common/model" + "golang.org/x/net/context" "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/retrieval/discovery/marathon" @@ -26,8 +27,8 @@ import ( var marathonValidLabel = map[string]string{"prometheus": "yes"} -func newTestDiscovery(client marathon.AppListClient) (chan config.TargetGroup, *MarathonDiscovery) { - ch := make(chan config.TargetGroup) +func newTestDiscovery(client marathon.AppListClient) (chan []*config.TargetGroup, *MarathonDiscovery) { + ch := make(chan []*config.TargetGroup) md := NewMarathonDiscovery(&config.MarathonSDConfig{ Servers: []string{"http://localhost:8080"}, }) @@ -60,7 +61,9 @@ func TestMarathonSDEmptyList(t *testing.T) { go func() { select { case tg := <-ch: - t.Fatalf("Got group: %v", tg) + if len(tg) > 0 { + t.Fatalf("Got group: %v", tg) + } default: } }() @@ -96,7 +99,9 @@ func TestMarathonSDSendGroup(t *testing.T) { }) go func() { select { - case tg := <-ch: + case tgs := <-ch: + tg := tgs[0] + if tg.Source != "test-service" { t.Fatalf("Wrong target group name: %s", tg.Source) } @@ -121,9 +126,10 @@ func TestMarathonSDRemoveApp(t *testing.T) { ch, md := newTestDiscovery(func(url string) (*marathon.AppList, error) { return marathonTestAppList(marathonValidLabel, 1), nil }) + go func() { - up1 := <-ch - up2 := <-ch + up1 := (<-ch)[0] + up2 := (<-ch)[0] if up2.Source != up1.Source { t.Fatalf("Source is different: %s", up2) if len(up2.Targets) > 0 { @@ -145,33 +151,25 @@ func TestMarathonSDRemoveApp(t *testing.T) { } } -func TestMarathonSDSources(t *testing.T) { - _, md := newTestDiscovery(func(url string) (*marathon.AppList, error) { - return marathonTestAppList(marathonValidLabel, 1), nil - }) - sources := md.Sources() - if len(sources) != 1 { - t.Fatalf("Wrong number of sources: %s", sources) - } -} - func TestMarathonSDRunAndStop(t *testing.T) { ch, md := newTestDiscovery(func(url string) (*marathon.AppList, error) { return marathonTestAppList(marathonValidLabel, 1), nil }) md.refreshInterval = time.Millisecond * 10 - done := make(chan struct{}) + ctx, cancel := context.WithCancel(context.Background()) go func() { select { case <-ch: - close(done) + cancel() case <-time.After(md.refreshInterval * 3): - close(done) + cancel() t.Fatalf("Update took too long.") } }() - md.Run(ch, done) + + md.Run(ctx, ch) + select { case <-ch: default: diff --git a/retrieval/discovery/nerve.go b/retrieval/discovery/nerve.go index d23416062..f99c9d61f 100644 --- a/retrieval/discovery/nerve.go +++ b/retrieval/discovery/nerve.go @@ -21,6 +21,7 @@ import ( "github.com/prometheus/common/model" "github.com/samuel/go-zookeeper/zk" + "golang.org/x/net/context" "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/util/treecache" @@ -47,7 +48,7 @@ type NerveDiscovery struct { conn *zk.Conn mu sync.RWMutex sources map[string]*config.TargetGroup - sdUpdates *chan<- config.TargetGroup + sdUpdates *chan<- []*config.TargetGroup updates chan treecache.ZookeeperTreeCacheEvent treeCaches []*treecache.ZookeeperTreeCache } @@ -73,17 +74,6 @@ func NewNerveDiscovery(conf *config.NerveSDConfig) *NerveDiscovery { return sd } -// Sources implements the TargetProvider interface. -func (sd *NerveDiscovery) Sources() []string { - sd.mu.RLock() - defer sd.mu.RUnlock() - srcs := []string{} - for t := range sd.sources { - srcs = append(srcs, t) - } - return srcs -} - func (sd *NerveDiscovery) processUpdates() { defer sd.conn.Close() for event := range sd.updates { @@ -104,7 +94,7 @@ func (sd *NerveDiscovery) processUpdates() { } sd.mu.Unlock() if sd.sdUpdates != nil { - *sd.sdUpdates <- *tg + *sd.sdUpdates <- []*config.TargetGroup{tg} } } @@ -114,17 +104,22 @@ func (sd *NerveDiscovery) processUpdates() { } // Run implements the TargetProvider interface. -func (sd *NerveDiscovery) Run(ch chan<- config.TargetGroup, done <-chan struct{}) { +func (sd *NerveDiscovery) Run(ctx context.Context, ch chan<- []*config.TargetGroup) { // Send on everything we have seen so far. sd.mu.Lock() - for _, targetGroup := range sd.sources { - ch <- *targetGroup + + all := make([]*config.TargetGroup, 0, len(sd.sources)) + + for _, tg := range sd.sources { + all = append(all, tg) } + ch <- all + // Tell processUpdates to send future updates. sd.sdUpdates = &ch sd.mu.Unlock() - <-done + <-ctx.Done() for _, tc := range sd.treeCaches { tc.Stop() } diff --git a/retrieval/discovery/serverset.go b/retrieval/discovery/serverset.go index d1172b243..5a13af63c 100644 --- a/retrieval/discovery/serverset.go +++ b/retrieval/discovery/serverset.go @@ -22,6 +22,7 @@ import ( "github.com/prometheus/common/model" "github.com/samuel/go-zookeeper/zk" + "golang.org/x/net/context" "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/util/strutil" @@ -57,7 +58,7 @@ type ServersetDiscovery struct { conn *zk.Conn mu sync.RWMutex sources map[string]*config.TargetGroup - sdUpdates *chan<- config.TargetGroup + sdUpdates *chan<- []*config.TargetGroup updates chan treecache.ZookeeperTreeCacheEvent treeCaches []*treecache.ZookeeperTreeCache } @@ -83,17 +84,6 @@ func NewServersetDiscovery(conf *config.ServersetSDConfig) *ServersetDiscovery { return sd } -// Sources implements the TargetProvider interface. -func (sd *ServersetDiscovery) Sources() []string { - sd.mu.RLock() - defer sd.mu.RUnlock() - srcs := []string{} - for t := range sd.sources { - srcs = append(srcs, t) - } - return srcs -} - func (sd *ServersetDiscovery) processUpdates() { defer sd.conn.Close() for event := range sd.updates { @@ -114,7 +104,7 @@ func (sd *ServersetDiscovery) processUpdates() { } sd.mu.Unlock() if sd.sdUpdates != nil { - *sd.sdUpdates <- *tg + *sd.sdUpdates <- []*config.TargetGroup{tg} } } @@ -124,17 +114,22 @@ func (sd *ServersetDiscovery) processUpdates() { } // Run implements the TargetProvider interface. -func (sd *ServersetDiscovery) Run(ch chan<- config.TargetGroup, done <-chan struct{}) { +func (sd *ServersetDiscovery) Run(ctx context.Context, ch chan<- []*config.TargetGroup) { // Send on everything we have seen so far. sd.mu.Lock() - for _, targetGroup := range sd.sources { - ch <- *targetGroup + + all := make([]*config.TargetGroup, 0, len(sd.sources)) + + for _, tg := range sd.sources { + all = append(all, tg) } + ch <- all + // Tell processUpdates to send future updates. sd.sdUpdates = &ch sd.mu.Unlock() - <-done + <-ctx.Done() for _, tc := range sd.treeCaches { tc.Stop() } @@ -142,8 +137,8 @@ func (sd *ServersetDiscovery) Run(ch chan<- config.TargetGroup, done <-chan stru func parseServersetMember(data []byte, path string) (*model.LabelSet, error) { member := serversetMember{} - err := json.Unmarshal(data, &member) - if err != nil { + + if err := json.Unmarshal(data, &member); err != nil { return nil, fmt.Errorf("error unmarshaling serverset member %q: %s", path, err) } diff --git a/retrieval/targetmanager.go b/retrieval/targetmanager.go index e223cc332..7328c9e03 100644 --- a/retrieval/targetmanager.go +++ b/retrieval/targetmanager.go @@ -40,7 +40,7 @@ type TargetProvider interface { // updated target groups. The channel must be closed by the target provider // if no more updates will be sent. // On receiving from done Run must return. - Run(up chan<- config.TargetGroup, done <-chan struct{}) + Run(ctx context.Context, up chan<- []*config.TargetGroup) } // TargetManager maintains a set of targets, starts and stops their scraping and @@ -178,7 +178,7 @@ func (ss *scrapeSet) run(ctx context.Context) { for name, prov := range providers { var ( - updates = make(chan config.TargetGroup) + updates = make(chan []*config.TargetGroup) ) wg.Add(1) @@ -192,23 +192,17 @@ func (ss *scrapeSet) run(ctx context.Context) { case <-ctx.Done(): ss.stopScrapers(name) return - case update := <-updates: - if err := ss.update(name, &update); err != nil { - log.With("target_group", update).Errorf("Target update failed: %s", err) + case tgs := <-updates: + for _, tg := range tgs { + if err := ss.update(name, tg); err != nil { + log.With("target_group", tg).Errorf("Target update failed: %s", err) + } } } } }(name, prov) - done := make(chan struct{}) - - // TODO(fabxc): Adjust the TargetProvider interface so we can remove this - // redirection of the termination signal. - go func() { - <-ctx.Done() - close(done) - }() - go prov.Run(updates, done) + go prov.Run(ctx, updates) } wg.Wait() @@ -421,21 +415,16 @@ func NewStaticProvider(groups []*config.TargetGroup) *StaticProvider { for i, tg := range groups { tg.Source = fmt.Sprintf("%d", i) } - return &StaticProvider{ - TargetGroups: groups, - } + return &StaticProvider{groups} } // Run implements the TargetProvider interface. -func (sd *StaticProvider) Run(ch chan<- config.TargetGroup, done <-chan struct{}) { - defer close(ch) - - for _, tg := range sd.TargetGroups { - select { - case <-done: - return - case ch <- *tg: - } +func (sd *StaticProvider) Run(ctx context.Context, ch chan<- []*config.TargetGroup) { + // We still have to consider that the consumer exits right away in which case + // the context will be canceled. + select { + case ch <- sd.TargetGroups: + case <-ctx.Done(): } - <-done + close(ch) } From d15adfc917b3d417980c6259c0c8f072f1f82082 Mon Sep 17 00:00:00 2001 From: Fabian Reinartz Date: Fri, 19 Feb 2016 22:54:51 +0100 Subject: [PATCH 04/23] 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. --- retrieval/discovery/dns.go | 2 + retrieval/target.go | 15 +- retrieval/targetmanager.go | 297 +++++++++++++++++++++++++++---------- 3 files changed, 231 insertions(+), 83 deletions(-) diff --git a/retrieval/discovery/dns.go b/retrieval/discovery/dns.go index 3c75f07fc..7c744f8c0 100644 --- a/retrieval/discovery/dns.go +++ b/retrieval/discovery/dns.go @@ -113,6 +113,7 @@ func (dd *DNSDiscovery) Run(ctx context.Context, ch chan<- []*config.TargetGroup func (dd *DNSDiscovery) refreshAll(ch chan<- []*config.TargetGroup) { var wg sync.WaitGroup + wg.Add(len(dd.names)) for _, name := range dd.names { go func(n string) { @@ -122,6 +123,7 @@ func (dd *DNSDiscovery) refreshAll(ch chan<- []*config.TargetGroup) { wg.Done() }(name) } + wg.Wait() } diff --git a/retrieval/target.go b/retrieval/target.go index bf00f7ca7..999b4a5dc 100644 --- a/retrieval/target.go +++ b/retrieval/target.go @@ -167,6 +167,7 @@ type Target struct { scraperStopping chan struct{} // Closing scraperStopped signals that scraping has been stopped. scraperStopped chan struct{} + running bool // Mutex protects the members below. sync.RWMutex @@ -411,9 +412,11 @@ func (t *Target) InstanceIdentifier() string { func (t *Target) RunScraper(sampleAppender storage.SampleAppender) { 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 { case <-time.After(t.offset(lastScrapeInterval)): @@ -471,6 +474,14 @@ func (t *Target) RunScraper(sampleAppender storage.SampleAppender) { // StopScraper implements Target. 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) close(t.scraperStopping) diff --git a/retrieval/targetmanager.go b/retrieval/targetmanager.go index 7328c9e03..ceb880b0d 100644 --- a/retrieval/targetmanager.go +++ b/retrieval/targetmanager.go @@ -17,6 +17,7 @@ import ( "fmt" "strings" "sync" + "time" "github.com/prometheus/common/log" "github.com/prometheus/common/model" @@ -47,21 +48,23 @@ type TargetProvider interface { // creates the new targets based on the target groups it receives from various // target providers. type TargetManager struct { - appender storage.SampleAppender + appender storage.SampleAppender + scrapeConfigs []*config.ScrapeConfig mtx sync.RWMutex ctx context.Context cancel func() wg sync.WaitGroup - // Providers by the scrape configs they are derived from. - scrapeSets []*scrapeSet + // Set of unqiue targets by scrape configuration. + targetSets map[string]*targetSet } // NewTargetManager creates a new TargetManager. func NewTargetManager(app storage.SampleAppender) *TargetManager { return &TargetManager{ - appender: app, + appender: app, + targetSets: map[string]*targetSet{}, } } @@ -72,16 +75,39 @@ func (tm *TargetManager) Run() { tm.mtx.Lock() 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) - go func(ss *scrapeSet) { - ss.run(tm.ctx) + go func(ts *targetSet) { + ts.run(tm.ctx) tm.wg.Done() - }(ss) + }(ts) } tm.mtx.Unlock() + tm.wg.Wait() } @@ -90,14 +116,16 @@ func (tm *TargetManager) Stop() { log.Infoln("Stopping target manager...") tm.mtx.Lock() - - // Cancel the base context, this will cause all in-flight scrapes to abort immmediately. + // Cancel the base context, this will cause all target providers to shut down + // and all in-flight scrapes to abort immmediately. // Started inserts will be finished before terminating. tm.cancel() tm.mtx.Unlock() - // Wait for all provider sets to terminate. + // Wait for all scrape inserts to complete. tm.wg.Wait() + + log.Debugln("Target manager stopped") } // 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{} // TODO(fabxc): this is just a hack to maintain compatibility for now. - for _, ps := range tm.scrapeSets { - for _, ts := range ps.tgroups { + for _, ps := range tm.targetSets { + for _, ts := range ps.scrapePool.tgroups { for _, t := range ts { job := string(t.Labels()[model.JobLabel]) pools[job] = append(pools[job], t) @@ -135,66 +163,120 @@ func (tm *TargetManager) ApplyConfig(cfg *config.Config) bool { } tm.mtx.Lock() - - tm.scrapeSets = tm.scrapeSets[:0] - - for _, scfg := range cfg.ScrapeConfigs { - tm.scrapeSets = append(tm.scrapeSets, newScrapeSet(tm.appender, scfg)) - } - + tm.scrapeConfigs = cfg.ScrapeConfigs tm.mtx.Unlock() 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 // receives target updates. -type scrapeSet struct { - appender storage.SampleAppender +type targetSet struct { + mtx sync.RWMutex + tgroups map[string]map[model.Fingerprint]*Target + providers map[string]TargetProvider - config *config.ScrapeConfig - tgroups map[string]map[model.Fingerprint]*Target + scrapePool *scrapePool + config *config.ScrapeConfig - mtx sync.RWMutex + stopProviders func() + syncCh chan struct{} } -func newScrapeSet(app storage.SampleAppender, cfg *config.ScrapeConfig) *scrapeSet { - return &scrapeSet{ - appender: app, - config: cfg, - tgroups: map[string]map[model.Fingerprint]*Target{}, +func newTargetSet(cfg *config.ScrapeConfig, app storage.SampleAppender) *targetSet { + ts := &targetSet{ + 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 -// and handles their updates. If the context is done, it blocks until the -// target scrapers have terminated. -func (ss *scrapeSet) run(ctx context.Context) { - var ( - providers = providersFromConfig(ss.config) - wg sync.WaitGroup - ) +func (ts *targetSet) run(ctx context.Context) { + ts.scrapePool.ctx = ctx + +Loop: + for { + // Throttle syncing to once per five seconds. + 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 { - var ( - updates = make(chan []*config.TargetGroup) - ) - 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 { select { case <-ctx.Done(): - ss.stopScrapers(name) return case tgs := <-updates: 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) } } @@ -206,18 +288,52 @@ func (ss *scrapeSet) run(ctx context.Context) { } wg.Wait() + + ts.sync() } -// stopScrapers shuts down all active scrapers for a provider. -func (ss *scrapeSet) stopScrapers(name string) { +// update handles a target group update from a target provider identified by the name. +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 - ss.mtx.RLock() - // TODO(fabxc): the prefixing is slightly hacky but this will be gone with subsequent changes. - for source, tgroup := range ss.tgroups { - if !strings.HasPrefix(source, name) { - continue - } + sp.mtx.RLock() + + for _, tgroup := range sp.tgroups { for _, t := range tgroup { wg.Add(1) @@ -227,41 +343,55 @@ func (ss *scrapeSet) stopScrapers(name string) { }(t) } } - ss.mtx.RUnlock() + sp.mtx.RUnlock() wg.Wait() } -// update handles a target group update from a target provider identified by the name. -func (ss *scrapeSet) update(name string, tgroup *config.TargetGroup) error { +func (sp *scrapePool) sync(tgroups map[string]map[model.Fingerprint]*Target) { + sp.mtx.Lock() + var ( - source = name + "/" + tgroup.Source - prevTargets = ss.tgroups[source] + wg sync.WaitGroup + newTgroups = map[string]map[model.Fingerprint]*Target{} ) - targets, err := targetsFromGroup(tgroup, ss.config) - if err != nil { - return err - } + for source, targets := range tgroups { + var ( + prevTargets = sp.tgroups[source] + newTargets = map[model.Fingerprint]*Target{} + ) + newTgroups[source] = newTargets - ss.mtx.Lock() - ss.tgroups[source] = targets + for fp, tnew := range targets { + // If the same target existed before, we let it run and replace + // the new one with it. + if told, ok := prevTargets[fp]; ok { + newTargets[fp] = told + } else { + newTargets[fp] = tnew + go tnew.RunScraper(sp.appender) + } + } + for fp, told := range targets { + // A previous target is no longer in the group. + if _, ok := targets[fp]; !ok { + wg.Add(1) - for fp, tnew := range targets { - // If the same target existed before, we let it run and replace - // the new one with it. - if told, ok := prevTargets[fp]; ok { - targets[fp] = told - } else { - go tnew.RunScraper(ss.appender) + go func(told *Target) { + told.StopScraper() + wg.Done() + }(told) + } } } - ss.mtx.Unlock() - var wg sync.WaitGroup - for fp, told := range prevTargets { - // A previous target is no longer in the group. - if _, ok := targets[fp]; !ok { + // 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) { @@ -270,13 +400,17 @@ func (ss *scrapeSet) update(name string, tgroup *config.TargetGroup) error { }(told) } } + + sp.tgroups = newTgroups + // 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 // may be active and tries to insert. The old scraper that didn't terminate yet could still // be inserting a previous sample set. 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. @@ -331,6 +465,7 @@ func providersFromConfig(cfg *config.ScrapeConfig) map[string]TargetProvider { // targetsFromGroup builds targets based on the given TargetGroup and config. func targetsFromGroup(tg *config.TargetGroup, cfg *config.ScrapeConfig) (map[model.Fingerprint]*Target, error) { targets := make(map[model.Fingerprint]*Target, len(tg.Targets)) + for i, labels := range tg.Targets { for k, v := range cfg.Params { if len(v) > 0 { From da99366f85bb9acc758ecc4ff0eeec43ac0310c7 Mon Sep 17 00:00:00 2001 From: Fabian Reinartz Date: Mon, 22 Feb 2016 10:54:01 +0100 Subject: [PATCH 05/23] Consolidate Target.Update into constructor. The Target.Update method is no longer needed. --- retrieval/target.go | 40 +++++++++------------------------------- retrieval/target_test.go | 9 +-------- 2 files changed, 10 insertions(+), 39 deletions(-) diff --git a/retrieval/target.go b/retrieval/target.go index 999b4a5dc..7dd8cbd2d 100644 --- a/retrieval/target.go +++ b/retrieval/target.go @@ -185,13 +185,20 @@ type Target struct { // NewTarget creates a reasonably configured target for querying. func NewTarget(cfg *config.ScrapeConfig, labels, metaLabels model.LabelSet) (*Target, error) { + client, err := newHTTPClient(cfg) + if err != nil { + return nil, err + } t := &Target{ status: &TargetStatus{}, scraperStopping: make(chan struct{}), scraperStopped: make(chan struct{}), + scrapeConfig: cfg, + labels: labels, + metaLabels: metaLabels, + httpClient: client, } - err := t.Update(cfg, labels, metaLabels) - return t, err + return t, nil } // Status returns the status of the target. @@ -199,28 +206,6 @@ func (t *Target) Status() *TargetStatus { return t.status } -// Update overwrites settings in the target that are derived from the job config -// it belongs to. -func (t *Target) Update(cfg *config.ScrapeConfig, labels, metaLabels model.LabelSet) error { - t.Lock() - - t.scrapeConfig = cfg - t.labels = labels - t.metaLabels = metaLabels - - t.Unlock() - - httpClient, err := t.client() - if err != nil { - return fmt.Errorf("cannot create HTTP client: %s", err) - } - t.Lock() - t.httpClient = httpClient - t.Unlock() - - return nil -} - func newHTTPClient(cfg *config.ScrapeConfig) (*http.Client, error) { tlsOpts := httputil.TLSOptions{ InsecureSkipVerify: cfg.TLSConfig.InsecureSkipVerify, @@ -293,13 +278,6 @@ func (t *Target) offset(interval time.Duration) time.Duration { return time.Duration(next) } -func (t *Target) client() (*http.Client, error) { - t.RLock() - defer t.RUnlock() - - return newHTTPClient(t.scrapeConfig) -} - func (t *Target) interval() time.Duration { t.RLock() defer t.RUnlock() diff --git a/retrieval/target_test.go b/retrieval/target_test.go index 5fc72423b..be8075c05 100644 --- a/retrieval/target_test.go +++ b/retrieval/target_test.go @@ -578,7 +578,7 @@ func newTestTarget(targetURL string, deadline time.Duration, labels model.LabelS labels[model.AddressLabel] = model.LabelValue(strings.TrimLeft(targetURL, "http://")) labels[model.MetricsPathLabel] = "/metrics" - t := &Target{ + return &Target{ scrapeConfig: &config.ScrapeConfig{ ScrapeInterval: model.Duration(time.Millisecond), ScrapeTimeout: model.Duration(deadline), @@ -588,13 +588,6 @@ func newTestTarget(targetURL string, deadline time.Duration, labels model.LabelS scraperStopping: make(chan struct{}), scraperStopped: make(chan struct{}), } - - var err error - if t.httpClient, err = t.client(); err != nil { - panic(err) - } - - return t } func TestNewHTTPBearerToken(t *testing.T) { From cebba3efbb143d35a9ffdb01930cc648235beaa3 Mon Sep 17 00:00:00 2001 From: Fabian Reinartz Date: Mon, 22 Feb 2016 12:42:11 +0100 Subject: [PATCH 06/23] Simplify and fix TargetManager reloading --- retrieval/target.go | 15 +---- retrieval/targetmanager.go | 115 ++++++++++++++++++++----------------- 2 files changed, 65 insertions(+), 65 deletions(-) diff --git a/retrieval/target.go b/retrieval/target.go index 7dd8cbd2d..43c19ca4e 100644 --- a/retrieval/target.go +++ b/retrieval/target.go @@ -167,7 +167,6 @@ type Target struct { scraperStopping chan struct{} // Closing scraperStopped signals that scraping has been stopped. scraperStopped chan struct{} - running bool // Mutex protects the members below. sync.RWMutex @@ -388,11 +387,9 @@ func (t *Target) InstanceIdentifier() string { // RunScraper implements Target. func (t *Target) RunScraper(sampleAppender storage.SampleAppender) { - defer close(t.scraperStopped) + log.Debugf("Running scraper for %v", t) - t.Lock() - t.running = true - t.Unlock() + defer close(t.scraperStopped) lastScrapeInterval := t.interval() @@ -452,14 +449,6 @@ func (t *Target) RunScraper(sampleAppender storage.SampleAppender) { // StopScraper implements Target. 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) close(t.scraperStopping) diff --git a/retrieval/targetmanager.go b/retrieval/targetmanager.go index ceb880b0d..6f2cd67f5 100644 --- a/retrieval/targetmanager.go +++ b/retrieval/targetmanager.go @@ -73,38 +73,9 @@ func (tm *TargetManager) Run() { log.Info("Starting target manager...") tm.mtx.Lock() + tm.ctx, tm.cancel = context.WithCancel(context.Background()) - - 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) - - go func(ts *targetSet) { - ts.run(tm.ctx) - tm.wg.Done() - }(ts) - } + tm.reload() tm.mtx.Unlock() @@ -128,6 +99,38 @@ func (tm *TargetManager) Stop() { log.Debugln("Target manager stopped") } +func (tm *TargetManager) reload() { + 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 + + tm.wg.Add(1) + + go func(ts *targetSet) { + ts.runScraping(tm.ctx) + tm.wg.Done() + }(ts) + } + ts.runProviders(tm.ctx, providersFromConfig(scfg)) + } + + // Remove old target sets. Waiting for stopping is already guaranteed + // by the goroutine that started the target set. + for name, ts := range tm.targetSets { + if _, ok := jobs[name]; !ok { + ts.cancel() + delete(tm.targetSets, name) + } + } +} + // Pools returns the targets currently being scraped bucketed by their job name. func (tm *TargetManager) Pools() map[string][]*Target { tm.mtx.RLock() @@ -151,21 +154,14 @@ func (tm *TargetManager) Pools() map[string][]*Target { // by the new cfg. The state of targets that are valid in the new configuration remains unchanged. // Returns true on success. func (tm *TargetManager) ApplyConfig(cfg *config.Config) bool { - tm.mtx.RLock() - running := tm.ctx != nil - tm.mtx.RUnlock() - - if running { - tm.Stop() - defer func() { - go tm.Run() - }() - } - tm.mtx.Lock() - tm.scrapeConfigs = cfg.ScrapeConfigs - tm.mtx.Unlock() + defer tm.mtx.Unlock() + tm.scrapeConfigs = cfg.ScrapeConfigs + + if tm.ctx != nil { + tm.reload() + } return true } @@ -180,8 +176,9 @@ type targetSet struct { scrapePool *scrapePool config *config.ScrapeConfig - stopProviders func() - syncCh chan struct{} + syncCh chan struct{} + cancelScraping func() + cancelProviders func() } func newTargetSet(cfg *config.ScrapeConfig, app storage.SampleAppender) *targetSet { @@ -194,7 +191,21 @@ func newTargetSet(cfg *config.ScrapeConfig, app storage.SampleAppender) *targetS return ts } -func (ts *targetSet) run(ctx context.Context) { +func (ts *targetSet) cancel() { + ts.mtx.RLock() + defer ts.mtx.RUnlock() + + if ts.cancelScraping != nil { + ts.cancelScraping() + } + if ts.cancelProviders != nil { + ts.cancelProviders() + } +} + +func (ts *targetSet) runScraping(ctx context.Context) { + ctx, ts.cancelScraping = context.WithCancel(ctx) + ts.scrapePool.ctx = ctx Loop: @@ -234,10 +245,10 @@ func (ts *targetSet) runProviders(ctx context.Context, providers map[string]Targ var wg sync.WaitGroup - if ts.stopProviders != nil { - ts.stopProviders() + if ts.cancelProviders != nil { + ts.cancelProviders() } - ctx, ts.stopProviders = context.WithCancel(ctx) + ctx, ts.cancelProviders = context.WithCancel(ctx) for name, prov := range providers { wg.Add(1) @@ -373,7 +384,7 @@ func (sp *scrapePool) sync(tgroups map[string]map[model.Fingerprint]*Target) { go tnew.RunScraper(sp.appender) } } - for fp, told := range targets { + for fp, told := range prevTargets { // A previous target is no longer in the group. if _, ok := targets[fp]; !ok { wg.Add(1) @@ -388,7 +399,7 @@ func (sp *scrapePool) sync(tgroups map[string]map[model.Fingerprint]*Target) { // Stop scrapers for target groups that disappeared completely. for source, targets := range sp.tgroups { - if _, ok := tgroups[source]; !ok { + if _, ok := tgroups[source]; ok { continue } for _, told := range targets { From 05de8b7f8d43bc37b7198cad16e9b4760ba255ff Mon Sep 17 00:00:00 2001 From: Fabian Reinartz Date: Mon, 22 Feb 2016 16:46:55 +0100 Subject: [PATCH 07/23] Extract target scraping into scrape loop. This commit factors out the scrape loop handling into its own data structure. For the transition it will be directly attached to the target. --- retrieval/scrape.go | 325 ++++++++++++++++++++++++++++++++----- retrieval/target.go | 219 +++---------------------- retrieval/targetmanager.go | 101 ------------ 3 files changed, 305 insertions(+), 340 deletions(-) diff --git a/retrieval/scrape.go b/retrieval/scrape.go index 40d398ded..da6fa5d8a 100644 --- a/retrieval/scrape.go +++ b/retrieval/scrape.go @@ -13,65 +13,302 @@ package retrieval -// import ( -// "sync" -// "time" +import ( + "errors" + "sync" + "time" -// "github.com/prometheus/common/log" -// "github.com/prometheus/common/model" -// "golang.org/x/net/context" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/common/log" + "github.com/prometheus/common/model" + "golang.org/x/net/context" -// "github.com/prometheus/prometheus/config" -// "github.com/prometheus/prometheus/storage" -// ) + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/storage/local" +) -// type scraper interface { -// scrape(context.Context) error -// report(start time.Time, dur time.Duration, err error) error -// } +const ( + scrapeHealthMetricName = "up" + scrapeDurationMetricName = "scrape_duration_seconds" -// type scrapePool struct { -// mtx sync.RWMutex -// targets map[model.Fingerprint]*Target -// loops map[model.Fingerprint]loop + // Capacity of the channel to buffer samples during ingestion. + ingestedSamplesCap = 256 -// config *config.ScrapeConfig + // Constants for instrumentation. + namespace = "prometheus" + interval = "interval" +) -// newLoop func(context.Context) -// } +var ( + errSkippedScrape = errors.New("scrape skipped due to throttled ingestion") -// func newScrapePool(c *config.ScrapeConfig) *scrapePool { -// return &scrapePool{config: c} -// } + targetIntervalLength = prometheus.NewSummaryVec( + prometheus.SummaryOpts{ + Namespace: namespace, + Name: "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}, + ) + targetSkippedScrapes = prometheus.NewCounterVec( + prometheus.CounterOpts{ + Namespace: namespace, + Name: "target_skipped_scrapes_total", + Help: "Total number of scrapes that were skipped because the metric storage was throttled.", + }, + []string{interval}, + ) +) -// func (sp *scrapePool) sync(targets []*Target) { -// sp.mtx.Lock() -// defer sp.mtx.Unlock() +func init() { + prometheus.MustRegister(targetIntervalLength) + prometheus.MustRegister(targetSkippedScrapes) +} -// uniqueTargets := make(map[string]*Target{}, len(targets)) +// A scraper retrieves samples and accepts a status report at the end. +type scraper interface { + scrape(context.Context) (model.Samples, error) + report(start time.Time, dur time.Duration, err error) + offset(interval time.Duration) time.Duration +} -// for _, t := range targets { -// uniqueTargets[t.fingerprint()] = t -// } +// scrapePool manages scrapes for sets of targets. +type scrapePool struct { + appender storage.SampleAppender -// sp.targets = uniqueTargets -// } + ctx context.Context + mtx sync.RWMutex + tgroups map[string]map[model.Fingerprint]*Target -// type scrapeLoop struct { -// scraper scraper -// mtx sync.RWMutex -// } + targets map[model.Fingerprint]loop +} -// func newScrapeLoop(ctx context.Context) +func newScrapePool(app storage.SampleAppender) *scrapePool { + return &scrapePool{ + appender: app, + tgroups: map[string]map[model.Fingerprint]*Target{}, + } +} -// func (sl *scrapeLoop) update() {} +func (sp *scrapePool) stop() { + var wg sync.WaitGroup -// func (sl *scrapeLoop) run(ctx context.Context) { -// var wg sync.WaitGroup + sp.mtx.RLock() -// wg.Wait() -// } + for _, tgroup := range sp.tgroups { + for _, t := range tgroup { + wg.Add(1) -// func (sl *scrapeLoop) stop() { + go func(t *Target) { + t.scrapeLoop.stop() + wg.Done() + }(t) + } + } + sp.mtx.RUnlock() -// } + wg.Wait() +} + +func (sp *scrapePool) sync(tgroups map[string]map[model.Fingerprint]*Target) { + sp.mtx.Lock() + + var ( + wg sync.WaitGroup + newTgroups = map[string]map[model.Fingerprint]*Target{} + ) + + for source, targets := range tgroups { + var ( + prevTargets = sp.tgroups[source] + newTargets = map[model.Fingerprint]*Target{} + ) + newTgroups[source] = newTargets + + for fp, tnew := range targets { + // If the same target existed before, we let it run and replace + // the new one with it. + if told, ok := prevTargets[fp]; ok { + newTargets[fp] = told + } else { + newTargets[fp] = tnew + + tnew.scrapeLoop = newScrapeLoop(sp.ctx, tnew, tnew.wrapAppender(sp.appender), tnew.wrapReportingAppender(sp.appender)) + go tnew.scrapeLoop.run(tnew.interval(), tnew.timeout()) + } + } + for fp, told := range prevTargets { + // A previous target is no longer in the group. + if _, ok := targets[fp]; !ok { + wg.Add(1) + + go func(told *Target) { + told.scrapeLoop.stop() + wg.Done() + }(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.scrapeLoop.stop() + wg.Done() + }(told) + } + } + + sp.tgroups = newTgroups + + // 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 + // may be active and tries to insert. The old scraper that didn't terminate yet could still + // be inserting a previous sample set. + wg.Wait() + + // TODO(fabxc): maybe this can be released earlier with subsequent refactoring. + sp.mtx.Unlock() +} + +type loop interface { + run(interval, timeout time.Duration) + stop() +} + +type scrapeLoop struct { + scraper scraper + + appender storage.SampleAppender + reportAppender storage.SampleAppender + + done chan struct{} + mtx sync.RWMutex + ctx context.Context + cancel func() +} + +func newScrapeLoop(ctx context.Context, sc scraper, app, reportApp storage.SampleAppender) *scrapeLoop { + sl := &scrapeLoop{ + scraper: sc, + appender: app, + reportAppender: reportApp, + done: make(chan struct{}), + } + sl.ctx, sl.cancel = context.WithCancel(ctx) + + return sl +} + +func (sl *scrapeLoop) run(interval, timeout time.Duration) { + defer close(sl.done) + + select { + case <-time.After(sl.scraper.offset(interval)): + // Continue after a scraping offset. + case <-sl.ctx.Done(): + return + } + + var last time.Time + + ticker := time.NewTicker(interval) + defer ticker.Stop() + + for { + select { + case <-sl.ctx.Done(): + return + default: + } + + if sl.appender.NeedsThrottling() { + targetSkippedScrapes.WithLabelValues(interval.String()).Inc() + continue + } + targetIntervalLength.WithLabelValues(interval.String()).Observe( + float64(time.Since(last)) / float64(time.Second), // Sub-second precision. + ) + + var ( + start = time.Now() + scrapeCtx, _ = context.WithTimeout(sl.ctx, timeout) + ) + + samples, err := sl.scraper.scrape(scrapeCtx) + if err == nil { + sl.append(samples) + } + + sl.report(start, time.Since(start), err) + + select { + case <-sl.ctx.Done(): + return + case <-ticker.C: + } + + last = start + } +} + +func (sl *scrapeLoop) stop() { + sl.mtx.RLock() + sl.cancel() + sl.mtx.RUnlock() + + <-sl.done +} + +func (sl *scrapeLoop) append(samples model.Samples) { + numOutOfOrder := 0 + + for _, s := range samples { + if err := sl.appender.Append(s); err != nil { + if err == local.ErrOutOfOrderSample { + numOutOfOrder++ + } else { + log.Warnf("Error inserting sample: %s", err) + } + } + } + if numOutOfOrder > 0 { + log.With("numDropped", numOutOfOrder).Warn("Error on ingesting out-of-order samples") + } +} + +func (sl *scrapeLoop) report(start time.Time, duration time.Duration, err error) { + sl.scraper.report(start, duration, err) + + ts := model.TimeFromUnixNano(start.UnixNano()) + + var health model.SampleValue + if err == nil { + health = 1 + } + + healthSample := &model.Sample{ + Metric: model.Metric{ + model.MetricNameLabel: scrapeHealthMetricName, + }, + Timestamp: ts, + Value: health, + } + durationSample := &model.Sample{ + Metric: model.Metric{ + model.MetricNameLabel: scrapeDurationMetricName, + }, + Timestamp: ts, + Value: model.SampleValue(float64(duration) / float64(time.Second)), + } + + sl.reportAppender.Append(healthSample) + sl.reportAppender.Append(durationSample) +} diff --git a/retrieval/target.go b/retrieval/target.go index 43c19ca4e..48e73cac7 100644 --- a/retrieval/target.go +++ b/retrieval/target.go @@ -14,7 +14,6 @@ package retrieval import ( - "errors" "fmt" "io" "io/ioutil" @@ -24,58 +23,16 @@ import ( "sync" "time" - "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/common/expfmt" - "github.com/prometheus/common/log" "github.com/prometheus/common/model" "golang.org/x/net/context" "golang.org/x/net/context/ctxhttp" "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/storage" - "github.com/prometheus/prometheus/storage/local" "github.com/prometheus/prometheus/util/httputil" ) -const ( - scrapeHealthMetricName = "up" - scrapeDurationMetricName = "scrape_duration_seconds" - - // Capacity of the channel to buffer samples during ingestion. - ingestedSamplesCap = 256 - - // Constants for instrumentation. - namespace = "prometheus" - interval = "interval" -) - -var ( - errSkippedScrape = errors.New("scrape skipped due to throttled ingestion") - - targetIntervalLength = prometheus.NewSummaryVec( - prometheus.SummaryOpts{ - Namespace: namespace, - Name: "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}, - ) - targetSkippedScrapes = prometheus.NewCounterVec( - prometheus.CounterOpts{ - Namespace: namespace, - Name: "target_skipped_scrapes_total", - Help: "Total number of scrapes that were skipped because the metric storage was throttled.", - }, - []string{interval}, - ) -) - -func init() { - prometheus.MustRegister(targetIntervalLength) - prometheus.MustRegister(targetSkippedScrapes) -} - // TargetHealth describes the health state of a target. type TargetHealth int @@ -163,10 +120,8 @@ func (ts *TargetStatus) setLastError(err error) { type Target struct { // The status object for the target. It is only set once on initialization. status *TargetStatus - // Closing scraperStopping signals that scraping should stop. - scraperStopping chan struct{} - // Closing scraperStopped signals that scraping has been stopped. - scraperStopped chan struct{} + + scrapeLoop *scrapeLoop // Mutex protects the members below. sync.RWMutex @@ -189,13 +144,11 @@ func NewTarget(cfg *config.ScrapeConfig, labels, metaLabels model.LabelSet) (*Ta return nil, err } t := &Target{ - status: &TargetStatus{}, - scraperStopping: make(chan struct{}), - scraperStopped: make(chan struct{}), - scrapeConfig: cfg, - labels: labels, - metaLabels: metaLabels, - httpClient: client, + status: &TargetStatus{}, + scrapeConfig: cfg, + labels: labels, + metaLabels: metaLabels, + httpClient: client, } return t, nil } @@ -385,184 +338,60 @@ func (t *Target) InstanceIdentifier() string { return t.host() } -// RunScraper implements Target. -func (t *Target) RunScraper(sampleAppender storage.SampleAppender) { - log.Debugf("Running scraper for %v", t) - - defer close(t.scraperStopped) - - lastScrapeInterval := t.interval() - - select { - case <-time.After(t.offset(lastScrapeInterval)): - // Continue after scraping offset. - case <-t.scraperStopping: - return - } - - ticker := time.NewTicker(lastScrapeInterval) - defer ticker.Stop() - - t.scrape(sampleAppender) - - // Explanation of the contraption below: - // - // In case t.scraperStopping has something to receive, we want to read - // from that channel rather than starting a new scrape (which might take very - // long). That's why the outer select has no ticker.C. Should t.scraperStopping - // not have anything to receive, we go into the inner select, where ticker.C - // is in the mix. - for { - select { - case <-t.scraperStopping: - return - default: - select { - case <-t.scraperStopping: - return - case <-ticker.C: - took := time.Since(t.status.LastScrape()) - - intervalStr := lastScrapeInterval.String() - - // On changed scrape interval the new interval becomes effective - // after the next scrape. - if iv := t.interval(); iv != lastScrapeInterval { - ticker.Stop() - ticker = time.NewTicker(iv) - lastScrapeInterval = iv - } - - targetIntervalLength.WithLabelValues(intervalStr).Observe( - float64(took) / float64(time.Second), // Sub-second precision. - ) - if sampleAppender.NeedsThrottling() { - targetSkippedScrapes.WithLabelValues(intervalStr).Inc() - t.status.setLastError(errSkippedScrape) - continue - } - t.scrape(sampleAppender) - } - } - } -} - -// StopScraper implements Target. -func (t *Target) StopScraper() { - log.Debugf("Stopping scraper for target %v...", t) - - close(t.scraperStopping) - <-t.scraperStopped - - log.Debugf("Scraper for target %v stopped.", t) -} - const acceptHeader = `application/vnd.google.protobuf;proto=io.prometheus.client.MetricFamily;encoding=delimited;q=0.7,text/plain;version=0.0.4;q=0.3,application/json;schema="prometheus/telemetry";version=0.0.2;q=0.2,*/*;q=0.1` -func (t *Target) scrape(appender storage.SampleAppender) error { - var ( - err error - start = time.Now() - ) - defer func(appender storage.SampleAppender) { - t.report(appender, start, time.Since(start), err) - }(appender) - +func (t *Target) scrape(ctx context.Context) (model.Samples, error) { t.RLock() - - appender = t.wrapAppender(appender) - client := t.httpClient t.RUnlock() + start := time.Now() + req, err := http.NewRequest("GET", t.URL().String(), nil) if err != nil { - return err + return nil, err } req.Header.Add("Accept", acceptHeader) - ctx, _ := context.WithTimeout(context.Background(), t.timeout()) resp, err := ctxhttp.Do(ctx, client, req) if err != nil { - return err + return nil, err } defer resp.Body.Close() if resp.StatusCode != http.StatusOK { - return fmt.Errorf("server returned HTTP status %s", resp.Status) + return nil, fmt.Errorf("server returned HTTP status %s", resp.Status) } - dec := expfmt.NewDecoder(resp.Body, expfmt.ResponseFormat(resp.Header)) - + var ( + allSamples = make(model.Samples, 0, 200) + decSamples = make(model.Vector, 0, 50) + ) sdec := expfmt.SampleDecoder{ - Dec: dec, + Dec: expfmt.NewDecoder(resp.Body, expfmt.ResponseFormat(resp.Header)), Opts: &expfmt.DecodeOptions{ Timestamp: model.TimeFromUnixNano(start.UnixNano()), }, } - var ( - samples model.Vector - numOutOfOrder int - logger = log.With("target", t.InstanceIdentifier()) - ) for { - if err = sdec.Decode(&samples); err != nil { + if err = sdec.Decode(&decSamples); err != nil { break } - for _, s := range samples { - err := appender.Append(s) - if err != nil { - if err == local.ErrOutOfOrderSample { - numOutOfOrder++ - } else { - logger.With("sample", s).Warnf("Error inserting sample: %s", err) - } - } - - } - } - if numOutOfOrder > 0 { - logger.With("numDropped", numOutOfOrder).Warn("Error on ingesting out-of-order samples") + allSamples = append(allSamples, decSamples...) + decSamples = decSamples[:0] } if err == io.EOF { // Set err to nil since it is used in the scrape health recording. err = nil } - return err + return allSamples, err } -func (t *Target) report(app storage.SampleAppender, start time.Time, duration time.Duration, err error) { - t.status.setLastScrape(start) +func (t *Target) report(start time.Time, dur time.Duration, err error) { t.status.setLastError(err) - - ts := model.TimeFromUnixNano(start.UnixNano()) - - var health model.SampleValue - if err == nil { - health = 1 - } - - healthSample := &model.Sample{ - Metric: model.Metric{ - model.MetricNameLabel: scrapeHealthMetricName, - }, - Timestamp: ts, - Value: health, - } - durationSample := &model.Sample{ - Metric: model.Metric{ - model.MetricNameLabel: scrapeDurationMetricName, - }, - Timestamp: ts, - Value: model.SampleValue(float64(duration) / float64(time.Second)), - } - - app = t.wrapReportingAppender(app) - - app.Append(healthSample) - app.Append(durationSample) + t.status.setLastScrape(start) } // Merges the ingested sample's metric with the label set. On a collision the diff --git a/retrieval/targetmanager.go b/retrieval/targetmanager.go index 6f2cd67f5..bc750f4b2 100644 --- a/retrieval/targetmanager.go +++ b/retrieval/targetmanager.go @@ -323,107 +323,6 @@ func (ts *targetSet) update(name string, tgroup *config.TargetGroup) error { 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 - - sp.mtx.RLock() - - for _, tgroup := range sp.tgroups { - for _, t := range tgroup { - wg.Add(1) - - go func(t *Target) { - t.StopScraper() - wg.Done() - }(t) - } - } - sp.mtx.RUnlock() - - wg.Wait() -} - -func (sp *scrapePool) sync(tgroups map[string]map[model.Fingerprint]*Target) { - sp.mtx.Lock() - - var ( - wg sync.WaitGroup - newTgroups = map[string]map[model.Fingerprint]*Target{} - ) - - for source, targets := range tgroups { - var ( - prevTargets = sp.tgroups[source] - newTargets = map[model.Fingerprint]*Target{} - ) - newTgroups[source] = newTargets - - for fp, tnew := range targets { - // If the same target existed before, we let it run and replace - // the new one with it. - if told, ok := prevTargets[fp]; ok { - newTargets[fp] = told - } else { - newTargets[fp] = tnew - go tnew.RunScraper(sp.appender) - } - } - for fp, told := range prevTargets { - // A previous target is no longer in the group. - if _, ok := targets[fp]; !ok { - wg.Add(1) - - go func(told *Target) { - told.StopScraper() - wg.Done() - }(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. - // 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 - // be inserting a previous sample set. - wg.Wait() - - // TODO(fabxc): maybe this can be released earlier with subsequent refactoring. - sp.mtx.Unlock() -} - // providersFromConfig returns all TargetProviders configured in cfg. func providersFromConfig(cfg *config.ScrapeConfig) map[string]TargetProvider { providers := map[string]TargetProvider{} From c7bbe95597fe6d8aa67b16011056b850afb7f790 Mon Sep 17 00:00:00 2001 From: Fabian Reinartz Date: Mon, 22 Feb 2016 18:49:26 +0100 Subject: [PATCH 08/23] Remove outdated target tests --- retrieval/scrape.go | 45 ++--- retrieval/target_test.go | 350 +-------------------------------------- 2 files changed, 28 insertions(+), 367 deletions(-) diff --git a/retrieval/scrape.go b/retrieval/scrape.go index da6fa5d8a..bbf1e113c 100644 --- a/retrieval/scrape.go +++ b/retrieval/scrape.go @@ -135,7 +135,7 @@ func (sp *scrapePool) sync(tgroups map[string]map[model.Fingerprint]*Target) { newTargets[fp] = tnew tnew.scrapeLoop = newScrapeLoop(sp.ctx, tnew, tnew.wrapAppender(sp.appender), tnew.wrapReportingAppender(sp.appender)) - go tnew.scrapeLoop.run(tnew.interval(), tnew.timeout()) + go tnew.scrapeLoop.run(tnew.interval(), tnew.timeout(), nil) } } for fp, told := range prevTargets { @@ -179,7 +179,7 @@ func (sp *scrapePool) sync(tgroups map[string]map[model.Fingerprint]*Target) { } type loop interface { - run(interval, timeout time.Duration) + run(interval, timeout time.Duration, errc chan<- error) stop() } @@ -207,7 +207,7 @@ func newScrapeLoop(ctx context.Context, sc scraper, app, reportApp storage.Sampl return sl } -func (sl *scrapeLoop) run(interval, timeout time.Duration) { +func (sl *scrapeLoop) run(interval, timeout time.Duration, errc chan<- error) { defer close(sl.done) select { @@ -229,33 +229,34 @@ func (sl *scrapeLoop) run(interval, timeout time.Duration) { default: } - if sl.appender.NeedsThrottling() { + if !sl.appender.NeedsThrottling() { + var ( + start = time.Now() + scrapeCtx, _ = context.WithTimeout(sl.ctx, timeout) + ) + + targetIntervalLength.WithLabelValues(interval.String()).Observe( + float64(time.Since(last)) / float64(time.Second), // Sub-second precision. + ) + + samples, err := sl.scraper.scrape(scrapeCtx) + if err == nil { + sl.append(samples) + } else if errc != nil { + errc <- err + } + + sl.report(start, time.Since(start), err) + last = start + } else { targetSkippedScrapes.WithLabelValues(interval.String()).Inc() - continue } - targetIntervalLength.WithLabelValues(interval.String()).Observe( - float64(time.Since(last)) / float64(time.Second), // Sub-second precision. - ) - - var ( - start = time.Now() - scrapeCtx, _ = context.WithTimeout(sl.ctx, timeout) - ) - - samples, err := sl.scraper.scrape(scrapeCtx) - if err == nil { - sl.append(samples) - } - - sl.report(start, time.Since(start), err) select { case <-sl.ctx.Done(): return case <-ticker.C: } - - last = start } } diff --git a/retrieval/target_test.go b/retrieval/target_test.go index be8075c05..2627d97f0 100644 --- a/retrieval/target_test.go +++ b/retrieval/target_test.go @@ -28,6 +28,7 @@ import ( "time" "github.com/prometheus/common/model" + "golang.org/x/net/context" "github.com/prometheus/prometheus/config" ) @@ -167,299 +168,6 @@ func TestTargetWrapAppender(t *testing.T) { } } -func TestOverwriteLabels(t *testing.T) { - type test struct { - metric string - resultNormal model.Metric - resultHonor model.Metric - } - var tests []test - - server := httptest.NewServer( - http.HandlerFunc( - func(w http.ResponseWriter, r *http.Request) { - w.Header().Set("Content-Type", `text/plain; version=0.0.4`) - for _, test := range tests { - w.Write([]byte(test.metric)) - w.Write([]byte(" 1\n")) - } - }, - ), - ) - defer server.Close() - addr := model.LabelValue(strings.Split(server.URL, "://")[1]) - - tests = []test{ - { - metric: `foo{}`, - resultNormal: model.Metric{ - model.MetricNameLabel: "foo", - model.InstanceLabel: addr, - }, - resultHonor: model.Metric{ - model.MetricNameLabel: "foo", - model.InstanceLabel: addr, - }, - }, - { - metric: `foo{instance=""}`, - resultNormal: model.Metric{ - model.MetricNameLabel: "foo", - model.InstanceLabel: addr, - }, - resultHonor: model.Metric{ - model.MetricNameLabel: "foo", - }, - }, - { - metric: `foo{instance="other_instance"}`, - resultNormal: model.Metric{ - model.MetricNameLabel: "foo", - model.InstanceLabel: addr, - model.ExportedLabelPrefix + model.InstanceLabel: "other_instance", - }, - resultHonor: model.Metric{ - model.MetricNameLabel: "foo", - model.InstanceLabel: "other_instance", - }, - }, - } - - target := newTestTarget(server.URL, time.Second, nil) - - target.scrapeConfig.HonorLabels = false - app := &collectResultAppender{} - if err := target.scrape(app); err != nil { - t.Fatal(err) - } - - for i, test := range tests { - if !reflect.DeepEqual(app.result[i].Metric, test.resultNormal) { - t.Errorf("Error comparing %q:\nExpected:\n%s\nGot:\n%s\n", test.metric, test.resultNormal, app.result[i].Metric) - } - } - - target.scrapeConfig.HonorLabels = true - app = &collectResultAppender{} - if err := target.scrape(app); err != nil { - t.Fatal(err) - } - - for i, test := range tests { - if !reflect.DeepEqual(app.result[i].Metric, test.resultHonor) { - t.Errorf("Error comparing %q:\nExpected:\n%s\nGot:\n%s\n", test.metric, test.resultHonor, app.result[i].Metric) - } - - } -} -func TestTargetScrapeUpdatesState(t *testing.T) { - testTarget := newTestTarget("bad schema", 0, nil) - - testTarget.scrape(nopAppender{}) - if testTarget.status.Health() != HealthBad { - t.Errorf("Expected target state %v, actual: %v", HealthBad, testTarget.status.Health()) - } -} - -func TestTargetScrapeWithThrottledStorage(t *testing.T) { - server := httptest.NewServer( - http.HandlerFunc( - func(w http.ResponseWriter, r *http.Request) { - w.Header().Set("Content-Type", `text/plain; version=0.0.4`) - for i := 0; i < 10; i++ { - w.Write([]byte( - fmt.Sprintf("test_metric_%d{foo=\"bar\"} 123.456\n", i), - )) - } - }, - ), - ) - defer server.Close() - - testTarget := newTestTarget(server.URL, time.Second, model.LabelSet{"dings": "bums"}) - - go testTarget.RunScraper(&collectResultAppender{throttled: true}) - - // Enough time for a scrape to happen. - time.Sleep(20 * time.Millisecond) - - testTarget.StopScraper() - // Wait for it to take effect. - time.Sleep(20 * time.Millisecond) - - if testTarget.status.Health() != HealthBad { - t.Errorf("Expected target state %v, actual: %v", HealthBad, testTarget.status.Health()) - } - if testTarget.status.LastError() != errSkippedScrape { - t.Errorf("Expected target error %q, actual: %q", errSkippedScrape, testTarget.status.LastError()) - } -} - -func TestTargetScrapeMetricRelabelConfigs(t *testing.T) { - server := httptest.NewServer( - http.HandlerFunc( - func(w http.ResponseWriter, r *http.Request) { - w.Header().Set("Content-Type", `text/plain; version=0.0.4`) - w.Write([]byte("test_metric_drop 0\n")) - w.Write([]byte("test_metric_relabel 1\n")) - }, - ), - ) - defer server.Close() - testTarget := newTestTarget(server.URL, time.Second, model.LabelSet{}) - testTarget.scrapeConfig.MetricRelabelConfigs = []*config.RelabelConfig{ - { - SourceLabels: model.LabelNames{"__name__"}, - Regex: config.MustNewRegexp(".*drop.*"), - Action: config.RelabelDrop, - }, - { - SourceLabels: model.LabelNames{"__name__"}, - Regex: config.MustNewRegexp(".*(relabel|up).*"), - TargetLabel: "foo", - Replacement: "bar", - Action: config.RelabelReplace, - }, - } - - appender := &collectResultAppender{} - if err := testTarget.scrape(appender); err != nil { - t.Fatal(err) - } - - // Remove variables part of result. - for _, sample := range appender.result { - sample.Timestamp = 0 - sample.Value = 0 - } - - expected := []*model.Sample{ - { - Metric: model.Metric{ - model.MetricNameLabel: "test_metric_relabel", - "foo": "bar", - model.InstanceLabel: model.LabelValue(testTarget.host()), - }, - Timestamp: 0, - Value: 0, - }, - // The metrics about the scrape are not affected. - { - Metric: model.Metric{ - model.MetricNameLabel: scrapeHealthMetricName, - model.InstanceLabel: model.LabelValue(testTarget.host()), - }, - Timestamp: 0, - Value: 0, - }, - { - Metric: model.Metric{ - model.MetricNameLabel: scrapeDurationMetricName, - model.InstanceLabel: model.LabelValue(testTarget.host()), - }, - Timestamp: 0, - Value: 0, - }, - } - - if !appender.result.Equal(expected) { - t.Fatalf("Expected and actual samples not equal. Expected: %s, actual: %s", expected, appender.result) - } - -} - -func TestTargetRecordScrapeHealth(t *testing.T) { - var ( - testTarget = newTestTarget("example.url:80", 0, model.LabelSet{model.JobLabel: "testjob"}) - now = model.Now() - appender = &collectResultAppender{} - ) - - testTarget.report(appender, now.Time(), 2*time.Second, nil) - - result := appender.result - - if len(result) != 2 { - t.Fatalf("Expected two samples, got %d", len(result)) - } - - actual := result[0] - expected := &model.Sample{ - Metric: model.Metric{ - model.MetricNameLabel: scrapeHealthMetricName, - model.InstanceLabel: "example.url:80", - model.JobLabel: "testjob", - }, - Timestamp: now, - Value: 1, - } - - if !actual.Equal(expected) { - t.Fatalf("Expected and actual samples not equal. Expected: %v, actual: %v", expected, actual) - } - - actual = result[1] - expected = &model.Sample{ - Metric: model.Metric{ - model.MetricNameLabel: scrapeDurationMetricName, - model.InstanceLabel: "example.url:80", - model.JobLabel: "testjob", - }, - Timestamp: now, - Value: 2.0, - } - - if !actual.Equal(expected) { - t.Fatalf("Expected and actual samples not equal. Expected: %v, actual: %v", expected, actual) - } -} - -func TestTargetScrapeTimeout(t *testing.T) { - signal := make(chan bool, 1) - server := httptest.NewServer( - http.HandlerFunc( - func(w http.ResponseWriter, r *http.Request) { - <-signal - w.Header().Set("Content-Type", `text/plain; version=0.0.4`) - w.Write([]byte{}) - }, - ), - ) - defer server.Close() - - testTarget := newTestTarget(server.URL, 50*time.Millisecond, model.LabelSet{}) - - appender := nopAppender{} - - // scrape once without timeout - signal <- true - if err := testTarget.scrape(appender); err != nil { - t.Fatal(err) - } - - // let the deadline lapse - time.Sleep(55 * time.Millisecond) - - // now scrape again - signal <- true - if err := testTarget.scrape(appender); err != nil { - t.Fatal(err) - } - - // now timeout - if err := testTarget.scrape(appender); err == nil { - t.Fatal("expected scrape to timeout") - } else { - signal <- true // let handler continue - } - - // now scrape again without timeout - signal <- true - if err := testTarget.scrape(appender); err != nil { - t.Fatal(err) - } -} - func TestTargetScrape404(t *testing.T) { server := httptest.NewServer( http.HandlerFunc( @@ -471,59 +179,14 @@ func TestTargetScrape404(t *testing.T) { defer server.Close() testTarget := newTestTarget(server.URL, time.Second, model.LabelSet{}) - appender := nopAppender{} want := errors.New("server returned HTTP status 404 Not Found") - got := testTarget.scrape(appender) + _, got := testTarget.scrape(context.Background()) if got == nil || want.Error() != got.Error() { t.Fatalf("want err %q, got %q", want, got) } } -func TestTargetRunScraperScrapes(t *testing.T) { - testTarget := newTestTarget("bad schema", 0, nil) - - go testTarget.RunScraper(nopAppender{}) - - // Enough time for a scrape to happen. - time.Sleep(20 * time.Millisecond) - if testTarget.status.LastScrape().IsZero() { - t.Errorf("Scrape hasn't occured.") - } - - testTarget.StopScraper() - // Wait for it to take effect. - time.Sleep(20 * time.Millisecond) - last := testTarget.status.LastScrape() - // Enough time for a scrape to happen. - time.Sleep(20 * time.Millisecond) - if testTarget.status.LastScrape() != last { - t.Errorf("Scrape occured after it was stopped.") - } -} - -func BenchmarkScrape(b *testing.B) { - server := httptest.NewServer( - http.HandlerFunc( - func(w http.ResponseWriter, r *http.Request) { - w.Header().Set("Content-Type", `text/plain; version=0.0.4`) - w.Write([]byte("test_metric{foo=\"bar\"} 123.456\n")) - }, - ), - ) - defer server.Close() - - testTarget := newTestTarget(server.URL, time.Second, model.LabelSet{"dings": "bums"}) - appender := nopAppender{} - - b.ResetTimer() - for i := 0; i < b.N; i++ { - if err := testTarget.scrape(appender); err != nil { - b.Fatal(err) - } - } -} - func TestURLParams(t *testing.T) { server := httptest.NewServer( http.HandlerFunc( @@ -566,8 +229,7 @@ func TestURLParams(t *testing.T) { if err != nil { t.Fatal(err) } - app := &collectResultAppender{} - if err = target.scrape(app); err != nil { + if _, err = target.scrape(context.Background()); err != nil { t.Fatal(err) } } @@ -583,10 +245,8 @@ func newTestTarget(targetURL string, deadline time.Duration, labels model.LabelS ScrapeInterval: model.Duration(time.Millisecond), ScrapeTimeout: model.Duration(deadline), }, - labels: labels, - status: &TargetStatus{}, - scraperStopping: make(chan struct{}), - scraperStopped: make(chan struct{}), + labels: labels, + status: &TargetStatus{}, } } From 2bb8ef99d1577082793cb32f45055dd128444faf Mon Sep 17 00:00:00 2001 From: Fabian Reinartz Date: Tue, 23 Feb 2016 10:58:16 +0100 Subject: [PATCH 09/23] Test scrape loop behavior. --- retrieval/scrape_test.go | 137 +++++++++++++++++++++++++++++++++++++++ 1 file changed, 137 insertions(+) create mode 100644 retrieval/scrape_test.go diff --git a/retrieval/scrape_test.go b/retrieval/scrape_test.go new file mode 100644 index 000000000..bcc4a5bd6 --- /dev/null +++ b/retrieval/scrape_test.go @@ -0,0 +1,137 @@ +// 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 retrieval + +import ( + "testing" + "time" + + "github.com/prometheus/common/model" + "golang.org/x/net/context" + + // "github.com/prometheus/prometheus/storage" +) + +func TestScrapeLoopRun(t *testing.T) { + var ( + signal = make(chan struct{}) + errc = make(chan error) + + scraper = &testScraper{} + app = &nopAppender{} + reportApp = &nopAppender{} + ) + defer close(signal) + + ctx, cancel := context.WithCancel(context.Background()) + sl := newScrapeLoop(ctx, scraper, app, reportApp) + + // The loop must terminate during the initial offset if the context + // is canceled. + scraper.offsetDur = time.Hour + + go func() { + sl.run(time.Second, time.Hour, errc) + signal <- struct{}{} + }() + + // Wait to make sure we are actually waiting on the offset. + time.Sleep(1 * time.Second) + + cancel() + select { + case <-signal: + case <-time.After(5 * time.Second): + t.Fatalf("Cancelation during initial offset failed") + case err := <-errc: + t.Fatalf("Unexpected error: %s", err) + } + + // The provided timeout must cause cancelation of the context passed down to the + // scraper. The scraper has to respect the context. + scraper.offsetDur = 0 + + block := make(chan struct{}) + scraper.scrapeFunc = func(ctx context.Context) (model.Samples, error) { + select { + case <-block: + case <-ctx.Done(): + return nil, ctx.Err() + } + return nil, nil + } + + ctx, cancel = context.WithCancel(context.Background()) + sl = newScrapeLoop(ctx, scraper, app, reportApp) + + go func() { + sl.run(time.Second, 100*time.Millisecond, errc) + signal <- struct{}{} + }() + + select { + case err := <-errc: + if err != context.DeadlineExceeded { + t.Fatalf("Expected timeout error but got: %s", err) + } + case <-time.After(3 * time.Second): + t.Fatalf("Expected timeout error but got none") + } + + // We already caught the timeout error and are certainly in the loop. + // Let the scrapes returns immediately to cause no further timeout errors + // and check whether canceling the parent context terminates the loop. + close(block) + cancel() + + select { + case <-signal: + // Loop terminated as expected. + case err := <-errc: + t.Fatalf("Unexpected error: %s", err) + case <-time.After(3 * time.Second): + t.Fatalf("Loop did not terminate on context cancelation") + } +} + +// testScraper implements the scraper interface and allows setting values +// returned by its methods. It also allows setting a custom scrape function. +type testScraper struct { + offsetDur time.Duration + + lastStart time.Time + lastDuration time.Duration + lastError error + + samples model.Samples + scrapeErr error + scrapeFunc func(context.Context) (model.Samples, error) +} + +func (ts *testScraper) offset(interval time.Duration) time.Duration { + return ts.offsetDur +} + +func (ts *testScraper) report(start time.Time, duration time.Duration, err error) { + ts.lastStart = start + ts.lastDuration = duration + ts.lastError = err +} + +func (ts *testScraper) scrape(ctx context.Context) (model.Samples, error) { + if ts.scrapeFunc != nil { + return ts.scrapeFunc(ctx) + } + return ts.samples, ts.scrapeErr +} From 1a3253e8ede7fa55641ccb291bf99b08fcc97287 Mon Sep 17 00:00:00 2001 From: Fabian Reinartz Date: Thu, 25 Feb 2016 13:58:46 +0100 Subject: [PATCH 10/23] Make scrape time unambigious. This commit changes the scraper interface to accept a timestamp so the reported timestamp by the caller and the timestamp attached to samples does not differ. --- retrieval/scrape.go | 16 ++++++++-------- retrieval/scrape_test.go | 8 ++++---- retrieval/target.go | 6 ++---- retrieval/target_test.go | 4 ++-- 4 files changed, 16 insertions(+), 18 deletions(-) diff --git a/retrieval/scrape.go b/retrieval/scrape.go index bbf1e113c..6ec7084ed 100644 --- a/retrieval/scrape.go +++ b/retrieval/scrape.go @@ -66,13 +66,6 @@ func init() { prometheus.MustRegister(targetSkippedScrapes) } -// A scraper retrieves samples and accepts a status report at the end. -type scraper interface { - scrape(context.Context) (model.Samples, error) - report(start time.Time, dur time.Duration, err error) - offset(interval time.Duration) time.Duration -} - // scrapePool manages scrapes for sets of targets. type scrapePool struct { appender storage.SampleAppender @@ -178,6 +171,13 @@ func (sp *scrapePool) sync(tgroups map[string]map[model.Fingerprint]*Target) { sp.mtx.Unlock() } +// A scraper retrieves samples and accepts a status report at the end. +type scraper interface { + scrape(ctx context.Context, ts time.Time) (model.Samples, error) + report(start time.Time, dur time.Duration, err error) + offset(interval time.Duration) time.Duration +} + type loop interface { run(interval, timeout time.Duration, errc chan<- error) stop() @@ -239,7 +239,7 @@ func (sl *scrapeLoop) run(interval, timeout time.Duration, errc chan<- error) { float64(time.Since(last)) / float64(time.Second), // Sub-second precision. ) - samples, err := sl.scraper.scrape(scrapeCtx) + samples, err := sl.scraper.scrape(scrapeCtx, start) if err == nil { sl.append(samples) } else if errc != nil { diff --git a/retrieval/scrape_test.go b/retrieval/scrape_test.go index bcc4a5bd6..142bc13af 100644 --- a/retrieval/scrape_test.go +++ b/retrieval/scrape_test.go @@ -63,7 +63,7 @@ func TestScrapeLoopRun(t *testing.T) { scraper.offsetDur = 0 block := make(chan struct{}) - scraper.scrapeFunc = func(ctx context.Context) (model.Samples, error) { + scraper.scrapeFunc = func(ctx context.Context, ts time.Time) (model.Samples, error) { select { case <-block: case <-ctx.Done(): @@ -116,7 +116,7 @@ type testScraper struct { samples model.Samples scrapeErr error - scrapeFunc func(context.Context) (model.Samples, error) + scrapeFunc func(context.Context, time.Time) (model.Samples, error) } func (ts *testScraper) offset(interval time.Duration) time.Duration { @@ -129,9 +129,9 @@ func (ts *testScraper) report(start time.Time, duration time.Duration, err error ts.lastError = err } -func (ts *testScraper) scrape(ctx context.Context) (model.Samples, error) { +func (ts *testScraper) scrape(ctx context.Context, t time.Time) (model.Samples, error) { if ts.scrapeFunc != nil { - return ts.scrapeFunc(ctx) + return ts.scrapeFunc(ctx, t) } return ts.samples, ts.scrapeErr } diff --git a/retrieval/target.go b/retrieval/target.go index 48e73cac7..d341d98df 100644 --- a/retrieval/target.go +++ b/retrieval/target.go @@ -340,13 +340,11 @@ func (t *Target) InstanceIdentifier() string { const acceptHeader = `application/vnd.google.protobuf;proto=io.prometheus.client.MetricFamily;encoding=delimited;q=0.7,text/plain;version=0.0.4;q=0.3,application/json;schema="prometheus/telemetry";version=0.0.2;q=0.2,*/*;q=0.1` -func (t *Target) scrape(ctx context.Context) (model.Samples, error) { +func (t *Target) scrape(ctx context.Context, ts time.Time) (model.Samples, error) { t.RLock() client := t.httpClient t.RUnlock() - start := time.Now() - req, err := http.NewRequest("GET", t.URL().String(), nil) if err != nil { return nil, err @@ -370,7 +368,7 @@ func (t *Target) scrape(ctx context.Context) (model.Samples, error) { sdec := expfmt.SampleDecoder{ Dec: expfmt.NewDecoder(resp.Body, expfmt.ResponseFormat(resp.Header)), Opts: &expfmt.DecodeOptions{ - Timestamp: model.TimeFromUnixNano(start.UnixNano()), + Timestamp: model.TimeFromUnixNano(ts.UnixNano()), }, } diff --git a/retrieval/target_test.go b/retrieval/target_test.go index 2627d97f0..54a89b229 100644 --- a/retrieval/target_test.go +++ b/retrieval/target_test.go @@ -181,7 +181,7 @@ func TestTargetScrape404(t *testing.T) { testTarget := newTestTarget(server.URL, time.Second, model.LabelSet{}) want := errors.New("server returned HTTP status 404 Not Found") - _, got := testTarget.scrape(context.Background()) + _, got := testTarget.scrape(context.Background(), time.Now()) if got == nil || want.Error() != got.Error() { t.Fatalf("want err %q, got %q", want, got) } @@ -229,7 +229,7 @@ func TestURLParams(t *testing.T) { if err != nil { t.Fatal(err) } - if _, err = target.scrape(context.Background()); err != nil { + if _, err = target.scrape(context.Background(), time.Now()); err != nil { t.Fatal(err) } } From fbe251c2df399b89bf87a11352ab26aff676d8d0 Mon Sep 17 00:00:00 2001 From: Fabian Reinartz Date: Thu, 25 Feb 2016 14:05:42 +0100 Subject: [PATCH 11/23] Fix scrape interval length calculation --- retrieval/scrape.go | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/retrieval/scrape.go b/retrieval/scrape.go index 6ec7084ed..507dab878 100644 --- a/retrieval/scrape.go +++ b/retrieval/scrape.go @@ -235,9 +235,12 @@ func (sl *scrapeLoop) run(interval, timeout time.Duration, errc chan<- error) { scrapeCtx, _ = context.WithTimeout(sl.ctx, timeout) ) - targetIntervalLength.WithLabelValues(interval.String()).Observe( - float64(time.Since(last)) / float64(time.Second), // Sub-second precision. - ) + // Only record after the first scrape. + if !last.IsZero() { + targetIntervalLength.WithLabelValues(interval.String()).Observe( + float64(time.Since(last)) / float64(time.Second), // Sub-second precision. + ) + } samples, err := sl.scraper.scrape(scrapeCtx, start) if err == nil { From 775316f8d2fffe840b9fa7403a16eb40cd60bbea Mon Sep 17 00:00:00 2001 From: Fabian Reinartz Date: Tue, 23 Feb 2016 11:56:09 +0100 Subject: [PATCH 12/23] Move appender construction from Target to scrapePool --- retrieval/scrape.go | 41 +++++++++++++++++++- retrieval/scrape_test.go | 79 +++++++++++++++++++++++++++++++++++++- retrieval/target.go | 41 +------------------- retrieval/target_test.go | 76 ------------------------------------ retrieval/targetmanager.go | 2 +- 5 files changed, 120 insertions(+), 119 deletions(-) diff --git a/retrieval/scrape.go b/retrieval/scrape.go index 507dab878..9c52e6e1e 100644 --- a/retrieval/scrape.go +++ b/retrieval/scrape.go @@ -23,6 +23,7 @@ import ( "github.com/prometheus/common/model" "golang.org/x/net/context" + "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/storage/local" ) @@ -69,6 +70,7 @@ func init() { // scrapePool manages scrapes for sets of targets. type scrapePool struct { appender storage.SampleAppender + config *config.ScrapeConfig ctx context.Context mtx sync.RWMutex @@ -77,9 +79,10 @@ type scrapePool struct { targets map[model.Fingerprint]loop } -func newScrapePool(app storage.SampleAppender) *scrapePool { +func newScrapePool(cfg *config.ScrapeConfig, app storage.SampleAppender) *scrapePool { return &scrapePool{ appender: app, + config: cfg, tgroups: map[string]map[model.Fingerprint]*Target{}, } } @@ -104,6 +107,40 @@ func (sp *scrapePool) stop() { wg.Wait() } +// sampleAppender returns an appender for ingested samples from the target. +func (sp *scrapePool) sampleAppender(target *Target) storage.SampleAppender { + app := sp.appender + // The relabelAppender has to be inside the label-modifying appenders + // so the relabeling rules are applied to the correct label set. + if mrc := sp.config.MetricRelabelConfigs; len(mrc) > 0 { + app = relabelAppender{ + SampleAppender: app, + relabelings: mrc, + } + } + + if sp.config.HonorLabels { + app = honorLabelsAppender{ + SampleAppender: app, + labels: target.Labels(), + } + } else { + app = ruleLabelsAppender{ + SampleAppender: app, + labels: target.Labels(), + } + } + return app +} + +// reportAppender returns an appender for reporting samples for the target. +func (sp *scrapePool) reportAppender(target *Target) storage.SampleAppender { + return ruleLabelsAppender{ + SampleAppender: sp.appender, + labels: target.Labels(), + } +} + func (sp *scrapePool) sync(tgroups map[string]map[model.Fingerprint]*Target) { sp.mtx.Lock() @@ -127,7 +164,7 @@ func (sp *scrapePool) sync(tgroups map[string]map[model.Fingerprint]*Target) { } else { newTargets[fp] = tnew - tnew.scrapeLoop = newScrapeLoop(sp.ctx, tnew, tnew.wrapAppender(sp.appender), tnew.wrapReportingAppender(sp.appender)) + tnew.scrapeLoop = newScrapeLoop(sp.ctx, tnew, sp.sampleAppender(tnew), sp.reportAppender(tnew)) go tnew.scrapeLoop.run(tnew.interval(), tnew.timeout(), nil) } } diff --git a/retrieval/scrape_test.go b/retrieval/scrape_test.go index 142bc13af..ee6dfbc20 100644 --- a/retrieval/scrape_test.go +++ b/retrieval/scrape_test.go @@ -20,9 +20,86 @@ import ( "github.com/prometheus/common/model" "golang.org/x/net/context" - // "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/config" ) +func TestScrapePoolReportAppender(t *testing.T) { + cfg := &config.ScrapeConfig{ + MetricRelabelConfigs: []*config.RelabelConfig{ + {}, {}, {}, + }, + } + target := newTestTarget("example.com:80", 10*time.Millisecond, nil) + app := &nopAppender{} + + sp := newScrapePool(cfg, app) + + cfg.HonorLabels = false + wrapped := sp.reportAppender(target) + + rl, ok := wrapped.(ruleLabelsAppender) + if !ok { + t.Fatalf("Expected ruleLabelsAppender but got %T", wrapped) + } + if rl.SampleAppender != app { + t.Fatalf("Expected base appender but got %T", rl.SampleAppender) + } + + cfg.HonorLabels = true + wrapped = sp.reportAppender(target) + + hl, ok := wrapped.(ruleLabelsAppender) + if !ok { + t.Fatalf("Expected ruleLabelsAppender but got %T", wrapped) + } + if hl.SampleAppender != app { + t.Fatalf("Expected base appender but got %T", hl.SampleAppender) + } +} + +func TestScrapePoolSampleAppender(t *testing.T) { + cfg := &config.ScrapeConfig{ + MetricRelabelConfigs: []*config.RelabelConfig{ + {}, {}, {}, + }, + } + + target := newTestTarget("example.com:80", 10*time.Millisecond, nil) + app := &nopAppender{} + + sp := newScrapePool(cfg, app) + + cfg.HonorLabels = false + wrapped := sp.sampleAppender(target) + + rl, ok := wrapped.(ruleLabelsAppender) + if !ok { + t.Fatalf("Expected ruleLabelsAppender but got %T", wrapped) + } + re, ok := rl.SampleAppender.(relabelAppender) + if !ok { + t.Fatalf("Expected relabelAppender but got %T", rl.SampleAppender) + } + if re.SampleAppender != app { + t.Fatalf("Expected base appender but got %T", re.SampleAppender) + } + + cfg.HonorLabels = true + wrapped = sp.sampleAppender(target) + + hl, ok := wrapped.(honorLabelsAppender) + if !ok { + t.Fatalf("Expected honorLabelsAppender but got %T", wrapped) + } + re, ok = hl.SampleAppender.(relabelAppender) + if !ok { + t.Fatalf("Expected relabelAppender but got %T", hl.SampleAppender) + } + if re.SampleAppender != app { + t.Fatalf("Expected base appender but got %T", re.SampleAppender) + } +} + func TestScrapeLoopRun(t *testing.T) { var ( signal = make(chan struct{}) diff --git a/retrieval/target.go b/retrieval/target.go index d341d98df..0074b6e2d 100644 --- a/retrieval/target.go +++ b/retrieval/target.go @@ -121,13 +121,12 @@ type Target struct { // The status object for the target. It is only set once on initialization. status *TargetStatus - scrapeLoop *scrapeLoop + scrapeLoop *scrapeLoop + scrapeConfig *config.ScrapeConfig // Mutex protects the members below. sync.RWMutex - scrapeConfig *config.ScrapeConfig - // Labels before any processing. metaLabels model.LabelSet // Any labels that are added to this target and its metrics. @@ -265,42 +264,6 @@ func (t *Target) path() string { return string(t.labels[model.MetricsPathLabel]) } -// wrapAppender wraps a SampleAppender for samples ingested from the target. -// RLock must be acquired by the caller. -func (t *Target) wrapAppender(app storage.SampleAppender) storage.SampleAppender { - // The relabelAppender has to be inside the label-modifying appenders - // so the relabeling rules are applied to the correct label set. - if mrc := t.scrapeConfig.MetricRelabelConfigs; len(mrc) > 0 { - app = relabelAppender{ - SampleAppender: app, - relabelings: mrc, - } - } - - if t.scrapeConfig.HonorLabels { - app = honorLabelsAppender{ - SampleAppender: app, - labels: t.unlockedLabels(), - } - } else { - app = ruleLabelsAppender{ - SampleAppender: app, - labels: t.unlockedLabels(), - } - } - return app -} - -// wrapReportingAppender wraps an appender for target status report samples. -// It ignores any relabeling rules set for the target. -// RLock must not be acquired by the caller. -func (t *Target) wrapReportingAppender(app storage.SampleAppender) storage.SampleAppender { - return ruleLabelsAppender{ - SampleAppender: app, - labels: t.Labels(), - } -} - // URL returns a copy of the target's URL. func (t *Target) URL() *url.URL { t.RLock() diff --git a/retrieval/target_test.go b/retrieval/target_test.go index 54a89b229..a6c77d232 100644 --- a/retrieval/target_test.go +++ b/retrieval/target_test.go @@ -92,82 +92,6 @@ func TestTargetOffset(t *testing.T) { } } -func TestTargetWrapReportingAppender(t *testing.T) { - cfg := &config.ScrapeConfig{ - MetricRelabelConfigs: []*config.RelabelConfig{ - {}, {}, {}, - }, - } - - target := newTestTarget("example.com:80", 10*time.Millisecond, nil) - target.scrapeConfig = cfg - app := &nopAppender{} - - cfg.HonorLabels = false - wrapped := target.wrapReportingAppender(app) - - rl, ok := wrapped.(ruleLabelsAppender) - if !ok { - t.Fatalf("Expected ruleLabelsAppender but got %T", wrapped) - } - if rl.SampleAppender != app { - t.Fatalf("Expected base appender but got %T", rl.SampleAppender) - } - - cfg.HonorLabels = true - wrapped = target.wrapReportingAppender(app) - - hl, ok := wrapped.(ruleLabelsAppender) - if !ok { - t.Fatalf("Expected ruleLabelsAppender but got %T", wrapped) - } - if hl.SampleAppender != app { - t.Fatalf("Expected base appender but got %T", hl.SampleAppender) - } -} - -func TestTargetWrapAppender(t *testing.T) { - cfg := &config.ScrapeConfig{ - MetricRelabelConfigs: []*config.RelabelConfig{ - {}, {}, {}, - }, - } - - target := newTestTarget("example.com:80", 10*time.Millisecond, nil) - target.scrapeConfig = cfg - app := &nopAppender{} - - cfg.HonorLabels = false - wrapped := target.wrapAppender(app) - - rl, ok := wrapped.(ruleLabelsAppender) - if !ok { - t.Fatalf("Expected ruleLabelsAppender but got %T", wrapped) - } - re, ok := rl.SampleAppender.(relabelAppender) - if !ok { - t.Fatalf("Expected relabelAppender but got %T", rl.SampleAppender) - } - if re.SampleAppender != app { - t.Fatalf("Expected base appender but got %T", re.SampleAppender) - } - - cfg.HonorLabels = true - wrapped = target.wrapAppender(app) - - hl, ok := wrapped.(honorLabelsAppender) - if !ok { - t.Fatalf("Expected honorLabelsAppender but got %T", wrapped) - } - re, ok = hl.SampleAppender.(relabelAppender) - if !ok { - t.Fatalf("Expected relabelAppender but got %T", hl.SampleAppender) - } - if re.SampleAppender != app { - t.Fatalf("Expected base appender but got %T", re.SampleAppender) - } -} - func TestTargetScrape404(t *testing.T) { server := httptest.NewServer( http.HandlerFunc( diff --git a/retrieval/targetmanager.go b/retrieval/targetmanager.go index bc750f4b2..9ffc0d621 100644 --- a/retrieval/targetmanager.go +++ b/retrieval/targetmanager.go @@ -184,7 +184,7 @@ type targetSet struct { func newTargetSet(cfg *config.ScrapeConfig, app storage.SampleAppender) *targetSet { ts := &targetSet{ tgroups: map[string]map[model.Fingerprint]*Target{}, - scrapePool: newScrapePool(app), + scrapePool: newScrapePool(cfg, app), syncCh: make(chan struct{}, 1), config: cfg, } From 02f635dc241f7de0cf99ee3af954bd2e764ec3a3 Mon Sep 17 00:00:00 2001 From: Fabian Reinartz Date: Tue, 23 Feb 2016 12:06:08 +0100 Subject: [PATCH 13/23] Remove interval/timeout from Target internals --- retrieval/scrape.go | 2 +- retrieval/target.go | 14 -------------- 2 files changed, 1 insertion(+), 15 deletions(-) diff --git a/retrieval/scrape.go b/retrieval/scrape.go index 9c52e6e1e..7f27d2d15 100644 --- a/retrieval/scrape.go +++ b/retrieval/scrape.go @@ -165,7 +165,7 @@ func (sp *scrapePool) sync(tgroups map[string]map[model.Fingerprint]*Target) { newTargets[fp] = tnew tnew.scrapeLoop = newScrapeLoop(sp.ctx, tnew, sp.sampleAppender(tnew), sp.reportAppender(tnew)) - go tnew.scrapeLoop.run(tnew.interval(), tnew.timeout(), nil) + go tnew.scrapeLoop.run(time.Duration(sp.config.ScrapeInterval), time.Duration(sp.config.ScrapeTimeout), nil) } } for fp, told := range prevTargets { diff --git a/retrieval/target.go b/retrieval/target.go index 0074b6e2d..81d3e1bd8 100644 --- a/retrieval/target.go +++ b/retrieval/target.go @@ -229,20 +229,6 @@ func (t *Target) offset(interval time.Duration) time.Duration { return time.Duration(next) } -func (t *Target) interval() time.Duration { - t.RLock() - defer t.RUnlock() - - return time.Duration(t.scrapeConfig.ScrapeInterval) -} - -func (t *Target) timeout() time.Duration { - t.RLock() - defer t.RUnlock() - - return time.Duration(t.scrapeConfig.ScrapeTimeout) -} - func (t *Target) scheme() string { t.RLock() defer t.RUnlock() From 84f74b9a840f375acbb95bb1259ba24a43269ca2 Mon Sep 17 00:00:00 2001 From: Fabian Reinartz Date: Tue, 23 Feb 2016 13:34:24 +0100 Subject: [PATCH 14/23] Apply new scrape config on reload. This commit updates a target set's scrape configuration on reload. This will cause all running scrape loops to be stopped and started again with new parameters. --- retrieval/scrape.go | 32 +++++++++++++++++++++++++++++--- retrieval/targetmanager.go | 10 ++++++++++ 2 files changed, 39 insertions(+), 3 deletions(-) diff --git a/retrieval/scrape.go b/retrieval/scrape.go index 7f27d2d15..5586671e1 100644 --- a/retrieval/scrape.go +++ b/retrieval/scrape.go @@ -107,6 +107,34 @@ func (sp *scrapePool) stop() { wg.Wait() } +func (sp *scrapePool) reload(cfg *config.ScrapeConfig) { + log.Debugln("reload scrapepool") + defer log.Debugln("reload done") + + sp.mtx.Lock() + defer sp.mtx.Unlock() + + sp.config = cfg + + var wg sync.WaitGroup + + for _, tgroup := range sp.tgroups { + for _, t := range tgroup { + wg.Add(1) + + go func(t *Target) { + t.scrapeLoop.stop() + + t.scrapeLoop = newScrapeLoop(sp.ctx, t, sp.sampleAppender(t), sp.reportAppender(t)) + go t.scrapeLoop.run(time.Duration(cfg.ScrapeInterval), time.Duration(cfg.ScrapeTimeout), nil) + wg.Done() + }(t) + } + } + + wg.Wait() +} + // sampleAppender returns an appender for ingested samples from the target. func (sp *scrapePool) sampleAppender(target *Target) storage.SampleAppender { app := sp.appender @@ -143,6 +171,7 @@ func (sp *scrapePool) reportAppender(target *Target) storage.SampleAppender { func (sp *scrapePool) sync(tgroups map[string]map[model.Fingerprint]*Target) { sp.mtx.Lock() + defer sp.mtx.Unlock() var ( wg sync.WaitGroup @@ -203,9 +232,6 @@ func (sp *scrapePool) sync(tgroups map[string]map[model.Fingerprint]*Target) { // may be active and tries to insert. The old scraper that didn't terminate yet could still // be inserting a previous sample set. wg.Wait() - - // TODO(fabxc): maybe this can be released earlier with subsequent refactoring. - sp.mtx.Unlock() } // A scraper retrieves samples and accepts a status report at the end. diff --git a/retrieval/targetmanager.go b/retrieval/targetmanager.go index 9ffc0d621..474dbaf86 100644 --- a/retrieval/targetmanager.go +++ b/retrieval/targetmanager.go @@ -117,6 +117,8 @@ func (tm *TargetManager) reload() { ts.runScraping(tm.ctx) tm.wg.Done() }(ts) + } else { + ts.reload(scfg) } ts.runProviders(tm.ctx, providersFromConfig(scfg)) } @@ -203,6 +205,14 @@ func (ts *targetSet) cancel() { } } +func (ts *targetSet) reload(cfg *config.ScrapeConfig) { + ts.mtx.Lock() + ts.config = cfg + ts.mtx.Unlock() + + ts.scrapePool.reload(cfg) +} + func (ts *targetSet) runScraping(ctx context.Context) { ctx, ts.cancelScraping = context.WithCancel(ctx) From 76a8c6160d4659f32d42a923d7f556ee69352f5a Mon Sep 17 00:00:00 2001 From: Fabian Reinartz Date: Tue, 23 Feb 2016 14:37:25 +0100 Subject: [PATCH 15/23] Deduplicate targets in scrape pool. With this commit the scrape pool deduplicates incoming targets before scraping them. This way multiple target providers can produce the same target but it will be scraped only once. --- retrieval/scrape.go | 130 ++++++++++++++++++------------------- retrieval/targetmanager.go | 35 ++++++---- 2 files changed, 85 insertions(+), 80 deletions(-) diff --git a/retrieval/scrape.go b/retrieval/scrape.go index 5586671e1..d23d38ff2 100644 --- a/retrieval/scrape.go +++ b/retrieval/scrape.go @@ -72,64 +72,72 @@ type scrapePool struct { appender storage.SampleAppender config *config.ScrapeConfig - ctx context.Context - mtx sync.RWMutex - tgroups map[string]map[model.Fingerprint]*Target + ctx context.Context - targets map[model.Fingerprint]loop + mtx sync.RWMutex + targets map[model.Fingerprint]*Target + loops map[model.Fingerprint]loop } func newScrapePool(cfg *config.ScrapeConfig, app storage.SampleAppender) *scrapePool { return &scrapePool{ appender: app, config: cfg, - tgroups: map[string]map[model.Fingerprint]*Target{}, + targets: map[model.Fingerprint]*Target{}, + loops: map[model.Fingerprint]loop{}, } } +// stop terminates all scrape loops and returns after they all terminated. +// A stopped scrape pool must not be used again. func (sp *scrapePool) stop() { var wg sync.WaitGroup sp.mtx.RLock() - for _, tgroup := range sp.tgroups { - for _, t := range tgroup { - wg.Add(1) + for _, l := range sp.loops { + wg.Add(1) - go func(t *Target) { - t.scrapeLoop.stop() - wg.Done() - }(t) - } + go func(l loop) { + l.stop() + wg.Done() + }(l) } sp.mtx.RUnlock() wg.Wait() } +// reload the scrape pool with the given scrape configuration. The target state is preserved +// but all scrape loops are restarted with the new scrape configuration. +// This method returns after all scrape loops that were stopped have fully terminated. func (sp *scrapePool) reload(cfg *config.ScrapeConfig) { - log.Debugln("reload scrapepool") - defer log.Debugln("reload done") - sp.mtx.Lock() defer sp.mtx.Unlock() sp.config = cfg - var wg sync.WaitGroup + var ( + wg sync.WaitGroup + interval = time.Duration(sp.config.ScrapeInterval) + timeout = time.Duration(sp.config.ScrapeTimeout) + ) - for _, tgroup := range sp.tgroups { - for _, t := range tgroup { - wg.Add(1) + for fp, oldLoop := range sp.loops { + var ( + t = sp.targets[fp] + newLoop = newScrapeLoop(sp.ctx, t, sp.sampleAppender(t), sp.reportAppender(t)) + ) + wg.Add(1) - go func(t *Target) { - t.scrapeLoop.stop() + go func(oldLoop, newLoop loop) { + oldLoop.stop() + wg.Done() - t.scrapeLoop = newScrapeLoop(sp.ctx, t, sp.sampleAppender(t), sp.reportAppender(t)) - go t.scrapeLoop.run(time.Duration(cfg.ScrapeInterval), time.Duration(cfg.ScrapeTimeout), nil) - wg.Done() - }(t) - } + go newLoop.run(interval, timeout, nil) + }(oldLoop, newLoop) + + sp.loops[fp] = newLoop } wg.Wait() @@ -169,64 +177,49 @@ func (sp *scrapePool) reportAppender(target *Target) storage.SampleAppender { } } -func (sp *scrapePool) sync(tgroups map[string]map[model.Fingerprint]*Target) { +// sync takes a list of potentially duplicated targets, deduplicates them, starts +// scrape loops for new targets, and stops scrape loops for disappeared targets. +// It returns after all stopped scrape loops terminated. +func (sp *scrapePool) sync(targets []*Target) { sp.mtx.Lock() defer sp.mtx.Unlock() var ( - wg sync.WaitGroup - newTgroups = map[string]map[model.Fingerprint]*Target{} + fingerprints = map[model.Fingerprint]struct{}{} + interval = time.Duration(sp.config.ScrapeInterval) + timeout = time.Duration(sp.config.ScrapeTimeout) ) - for source, targets := range tgroups { - var ( - prevTargets = sp.tgroups[source] - newTargets = map[model.Fingerprint]*Target{} - ) - newTgroups[source] = newTargets + for _, t := range targets { + fp := t.fingerprint() + fingerprints[fp] = struct{}{} - for fp, tnew := range targets { - // If the same target existed before, we let it run and replace - // the new one with it. - if told, ok := prevTargets[fp]; ok { - newTargets[fp] = told - } else { - newTargets[fp] = tnew + if _, ok := sp.targets[fp]; !ok { + l := newScrapeLoop(sp.ctx, t, sp.sampleAppender(t), sp.reportAppender(t)) - tnew.scrapeLoop = newScrapeLoop(sp.ctx, tnew, sp.sampleAppender(tnew), sp.reportAppender(tnew)) - go tnew.scrapeLoop.run(time.Duration(sp.config.ScrapeInterval), time.Duration(sp.config.ScrapeTimeout), nil) - } - } - for fp, told := range prevTargets { - // A previous target is no longer in the group. - if _, ok := targets[fp]; !ok { - wg.Add(1) + sp.targets[fp] = t + sp.loops[fp] = l - go func(told *Target) { - told.scrapeLoop.stop() - wg.Done() - }(told) - } + go l.run(interval, timeout, nil) } } - // Stop scrapers for target groups that disappeared completely. - for source, targets := range sp.tgroups { - if _, ok := tgroups[source]; ok { - continue - } - for _, told := range targets { + var wg sync.WaitGroup + + // Stop and remove old targets and scraper loops. + for fp := range sp.targets { + if _, ok := fingerprints[fp]; !ok { wg.Add(1) - - go func(told *Target) { - told.scrapeLoop.stop() + go func(l loop) { + l.stop() wg.Done() - }(told) + }(sp.loops[fp]) + + delete(sp.loops, fp) + delete(sp.targets, fp) } } - sp.tgroups = newTgroups - // 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 // may be active and tries to insert. The old scraper that didn't terminate yet could still @@ -241,6 +234,7 @@ type scraper interface { offset(interval time.Duration) time.Duration } +// A loop can run and be stopped again. It must be reused after it was stopped. type loop interface { run(interval, timeout time.Duration, errc chan<- error) stop() diff --git a/retrieval/targetmanager.go b/retrieval/targetmanager.go index 474dbaf86..d0f0960a1 100644 --- a/retrieval/targetmanager.go +++ b/retrieval/targetmanager.go @@ -142,12 +142,14 @@ func (tm *TargetManager) Pools() map[string][]*Target { // TODO(fabxc): this is just a hack to maintain compatibility for now. for _, ps := range tm.targetSets { - for _, ts := range ps.scrapePool.tgroups { - for _, t := range ts { - job := string(t.Labels()[model.JobLabel]) - pools[job] = append(pools[job], t) - } + ps.scrapePool.mtx.RLock() + + for _, t := range ps.scrapePool.targets { + job := string(t.Labels()[model.JobLabel]) + pools[job] = append(pools[job], t) } + + ps.scrapePool.mtx.RUnlock() } return pools } @@ -168,10 +170,12 @@ func (tm *TargetManager) ApplyConfig(cfg *config.Config) bool { } // targetSet holds several TargetProviders for which the same scrape configuration -// is used. It runs the target providers and starts and stops scrapers as it -// receives target updates. +// is used. It maintains target groups from all given providers and sync them +// to a scrape pool. type targetSet struct { - mtx sync.RWMutex + mtx sync.RWMutex + + // Sets of targets by a source string that is unique across target providers. tgroups map[string]map[model.Fingerprint]*Target providers map[string]TargetProvider @@ -231,7 +235,9 @@ Loop: case <-ctx.Done(): break Loop case <-ts.syncCh: + ts.mtx.RLock() ts.sync() + ts.mtx.RUnlock() } } @@ -241,9 +247,13 @@ Loop: } 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) + targets := []*Target{} + for _, tgroup := range ts.tgroups { + for _, t := range tgroup { + targets = append(targets, t) + } + } + ts.scrapePool.sync(targets) } func (ts *targetSet) runProviders(ctx context.Context, providers map[string]TargetProvider) { @@ -308,8 +318,9 @@ func (ts *targetSet) runProviders(ctx context.Context, providers map[string]Targ go prov.Run(ctx, updates) } + // We wait for a full initial set of target groups before releasing the mutex + // to ensure the initial sync is complete and there are no races with subsequent updates. wg.Wait() - ts.sync() } From 9bea27ae8a7425d56e17ad19d44b84a7b6cf43cf Mon Sep 17 00:00:00 2001 From: Fabian Reinartz Date: Sun, 28 Feb 2016 09:51:02 +0100 Subject: [PATCH 16/23] Add scraping tests --- retrieval/scrape.go | 126 ++++++++++---------- retrieval/scrape_test.go | 241 +++++++++++++++++++++++++++++++++++++++ 2 files changed, 306 insertions(+), 61 deletions(-) diff --git a/retrieval/scrape.go b/retrieval/scrape.go index d23d38ff2..34326c13e 100644 --- a/retrieval/scrape.go +++ b/retrieval/scrape.go @@ -74,9 +74,14 @@ type scrapePool struct { ctx context.Context + // Targets and loops must always be synchronized to have the same + // set of fingerprints. mtx sync.RWMutex targets map[model.Fingerprint]*Target loops map[model.Fingerprint]loop + + // Constructor for new scrape loops. This is settable for testing convenience. + newLoop func(context.Context, scraper, storage.SampleAppender, storage.SampleAppender) loop } func newScrapePool(cfg *config.ScrapeConfig, app storage.SampleAppender) *scrapePool { @@ -85,25 +90,28 @@ func newScrapePool(cfg *config.ScrapeConfig, app storage.SampleAppender) *scrape config: cfg, targets: map[model.Fingerprint]*Target{}, loops: map[model.Fingerprint]loop{}, + newLoop: newScrapeLoop, } } // stop terminates all scrape loops and returns after they all terminated. -// A stopped scrape pool must not be used again. func (sp *scrapePool) stop() { var wg sync.WaitGroup - sp.mtx.RLock() + sp.mtx.Lock() + defer sp.mtx.Unlock() - for _, l := range sp.loops { + for fp, l := range sp.loops { wg.Add(1) go func(l loop) { l.stop() wg.Done() }(l) + + delete(sp.loops, fp) + delete(sp.targets, fp) } - sp.mtx.RUnlock() wg.Wait() } @@ -126,7 +134,7 @@ func (sp *scrapePool) reload(cfg *config.ScrapeConfig) { for fp, oldLoop := range sp.loops { var ( t = sp.targets[fp] - newLoop = newScrapeLoop(sp.ctx, t, sp.sampleAppender(t), sp.reportAppender(t)) + newLoop = sp.newLoop(sp.ctx, t, sp.sampleAppender(t), sp.reportAppender(t)) ) wg.Add(1) @@ -143,6 +151,56 @@ func (sp *scrapePool) reload(cfg *config.ScrapeConfig) { wg.Wait() } +// sync takes a list of potentially duplicated targets, deduplicates them, starts +// scrape loops for new targets, and stops scrape loops for disappeared targets. +// It returns after all stopped scrape loops terminated. +func (sp *scrapePool) sync(targets []*Target) { + sp.mtx.Lock() + defer sp.mtx.Unlock() + + var ( + fingerprints = map[model.Fingerprint]struct{}{} + interval = time.Duration(sp.config.ScrapeInterval) + timeout = time.Duration(sp.config.ScrapeTimeout) + ) + + for _, t := range targets { + fp := t.fingerprint() + fingerprints[fp] = struct{}{} + + if _, ok := sp.targets[fp]; !ok { + l := sp.newLoop(sp.ctx, t, sp.sampleAppender(t), sp.reportAppender(t)) + + sp.targets[fp] = t + sp.loops[fp] = l + + go l.run(interval, timeout, nil) + } + } + + var wg sync.WaitGroup + + // Stop and remove old targets and scraper loops. + for fp := range sp.targets { + if _, ok := fingerprints[fp]; !ok { + wg.Add(1) + go func(l loop) { + l.stop() + wg.Done() + }(sp.loops[fp]) + + delete(sp.loops, fp) + delete(sp.targets, fp) + } + } + + // 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 + // may be active and tries to insert. The old scraper that didn't terminate yet could still + // be inserting a previous sample set. + wg.Wait() +} + // sampleAppender returns an appender for ingested samples from the target. func (sp *scrapePool) sampleAppender(target *Target) storage.SampleAppender { app := sp.appender @@ -177,56 +235,6 @@ func (sp *scrapePool) reportAppender(target *Target) storage.SampleAppender { } } -// sync takes a list of potentially duplicated targets, deduplicates them, starts -// scrape loops for new targets, and stops scrape loops for disappeared targets. -// It returns after all stopped scrape loops terminated. -func (sp *scrapePool) sync(targets []*Target) { - sp.mtx.Lock() - defer sp.mtx.Unlock() - - var ( - fingerprints = map[model.Fingerprint]struct{}{} - interval = time.Duration(sp.config.ScrapeInterval) - timeout = time.Duration(sp.config.ScrapeTimeout) - ) - - for _, t := range targets { - fp := t.fingerprint() - fingerprints[fp] = struct{}{} - - if _, ok := sp.targets[fp]; !ok { - l := newScrapeLoop(sp.ctx, t, sp.sampleAppender(t), sp.reportAppender(t)) - - sp.targets[fp] = t - sp.loops[fp] = l - - go l.run(interval, timeout, nil) - } - } - - var wg sync.WaitGroup - - // Stop and remove old targets and scraper loops. - for fp := range sp.targets { - if _, ok := fingerprints[fp]; !ok { - wg.Add(1) - go func(l loop) { - l.stop() - wg.Done() - }(sp.loops[fp]) - - delete(sp.loops, fp) - delete(sp.targets, fp) - } - } - - // 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 - // may be active and tries to insert. The old scraper that didn't terminate yet could still - // be inserting a previous sample set. - wg.Wait() -} - // A scraper retrieves samples and accepts a status report at the end. type scraper interface { scrape(ctx context.Context, ts time.Time) (model.Samples, error) @@ -234,7 +242,7 @@ type scraper interface { offset(interval time.Duration) time.Duration } -// A loop can run and be stopped again. It must be reused after it was stopped. +// A loop can run and be stopped again. It must not be reused after it was stopped. type loop interface { run(interval, timeout time.Duration, errc chan<- error) stop() @@ -247,12 +255,11 @@ type scrapeLoop struct { reportAppender storage.SampleAppender done chan struct{} - mtx sync.RWMutex ctx context.Context cancel func() } -func newScrapeLoop(ctx context.Context, sc scraper, app, reportApp storage.SampleAppender) *scrapeLoop { +func newScrapeLoop(ctx context.Context, sc scraper, app, reportApp storage.SampleAppender) loop { sl := &scrapeLoop{ scraper: sc, appender: app, @@ -321,10 +328,7 @@ func (sl *scrapeLoop) run(interval, timeout time.Duration, errc chan<- error) { } func (sl *scrapeLoop) stop() { - sl.mtx.RLock() sl.cancel() - sl.mtx.RUnlock() - <-sl.done } diff --git a/retrieval/scrape_test.go b/retrieval/scrape_test.go index ee6dfbc20..f40bf9b9a 100644 --- a/retrieval/scrape_test.go +++ b/retrieval/scrape_test.go @@ -14,6 +14,9 @@ package retrieval import ( + "fmt" + "reflect" + "sync" "testing" "time" @@ -21,8 +24,199 @@ import ( "golang.org/x/net/context" "github.com/prometheus/prometheus/config" + "github.com/prometheus/prometheus/storage" ) +func TestNewScrapePool(t *testing.T) { + var ( + app = &nopAppender{} + cfg = &config.ScrapeConfig{} + sp = newScrapePool(cfg, app) + ) + + if a, ok := sp.appender.(*nopAppender); !ok || a != app { + t.Fatalf("Wrong sample appender") + } + if sp.config != cfg { + t.Fatalf("Wrong scrape config") + } + if sp.newLoop == nil { + t.Fatalf("newLoop function not initialized") + } +} + +type testLoop struct { + startFunc func(interval, timeout time.Duration, errc chan<- error) + stopFunc func() +} + +func (l *testLoop) run(interval, timeout time.Duration, errc chan<- error) { + l.startFunc(interval, timeout, errc) +} + +func (l *testLoop) stop() { + l.stopFunc() +} + +func TestScrapePoolStop(t *testing.T) { + sp := &scrapePool{ + targets: map[model.Fingerprint]*Target{}, + loops: map[model.Fingerprint]loop{}, + } + var mtx sync.Mutex + stopped := map[model.Fingerprint]bool{} + numTargets := 20 + + // Stopping the scrape pool must call stop() on all scrape loops, + // clean them and the respective targets up. It must wait until each loop's + // stop function returned before returning itself. + + for i := 0; i < numTargets; i++ { + t := &Target{ + labels: model.LabelSet{ + model.AddressLabel: model.LabelValue(fmt.Sprintf("example.com:%d", i)), + }, + } + l := &testLoop{} + l.stopFunc = func() { + time.Sleep(time.Duration(i*20) * time.Millisecond) + + mtx.Lock() + stopped[t.fingerprint()] = true + mtx.Unlock() + } + + sp.targets[t.fingerprint()] = t + sp.loops[t.fingerprint()] = l + } + + done := make(chan struct{}) + stopTime := time.Now() + + go func() { + sp.stop() + close(done) + }() + + select { + case <-time.After(5 * time.Second): + t.Fatalf("scrapeLoop.stop() did not return as expected") + case <-done: + // This should have taken at least as long as the last target slept. + if time.Since(stopTime) < time.Duration(numTargets*20)*time.Millisecond { + t.Fatalf("scrapeLoop.stop() exited before all targets stopped") + } + } + + mtx.Lock() + if len(stopped) != numTargets { + t.Fatalf("Expected 20 stopped loops, got %d", len(stopped)) + } + mtx.Unlock() + + if len(sp.targets) > 0 { + t.Fatalf("Targets were not cleared on stopping: %d left", len(sp.targets)) + } + if len(sp.loops) > 0 { + t.Fatalf("Loops were not cleared on stopping: %d left", len(sp.loops)) + } +} + +func TestScrapePoolReload(t *testing.T) { + var mtx sync.Mutex + numTargets := 20 + + stopped := map[model.Fingerprint]bool{} + + reloadCfg := &config.ScrapeConfig{ + ScrapeInterval: model.Duration(3 * time.Second), + ScrapeTimeout: model.Duration(2 * time.Second), + } + // On starting to run, new loops created on reload check whether their preceeding + // equivalents have been stopped. + newLoop := func(ctx context.Context, s scraper, app, reportApp storage.SampleAppender) loop { + l := &testLoop{} + l.startFunc = func(interval, timeout time.Duration, errc chan<- error) { + if interval != 3*time.Second { + t.Errorf("Expected scrape interval %d but got %d", 3*time.Second, interval) + } + if timeout != 2*time.Second { + t.Errorf("Expected scrape timeout %d but got %d", 2*time.Second, timeout) + } + mtx.Lock() + if !stopped[s.(*Target).fingerprint()] { + t.Errorf("Scrape loop for %v not stopped yet", s.(*Target)) + } + mtx.Unlock() + } + return l + } + sp := &scrapePool{ + targets: map[model.Fingerprint]*Target{}, + loops: map[model.Fingerprint]loop{}, + newLoop: newLoop, + } + + // Reloading a scrape pool with a new scrape configuration must stop all scrape + // loops and start new ones. A new loop must not be started before the preceeding + // one terminated. + + for i := 0; i < numTargets; i++ { + t := &Target{ + labels: model.LabelSet{ + model.AddressLabel: model.LabelValue(fmt.Sprintf("example.com:%d", i)), + }, + } + l := &testLoop{} + l.stopFunc = func() { + time.Sleep(time.Duration(i*20) * time.Millisecond) + + mtx.Lock() + stopped[t.fingerprint()] = true + mtx.Unlock() + } + + sp.targets[t.fingerprint()] = t + sp.loops[t.fingerprint()] = l + } + done := make(chan struct{}) + + beforeTargets := map[model.Fingerprint]*Target{} + for fp, t := range sp.targets { + beforeTargets[fp] = t + } + + reloadTime := time.Now() + + go func() { + sp.reload(reloadCfg) + close(done) + }() + + select { + case <-time.After(5 * time.Second): + t.Fatalf("scrapeLoop.reload() did not return as expected") + case <-done: + // This should have taken at least as long as the last target slept. + if time.Since(reloadTime) < time.Duration(numTargets*20)*time.Millisecond { + t.Fatalf("scrapeLoop.stop() exited before all targets stopped") + } + } + + mtx.Lock() + if len(stopped) != numTargets { + t.Fatalf("Expected 20 stopped loops, got %d", stopped) + } + mtx.Unlock() + + if !reflect.DeepEqual(sp.targets, beforeTargets) { + t.Fatalf("Reloading affected target states unexpectedly") + } + if len(sp.loops) != numTargets { + t.Fatalf("Expected %d loops after reload but got %d", numTargets, len(sp.loops)) + } +} + func TestScrapePoolReportAppender(t *testing.T) { cfg := &config.ScrapeConfig{ MetricRelabelConfigs: []*config.RelabelConfig{ @@ -100,6 +294,53 @@ func TestScrapePoolSampleAppender(t *testing.T) { } } +func TestScrapeLoopStop(t *testing.T) { + scraper := &testScraper{} + sl := newScrapeLoop(context.Background(), scraper, nil, nil) + + // The scrape pool synchronizes on stopping scrape loops. However, new scrape + // loops are syarted asynchronously. Thus it's possible, that a loop is stopped + // again before having started properly. + // Stopping not-yet-started loops must block until the run method was called and exited. + // The run method must exit immediately. + + stopDone := make(chan struct{}) + go func() { + sl.stop() + close(stopDone) + }() + + select { + case <-stopDone: + t.Fatalf("Stopping terminated before run exited successfully") + case <-time.After(500 * time.Millisecond): + } + + // Running the scrape loop must exit before calling the scraper even once. + scraper.scrapeFunc = func(context.Context, time.Time) (model.Samples, error) { + t.Fatalf("scraper was called for terminated scrape loop") + return nil, nil + } + + runDone := make(chan struct{}) + go func() { + sl.run(0, 0, nil) + close(runDone) + }() + + select { + case <-runDone: + case <-time.After(1 * time.Second): + t.Fatalf("Running terminated scrape loop did not exit") + } + + select { + case <-stopDone: + case <-time.After(1 * time.Second): + t.Fatalf("Stopping did not terminate after running exited") + } +} + func TestScrapeLoopRun(t *testing.T) { var ( signal = make(chan struct{}) From 75681b691ac2834a8566df1ede6c7418394570b5 Mon Sep 17 00:00:00 2001 From: Fabian Reinartz Date: Sun, 28 Feb 2016 19:21:50 +0100 Subject: [PATCH 17/23] Extract HTTP client from Target. The HTTP client is the same across all targets with the same scrape configuration. Thus, this commit moves it into the scrape pool. --- retrieval/scrape.go | 79 +++++++++++++++++++++++-- retrieval/scrape_test.go | 4 +- retrieval/target.go | 72 +---------------------- retrieval/target_test.go | 115 +++++++++++++++---------------------- retrieval/targetmanager.go | 5 +- 5 files changed, 126 insertions(+), 149 deletions(-) diff --git a/retrieval/scrape.go b/retrieval/scrape.go index 34326c13e..a810edc72 100644 --- a/retrieval/scrape.go +++ b/retrieval/scrape.go @@ -15,13 +15,18 @@ package retrieval import ( "errors" + "fmt" + "io" + "net/http" "sync" "time" "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/common/expfmt" "github.com/prometheus/common/log" "github.com/prometheus/common/model" "golang.org/x/net/context" + "golang.org/x/net/context/ctxhttp" "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/storage" @@ -70,13 +75,14 @@ func init() { // scrapePool manages scrapes for sets of targets. type scrapePool struct { appender storage.SampleAppender - config *config.ScrapeConfig ctx context.Context + mtx sync.RWMutex + config *config.ScrapeConfig + client *http.Client // Targets and loops must always be synchronized to have the same // set of fingerprints. - mtx sync.RWMutex targets map[model.Fingerprint]*Target loops map[model.Fingerprint]loop @@ -85,9 +91,15 @@ type scrapePool struct { } func newScrapePool(cfg *config.ScrapeConfig, app storage.SampleAppender) *scrapePool { + client, err := newHTTPClient(cfg) + if err != nil { + // Any errors that could occur here should be caught during config validation. + log.Errorf("Error creating HTTP client for job %q: %s", cfg.JobName, err) + } return &scrapePool{ appender: app, config: cfg, + client: client, targets: map[model.Fingerprint]*Target{}, loops: map[model.Fingerprint]loop{}, newLoop: newScrapeLoop, @@ -123,7 +135,13 @@ func (sp *scrapePool) reload(cfg *config.ScrapeConfig) { sp.mtx.Lock() defer sp.mtx.Unlock() + client, err := newHTTPClient(cfg) + if err != nil { + // Any errors that could occur here should be caught during config validation. + log.Errorf("Error creating HTTP client for job %q: %s", cfg.JobName, err) + } sp.config = cfg + sp.client = client var ( wg sync.WaitGroup @@ -134,7 +152,8 @@ func (sp *scrapePool) reload(cfg *config.ScrapeConfig) { for fp, oldLoop := range sp.loops { var ( t = sp.targets[fp] - newLoop = sp.newLoop(sp.ctx, t, sp.sampleAppender(t), sp.reportAppender(t)) + s = &targetScraper{Target: t, client: sp.client} + newLoop = sp.newLoop(sp.ctx, s, sp.sampleAppender(t), sp.reportAppender(t)) ) wg.Add(1) @@ -169,7 +188,8 @@ func (sp *scrapePool) sync(targets []*Target) { fingerprints[fp] = struct{}{} if _, ok := sp.targets[fp]; !ok { - l := sp.newLoop(sp.ctx, t, sp.sampleAppender(t), sp.reportAppender(t)) + s := &targetScraper{Target: t, client: sp.client} + l := sp.newLoop(sp.ctx, s, sp.sampleAppender(t), sp.reportAppender(t)) sp.targets[fp] = t sp.loops[fp] = l @@ -242,6 +262,57 @@ type scraper interface { offset(interval time.Duration) time.Duration } +// targetScraper implements the scraper interface for a target. +type targetScraper struct { + *Target + client *http.Client +} + +const acceptHeader = `application/vnd.google.protobuf;proto=io.prometheus.client.MetricFamily;encoding=delimited;q=0.7,text/plain;version=0.0.4;q=0.3,application/json;schema="prometheus/telemetry";version=0.0.2;q=0.2,*/*;q=0.1` + +func (s *targetScraper) scrape(ctx context.Context, ts time.Time) (model.Samples, error) { + req, err := http.NewRequest("GET", s.URL().String(), nil) + if err != nil { + return nil, err + } + req.Header.Add("Accept", acceptHeader) + + resp, err := ctxhttp.Do(ctx, s.client, req) + if err != nil { + return nil, err + } + defer resp.Body.Close() + + if resp.StatusCode != http.StatusOK { + return nil, fmt.Errorf("server returned HTTP status %s", resp.Status) + } + + var ( + allSamples = make(model.Samples, 0, 200) + decSamples = make(model.Vector, 0, 50) + ) + sdec := expfmt.SampleDecoder{ + Dec: expfmt.NewDecoder(resp.Body, expfmt.ResponseFormat(resp.Header)), + Opts: &expfmt.DecodeOptions{ + Timestamp: model.TimeFromUnixNano(ts.UnixNano()), + }, + } + + for { + if err = sdec.Decode(&decSamples); err != nil { + break + } + allSamples = append(allSamples, decSamples...) + decSamples = decSamples[:0] + } + + if err == io.EOF { + // Set err to nil since it is used in the scrape health recording. + err = nil + } + return allSamples, err +} + // A loop can run and be stopped again. It must not be reused after it was stopped. type loop interface { run(interval, timeout time.Duration, errc chan<- error) diff --git a/retrieval/scrape_test.go b/retrieval/scrape_test.go index f40bf9b9a..d36919214 100644 --- a/retrieval/scrape_test.go +++ b/retrieval/scrape_test.go @@ -144,8 +144,8 @@ func TestScrapePoolReload(t *testing.T) { t.Errorf("Expected scrape timeout %d but got %d", 2*time.Second, timeout) } mtx.Lock() - if !stopped[s.(*Target).fingerprint()] { - t.Errorf("Scrape loop for %v not stopped yet", s.(*Target)) + if !stopped[s.(*targetScraper).fingerprint()] { + t.Errorf("Scrape loop for %v not stopped yet", s.(*targetScraper)) } mtx.Unlock() } diff --git a/retrieval/target.go b/retrieval/target.go index 81d3e1bd8..e48e28449 100644 --- a/retrieval/target.go +++ b/retrieval/target.go @@ -15,7 +15,6 @@ package retrieval import ( "fmt" - "io" "io/ioutil" "net/http" "net/url" @@ -23,10 +22,7 @@ import ( "sync" "time" - "github.com/prometheus/common/expfmt" "github.com/prometheus/common/model" - "golang.org/x/net/context" - "golang.org/x/net/context/ctxhttp" "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/storage" @@ -121,7 +117,6 @@ type Target struct { // The status object for the target. It is only set once on initialization. status *TargetStatus - scrapeLoop *scrapeLoop scrapeConfig *config.ScrapeConfig // Mutex protects the members below. @@ -131,25 +126,16 @@ type Target struct { metaLabels model.LabelSet // Any labels that are added to this target and its metrics. labels model.LabelSet - - // The HTTP client used to scrape the target's endpoint. - httpClient *http.Client } // NewTarget creates a reasonably configured target for querying. -func NewTarget(cfg *config.ScrapeConfig, labels, metaLabels model.LabelSet) (*Target, error) { - client, err := newHTTPClient(cfg) - if err != nil { - return nil, err - } - t := &Target{ +func NewTarget(cfg *config.ScrapeConfig, labels, metaLabels model.LabelSet) *Target { + return &Target{ status: &TargetStatus{}, scrapeConfig: cfg, labels: labels, metaLabels: metaLabels, - httpClient: client, } - return t, nil } // Status returns the status of the target. @@ -282,60 +268,6 @@ func (t *Target) URL() *url.URL { } } -// InstanceIdentifier returns the identifier for the target. -func (t *Target) InstanceIdentifier() string { - return t.host() -} - -const acceptHeader = `application/vnd.google.protobuf;proto=io.prometheus.client.MetricFamily;encoding=delimited;q=0.7,text/plain;version=0.0.4;q=0.3,application/json;schema="prometheus/telemetry";version=0.0.2;q=0.2,*/*;q=0.1` - -func (t *Target) scrape(ctx context.Context, ts time.Time) (model.Samples, error) { - t.RLock() - client := t.httpClient - t.RUnlock() - - req, err := http.NewRequest("GET", t.URL().String(), nil) - if err != nil { - return nil, err - } - req.Header.Add("Accept", acceptHeader) - - resp, err := ctxhttp.Do(ctx, client, req) - if err != nil { - return nil, err - } - defer resp.Body.Close() - - if resp.StatusCode != http.StatusOK { - return nil, fmt.Errorf("server returned HTTP status %s", resp.Status) - } - - var ( - allSamples = make(model.Samples, 0, 200) - decSamples = make(model.Vector, 0, 50) - ) - sdec := expfmt.SampleDecoder{ - Dec: expfmt.NewDecoder(resp.Body, expfmt.ResponseFormat(resp.Header)), - Opts: &expfmt.DecodeOptions{ - Timestamp: model.TimeFromUnixNano(ts.UnixNano()), - }, - } - - for { - if err = sdec.Decode(&decSamples); err != nil { - break - } - allSamples = append(allSamples, decSamples...) - decSamples = decSamples[:0] - } - - if err == io.EOF { - // Set err to nil since it is used in the scrape health recording. - err = nil - } - return allSamples, err -} - func (t *Target) report(start time.Time, dur time.Duration, err error) { t.status.setLastError(err) t.status.setLastScrape(start) diff --git a/retrieval/target_test.go b/retrieval/target_test.go index a6c77d232..8d6233f05 100644 --- a/retrieval/target_test.go +++ b/retrieval/target_test.go @@ -16,19 +16,17 @@ package retrieval import ( "crypto/tls" "crypto/x509" - "errors" "fmt" "io/ioutil" "net/http" "net/http/httptest" - "net/url" + // "net/url" "reflect" "strings" "testing" "time" "github.com/prometheus/common/model" - "golang.org/x/net/context" "github.com/prometheus/prometheus/config" ) @@ -92,71 +90,50 @@ func TestTargetOffset(t *testing.T) { } } -func TestTargetScrape404(t *testing.T) { - server := httptest.NewServer( - http.HandlerFunc( - func(w http.ResponseWriter, r *http.Request) { - w.WriteHeader(http.StatusNotFound) - }, - ), - ) - defer server.Close() +// func TestTargetURLParams(t *testing.T) { +// server := httptest.NewServer( +// http.HandlerFunc( +// func(w http.ResponseWriter, r *http.Request) { +// w.Header().Set("Content-Type", `text/plain; version=0.0.4`) +// w.Write([]byte{}) +// r.ParseForm() +// if r.Form["foo"][0] != "bar" { +// t.Fatalf("URL parameter 'foo' had unexpected first value '%v'", r.Form["foo"][0]) +// } +// if r.Form["foo"][1] != "baz" { +// t.Fatalf("URL parameter 'foo' had unexpected second value '%v'", r.Form["foo"][1]) +// } +// }, +// ), +// ) +// defer server.Close() +// serverURL, err := url.Parse(server.URL) +// if err != nil { +// t.Fatal(err) +// } - testTarget := newTestTarget(server.URL, time.Second, model.LabelSet{}) - - want := errors.New("server returned HTTP status 404 Not Found") - _, got := testTarget.scrape(context.Background(), time.Now()) - if got == nil || want.Error() != got.Error() { - t.Fatalf("want err %q, got %q", want, got) - } -} - -func TestURLParams(t *testing.T) { - server := httptest.NewServer( - http.HandlerFunc( - func(w http.ResponseWriter, r *http.Request) { - w.Header().Set("Content-Type", `text/plain; version=0.0.4`) - w.Write([]byte{}) - r.ParseForm() - if r.Form["foo"][0] != "bar" { - t.Fatalf("URL parameter 'foo' had unexpected first value '%v'", r.Form["foo"][0]) - } - if r.Form["foo"][1] != "baz" { - t.Fatalf("URL parameter 'foo' had unexpected second value '%v'", r.Form["foo"][1]) - } - }, - ), - ) - defer server.Close() - serverURL, err := url.Parse(server.URL) - if err != nil { - t.Fatal(err) - } - - target, err := NewTarget( - &config.ScrapeConfig{ - JobName: "test_job1", - ScrapeInterval: model.Duration(1 * time.Minute), - ScrapeTimeout: model.Duration(1 * time.Second), - Scheme: serverURL.Scheme, - Params: url.Values{ - "foo": []string{"bar", "baz"}, - }, - }, - model.LabelSet{ - model.SchemeLabel: model.LabelValue(serverURL.Scheme), - model.AddressLabel: model.LabelValue(serverURL.Host), - "__param_foo": "bar", - }, - nil, - ) - if err != nil { - t.Fatal(err) - } - if _, err = target.scrape(context.Background(), time.Now()); err != nil { - t.Fatal(err) - } -} +// target, err := NewTarget( +// &config.ScrapeConfig{ +// JobName: "test_job1", +// Scheme: "https", +// Params: url.Values{ +// "foo": []string{"bar", "baz"}, +// }, +// }, +// model.LabelSet{ +// model.SchemeLabel: model.LabelValue(serverURL.Scheme), +// model.AddressLabel: model.LabelValue(serverURL.Host), +// "__param_foo": "bar_override", +// }, +// nil, +// ) +// if err != nil { +// t.Fatal(err) +// } +// if _, err = target.scrape(context.Background(), time.Now()); err != nil { +// t.Fatal(err) +// } +// } func newTestTarget(targetURL string, deadline time.Duration, labels model.LabelSet) *Target { labels = labels.Clone() @@ -343,7 +320,7 @@ func newTLSConfig(t *testing.T) *tls.Config { return tlsConfig } -func TestNewTargetWithBadTLSConfig(t *testing.T) { +func TestNewClientWithBadTLSConfig(t *testing.T) { cfg := &config.ScrapeConfig{ ScrapeTimeout: model.Duration(1 * time.Second), TLSConfig: config.TLSConfig{ @@ -352,7 +329,7 @@ func TestNewTargetWithBadTLSConfig(t *testing.T) { KeyFile: "testdata/nonexistent_client.key", }, } - _, err := NewTarget(cfg, nil, nil) + _, err := newHTTPClient(cfg) if err == nil { t.Fatalf("Expected error, got nil.") } diff --git a/retrieval/targetmanager.go b/retrieval/targetmanager.go index d0f0960a1..e30a85ff6 100644 --- a/retrieval/targetmanager.go +++ b/retrieval/targetmanager.go @@ -459,11 +459,8 @@ func targetsFromGroup(tg *config.TargetGroup, cfg *config.ScrapeConfig) (map[mod delete(labels, ln) } } - tr, err := NewTarget(cfg, labels, preRelabelLabels) - if err != nil { - return nil, fmt.Errorf("error while creating instance %d in target group %s: %s", i, tg, err) - } + tr := NewTarget(cfg, labels, preRelabelLabels) targets[tr.fingerprint()] = tr } From 0d7105abeeb7014e4d2496e76adbc3cedcda64d3 Mon Sep 17 00:00:00 2001 From: Fabian Reinartz Date: Sun, 28 Feb 2016 19:56:18 +0100 Subject: [PATCH 18/23] Remove scrape config from Target. This commit removes the scrapeConfig entirely from Target. All identity defining parameters are thus immutable now and the mutex can be removed.. Target identity is now correctly defined by the labels and the full URL. This in particular includes URL parameters that are not specified in the label set. Fingerprint is also removed from hash to remove an unnecessary tight coupling to the common/model package. --- retrieval/scrape.go | 36 ++++++------- retrieval/scrape_test.go | 32 ++++++------ retrieval/target.go | 100 +++++++++++-------------------------- retrieval/target_test.go | 9 +--- retrieval/targetmanager.go | 16 +++--- 5 files changed, 75 insertions(+), 118 deletions(-) diff --git a/retrieval/scrape.go b/retrieval/scrape.go index a810edc72..3697a2216 100644 --- a/retrieval/scrape.go +++ b/retrieval/scrape.go @@ -82,9 +82,9 @@ type scrapePool struct { config *config.ScrapeConfig client *http.Client // Targets and loops must always be synchronized to have the same - // set of fingerprints. - targets map[model.Fingerprint]*Target - loops map[model.Fingerprint]loop + // set of hashes. + targets map[uint64]*Target + loops map[uint64]loop // Constructor for new scrape loops. This is settable for testing convenience. newLoop func(context.Context, scraper, storage.SampleAppender, storage.SampleAppender) loop @@ -100,8 +100,8 @@ func newScrapePool(cfg *config.ScrapeConfig, app storage.SampleAppender) *scrape appender: app, config: cfg, client: client, - targets: map[model.Fingerprint]*Target{}, - loops: map[model.Fingerprint]loop{}, + targets: map[uint64]*Target{}, + loops: map[uint64]loop{}, newLoop: newScrapeLoop, } } @@ -178,21 +178,21 @@ func (sp *scrapePool) sync(targets []*Target) { defer sp.mtx.Unlock() var ( - fingerprints = map[model.Fingerprint]struct{}{} - interval = time.Duration(sp.config.ScrapeInterval) - timeout = time.Duration(sp.config.ScrapeTimeout) + uniqueTargets = map[uint64]struct{}{} + interval = time.Duration(sp.config.ScrapeInterval) + timeout = time.Duration(sp.config.ScrapeTimeout) ) for _, t := range targets { - fp := t.fingerprint() - fingerprints[fp] = struct{}{} + hash := t.hash() + uniqueTargets[hash] = struct{}{} - if _, ok := sp.targets[fp]; !ok { + if _, ok := sp.targets[hash]; !ok { s := &targetScraper{Target: t, client: sp.client} l := sp.newLoop(sp.ctx, s, sp.sampleAppender(t), sp.reportAppender(t)) - sp.targets[fp] = t - sp.loops[fp] = l + sp.targets[hash] = t + sp.loops[hash] = l go l.run(interval, timeout, nil) } @@ -201,16 +201,16 @@ func (sp *scrapePool) sync(targets []*Target) { var wg sync.WaitGroup // Stop and remove old targets and scraper loops. - for fp := range sp.targets { - if _, ok := fingerprints[fp]; !ok { + for hash := range sp.targets { + if _, ok := uniqueTargets[hash]; !ok { wg.Add(1) go func(l loop) { l.stop() wg.Done() - }(sp.loops[fp]) + }(sp.loops[hash]) - delete(sp.loops, fp) - delete(sp.targets, fp) + delete(sp.loops, hash) + delete(sp.targets, hash) } } diff --git a/retrieval/scrape_test.go b/retrieval/scrape_test.go index d36919214..55e9c35ad 100644 --- a/retrieval/scrape_test.go +++ b/retrieval/scrape_test.go @@ -60,11 +60,11 @@ func (l *testLoop) stop() { func TestScrapePoolStop(t *testing.T) { sp := &scrapePool{ - targets: map[model.Fingerprint]*Target{}, - loops: map[model.Fingerprint]loop{}, + targets: map[uint64]*Target{}, + loops: map[uint64]loop{}, } var mtx sync.Mutex - stopped := map[model.Fingerprint]bool{} + stopped := map[uint64]bool{} numTargets := 20 // Stopping the scrape pool must call stop() on all scrape loops, @@ -82,12 +82,12 @@ func TestScrapePoolStop(t *testing.T) { time.Sleep(time.Duration(i*20) * time.Millisecond) mtx.Lock() - stopped[t.fingerprint()] = true + stopped[t.hash()] = true mtx.Unlock() } - sp.targets[t.fingerprint()] = t - sp.loops[t.fingerprint()] = l + sp.targets[t.hash()] = t + sp.loops[t.hash()] = l } done := make(chan struct{}) @@ -126,7 +126,7 @@ func TestScrapePoolReload(t *testing.T) { var mtx sync.Mutex numTargets := 20 - stopped := map[model.Fingerprint]bool{} + stopped := map[uint64]bool{} reloadCfg := &config.ScrapeConfig{ ScrapeInterval: model.Duration(3 * time.Second), @@ -144,7 +144,7 @@ func TestScrapePoolReload(t *testing.T) { t.Errorf("Expected scrape timeout %d but got %d", 2*time.Second, timeout) } mtx.Lock() - if !stopped[s.(*targetScraper).fingerprint()] { + if !stopped[s.(*targetScraper).hash()] { t.Errorf("Scrape loop for %v not stopped yet", s.(*targetScraper)) } mtx.Unlock() @@ -152,8 +152,8 @@ func TestScrapePoolReload(t *testing.T) { return l } sp := &scrapePool{ - targets: map[model.Fingerprint]*Target{}, - loops: map[model.Fingerprint]loop{}, + targets: map[uint64]*Target{}, + loops: map[uint64]loop{}, newLoop: newLoop, } @@ -172,18 +172,18 @@ func TestScrapePoolReload(t *testing.T) { time.Sleep(time.Duration(i*20) * time.Millisecond) mtx.Lock() - stopped[t.fingerprint()] = true + stopped[t.hash()] = true mtx.Unlock() } - sp.targets[t.fingerprint()] = t - sp.loops[t.fingerprint()] = l + sp.targets[t.hash()] = t + sp.loops[t.hash()] = l } done := make(chan struct{}) - beforeTargets := map[model.Fingerprint]*Target{} - for fp, t := range sp.targets { - beforeTargets[fp] = t + beforeTargets := map[uint64]*Target{} + for h, t := range sp.targets { + beforeTargets[h] = t } reloadTime := time.Now() diff --git a/retrieval/target.go b/retrieval/target.go index e48e28449..8aff3f8d8 100644 --- a/retrieval/target.go +++ b/retrieval/target.go @@ -15,6 +15,7 @@ package retrieval import ( "fmt" + "hash/fnv" "io/ioutil" "net/http" "net/url" @@ -117,24 +118,21 @@ type Target struct { // The status object for the target. It is only set once on initialization. status *TargetStatus - scrapeConfig *config.ScrapeConfig - - // Mutex protects the members below. - sync.RWMutex - // Labels before any processing. metaLabels model.LabelSet // Any labels that are added to this target and its metrics. labels model.LabelSet + // Additional URL parmeters that are part of the target URL. + params url.Values } // NewTarget creates a reasonably configured target for querying. -func NewTarget(cfg *config.ScrapeConfig, labels, metaLabels model.LabelSet) *Target { +func NewTarget(labels, metaLabels model.LabelSet, params url.Values) *Target { return &Target{ - status: &TargetStatus{}, - scrapeConfig: cfg, - labels: labels, - metaLabels: metaLabels, + status: &TargetStatus{}, + labels: labels, + metaLabels: metaLabels, + params: params, } } @@ -188,15 +186,16 @@ func newHTTPClient(cfg *config.ScrapeConfig) (*http.Client, error) { } func (t *Target) String() string { - return t.host() + return t.URL().String() } -// fingerprint returns an identifying hash for the target. -func (t *Target) fingerprint() model.Fingerprint { - t.RLock() - defer t.RUnlock() +// hash returns an identifying hash for the target. +func (t *Target) hash() uint64 { + h := fnv.New64a() + h.Write([]byte(t.labels.Fingerprint().String())) + h.Write([]byte(t.URL().String())) - return t.labels.Fingerprint() + return h.Sum64() } // offset returns the time until the next scrape cycle for the target. @@ -205,7 +204,7 @@ func (t *Target) offset(interval time.Duration) time.Duration { var ( base = now % int64(interval) - offset = uint64(t.fingerprint()) % uint64(interval) + offset = t.hash() % uint64(interval) next = base + int64(offset) ) @@ -215,35 +214,27 @@ func (t *Target) offset(interval time.Duration) time.Duration { return time.Duration(next) } -func (t *Target) scheme() string { - t.RLock() - defer t.RUnlock() - - return string(t.labels[model.SchemeLabel]) +// Labels returns a copy of the set of all public labels of the target. +func (t *Target) Labels() model.LabelSet { + lset := make(model.LabelSet, len(t.labels)) + for ln, lv := range t.labels { + if !strings.HasPrefix(string(ln), model.ReservedLabelPrefix) { + lset[ln] = lv + } + } + return lset } -func (t *Target) host() string { - t.RLock() - defer t.RUnlock() - - return string(t.labels[model.AddressLabel]) -} - -func (t *Target) path() string { - t.RLock() - defer t.RUnlock() - - return string(t.labels[model.MetricsPathLabel]) +// MetaLabels returns a copy of the target's labels before any processing. +func (t *Target) MetaLabels() model.LabelSet { + return t.metaLabels.Clone() } // URL returns a copy of the target's URL. func (t *Target) URL() *url.URL { - t.RLock() - defer t.RUnlock() - params := url.Values{} - for k, v := range t.scrapeConfig.Params { + for k, v := range t.params { params[k] = make([]string, len(v)) copy(params[k], v) } @@ -329,36 +320,3 @@ func (app relabelAppender) Append(s *model.Sample) error { return app.SampleAppender.Append(s) } - -// Labels returns a copy of the set of all public labels of the target. -func (t *Target) Labels() model.LabelSet { - t.RLock() - defer t.RUnlock() - - return t.unlockedLabels() -} - -// unlockedLabels does the same as Labels but does not lock the mutex (useful -// for internal usage when the mutex is already locked). -func (t *Target) unlockedLabels() model.LabelSet { - lset := make(model.LabelSet, len(t.labels)) - for ln, lv := range t.labels { - if !strings.HasPrefix(string(ln), model.ReservedLabelPrefix) { - lset[ln] = lv - } - } - - if _, ok := lset[model.InstanceLabel]; !ok { - lset[model.InstanceLabel] = t.labels[model.AddressLabel] - } - - return lset -} - -// MetaLabels returns a copy of the target's labels before any processing. -func (t *Target) MetaLabels() model.LabelSet { - t.RLock() - defer t.RUnlock() - - return t.metaLabels.Clone() -} diff --git a/retrieval/target_test.go b/retrieval/target_test.go index 8d6233f05..e679a568d 100644 --- a/retrieval/target_test.go +++ b/retrieval/target_test.go @@ -34,9 +34,8 @@ import ( func TestTargetLabels(t *testing.T) { target := newTestTarget("example.com:80", 0, model.LabelSet{"job": "some_job", "foo": "bar"}) want := model.LabelSet{ - model.JobLabel: "some_job", - model.InstanceLabel: "example.com:80", - "foo": "bar", + model.JobLabel: "some_job", + "foo": "bar", } got := target.Labels() if !reflect.DeepEqual(want, got) { @@ -142,10 +141,6 @@ func newTestTarget(targetURL string, deadline time.Duration, labels model.LabelS labels[model.MetricsPathLabel] = "/metrics" return &Target{ - scrapeConfig: &config.ScrapeConfig{ - ScrapeInterval: model.Duration(time.Millisecond), - ScrapeTimeout: model.Duration(deadline), - }, labels: labels, status: &TargetStatus{}, } diff --git a/retrieval/targetmanager.go b/retrieval/targetmanager.go index e30a85ff6..5e590e44a 100644 --- a/retrieval/targetmanager.go +++ b/retrieval/targetmanager.go @@ -176,7 +176,7 @@ type targetSet struct { mtx sync.RWMutex // Sets of targets by a source string that is unique across target providers. - tgroups map[string]map[model.Fingerprint]*Target + tgroups map[string]map[uint64]*Target providers map[string]TargetProvider scrapePool *scrapePool @@ -189,7 +189,7 @@ type targetSet struct { func newTargetSet(cfg *config.ScrapeConfig, app storage.SampleAppender) *targetSet { ts := &targetSet{ - tgroups: map[string]map[model.Fingerprint]*Target{}, + tgroups: map[string]map[uint64]*Target{}, scrapePool: newScrapePool(cfg, app), syncCh: make(chan struct{}, 1), config: cfg, @@ -394,8 +394,8 @@ func providersFromConfig(cfg *config.ScrapeConfig) map[string]TargetProvider { } // targetsFromGroup builds targets based on the given TargetGroup and config. -func targetsFromGroup(tg *config.TargetGroup, cfg *config.ScrapeConfig) (map[model.Fingerprint]*Target, error) { - targets := make(map[model.Fingerprint]*Target, len(tg.Targets)) +func targetsFromGroup(tg *config.TargetGroup, cfg *config.ScrapeConfig) (map[uint64]*Target, error) { + targets := make(map[uint64]*Target, len(tg.Targets)) for i, labels := range tg.Targets { for k, v := range cfg.Params { @@ -460,8 +460,12 @@ func targetsFromGroup(tg *config.TargetGroup, cfg *config.ScrapeConfig) (map[mod } } - tr := NewTarget(cfg, labels, preRelabelLabels) - targets[tr.fingerprint()] = tr + if _, ok := labels[model.InstanceLabel]; !ok { + labels[model.InstanceLabel] = labels[model.AddressLabel] + } + + tr := NewTarget(labels, preRelabelLabels, cfg.Params) + targets[tr.hash()] = tr } return targets, nil From 2060a0a15b93fd2cb7e44d03edb4f0e0a6a0529f Mon Sep 17 00:00:00 2001 From: Fabian Reinartz Date: Sun, 28 Feb 2016 20:03:44 +0100 Subject: [PATCH 19/23] Turn target group members into plain lists. As the scrape pool deduplicates targets now, it is no longer necessary to store a hash map for members of each group. --- retrieval/targetmanager.go | 21 +++++++++------------ 1 file changed, 9 insertions(+), 12 deletions(-) diff --git a/retrieval/targetmanager.go b/retrieval/targetmanager.go index 5e590e44a..6daa708fc 100644 --- a/retrieval/targetmanager.go +++ b/retrieval/targetmanager.go @@ -176,7 +176,7 @@ type targetSet struct { mtx sync.RWMutex // Sets of targets by a source string that is unique across target providers. - tgroups map[string]map[uint64]*Target + tgroups map[string][]*Target providers map[string]TargetProvider scrapePool *scrapePool @@ -189,7 +189,7 @@ type targetSet struct { func newTargetSet(cfg *config.ScrapeConfig, app storage.SampleAppender) *targetSet { ts := &targetSet{ - tgroups: map[string]map[uint64]*Target{}, + tgroups: map[string][]*Target{}, scrapePool: newScrapePool(cfg, app), syncCh: make(chan struct{}, 1), config: cfg, @@ -247,13 +247,11 @@ Loop: } func (ts *targetSet) sync() { - targets := []*Target{} - for _, tgroup := range ts.tgroups { - for _, t := range tgroup { - targets = append(targets, t) - } + var all []*Target + for _, targets := range ts.tgroups { + all = append(all, targets...) } - ts.scrapePool.sync(targets) + ts.scrapePool.sync(all) } func (ts *targetSet) runProviders(ctx context.Context, providers map[string]TargetProvider) { @@ -394,8 +392,8 @@ func providersFromConfig(cfg *config.ScrapeConfig) map[string]TargetProvider { } // targetsFromGroup builds targets based on the given TargetGroup and config. -func targetsFromGroup(tg *config.TargetGroup, cfg *config.ScrapeConfig) (map[uint64]*Target, error) { - targets := make(map[uint64]*Target, len(tg.Targets)) +func targetsFromGroup(tg *config.TargetGroup, cfg *config.ScrapeConfig) ([]*Target, error) { + targets := make([]*Target, 0, len(tg.Targets)) for i, labels := range tg.Targets { for k, v := range cfg.Params { @@ -464,8 +462,7 @@ func targetsFromGroup(tg *config.TargetGroup, cfg *config.ScrapeConfig) (map[uin labels[model.InstanceLabel] = labels[model.AddressLabel] } - tr := NewTarget(labels, preRelabelLabels, cfg.Params) - targets[tr.hash()] = tr + targets = append(targets, NewTarget(labels, preRelabelLabels, cfg.Params)) } return targets, nil From 1ede7b9d729af17f85565cddea41aaba5581d864 Mon Sep 17 00:00:00 2001 From: Fabian Reinartz Date: Sun, 28 Feb 2016 20:23:26 +0100 Subject: [PATCH 20/23] Consolidate TargetStatus into Target. This commit simplifies the TargetHealth type and moves the target status into the target itself. This also removes a race where error and last scrape time could have been out of sync. --- retrieval/target.go | 135 ++++++++++++----------------------- retrieval/target_test.go | 1 - web/ui/bindata.go | 4 +- web/ui/templates/status.html | 10 +-- 4 files changed, 53 insertions(+), 97 deletions(-) diff --git a/retrieval/target.go b/retrieval/target.go index 8aff3f8d8..3f6bfa5e1 100644 --- a/retrieval/target.go +++ b/retrieval/target.go @@ -31,116 +31,40 @@ import ( ) // TargetHealth describes the health state of a target. -type TargetHealth int - -func (t TargetHealth) String() string { - switch t { - case HealthUnknown: - return "unknown" - case HealthGood: - return "up" - case HealthBad: - return "down" - } - panic("unknown state") -} - -func (t TargetHealth) value() model.SampleValue { - if t == HealthGood { - return 1 - } - return 0 -} +type TargetHealth string +// The possible health states of a target based on the last performed scrape. const ( - // HealthUnknown is the state of a Target before it is first scraped. - HealthUnknown TargetHealth = iota - // HealthGood is the state of a Target that has been successfully scraped. - HealthGood - // HealthBad is the state of a Target that was scraped unsuccessfully. - HealthBad + HealthUnknown TargetHealth = "unknown" + HealthGood TargetHealth = "up" + HealthBad TargetHealth = "down" ) -// TargetStatus contains information about the current status of a scrape target. -type TargetStatus struct { - lastError error - lastScrape time.Time - health TargetHealth - - mu sync.RWMutex -} - -// LastError returns the error encountered during the last scrape. -func (ts *TargetStatus) LastError() error { - ts.mu.RLock() - defer ts.mu.RUnlock() - - return ts.lastError -} - -// LastScrape returns the time of the last scrape. -func (ts *TargetStatus) LastScrape() time.Time { - ts.mu.RLock() - defer ts.mu.RUnlock() - - return ts.lastScrape -} - -// Health returns the last known health state of the target. -func (ts *TargetStatus) Health() TargetHealth { - ts.mu.RLock() - defer ts.mu.RUnlock() - - return ts.health -} - -func (ts *TargetStatus) setLastScrape(t time.Time) { - ts.mu.Lock() - defer ts.mu.Unlock() - - ts.lastScrape = t -} - -func (ts *TargetStatus) setLastError(err error) { - ts.mu.Lock() - defer ts.mu.Unlock() - - if err == nil { - ts.health = HealthGood - } else { - ts.health = HealthBad - } - ts.lastError = err -} - // Target refers to a singular HTTP or HTTPS endpoint. type Target struct { - // The status object for the target. It is only set once on initialization. - status *TargetStatus - // Labels before any processing. metaLabels model.LabelSet // Any labels that are added to this target and its metrics. labels model.LabelSet // Additional URL parmeters that are part of the target URL. params url.Values + + mtx sync.RWMutex + lastError error + lastScrape time.Time + health TargetHealth } // NewTarget creates a reasonably configured target for querying. func NewTarget(labels, metaLabels model.LabelSet, params url.Values) *Target { return &Target{ - status: &TargetStatus{}, labels: labels, metaLabels: metaLabels, params: params, + health: HealthUnknown, } } -// Status returns the status of the target. -func (t *Target) Status() *TargetStatus { - return t.status -} - func newHTTPClient(cfg *config.ScrapeConfig) (*http.Client, error) { tlsOpts := httputil.TLSOptions{ InsecureSkipVerify: cfg.TLSConfig.InsecureSkipVerify, @@ -260,8 +184,41 @@ func (t *Target) URL() *url.URL { } func (t *Target) report(start time.Time, dur time.Duration, err error) { - t.status.setLastError(err) - t.status.setLastScrape(start) + t.mtx.Lock() + defer t.mtx.Unlock() + + if err == nil { + t.health = HealthGood + } else { + t.health = HealthBad + } + + t.lastError = err + t.lastScrape = start +} + +// LastError returns the error encountered during the last scrape. +func (t *Target) LastError() error { + t.mtx.RLock() + defer t.mtx.RUnlock() + + return t.lastError +} + +// LastScrape returns the time of the last scrape. +func (t *Target) LastScrape() time.Time { + t.mtx.RLock() + defer t.mtx.RUnlock() + + return t.lastScrape +} + +// Health returns the last known health state of the target. +func (t *Target) Health() TargetHealth { + t.mtx.RLock() + defer t.mtx.RUnlock() + + return t.health } // Merges the ingested sample's metric with the label set. On a collision the diff --git a/retrieval/target_test.go b/retrieval/target_test.go index e679a568d..e2d8a8778 100644 --- a/retrieval/target_test.go +++ b/retrieval/target_test.go @@ -142,7 +142,6 @@ func newTestTarget(targetURL string, deadline time.Duration, labels model.LabelS return &Target{ labels: labels, - status: &TargetStatus{}, } } diff --git a/web/ui/bindata.go b/web/ui/bindata.go index 43c15e3b7..beed94aac 100644 --- a/web/ui/bindata.go +++ b/web/ui/bindata.go @@ -159,7 +159,7 @@ func webUiTemplatesGraphHtml() (*asset, error) { return a, nil } -var _webUiTemplatesStatusHtml = []byte("\x1f\x8b\x08\x00\x00\x09\x6e\x88\x00\xff\xcc\x57\xcd\x6e\xdc\x36\x10\xbe\xef\x53\xb0\x44\x8e\xd5\x2e\x10\xa0\x17\x63\x57\x07\x1b\x29\x1c\xc0\x29\xdc\xac\x7d\xe9\x25\xe0\x8a\xb3\x12\x5b\x9a\x14\x48\xca\xb5\xa1\xe8\xdd\x3b\x43\x49\x5e\xfd\x6d\x9a\x34\x68\xeb\xcb\x9a\x43\x0e\xe7\xe7\x9b\x6f\x46\x74\x5d\x4b\x38\x2a\x03\x8c\x17\x20\x24\x6f\x9a\xed\x0f\x49\xc2\x8c\x7a\x62\x49\x92\xd6\x35\x18\xd9\x34\xab\xd5\x49\x2b\xb3\x26\x80\x09\xa8\xb8\x62\x6c\x2b\xd5\x23\xcb\xb4\xf0\x7e\x17\x0f\x04\xaa\xb8\xe4\xa8\x2b\x25\x79\x8a\xe7\xa8\x51\xbc\x65\x4a\xee\xb8\xab\x4c\x50\x0f\xc0\xd3\x8f\xed\x82\xbd\x37\x47\xeb\x1e\x44\x50\xd6\x6c\x37\xc5\xdb\x4e\x3b\x88\x83\x86\xde\x62\x2b\xc4\xdf\x04\xad\x4b\x30\x1e\x64\x27\x1f\xac\x93\xe0\x5e\x44\x1f\x9c\x2a\x5f\xa4\xc2\x3e\x82\xeb\x02\x20\xa3\x07\x2b\x9f\x7b\x89\x64\x77\x12\x48\x2c\xd2\xfb\x92\x62\xda\x6e\x70\x39\x3a\x91\x88\xc0\x7a\x1f\x44\xa8\xfc\xfa\x52\xb9\x50\xac\xef\xef\xae\x10\xa2\x0d\x9e\x9c\xec\x6d\x4e\x06\x71\x7d\x72\x86\x02\x85\x93\xae\x46\x48\x1c\x2a\xa5\xa5\x3a\x65\xcf\xd3\x4b\xda\xf9\x1f\x01\xa9\x6b\x27\x4c\x0e\xec\xcd\x1f\xf0\xfc\x23\x7b\xf3\x28\x74\x05\xec\x62\xc7\xd6\x14\x52\xac\xf3\x39\xe0\x98\xcf\x6c\x09\x58\x5d\xfb\x27\x47\xa8\xc8\x40\x44\x67\x01\xc6\xd6\xec\x97\xb0\xa3\x40\x5a\xba\x7d\x35\x96\x08\xc2\x51\xe5\x95\xeb\x80\xbc\x1a\x8a\x03\x10\x4b\x07\x83\x42\xb6\x5a\x14\x09\xed\xaf\x26\x34\xd5\xe0\x89\xa4\xf8\x67\x66\xa0\x45\x29\x13\x5a\xb3\xde\x56\x54\x6c\x1a\x34\x7e\x7d\xf7\xe1\x66\x6f\x54\x59\x42\x60\xa5\x08\xc5\xad\xc3\x86\x79\x42\x2f\x07\xb7\xe9\xfb\x68\xc9\x63\x10\x2e\x87\x80\x3e\xef\xda\xc5\xc9\xeb\xbf\x54\xfd\x41\xbd\x7f\xb7\x07\xac\x77\x69\xad\xa6\x72\x8f\x12\x6b\xa3\xb9\xc5\x23\x3f\x60\x40\x2c\x3a\x8e\x89\x61\x79\x5b\x5e\x10\x19\x32\x54\x2e\x85\xd9\xf1\x9f\x78\x1f\x33\x7a\xf8\x44\x17\xc8\x3f\x72\x00\xc5\x8e\x1f\xe3\xc2\x2f\xb0\xab\x73\x96\xbe\x33\xb2\xb4\xca\x84\x29\xab\xfa\x73\x8a\x77\xd6\xb9\xfd\xe1\x8d\x38\x80\xf6\xe7\x4f\x7d\x60\xfb\xcc\x89\xf2\xac\x81\x77\xce\x59\x37\x3f\x9c\x46\x4f\x1a\x13\x58\xa6\x4d\x36\x80\x9d\x00\x1f\x81\x7a\x26\x79\x39\xdb\x12\xac\x40\x5a\xed\x38\xf2\xed\xfe\xe3\x0d\xfb\xcc\x72\x6d\x0f\x42\xe3\xba\x69\x08\x60\xda\x5d\xef\xb3\x02\x1e\xb0\xd3\x2e\x36\x9b\x6e\xe7\xda\xfa\x10\x49\x4a\xc2\x2d\x92\x93\x8a\x20\x52\xa4\xe6\xd4\xc3\x20\x4a\x4d\xd8\xf5\xe3\xc0\xc7\x79\x40\xd7\x7f\xad\xc0\x3d\xb3\x49\xf8\x93\xab\x6a\x38\x45\x3a\x03\x8b\x37\x30\x25\x62\x4c\xcf\x96\xe8\x92\xc5\xdf\xa4\x74\xea\x41\xb8\xe7\x48\x9b\xb8\xd3\x34\x94\x77\x3f\x46\xf8\x76\x43\x37\xe7\xf1\x4f\xa7\xc8\xdf\xed\x8f\xe7\xd1\x59\xe8\x27\x91\x0a\x0d\x2e\xb0\xf8\x9b\xd4\xf5\x4b\xd7\x5c\x83\xd0\xd8\x08\x9f\x59\x11\x17\x77\xf6\x8a\xd4\x11\x2d\xe6\x89\xa6\x9f\x94\x91\x2a\x13\xc1\x3a\x16\xe0\x29\x24\x15\x4e\x0b\x97\x09\x0f\x7c\x39\x8f\xb1\xd9\x85\x94\x96\x41\xf8\x67\x29\x65\x95\xf3\xd6\x25\xb1\xd9\xb0\x5d\x99\x14\x41\x24\xc1\xe6\xb9\xc6\x01\x1f\x90\xb2\x41\x95\x9c\x05\x15\x48\xee\x8e\xad\x53\xb9\x32\x42\x27\xdd\xf6\x25\xe0\x37\x0c\x98\x83\x58\x31\x65\xf2\x0b\xca\xe2\x03\x04\xd1\x76\x22\xb1\x74\x31\xd3\xb6\xc4\x91\x65\x71\x76\xb5\xea\x6c\xdd\xfd\xa5\x39\xc2\x19\x57\x06\x61\x34\x19\xf0\x2f\xd3\x6f\xc4\xdc\x48\x41\xdd\x79\xff\x0f\x29\xa8\x3d\x7c\xab\x3f\x7c\x62\x89\x4a\x07\x9e\x1a\x6b\xe0\xdb\xf9\xfd\x9d\x64\xa8\x6b\x75\x7c\x21\x32\x8d\xc6\x76\x32\xae\xdf\xfb\xdf\xc0\xe1\x33\xe0\x17\xc0\xaf\x48\x9f\x58\x5d\x7b\x85\x85\x58\xd0\x47\xae\x8b\xdc\x7e\x67\xaf\xcd\x62\x89\x73\x78\x29\xe7\x73\x4d\x29\x89\x0a\x6e\xda\x76\x7c\xf0\x0c\x18\x98\x3d\x87\xf5\xd7\x66\x31\xfd\x1c\xcc\xef\x8d\xde\x32\x73\x95\xe5\xd7\x0d\x06\xef\x42\x55\x1e\xb5\xc8\xf1\x7d\xb0\x6f\x25\xf6\x33\x89\xaf\xe5\x85\xd8\x61\x19\x63\x7a\x6d\x2f\x45\x5a\xe2\xff\x27\xe9\xaa\x57\xfe\x2b\x00\x00\xff\xff\xb2\x36\x91\x1f\xeb\x0c\x00\x00") +var _webUiTemplatesStatusHtml = []byte("\x1f\x8b\x08\x00\x00\x09\x6e\x88\x00\xff\xcc\x57\xcd\x6e\xdc\x36\x10\xbe\xef\x53\xb0\x44\x8e\xd5\x2e\x10\xa0\x17\x63\x57\x07\x1b\x29\x1c\xc0\x29\xdc\xac\x7d\xe9\x25\xe0\x8a\xb3\x12\x5b\x9a\x14\x48\xca\xf5\x42\xd1\xbb\x77\x86\x92\xa2\x9f\xd5\xa6\x49\x83\xb6\xb9\xac\x39\xe4\x70\x7e\xbe\xf9\x66\x44\xd7\xb5\x84\xa3\x32\xc0\x78\x01\x42\xf2\xa6\xd9\xfe\x90\x24\xcc\xa8\x17\x96\x24\x69\x5d\x83\x91\x4d\xb3\x5a\x0d\x5a\x99\x35\x01\x4c\x40\xc5\x15\x63\x5b\xa9\x9e\x59\xa6\x85\xf7\xbb\x78\x20\x50\xc5\x25\x47\x5d\x29\xc9\x53\x3c\x47\x8d\xe2\x35\x53\x72\xc7\x5d\x65\x82\x7a\x02\x9e\xbe\x6f\x17\xec\xad\x39\x5a\xf7\x24\x82\xb2\x66\xbb\x29\x5e\x77\xda\x41\x1c\x34\xf4\x16\x5b\x21\xfe\x26\x68\x5d\x82\xf1\x20\x3b\xf9\x60\x9d\x04\xf7\x49\xf4\xc1\xa9\xf2\x93\x54\xd8\x67\x70\x5d\x00\x64\xf4\x60\xe5\xa9\x97\x48\x76\x83\x40\x62\x91\x3e\x96\x14\xd3\x76\x83\xcb\xc9\x89\x44\x04\xd6\xfb\x20\x42\xe5\xd7\xd7\xca\x85\x62\xfd\xf8\x70\x83\x10\x6d\xf0\x64\xb0\xb7\x19\x0c\xe2\x7a\x70\x86\x02\x85\x93\xae\x26\x48\x1c\x2a\xa5\xa5\x1a\xb2\xe7\xe9\x35\xed\xfc\x8f\x80\xd4\xb5\x13\x26\x07\xf6\xea\x0f\x38\xfd\xc8\x5e\x3d\x0b\x5d\x01\xbb\xda\xb1\x35\x85\x14\xeb\x7c\x09\x38\xe6\x33\x5b\x02\x56\xd7\xfe\xc9\x11\x2a\x32\x10\xd1\x59\x80\xb1\x35\xfb\x39\xec\x28\x90\x96\x6e\x5f\x8c\x25\x82\x70\x54\x79\xe5\x3a\x20\x6f\xc6\xe2\x08\xc4\xd2\xc1\xa8\x90\xad\x16\x45\x42\xfb\xab\x19\x4d\x35\x78\x22\x29\xfe\x39\x33\xd0\xa2\x94\x09\xad\x59\x6f\x2b\x2a\x36\x0d\x1a\xbf\x7d\x78\x77\xb7\x37\xaa\x2c\x21\xb0\x52\x84\xe2\xde\x61\xc3\xbc\xa0\x97\x83\xdb\xf4\x7d\xb4\xe4\x31\x08\x97\x43\x40\x9f\x0f\xed\x62\xf0\xfa\x2f\x55\x7f\x54\xef\xdf\xed\x01\xeb\x5d\x5a\xab\xa9\xdc\x93\xc4\xda\x68\xee\xf1\xc8\x8f\x18\x10\x8b\x8e\x63\x62\x5c\xde\x96\x17\x44\x86\x0c\x95\x4b\x61\x76\xfc\x27\xde\xc7\x8c\x1e\x3e\xd0\x05\xf2\x8f\x1c\x40\xb1\xe3\xc7\xb4\xf0\x0b\xec\xea\x9c\xa5\x6f\x8c\x2c\xad\x32\x61\xce\xaa\xfe\x9c\xe2\x3d\xeb\xdc\xfe\xf0\x4e\x1c\x40\xfb\xcb\xa7\x3e\xb0\x7d\xe6\x44\x79\xd1\xc0\x1b\xe7\xac\x3b\x3f\x9c\x47\x4f\x1a\x33\x58\xe6\x4d\x36\x82\x9d\x00\x9f\x80\x7a\x21\x79\x79\xb6\x25\x58\x81\xb4\xda\x71\xe4\xdb\xe3\xfb\x3b\xf6\x91\xe5\xda\x1e\x84\xc6\x75\xd3\x10\xc0\xb4\xbb\xde\x67\x05\x3c\x61\xa7\x5d\x6d\x36\xdd\xce\xad\xf5\x21\x92\x94\x84\x7b\x24\x27\x15\x41\xa4\x48\xcd\xb9\x87\x51\x94\x9a\xb0\xeb\xc7\x81\x8f\xf3\x80\xae\xff\x5a\x81\x3b\xb1\x59\xf8\xb3\xab\x6a\x3c\x45\x3a\x03\x8b\x37\x30\x25\x62\x4c\xcf\x96\xe8\x92\xc5\xdf\xa4\x74\xea\x49\xb8\x53\xa4\x4d\xdc\x69\x1a\xca\xbb\x1f\x23\x7c\xbb\xa1\x9b\xe7\xf1\xcf\xa7\xc8\xdf\xed\x4f\xe7\xd1\x45\xe8\x67\x91\x0a\x0d\x2e\xb0\xf8\x9b\xd4\x35\x5b\xdf\x82\xd0\xd8\x01\x1f\x59\x11\x17\x0f\xf6\x86\xf4\x10\x26\xe6\x89\x9f\x1f\x94\x91\x2a\x13\xc1\x3a\x16\xe0\x25\x24\x15\x8e\x09\x97\x09\x0f\x7c\x39\x81\xce\xde\x42\x12\xcb\x69\xff\xb3\x24\xb2\xca\x79\xeb\x92\xd8\x5e\xd8\xa0\x4c\x8a\x20\x92\x60\xf3\x5c\xe3\x48\x0f\x48\xd2\xa0\x4a\xce\x82\x0a\x24\x77\xc7\xd6\xa9\x5c\x19\xa1\x93\x6e\xfb\x1a\xf0\xab\x05\xcc\x41\xac\x91\x32\xf9\x15\x85\xff\x0e\x82\x68\x7b\x8f\x78\xb9\x98\x62\x5b\xd4\xc8\xab\x38\xad\x5a\x75\xb6\xee\xfe\xd2\xe4\xe0\x8c\x2b\x83\xf8\x99\x0c\xf8\xe7\x09\x37\xe1\x6a\x24\x9d\xee\xbc\xff\x87\xa4\xd3\x1e\xbe\xd6\x1f\x3e\xaa\x44\xa5\x03\x4f\x8d\x35\xf0\xf5\x8c\xfe\x46\x32\xd4\xb5\x3a\x12\xe0\x3e\xb4\x43\x70\xfd\xd6\xff\x06\x0e\xbf\xf8\xbf\x00\x7e\x30\xfa\x8c\xea\xda\x2b\xac\xc0\x58\x11\x69\x2d\x72\xfb\x8d\xfd\x34\x78\x8f\x43\x76\x29\xbd\x4b\x1d\x27\xa9\xea\x6e\xde\x5a\x71\x00\x8e\xec\x5d\xc2\xf3\x4b\xe3\x9e\x0f\xf9\xf3\x7b\x93\x17\xca\xb9\xca\xf2\x9b\x05\xa3\x76\xa1\x2a\x8f\x5a\xe4\xf8\xd5\xdf\xb7\x12\xfb\x99\xc4\xef\xe5\xdd\xd7\xbd\x01\x62\x4c\xdf\xdb\xfb\x8f\x96\xf8\x5f\x47\xba\xea\x95\xff\x0a\x00\x00\xff\xff\xd0\x50\xb7\x0a\xc1\x0c\x00\x00") func webUiTemplatesStatusHtmlBytes() ([]byte, error) { return bindataRead( @@ -174,7 +174,7 @@ func webUiTemplatesStatusHtml() (*asset, error) { return nil, err } - info := bindataFileInfo{name: "web/ui/templates/status.html", size: 3307, mode: os.FileMode(420), modTime: time.Unix(1455530985, 0)} + info := bindataFileInfo{name: "web/ui/templates/status.html", size: 3265, mode: os.FileMode(420), modTime: time.Unix(1456687049, 0)} a := &asset{bytes: bytes, info: info} return a, nil } diff --git a/web/ui/templates/status.html b/web/ui/templates/status.html index 085566e0c..f28069008 100644 --- a/web/ui/templates/status.html +++ b/web/ui/templates/status.html @@ -55,8 +55,8 @@ {{end}} - - {{.Status.Health}} + + {{.Health}} @@ -70,11 +70,11 @@ - {{if .Status.LastScrape.IsZero}}Never{{else}}{{since .Status.LastScrape}} ago{{end}} + {{if .LastScrape.IsZero}}Never{{else}}{{since .LastScrape}} ago{{end}} - {{if .Status.LastError}} - {{.Status.LastError}} + {{if .LastError}} + {{.LastError}} {{end}} From 50c2f20756aa37afc75e5ccd2871a3238e0e2d8e Mon Sep 17 00:00:00 2001 From: Fabian Reinartz Date: Sun, 28 Feb 2016 23:59:03 +0100 Subject: [PATCH 21/23] Add targetScraper tests --- retrieval/scrape_test.go | 132 +++++++++++++++++++++++++++++++++++++++ 1 file changed, 132 insertions(+) diff --git a/retrieval/scrape_test.go b/retrieval/scrape_test.go index 55e9c35ad..7ea1eba5a 100644 --- a/retrieval/scrape_test.go +++ b/retrieval/scrape_test.go @@ -15,7 +15,11 @@ package retrieval import ( "fmt" + "net/http" + "net/http/httptest" + "net/url" "reflect" + "strings" "sync" "testing" "time" @@ -423,6 +427,134 @@ func TestScrapeLoopRun(t *testing.T) { } } +func TestTargetScraperScrapeOK(t *testing.T) { + server := httptest.NewServer( + http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + w.Header().Set("Content-Type", `text/plain; version=0.0.4`) + w.Write([]byte("metric_a 1\nmetric_b 2\n")) + }), + ) + defer server.Close() + + serverURL, err := url.Parse(server.URL) + if err != nil { + panic(err) + } + + ts := &targetScraper{ + Target: &Target{ + labels: model.LabelSet{ + model.SchemeLabel: model.LabelValue(serverURL.Scheme), + model.AddressLabel: model.LabelValue(serverURL.Host), + }, + }, + client: http.DefaultClient, + } + now := time.Now() + + samples, err := ts.scrape(context.Background(), now) + if err != nil { + t.Fatalf("Unexpected scrape error: %s", err) + } + + expectedSamples := model.Samples{ + { + Metric: model.Metric{"__name__": "metric_a"}, + Timestamp: model.TimeFromUnixNano(now.UnixNano()), + Value: 1, + }, + { + Metric: model.Metric{"__name__": "metric_b"}, + Timestamp: model.TimeFromUnixNano(now.UnixNano()), + Value: 2, + }, + } + + if !reflect.DeepEqual(samples, expectedSamples) { + t.Errorf("Scraped samples did not match served metrics") + t.Errorf("Expected: %v", expectedSamples) + t.Fatalf("Got: %v", samples) + } +} + +func TestTargetScrapeScrapeCancel(t *testing.T) { + block := make(chan struct{}) + + server := httptest.NewServer( + http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + <-block + }), + ) + defer server.Close() + + serverURL, err := url.Parse(server.URL) + if err != nil { + panic(err) + } + + ts := &targetScraper{ + Target: &Target{ + labels: model.LabelSet{ + model.SchemeLabel: model.LabelValue(serverURL.Scheme), + model.AddressLabel: model.LabelValue(serverURL.Host), + }, + }, + client: http.DefaultClient, + } + ctx, cancel := context.WithCancel(context.Background()) + + done := make(chan struct{}) + + go func() { + time.Sleep(1 * time.Second) + cancel() + }() + + go func() { + if _, err := ts.scrape(ctx, time.Now()); err != context.Canceled { + t.Fatalf("Expected context cancelation error but got: %s", err) + } + close(done) + }() + + select { + case <-time.After(5 * time.Second): + t.Fatalf("Scrape function did not return unexpectedly") + case <-done: + } + // If this is closed in a defer above the function the test server + // does not terminate and the test doens't complete. + close(block) +} + +func TestTargetScrapeScrapeNotFound(t *testing.T) { + server := httptest.NewServer( + http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + w.WriteHeader(http.StatusNotFound) + }), + ) + defer server.Close() + + serverURL, err := url.Parse(server.URL) + if err != nil { + panic(err) + } + + ts := &targetScraper{ + Target: &Target{ + labels: model.LabelSet{ + model.SchemeLabel: model.LabelValue(serverURL.Scheme), + model.AddressLabel: model.LabelValue(serverURL.Host), + }, + }, + client: http.DefaultClient, + } + + if _, err := ts.scrape(context.Background(), time.Now()); !strings.Contains(err.Error(), "404") { + t.Fatalf("Expected \"404 NotFound\" error but got: %s", err) + } +} + // testScraper implements the scraper interface and allows setting values // returned by its methods. It also allows setting a custom scrape function. type testScraper struct { From 499f4af4aa109185410697d9d5a7f7e4c38d8044 Mon Sep 17 00:00:00 2001 From: Fabian Reinartz Date: Tue, 1 Mar 2016 14:49:57 +0100 Subject: [PATCH 22/23] Test target URL --- retrieval/target_test.go | 76 +++++++++++++++++----------------------- 1 file changed, 32 insertions(+), 44 deletions(-) diff --git a/retrieval/target_test.go b/retrieval/target_test.go index e2d8a8778..851dc5403 100644 --- a/retrieval/target_test.go +++ b/retrieval/target_test.go @@ -20,7 +20,7 @@ import ( "io/ioutil" "net/http" "net/http/httptest" - // "net/url" + "net/url" "reflect" "strings" "testing" @@ -89,50 +89,38 @@ func TestTargetOffset(t *testing.T) { } } -// func TestTargetURLParams(t *testing.T) { -// server := httptest.NewServer( -// http.HandlerFunc( -// func(w http.ResponseWriter, r *http.Request) { -// w.Header().Set("Content-Type", `text/plain; version=0.0.4`) -// w.Write([]byte{}) -// r.ParseForm() -// if r.Form["foo"][0] != "bar" { -// t.Fatalf("URL parameter 'foo' had unexpected first value '%v'", r.Form["foo"][0]) -// } -// if r.Form["foo"][1] != "baz" { -// t.Fatalf("URL parameter 'foo' had unexpected second value '%v'", r.Form["foo"][1]) -// } -// }, -// ), -// ) -// defer server.Close() -// serverURL, err := url.Parse(server.URL) -// if err != nil { -// t.Fatal(err) -// } +func TestTargetURL(t *testing.T) { + params := url.Values{ + "abc": []string{"foo", "bar", "baz"}, + "xyz": []string{"hoo"}, + } + labels := model.LabelSet{ + model.AddressLabel: "example.com:1234", + model.SchemeLabel: "https", + model.MetricsPathLabel: "/metricz", + "__param_abc": "overwrite", + "__param_cde": "huu", + } + target := NewTarget(labels, labels, params) -// target, err := NewTarget( -// &config.ScrapeConfig{ -// JobName: "test_job1", -// Scheme: "https", -// Params: url.Values{ -// "foo": []string{"bar", "baz"}, -// }, -// }, -// model.LabelSet{ -// model.SchemeLabel: model.LabelValue(serverURL.Scheme), -// model.AddressLabel: model.LabelValue(serverURL.Host), -// "__param_foo": "bar_override", -// }, -// nil, -// ) -// if err != nil { -// t.Fatal(err) -// } -// if _, err = target.scrape(context.Background(), time.Now()); err != nil { -// t.Fatal(err) -// } -// } + // The reserved labels are concatenated into a full URL. The first value for each + // URL query parameter can be set/modified via labels as well. + expectedParams := url.Values{ + "abc": []string{"overwrite", "bar", "baz"}, + "cde": []string{"huu"}, + "xyz": []string{"hoo"}, + } + expectedURL := url.URL{ + Scheme: "https", + Host: "example.com:1234", + Path: "/metricz", + RawQuery: expectedParams.Encode(), + } + + if u := target.URL(); !reflect.DeepEqual(u.String(), expectedURL.String()) { + t.Fatalf("Expected URL %q but got %q", expectedURL, u) + } +} func newTestTarget(targetURL string, deadline time.Duration, labels model.LabelSet) *Target { labels = labels.Clone() From ddc74f712b05a26af1c7463fbb6003c78a53329b Mon Sep 17 00:00:00 2001 From: Fabian Reinartz Date: Wed, 2 Mar 2016 09:10:20 +0100 Subject: [PATCH 23/23] Add sortable target list --- retrieval/target.go | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/retrieval/target.go b/retrieval/target.go index 3f6bfa5e1..65d5ba123 100644 --- a/retrieval/target.go +++ b/retrieval/target.go @@ -221,6 +221,13 @@ func (t *Target) Health() TargetHealth { return t.health } +// Targets is a sortable list of targets. +type Targets []*Target + +func (ts Targets) Len() int { return len(ts) } +func (ts Targets) Less(i, j int) bool { return ts[i].URL().String() < ts[j].URL().String() } +func (ts Targets) Swap(i, j int) { ts[i], ts[j] = ts[j], ts[i] } + // Merges the ingested sample's metric with the label set. On a collision the // value of the ingested label is stored in a label prefixed with 'exported_'. type ruleLabelsAppender struct {