Adding upstream version 1.34.4.
Signed-off-by: Daniel Baumann <daniel@debian.org>
This commit is contained in:
parent
e393c3af3f
commit
4978089aab
4963 changed files with 677545 additions and 0 deletions
193
plugins/inputs/amqp_consumer/README.md
Normal file
193
plugins/inputs/amqp_consumer/README.md
Normal file
|
@ -0,0 +1,193 @@
|
|||
# AMQP Consumer Input Plugin
|
||||
|
||||
This plugin consumes messages from an Advanced Message Queuing Protocol v0.9.1
|
||||
broker. A prominent implementation of this protocol is [RabbitMQ][rabbitmq].
|
||||
|
||||
Metrics are read from a topic exchange using the configured queue and binding
|
||||
key. The message payloads must be formatted in one of the supported
|
||||
[data formats][data_formats].
|
||||
|
||||
For an introduction check the [AMQP concepts page][amqp_concepts] and the
|
||||
[RabbitMQ getting started guide][rabbitmq_getting_started].
|
||||
|
||||
⭐ Telegraf v1.3.0
|
||||
🏷️ messaging
|
||||
💻 all
|
||||
|
||||
[amqp_concepts]: https://www.rabbitmq.com/tutorials/amqp-concepts.html
|
||||
[data_formats]: /docs/DATA_FORMATS_INPUT.md
|
||||
[rabbitmq]: https://www.rabbitmq.com
|
||||
[rabbitmq_getting_started]: https://www.rabbitmq.com/getstarted.html
|
||||
|
||||
## 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 `username` and
|
||||
`password` 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
|
||||
# AMQP consumer plugin
|
||||
[[inputs.amqp_consumer]]
|
||||
## Brokers to consume from. If multiple brokers are specified a random broker
|
||||
## will be selected anytime a connection is established. This can be
|
||||
## helpful for load balancing when not using a dedicated load balancer.
|
||||
brokers = ["amqp://localhost:5672/influxdb"]
|
||||
|
||||
## Authentication credentials for the PLAIN auth_method.
|
||||
# username = ""
|
||||
# password = ""
|
||||
|
||||
## Name of the exchange to declare. If unset, no exchange will be declared.
|
||||
exchange = "telegraf"
|
||||
|
||||
## Exchange type; common types are "direct", "fanout", "topic", "header", "x-consistent-hash".
|
||||
# exchange_type = "topic"
|
||||
|
||||
## If true, exchange will be passively declared.
|
||||
# exchange_passive = false
|
||||
|
||||
## Exchange durability can be either "transient" or "durable".
|
||||
# exchange_durability = "durable"
|
||||
|
||||
## Additional exchange arguments.
|
||||
# exchange_arguments = { }
|
||||
# exchange_arguments = {"hash_property" = "timestamp"}
|
||||
|
||||
## AMQP queue name.
|
||||
queue = "telegraf"
|
||||
|
||||
## AMQP queue durability can be "transient" or "durable".
|
||||
queue_durability = "durable"
|
||||
|
||||
## If true, queue will be passively declared.
|
||||
# queue_passive = false
|
||||
|
||||
## Additional arguments when consuming from Queue
|
||||
# queue_consume_arguments = { }
|
||||
# queue_consume_arguments = {"x-stream-offset" = "first"}
|
||||
|
||||
## Additional queue arguments.
|
||||
# queue_arguments = { }
|
||||
# queue_arguments = {"x-max-length" = 100}
|
||||
|
||||
## A binding between the exchange and queue using this binding key is
|
||||
## created. If unset, no binding is created.
|
||||
binding_key = "#"
|
||||
|
||||
## Maximum number of messages server should give to the worker.
|
||||
# prefetch_count = 50
|
||||
|
||||
## 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
|
||||
|
||||
## Timeout for establishing the connection to a broker
|
||||
# timeout = "30s"
|
||||
|
||||
## Auth method. PLAIN and EXTERNAL are supported
|
||||
## Using EXTERNAL requires enabling the rabbitmq_auth_mechanism_ssl plugin as
|
||||
## described here: https://www.rabbitmq.com/plugins.html
|
||||
# auth_method = "PLAIN"
|
||||
|
||||
## Optional TLS Config
|
||||
# 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
|
||||
|
||||
## Content encoding for message payloads, can be set to
|
||||
## "gzip", "identity" or "auto"
|
||||
## - Use "gzip" to decode gzip
|
||||
## - Use "identity" to apply no encoding
|
||||
## - Use "auto" determine the encoding using the ContentEncoding header
|
||||
# content_encoding = "identity"
|
||||
|
||||
## Maximum size of decoded message.
|
||||
## Acceptable units are B, KiB, KB, MiB, MB...
|
||||
## Without quotes and units, interpreted as size in bytes.
|
||||
# max_decompression_size = "500MB"
|
||||
|
||||
## 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"
|
||||
```
|
||||
|
||||
## Message acknowledgement behavior
|
||||
|
||||
This plugin tracks metrics to report the delivery state to the broker.
|
||||
|
||||
Messages are **acknowledged** (ACK) in the broker if they were successfully
|
||||
parsed and delivered to all corresponding output sinks.
|
||||
|
||||
Messages are **not acknowledged** (NACK) if parsing of the messages fails and no
|
||||
metrics were created. In this case requeueing is disabled so messages will not
|
||||
be sent out to any other queue. The message will then be discarded or sent to a
|
||||
dead-letter exchange depending on the server configuration. See
|
||||
[RabitMQ documentation][rabbitmq_doc] for more details.
|
||||
|
||||
Messages are **rejected** (REJECT) if the messages were parsed correctly but
|
||||
could not be delivered e.g. due to output-service outages. Requeueing is
|
||||
disabled in this case and messages will be discarded by the server. See
|
||||
[RabitMQ documentation][rabbitmq_doc] for more details.
|
||||
|
||||
[rabbitmq_doc]: https://www.rabbitmq.com/docs/confirms
|
||||
|
||||
## Metrics
|
||||
|
||||
The format of metrics produced by this plugin depends on the content and
|
||||
data format of received messages.
|
||||
|
||||
## Example Output
|
504
plugins/inputs/amqp_consumer/amqp_consumer.go
Normal file
504
plugins/inputs/amqp_consumer/amqp_consumer.go
Normal file
|
@ -0,0 +1,504 @@
|
|||
//go:generate ../../../tools/readme_config_includer/generator
|
||||
package amqp_consumer
|
||||
|
||||
import (
|
||||
"context"
|
||||
_ "embed"
|
||||
"errors"
|
||||
"fmt"
|
||||
"math/rand"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
amqp "github.com/rabbitmq/amqp091-go"
|
||||
|
||||
"github.com/influxdata/telegraf"
|
||||
"github.com/influxdata/telegraf/config"
|
||||
"github.com/influxdata/telegraf/internal"
|
||||
"github.com/influxdata/telegraf/plugins/common/tls"
|
||||
"github.com/influxdata/telegraf/plugins/inputs"
|
||||
)
|
||||
|
||||
//go:embed sample.conf
|
||||
var sampleConfig string
|
||||
|
||||
var once sync.Once
|
||||
|
||||
type empty struct{}
|
||||
type externalAuth struct{}
|
||||
|
||||
type semaphore chan empty
|
||||
|
||||
type AMQPConsumer struct {
|
||||
URL string `toml:"url" deprecated:"1.7.0;1.35.0;use 'brokers' instead"`
|
||||
Brokers []string `toml:"brokers"`
|
||||
Username config.Secret `toml:"username"`
|
||||
Password config.Secret `toml:"password"`
|
||||
Exchange string `toml:"exchange"`
|
||||
ExchangeType string `toml:"exchange_type"`
|
||||
ExchangeDurability string `toml:"exchange_durability"`
|
||||
ExchangePassive bool `toml:"exchange_passive"`
|
||||
ExchangeArguments map[string]string `toml:"exchange_arguments"`
|
||||
MaxUndeliveredMessages int `toml:"max_undelivered_messages"`
|
||||
Queue string `toml:"queue"`
|
||||
QueueDurability string `toml:"queue_durability"`
|
||||
QueuePassive bool `toml:"queue_passive"`
|
||||
QueueArguments map[string]int `toml:"queue_arguments"`
|
||||
QueueConsumeArguments map[string]string `toml:"queue_consume_arguments"`
|
||||
BindingKey string `toml:"binding_key"`
|
||||
PrefetchCount int `toml:"prefetch_count"`
|
||||
AuthMethod string `toml:"auth_method"`
|
||||
ContentEncoding string `toml:"content_encoding"`
|
||||
MaxDecompressionSize config.Size `toml:"max_decompression_size"`
|
||||
Timeout config.Duration `toml:"timeout"`
|
||||
Log telegraf.Logger `toml:"-"`
|
||||
tls.ClientConfig
|
||||
|
||||
deliveries map[telegraf.TrackingID]amqp.Delivery
|
||||
|
||||
parser telegraf.Parser
|
||||
conn *amqp.Connection
|
||||
wg *sync.WaitGroup
|
||||
cancel context.CancelFunc
|
||||
decoder internal.ContentDecoder
|
||||
}
|
||||
|
||||
// Mechanism represents the authentication mechanism used for AMQP connections.
|
||||
func (*externalAuth) Mechanism() string {
|
||||
return "EXTERNAL"
|
||||
}
|
||||
|
||||
// Response represents the response returned by the AMQP server.
|
||||
func (*externalAuth) Response() string {
|
||||
return "\000"
|
||||
}
|
||||
|
||||
func (*AMQPConsumer) SampleConfig() string {
|
||||
return sampleConfig
|
||||
}
|
||||
|
||||
func (a *AMQPConsumer) Init() error {
|
||||
// Defaults
|
||||
if a.URL != "" {
|
||||
a.Brokers = append(a.Brokers, a.URL)
|
||||
}
|
||||
if len(a.Brokers) == 0 {
|
||||
a.Brokers = []string{"amqp://localhost:5672/influxdb"}
|
||||
}
|
||||
|
||||
if a.AuthMethod == "" {
|
||||
a.AuthMethod = "PLAIN"
|
||||
}
|
||||
|
||||
if a.ExchangeType == "" {
|
||||
a.ExchangeType = "topic"
|
||||
}
|
||||
|
||||
if a.ExchangeDurability == "" {
|
||||
a.ExchangeDurability = "durable"
|
||||
}
|
||||
|
||||
if a.QueueDurability == "" {
|
||||
a.QueueDurability = "durable"
|
||||
}
|
||||
|
||||
if a.PrefetchCount == 0 {
|
||||
a.PrefetchCount = 50
|
||||
}
|
||||
|
||||
if a.MaxUndeliveredMessages == 0 {
|
||||
a.MaxUndeliveredMessages = 1000
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (a *AMQPConsumer) SetParser(parser telegraf.Parser) {
|
||||
a.parser = parser
|
||||
}
|
||||
|
||||
func (a *AMQPConsumer) Start(acc telegraf.Accumulator) error {
|
||||
amqpConf, err := a.createConfig()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
var options []internal.DecodingOption
|
||||
if a.MaxDecompressionSize > 0 {
|
||||
options = append(options, internal.WithMaxDecompressionSize(int64(a.MaxDecompressionSize)))
|
||||
}
|
||||
a.decoder, err = internal.NewContentDecoder(a.ContentEncoding, options...)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
msgs, err := a.connect(amqpConf)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
a.cancel = cancel
|
||||
|
||||
a.wg = &sync.WaitGroup{}
|
||||
a.wg.Add(1)
|
||||
go func() {
|
||||
defer a.wg.Done()
|
||||
a.process(ctx, msgs, acc)
|
||||
}()
|
||||
|
||||
go func() {
|
||||
for {
|
||||
err := <-a.conn.NotifyClose(make(chan *amqp.Error))
|
||||
if err == nil {
|
||||
break
|
||||
}
|
||||
|
||||
a.Log.Infof("Connection closed: %s; trying to reconnect", err)
|
||||
for {
|
||||
msgs, err := a.connect(amqpConf)
|
||||
if err != nil {
|
||||
a.Log.Errorf("AMQP connection failed: %s", err)
|
||||
time.Sleep(10 * time.Second)
|
||||
continue
|
||||
}
|
||||
|
||||
a.wg.Add(1)
|
||||
go func() {
|
||||
defer a.wg.Done()
|
||||
a.process(ctx, msgs, acc)
|
||||
}()
|
||||
break
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (*AMQPConsumer) Gather(_ telegraf.Accumulator) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (a *AMQPConsumer) Stop() {
|
||||
// We did not connect successfully so there is nothing to do here.
|
||||
if a.conn == nil || a.conn.IsClosed() {
|
||||
return
|
||||
}
|
||||
a.cancel()
|
||||
a.wg.Wait()
|
||||
err := a.conn.Close()
|
||||
if err != nil && !errors.Is(err, amqp.ErrClosed) {
|
||||
a.Log.Errorf("Error closing AMQP connection: %s", err)
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
func (a *AMQPConsumer) createConfig() (*amqp.Config, error) {
|
||||
// make new tls config
|
||||
tlsCfg, err := a.ClientConfig.TLSConfig()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
var auth []amqp.Authentication
|
||||
|
||||
if strings.EqualFold(a.AuthMethod, "EXTERNAL") {
|
||||
auth = []amqp.Authentication{&externalAuth{}}
|
||||
} else if !a.Username.Empty() || !a.Password.Empty() {
|
||||
username, err := a.Username.Get()
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("getting username failed: %w", err)
|
||||
}
|
||||
defer username.Destroy()
|
||||
|
||||
password, err := a.Password.Get()
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("getting password failed: %w", err)
|
||||
}
|
||||
defer password.Destroy()
|
||||
|
||||
auth = []amqp.Authentication{
|
||||
&amqp.PlainAuth{
|
||||
Username: username.String(),
|
||||
Password: password.String(),
|
||||
},
|
||||
}
|
||||
}
|
||||
amqpConfig := amqp.Config{
|
||||
TLSClientConfig: tlsCfg,
|
||||
SASL: auth, // if nil, it will be PLAIN
|
||||
Dial: amqp.DefaultDial(time.Duration(a.Timeout)),
|
||||
}
|
||||
return &amqpConfig, nil
|
||||
}
|
||||
|
||||
func (a *AMQPConsumer) connect(amqpConf *amqp.Config) (<-chan amqp.Delivery, error) {
|
||||
brokers := a.Brokers
|
||||
|
||||
p := rand.Perm(len(brokers))
|
||||
for _, n := range p {
|
||||
broker := brokers[n]
|
||||
a.Log.Debugf("Connecting to %q", broker)
|
||||
conn, err := amqp.DialConfig(broker, *amqpConf)
|
||||
if err == nil {
|
||||
a.conn = conn
|
||||
a.Log.Debugf("Connected to %q", broker)
|
||||
break
|
||||
}
|
||||
a.Log.Errorf("Error connecting to %q: %s", broker, err)
|
||||
}
|
||||
|
||||
if a.conn == nil {
|
||||
return nil, &internal.StartupError{
|
||||
Err: errors.New("could not connect to any broker"),
|
||||
Retry: true,
|
||||
}
|
||||
}
|
||||
|
||||
ch, err := a.conn.Channel()
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to open a channel: %w", err)
|
||||
}
|
||||
|
||||
if a.Exchange != "" {
|
||||
exchangeDurable := true
|
||||
if a.ExchangeDurability == "transient" {
|
||||
exchangeDurable = false
|
||||
}
|
||||
|
||||
exchangeArgs := make(amqp.Table, len(a.ExchangeArguments))
|
||||
for k, v := range a.ExchangeArguments {
|
||||
exchangeArgs[k] = v
|
||||
}
|
||||
|
||||
err = a.declareExchange(
|
||||
ch,
|
||||
exchangeDurable,
|
||||
exchangeArgs)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
q, err := a.declareQueue(ch)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if a.BindingKey != "" {
|
||||
err = ch.QueueBind(
|
||||
q.Name, // queue
|
||||
a.BindingKey, // binding-key
|
||||
a.Exchange, // exchange
|
||||
false,
|
||||
nil,
|
||||
)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to bind a queue: %w", err)
|
||||
}
|
||||
}
|
||||
|
||||
err = ch.Qos(
|
||||
a.PrefetchCount,
|
||||
0, // prefetch-size
|
||||
false, // global
|
||||
)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to set QoS: %w", err)
|
||||
}
|
||||
|
||||
consumeArgs := make(amqp.Table, len(a.QueueConsumeArguments))
|
||||
for k, v := range a.QueueConsumeArguments {
|
||||
consumeArgs[k] = v
|
||||
}
|
||||
|
||||
msgs, err := ch.Consume(
|
||||
q.Name, // queue
|
||||
"", // consumer
|
||||
false, // auto-ack
|
||||
false, // exclusive
|
||||
false, // no-local
|
||||
false, // no-wait
|
||||
consumeArgs, // arguments
|
||||
)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed establishing connection to queue: %w", err)
|
||||
}
|
||||
|
||||
return msgs, err
|
||||
}
|
||||
|
||||
func (a *AMQPConsumer) declareExchange(
|
||||
channel *amqp.Channel,
|
||||
exchangeDurable bool,
|
||||
exchangeArguments amqp.Table,
|
||||
) error {
|
||||
var err error
|
||||
if a.ExchangePassive {
|
||||
err = channel.ExchangeDeclarePassive(
|
||||
a.Exchange,
|
||||
a.ExchangeType,
|
||||
exchangeDurable,
|
||||
false, // delete when unused
|
||||
false, // internal
|
||||
false, // no-wait
|
||||
exchangeArguments,
|
||||
)
|
||||
} else {
|
||||
err = channel.ExchangeDeclare(
|
||||
a.Exchange,
|
||||
a.ExchangeType,
|
||||
exchangeDurable,
|
||||
false, // delete when unused
|
||||
false, // internal
|
||||
false, // no-wait
|
||||
exchangeArguments,
|
||||
)
|
||||
}
|
||||
if err != nil {
|
||||
return fmt.Errorf("error declaring exchange: %w", err)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (a *AMQPConsumer) declareQueue(channel *amqp.Channel) (*amqp.Queue, error) {
|
||||
var queue amqp.Queue
|
||||
var err error
|
||||
|
||||
queueDurable := true
|
||||
if a.QueueDurability == "transient" {
|
||||
queueDurable = false
|
||||
}
|
||||
|
||||
queueArgs := make(amqp.Table, len(a.QueueArguments))
|
||||
for k, v := range a.QueueArguments {
|
||||
queueArgs[k] = v
|
||||
}
|
||||
|
||||
if a.QueuePassive {
|
||||
queue, err = channel.QueueDeclarePassive(
|
||||
a.Queue, // queue
|
||||
queueDurable, // durable
|
||||
false, // delete when unused
|
||||
false, // exclusive
|
||||
false, // no-wait
|
||||
queueArgs, // arguments
|
||||
)
|
||||
} else {
|
||||
queue, err = channel.QueueDeclare(
|
||||
a.Queue, // queue
|
||||
queueDurable, // durable
|
||||
false, // delete when unused
|
||||
false, // exclusive
|
||||
false, // no-wait
|
||||
queueArgs, // arguments
|
||||
)
|
||||
}
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("error declaring queue: %w", err)
|
||||
}
|
||||
return &queue, nil
|
||||
}
|
||||
|
||||
// Read messages from queue and add them to the Accumulator
|
||||
func (a *AMQPConsumer) process(ctx context.Context, msgs <-chan amqp.Delivery, ac telegraf.Accumulator) {
|
||||
a.deliveries = make(map[telegraf.TrackingID]amqp.Delivery)
|
||||
|
||||
acc := ac.WithTracking(a.MaxUndeliveredMessages)
|
||||
sem := make(semaphore, a.MaxUndeliveredMessages)
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case track := <-acc.Delivered():
|
||||
if a.onDelivery(track) {
|
||||
<-sem
|
||||
}
|
||||
case sem <- empty{}:
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case track := <-acc.Delivered():
|
||||
if a.onDelivery(track) {
|
||||
<-sem
|
||||
<-sem
|
||||
}
|
||||
case d, ok := <-msgs:
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
err := a.onMessage(acc, d)
|
||||
if err != nil {
|
||||
acc.AddError(err)
|
||||
<-sem
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (a *AMQPConsumer) onMessage(acc telegraf.TrackingAccumulator, d amqp.Delivery) error {
|
||||
onError := func() {
|
||||
// Discard the message from the queue; will never be able to process it
|
||||
if err := d.Nack(false, false); err != nil {
|
||||
a.Log.Errorf("Unable to NACK message: %d: %v", d.DeliveryTag, err)
|
||||
a.conn.Close()
|
||||
}
|
||||
}
|
||||
|
||||
a.decoder.SetEncoding(d.ContentEncoding)
|
||||
body, err := a.decoder.Decode(d.Body)
|
||||
if err != nil {
|
||||
onError()
|
||||
return err
|
||||
}
|
||||
|
||||
metrics, err := a.parser.Parse(body)
|
||||
if err != nil {
|
||||
onError()
|
||||
return err
|
||||
}
|
||||
if len(metrics) == 0 {
|
||||
once.Do(func() {
|
||||
a.Log.Debug(internal.NoMetricsCreatedMsg)
|
||||
})
|
||||
}
|
||||
|
||||
id := acc.AddTrackingMetricGroup(metrics)
|
||||
a.deliveries[id] = d
|
||||
return nil
|
||||
}
|
||||
|
||||
func (a *AMQPConsumer) onDelivery(track telegraf.DeliveryInfo) bool {
|
||||
delivery, ok := a.deliveries[track.ID()]
|
||||
if !ok {
|
||||
// Added by a previous connection
|
||||
return false
|
||||
}
|
||||
|
||||
if track.Delivered() {
|
||||
err := delivery.Ack(false)
|
||||
if err != nil {
|
||||
a.Log.Errorf("Unable to ack written delivery: %d: %v", delivery.DeliveryTag, err)
|
||||
a.conn.Close()
|
||||
}
|
||||
} else {
|
||||
err := delivery.Reject(false)
|
||||
if err != nil {
|
||||
a.Log.Errorf("Unable to reject failed delivery: %d: %v", delivery.DeliveryTag, err)
|
||||
a.conn.Close()
|
||||
}
|
||||
}
|
||||
|
||||
delete(a.deliveries, track.ID())
|
||||
return true
|
||||
}
|
||||
|
||||
func init() {
|
||||
inputs.Add("amqp_consumer", func() telegraf.Input {
|
||||
return &AMQPConsumer{Timeout: config.Duration(30 * time.Second)}
|
||||
})
|
||||
}
|
444
plugins/inputs/amqp_consumer/amqp_consumer_test.go
Normal file
444
plugins/inputs/amqp_consumer/amqp_consumer_test.go
Normal file
|
@ -0,0 +1,444 @@
|
|||
package amqp_consumer
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/docker/go-connections/nat"
|
||||
"github.com/rabbitmq/amqp091-go"
|
||||
"github.com/stretchr/testify/require"
|
||||
"github.com/testcontainers/testcontainers-go/wait"
|
||||
|
||||
"github.com/influxdata/telegraf"
|
||||
"github.com/influxdata/telegraf/config"
|
||||
"github.com/influxdata/telegraf/internal"
|
||||
"github.com/influxdata/telegraf/models"
|
||||
"github.com/influxdata/telegraf/plugins/parsers/influx"
|
||||
"github.com/influxdata/telegraf/testutil"
|
||||
)
|
||||
|
||||
func TestAutoEncoding(t *testing.T) {
|
||||
// Setup a gzipped payload
|
||||
enc, err := internal.NewGzipEncoder()
|
||||
require.NoError(t, err)
|
||||
payloadGZip, err := enc.Encode([]byte(`measurementName fieldKey="gzip" 1556813561098000000`))
|
||||
require.NoError(t, err)
|
||||
|
||||
// Setup the plugin including the message parser
|
||||
decoder, err := internal.NewContentDecoder("auto")
|
||||
require.NoError(t, err)
|
||||
plugin := &AMQPConsumer{
|
||||
deliveries: make(map[telegraf.TrackingID]amqp091.Delivery),
|
||||
decoder: decoder,
|
||||
}
|
||||
|
||||
parser := &influx.Parser{}
|
||||
require.NoError(t, parser.Init())
|
||||
plugin.SetParser(parser)
|
||||
|
||||
// Setup the message creator
|
||||
msg := amqp091.Delivery{
|
||||
ContentEncoding: "gzip",
|
||||
Body: payloadGZip,
|
||||
}
|
||||
|
||||
// Simulate a message receive event
|
||||
var acc testutil.Accumulator
|
||||
require.NoError(t, plugin.onMessage(&acc, msg))
|
||||
acc.AssertContainsFields(t, "measurementName", map[string]interface{}{"fieldKey": "gzip"})
|
||||
|
||||
// Check the decoding
|
||||
encIdentity, err := internal.NewIdentityEncoder()
|
||||
require.NoError(t, err)
|
||||
payload, err := encIdentity.Encode([]byte(`measurementName2 fieldKey="identity" 1556813561098000000`))
|
||||
require.NoError(t, err)
|
||||
|
||||
// Setup a non-encoded payload
|
||||
msg = amqp091.Delivery{
|
||||
ContentEncoding: "not_gzip",
|
||||
Body: payload,
|
||||
}
|
||||
|
||||
// Simulate a message receive event
|
||||
require.NoError(t, plugin.onMessage(&acc, msg))
|
||||
require.NoError(t, err)
|
||||
acc.AssertContainsFields(t, "measurementName2", map[string]interface{}{"fieldKey": "identity"})
|
||||
}
|
||||
|
||||
func TestIntegration(t *testing.T) {
|
||||
if testing.Short() {
|
||||
t.Skip("Skipping integration test in short mode")
|
||||
}
|
||||
|
||||
// Define common properties
|
||||
servicePort := "5672"
|
||||
vhost := "/"
|
||||
exchange := "telegraf"
|
||||
exchangeType := "direct"
|
||||
queueName := "test"
|
||||
bindingKey := "test"
|
||||
|
||||
// Setup the container
|
||||
container := testutil.Container{
|
||||
Image: "rabbitmq",
|
||||
ExposedPorts: []string{servicePort},
|
||||
WaitingFor: wait.ForAll(
|
||||
wait.ForListeningPort(nat.Port(servicePort)),
|
||||
wait.ForLog("Server startup complete"),
|
||||
),
|
||||
}
|
||||
require.NoError(t, container.Start(), "failed to start container")
|
||||
defer container.Terminate()
|
||||
url := fmt.Sprintf("amqp://%s:%s%s", container.Address, container.Ports[servicePort], vhost)
|
||||
|
||||
// Setup a AMQP producer to send messages
|
||||
client, err := newProducer(url, vhost, exchange, exchangeType, queueName, bindingKey)
|
||||
require.NoError(t, err)
|
||||
defer client.close()
|
||||
|
||||
// Setup the plugin with an Influx line-protocol parser
|
||||
plugin := &AMQPConsumer{
|
||||
Brokers: []string{url},
|
||||
Username: config.NewSecret([]byte("guest")),
|
||||
Password: config.NewSecret([]byte("guest")),
|
||||
Timeout: config.Duration(3 * time.Second),
|
||||
Exchange: exchange,
|
||||
ExchangeType: exchangeType,
|
||||
Queue: queueName,
|
||||
BindingKey: bindingKey,
|
||||
Log: testutil.Logger{},
|
||||
}
|
||||
|
||||
parser := &influx.Parser{}
|
||||
require.NoError(t, parser.Init())
|
||||
plugin.SetParser(parser)
|
||||
require.NoError(t, plugin.Init())
|
||||
|
||||
// Setup the metrics
|
||||
metrics := []string{
|
||||
"test,source=A value=0i 1712780301000000000",
|
||||
"test,source=B value=1i 1712780301000000100",
|
||||
"test,source=C value=2i 1712780301000000200",
|
||||
}
|
||||
expected := make([]telegraf.Metric, 0, len(metrics))
|
||||
for _, x := range metrics {
|
||||
m, err := parser.Parse([]byte(x))
|
||||
require.NoError(t, err)
|
||||
expected = append(expected, m...)
|
||||
}
|
||||
|
||||
// Start the plugin
|
||||
var acc testutil.Accumulator
|
||||
require.NoError(t, plugin.Start(&acc))
|
||||
defer plugin.Stop()
|
||||
|
||||
// Write metrics
|
||||
for _, x := range metrics {
|
||||
require.NoError(t, client.write(t.Context(), exchange, queueName, []byte(x)))
|
||||
}
|
||||
|
||||
// Verify that the metrics were actually written
|
||||
require.Eventually(t, func() bool {
|
||||
return acc.NMetrics() >= uint64(len(expected))
|
||||
}, 3*time.Second, 100*time.Millisecond)
|
||||
|
||||
client.close()
|
||||
plugin.Stop()
|
||||
testutil.RequireMetricsEqual(t, expected, acc.GetTelegrafMetrics())
|
||||
}
|
||||
|
||||
func TestStartupErrorBehaviorError(t *testing.T) {
|
||||
if testing.Short() {
|
||||
t.Skip("Skipping integration test in short mode")
|
||||
}
|
||||
|
||||
// Define common properties
|
||||
servicePort := "5672"
|
||||
vhost := "/"
|
||||
exchange := "telegraf"
|
||||
exchangeType := "direct"
|
||||
queueName := "test"
|
||||
bindingKey := "test"
|
||||
|
||||
// Setup the container
|
||||
container := testutil.Container{
|
||||
Image: "rabbitmq",
|
||||
ExposedPorts: []string{servicePort},
|
||||
WaitingFor: wait.ForAll(
|
||||
wait.ForListeningPort(nat.Port(servicePort)),
|
||||
wait.ForLog("Server startup complete"),
|
||||
),
|
||||
}
|
||||
require.NoError(t, container.Start(), "failed to start container")
|
||||
defer container.Terminate()
|
||||
url := fmt.Sprintf("amqp://%s:%s%s", container.Address, container.Ports[servicePort], vhost)
|
||||
|
||||
// Pause the container for simulating connectivity issues
|
||||
require.NoError(t, container.Pause())
|
||||
defer container.Resume() //nolint:errcheck // Ignore the returned error as we cannot do anything about it anyway
|
||||
|
||||
// Setup the plugin with an Influx line-protocol parser
|
||||
plugin := &AMQPConsumer{
|
||||
Brokers: []string{url},
|
||||
Username: config.NewSecret([]byte("guest")),
|
||||
Password: config.NewSecret([]byte("guest")),
|
||||
Timeout: config.Duration(1 * time.Second),
|
||||
Exchange: exchange,
|
||||
ExchangeType: exchangeType,
|
||||
Queue: queueName,
|
||||
BindingKey: bindingKey,
|
||||
Log: testutil.Logger{},
|
||||
}
|
||||
|
||||
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: "amqp",
|
||||
},
|
||||
)
|
||||
require.NoError(t, model.Init())
|
||||
|
||||
// Starting the plugin will fail with an error because the container
|
||||
// is paused.
|
||||
var acc testutil.Accumulator
|
||||
require.ErrorContains(t, model.Start(&acc), "could not connect to any broker")
|
||||
}
|
||||
|
||||
func TestStartupErrorBehaviorIgnore(t *testing.T) {
|
||||
if testing.Short() {
|
||||
t.Skip("Skipping integration test in short mode")
|
||||
}
|
||||
|
||||
// Define common properties
|
||||
servicePort := "5672"
|
||||
vhost := "/"
|
||||
exchange := "telegraf"
|
||||
exchangeType := "direct"
|
||||
queueName := "test"
|
||||
bindingKey := "test"
|
||||
|
||||
// Setup the container
|
||||
container := testutil.Container{
|
||||
Image: "rabbitmq",
|
||||
ExposedPorts: []string{servicePort},
|
||||
WaitingFor: wait.ForAll(
|
||||
wait.ForListeningPort(nat.Port(servicePort)),
|
||||
wait.ForLog("Server startup complete"),
|
||||
),
|
||||
}
|
||||
require.NoError(t, container.Start(), "failed to start container")
|
||||
defer container.Terminate()
|
||||
url := fmt.Sprintf("amqp://%s:%s%s", container.Address, container.Ports[servicePort], vhost)
|
||||
|
||||
// Pause the container for simulating connectivity issues
|
||||
require.NoError(t, container.Pause())
|
||||
defer container.Resume() //nolint:errcheck // Ignore the returned error as we cannot do anything about it anyway
|
||||
|
||||
// Setup the plugin with an Influx line-protocol parser
|
||||
plugin := &AMQPConsumer{
|
||||
Brokers: []string{url},
|
||||
Username: config.NewSecret([]byte("guest")),
|
||||
Password: config.NewSecret([]byte("guest")),
|
||||
Timeout: config.Duration(1 * time.Second),
|
||||
Exchange: exchange,
|
||||
ExchangeType: exchangeType,
|
||||
Queue: queueName,
|
||||
BindingKey: bindingKey,
|
||||
Log: testutil.Logger{},
|
||||
}
|
||||
|
||||
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: "amqp",
|
||||
StartupErrorBehavior: "ignore",
|
||||
},
|
||||
)
|
||||
require.NoError(t, model.Init())
|
||||
|
||||
// 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, "could not connect to any broker")
|
||||
var fatalErr *internal.FatalError
|
||||
require.ErrorAs(t, err, &fatalErr)
|
||||
}
|
||||
|
||||
func TestStartupErrorBehaviorRetry(t *testing.T) {
|
||||
if testing.Short() {
|
||||
t.Skip("Skipping integration test in short mode")
|
||||
}
|
||||
|
||||
// Define common properties
|
||||
servicePort := "5672"
|
||||
vhost := "/"
|
||||
exchange := "telegraf"
|
||||
exchangeType := "direct"
|
||||
queueName := "test"
|
||||
bindingKey := "test"
|
||||
|
||||
// Setup the container
|
||||
container := testutil.Container{
|
||||
Image: "rabbitmq",
|
||||
ExposedPorts: []string{servicePort},
|
||||
WaitingFor: wait.ForAll(
|
||||
wait.ForListeningPort(nat.Port(servicePort)),
|
||||
wait.ForLog("Server startup complete"),
|
||||
),
|
||||
}
|
||||
require.NoError(t, container.Start(), "failed to start container")
|
||||
defer container.Terminate()
|
||||
url := fmt.Sprintf("amqp://%s:%s%s", container.Address, container.Ports[servicePort], vhost)
|
||||
|
||||
// Pause the container for simulating connectivity issues
|
||||
require.NoError(t, container.Pause())
|
||||
defer container.Resume() //nolint:errcheck // Ignore the returned error as we cannot do anything about it anyway
|
||||
|
||||
// Setup the plugin with an Influx line-protocol parser
|
||||
plugin := &AMQPConsumer{
|
||||
Brokers: []string{url},
|
||||
Username: config.NewSecret([]byte("guest")),
|
||||
Password: config.NewSecret([]byte("guest")),
|
||||
Timeout: config.Duration(1 * time.Second),
|
||||
Exchange: exchange,
|
||||
ExchangeType: exchangeType,
|
||||
Queue: queueName,
|
||||
BindingKey: bindingKey,
|
||||
Log: testutil.Logger{},
|
||||
}
|
||||
|
||||
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: "amqp",
|
||||
StartupErrorBehavior: "retry",
|
||||
},
|
||||
)
|
||||
require.NoError(t, model.Init())
|
||||
|
||||
// Setup the metrics
|
||||
metrics := []string{
|
||||
"test,source=A value=0i 1712780301000000000",
|
||||
"test,source=B value=1i 1712780301000000100",
|
||||
"test,source=C value=2i 1712780301000000200",
|
||||
}
|
||||
expected := make([]telegraf.Metric, 0, len(metrics))
|
||||
for _, x := range metrics {
|
||||
m, err := parser.Parse([]byte(x))
|
||||
require.NoError(t, err)
|
||||
expected = append(expected, m...)
|
||||
}
|
||||
|
||||
// Starting the plugin should succeed as we will retry to startup later
|
||||
var acc testutil.Accumulator
|
||||
require.NoError(t, model.Start(&acc))
|
||||
|
||||
// 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, container.Resume())
|
||||
require.NoError(t, model.Gather(&acc))
|
||||
defer model.Stop()
|
||||
|
||||
// Setup a AMQP producer and send messages
|
||||
client, err := newProducer(url, vhost, exchange, exchangeType, queueName, bindingKey)
|
||||
require.NoError(t, err)
|
||||
defer client.close()
|
||||
|
||||
// Write metrics
|
||||
for _, x := range metrics {
|
||||
require.NoError(t, client.write(t.Context(), exchange, queueName, []byte(x)))
|
||||
}
|
||||
|
||||
// Verify that the metrics were actually collected
|
||||
require.Eventually(t, func() bool {
|
||||
return acc.NMetrics() >= uint64(len(expected))
|
||||
}, 3*time.Second, 100*time.Millisecond)
|
||||
|
||||
client.close()
|
||||
plugin.Stop()
|
||||
testutil.RequireMetricsEqual(t, expected, acc.GetTelegrafMetrics())
|
||||
}
|
||||
|
||||
type producer struct {
|
||||
conn *amqp091.Connection
|
||||
channel *amqp091.Channel
|
||||
queue amqp091.Queue
|
||||
}
|
||||
|
||||
func newProducer(url, vhost, exchange, exchangeType, queueName, key string) (*producer, error) {
|
||||
cfg := amqp091.Config{
|
||||
Vhost: vhost,
|
||||
Properties: amqp091.NewConnectionProperties(),
|
||||
}
|
||||
cfg.Properties.SetClientConnectionName("test-producer")
|
||||
conn, err := amqp091.DialConfig(url, cfg)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
channel, err := conn.Channel()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if err := channel.ExchangeDeclare(exchange, exchangeType, true, false, false, false, nil); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
queue, err := channel.QueueDeclare(queueName, true, false, false, false, nil)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if err := channel.QueueBind(queue.Name, key, exchange, false, nil); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &producer{
|
||||
conn: conn,
|
||||
channel: channel,
|
||||
queue: queue,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (p *producer) close() {
|
||||
p.channel.Close()
|
||||
p.conn.Close()
|
||||
}
|
||||
|
||||
func (p *producer) write(testContext context.Context, exchange, key string, payload []byte) error {
|
||||
msg := amqp091.Publishing{
|
||||
DeliveryMode: amqp091.Persistent,
|
||||
Timestamp: time.Now(),
|
||||
ContentType: "text/plain",
|
||||
Body: payload,
|
||||
}
|
||||
|
||||
ctx, cancel := context.WithTimeout(testContext, 3*time.Second)
|
||||
defer cancel()
|
||||
|
||||
return p.channel.PublishWithContext(ctx, exchange, key, true, false, msg)
|
||||
}
|
95
plugins/inputs/amqp_consumer/sample.conf
Normal file
95
plugins/inputs/amqp_consumer/sample.conf
Normal file
|
@ -0,0 +1,95 @@
|
|||
# AMQP consumer plugin
|
||||
[[inputs.amqp_consumer]]
|
||||
## Brokers to consume from. If multiple brokers are specified a random broker
|
||||
## will be selected anytime a connection is established. This can be
|
||||
## helpful for load balancing when not using a dedicated load balancer.
|
||||
brokers = ["amqp://localhost:5672/influxdb"]
|
||||
|
||||
## Authentication credentials for the PLAIN auth_method.
|
||||
# username = ""
|
||||
# password = ""
|
||||
|
||||
## Name of the exchange to declare. If unset, no exchange will be declared.
|
||||
exchange = "telegraf"
|
||||
|
||||
## Exchange type; common types are "direct", "fanout", "topic", "header", "x-consistent-hash".
|
||||
# exchange_type = "topic"
|
||||
|
||||
## If true, exchange will be passively declared.
|
||||
# exchange_passive = false
|
||||
|
||||
## Exchange durability can be either "transient" or "durable".
|
||||
# exchange_durability = "durable"
|
||||
|
||||
## Additional exchange arguments.
|
||||
# exchange_arguments = { }
|
||||
# exchange_arguments = {"hash_property" = "timestamp"}
|
||||
|
||||
## AMQP queue name.
|
||||
queue = "telegraf"
|
||||
|
||||
## AMQP queue durability can be "transient" or "durable".
|
||||
queue_durability = "durable"
|
||||
|
||||
## If true, queue will be passively declared.
|
||||
# queue_passive = false
|
||||
|
||||
## Additional arguments when consuming from Queue
|
||||
# queue_consume_arguments = { }
|
||||
# queue_consume_arguments = {"x-stream-offset" = "first"}
|
||||
|
||||
## Additional queue arguments.
|
||||
# queue_arguments = { }
|
||||
# queue_arguments = {"x-max-length" = 100}
|
||||
|
||||
## A binding between the exchange and queue using this binding key is
|
||||
## created. If unset, no binding is created.
|
||||
binding_key = "#"
|
||||
|
||||
## Maximum number of messages server should give to the worker.
|
||||
# prefetch_count = 50
|
||||
|
||||
## 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
|
||||
|
||||
## Timeout for establishing the connection to a broker
|
||||
# timeout = "30s"
|
||||
|
||||
## Auth method. PLAIN and EXTERNAL are supported
|
||||
## Using EXTERNAL requires enabling the rabbitmq_auth_mechanism_ssl plugin as
|
||||
## described here: https://www.rabbitmq.com/plugins.html
|
||||
# auth_method = "PLAIN"
|
||||
|
||||
## Optional TLS Config
|
||||
# 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
|
||||
|
||||
## Content encoding for message payloads, can be set to
|
||||
## "gzip", "identity" or "auto"
|
||||
## - Use "gzip" to decode gzip
|
||||
## - Use "identity" to apply no encoding
|
||||
## - Use "auto" determine the encoding using the ContentEncoding header
|
||||
# content_encoding = "identity"
|
||||
|
||||
## Maximum size of decoded message.
|
||||
## Acceptable units are B, KiB, KB, MiB, MB...
|
||||
## Without quotes and units, interpreted as size in bytes.
|
||||
# max_decompression_size = "500MB"
|
||||
|
||||
## 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"
|
Loading…
Add table
Add a link
Reference in a new issue