1
0
Fork 0

Adding upstream version 1.34.4.

Signed-off-by: Daniel Baumann <daniel@debian.org>
This commit is contained in:
Daniel Baumann 2025-05-24 07:26:29 +02:00
parent e393c3af3f
commit 4978089aab
Signed by: daniel
GPG key ID: FBB4F0E80A80222F
4963 changed files with 677545 additions and 0 deletions

View file

@ -0,0 +1,237 @@
# Apache Kafka Consumer Input Plugin
This service plugin consumes messages from [Kafka brokers][kafka] in one of the
supported [data formats][data_formats]. The plugin uses
[consumer groups][consumer_groups] when talking to the Kafka cluster so multiple
instances of Telegraf can consume messages from the same topic in parallel.
⭐ Telegraf v0.2.3
🏷️ messaging
💻 all
[kafka]: https://kafka.apache.org
[consumer_groups]: http://godoc.org/github.com/wvanbergen/kafka/consumergroup
[data_formats]: /docs/DATA_FORMATS_INPUT.md
## Service Input <!-- @/docs/includes/service_input.md -->
This plugin is a service input. Normal plugins gather metrics determined by the
interval setting. Service plugins start a service to listen and wait for
metrics or events to occur. Service plugins have two key differences from
normal plugins:
1. The global or plugin specific `interval` setting may not apply
2. The CLI options of `--test`, `--test-wait`, and `--once` may not produce
output for this plugin
## Global configuration options <!-- @/docs/includes/plugin_config.md -->
In addition to the plugin-specific configuration settings, plugins support
additional global and plugin configuration settings. These settings are used to
modify metrics, tags, and field or create aliases and configure ordering, etc.
See the [CONFIGURATION.md][CONFIGURATION.md] for more details.
[CONFIGURATION.md]: ../../../docs/CONFIGURATION.md#plugins
## Startup error behavior options <!-- @/docs/includes/startup_error_behavior.md -->
In addition to the plugin-specific and global configuration settings the plugin
supports options for specifying the behavior when experiencing startup errors
using the `startup_error_behavior` setting. Available values are:
- `error`: Telegraf with stop and exit in case of startup errors. This is the
default behavior.
- `ignore`: Telegraf will ignore startup errors for this plugin and disables it
but continues processing for all other plugins.
- `retry`: Telegraf will try to startup the plugin in every gather or write
cycle in case of startup errors. The plugin is disabled until
the startup succeeds.
- `probe`: Telegraf will probe the plugin's function (if possible) and disables the plugin
in case probing fails. If the plugin does not support probing, Telegraf will
behave as if `ignore` was set instead.
## Secret-store support
This plugin supports secrets from secret-stores for the `sasl_username`,
`sasl_password` and `sasl_access_token` option.
See the [secret-store documentation][SECRETSTORE] for more details on how
to use them.
[SECRETSTORE]: ../../../docs/CONFIGURATION.md#secret-store-secrets
## Configuration
```toml @sample.conf
# Read metrics from Kafka topics
[[inputs.kafka_consumer]]
## Kafka brokers.
brokers = ["localhost:9092"]
## Set the minimal supported Kafka version. Should be a string contains
## 4 digits in case if it is 0 version and 3 digits for versions starting
## from 1.0.0 separated by dot. This setting enables the use of new
## Kafka features and APIs. Must be 0.10.2.0(used as default) or greater.
## Please, check the list of supported versions at
## https://pkg.go.dev/github.com/Shopify/sarama#SupportedVersions
## ex: kafka_version = "2.6.0"
## ex: kafka_version = "0.10.2.0"
# kafka_version = "0.10.2.0"
## Topics to consume.
topics = ["telegraf"]
## Topic regular expressions to consume. Matches will be added to topics.
## Example: topic_regexps = [ "*test", "metric[0-9A-z]*" ]
# topic_regexps = [ ]
## When set this tag will be added to all metrics with the topic as the value.
# topic_tag = ""
## The list of Kafka message headers that should be pass as metric tags
## works only for Kafka version 0.11+, on lower versions the message headers
## are not available
# msg_headers_as_tags = []
## The name of kafka message header which value should override the metric name.
## In case when the same header specified in current option and in msg_headers_as_tags
## option, it will be excluded from the msg_headers_as_tags list.
# msg_header_as_metric_name = ""
## Set metric(s) timestamp using the given source.
## Available options are:
## metric -- do not modify the metric timestamp
## inner -- use the inner message timestamp (Kafka v0.10+)
## outer -- use the outer (compressed) block timestamp (Kafka v0.10+)
# timestamp_source = "metric"
## Optional Client id
# client_id = "Telegraf"
## Optional TLS Config
# enable_tls = false
# tls_ca = "/etc/telegraf/ca.pem"
# tls_cert = "/etc/telegraf/cert.pem"
# tls_key = "/etc/telegraf/key.pem"
## Use TLS but skip chain & host verification
# insecure_skip_verify = false
## Period between keep alive probes.
## Defaults to the OS configuration if not specified or zero.
# keep_alive_period = "15s"
## SASL authentication credentials. These settings should typically be used
## with TLS encryption enabled
# sasl_username = "kafka"
# sasl_password = "secret"
## Optional SASL:
## one of: OAUTHBEARER, PLAIN, SCRAM-SHA-256, SCRAM-SHA-512, GSSAPI
## (defaults to PLAIN)
# sasl_mechanism = ""
## used if sasl_mechanism is GSSAPI
# sasl_gssapi_service_name = ""
# ## One of: KRB5_USER_AUTH and KRB5_KEYTAB_AUTH
# sasl_gssapi_auth_type = "KRB5_USER_AUTH"
# sasl_gssapi_kerberos_config_path = "/"
# sasl_gssapi_realm = "realm"
# sasl_gssapi_key_tab_path = ""
# sasl_gssapi_disable_pafxfast = false
## used if sasl_mechanism is OAUTHBEARER
# sasl_access_token = ""
## SASL protocol version. When connecting to Azure EventHub set to 0.
# sasl_version = 1
# Disable Kafka metadata full fetch
# metadata_full = false
## Name of the consumer group.
# consumer_group = "telegraf_metrics_consumers"
## Compression codec represents the various compression codecs recognized by
## Kafka in messages.
## 0 : None
## 1 : Gzip
## 2 : Snappy
## 3 : LZ4
## 4 : ZSTD
# compression_codec = 0
## Initial offset position; one of "oldest" or "newest".
# offset = "oldest"
## Consumer group partition assignment strategy; one of "range", "roundrobin" or "sticky".
# balance_strategy = "range"
## Maximum number of retries for metadata operations including
## connecting. Sets Sarama library's Metadata.Retry.Max config value. If 0 or
## unset, use the Sarama default of 3,
# metadata_retry_max = 0
## Type of retry backoff. Valid options: "constant", "exponential"
# metadata_retry_type = "constant"
## Amount of time to wait before retrying. When metadata_retry_type is
## "constant", each retry is delayed this amount. When "exponential", the
## first retry is delayed this amount, and subsequent delays are doubled. If 0
## or unset, use the Sarama default of 250 ms
# metadata_retry_backoff = 0
## Maximum amount of time to wait before retrying when metadata_retry_type is
## "exponential". Ignored for other retry types. If 0, there is no backoff
## limit.
# metadata_retry_max_duration = 0
## When set to true, this turns each bootstrap broker address into a set of
## IPs, then does a reverse lookup on each one to get its canonical hostname.
## This list of hostnames then replaces the original address list.
## resolve_canonical_bootstrap_servers_only = false
## Maximum length of a message to consume, in bytes (default 0/unlimited);
## larger messages are dropped
max_message_len = 1000000
## Max undelivered messages
## This plugin uses tracking metrics, which ensure messages are read to
## outputs before acknowledging them to the original broker to ensure data
## is not lost. This option sets the maximum messages to read from the
## broker that have not been written by an output.
##
## This value needs to be picked with awareness of the agent's
## metric_batch_size value as well. Setting max undelivered messages too high
## can result in a constant stream of data batches to the output. While
## setting it too low may never flush the broker's messages.
# max_undelivered_messages = 1000
## Maximum amount of time the consumer should take to process messages. If
## the debug log prints messages from sarama about 'abandoning subscription
## to [topic] because consuming was taking too long', increase this value to
## longer than the time taken by the output plugin(s).
##
## Note that the effective timeout could be between 'max_processing_time' and
## '2 * max_processing_time'.
# max_processing_time = "100ms"
## The default number of message bytes to fetch from the broker in each
## request (default 1MB). This should be larger than the majority of
## your messages, or else the consumer will spend a lot of time
## negotiating sizes and not actually consuming. Similar to the JVM's
## `fetch.message.max.bytes`.
# consumer_fetch_default = "1MB"
## Data format to consume.
## Each data format has its own unique set of configuration options, read
## more about them here:
## https://github.com/influxdata/telegraf/blob/master/docs/DATA_FORMATS_INPUT.md
data_format = "influx"
```
## Metrics
The plugin accepts arbitrary input and parses it according to the `data_format`
setting. There is no predefined metric format.
## Example Output
There is no predefined metric format, so output depends on plugin input.

