forked from IBM/sarama
-
Notifications
You must be signed in to change notification settings - Fork 0
/
producer.go
806 lines (683 loc) · 23.8 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
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
package sarama
import (
"fmt"
"sync"
"time"
"github.com/eapache/go-resiliency/breaker"
)
func forceFlushThreshold() int {
return int(MaxRequestSize - (10 * 1024)) // 10KiB is safety room for misc. overhead, we might want to calculate this more precisely?
}
// ProducerConfig is used to pass multiple configuration options to NewProducer.
type ProducerConfig struct {
Partitioner PartitionerConstructor // Generates partitioners for choosing the partition to send messages to (defaults to hash).
RequiredAcks RequiredAcks // The level of acknowledgement reliability needed from the broker (defaults to WaitForLocal).
Timeout time.Duration // The maximum duration the broker will wait the receipt of the number of RequiredAcks. This is only relevant when RequiredAcks is set to WaitForAll or a number > 1. Only supports millisecond resolution, nanoseconds will be truncated.
Compression CompressionCodec // The type of compression to use on messages (defaults to no compression).
FlushMsgCount int // The number of messages needed to trigger a flush. This is a minimum, not an upper limit (use MaxMessagesPerReq for that).
FlushFrequency time.Duration // If this amount of time elapses without a flush, one will be queued. This is a minimum, not an upper limit.
FlushByteCount int // If this many bytes of messages are accumulated, a flush will be triggered. This is a minimum, not an upper limit.
AckSuccesses bool // If enabled, successfully delivered messages will be returned on the Successes channel.
MaxMessageBytes int // The maximum permitted size of a message (defaults to 1000000)
MaxMessagesPerReq int // The maximum number of messages the producer will send in a single broker request. Defaults to 0 for unlimited. The global setting MaxRequestSize still applies.
ChannelBufferSize int // The size of the buffers of the channels between the different goroutines. Defaults to 0 (unbuffered).
RetryBackoff time.Duration // The amount of time to wait for the cluster to elect a new leader before processing retries. Defaults to 250ms.
}
// NewProducerConfig creates a new ProducerConfig instance with sensible defaults.
func NewProducerConfig() *ProducerConfig {
return &ProducerConfig{
Partitioner: NewHashPartitioner,
RequiredAcks: WaitForLocal,
MaxMessageBytes: 1000000,
RetryBackoff: 250 * time.Millisecond,
}
}
// Validate checks a ProducerConfig instance. It will return a
// ConfigurationError if the specified value doesn't make sense.
func (config *ProducerConfig) Validate() error {
if config.RequiredAcks < -1 {
return ConfigurationError("Invalid RequiredAcks")
} else if config.RequiredAcks > 1 {
Logger.Println("ProducerConfig.RequiredAcks > 1 is deprecated and will raise an exception with kafka >= 0.8.2.0.")
}
if config.Timeout < 0 {
return ConfigurationError("Invalid Timeout")
} else if config.Timeout%time.Millisecond != 0 {
Logger.Println("ProducerConfig.Timeout only supports millisecond resolution; nanoseconds will be truncated.")
}
if config.FlushMsgCount < 0 {
return ConfigurationError("Invalid FlushMsgCount")
}
if config.FlushByteCount < 0 {
return ConfigurationError("Invalid FlushByteCount")
} else if config.FlushByteCount >= forceFlushThreshold() {
Logger.Println("ProducerConfig.FlushByteCount too close to MaxRequestSize; it will be ignored.")
}
if config.FlushFrequency < 0 {
return ConfigurationError("Invalid FlushFrequency")
}
if config.Partitioner == nil {
return ConfigurationError("No partitioner set")
}
if config.MaxMessageBytes <= 0 {
return ConfigurationError("Invalid MaxMessageBytes")
} else if config.MaxMessageBytes >= forceFlushThreshold() {
Logger.Println("ProducerConfig.MaxMessageBytes too close to MaxRequestSize; it will be ignored.")
}
if config.MaxMessagesPerReq < 0 || (config.MaxMessagesPerReq > 0 && config.MaxMessagesPerReq < config.FlushMsgCount) {
return ConfigurationError("Invalid MaxMessagesPerReq, must be non-negative and >= FlushMsgCount if set")
}
if config.RetryBackoff < 0 {
return ConfigurationError("Invalid RetryBackoff")
}
return nil
}
// Producer publishes Kafka messages. 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() on a producer to avoid
// leaks: it will not be garbage-collected automatically when it passes out of
// scope (this is in addition to calling Close on the underlying client, which
// is still necessary).
type Producer struct {
client *Client
config ProducerConfig
errors chan *ProduceError
input, successes, retries chan *MessageToSend
brokers map[*Broker]*brokerWorker
brokerLock sync.Mutex
}
// NewProducer creates a new Producer using the given client.
func NewProducer(client *Client, config *ProducerConfig) (*Producer, error) {
// Check that we are not dealing with a closed Client before processing
// any other arguments
if client.Closed() {
return nil, ClosedClient
}
if config == nil {
config = NewProducerConfig()
}
if err := config.Validate(); err != nil {
return nil, err
}
p := &Producer{
client: client,
config: *config,
errors: make(chan *ProduceError),
input: make(chan *MessageToSend),
successes: make(chan *MessageToSend),
retries: make(chan *MessageToSend),
brokers: make(map[*Broker]*brokerWorker),
}
// launch our singleton dispatchers
go withRecover(p.topicDispatcher)
go withRecover(p.retryHandler)
return p, nil
}
type flagSet int8
const (
retried flagSet = 1 << iota // message has been retried
chaser // message is last in a group that failed
ref // add a reference to a singleton channel
unref // remove a reference from a singleton channel
shutdown // start the shutdown process
)
// MessageToSend is the collection of elements passed to the Producer in order to send a message.
type MessageToSend struct {
Topic string // The Kafka topic for this message.
Key Encoder // The partitioning key for this message. It must implement the Encoder interface. Pre-existing Encoders include StringEncoder and ByteEncoder.
Value Encoder // The actual message to store in Kafka. It must implement the Encoder interface. Pre-existing Encoders include StringEncoder and ByteEncoder.
Metadata interface{} // 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.
// these are filled in by the producer as the message is processed
offset int64
partition int32
flags flagSet
}
// 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.
func (m *MessageToSend) Offset() int64 {
return m.offset
}
// Partition is the partition that the message was sent to. This is only guaranteed to be defined if
// the message was successfully delivered.
func (m *MessageToSend) Partition() int32 {
return m.partition
}
func (m *MessageToSend) byteSize() int {
size := 26 // the metadata overhead of CRC, flags, etc.
if m.Key != nil {
size += m.Key.Length()
}
if m.Value != nil {
size += m.Value.Length()
}
return size
}
// ProduceError is the type of error generated when the producer fails to deliver a message.
// It contains the original MessageToSend as well as the actual error value.
type ProduceError struct {
Msg *MessageToSend
Err error
}
// ProduceErrors is a type that wraps a batch of "ProduceError"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 ProduceErrors []*ProduceError
func (pe ProduceErrors) Error() string {
return fmt.Sprintf("kafka: Failed to deliver %d messages.", len(pe))
}
// Errors is the error output channel back to the user. You MUST read from this channel or the Producer will deadlock.
// It is suggested that you send messages and read errors together in a single select statement.
func (p *Producer) Errors() <-chan *ProduceError {
return p.errors
}
// Successes is the success output channel back to the user when AckSuccesses is configured.
// If AckSuccesses 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.
func (p *Producer) Successes() <-chan *MessageToSend {
return p.successes
}
// Input is the input channel for the user to write messages to that they wish to send.
func (p *Producer) Input() chan<- *MessageToSend {
return p.input
}
// Close shuts down the producer and flushes any messages it may have buffered.
// 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.
func (p *Producer) Close() error {
go withRecover(func() {
p.input <- &MessageToSend{flags: shutdown}
})
if p.config.AckSuccesses {
go withRecover(func() {
for _ = range p.successes {
}
})
}
var errors ProduceErrors
for event := range p.errors {
errors = append(errors, event)
}
close(p.successes)
if len(errors) > 0 {
return errors
}
return nil
}
///////////////////////////////////////////
// In normal processing, a message flows through the following functions from top to bottom,
// starting at topicDispatcher (which reads from Producer.input) and ending in flusher
// (which sends the message to the broker). In cases where a message must be retried, it goes
// through retryHandler before being returned to the top of the flow.
///////////////////////////////////////////
// singleton
// dispatches messages by topic
func (p *Producer) topicDispatcher() {
handlers := make(map[string]chan *MessageToSend)
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 {
Logger.Println("Producer shutting down.")
break
}
if (p.config.Compression == CompressionNone && msg.Value != nil && msg.Value.Length() > p.config.MaxMessageBytes) ||
(msg.byteSize() > p.config.MaxMessageBytes) {
p.returnError(msg, MessageSizeTooLarge)
continue
}
handler := handlers[msg.Topic]
if handler == nil {
p.retries <- &MessageToSend{flags: ref}
newHandler := make(chan *MessageToSend, p.config.ChannelBufferSize)
topic := msg.Topic // block local because go's closure semantics suck
go withRecover(func() { p.partitionDispatcher(topic, newHandler) })
handler = newHandler
handlers[msg.Topic] = handler
}
handler <- msg
}
for _, handler := range handlers {
close(handler)
}
p.retries <- &MessageToSend{flags: shutdown}
for msg := range p.input {
p.returnError(msg, ShuttingDown)
}
close(p.errors)
}
// one per topic
// partitions messages, then dispatches them by partition
func (p *Producer) partitionDispatcher(topic string, input chan *MessageToSend) {
handlers := make(map[int32]chan *MessageToSend)
partitioner := p.config.Partitioner()
for msg := range input {
if msg.flags&retried == 0 {
err := p.assignPartition(partitioner, msg)
if err != nil {
p.returnError(msg, err)
continue
}
}
handler := handlers[msg.partition]
if handler == nil {
p.retries <- &MessageToSend{flags: ref}
newHandler := make(chan *MessageToSend, p.config.ChannelBufferSize)
topic := msg.Topic // block local because go's closure semantics suck
partition := msg.partition // block local because go's closure semantics suck
go withRecover(func() { p.leaderDispatcher(topic, partition, newHandler) })
handler = newHandler
handlers[msg.partition] = handler
}
handler <- msg
}
for _, handler := range handlers {
close(handler)
}
p.retries <- &MessageToSend{flags: unref}
}
// one per partition per topic
// dispatches messages to the appropriate broker
// also responsible for maintaining message order during retries
func (p *Producer) leaderDispatcher(topic string, partition int32, input chan *MessageToSend) {
var leader *Broker
var output chan *MessageToSend
var backlog []*MessageToSend
breaker := breaker.New(3, 1, 10*time.Second)
doUpdate := func() (err error) {
if err = p.client.RefreshTopicMetadata(topic); err != nil {
return err
}
if leader, err = p.client.Leader(topic, partition); err != nil {
return err
}
output = p.getBrokerWorker(leader)
return nil
}
// try to prefetch the leader; if this doesn't work, we'll do a proper breaker-protected refresh-and-fetch
// on the first message
leader, _ = p.client.Leader(topic, partition)
if leader != nil {
output = p.getBrokerWorker(leader)
}
for msg := range input {
if msg.flags&retried == 0 {
// normal case
if backlog != nil {
backlog = append(backlog, msg)
continue
}
} else if msg.flags&chaser == 0 {
// retry flag set, chaser flag not set
if backlog == nil {
// on the very first retried message we send off a chaser 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)
Logger.Printf("producer/leader state change to [retrying] on %s/%d\n", topic, partition)
output <- &MessageToSend{Topic: topic, partition: partition, flags: chaser}
backlog = make([]*MessageToSend, 0)
p.unrefBrokerWorker(leader)
output = nil
time.Sleep(p.config.RetryBackoff)
}
} else {
// retry *and* chaser flag set, flush the backlog and return to normal processing
Logger.Printf("producer/leader state change to [flushing] on %s/%d\n", topic, partition)
if output == nil {
if err := breaker.Run(doUpdate); err != nil {
p.returnErrors(backlog, err)
backlog = nil
continue
}
}
for _, msg := range backlog {
output <- msg
}
Logger.Printf("producer/leader state change to [normal] on %s/%d\n", topic, partition)
backlog = nil
continue
}
if output == nil {
if err := breaker.Run(doUpdate); err != nil {
p.returnError(msg, err)
continue
}
}
output <- msg
}
p.unrefBrokerWorker(leader)
p.retries <- &MessageToSend{flags: unref}
}
// one per broker
// groups messages together into appropriately-sized batches for sending to the broker
// based on https://godoc.org/github.com/eapache/channels#BatchingChannel
func (p *Producer) messageAggregator(broker *Broker, input chan *MessageToSend) {
var ticker *time.Ticker
var timer <-chan time.Time
if p.config.FlushFrequency > 0 {
ticker = time.NewTicker(p.config.FlushFrequency)
timer = ticker.C
}
var buffer []*MessageToSend
var doFlush chan []*MessageToSend
var bytesAccumulated int
flusher := make(chan []*MessageToSend)
go withRecover(func() { p.flusher(broker, flusher) })
for {
select {
case msg := <-input:
if msg == nil {
goto shutdown
}
if (bytesAccumulated+msg.byteSize() >= forceFlushThreshold()) ||
(p.config.Compression != CompressionNone && bytesAccumulated+msg.byteSize() >= p.config.MaxMessageBytes) ||
(p.config.MaxMessagesPerReq > 0 && len(buffer) >= p.config.MaxMessagesPerReq) {
Logger.Println("producer/aggregator maximum request accumulated, forcing blocking flush")
flusher <- buffer
buffer = nil
doFlush = nil
bytesAccumulated = 0
}
buffer = append(buffer, msg)
bytesAccumulated += msg.byteSize()
if len(buffer) >= p.config.FlushMsgCount ||
(p.config.FlushByteCount > 0 && bytesAccumulated >= p.config.FlushByteCount) {
doFlush = flusher
}
case <-timer:
doFlush = flusher
case doFlush <- buffer:
buffer = nil
doFlush = nil
bytesAccumulated = 0
}
}
shutdown:
if ticker != nil {
ticker.Stop()
}
if len(buffer) > 0 {
flusher <- buffer
}
close(flusher)
}
// one per broker
// takes a batch at a time from the messageAggregator and sends to the broker
func (p *Producer) flusher(broker *Broker, input chan []*MessageToSend) {
var closing error
currentRetries := make(map[string]map[int32]error)
for batch := range input {
if closing != nil {
p.retryMessages(batch, closing)
continue
}
// group messages by topic/partition
msgSets := make(map[string]map[int32][]*MessageToSend)
for i, msg := range batch {
if currentRetries[msg.Topic] != nil && currentRetries[msg.Topic][msg.partition] != nil {
if msg.flags&chaser == chaser {
// we can start processing this topic/partition again
Logger.Printf("producer/flusher state change to [normal] on %s/%d\n",
msg.Topic, msg.partition)
currentRetries[msg.Topic][msg.partition] = nil
}
p.retryMessages([]*MessageToSend{msg}, currentRetries[msg.Topic][msg.partition])
batch[i] = nil // to prevent it being returned/retried twice
continue
}
partitionSet := msgSets[msg.Topic]
if partitionSet == nil {
partitionSet = make(map[int32][]*MessageToSend)
msgSets[msg.Topic] = partitionSet
}
partitionSet[msg.partition] = append(partitionSet[msg.partition], msg)
}
request := p.buildRequest(msgSets)
if request == nil {
continue
}
response, err := broker.Produce(p.client.id, request)
switch err {
case nil:
break
case EncodingError:
p.returnErrors(batch, err)
continue
default:
p.client.disconnectBroker(broker)
Logger.Println("producer/flusher state change to [closing] because", err)
closing = err
p.retryMessages(batch, err)
continue
}
if response == nil {
// this only happens when RequiredAcks is NoResponse, so we have to assume success
if p.config.AckSuccesses {
p.returnSuccesses(batch)
}
continue
}
// we iterate through the blocks in the request, not the response, so that we notice
// if the response is missing a block completely
for topic, partitionSet := range msgSets {
for partition, msgs := range partitionSet {
block := response.GetBlock(topic, partition)
if block == nil {
p.returnErrors(msgs, IncompleteResponse)
continue
}
switch block.Err {
case NoError:
// All the messages for this topic-partition were delivered successfully!
if p.config.AckSuccesses {
for i := range msgs {
msgs[i].offset = block.Offset + int64(i)
}
p.returnSuccesses(msgs)
}
case UnknownTopicOrPartition, NotLeaderForPartition, LeaderNotAvailable:
Logger.Printf("producer/flusher state change to [retrying] on %s/%d because %v\n",
topic, partition, block.Err)
if currentRetries[topic] == nil {
currentRetries[topic] = make(map[int32]error)
}
currentRetries[topic][partition] = block.Err
p.retryMessages(msgs, block.Err)
default:
p.returnErrors(msgs, block.Err)
}
}
}
}
p.retries <- &MessageToSend{flags: unref}
}
// singleton
// effectively a "bridge" between the flushers and the topicDispatcher in order to avoid deadlock
// based on https://godoc.org/github.com/eapache/channels#InfiniteChannel
func (p *Producer) retryHandler() {
var buf []*MessageToSend
var msg *MessageToSend
refs := 0
shuttingDown := false
for {
if len(buf) == 0 {
msg = <-p.retries
} else {
select {
case msg = <-p.retries:
case p.input <- buf[0]:
buf = buf[1:]
continue
}
}
if msg.flags&ref != 0 {
refs++
} else if msg.flags&unref != 0 {
refs--
if refs == 0 && shuttingDown {
break
}
} else if msg.flags&shutdown != 0 {
shuttingDown = true
if refs == 0 {
break
}
} else {
buf = append(buf, msg)
}
}
close(p.retries)
for i := range buf {
p.input <- buf[i]
}
close(p.input)
}
///////////////////////////////////////////
///////////////////////////////////////////
// utility functions
func (p *Producer) assignPartition(partitioner Partitioner, msg *MessageToSend) error {
var partitions []int32
var err error
if partitioner.RequiresConsistency() {
partitions, err = p.client.Partitions(msg.Topic)
} else {
partitions, err = p.client.WritablePartitions(msg.Topic)
}
if err != nil {
return err
}
numPartitions := int32(len(partitions))
if numPartitions == 0 {
return LeaderNotAvailable
}
choice, err := partitioner.Partition(msg.Key, numPartitions)
if err != nil {
return err
} else if choice < 0 || choice >= numPartitions {
return InvalidPartition
}
msg.partition = partitions[choice]
return nil
}
func (p *Producer) buildRequest(batch map[string]map[int32][]*MessageToSend) *ProduceRequest {
req := &ProduceRequest{RequiredAcks: p.config.RequiredAcks, Timeout: int32(p.config.Timeout / time.Millisecond)}
empty := true
for topic, partitionSet := range batch {
for partition, msgSet := range partitionSet {
setToSend := new(MessageSet)
setSize := 0
for _, msg := range msgSet {
var keyBytes, valBytes []byte
var err error
if msg.Key != nil {
if keyBytes, err = msg.Key.Encode(); err != nil {
p.returnError(msg, err)
continue
}
}
if msg.Value != nil {
if valBytes, err = msg.Value.Encode(); err != nil {
p.returnError(msg, err)
continue
}
}
if p.config.Compression != CompressionNone && setSize+msg.byteSize() > p.config.MaxMessageBytes {
// compression causes message-sets to be wrapped as single messages, which have tighter
// size requirements, so we have to respect those limits
valBytes, err := encode(setToSend)
if err != nil {
Logger.Println(err) // if this happens, it's basically our fault.
panic(err)
}
req.AddMessage(topic, partition, &Message{Codec: p.config.Compression, Key: nil, Value: valBytes})
setToSend = new(MessageSet)
setSize = 0
}
setSize += msg.byteSize()
setToSend.addMessage(&Message{Codec: CompressionNone, Key: keyBytes, Value: valBytes})
empty = false
}
if p.config.Compression == CompressionNone {
req.AddSet(topic, partition, setToSend)
} else {
valBytes, err := encode(setToSend)
if err != nil {
Logger.Println(err) // if this happens, it's basically our fault.
panic(err)
}
req.AddMessage(topic, partition, &Message{Codec: p.config.Compression, Key: nil, Value: valBytes})
}
}
}
if empty {
return nil
}
return req
}
func (p *Producer) returnError(msg *MessageToSend, err error) {
msg.flags = 0
p.errors <- &ProduceError{Msg: msg, Err: err}
}
func (p *Producer) returnErrors(batch []*MessageToSend, err error) {
for _, msg := range batch {
if msg != nil {
p.returnError(msg, err)
}
}
}
func (p *Producer) returnSuccesses(batch []*MessageToSend) {
for _, msg := range batch {
if msg != nil {
msg.flags = 0
p.successes <- msg
}
}
}
func (p *Producer) retryMessages(batch []*MessageToSend, err error) {
for _, msg := range batch {
if msg == nil {
continue
}
if msg.flags&retried == retried {
p.returnError(msg, err)
} else {
msg.flags |= retried
p.retries <- msg
}
}
}
type brokerWorker struct {
input chan *MessageToSend
refs int
}
func (p *Producer) getBrokerWorker(broker *Broker) chan *MessageToSend {
p.brokerLock.Lock()
defer p.brokerLock.Unlock()
worker := p.brokers[broker]
if worker == nil {
p.retries <- &MessageToSend{flags: ref}
worker = &brokerWorker{
refs: 1,
input: make(chan *MessageToSend),
}
p.brokers[broker] = worker
go withRecover(func() { p.messageAggregator(broker, worker.input) })
} else {
worker.refs++
}
return worker.input
}
func (p *Producer) unrefBrokerWorker(broker *Broker) {
p.brokerLock.Lock()
defer p.brokerLock.Unlock()
worker := p.brokers[broker]
if worker != nil {
worker.refs--
if worker.refs == 0 {
close(worker.input)
delete(p.brokers, broker)
}
}
}