mirror of
https://github.com/akvorado/akvorado.git
synced 2025-12-11 22:14:02 +01:00
This change split the inlet component into a simpler inlet and a new outlet component. The new inlet component receive flows and put them in Kafka, unparsed. The outlet component takes them from Kafka and resume the processing from here (flow parsing, enrichment) and puts them in ClickHouse. The main goal is to ensure the inlet does a minimal work to not be late when processing packets (and restart faster). It also brings some simplification as the number of knobs to tune everything is reduced: for inlet, we only need to tune the queue size for UDP, the number of workers and a few Kafka parameters; for outlet, we need to tune a few Kafka parameters, the number of workers and a few ClickHouse parameters. The outlet component features a simple Kafka input component. The core component becomes just a callback function. There is also a new ClickHouse component to push data to ClickHouse using the low-level ch-go library with batch inserts. This processing has an impact on the internal representation of a FlowMessage. Previously, it was tailored to dynamically build the protobuf message to be put in Kafka. Now, it builds the batch request to be sent to ClickHouse. This makes the FlowMessage structure hides the content of the next batch request and therefore, it should be reused. This also changes the way we decode flows as they don't output FlowMessage anymore, they reuse one that is provided to each worker. The ClickHouse tables are slightly updated. Instead of using Kafka engine, the Null engine is used instead. Fix #1122
142 lines
3.9 KiB
Go
142 lines
3.9 KiB
Go
// SPDX-FileCopyrightText: 2022 Free Mobile
|
|
// SPDX-License-Identifier: AGPL-3.0-only
|
|
|
|
// Package kafka handles flow exports to Kafka.
|
|
package kafka
|
|
|
|
import (
|
|
"encoding/binary"
|
|
"fmt"
|
|
"math/rand"
|
|
"strings"
|
|
"time"
|
|
|
|
"github.com/IBM/sarama"
|
|
"gopkg.in/tomb.v2"
|
|
|
|
"akvorado/common/daemon"
|
|
"akvorado/common/kafka"
|
|
"akvorado/common/pb"
|
|
"akvorado/common/reporter"
|
|
)
|
|
|
|
// Component represents the Kafka exporter.
|
|
type Component struct {
|
|
r *reporter.Reporter
|
|
d *Dependencies
|
|
t tomb.Tomb
|
|
config Configuration
|
|
|
|
kafkaConfig *sarama.Config
|
|
kafkaTopic string
|
|
kafkaProducer sarama.AsyncProducer
|
|
createKafkaProducer func() (sarama.AsyncProducer, error)
|
|
metrics metrics
|
|
}
|
|
|
|
// Dependencies define the dependencies of the Kafka exporter.
|
|
type Dependencies struct {
|
|
Daemon daemon.Component
|
|
}
|
|
|
|
// New creates a new Kafka exporter component.
|
|
func New(reporter *reporter.Reporter, configuration Configuration, dependencies Dependencies) (*Component, error) {
|
|
// Build Kafka configuration
|
|
kafkaConfig, err := kafka.NewConfig(configuration.Configuration)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
kafkaConfig.Metadata.AllowAutoTopicCreation = true
|
|
kafkaConfig.Producer.MaxMessageBytes = configuration.MaxMessageBytes
|
|
kafkaConfig.Producer.Compression = sarama.CompressionCodec(configuration.CompressionCodec)
|
|
kafkaConfig.Producer.Return.Successes = false
|
|
kafkaConfig.Producer.Return.Errors = true
|
|
kafkaConfig.Producer.Flush.Bytes = configuration.FlushBytes
|
|
kafkaConfig.Producer.Flush.Frequency = configuration.FlushInterval
|
|
kafkaConfig.Producer.Partitioner = sarama.NewHashPartitioner
|
|
kafkaConfig.ChannelBufferSize = configuration.QueueSize
|
|
if err := kafkaConfig.Validate(); err != nil {
|
|
return nil, fmt.Errorf("cannot validate Kafka configuration: %w", err)
|
|
}
|
|
|
|
c := Component{
|
|
r: reporter,
|
|
d: &dependencies,
|
|
config: configuration,
|
|
|
|
kafkaConfig: kafkaConfig,
|
|
kafkaTopic: fmt.Sprintf("%s-v%d", configuration.Topic, pb.Version),
|
|
}
|
|
c.initMetrics()
|
|
c.createKafkaProducer = func() (sarama.AsyncProducer, error) {
|
|
return sarama.NewAsyncProducer(c.config.Brokers, c.kafkaConfig)
|
|
}
|
|
c.d.Daemon.Track(&c.t, "inlet/kafka")
|
|
return &c, nil
|
|
}
|
|
|
|
// Start starts the Kafka component.
|
|
func (c *Component) Start() error {
|
|
c.r.Info().Msg("starting Kafka component")
|
|
kafka.GlobalKafkaLogger.Register(c.r)
|
|
|
|
// Create producer
|
|
kafkaProducer, err := c.createKafkaProducer()
|
|
if err != nil {
|
|
c.r.Err(err).
|
|
Str("brokers", strings.Join(c.config.Brokers, ",")).
|
|
Msg("unable to create async producer")
|
|
return fmt.Errorf("unable to create Kafka async producer: %w", err)
|
|
}
|
|
c.kafkaProducer = kafkaProducer
|
|
|
|
// Main loop
|
|
c.t.Go(func() error {
|
|
defer kafkaProducer.Close()
|
|
errLogger := c.r.Sample(reporter.BurstSampler(10*time.Second, 3))
|
|
dying := c.t.Dying()
|
|
for {
|
|
select {
|
|
case <-dying:
|
|
c.r.Debug().Msg("stop error logger")
|
|
return nil
|
|
case msg := <-kafkaProducer.Errors():
|
|
if msg != nil {
|
|
c.metrics.errors.WithLabelValues(msg.Error()).Inc()
|
|
errLogger.Err(msg.Err).
|
|
Str("topic", msg.Msg.Topic).
|
|
Int64("offset", msg.Msg.Offset).
|
|
Int32("partition", msg.Msg.Partition).
|
|
Msg("Kafka producer error")
|
|
}
|
|
}
|
|
}
|
|
})
|
|
return nil
|
|
}
|
|
|
|
// Stop stops the Kafka component
|
|
func (c *Component) Stop() error {
|
|
defer func() {
|
|
c.kafkaConfig.MetricRegistry.UnregisterAll()
|
|
kafka.GlobalKafkaLogger.Unregister()
|
|
c.r.Info().Msg("Kafka component stopped")
|
|
}()
|
|
c.r.Info().Msg("stopping Kafka component")
|
|
c.t.Kill(nil)
|
|
return c.t.Wait()
|
|
}
|
|
|
|
// Send a message to Kafka.
|
|
func (c *Component) Send(exporter string, payload []byte) {
|
|
c.metrics.bytesSent.WithLabelValues(exporter).Add(float64(len(payload)))
|
|
c.metrics.messagesSent.WithLabelValues(exporter).Inc()
|
|
key := make([]byte, 4)
|
|
binary.BigEndian.PutUint32(key, rand.Uint32())
|
|
c.kafkaProducer.Input() <- &sarama.ProducerMessage{
|
|
Topic: c.kafkaTopic,
|
|
Key: sarama.ByteEncoder(key),
|
|
Value: sarama.ByteEncoder(payload),
|
|
}
|
|
}
|