Add ability to label inputs for logging (#6207)

This commit is contained in:
Greg 2019-08-21 17:49:07 -06:00 committed by Daniel Nelson
parent bc52592c87
commit 5c8d0e3ac9
22 changed files with 475 additions and 103 deletions

View File

@ -14,7 +14,7 @@ var (
) )
type MetricMaker interface { type MetricMaker interface {
Name() string LogName() string
MakeMetric(metric telegraf.Metric) telegraf.Metric MakeMetric(metric telegraf.Metric) telegraf.Metric
} }
@ -111,7 +111,7 @@ func (ac *accumulator) AddError(err error) {
return return
} }
NErrors.Incr(1) NErrors.Incr(1)
log.Printf("E! [%s]: Error in plugin: %v", ac.maker.Name(), err) log.Printf("E! [%s] Error in plugin: %v", ac.maker.LogName(), err)
} }
func (ac *accumulator) SetPrecision(precision time.Duration) { func (ac *accumulator) SetPrecision(precision time.Duration) {

View File

@ -147,6 +147,10 @@ func (tm *TestMetricMaker) Name() string {
return "TestPlugin" return "TestPlugin"
} }
func (tm *TestMetricMaker) LogName() string {
return tm.Name()
}
func (tm *TestMetricMaker) MakeMetric(metric telegraf.Metric) telegraf.Metric { func (tm *TestMetricMaker) MakeMetric(metric telegraf.Metric) telegraf.Metric {
return metric return metric
} }

View File

@ -209,7 +209,7 @@ func (a *Agent) Test(ctx context.Context, waitDuration time.Duration) error {
// Special instructions for some inputs. cpu, for example, needs to be // Special instructions for some inputs. cpu, for example, needs to be
// run twice in order to return cpu usage percentages. // run twice in order to return cpu usage percentages.
switch input.Name() { switch input.Config.Name {
case "inputs.cpu", "inputs.mongodb", "inputs.procstat": case "inputs.cpu", "inputs.mongodb", "inputs.procstat":
nulAcc := NewAccumulator(input, nulC) nulAcc := NewAccumulator(input, nulC)
nulAcc.SetPrecision(a.Precision()) nulAcc.SetPrecision(a.Precision())
@ -337,8 +337,8 @@ func (a *Agent) gatherOnce(
case err := <-done: case err := <-done:
return err return err
case <-ticker.C: case <-ticker.C:
log.Printf("W! [agent] input %q did not complete within its interval", log.Printf("W! [agent] [%s] did not complete within its interval",
input.Name()) input.LogName())
} }
} }
} }
@ -551,7 +551,7 @@ func (a *Agent) flush(
logError := func(err error) { logError := func(err error) {
if err != nil { if err != nil {
log.Printf("E! [agent] Error writing to output [%s]: %v", output.Name, err) log.Printf("E! [agent] Error writing to %s: %v", output.LogName(), err)
} }
} }
@ -603,8 +603,8 @@ func (a *Agent) flushOnce(
output.LogBufferStatus() output.LogBufferStatus()
return err return err
case <-ticker.C: case <-ticker.C:
log.Printf("W! [agent] output %q did not complete within its flush interval", log.Printf("W! [agent] [%q] did not complete within its flush interval",
output.Name) output.LogName())
output.LogBufferStatus() output.LogBufferStatus()
} }
} }
@ -617,7 +617,7 @@ func (a *Agent) initPlugins() error {
err := input.Init() err := input.Init()
if err != nil { if err != nil {
return fmt.Errorf("could not initialize input %s: %v", return fmt.Errorf("could not initialize input %s: %v",
input.Config.Name, err) input.LogName(), err)
} }
} }
for _, processor := range a.Config.Processors { for _, processor := range a.Config.Processors {
@ -647,11 +647,11 @@ func (a *Agent) initPlugins() error {
// connectOutputs connects to all outputs. // connectOutputs connects to all outputs.
func (a *Agent) connectOutputs(ctx context.Context) error { func (a *Agent) connectOutputs(ctx context.Context) error {
for _, output := range a.Config.Outputs { for _, output := range a.Config.Outputs {
log.Printf("D! [agent] Attempting connection to output: %s\n", output.Name) log.Printf("D! [agent] Attempting connection to [%s]", output.LogName())
err := output.Output.Connect() err := output.Output.Connect()
if err != nil { if err != nil {
log.Printf("E! [agent] Failed to connect to output %s, retrying in 15s, "+ log.Printf("E! [agent] Failed to connect to [%s], retrying in 15s, "+
"error was '%s' \n", output.Name, err) "error was '%s'", output.LogName(), err)
err := internal.SleepContext(ctx, 15*time.Second) err := internal.SleepContext(ctx, 15*time.Second)
if err != nil { if err != nil {
@ -663,7 +663,7 @@ func (a *Agent) connectOutputs(ctx context.Context) error {
return err return err
} }
} }
log.Printf("D! [agent] Successfully connected to output: %s\n", output.Name) log.Printf("D! [agent] Successfully connected to %s", output.LogName())
} }
return nil return nil
} }
@ -693,8 +693,8 @@ func (a *Agent) startServiceInputs(
err := si.Start(acc) err := si.Start(acc)
if err != nil { if err != nil {
log.Printf("E! [agent] Service for input %s failed to start: %v", log.Printf("E! [agent] Service for [%s] failed to start: %v",
input.Name(), err) input.LogName(), err)
for _, si := range started { for _, si := range started {
si.Stop() si.Stop()
@ -745,8 +745,8 @@ func panicRecover(input *models.RunningInput) {
if err := recover(); err != nil { if err := recover(); err != nil {
trace := make([]byte, 2048) trace := make([]byte, 2048)
runtime.Stack(trace, true) runtime.Stack(trace, true)
log.Printf("E! FATAL: Input [%s] panicked: %s, Stack:\n%s\n", log.Printf("E! FATAL: [%s] panicked: %s, Stack:\n%s",
input.Name(), err, trace) input.LogName(), err, trace)
log.Println("E! PLEASE REPORT THIS PANIC ON GITHUB with " + log.Println("E! PLEASE REPORT THIS PANIC ON GITHUB with " +
"stack trace, configuration, and OS information: " + "stack trace, configuration, and OS information: " +
"https://github.com/influxdata/telegraf/issues/new/choose") "https://github.com/influxdata/telegraf/issues/new/choose")

View File

@ -1,14 +1,5 @@
package telegraf package telegraf
// Initializer is an interface that all plugin types: Inputs, Outputs,
// Processors, and Aggregators can optionally implement to initialize the
// plugin.
type Initializer interface {
// Init performs one time setup of the plugin and returns an error if the
// configuration is invalid.
Init() error
}
type Input interface { type Input interface {
// SampleConfig returns the default configuration of the Input // SampleConfig returns the default configuration of the Input
SampleConfig() string SampleConfig() string

View File

@ -187,7 +187,7 @@ func (c *Config) AggregatorNames() []string {
func (c *Config) ProcessorNames() []string { func (c *Config) ProcessorNames() []string {
var name []string var name []string
for _, processor := range c.Processors { for _, processor := range c.Processors {
name = append(name, processor.Name) name = append(name, processor.Config.Name)
} }
return name return name
} }
@ -196,7 +196,7 @@ func (c *Config) ProcessorNames() []string {
func (c *Config) OutputNames() []string { func (c *Config) OutputNames() []string {
var name []string var name []string
for _, output := range c.Outputs { for _, output := range c.Outputs {
name = append(name, output.Name) name = append(name, output.Config.Name)
} }
return name return name
} }
@ -920,11 +920,7 @@ func (c *Config) addProcessor(name string, table *ast.Table) error {
return err return err
} }
rf := &models.RunningProcessor{ rf := models.NewRunningProcessor(processor, processorConfig)
Name: name,
Processor: processor,
Config: processorConfig,
}
c.Processors = append(c.Processors, rf) c.Processors = append(c.Processors, rf)
return nil return nil
@ -1103,6 +1099,14 @@ func buildAggregator(name string, tbl *ast.Table) (*models.AggregatorConfig, err
} }
} }
if node, ok := tbl.Fields["alias"]; ok {
if kv, ok := node.(*ast.KeyValue); ok {
if str, ok := kv.Value.(*ast.String); ok {
conf.Alias = str.Value
}
}
}
conf.Tags = make(map[string]string) conf.Tags = make(map[string]string)
if node, ok := tbl.Fields["tags"]; ok { if node, ok := tbl.Fields["tags"]; ok {
if subtbl, ok := node.(*ast.Table); ok { if subtbl, ok := node.(*ast.Table); ok {
@ -1119,6 +1123,7 @@ func buildAggregator(name string, tbl *ast.Table) (*models.AggregatorConfig, err
delete(tbl.Fields, "name_prefix") delete(tbl.Fields, "name_prefix")
delete(tbl.Fields, "name_suffix") delete(tbl.Fields, "name_suffix")
delete(tbl.Fields, "name_override") delete(tbl.Fields, "name_override")
delete(tbl.Fields, "alias")
delete(tbl.Fields, "tags") delete(tbl.Fields, "tags")
var err error var err error
conf.Filter, err = buildFilter(tbl) conf.Filter, err = buildFilter(tbl)
@ -1146,6 +1151,15 @@ func buildProcessor(name string, tbl *ast.Table) (*models.ProcessorConfig, error
} }
} }
if node, ok := tbl.Fields["alias"]; ok {
if kv, ok := node.(*ast.KeyValue); ok {
if str, ok := kv.Value.(*ast.String); ok {
conf.Alias = str.Value
}
}
}
delete(tbl.Fields, "alias")
delete(tbl.Fields, "order") delete(tbl.Fields, "order")
var err error var err error
conf.Filter, err = buildFilter(tbl) conf.Filter, err = buildFilter(tbl)
@ -1334,6 +1348,14 @@ func buildInput(name string, tbl *ast.Table) (*models.InputConfig, error) {
} }
} }
if node, ok := tbl.Fields["alias"]; ok {
if kv, ok := node.(*ast.KeyValue); ok {
if str, ok := kv.Value.(*ast.String); ok {
cp.Alias = str.Value
}
}
}
cp.Tags = make(map[string]string) cp.Tags = make(map[string]string)
if node, ok := tbl.Fields["tags"]; ok { if node, ok := tbl.Fields["tags"]; ok {
if subtbl, ok := node.(*ast.Table); ok { if subtbl, ok := node.(*ast.Table); ok {
@ -1346,6 +1368,7 @@ func buildInput(name string, tbl *ast.Table) (*models.InputConfig, error) {
delete(tbl.Fields, "name_prefix") delete(tbl.Fields, "name_prefix")
delete(tbl.Fields, "name_suffix") delete(tbl.Fields, "name_suffix")
delete(tbl.Fields, "name_override") delete(tbl.Fields, "name_override")
delete(tbl.Fields, "alias")
delete(tbl.Fields, "interval") delete(tbl.Fields, "interval")
delete(tbl.Fields, "tags") delete(tbl.Fields, "tags")
var err error var err error
@ -2007,9 +2030,18 @@ func buildOutput(name string, tbl *ast.Table) (*models.OutputConfig, error) {
} }
} }
if node, ok := tbl.Fields["alias"]; ok {
if kv, ok := node.(*ast.KeyValue); ok {
if str, ok := kv.Value.(*ast.String); ok {
oc.Alias = str.Value
}
}
}
delete(tbl.Fields, "flush_interval") delete(tbl.Fields, "flush_interval")
delete(tbl.Fields, "metric_buffer_limit") delete(tbl.Fields, "metric_buffer_limit")
delete(tbl.Fields, "metric_batch_size") delete(tbl.Fields, "metric_batch_size")
delete(tbl.Fields, "alias")
return oc, nil return oc, nil
} }

87
internal/models/log.go Normal file
View File

@ -0,0 +1,87 @@
package models
import (
"log"
"reflect"
"github.com/influxdata/telegraf"
"github.com/influxdata/telegraf/selfstat"
)
// Logger defines a logging structure for plugins.
type Logger struct {
Errs selfstat.Stat
Name string // Name is the plugin name, will be printed in the `[]`.
}
// Errorf logs an error message, patterned after log.Printf.
func (l *Logger) Errorf(format string, args ...interface{}) {
l.Errs.Incr(1)
log.Printf("E! ["+l.Name+"] "+format, args...)
}
// Error logs an error message, patterned after log.Print.
func (l *Logger) Error(args ...interface{}) {
l.Errs.Incr(1)
log.Print(append([]interface{}{"E! [" + l.Name + "] "}, args...)...)
}
// Debugf logs a debug message, patterned after log.Printf.
func (l *Logger) Debugf(format string, args ...interface{}) {
log.Printf("D! ["+l.Name+"] "+format, args...)
}
// Debug logs a debug message, patterned after log.Print.
func (l *Logger) Debug(args ...interface{}) {
log.Print(append([]interface{}{"D! [" + l.Name + "] "}, args...)...)
}
// Warnf logs a warning message, patterned after log.Printf.
func (l *Logger) Warnf(format string, args ...interface{}) {
log.Printf("W! ["+l.Name+"] "+format, args...)
}
// Warn logs a warning message, patterned after log.Print.
func (l *Logger) Warn(args ...interface{}) {
log.Print(append([]interface{}{"W! [" + l.Name + "] "}, args...)...)
}
// Infof logs an information message, patterned after log.Printf.
func (l *Logger) Infof(format string, args ...interface{}) {
log.Printf("I! ["+l.Name+"] "+format, args...)
}
// Info logs an information message, patterned after log.Print.
func (l *Logger) Info(args ...interface{}) {
log.Print(append([]interface{}{"I! [" + l.Name + "] "}, args...)...)
}
// logName returns the log-friendly name/type.
func logName(pluginType, name, alias string) string {
if alias == "" {
return pluginType + "." + name
}
return pluginType + "." + name + "::" + alias
}
func setLogIfExist(i interface{}, log telegraf.Logger) {
valI := reflect.ValueOf(i)
if valI.Type().Kind() != reflect.Ptr {
valI = reflect.New(reflect.TypeOf(i))
}
field := valI.Elem().FieldByName("Log")
if !field.IsValid() {
return
}
switch field.Type().String() {
case "telegraf.Logger":
if field.CanSet() {
field.Set(reflect.ValueOf(log))
}
}
return
}

View File

@ -0,0 +1,70 @@
package models
import (
"testing"
"github.com/influxdata/telegraf/selfstat"
"github.com/stretchr/testify/require"
)
func TestErrorCounting(t *testing.T) {
iLog := Logger{Name: "inputs.test", Errs: selfstat.Register(
"gather",
"errors",
map[string]string{"input": "test"},
)}
iLog.Error("something went wrong")
iLog.Errorf("something went wrong")
aLog := Logger{Name: "aggregators.test", Errs: selfstat.Register(
"aggregate",
"errors",
map[string]string{"aggregator": "test"},
)}
aLog.Name = "aggregators.test"
aLog.Error("another thing happened")
oLog := Logger{Name: "outputs.test", Errs: selfstat.Register(
"write",
"errors",
map[string]string{"output": "test"},
)}
oLog.Error("another thing happened")
pLog := Logger{Name: "processors.test", Errs: selfstat.Register(
"process",
"errors",
map[string]string{"processor": "test"},
)}
pLog.Error("another thing happened")
require.Equal(t, int64(2), iLog.Errs.Get())
require.Equal(t, int64(1), aLog.Errs.Get())
require.Equal(t, int64(1), oLog.Errs.Get())
require.Equal(t, int64(1), pLog.Errs.Get())
}
func TestLogging(t *testing.T) {
log := Logger{Name: "inputs.test", Errs: selfstat.Register(
"gather",
"errors",
map[string]string{"input": "test"},
)}
log.Errs.Set(0)
log.Debugf("something happened")
log.Debug("something happened")
log.Warnf("something happened")
log.Warn("something happened")
require.Equal(t, int64(0), log.Errs.Get())
log.Infof("something happened")
log.Info("something happened")
require.Equal(t, int64(0), log.Errs.Get())
log.Errorf("something happened")
log.Error("something happened")
require.Equal(t, int64(2), log.Errs.Get())
}

View File

@ -1,7 +1,6 @@
package models package models
import ( import (
"log"
"sync" "sync"
"time" "time"
@ -16,6 +15,7 @@ type RunningAggregator struct {
Config *AggregatorConfig Config *AggregatorConfig
periodStart time.Time periodStart time.Time
periodEnd time.Time periodEnd time.Time
log telegraf.Logger
MetricsPushed selfstat.Stat MetricsPushed selfstat.Stat
MetricsFiltered selfstat.Stat MetricsFiltered selfstat.Stat
@ -23,39 +23,46 @@ type RunningAggregator struct {
PushTime selfstat.Stat PushTime selfstat.Stat
} }
func NewRunningAggregator( func NewRunningAggregator(aggregator telegraf.Aggregator, config *AggregatorConfig) *RunningAggregator {
aggregator telegraf.Aggregator, logger := &Logger{
config *AggregatorConfig, Name: logName("aggregators", config.Name, config.Alias),
) *RunningAggregator { Errs: selfstat.Register("aggregate", "errors",
map[string]string{"input": config.Name, "alias": config.Alias}),
}
setLogIfExist(aggregator, logger)
return &RunningAggregator{ return &RunningAggregator{
Aggregator: aggregator, Aggregator: aggregator,
Config: config, Config: config,
MetricsPushed: selfstat.Register( MetricsPushed: selfstat.Register(
"aggregate", "aggregate",
"metrics_pushed", "metrics_pushed",
map[string]string{"aggregator": config.Name}, map[string]string{"aggregator": config.Name, "alias": config.Alias},
), ),
MetricsFiltered: selfstat.Register( MetricsFiltered: selfstat.Register(
"aggregate", "aggregate",
"metrics_filtered", "metrics_filtered",
map[string]string{"aggregator": config.Name}, map[string]string{"aggregator": config.Name, "alias": config.Alias},
), ),
MetricsDropped: selfstat.Register( MetricsDropped: selfstat.Register(
"aggregate", "aggregate",
"metrics_dropped", "metrics_dropped",
map[string]string{"aggregator": config.Name}, map[string]string{"aggregator": config.Name, "alias": config.Alias},
), ),
PushTime: selfstat.Register( PushTime: selfstat.Register(
"aggregate", "aggregate",
"push_time_ns", "push_time_ns",
map[string]string{"aggregator": config.Name}, map[string]string{"aggregator": config.Name, "alias": config.Alias},
), ),
log: logger,
} }
} }
// AggregatorConfig is the common config for all aggregators. // AggregatorConfig is the common config for all aggregators.
type AggregatorConfig struct { type AggregatorConfig struct {
Name string Name string
Alias string
DropOriginal bool DropOriginal bool
Period time.Duration Period time.Duration
Delay time.Duration Delay time.Duration
@ -68,8 +75,8 @@ type AggregatorConfig struct {
Filter Filter Filter Filter
} }
func (r *RunningAggregator) Name() string { func (r *RunningAggregator) LogName() string {
return "aggregators." + r.Config.Name return logName("aggregators", r.Config.Name, r.Config.Alias)
} }
func (r *RunningAggregator) Init() error { func (r *RunningAggregator) Init() error {
@ -93,7 +100,7 @@ func (r *RunningAggregator) EndPeriod() time.Time {
func (r *RunningAggregator) UpdateWindow(start, until time.Time) { func (r *RunningAggregator) UpdateWindow(start, until time.Time) {
r.periodStart = start r.periodStart = start
r.periodEnd = until r.periodEnd = until
log.Printf("D! [%s] Updated aggregation range [%s, %s]", r.Name(), start, until) r.log.Debugf("Updated aggregation range [%s, %s]", start, until)
} }
func (r *RunningAggregator) MakeMetric(metric telegraf.Metric) telegraf.Metric { func (r *RunningAggregator) MakeMetric(metric telegraf.Metric) telegraf.Metric {
@ -137,8 +144,8 @@ func (r *RunningAggregator) Add(m telegraf.Metric) bool {
defer r.Unlock() defer r.Unlock()
if m.Time().Before(r.periodStart.Add(-r.Config.Grace)) || m.Time().After(r.periodEnd.Add(r.Config.Delay)) { if m.Time().Before(r.periodStart.Add(-r.Config.Grace)) || m.Time().After(r.periodEnd.Add(r.Config.Delay)) {
log.Printf("D! [%s] metric is outside aggregation window; discarding. %s: m: %s e: %s g: %s", r.log.Debugf("metric is outside aggregation window; discarding. %s: m: %s e: %s g: %s",
r.Name(), m.Time(), r.periodStart, r.periodEnd, r.Config.Grace) m.Time(), r.periodStart, r.periodEnd, r.Config.Grace)
r.MetricsDropped.Incr(1) r.MetricsDropped.Incr(1)
return r.Config.DropOriginal return r.Config.DropOriginal
} }

View File

@ -13,6 +13,7 @@ type RunningInput struct {
Input telegraf.Input Input telegraf.Input
Config *InputConfig Config *InputConfig
log telegraf.Logger
defaultTags map[string]string defaultTags map[string]string
MetricsGathered selfstat.Stat MetricsGathered selfstat.Stat
@ -20,25 +21,35 @@ type RunningInput struct {
} }
func NewRunningInput(input telegraf.Input, config *InputConfig) *RunningInput { func NewRunningInput(input telegraf.Input, config *InputConfig) *RunningInput {
logger := &Logger{
Name: logName("inputs", config.Name, config.Alias),
Errs: selfstat.Register("gather", "errors",
map[string]string{"input": config.Name, "alias": config.Alias}),
}
setLogIfExist(input, logger)
return &RunningInput{ return &RunningInput{
Input: input, Input: input,
Config: config, Config: config,
MetricsGathered: selfstat.Register( MetricsGathered: selfstat.Register(
"gather", "gather",
"metrics_gathered", "metrics_gathered",
map[string]string{"input": config.Name}, map[string]string{"input": config.Name, "alias": config.Alias},
), ),
GatherTime: selfstat.RegisterTiming( GatherTime: selfstat.RegisterTiming(
"gather", "gather",
"gather_time_ns", "gather_time_ns",
map[string]string{"input": config.Name}, map[string]string{"input": config.Name, "alias": config.Alias},
), ),
log: logger,
} }
} }
// InputConfig is the common config for all inputs. // InputConfig is the common config for all inputs.
type InputConfig struct { type InputConfig struct {
Name string Name string
Alias string
Interval time.Duration Interval time.Duration
NameOverride string NameOverride string
@ -48,14 +59,14 @@ type InputConfig struct {
Filter Filter Filter Filter
} }
func (r *RunningInput) Name() string {
return "inputs." + r.Config.Name
}
func (r *RunningInput) metricFiltered(metric telegraf.Metric) { func (r *RunningInput) metricFiltered(metric telegraf.Metric) {
metric.Drop() metric.Drop()
} }
func (r *RunningInput) LogName() string {
return logName("inputs", r.Config.Name, r.Config.Alias)
}
func (r *RunningInput) Init() error { func (r *RunningInput) Init() error {
if p, ok := r.Input.(telegraf.Initializer); ok { if p, ok := r.Input.(telegraf.Initializer); ok {
err := p.Init() err := p.Init()

View File

@ -1,7 +1,6 @@
package models package models
import ( import (
"log"
"sync" "sync"
"sync/atomic" "sync/atomic"
"time" "time"
@ -21,6 +20,7 @@ const (
// OutputConfig containing name and filter // OutputConfig containing name and filter
type OutputConfig struct { type OutputConfig struct {
Name string Name string
Alias string
Filter Filter Filter Filter
FlushInterval time.Duration FlushInterval time.Duration
@ -34,7 +34,6 @@ type RunningOutput struct {
newMetricsCount int64 newMetricsCount int64
droppedMetrics int64 droppedMetrics int64
Name string
Output telegraf.Output Output telegraf.Output
Config *OutputConfig Config *OutputConfig
MetricBufferLimit int MetricBufferLimit int
@ -46,6 +45,7 @@ type RunningOutput struct {
BatchReady chan time.Time BatchReady chan time.Time
buffer *Buffer buffer *Buffer
log telegraf.Logger
aggMutex sync.Mutex aggMutex sync.Mutex
} }
@ -53,56 +53,77 @@ type RunningOutput struct {
func NewRunningOutput( func NewRunningOutput(
name string, name string,
output telegraf.Output, output telegraf.Output,
conf *OutputConfig, config *OutputConfig,
batchSize int, batchSize int,
bufferLimit int, bufferLimit int,
) *RunningOutput { ) *RunningOutput {
if conf.MetricBufferLimit > 0 { logger := &Logger{
bufferLimit = conf.MetricBufferLimit Name: logName("outputs", config.Name, config.Alias),
Errs: selfstat.Register("gather", "errors",
map[string]string{"output": config.Name, "alias": config.Alias}),
}
setLogIfExist(output, logger)
if config.MetricBufferLimit > 0 {
bufferLimit = config.MetricBufferLimit
} }
if bufferLimit == 0 { if bufferLimit == 0 {
bufferLimit = DEFAULT_METRIC_BUFFER_LIMIT bufferLimit = DEFAULT_METRIC_BUFFER_LIMIT
} }
if conf.MetricBatchSize > 0 { if config.MetricBatchSize > 0 {
batchSize = conf.MetricBatchSize batchSize = config.MetricBatchSize
} }
if batchSize == 0 { if batchSize == 0 {
batchSize = DEFAULT_METRIC_BATCH_SIZE batchSize = DEFAULT_METRIC_BATCH_SIZE
} }
ro := &RunningOutput{ ro := &RunningOutput{
Name: name, buffer: NewBuffer(config.LogName(), bufferLimit),
buffer: NewBuffer(name, bufferLimit),
BatchReady: make(chan time.Time, 1), BatchReady: make(chan time.Time, 1),
Output: output, Output: output,
Config: conf, Config: config,
MetricBufferLimit: bufferLimit, MetricBufferLimit: bufferLimit,
MetricBatchSize: batchSize, MetricBatchSize: batchSize,
MetricsFiltered: selfstat.Register( MetricsFiltered: selfstat.Register(
"write", "write",
"metrics_filtered", "metrics_filtered",
map[string]string{"output": name}, map[string]string{"output": config.Name, "alias": config.Alias},
), ),
WriteTime: selfstat.RegisterTiming( WriteTime: selfstat.RegisterTiming(
"write", "write",
"write_time_ns", "write_time_ns",
map[string]string{"output": name}, map[string]string{"output": config.Name, "alias": config.Alias},
), ),
log: logger,
} }
return ro return ro
} }
func (c *OutputConfig) LogName() string {
if c.Alias == "" {
return c.Name
}
return c.Name + "::" + c.Alias
}
func (r *RunningOutput) LogName() string {
return logName("outputs", r.Config.Name, r.Config.Alias)
}
func (ro *RunningOutput) metricFiltered(metric telegraf.Metric) { func (ro *RunningOutput) metricFiltered(metric telegraf.Metric) {
ro.MetricsFiltered.Incr(1) ro.MetricsFiltered.Incr(1)
metric.Drop() metric.Drop()
} }
func (ro *RunningOutput) Init() error { func (r *RunningOutput) Init() error {
if p, ok := ro.Output.(telegraf.Initializer); ok { if p, ok := r.Output.(telegraf.Initializer); ok {
err := p.Init() err := p.Init()
if err != nil { if err != nil {
return err return err
} }
} }
return nil return nil
} }
@ -192,35 +213,32 @@ func (ro *RunningOutput) WriteBatch() error {
return nil return nil
} }
func (ro *RunningOutput) Close() { func (r *RunningOutput) Close() {
err := ro.Output.Close() err := r.Output.Close()
if err != nil { if err != nil {
log.Printf("E! [outputs.%s] Error closing output: %v", ro.Name, err) r.log.Errorf("Error closing output: %v", err)
} }
} }
func (ro *RunningOutput) write(metrics []telegraf.Metric) error { func (r *RunningOutput) write(metrics []telegraf.Metric) error {
dropped := atomic.LoadInt64(&ro.droppedMetrics) dropped := atomic.LoadInt64(&r.droppedMetrics)
if dropped > 0 { if dropped > 0 {
log.Printf("W! [outputs.%s] Metric buffer overflow; %d metrics have been dropped", r.log.Warnf("Metric buffer overflow; %d metrics have been dropped", dropped)
ro.Name, dropped) atomic.StoreInt64(&r.droppedMetrics, 0)
atomic.StoreInt64(&ro.droppedMetrics, 0)
} }
start := time.Now() start := time.Now()
err := ro.Output.Write(metrics) err := r.Output.Write(metrics)
elapsed := time.Since(start) elapsed := time.Since(start)
ro.WriteTime.Incr(elapsed.Nanoseconds()) r.WriteTime.Incr(elapsed.Nanoseconds())
if err == nil { if err == nil {
log.Printf("D! [outputs.%s] wrote batch of %d metrics in %s\n", r.log.Debugf("Wrote batch of %d metrics in %s", len(metrics), elapsed)
ro.Name, len(metrics), elapsed)
} }
return err return err
} }
func (ro *RunningOutput) LogBufferStatus() { func (r *RunningOutput) LogBufferStatus() {
nBuffer := ro.buffer.Len() nBuffer := r.buffer.Len()
log.Printf("D! [outputs.%s] buffer fullness: %d / %d metrics. ", r.log.Debugf("Buffer fullness: %d / %d metrics", nBuffer, r.MetricBufferLimit)
ro.Name, nBuffer, ro.MetricBufferLimit)
} }

View File

@ -4,12 +4,12 @@ import (
"sync" "sync"
"github.com/influxdata/telegraf" "github.com/influxdata/telegraf"
"github.com/influxdata/telegraf/selfstat"
) )
type RunningProcessor struct { type RunningProcessor struct {
Name string
sync.Mutex sync.Mutex
log telegraf.Logger
Processor telegraf.Processor Processor telegraf.Processor
Config *ProcessorConfig Config *ProcessorConfig
} }
@ -23,10 +23,27 @@ func (rp RunningProcessors) Less(i, j int) bool { return rp[i].Config.Order < rp
// FilterConfig containing a name and filter // FilterConfig containing a name and filter
type ProcessorConfig struct { type ProcessorConfig struct {
Name string Name string
Alias string
Order int64 Order int64
Filter Filter Filter Filter
} }
func NewRunningProcessor(processor telegraf.Processor, config *ProcessorConfig) *RunningProcessor {
logger := &Logger{
Name: logName("processors", config.Name, config.Alias),
Errs: selfstat.Register("process", "errors",
map[string]string{"input": config.Name, "alias": config.Alias}),
}
setLogIfExist(processor, logger)
return &RunningProcessor{
Processor: processor,
Config: config,
log: logger,
}
}
func (rp *RunningProcessor) metricFiltered(metric telegraf.Metric) { func (rp *RunningProcessor) metricFiltered(metric telegraf.Metric) {
metric.Drop() metric.Drop()
} }
@ -40,8 +57,8 @@ func containsMetric(item telegraf.Metric, metrics []telegraf.Metric) bool {
return false return false
} }
func (rp *RunningProcessor) Init() error { func (r *RunningProcessor) Init() error {
if p, ok := rp.Processor.(telegraf.Initializer); ok { if p, ok := r.Processor.(telegraf.Initializer); ok {
err := p.Init() err := p.Init()
if err != nil { if err != nil {
return err return err

30
plugin.go Normal file
View File

@ -0,0 +1,30 @@
package telegraf
// Initializer is an interface that all plugin types: Inputs, Outputs,
// Processors, and Aggregators can optionally implement to initialize the
// plugin.
type Initializer interface {
// Init performs one time setup of the plugin and returns an error if the
// configuration is invalid.
Init() error
}
// Logger defines an interface for logging.
type Logger interface {
// Errorf logs an error message, patterned after log.Printf.
Errorf(format string, args ...interface{})
// Error logs an error message, patterned after log.Print.
Error(args ...interface{})
// Debugf logs a debug message, patterned after log.Printf.
Debugf(format string, args ...interface{})
// Debug logs a debug message, patterned after log.Print.
Debug(args ...interface{})
// Warnf logs a warning message, patterned after log.Printf.
Warnf(format string, args ...interface{})
// Warn logs a warning message, patterned after log.Print.
Warn(args ...interface{})
// Infof logs an information message, patterned after log.Printf.
Infof(format string, args ...interface{})
// Info logs an information message, patterned after log.Print.
Info(args ...interface{})
}

View File

@ -183,6 +183,10 @@ func (tm *testMetricMaker) Name() string {
return "TestPlugin" return "TestPlugin"
} }
func (tm *testMetricMaker) LogName() string {
return tm.Name()
}
func (tm *testMetricMaker) MakeMetric(metric telegraf.Metric) telegraf.Metric { func (tm *testMetricMaker) MakeMetric(metric telegraf.Metric) telegraf.Metric {
return metric return metric
} }

View File

@ -3,7 +3,6 @@ package exec
import ( import (
"bytes" "bytes"
"fmt" "fmt"
"log"
"os/exec" "os/exec"
"path/filepath" "path/filepath"
"runtime" "runtime"
@ -51,6 +50,7 @@ type Exec struct {
parser parsers.Parser parser parsers.Parser
runner Runner runner Runner
log telegraf.Logger
} }
func NewExec() *Exec { func NewExec() *Exec {
@ -161,7 +161,7 @@ func (e *Exec) ProcessCommand(command string, acc telegraf.Accumulator, wg *sync
if isNagios { if isNagios {
metrics, err = nagios.TryAddState(runErr, metrics) metrics, err = nagios.TryAddState(runErr, metrics)
if err != nil { if err != nil {
log.Printf("E! [inputs.exec] failed to add nagios state: %s", err) e.log.Errorf("failed to add nagios state: %s", err)
} }
} }
@ -229,6 +229,10 @@ func (e *Exec) Gather(acc telegraf.Accumulator) error {
return nil return nil
} }
func (e *Exec) Init() error {
return nil
}
func init() { func init() {
inputs.Add("exec", func() telegraf.Input { inputs.Add("exec", func() telegraf.Input {
return NewExec() return NewExec()

View File

@ -8,7 +8,6 @@ import (
"time" "time"
"github.com/influxdata/telegraf/plugins/parsers" "github.com/influxdata/telegraf/plugins/parsers"
"github.com/influxdata/telegraf/testutil" "github.com/influxdata/telegraf/testutil"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
@ -97,6 +96,7 @@ func TestExec(t *testing.T) {
MetricName: "exec", MetricName: "exec",
}) })
e := &Exec{ e := &Exec{
log: testutil.Logger{},
runner: newRunnerMock([]byte(validJson), nil, nil), runner: newRunnerMock([]byte(validJson), nil, nil),
Commands: []string{"testcommand arg1"}, Commands: []string{"testcommand arg1"},
parser: parser, parser: parser,
@ -126,6 +126,7 @@ func TestExecMalformed(t *testing.T) {
MetricName: "exec", MetricName: "exec",
}) })
e := &Exec{ e := &Exec{
log: testutil.Logger{},
runner: newRunnerMock([]byte(malformedJson), nil, nil), runner: newRunnerMock([]byte(malformedJson), nil, nil),
Commands: []string{"badcommand arg1"}, Commands: []string{"badcommand arg1"},
parser: parser, parser: parser,
@ -142,6 +143,7 @@ func TestCommandError(t *testing.T) {
MetricName: "exec", MetricName: "exec",
}) })
e := &Exec{ e := &Exec{
log: testutil.Logger{},
runner: newRunnerMock(nil, nil, fmt.Errorf("exit status code 1")), runner: newRunnerMock(nil, nil, fmt.Errorf("exit status code 1")),
Commands: []string{"badcommand"}, Commands: []string{"badcommand"},
parser: parser, parser: parser,

View File

@ -6,7 +6,6 @@ import (
"encoding/json" "encoding/json"
"fmt" "fmt"
"io" "io"
"log"
"net" "net"
"net/http" "net/http"
"net/url" "net/url"
@ -101,12 +100,15 @@ type HTTPConfig struct {
InfluxUintSupport bool `toml:"influx_uint_support"` InfluxUintSupport bool `toml:"influx_uint_support"`
Serializer *influx.Serializer Serializer *influx.Serializer
log telegraf.Logger
} }
type httpClient struct { type httpClient struct {
client *http.Client client *http.Client
config HTTPConfig config HTTPConfig
createdDatabases map[string]bool createdDatabases map[string]bool
log telegraf.Logger
} }
func NewHTTPClient(config HTTPConfig) (*httpClient, error) { func NewHTTPClient(config HTTPConfig) (*httpClient, error) {
@ -174,6 +176,7 @@ func NewHTTPClient(config HTTPConfig) (*httpClient, error) {
}, },
createdDatabases: make(map[string]bool), createdDatabases: make(map[string]bool),
config: config, config: config,
log: config.log,
} }
return client, nil return client, nil
} }
@ -183,6 +186,10 @@ func (c *httpClient) URL() string {
return c.config.URL.String() return c.config.URL.String()
} }
func (c *httpClient) SetLogger(log telegraf.Logger) {
c.log = log
}
// Database returns the default database that this client connects too. // Database returns the default database that this client connects too.
func (c *httpClient) Database() string { func (c *httpClient) Database() string {
return c.config.Database return c.config.Database
@ -262,7 +269,7 @@ func (c *httpClient) Write(ctx context.Context, metrics []telegraf.Metric) error
if !c.config.SkipDatabaseCreation && !c.createdDatabases[db] { if !c.config.SkipDatabaseCreation && !c.createdDatabases[db] {
err := c.CreateDatabase(ctx, db) err := c.CreateDatabase(ctx, db)
if err != nil { if err != nil {
log.Printf("W! [outputs.influxdb] when writing to [%s]: database %q creation failed: %v", c.log.Warnf("when writing to [%s]: database %q creation failed: %v",
c.config.URL, db, err) c.config.URL, db, err)
} }
} }
@ -328,7 +335,7 @@ func (c *httpClient) writeBatch(ctx context.Context, db string, metrics []telegr
// discarded for being older than the retention policy. Usually this not // discarded for being older than the retention policy. Usually this not
// a cause for concern and we don't want to retry. // a cause for concern and we don't want to retry.
if strings.Contains(desc, errStringPointsBeyondRP) { if strings.Contains(desc, errStringPointsBeyondRP) {
log.Printf("W! [outputs.influxdb]: when writing to [%s]: received error %v", c.log.Warnf("when writing to [%s]: received error %v",
c.URL(), desc) c.URL(), desc)
return nil return nil
} }
@ -337,7 +344,7 @@ func (c *httpClient) writeBatch(ctx context.Context, db string, metrics []telegr
// correctable at this point and so the point is dropped instead of // correctable at this point and so the point is dropped instead of
// retrying. // retrying.
if strings.Contains(desc, errStringPartialWrite) { if strings.Contains(desc, errStringPartialWrite) {
log.Printf("E! [outputs.influxdb]: when writing to [%s]: received error %v; discarding points", c.log.Errorf("when writing to [%s]: received error %v; discarding points",
c.URL(), desc) c.URL(), desc)
return nil return nil
} }
@ -345,7 +352,7 @@ func (c *httpClient) writeBatch(ctx context.Context, db string, metrics []telegr
// This error indicates a bug in either Telegraf line protocol // This error indicates a bug in either Telegraf line protocol
// serialization, retries would not be successful. // serialization, retries would not be successful.
if strings.Contains(desc, errStringUnableToParse) { if strings.Contains(desc, errStringUnableToParse) {
log.Printf("E! [outputs.influxdb]: when writing to [%s]: received error %v; discarding points", c.log.Errorf("when writing to [%s]: received error %v; discarding points",
c.URL(), desc) c.URL(), desc)
return nil return nil
} }

View File

@ -21,6 +21,7 @@ import (
"github.com/influxdata/telegraf/internal" "github.com/influxdata/telegraf/internal"
"github.com/influxdata/telegraf/metric" "github.com/influxdata/telegraf/metric"
"github.com/influxdata/telegraf/plugins/outputs/influxdb" "github.com/influxdata/telegraf/plugins/outputs/influxdb"
"github.com/influxdata/telegraf/testutil"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
) )
@ -471,6 +472,7 @@ func TestHTTP_Write(t *testing.T) {
client, err := influxdb.NewHTTPClient(tt.config) client, err := influxdb.NewHTTPClient(tt.config)
require.NoError(t, err) require.NoError(t, err)
client.SetLogger(testutil.Logger{})
err = client.Write(ctx, metrics) err = client.Write(ctx, metrics)
if tt.errFunc != nil { if tt.errFunc != nil {
tt.errFunc(t, err) tt.errFunc(t, err)

View File

@ -4,7 +4,6 @@ import (
"context" "context"
"errors" "errors"
"fmt" "fmt"
"log"
"math/rand" "math/rand"
"net/url" "net/url"
"time" "time"
@ -28,6 +27,7 @@ type Client interface {
Database() string Database() string
URL() string URL() string
Close() Close()
SetLogger(telegraf.Logger)
} }
// InfluxDB struct is the primary data structure for the plugin // InfluxDB struct is the primary data structure for the plugin
@ -59,6 +59,7 @@ type InfluxDB struct {
CreateUDPClientF func(config *UDPConfig) (Client, error) CreateUDPClientF func(config *UDPConfig) (Client, error)
serializer *influx.Serializer serializer *influx.Serializer
Log telegraf.Logger
} }
var sampleConfig = ` var sampleConfig = `
@ -171,6 +172,8 @@ func (i *InfluxDB) Connect() error {
return err return err
} }
c.SetLogger(i.Log)
i.clients = append(i.clients, c) i.clients = append(i.clients, c)
case "http", "https", "unix": case "http", "https", "unix":
c, err := i.httpClient(ctx, parts, proxy) c, err := i.httpClient(ctx, parts, proxy)
@ -178,6 +181,8 @@ func (i *InfluxDB) Connect() error {
return err return err
} }
c.SetLogger(i.Log)
i.clients = append(i.clients, c) i.clients = append(i.clients, c)
default: default:
return fmt.Errorf("unsupported scheme [%q]: %q", u, parts.Scheme) return fmt.Errorf("unsupported scheme [%q]: %q", u, parts.Scheme)
@ -221,13 +226,13 @@ func (i *InfluxDB) Write(metrics []telegraf.Metric) error {
if !i.SkipDatabaseCreation { if !i.SkipDatabaseCreation {
err := client.CreateDatabase(ctx, apiError.Database) err := client.CreateDatabase(ctx, apiError.Database)
if err != nil { if err != nil {
log.Printf("E! [outputs.influxdb] when writing to [%s]: database %q not found and failed to recreate", i.Log.Errorf("when writing to [%s]: database %q not found and failed to recreate",
client.URL(), apiError.Database) client.URL(), apiError.Database)
} }
} }
} }
log.Printf("E! [outputs.influxdb] when writing to [%s]: %v", client.URL(), err) i.Log.Errorf("when writing to [%s]: %v", client.URL(), err)
} }
return errors.New("could not write any address") return errors.New("could not write any address")
@ -281,7 +286,7 @@ func (i *InfluxDB) httpClient(ctx context.Context, url *url.URL, proxy *url.URL)
if !i.SkipDatabaseCreation { if !i.SkipDatabaseCreation {
err = c.CreateDatabase(ctx, c.Database()) err = c.CreateDatabase(ctx, c.Database())
if err != nil { if err != nil {
log.Printf("W! [outputs.influxdb] when writing to [%s]: database %q creation failed: %v", i.Log.Warnf("when writing to [%s]: database %q creation failed: %v",
c.URL(), i.Database, err) c.URL(), i.Database, err)
} }
} }

View File

@ -11,6 +11,7 @@ import (
"github.com/influxdata/telegraf/internal/tls" "github.com/influxdata/telegraf/internal/tls"
"github.com/influxdata/telegraf/metric" "github.com/influxdata/telegraf/metric"
"github.com/influxdata/telegraf/plugins/outputs/influxdb" "github.com/influxdata/telegraf/plugins/outputs/influxdb"
"github.com/influxdata/telegraf/testutil"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
) )
@ -20,6 +21,8 @@ type MockClient struct {
CreateDatabaseF func(ctx context.Context, database string) error CreateDatabaseF func(ctx context.Context, database string) error
DatabaseF func() string DatabaseF func() string
CloseF func() CloseF func()
log telegraf.Logger
} }
func (c *MockClient) URL() string { func (c *MockClient) URL() string {
@ -42,6 +45,10 @@ func (c *MockClient) Close() {
c.CloseF() c.CloseF()
} }
func (c *MockClient) SetLogger(log telegraf.Logger) {
c.log = log
}
func TestDeprecatedURLSupport(t *testing.T) { func TestDeprecatedURLSupport(t *testing.T) {
var actual *influxdb.UDPConfig var actual *influxdb.UDPConfig
output := influxdb.InfluxDB{ output := influxdb.InfluxDB{
@ -52,6 +59,9 @@ func TestDeprecatedURLSupport(t *testing.T) {
return &MockClient{}, nil return &MockClient{}, nil
}, },
} }
output.Log = testutil.Logger{}
err := output.Connect() err := output.Connect()
require.NoError(t, err) require.NoError(t, err)
require.Equal(t, "udp://localhost:8089", actual.URL.String()) require.Equal(t, "udp://localhost:8089", actual.URL.String())
@ -72,6 +82,9 @@ func TestDefaultURL(t *testing.T) {
}, nil }, nil
}, },
} }
output.Log = testutil.Logger{}
err := output.Connect() err := output.Connect()
require.NoError(t, err) require.NoError(t, err)
require.Equal(t, "http://localhost:8086", actual.URL.String()) require.Equal(t, "http://localhost:8086", actual.URL.String())
@ -89,6 +102,8 @@ func TestConnectUDPConfig(t *testing.T) {
return &MockClient{}, nil return &MockClient{}, nil
}, },
} }
output.Log = testutil.Logger{}
err := output.Connect() err := output.Connect()
require.NoError(t, err) require.NoError(t, err)
@ -130,6 +145,9 @@ func TestConnectHTTPConfig(t *testing.T) {
}, nil }, nil
}, },
} }
output.Log = testutil.Logger{}
err := output.Connect() err := output.Connect()
require.NoError(t, err) require.NoError(t, err)
@ -153,7 +171,6 @@ func TestConnectHTTPConfig(t *testing.T) {
func TestWriteRecreateDatabaseIfDatabaseNotFound(t *testing.T) { func TestWriteRecreateDatabaseIfDatabaseNotFound(t *testing.T) {
output := influxdb.InfluxDB{ output := influxdb.InfluxDB{
URLs: []string{"http://localhost:8086"}, URLs: []string{"http://localhost:8086"},
CreateHTTPClientF: func(config *influxdb.HTTPConfig) (influxdb.Client, error) { CreateHTTPClientF: func(config *influxdb.HTTPConfig) (influxdb.Client, error) {
return &MockClient{ return &MockClient{
DatabaseF: func() string { DatabaseF: func() string {
@ -173,12 +190,13 @@ func TestWriteRecreateDatabaseIfDatabaseNotFound(t *testing.T) {
}, },
URLF: func() string { URLF: func() string {
return "http://localhost:8086" return "http://localhost:8086"
}, },
}, nil }, nil
}, },
} }
output.Log = testutil.Logger{}
err := output.Connect() err := output.Connect()
require.NoError(t, err) require.NoError(t, err)

View File

@ -5,7 +5,6 @@ import (
"bytes" "bytes"
"context" "context"
"fmt" "fmt"
"log"
"net" "net"
"net/url" "net/url"
@ -32,6 +31,7 @@ type UDPConfig struct {
URL *url.URL URL *url.URL
Serializer *influx.Serializer Serializer *influx.Serializer
Dialer Dialer Dialer Dialer
Log telegraf.Logger
} }
func NewUDPClient(config UDPConfig) (*udpClient, error) { func NewUDPClient(config UDPConfig) (*udpClient, error) {
@ -69,12 +69,18 @@ type udpClient struct {
dialer Dialer dialer Dialer
serializer *influx.Serializer serializer *influx.Serializer
url *url.URL url *url.URL
log telegraf.Logger
} }
func (c *udpClient) URL() string { func (c *udpClient) URL() string {
return c.url.String() return c.url.String()
} }
func (c *udpClient) SetLogger(log telegraf.Logger) {
c.log = log
}
func (c *udpClient) Database() string { func (c *udpClient) Database() string {
return "" return ""
} }
@ -93,7 +99,7 @@ func (c *udpClient) Write(ctx context.Context, metrics []telegraf.Metric) error
if err != nil { if err != nil {
// Since we are serializing multiple metrics, don't fail the // Since we are serializing multiple metrics, don't fail the
// entire batch just because of one unserializable metric. // entire batch just because of one unserializable metric.
log.Printf("E! [outputs.influxdb] when writing to [%s] could not serialize metric: %v", c.log.Errorf("when writing to [%s] could not serialize metric: %v",
c.URL(), err) c.URL(), err)
continue continue
} }

View File

@ -14,6 +14,7 @@ import (
"github.com/influxdata/telegraf" "github.com/influxdata/telegraf"
"github.com/influxdata/telegraf/metric" "github.com/influxdata/telegraf/metric"
"github.com/influxdata/telegraf/plugins/outputs/influxdb" "github.com/influxdata/telegraf/plugins/outputs/influxdb"
"github.com/influxdata/telegraf/testutil"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
) )
@ -78,6 +79,7 @@ func TestUDP_URL(t *testing.T) {
} }
client, err := influxdb.NewUDPClient(config) client, err := influxdb.NewUDPClient(config)
client.SetLogger(testutil.Logger{})
require.NoError(t, err) require.NoError(t, err)
require.Equal(t, u.String(), client.URL()) require.Equal(t, u.String(), client.URL())
@ -101,6 +103,7 @@ func TestUDP_Simple(t *testing.T) {
}, },
} }
client, err := influxdb.NewUDPClient(config) client, err := influxdb.NewUDPClient(config)
client.SetLogger(testutil.Logger{})
require.NoError(t, err) require.NoError(t, err)
ctx := context.Background() ctx := context.Background()
@ -127,6 +130,7 @@ func TestUDP_DialError(t *testing.T) {
}, },
} }
client, err := influxdb.NewUDPClient(config) client, err := influxdb.NewUDPClient(config)
client.SetLogger(testutil.Logger{})
require.NoError(t, err) require.NoError(t, err)
ctx := context.Background() ctx := context.Background()
@ -156,6 +160,7 @@ func TestUDP_WriteError(t *testing.T) {
}, },
} }
client, err := influxdb.NewUDPClient(config) client, err := influxdb.NewUDPClient(config)
client.SetLogger(testutil.Logger{})
require.NoError(t, err) require.NoError(t, err)
ctx := context.Background() ctx := context.Background()
@ -219,6 +224,7 @@ func TestUDP_ErrorLogging(t *testing.T) {
log.SetOutput(&b) log.SetOutput(&b)
client, err := influxdb.NewUDPClient(tt.config) client, err := influxdb.NewUDPClient(tt.config)
client.SetLogger(testutil.Logger{})
require.NoError(t, err) require.NoError(t, err)
ctx := context.Background() ctx := context.Background()
@ -262,6 +268,7 @@ func TestUDP_WriteWithRealConn(t *testing.T) {
URL: u, URL: u,
} }
client, err := influxdb.NewUDPClient(config) client, err := influxdb.NewUDPClient(config)
client.SetLogger(testutil.Logger{})
require.NoError(t, err) require.NoError(t, err)
ctx := context.Background() ctx := context.Background()

50
testutil/log.go Normal file
View File

@ -0,0 +1,50 @@
package testutil
import (
"log"
)
// Logger defines a logging structure for plugins.
type Logger struct {
Name string // Name is the plugin name, will be printed in the `[]`.
}
// Errorf logs an error message, patterned after log.Printf.
func (l Logger) Errorf(format string, args ...interface{}) {
log.Printf("E! ["+l.Name+"] "+format, args...)
}
// Error logs an error message, patterned after log.Print.
func (l Logger) Error(args ...interface{}) {
log.Print(append([]interface{}{"E! [" + l.Name + "] "}, args...)...)
}
// Debugf logs a debug message, patterned after log.Printf.
func (l Logger) Debugf(format string, args ...interface{}) {
log.Printf("D! ["+l.Name+"] "+format, args...)
}
// Debug logs a debug message, patterned after log.Print.
func (l Logger) Debug(args ...interface{}) {
log.Print(append([]interface{}{"D! [" + l.Name + "] "}, args...)...)
}
// Warnf logs a warning message, patterned after log.Printf.
func (l Logger) Warnf(format string, args ...interface{}) {
log.Printf("W! ["+l.Name+"] "+format, args...)
}
// Warn logs a warning message, patterned after log.Print.
func (l Logger) Warn(args ...interface{}) {
log.Print(append([]interface{}{"W! [" + l.Name + "] "}, args...)...)
}
// Infof logs an information message, patterned after log.Printf.
func (l Logger) Infof(format string, args ...interface{}) {
log.Printf("I! ["+l.Name+"] "+format, args...)
}
// Info logs an information message, patterned after log.Print.
func (l Logger) Info(args ...interface{}) {
log.Print(append([]interface{}{"I! [" + l.Name + "] "}, args...)...)
}