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
116
plugins/outputs/influxdb_v2/README.md
Normal file
116
plugins/outputs/influxdb_v2/README.md
Normal file
|
@ -0,0 +1,116 @@
|
|||
# InfluxDB v2.x Output Plugin
|
||||
|
||||
This plugin writes metrics to a [InfluxDB v2.x][influxdb_v2] instance via HTTP.
|
||||
|
||||
⭐ Telegraf v1.8.0
|
||||
🏷️ datastore
|
||||
💻 all
|
||||
|
||||
[influxdb_v2]: https://docs.influxdata.com/influxdb/v2
|
||||
|
||||
## 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
|
||||
|
||||
## Secret-store support
|
||||
|
||||
This plugin supports secrets from secret-stores for the `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 sending metrics to InfluxDB 2.0
|
||||
[[outputs.influxdb_v2]]
|
||||
## The URLs of the InfluxDB cluster nodes.
|
||||
##
|
||||
## Multiple URLs can be specified for a single cluster, only ONE of the
|
||||
## urls will be written to each interval.
|
||||
## ex: urls = ["https://us-west-2-1.aws.cloud2.influxdata.com"]
|
||||
urls = ["http://127.0.0.1:8086"]
|
||||
|
||||
## Local address to bind when connecting to the server
|
||||
## If empty or not set, the local address is automatically chosen.
|
||||
# local_address = ""
|
||||
|
||||
## Token for authentication.
|
||||
token = ""
|
||||
|
||||
## Organization is the name of the organization you wish to write to.
|
||||
organization = ""
|
||||
|
||||
## Destination bucket to write into.
|
||||
bucket = ""
|
||||
|
||||
## The value of this tag will be used to determine the bucket. If this
|
||||
## tag is not set the 'bucket' option is used as the default.
|
||||
# bucket_tag = ""
|
||||
|
||||
## If true, the bucket tag will not be added to the metric.
|
||||
# exclude_bucket_tag = false
|
||||
|
||||
## Timeout for HTTP messages.
|
||||
# timeout = "5s"
|
||||
|
||||
## Additional HTTP headers
|
||||
# http_headers = {"X-Special-Header" = "Special-Value"}
|
||||
|
||||
## HTTP Proxy override, if unset values the standard proxy environment
|
||||
## variables are consulted to determine which proxy, if any, should be used.
|
||||
# http_proxy = "http://corporate.proxy:3128"
|
||||
|
||||
## HTTP User-Agent
|
||||
# user_agent = "telegraf"
|
||||
|
||||
## Content-Encoding for write request body, can be set to "gzip" to
|
||||
## compress body or "identity" to apply no encoding.
|
||||
# content_encoding = "gzip"
|
||||
|
||||
## Enable or disable uint support for writing uints influxdb 2.0.
|
||||
# influx_uint_support = false
|
||||
|
||||
## When true, Telegraf will omit the timestamp on data to allow InfluxDB
|
||||
## to set the timestamp of the data during ingestion. This is generally NOT
|
||||
## what you want as it can lead to data points captured at different times
|
||||
## getting omitted due to similar data.
|
||||
# influx_omit_timestamp = false
|
||||
|
||||
## HTTP/2 Timeouts
|
||||
## The following values control the HTTP/2 client's timeouts. These settings
|
||||
## are generally not required unless a user is seeing issues with client
|
||||
## disconnects. If a user does see issues, then it is suggested to set these
|
||||
## values to "15s" for ping timeout and "30s" for read idle timeout and
|
||||
## retry.
|
||||
##
|
||||
## Note that the timer for read_idle_timeout begins at the end of the last
|
||||
## successful write and not at the beginning of the next write.
|
||||
# ping_timeout = "0s"
|
||||
# read_idle_timeout = "0s"
|
||||
|
||||
## Optional TLS Config for use on HTTP connections.
|
||||
# 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
|
||||
|
||||
## Rate limits for sending data (disabled by default)
|
||||
## Available, uncompressed payload size e.g. "5MB"
|
||||
# rate_limit = "unlimited"
|
||||
## Fixed time-window for the available payload size e.g. "5m"
|
||||
# rate_limit_period = "0s"
|
||||
```
|
||||
|
||||
## Metrics
|
||||
|
||||
Reference the [influx serializer][] for details about metric production.
|
||||
|
||||
[influx serializer]: /plugins/serializers/influx/README.md#Metrics
|
434
plugins/outputs/influxdb_v2/http.go
Normal file
434
plugins/outputs/influxdb_v2/http.go
Normal file
|
@ -0,0 +1,434 @@
|
|||
package influxdb_v2
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"crypto/tls"
|
||||
"encoding/json"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
"math"
|
||||
"net"
|
||||
"net/http"
|
||||
"net/url"
|
||||
"path"
|
||||
"strconv"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"golang.org/x/net/http2"
|
||||
|
||||
"github.com/influxdata/telegraf"
|
||||
"github.com/influxdata/telegraf/config"
|
||||
"github.com/influxdata/telegraf/internal"
|
||||
"github.com/influxdata/telegraf/plugins/common/ratelimiter"
|
||||
)
|
||||
|
||||
type APIError struct {
|
||||
Err error
|
||||
StatusCode int
|
||||
Retryable bool
|
||||
}
|
||||
|
||||
func (e APIError) Error() string {
|
||||
return e.Err.Error()
|
||||
}
|
||||
|
||||
func (e APIError) Unwrap() error {
|
||||
return e.Err
|
||||
}
|
||||
|
||||
const (
|
||||
defaultMaxWaitSeconds = 60
|
||||
defaultMaxWaitRetryAfterSeconds = 10 * 60
|
||||
)
|
||||
|
||||
type httpClient struct {
|
||||
url *url.URL
|
||||
localAddr *net.TCPAddr
|
||||
token config.Secret
|
||||
organization string
|
||||
bucket string
|
||||
bucketTag string
|
||||
excludeBucketTag bool
|
||||
timeout time.Duration
|
||||
headers map[string]string
|
||||
proxy *url.URL
|
||||
userAgent string
|
||||
contentEncoding string
|
||||
pingTimeout config.Duration
|
||||
readIdleTimeout config.Duration
|
||||
tlsConfig *tls.Config
|
||||
encoder internal.ContentEncoder
|
||||
serializer ratelimiter.Serializer
|
||||
rateLimiter *ratelimiter.RateLimiter
|
||||
client *http.Client
|
||||
params url.Values
|
||||
retryTime time.Time
|
||||
retryCount int
|
||||
log telegraf.Logger
|
||||
}
|
||||
|
||||
func (c *httpClient) Init() error {
|
||||
if c.headers == nil {
|
||||
c.headers = make(map[string]string, 1)
|
||||
}
|
||||
|
||||
if _, ok := c.headers["User-Agent"]; !ok {
|
||||
c.headers["User-Agent"] = c.userAgent
|
||||
}
|
||||
|
||||
var proxy func(*http.Request) (*url.URL, error)
|
||||
if c.proxy != nil {
|
||||
proxy = http.ProxyURL(c.proxy)
|
||||
} else {
|
||||
proxy = http.ProxyFromEnvironment
|
||||
}
|
||||
|
||||
var transport *http.Transport
|
||||
switch c.url.Scheme {
|
||||
case "http", "https":
|
||||
var dialerFunc func(ctx context.Context, network, addr string) (net.Conn, error)
|
||||
if c.localAddr != nil {
|
||||
dialer := &net.Dialer{LocalAddr: c.localAddr}
|
||||
dialerFunc = dialer.DialContext
|
||||
}
|
||||
transport = &http.Transport{
|
||||
Proxy: proxy,
|
||||
TLSClientConfig: c.tlsConfig,
|
||||
DialContext: dialerFunc,
|
||||
}
|
||||
if c.readIdleTimeout != 0 || c.pingTimeout != 0 {
|
||||
http2Trans, err := http2.ConfigureTransports(transport)
|
||||
if err == nil {
|
||||
http2Trans.ReadIdleTimeout = time.Duration(c.readIdleTimeout)
|
||||
http2Trans.PingTimeout = time.Duration(c.pingTimeout)
|
||||
}
|
||||
}
|
||||
case "unix":
|
||||
transport = &http.Transport{
|
||||
Dial: func(_, _ string) (net.Conn, error) {
|
||||
return net.DialTimeout(
|
||||
c.url.Scheme,
|
||||
c.url.Path,
|
||||
c.timeout,
|
||||
)
|
||||
},
|
||||
}
|
||||
default:
|
||||
return fmt.Errorf("unsupported scheme %q", c.url.Scheme)
|
||||
}
|
||||
|
||||
preppedURL, params, err := prepareWriteURL(*c.url, c.organization)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
c.url = preppedURL
|
||||
c.client = &http.Client{
|
||||
Timeout: c.timeout,
|
||||
Transport: transport,
|
||||
}
|
||||
c.params = params
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
type genericRespError struct {
|
||||
Code string
|
||||
Message string
|
||||
Line *int32
|
||||
MaxLength *int32
|
||||
}
|
||||
|
||||
func (g genericRespError) Error() string {
|
||||
errString := fmt.Sprintf("%s: %s", g.Code, g.Message)
|
||||
if g.Line != nil {
|
||||
return fmt.Sprintf("%s - line[%d]", errString, g.Line)
|
||||
} else if g.MaxLength != nil {
|
||||
return fmt.Sprintf("%s - maxlen[%d]", errString, g.MaxLength)
|
||||
}
|
||||
return errString
|
||||
}
|
||||
|
||||
func (c *httpClient) Write(ctx context.Context, metrics []telegraf.Metric) error {
|
||||
if c.retryTime.After(time.Now()) {
|
||||
return errors.New("retry time has not elapsed")
|
||||
}
|
||||
|
||||
batches := make(map[string][]telegraf.Metric)
|
||||
batchIndices := make(map[string][]int)
|
||||
if c.bucketTag == "" {
|
||||
batches[c.bucket] = metrics
|
||||
batchIndices[c.bucket] = make([]int, len(metrics))
|
||||
for i := range metrics {
|
||||
batchIndices[c.bucket][i] = i
|
||||
}
|
||||
} else {
|
||||
for i, metric := range metrics {
|
||||
bucket, ok := metric.GetTag(c.bucketTag)
|
||||
if !ok {
|
||||
bucket = c.bucket
|
||||
} else if c.excludeBucketTag {
|
||||
// Avoid modifying the metric if we do remove the tag
|
||||
metric = metric.Copy()
|
||||
metric.Accept()
|
||||
metric.RemoveTag(c.bucketTag)
|
||||
}
|
||||
|
||||
batches[bucket] = append(batches[bucket], metric)
|
||||
batchIndices[bucket] = append(batchIndices[bucket], i)
|
||||
}
|
||||
}
|
||||
|
||||
var wErr internal.PartialWriteError
|
||||
for bucket, batch := range batches {
|
||||
err := c.writeBatch(ctx, bucket, batch)
|
||||
if err == nil {
|
||||
wErr.MetricsAccept = append(wErr.MetricsAccept, batchIndices[bucket]...)
|
||||
continue
|
||||
}
|
||||
|
||||
// Check if the request was too large and split it
|
||||
var apiErr *APIError
|
||||
if errors.As(err, &apiErr) {
|
||||
if apiErr.StatusCode == http.StatusRequestEntityTooLarge {
|
||||
// TODO: Need a testcase to verify rejected metrics are not retried...
|
||||
return c.splitAndWriteBatch(ctx, c.bucket, metrics)
|
||||
}
|
||||
wErr.Err = err
|
||||
if !apiErr.Retryable {
|
||||
wErr.MetricsReject = append(wErr.MetricsReject, batchIndices[bucket]...)
|
||||
}
|
||||
// TODO: Clarify if we should continue here to try the remaining buckets?
|
||||
return &wErr
|
||||
}
|
||||
|
||||
// Check if we got a write error and if so, translate the returned
|
||||
// metric indices to return the original indices in case of bucketing
|
||||
var writeErr *internal.PartialWriteError
|
||||
if errors.As(err, &writeErr) {
|
||||
wErr.Err = writeErr.Err
|
||||
for _, idx := range writeErr.MetricsAccept {
|
||||
wErr.MetricsAccept = append(wErr.MetricsAccept, batchIndices[bucket][idx])
|
||||
}
|
||||
for _, idx := range writeErr.MetricsReject {
|
||||
wErr.MetricsReject = append(wErr.MetricsReject, batchIndices[bucket][idx])
|
||||
}
|
||||
if !errors.Is(writeErr.Err, internal.ErrSizeLimitReached) {
|
||||
continue
|
||||
}
|
||||
return &wErr
|
||||
}
|
||||
|
||||
// Return the error without special treatment
|
||||
wErr.Err = err
|
||||
return &wErr
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *httpClient) splitAndWriteBatch(ctx context.Context, bucket string, metrics []telegraf.Metric) error {
|
||||
c.log.Warnf("Retrying write after splitting metric payload in half to reduce batch size")
|
||||
midpoint := len(metrics) / 2
|
||||
|
||||
if err := c.writeBatch(ctx, bucket, metrics[:midpoint]); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return c.writeBatch(ctx, bucket, metrics[midpoint:])
|
||||
}
|
||||
|
||||
func (c *httpClient) writeBatch(ctx context.Context, bucket string, metrics []telegraf.Metric) error {
|
||||
// Get the current limit for the outbound data
|
||||
ratets := time.Now()
|
||||
limit := c.rateLimiter.Remaining(ratets)
|
||||
|
||||
// Serialize the metrics with the remaining limit, exit early if nothing was serialized
|
||||
body, werr := c.serializer.SerializeBatch(metrics, limit)
|
||||
if werr != nil && !errors.Is(werr, internal.ErrSizeLimitReached) || len(body) == 0 {
|
||||
return werr
|
||||
}
|
||||
used := int64(len(body))
|
||||
|
||||
// Encode the content if requested
|
||||
if c.encoder != nil {
|
||||
var err error
|
||||
if body, err = c.encoder.Encode(body); err != nil {
|
||||
return fmt.Errorf("encoding failed: %w", err)
|
||||
}
|
||||
}
|
||||
|
||||
// Setup the request
|
||||
address := makeWriteURL(*c.url, c.params, bucket)
|
||||
req, err := http.NewRequest("POST", address, io.NopCloser(bytes.NewBuffer(body)))
|
||||
if err != nil {
|
||||
return fmt.Errorf("creating request failed: %w", err)
|
||||
}
|
||||
if c.encoder != nil {
|
||||
req.Header.Set("Content-Encoding", c.contentEncoding)
|
||||
}
|
||||
req.Header.Set("Content-Type", "text/plain; charset=utf-8")
|
||||
|
||||
// Set authorization
|
||||
token, err := c.token.Get()
|
||||
if err != nil {
|
||||
return fmt.Errorf("getting token failed: %w", err)
|
||||
}
|
||||
req.Header.Set("Authorization", "Token "+token.String())
|
||||
token.Destroy()
|
||||
|
||||
c.addHeaders(req)
|
||||
|
||||
// Execute the request
|
||||
c.rateLimiter.Accept(ratets, used)
|
||||
resp, err := c.client.Do(req.WithContext(ctx))
|
||||
if err != nil {
|
||||
internal.OnClientError(c.client, err)
|
||||
return err
|
||||
}
|
||||
defer resp.Body.Close()
|
||||
|
||||
// Check for success
|
||||
switch resp.StatusCode {
|
||||
case
|
||||
// this is the expected response:
|
||||
http.StatusNoContent,
|
||||
// but if we get these we should still accept it as delivered:
|
||||
http.StatusOK,
|
||||
http.StatusCreated,
|
||||
http.StatusAccepted,
|
||||
http.StatusPartialContent,
|
||||
http.StatusMultiStatus,
|
||||
http.StatusAlreadyReported:
|
||||
c.retryCount = 0
|
||||
return werr
|
||||
}
|
||||
|
||||
// We got an error and now try to decode further
|
||||
var desc string
|
||||
writeResp := &genericRespError{}
|
||||
if json.NewDecoder(resp.Body).Decode(writeResp) == nil {
|
||||
desc = ": " + writeResp.Error()
|
||||
}
|
||||
|
||||
switch resp.StatusCode {
|
||||
// request was too large, send back to try again
|
||||
case http.StatusRequestEntityTooLarge:
|
||||
c.log.Errorf("Failed to write metric to %s, request was too large (413)", bucket)
|
||||
return &APIError{
|
||||
Err: fmt.Errorf("%s: %s", resp.Status, desc),
|
||||
StatusCode: resp.StatusCode,
|
||||
}
|
||||
case
|
||||
// request was malformed:
|
||||
http.StatusBadRequest,
|
||||
// request was received but server refused to process it due to a semantic problem with the request.
|
||||
// for example, submitting metrics outside the retention period.
|
||||
http.StatusUnprocessableEntity,
|
||||
http.StatusNotAcceptable:
|
||||
|
||||
// Clients should *not* repeat the request and the metrics should be rejected.
|
||||
return &APIError{
|
||||
Err: fmt.Errorf("failed to write metric to %s (will be dropped: %s)%s", bucket, resp.Status, desc),
|
||||
StatusCode: resp.StatusCode,
|
||||
}
|
||||
case http.StatusUnauthorized, http.StatusForbidden:
|
||||
return fmt.Errorf("failed to write metric to %s (%s)%s", bucket, resp.Status, desc)
|
||||
case http.StatusTooManyRequests,
|
||||
http.StatusServiceUnavailable,
|
||||
http.StatusBadGateway,
|
||||
http.StatusGatewayTimeout:
|
||||
// ^ these handle the cases where the server is likely overloaded, and may not be able to say so.
|
||||
c.retryCount++
|
||||
retryDuration := c.getRetryDuration(resp.Header)
|
||||
c.retryTime = time.Now().Add(retryDuration)
|
||||
c.log.Warnf("Failed to write to %s; will retry in %s. (%s)\n", bucket, retryDuration, resp.Status)
|
||||
return fmt.Errorf("waiting %s for server (%s) before sending metric again", retryDuration, bucket)
|
||||
}
|
||||
|
||||
// if it's any other 4xx code, the client should not retry as it's the client's mistake.
|
||||
// retrying will not make the request magically work.
|
||||
if len(resp.Status) > 0 && resp.Status[0] == '4' {
|
||||
return &APIError{
|
||||
Err: fmt.Errorf("failed to write metric to %s (will be dropped: %s)%s", bucket, resp.Status, desc),
|
||||
StatusCode: resp.StatusCode,
|
||||
}
|
||||
}
|
||||
|
||||
// This is only until platform spec is fully implemented. As of the
|
||||
// time of writing, there is no error body returned.
|
||||
if xErr := resp.Header.Get("X-Influx-Error"); xErr != "" {
|
||||
desc = fmt.Sprintf(": %s; %s", desc, xErr)
|
||||
}
|
||||
|
||||
return &APIError{
|
||||
Err: fmt.Errorf("failed to write metric to bucket %q: %s%s", bucket, resp.Status, desc),
|
||||
StatusCode: resp.StatusCode,
|
||||
Retryable: true,
|
||||
}
|
||||
}
|
||||
|
||||
// retryDuration takes the longer of the Retry-After header and our own back-off calculation
|
||||
func (c *httpClient) getRetryDuration(headers http.Header) time.Duration {
|
||||
// basic exponential backoff (x^2)/40 (denominator to widen the slope)
|
||||
// at 40 denominator, it'll take 49 retries to hit the max defaultMaxWait of 60s
|
||||
backoff := math.Pow(float64(c.retryCount), 2) / 40
|
||||
backoff = math.Min(backoff, defaultMaxWaitSeconds)
|
||||
|
||||
// get any value from the header, if available
|
||||
retryAfterHeader := float64(0)
|
||||
retryAfterHeaderString := headers.Get("Retry-After")
|
||||
if len(retryAfterHeaderString) > 0 {
|
||||
var err error
|
||||
retryAfterHeader, err = strconv.ParseFloat(retryAfterHeaderString, 64)
|
||||
if err != nil {
|
||||
// there was a value but we couldn't parse it? guess minimum 10 sec
|
||||
retryAfterHeader = 10
|
||||
}
|
||||
// protect against excessively large retry-after
|
||||
retryAfterHeader = math.Min(retryAfterHeader, defaultMaxWaitRetryAfterSeconds)
|
||||
}
|
||||
// take the highest value of backoff and retry-after.
|
||||
retry := math.Max(backoff, retryAfterHeader)
|
||||
return time.Duration(retry*1000) * time.Millisecond
|
||||
}
|
||||
|
||||
func (c *httpClient) addHeaders(req *http.Request) {
|
||||
for header, value := range c.headers {
|
||||
if strings.EqualFold(header, "host") {
|
||||
req.Host = value
|
||||
} else {
|
||||
req.Header.Set(header, value)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func makeWriteURL(loc url.URL, params url.Values, bucket string) string {
|
||||
params.Set("bucket", bucket)
|
||||
loc.RawQuery = params.Encode()
|
||||
return loc.String()
|
||||
}
|
||||
|
||||
func prepareWriteURL(loc url.URL, org string) (*url.URL, url.Values, error) {
|
||||
switch loc.Scheme {
|
||||
case "unix":
|
||||
loc.Scheme = "http"
|
||||
loc.Host = "127.0.0.1"
|
||||
loc.Path = "/api/v2/write"
|
||||
case "http", "https":
|
||||
loc.Path = path.Join(loc.Path, "/api/v2/write")
|
||||
default:
|
||||
return nil, nil, fmt.Errorf("unsupported scheme: %q", loc.Scheme)
|
||||
}
|
||||
|
||||
params := loc.Query()
|
||||
params.Set("org", org)
|
||||
|
||||
return &loc, params, nil
|
||||
}
|
||||
|
||||
func (c *httpClient) Close() {
|
||||
c.client.CloseIdleConnections()
|
||||
}
|
269
plugins/outputs/influxdb_v2/http_test.go
Normal file
269
plugins/outputs/influxdb_v2/http_test.go
Normal file
|
@ -0,0 +1,269 @@
|
|||
package influxdb_v2
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"net/http"
|
||||
"net/url"
|
||||
"path"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
"github.com/influxdata/telegraf/config"
|
||||
)
|
||||
|
||||
func TestHTTPClientInit(t *testing.T) {
|
||||
tests := []struct {
|
||||
name string
|
||||
addr string
|
||||
client *httpClient
|
||||
}{
|
||||
{
|
||||
name: "unix socket",
|
||||
addr: "unix://var/run/influxd.sock",
|
||||
client: &httpClient{},
|
||||
},
|
||||
{
|
||||
name: "unix socket with timeouts",
|
||||
addr: "unix://var/run/influxd.sock",
|
||||
client: &httpClient{
|
||||
pingTimeout: config.Duration(15 * time.Second),
|
||||
readIdleTimeout: config.Duration(30 * time.Second),
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
u, err := url.Parse(tt.addr)
|
||||
require.NoError(t, err)
|
||||
tt.client.url = u
|
||||
|
||||
require.NoError(t, tt.client.Init())
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestHTTPClientInitFail(t *testing.T) {
|
||||
tests := []struct {
|
||||
name string
|
||||
addr string
|
||||
client *httpClient
|
||||
}{
|
||||
{
|
||||
name: "udp unsupported",
|
||||
addr: "udp://localhost:9999",
|
||||
client: &httpClient{},
|
||||
},
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
u, err := url.Parse(tt.addr)
|
||||
require.NoError(t, err)
|
||||
tt.client.url = u
|
||||
|
||||
require.Error(t, tt.client.Init())
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestMakeWriteURL(t *testing.T) {
|
||||
tests := []struct {
|
||||
name string
|
||||
addr string
|
||||
expected string
|
||||
bucket string
|
||||
org string
|
||||
}{
|
||||
{
|
||||
name: "http default",
|
||||
addr: "http://localhost:9999",
|
||||
expected: "http://localhost:9999/api/v2/write?bucket=telegraf0&org=influx0",
|
||||
bucket: "telegraf0",
|
||||
org: "influx0",
|
||||
},
|
||||
{
|
||||
name: "http with param",
|
||||
addr: "http://localhost:9999?id=abc",
|
||||
expected: "http://localhost:9999/api/v2/write?bucket=telegraf1&id=abc&org=influx1",
|
||||
bucket: "telegraf1",
|
||||
org: "influx1",
|
||||
},
|
||||
{
|
||||
name: "unix socket default",
|
||||
addr: "unix://var/run/influxd.sock",
|
||||
expected: "http://127.0.0.1/api/v2/write?bucket=telegraf2&org=influx2",
|
||||
bucket: "telegraf2",
|
||||
org: "influx2",
|
||||
},
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
u, err := url.Parse(tt.addr)
|
||||
require.NoError(t, err)
|
||||
|
||||
preppedURL, params, err := prepareWriteURL(*u, tt.org)
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, tt.expected, makeWriteURL(*preppedURL, params, tt.bucket))
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestMakeWriteURLFail(t *testing.T) {
|
||||
tests := []struct {
|
||||
name string
|
||||
addr string
|
||||
expected string
|
||||
bucket string
|
||||
org string
|
||||
}{
|
||||
{
|
||||
name: "default values",
|
||||
addr: "udp://localhost:9999",
|
||||
},
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
u, err := url.Parse(tt.addr)
|
||||
require.NoError(t, err)
|
||||
|
||||
_, _, err = prepareWriteURL(*u, tt.org)
|
||||
require.Error(t, err)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestExponentialBackoffCalculation(t *testing.T) {
|
||||
c := &httpClient{}
|
||||
tests := []struct {
|
||||
retryCount int
|
||||
expected time.Duration
|
||||
}{
|
||||
{retryCount: 0, expected: 0},
|
||||
{retryCount: 1, expected: 25 * time.Millisecond},
|
||||
{retryCount: 5, expected: 625 * time.Millisecond},
|
||||
{retryCount: 10, expected: 2500 * time.Millisecond},
|
||||
{retryCount: 30, expected: 22500 * time.Millisecond},
|
||||
{retryCount: 40, expected: 40 * time.Second},
|
||||
{retryCount: 50, expected: 60 * time.Second}, // max hit
|
||||
{retryCount: 100, expected: 60 * time.Second},
|
||||
{retryCount: 1000, expected: 60 * time.Second},
|
||||
}
|
||||
for _, test := range tests {
|
||||
t.Run(fmt.Sprintf("%d_retries", test.retryCount), func(t *testing.T) {
|
||||
c.retryCount = test.retryCount
|
||||
require.EqualValues(t, test.expected, c.getRetryDuration(http.Header{}))
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestExponentialBackoffCalculationWithRetryAfter(t *testing.T) {
|
||||
c := &httpClient{}
|
||||
tests := []struct {
|
||||
retryCount int
|
||||
retryAfter string
|
||||
expected time.Duration
|
||||
}{
|
||||
{retryCount: 0, retryAfter: "0", expected: 0},
|
||||
{retryCount: 0, retryAfter: "10", expected: 10 * time.Second},
|
||||
{retryCount: 0, retryAfter: "60", expected: 60 * time.Second},
|
||||
{retryCount: 0, retryAfter: "600", expected: 600 * time.Second},
|
||||
{retryCount: 0, retryAfter: "601", expected: 600 * time.Second}, // max hit
|
||||
{retryCount: 40, retryAfter: "39", expected: 40 * time.Second}, // retryCount wins
|
||||
{retryCount: 40, retryAfter: "41", expected: 41 * time.Second}, // retryAfter wins
|
||||
{retryCount: 100, retryAfter: "100", expected: 100 * time.Second},
|
||||
}
|
||||
for _, test := range tests {
|
||||
t.Run(fmt.Sprintf("%d_retries", test.retryCount), func(t *testing.T) {
|
||||
c.retryCount = test.retryCount
|
||||
hdr := http.Header{}
|
||||
hdr.Add("Retry-After", test.retryAfter)
|
||||
require.EqualValues(t, test.expected, c.getRetryDuration(hdr))
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestHeadersDoNotOverrideConfig(t *testing.T) {
|
||||
testURL, err := url.Parse("https://localhost:8181")
|
||||
require.NoError(t, err)
|
||||
c := &httpClient{
|
||||
headers: map[string]string{
|
||||
"Authorization": "Bearer foo",
|
||||
"User-Agent": "foo",
|
||||
},
|
||||
// URL to make Init() happy
|
||||
url: testURL,
|
||||
}
|
||||
require.NoError(t, c.Init())
|
||||
require.Equal(t, "Bearer foo", c.headers["Authorization"])
|
||||
require.Equal(t, "foo", c.headers["User-Agent"])
|
||||
}
|
||||
|
||||
// goos: linux
|
||||
// goarch: amd64
|
||||
// pkg: github.com/influxdata/telegraf/plugins/outputs/influxdb_v2
|
||||
// cpu: 11th Gen Intel(R) Core(TM) i7-11850H @ 2.50GHz
|
||||
// BenchmarkOldMakeWriteURL
|
||||
// BenchmarkOldMakeWriteURL-16 1556631 683.2 ns/op 424 B/op 14 allocs/op
|
||||
// PASS
|
||||
// ok github.com/influxdata/telegraf/plugins/outputs/influxdb_v2 1.851s
|
||||
func BenchmarkOldMakeWriteURL(b *testing.B) {
|
||||
org := "org"
|
||||
|
||||
u, err := url.Parse("http://localhost:8086")
|
||||
require.NoError(b, err)
|
||||
loc, _, err := prepareWriteURL(*u, org)
|
||||
require.NoError(b, err)
|
||||
|
||||
b.ReportAllocs()
|
||||
for n := 0; n < b.N; n++ {
|
||||
//nolint:errcheck // Skip error for benchmarking
|
||||
oldMakeWriteURL(*loc)
|
||||
}
|
||||
}
|
||||
|
||||
// goos: linux
|
||||
// goarch: amd64
|
||||
// pkg: github.com/influxdata/telegraf/plugins/outputs/influxdb_v2
|
||||
// cpu: 11th Gen Intel(R) Core(TM) i7-11850H @ 2.50GHz
|
||||
// BenchmarkNewMakeWriteURL
|
||||
// BenchmarkNewMakeWriteURL-16 2084415 496.5 ns/op 280 B/op 9 allocs/op
|
||||
// PASS
|
||||
// ok github.com/influxdata/telegraf/plugins/outputs/influxdb_v2 1.626s
|
||||
func BenchmarkNewMakeWriteURL(b *testing.B) {
|
||||
bucket := "bkt"
|
||||
org := "org"
|
||||
|
||||
u, err := url.Parse("http://localhost:8086")
|
||||
require.NoError(b, err)
|
||||
loc, params, err := prepareWriteURL(*u, org)
|
||||
require.NoError(b, err)
|
||||
|
||||
b.ReportAllocs()
|
||||
for n := 0; n < b.N; n++ {
|
||||
makeWriteURL(*loc, params, bucket)
|
||||
}
|
||||
}
|
||||
|
||||
func oldMakeWriteURL(loc url.URL) (string, error) {
|
||||
params := url.Values{}
|
||||
params.Set("bucket", "bkt")
|
||||
params.Set("org", "org")
|
||||
|
||||
switch loc.Scheme {
|
||||
case "unix":
|
||||
loc.Scheme = "http"
|
||||
loc.Host = "127.0.0.1"
|
||||
loc.Path = "/api/v2/write"
|
||||
case "http", "https":
|
||||
loc.Path = path.Join(loc.Path, "/api/v2/write")
|
||||
default:
|
||||
return "", fmt.Errorf("unsupported scheme: %q", loc.Scheme)
|
||||
}
|
||||
loc.RawQuery = params.Encode()
|
||||
return loc.String(), nil
|
||||
}
|
221
plugins/outputs/influxdb_v2/influxdb_v2.go
Normal file
221
plugins/outputs/influxdb_v2/influxdb_v2.go
Normal file
|
@ -0,0 +1,221 @@
|
|||
//go:generate ../../../tools/readme_config_includer/generator
|
||||
package influxdb_v2
|
||||
|
||||
import (
|
||||
"context"
|
||||
"crypto/tls"
|
||||
_ "embed"
|
||||
"errors"
|
||||
"fmt"
|
||||
"math/rand"
|
||||
"net"
|
||||
"net/url"
|
||||
"strconv"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/influxdata/telegraf"
|
||||
"github.com/influxdata/telegraf/config"
|
||||
"github.com/influxdata/telegraf/internal"
|
||||
"github.com/influxdata/telegraf/plugins/common/ratelimiter"
|
||||
commontls "github.com/influxdata/telegraf/plugins/common/tls"
|
||||
"github.com/influxdata/telegraf/plugins/outputs"
|
||||
"github.com/influxdata/telegraf/plugins/serializers/influx"
|
||||
)
|
||||
|
||||
//go:embed sample.conf
|
||||
var sampleConfig string
|
||||
|
||||
type InfluxDB struct {
|
||||
URLs []string `toml:"urls"`
|
||||
LocalAddr string `toml:"local_address"`
|
||||
Token config.Secret `toml:"token"`
|
||||
Organization string `toml:"organization"`
|
||||
Bucket string `toml:"bucket"`
|
||||
BucketTag string `toml:"bucket_tag"`
|
||||
ExcludeBucketTag bool `toml:"exclude_bucket_tag"`
|
||||
Timeout config.Duration `toml:"timeout"`
|
||||
HTTPHeaders map[string]string `toml:"http_headers"`
|
||||
HTTPProxy string `toml:"http_proxy"`
|
||||
UserAgent string `toml:"user_agent"`
|
||||
ContentEncoding string `toml:"content_encoding"`
|
||||
UintSupport bool `toml:"influx_uint_support"`
|
||||
OmitTimestamp bool `toml:"influx_omit_timestamp"`
|
||||
PingTimeout config.Duration `toml:"ping_timeout"`
|
||||
ReadIdleTimeout config.Duration `toml:"read_idle_timeout"`
|
||||
Log telegraf.Logger `toml:"-"`
|
||||
commontls.ClientConfig
|
||||
ratelimiter.RateLimitConfig
|
||||
|
||||
clients []*httpClient
|
||||
encoder internal.ContentEncoder
|
||||
serializer ratelimiter.Serializer
|
||||
tlsCfg *tls.Config
|
||||
}
|
||||
|
||||
func (*InfluxDB) SampleConfig() string {
|
||||
return sampleConfig
|
||||
}
|
||||
|
||||
func (i *InfluxDB) Init() error {
|
||||
// Set defaults
|
||||
if i.UserAgent == "" {
|
||||
i.UserAgent = internal.ProductToken()
|
||||
}
|
||||
|
||||
if len(i.URLs) == 0 {
|
||||
i.URLs = append(i.URLs, "http://localhost:8086")
|
||||
}
|
||||
|
||||
// Init encoding if configured
|
||||
switch i.ContentEncoding {
|
||||
case "", "gzip":
|
||||
i.ContentEncoding = "gzip"
|
||||
enc, err := internal.NewGzipEncoder()
|
||||
if err != nil {
|
||||
return fmt.Errorf("setting up gzip encoder failed: %w", err)
|
||||
}
|
||||
i.encoder = enc
|
||||
case "identity":
|
||||
default:
|
||||
return fmt.Errorf("invalid content encoding %q", i.ContentEncoding)
|
||||
}
|
||||
|
||||
// Setup the limited serializer
|
||||
serializer := &influx.Serializer{
|
||||
UintSupport: i.UintSupport,
|
||||
OmitTimestamp: i.OmitTimestamp,
|
||||
}
|
||||
if err := serializer.Init(); err != nil {
|
||||
return fmt.Errorf("setting up serializer failed: %w", err)
|
||||
}
|
||||
i.serializer = ratelimiter.NewIndividualSerializer(serializer)
|
||||
|
||||
// Setup the client config
|
||||
tlsCfg, err := i.ClientConfig.TLSConfig()
|
||||
if err != nil {
|
||||
return fmt.Errorf("setting up TLS failed: %w", err)
|
||||
}
|
||||
i.tlsCfg = tlsCfg
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (i *InfluxDB) Connect() error {
|
||||
for _, u := range i.URLs {
|
||||
parts, err := url.Parse(u)
|
||||
if err != nil {
|
||||
return fmt.Errorf("error parsing url [%q]: %w", u, err)
|
||||
}
|
||||
|
||||
var proxy *url.URL
|
||||
if len(i.HTTPProxy) > 0 {
|
||||
proxy, err = url.Parse(i.HTTPProxy)
|
||||
if err != nil {
|
||||
return fmt.Errorf("error parsing proxy_url [%s]: %w", i.HTTPProxy, err)
|
||||
}
|
||||
}
|
||||
|
||||
var localAddr *net.TCPAddr
|
||||
if i.LocalAddr != "" {
|
||||
// Resolve the local address into IP address and the given port if any
|
||||
addr, sPort, err := net.SplitHostPort(i.LocalAddr)
|
||||
if err != nil {
|
||||
if !strings.Contains(err.Error(), "missing port") {
|
||||
return fmt.Errorf("invalid local address: %w", err)
|
||||
}
|
||||
addr = i.LocalAddr
|
||||
}
|
||||
local, err := net.ResolveIPAddr("ip", addr)
|
||||
if err != nil {
|
||||
return fmt.Errorf("cannot resolve local address: %w", err)
|
||||
}
|
||||
|
||||
var port int
|
||||
if sPort != "" {
|
||||
p, err := strconv.ParseUint(sPort, 10, 16)
|
||||
if err != nil {
|
||||
return fmt.Errorf("invalid port: %w", err)
|
||||
}
|
||||
port = int(p)
|
||||
}
|
||||
|
||||
localAddr = &net.TCPAddr{IP: local.IP, Port: port, Zone: local.Zone}
|
||||
}
|
||||
|
||||
switch parts.Scheme {
|
||||
case "http", "https", "unix":
|
||||
limiter, err := i.RateLimitConfig.CreateRateLimiter()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
c := &httpClient{
|
||||
url: parts,
|
||||
localAddr: localAddr,
|
||||
token: i.Token,
|
||||
organization: i.Organization,
|
||||
bucket: i.Bucket,
|
||||
bucketTag: i.BucketTag,
|
||||
excludeBucketTag: i.ExcludeBucketTag,
|
||||
timeout: time.Duration(i.Timeout),
|
||||
headers: i.HTTPHeaders,
|
||||
proxy: proxy,
|
||||
userAgent: i.UserAgent,
|
||||
contentEncoding: i.ContentEncoding,
|
||||
tlsConfig: i.tlsCfg,
|
||||
pingTimeout: i.PingTimeout,
|
||||
readIdleTimeout: i.ReadIdleTimeout,
|
||||
encoder: i.encoder,
|
||||
rateLimiter: limiter,
|
||||
serializer: i.serializer,
|
||||
log: i.Log,
|
||||
}
|
||||
|
||||
if err := c.Init(); err != nil {
|
||||
return fmt.Errorf("error creating HTTP client [%s]: %w", parts, err)
|
||||
}
|
||||
|
||||
i.clients = append(i.clients, c)
|
||||
default:
|
||||
return fmt.Errorf("unsupported scheme [%q]: %q", u, parts.Scheme)
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (i *InfluxDB) Close() error {
|
||||
for _, client := range i.clients {
|
||||
client.Close()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// Write sends metrics to one of the configured servers, logging each
|
||||
// unsuccessful. If all servers fail, return an error.
|
||||
func (i *InfluxDB) Write(metrics []telegraf.Metric) error {
|
||||
ctx := context.Background()
|
||||
|
||||
for _, n := range rand.Perm(len(i.clients)) {
|
||||
client := i.clients[n]
|
||||
if err := client.Write(ctx, metrics); err != nil {
|
||||
i.Log.Errorf("When writing to [%s]: %v", client.url, err)
|
||||
var werr *internal.PartialWriteError
|
||||
if errors.As(err, &werr) || errors.Is(err, internal.ErrSizeLimitReached) {
|
||||
return err
|
||||
}
|
||||
continue
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
return errors.New("failed to send metrics to any configured server(s)")
|
||||
}
|
||||
|
||||
func init() {
|
||||
outputs.Add("influxdb_v2", func() telegraf.Output {
|
||||
return &InfluxDB{
|
||||
Timeout: config.Duration(time.Second * 5),
|
||||
}
|
||||
})
|
||||
}
|
1172
plugins/outputs/influxdb_v2/influxdb_v2_test.go
Normal file
1172
plugins/outputs/influxdb_v2/influxdb_v2_test.go
Normal file
File diff suppressed because it is too large
Load diff
79
plugins/outputs/influxdb_v2/sample.conf
Normal file
79
plugins/outputs/influxdb_v2/sample.conf
Normal file
|
@ -0,0 +1,79 @@
|
|||
# Configuration for sending metrics to InfluxDB 2.0
|
||||
[[outputs.influxdb_v2]]
|
||||
## The URLs of the InfluxDB cluster nodes.
|
||||
##
|
||||
## Multiple URLs can be specified for a single cluster, only ONE of the
|
||||
## urls will be written to each interval.
|
||||
## ex: urls = ["https://us-west-2-1.aws.cloud2.influxdata.com"]
|
||||
urls = ["http://127.0.0.1:8086"]
|
||||
|
||||
## Local address to bind when connecting to the server
|
||||
## If empty or not set, the local address is automatically chosen.
|
||||
# local_address = ""
|
||||
|
||||
## Token for authentication.
|
||||
token = ""
|
||||
|
||||
## Organization is the name of the organization you wish to write to.
|
||||
organization = ""
|
||||
|
||||
## Destination bucket to write into.
|
||||
bucket = ""
|
||||
|
||||
## The value of this tag will be used to determine the bucket. If this
|
||||
## tag is not set the 'bucket' option is used as the default.
|
||||
# bucket_tag = ""
|
||||
|
||||
## If true, the bucket tag will not be added to the metric.
|
||||
# exclude_bucket_tag = false
|
||||
|
||||
## Timeout for HTTP messages.
|
||||
# timeout = "5s"
|
||||
|
||||
## Additional HTTP headers
|
||||
# http_headers = {"X-Special-Header" = "Special-Value"}
|
||||
|
||||
## HTTP Proxy override, if unset values the standard proxy environment
|
||||
## variables are consulted to determine which proxy, if any, should be used.
|
||||
# http_proxy = "http://corporate.proxy:3128"
|
||||
|
||||
## HTTP User-Agent
|
||||
# user_agent = "telegraf"
|
||||
|
||||
## Content-Encoding for write request body, can be set to "gzip" to
|
||||
## compress body or "identity" to apply no encoding.
|
||||
# content_encoding = "gzip"
|
||||
|
||||
## Enable or disable uint support for writing uints influxdb 2.0.
|
||||
# influx_uint_support = false
|
||||
|
||||
## When true, Telegraf will omit the timestamp on data to allow InfluxDB
|
||||
## to set the timestamp of the data during ingestion. This is generally NOT
|
||||
## what you want as it can lead to data points captured at different times
|
||||
## getting omitted due to similar data.
|
||||
# influx_omit_timestamp = false
|
||||
|
||||
## HTTP/2 Timeouts
|
||||
## The following values control the HTTP/2 client's timeouts. These settings
|
||||
## are generally not required unless a user is seeing issues with client
|
||||
## disconnects. If a user does see issues, then it is suggested to set these
|
||||
## values to "15s" for ping timeout and "30s" for read idle timeout and
|
||||
## retry.
|
||||
##
|
||||
## Note that the timer for read_idle_timeout begins at the end of the last
|
||||
## successful write and not at the beginning of the next write.
|
||||
# ping_timeout = "0s"
|
||||
# read_idle_timeout = "0s"
|
||||
|
||||
## Optional TLS Config for use on HTTP connections.
|
||||
# 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
|
||||
|
||||
## Rate limits for sending data (disabled by default)
|
||||
## Available, uncompressed payload size e.g. "5MB"
|
||||
# rate_limit = "unlimited"
|
||||
## Fixed time-window for the available payload size e.g. "5m"
|
||||
# rate_limit_period = "0s"
|
Loading…
Add table
Add a link
Reference in a new issue