Flush based on buffer size rather than time
this includes: - Add Accumulator to the Start() function of service inputs - For message consumer plugins, use the Accumulator to constantly add metrics and make Gather a dummy function - rework unit tests to match this new behavior. - make "flush_buffer_when_full" a config option that defaults to true closes #666
This commit is contained in:
@@ -61,7 +61,7 @@ func (gh *GithubWebhooks) Listen() {
|
||||
}
|
||||
}
|
||||
|
||||
func (gh *GithubWebhooks) Start() error {
|
||||
func (gh *GithubWebhooks) Start(_ telegraf.Accumulator) error {
|
||||
go gh.Listen()
|
||||
log.Printf("Started the github_webhooks service on %s\n", gh.ServiceAddress)
|
||||
return nil
|
||||
|
||||
@@ -1,7 +1,6 @@
|
||||
package kafka_consumer
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"log"
|
||||
"strings"
|
||||
"sync"
|
||||
@@ -19,11 +18,13 @@ type Kafka struct {
|
||||
Topics []string
|
||||
ZookeeperPeers []string
|
||||
Consumer *consumergroup.ConsumerGroup
|
||||
MetricBuffer int
|
||||
|
||||
// Legacy metric buffer support
|
||||
MetricBuffer int
|
||||
// TODO remove PointBuffer, legacy support
|
||||
PointBuffer int
|
||||
Offset string
|
||||
|
||||
Offset string
|
||||
parser parsers.Parser
|
||||
|
||||
sync.Mutex
|
||||
@@ -32,9 +33,10 @@ type Kafka struct {
|
||||
in <-chan *sarama.ConsumerMessage
|
||||
// channel for all kafka consumer errors
|
||||
errs <-chan *sarama.ConsumerError
|
||||
// channel for all incoming parsed kafka metrics
|
||||
metricC chan telegraf.Metric
|
||||
done chan struct{}
|
||||
done chan struct{}
|
||||
|
||||
// keep the accumulator internally:
|
||||
acc telegraf.Accumulator
|
||||
|
||||
// doNotCommitMsgs tells the parser not to call CommitUpTo on the consumer
|
||||
// this is mostly for test purposes, but there may be a use-case for it later.
|
||||
@@ -48,8 +50,6 @@ var sampleConfig = `
|
||||
zookeeper_peers = ["localhost:2181"]
|
||||
### the name of the consumer group
|
||||
consumer_group = "telegraf_metrics_consumers"
|
||||
### Maximum number of metrics to buffer between collection intervals
|
||||
metric_buffer = 100000
|
||||
### Offset (must be either "oldest" or "newest")
|
||||
offset = "oldest"
|
||||
|
||||
@@ -72,11 +72,13 @@ func (k *Kafka) SetParser(parser parsers.Parser) {
|
||||
k.parser = parser
|
||||
}
|
||||
|
||||
func (k *Kafka) Start() error {
|
||||
func (k *Kafka) Start(acc telegraf.Accumulator) error {
|
||||
k.Lock()
|
||||
defer k.Unlock()
|
||||
var consumerErr error
|
||||
|
||||
k.acc = acc
|
||||
|
||||
config := consumergroup.NewConfig()
|
||||
switch strings.ToLower(k.Offset) {
|
||||
case "oldest", "":
|
||||
@@ -106,13 +108,6 @@ func (k *Kafka) Start() error {
|
||||
}
|
||||
|
||||
k.done = make(chan struct{})
|
||||
if k.PointBuffer == 0 && k.MetricBuffer == 0 {
|
||||
k.MetricBuffer = 100000
|
||||
} else if k.PointBuffer > 0 {
|
||||
// Legacy support of PointBuffer field TODO remove
|
||||
k.MetricBuffer = k.PointBuffer
|
||||
}
|
||||
k.metricC = make(chan telegraf.Metric, k.MetricBuffer)
|
||||
|
||||
// Start the kafka message reader
|
||||
go k.receiver()
|
||||
@@ -138,14 +133,7 @@ func (k *Kafka) receiver() {
|
||||
}
|
||||
|
||||
for _, metric := range metrics {
|
||||
fmt.Println(string(metric.Name()))
|
||||
select {
|
||||
case k.metricC <- metric:
|
||||
continue
|
||||
default:
|
||||
log.Printf("Kafka Consumer buffer is full, dropping a metric." +
|
||||
" You may want to increase the metric_buffer setting")
|
||||
}
|
||||
k.acc.AddFields(metric.Name(), metric.Fields(), metric.Tags(), metric.Time())
|
||||
}
|
||||
|
||||
if !k.doNotCommitMsgs {
|
||||
@@ -169,13 +157,6 @@ func (k *Kafka) Stop() {
|
||||
}
|
||||
|
||||
func (k *Kafka) Gather(acc telegraf.Accumulator) error {
|
||||
k.Lock()
|
||||
defer k.Unlock()
|
||||
nmetrics := len(k.metricC)
|
||||
for i := 0; i < nmetrics; i++ {
|
||||
metric := <-k.metricC
|
||||
acc.AddFields(metric.Name(), metric.Fields(), metric.Tags(), metric.Time())
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
|
||||
@@ -44,18 +44,19 @@ func TestReadsMetricsFromKafka(t *testing.T) {
|
||||
}
|
||||
p, _ := parsers.NewInfluxParser()
|
||||
k.SetParser(p)
|
||||
if err := k.Start(); err != nil {
|
||||
|
||||
// Verify that we can now gather the sent message
|
||||
var acc testutil.Accumulator
|
||||
|
||||
// Sanity check
|
||||
assert.Equal(t, 0, len(acc.Metrics), "There should not be any points")
|
||||
if err := k.Start(&acc); err != nil {
|
||||
t.Fatal(err.Error())
|
||||
} else {
|
||||
defer k.Stop()
|
||||
}
|
||||
|
||||
waitForPoint(k, t)
|
||||
|
||||
// Verify that we can now gather the sent message
|
||||
var acc testutil.Accumulator
|
||||
// Sanity check
|
||||
assert.Equal(t, 0, len(acc.Metrics), "There should not be any points")
|
||||
waitForPoint(&acc, t)
|
||||
|
||||
// Gather points
|
||||
err = k.Gather(&acc)
|
||||
@@ -77,7 +78,7 @@ func TestReadsMetricsFromKafka(t *testing.T) {
|
||||
|
||||
// Waits for the metric that was sent to the kafka broker to arrive at the kafka
|
||||
// consumer
|
||||
func waitForPoint(k *Kafka, t *testing.T) {
|
||||
func waitForPoint(acc *testutil.Accumulator, t *testing.T) {
|
||||
// Give the kafka container up to 2 seconds to get the point to the consumer
|
||||
ticker := time.NewTicker(5 * time.Millisecond)
|
||||
counter := 0
|
||||
@@ -87,7 +88,7 @@ func waitForPoint(k *Kafka, t *testing.T) {
|
||||
counter++
|
||||
if counter > 1000 {
|
||||
t.Fatal("Waited for 5s, point never arrived to consumer")
|
||||
} else if len(k.metricC) == 1 {
|
||||
} else if acc.NFields() == 1 {
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
@@ -4,7 +4,6 @@ import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/influxdata/telegraf"
|
||||
"github.com/influxdata/telegraf/plugins/parsers"
|
||||
"github.com/influxdata/telegraf/testutil"
|
||||
|
||||
@@ -17,29 +16,28 @@ const (
|
||||
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) {
|
||||
in := make(chan *sarama.ConsumerMessage, pointBuffer)
|
||||
func newTestKafka() (*Kafka, chan *sarama.ConsumerMessage) {
|
||||
in := make(chan *sarama.ConsumerMessage, 1000)
|
||||
k := Kafka{
|
||||
ConsumerGroup: "test",
|
||||
Topics: []string{"telegraf"},
|
||||
ZookeeperPeers: []string{"localhost:2181"},
|
||||
PointBuffer: pointBuffer,
|
||||
Offset: "oldest",
|
||||
in: in,
|
||||
doNotCommitMsgs: true,
|
||||
errs: make(chan *sarama.ConsumerError, pointBuffer),
|
||||
errs: make(chan *sarama.ConsumerError, 1000),
|
||||
done: make(chan struct{}),
|
||||
metricC: make(chan telegraf.Metric, pointBuffer),
|
||||
}
|
||||
return &k, in
|
||||
}
|
||||
|
||||
// Test that the parser parses kafka messages into points
|
||||
func TestRunParser(t *testing.T) {
|
||||
k, in := NewTestKafka()
|
||||
k, in := newTestKafka()
|
||||
acc := testutil.Accumulator{}
|
||||
k.acc = &acc
|
||||
defer close(k.done)
|
||||
|
||||
k.parser, _ = parsers.NewInfluxParser()
|
||||
@@ -47,12 +45,14 @@ func TestRunParser(t *testing.T) {
|
||||
in <- saramaMsg(testMsg)
|
||||
time.Sleep(time.Millisecond)
|
||||
|
||||
assert.Equal(t, len(k.metricC), 1)
|
||||
assert.Equal(t, acc.NFields(), 1)
|
||||
}
|
||||
|
||||
// Test that the parser ignores invalid messages
|
||||
func TestRunParserInvalidMsg(t *testing.T) {
|
||||
k, in := NewTestKafka()
|
||||
k, in := newTestKafka()
|
||||
acc := testutil.Accumulator{}
|
||||
k.acc = &acc
|
||||
defer close(k.done)
|
||||
|
||||
k.parser, _ = parsers.NewInfluxParser()
|
||||
@@ -60,27 +60,14 @@ func TestRunParserInvalidMsg(t *testing.T) {
|
||||
in <- saramaMsg(invalidMsg)
|
||||
time.Sleep(time.Millisecond)
|
||||
|
||||
assert.Equal(t, len(k.metricC), 0)
|
||||
}
|
||||
|
||||
// Test that points are dropped when we hit the buffer limit
|
||||
func TestRunParserRespectsBuffer(t *testing.T) {
|
||||
k, in := NewTestKafka()
|
||||
defer close(k.done)
|
||||
|
||||
k.parser, _ = parsers.NewInfluxParser()
|
||||
go k.receiver()
|
||||
for i := 0; i < pointBuffer+1; i++ {
|
||||
in <- saramaMsg(testMsg)
|
||||
}
|
||||
time.Sleep(time.Millisecond)
|
||||
|
||||
assert.Equal(t, len(k.metricC), 5)
|
||||
assert.Equal(t, acc.NFields(), 0)
|
||||
}
|
||||
|
||||
// Test that the parser parses kafka messages into points
|
||||
func TestRunParserAndGather(t *testing.T) {
|
||||
k, in := NewTestKafka()
|
||||
k, in := newTestKafka()
|
||||
acc := testutil.Accumulator{}
|
||||
k.acc = &acc
|
||||
defer close(k.done)
|
||||
|
||||
k.parser, _ = parsers.NewInfluxParser()
|
||||
@@ -88,17 +75,18 @@ func TestRunParserAndGather(t *testing.T) {
|
||||
in <- saramaMsg(testMsg)
|
||||
time.Sleep(time.Millisecond)
|
||||
|
||||
acc := testutil.Accumulator{}
|
||||
k.Gather(&acc)
|
||||
|
||||
assert.Equal(t, len(acc.Metrics), 1)
|
||||
assert.Equal(t, acc.NFields(), 1)
|
||||
acc.AssertContainsFields(t, "cpu_load_short",
|
||||
map[string]interface{}{"value": float64(23422)})
|
||||
}
|
||||
|
||||
// Test that the parser parses kafka messages into points
|
||||
func TestRunParserAndGatherGraphite(t *testing.T) {
|
||||
k, in := NewTestKafka()
|
||||
k, in := newTestKafka()
|
||||
acc := testutil.Accumulator{}
|
||||
k.acc = &acc
|
||||
defer close(k.done)
|
||||
|
||||
k.parser, _ = parsers.NewGraphiteParser("_", []string{}, nil)
|
||||
@@ -106,17 +94,18 @@ func TestRunParserAndGatherGraphite(t *testing.T) {
|
||||
in <- saramaMsg(testMsgGraphite)
|
||||
time.Sleep(time.Millisecond)
|
||||
|
||||
acc := testutil.Accumulator{}
|
||||
k.Gather(&acc)
|
||||
|
||||
assert.Equal(t, len(acc.Metrics), 1)
|
||||
assert.Equal(t, acc.NFields(), 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()
|
||||
k, in := newTestKafka()
|
||||
acc := testutil.Accumulator{}
|
||||
k.acc = &acc
|
||||
defer close(k.done)
|
||||
|
||||
k.parser, _ = parsers.NewJSONParser("kafka_json_test", []string{}, nil)
|
||||
@@ -124,10 +113,9 @@ func TestRunParserAndGatherJSON(t *testing.T) {
|
||||
in <- saramaMsg(testMsgJSON)
|
||||
time.Sleep(time.Millisecond)
|
||||
|
||||
acc := testutil.Accumulator{}
|
||||
k.Gather(&acc)
|
||||
|
||||
assert.Equal(t, len(acc.Metrics), 1)
|
||||
assert.Equal(t, acc.NFields(), 2)
|
||||
acc.AssertContainsFields(t, "kafka_json_test",
|
||||
map[string]interface{}{
|
||||
"a": float64(5),
|
||||
|
||||
@@ -15,15 +15,17 @@ import (
|
||||
)
|
||||
|
||||
type MQTTConsumer struct {
|
||||
Servers []string
|
||||
Topics []string
|
||||
Username string
|
||||
Password string
|
||||
MetricBuffer int
|
||||
QoS int `toml:"qos"`
|
||||
Servers []string
|
||||
Topics []string
|
||||
Username string
|
||||
Password string
|
||||
QoS int `toml:"qos"`
|
||||
|
||||
parser parsers.Parser
|
||||
|
||||
// Legacy metric buffer support
|
||||
MetricBuffer int
|
||||
|
||||
// Path to CA file
|
||||
SSLCA string `toml:"ssl_ca"`
|
||||
// Path to host cert file
|
||||
@@ -35,13 +37,12 @@ type MQTTConsumer struct {
|
||||
|
||||
sync.Mutex
|
||||
client *mqtt.Client
|
||||
// channel for all incoming parsed mqtt metrics
|
||||
metricC chan telegraf.Metric
|
||||
// channel for the topics of all incoming metrics (for tagging metrics)
|
||||
topicC chan string
|
||||
// channel of all incoming raw mqtt messages
|
||||
in chan mqtt.Message
|
||||
done chan struct{}
|
||||
|
||||
// keep the accumulator internally:
|
||||
acc telegraf.Accumulator
|
||||
}
|
||||
|
||||
var sampleConfig = `
|
||||
@@ -56,9 +57,6 @@ var sampleConfig = `
|
||||
"sensors/#",
|
||||
]
|
||||
|
||||
### Maximum number of metrics to buffer between collection intervals
|
||||
metric_buffer = 100000
|
||||
|
||||
### username and password to connect MQTT server.
|
||||
# username = "telegraf"
|
||||
# password = "metricsmetricsmetricsmetrics"
|
||||
@@ -89,9 +87,11 @@ func (m *MQTTConsumer) SetParser(parser parsers.Parser) {
|
||||
m.parser = parser
|
||||
}
|
||||
|
||||
func (m *MQTTConsumer) Start() error {
|
||||
func (m *MQTTConsumer) Start(acc telegraf.Accumulator) error {
|
||||
m.Lock()
|
||||
defer m.Unlock()
|
||||
|
||||
m.acc = acc
|
||||
if m.QoS > 2 || m.QoS < 0 {
|
||||
return fmt.Errorf("MQTT Consumer, invalid QoS value: %d", m.QoS)
|
||||
}
|
||||
@@ -106,13 +106,8 @@ func (m *MQTTConsumer) Start() error {
|
||||
return token.Error()
|
||||
}
|
||||
|
||||
m.in = make(chan mqtt.Message, m.MetricBuffer)
|
||||
m.in = make(chan mqtt.Message, 1000)
|
||||
m.done = make(chan struct{})
|
||||
if m.MetricBuffer == 0 {
|
||||
m.MetricBuffer = 100000
|
||||
}
|
||||
m.metricC = make(chan telegraf.Metric, m.MetricBuffer)
|
||||
m.topicC = make(chan string, m.MetricBuffer)
|
||||
|
||||
topics := make(map[string]byte)
|
||||
for _, topic := range m.Topics {
|
||||
@@ -145,13 +140,9 @@ func (m *MQTTConsumer) receiver() {
|
||||
}
|
||||
|
||||
for _, metric := range metrics {
|
||||
select {
|
||||
case m.metricC <- metric:
|
||||
m.topicC <- topic
|
||||
default:
|
||||
log.Printf("MQTT Consumer buffer is full, dropping a metric." +
|
||||
" You may want to increase the metric_buffer setting")
|
||||
}
|
||||
tags := metric.Tags()
|
||||
tags["topic"] = topic
|
||||
m.acc.AddFields(metric.Name(), metric.Fields(), tags, metric.Time())
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -169,16 +160,6 @@ func (m *MQTTConsumer) Stop() {
|
||||
}
|
||||
|
||||
func (m *MQTTConsumer) Gather(acc telegraf.Accumulator) error {
|
||||
m.Lock()
|
||||
defer m.Unlock()
|
||||
nmetrics := len(m.metricC)
|
||||
for i := 0; i < nmetrics; i++ {
|
||||
metric := <-m.metricC
|
||||
topic := <-m.topicC
|
||||
tags := metric.Tags()
|
||||
tags["topic"] = topic
|
||||
acc.AddFields(metric.Name(), metric.Fields(), tags, metric.Time())
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
|
||||
@@ -4,7 +4,6 @@ import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/influxdata/telegraf"
|
||||
"github.com/influxdata/telegraf/plugins/parsers"
|
||||
"github.com/influxdata/telegraf/testutil"
|
||||
|
||||
@@ -16,19 +15,15 @@ const (
|
||||
testMsgGraphite = "cpu.load.short.graphite 23422 1454780029"
|
||||
testMsgJSON = "{\"a\": 5, \"b\": {\"c\": 6}}\n"
|
||||
invalidMsg = "cpu_load_short,host=server01 1422568543702900257"
|
||||
metricBuffer = 5
|
||||
)
|
||||
|
||||
func newTestMQTTConsumer() (*MQTTConsumer, chan mqtt.Message) {
|
||||
in := make(chan mqtt.Message, metricBuffer)
|
||||
in := make(chan mqtt.Message, 100)
|
||||
n := &MQTTConsumer{
|
||||
Topics: []string{"telegraf"},
|
||||
Servers: []string{"localhost:1883"},
|
||||
MetricBuffer: metricBuffer,
|
||||
in: in,
|
||||
done: make(chan struct{}),
|
||||
metricC: make(chan telegraf.Metric, metricBuffer),
|
||||
topicC: make(chan string, metricBuffer),
|
||||
Topics: []string{"telegraf"},
|
||||
Servers: []string{"localhost:1883"},
|
||||
in: in,
|
||||
done: make(chan struct{}),
|
||||
}
|
||||
return n, in
|
||||
}
|
||||
@@ -36,14 +31,16 @@ func newTestMQTTConsumer() (*MQTTConsumer, chan mqtt.Message) {
|
||||
// Test that the parser parses NATS messages into metrics
|
||||
func TestRunParser(t *testing.T) {
|
||||
n, in := newTestMQTTConsumer()
|
||||
acc := testutil.Accumulator{}
|
||||
n.acc = &acc
|
||||
defer close(n.done)
|
||||
|
||||
n.parser, _ = parsers.NewInfluxParser()
|
||||
go n.receiver()
|
||||
in <- mqttMsg(testMsg)
|
||||
time.Sleep(time.Millisecond)
|
||||
time.Sleep(time.Millisecond * 25)
|
||||
|
||||
if a := len(n.metricC); a != 1 {
|
||||
if a := acc.NFields(); a != 1 {
|
||||
t.Errorf("got %v, expected %v", a, 1)
|
||||
}
|
||||
}
|
||||
@@ -51,51 +48,34 @@ func TestRunParser(t *testing.T) {
|
||||
// Test that the parser ignores invalid messages
|
||||
func TestRunParserInvalidMsg(t *testing.T) {
|
||||
n, in := newTestMQTTConsumer()
|
||||
acc := testutil.Accumulator{}
|
||||
n.acc = &acc
|
||||
defer close(n.done)
|
||||
|
||||
n.parser, _ = parsers.NewInfluxParser()
|
||||
go n.receiver()
|
||||
in <- mqttMsg(invalidMsg)
|
||||
time.Sleep(time.Millisecond)
|
||||
time.Sleep(time.Millisecond * 25)
|
||||
|
||||
if a := len(n.metricC); a != 0 {
|
||||
if a := acc.NFields(); a != 0 {
|
||||
t.Errorf("got %v, expected %v", a, 0)
|
||||
}
|
||||
}
|
||||
|
||||
// Test that metrics are dropped when we hit the buffer limit
|
||||
func TestRunParserRespectsBuffer(t *testing.T) {
|
||||
n, in := newTestMQTTConsumer()
|
||||
defer close(n.done)
|
||||
|
||||
n.parser, _ = parsers.NewInfluxParser()
|
||||
go n.receiver()
|
||||
for i := 0; i < metricBuffer+1; i++ {
|
||||
in <- mqttMsg(testMsg)
|
||||
}
|
||||
time.Sleep(time.Millisecond)
|
||||
|
||||
if a := len(n.metricC); a != metricBuffer {
|
||||
t.Errorf("got %v, expected %v", a, metricBuffer)
|
||||
}
|
||||
}
|
||||
|
||||
// Test that the parser parses line format messages into metrics
|
||||
func TestRunParserAndGather(t *testing.T) {
|
||||
n, in := newTestMQTTConsumer()
|
||||
acc := testutil.Accumulator{}
|
||||
n.acc = &acc
|
||||
defer close(n.done)
|
||||
|
||||
n.parser, _ = parsers.NewInfluxParser()
|
||||
go n.receiver()
|
||||
in <- mqttMsg(testMsg)
|
||||
time.Sleep(time.Millisecond)
|
||||
time.Sleep(time.Millisecond * 25)
|
||||
|
||||
acc := testutil.Accumulator{}
|
||||
n.Gather(&acc)
|
||||
|
||||
if a := len(acc.Metrics); a != 1 {
|
||||
t.Errorf("got %v, expected %v", a, 1)
|
||||
}
|
||||
acc.AssertContainsFields(t, "cpu_load_short",
|
||||
map[string]interface{}{"value": float64(23422)})
|
||||
}
|
||||
@@ -103,19 +83,17 @@ func TestRunParserAndGather(t *testing.T) {
|
||||
// Test that the parser parses graphite format messages into metrics
|
||||
func TestRunParserAndGatherGraphite(t *testing.T) {
|
||||
n, in := newTestMQTTConsumer()
|
||||
acc := testutil.Accumulator{}
|
||||
n.acc = &acc
|
||||
defer close(n.done)
|
||||
|
||||
n.parser, _ = parsers.NewGraphiteParser("_", []string{}, nil)
|
||||
go n.receiver()
|
||||
in <- mqttMsg(testMsgGraphite)
|
||||
time.Sleep(time.Millisecond)
|
||||
time.Sleep(time.Millisecond * 25)
|
||||
|
||||
acc := testutil.Accumulator{}
|
||||
n.Gather(&acc)
|
||||
|
||||
if a := len(acc.Metrics); a != 1 {
|
||||
t.Errorf("got %v, expected %v", a, 1)
|
||||
}
|
||||
acc.AssertContainsFields(t, "cpu_load_short_graphite",
|
||||
map[string]interface{}{"value": float64(23422)})
|
||||
}
|
||||
@@ -123,19 +101,17 @@ func TestRunParserAndGatherGraphite(t *testing.T) {
|
||||
// Test that the parser parses json format messages into metrics
|
||||
func TestRunParserAndGatherJSON(t *testing.T) {
|
||||
n, in := newTestMQTTConsumer()
|
||||
acc := testutil.Accumulator{}
|
||||
n.acc = &acc
|
||||
defer close(n.done)
|
||||
|
||||
n.parser, _ = parsers.NewJSONParser("nats_json_test", []string{}, nil)
|
||||
go n.receiver()
|
||||
in <- mqttMsg(testMsgJSON)
|
||||
time.Sleep(time.Millisecond)
|
||||
time.Sleep(time.Millisecond * 25)
|
||||
|
||||
acc := testutil.Accumulator{}
|
||||
n.Gather(&acc)
|
||||
|
||||
if a := len(acc.Metrics); a != 1 {
|
||||
t.Errorf("got %v, expected %v", a, 1)
|
||||
}
|
||||
acc.AssertContainsFields(t, "nats_json_test",
|
||||
map[string]interface{}{
|
||||
"a": float64(5),
|
||||
|
||||
@@ -28,8 +28,10 @@ type natsConsumer struct {
|
||||
Servers []string
|
||||
Secure bool
|
||||
|
||||
// Legacy metric buffer support
|
||||
MetricBuffer int
|
||||
parser parsers.Parser
|
||||
|
||||
parser parsers.Parser
|
||||
|
||||
sync.Mutex
|
||||
Conn *nats.Conn
|
||||
@@ -39,9 +41,8 @@ type natsConsumer struct {
|
||||
in chan *nats.Msg
|
||||
// channel for all NATS read errors
|
||||
errs chan error
|
||||
// channel for all incoming parsed metrics
|
||||
metricC chan telegraf.Metric
|
||||
done chan struct{}
|
||||
done chan struct{}
|
||||
acc telegraf.Accumulator
|
||||
}
|
||||
|
||||
var sampleConfig = `
|
||||
@@ -53,9 +54,7 @@ var sampleConfig = `
|
||||
subjects = ["telegraf"]
|
||||
### name a queue group
|
||||
queue_group = "telegraf_consumers"
|
||||
### Maximum number of metrics to buffer between collection intervals
|
||||
metric_buffer = 100000
|
||||
|
||||
|
||||
### 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:
|
||||
@@ -84,10 +83,12 @@ func (n *natsConsumer) natsErrHandler(c *nats.Conn, s *nats.Subscription, e erro
|
||||
}
|
||||
|
||||
// Start the nats consumer. Caller must call *natsConsumer.Stop() to clean up.
|
||||
func (n *natsConsumer) Start() error {
|
||||
func (n *natsConsumer) Start(acc telegraf.Accumulator) error {
|
||||
n.Lock()
|
||||
defer n.Unlock()
|
||||
|
||||
n.acc = acc
|
||||
|
||||
var connectErr error
|
||||
|
||||
opts := nats.DefaultOptions
|
||||
@@ -115,11 +116,6 @@ func (n *natsConsumer) Start() error {
|
||||
}
|
||||
|
||||
n.done = make(chan struct{})
|
||||
if n.MetricBuffer == 0 {
|
||||
n.MetricBuffer = 100000
|
||||
}
|
||||
|
||||
n.metricC = make(chan telegraf.Metric, n.MetricBuffer)
|
||||
|
||||
// Start the message reader
|
||||
go n.receiver()
|
||||
@@ -146,13 +142,7 @@ func (n *natsConsumer) receiver() {
|
||||
}
|
||||
|
||||
for _, metric := range metrics {
|
||||
select {
|
||||
case n.metricC <- metric:
|
||||
continue
|
||||
default:
|
||||
log.Printf("NATS Consumer buffer is full, dropping a metric." +
|
||||
" You may want to increase the metric_buffer setting")
|
||||
}
|
||||
n.acc.AddFields(metric.Name(), metric.Fields(), metric.Tags(), metric.Time())
|
||||
}
|
||||
|
||||
}
|
||||
@@ -163,7 +153,6 @@ func (n *natsConsumer) clean() {
|
||||
n.Lock()
|
||||
defer n.Unlock()
|
||||
close(n.in)
|
||||
close(n.metricC)
|
||||
close(n.errs)
|
||||
|
||||
for _, sub := range n.Subs {
|
||||
@@ -185,13 +174,6 @@ func (n *natsConsumer) Stop() {
|
||||
}
|
||||
|
||||
func (n *natsConsumer) Gather(acc telegraf.Accumulator) error {
|
||||
n.Lock()
|
||||
defer n.Unlock()
|
||||
nmetrics := len(n.metricC)
|
||||
for i := 0; i < nmetrics; i++ {
|
||||
metric := <-n.metricC
|
||||
acc.AddFields(metric.Name(), metric.Fields(), metric.Tags(), metric.Time())
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
|
||||
@@ -4,7 +4,6 @@ import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/influxdata/telegraf"
|
||||
"github.com/influxdata/telegraf/plugins/parsers"
|
||||
"github.com/influxdata/telegraf/testutil"
|
||||
"github.com/nats-io/nats"
|
||||
@@ -21,15 +20,13 @@ const (
|
||||
func newTestNatsConsumer() (*natsConsumer, chan *nats.Msg) {
|
||||
in := make(chan *nats.Msg, metricBuffer)
|
||||
n := &natsConsumer{
|
||||
QueueGroup: "test",
|
||||
Subjects: []string{"telegraf"},
|
||||
Servers: []string{"nats://localhost:4222"},
|
||||
Secure: false,
|
||||
MetricBuffer: metricBuffer,
|
||||
in: in,
|
||||
errs: make(chan error, metricBuffer),
|
||||
done: make(chan struct{}),
|
||||
metricC: make(chan telegraf.Metric, metricBuffer),
|
||||
QueueGroup: "test",
|
||||
Subjects: []string{"telegraf"},
|
||||
Servers: []string{"nats://localhost:4222"},
|
||||
Secure: false,
|
||||
in: in,
|
||||
errs: make(chan error, metricBuffer),
|
||||
done: make(chan struct{}),
|
||||
}
|
||||
return n, in
|
||||
}
|
||||
@@ -37,66 +34,51 @@ func newTestNatsConsumer() (*natsConsumer, chan *nats.Msg) {
|
||||
// Test that the parser parses NATS messages into metrics
|
||||
func TestRunParser(t *testing.T) {
|
||||
n, in := newTestNatsConsumer()
|
||||
acc := testutil.Accumulator{}
|
||||
n.acc = &acc
|
||||
defer close(n.done)
|
||||
|
||||
n.parser, _ = parsers.NewInfluxParser()
|
||||
go n.receiver()
|
||||
in <- natsMsg(testMsg)
|
||||
time.Sleep(time.Millisecond)
|
||||
time.Sleep(time.Millisecond * 25)
|
||||
|
||||
if a := len(n.metricC); a != 1 {
|
||||
t.Errorf("got %v, expected %v", a, 1)
|
||||
if acc.NFields() != 1 {
|
||||
t.Errorf("got %v, expected %v", acc.NFields(), 1)
|
||||
}
|
||||
}
|
||||
|
||||
// Test that the parser ignores invalid messages
|
||||
func TestRunParserInvalidMsg(t *testing.T) {
|
||||
n, in := newTestNatsConsumer()
|
||||
acc := testutil.Accumulator{}
|
||||
n.acc = &acc
|
||||
defer close(n.done)
|
||||
|
||||
n.parser, _ = parsers.NewInfluxParser()
|
||||
go n.receiver()
|
||||
in <- natsMsg(invalidMsg)
|
||||
time.Sleep(time.Millisecond)
|
||||
time.Sleep(time.Millisecond * 25)
|
||||
|
||||
if a := len(n.metricC); a != 0 {
|
||||
t.Errorf("got %v, expected %v", a, 0)
|
||||
}
|
||||
}
|
||||
|
||||
// Test that metrics are dropped when we hit the buffer limit
|
||||
func TestRunParserRespectsBuffer(t *testing.T) {
|
||||
n, in := newTestNatsConsumer()
|
||||
defer close(n.done)
|
||||
|
||||
n.parser, _ = parsers.NewInfluxParser()
|
||||
go n.receiver()
|
||||
for i := 0; i < metricBuffer+1; i++ {
|
||||
in <- natsMsg(testMsg)
|
||||
}
|
||||
time.Sleep(time.Millisecond)
|
||||
|
||||
if a := len(n.metricC); a != metricBuffer {
|
||||
t.Errorf("got %v, expected %v", a, metricBuffer)
|
||||
if acc.NFields() != 0 {
|
||||
t.Errorf("got %v, expected %v", acc.NFields(), 0)
|
||||
}
|
||||
}
|
||||
|
||||
// Test that the parser parses line format messages into metrics
|
||||
func TestRunParserAndGather(t *testing.T) {
|
||||
n, in := newTestNatsConsumer()
|
||||
acc := testutil.Accumulator{}
|
||||
n.acc = &acc
|
||||
defer close(n.done)
|
||||
|
||||
n.parser, _ = parsers.NewInfluxParser()
|
||||
go n.receiver()
|
||||
in <- natsMsg(testMsg)
|
||||
time.Sleep(time.Millisecond)
|
||||
time.Sleep(time.Millisecond * 25)
|
||||
|
||||
acc := testutil.Accumulator{}
|
||||
n.Gather(&acc)
|
||||
|
||||
if a := len(acc.Metrics); a != 1 {
|
||||
t.Errorf("got %v, expected %v", a, 1)
|
||||
}
|
||||
acc.AssertContainsFields(t, "cpu_load_short",
|
||||
map[string]interface{}{"value": float64(23422)})
|
||||
}
|
||||
@@ -104,19 +86,17 @@ func TestRunParserAndGather(t *testing.T) {
|
||||
// Test that the parser parses graphite format messages into metrics
|
||||
func TestRunParserAndGatherGraphite(t *testing.T) {
|
||||
n, in := newTestNatsConsumer()
|
||||
acc := testutil.Accumulator{}
|
||||
n.acc = &acc
|
||||
defer close(n.done)
|
||||
|
||||
n.parser, _ = parsers.NewGraphiteParser("_", []string{}, nil)
|
||||
go n.receiver()
|
||||
in <- natsMsg(testMsgGraphite)
|
||||
time.Sleep(time.Millisecond)
|
||||
time.Sleep(time.Millisecond * 25)
|
||||
|
||||
acc := testutil.Accumulator{}
|
||||
n.Gather(&acc)
|
||||
|
||||
if a := len(acc.Metrics); a != 1 {
|
||||
t.Errorf("got %v, expected %v", a, 1)
|
||||
}
|
||||
acc.AssertContainsFields(t, "cpu_load_short_graphite",
|
||||
map[string]interface{}{"value": float64(23422)})
|
||||
}
|
||||
@@ -124,19 +104,17 @@ func TestRunParserAndGatherGraphite(t *testing.T) {
|
||||
// Test that the parser parses json format messages into metrics
|
||||
func TestRunParserAndGatherJSON(t *testing.T) {
|
||||
n, in := newTestNatsConsumer()
|
||||
acc := testutil.Accumulator{}
|
||||
n.acc = &acc
|
||||
defer close(n.done)
|
||||
|
||||
n.parser, _ = parsers.NewJSONParser("nats_json_test", []string{}, nil)
|
||||
go n.receiver()
|
||||
in <- natsMsg(testMsgJSON)
|
||||
time.Sleep(time.Millisecond)
|
||||
time.Sleep(time.Millisecond * 25)
|
||||
|
||||
acc := testutil.Accumulator{}
|
||||
n.Gather(&acc)
|
||||
|
||||
if a := len(acc.Metrics); a != 1 {
|
||||
t.Errorf("got %v, expected %v", a, 1)
|
||||
}
|
||||
acc.AssertContainsFields(t, "nats_json_test",
|
||||
map[string]interface{}{
|
||||
"a": float64(5),
|
||||
|
||||
@@ -213,7 +213,7 @@ func (s *Statsd) Gather(acc telegraf.Accumulator) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *Statsd) Start() error {
|
||||
func (s *Statsd) Start(_ telegraf.Accumulator) error {
|
||||
// Make data structures
|
||||
s.done = make(chan struct{})
|
||||
s.in = make(chan []byte, s.AllowedPendingMessages)
|
||||
|
||||
Reference in New Issue
Block a user