/
kafka.go
143 lines (131 loc) · 4.21 KB
/
kafka.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
package kafka
import (
"context"
"fmt"
"math"
"time"
dfv1 "github.com/argoproj-labs/argo-dataflow/api/v1alpha1"
sharedkafka "github.com/argoproj-labs/argo-dataflow/runner/sidecar/shared/kafka"
"github.com/argoproj-labs/argo-dataflow/runner/sidecar/sink"
sharedutil "github.com/argoproj-labs/argo-dataflow/shared/util"
kafka "github.com/confluentinc/confluent-kafka-go/kafka"
"github.com/opentracing/opentracing-go"
"github.com/prometheus/client_golang/prometheus"
"golang.org/x/sync/semaphore"
"k8s.io/apimachinery/pkg/util/wait"
corev1 "k8s.io/client-go/kubernetes/typed/core/v1"
)
var logger = sharedutil.NewLogger()
type kafkaSink struct {
sinkName string
producer *kafka.Producer
topic string
async bool
inflight *semaphore.Weighted
}
func New(ctx context.Context, sinkName string, secretInterface corev1.SecretInterface, x dfv1.KafkaSink, errorsCounter prometheus.Counter) (sink.Interface, error) {
logger := logger.WithValues("sink", sinkName)
config, err := sharedkafka.GetConfig(ctx, secretInterface, x.KafkaConfig)
if err != nil {
return nil, err
}
config["go.logs.channel.enable"] = true
if x.MaxMessageBytes > 0 {
config["message.max.bytes"] = x.GetMessageMaxBytes()
}
// https://docs.confluent.io/cloud/current/client-apps/optimizing/throughput.html
config["batch.size"] = x.GetBatchSize()
config["linger.ms"] = x.GetLingerMs()
config["message.timeout.ms"] = x.GetMessageTimeoutMs()
config["compression.type"] = x.CompressionType
config["acks"] = x.GetAcks()
config["enable.idempotence"] = x.EnableIdempotence
if x.Async { // this is meant to be set by `enable.idempotence` automatically, but I'm not sure it is
config["retries"] = math.MaxInt32
}
logger.Info("kafka config", "config", sharedutil.MustJSON(sharedkafka.RedactConfigMap(config)))
// https://github.com/confluentinc/confluent-kafka-go/blob/master/examples/producer_example/producer_example.go
producer, err := kafka.NewProducer(&config)
if err != nil {
return nil, err
}
go wait.JitterUntilWithContext(ctx, func(context.Context) {
logger.Info("consuming Kafka logs")
for e := range producer.Logs() {
logger.WithValues("name", e.Name, "tag", e.Tag).Info(e.Message)
}
}, 3*time.Second, 1.2, true)
inflight := semaphore.NewWeighted(int64(x.GetMessageInflight()))
go wait.JitterUntilWithContext(ctx, func(context.Context) {
logger.Info("starting producer event consuming loop")
for e := range producer.Events() {
switch ev := e.(type) {
case *kafka.Message:
inflight.Release(1)
if err := ev.TopicPartition.Error; err != nil {
logger.Error(err, "Async to Kafka failed", "topic", x.Topic)
errorsCounter.Inc()
}
}
}
}, time.Second, 1.2, true)
return &kafkaSink{
sinkName,
producer,
x.Topic,
x.Async,
inflight,
}, nil
}
func (h *kafkaSink) Sink(ctx context.Context, msg []byte) error {
span, ctx := opentracing.StartSpanFromContext(ctx, fmt.Sprintf("kafka-sink-%s", h.sinkName))
defer span.Finish()
m, err := dfv1.MetaFromContext(ctx)
if err != nil {
return err
}
var deliveryChan chan kafka.Event
if !h.async {
deliveryChan = make(chan kafka.Event)
defer close(deliveryChan)
}
if err := h.inflight.Acquire(ctx, 1); err != nil {
return err
}
if err := h.producer.Produce(&kafka.Message{
TopicPartition: kafka.TopicPartition{Topic: &h.topic, Partition: kafka.PartitionAny},
Headers: []kafka.Header{
{Key: "source", Value: []byte(m.Source)},
{Key: "id", Value: []byte(m.ID)},
},
Value: msg,
}, deliveryChan); err != nil {
return err
}
if deliveryChan != nil {
defer h.inflight.Release(1)
select {
case <-ctx.Done():
return fmt.Errorf("failed to get delivery: %w", ctx.Err())
case e := <-deliveryChan:
switch ev := e.(type) {
case *kafka.Message:
return ev.TopicPartition.Error
default:
return fmt.Errorf("failed to read delivery report: %s", e.String())
}
}
}
return nil
}
func (h *kafkaSink) Close() error {
logger.Info("flushing producer")
unflushedMessages := h.producer.Flush(15 * 1000)
if unflushedMessages > 0 {
logger.Error(fmt.Errorf("unflushed messagesd %d", unflushedMessages), "failed to flush producer", "sinkName", h.sinkName)
}
logger.Info("closing producer")
h.producer.Close()
logger.Info("producer closed")
return nil
}