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,257 @@
# Kafka Output Plugin
This plugin writes metrics to a [Kafka Broker][kafka] acting a Kafka Producer.
⭐ Telegraf v0.1.7
🏷️ messaging
💻 all
[kafka]: http://kafka.apache.org
## 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
# Configuration for the Kafka server to send metrics to
[[outputs.kafka]]
## URLs of kafka brokers
## The brokers listed here are used to connect to collect metadata about a
## cluster. However, once the initial metadata collect is completed, telegraf
## will communicate solely with the kafka leader and not all defined brokers.
brokers = ["localhost:9092"]
## Kafka topic for producer messages
topic = "telegraf"
## The value of this tag will be used as the topic. If not set the 'topic'
## option is used.
# topic_tag = ""
## If true, the 'topic_tag' will be removed from to the metric.
# exclude_topic_tag = false
## Optional Client id
# client_id = "Telegraf"
## Set the minimal supported Kafka version. Setting this enables the use of new
## Kafka features and APIs. Of particular interested, lz4 compression
## requires at least version 0.10.0.0.
## ex: version = "1.1.0"
# version = ""
## The routing tag specifies a tagkey on the metric whose value is used as
## the message key. The message key is used to determine which partition to
## send the message to. This tag is preferred over the routing_key option.
routing_tag = "host"
## The routing key is set as the message key and used to determine which
## partition to send the message to. This value is only used when no
## routing_tag is set or as a fallback when the tag specified in routing tag
## is not found.
##
## If set to "random", a random value will be generated for each message.
##
## When unset, no message key is added and each message is routed to a random
## partition.
##
## ex: routing_key = "random"
## routing_key = "telegraf"
# routing_key = ""
## 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
## Idempotent Writes
## If enabled, exactly one copy of each message is written.
# idempotent_writes = false
## RequiredAcks is used in Produce Requests to tell the broker how many
## replica acknowledgements it must see before responding
## 0 : the producer never waits for an acknowledgement from the broker.
## This option provides the lowest latency but the weakest durability
## guarantees (some data will be lost when a server fails).
## 1 : the producer gets an acknowledgement after the leader replica has
## received the data. This option provides better durability as the
## client waits until the server acknowledges the request as successful
## (only messages that were written to the now-dead leader but not yet
## replicated will be lost).
## -1: the producer gets an acknowledgement after all in-sync replicas have
## received the data. This option provides the best durability, we
## guarantee that no messages will be lost as long as at least one in
## sync replica remains.
# required_acks = -1
## The maximum number of times to retry sending a metric before failing
## until the next flush.
# max_retry = 3
## The maximum permitted size of a message. Should be set equal to or
## smaller than the broker's 'message.max.bytes'.
# max_message_bytes = 1000000
## Producer timestamp
## This option sets the timestamp of the kafka producer message, choose from:
## * metric: Uses the metric's timestamp
## * now: Uses the time of write
# producer_timestamp = metric
## Add metric name as specified kafka header if not empty
# metric_name_header = ""
## 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"
## Optional SOCKS5 proxy to use when connecting to brokers
# socks5_enabled = true
# socks5_address = "127.0.0.1:1080"
# socks5_username = "alice"
# socks5_password = "pass123"
## Optional SASL Config
# 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
## Access token used if sasl_mechanism is OAUTHBEARER
# sasl_access_token = ""
## Arbitrary key value string pairs to pass as a TOML table. For example:
# {logicalCluster = "cluster-042", poolId = "pool-027"}
# sasl_extensions = {}
## SASL protocol version. When connecting to Azure EventHub set to 0.
# sasl_version = 1
# Disable Kafka metadata full fetch
# metadata_full = false
## 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
## Data format to output.
## 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_OUTPUT.md
# data_format = "influx"
## NOTE: Due to the way TOML is parsed, tables must be at the END of the
## plugin definition, otherwise additional config options are read as part of
## the table
## Optional topic suffix configuration.
## If the section is omitted, no suffix is used.
## Following topic suffix methods are supported:
## measurement - suffix equals to separator + measurement's name
## tags - suffix equals to separator + specified tags' values
## interleaved with separator
## Suffix equals to "_" + measurement name
# [outputs.kafka.topic_suffix]
# method = "measurement"
# separator = "_"
## Suffix equals to "__" + measurement's "foo" tag value.
## If there's no such a tag, suffix equals to an empty string
# [outputs.kafka.topic_suffix]
# method = "tags"
# keys = ["foo"]
# separator = "__"
## Suffix equals to "_" + measurement's "foo" and "bar"
## tag values, separated by "_". If there is no such tags,
## their values treated as empty strings.
# [outputs.kafka.topic_suffix]
# method = "tags"
# keys = ["foo", "bar"]
# separator = "_"
```
### `max_retry`
This option controls the number of retries before a failure notification is
displayed for each message when no acknowledgement is received from the
broker. When the setting is greater than `0`, message latency can be reduced,
duplicate messages can occur in cases of transient errors, and broker loads can
increase during downtime.
The option is similar to the
[retries](https://kafka.apache.org/documentation/#producerconfigs) Producer
option in the Java Kafka Producer.

View file

@ -0,0 +1,273 @@
//go:generate ../../../tools/readme_config_includer/generator
package kafka
import (
_ "embed"
"errors"
"fmt"
"strings"
"time"
"github.com/IBM/sarama"
"github.com/gofrs/uuid/v5"
"github.com/influxdata/telegraf"
"github.com/influxdata/telegraf/internal"
"github.com/influxdata/telegraf/plugins/common/kafka"
"github.com/influxdata/telegraf/plugins/common/proxy"
"github.com/influxdata/telegraf/plugins/outputs"
)
//go:embed sample.conf
var sampleConfig string
var ValidTopicSuffixMethods = []string{
"",
"measurement",
"tags",
}
var zeroTime = time.Unix(0, 0)
type Kafka struct {
Brokers []string `toml:"brokers"`
Topic string `toml:"topic"`
TopicTag string `toml:"topic_tag"`
ExcludeTopicTag bool `toml:"exclude_topic_tag"`
TopicSuffix TopicSuffix `toml:"topic_suffix"`
RoutingTag string `toml:"routing_tag"`
RoutingKey string `toml:"routing_key"`
ProducerTimestamp string `toml:"producer_timestamp"`
MetricNameHeader string `toml:"metric_name_header"`
Log telegraf.Logger `toml:"-"`
proxy.Socks5ProxyConfig
kafka.WriteConfig
// Legacy TLS config options
// TLS client certificate
Certificate string
// TLS client key
Key string
// TLS certificate authority
CA string
saramaConfig *sarama.Config
producerFunc func(addrs []string, config *sarama.Config) (sarama.SyncProducer, error)
producer sarama.SyncProducer
serializer telegraf.Serializer
}
type TopicSuffix struct {
Method string `toml:"method"`
Keys []string `toml:"keys"`
Separator string `toml:"separator"`
}
func ValidateTopicSuffixMethod(method string) error {
for _, validMethod := range ValidTopicSuffixMethods {
if method == validMethod {
return nil
}
}
return fmt.Errorf("unknown topic suffix method provided: %s", method)
}
func (*Kafka) SampleConfig() string {
return sampleConfig
}
func (k *Kafka) GetTopicName(metric telegraf.Metric) (telegraf.Metric, string) {
topic := k.Topic
if k.TopicTag != "" {
if t, ok := metric.GetTag(k.TopicTag); ok {
topic = t
// If excluding the topic tag, a copy is required to avoid modifying
// the metric buffer.
if k.ExcludeTopicTag {
metric = metric.Copy()
metric.Accept()
metric.RemoveTag(k.TopicTag)
}
}
}
var topicName string
switch k.TopicSuffix.Method {
case "measurement":
topicName = topic + k.TopicSuffix.Separator + metric.Name()
case "tags":
var topicNameComponents []string
topicNameComponents = append(topicNameComponents, topic)
for _, tag := range k.TopicSuffix.Keys {
tagValue := metric.Tags()[tag]
if tagValue != "" {
topicNameComponents = append(topicNameComponents, tagValue)
}
}
topicName = strings.Join(topicNameComponents, k.TopicSuffix.Separator)
default:
topicName = topic
}
return metric, topicName
}
func (k *Kafka) SetSerializer(serializer telegraf.Serializer) {
k.serializer = serializer
}
func (k *Kafka) Init() error {
kafka.SetLogger(k.Log.Level())
if err := ValidateTopicSuffixMethod(k.TopicSuffix.Method); err != nil {
return err
}
config := sarama.NewConfig()
if err := k.SetConfig(config, k.Log); err != nil {
return err
}
// Legacy support ssl config
if k.Certificate != "" {
k.TLSCert = k.Certificate
k.TLSCA = k.CA
k.TLSKey = k.Key
}
if k.Socks5ProxyEnabled {
config.Net.Proxy.Enable = true
dialer, err := k.Socks5ProxyConfig.GetDialer()
if err != nil {
return fmt.Errorf("connecting to proxy server failed: %w", err)
}
config.Net.Proxy.Dialer = dialer
}
k.saramaConfig = config
switch k.ProducerTimestamp {
case "":
k.ProducerTimestamp = "metric"
case "metric", "now":
default:
return fmt.Errorf("unknown producer_timestamp option: %s", k.ProducerTimestamp)
}
return nil
}
func (k *Kafka) Connect() error {
producer, err := k.producerFunc(k.Brokers, k.saramaConfig)
if err != nil {
return &internal.StartupError{Err: err, Retry: true}
}
k.producer = producer
return nil
}
func (k *Kafka) Close() error {
if k.producer == nil {
return nil
}
return k.producer.Close()
}
func (k *Kafka) routingKey(metric telegraf.Metric) (string, error) {
if k.RoutingTag != "" {
key, ok := metric.GetTag(k.RoutingTag)
if ok {
return key, nil
}
}
if k.RoutingKey == "random" {
u, err := uuid.NewV4()
if err != nil {
return "", err
}
return u.String(), nil
}
return k.RoutingKey, nil
}
func (k *Kafka) Write(metrics []telegraf.Metric) error {
msgs := make([]*sarama.ProducerMessage, 0, len(metrics))
for _, metric := range metrics {
metric, topic := k.GetTopicName(metric)
buf, err := k.serializer.Serialize(metric)
if err != nil {
k.Log.Debugf("Could not serialize metric: %v", err)
continue
}
m := &sarama.ProducerMessage{
Topic: topic,
Value: sarama.ByteEncoder(buf),
}
if k.MetricNameHeader != "" {
m.Headers = []sarama.RecordHeader{
{
Key: []byte(k.MetricNameHeader),
Value: []byte(metric.Name()),
},
}
}
// Negative timestamps are not allowed by the Kafka protocol.
if k.ProducerTimestamp == "metric" && !metric.Time().Before(zeroTime) {
m.Timestamp = metric.Time()
}
key, err := k.routingKey(metric)
if err != nil {
return fmt.Errorf("could not generate routing key: %w", err)
}
if key != "" {
m.Key = sarama.StringEncoder(key)
}
msgs = append(msgs, m)
}
err := k.producer.SendMessages(msgs)
if err != nil {
// We could have many errors, return only the first encountered.
var errs sarama.ProducerErrors
if errors.As(err, &errs) && len(errs) > 0 {
// Just return the first error encountered
firstErr := errs[0]
if errors.Is(firstErr.Err, sarama.ErrMessageSizeTooLarge) {
k.Log.Error("Message too large, consider increasing `max_message_bytes`; dropping batch")
return nil
}
if errors.Is(firstErr.Err, sarama.ErrInvalidTimestamp) {
k.Log.Error(
"The timestamp of the message is out of acceptable range, consider increasing broker `message.timestamp.difference.max.ms`; " +
"dropping batch",
)
return nil
}
return firstErr
}
return err
}
return nil
}
func init() {
outputs.Add("kafka", func() telegraf.Output {
return &Kafka{
WriteConfig: kafka.WriteConfig{
MaxRetry: 3,
RequiredAcks: -1,
},
producerFunc: sarama.NewSyncProducer,
}
})
}

View file

@ -0,0 +1,312 @@
package kafka
import (
"testing"
"time"
"github.com/IBM/sarama"
"github.com/stretchr/testify/require"
kafkacontainer "github.com/testcontainers/testcontainers-go/modules/kafka"
"github.com/influxdata/telegraf"
"github.com/influxdata/telegraf/metric"
"github.com/influxdata/telegraf/plugins/serializers/influx"
"github.com/influxdata/telegraf/testutil"
)
type topicSuffixTestpair struct {
topicSuffix TopicSuffix
expectedTopic string
}
func TestConnectAndWriteIntegration(t *testing.T) {
if testing.Short() {
t.Skip("Skipping integration test in short mode")
}
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)
// Setup the plugin
plugin := &Kafka{
Brokers: brokers,
Topic: "Test",
Log: testutil.Logger{},
producerFunc: sarama.NewSyncProducer,
}
// Setup the metric serializer
s := &influx.Serializer{}
require.NoError(t, s.Init())
plugin.SetSerializer(s)
// Verify that we can connect to the Kafka broker
require.NoError(t, plugin.Init())
require.NoError(t, plugin.Connect())
defer plugin.Close()
// Verify that we can successfully write data to the kafka broker
require.NoError(t, plugin.Write(testutil.MockMetrics()))
}
func TestTopicSuffixes(t *testing.T) {
topic := "Test"
m := testutil.TestMetric(1)
metricTagName := "tag1"
metricTagValue := m.Tags()[metricTagName]
metricName := m.Name()
var testcases = []topicSuffixTestpair{
// This ensures empty separator is okay
{TopicSuffix{Method: "measurement"},
topic + metricName},
{TopicSuffix{Method: "measurement", Separator: "sep"},
topic + "sep" + metricName},
{TopicSuffix{Method: "tags", Keys: []string{metricTagName}, Separator: "_"},
topic + "_" + metricTagValue},
{TopicSuffix{Method: "tags", Keys: []string{metricTagName, metricTagName, metricTagName}, Separator: "___"},
topic + "___" + metricTagValue + "___" + metricTagValue + "___" + metricTagValue},
{TopicSuffix{Method: "tags", Keys: []string{metricTagName, metricTagName, metricTagName}},
topic + metricTagValue + metricTagValue + metricTagValue},
// This ensures non-existing tags are ignored
{TopicSuffix{Method: "tags", Keys: []string{"non_existing_tag", "non_existing_tag"}, Separator: "___"},
topic},
{TopicSuffix{Method: "tags", Keys: []string{metricTagName, "non_existing_tag"}, Separator: "___"},
topic + "___" + metricTagValue},
// This ensures backward compatibility
{TopicSuffix{},
topic},
}
for _, testcase := range testcases {
topicSuffix := testcase.topicSuffix
expectedTopic := testcase.expectedTopic
k := &Kafka{
Topic: topic,
TopicSuffix: topicSuffix,
Log: testutil.Logger{},
}
_, topic := k.GetTopicName(m)
require.Equal(t, expectedTopic, topic)
}
}
func TestValidateTopicSuffixMethod(t *testing.T) {
err := ValidateTopicSuffixMethod("invalid_topic_suffix_method")
require.Error(t, err, "Topic suffix method used should be invalid.")
for _, method := range ValidTopicSuffixMethods {
err := ValidateTopicSuffixMethod(method)
require.NoError(t, err, "Topic suffix method used should be valid.")
}
}
func TestRoutingKey(t *testing.T) {
tests := []struct {
name string
kafka *Kafka
metric telegraf.Metric
check func(t *testing.T, routingKey string)
}{
{
name: "static routing key",
kafka: &Kafka{
RoutingKey: "static",
},
metric: func() telegraf.Metric {
m := metric.New(
"cpu",
map[string]string{},
map[string]interface{}{
"value": 42.0,
},
time.Unix(0, 0),
)
return m
}(),
check: func(t *testing.T, routingKey string) {
require.Equal(t, "static", routingKey)
},
},
{
name: "random routing key",
kafka: &Kafka{
RoutingKey: "random",
},
metric: func() telegraf.Metric {
m := metric.New(
"cpu",
map[string]string{},
map[string]interface{}{
"value": 42.0,
},
time.Unix(0, 0),
)
return m
}(),
check: func(t *testing.T, routingKey string) {
require.Len(t, routingKey, 36)
},
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
tt.kafka.Log = testutil.Logger{}
key, err := tt.kafka.routingKey(tt.metric)
require.NoError(t, err)
tt.check(t, key)
})
}
}
type MockProducer struct {
sent []*sarama.ProducerMessage
sarama.SyncProducer
}
func (p *MockProducer) SendMessage(msg *sarama.ProducerMessage) (partition int32, offset int64, err error) {
p.sent = append(p.sent, msg)
return 0, 0, nil
}
func (p *MockProducer) SendMessages(msgs []*sarama.ProducerMessage) error {
p.sent = append(p.sent, msgs...)
return nil
}
func (*MockProducer) Close() error {
return nil
}
func NewMockProducer(_ []string, _ *sarama.Config) (sarama.SyncProducer, error) {
return &MockProducer{}, nil
}
func TestTopicTag(t *testing.T) {
tests := []struct {
name string
plugin *Kafka
input []telegraf.Metric
topic string
value string
}{
{
name: "static topic",
plugin: &Kafka{
Brokers: []string{"127.0.0.1"},
Topic: "telegraf",
producerFunc: NewMockProducer,
},
input: []telegraf.Metric{
testutil.MustMetric(
"cpu",
map[string]string{},
map[string]interface{}{
"time_idle": 42.0,
},
time.Unix(0, 0),
),
},
topic: "telegraf",
value: "cpu time_idle=42 0\n",
},
{
name: "topic tag overrides static topic",
plugin: &Kafka{
Brokers: []string{"127.0.0.1"},
Topic: "telegraf",
TopicTag: "topic",
producerFunc: NewMockProducer,
},
input: []telegraf.Metric{
testutil.MustMetric(
"cpu",
map[string]string{
"topic": "xyzzy",
},
map[string]interface{}{
"time_idle": 42.0,
},
time.Unix(0, 0),
),
},
topic: "xyzzy",
value: "cpu,topic=xyzzy time_idle=42 0\n",
},
{
name: "missing topic tag falls back to static topic",
plugin: &Kafka{
Brokers: []string{"127.0.0.1"},
Topic: "telegraf",
TopicTag: "topic",
producerFunc: NewMockProducer,
},
input: []telegraf.Metric{
testutil.MustMetric(
"cpu",
map[string]string{},
map[string]interface{}{
"time_idle": 42.0,
},
time.Unix(0, 0),
),
},
topic: "telegraf",
value: "cpu time_idle=42 0\n",
},
{
name: "exclude topic tag removes tag",
plugin: &Kafka{
Brokers: []string{"127.0.0.1"},
Topic: "telegraf",
TopicTag: "topic",
ExcludeTopicTag: true,
producerFunc: NewMockProducer,
},
input: []telegraf.Metric{
testutil.MustMetric(
"cpu",
map[string]string{
"topic": "xyzzy",
},
map[string]interface{}{
"time_idle": 42.0,
},
time.Unix(0, 0),
),
},
topic: "xyzzy",
value: "cpu time_idle=42 0\n",
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
tt.plugin.Log = testutil.Logger{}
s := &influx.Serializer{}
require.NoError(t, s.Init())
tt.plugin.SetSerializer(s)
err := tt.plugin.Connect()
require.NoError(t, err)
producer := &MockProducer{}
tt.plugin.producer = producer
err = tt.plugin.Write(tt.input)
require.NoError(t, err)
require.Equal(t, tt.topic, producer.sent[0].Topic)
encoded, err := producer.sent[0].Value.Encode()
require.NoError(t, err)
require.Equal(t, tt.value, string(encoded))
})
}
}

View file

@ -0,0 +1,196 @@
# Configuration for the Kafka server to send metrics to
[[outputs.kafka]]
## URLs of kafka brokers
## The brokers listed here are used to connect to collect metadata about a
## cluster. However, once the initial metadata collect is completed, telegraf
## will communicate solely with the kafka leader and not all defined brokers.
brokers = ["localhost:9092"]
## Kafka topic for producer messages
topic = "telegraf"
## The value of this tag will be used as the topic. If not set the 'topic'
## option is used.
# topic_tag = ""
## If true, the 'topic_tag' will be removed from to the metric.
# exclude_topic_tag = false
## Optional Client id
# client_id = "Telegraf"
## Set the minimal supported Kafka version. Setting this enables the use of new
## Kafka features and APIs. Of particular interested, lz4 compression
## requires at least version 0.10.0.0.
## ex: version = "1.1.0"
# version = ""
## The routing tag specifies a tagkey on the metric whose value is used as
## the message key. The message key is used to determine which partition to
## send the message to. This tag is preferred over the routing_key option.
routing_tag = "host"
## The routing key is set as the message key and used to determine which
## partition to send the message to. This value is only used when no
## routing_tag is set or as a fallback when the tag specified in routing tag
## is not found.
##
## If set to "random", a random value will be generated for each message.
##
## When unset, no message key is added and each message is routed to a random
## partition.
##
## ex: routing_key = "random"
## routing_key = "telegraf"
# routing_key = ""
## 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
## Idempotent Writes
## If enabled, exactly one copy of each message is written.
# idempotent_writes = false
## RequiredAcks is used in Produce Requests to tell the broker how many
## replica acknowledgements it must see before responding
## 0 : the producer never waits for an acknowledgement from the broker.
## This option provides the lowest latency but the weakest durability
## guarantees (some data will be lost when a server fails).
## 1 : the producer gets an acknowledgement after the leader replica has
## received the data. This option provides better durability as the
## client waits until the server acknowledges the request as successful
## (only messages that were written to the now-dead leader but not yet
## replicated will be lost).
## -1: the producer gets an acknowledgement after all in-sync replicas have
## received the data. This option provides the best durability, we
## guarantee that no messages will be lost as long as at least one in
## sync replica remains.
# required_acks = -1
## The maximum number of times to retry sending a metric before failing
## until the next flush.
# max_retry = 3
## The maximum permitted size of a message. Should be set equal to or
## smaller than the broker's 'message.max.bytes'.
# max_message_bytes = 1000000
## Producer timestamp
## This option sets the timestamp of the kafka producer message, choose from:
## * metric: Uses the metric's timestamp
## * now: Uses the time of write
# producer_timestamp = metric
## Add metric name as specified kafka header if not empty
# metric_name_header = ""
## 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"
## Optional SOCKS5 proxy to use when connecting to brokers
# socks5_enabled = true
# socks5_address = "127.0.0.1:1080"
# socks5_username = "alice"
# socks5_password = "pass123"
## Optional SASL Config
# 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
## Access token used if sasl_mechanism is OAUTHBEARER
# sasl_access_token = ""
## Arbitrary key value string pairs to pass as a TOML table. For example:
# {logicalCluster = "cluster-042", poolId = "pool-027"}
# sasl_extensions = {}
## SASL protocol version. When connecting to Azure EventHub set to 0.
# sasl_version = 1
# Disable Kafka metadata full fetch
# metadata_full = false
## 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
## Data format to output.
## 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_OUTPUT.md
# data_format = "influx"
## NOTE: Due to the way TOML is parsed, tables must be at the END of the
## plugin definition, otherwise additional config options are read as part of
## the table
## Optional topic suffix configuration.
## If the section is omitted, no suffix is used.
## Following topic suffix methods are supported:
## measurement - suffix equals to separator + measurement's name
## tags - suffix equals to separator + specified tags' values
## interleaved with separator
## Suffix equals to "_" + measurement name
# [outputs.kafka.topic_suffix]
# method = "measurement"
# separator = "_"
## Suffix equals to "__" + measurement's "foo" tag value.
## If there's no such a tag, suffix equals to an empty string
# [outputs.kafka.topic_suffix]
# method = "tags"
# keys = ["foo"]
# separator = "__"
## Suffix equals to "_" + measurement's "foo" and "bar"
## tag values, separated by "_". If there is no such tags,
## their values treated as empty strings.
# [outputs.kafka.topic_suffix]
# method = "tags"
# keys = ["foo", "bar"]
# separator = "_"