Implementing generic parser plugins and documentation

This constitutes a large change in how we will parse different data
formats going forward (for the plugins that support it)

This is working off @henrypfhu's changes.
This commit is contained in:
Cameron Sparr
2016-02-05 17:36:35 -07:00
parent 1449c8b887
commit e619493ece
32 changed files with 1971 additions and 522 deletions

View File

@@ -10,8 +10,8 @@ import (
"time"
"github.com/influxdata/telegraf"
"github.com/influxdata/telegraf/internal"
"github.com/influxdata/telegraf/plugins/inputs"
jsonparser "github.com/influxdata/telegraf/plugins/parsers/json"
)
const statsPath = "/_nodes/stats"
@@ -168,7 +168,7 @@ func (e *Elasticsearch) gatherNodeStats(url string, acc telegraf.Accumulator) er
now := time.Now()
for p, s := range stats {
f := internal.JSONFlattener{}
f := jsonparser.JSONFlattener{}
err := f.FlattenJSON("", s)
if err != nil {
return err

View File

@@ -8,8 +8,8 @@ The exec plugin can execute arbitrary commands which output:
> Graphite understands messages with this format:
> ```
metric_path value timestamp\n
> ```
metric_path value timestamp\n
```
> __metric_path__ is the metric namespace that you want to populate.
@@ -28,10 +28,7 @@ and strings will be ignored.
# Read flattened metrics from one or more commands that output JSON to stdout
[[inputs.exec]]
# Shell/commands array
# compatible with old version
# we can still use the old command configuration
# command = "/usr/bin/mycollector --foo=bar"
commands = ["/tmp/test.sh","/tmp/test2.sh"]
commands = ["/tmp/test.sh", "/tmp/test2.sh"]
# Data format to consume. This can be "json", "influx" or "graphite" (line-protocol)
# NOTE json only reads numerical measurements, strings and booleans are ignored.
@@ -128,7 +125,7 @@ and usage_busy. They will receive a timestamp at collection time.
We can also change the data_format to "graphite" to use the metrics collecting scripts such as (compatible with graphite):
* Nagios [Mertics Plugins] (https://exchange.nagios.org/directory/Plugins)
* Sensu [Mertics Plugins] (https://github.com/sensu-plugins)
* Sensu [Mertics Plugins] (https://github.com/sensu-plugins)
#### Configuration
```
@@ -180,4 +177,4 @@ sensu.metric.net.server0.eth0.rx_dropped 0 1444234982
The templates configuration will be used to parse the graphite metrics to support influxdb/opentsdb tagging store engines.
More detail information about templates, please refer to [The graphite Input] (https://github.com/influxdata/influxdb/blob/master/services/graphite/README.md)

View File

@@ -9,66 +9,40 @@ import (
"github.com/gonuts/go-shellquote"
"github.com/influxdata/telegraf"
"github.com/influxdata/telegraf/internal/encoding"
"github.com/influxdata/telegraf/plugins/inputs"
_ "github.com/influxdata/telegraf/internal/encoding/graphite"
_ "github.com/influxdata/telegraf/internal/encoding/influx"
_ "github.com/influxdata/telegraf/internal/encoding/json"
"github.com/influxdata/telegraf/plugins/parsers"
)
const sampleConfig = `
# Shell/commands array
# compatible with old version
# we can still use the old command configuration
# command = "/usr/bin/mycollector --foo=bar"
commands = ["/tmp/test.sh","/tmp/test2.sh"]
### Commands array
commands = ["/tmp/test.sh", "/usr/bin/mycollector --foo=bar"]
# Data format to consume. This can be "json", "influx" or "graphite" (line-protocol)
# NOTE json only reads numerical measurements, strings and booleans are ignored.
data_format = "json"
# measurement name suffix (for separating different commands)
### measurement name suffix (for separating different commands)
name_suffix = "_mycollector"
### Below configuration will be used for data_format = "graphite", can be ignored for other data_format
### If matching multiple measurement files, this string will be used to join the matched values.
separator = "."
### Each template line requires a template pattern. It can have an optional
### filter before the template and separated by spaces. It can also have optional extra
### tags following the template. Multiple tags should be separated by commas and no spaces
### similar to the line protocol format. The can be only one default template.
### Templates support below format:
### 1. filter + template
### 2. filter + template + extra tag
### 3. filter + template with field key
### 4. default template
templates = [
"*.app env.service.resource.measurement",
"stats.* .host.measurement* region=us-west,agent=sensu",
"stats2.* .host.measurement.field",
"measurement*"
]
### Data format to consume. This can be "json", "influx" or "graphite"
### Each data format has it's own unique set of configuration options, read
### more about them here:
### https://github.com/influxdata/telegraf/blob/master/DATA_FORMATS.md
data_format = "influx"
`
type Exec struct {
Commands []string
Command string
DataFormat string
Commands []string
Command string
Separator string
Templates []string
encodingParser encoding.Parser
initedConfig bool
parser parsers.Parser
wg sync.WaitGroup
sync.Mutex
runner Runner
errc chan error
runner Runner
errChan chan error
}
func NewExec() *Exec {
return &Exec{
runner: CommandRunner{},
}
}
type Runner interface {
@@ -95,22 +69,18 @@ func (c CommandRunner) Run(e *Exec, command string) ([]byte, error) {
return out.Bytes(), nil
}
func NewExec() *Exec {
return &Exec{runner: CommandRunner{}}
}
func (e *Exec) ProcessCommand(command string, acc telegraf.Accumulator) {
defer e.wg.Done()
out, err := e.runner.Run(e, command)
if err != nil {
e.errc <- err
e.errChan <- err
return
}
metrics, err := e.encodingParser.Parse(out)
metrics, err := e.parser.Parse(out)
if err != nil {
e.errc <- err
e.errChan <- err
} else {
for _, metric := range metrics {
acc.AddFields(metric.Name(), metric.Fields(), metric.Tags(), metric.Time())
@@ -118,66 +88,33 @@ func (e *Exec) ProcessCommand(command string, acc telegraf.Accumulator) {
}
}
func (e *Exec) initConfig() error {
e.Lock()
defer e.Unlock()
if e.Command != "" && len(e.Commands) < 1 {
e.Commands = []string{e.Command}
}
if e.DataFormat == "" {
e.DataFormat = "json"
}
var err error
configs := make(map[string]interface{})
configs["Separator"] = e.Separator
configs["Templates"] = e.Templates
e.encodingParser, err = encoding.NewParser(e.DataFormat, configs)
if err != nil {
return fmt.Errorf("exec configuration is error: %s ", err.Error())
}
return nil
}
func (e *Exec) SampleConfig() string {
return sampleConfig
}
func (e *Exec) Description() string {
return "Read metrics from one or more commands that can output JSON, influx or graphite line protocol to stdout"
return "Read metrics from one or more commands that can output to stdout"
}
func (e *Exec) SetParser(parser parsers.Parser) {
e.parser = parser
}
func (e *Exec) Gather(acc telegraf.Accumulator) error {
e.errChan = make(chan error, len(e.Commands))
if !e.initedConfig {
if err := e.initConfig(); err != nil {
return err
}
e.initedConfig = true
}
e.Lock()
e.errc = make(chan error, 10)
e.Unlock()
e.wg.Add(len(e.Commands))
for _, command := range e.Commands {
e.wg.Add(1)
go e.ProcessCommand(command, acc)
}
e.wg.Wait()
select {
default:
close(e.errc)
close(e.errChan)
return nil
case err := <-e.errc:
close(e.errc)
case err := <-e.errChan:
close(e.errChan)
return err
}

View File

@@ -4,6 +4,8 @@ import (
"fmt"
"testing"
"github.com/influxdata/telegraf/plugins/parsers"
"github.com/influxdata/telegraf/testutil"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
@@ -63,9 +65,11 @@ func (r runnerMock) Run(e *Exec, command string) ([]byte, error) {
}
func TestExec(t *testing.T) {
parser, _ := parsers.NewJSONParser("exec", []string{}, nil)
e := &Exec{
runner: newRunnerMock([]byte(validJson), nil),
Commands: []string{"testcommand arg1"},
parser: parser,
}
var acc testutil.Accumulator
@@ -87,9 +91,11 @@ func TestExec(t *testing.T) {
}
func TestExecMalformed(t *testing.T) {
parser, _ := parsers.NewJSONParser("exec", []string{}, nil)
e := &Exec{
runner: newRunnerMock([]byte(malformedJson), nil),
Commands: []string{"badcommand arg1"},
parser: parser,
}
var acc testutil.Accumulator
@@ -99,9 +105,11 @@ func TestExecMalformed(t *testing.T) {
}
func TestCommandError(t *testing.T) {
parser, _ := parsers.NewJSONParser("exec", []string{}, nil)
e := &Exec{
runner: newRunnerMock(nil, fmt.Errorf("exit status code 1")),
Commands: []string{"badcommand"},
parser: parser,
}
var acc testutil.Accumulator
@@ -111,10 +119,11 @@ func TestCommandError(t *testing.T) {
}
func TestLineProtocolParse(t *testing.T) {
parser, _ := parsers.NewInfluxParser()
e := &Exec{
runner: newRunnerMock([]byte(lineProtocol), nil),
Commands: []string{"line-protocol"},
DataFormat: "influx",
runner: newRunnerMock([]byte(lineProtocol), nil),
Commands: []string{"line-protocol"},
parser: parser,
}
var acc testutil.Accumulator
@@ -133,10 +142,11 @@ func TestLineProtocolParse(t *testing.T) {
}
func TestLineProtocolParseMultiple(t *testing.T) {
parser, _ := parsers.NewInfluxParser()
e := &Exec{
runner: newRunnerMock([]byte(lineProtocolMulti), nil),
Commands: []string{"line-protocol"},
DataFormat: "influx",
runner: newRunnerMock([]byte(lineProtocolMulti), nil),
Commands: []string{"line-protocol"},
parser: parser,
}
var acc testutil.Accumulator
@@ -158,15 +168,3 @@ func TestLineProtocolParseMultiple(t *testing.T) {
acc.AssertContainsTaggedFields(t, "cpu", fields, tags)
}
}
func TestInvalidDataFormat(t *testing.T) {
e := &Exec{
runner: newRunnerMock([]byte(lineProtocol), nil),
Commands: []string{"bad data format"},
DataFormat: "FooBar",
}
var acc testutil.Accumulator
err := e.Gather(&acc)
require.Error(t, err)
}

View File

@@ -1,7 +1,6 @@
package httpjson
import (
"encoding/json"
"errors"
"fmt"
"io/ioutil"
@@ -12,8 +11,8 @@ import (
"time"
"github.com/influxdata/telegraf"
"github.com/influxdata/telegraf/internal"
"github.com/influxdata/telegraf/plugins/inputs"
"github.com/influxdata/telegraf/plugins/parsers"
)
type HttpJson struct {
@@ -137,39 +136,34 @@ func (h *HttpJson) gatherServer(
return err
}
var jsonOut map[string]interface{}
if err = json.Unmarshal([]byte(resp), &jsonOut); err != nil {
return errors.New("Error decoding JSON response")
}
tags := map[string]string{
"server": serverURL,
}
for _, tag := range h.TagKeys {
switch v := jsonOut[tag].(type) {
case string:
tags[tag] = v
}
delete(jsonOut, tag)
}
if responseTime >= 0 {
jsonOut["response_time"] = responseTime
}
f := internal.JSONFlattener{}
err = f.FlattenJSON("", jsonOut)
if err != nil {
return err
}
var msrmnt_name string
if h.Name == "" {
msrmnt_name = "httpjson"
} else {
msrmnt_name = "httpjson_" + h.Name
}
acc.AddFields(msrmnt_name, f.Fields, tags)
tags := map[string]string{
"server": serverURL,
}
parser, err := parsers.NewJSONParser(msrmnt_name, h.TagKeys, tags)
if err != nil {
return err
}
metrics, err := parser.Parse([]byte(resp))
if err != nil {
return err
}
for _, metric := range metrics {
fields := make(map[string]interface{})
for k, v := range metric.Fields() {
fields[k] = v
}
fields["response_time"] = responseTime
acc.AddFields(metric.Name(), fields, metric.Tags())
}
return nil
}

View File

@@ -1,12 +1,14 @@
package kafka_consumer
import (
"fmt"
"log"
"strings"
"sync"
"github.com/influxdata/telegraf"
"github.com/influxdata/telegraf/plugins/inputs"
"github.com/influxdata/telegraf/plugins/parsers"
"github.com/Shopify/sarama"
"github.com/wvanbergen/kafka/consumergroup"
@@ -20,6 +22,8 @@ type Kafka struct {
PointBuffer int
Offset string
parser parsers.Parser
sync.Mutex
// channel for all incoming kafka messages
@@ -36,16 +40,22 @@ type Kafka struct {
}
var sampleConfig = `
# topic(s) to consume
### topic(s) to consume
topics = ["telegraf"]
# an array of Zookeeper connection strings
### an array of Zookeeper connection strings
zookeeper_peers = ["localhost:2181"]
# the name of the consumer group
### the name of the consumer group
consumer_group = "telegraf_metrics_consumers"
# Maximum number of points to buffer between collection intervals
### Maximum number of points to buffer between collection intervals
point_buffer = 100000
# Offset (must be either "oldest" or "newest")
### Offset (must be either "oldest" or "newest")
offset = "oldest"
### Data format to consume. This can be "json", "influx" or "graphite"
### Each data format has it's own unique set of configuration options, read
### more about them here:
### https://github.com/influxdata/telegraf/blob/master/DATA_FORMATS.md
data_format = "influx"
`
func (k *Kafka) SampleConfig() string {
@@ -53,7 +63,11 @@ func (k *Kafka) SampleConfig() string {
}
func (k *Kafka) Description() string {
return "Read line-protocol metrics from Kafka topic(s)"
return "Read metrics from Kafka topic(s)"
}
func (k *Kafka) SetParser(parser parsers.Parser) {
k.parser = parser
}
func (k *Kafka) Start() error {
@@ -96,15 +110,15 @@ func (k *Kafka) Start() error {
k.metricC = make(chan telegraf.Metric, k.PointBuffer)
// Start the kafka message reader
go k.parser()
go k.receiver()
log.Printf("Started the kafka consumer service, peers: %v, topics: %v\n",
k.ZookeeperPeers, k.Topics)
return nil
}
// parser() reads all incoming messages from the consumer, and parses them into
// receiver() reads all incoming messages from the consumer, and parses them into
// influxdb metric points.
func (k *Kafka) parser() {
func (k *Kafka) receiver() {
for {
select {
case <-k.done:
@@ -112,13 +126,14 @@ func (k *Kafka) parser() {
case err := <-k.errs:
log.Printf("Kafka Consumer Error: %s\n", err.Error())
case msg := <-k.in:
metrics, err := telegraf.ParseMetrics(msg.Value)
metrics, err := k.parser.Parse(msg.Value)
if err != nil {
log.Printf("Could not parse kafka message: %s, error: %s",
string(msg.Value), err.Error())
}
for _, metric := range metrics {
fmt.Println(string(metric.Name()))
select {
case k.metricC <- metric:
continue

View File

@@ -9,6 +9,8 @@ import (
"github.com/influxdata/telegraf/testutil"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"github.com/influxdata/telegraf/plugins/parsers"
)
func TestReadsMetricsFromKafka(t *testing.T) {
@@ -40,6 +42,8 @@ func TestReadsMetricsFromKafka(t *testing.T) {
PointBuffer: 100000,
Offset: "oldest",
}
p, _ := parsers.NewInfluxParser()
k.SetParser(p)
if err := k.Start(); err != nil {
t.Fatal(err.Error())
} else {

View File

@@ -5,6 +5,7 @@ import (
"time"
"github.com/influxdata/telegraf"
"github.com/influxdata/telegraf/plugins/parsers"
"github.com/influxdata/telegraf/testutil"
"github.com/Shopify/sarama"
@@ -12,9 +13,11 @@ import (
)
const (
testMsg = "cpu_load_short,host=server01 value=23422.0 1422568543702900257"
invalidMsg = "cpu_load_short,host=server01 1422568543702900257"
pointBuffer = 5
testMsg = "cpu_load_short,host=server01 value=23422.0 1422568543702900257"
testMsgGraphite = "cpu.load.short.graphite 23422 1454780029"
testMsgJSON = "{\"a\": 5, \"b\": {\"c\": 6}}\n"
invalidMsg = "cpu_load_short,host=server01 1422568543702900257"
pointBuffer = 5
)
func NewTestKafka() (*Kafka, chan *sarama.ConsumerMessage) {
@@ -39,7 +42,8 @@ func TestRunParser(t *testing.T) {
k, in := NewTestKafka()
defer close(k.done)
go k.parser()
k.parser, _ = parsers.NewInfluxParser()
go k.receiver()
in <- saramaMsg(testMsg)
time.Sleep(time.Millisecond)
@@ -51,7 +55,8 @@ func TestRunParserInvalidMsg(t *testing.T) {
k, in := NewTestKafka()
defer close(k.done)
go k.parser()
k.parser, _ = parsers.NewInfluxParser()
go k.receiver()
in <- saramaMsg(invalidMsg)
time.Sleep(time.Millisecond)
@@ -63,7 +68,8 @@ func TestRunParserRespectsBuffer(t *testing.T) {
k, in := NewTestKafka()
defer close(k.done)
go k.parser()
k.parser, _ = parsers.NewInfluxParser()
go k.receiver()
for i := 0; i < pointBuffer+1; i++ {
in <- saramaMsg(testMsg)
}
@@ -77,7 +83,8 @@ func TestRunParserAndGather(t *testing.T) {
k, in := NewTestKafka()
defer close(k.done)
go k.parser()
k.parser, _ = parsers.NewInfluxParser()
go k.receiver()
in <- saramaMsg(testMsg)
time.Sleep(time.Millisecond)
@@ -89,6 +96,45 @@ func TestRunParserAndGather(t *testing.T) {
map[string]interface{}{"value": float64(23422)})
}
// Test that the parser parses kafka messages into points
func TestRunParserAndGatherGraphite(t *testing.T) {
k, in := NewTestKafka()
defer close(k.done)
k.parser, _ = parsers.NewGraphiteParser("_", []string{}, nil)
go k.receiver()
in <- saramaMsg(testMsgGraphite)
time.Sleep(time.Millisecond)
acc := testutil.Accumulator{}
k.Gather(&acc)
assert.Equal(t, len(acc.Metrics), 1)
acc.AssertContainsFields(t, "cpu_load_short_graphite",
map[string]interface{}{"value": float64(23422)})
}
// Test that the parser parses kafka messages into points
func TestRunParserAndGatherJSON(t *testing.T) {
k, in := NewTestKafka()
defer close(k.done)
k.parser, _ = parsers.NewJSONParser("kafka_json_test", []string{}, nil)
go k.receiver()
in <- saramaMsg(testMsgJSON)
time.Sleep(time.Millisecond)
acc := testutil.Accumulator{}
k.Gather(&acc)
assert.Equal(t, len(acc.Metrics), 1)
acc.AssertContainsFields(t, "kafka_json_test",
map[string]interface{}{
"a": float64(5),
"b_c": float64(6),
})
}
func saramaMsg(val string) *sarama.ConsumerMessage {
return &sarama.ConsumerMessage{
Key: nil,

View File

@@ -11,7 +11,7 @@ import (
"sync"
"time"
"github.com/influxdata/influxdb/services/graphite"
"github.com/influxdata/telegraf/plugins/parsers/graphite"
"github.com/influxdata/telegraf"
"github.com/influxdata/telegraf/plugins/inputs"
@@ -123,37 +123,39 @@ func (_ *Statsd) Description() string {
}
const sampleConfig = `
# Address and port to host UDP listener on
### Address and port to host UDP listener on
service_address = ":8125"
# Delete gauges every interval (default=false)
### Delete gauges every interval (default=false)
delete_gauges = false
# Delete counters every interval (default=false)
### Delete counters every interval (default=false)
delete_counters = false
# Delete sets every interval (default=false)
### Delete sets every interval (default=false)
delete_sets = false
# Delete timings & histograms every interval (default=true)
### Delete timings & histograms every interval (default=true)
delete_timings = true
# Percentiles to calculate for timing & histogram stats
### Percentiles to calculate for timing & histogram stats
percentiles = [90]
# convert measurement names, "." to "_" and "-" to "__"
### convert measurement names, "." to "_" and "-" to "__"
convert_names = true
### Statsd data translation templates, more info can be read here:
### https://github.com/influxdata/telegraf/blob/master/DATA_FORMATS.md#graphite
# templates = [
# "cpu.* measurement*"
# ]
# Number of UDP messages allowed to queue up, once filled,
# the statsd server will start dropping packets
### Number of UDP messages allowed to queue up, once filled,
### the statsd server will start dropping packets
allowed_pending_messages = 10000
# Number of timing/histogram values to track per-measurement in the
# calculation of percentiles. Raising this limit increases the accuracy
# of percentiles but also increases the memory usage and cpu time.
### Number of timing/histogram values to track per-measurement in the
### calculation of percentiles. Raising this limit increases the accuracy
### of percentiles but also increases the memory usage and cpu time.
percentile_limit = 1000
# UDP packet size for the server to listen for. This will depend on the size
# of the packets that the client is sending, which is usually 1500 bytes.
### UDP packet size for the server to listen for. This will depend on the size
### of the packets that the client is sending, which is usually 1500 bytes.
udp_packet_size = 1500
`
@@ -418,18 +420,14 @@ func (s *Statsd) parseName(bucket string) (string, string, map[string]string) {
}
}
o := graphite.Options{
Separator: "_",
Templates: s.Templates,
DefaultTags: tags,
}
var field string
name := bucketparts[0]
p, err := graphite.NewParserWithOptions(o)
p, err := graphite.NewGraphiteParser(".", s.Templates, nil)
if err == nil {
p.DefaultTags = tags
name, tags, field, _ = p.ApplyTemplate(name)
}
if s.ConvertNames {
name = strings.Replace(name, ".", "_", -1)
name = strings.Replace(name, "-", "__", -1)

View File

@@ -71,16 +71,11 @@ func TestGraphiteOK(t *testing.T) {
// Start TCP server
wg.Add(1)
go TCPServer(t, &wg)
wg.Wait()
// Connect
wg.Add(1)
err1 := g.Connect()
wg.Wait()
require.NoError(t, err1)
// Send Data
err2 := g.Write(metrics)
require.NoError(t, err2)
wg.Add(1)
// Waiting TCPserver
wg.Wait()
g.Close()
@@ -88,9 +83,8 @@ func TestGraphiteOK(t *testing.T) {
func TCPServer(t *testing.T, wg *sync.WaitGroup) {
tcpServer, _ := net.Listen("tcp", "127.0.0.1:2003")
wg.Done()
defer wg.Done()
conn, _ := tcpServer.Accept()
wg.Done()
reader := bufio.NewReader(conn)
tp := textproto.NewReader(reader)
data1, _ := tp.ReadLine()
@@ -100,7 +94,6 @@ func TCPServer(t *testing.T, wg *sync.WaitGroup) {
data3, _ := tp.ReadLine()
assert.Equal(t, "my.prefix.192_168_0_1.my_measurement.value 3.14 1289430000", data3)
conn.Close()
wg.Done()
}
func TestGraphiteTags(t *testing.T) {

View File

@@ -0,0 +1,135 @@
package graphite
import (
"fmt"
"strings"
)
const (
// DefaultSeparator is the default join character to use when joining multiple
// measurment parts in a template.
DefaultSeparator = "."
)
// Config represents the configuration for Graphite endpoints.
type Config struct {
Separator string
Templates []string
}
// Validate validates the config's templates and tags.
func (c *Config) Validate() error {
if err := c.validateTemplates(); err != nil {
return err
}
return nil
}
func (c *Config) validateTemplates() error {
// map to keep track of filters we see
filters := map[string]struct{}{}
for i, t := range c.Templates {
parts := strings.Fields(t)
// Ensure template string is non-empty
if len(parts) == 0 {
return fmt.Errorf("missing template at position: %d", i)
}
if len(parts) == 1 && parts[0] == "" {
return fmt.Errorf("missing template at position: %d", i)
}
if len(parts) > 3 {
return fmt.Errorf("invalid template format: '%s'", t)
}
template := t
filter := ""
tags := ""
if len(parts) >= 2 {
// We could have <filter> <template> or <template> <tags>. Equals is only allowed in
// tags section.
if strings.Contains(parts[1], "=") {
template = parts[0]
tags = parts[1]
} else {
filter = parts[0]
template = parts[1]
}
}
if len(parts) == 3 {
tags = parts[2]
}
// Validate the template has one and only one measurement
if err := c.validateTemplate(template); err != nil {
return err
}
// Prevent duplicate filters in the config
if _, ok := filters[filter]; ok {
return fmt.Errorf("duplicate filter '%s' found at position: %d", filter, i)
}
filters[filter] = struct{}{}
if filter != "" {
// Validate filter expression is valid
if err := c.validateFilter(filter); err != nil {
return err
}
}
if tags != "" {
// Validate tags
for _, tagStr := range strings.Split(tags, ",") {
if err := c.validateTag(tagStr); err != nil {
return err
}
}
}
}
return nil
}
func (c *Config) validateTemplate(template string) error {
hasMeasurement := false
for _, p := range strings.Split(template, ".") {
if p == "measurement" || p == "measurement*" {
hasMeasurement = true
}
}
if !hasMeasurement {
return fmt.Errorf("no measurement in template `%s`", template)
}
return nil
}
func (c *Config) validateFilter(filter string) error {
for _, p := range strings.Split(filter, ".") {
if p == "" {
return fmt.Errorf("filter contains blank section: %s", filter)
}
if strings.Contains(p, "*") && p != "*" {
return fmt.Errorf("invalid filter wildcard section: %s", filter)
}
}
return nil
}
func (c *Config) validateTag(keyValue string) error {
parts := strings.Split(keyValue, "=")
if len(parts) != 2 {
return fmt.Errorf("invalid template tags: '%s'", keyValue)
}
if parts[0] == "" || parts[1] == "" {
return fmt.Errorf("invalid template tags: %s'", keyValue)
}
return nil
}

View File

@@ -0,0 +1,14 @@
package graphite
import "fmt"
// An UnsupposedValueError is returned when a parsed value is not
// supposed.
type UnsupposedValueError struct {
Field string
Value float64
}
func (err *UnsupposedValueError) Error() string {
return fmt.Sprintf(`field "%s" value: "%v" is unsupported`, err.Field, err.Value)
}

View File

@@ -0,0 +1,425 @@
package graphite
import (
"bufio"
"bytes"
"fmt"
"io"
"math"
"sort"
"strconv"
"strings"
"time"
"github.com/influxdata/telegraf"
)
// Minimum and maximum supported dates for timestamps.
var (
MinDate = time.Date(1901, 12, 13, 0, 0, 0, 0, time.UTC)
MaxDate = time.Date(2038, 1, 19, 0, 0, 0, 0, time.UTC)
)
// Parser encapsulates a Graphite Parser.
type GraphiteParser struct {
Separator string
Templates []string
DefaultTags map[string]string
matcher *matcher
}
func NewGraphiteParser(
separator string,
templates []string,
defaultTags map[string]string,
) (*GraphiteParser, error) {
var err error
if separator == "" {
separator = DefaultSeparator
}
p := &GraphiteParser{
Separator: separator,
Templates: templates,
}
if defaultTags != nil {
p.DefaultTags = defaultTags
}
matcher := newMatcher()
p.matcher = matcher
defaultTemplate, _ := NewTemplate("measurement*", nil, p.Separator)
matcher.AddDefaultTemplate(defaultTemplate)
for _, pattern := range p.Templates {
template := pattern
filter := ""
// Format is [filter] <template> [tag1=value1,tag2=value2]
parts := strings.Fields(pattern)
if len(parts) < 1 {
continue
} else if len(parts) >= 2 {
if strings.Contains(parts[1], "=") {
template = parts[0]
} else {
filter = parts[0]
template = parts[1]
}
}
// Parse out the default tags specific to this template
tags := map[string]string{}
if strings.Contains(parts[len(parts)-1], "=") {
tagStrs := strings.Split(parts[len(parts)-1], ",")
for _, kv := range tagStrs {
parts := strings.Split(kv, "=")
tags[parts[0]] = parts[1]
}
}
tmpl, err1 := NewTemplate(template, tags, p.Separator)
if err1 != nil {
err = err1
break
}
matcher.Add(filter, tmpl)
}
if err != nil {
return p, fmt.Errorf("exec input parser config is error: %s ", err.Error())
} else {
return p, nil
}
}
func (p *GraphiteParser) Parse(buf []byte) ([]telegraf.Metric, error) {
// parse even if the buffer begins with a newline
buf = bytes.TrimPrefix(buf, []byte("\n"))
// add newline to end if not exists:
if len(buf) > 0 && !bytes.HasSuffix(buf, []byte("\n")) {
buf = append(buf, []byte("\n")...)
}
metrics := make([]telegraf.Metric, 0)
buffer := bytes.NewBuffer(buf)
reader := bufio.NewReader(buffer)
for {
// Read up to the next newline.
buf, err := reader.ReadBytes('\n')
if err == io.EOF {
return metrics, nil
}
if err != nil && err != io.EOF {
return metrics, err
}
// Trim the buffer, even though there should be no padding
line := strings.TrimSpace(string(buf))
if metric, err := p.ParseLine(line); err == nil {
metrics = append(metrics, metric)
}
}
}
// Parse performs Graphite parsing of a single line.
func (p *GraphiteParser) ParseLine(line string) (telegraf.Metric, error) {
// Break into 3 fields (name, value, timestamp).
fields := strings.Fields(line)
if len(fields) != 2 && len(fields) != 3 {
return nil, fmt.Errorf("received %q which doesn't have required fields", line)
}
// decode the name and tags
template := p.matcher.Match(fields[0])
measurement, tags, field, err := template.Apply(fields[0])
if err != nil {
return nil, err
}
// Could not extract measurement, use the raw value
if measurement == "" {
measurement = fields[0]
}
// Parse value.
v, err := strconv.ParseFloat(fields[1], 64)
if err != nil {
return nil, fmt.Errorf(`field "%s" value: %s`, fields[0], err)
}
if math.IsNaN(v) || math.IsInf(v, 0) {
return nil, &UnsupposedValueError{Field: fields[0], Value: v}
}
fieldValues := map[string]interface{}{}
if field != "" {
fieldValues[field] = v
} else {
fieldValues["value"] = v
}
// If no 3rd field, use now as timestamp
timestamp := time.Now().UTC()
if len(fields) == 3 {
// Parse timestamp.
unixTime, err := strconv.ParseFloat(fields[2], 64)
if err != nil {
return nil, fmt.Errorf(`field "%s" time: %s`, fields[0], err)
}
// -1 is a special value that gets converted to current UTC time
// See https://github.com/graphite-project/carbon/issues/54
if unixTime != float64(-1) {
// Check if we have fractional seconds
timestamp = time.Unix(int64(unixTime), int64((unixTime-math.Floor(unixTime))*float64(time.Second)))
if timestamp.Before(MinDate) || timestamp.After(MaxDate) {
return nil, fmt.Errorf("timestamp out of range")
}
}
}
// Set the default tags on the point if they are not already set
for k, v := range p.DefaultTags {
if _, ok := tags[k]; !ok {
tags[k] = v
}
}
return telegraf.NewMetric(measurement, tags, fieldValues, timestamp)
}
// ApplyTemplate extracts the template fields from the given line and
// returns the measurement name and tags.
func (p *GraphiteParser) ApplyTemplate(line string) (string, map[string]string, string, error) {
// Break line into fields (name, value, timestamp), only name is used
fields := strings.Fields(line)
if len(fields) == 0 {
return "", make(map[string]string), "", nil
}
// decode the name and tags
template := p.matcher.Match(fields[0])
name, tags, field, err := template.Apply(fields[0])
// Set the default tags on the point if they are not already set
for k, v := range p.DefaultTags {
if _, ok := tags[k]; !ok {
tags[k] = v
}
}
return name, tags, field, err
}
// template represents a pattern and tags to map a graphite metric string to a influxdb Point
type template struct {
tags []string
defaultTags map[string]string
greedyMeasurement bool
separator string
}
// NewTemplate returns a new template ensuring it has a measurement
// specified.
func NewTemplate(pattern string, defaultTags map[string]string, separator string) (*template, error) {
tags := strings.Split(pattern, ".")
hasMeasurement := false
template := &template{tags: tags, defaultTags: defaultTags, separator: separator}
for _, tag := range tags {
if strings.HasPrefix(tag, "measurement") {
hasMeasurement = true
}
if tag == "measurement*" {
template.greedyMeasurement = true
}
}
if !hasMeasurement {
return nil, fmt.Errorf("no measurement specified for template. %q", pattern)
}
return template, nil
}
// Apply extracts the template fields from the given line and returns the measurement
// name and tags
func (t *template) Apply(line string) (string, map[string]string, string, error) {
fields := strings.Split(line, ".")
var (
measurement []string
tags = make(map[string]string)
field string
)
// Set any default tags
for k, v := range t.defaultTags {
tags[k] = v
}
for i, tag := range t.tags {
if i >= len(fields) {
continue
}
if tag == "measurement" {
measurement = append(measurement, fields[i])
} else if tag == "field" {
if len(field) != 0 {
return "", nil, "", fmt.Errorf("'field' can only be used once in each template: %q", line)
}
field = fields[i]
} else if tag == "measurement*" {
measurement = append(measurement, fields[i:]...)
break
} else if tag != "" {
tags[tag] = fields[i]
}
}
return strings.Join(measurement, t.separator), tags, field, nil
}
// matcher determines which template should be applied to a given metric
// based on a filter tree.
type matcher struct {
root *node
defaultTemplate *template
}
func newMatcher() *matcher {
return &matcher{
root: &node{},
}
}
// Add inserts the template in the filter tree based the given filter
func (m *matcher) Add(filter string, template *template) {
if filter == "" {
m.AddDefaultTemplate(template)
return
}
m.root.Insert(filter, template)
}
func (m *matcher) AddDefaultTemplate(template *template) {
m.defaultTemplate = template
}
// Match returns the template that matches the given graphite line
func (m *matcher) Match(line string) *template {
tmpl := m.root.Search(line)
if tmpl != nil {
return tmpl
}
return m.defaultTemplate
}
// node is an item in a sorted k-ary tree. Each child is sorted by its value.
// The special value of "*", is always last.
type node struct {
value string
children nodes
template *template
}
func (n *node) insert(values []string, template *template) {
// Add the end, set the template
if len(values) == 0 {
n.template = template
return
}
// See if the the current element already exists in the tree. If so, insert the
// into that sub-tree
for _, v := range n.children {
if v.value == values[0] {
v.insert(values[1:], template)
return
}
}
// New element, add it to the tree and sort the children
newNode := &node{value: values[0]}
n.children = append(n.children, newNode)
sort.Sort(&n.children)
// Now insert the rest of the tree into the new element
newNode.insert(values[1:], template)
}
// Insert inserts the given string template into the tree. The filter string is separated
// on "." and each part is used as the path in the tree.
func (n *node) Insert(filter string, template *template) {
n.insert(strings.Split(filter, "."), template)
}
func (n *node) search(lineParts []string) *template {
// Nothing to search
if len(lineParts) == 0 || len(n.children) == 0 {
return n.template
}
// If last element is a wildcard, don't include in this search since it's sorted
// to the end but lexicographically it would not always be and sort.Search assumes
// the slice is sorted.
length := len(n.children)
if n.children[length-1].value == "*" {
length--
}
// Find the index of child with an exact match
i := sort.Search(length, func(i int) bool {
return n.children[i].value >= lineParts[0]
})
// Found an exact match, so search that child sub-tree
if i < len(n.children) && n.children[i].value == lineParts[0] {
return n.children[i].search(lineParts[1:])
}
// Not an exact match, see if we have a wildcard child to search
if n.children[len(n.children)-1].value == "*" {
return n.children[len(n.children)-1].search(lineParts[1:])
}
return n.template
}
func (n *node) Search(line string) *template {
return n.search(strings.Split(line, "."))
}
type nodes []*node
// Less returns a boolean indicating whether the filter at position j
// is less than the filter at position k. Filters are order by string
// comparison of each component parts. A wildcard value "*" is never
// less than a non-wildcard value.
//
// For example, the filters:
// "*.*"
// "servers.*"
// "servers.localhost"
// "*.localhost"
//
// Would be sorted as:
// "servers.localhost"
// "servers.*"
// "*.localhost"
// "*.*"
func (n *nodes) Less(j, k int) bool {
if (*n)[j].value == "*" && (*n)[k].value != "*" {
return false
}
if (*n)[j].value != "*" && (*n)[k].value == "*" {
return true
}
return (*n)[j].value < (*n)[k].value
}
func (n *nodes) Swap(i, j int) { (*n)[i], (*n)[j] = (*n)[j], (*n)[i] }
func (n *nodes) Len() int { return len(*n) }

View File

@@ -0,0 +1,595 @@
package graphite
import (
"reflect"
"strconv"
"testing"
"time"
"github.com/influxdata/telegraf"
"github.com/stretchr/testify/assert"
)
func BenchmarkParse(b *testing.B) {
p, err := NewGraphiteParser("_", []string{
"*.* .wrong.measurement*",
"servers.* .host.measurement*",
"servers.localhost .host.measurement*",
"*.localhost .host.measurement*",
"*.*.cpu .host.measurement*",
"a.b.c .host.measurement*",
"influxd.*.foo .host.measurement*",
"prod.*.mem .host.measurement*",
}, nil)
if err != nil {
b.Fatalf("unexpected error creating parser, got %v", err)
}
for i := 0; i < b.N; i++ {
p.Parse([]byte("servers.localhost.cpu.load 11 1435077219"))
}
}
func TestTemplateApply(t *testing.T) {
var tests = []struct {
test string
input string
template string
measurement string
tags map[string]string
err string
}{
{
test: "metric only",
input: "cpu",
template: "measurement",
measurement: "cpu",
},
{
test: "metric with single series",
input: "cpu.server01",
template: "measurement.hostname",
measurement: "cpu",
tags: map[string]string{"hostname": "server01"},
},
{
test: "metric with multiple series",
input: "cpu.us-west.server01",
template: "measurement.region.hostname",
measurement: "cpu",
tags: map[string]string{"hostname": "server01", "region": "us-west"},
},
{
test: "no metric",
tags: make(map[string]string),
err: `no measurement specified for template. ""`,
},
{
test: "ignore unnamed",
input: "foo.cpu",
template: "measurement",
measurement: "foo",
tags: make(map[string]string),
},
{
test: "name shorter than template",
input: "foo",
template: "measurement.A.B.C",
measurement: "foo",
tags: make(map[string]string),
},
{
test: "wildcard measurement at end",
input: "prod.us-west.server01.cpu.load",
template: "env.zone.host.measurement*",
measurement: "cpu.load",
tags: map[string]string{"env": "prod", "zone": "us-west", "host": "server01"},
},
{
test: "skip fields",
input: "ignore.us-west.ignore-this-too.cpu.load",
template: ".zone..measurement*",
measurement: "cpu.load",
tags: map[string]string{"zone": "us-west"},
},
}
for _, test := range tests {
tmpl, err := NewTemplate(test.template, nil, DefaultSeparator)
if errstr(err) != test.err {
t.Fatalf("err does not match. expected %v, got %v", test.err, err)
}
if err != nil {
// If we erred out,it was intended and the following tests won't work
continue
}
measurement, tags, _, _ := tmpl.Apply(test.input)
if measurement != test.measurement {
t.Fatalf("name parse failer. expected %v, got %v", test.measurement, measurement)
}
if len(tags) != len(test.tags) {
t.Fatalf("unexpected number of tags. expected %v, got %v", test.tags, tags)
}
for k, v := range test.tags {
if tags[k] != v {
t.Fatalf("unexpected tag value for tags[%s]. expected %q, got %q", k, v, tags[k])
}
}
}
}
func TestParseMissingMeasurement(t *testing.T) {
_, err := NewGraphiteParser("", []string{"a.b.c"}, nil)
if err == nil {
t.Fatalf("expected error creating parser, got nil")
}
}
func TestParse(t *testing.T) {
testTime := time.Now().Round(time.Second)
epochTime := testTime.Unix()
strTime := strconv.FormatInt(epochTime, 10)
var tests = []struct {
test string
input string
measurement string
tags map[string]string
value float64
time time.Time
template string
err string
}{
{
test: "normal case",
input: `cpu.foo.bar 50 ` + strTime,
template: "measurement.foo.bar",
measurement: "cpu",
tags: map[string]string{
"foo": "foo",
"bar": "bar",
},
value: 50,
time: testTime,
},
{
test: "metric only with float value",
input: `cpu 50.554 ` + strTime,
measurement: "cpu",
template: "measurement",
value: 50.554,
time: testTime,
},
{
test: "missing metric",
input: `1419972457825`,
template: "measurement",
err: `received "1419972457825" which doesn't have required fields`,
},
{
test: "should error parsing invalid float",
input: `cpu 50.554z 1419972457825`,
template: "measurement",
err: `field "cpu" value: strconv.ParseFloat: parsing "50.554z": invalid syntax`,
},
{
test: "should error parsing invalid int",
input: `cpu 50z 1419972457825`,
template: "measurement",
err: `field "cpu" value: strconv.ParseFloat: parsing "50z": invalid syntax`,
},
{
test: "should error parsing invalid time",
input: `cpu 50.554 14199724z57825`,
template: "measurement",
err: `field "cpu" time: strconv.ParseFloat: parsing "14199724z57825": invalid syntax`,
},
}
for _, test := range tests {
p, err := NewGraphiteParser("", []string{test.template}, nil)
if err != nil {
t.Fatalf("unexpected error creating graphite parser: %v", err)
}
metric, err := p.ParseLine(test.input)
if errstr(err) != test.err {
t.Fatalf("err does not match. expected %v, got %v", test.err, err)
}
if err != nil {
// If we erred out,it was intended and the following tests won't work
continue
}
if metric.Name() != test.measurement {
t.Fatalf("name parse failer. expected %v, got %v",
test.measurement, metric.Name())
}
if len(metric.Tags()) != len(test.tags) {
t.Fatalf("tags len mismatch. expected %d, got %d",
len(test.tags), len(metric.Tags()))
}
f := metric.Fields()["value"].(float64)
if metric.Fields()["value"] != f {
t.Fatalf("floatValue value mismatch. expected %v, got %v",
test.value, f)
}
if metric.Time().UnixNano()/1000000 != test.time.UnixNano()/1000000 {
t.Fatalf("time value mismatch. expected %v, got %v",
test.time.UnixNano(), metric.Time().UnixNano())
}
}
}
func TestParseNaN(t *testing.T) {
p, err := NewGraphiteParser("", []string{"measurement*"}, nil)
assert.NoError(t, err)
_, err = p.ParseLine("servers.localhost.cpu_load NaN 1435077219")
assert.Error(t, err)
if _, ok := err.(*UnsupposedValueError); !ok {
t.Fatalf("expected *ErrUnsupportedValue, got %v", reflect.TypeOf(err))
}
}
func TestFilterMatchDefault(t *testing.T) {
p, err := NewGraphiteParser("", []string{"servers.localhost .host.measurement*"}, nil)
if err != nil {
t.Fatalf("unexpected error creating parser, got %v", err)
}
exp, err := telegraf.NewMetric("miss.servers.localhost.cpu_load",
map[string]string{},
map[string]interface{}{"value": float64(11)},
time.Unix(1435077219, 0))
assert.NoError(t, err)
m, err := p.ParseLine("miss.servers.localhost.cpu_load 11 1435077219")
assert.NoError(t, err)
assert.Equal(t, exp.String(), m.String())
}
func TestFilterMatchMultipleMeasurement(t *testing.T) {
p, err := NewGraphiteParser("", []string{"servers.localhost .host.measurement.measurement*"}, nil)
if err != nil {
t.Fatalf("unexpected error creating parser, got %v", err)
}
exp, err := telegraf.NewMetric("cpu.cpu_load.10",
map[string]string{"host": "localhost"},
map[string]interface{}{"value": float64(11)},
time.Unix(1435077219, 0))
assert.NoError(t, err)
m, err := p.ParseLine("servers.localhost.cpu.cpu_load.10 11 1435077219")
assert.NoError(t, err)
assert.Equal(t, exp.String(), m.String())
}
func TestFilterMatchMultipleMeasurementSeparator(t *testing.T) {
p, err := NewGraphiteParser("_",
[]string{"servers.localhost .host.measurement.measurement*"},
nil,
)
assert.NoError(t, err)
exp, err := telegraf.NewMetric("cpu_cpu_load_10",
map[string]string{"host": "localhost"},
map[string]interface{}{"value": float64(11)},
time.Unix(1435077219, 0))
assert.NoError(t, err)
m, err := p.ParseLine("servers.localhost.cpu.cpu_load.10 11 1435077219")
assert.NoError(t, err)
assert.Equal(t, exp.String(), m.String())
}
func TestFilterMatchSingle(t *testing.T) {
p, err := NewGraphiteParser("", []string{"servers.localhost .host.measurement*"}, nil)
if err != nil {
t.Fatalf("unexpected error creating parser, got %v", err)
}
exp, err := telegraf.NewMetric("cpu_load",
map[string]string{"host": "localhost"},
map[string]interface{}{"value": float64(11)},
time.Unix(1435077219, 0))
m, err := p.ParseLine("servers.localhost.cpu_load 11 1435077219")
assert.NoError(t, err)
assert.Equal(t, exp.String(), m.String())
}
func TestParseNoMatch(t *testing.T) {
p, err := NewGraphiteParser("", []string{"servers.*.cpu .host.measurement.cpu.measurement"}, nil)
if err != nil {
t.Fatalf("unexpected error creating parser, got %v", err)
}
exp, err := telegraf.NewMetric("servers.localhost.memory.VmallocChunk",
map[string]string{},
map[string]interface{}{"value": float64(11)},
time.Unix(1435077219, 0))
assert.NoError(t, err)
m, err := p.ParseLine("servers.localhost.memory.VmallocChunk 11 1435077219")
assert.NoError(t, err)
assert.Equal(t, exp.String(), m.String())
}
func TestFilterMatchWildcard(t *testing.T) {
p, err := NewGraphiteParser("", []string{"servers.* .host.measurement*"}, nil)
if err != nil {
t.Fatalf("unexpected error creating parser, got %v", err)
}
exp, err := telegraf.NewMetric("cpu_load",
map[string]string{"host": "localhost"},
map[string]interface{}{"value": float64(11)},
time.Unix(1435077219, 0))
assert.NoError(t, err)
m, err := p.ParseLine("servers.localhost.cpu_load 11 1435077219")
assert.NoError(t, err)
assert.Equal(t, exp.String(), m.String())
}
func TestFilterMatchExactBeforeWildcard(t *testing.T) {
p, err := NewGraphiteParser("", []string{
"servers.* .wrong.measurement*",
"servers.localhost .host.measurement*"}, nil)
if err != nil {
t.Fatalf("unexpected error creating parser, got %v", err)
}
exp, err := telegraf.NewMetric("cpu_load",
map[string]string{"host": "localhost"},
map[string]interface{}{"value": float64(11)},
time.Unix(1435077219, 0))
assert.NoError(t, err)
m, err := p.ParseLine("servers.localhost.cpu_load 11 1435077219")
assert.NoError(t, err)
assert.Equal(t, exp.String(), m.String())
}
func TestFilterMatchMostLongestFilter(t *testing.T) {
p, err := NewGraphiteParser("", []string{
"*.* .wrong.measurement*",
"servers.* .wrong.measurement*",
"servers.localhost .wrong.measurement*",
"servers.localhost.cpu .host.resource.measurement*", // should match this
"*.localhost .wrong.measurement*",
}, nil)
if err != nil {
t.Fatalf("unexpected error creating parser, got %v", err)
}
exp, err := telegraf.NewMetric("cpu_load",
map[string]string{"host": "localhost", "resource": "cpu"},
map[string]interface{}{"value": float64(11)},
time.Unix(1435077219, 0))
assert.NoError(t, err)
m, err := p.ParseLine("servers.localhost.cpu.cpu_load 11 1435077219")
assert.NoError(t, err)
assert.Equal(t, exp.String(), m.String())
}
func TestFilterMatchMultipleWildcards(t *testing.T) {
p, err := NewGraphiteParser("", []string{
"*.* .wrong.measurement*",
"servers.* .host.measurement*", // should match this
"servers.localhost .wrong.measurement*",
"*.localhost .wrong.measurement*",
}, nil)
if err != nil {
t.Fatalf("unexpected error creating parser, got %v", err)
}
exp, err := telegraf.NewMetric("cpu_load",
map[string]string{"host": "server01"},
map[string]interface{}{"value": float64(11)},
time.Unix(1435077219, 0))
assert.NoError(t, err)
m, err := p.ParseLine("servers.server01.cpu_load 11 1435077219")
assert.NoError(t, err)
assert.Equal(t, exp.String(), m.String())
}
func TestParseDefaultTags(t *testing.T) {
p, err := NewGraphiteParser("", []string{"servers.localhost .host.measurement*"}, map[string]string{
"region": "us-east",
"zone": "1c",
"host": "should not set",
})
if err != nil {
t.Fatalf("unexpected error creating parser, got %v", err)
}
exp, err := telegraf.NewMetric("cpu_load",
map[string]string{"host": "localhost", "region": "us-east", "zone": "1c"},
map[string]interface{}{"value": float64(11)},
time.Unix(1435077219, 0))
assert.NoError(t, err)
m, err := p.ParseLine("servers.localhost.cpu_load 11 1435077219")
assert.NoError(t, err)
assert.Equal(t, exp.String(), m.String())
}
func TestParseDefaultTemplateTags(t *testing.T) {
p, err := NewGraphiteParser("", []string{"servers.localhost .host.measurement* zone=1c"}, map[string]string{
"region": "us-east",
"host": "should not set",
})
if err != nil {
t.Fatalf("unexpected error creating parser, got %v", err)
}
exp, err := telegraf.NewMetric("cpu_load",
map[string]string{"host": "localhost", "region": "us-east", "zone": "1c"},
map[string]interface{}{"value": float64(11)},
time.Unix(1435077219, 0))
assert.NoError(t, err)
m, err := p.ParseLine("servers.localhost.cpu_load 11 1435077219")
assert.NoError(t, err)
assert.Equal(t, exp.String(), m.String())
}
func TestParseDefaultTemplateTagsOverridGlobal(t *testing.T) {
p, err := NewGraphiteParser("", []string{"servers.localhost .host.measurement* zone=1c,region=us-east"}, map[string]string{
"region": "shot not be set",
"host": "should not set",
})
if err != nil {
t.Fatalf("unexpected error creating parser, got %v", err)
}
exp, err := telegraf.NewMetric("cpu_load",
map[string]string{"host": "localhost", "region": "us-east", "zone": "1c"},
map[string]interface{}{"value": float64(11)},
time.Unix(1435077219, 0))
assert.NoError(t, err)
m, err := p.ParseLine("servers.localhost.cpu_load 11 1435077219")
assert.NoError(t, err)
assert.Equal(t, exp.String(), m.String())
}
func TestParseTemplateWhitespace(t *testing.T) {
p, err := NewGraphiteParser("",
[]string{"servers.localhost .host.measurement* zone=1c"},
map[string]string{
"region": "us-east",
"host": "should not set",
})
if err != nil {
t.Fatalf("unexpected error creating parser, got %v", err)
}
exp, err := telegraf.NewMetric("cpu_load",
map[string]string{"host": "localhost", "region": "us-east", "zone": "1c"},
map[string]interface{}{"value": float64(11)},
time.Unix(1435077219, 0))
assert.NoError(t, err)
m, err := p.ParseLine("servers.localhost.cpu_load 11 1435077219")
assert.NoError(t, err)
assert.Equal(t, exp.String(), m.String())
}
// Test basic functionality of ApplyTemplate
func TestApplyTemplate(t *testing.T) {
p, err := NewGraphiteParser("_",
[]string{"current.* measurement.measurement"},
nil)
assert.NoError(t, err)
measurement, _, _, _ := p.ApplyTemplate("current.users")
assert.Equal(t, "current_users", measurement)
}
// Test basic functionality of ApplyTemplate
func TestApplyTemplateNoMatch(t *testing.T) {
p, err := NewGraphiteParser(".",
[]string{"foo.bar measurement.measurement"},
nil)
assert.NoError(t, err)
measurement, _, _, _ := p.ApplyTemplate("current.users")
assert.Equal(t, "current.users", measurement)
}
// Test that most specific template is chosen
func TestApplyTemplateSpecific(t *testing.T) {
p, err := NewGraphiteParser("_",
[]string{
"current.* measurement.measurement",
"current.*.* measurement.measurement.service",
}, nil)
assert.NoError(t, err)
measurement, tags, _, _ := p.ApplyTemplate("current.users.facebook")
assert.Equal(t, "current_users", measurement)
service, ok := tags["service"]
if !ok {
t.Error("Expected for template to apply a 'service' tag, but not found")
}
if service != "facebook" {
t.Errorf("Expected service='facebook' tag, got service='%s'", service)
}
}
func TestApplyTemplateTags(t *testing.T) {
p, err := NewGraphiteParser("_",
[]string{"current.* measurement.measurement region=us-west"}, nil)
assert.NoError(t, err)
measurement, tags, _, _ := p.ApplyTemplate("current.users")
assert.Equal(t, "current_users", measurement)
region, ok := tags["region"]
if !ok {
t.Error("Expected for template to apply a 'region' tag, but not found")
}
if region != "us-west" {
t.Errorf("Expected region='us-west' tag, got region='%s'", region)
}
}
func TestApplyTemplateField(t *testing.T) {
p, err := NewGraphiteParser("_",
[]string{"current.* measurement.measurement.field"}, nil)
assert.NoError(t, err)
measurement, _, field, err := p.ApplyTemplate("current.users.logged_in")
assert.Equal(t, "current_users", measurement)
if field != "logged_in" {
t.Errorf("Parser.ApplyTemplate unexpected result. got %s, exp %s",
field, "logged_in")
}
}
func TestApplyTemplateFieldError(t *testing.T) {
p, err := NewGraphiteParser("_",
[]string{"current.* measurement.field.field"}, nil)
assert.NoError(t, err)
_, _, _, err = p.ApplyTemplate("current.users.logged_in")
if err == nil {
t.Errorf("Parser.ApplyTemplate unexpected result. got %s, exp %s", err,
"'field' can only be used once in each template: current.users.logged_in")
}
}
// Test Helpers
func errstr(err error) string {
if err != nil {
return err.Error()
}
return ""
}

View File

@@ -0,0 +1,57 @@
package influx
import (
"bytes"
"fmt"
"github.com/influxdata/telegraf"
"github.com/influxdata/influxdb/models"
)
// InfluxParser is an object for Parsing incoming metrics.
type InfluxParser struct {
// DefaultTags will be added to every parsed metric
DefaultTags map[string]string
}
// ParseMetrics returns a slice of Metrics from a text representation of a
// metric (in line-protocol format)
// with each metric separated by newlines. If any metrics fail to parse,
// a non-nil error will be returned in addition to the metrics that parsed
// successfully.
func (p *InfluxParser) Parse(buf []byte) ([]telegraf.Metric, error) {
// parse even if the buffer begins with a newline
buf = bytes.TrimPrefix(buf, []byte("\n"))
points, err := models.ParsePoints(buf)
metrics := make([]telegraf.Metric, len(points))
for i, point := range points {
tags := point.Tags()
for k, v := range p.DefaultTags {
// Only set tags not in parsed metric
if _, ok := tags[k]; !ok {
tags[k] = v
}
}
// Ignore error here because it's impossible that a model.Point
// wouldn't parse into client.Point properly
metrics[i], _ = telegraf.NewMetric(point.Name(), tags,
point.Fields(), point.Time())
}
return metrics, err
}
func (p *InfluxParser) ParseLine(line string) (telegraf.Metric, error) {
metrics, err := p.Parse([]byte(line + "\n"))
if err != nil {
return nil, err
}
if len(metrics) < 1 {
return nil, fmt.Errorf(
"Can not parse the line: %s, for data format: influx ", line)
}
return metrics[0], nil
}

View File

@@ -0,0 +1,194 @@
package influx
import (
"testing"
"time"
"github.com/stretchr/testify/assert"
)
var exptime = time.Date(2009, time.November, 10, 23, 0, 0, 0, time.UTC)
const (
validInflux = "cpu_load_short,cpu=cpu0 value=10 1257894000000000000"
validInfluxNewline = "\ncpu_load_short,cpu=cpu0 value=10 1257894000000000000\n"
invalidInflux = "I don't think this is line protocol"
invalidInflux2 = "{\"a\": 5, \"b\": {\"c\": 6}}"
)
const influxMulti = `
cpu,host=foo,datacenter=us-east usage_idle=99,usage_busy=1
cpu,host=foo,datacenter=us-east usage_idle=99,usage_busy=1
cpu,host=foo,datacenter=us-east usage_idle=99,usage_busy=1
cpu,host=foo,datacenter=us-east usage_idle=99,usage_busy=1
cpu,host=foo,datacenter=us-east usage_idle=99,usage_busy=1
cpu,host=foo,datacenter=us-east usage_idle=99,usage_busy=1
cpu,host=foo,datacenter=us-east usage_idle=99,usage_busy=1
`
const influxMultiSomeInvalid = `
cpu,host=foo,datacenter=us-east usage_idle=99,usage_busy=1
cpu,host=foo,datacenter=us-east usage_idle=99,usage_busy=1
cpu,host=foo,datacenter=us-east usage_idle=99,usage_busy=1
cpu,cpu=cpu3, host=foo,datacenter=us-east usage_idle=99,usage_busy=1
cpu,cpu=cpu4 , usage_idle=99,usage_busy=1
cpu,host=foo,datacenter=us-east usage_idle=99,usage_busy=1
`
func TestParseValidInflux(t *testing.T) {
parser := InfluxParser{}
metrics, err := parser.Parse([]byte(validInflux))
assert.NoError(t, err)
assert.Len(t, metrics, 1)
assert.Equal(t, "cpu_load_short", metrics[0].Name())
assert.Equal(t, map[string]interface{}{
"value": float64(10),
}, metrics[0].Fields())
assert.Equal(t, map[string]string{
"cpu": "cpu0",
}, metrics[0].Tags())
assert.Equal(t, exptime, metrics[0].Time())
metrics, err = parser.Parse([]byte(validInfluxNewline))
assert.NoError(t, err)
assert.Len(t, metrics, 1)
assert.Equal(t, "cpu_load_short", metrics[0].Name())
assert.Equal(t, map[string]interface{}{
"value": float64(10),
}, metrics[0].Fields())
assert.Equal(t, map[string]string{
"cpu": "cpu0",
}, metrics[0].Tags())
assert.Equal(t, exptime, metrics[0].Time())
}
func TestParseLineValidInflux(t *testing.T) {
parser := InfluxParser{}
metric, err := parser.ParseLine(validInflux)
assert.NoError(t, err)
assert.Equal(t, "cpu_load_short", metric.Name())
assert.Equal(t, map[string]interface{}{
"value": float64(10),
}, metric.Fields())
assert.Equal(t, map[string]string{
"cpu": "cpu0",
}, metric.Tags())
assert.Equal(t, exptime, metric.Time())
metric, err = parser.ParseLine(validInfluxNewline)
assert.NoError(t, err)
assert.Equal(t, "cpu_load_short", metric.Name())
assert.Equal(t, map[string]interface{}{
"value": float64(10),
}, metric.Fields())
assert.Equal(t, map[string]string{
"cpu": "cpu0",
}, metric.Tags())
assert.Equal(t, exptime, metric.Time())
}
func TestParseMultipleValid(t *testing.T) {
parser := InfluxParser{}
metrics, err := parser.Parse([]byte(influxMulti))
assert.NoError(t, err)
assert.Len(t, metrics, 7)
for _, metric := range metrics {
assert.Equal(t, "cpu", metric.Name())
assert.Equal(t, map[string]string{
"datacenter": "us-east",
"host": "foo",
}, metrics[0].Tags())
assert.Equal(t, map[string]interface{}{
"usage_idle": float64(99),
"usage_busy": float64(1),
}, metrics[0].Fields())
}
}
func TestParseSomeValid(t *testing.T) {
parser := InfluxParser{}
metrics, err := parser.Parse([]byte(influxMultiSomeInvalid))
assert.Error(t, err)
assert.Len(t, metrics, 4)
for _, metric := range metrics {
assert.Equal(t, "cpu", metric.Name())
assert.Equal(t, map[string]string{
"datacenter": "us-east",
"host": "foo",
}, metrics[0].Tags())
assert.Equal(t, map[string]interface{}{
"usage_idle": float64(99),
"usage_busy": float64(1),
}, metrics[0].Fields())
}
}
// Test that default tags are applied.
func TestParseDefaultTags(t *testing.T) {
parser := InfluxParser{
DefaultTags: map[string]string{
"tag": "default",
},
}
metrics, err := parser.Parse([]byte(influxMultiSomeInvalid))
assert.Error(t, err)
assert.Len(t, metrics, 4)
for _, metric := range metrics {
assert.Equal(t, "cpu", metric.Name())
assert.Equal(t, map[string]string{
"datacenter": "us-east",
"host": "foo",
"tag": "default",
}, metrics[0].Tags())
assert.Equal(t, map[string]interface{}{
"usage_idle": float64(99),
"usage_busy": float64(1),
}, metrics[0].Fields())
}
}
// Verify that metric tags will override default tags
func TestParseDefaultTagsOverride(t *testing.T) {
parser := InfluxParser{
DefaultTags: map[string]string{
"host": "default",
},
}
metrics, err := parser.Parse([]byte(influxMultiSomeInvalid))
assert.Error(t, err)
assert.Len(t, metrics, 4)
for _, metric := range metrics {
assert.Equal(t, "cpu", metric.Name())
assert.Equal(t, map[string]string{
"datacenter": "us-east",
"host": "foo",
}, metrics[0].Tags())
assert.Equal(t, map[string]interface{}{
"usage_idle": float64(99),
"usage_busy": float64(1),
}, metrics[0].Fields())
}
}
func TestParseInvalidInflux(t *testing.T) {
parser := InfluxParser{}
_, err := parser.Parse([]byte(invalidInflux))
assert.Error(t, err)
_, err = parser.Parse([]byte(invalidInflux2))
assert.Error(t, err)
_, err = parser.ParseLine(invalidInflux)
assert.Error(t, err)
_, err = parser.ParseLine(invalidInflux2)
assert.Error(t, err)
}

View File

@@ -0,0 +1,109 @@
package json
import (
"encoding/json"
"fmt"
"strconv"
"strings"
"time"
"github.com/influxdata/telegraf"
)
type JSONParser struct {
MetricName string
TagKeys []string
DefaultTags map[string]string
}
func (p *JSONParser) Parse(buf []byte) ([]telegraf.Metric, error) {
metrics := make([]telegraf.Metric, 0)
var jsonOut map[string]interface{}
err := json.Unmarshal(buf, &jsonOut)
if err != nil {
err = fmt.Errorf("unable to parse out as JSON, %s", err)
return nil, err
}
tags := make(map[string]string)
for k, v := range p.DefaultTags {
tags[k] = v
}
for _, tag := range p.TagKeys {
switch v := jsonOut[tag].(type) {
case string:
tags[tag] = v
}
delete(jsonOut, tag)
}
f := JSONFlattener{}
err = f.FlattenJSON("", jsonOut)
if err != nil {
return nil, err
}
metric, err := telegraf.NewMetric(p.MetricName, tags, f.Fields, time.Now().UTC())
if err != nil {
return nil, err
}
return append(metrics, metric), nil
}
func (p *JSONParser) ParseLine(line string) (telegraf.Metric, error) {
metrics, err := p.Parse([]byte(line + "\n"))
if err != nil {
return nil, err
}
if len(metrics) < 1 {
return nil, fmt.Errorf("Can not parse the line: %s, for data format: influx ", line)
}
return metrics[0], nil
}
type JSONFlattener struct {
Fields map[string]interface{}
}
// FlattenJSON flattens nested maps/interfaces into a fields map
func (f *JSONFlattener) FlattenJSON(
fieldname string,
v interface{},
) error {
if f.Fields == nil {
f.Fields = make(map[string]interface{})
}
fieldname = strings.Trim(fieldname, "_")
switch t := v.(type) {
case map[string]interface{}:
for k, v := range t {
err := f.FlattenJSON(fieldname+"_"+k+"_", v)
if err != nil {
return err
}
}
case []interface{}:
for i, v := range t {
k := strconv.Itoa(i)
err := f.FlattenJSON(fieldname+"_"+k+"_", v)
if err != nil {
return nil
}
}
case float64:
f.Fields[fieldname] = t
case bool, string, nil:
// ignored types
return nil
default:
return fmt.Errorf("JSON Flattener: got unexpected type %T with value %v (%s)",
t, t, fieldname)
}
return nil
}

View File

@@ -0,0 +1,284 @@
package json
import (
"testing"
"github.com/stretchr/testify/assert"
)
const (
validJSON = "{\"a\": 5, \"b\": {\"c\": 6}}"
validJSONNewline = "\n{\"d\": 7, \"b\": {\"d\": 8}}\n"
invalidJSON = "I don't think this is JSON"
invalidJSON2 = "{\"a\": 5, \"b\": \"c\": 6}}"
)
const validJSONTags = `
{
"a": 5,
"b": {
"c": 6
},
"mytag": "foobar",
"othertag": "baz"
}
`
func TestParseValidJSON(t *testing.T) {
parser := JSONParser{
MetricName: "json_test",
}
// Most basic vanilla test
metrics, err := parser.Parse([]byte(validJSON))
assert.NoError(t, err)
assert.Len(t, metrics, 1)
assert.Equal(t, "json_test", metrics[0].Name())
assert.Equal(t, map[string]interface{}{
"a": float64(5),
"b_c": float64(6),
}, metrics[0].Fields())
assert.Equal(t, map[string]string{}, metrics[0].Tags())
// Test that newlines are fine
metrics, err = parser.Parse([]byte(validJSONNewline))
assert.NoError(t, err)
assert.Len(t, metrics, 1)
assert.Equal(t, "json_test", metrics[0].Name())
assert.Equal(t, map[string]interface{}{
"d": float64(7),
"b_d": float64(8),
}, metrics[0].Fields())
assert.Equal(t, map[string]string{}, metrics[0].Tags())
// Test that strings without TagKeys defined are ignored
metrics, err = parser.Parse([]byte(validJSONTags))
assert.NoError(t, err)
assert.Len(t, metrics, 1)
assert.Equal(t, "json_test", metrics[0].Name())
assert.Equal(t, map[string]interface{}{
"a": float64(5),
"b_c": float64(6),
}, metrics[0].Fields())
assert.Equal(t, map[string]string{}, metrics[0].Tags())
}
func TestParseLineValidJSON(t *testing.T) {
parser := JSONParser{
MetricName: "json_test",
}
// Most basic vanilla test
metric, err := parser.ParseLine(validJSON)
assert.NoError(t, err)
assert.Equal(t, "json_test", metric.Name())
assert.Equal(t, map[string]interface{}{
"a": float64(5),
"b_c": float64(6),
}, metric.Fields())
assert.Equal(t, map[string]string{}, metric.Tags())
// Test that newlines are fine
metric, err = parser.ParseLine(validJSONNewline)
assert.NoError(t, err)
assert.Equal(t, "json_test", metric.Name())
assert.Equal(t, map[string]interface{}{
"d": float64(7),
"b_d": float64(8),
}, metric.Fields())
assert.Equal(t, map[string]string{}, metric.Tags())
// Test that strings without TagKeys defined are ignored
metric, err = parser.ParseLine(validJSONTags)
assert.NoError(t, err)
assert.Equal(t, "json_test", metric.Name())
assert.Equal(t, map[string]interface{}{
"a": float64(5),
"b_c": float64(6),
}, metric.Fields())
assert.Equal(t, map[string]string{}, metric.Tags())
}
func TestParseInvalidJSON(t *testing.T) {
parser := JSONParser{
MetricName: "json_test",
}
_, err := parser.Parse([]byte(invalidJSON))
assert.Error(t, err)
_, err = parser.Parse([]byte(invalidJSON2))
assert.Error(t, err)
_, err = parser.ParseLine(invalidJSON)
assert.Error(t, err)
}
func TestParseWithTagKeys(t *testing.T) {
// Test that strings not matching tag keys are ignored
parser := JSONParser{
MetricName: "json_test",
TagKeys: []string{"wrongtagkey"},
}
metrics, err := parser.Parse([]byte(validJSONTags))
assert.NoError(t, err)
assert.Len(t, metrics, 1)
assert.Equal(t, "json_test", metrics[0].Name())
assert.Equal(t, map[string]interface{}{
"a": float64(5),
"b_c": float64(6),
}, metrics[0].Fields())
assert.Equal(t, map[string]string{}, metrics[0].Tags())
// Test that single tag key is found and applied
parser = JSONParser{
MetricName: "json_test",
TagKeys: []string{"mytag"},
}
metrics, err = parser.Parse([]byte(validJSONTags))
assert.NoError(t, err)
assert.Len(t, metrics, 1)
assert.Equal(t, "json_test", metrics[0].Name())
assert.Equal(t, map[string]interface{}{
"a": float64(5),
"b_c": float64(6),
}, metrics[0].Fields())
assert.Equal(t, map[string]string{
"mytag": "foobar",
}, metrics[0].Tags())
// Test that both tag keys are found and applied
parser = JSONParser{
MetricName: "json_test",
TagKeys: []string{"mytag", "othertag"},
}
metrics, err = parser.Parse([]byte(validJSONTags))
assert.NoError(t, err)
assert.Len(t, metrics, 1)
assert.Equal(t, "json_test", metrics[0].Name())
assert.Equal(t, map[string]interface{}{
"a": float64(5),
"b_c": float64(6),
}, metrics[0].Fields())
assert.Equal(t, map[string]string{
"mytag": "foobar",
"othertag": "baz",
}, metrics[0].Tags())
}
func TestParseLineWithTagKeys(t *testing.T) {
// Test that strings not matching tag keys are ignored
parser := JSONParser{
MetricName: "json_test",
TagKeys: []string{"wrongtagkey"},
}
metric, err := parser.ParseLine(validJSONTags)
assert.NoError(t, err)
assert.Equal(t, "json_test", metric.Name())
assert.Equal(t, map[string]interface{}{
"a": float64(5),
"b_c": float64(6),
}, metric.Fields())
assert.Equal(t, map[string]string{}, metric.Tags())
// Test that single tag key is found and applied
parser = JSONParser{
MetricName: "json_test",
TagKeys: []string{"mytag"},
}
metric, err = parser.ParseLine(validJSONTags)
assert.NoError(t, err)
assert.Equal(t, "json_test", metric.Name())
assert.Equal(t, map[string]interface{}{
"a": float64(5),
"b_c": float64(6),
}, metric.Fields())
assert.Equal(t, map[string]string{
"mytag": "foobar",
}, metric.Tags())
// Test that both tag keys are found and applied
parser = JSONParser{
MetricName: "json_test",
TagKeys: []string{"mytag", "othertag"},
}
metric, err = parser.ParseLine(validJSONTags)
assert.NoError(t, err)
assert.Equal(t, "json_test", metric.Name())
assert.Equal(t, map[string]interface{}{
"a": float64(5),
"b_c": float64(6),
}, metric.Fields())
assert.Equal(t, map[string]string{
"mytag": "foobar",
"othertag": "baz",
}, metric.Tags())
}
func TestParseValidJSONDefaultTags(t *testing.T) {
parser := JSONParser{
MetricName: "json_test",
TagKeys: []string{"mytag"},
DefaultTags: map[string]string{
"t4g": "default",
},
}
// Most basic vanilla test
metrics, err := parser.Parse([]byte(validJSON))
assert.NoError(t, err)
assert.Len(t, metrics, 1)
assert.Equal(t, "json_test", metrics[0].Name())
assert.Equal(t, map[string]interface{}{
"a": float64(5),
"b_c": float64(6),
}, metrics[0].Fields())
assert.Equal(t, map[string]string{"t4g": "default"}, metrics[0].Tags())
// Test that tagkeys and default tags are applied
metrics, err = parser.Parse([]byte(validJSONTags))
assert.NoError(t, err)
assert.Len(t, metrics, 1)
assert.Equal(t, "json_test", metrics[0].Name())
assert.Equal(t, map[string]interface{}{
"a": float64(5),
"b_c": float64(6),
}, metrics[0].Fields())
assert.Equal(t, map[string]string{
"t4g": "default",
"mytag": "foobar",
}, metrics[0].Tags())
}
// Test that default tags are overridden by tag keys
func TestParseValidJSONDefaultTagsOverride(t *testing.T) {
parser := JSONParser{
MetricName: "json_test",
TagKeys: []string{"mytag"},
DefaultTags: map[string]string{
"mytag": "default",
},
}
// Most basic vanilla test
metrics, err := parser.Parse([]byte(validJSON))
assert.NoError(t, err)
assert.Len(t, metrics, 1)
assert.Equal(t, "json_test", metrics[0].Name())
assert.Equal(t, map[string]interface{}{
"a": float64(5),
"b_c": float64(6),
}, metrics[0].Fields())
assert.Equal(t, map[string]string{"mytag": "default"}, metrics[0].Tags())
// Test that tagkeys override default tags
metrics, err = parser.Parse([]byte(validJSONTags))
assert.NoError(t, err)
assert.Len(t, metrics, 1)
assert.Equal(t, "json_test", metrics[0].Name())
assert.Equal(t, map[string]interface{}{
"a": float64(5),
"b_c": float64(6),
}, metrics[0].Fields())
assert.Equal(t, map[string]string{
"mytag": "foobar",
}, metrics[0].Tags())
}

View File

@@ -0,0 +1,95 @@
package parsers
import (
"fmt"
"github.com/influxdata/telegraf"
"github.com/influxdata/telegraf/plugins/parsers/graphite"
"github.com/influxdata/telegraf/plugins/parsers/influx"
"github.com/influxdata/telegraf/plugins/parsers/json"
)
// ParserInput is an interface for input plugins that are able to parse
// arbitrary data formats.
type ParserInput interface {
// SetParser sets the parser function for the interface
SetParser(parser Parser)
}
// Parser is an interface defining functions that a parser plugin must satisfy.
type Parser interface {
// Parse takes a byte buffer separated by newlines
// ie, `cpu.usage.idle 90\ncpu.usage.busy 10`
// and parses it into telegraf metrics
Parse(buf []byte) ([]telegraf.Metric, error)
// ParseLine takes a single string metric
// ie, "cpu.usage.idle 90"
// and parses it into a telegraf metric.
ParseLine(line string) (telegraf.Metric, error)
}
// Config is a struct that covers the data types needed for all parser types,
// and can be used to instantiate _any_ of the parsers.
type Config struct {
// Dataformat can be one of: json, influx, graphite
DataFormat string
// Separator only applied to Graphite data.
Separator string
// Templates only apply to Graphite data.
Templates []string
// TagKeys only apply to JSON data
TagKeys []string
// MetricName only applies to JSON data. This will be the name of the measurement.
MetricName string
// DefaultTags are the default tags that will be added to all parsed metrics.
DefaultTags map[string]string
}
// NewParser returns a Parser interface based on the given config.
func NewParser(config *Config) (Parser, error) {
var err error
var parser Parser
switch config.DataFormat {
case "json":
parser, err = NewJSONParser(config.MetricName,
config.TagKeys, config.DefaultTags)
case "influx":
parser, err = NewInfluxParser()
case "graphite":
parser, err = NewGraphiteParser(config.Separator,
config.Templates, config.DefaultTags)
default:
err = fmt.Errorf("Invalid data format: %s", config.DataFormat)
}
return parser, err
}
func NewJSONParser(
metricName string,
tagKeys []string,
defaultTags map[string]string,
) (Parser, error) {
parser := &json.JSONParser{
MetricName: metricName,
TagKeys: tagKeys,
DefaultTags: defaultTags,
}
return parser, nil
}
func NewInfluxParser() (Parser, error) {
return &influx.InfluxParser{}, nil
}
func NewGraphiteParser(
separator string,
templates []string,
defaultTags map[string]string,
) (Parser, error) {
return graphite.NewGraphiteParser(separator, templates, defaultTags)
}