View file

@ -0,0 +1,591 @@
//go:generate ../../../tools/readme_config_includer/generator
package kafka_consumer
import (
"context"
_ "embed"
"errors"
"fmt"
"regexp"
"sort"
"strings"
"sync"
"time"
"github.com/IBM/sarama"
"github.com/influxdata/telegraf"
"github.com/influxdata/telegraf/config"
"github.com/influxdata/telegraf/internal"
"github.com/influxdata/telegraf/plugins/common/kafka"
"github.com/influxdata/telegraf/plugins/inputs"
)
//go:embed sample.conf
var sampleConfig string
var once sync.Once
const (
defaultMaxUndeliveredMessages = 1000
defaultMaxProcessingTime = config.Duration(100 * time.Millisecond)
defaultConsumerGroup = "telegraf_metrics_consumers"
reconnectDelay = 5 * time.Second
)
type KafkaConsumer struct {
Brokers []string `toml:"brokers"`
Version string `toml:"kafka_version"`
ConsumerGroup string `toml:"consumer_group"`
MaxMessageLen int `toml:"max_message_len"`
MaxUndeliveredMessages int `toml:"max_undelivered_messages"`
MaxProcessingTime config.Duration `toml:"max_processing_time"`
Offset string `toml:"offset"`
BalanceStrategy string `toml:"balance_strategy"`
Topics []string `toml:"topics"`
TopicRegexps []string `toml:"topic_regexps"`
TopicTag string `toml:"topic_tag"`
MsgHeadersAsTags []string `toml:"msg_headers_as_tags"`
MsgHeaderAsMetricName string `toml:"msg_header_as_metric_name"`
TimestampSource string `toml:"timestamp_source"`
ConsumerFetchDefault config.Size `toml:"consumer_fetch_default"`
ConnectionStrategy string `toml:"connection_strategy" deprecated:"1.33.0;1.40.0;use 'startup_error_behavior' instead"`
ResolveCanonicalBootstrapServersOnly bool `toml:"resolve_canonical_bootstrap_servers_only"`
Log telegraf.Logger `toml:"-"`
kafka.ReadConfig
consumerCreator consumerGroupCreator
consumer consumerGroup
config *sarama.Config
topicClient sarama.Client
regexps []regexp.Regexp
allWantedTopics []string
fingerprint string
parser telegraf.Parser
topicLock sync.Mutex
wg sync.WaitGroup
cancel context.CancelFunc
}
// consumerGroupHandler is a sarama.ConsumerGroupHandler implementation.
type consumerGroupHandler struct {
maxMessageLen int
topicTag string
msgHeadersToTags map[string]bool
msgHeaderToMetricName string
timestampSource string
acc telegraf.TrackingAccumulator
sem semaphore
parser telegraf.Parser
wg sync.WaitGroup
cancel context.CancelFunc
mu sync.Mutex
undelivered map[telegraf.TrackingID]message
log telegraf.Logger
}
// message is an aggregate type binding the Kafka message and the session so that offsets can be updated.
type message struct {
message *sarama.ConsumerMessage
session sarama.ConsumerGroupSession
}
type (
empty struct{}
semaphore chan empty
)
type consumerGroup interface {
Consume(ctx context.Context, topics []string, handler sarama.ConsumerGroupHandler) error
Errors() <-chan error
Close() error
}
type consumerGroupCreator interface {
create(brokers []string, group string, cfg *sarama.Config) (consumerGroup, error)
}
type saramaCreator struct{}
func (*saramaCreator) create(brokers []string, group string, cfg *sarama.Config) (consumerGroup, error) {
return sarama.NewConsumerGroup(brokers, group, cfg)
}
func (*KafkaConsumer) SampleConfig() string {
return sampleConfig
}
func (k *KafkaConsumer) Init() error {
kafka.SetLogger(k.Log.Level())
if k.MaxUndeliveredMessages == 0 {
k.MaxUndeliveredMessages = defaultMaxUndeliveredMessages
}
if time.Duration(k.MaxProcessingTime) == 0 {
k.MaxProcessingTime = defaultMaxProcessingTime
}
if k.ConsumerGroup == "" {
k.ConsumerGroup = defaultConsumerGroup
}
switch k.TimestampSource {
case "":
k.TimestampSource = "metric"
case "metric", "inner", "outer":
default:
return fmt.Errorf("invalid timestamp source %q", k.TimestampSource)
}
cfg := sarama.NewConfig()
// Kafka version 0.10.2.0 is required for consumer groups.
// Try to parse version from config. If can not, set default
cfg.Version = sarama.V0_10_2_0
if k.Version != "" {
version, err := sarama.ParseKafkaVersion(k.Version)
if err != nil {
return fmt.Errorf("invalid version: %w", err)
}
cfg.Version = version
}
if err := k.SetConfig(cfg, k.Log); err != nil {
return fmt.Errorf("setting config failed: %w", err)
}
switch strings.ToLower(k.Offset) {
case "oldest", "":
cfg.Consumer.Offsets.Initial = sarama.OffsetOldest
case "newest":
cfg.Consumer.Offsets.Initial = sarama.OffsetNewest
default:
return fmt.Errorf("invalid offset %q", k.Offset)
}
switch strings.ToLower(k.BalanceStrategy) {
case "range", "":
cfg.Consumer.Group.Rebalance.GroupStrategies = []sarama.BalanceStrategy{sarama.NewBalanceStrategyRange()}
case "roundrobin":
cfg.Consumer.Group.Rebalance.GroupStrategies = []sarama.BalanceStrategy{sarama.NewBalanceStrategyRoundRobin()}
case "sticky":
cfg.Consumer.Group.Rebalance.GroupStrategies = []sarama.BalanceStrategy{sarama.NewBalanceStrategySticky()}
default:
return fmt.Errorf("invalid balance strategy %q", k.BalanceStrategy)
}
if k.consumerCreator == nil {
k.consumerCreator = &saramaCreator{}
}
cfg.Net.ResolveCanonicalBootstrapServers = k.ResolveCanonicalBootstrapServersOnly
cfg.Consumer.MaxProcessingTime = time.Duration(k.MaxProcessingTime)
if k.ConsumerFetchDefault != 0 {
cfg.Consumer.Fetch.Default = int32(k.ConsumerFetchDefault)
}
switch strings.ToLower(k.ConnectionStrategy) {
default:
return fmt.Errorf("invalid connection strategy %q", k.ConnectionStrategy)
case "defer", "startup", "":
}
k.config = cfg
if len(k.TopicRegexps) == 0 {
k.allWantedTopics = k.Topics
} else {
if err := k.compileTopicRegexps(); err != nil {
return err
}
// We have regexps, so we're going to need a client to ask
// the broker for topics
client, err := sarama.NewClient(k.Brokers, k.config)
if err != nil {
return err
}
k.topicClient = client
}
return nil
}
func (k *KafkaConsumer) SetParser(parser telegraf.Parser) {
k.parser = parser
}
func (k *KafkaConsumer) Start(acc telegraf.Accumulator) error {
var err error
// If TopicRegexps is set, add matches to Topics
if len(k.TopicRegexps) > 0 {
if err := k.refreshTopics(); err != nil {
return err
}
}
ctx, cancel := context.WithCancel(context.Background())
k.cancel = cancel
if k.ConnectionStrategy != "defer" {
err = k.create()
if err != nil {
return &internal.StartupError{
Err: fmt.Errorf("create consumer: %w", err),
Retry: errors.Is(err, sarama.ErrOutOfBrokers),
}
}
k.startErrorAdder(acc)
}
// Start consumer goroutine
k.wg.Add(1)
go func() {
var err error
defer k.wg.Done()
if k.consumer == nil {
err = k.create()
if err != nil {
acc.AddError(fmt.Errorf("create consumer async: %w", err))
return
}
}
k.startErrorAdder(acc)
for ctx.Err() == nil {
handler := newConsumerGroupHandler(acc, k.MaxUndeliveredMessages, k.parser, k.Log)
handler.maxMessageLen = k.MaxMessageLen
handler.topicTag = k.TopicTag
handler.msgHeaderToMetricName = k.MsgHeaderAsMetricName
// if message headers list specified, put it as map to handler
msgHeadersMap := make(map[string]bool, len(k.MsgHeadersAsTags))
if len(k.MsgHeadersAsTags) > 0 {
for _, header := range k.MsgHeadersAsTags {
if k.MsgHeaderAsMetricName != header {
msgHeadersMap[header] = true
}
}
}
handler.msgHeadersToTags = msgHeadersMap
handler.timestampSource = k.TimestampSource
// We need to copy allWantedTopics; the Consume() is
// long-running and we can easily deadlock if our
// topic-update-checker fires.
topics := make([]string, len(k.allWantedTopics))
k.topicLock.Lock()
copy(topics, k.allWantedTopics)
k.topicLock.Unlock()
err := k.consumer.Consume(ctx, topics, handler)
if err != nil {
acc.AddError(fmt.Errorf("consume: %w", err))
internal.SleepContext(ctx, reconnectDelay) //nolint:errcheck // ignore returned error as we cannot do anything about it anyway
}
}
err = k.consumer.Close()
if err != nil {
acc.AddError(fmt.Errorf("close: %w", err))
}
}()
return nil
}
func (*KafkaConsumer) Gather(telegraf.Accumulator) error {
return nil
}
func (k *KafkaConsumer) Stop() {
// Lock so that a topic refresh cannot start while we are stopping.
k.topicLock.Lock()
if k.topicClient != nil {
k.topicClient.Close()
}
k.topicLock.Unlock()
k.cancel()
k.wg.Wait()
}
func (k *KafkaConsumer) compileTopicRegexps() error {
// While we can add new topics matching extant regexps, we can't
// update that list on the fly. We compile them once at startup.
// Changing them is a configuration change and requires a restart.
k.regexps = make([]regexp.Regexp, 0, len(k.TopicRegexps))
for _, r := range k.TopicRegexps {
re, err := regexp.Compile(r)
if err != nil {
return fmt.Errorf("regular expression %q did not compile: '%w", r, err)
}
k.regexps = append(k.regexps, *re)
}
return nil
}
func (k *KafkaConsumer) refreshTopics() error {
// We have instantiated a new generic Kafka client, so we can ask
// it for all the topics it knows about. Then we build
// regexps from our strings, loop over those, loop over the
// topics, and if we find a match, add that topic to
// out topic set, which then we turn back into a list at the end.
if len(k.regexps) == 0 {
return nil
}
allDiscoveredTopics, err := k.topicClient.Topics()
if err != nil {
return err
}
k.Log.Debugf("discovered topics: %v", allDiscoveredTopics)
extantTopicSet := make(map[string]bool, len(allDiscoveredTopics))
for _, t := range allDiscoveredTopics {
extantTopicSet[t] = true
}
// Even if a topic specified by a literal string (that is, k.Topics)
// does not appear in the topic list, we want to keep it around, in
// case it pops back up--it is not guaranteed to be matched by any
// of our regular expressions. Therefore, we pretend that it's in
// extantTopicSet, even if it isn't.
//
// Assuming that literally-specified topics are usually in the topics
// present on the broker, this should not need a resizing (although if
// you have many topics that you don't care about, it will be too big)
wantedTopicSet := make(map[string]bool, len(allDiscoveredTopics))
for _, t := range k.Topics {
// Get our pre-specified topics
k.Log.Debugf("adding literally-specified topic %s", t)
wantedTopicSet[t] = true
}
for _, t := range allDiscoveredTopics {
// Add topics that match regexps
for _, r := range k.regexps {
if r.MatchString(t) {
wantedTopicSet[t] = true
k.Log.Debugf("adding regexp-matched topic %q", t)
break
}
}
}
topicList := make([]string, 0, len(wantedTopicSet))
for t := range wantedTopicSet {
topicList = append(topicList, t)
}
sort.Strings(topicList)
fingerprint := strings.Join(topicList, ";")
if fingerprint != k.fingerprint {
k.Log.Infof("updating topics: replacing %q with %q", k.allWantedTopics, topicList)
}
k.topicLock.Lock()
k.fingerprint = fingerprint
k.allWantedTopics = topicList
k.topicLock.Unlock()
return nil
}
func (k *KafkaConsumer) create() error {
var err error
k.consumer, err = k.consumerCreator.create(
k.Brokers,
k.ConsumerGroup,
k.config,
)
return err
}
func (k *KafkaConsumer) startErrorAdder(acc telegraf.Accumulator) {
k.wg.Add(1)
go func() {
defer k.wg.Done()
for err := range k.consumer.Errors() {
acc.AddError(fmt.Errorf("channel: %w", err))
}
}()
}
func newConsumerGroupHandler(acc telegraf.Accumulator, maxUndelivered int, parser telegraf.Parser, log telegraf.Logger) *consumerGroupHandler {
handler := &consumerGroupHandler{
acc: acc.WithTracking(maxUndelivered),
sem: make(chan empty, maxUndelivered),
undelivered: make(map[telegraf.TrackingID]message, maxUndelivered),
parser: parser,
log: log,
}
return handler
}
// Setup is called once when a new session is opened. It setups up the handler and begins processing delivered messages.
func (h *consumerGroupHandler) Setup(sarama.ConsumerGroupSession) error {
h.undelivered = make(map[telegraf.TrackingID]message)
ctx, cancel := context.WithCancel(context.Background())
h.cancel = cancel
h.wg.Add(1)
go func() {
defer h.wg.Done()
h.run(ctx)
}()
return nil
}
// ConsumeClaim is called once each claim in a goroutine and must be thread-safe. Should run until the claim is closed.
func (h *consumerGroupHandler) ConsumeClaim(session sarama.ConsumerGroupSession, claim sarama.ConsumerGroupClaim) error {
ctx := session.Context()
for {
err := h.reserve(ctx)
if err != nil {
return err
}
select {
case <-ctx.Done():
return nil
case msg, ok := <-claim.Messages():
if !ok {
return nil
}
err := h.handle(session, msg)
if err != nil {
h.acc.AddError(err)
}
}
}
}
// Cleanup stops the internal goroutine and is called after all ConsumeClaim functions have completed.
func (h *consumerGroupHandler) Cleanup(sarama.ConsumerGroupSession) error {
h.cancel()
h.wg.Wait()
return nil
}
// Run processes any delivered metrics during the lifetime of the session.
func (h *consumerGroupHandler) run(ctx context.Context) {
for {
select {
case <-ctx.Done():
return
case track := <-h.acc.Delivered():
h.onDelivery(track)
}
}
}
func (h *consumerGroupHandler) onDelivery(track telegraf.DeliveryInfo) {
h.mu.Lock()
defer h.mu.Unlock()
msg, ok := h.undelivered[track.ID()]
if !ok {
h.log.Errorf("Could not mark message delivered: %d", track.ID())
return
}
if track.Delivered() {
msg.session.MarkMessage(msg.message, "")
}
delete(h.undelivered, track.ID())
<-h.sem
}
// reserve blocks until there is an available slot for a new message.
func (h *consumerGroupHandler) reserve(ctx context.Context) error {
select {
case <-ctx.Done():
return ctx.Err()
case h.sem <- empty{}:
return nil
}
}
func (h *consumerGroupHandler) release() {
<-h.sem
}
// handle processes a message and if successful saves it to be acknowledged after delivery.
func (h *consumerGroupHandler) handle(session sarama.ConsumerGroupSession, msg *sarama.ConsumerMessage) error {
if h.maxMessageLen != 0 && len(msg.Value) > h.maxMessageLen {
session.MarkMessage(msg, "")
h.release()
return fmt.Errorf("message exceeds max_message_len (actual %d, max %d)",
len(msg.Value), h.maxMessageLen)
}
metrics, err := h.parser.Parse(msg.Value)
if err != nil {
session.MarkMessage(msg, "")
h.release()
return err
}
if len(metrics) == 0 {
once.Do(func() {
h.log.Debug(internal.NoMetricsCreatedMsg)
})
}
// Check if any message header should override metric name or should be pass as tag
if len(h.msgHeadersToTags) > 0 || h.msgHeaderToMetricName != "" {
for _, header := range msg.Headers {
// convert to a string as the header and value are byte arrays.
headerKey := string(header.Key)
if _, exists := h.msgHeadersToTags[headerKey]; exists {
// If message header should be pass as tag then add it to the metrics
for _, metric := range metrics {
metric.AddTag(headerKey, string(header.Value))
}
} else {
if h.msgHeaderToMetricName == headerKey {
for _, metric := range metrics {
metric.SetName(string(header.Value))
}
}
}
}
}
// Add topic name as tag with topicTag name specified in the config
if len(h.topicTag) > 0 {
for _, metric := range metrics {
metric.AddTag(h.topicTag, msg.Topic)
}
}
// Do override the metric timestamp if required
switch h.timestampSource {
case "inner":
for _, metric := range metrics {
metric.SetTime(msg.Timestamp)
}
case "outer":
for _, metric := range metrics {
metric.SetTime(msg.BlockTimestamp)
}
}
h.mu.Lock()
id := h.acc.AddTrackingMetricGroup(metrics)
h.undelivered[id] = message{session: session, message: msg}
h.mu.Unlock()
return nil
}
func init() {
inputs.Add("kafka_consumer", func() telegraf.Input {
return &KafkaConsumer{}
})
}

