This repository has been archived by the owner on May 13, 2019. It is now read-only.
-
Notifications
You must be signed in to change notification settings - Fork 141
/
consumer_group.go
512 lines (421 loc) · 14.1 KB
/
consumer_group.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
package consumergroup
import (
"errors"
"fmt"
"sync"
"time"
"github.com/Shopify/sarama"
"github.com/wvanbergen/kazoo-go"
)
var (
AlreadyClosing = errors.New("The consumer group is already shutting down.")
)
type Config struct {
*sarama.Config
Zookeeper *kazoo.Config
Offsets struct {
Initial int64 // The initial offset method to use if the consumer has no previously stored offset. Must be either sarama.OffsetOldest (default) or sarama.OffsetNewest.
ProcessingTimeout time.Duration // Time to wait for all the offsets for a partition to be processed after stopping to consume from it. Defaults to 1 minute.
CommitInterval time.Duration // The interval between which the processed offsets are commited.
ResetOffsets bool // Resets the offsets for the consumergroup so that it won't resume from where it left off previously.
}
}
func NewConfig() *Config {
config := &Config{}
config.Config = sarama.NewConfig()
config.Zookeeper = kazoo.NewConfig()
config.Offsets.Initial = sarama.OffsetOldest
config.Offsets.ProcessingTimeout = 60 * time.Second
config.Offsets.CommitInterval = 10 * time.Second
return config
}
func (cgc *Config) Validate() error {
if cgc.Zookeeper.Timeout <= 0 {
return sarama.ConfigurationError("ZookeeperTimeout should have a duration > 0")
}
if cgc.Offsets.CommitInterval < 0 {
return sarama.ConfigurationError("CommitInterval should have a duration >= 0")
}
if cgc.Offsets.Initial != sarama.OffsetOldest && cgc.Offsets.Initial != sarama.OffsetNewest {
return errors.New("Offsets.Initial should be sarama.OffsetOldest or sarama.OffsetNewest.")
}
if cgc.Config != nil {
if err := cgc.Config.Validate(); err != nil {
return err
}
}
return nil
}
// The ConsumerGroup type holds all the information for a consumer that is part
// of a consumer group. Call JoinConsumerGroup to start a consumer.
type ConsumerGroup struct {
config *Config
consumer sarama.Consumer
kazoo *kazoo.Kazoo
group *kazoo.Consumergroup
instance *kazoo.ConsumergroupInstance
wg sync.WaitGroup
singleShutdown sync.Once
messages chan *sarama.ConsumerMessage
errors chan error
stopper chan struct{}
consumers kazoo.ConsumergroupInstanceList
offsetManager OffsetManager
}
// Connects to a consumer group, using Zookeeper for auto-discovery
func JoinConsumerGroup(name string, topics []string, zookeeper []string, config *Config) (cg *ConsumerGroup, err error) {
if name == "" {
return nil, sarama.ConfigurationError("Empty consumergroup name")
}
if len(topics) == 0 {
return nil, sarama.ConfigurationError("No topics provided")
}
if len(zookeeper) == 0 {
return nil, errors.New("You need to provide at least one zookeeper node address!")
}
if config == nil {
config = NewConfig()
}
config.ClientID = name
// Validate configuration
if err = config.Validate(); err != nil {
return
}
var kz *kazoo.Kazoo
if kz, err = kazoo.NewKazoo(zookeeper, config.Zookeeper); err != nil {
return
}
brokers, err := kz.BrokerList()
if err != nil {
kz.Close()
return
}
group := kz.Consumergroup(name)
if config.Offsets.ResetOffsets {
err = group.ResetOffsets()
if err != nil {
kz.Close()
return
}
}
instance := group.NewInstance()
var consumer sarama.Consumer
if consumer, err = sarama.NewConsumer(brokers, config.Config); err != nil {
kz.Close()
return
}
cg = &ConsumerGroup{
config: config,
consumer: consumer,
kazoo: kz,
group: group,
instance: instance,
messages: make(chan *sarama.ConsumerMessage, config.ChannelBufferSize),
errors: make(chan error, config.ChannelBufferSize),
stopper: make(chan struct{}),
}
// Register consumer group
if exists, err := cg.group.Exists(); err != nil {
cg.Logf("FAILED to check for existence of consumergroup: %s!\n", err)
_ = consumer.Close()
_ = kz.Close()
return nil, err
} else if !exists {
cg.Logf("Consumergroup `%s` does not yet exists, creating...\n", cg.group.Name)
if err := cg.group.Create(); err != nil {
cg.Logf("FAILED to create consumergroup in Zookeeper: %s!\n", err)
_ = consumer.Close()
_ = kz.Close()
return nil, err
}
}
// Register itself with zookeeper
if err := cg.instance.Register(topics); err != nil {
cg.Logf("FAILED to register consumer instance: %s!\n", err)
return nil, err
} else {
cg.Logf("Consumer instance registered (%s).", cg.instance.ID)
}
offsetConfig := OffsetManagerConfig{CommitInterval: config.Offsets.CommitInterval}
cg.offsetManager = NewZookeeperOffsetManager(cg, &offsetConfig)
go cg.topicListConsumer(topics)
return
}
// Returns a channel that you can read to obtain events from Kafka to process.
func (cg *ConsumerGroup) Messages() <-chan *sarama.ConsumerMessage {
return cg.messages
}
// Returns a channel that you can read to obtain events from Kafka to process.
func (cg *ConsumerGroup) Errors() <-chan error {
return cg.errors
}
func (cg *ConsumerGroup) Closed() bool {
return cg.instance == nil
}
func (cg *ConsumerGroup) Close() error {
shutdownError := AlreadyClosing
cg.singleShutdown.Do(func() {
defer cg.kazoo.Close()
shutdownError = nil
close(cg.stopper)
cg.wg.Wait()
if err := cg.offsetManager.Close(); err != nil {
cg.Logf("FAILED closing the offset manager: %s!\n", err)
}
if shutdownError = cg.instance.Deregister(); shutdownError != nil {
cg.Logf("FAILED deregistering consumer instance: %s!\n", shutdownError)
} else {
cg.Logf("Deregistered consumer instance %s.\n", cg.instance.ID)
}
if shutdownError = cg.consumer.Close(); shutdownError != nil {
cg.Logf("FAILED closing the Sarama client: %s\n", shutdownError)
}
close(cg.messages)
close(cg.errors)
cg.instance = nil
})
return shutdownError
}
func (cg *ConsumerGroup) Logf(format string, args ...interface{}) {
var identifier string
if cg.instance == nil {
identifier = "(defunct)"
} else {
identifier = cg.instance.ID[len(cg.instance.ID)-12:]
}
sarama.Logger.Printf("[%s/%s] %s", cg.group.Name, identifier, fmt.Sprintf(format, args...))
}
func (cg *ConsumerGroup) InstanceRegistered() (bool, error) {
return cg.instance.Registered()
}
func (cg *ConsumerGroup) CommitUpto(message *sarama.ConsumerMessage) error {
cg.offsetManager.MarkAsProcessed(message.Topic, message.Partition, message.Offset)
return nil
}
func (cg *ConsumerGroup) FlushOffsets() error {
return cg.offsetManager.Flush()
}
func (cg *ConsumerGroup) topicListConsumer(topics []string) {
for {
select {
case <-cg.stopper:
return
default:
}
consumers, consumerChanges, err := cg.group.WatchInstances()
if err != nil {
cg.Logf("FAILED to get list of registered consumer instances: %s\n", err)
return
}
cg.consumers = consumers
cg.Logf("Currently registered consumers: %d\n", len(cg.consumers))
stopper := make(chan struct{})
for _, topic := range topics {
cg.wg.Add(1)
go cg.topicConsumer(topic, cg.messages, cg.errors, stopper)
}
select {
case <-cg.stopper:
close(stopper)
return
case <-consumerChanges:
registered, err := cg.instance.Registered()
if err != nil {
cg.Logf("FAILED to get register status: %s\n", err)
} else if !registered {
err = cg.instance.Register(topics)
if err != nil {
cg.Logf("FAILED to register consumer instance: %s!\n", err)
} else {
cg.Logf("Consumer instance registered (%s).", cg.instance.ID)
}
}
cg.Logf("Triggering rebalance due to consumer list change\n")
close(stopper)
cg.wg.Wait()
}
}
}
func (cg *ConsumerGroup) topicConsumer(topic string, messages chan<- *sarama.ConsumerMessage, errors chan<- error, stopper <-chan struct{}) {
defer cg.wg.Done()
select {
case <-stopper:
return
default:
}
cg.Logf("%s :: Started topic consumer\n", topic)
// Fetch a list of partition IDs
partitions, err := cg.kazoo.Topic(topic).Partitions()
if err != nil {
cg.Logf("%s :: FAILED to get list of partitions: %s\n", topic, err)
cg.errors <- &sarama.ConsumerError{
Topic: topic,
Partition: -1,
Err: err,
}
return
}
partitionLeaders, err := retrievePartitionLeaders(partitions)
if err != nil {
cg.Logf("%s :: FAILED to get leaders of partitions: %s\n", topic, err)
cg.errors <- &sarama.ConsumerError{
Topic: topic,
Partition: -1,
Err: err,
}
return
}
dividedPartitions := dividePartitionsBetweenConsumers(cg.consumers, partitionLeaders)
myPartitions := dividedPartitions[cg.instance.ID]
cg.Logf("%s :: Claiming %d of %d partitions", topic, len(myPartitions), len(partitionLeaders))
// Consume all the assigned partitions
var wg sync.WaitGroup
for _, pid := range myPartitions {
wg.Add(1)
go cg.partitionConsumer(topic, pid.ID, messages, errors, &wg, stopper)
}
wg.Wait()
cg.Logf("%s :: Stopped topic consumer\n", topic)
}
func (cg *ConsumerGroup) consumePartition(topic string, partition int32, nextOffset int64) (sarama.PartitionConsumer, error) {
consumer, err := cg.consumer.ConsumePartition(topic, partition, nextOffset)
if err == sarama.ErrOffsetOutOfRange {
cg.Logf("%s/%d :: Partition consumer offset out of Range.\n", topic, partition)
// if the offset is out of range, simplistically decide whether to use OffsetNewest or OffsetOldest
// if the configuration specified offsetOldest, then switch to the oldest available offset, else
// switch to the newest available offset.
if cg.config.Offsets.Initial == sarama.OffsetOldest {
nextOffset = sarama.OffsetOldest
cg.Logf("%s/%d :: Partition consumer offset reset to oldest available offset.\n", topic, partition)
} else {
nextOffset = sarama.OffsetNewest
cg.Logf("%s/%d :: Partition consumer offset reset to newest available offset.\n", topic, partition)
}
// retry the consumePartition with the adjusted offset
consumer, err = cg.consumer.ConsumePartition(topic, partition, nextOffset)
}
if err != nil {
cg.Logf("%s/%d :: FAILED to start partition consumer: %s\n", topic, partition, err)
return nil, err
}
return consumer, err
}
// Consumes a partition
func (cg *ConsumerGroup) partitionConsumer(topic string, partition int32, messages chan<- *sarama.ConsumerMessage, errors chan<- error, wg *sync.WaitGroup, stopper <-chan struct{}) {
defer wg.Done()
// Since ProcessingTimeout is the amount of time we'll wait for the final batch
// of messages to be processed before releasing a partition, we need to wait slightly
// longer than that before timing out here to ensure that another consumer has had
// enough time to release the partition. Hence, +2 seconds.
maxRetries := int(cg.config.Offsets.ProcessingTimeout/time.Second) + 2
partitionClaimLoop:
for tries := 0; tries < maxRetries; tries++ {
select {
case <-stopper:
return
case <-time.After(1 * time.Second):
if err := cg.instance.ClaimPartition(topic, partition); err == nil {
break partitionClaimLoop
} else if tries+1 < maxRetries {
if err == kazoo.ErrPartitionClaimedByOther {
// Another consumer still owns this partition. We should wait longer for it to release it.
} else {
// An unexpected error occurred. Log it and continue trying until we hit the timeout.
cg.Logf("%s/%d :: FAILED to claim partition on attempt %v of %v; retrying in 1 second. Error: %v", topic, partition, tries+1, maxRetries, err)
}
} else {
cg.Logf("%s/%d :: FAILED to claim the partition: %s\n", topic, partition, err)
cg.errors <- &sarama.ConsumerError{
Topic: topic,
Partition: partition,
Err: err,
}
return
}
}
}
defer func() {
err := cg.instance.ReleasePartition(topic, partition)
if err != nil {
cg.Logf("%s/%d :: FAILED to release partition: %s\n", topic, partition, err)
cg.errors <- &sarama.ConsumerError{
Topic: topic,
Partition: partition,
Err: err,
}
}
}()
nextOffset, err := cg.offsetManager.InitializePartition(topic, partition)
if err != nil {
cg.Logf("%s/%d :: FAILED to determine initial offset: %s\n", topic, partition, err)
return
}
if nextOffset >= 0 {
cg.Logf("%s/%d :: Partition consumer starting at offset %d.\n", topic, partition, nextOffset)
} else {
nextOffset = cg.config.Offsets.Initial
if nextOffset == sarama.OffsetOldest {
cg.Logf("%s/%d :: Partition consumer starting at the oldest available offset.\n", topic, partition)
} else if nextOffset == sarama.OffsetNewest {
cg.Logf("%s/%d :: Partition consumer listening for new messages only.\n", topic, partition)
}
}
consumer, err := cg.consumePartition(topic, partition, nextOffset)
if err != nil {
cg.Logf("%s/%d :: FAILED to start partition consumer: %s\n", topic, partition, err)
return
}
defer consumer.Close()
err = nil
var lastOffset int64 = -1 // aka unknown
partitionConsumerLoop:
for {
select {
case <-stopper:
break partitionConsumerLoop
case err := <-consumer.Errors():
if err == nil {
cg.Logf("%s/%d :: Consumer encountered an invalid state: re-establishing consumption of partition.\n", topic, partition)
// Errors encountered (if any) are logged in the consumerPartition function
var cErr error
consumer, cErr = cg.consumePartition(topic, partition, lastOffset)
if cErr != nil {
break partitionConsumerLoop
}
continue partitionConsumerLoop
}
for {
select {
case errors <- err:
continue partitionConsumerLoop
case <-stopper:
break partitionConsumerLoop
}
}
case message := <-consumer.Messages():
if message == nil {
cg.Logf("%s/%d :: Consumer encountered an invalid state: re-establishing consumption of partition.\n", topic, partition)
// Errors encountered (if any) are logged in the consumerPartition function
var cErr error
consumer, cErr = cg.consumePartition(topic, partition, lastOffset)
if cErr != nil {
break partitionConsumerLoop
}
continue partitionConsumerLoop
}
for {
select {
case <-stopper:
break partitionConsumerLoop
case messages <- message:
lastOffset = message.Offset
continue partitionConsumerLoop
}
}
}
}
cg.Logf("%s/%d :: Stopping partition consumer at offset %d\n", topic, partition, lastOffset)
if err := cg.offsetManager.FinalizePartition(topic, partition, lastOffset, cg.config.Offsets.ProcessingTimeout); err != nil {
cg.Logf("%s/%d :: %s\n", topic, partition, err)
}
}