Remove outputs blocking inputs when output is slow (#4938)
This commit is contained in:
@@ -20,13 +20,13 @@ type MetricMaker interface {
|
||||
|
||||
type accumulator struct {
|
||||
maker MetricMaker
|
||||
metrics chan telegraf.Metric
|
||||
metrics chan<- telegraf.Metric
|
||||
precision time.Duration
|
||||
}
|
||||
|
||||
func NewAccumulator(
|
||||
maker MetricMaker,
|
||||
metrics chan telegraf.Metric,
|
||||
metrics chan<- telegraf.Metric,
|
||||
) telegraf.Accumulator {
|
||||
acc := accumulator{
|
||||
maker: maker,
|
||||
@@ -42,7 +42,7 @@ func (ac *accumulator) AddFields(
|
||||
tags map[string]string,
|
||||
t ...time.Time,
|
||||
) {
|
||||
ac.addMetric(measurement, tags, fields, telegraf.Untyped, t...)
|
||||
ac.addFields(measurement, tags, fields, telegraf.Untyped, t...)
|
||||
}
|
||||
|
||||
func (ac *accumulator) AddGauge(
|
||||
@@ -51,7 +51,7 @@ func (ac *accumulator) AddGauge(
|
||||
tags map[string]string,
|
||||
t ...time.Time,
|
||||
) {
|
||||
ac.addMetric(measurement, tags, fields, telegraf.Gauge, t...)
|
||||
ac.addFields(measurement, tags, fields, telegraf.Gauge, t...)
|
||||
}
|
||||
|
||||
func (ac *accumulator) AddCounter(
|
||||
@@ -60,7 +60,7 @@ func (ac *accumulator) AddCounter(
|
||||
tags map[string]string,
|
||||
t ...time.Time,
|
||||
) {
|
||||
ac.addMetric(measurement, tags, fields, telegraf.Counter, t...)
|
||||
ac.addFields(measurement, tags, fields, telegraf.Counter, t...)
|
||||
}
|
||||
|
||||
func (ac *accumulator) AddSummary(
|
||||
@@ -69,7 +69,7 @@ func (ac *accumulator) AddSummary(
|
||||
tags map[string]string,
|
||||
t ...time.Time,
|
||||
) {
|
||||
ac.addMetric(measurement, tags, fields, telegraf.Summary, t...)
|
||||
ac.addFields(measurement, tags, fields, telegraf.Summary, t...)
|
||||
}
|
||||
|
||||
func (ac *accumulator) AddHistogram(
|
||||
@@ -78,10 +78,16 @@ func (ac *accumulator) AddHistogram(
|
||||
tags map[string]string,
|
||||
t ...time.Time,
|
||||
) {
|
||||
ac.addMetric(measurement, tags, fields, telegraf.Histogram, t...)
|
||||
ac.addFields(measurement, tags, fields, telegraf.Histogram, t...)
|
||||
}
|
||||
|
||||
func (ac *accumulator) addMetric(
|
||||
func (ac *accumulator) AddMetric(m telegraf.Metric) {
|
||||
if m := ac.maker.MakeMetric(m); m != nil {
|
||||
ac.metrics <- m
|
||||
}
|
||||
}
|
||||
|
||||
func (ac *accumulator) addFields(
|
||||
measurement string,
|
||||
tags map[string]string,
|
||||
fields map[string]interface{},
|
||||
@@ -104,13 +110,9 @@ func (ac *accumulator) AddError(err error) {
|
||||
return
|
||||
}
|
||||
NErrors.Incr(1)
|
||||
log.Printf("E! Error in plugin [%s]: %s", ac.maker.Name(), err)
|
||||
log.Printf("E! [%s]: Error in plugin: %v", ac.maker.Name(), err)
|
||||
}
|
||||
|
||||
// SetPrecision takes two time.Duration objects. If the first is non-zero,
|
||||
// it sets that as the precision. Otherwise, it takes the second argument
|
||||
// as the order of time that the metrics should be rounded to, with the
|
||||
// maximum being 1s.
|
||||
func (ac *accumulator) SetPrecision(precision, interval time.Duration) {
|
||||
if precision > 0 {
|
||||
ac.precision = precision
|
||||
@@ -128,7 +130,7 @@ func (ac *accumulator) SetPrecision(precision, interval time.Duration) {
|
||||
}
|
||||
}
|
||||
|
||||
func (ac accumulator) getTime(t []time.Time) time.Time {
|
||||
func (ac *accumulator) getTime(t []time.Time) time.Time {
|
||||
var timestamp time.Time
|
||||
if len(t) > 0 {
|
||||
timestamp = t[0]
|
||||
@@ -137,3 +139,43 @@ func (ac accumulator) getTime(t []time.Time) time.Time {
|
||||
}
|
||||
return timestamp.Round(ac.precision)
|
||||
}
|
||||
|
||||
func (ac *accumulator) WithTracking(maxTracked int) telegraf.TrackingAccumulator {
|
||||
return &trackingAccumulator{
|
||||
Accumulator: ac,
|
||||
delivered: make(chan telegraf.DeliveryInfo, maxTracked),
|
||||
}
|
||||
}
|
||||
|
||||
type trackingAccumulator struct {
|
||||
telegraf.Accumulator
|
||||
delivered chan telegraf.DeliveryInfo
|
||||
}
|
||||
|
||||
func (a *trackingAccumulator) AddTrackingMetric(m telegraf.Metric) telegraf.TrackingID {
|
||||
dm, id := metric.WithTracking(m, a.onDelivery)
|
||||
a.AddMetric(dm)
|
||||
return id
|
||||
}
|
||||
|
||||
func (a *trackingAccumulator) AddTrackingMetricGroup(group []telegraf.Metric) telegraf.TrackingID {
|
||||
db, id := metric.WithGroupTracking(group, a.onDelivery)
|
||||
for _, m := range db {
|
||||
a.AddMetric(m)
|
||||
}
|
||||
return id
|
||||
}
|
||||
|
||||
func (a *trackingAccumulator) Delivered() <-chan telegraf.DeliveryInfo {
|
||||
return a.delivered
|
||||
}
|
||||
|
||||
func (a *trackingAccumulator) onDelivery(info telegraf.DeliveryInfo) {
|
||||
select {
|
||||
case a.delivered <- info:
|
||||
default:
|
||||
// This is a programming error in the input. More items were sent for
|
||||
// tracking than space requested.
|
||||
panic("channel is full")
|
||||
}
|
||||
}
|
||||
|
||||
898
agent/agent.go
898
agent/agent.go
@@ -1,9 +1,9 @@
|
||||
package agent
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"log"
|
||||
"os"
|
||||
"runtime"
|
||||
"sync"
|
||||
"time"
|
||||
@@ -12,187 +12,157 @@ import (
|
||||
"github.com/influxdata/telegraf/internal"
|
||||
"github.com/influxdata/telegraf/internal/config"
|
||||
"github.com/influxdata/telegraf/internal/models"
|
||||
"github.com/influxdata/telegraf/selfstat"
|
||||
"github.com/influxdata/telegraf/plugins/serializers/influx"
|
||||
)
|
||||
|
||||
// Agent runs telegraf and collects data based on the given config
|
||||
// Agent runs a set of plugins.
|
||||
type Agent struct {
|
||||
Config *config.Config
|
||||
}
|
||||
|
||||
// NewAgent returns an Agent struct based off the given Config
|
||||
// NewAgent returns an Agent for the given Config.
|
||||
func NewAgent(config *config.Config) (*Agent, error) {
|
||||
a := &Agent{
|
||||
Config: config,
|
||||
}
|
||||
|
||||
if !a.Config.Agent.OmitHostname {
|
||||
if a.Config.Agent.Hostname == "" {
|
||||
hostname, err := os.Hostname()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
a.Config.Agent.Hostname = hostname
|
||||
}
|
||||
|
||||
config.Tags["host"] = a.Config.Agent.Hostname
|
||||
}
|
||||
|
||||
return a, nil
|
||||
}
|
||||
|
||||
// Connect connects to all configured outputs
|
||||
func (a *Agent) Connect() error {
|
||||
for _, o := range a.Config.Outputs {
|
||||
switch ot := o.Output.(type) {
|
||||
case telegraf.ServiceOutput:
|
||||
if err := ot.Start(); err != nil {
|
||||
log.Printf("E! Service for output %s failed to start, exiting\n%s\n",
|
||||
o.Name, err.Error())
|
||||
return err
|
||||
}
|
||||
// Run starts and runs the Agent until the context is done.
|
||||
func (a *Agent) Run(ctx context.Context) error {
|
||||
log.Printf("I! [agent] Config: Interval:%s, Quiet:%#v, Hostname:%#v, "+
|
||||
"Flush Interval:%s",
|
||||
a.Config.Agent.Interval.Duration, a.Config.Agent.Quiet,
|
||||
a.Config.Agent.Hostname, a.Config.Agent.FlushInterval.Duration)
|
||||
|
||||
if ctx.Err() != nil {
|
||||
return ctx.Err()
|
||||
}
|
||||
|
||||
log.Printf("D! [agent] Connecting outputs")
|
||||
err := a.connectOutputs(ctx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
inputC := make(chan telegraf.Metric, 100)
|
||||
procC := make(chan telegraf.Metric, 100)
|
||||
outputC := make(chan telegraf.Metric, 100)
|
||||
|
||||
startTime := time.Now()
|
||||
|
||||
log.Printf("D! [agent] Starting service inputs")
|
||||
err = a.startServiceInputs(ctx, inputC)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
var wg sync.WaitGroup
|
||||
|
||||
src := inputC
|
||||
dst := inputC
|
||||
|
||||
wg.Add(1)
|
||||
go func(dst chan telegraf.Metric) {
|
||||
defer wg.Done()
|
||||
|
||||
err := a.runInputs(ctx, startTime, dst)
|
||||
if err != nil {
|
||||
log.Printf("E! [agent] Error running inputs: %v", err)
|
||||
}
|
||||
|
||||
log.Printf("D! Attempting connection to output: %s\n", o.Name)
|
||||
err := o.Output.Connect()
|
||||
if err != nil {
|
||||
log.Printf("E! Failed to connect to output %s, retrying in 15s, "+
|
||||
"error was '%s' \n", o.Name, err)
|
||||
time.Sleep(15 * time.Second)
|
||||
err = o.Output.Connect()
|
||||
log.Printf("D! [agent] Stopping service inputs")
|
||||
a.stopServiceInputs()
|
||||
|
||||
close(dst)
|
||||
log.Printf("D! [agent] Input channel closed")
|
||||
}(dst)
|
||||
|
||||
src = dst
|
||||
|
||||
if len(a.Config.Processors) > 0 {
|
||||
dst = procC
|
||||
|
||||
wg.Add(1)
|
||||
go func(src, dst chan telegraf.Metric) {
|
||||
defer wg.Done()
|
||||
|
||||
err := a.runProcessors(src, dst)
|
||||
if err != nil {
|
||||
return err
|
||||
log.Printf("E! [agent] Error running processors: %v", err)
|
||||
}
|
||||
}
|
||||
log.Printf("D! Successfully connected to output: %s\n", o.Name)
|
||||
close(dst)
|
||||
log.Printf("D! [agent] Processor channel closed")
|
||||
}(src, dst)
|
||||
|
||||
src = dst
|
||||
}
|
||||
|
||||
if len(a.Config.Aggregators) > 0 {
|
||||
dst = outputC
|
||||
|
||||
wg.Add(1)
|
||||
go func(src, dst chan telegraf.Metric) {
|
||||
defer wg.Done()
|
||||
|
||||
err := a.runAggregators(startTime, src, dst)
|
||||
if err != nil {
|
||||
log.Printf("E! [agent] Error running aggregators: %v", err)
|
||||
}
|
||||
close(dst)
|
||||
log.Printf("D! [agent] Output channel closed")
|
||||
}(src, dst)
|
||||
|
||||
src = dst
|
||||
}
|
||||
|
||||
wg.Add(1)
|
||||
go func(src chan telegraf.Metric) {
|
||||
defer wg.Done()
|
||||
|
||||
err := a.runOutputs(startTime, src)
|
||||
if err != nil {
|
||||
log.Printf("E! [agent] Error running outputs: %v", err)
|
||||
}
|
||||
}(src)
|
||||
|
||||
wg.Wait()
|
||||
|
||||
log.Printf("D! [agent] Closing outputs")
|
||||
err = a.closeOutputs()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// Close closes the connection to all configured outputs
|
||||
func (a *Agent) Close() error {
|
||||
var err error
|
||||
for _, o := range a.Config.Outputs {
|
||||
err = o.Output.Close()
|
||||
switch ot := o.Output.(type) {
|
||||
case telegraf.ServiceOutput:
|
||||
ot.Stop()
|
||||
}
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
func panicRecover(input *models.RunningInput) {
|
||||
if err := recover(); err != nil {
|
||||
trace := make([]byte, 2048)
|
||||
runtime.Stack(trace, true)
|
||||
log.Printf("E! FATAL: Input [%s] panicked: %s, Stack:\n%s\n",
|
||||
input.Name(), err, trace)
|
||||
log.Println("E! PLEASE REPORT THIS PANIC ON GITHUB with " +
|
||||
"stack trace, configuration, and OS information: " +
|
||||
"https://github.com/influxdata/telegraf/issues/new")
|
||||
}
|
||||
}
|
||||
|
||||
// gatherer runs the inputs that have been configured with their own
|
||||
// reporting interval.
|
||||
func (a *Agent) gatherer(
|
||||
shutdown chan struct{},
|
||||
input *models.RunningInput,
|
||||
interval time.Duration,
|
||||
metricC chan telegraf.Metric,
|
||||
) {
|
||||
defer panicRecover(input)
|
||||
|
||||
GatherTime := selfstat.RegisterTiming("gather",
|
||||
"gather_time_ns",
|
||||
map[string]string{"input": input.Config.Name},
|
||||
)
|
||||
|
||||
acc := NewAccumulator(input, metricC)
|
||||
acc.SetPrecision(a.Config.Agent.Precision.Duration,
|
||||
a.Config.Agent.Interval.Duration)
|
||||
|
||||
ticker := time.NewTicker(interval)
|
||||
defer ticker.Stop()
|
||||
|
||||
for {
|
||||
internal.RandomSleep(a.Config.Agent.CollectionJitter.Duration, shutdown)
|
||||
|
||||
start := time.Now()
|
||||
gatherWithTimeout(shutdown, input, acc, interval)
|
||||
elapsed := time.Since(start)
|
||||
|
||||
GatherTime.Incr(elapsed.Nanoseconds())
|
||||
|
||||
select {
|
||||
case <-shutdown:
|
||||
return
|
||||
case <-ticker.C:
|
||||
continue
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// gatherWithTimeout gathers from the given input, with the given timeout.
|
||||
// when the given timeout is reached, gatherWithTimeout logs an error message
|
||||
// but continues waiting for it to return. This is to avoid leaving behind
|
||||
// hung processes, and to prevent re-calling the same hung process over and
|
||||
// over.
|
||||
func gatherWithTimeout(
|
||||
shutdown chan struct{},
|
||||
input *models.RunningInput,
|
||||
acc telegraf.Accumulator,
|
||||
timeout time.Duration,
|
||||
) {
|
||||
ticker := time.NewTicker(timeout)
|
||||
defer ticker.Stop()
|
||||
done := make(chan error)
|
||||
go func() {
|
||||
done <- input.Input.Gather(acc)
|
||||
// Test runs the inputs once and prints the output to stdout in line protocol.
|
||||
func (a *Agent) Test() error {
|
||||
var wg sync.WaitGroup
|
||||
metricC := make(chan telegraf.Metric)
|
||||
defer func() {
|
||||
close(metricC)
|
||||
wg.Wait()
|
||||
}()
|
||||
|
||||
for {
|
||||
select {
|
||||
case err := <-done:
|
||||
if err != nil {
|
||||
acc.AddError(err)
|
||||
}
|
||||
return
|
||||
case <-ticker.C:
|
||||
err := fmt.Errorf("took longer to collect than collection interval (%s)",
|
||||
timeout)
|
||||
acc.AddError(err)
|
||||
continue
|
||||
case <-shutdown:
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Test verifies that we can 'Gather' from all inputs with their configured
|
||||
// Config struct
|
||||
func (a *Agent) Test() error {
|
||||
shutdown := make(chan struct{})
|
||||
defer close(shutdown)
|
||||
metricC := make(chan telegraf.Metric)
|
||||
|
||||
// dummy receiver for the point channel
|
||||
wg.Add(1)
|
||||
go func() {
|
||||
for {
|
||||
select {
|
||||
case <-metricC:
|
||||
// do nothing
|
||||
case <-shutdown:
|
||||
return
|
||||
defer wg.Done()
|
||||
|
||||
s := influx.NewSerializer()
|
||||
s.SetFieldSortOrder(influx.SortFields)
|
||||
for metric := range metricC {
|
||||
octets, err := s.Serialize(metric)
|
||||
if err == nil {
|
||||
fmt.Print("> ", string(octets))
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
for _, input := range a.Config.Inputs {
|
||||
if _, ok := input.Input.(telegraf.ServiceInput); ok {
|
||||
fmt.Printf("\nWARNING: skipping plugin [[%s]]: service inputs not supported in --test mode\n",
|
||||
log.Printf("W!: [agent] skipping plugin [[%s]]: service inputs not supported in --test mode",
|
||||
input.Name())
|
||||
continue
|
||||
}
|
||||
@@ -200,7 +170,6 @@ func (a *Agent) Test() error {
|
||||
acc := NewAccumulator(input, metricC)
|
||||
acc.SetPrecision(a.Config.Agent.Precision.Duration,
|
||||
a.Config.Agent.Interval.Duration)
|
||||
input.SetTrace(true)
|
||||
input.SetDefaultTags(a.Config.Tags)
|
||||
|
||||
if err := input.Input.Gather(acc); err != nil {
|
||||
@@ -218,216 +187,445 @@ func (a *Agent) Test() error {
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// flush writes a list of metrics to all configured outputs
|
||||
func (a *Agent) flush() {
|
||||
var wg sync.WaitGroup
|
||||
|
||||
wg.Add(len(a.Config.Outputs))
|
||||
for _, o := range a.Config.Outputs {
|
||||
go func(output *models.RunningOutput) {
|
||||
defer wg.Done()
|
||||
err := output.Write()
|
||||
if err != nil {
|
||||
log.Printf("E! Error writing to output [%s]: %s\n",
|
||||
output.Name, err.Error())
|
||||
}
|
||||
}(o)
|
||||
}
|
||||
|
||||
wg.Wait()
|
||||
}
|
||||
|
||||
// flusher monitors the metrics input channel and flushes on the minimum interval
|
||||
func (a *Agent) flusher(
|
||||
shutdown chan struct{},
|
||||
metricC chan telegraf.Metric,
|
||||
aggMetricC chan telegraf.Metric,
|
||||
outMetricC chan telegraf.Metric,
|
||||
// runInputs starts and triggers the periodic gather for Inputs.
|
||||
//
|
||||
// When the context is done the timers are stopped and this function returns
|
||||
// after all ongoing Gather calls complete.
|
||||
func (a *Agent) runInputs(
|
||||
ctx context.Context,
|
||||
startTime time.Time,
|
||||
dst chan<- telegraf.Metric,
|
||||
) error {
|
||||
var wg sync.WaitGroup
|
||||
wg.Add(1)
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
for {
|
||||
select {
|
||||
case <-shutdown:
|
||||
if len(outMetricC) > 0 {
|
||||
// keep going until channel is empty
|
||||
continue
|
||||
}
|
||||
return
|
||||
case metric := <-outMetricC:
|
||||
for i, o := range a.Config.Outputs {
|
||||
if i == len(a.Config.Outputs)-1 {
|
||||
o.AddMetric(metric)
|
||||
} else {
|
||||
o.AddMetric(metric.Copy())
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
wg.Add(1)
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
for metric := range aggMetricC {
|
||||
// Apply Processors
|
||||
metrics := []telegraf.Metric{metric}
|
||||
for _, processor := range a.Config.Processors {
|
||||
metrics = processor.Apply(metrics...)
|
||||
}
|
||||
outMetricC <- metric
|
||||
}
|
||||
}()
|
||||
|
||||
wg.Add(1)
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
for {
|
||||
select {
|
||||
case <-shutdown:
|
||||
if len(metricC) > 0 {
|
||||
// keep going until channel is empty
|
||||
continue
|
||||
}
|
||||
close(aggMetricC)
|
||||
return
|
||||
case metric := <-metricC:
|
||||
// Apply Processors
|
||||
metrics := []telegraf.Metric{metric}
|
||||
for _, processor := range a.Config.Processors {
|
||||
metrics = processor.Apply(metrics...)
|
||||
}
|
||||
|
||||
for _, metric := range metrics {
|
||||
// Apply Aggregators
|
||||
var dropOriginal bool
|
||||
for _, agg := range a.Config.Aggregators {
|
||||
if ok := agg.Add(metric.Copy()); ok {
|
||||
dropOriginal = true
|
||||
}
|
||||
}
|
||||
|
||||
// Forward metric to Outputs
|
||||
if !dropOriginal {
|
||||
outMetricC <- metric
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
ticker := time.NewTicker(a.Config.Agent.FlushInterval.Duration)
|
||||
semaphore := make(chan struct{}, 1)
|
||||
for {
|
||||
select {
|
||||
case <-shutdown:
|
||||
log.Println("I! Hang on, flushing any cached metrics before shutdown")
|
||||
// wait for outMetricC to get flushed before flushing outputs
|
||||
wg.Wait()
|
||||
a.flush()
|
||||
return nil
|
||||
case <-ticker.C:
|
||||
go func() {
|
||||
select {
|
||||
case semaphore <- struct{}{}:
|
||||
internal.RandomSleep(a.Config.Agent.FlushJitter.Duration, shutdown)
|
||||
a.flush()
|
||||
<-semaphore
|
||||
default:
|
||||
// skipping this flush because one is already happening
|
||||
log.Println("W! Skipping a scheduled flush because there is" +
|
||||
" already a flush ongoing.")
|
||||
}
|
||||
}()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Run runs the agent daemon, gathering every Interval
|
||||
func (a *Agent) Run(shutdown chan struct{}) error {
|
||||
var wg sync.WaitGroup
|
||||
|
||||
log.Printf("I! Agent Config: Interval:%s, Quiet:%#v, Hostname:%#v, "+
|
||||
"Flush Interval:%s \n",
|
||||
a.Config.Agent.Interval.Duration, a.Config.Agent.Quiet,
|
||||
a.Config.Agent.Hostname, a.Config.Agent.FlushInterval.Duration)
|
||||
|
||||
// Channel shared between all input threads for accumulating metrics
|
||||
metricC := make(chan telegraf.Metric, 100)
|
||||
|
||||
// Channel for metrics ready to be output
|
||||
outMetricC := make(chan telegraf.Metric, 100)
|
||||
|
||||
// Channel for aggregated metrics
|
||||
aggMetricC := make(chan telegraf.Metric, 100)
|
||||
|
||||
// Round collection to nearest interval by sleeping
|
||||
if a.Config.Agent.RoundInterval {
|
||||
i := int64(a.Config.Agent.Interval.Duration)
|
||||
time.Sleep(time.Duration(i - (time.Now().UnixNano() % i)))
|
||||
}
|
||||
|
||||
wg.Add(1)
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
if err := a.flusher(shutdown, metricC, aggMetricC, outMetricC); err != nil {
|
||||
log.Printf("E! Flusher routine failed, exiting: %s\n", err.Error())
|
||||
close(shutdown)
|
||||
}
|
||||
}()
|
||||
|
||||
wg.Add(len(a.Config.Aggregators))
|
||||
for _, aggregator := range a.Config.Aggregators {
|
||||
go func(agg *models.RunningAggregator) {
|
||||
defer wg.Done()
|
||||
acc := NewAccumulator(agg, aggMetricC)
|
||||
acc.SetPrecision(a.Config.Agent.Precision.Duration,
|
||||
a.Config.Agent.Interval.Duration)
|
||||
agg.Run(acc, shutdown)
|
||||
}(aggregator)
|
||||
}
|
||||
|
||||
// Service inputs may immediately add metrics, if metrics are added before
|
||||
// the aggregator starts they will be dropped. Generally this occurs
|
||||
// only during testing but it is an outstanding issue.
|
||||
//
|
||||
// https://github.com/influxdata/telegraf/issues/4394
|
||||
for _, input := range a.Config.Inputs {
|
||||
input.SetDefaultTags(a.Config.Tags)
|
||||
switch p := input.Input.(type) {
|
||||
case telegraf.ServiceInput:
|
||||
acc := NewAccumulator(input, metricC)
|
||||
// Service input plugins should set their own precision of their
|
||||
// metrics.
|
||||
acc.SetPrecision(time.Nanosecond, 0)
|
||||
if err := p.Start(acc); err != nil {
|
||||
log.Printf("E! Service for input %s failed to start, exiting\n%s\n",
|
||||
input.Name(), err.Error())
|
||||
return err
|
||||
}
|
||||
defer p.Stop()
|
||||
}
|
||||
}
|
||||
|
||||
wg.Add(len(a.Config.Inputs))
|
||||
for _, input := range a.Config.Inputs {
|
||||
interval := a.Config.Agent.Interval.Duration
|
||||
// overwrite global interval if this plugin has it's own.
|
||||
precision := a.Config.Agent.Precision.Duration
|
||||
jitter := a.Config.Agent.CollectionJitter.Duration
|
||||
|
||||
// Overwrite agent interval if this plugin has its own.
|
||||
if input.Config.Interval != 0 {
|
||||
interval = input.Config.Interval
|
||||
}
|
||||
go func(in *models.RunningInput, interv time.Duration) {
|
||||
|
||||
acc := NewAccumulator(input, dst)
|
||||
acc.SetPrecision(precision, interval)
|
||||
|
||||
wg.Add(1)
|
||||
go func(input *models.RunningInput) {
|
||||
defer wg.Done()
|
||||
a.gatherer(shutdown, in, interv, metricC)
|
||||
}(input, interval)
|
||||
|
||||
if a.Config.Agent.RoundInterval {
|
||||
err := internal.SleepContext(
|
||||
ctx, internal.AlignDuration(startTime, interval))
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
a.gatherOnInterval(ctx, acc, input, interval, jitter)
|
||||
}(input)
|
||||
}
|
||||
wg.Wait()
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// gather runs an input's gather function periodically until the context is
|
||||
// done.
|
||||
func (a *Agent) gatherOnInterval(
|
||||
ctx context.Context,
|
||||
acc telegraf.Accumulator,
|
||||
input *models.RunningInput,
|
||||
interval time.Duration,
|
||||
jitter time.Duration,
|
||||
) {
|
||||
defer panicRecover(input)
|
||||
|
||||
ticker := time.NewTicker(interval)
|
||||
defer ticker.Stop()
|
||||
|
||||
for {
|
||||
err := internal.SleepContext(ctx, internal.RandomDuration(jitter))
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
err = a.gatherOnce(acc, input, interval)
|
||||
if err != nil {
|
||||
acc.AddError(err)
|
||||
}
|
||||
|
||||
select {
|
||||
case <-ticker.C:
|
||||
continue
|
||||
case <-ctx.Done():
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// gatherOnce runs the input's Gather function once, logging a warning each
|
||||
// interval it fails to complete before.
|
||||
func (a *Agent) gatherOnce(
|
||||
acc telegraf.Accumulator,
|
||||
input *models.RunningInput,
|
||||
timeout time.Duration,
|
||||
) error {
|
||||
ticker := time.NewTicker(timeout)
|
||||
defer ticker.Stop()
|
||||
|
||||
done := make(chan error)
|
||||
go func() {
|
||||
done <- input.Gather(acc)
|
||||
}()
|
||||
|
||||
for {
|
||||
select {
|
||||
case err := <-done:
|
||||
return err
|
||||
case <-ticker.C:
|
||||
log.Printf("W! [agent] input %q did not complete within its interval",
|
||||
input.Name())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// runProcessors applies processors to metrics.
|
||||
func (a *Agent) runProcessors(
|
||||
src <-chan telegraf.Metric,
|
||||
agg chan<- telegraf.Metric,
|
||||
) error {
|
||||
for metric := range src {
|
||||
metrics := a.applyProcessors(metric)
|
||||
|
||||
for _, metric := range metrics {
|
||||
agg <- metric
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// applyProcessors applies all processors to a metric.
|
||||
func (a *Agent) applyProcessors(m telegraf.Metric) []telegraf.Metric {
|
||||
metrics := []telegraf.Metric{m}
|
||||
for _, processor := range a.Config.Processors {
|
||||
metrics = processor.Apply(metrics...)
|
||||
}
|
||||
|
||||
return metrics
|
||||
}
|
||||
|
||||
// runAggregators triggers the periodic push for Aggregators.
|
||||
//
|
||||
// When the context is done a final push will occur and then this function
|
||||
// will return.
|
||||
func (a *Agent) runAggregators(
|
||||
startTime time.Time,
|
||||
src <-chan telegraf.Metric,
|
||||
dst chan<- telegraf.Metric,
|
||||
) error {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
|
||||
var wg sync.WaitGroup
|
||||
wg.Add(1)
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
for metric := range src {
|
||||
var dropOriginal bool
|
||||
for _, agg := range a.Config.Aggregators {
|
||||
if ok := agg.Add(metric); ok {
|
||||
dropOriginal = true
|
||||
}
|
||||
}
|
||||
|
||||
if !dropOriginal {
|
||||
dst <- metric
|
||||
}
|
||||
}
|
||||
cancel()
|
||||
}()
|
||||
|
||||
precision := a.Config.Agent.Precision.Duration
|
||||
interval := a.Config.Agent.Interval.Duration
|
||||
aggregations := make(chan telegraf.Metric, 100)
|
||||
for _, agg := range a.Config.Aggregators {
|
||||
wg.Add(1)
|
||||
go func(agg *models.RunningAggregator) {
|
||||
defer wg.Done()
|
||||
|
||||
if a.Config.Agent.RoundInterval {
|
||||
// Aggregators are aligned to the agent interval regardless of
|
||||
// their period.
|
||||
err := internal.SleepContext(ctx, internal.AlignDuration(startTime, interval))
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
agg.SetPeriodStart(startTime)
|
||||
|
||||
acc := NewAccumulator(agg, aggregations)
|
||||
acc.SetPrecision(precision, interval)
|
||||
a.push(ctx, agg, acc)
|
||||
close(aggregations)
|
||||
}(agg)
|
||||
}
|
||||
|
||||
for metric := range aggregations {
|
||||
metrics := a.applyProcessors(metric)
|
||||
for _, metric := range metrics {
|
||||
dst <- metric
|
||||
}
|
||||
}
|
||||
|
||||
wg.Wait()
|
||||
a.Close()
|
||||
return nil
|
||||
}
|
||||
|
||||
// push runs the push for a single aggregator every period. More simple than
|
||||
// the output/input version as timeout should be less likely.... not really
|
||||
// because the output channel can block for now.
|
||||
func (a *Agent) push(
|
||||
ctx context.Context,
|
||||
aggregator *models.RunningAggregator,
|
||||
acc telegraf.Accumulator,
|
||||
) {
|
||||
ticker := time.NewTicker(aggregator.Period())
|
||||
defer ticker.Stop()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ticker.C:
|
||||
break
|
||||
case <-ctx.Done():
|
||||
aggregator.Push(acc)
|
||||
return
|
||||
}
|
||||
|
||||
aggregator.Push(acc)
|
||||
}
|
||||
}
|
||||
|
||||
// runOutputs triggers the periodic write for Outputs.
|
||||
//
|
||||
// When the context is done, outputs continue to run until their buffer is
|
||||
// closed, afterwich they run flush once more.
|
||||
func (a *Agent) runOutputs(
|
||||
startTime time.Time,
|
||||
src <-chan telegraf.Metric,
|
||||
) error {
|
||||
interval := a.Config.Agent.FlushInterval.Duration
|
||||
jitter := a.Config.Agent.FlushJitter.Duration
|
||||
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
|
||||
var wg sync.WaitGroup
|
||||
for _, output := range a.Config.Outputs {
|
||||
interval := interval
|
||||
// Overwrite agent flush_interval if this plugin has its own.
|
||||
if output.Config.FlushInterval != 0 {
|
||||
interval = output.Config.FlushInterval
|
||||
}
|
||||
|
||||
wg.Add(1)
|
||||
go func(output *models.RunningOutput) {
|
||||
defer wg.Done()
|
||||
|
||||
if a.Config.Agent.RoundInterval {
|
||||
err := internal.SleepContext(
|
||||
ctx, internal.AlignDuration(startTime, interval))
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
a.flush(ctx, output, interval, jitter)
|
||||
}(output)
|
||||
}
|
||||
|
||||
for metric := range src {
|
||||
for i, output := range a.Config.Outputs {
|
||||
if i == len(a.Config.Outputs)-1 {
|
||||
output.AddMetric(metric)
|
||||
} else {
|
||||
output.AddMetric(metric.Copy())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
log.Println("I! [agent] Hang on, flushing any cached metrics before shutdown")
|
||||
cancel()
|
||||
wg.Wait()
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// flush runs an output's flush function periodically until the context is
|
||||
// done.
|
||||
func (a *Agent) flush(
|
||||
ctx context.Context,
|
||||
output *models.RunningOutput,
|
||||
interval time.Duration,
|
||||
jitter time.Duration,
|
||||
) {
|
||||
// since we are watching two channels we need a ticker with the jitter
|
||||
// integrated.
|
||||
ticker := NewTicker(interval, jitter)
|
||||
defer ticker.Stop()
|
||||
|
||||
logError := func(err error) {
|
||||
if err != nil {
|
||||
log.Printf("E! [agent] Error writing to output [%s]: %v", output.Name, err)
|
||||
}
|
||||
}
|
||||
|
||||
for {
|
||||
// Favor shutdown over other methods.
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
logError(a.flushOnce(output, interval, output.Write))
|
||||
return
|
||||
default:
|
||||
}
|
||||
|
||||
select {
|
||||
case <-ticker.C:
|
||||
logError(a.flushOnce(output, interval, output.Write))
|
||||
case <-output.BatchReady:
|
||||
// Favor the ticker over batch ready
|
||||
select {
|
||||
case <-ticker.C:
|
||||
logError(a.flushOnce(output, interval, output.Write))
|
||||
default:
|
||||
logError(a.flushOnce(output, interval, output.WriteBatch))
|
||||
}
|
||||
case <-ctx.Done():
|
||||
logError(a.flushOnce(output, interval, output.Write))
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// flushOnce runs the output's Write function once, logging a warning each
|
||||
// interval it fails to complete before.
|
||||
func (a *Agent) flushOnce(
|
||||
output *models.RunningOutput,
|
||||
timeout time.Duration,
|
||||
writeFunc func() error,
|
||||
) error {
|
||||
ticker := time.NewTicker(timeout)
|
||||
defer ticker.Stop()
|
||||
|
||||
done := make(chan error)
|
||||
go func() {
|
||||
done <- writeFunc()
|
||||
}()
|
||||
|
||||
for {
|
||||
select {
|
||||
case err := <-done:
|
||||
output.LogBufferStatus()
|
||||
return err
|
||||
case <-ticker.C:
|
||||
log.Printf("W! [agent] output %q did not complete within its flush interval",
|
||||
output.Name)
|
||||
output.LogBufferStatus()
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
// connectOutputs connects to all outputs.
|
||||
func (a *Agent) connectOutputs(ctx context.Context) error {
|
||||
for _, output := range a.Config.Outputs {
|
||||
log.Printf("D! [agent] Attempting connection to output: %s\n", output.Name)
|
||||
err := output.Output.Connect()
|
||||
if err != nil {
|
||||
log.Printf("E! [agent] Failed to connect to output %s, retrying in 15s, "+
|
||||
"error was '%s' \n", output.Name, err)
|
||||
|
||||
err := internal.SleepContext(ctx, 15*time.Second)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
err = output.Output.Connect()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
log.Printf("D! [agent] Successfully connected to output: %s\n", output.Name)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// closeOutputs closes all outputs.
|
||||
func (a *Agent) closeOutputs() error {
|
||||
var err error
|
||||
for _, output := range a.Config.Outputs {
|
||||
err = output.Output.Close()
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
// startServiceInputs starts all service inputs.
|
||||
func (a *Agent) startServiceInputs(
|
||||
ctx context.Context,
|
||||
dst chan<- telegraf.Metric,
|
||||
) error {
|
||||
started := []telegraf.ServiceInput{}
|
||||
|
||||
for _, input := range a.Config.Inputs {
|
||||
if si, ok := input.Input.(telegraf.ServiceInput); ok {
|
||||
// Service input plugins are not subject to timestamp rounding.
|
||||
// This only applies to the accumulator passed to Start(), the
|
||||
// Gather() accumulator does apply rounding according to the
|
||||
// precision agent setting.
|
||||
acc := NewAccumulator(input, dst)
|
||||
acc.SetPrecision(time.Nanosecond, 0)
|
||||
|
||||
err := si.Start(acc)
|
||||
if err != nil {
|
||||
log.Printf("E! [agent] Service for input %s failed to start: %v",
|
||||
input.Name(), err)
|
||||
|
||||
for _, si := range started {
|
||||
si.Stop()
|
||||
}
|
||||
|
||||
return err
|
||||
}
|
||||
|
||||
started = append(started, si)
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// stopServiceInputs stops all service inputs.
|
||||
func (a *Agent) stopServiceInputs() {
|
||||
for _, input := range a.Config.Inputs {
|
||||
if si, ok := input.Input.(telegraf.ServiceInput); ok {
|
||||
si.Stop()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// panicRecover displays an error if an input panics.
|
||||
func panicRecover(input *models.RunningInput) {
|
||||
if err := recover(); err != nil {
|
||||
trace := make([]byte, 2048)
|
||||
runtime.Stack(trace, true)
|
||||
log.Printf("E! FATAL: Input [%s] panicked: %s, Stack:\n%s\n",
|
||||
input.Name(), err, trace)
|
||||
log.Println("E! PLEASE REPORT THIS PANIC ON GITHUB with " +
|
||||
"stack trace, configuration, and OS information: " +
|
||||
"https://github.com/influxdata/telegraf/issues/new/choose")
|
||||
}
|
||||
}
|
||||
|
||||
57
agent/tick.go
Normal file
57
agent/tick.go
Normal file
@@ -0,0 +1,57 @@
|
||||
package agent
|
||||
|
||||
import (
|
||||
"context"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/influxdata/telegraf/internal"
|
||||
)
|
||||
|
||||
type Ticker struct {
|
||||
C chan time.Time
|
||||
ticker *time.Ticker
|
||||
jitter time.Duration
|
||||
wg sync.WaitGroup
|
||||
cancelFunc context.CancelFunc
|
||||
}
|
||||
|
||||
func NewTicker(
|
||||
interval time.Duration,
|
||||
jitter time.Duration,
|
||||
) *Ticker {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
|
||||
t := &Ticker{
|
||||
C: make(chan time.Time, 1),
|
||||
ticker: time.NewTicker(interval),
|
||||
jitter: jitter,
|
||||
cancelFunc: cancel,
|
||||
}
|
||||
|
||||
t.wg.Add(1)
|
||||
go t.relayTime(ctx)
|
||||
|
||||
return t
|
||||
}
|
||||
|
||||
func (t *Ticker) Stop() {
|
||||
t.cancelFunc()
|
||||
t.wg.Wait()
|
||||
}
|
||||
|
||||
func (t *Ticker) relayTime(ctx context.Context) {
|
||||
defer t.wg.Done()
|
||||
for {
|
||||
select {
|
||||
case tm := <-t.ticker.C:
|
||||
internal.SleepContext(ctx, internal.RandomDuration(t.jitter))
|
||||
select {
|
||||
case t.C <- tm:
|
||||
default:
|
||||
}
|
||||
case <-ctx.Done():
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user