-
Notifications
You must be signed in to change notification settings - Fork 1.8k
/
Copy pathconsumer.go
554 lines (467 loc) · 15.2 KB
/
consumer.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
package sarama
import (
"fmt"
"sync"
"time"
)
// ConsumerMessage encapsulates a Kafka message returned by the consumer.
type ConsumerMessage struct {
Key, Value []byte
Topic string
Partition int32
Offset int64
}
// ConsumerError is what is provided to the user when an error occurs.
// It wraps an error and includes the topic and partition.
type ConsumerError struct {
Topic string
Partition int32
Err error
}
func (ce ConsumerError) Error() string {
return fmt.Sprintf("kafka: error while consuming %s/%d: %s", ce.Topic, ce.Partition, ce.Err)
}
// ConsumerErrors is a type that wraps a batch of errors and implements the Error interface.
// It can be returned from the PartitionConsumer's Close methods to avoid the need to manually drain errors
// when stopping.
type ConsumerErrors []*ConsumerError
func (ce ConsumerErrors) Error() string {
return fmt.Sprintf("kafka: %d errors while consuming", len(ce))
}
// Consumer manages PartitionConsumers which process Kafka messages from brokers. You MUST call Close()
// on a consumer to avoid leaks, it will not be garbage-collected automatically when it passes out of
// scope.
type Consumer struct {
client *Client
conf *Config
ownClient bool
lock sync.Mutex
children map[string]map[int32]*PartitionConsumer
brokerConsumers map[*Broker]*brokerConsumer
}
// NewConsumer creates a new consumer using the given broker addresses and configuration.
func NewConsumer(addrs []string, config *Config) (*Consumer, error) {
client, err := NewClient(addrs, config)
if err != nil {
return nil, err
}
c, err := NewConsumerFromClient(client)
if err != nil {
return nil, err
}
c.ownClient = true
return c, nil
}
// NewConsumerFromClient creates a new consumer using the given client.
func NewConsumerFromClient(client *Client) (*Consumer, error) {
// Check that we are not dealing with a closed Client before processing any other arguments
if client.Closed() {
return nil, ErrClosedClient
}
c := &Consumer{
client: client,
conf: client.conf,
children: make(map[string]map[int32]*PartitionConsumer),
brokerConsumers: make(map[*Broker]*brokerConsumer),
}
return c, nil
}
// Close shuts down the consumer. It must be called after all child PartitionConsumers have already been closed.
func (c *Consumer) Close() error {
if c.ownClient {
return c.client.Close()
}
return nil
}
const (
// OffsetNewest causes the consumer to start at the most recent available offset, as
// determined by querying the broker.
OffsetNewest int64 = -1
// OffsetOldest causes the consumer to start at the oldest available offset, as
// determined by querying the broker.
OffsetOldest int64 = -2
)
// ConsumePartition creates a PartitionConsumer on the given topic/partition with the given offset. It will
// return an error if this Consumer is already consuming on the given topic/partition. Offset can be a
// literal offset, or OffsetNewest or OffsetOldest
func (c *Consumer) ConsumePartition(topic string, partition int32, offset int64) (*PartitionConsumer, error) {
child := &PartitionConsumer{
consumer: c,
conf: c.conf,
topic: topic,
partition: partition,
messages: make(chan *ConsumerMessage, c.conf.ChannelBufferSize),
errors: make(chan *ConsumerError, c.conf.ChannelBufferSize),
trigger: make(chan none, 1),
dying: make(chan none),
fetchSize: c.conf.Consumer.Fetch.Default,
}
if err := child.chooseStartingOffset(offset); err != nil {
return nil, err
}
if leader, err := c.client.Leader(child.topic, child.partition); err != nil {
return nil, err
} else {
child.broker = leader
}
if err := c.addChild(child); err != nil {
return nil, err
}
go withRecover(child.dispatcher)
brokerWorker := c.refBrokerConsumer(child.broker)
brokerWorker.input <- child
return child, nil
}
func (c *Consumer) addChild(child *PartitionConsumer) error {
c.lock.Lock()
defer c.lock.Unlock()
topicChildren := c.children[child.topic]
if topicChildren == nil {
topicChildren = make(map[int32]*PartitionConsumer)
c.children[child.topic] = topicChildren
}
if topicChildren[child.partition] != nil {
return ConfigurationError("That topic/partition is already being consumed")
}
topicChildren[child.partition] = child
return nil
}
func (c *Consumer) removeChild(child *PartitionConsumer) {
c.lock.Lock()
defer c.lock.Unlock()
delete(c.children[child.topic], child.partition)
}
func (c *Consumer) refBrokerConsumer(broker *Broker) *brokerConsumer {
c.lock.Lock()
defer c.lock.Unlock()
brokerWorker := c.brokerConsumers[broker]
if brokerWorker == nil {
brokerWorker = &brokerConsumer{
consumer: c,
broker: broker,
input: make(chan *PartitionConsumer),
newSubscriptions: make(chan []*PartitionConsumer),
wait: make(chan none),
subscriptions: make(map[*PartitionConsumer]none),
refs: 1,
}
go withRecover(brokerWorker.subscriptionManager)
go withRecover(brokerWorker.subscriptionConsumer)
c.brokerConsumers[broker] = brokerWorker
} else {
brokerWorker.refs++
}
return brokerWorker
}
func (c *Consumer) unrefBrokerConsumer(broker *Broker) {
c.lock.Lock()
defer c.lock.Unlock()
brokerWorker := c.brokerConsumers[broker]
brokerWorker.refs--
if brokerWorker.refs == 0 {
close(brokerWorker.input)
delete(c.brokerConsumers, broker)
}
}
// PartitionConsumer
// PartitionConsumer processes Kafka messages from a given topic and partition. You MUST call Close()
// on a consumer 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 consumer's client, which is still necessary).
// You have to read from both the Messages and Errors channels to prevent the consumer from locking eventually.
type PartitionConsumer struct {
consumer *Consumer
conf *Config
topic string
partition int32
broker *Broker
messages chan *ConsumerMessage
errors chan *ConsumerError
trigger, dying chan none
fetchSize int32
offset int64
}
func (child *PartitionConsumer) sendError(err error) {
child.errors <- &ConsumerError{
Topic: child.topic,
Partition: child.partition,
Err: err,
}
}
func (child *PartitionConsumer) dispatcher() {
for _ = range child.trigger {
select {
case <-child.dying:
close(child.trigger)
default:
if child.broker != nil {
child.consumer.unrefBrokerConsumer(child.broker)
child.broker = nil
}
if err := child.dispatch(); err != nil {
child.sendError(err)
child.trigger <- none{}
// there's no point in trying again *right* away
select {
case <-child.dying:
close(child.trigger)
case <-time.After(10 * time.Second):
}
}
}
}
if child.broker != nil {
child.consumer.unrefBrokerConsumer(child.broker)
}
child.consumer.removeChild(child)
close(child.messages)
close(child.errors)
}
func (child *PartitionConsumer) dispatch() error {
if err := child.consumer.client.RefreshTopicMetadata(child.topic); err != nil {
return err
}
if leader, err := child.consumer.client.Leader(child.topic, child.partition); err != nil {
return err
} else {
child.broker = leader
}
brokerWorker := child.consumer.refBrokerConsumer(child.broker)
brokerWorker.input <- child
return nil
}
func (child *PartitionConsumer) chooseStartingOffset(offset int64) (err error) {
var where OffsetTime
switch offset {
case OffsetNewest:
where = LatestOffsets
case OffsetOldest:
where = EarliestOffset
default:
if offset < 0 {
return ConfigurationError("Invalid offset")
}
child.offset = offset
return nil
}
child.offset, err = child.consumer.client.GetOffset(child.topic, child.partition, where)
return err
}
// Messages returns the read channel for the messages that are returned by the broker
func (child *PartitionConsumer) Messages() <-chan *ConsumerMessage {
return child.messages
}
// Errors returns the read channel for any errors that occurred while consuming the partition.
// You have to read this channel to prevent the consumer from deadlock. Under no circumstances,
// the partition consumer will shut down by itself. It will just wait until it is able to continue
// consuming messages. If you want to shut down your consumer, you will have trigger it yourself
// by consuming this channel and calling Close or AsyncClose when appropriate.
func (child *PartitionConsumer) Errors() <-chan *ConsumerError {
return child.errors
}
// AsyncClose initiates a shutdown of the PartitionConsumer. This method will return immediately,
// after which you should wait until the 'messages' and 'errors' channel are drained.
// It is required to call this function, or Close before a consumer object passes out of scope,
// as it will otherwise leak memory. You must call this before calling Close on the underlying
// client.
func (child *PartitionConsumer) AsyncClose() {
// this triggers whatever worker owns this child to abandon it and close its trigger channel, which causes
// the dispatcher to exit its loop, which removes it from the consumer then closes its 'messages' and
// 'errors' channel (alternatively, if the child is already at the dispatcher for some reason, that will
// also just close itself)
close(child.dying)
}
// Close stops the PartitionConsumer from fetching messages. It is required to call this function
// (or AsyncClose) before a consumer object passes out of scope, as it will otherwise leak memory. You must
// call this before calling Close on the underlying client.
func (child *PartitionConsumer) Close() error {
child.AsyncClose()
go withRecover(func() {
for _ = range child.messages {
// drain
}
})
var errors ConsumerErrors
for err := range child.errors {
errors = append(errors, err)
}
if len(errors) > 0 {
return errors
}
return nil
}
// brokerConsumer
type brokerConsumer struct {
consumer *Consumer
broker *Broker
input chan *PartitionConsumer
newSubscriptions chan []*PartitionConsumer
wait chan none
subscriptions map[*PartitionConsumer]none
refs int
}
func (w *brokerConsumer) subscriptionManager() {
var buffer []*PartitionConsumer
// The subscriptionManager constantly accepts new subscriptions on `input` (even when the main subscriptionConsumer
// goroutine is in the middle of a network request) and batches it up. The main worker goroutine picks
// up a batch of new subscriptions between every network request by reading from `newSubscriptions`, so we give
// it nil if no new subscriptions are available. We also write to `wait` only when new subscriptions is available,
// so the main goroutine can block waiting for work if it has none.
for {
if len(buffer) > 0 {
select {
case event, ok := <-w.input:
if !ok {
goto done
}
buffer = append(buffer, event)
case w.newSubscriptions <- buffer:
buffer = nil
case w.wait <- none{}:
}
} else {
select {
case event, ok := <-w.input:
if !ok {
goto done
}
buffer = append(buffer, event)
case w.newSubscriptions <- nil:
}
}
}
done:
close(w.wait)
if len(buffer) > 0 {
w.newSubscriptions <- buffer
}
close(w.newSubscriptions)
}
func (w *brokerConsumer) subscriptionConsumer() {
<-w.wait // wait for our first piece of work
// the subscriptionConsumer ensures we will get nil right away if no new subscriptions is available
for newSubscriptions := range w.newSubscriptions {
w.updateSubscriptionCache(newSubscriptions)
if len(w.subscriptions) == 0 {
// We're about to be shut down or we're about to receive more subscriptions.
// Either way, the signal just hasn't propagated to our goroutine yet.
<-w.wait
continue
}
response, err := w.fetchNewMessages()
if err != nil {
Logger.Printf("Unexpected error processing FetchRequest; disconnecting broker %s: %s\n", w.broker.addr, err)
w.abort(err)
return
}
for child := range w.subscriptions {
block := response.GetBlock(child.topic, child.partition)
if block == nil {
child.sendError(ErrIncompleteResponse)
child.trigger <- none{}
delete(w.subscriptions, child)
continue
}
w.handleResponse(child, block)
}
}
}
func (w *brokerConsumer) updateSubscriptionCache(newSubscriptions []*PartitionConsumer) {
// take new subscriptions, and abandon subscriptions that have been closed
for _, child := range newSubscriptions {
w.subscriptions[child] = none{}
}
for child := range w.subscriptions {
select {
case <-child.dying:
close(child.trigger)
delete(w.subscriptions, child)
default:
}
}
}
func (w *brokerConsumer) abort(err error) {
_ = w.broker.Close() // we don't care about the error this might return, we already have one
w.consumer.client.disconnectBroker(w.broker)
for child := range w.subscriptions {
child.sendError(err)
child.trigger <- none{}
}
for newSubscription := range w.newSubscriptions {
for _, child := range newSubscription {
child.sendError(err)
child.trigger <- none{}
}
}
}
func (w *brokerConsumer) fetchNewMessages() (*FetchResponse, error) {
request := &FetchRequest{
MinBytes: w.consumer.conf.Consumer.Fetch.Min,
MaxWaitTime: int32(w.consumer.conf.Consumer.MaxWaitTime / time.Millisecond),
}
for child := range w.subscriptions {
request.AddBlock(child.topic, child.partition, child.offset, child.fetchSize)
}
return w.broker.Fetch(request)
}
func (w *brokerConsumer) handleResponse(child *PartitionConsumer, block *FetchResponseBlock) {
switch block.Err {
case ErrNoError:
break
default:
child.sendError(block.Err)
fallthrough
case ErrUnknownTopicOrPartition, ErrNotLeaderForPartition, ErrLeaderNotAvailable:
// doesn't belong to us, redispatch it
child.trigger <- none{}
delete(w.subscriptions, child)
return
}
if len(block.MsgSet.Messages) == 0 {
// We got no messages. If we got a trailing one then we need to ask for more data.
// Otherwise we just poll again and wait for one to be produced...
if block.MsgSet.PartialTrailingMessage {
if child.conf.Consumer.Fetch.Max > 0 && child.fetchSize == child.conf.Consumer.Fetch.Max {
// we can't ask for more data, we've hit the configured limit
child.sendError(ErrMessageTooLarge)
child.offset++ // skip this one so we can keep processing future messages
} else {
child.fetchSize *= 2
if child.conf.Consumer.Fetch.Max > 0 && child.fetchSize > child.conf.Consumer.Fetch.Max {
child.fetchSize = child.conf.Consumer.Fetch.Max
}
}
}
return
}
// we got messages, reset our fetch size in case it was increased for a previous request
child.fetchSize = child.conf.Consumer.Fetch.Default
incomplete := false
atLeastOne := false
prelude := true
for _, msgBlock := range block.MsgSet.Messages {
for _, msg := range msgBlock.Messages() {
if prelude && msg.Offset < child.offset {
continue
}
prelude = false
if msg.Offset >= child.offset {
atLeastOne = true
child.messages <- &ConsumerMessage{
Topic: child.topic,
Partition: child.partition,
Key: msg.Msg.Key,
Value: msg.Msg.Value,
Offset: msg.Offset,
}
child.offset = msg.Offset + 1
} else {
incomplete = true
}
}
}
if incomplete || !atLeastOne {
child.sendError(ErrIncompleteResponse)
child.trigger <- none{}
delete(w.subscriptions, child)
}
}