forked from ConduitIO/conduit-connector-kafka
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathproducer.go
238 lines (213 loc) · 7.39 KB
/
producer.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
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
// Copyright © 2022 Meroxa, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
//go:generate mockgen -destination mock/producer.go -package mock -mock_names=Producer=Producer . Producer
package kafka
import (
"context"
"errors"
"fmt"
"sync"
"time"
"github.com/avast/retry-go"
sdk "github.com/conduitio/conduit-connector-sdk"
"github.com/segmentio/kafka-go"
)
type Producer interface {
// Send sends all records to Kafka synchronously.
Send(ctx context.Context, records []sdk.Record) (int, error)
// Close this producer and the associated resources (e.g. connections to the broker)
Close() error
}
type segmentProducer struct {
writer *kafka.Writer
balancer *batchSizeAdjustingBalancer
}
// NewProducer creates a new Kafka producer.
// The current implementation uses Segment's kafka-go client.
func NewProducer(cfg Config) (Producer, error) {
if len(cfg.Servers) == 0 {
return nil, ErrServersMissing
}
if cfg.Topic == "" {
return nil, ErrTopicMissing
}
p := &segmentProducer{}
err := p.init(cfg)
if err != nil {
return nil, fmt.Errorf("couldn't initialize producer: %w", err)
}
return p, nil
}
func (p *segmentProducer) init(cfg Config) error {
p.balancer = &batchSizeAdjustingBalancer{}
p.writer = &kafka.Writer{
Addr: kafka.TCP(cfg.Servers...),
Topic: cfg.Topic,
WriteTimeout: cfg.DeliveryTimeout,
RequiredAcks: cfg.Acks,
MaxAttempts: 3,
AllowAutoTopicCreation: true,
Balancer: p.balancer,
BatchTimeout: time.Millisecond, // partial batches will be written after 1 millisecond
}
p.balancer.writer = p.writer
err := p.configureSecurity(cfg)
if err != nil {
return fmt.Errorf("couldn't configure security: %w", err)
}
return nil
}
func (p *segmentProducer) configureSecurity(cfg Config) error {
transport := &kafka.Transport{}
// TLS settings
if cfg.useTLS() {
tlsCfg, err := newTLSConfig(cfg.ClientCert, cfg.ClientKey, cfg.CACert, cfg.InsecureSkipVerify)
if err != nil {
return fmt.Errorf("invalid TLS config: %w", err)
}
transport.TLS = tlsCfg
}
// SASL
if cfg.saslEnabled() {
mechanism, err := newSASLMechanism(cfg.SASLMechanism, cfg.SASLUsername, cfg.SASLPassword)
if err != nil {
return fmt.Errorf("couldn't configure SASL: %w", err)
}
transport.SASL = mechanism
}
p.writer.Transport = transport
return nil
}
func (p *segmentProducer) Send(ctx context.Context, records []sdk.Record) (int, error) {
p.balancer.SetRecordCount(len(records))
messages := make([]kafka.Message, len(records))
for i, r := range records {
messages[i] = kafka.Message{
Key: r.Key.Bytes(),
Value: r.Bytes(),
}
}
err := p.sendRetryable(ctx, messages)
if err != nil {
werr, ok := err.(kafka.WriteErrors)
if !ok {
return 0, fmt.Errorf("failed to produce messages: %w", err)
}
// multiple errors occurred, we loop through the errors and fetch the
// first non-nil error, we log all others
count := 0
err = nil
for i := range werr {
switch {
case werr[i] != nil && err == nil:
// the first message that failed to be produced - we will return
// this one
count = i
err = werr[i]
case werr[i] != nil && err != nil:
// a message that failed to be produced after an already failed
// message - we will log this one
sdk.Logger(ctx).Err(werr[i]).Bytes("record_position", records[i].Position).Msg("failed to produce message")
case werr[i] == nil && err != nil:
// a message that we successfully produced after a message that
// failed - we need to log a warning, this one will be
// duplicated if it's reprocessed
sdk.Logger(ctx).Warn().Bytes("record_position", records[i].Position).Msg("this message was produced after a previous message failed to be produced - if you restart the pipeline this message will be produced again")
}
}
return count, err
}
return len(records), nil
}
func (p *segmentProducer) sendRetryable(ctx context.Context, messages []kafka.Message) error {
return retry.Do(
func() error {
return p.writer.WriteMessages(
context.Background(),
messages...,
)
},
retry.RetryIf(func(err error) bool {
// this can happen when the topic doesn't exist and the broker has auto-create enabled
// we give it some time to process topic metadata and retry
return errors.Is(err, kafka.LeaderNotAvailable)
}),
retry.OnRetry(func(n uint, err error) {
sdk.Logger(ctx).
Info().
Err(err).
Msgf("retrying write, attempt #%v", n)
}),
retry.Delay(time.Second),
retry.Attempts(10),
retry.LastErrorOnly(true),
)
}
func (p *segmentProducer) Close() error {
if p.writer == nil {
return nil
}
// this will also make the loops in the reader goroutines stop
err := p.writer.Close()
if err != nil {
return fmt.Errorf("couldn't close writer: %w", err)
}
// close idle connections if possible
closeIdleTransport, ok := p.writer.Transport.(interface {
CloseIdleConnections()
})
if ok {
closeIdleTransport.CloseIdleConnections()
}
return nil
}
// batchSizeAdjustingBalancer is a balancer that adjusts the batch size of the
// writer based on the number of messages that we want to write.
// Before calling Writer.WriteMessages you can call SetRecordCount on the
// balancer to let it know how many messages will be written. First time the
// Balance method is called it will calculate the appropriate batch size based
// on the number of partitions and adjust the setting in the writer.
// The actual balancing logic after that is delegated to kafka.RoundRobin.
//
// When the record count is divisible by the number of partitions, this balancer
// will ensure that all batches are flushed instantly. If it's not divisible and
// the record count is greater than the number of partitions, some batches will
// be flushed only after the batch delay is reached.
//
// This is a workaround and best approximation for achieving synchronous writes,
// because the segment client doesn't allow you to write without batching.
// For more info see https://github.com/segmentio/kafka-go/issues/852.
type batchSizeAdjustingBalancer struct {
kafka.RoundRobin
writer *kafka.Writer
recordCount int
once sync.Once
}
func (b *batchSizeAdjustingBalancer) SetRecordCount(count int) {
b.recordCount = count
b.once = sync.Once{} // make sure new batch size is calculated
}
// Balance satisfies the kafka.Balancer interface.
func (b *batchSizeAdjustingBalancer) Balance(msg kafka.Message, partitions ...int) int {
b.once.Do(func() {
// kafka.Writer.BatchSize is the batch size for a single partition. We
// know the number of messages we are writing and because we are using
// the round robin balancer we know that a single partition will receive
// at most ceil(recordCount/partitions) messages. We set the batch size
// accordingly.
// trick for ceil without cast to float: ceil(a/b) = (a+b-1)/b
b.writer.BatchSize = (b.recordCount + len(partitions) - 1) / len(partitions)
})
return b.RoundRobin.Balance(msg, partitions...)
}