Merge branch 'develop' into main

- MetricRouter: Fix interval_timestamp option
- InfluxSink & InfluxAsyncSink: Add own flush mechanism
- InfluxSink & InfluxAsyncSink: Use default options, overwrite if
configured otherwise
- InfinibandCollector: Add units to metrics
- LikwidCollector: Fix for dying accessdaemon
This commit is contained in:
Thomas Roehl 2022-04-04 02:59:42 +02:00
commit 2f0b6057ca
7 changed files with 236 additions and 116 deletions

View File

@ -18,13 +18,18 @@ import (
const IB_BASEPATH = "/sys/class/infiniband/" const IB_BASEPATH = "/sys/class/infiniband/"
type InfinibandCollectorMetric struct {
path string
unit string
}
type InfinibandCollectorInfo struct { type InfinibandCollectorInfo struct {
LID string // IB local Identifier (LID) LID string // IB local Identifier (LID)
device string // IB device device string // IB device
port string // IB device port port string // IB device port
portCounterFiles map[string]string // mapping counter name -> sysfs file portCounterFiles map[string]InfinibandCollectorMetric // mapping counter name -> InfinibandCollectorMetric
tagSet map[string]string // corresponding tag list tagSet map[string]string // corresponding tag list
lastState map[string]int64 // State from last measurement lastState map[string]int64 // State from last measurement
} }
type InfinibandCollector struct { type InfinibandCollector struct {
@ -106,16 +111,16 @@ func (m *InfinibandCollector) Init(config json.RawMessage) error {
// Check access to counter files // Check access to counter files
countersDir := filepath.Join(path, "counters") countersDir := filepath.Join(path, "counters")
portCounterFiles := map[string]string{ portCounterFiles := map[string]InfinibandCollectorMetric{
"ib_recv": filepath.Join(countersDir, "port_rcv_data"), "ib_recv": {path: filepath.Join(countersDir, "port_rcv_data"), unit: "bytes"},
"ib_xmit": filepath.Join(countersDir, "port_xmit_data"), "ib_xmit": {path: filepath.Join(countersDir, "port_xmit_data"), unit: "bytes"},
"ib_recv_pkts": filepath.Join(countersDir, "port_rcv_packets"), "ib_recv_pkts": {path: filepath.Join(countersDir, "port_rcv_packets"), unit: "packets"},
"ib_xmit_pkts": filepath.Join(countersDir, "port_xmit_packets"), "ib_xmit_pkts": {path: filepath.Join(countersDir, "port_xmit_packets"), unit: "packets"},
} }
for _, counterFile := range portCounterFiles { for _, counter := range portCounterFiles {
err := unix.Access(counterFile, unix.R_OK) err := unix.Access(counter.path, unix.R_OK)
if err != nil { if err != nil {
return fmt.Errorf("unable to access %s: %v", counterFile, err) return fmt.Errorf("unable to access %s: %v", counter.path, err)
} }
} }
@ -165,14 +170,14 @@ func (m *InfinibandCollector) Read(interval time.Duration, output chan lp.CCMetr
m.lastTimestamp = now m.lastTimestamp = now
for _, info := range m.info { for _, info := range m.info {
for counterName, counterFile := range info.portCounterFiles { for counterName, counterDef := range info.portCounterFiles {
// Read counter file // Read counter file
line, err := ioutil.ReadFile(counterFile) line, err := ioutil.ReadFile(counterDef.path)
if err != nil { if err != nil {
cclog.ComponentError( cclog.ComponentError(
m.name, m.name,
fmt.Sprintf("Read(): Failed to read from file '%s': %v", counterFile, err)) fmt.Sprintf("Read(): Failed to read from file '%s': %v", counterDef.path, err))
continue continue
} }
data := strings.TrimSpace(string(line)) data := strings.TrimSpace(string(line))
@ -189,6 +194,7 @@ func (m *InfinibandCollector) Read(interval time.Duration, output chan lp.CCMetr
// Send absolut values // Send absolut values
if m.config.SendAbsoluteValues { if m.config.SendAbsoluteValues {
if y, err := lp.New(counterName, info.tagSet, m.meta, map[string]interface{}{"value": v}, now); err == nil { if y, err := lp.New(counterName, info.tagSet, m.meta, map[string]interface{}{"value": v}, now); err == nil {
y.AddMeta("unit", counterDef.unit)
output <- y output <- y
} }
} }
@ -198,6 +204,7 @@ func (m *InfinibandCollector) Read(interval time.Duration, output chan lp.CCMetr
if info.lastState[counterName] >= 0 { if info.lastState[counterName] >= 0 {
rate := float64((v - info.lastState[counterName])) / timeDiff rate := float64((v - info.lastState[counterName])) / timeDiff
if y, err := lp.New(counterName+"_bw", info.tagSet, m.meta, map[string]interface{}{"value": rate}, now); err == nil { if y, err := lp.New(counterName+"_bw", info.tagSet, m.meta, map[string]interface{}{"value": rate}, now); err == nil {
y.AddMeta("unit", counterDef.unit+"/sec")
output <- y output <- y
} }
} }

View File

@ -16,6 +16,7 @@ import (
"math" "math"
"os" "os"
"os/signal" "os/signal"
"sort"
"strconv" "strconv"
"strings" "strings"
"sync" "sync"
@ -54,6 +55,7 @@ type LikwidEventsetConfig struct {
gid C.int gid C.int
eorder []*C.char eorder []*C.char
estr *C.char estr *C.char
go_estr string
results map[int]map[string]interface{} results map[int]map[string]interface{}
metrics map[int]map[string]float64 metrics map[int]map[string]float64
} }
@ -101,8 +103,14 @@ func eventsToEventStr(events map[string]string) string {
func genLikwidEventSet(input LikwidCollectorEventsetConfig) LikwidEventsetConfig { func genLikwidEventSet(input LikwidCollectorEventsetConfig) LikwidEventsetConfig {
tmplist := make([]string, 0) tmplist := make([]string, 0)
clist := make([]string, 0)
for k := range input.Events {
clist = append(clist, k)
}
sort.Strings(clist)
elist := make([]*C.char, 0) elist := make([]*C.char, 0)
for k, v := range input.Events { for _, k := range clist {
v := input.Events[k]
tmplist = append(tmplist, fmt.Sprintf("%s:%s", v, k)) tmplist = append(tmplist, fmt.Sprintf("%s:%s", v, k))
c_counter := C.CString(k) c_counter := C.CString(k)
elist = append(elist, c_counter) elist = append(elist, c_counter)
@ -124,6 +132,7 @@ func genLikwidEventSet(input LikwidCollectorEventsetConfig) LikwidEventsetConfig
gid: -1, gid: -1,
eorder: elist, eorder: elist,
estr: C.CString(estr), estr: C.CString(estr),
go_estr: estr,
results: res, results: res,
metrics: met, metrics: met,
} }
@ -193,7 +202,7 @@ func (m *LikwidCollector) Init(config json.RawMessage) error {
} }
m.setup() m.setup()
m.meta = map[string]string{"source": m.name, "group": "PerfCounter"} m.meta = map[string]string{"group": "PerfCounter"}
cclog.ComponentDebug(m.name, "Get cpulist and init maps and lists") cclog.ComponentDebug(m.name, "Get cpulist and init maps and lists")
cpulist := topo.CpuList() cpulist := topo.CpuList()
m.cpulist = make([]C.int, len(cpulist)) m.cpulist = make([]C.int, len(cpulist))
@ -425,6 +434,9 @@ func (m *LikwidCollector) calcGlobalMetrics(interval time.Duration, output chan
func (m *LikwidCollector) LateInit() error { func (m *LikwidCollector) LateInit() error {
var ret C.int var ret C.int
if m.initialized {
return nil
}
switch m.config.AccessMode { switch m.config.AccessMode {
case "direct": case "direct":
C.HPMmode(0) C.HPMmode(0)
@ -475,7 +487,17 @@ func (m *LikwidCollector) LateInit() error {
for i, evset := range m.config.Eventsets { for i, evset := range m.config.Eventsets {
var gid C.int var gid C.int
if len(evset.Events) > 0 { if len(evset.Events) > 0 {
skip := false
likwidGroup := genLikwidEventSet(evset) likwidGroup := genLikwidEventSet(evset)
for _, g := range m.likwidGroups {
if likwidGroup.go_estr == g.go_estr {
skip = true
break
}
}
if skip {
continue
}
// Now we add the list of events to likwid // Now we add the list of events to likwid
gid = C.perfmon_addEventSet(likwidGroup.estr) gid = C.perfmon_addEventSet(likwidGroup.estr)
if gid >= 0 { if gid >= 0 {
@ -520,9 +542,14 @@ func (m *LikwidCollector) Read(interval time.Duration, output chan lp.CCMetric)
} }
if !m.initialized { if !m.initialized {
if m.LateInit() != nil { m.lock.Lock()
err = m.LateInit()
if err != nil {
m.lock.Unlock()
return return
} }
m.initialized = true
m.lock.Unlock()
} }
if m.initialized && !skip { if m.initialized && !skip {

View File

@ -48,7 +48,6 @@ type metricRouter struct {
done chan bool // channel to finish / stop metric router done chan bool // channel to finish / stop metric router
wg *sync.WaitGroup // wait group for all goroutines in cc-metric-collector wg *sync.WaitGroup // wait group for all goroutines in cc-metric-collector
timestamp time.Time // timestamp periodically updated by ticker each interval timestamp time.Time // timestamp periodically updated by ticker each interval
timerdone chan bool // channel to finish / stop timestamp updater
ticker mct.MultiChanTicker // periodically ticking once each interval ticker mct.MultiChanTicker // periodically ticking once each interval
config metricRouterConfig // json encoded config for metric router config metricRouterConfig // json encoded config for metric router
cache MetricCache // pointer to MetricCache cache MetricCache // pointer to MetricCache
@ -124,29 +123,6 @@ func (r *metricRouter) Init(ticker mct.MultiChanTicker, wg *sync.WaitGroup, rout
return nil return nil
} }
// StartTimer starts a timer which updates timestamp periodically
func (r *metricRouter) StartTimer() {
m := make(chan time.Time)
r.ticker.AddChannel(m)
r.timerdone = make(chan bool)
r.wg.Add(1)
go func() {
defer r.wg.Done()
for {
select {
case <-r.timerdone:
close(r.timerdone)
cclog.ComponentDebug("MetricRouter", "TIMER DONE")
return
case t := <-m:
r.timestamp = t
}
}
}()
cclog.ComponentDebug("MetricRouter", "TIMER START")
}
func getParamMap(point lp.CCMetric) map[string]interface{} { func getParamMap(point lp.CCMetric) map[string]interface{} {
params := make(map[string]interface{}) params := make(map[string]interface{})
params["metric"] = point params["metric"] = point
@ -235,8 +211,9 @@ func (r *metricRouter) dropMetric(point lp.CCMetric) bool {
func (r *metricRouter) Start() { func (r *metricRouter) Start() {
// start timer if configured // start timer if configured
r.timestamp = time.Now() r.timestamp = time.Now()
timeChan := make(chan time.Time)
if r.config.IntervalStamp { if r.config.IntervalStamp {
r.StartTimer() r.ticker.AddChannel(timeChan)
} }
// Router manager is done // Router manager is done
@ -316,6 +293,10 @@ func (r *metricRouter) Start() {
done() done()
return return
case timestamp := <-timeChan:
r.timestamp = timestamp
cclog.ComponentDebug("MetricRouter", "Update timestamp", r.timestamp.UnixNano())
case p := <-r.coll_input: case p := <-r.coll_input:
coll_forward(p) coll_forward(p)
for i := 0; len(r.coll_input) > 0 && i < (r.maxForward-1); i++ { for i := 0; len(r.coll_input) > 0 && i < (r.maxForward-1); i++ {
@ -361,14 +342,6 @@ func (r *metricRouter) Close() {
// wait for close of channel r.done // wait for close of channel r.done
<-r.done <-r.done
// stop timer
if r.config.IntervalStamp {
cclog.ComponentDebug("MetricRouter", "TIMER CLOSE")
r.timerdone <- true
// wait for close of channel r.timerdone
<-r.timerdone
}
// stop metric cache // stop metric cache
if r.config.NumCacheIntervals > 0 { if r.config.NumCacheIntervals > 0 {
cclog.ComponentDebug("MetricRouter", "CACHE CLOSE") cclog.ComponentDebug("MetricRouter", "CACHE CLOSE")

View File

@ -42,13 +42,13 @@ func (s *HttpSink) Write(m lp.CCMetric) error {
if s.buffer.Len() == 0 && s.flushDelay != 0 { if s.buffer.Len() == 0 && s.flushDelay != 0 {
// This is the first write since the last flush, start the flushTimer! // This is the first write since the last flush, start the flushTimer!
if s.flushTimer != nil && s.flushTimer.Stop() { if s.flushTimer != nil && s.flushTimer.Stop() {
cclog.ComponentDebug("HttpSink", "unexpected: the flushTimer was already running?") cclog.ComponentDebug(s.name, "unexpected: the flushTimer was already running?")
} }
// Run a batched flush for all lines that have arrived in the last second // Run a batched flush for all lines that have arrived in the last second
s.flushTimer = time.AfterFunc(s.flushDelay, func() { s.flushTimer = time.AfterFunc(s.flushDelay, func() {
if err := s.Flush(); err != nil { if err := s.Flush(); err != nil {
cclog.ComponentError("HttpSink", "flush failed:", err.Error()) cclog.ComponentError(s.name, "flush failed:", err.Error())
} }
}) })
} }
@ -60,6 +60,7 @@ func (s *HttpSink) Write(m lp.CCMetric) error {
s.lock.Unlock() // defer does not work here as Flush() takes the lock as well s.lock.Unlock() // defer does not work here as Flush() takes the lock as well
if err != nil { if err != nil {
cclog.ComponentError(s.name, "encoding failed:", err.Error())
return err return err
} }
@ -84,6 +85,7 @@ func (s *HttpSink) Flush() error {
// Create new request to send buffer // Create new request to send buffer
req, err := http.NewRequest(http.MethodPost, s.config.URL, s.buffer) req, err := http.NewRequest(http.MethodPost, s.config.URL, s.buffer)
if err != nil { if err != nil {
cclog.ComponentError(s.name, "failed to create request:", err.Error())
return err return err
} }
@ -100,12 +102,15 @@ func (s *HttpSink) Flush() error {
// Handle transport/tcp errors // Handle transport/tcp errors
if err != nil { if err != nil {
cclog.ComponentError(s.name, "transport/tcp error:", err.Error())
return err return err
} }
// Handle application errors // Handle application errors
if res.StatusCode != http.StatusOK { if res.StatusCode != http.StatusOK {
return errors.New(res.Status) err = errors.New(res.Status)
cclog.ComponentError(s.name, "application error:", err.Error())
return err
} }
return nil return nil
@ -114,7 +119,7 @@ func (s *HttpSink) Flush() error {
func (s *HttpSink) Close() { func (s *HttpSink) Close() {
s.flushTimer.Stop() s.flushTimer.Stop()
if err := s.Flush(); err != nil { if err := s.Flush(); err != nil {
cclog.ComponentError("HttpSink", "flush failed:", err.Error()) cclog.ComponentError(s.name, "flush failed:", err.Error())
} }
s.client.CloseIdleConnections() s.client.CloseIdleConnections()
} }

View File

@ -28,10 +28,11 @@ type InfluxAsyncSinkConfig struct {
BatchSize uint `json:"batch_size,omitempty"` BatchSize uint `json:"batch_size,omitempty"`
// Interval, in ms, in which is buffer flushed if it has not been already written (by reaching batch size) . Default 1000ms // Interval, in ms, in which is buffer flushed if it has not been already written (by reaching batch size) . Default 1000ms
FlushInterval uint `json:"flush_interval,omitempty"` FlushInterval uint `json:"flush_interval,omitempty"`
InfluxRetryInterval string `json:"retry_interval"` InfluxRetryInterval string `json:"retry_interval,omitempty"`
InfluxExponentialBase uint `json:"retry_exponential_base"` InfluxExponentialBase uint `json:"retry_exponential_base,omitempty"`
InfluxMaxRetries uint `json:"max_retries"` InfluxMaxRetries uint `json:"max_retries,omitempty"`
InfluxMaxRetryTime string `json:"max_retry_time"` InfluxMaxRetryTime string `json:"max_retry_time,omitempty"`
CustomFlushInterval string `json:"custom_flush_interval,omitempty"`
} }
type InfluxAsyncSink struct { type InfluxAsyncSink struct {
@ -42,6 +43,8 @@ type InfluxAsyncSink struct {
config InfluxAsyncSinkConfig config InfluxAsyncSinkConfig
influxRetryInterval uint influxRetryInterval uint
influxMaxRetryTime uint influxMaxRetryTime uint
customFlushInterval time.Duration
flushTimer *time.Timer
} }
func (s *InfluxAsyncSink) connect() error { func (s *InfluxAsyncSink) connect() error {
@ -60,20 +63,34 @@ func (s *InfluxAsyncSink) connect() error {
cclog.ComponentDebug(s.name, "Using URI", uri, "Org", s.config.Organization, "Bucket", s.config.Database) cclog.ComponentDebug(s.name, "Using URI", uri, "Org", s.config.Organization, "Bucket", s.config.Database)
clientOptions := influxdb2.DefaultOptions() clientOptions := influxdb2.DefaultOptions()
if s.config.BatchSize != 0 { if s.config.BatchSize != 0 {
cclog.ComponentDebug(s.name, "Batch size", s.config.BatchSize)
clientOptions.SetBatchSize(s.config.BatchSize) clientOptions.SetBatchSize(s.config.BatchSize)
} }
if s.config.FlushInterval != 0 { if s.config.FlushInterval != 0 {
cclog.ComponentDebug(s.name, "Flush interval", s.config.FlushInterval)
clientOptions.SetFlushInterval(s.config.FlushInterval) clientOptions.SetFlushInterval(s.config.FlushInterval)
} }
if s.influxRetryInterval != 0 {
cclog.ComponentDebug(s.name, "MaxRetryInterval", s.influxRetryInterval)
clientOptions.SetMaxRetryInterval(s.influxRetryInterval)
}
if s.influxMaxRetryTime != 0 {
cclog.ComponentDebug(s.name, "MaxRetryTime", s.influxMaxRetryTime)
clientOptions.SetMaxRetryTime(s.influxMaxRetryTime)
}
if s.config.InfluxExponentialBase != 0 {
cclog.ComponentDebug(s.name, "Exponential Base", s.config.InfluxExponentialBase)
clientOptions.SetExponentialBase(s.config.InfluxExponentialBase)
}
if s.config.InfluxMaxRetries != 0 {
cclog.ComponentDebug(s.name, "Max Retries", s.config.InfluxMaxRetries)
clientOptions.SetMaxRetries(s.config.InfluxMaxRetries)
}
clientOptions.SetTLSConfig( clientOptions.SetTLSConfig(
&tls.Config{ &tls.Config{
InsecureSkipVerify: true, InsecureSkipVerify: true,
}, },
) ).SetPrecision(time.Second)
clientOptions.SetMaxRetryInterval(s.influxRetryInterval)
clientOptions.SetMaxRetryTime(s.influxMaxRetryTime)
clientOptions.SetExponentialBase(s.config.InfluxExponentialBase)
clientOptions.SetMaxRetries(s.config.InfluxMaxRetries)
s.client = influxdb2.NewClientWithOptions(uri, auth, clientOptions) s.client = influxdb2.NewClientWithOptions(uri, auth, clientOptions)
s.writeApi = s.client.WriteAPI(s.config.Organization, s.config.Database) s.writeApi = s.client.WriteAPI(s.config.Organization, s.config.Database)
@ -88,6 +105,15 @@ func (s *InfluxAsyncSink) connect() error {
} }
func (s *InfluxAsyncSink) Write(m lp.CCMetric) error { func (s *InfluxAsyncSink) Write(m lp.CCMetric) error {
if s.customFlushInterval != 0 && s.flushTimer == nil {
// Run a batched flush for all lines that have arrived in the defined interval
s.flushTimer = time.AfterFunc(s.customFlushInterval, func() {
if err := s.Flush(); err != nil {
cclog.ComponentError(s.name, "flush failed:", err.Error())
}
})
}
s.writeApi.WritePoint( s.writeApi.WritePoint(
m.ToPoint(s.meta_as_tags), m.ToPoint(s.meta_as_tags),
) )
@ -95,7 +121,11 @@ func (s *InfluxAsyncSink) Write(m lp.CCMetric) error {
} }
func (s *InfluxAsyncSink) Flush() error { func (s *InfluxAsyncSink) Flush() error {
cclog.ComponentDebug(s.name, "Flushing")
s.writeApi.Flush() s.writeApi.Flush()
if s.customFlushInterval != 0 && s.flushTimer != nil {
s.flushTimer = nil
}
return nil return nil
} }
@ -110,13 +140,16 @@ func NewInfluxAsyncSink(name string, config json.RawMessage) (Sink, error) {
s.name = fmt.Sprintf("InfluxSink(%s)", name) s.name = fmt.Sprintf("InfluxSink(%s)", name)
// Set default for maximum number of points sent to server in single request. // Set default for maximum number of points sent to server in single request.
s.config.BatchSize = 100 s.config.BatchSize = 0
s.influxRetryInterval = uint(time.Duration(1) * time.Second) s.influxRetryInterval = 0
s.config.InfluxRetryInterval = "1s" //s.config.InfluxRetryInterval = "1s"
s.influxMaxRetryTime = uint(7 * time.Duration(24) * time.Hour) s.influxMaxRetryTime = 0
s.config.InfluxMaxRetryTime = "168h" //s.config.InfluxMaxRetryTime = "168h"
s.config.InfluxMaxRetries = 20 s.config.InfluxMaxRetries = 0
s.config.InfluxExponentialBase = 2 s.config.InfluxExponentialBase = 0
s.config.FlushInterval = 0
s.config.CustomFlushInterval = ""
s.customFlushInterval = time.Duration(0)
// Default retry intervals (in seconds) // Default retry intervals (in seconds)
// 1 2 // 1 2
@ -168,6 +201,15 @@ func NewInfluxAsyncSink(name string, config json.RawMessage) (Sink, error) {
s.influxRetryInterval = toUint(s.config.InfluxRetryInterval, s.influxRetryInterval) s.influxRetryInterval = toUint(s.config.InfluxRetryInterval, s.influxRetryInterval)
s.influxMaxRetryTime = toUint(s.config.InfluxMaxRetryTime, s.influxMaxRetryTime) s.influxMaxRetryTime = toUint(s.config.InfluxMaxRetryTime, s.influxMaxRetryTime)
// Use a own timer for calling Flush()
if len(s.config.CustomFlushInterval) > 0 {
t, err := time.ParseDuration(s.config.CustomFlushInterval)
if err != nil {
return nil, fmt.Errorf("invalid duration in 'custom_flush_interval': %v", err)
}
s.customFlushInterval = t
}
// Connect to InfluxDB server // Connect to InfluxDB server
if err := s.connect(); err != nil { if err := s.connect(); err != nil {
return nil, fmt.Errorf("unable to connect: %v", err) return nil, fmt.Errorf("unable to connect: %v", err)

View File

@ -6,28 +6,32 @@ import (
"encoding/json" "encoding/json"
"errors" "errors"
"fmt" "fmt"
"sync"
"time" "time"
cclog "github.com/ClusterCockpit/cc-metric-collector/internal/ccLogger" cclog "github.com/ClusterCockpit/cc-metric-collector/internal/ccLogger"
lp "github.com/ClusterCockpit/cc-metric-collector/internal/ccMetric" lp "github.com/ClusterCockpit/cc-metric-collector/internal/ccMetric"
influxdb2 "github.com/influxdata/influxdb-client-go/v2" influxdb2 "github.com/influxdata/influxdb-client-go/v2"
influxdb2Api "github.com/influxdata/influxdb-client-go/v2/api" influxdb2Api "github.com/influxdata/influxdb-client-go/v2/api"
"github.com/influxdata/influxdb-client-go/v2/api/write"
) )
type InfluxSinkConfig struct { type InfluxSinkConfig struct {
defaultSinkConfig defaultSinkConfig
Host string `json:"host,omitempty"` Host string `json:"host,omitempty"`
Port string `json:"port,omitempty"` Port string `json:"port,omitempty"`
Database string `json:"database,omitempty"` Database string `json:"database,omitempty"`
User string `json:"user,omitempty"` User string `json:"user,omitempty"`
Password string `json:"password,omitempty"` Password string `json:"password,omitempty"`
Organization string `json:"organization,omitempty"` Organization string `json:"organization,omitempty"`
SSL bool `json:"ssl,omitempty"` SSL bool `json:"ssl,omitempty"`
RetentionPol string `json:"retention_policy,omitempty"` FlushDelay string `json:"flush_delay,omitempty"`
InfluxRetryInterval string `json:"retry_interval"` BatchSize int `json:"batch_size,omitempty"`
InfluxExponentialBase uint `json:"retry_exponential_base"` RetentionPol string `json:"retention_policy,omitempty"`
InfluxMaxRetries uint `json:"max_retries"` // InfluxRetryInterval string `json:"retry_interval"`
InfluxMaxRetryTime string `json:"max_retry_time"` // InfluxExponentialBase uint `json:"retry_exponential_base"`
// InfluxMaxRetries uint `json:"max_retries"`
// InfluxMaxRetryTime string `json:"max_retry_time"`
//InfluxMaxRetryDelay string `json:"max_retry_delay"` // It is mentioned in the docs but there is no way to set it //InfluxMaxRetryDelay string `json:"max_retry_delay"` // It is mentioned in the docs but there is no way to set it
} }
@ -38,6 +42,10 @@ type InfluxSink struct {
config InfluxSinkConfig config InfluxSinkConfig
influxRetryInterval uint influxRetryInterval uint
influxMaxRetryTime uint influxMaxRetryTime uint
batch []*write.Point
flushTimer *time.Timer
flushDelay time.Duration
lock sync.Mutex // Flush() runs in another goroutine, so this lock has to protect the buffer
//influxMaxRetryDelay uint //influxMaxRetryDelay uint
} }
@ -56,16 +64,31 @@ func (s *InfluxSink) connect() error {
} }
cclog.ComponentDebug(s.name, "Using URI", uri, "Org", s.config.Organization, "Bucket", s.config.Database) cclog.ComponentDebug(s.name, "Using URI", uri, "Org", s.config.Organization, "Bucket", s.config.Database)
clientOptions := influxdb2.DefaultOptions() clientOptions := influxdb2.DefaultOptions()
// if s.influxRetryInterval != 0 {
// cclog.ComponentDebug(s.name, "MaxRetryInterval", s.influxRetryInterval)
// clientOptions.SetMaxRetryInterval(s.influxRetryInterval)
// }
// if s.influxMaxRetryTime != 0 {
// cclog.ComponentDebug(s.name, "MaxRetryTime", s.influxMaxRetryTime)
// clientOptions.SetMaxRetryTime(s.influxMaxRetryTime)
// }
// if s.config.InfluxExponentialBase != 0 {
// cclog.ComponentDebug(s.name, "Exponential Base", s.config.InfluxExponentialBase)
// clientOptions.SetExponentialBase(s.config.InfluxExponentialBase)
// }
// if s.config.InfluxMaxRetries != 0 {
// cclog.ComponentDebug(s.name, "Max Retries", s.config.InfluxMaxRetries)
// clientOptions.SetMaxRetries(s.config.InfluxMaxRetries)
// }
clientOptions.SetTLSConfig( clientOptions.SetTLSConfig(
&tls.Config{ &tls.Config{
InsecureSkipVerify: true, InsecureSkipVerify: true,
}, },
) )
clientOptions.SetMaxRetryInterval(s.influxRetryInterval) clientOptions.SetPrecision(time.Second)
clientOptions.SetMaxRetryTime(s.influxMaxRetryTime)
clientOptions.SetExponentialBase(s.config.InfluxExponentialBase)
clientOptions.SetMaxRetries(s.config.InfluxMaxRetries)
s.client = influxdb2.NewClientWithOptions(uri, auth, clientOptions) s.client = influxdb2.NewClientWithOptions(uri, auth, clientOptions)
s.writeApi = s.client.WriteAPIBlocking(s.config.Organization, s.config.Database) s.writeApi = s.client.WriteAPIBlocking(s.config.Organization, s.config.Database)
@ -80,38 +103,76 @@ func (s *InfluxSink) connect() error {
} }
func (s *InfluxSink) Write(m lp.CCMetric) error { func (s *InfluxSink) Write(m lp.CCMetric) error {
err := // err :=
s.writeApi.WritePoint( // s.writeApi.WritePoint(
context.Background(), // context.Background(),
m.ToPoint(s.meta_as_tags), // m.ToPoint(s.meta_as_tags),
) // )
return err if len(s.batch) == 0 && s.flushDelay != 0 {
// This is the first write since the last flush, start the flushTimer!
if s.flushTimer != nil && s.flushTimer.Stop() {
cclog.ComponentDebug(s.name, "unexpected: the flushTimer was already running?")
}
// Run a batched flush for all lines that have arrived in the last second
s.flushTimer = time.AfterFunc(s.flushDelay, func() {
if err := s.Flush(); err != nil {
cclog.ComponentError(s.name, "flush failed:", err.Error())
}
})
}
p := m.ToPoint(s.meta_as_tags)
s.lock.Lock()
s.batch = append(s.batch, p)
s.lock.Unlock()
// Flush synchronously if "flush_delay" is zero
if s.flushDelay == 0 {
return s.Flush()
}
return nil
} }
func (s *InfluxSink) Flush() error { func (s *InfluxSink) Flush() error {
s.lock.Lock()
defer s.lock.Unlock()
if len(s.batch) == 0 {
return nil
}
err := s.writeApi.WritePoint(context.Background(), s.batch...)
if err != nil {
cclog.ComponentError(s.name, "flush failed:", err.Error())
return err
}
s.batch = s.batch[:0]
return nil return nil
} }
func (s *InfluxSink) Close() { func (s *InfluxSink) Close() {
cclog.ComponentDebug(s.name, "Closing InfluxDB connection") cclog.ComponentDebug(s.name, "Closing InfluxDB connection")
s.flushTimer.Stop()
s.Flush()
s.client.Close() s.client.Close()
} }
func NewInfluxSink(name string, config json.RawMessage) (Sink, error) { func NewInfluxSink(name string, config json.RawMessage) (Sink, error) {
s := new(InfluxSink) s := new(InfluxSink)
s.name = fmt.Sprintf("InfluxSink(%s)", name) s.name = fmt.Sprintf("InfluxSink(%s)", name)
s.config.BatchSize = 100
s.config.FlushDelay = "1s"
if len(config) > 0 { if len(config) > 0 {
err := json.Unmarshal(config, &s.config) err := json.Unmarshal(config, &s.config)
if err != nil { if err != nil {
return nil, err return nil, err
} }
} }
s.influxRetryInterval = uint(time.Duration(1) * time.Second) s.influxRetryInterval = 0
s.config.InfluxRetryInterval = "1s" s.influxMaxRetryTime = 0
s.influxMaxRetryTime = uint(7 * time.Duration(24) * time.Hour) // s.config.InfluxRetryInterval = ""
s.config.InfluxMaxRetryTime = "168h" // s.config.InfluxMaxRetryTime = ""
s.config.InfluxMaxRetries = 20 // s.config.InfluxMaxRetries = 0
s.config.InfluxExponentialBase = 2 // s.config.InfluxExponentialBase = 0
if len(s.config.Host) == 0 || if len(s.config.Host) == 0 ||
len(s.config.Port) == 0 || len(s.config.Port) == 0 ||
@ -126,15 +187,25 @@ func NewInfluxSink(name string, config json.RawMessage) (Sink, error) {
s.meta_as_tags[k] = true s.meta_as_tags[k] = true
} }
toUint := func(duration string, def uint) uint { // toUint := func(duration string, def uint) uint {
t, err := time.ParseDuration(duration) // if len(duration) > 0 {
// t, err := time.ParseDuration(duration)
// if err == nil {
// return uint(t.Milliseconds())
// }
// }
// return def
// }
// s.influxRetryInterval = toUint(s.config.InfluxRetryInterval, s.influxRetryInterval)
// s.influxMaxRetryTime = toUint(s.config.InfluxMaxRetryTime, s.influxMaxRetryTime)
if len(s.config.FlushDelay) > 0 {
t, err := time.ParseDuration(s.config.FlushDelay)
if err == nil { if err == nil {
return uint(t.Milliseconds()) s.flushDelay = t
} }
return def
} }
s.influxRetryInterval = toUint(s.config.InfluxRetryInterval, s.influxRetryInterval) s.batch = make([]*write.Point, 0, s.config.BatchSize)
s.influxMaxRetryTime = toUint(s.config.InfluxMaxRetryTime, s.influxMaxRetryTime)
// Connect to InfluxDB server // Connect to InfluxDB server
if err := s.connect(); err != nil { if err := s.connect(); err != nil {

View File

@ -17,10 +17,8 @@ The `influxdb` sink uses the official [InfluxDB golang client](https://pkg.go.de
"password" : "examplepw", "password" : "examplepw",
"organization": "myorg", "organization": "myorg",
"ssl": true, "ssl": true,
"retry_interval" : "1s", "flush_delay" : "1s",
"retry_exponential_base" : 2, "batch_size" : 100
"max_retries": 20,
"max_retry_time" : "168h"
} }
} }
``` ```
@ -34,9 +32,6 @@ The `influxdb` sink uses the official [InfluxDB golang client](https://pkg.go.de
- `password`: Password for basic authentification - `password`: Password for basic authentification
- `organization`: Organization in the InfluxDB - `organization`: Organization in the InfluxDB
- `ssl`: Use SSL connection - `ssl`: Use SSL connection
- `retry_interval`: Base retry interval for failed write requests, default 1s - `flush_delay`: Group metrics coming in to a single batch
- `retry_exponential_base`: The retry interval is exponentially increased with this base, default 2 - `batch_size`: Maximal batch size
- `max_retries`: Maximal number of retry attempts
- `max_retry_time`: Maximal time to retry failed writes, default 168h (one week)
For information about the calculation of the retry interval settings, see [offical influxdb-client-go documentation](https://github.com/influxdata/influxdb-client-go#handling-of-failed-async-writes)