View file

@ -0,0 +1,929 @@
package kafka_consumer
import (
"context"
"fmt"
"math"
"net"
"testing"
"time"
"github.com/IBM/sarama"
"github.com/stretchr/testify/require"
"github.com/testcontainers/testcontainers-go"
kafkacontainer "github.com/testcontainers/testcontainers-go/modules/kafka"
"github.com/influxdata/telegraf"
"github.com/influxdata/telegraf/config"
"github.com/influxdata/telegraf/internal"
"github.com/influxdata/telegraf/metric"
"github.com/influxdata/telegraf/models"
"github.com/influxdata/telegraf/plugins/common/kafka"
"github.com/influxdata/telegraf/plugins/common/tls"
"github.com/influxdata/telegraf/plugins/outputs"
outputs_kafka "github.com/influxdata/telegraf/plugins/outputs/kafka"
"github.com/influxdata/telegraf/plugins/parsers/influx"
"github.com/influxdata/telegraf/plugins/parsers/value"
serializers_influx "github.com/influxdata/telegraf/plugins/serializers/influx"
"github.com/influxdata/telegraf/testutil"
)
type fakeConsumerGroup struct {
brokers []string
group string
config *sarama.Config
handler sarama.ConsumerGroupHandler
errors chan error
}
func (g *fakeConsumerGroup) Consume(_ context.Context, _ []string, handler sarama.ConsumerGroupHandler) error {
g.handler = handler
return g.handler.Setup(nil)
}
func (g *fakeConsumerGroup) Errors() <-chan error {
return g.errors
}
func (g *fakeConsumerGroup) Close() error {
close(g.errors)
return nil
}
type fakeCreator struct {
consumerGroup *fakeConsumerGroup
}
func (c *fakeCreator) create(brokers []string, group string, cfg *sarama.Config) (consumerGroup, error) {
c.consumerGroup.brokers = brokers
c.consumerGroup.group = group
c.consumerGroup.config = cfg
return c.consumerGroup, nil
}
func TestInit(t *testing.T) {
tests := []struct {
name string
plugin *KafkaConsumer
initError bool
check func(t *testing.T, plugin *KafkaConsumer)
}{
{
name: "default config",
plugin: &KafkaConsumer{Log: testutil.Logger{}},
check: func(t *testing.T, plugin *KafkaConsumer) {
require.Equal(t, defaultConsumerGroup, plugin.ConsumerGroup)
require.Equal(t, defaultMaxUndeliveredMessages, plugin.MaxUndeliveredMessages)
require.Equal(t, "Telegraf", plugin.config.ClientID)
require.Equal(t, sarama.OffsetOldest, plugin.config.Consumer.Offsets.Initial)
require.Equal(t, 100*time.Millisecond, plugin.config.Consumer.MaxProcessingTime)
},
},
{
name: "parses valid version string",
plugin: &KafkaConsumer{
ReadConfig: kafka.ReadConfig{
Config: kafka.Config{
Version: "1.0.0",
},
},
Log: testutil.Logger{},
},
check: func(t *testing.T, plugin *KafkaConsumer) {
require.Equal(t, plugin.config.Version, sarama.V1_0_0_0)
},
},
{
name: "invalid version string",
plugin: &KafkaConsumer{
ReadConfig: kafka.ReadConfig{
Config: kafka.Config{
Version: "100",
},
},
Log: testutil.Logger{},
},
initError: true,
},
{
name: "custom client_id",
plugin: &KafkaConsumer{
ReadConfig: kafka.ReadConfig{
Config: kafka.Config{
ClientID: "custom",
},
},
Log: testutil.Logger{},
},
check: func(t *testing.T, plugin *KafkaConsumer) {
require.Equal(t, "custom", plugin.config.ClientID)
},
},
{
name: "custom offset",
plugin: &KafkaConsumer{
Offset: "newest",
Log: testutil.Logger{},
},
check: func(t *testing.T, plugin *KafkaConsumer) {
require.Equal(t, sarama.OffsetNewest, plugin.config.Consumer.Offsets.Initial)
},
},
{
name: "invalid offset",
plugin: &KafkaConsumer{
Offset: "middle",
Log: testutil.Logger{},
},
initError: true,
},
{
name: "default tls without tls config",
plugin: &KafkaConsumer{
Log: testutil.Logger{},
},
check: func(t *testing.T, plugin *KafkaConsumer) {
require.False(t, plugin.config.Net.TLS.Enable)
},
},
{
name: "enabled tls without tls config",
plugin: &KafkaConsumer{
ReadConfig: kafka.ReadConfig{
Config: kafka.Config{
EnableTLS: func(b bool) *bool { return &b }(true),
},
},
Log: testutil.Logger{},
},
check: func(t *testing.T, plugin *KafkaConsumer) {
require.True(t, plugin.config.Net.TLS.Enable)
},
},
{
name: "default tls with a tls config",
plugin: &KafkaConsumer{
ReadConfig: kafka.ReadConfig{
Config: kafka.Config{
ClientConfig: tls.ClientConfig{
InsecureSkipVerify: true,
},
},
},
Log: testutil.Logger{},
},
check: func(t *testing.T, plugin *KafkaConsumer) {
require.True(t, plugin.config.Net.TLS.Enable)
},
},
{
name: "Insecure tls",
plugin: &KafkaConsumer{
ReadConfig: kafka.ReadConfig{
Config: kafka.Config{
ClientConfig: tls.ClientConfig{
InsecureSkipVerify: true,
},
},
},
Log: testutil.Logger{},
},
check: func(t *testing.T, plugin *KafkaConsumer) {
require.True(t, plugin.config.Net.TLS.Enable)
},
},
{
name: "custom max_processing_time",
plugin: &KafkaConsumer{
MaxProcessingTime: config.Duration(1000 * time.Millisecond),
Log: testutil.Logger{},
},
check: func(t *testing.T, plugin *KafkaConsumer) {
require.Equal(t, 1000*time.Millisecond, plugin.config.Consumer.MaxProcessingTime)
},
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
cg := &fakeConsumerGroup{}
tt.plugin.consumerCreator = &fakeCreator{consumerGroup: cg}
err := tt.plugin.Init()
if tt.initError {
require.Error(t, err)
return
}
// No error path
require.NoError(t, err)
tt.check(t, tt.plugin)
})
}
}
func TestStartStop(t *testing.T) {
cg := &fakeConsumerGroup{errors: make(chan error)}
plugin := &KafkaConsumer{
consumerCreator: &fakeCreator{consumerGroup: cg},
Log: testutil.Logger{},
}
err := plugin.Init()
require.NoError(t, err)
var acc testutil.Accumulator
require.NoError(t, plugin.Start(&acc))
plugin.Stop()
}
type FakeConsumerGroupSession struct {
ctx context.Context
}
func (*FakeConsumerGroupSession) Claims() map[string][]int32 {
panic("not implemented")
}
func (*FakeConsumerGroupSession) MemberID() string {
panic("not implemented")
}
func (*FakeConsumerGroupSession) GenerationID() int32 {
panic("not implemented")
}
func (*FakeConsumerGroupSession) MarkOffset(string, int32, int64, string) {
panic("not implemented")
}
func (*FakeConsumerGroupSession) ResetOffset(string, int32, int64, string) {
panic("not implemented")
}
func (*FakeConsumerGroupSession) MarkMessage(*sarama.ConsumerMessage, string) {
}
func (s *FakeConsumerGroupSession) Context() context.Context {
return s.ctx
}
func (*FakeConsumerGroupSession) Commit() {
}
type FakeConsumerGroupClaim struct {
messages chan *sarama.ConsumerMessage
}
func (*FakeConsumerGroupClaim) Topic() string {
panic("not implemented")
}
func (*FakeConsumerGroupClaim) Partition() int32 {
panic("not implemented")
}
func (*FakeConsumerGroupClaim) InitialOffset() int64 {
panic("not implemented")
}
func (*FakeConsumerGroupClaim) HighWaterMarkOffset() int64 {
panic("not implemented")
}
func (c *FakeConsumerGroupClaim) Messages() <-chan *sarama.ConsumerMessage {
return c.messages
}
func TestConsumerGroupHandlerLifecycle(t *testing.T) {
acc := &testutil.Accumulator{}
parser := value.Parser{
MetricName: "cpu",
DataType: "int",
}
cg := newConsumerGroupHandler(acc, 1, &parser, testutil.Logger{})
ctx, cancel := context.WithCancel(t.Context())
defer cancel()
session := &FakeConsumerGroupSession{
ctx: ctx,
}
var claim FakeConsumerGroupClaim
var err error
err = cg.Setup(session)
require.NoError(t, err)
cancel()
// This produces a flappy testcase probably due to a race between context cancellation and consumption.
// Furthermore, it is not clear what the outcome of this test should be...
// err = cg.ConsumeClaim(session, &claim)
// require.NoError(t, err)
// So stick with the line below for now.
//nolint:errcheck // see above
cg.ConsumeClaim(session, &claim)
err = cg.Cleanup(session)
require.NoError(t, err)
}
func TestConsumerGroupHandlerConsumeClaim(t *testing.T) {
acc := &testutil.Accumulator{}
parser := value.Parser{
MetricName: "cpu",
DataType: "int",
}
require.NoError(t, parser.Init())
cg := newConsumerGroupHandler(acc, 1, &parser, testutil.Logger{})
ctx, cancel := context.WithCancel(t.Context())
defer cancel()
session := &FakeConsumerGroupSession{ctx: ctx}
claim := &FakeConsumerGroupClaim{
messages: make(chan *sarama.ConsumerMessage, 1),
}
err := cg.Setup(session)
require.NoError(t, err)
claim.messages <- &sarama.ConsumerMessage{
Topic: "telegraf",
Value: []byte("42"),
}
go func() {
err := cg.ConsumeClaim(session, claim)
if err == nil {
t.Error("An error was expected.")
return
}
if err.Error() != "context canceled" {
t.Errorf("Expected 'context canceled' error, got: %v", err)
return
}
}()
acc.Wait(1)
cancel()
err = cg.Cleanup(session)
require.NoError(t, err)
expected := []telegraf.Metric{
testutil.MustMetric(
"cpu",
map[string]string{},
map[string]interface{}{
"value": 42,
},
time.Now(),
),
}
testutil.RequireMetricsEqual(t, expected, acc.GetTelegrafMetrics(), testutil.IgnoreTime())
}
func TestConsumerGroupHandlerHandle(t *testing.T) {
tests := []struct {
name string
maxMessageLen int
topicTag string
msg *sarama.ConsumerMessage
expected []telegraf.Metric
expectedHandleError string
}{
{
name: "happy path",
msg: &sarama.ConsumerMessage{
Topic: "telegraf",
Value: []byte("42"),
},
expected: []telegraf.Metric{
testutil.MustMetric(
"cpu",
map[string]string{},
map[string]interface{}{
"value": 42,
},
time.Now(),
),
},
},
{
name: "message to long",
maxMessageLen: 4,
msg: &sarama.ConsumerMessage{
Topic: "telegraf",
Value: []byte("12345"),
},
expectedHandleError: "message exceeds max_message_len (actual 5, max 4)",
},
{
name: "parse error",
msg: &sarama.ConsumerMessage{
Topic: "telegraf",
Value: []byte("not an integer"),
},
expectedHandleError: "strconv.Atoi: parsing \"integer\": invalid syntax",
},
{
name: "add topic tag",
topicTag: "topic",
msg: &sarama.ConsumerMessage{
Topic: "telegraf",
Value: []byte("42"),
},
expected: []telegraf.Metric{
testutil.MustMetric(
"cpu",
map[string]string{
"topic": "telegraf",
},
map[string]interface{}{
"value": 42,
},
time.Now(),
),
},
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
acc := &testutil.Accumulator{}
parser := value.Parser{
MetricName: "cpu",
DataType: "int",
}
require.NoError(t, parser.Init())
cg := newConsumerGroupHandler(acc, 1, &parser, testutil.Logger{})
cg.maxMessageLen = tt.maxMessageLen
cg.topicTag = tt.topicTag
session := &FakeConsumerGroupSession{ctx: t.Context()}
require.NoError(t, cg.reserve(t.Context()))
err := cg.handle(session, tt.msg)
if tt.expectedHandleError != "" {
require.Error(t, err)
require.EqualValues(t, tt.expectedHandleError, err.Error())
} else {
require.NoError(t, err)
}
testutil.RequireMetricsEqual(t, tt.expected, acc.GetTelegrafMetrics(), testutil.IgnoreTime())
})
}
}
func TestExponentialBackoff(t *testing.T) {
var err error
backoff := 10 * time.Millisecond
limit := 3
// get an unused port by listening on next available port, then closing it
listener, err := net.Listen("tcp", "127.0.0.1:0")
require.NoError(t, err)
port := listener.Addr().(*net.TCPAddr).Port
require.NoError(t, listener.Close())
// try to connect to kafka on that unused port
brokers := []string{
fmt.Sprintf("localhost:%d", port),
}
input := KafkaConsumer{
Brokers: brokers,
Log: testutil.Logger{},
Topics: []string{"topic"},
MaxUndeliveredMessages: 1,
ReadConfig: kafka.ReadConfig{
Config: kafka.Config{
MetadataRetryMax: limit,
MetadataRetryBackoff: config.Duration(backoff),
MetadataRetryType: "exponential",
},
},
}
parser := &influx.Parser{}
require.NoError(t, parser.Init())
input.SetParser(parser)
// time how long initialization (connection) takes
start := time.Now()
require.NoError(t, input.Init())
acc := testutil.Accumulator{}
require.Error(t, input.Start(&acc))
elapsed := time.Since(start)
t.Logf("elapsed %d", elapsed)
var expectedRetryDuration time.Duration
for i := 0; i < limit; i++ {
expectedRetryDuration += backoff * time.Duration(math.Pow(2, float64(i)))
}
t.Logf("expected > %d", expectedRetryDuration)
// Other than the expected retry delay, initializing and starting the
// plugin, including initializing a sarama consumer takes some time.
//
// It would be nice to check that the actual time is within an expected
// range, but we don't know how long the non-retry time should be.
//
// For now, just check that elapsed time isn't shorter than we expect the
// retry delays to be
require.GreaterOrEqual(t, elapsed, expectedRetryDuration)
input.Stop()
}
func TestExponentialBackoffDefault(t *testing.T) {
input := KafkaConsumer{
Brokers: []string{"broker"},
Log: testutil.Logger{},
Topics: []string{"topic"},
MaxUndeliveredMessages: 1,
ReadConfig: kafka.ReadConfig{
Config: kafka.Config{
MetadataRetryType: "exponential",
},
},
}
parser := &influx.Parser{}
require.NoError(t, parser.Init())
input.SetParser(parser)
require.NoError(t, input.Init())
// We don't need to start the plugin here since we're only testing
// initialization
// if input.MetadataRetryBackoff isn't set, it should be 250 ms
require.Equal(t, input.MetadataRetryBackoff, config.Duration(250*time.Millisecond))
}
func TestKafkaRoundTripIntegration(t *testing.T) {
if testing.Short() {
t.Skip("Skipping integration test in short mode")
}
var tests = []struct {
name string
connectionStrategy string
topics []string
topicRegexps []string
topicRefreshInterval config.Duration
}{
{"connection strategy startup", "startup", []string{"Test"}, nil, config.Duration(0)},
{"connection strategy defer", "defer", []string{"Test"}, nil, config.Duration(0)},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
kafkaContainer, err := kafkacontainer.Run(t.Context(), "confluentinc/confluent-local:7.5.0")
require.NoError(t, err)
defer kafkaContainer.Terminate(t.Context()) //nolint:errcheck // ignored
brokers, err := kafkaContainer.Brokers(t.Context())
require.NoError(t, err)
// Make kafka output
t.Logf("rt: starting output plugin")
creator := outputs.Outputs["kafka"]
output, ok := creator().(*outputs_kafka.Kafka)
require.True(t, ok)
s := &serializers_influx.Serializer{}
require.NoError(t, s.Init())
output.SetSerializer(s)
output.Brokers = brokers
output.Topic = "Test"
output.Log = testutil.Logger{}
require.NoError(t, output.Init())
require.NoError(t, output.Connect())
// Make kafka input
t.Logf("rt: starting input plugin")
input := KafkaConsumer{
Brokers: brokers,
Log: testutil.Logger{},
Topics: tt.topics,
TopicRegexps: tt.topicRegexps,
MaxUndeliveredMessages: 1,
ConnectionStrategy: tt.connectionStrategy,
}
parser := &influx.Parser{}
require.NoError(t, parser.Init())
input.SetParser(parser)
require.NoError(t, input.Init())
acc := testutil.Accumulator{}
require.NoError(t, input.Start(&acc))
// Shove some metrics through
expected := testutil.MockMetrics()
t.Logf("rt: writing")
require.NoError(t, output.Write(expected))
// Check that they were received
t.Logf("rt: expecting")
acc.Wait(len(expected))
testutil.RequireMetricsEqual(t, expected, acc.GetTelegrafMetrics())
t.Logf("rt: shutdown")
require.NoError(t, output.Close())
input.Stop()
t.Logf("rt: done")
})
}
}
func TestKafkaTimestampSourceIntegration(t *testing.T) {
if testing.Short() {
t.Skip("Skipping integration test in short mode")
}
metrics := []telegraf.Metric{
metric.New(
"test",
map[string]string{},
map[string]interface{}{"value": 42},
time.Unix(1704067200, 0),
),
}
for _, source := range []string{"metric", "inner", "outer"} {
t.Run(source, func(t *testing.T) {
kafkaContainer, err := kafkacontainer.Run(t.Context(), "confluentinc/confluent-local:7.5.0")
require.NoError(t, err)
defer kafkaContainer.Terminate(t.Context()) //nolint:errcheck // ignored
brokers, err := kafkaContainer.Brokers(t.Context())
require.NoError(t, err)
// Make kafka output
creator := outputs.Outputs["kafka"]
output, ok := creator().(*outputs_kafka.Kafka)
require.True(t, ok)
s := &serializers_influx.Serializer{}
require.NoError(t, s.Init())
output.SetSerializer(s)
output.Brokers = brokers
output.Topic = "Test"
output.Log = &testutil.Logger{}
require.NoError(t, output.Init())
require.NoError(t, output.Connect())
defer output.Close()
// Make kafka input
input := KafkaConsumer{
Brokers: brokers,
Log: testutil.Logger{},
Topics: []string{"Test"},
MaxUndeliveredMessages: 1,
}
parser := &influx.Parser{}
require.NoError(t, parser.Init())
input.SetParser(parser)
require.NoError(t, input.Init())
var acc testutil.Accumulator
require.NoError(t, input.Start(&acc))
defer input.Stop()
// Send the metrics and check that we got it back
sendTimestamp := time.Now().Unix()
require.NoError(t, output.Write(metrics))
require.Eventually(t, func() bool { return acc.NMetrics() > 0 }, 5*time.Second, 100*time.Millisecond)
actual := acc.GetTelegrafMetrics()
testutil.RequireMetricsEqual(t, metrics, actual, testutil.IgnoreTime())
// Check the timestamp
m := actual[0]
switch source {
case "metric":
require.EqualValues(t, 1704067200, m.Time().Unix())
case "inner", "outer":
require.GreaterOrEqual(t, sendTimestamp, m.Time().Unix())
}
})
}
}
func TestStartupErrorBehaviorErrorIntegration(t *testing.T) {
if testing.Short() {
t.Skip("Skipping integration test in short mode")
}
// Startup the container
container, err := kafkacontainer.Run(t.Context(), "confluentinc/confluent-local:7.5.0")
require.NoError(t, err)
defer container.Terminate(t.Context()) //nolint:errcheck // ignored
brokers, err := container.Brokers(t.Context())
require.NoError(t, err)
// Pause the container for simulating connectivity issues
containerID := container.GetContainerID()
provider, err := testcontainers.NewDockerProvider()
require.NoError(t, err)
require.NoError(t, provider.Client().ContainerPause(t.Context(), containerID))
//nolint:errcheck // Ignore the returned error as we cannot do anything about it anyway
defer provider.Client().ContainerUnpause(t.Context(), containerID)
// Setup the plugin and connect to the broker
plugin := &KafkaConsumer{
Brokers: brokers,
Log: testutil.Logger{},
Topics: []string{"test"},
MaxUndeliveredMessages: 1,
}
parser := &influx.Parser{}
require.NoError(t, parser.Init())
plugin.SetParser(parser)
// Create a model to be able to use the startup retry strategy
model := models.NewRunningInput(
plugin,
&models.InputConfig{
Name: "kafka_consumer",
Alias: "error-test",
},
)
model.StartupErrors.Set(0)
require.NoError(t, model.Init())
// Speed up test
plugin.config.Net.DialTimeout = 100 * time.Millisecond
plugin.config.Net.WriteTimeout = 100 * time.Millisecond
plugin.config.Net.ReadTimeout = 100 * time.Millisecond
// Starting the plugin will fail with an error because the container is paused.
var acc testutil.Accumulator
require.ErrorContains(t, model.Start(&acc), "client has run out of available brokers to talk to")
}
func TestStartupErrorBehaviorIgnoreIntegration(t *testing.T) {
if testing.Short() {
t.Skip("Skipping integration test in short mode")
}
// Startup the container
container, err := kafkacontainer.Run(t.Context(), "confluentinc/confluent-local:7.5.0")
require.NoError(t, err)
defer container.Terminate(t.Context()) //nolint:errcheck // ignored
brokers, err := container.Brokers(t.Context())
require.NoError(t, err)
// Pause the container for simulating connectivity issues
containerID := container.GetContainerID()
provider, err := testcontainers.NewDockerProvider()
require.NoError(t, err)
require.NoError(t, provider.Client().ContainerPause(t.Context(), containerID))
//nolint:errcheck // Ignore the returned error as we cannot do anything about it anyway
defer provider.Client().ContainerUnpause(t.Context(), containerID)
// Setup the plugin and connect to the broker
plugin := &KafkaConsumer{
Brokers: brokers,
Log: testutil.Logger{},
Topics: []string{"test"},
MaxUndeliveredMessages: 1,
}
parser := &influx.Parser{}
require.NoError(t, parser.Init())
plugin.SetParser(parser)
// Create a model to be able to use the startup retry strategy
model := models.NewRunningInput(
plugin,
&models.InputConfig{
Name: "kafka_consumer",
Alias: "ignore-test",
StartupErrorBehavior: "ignore",
},
)
model.StartupErrors.Set(0)
require.NoError(t, model.Init())
// Speed up test
plugin.config.Net.DialTimeout = 100 * time.Millisecond
plugin.config.Net.WriteTimeout = 100 * time.Millisecond
plugin.config.Net.ReadTimeout = 100 * time.Millisecond
// Starting the plugin will fail because the container is paused.
// The model code should convert it to a fatal error for the agent to remove
// the plugin.
var acc testutil.Accumulator
err = model.Start(&acc)
require.ErrorContains(t, err, "client has run out of available brokers to talk to")
var fatalErr *internal.FatalError
require.ErrorAs(t, err, &fatalErr)
}
func TestStartupErrorBehaviorRetryIntegration(t *testing.T) {
if testing.Short() {
t.Skip("Skipping integration test in short mode")
}
// Startup the container
container, err := kafkacontainer.Run(t.Context(), "confluentinc/confluent-local:7.5.0")
require.NoError(t, err)
defer container.Terminate(t.Context()) //nolint:errcheck // ignored
brokers, err := container.Brokers(t.Context())
require.NoError(t, err)
// Pause the container for simulating connectivity issues
containerID := container.GetContainerID()
provider, err := testcontainers.NewDockerProvider()
require.NoError(t, err)
require.NoError(t, provider.Client().ContainerPause(t.Context(), containerID))
//nolint:errcheck // Ignore the returned error as we cannot do anything about it anyway
defer provider.Client().ContainerUnpause(t.Context(), containerID)
// Setup the plugin and connect to the broker
plugin := &KafkaConsumer{
Brokers: brokers,
Log: testutil.Logger{},
Topics: []string{"test"},
MaxUndeliveredMessages: 1,
}
parser := &influx.Parser{}
require.NoError(t, parser.Init())
plugin.SetParser(parser)
// Create a model to be able to use the startup retry strategy
model := models.NewRunningInput(
plugin,
&models.InputConfig{
Name: "kafka_consumer",
Alias: "retry-test",
StartupErrorBehavior: "retry",
},
)
model.StartupErrors.Set(0)
require.NoError(t, model.Init())
// Speed up test
plugin.config.Net.DialTimeout = 100 * time.Millisecond
plugin.config.Net.WriteTimeout = 100 * time.Millisecond
plugin.config.Net.ReadTimeout = 100 * time.Millisecond
// Starting the plugin will not fail but should retry to connect in every gather cycle
var acc testutil.Accumulator
require.NoError(t, model.Start(&acc))
require.EqualValues(t, 1, model.StartupErrors.Get())
// There should be no metrics as the plugin is not fully started up yet
require.Empty(t, acc.GetTelegrafMetrics())
require.ErrorIs(t, model.Gather(&acc), internal.ErrNotConnected)
require.Equal(t, int64(2), model.StartupErrors.Get())
// Unpause the container, now writes should succeed
require.NoError(t, provider.Client().ContainerUnpause(t.Context(), containerID))
require.NoError(t, model.Gather(&acc))
defer model.Stop()
require.Equal(t, int64(2), model.StartupErrors.Get())
// Setup a writer
creator := outputs.Outputs["kafka"]
output, ok := creator().(*outputs_kafka.Kafka)
require.True(t, ok)
s := &serializers_influx.Serializer{}
require.NoError(t, s.Init())
output.SetSerializer(s)
output.Brokers = brokers
output.Topic = "test"
output.Log = &testutil.Logger{}
require.NoError(t, output.Init())
require.NoError(t, output.Connect())
defer output.Close()
// Send some data to the broker so we have something to receive
metrics := []telegraf.Metric{
metric.New(
"test",
map[string]string{},
map[string]interface{}{"value": 42},
time.Unix(1704067200, 0),
),
}
require.NoError(t, output.Write(metrics))
// Verify that the metrics were actually written
require.Eventually(t, func() bool {
return acc.NMetrics() >= 1
}, 3*time.Second, 100*time.Millisecond)
testutil.RequireMetricsEqual(t, metrics, acc.GetTelegrafMetrics())
}

