forked from bsm/sarama-cluster
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathconsumer.go
508 lines (433 loc) · 11.6 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
package cluster
import (
"sync"
"time"
"github.com/Shopify/sarama"
"github.com/samuel/go-zookeeper/zk"
"gopkg.in/tomb.v2"
)
// Config contains the consumer configuration options
type Config struct {
// Config contains the standard consumer configuration
*sarama.Config
// IDPrefix allows to force custom prefixes for consumer IDs.
// By default IDs have the form of PREFIX:HOSTNAME:UUID.
// Defaults to the consumer group name.
IDPrefix string
// AutoAck will force the consumer to automatically ack
// every message once it has been consumed through
// the Consumer.Messages() channel if set to true
AutoAck bool
// CommitEvery enables automatic commits in periodic cycles.
// Automatic commits will remain disabled if is set to < 10ms
// Default: 0
CommitEvery time.Duration
// Notifier instance to handle info/error
// notifications from the consumer
// Default: *LogNotifier
Notifier Notifier
// DefaultOffsetMode tells the consumer where to resume, if no offset
// is stored or the stored offset is out-of-range.
// Permitted values are sarama.OffsetNewest and sarama.OffsetOldest.
// Default: sarama.OffsetOldest
DefaultOffsetMode int64
// ZKSessionTimeout sets the timeout for the underlying zookeeper client
// session.
// Default: 1s
ZKSessionTimeout time.Duration
// Chroot path of the kafka cluster inside zookeeper
// Default: ""
ZKChrootPath string
customID string
}
func (c *Config) normalize() {
if c.Config == nil {
c.Config = sarama.NewConfig()
}
if c.Notifier == nil {
c.Notifier = &LogNotifier{Logger}
}
if c.CommitEvery < 10*time.Millisecond {
c.CommitEvery = 0
}
if c.DefaultOffsetMode != sarama.OffsetOldest && c.DefaultOffsetMode != sarama.OffsetOldest {
c.DefaultOffsetMode = sarama.OffsetOldest
}
if c.ZKSessionTimeout == 0 {
c.ZKSessionTimeout = time.Second
}
}
type Consumer struct {
id, group, topic string
client *sarama.Client
consumer *sarama.Consumer
config *Config
zoo *ZK
messages chan *sarama.ConsumerMessage
errors chan *sarama.ConsumerError
read map[int32]int64
rLock sync.Mutex
acked map[int32]int64
aLock sync.Mutex
partIDs []int32
pLock sync.Mutex
notifier Notifier
closer tomb.Tomb
ownClient bool
}
// NewConsumer creates a new consumer instance.
// You MUST call Close() to avoid leaks.
func NewConsumer(addrs, zookeepers []string, group, topic string, config *Config) (*Consumer, error) {
if config == nil {
config = new(Config)
}
client, err := sarama.NewClient(addrs, config.Config)
if err != nil {
return nil, err
}
c, err := NewConsumerFromClient(client, zookeepers, group, topic, config)
if err != nil {
client.Close()
return nil, err
}
c.ownClient = true
return c, nil
}
// NewConsumerFromClient creates a new consumer for a given topic, reuing an existing client
// You MUST call Close() to avoid leaks.
func NewConsumerFromClient(client *sarama.Client, zookeepers []string, group, topic string, config *Config) (*Consumer, error) {
if config == nil {
config = new(Config)
}
config.normalize()
// Validate configuration
if err := config.Validate(); err != nil {
return nil, err
} else if topic == "" {
return nil, sarama.ConfigurationError("Empty topic")
} else if group == "" {
return nil, sarama.ConfigurationError("Empty group")
}
// Generate unique consumer ID
id := config.customID
if id == "" {
prefix := config.IDPrefix
if prefix == "" {
prefix = group
}
id = newGUID(prefix)
}
// Create sarama consumer instance
scsmr, err := sarama.NewConsumerFromClient(client)
if err != nil {
return nil, err
}
// Connect to zookeeper
zoo, err := NewZK(zookeepers, config.ZKChrootPath, config.ZKSessionTimeout)
if err != nil {
scsmr.Close()
return nil, err
}
// Initialize consumer
consumer := &Consumer{
id: id,
group: group,
topic: topic,
zoo: zoo,
config: config,
client: client,
consumer: scsmr,
read: make(map[int32]int64),
acked: make(map[int32]int64),
partIDs: make([]int32, 0),
messages: make(chan *sarama.ConsumerMessage),
errors: make(chan *sarama.ConsumerError),
}
// Register consumer group and consumer itself
if err := consumer.register(); err != nil {
consumer.closeAll()
return nil, err
}
consumer.closer.Go(consumer.signalLoop)
if config.CommitEvery > 0 {
consumer.closer.Go(consumer.commitLoop)
}
return consumer, nil
}
// Messages returns the read channel for the messages that are returned by the broker
func (c *Consumer) Messages() <-chan *sarama.ConsumerMessage { return c.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.
func (c *Consumer) Errors() <-chan *sarama.ConsumerError { return c.errors }
// Claims exposes the partIDs partition ID
func (c *Consumer) Claims() []int32 {
c.pLock.Lock()
ids := c.partIDs
c.pLock.Unlock()
return ids
}
// ID exposes the consumer ID
func (c *Consumer) ID() string { return c.id }
// Group exposes the group name
func (c *Consumer) Group() string { return c.group }
// Topic exposes the group topic
func (c *Consumer) Topic() string { return c.topic }
// Offset manually retrives the stored offset for a partition ID
func (c *Consumer) Offset(partitionID int32) (int64, error) {
return c.zoo.Offset(c.group, c.topic, partitionID)
}
// Ack marks a consumer message as processed and stores the offset
// for the next Commit() call.
func (c *Consumer) Ack(msg *sarama.ConsumerMessage) {
c.aLock.Lock()
if msg.Offset > c.acked[msg.Partition] {
c.acked[msg.Partition] = msg.Offset
}
c.aLock.Unlock()
}
// Commit persists ack'd offsets
func (c *Consumer) Commit() error {
snap := c.resetAcked()
if len(snap) < 1 {
return nil
}
for partitionID, offset := range snap {
// fmt.Printf("$,%s,%d,%d\n", c.id, partitionID, offset+1)
if err := c.zoo.Commit(c.group, c.topic, partitionID, offset+1); err != nil {
return err
}
}
return nil
}
// Close closes the consumer instance.
// Also triggers a final Commit() call.
func (c *Consumer) Close() error {
c.closer.Kill(nil)
return c.closer.Wait()
}
// LOOPS
// Main signal loop
func (c *Consumer) signalLoop() error {
claims := make(Claims)
for {
// Check if shutdown was requested
select {
case <-c.closer.Dying():
return c.shutdown(claims)
default:
}
// Start a rebalance cycle
watch, err := c.rebalance(claims)
if err != nil {
c.config.Notifier.RebalanceError(c, err)
c.reset(claims)
continue
}
// Start a goroutine for each partition
done := make(chan struct{})
errs := make(chan struct{}, len(claims))
wait := new(sync.WaitGroup)
for _, pcsm := range claims {
wait.Add(1)
go c.consumeLoop(done, errs, wait, pcsm)
}
// Wait for signals
select {
case <-c.closer.Dying(): // on Close()
close(done)
wait.Wait()
return c.shutdown(claims)
case <-watch: // on rebalance signal
close(done)
wait.Wait()
case <-errs: // on consume errors
close(done)
wait.Wait()
}
}
}
// Commit loop, triggers periodic commits configured in CommitEvery
func (c *Consumer) commitLoop() error {
for {
select {
case <-c.closer.Dying():
return nil
case <-time.After(c.config.CommitEvery):
}
if err := c.Commit(); err != nil {
c.config.Notifier.CommitError(c, err)
}
}
}
// Message consumer loop for a single partition consumer
func (c *Consumer) consumeLoop(done, errs chan struct{}, wait *sync.WaitGroup, pcsm *sarama.PartitionConsumer) {
defer wait.Done()
for {
select {
case msg := <-pcsm.Messages():
// fmt.Printf("*,%s,%d,%d\n", c.id, msg.Partition, msg.Offset)
select {
case c.messages <- msg:
// fmt.Printf("+,%s,%d,%d\n", c.id, msg.Partition, msg.Offset)
c.rLock.Lock()
c.read[msg.Partition] = msg.Offset + 1
c.rLock.Unlock()
if c.config.AutoAck {
c.Ack(msg)
}
case <-done:
// fmt.Printf("@,%s\n", c.id)
return
}
case msg := <-pcsm.Errors():
if msg.Err == sarama.ErrOffsetOutOfRange {
offset, err := c.client.GetOffset(c.topic, msg.Partition, sarama.EarliestOffset)
if err == nil {
c.rLock.Lock()
c.read[msg.Partition] = offset
c.rLock.Unlock()
}
errs <- struct{}{}
}
select {
case c.errors <- msg:
// fmt.Printf("!,%s,%d,%s\n", c.id, msg.Partition, msg.Error())
case <-done:
// fmt.Printf("@,%s\n", c.id)
return
}
case <-done:
// fmt.Printf("@,%s\n", c.id)
return
}
}
}
// PRIVATE
// Shutdown the consumer, triggered by the main loop
func (c *Consumer) shutdown(claims Claims) error {
err := c.reset(claims)
c.closeAll()
return err
}
// Close all connections and channels
func (c *Consumer) closeAll() {
close(c.messages)
close(c.errors)
c.zoo.Close()
c.consumer.Close()
if c.ownClient {
c.client.Close()
}
}
// Rebalance cycle, triggered by the main loop
func (c *Consumer) rebalance(claims Claims) (<-chan zk.Event, error) {
c.config.Notifier.RebalanceStart(c)
// Commit and release existing claims
if err := c.reset(claims); err != nil {
return nil, err
}
// Fetch consumer list
consumerIDs, watch, err := c.zoo.Consumers(c.group)
if err != nil {
return nil, err
}
// Fetch partitions list
partitions, err := c.partitions()
if err != nil {
return nil, err
}
// Determine partitions and claim if changed
partitions = partitions.Select(c.id, consumerIDs)
// Make new claims
for _, part := range partitions {
pcsm, err := c.claim(part.ID)
if err != nil {
return nil, err
}
claims[part.ID] = pcsm
}
c.pLock.Lock()
c.partIDs = claims.PartitionIDs()
c.pLock.Unlock()
c.config.Notifier.RebalanceOK(c)
return watch, nil
}
// Commits offset and releases all claims
func (c *Consumer) reset(claims Claims) (err error) {
// Commit BEFORE releasing locks on partitions
err = c.Commit()
// Close all existing consumers (async)
wait := sync.WaitGroup{}
for _, pcsm := range claims {
wait.Add(1)
go func(c *sarama.PartitionConsumer) {
defer wait.Done()
c.Close()
}(pcsm)
}
wait.Wait()
// Release claimed partitions, ignore errors
for partitionID := range claims {
c.zoo.Release(c.group, c.topic, partitionID, c.id)
delete(claims, partitionID)
}
return
}
// Claims a partition
func (c *Consumer) claim(partitionID int32) (*sarama.PartitionConsumer, error) {
err := c.zoo.Claim(c.group, c.topic, partitionID, c.id)
if err != nil {
return nil, err
}
offset, err := c.Offset(partitionID)
if err != nil {
return nil, err
} else if offset < 1 {
offset = c.config.DefaultOffsetMode
}
c.rLock.Lock()
last := c.read[partitionID]
c.rLock.Unlock()
if offset < last {
offset = last
}
// fmt.Printf(">,%s,%d,%d\n", c.id, partitionID, offset)
return c.consumer.ConsumePartition(c.topic, partitionID, offset)
}
// Registers consumer with zookeeper
func (c *Consumer) register() error {
if err := c.zoo.RegisterGroup(c.group); err != nil {
return err
}
if err := c.zoo.RegisterConsumer(c.group, c.id, c.topic); err != nil {
return err
}
return nil
}
// Fetch all partitions for a topic
func (c *Consumer) partitions() (PartitionSlice, error) {
ids, err := c.client.Partitions(c.topic)
if err != nil {
return nil, err
}
slice := make(PartitionSlice, len(ids))
for n, id := range ids {
broker, err := c.client.Leader(c.topic, id)
if err != nil {
return nil, err
}
slice[n] = Partition{ID: id, Addr: broker.Addr()}
}
return slice, nil
}
// Creates a snapshot of acked and reset the current value
func (c *Consumer) resetAcked() map[int32]int64 {
c.aLock.Lock()
defer c.aLock.Unlock()
snap := make(map[int32]int64, len(c.acked))
for num, offset := range c.acked {
snap[num] = offset
delete(c.acked, num)
}
return snap
}