forked from IBM/sarama
-
Notifications
You must be signed in to change notification settings - Fork 0
/
async_producer.go
1116 lines (953 loc) · 31.5 KB
/
async_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
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
package sarama
import (
"encoding/binary"
"fmt"
"sync"
"time"
"github.com/eapache/go-resiliency/breaker"
"github.com/eapache/queue"
)
// AsyncProducer publishes Kafka messages using a non-blocking API. It routes messages
// to the correct broker for the provided topic-partition, refreshing metadata as appropriate,
// and parses responses for errors. You must read from the Errors() channel or the
// producer will deadlock. You must call Close() or AsyncClose() on a producer to avoid
// leaks: it will not be garbage-collected automatically when it passes out of
// scope.
type AsyncProducer interface {
// AsyncClose triggers a shutdown of the producer. The shutdown has completed
// when both the Errors and Successes channels have been closed. When calling
// AsyncClose, you *must* continue to read from those channels in order to
// drain the results of any messages in flight.
AsyncClose()
// Close shuts down the producer and waits for any buffered messages to be
// flushed. You must call this function before a producer object passes out of
// scope, as it may otherwise leak memory. You must call this before calling
// Close on the underlying client.
Close() error
// Input is the input channel for the user to write messages to that they
// wish to send.
Input() chan<- *ProducerMessage
// Successes is the success output channel back to the user when Return.Successes is
// enabled. If Return.Successes is true, you MUST read from this channel or the
// Producer will deadlock. It is suggested that you send and read messages
// together in a single select statement.
Successes() <-chan *ProducerMessage
// Errors is the error output channel back to the user. You MUST read from this
// channel or the Producer will deadlock when the channel is full. Alternatively,
// you can set Producer.Return.Errors in your config to false, which prevents
// errors to be returned.
Errors() <-chan *ProducerError
}
// transactionManager keeps the state necessary to ensure idempotent production
type transactionManager struct {
producerID int64
producerEpoch int16
sequenceNumbers map[string]int32
mutex sync.Mutex
}
const (
noProducerID = -1
noProducerEpoch = -1
)
func (t *transactionManager) getAndIncrementSequenceNumber(topic string, partition int32) int32 {
key := fmt.Sprintf("%s-%d", topic, partition)
t.mutex.Lock()
defer t.mutex.Unlock()
sequence := t.sequenceNumbers[key]
t.sequenceNumbers[key] = sequence + 1
return sequence
}
func newTransactionManager(conf *Config, client Client) (*transactionManager, error) {
txnmgr := &transactionManager{
producerID: noProducerID,
producerEpoch: noProducerEpoch,
}
if conf.Producer.Idempotent {
initProducerIDResponse, err := client.InitProducerID()
if err != nil {
return nil, err
}
txnmgr.producerID = initProducerIDResponse.ProducerID
txnmgr.producerEpoch = initProducerIDResponse.ProducerEpoch
txnmgr.sequenceNumbers = make(map[string]int32)
txnmgr.mutex = sync.Mutex{}
Logger.Printf("Obtained a ProducerId: %d and ProducerEpoch: %d\n", txnmgr.producerID, txnmgr.producerEpoch)
}
return txnmgr, nil
}
type asyncProducer struct {
client Client
conf *Config
errors chan *ProducerError
input, successes, retries chan *ProducerMessage
inFlight sync.WaitGroup
brokers map[*Broker]*brokerProducer
brokerRefs map[*brokerProducer]int
brokerLock sync.Mutex
txnmgr *transactionManager
}
// NewAsyncProducer creates a new AsyncProducer using the given broker addresses and configuration.
func NewAsyncProducer(addrs []string, conf *Config) (AsyncProducer, error) {
client, err := NewClient(addrs, conf)
if err != nil {
return nil, err
}
return newAsyncProducer(client)
}
// NewAsyncProducerFromClient creates a new Producer using the given client. It is still
// necessary to call Close() on the underlying client when shutting down this producer.
func NewAsyncProducerFromClient(client Client) (AsyncProducer, error) {
// For clients passed in by the client, ensure we don't
// call Close() on it.
cli := &nopCloserClient{client}
return newAsyncProducer(cli)
}
func newAsyncProducer(client Client) (AsyncProducer, error) {
// Check that we are not dealing with a closed Client before processing any other arguments
if client.Closed() {
return nil, ErrClosedClient
}
txnmgr, err := newTransactionManager(client.Config(), client)
if err != nil {
return nil, err
}
p := &asyncProducer{
client: client,
conf: client.Config(),
errors: make(chan *ProducerError),
input: make(chan *ProducerMessage),
successes: make(chan *ProducerMessage),
retries: make(chan *ProducerMessage),
brokers: make(map[*Broker]*brokerProducer),
brokerRefs: make(map[*brokerProducer]int),
txnmgr: txnmgr,
}
// launch our singleton dispatchers
go withRecover(p.dispatcher)
go withRecover(p.retryHandler)
return p, nil
}
type flagSet int8
const (
syn flagSet = 1 << iota // first message from partitionProducer to brokerProducer
fin // final message from partitionProducer to brokerProducer and back
shutdown // start the shutdown process
)
// ProducerMessage is the collection of elements passed to the Producer in order to send a message.
type ProducerMessage struct {
Topic string // The Kafka topic for this message.
// The partitioning key for this message. Pre-existing Encoders include
// StringEncoder and ByteEncoder.
Key Encoder
// The actual message to store in Kafka. Pre-existing Encoders include
// StringEncoder and ByteEncoder.
Value Encoder
// The headers are key-value pairs that are transparently passed
// by Kafka between producers and consumers.
Headers []RecordHeader
// This field is used to hold arbitrary data you wish to include so it
// will be available when receiving on the Successes and Errors channels.
// Sarama completely ignores this field and is only to be used for
// pass-through data.
Metadata interface{}
// Below this point are filled in by the producer as the message is processed
// Offset is the offset of the message stored on the broker. This is only
// guaranteed to be defined if the message was successfully delivered and
// RequiredAcks is not NoResponse.
Offset int64
// Partition is the partition that the message was sent to. This is only
// guaranteed to be defined if the message was successfully delivered.
Partition int32
// Timestamp can vary in behaviour depending on broker configuration, being
// in either one of the CreateTime or LogAppendTime modes (default CreateTime),
// and requiring version at least 0.10.0.
//
// When configured to CreateTime, the timestamp is specified by the producer
// either by explicitly setting this field, or when the message is added
// to a produce set.
//
// When configured to LogAppendTime, the timestamp assigned to the message
// by the broker. This is only guaranteed to be defined if the message was
// successfully delivered and RequiredAcks is not NoResponse.
Timestamp time.Time
retries int
flags flagSet
expectation chan *ProducerError
sequenceNumber int32
}
const producerMessageOverhead = 26 // the metadata overhead of CRC, flags, etc.
func (m *ProducerMessage) byteSize(version int) int {
var size int
if version >= 2 {
size = maximumRecordOverhead
for _, h := range m.Headers {
size += len(h.Key) + len(h.Value) + 2*binary.MaxVarintLen32
}
} else {
size = producerMessageOverhead
}
if m.Key != nil {
size += m.Key.Length()
}
if m.Value != nil {
size += m.Value.Length()
}
return size
}
func (m *ProducerMessage) clear() {
m.flags = 0
m.retries = 0
}
// ProducerError is the type of error generated when the producer fails to deliver a message.
// It contains the original ProducerMessage as well as the actual error value.
type ProducerError struct {
Msg *ProducerMessage
Err error
}
func (pe ProducerError) Error() string {
return fmt.Sprintf("kafka: Failed to produce message to topic %s: %s", pe.Msg.Topic, pe.Err)
}
// ProducerErrors is a type that wraps a batch of "ProducerError"s and implements the Error interface.
// It can be returned from the Producer's Close method to avoid the need to manually drain the Errors channel
// when closing a producer.
type ProducerErrors []*ProducerError
func (pe ProducerErrors) Error() string {
return fmt.Sprintf("kafka: Failed to deliver %d messages.", len(pe))
}
func (p *asyncProducer) Errors() <-chan *ProducerError {
return p.errors
}
func (p *asyncProducer) Successes() <-chan *ProducerMessage {
return p.successes
}
func (p *asyncProducer) Input() chan<- *ProducerMessage {
return p.input
}
func (p *asyncProducer) Close() error {
p.AsyncClose()
if p.conf.Producer.Return.Successes {
go withRecover(func() {
for range p.successes {
}
})
}
var errors ProducerErrors
if p.conf.Producer.Return.Errors {
for event := range p.errors {
errors = append(errors, event)
}
} else {
<-p.errors
}
if len(errors) > 0 {
return errors
}
return nil
}
func (p *asyncProducer) AsyncClose() {
go withRecover(p.shutdown)
}
// singleton
// dispatches messages by topic
func (p *asyncProducer) dispatcher() {
handlers := make(map[string]chan<- *ProducerMessage)
shuttingDown := false
for msg := range p.input {
if msg == nil {
Logger.Println("Something tried to send a nil message, it was ignored.")
continue
}
if msg.flags&shutdown != 0 {
shuttingDown = true
p.inFlight.Done()
continue
} else if msg.retries == 0 {
if shuttingDown {
// we can't just call returnError here because that decrements the wait group,
// which hasn't been incremented yet for this message, and shouldn't be
pErr := &ProducerError{Msg: msg, Err: ErrShuttingDown}
if p.conf.Producer.Return.Errors {
p.errors <- pErr
} else {
Logger.Println(pErr)
}
continue
}
p.inFlight.Add(1)
}
version := 1
if p.conf.Version.IsAtLeast(V0_11_0_0) {
version = 2
} else if msg.Headers != nil {
p.returnError(msg, ConfigurationError("Producing headers requires Kafka at least v0.11"))
continue
}
if msg.byteSize(version) > p.conf.Producer.MaxMessageBytes {
p.returnError(msg, ErrMessageSizeTooLarge)
continue
}
handler := handlers[msg.Topic]
if handler == nil {
handler = p.newTopicProducer(msg.Topic)
handlers[msg.Topic] = handler
}
handler <- msg
}
for _, handler := range handlers {
close(handler)
}
}
// one per topic
// partitions messages, then dispatches them by partition
type topicProducer struct {
parent *asyncProducer
topic string
input <-chan *ProducerMessage
breaker *breaker.Breaker
handlers map[int32]chan<- *ProducerMessage
partitioner Partitioner
}
func (p *asyncProducer) newTopicProducer(topic string) chan<- *ProducerMessage {
input := make(chan *ProducerMessage, p.conf.ChannelBufferSize)
tp := &topicProducer{
parent: p,
topic: topic,
input: input,
breaker: breaker.New(3, 1, 10*time.Second),
handlers: make(map[int32]chan<- *ProducerMessage),
partitioner: p.conf.Producer.Partitioner(topic),
}
go withRecover(tp.dispatch)
return input
}
func (tp *topicProducer) dispatch() {
for msg := range tp.input {
if msg.retries == 0 {
if err := tp.partitionMessage(msg); err != nil {
tp.parent.returnError(msg, err)
continue
}
}
// All messages being retried (sent or not) have already had their retry count updated
if tp.parent.conf.Producer.Idempotent && msg.retries == 0 {
msg.sequenceNumber = tp.parent.txnmgr.getAndIncrementSequenceNumber(msg.Topic, msg.Partition)
}
handler := tp.handlers[msg.Partition]
if handler == nil {
handler = tp.parent.newPartitionProducer(msg.Topic, msg.Partition)
tp.handlers[msg.Partition] = handler
}
handler <- msg
}
for _, handler := range tp.handlers {
close(handler)
}
}
func (tp *topicProducer) partitionMessage(msg *ProducerMessage) error {
var partitions []int32
err := tp.breaker.Run(func() (err error) {
requiresConsistency := false
if ep, ok := tp.partitioner.(DynamicConsistencyPartitioner); ok {
requiresConsistency = ep.MessageRequiresConsistency(msg)
} else {
requiresConsistency = tp.partitioner.RequiresConsistency()
}
if requiresConsistency {
partitions, err = tp.parent.client.Partitions(msg.Topic)
} else {
partitions, err = tp.parent.client.WritablePartitions(msg.Topic)
}
return
})
if err != nil {
return err
}
numPartitions := int32(len(partitions))
if numPartitions == 0 {
return ErrLeaderNotAvailable
}
choice, err := tp.partitioner.Partition(msg, numPartitions)
if err != nil {
return err
} else if choice < 0 || choice >= numPartitions {
return ErrInvalidPartition
}
msg.Partition = partitions[choice]
return nil
}
// one per partition per topic
// dispatches messages to the appropriate broker
// also responsible for maintaining message order during retries
type partitionProducer struct {
parent *asyncProducer
topic string
partition int32
input <-chan *ProducerMessage
leader *Broker
breaker *breaker.Breaker
brokerProducer *brokerProducer
// highWatermark tracks the "current" retry level, which is the only one where we actually let messages through,
// all other messages get buffered in retryState[msg.retries].buf to preserve ordering
// retryState[msg.retries].expectChaser simply tracks whether we've seen a fin message for a given level (and
// therefore whether our buffer is complete and safe to flush)
highWatermark int
retryState []partitionRetryState
}
type partitionRetryState struct {
buf []*ProducerMessage
expectChaser bool
}
func (p *asyncProducer) newPartitionProducer(topic string, partition int32) chan<- *ProducerMessage {
input := make(chan *ProducerMessage, p.conf.ChannelBufferSize)
pp := &partitionProducer{
parent: p,
topic: topic,
partition: partition,
input: input,
breaker: breaker.New(3, 1, 10*time.Second),
retryState: make([]partitionRetryState, p.conf.Producer.Retry.Max+1),
}
go withRecover(pp.dispatch)
return input
}
func (pp *partitionProducer) backoff(retries int) {
var backoff time.Duration
if pp.parent.conf.Producer.Retry.BackoffFunc != nil {
maxRetries := pp.parent.conf.Producer.Retry.Max
backoff = pp.parent.conf.Producer.Retry.BackoffFunc(retries, maxRetries)
} else {
backoff = pp.parent.conf.Producer.Retry.Backoff
}
if backoff > 0 {
time.Sleep(backoff)
}
}
func (pp *partitionProducer) dispatch() {
// try to prefetch the leader; if this doesn't work, we'll do a proper call to `updateLeader`
// on the first message
pp.leader, _ = pp.parent.client.Leader(pp.topic, pp.partition)
if pp.leader != nil {
pp.brokerProducer = pp.parent.getBrokerProducer(pp.leader)
pp.parent.inFlight.Add(1) // we're generating a syn message; track it so we don't shut down while it's still inflight
pp.brokerProducer.input <- &ProducerMessage{Topic: pp.topic, Partition: pp.partition, flags: syn}
}
defer func() {
if pp.brokerProducer != nil {
pp.parent.unrefBrokerProducer(pp.leader, pp.brokerProducer)
}
}()
for msg := range pp.input {
if pp.brokerProducer != nil && pp.brokerProducer.abandoned != nil {
select {
case <-pp.brokerProducer.abandoned:
// a message on the abandoned channel means that our current broker selection is out of date
Logger.Printf("producer/leader/%s/%d abandoning broker %d\n", pp.topic, pp.partition, pp.leader.ID())
pp.parent.unrefBrokerProducer(pp.leader, pp.brokerProducer)
pp.brokerProducer = nil
time.Sleep(pp.parent.conf.Producer.Retry.Backoff)
default:
// producer connection is still open.
}
}
if msg.retries > pp.highWatermark {
// a new, higher, retry level; handle it and then back off
pp.newHighWatermark(msg.retries)
pp.backoff(msg.retries)
} else if pp.highWatermark > 0 {
// we are retrying something (else highWatermark would be 0) but this message is not a *new* retry level
if msg.retries < pp.highWatermark {
// in fact this message is not even the current retry level, so buffer it for now (unless it's a just a fin)
if msg.flags&fin == fin {
pp.retryState[msg.retries].expectChaser = false
pp.parent.inFlight.Done() // this fin is now handled and will be garbage collected
} else {
pp.retryState[msg.retries].buf = append(pp.retryState[msg.retries].buf, msg)
}
continue
} else if msg.flags&fin == fin {
// this message is of the current retry level (msg.retries == highWatermark) and the fin flag is set,
// meaning this retry level is done and we can go down (at least) one level and flush that
pp.retryState[pp.highWatermark].expectChaser = false
pp.flushRetryBuffers()
pp.parent.inFlight.Done() // this fin is now handled and will be garbage collected
continue
}
}
// if we made it this far then the current msg contains real data, and can be sent to the next goroutine
// without breaking any of our ordering guarantees
if pp.brokerProducer == nil {
if err := pp.updateLeader(); err != nil {
pp.parent.returnError(msg, err)
pp.backoff(msg.retries)
continue
}
Logger.Printf("producer/leader/%s/%d selected broker %d\n", pp.topic, pp.partition, pp.leader.ID())
}
pp.brokerProducer.input <- msg
}
}
func (pp *partitionProducer) newHighWatermark(hwm int) {
Logger.Printf("producer/leader/%s/%d state change to [retrying-%d]\n", pp.topic, pp.partition, hwm)
pp.highWatermark = hwm
// send off a fin so that we know when everything "in between" has made it
// back to us and we can safely flush the backlog (otherwise we risk re-ordering messages)
pp.retryState[pp.highWatermark].expectChaser = true
pp.parent.inFlight.Add(1) // we're generating a fin message; track it so we don't shut down while it's still inflight
pp.brokerProducer.input <- &ProducerMessage{Topic: pp.topic, Partition: pp.partition, flags: fin, retries: pp.highWatermark - 1}
// a new HWM means that our current broker selection is out of date
Logger.Printf("producer/leader/%s/%d abandoning broker %d\n", pp.topic, pp.partition, pp.leader.ID())
pp.parent.unrefBrokerProducer(pp.leader, pp.brokerProducer)
pp.brokerProducer = nil
}
func (pp *partitionProducer) flushRetryBuffers() {
Logger.Printf("producer/leader/%s/%d state change to [flushing-%d]\n", pp.topic, pp.partition, pp.highWatermark)
for {
pp.highWatermark--
if pp.brokerProducer == nil {
if err := pp.updateLeader(); err != nil {
pp.parent.returnErrors(pp.retryState[pp.highWatermark].buf, err)
goto flushDone
}
Logger.Printf("producer/leader/%s/%d selected broker %d\n", pp.topic, pp.partition, pp.leader.ID())
}
for _, msg := range pp.retryState[pp.highWatermark].buf {
pp.brokerProducer.input <- msg
}
flushDone:
pp.retryState[pp.highWatermark].buf = nil
if pp.retryState[pp.highWatermark].expectChaser {
Logger.Printf("producer/leader/%s/%d state change to [retrying-%d]\n", pp.topic, pp.partition, pp.highWatermark)
break
} else if pp.highWatermark == 0 {
Logger.Printf("producer/leader/%s/%d state change to [normal]\n", pp.topic, pp.partition)
break
}
}
}
func (pp *partitionProducer) updateLeader() error {
return pp.breaker.Run(func() (err error) {
if err = pp.parent.client.RefreshMetadata(pp.topic); err != nil {
return err
}
if pp.leader, err = pp.parent.client.Leader(pp.topic, pp.partition); err != nil {
return err
}
pp.brokerProducer = pp.parent.getBrokerProducer(pp.leader)
pp.parent.inFlight.Add(1) // we're generating a syn message; track it so we don't shut down while it's still inflight
pp.brokerProducer.input <- &ProducerMessage{Topic: pp.topic, Partition: pp.partition, flags: syn}
return nil
})
}
// one per broker; also constructs an associated flusher
func (p *asyncProducer) newBrokerProducer(broker *Broker) *brokerProducer {
var (
input = make(chan *ProducerMessage)
bridge = make(chan *produceSet)
responses = make(chan *brokerProducerResponse)
)
bp := &brokerProducer{
parent: p,
broker: broker,
input: input,
output: bridge,
responses: responses,
stopchan: make(chan struct{}),
buffer: newProduceSet(p),
currentRetries: make(map[string]map[int32]error),
}
go withRecover(bp.run)
// minimal bridge to make the network response `select`able
go withRecover(func() {
for set := range bridge {
request := set.buildRequest()
response, err := broker.Produce(request)
responses <- &brokerProducerResponse{
set: set,
err: err,
res: response,
}
}
close(responses)
})
if p.conf.Producer.Retry.Max <= 0 {
bp.abandoned = make(chan struct{})
}
return bp
}
type brokerProducerResponse struct {
set *produceSet
err error
res *ProduceResponse
}
// groups messages together into appropriately-sized batches for sending to the broker
// handles state related to retries etc
type brokerProducer struct {
parent *asyncProducer
broker *Broker
input chan *ProducerMessage
output chan<- *produceSet
responses <-chan *brokerProducerResponse
abandoned chan struct{}
stopchan chan struct{}
buffer *produceSet
timer <-chan time.Time
timerFired bool
closing error
currentRetries map[string]map[int32]error
}
func (bp *brokerProducer) run() {
var output chan<- *produceSet
Logger.Printf("producer/broker/%d starting up\n", bp.broker.ID())
for {
select {
case msg, ok := <-bp.input:
if !ok {
Logger.Printf("producer/broker/%d input chan closed\n", bp.broker.ID())
bp.shutdown()
return
}
if msg == nil {
continue
}
if msg.flags&syn == syn {
Logger.Printf("producer/broker/%d state change to [open] on %s/%d\n",
bp.broker.ID(), msg.Topic, msg.Partition)
if bp.currentRetries[msg.Topic] == nil {
bp.currentRetries[msg.Topic] = make(map[int32]error)
}
bp.currentRetries[msg.Topic][msg.Partition] = nil
bp.parent.inFlight.Done()
continue
}
if reason := bp.needsRetry(msg); reason != nil {
bp.parent.retryMessage(msg, reason)
if bp.closing == nil && msg.flags&fin == fin {
// we were retrying this partition but we can start processing again
delete(bp.currentRetries[msg.Topic], msg.Partition)
Logger.Printf("producer/broker/%d state change to [closed] on %s/%d\n",
bp.broker.ID(), msg.Topic, msg.Partition)
}
continue
}
if bp.buffer.wouldOverflow(msg) {
if err := bp.waitForSpace(msg); err != nil {
bp.parent.retryMessage(msg, err)
continue
}
}
if err := bp.buffer.add(msg); err != nil {
bp.parent.returnError(msg, err)
continue
}
if bp.parent.conf.Producer.Flush.Frequency > 0 && bp.timer == nil {
bp.timer = time.After(bp.parent.conf.Producer.Flush.Frequency)
}
case <-bp.timer:
bp.timerFired = true
case output <- bp.buffer:
bp.rollOver()
case response, ok := <-bp.responses:
if ok {
bp.handleResponse(response)
}
case <-bp.stopchan:
Logger.Printf(
"producer/broker/%d run loop asked to stop\n", bp.broker.ID())
return
}
if bp.timerFired || bp.buffer.readyToFlush() {
output = bp.output
} else {
output = nil
}
}
}
func (bp *brokerProducer) shutdown() {
for !bp.buffer.empty() {
select {
case response := <-bp.responses:
bp.handleResponse(response)
case bp.output <- bp.buffer:
bp.rollOver()
}
}
close(bp.output)
for response := range bp.responses {
bp.handleResponse(response)
}
close(bp.stopchan)
Logger.Printf("producer/broker/%d shut down\n", bp.broker.ID())
}
func (bp *brokerProducer) needsRetry(msg *ProducerMessage) error {
if bp.closing != nil {
return bp.closing
}
return bp.currentRetries[msg.Topic][msg.Partition]
}
func (bp *brokerProducer) waitForSpace(msg *ProducerMessage) error {
Logger.Printf("producer/broker/%d maximum request accumulated, waiting for space\n", bp.broker.ID())
for {
select {
case response := <-bp.responses:
bp.handleResponse(response)
// handling a response can change our state, so re-check some things
if reason := bp.needsRetry(msg); reason != nil {
return reason
} else if !bp.buffer.wouldOverflow(msg) {
return nil
}
case bp.output <- bp.buffer:
bp.rollOver()
return nil
}
}
}
func (bp *brokerProducer) rollOver() {
bp.timer = nil
bp.timerFired = false
bp.buffer = newProduceSet(bp.parent)
}
func (bp *brokerProducer) handleResponse(response *brokerProducerResponse) {
if response.err != nil {
bp.handleError(response.set, response.err)
} else {
bp.handleSuccess(response.set, response.res)
}
if bp.buffer.empty() {
bp.rollOver() // this can happen if the response invalidated our buffer
}
}
func (bp *brokerProducer) handleSuccess(sent *produceSet, response *ProduceResponse) {
// we iterate through the blocks in the request set, not the response, so that we notice
// if the response is missing a block completely
var retryTopics []string
sent.eachPartition(func(topic string, partition int32, pSet *partitionSet) {
if response == nil {
// this only happens when RequiredAcks is NoResponse, so we have to assume success
bp.parent.returnSuccesses(pSet.msgs)
return
}
block := response.GetBlock(topic, partition)
if block == nil {
bp.parent.returnErrors(pSet.msgs, ErrIncompleteResponse)
return
}
switch block.Err {
// Success
case ErrNoError:
if bp.parent.conf.Version.IsAtLeast(V0_10_0_0) && !block.Timestamp.IsZero() {
for _, msg := range pSet.msgs {
msg.Timestamp = block.Timestamp
}
}
for i, msg := range pSet.msgs {
msg.Offset = block.Offset + int64(i)
}
bp.parent.returnSuccesses(pSet.msgs)
// Duplicate
case ErrDuplicateSequenceNumber:
bp.parent.returnSuccesses(pSet.msgs)
// Retriable errors
case ErrInvalidMessage, ErrUnknownTopicOrPartition, ErrLeaderNotAvailable, ErrNotLeaderForPartition,
ErrRequestTimedOut, ErrNotEnoughReplicas, ErrNotEnoughReplicasAfterAppend:
if bp.parent.conf.Producer.Retry.Max <= 0 {
bp.parent.abandonBrokerConnection(bp.broker)
bp.parent.returnErrors(pSet.msgs, block.Err)
} else {
retryTopics = append(retryTopics, topic)
}
// Other non-retriable errors
default:
if bp.parent.conf.Producer.Retry.Max <= 0 {
bp.parent.abandonBrokerConnection(bp.broker)
}
bp.parent.returnErrors(pSet.msgs, block.Err)
}
})
if len(retryTopics) > 0 {
if bp.parent.conf.Producer.Idempotent {
err := bp.parent.client.RefreshMetadata(retryTopics...)
if err != nil {
Logger.Printf("Failed refreshing metadata because of %v\n", err)
}
}
sent.eachPartition(func(topic string, partition int32, pSet *partitionSet) {
block := response.GetBlock(topic, partition)
if block == nil {
// handled in the previous "eachPartition" loop
return
}
switch block.Err {
case ErrInvalidMessage, ErrUnknownTopicOrPartition, ErrLeaderNotAvailable, ErrNotLeaderForPartition,
ErrRequestTimedOut, ErrNotEnoughReplicas, ErrNotEnoughReplicasAfterAppend:
Logger.Printf("producer/broker/%d state change to [retrying] on %s/%d because %v\n",
bp.broker.ID(), topic, partition, block.Err)
if bp.currentRetries[topic] == nil {
bp.currentRetries[topic] = make(map[int32]error)
}
bp.currentRetries[topic][partition] = block.Err
if bp.parent.conf.Producer.Idempotent {
go bp.parent.retryBatch(topic, partition, pSet, block.Err)
} else {
bp.parent.retryMessages(pSet.msgs, block.Err)
}
// dropping the following messages has the side effect of incrementing their retry count
bp.parent.retryMessages(bp.buffer.dropPartition(topic, partition), block.Err)
}
})
}
}
func (p *asyncProducer) retryBatch(topic string, partition int32, pSet *partitionSet, kerr KError) {
Logger.Printf("Retrying batch for %v-%d because of %s\n", topic, partition, kerr)
produceSet := newProduceSet(p)
produceSet.msgs[topic] = make(map[int32]*partitionSet)
produceSet.msgs[topic][partition] = pSet
produceSet.bufferBytes += pSet.bufferBytes
produceSet.bufferCount += len(pSet.msgs)
for _, msg := range pSet.msgs {
if msg.retries >= p.conf.Producer.Retry.Max {
p.returnError(msg, kerr)
return
}
msg.retries++
}
// it's expected that a metadata refresh has been requested prior to calling retryBatch
leader, err := p.client.Leader(topic, partition)
if err != nil {
Logger.Printf("Failed retrying batch for %v-%d because of %v while looking up for new leader\n", topic, partition, err)
for _, msg := range pSet.msgs {
p.returnError(msg, kerr)
}
return
}
bp := p.getBrokerProducer(leader)
bp.output <- produceSet
}
func (bp *brokerProducer) handleError(sent *produceSet, err error) {
switch err.(type) {
case PacketEncodingError:
sent.eachPartition(func(topic string, partition int32, pSet *partitionSet) {
bp.parent.returnErrors(pSet.msgs, err)
})
default:
Logger.Printf("producer/broker/%d state change to [closing] because %s\n", bp.broker.ID(), err)
bp.parent.abandonBrokerConnection(bp.broker)
_ = bp.broker.Close()
bp.closing = err
sent.eachPartition(func(topic string, partition int32, pSet *partitionSet) {
bp.parent.retryMessages(pSet.msgs, err)
})
bp.buffer.eachPartition(func(topic string, partition int32, pSet *partitionSet) {
bp.parent.retryMessages(pSet.msgs, err)
})
bp.rollOver()
}
}
// singleton
// effectively a "bridge" between the flushers and the dispatcher in order to avoid deadlock
// based on https://godoc.org/github.com/eapache/channels#InfiniteChannel
func (p *asyncProducer) retryHandler() {
var msg *ProducerMessage
buf := queue.New()
for {
if buf.Length() == 0 {
msg = <-p.retries
} else {
select {
case msg = <-p.retries:
case p.input <- buf.Peek().(*ProducerMessage):
buf.Remove()
continue