View file

@ -0,0 +1,163 @@
# Read metrics from Kafka topics
[[inputs.kafka_consumer]]
## Kafka brokers.
brokers = ["localhost:9092"]
## Set the minimal supported Kafka version. Should be a string contains
## 4 digits in case if it is 0 version and 3 digits for versions starting
## from 1.0.0 separated by dot. This setting enables the use of new
## Kafka features and APIs. Must be 0.10.2.0(used as default) or greater.
## Please, check the list of supported versions at
## https://pkg.go.dev/github.com/Shopify/sarama#SupportedVersions
## ex: kafka_version = "2.6.0"
## ex: kafka_version = "0.10.2.0"
# kafka_version = "0.10.2.0"
## Topics to consume.
topics = ["telegraf"]
## Topic regular expressions to consume. Matches will be added to topics.
## Example: topic_regexps = [ "*test", "metric[0-9A-z]*" ]
# topic_regexps = [ ]
## When set this tag will be added to all metrics with the topic as the value.
# topic_tag = ""
## The list of Kafka message headers that should be pass as metric tags
## works only for Kafka version 0.11+, on lower versions the message headers
## are not available
# msg_headers_as_tags = []
## The name of kafka message header which value should override the metric name.
## In case when the same header specified in current option and in msg_headers_as_tags
## option, it will be excluded from the msg_headers_as_tags list.
# msg_header_as_metric_name = ""
## Set metric(s) timestamp using the given source.
## Available options are:
## metric -- do not modify the metric timestamp
## inner -- use the inner message timestamp (Kafka v0.10+)
## outer -- use the outer (compressed) block timestamp (Kafka v0.10+)
# timestamp_source = "metric"
## Optional Client id
# client_id = "Telegraf"
## Optional TLS Config
# enable_tls = false
# tls_ca = "/etc/telegraf/ca.pem"
# tls_cert = "/etc/telegraf/cert.pem"
# tls_key = "/etc/telegraf/key.pem"
## Use TLS but skip chain & host verification
# insecure_skip_verify = false
## Period between keep alive probes.
## Defaults to the OS configuration if not specified or zero.
# keep_alive_period = "15s"
## SASL authentication credentials. These settings should typically be used
## with TLS encryption enabled
# sasl_username = "kafka"
# sasl_password = "secret"
## Optional SASL:
## one of: OAUTHBEARER, PLAIN, SCRAM-SHA-256, SCRAM-SHA-512, GSSAPI
## (defaults to PLAIN)
# sasl_mechanism = ""
## used if sasl_mechanism is GSSAPI
# sasl_gssapi_service_name = ""
# ## One of: KRB5_USER_AUTH and KRB5_KEYTAB_AUTH
# sasl_gssapi_auth_type = "KRB5_USER_AUTH"
# sasl_gssapi_kerberos_config_path = "/"
# sasl_gssapi_realm = "realm"
# sasl_gssapi_key_tab_path = ""
# sasl_gssapi_disable_pafxfast = false
## used if sasl_mechanism is OAUTHBEARER
# sasl_access_token = ""
## SASL protocol version. When connecting to Azure EventHub set to 0.
# sasl_version = 1
# Disable Kafka metadata full fetch
# metadata_full = false
## Name of the consumer group.
# consumer_group = "telegraf_metrics_consumers"
## Compression codec represents the various compression codecs recognized by
## Kafka in messages.
## 0 : None
## 1 : Gzip
## 2 : Snappy
## 3 : LZ4
## 4 : ZSTD
# compression_codec = 0
## Initial offset position; one of "oldest" or "newest".
# offset = "oldest"
## Consumer group partition assignment strategy; one of "range", "roundrobin" or "sticky".
# balance_strategy = "range"
## Maximum number of retries for metadata operations including
## connecting. Sets Sarama library's Metadata.Retry.Max config value. If 0 or
## unset, use the Sarama default of 3,
# metadata_retry_max = 0
## Type of retry backoff. Valid options: "constant", "exponential"
# metadata_retry_type = "constant"
## Amount of time to wait before retrying. When metadata_retry_type is
## "constant", each retry is delayed this amount. When "exponential", the
## first retry is delayed this amount, and subsequent delays are doubled. If 0
## or unset, use the Sarama default of 250 ms
# metadata_retry_backoff = 0
## Maximum amount of time to wait before retrying when metadata_retry_type is
## "exponential". Ignored for other retry types. If 0, there is no backoff
## limit.
# metadata_retry_max_duration = 0
## When set to true, this turns each bootstrap broker address into a set of
## IPs, then does a reverse lookup on each one to get its canonical hostname.
## This list of hostnames then replaces the original address list.
## resolve_canonical_bootstrap_servers_only = false
## Maximum length of a message to consume, in bytes (default 0/unlimited);
## larger messages are dropped
max_message_len = 1000000
## Max undelivered messages
## This plugin uses tracking metrics, which ensure messages are read to
## outputs before acknowledging them to the original broker to ensure data
## is not lost. This option sets the maximum messages to read from the
## broker that have not been written by an output.
##
## This value needs to be picked with awareness of the agent's
## metric_batch_size value as well. Setting max undelivered messages too high
## can result in a constant stream of data batches to the output. While
## setting it too low may never flush the broker's messages.
# max_undelivered_messages = 1000
## Maximum amount of time the consumer should take to process messages. If
## the debug log prints messages from sarama about 'abandoning subscription
## to [topic] because consuming was taking too long', increase this value to
## longer than the time taken by the output plugin(s).
##
## Note that the effective timeout could be between 'max_processing_time' and
## '2 * max_processing_time'.
# max_processing_time = "100ms"
## The default number of message bytes to fetch from the broker in each
## request (default 1MB). This should be larger than the majority of
## your messages, or else the consumer will spend a lot of time
## negotiating sizes and not actually consuming. Similar to the JVM's
## `fetch.message.max.bytes`.
# consumer_fetch_default = "1MB"
## Data format to consume.
## Each data format has its own unique set of configuration options, read
## more about them here:
## https://github.com/influxdata/telegraf/blob/master/docs/DATA_FORMATS_INPUT.md
data_format = "influx"