-
Notifications
You must be signed in to change notification settings - Fork 15
/
consumer_group.go
418 lines (383 loc) · 11.2 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
package consumergroup
import (
"errors"
"fmt"
"reflect"
"runtime/debug"
"sync"
"time"
"github.com/Shopify/sarama"
"github.com/meitu/go-zookeeper/zk"
"github.com/sirupsen/logrus"
)
const (
cgStarted = iota + 1
cgStopped
)
const (
restartEvent = 1
)
// ConsumerGroup consume message from Kafka with rebalancing supports
type ConsumerGroup struct {
name string
storage groupStorage
topicConsumers map[string]*topicConsumer
saramaClients map[string]sarama.Client
saramaConsumers map[string]sarama.Consumer
id string
state int
wg sync.WaitGroup
stopCh chan struct{}
triggerCh chan int
triggerOnce *sync.Once
owners map[string]map[int32]string
config *Config
logger *logrus.Logger
onLoad, onClose []func()
}
// NewConsumerGroup create the ConsumerGroup instance with config
func NewConsumerGroup(config *Config) (*ConsumerGroup, error) {
if config == nil {
return nil, errors.New("config can't be empty")
}
err := config.validate()
if err != nil {
return nil, fmt.Errorf("vaildate config failed, as %s", err)
}
cg := new(ConsumerGroup)
cg.state = cgStopped
cg.config = config
cg.id = config.ConsumerID
if cg.id == "" {
cg.id = genConsumerID()
}
cg.name = config.GroupID
cg.triggerCh = make(chan int)
cg.topicConsumers = make(map[string]*topicConsumer)
cg.saramaClients = make(map[string]sarama.Client)
cg.saramaConsumers = make(map[string]sarama.Consumer)
cg.onLoad = make([]func(), 0)
cg.onClose = make([]func(), 0)
cg.storage = newZKGroupStorage(config.ZkList, config.ZkSessionTimeout)
cg.logger = logrus.New()
if _, ok := cg.storage.(*zkGroupStorage); ok {
cg.storage.(*zkGroupStorage).Chroot(config.Chroot)
}
err = cg.initSaramaConsumer()
if err != nil {
return nil, fmt.Errorf("init sarama consumer, as %s", err)
}
cg.owners = make(map[string]map[int32]string)
for _, topic := range config.TopicList {
cg.topicConsumers[topic] = newTopicConsumer(cg, topic)
cg.owners[topic] = make(map[int32]string)
}
return cg, nil
}
func (cg *ConsumerGroup) initSaramaConsumer() error {
brokerList, err := cg.storage.getBrokerList()
if err != nil {
return err
}
if len(brokerList) == 0 {
return errors.New("no broker alive")
}
for _, topic := range cg.config.TopicList {
saramaClient, err := sarama.NewClient(brokerList, cg.config.SaramaConfig)
if err != nil {
return err
}
saramaConsumer, err := sarama.NewConsumerFromClient(saramaClient)
if err != nil {
return err
}
cg.saramaClients[topic] = saramaClient
cg.saramaConsumers[topic] = saramaConsumer
}
return nil
}
// Start would register ConsumerGroup, and rebalance would be triggered.
// ConsumerGroup computes the partitions which should be consumed by consumer's num, and start fetching message.
func (cg *ConsumerGroup) Start() error {
// exit when failed to register the consumer
err := cg.storage.registerConsumer(cg.name, cg.id, nil)
if err != nil && err != zk.ErrNodeExists {
return err
}
cg.wg.Add(1)
go cg.start()
return nil
}
// Stop would unregister ConsumerGroup, and rebalance would be triggered.
// The partitions which consumed by this ConsumerGroup would be assigned to others.
func (cg *ConsumerGroup) Stop() {
cg.stop()
cg.wg.Wait()
}
// SetLogger use to set the user's logger the consumer group
func (cg *ConsumerGroup) SetLogger(l *logrus.Logger) {
if l != nil {
cg.logger = l
}
}
// IsStopped return whether the ConsumerGroup was stopped or not.
func (cg *ConsumerGroup) IsStopped() bool {
return cg.state == cgStopped
}
func (cg *ConsumerGroup) callRecover() {
if err := recover(); err != nil {
cg.logger.WithFields(logrus.Fields{
"group": cg.name,
"err": err,
"stack": string(debug.Stack()),
}).Error("Recover panic")
cg.stop()
}
}
func (cg *ConsumerGroup) start() {
var wg sync.WaitGroup
defer cg.callRecover()
defer func() {
cg.state = cgStopped
err := cg.storage.deleteConsumer(cg.name, cg.id)
if err != nil {
cg.logger.WithFields(logrus.Fields{
"group": cg.name,
"err": err,
}).Error("Failed to delete consumer from zk")
}
for _, tc := range cg.topicConsumers {
close(tc.messages)
close(tc.errors)
}
cg.wg.Done()
}()
CONSUME_TOPIC_LOOP:
for {
cg.logger.WithField("group", cg.name).Info("Consumer group started")
cg.triggerOnce = new(sync.Once)
cg.stopCh = make(chan struct{})
err := cg.watchRebalance()
if err != nil {
cg.logger.WithFields(logrus.Fields{
"group": cg.name,
"err": err,
}).Error("Failed to watch rebalance")
cg.stop()
return
}
wg.Add(1)
go func() {
defer cg.callRecover()
defer wg.Done()
cg.autoReconnect(cg.storage.(*zkGroupStorage).sessionTimeout / 3)
}()
for _, consumer := range cg.topicConsumers {
wg.Add(1)
consumer.start()
go func(tc *topicConsumer) {
defer cg.callRecover()
defer wg.Done()
tc.wg.Wait()
cg.logger.WithFields(logrus.Fields{
"group": tc.group,
"topic": tc.name,
}).Info("Stop the topic consumer")
}(consumer)
}
cg.state = cgStarted
for _, onLoadFunc := range cg.onLoad {
onLoadFunc()
}
msg := <-cg.triggerCh
for _, onCloseFunc := range cg.onClose {
onCloseFunc()
}
switch msg {
case restartEvent:
close(cg.stopCh)
// The stop channel was used to notify partition's consumer to stop consuming when rebalance is triggered.
// So we should reinit when rebalance was triggered, as it would be closed.
wg.Wait()
continue CONSUME_TOPIC_LOOP
default:
close(cg.stopCh)
cg.logger.WithField("group", cg.name).Info("ConsumerGroup is stopping")
wg.Wait()
cg.logger.WithField("group", cg.name).Info("ConsumerGroup was stopped")
return
}
}
}
func (cg *ConsumerGroup) stop() {
cg.triggerOnce.Do(func() { close(cg.triggerCh) })
}
func (cg *ConsumerGroup) triggerRebalance() {
cg.triggerOnce.Do(func() { cg.triggerCh <- restartEvent })
}
func (cg *ConsumerGroup) getPartitionConsumer(topic string, partition int32, nextOffset int64) (sarama.PartitionConsumer, error) {
saramaConsumer := cg.saramaConsumers[topic]
consumer, err := saramaConsumer.ConsumePartition(topic, partition, nextOffset)
if err == sarama.ErrOffsetOutOfRange {
cg.logger.WithFields(logrus.Fields{
"group": cg.name,
"topic": topic,
"partition": partition,
"offset": nextOffset,
}).Error("Partition's offset was out of range, use auto-reset")
nextOffset = cg.config.OffsetAutoReset
consumer, err = saramaConsumer.ConsumePartition(topic, partition, nextOffset)
}
return consumer, err
}
// GetMessages was used to get a unbuffered message's channel from specified topic
func (cg *ConsumerGroup) GetMessages(topic string) (<-chan *sarama.ConsumerMessage, bool) {
if topicConsumer, ok := cg.topicConsumers[topic]; ok {
return topicConsumer.messages, true
}
return nil, false
}
// GetErrors was used to get a unbuffered error's channel from specified topic
func (cg *ConsumerGroup) GetErrors(topic string) (<-chan *sarama.ConsumerError, bool) {
if topicConsumer, ok := cg.topicConsumers[topic]; ok {
return topicConsumer.errors, true
}
return nil, false
}
// OnLoad load callback function that runs after startup
func (cg *ConsumerGroup) OnLoad(cb func()) {
cg.onLoad = append(cg.onLoad, cb)
}
// OnClose load callback function that runs before the end
func (cg *ConsumerGroup) OnClose(cb func()) {
cg.onClose = append(cg.onClose, cb)
}
func (cg *ConsumerGroup) autoReconnect(interval time.Duration) {
timer := time.NewTimer(interval)
cg.logger.WithField("group", cg.name).Info("The auto-reconnect consumer thread was started")
defer cg.logger.WithField("group", cg.name).Info("The auto-reconnect consumer thread was stopped")
for {
select {
case <-cg.stopCh:
return
case <-timer.C:
timer.Reset(interval)
exist, err := cg.storage.existsConsumer(cg.name, cg.id)
if err != nil {
cg.logger.WithFields(logrus.Fields{
"group": cg.name,
"err": err,
}).Error("Failed to check consumer existence")
break
}
if exist {
break
}
err = cg.storage.registerConsumer(cg.name, cg.id, nil)
if err != nil {
cg.logger.WithFields(logrus.Fields{
"group": cg.name,
"err": err,
}).Error("Failed to re-register consumer")
}
}
}
}
func (cg *ConsumerGroup) watchRebalance() error {
consumersWatcher, err := cg.storage.watchConsumerList(cg.name)
if err != nil {
return err
}
topicsChange, topicWatchers, err := cg.watchTopics(cg.config.TopicList)
if err != nil {
return err
}
go func() {
defer cg.callRecover()
cg.logger.WithField("group", cg.name).Info("Rebalance watcher thread was started")
select {
case <-consumersWatcher.EvCh:
cg.triggerRebalance()
cg.logger.WithField("group", cg.name).Info("Trigger rebalance while consumers was changed")
for _, tw := range topicWatchers {
cg.storage.removeWatcher(tw)
}
case topic := <-topicsChange:
cg.triggerRebalance()
cg.logger.WithFields(logrus.Fields{
"group": cg.name,
"topic": topic,
}).Info("Trigger rebalance while partitions was changed")
cg.storage.removeWatcher(consumersWatcher)
case <-cg.stopCh:
}
cg.logger.WithField("group", cg.name).Info("Rebalance watcher thread was exited")
}()
return nil
}
// CommitOffset is used to commit offset when auto commit was disabled.
func (cg *ConsumerGroup) CommitOffset(topic string, partition int32, offset int64) error {
if cg.config.OffsetAutoCommitEnable {
return errors.New("commit offset take effect when offset auto commit was disabled")
}
return cg.storage.commitOffset(cg.name, topic, partition, offset)
}
// GetOffsets return the offset in memory for debug
func (cg *ConsumerGroup) GetOffsets() map[string]interface{} {
topics := make(map[string]interface{})
for topic, tc := range cg.topicConsumers {
topics[topic] = tc.getOffsets()
}
return topics
}
// Owners return owners of all partitions
func (cg *ConsumerGroup) Owners() map[string]map[int32]string {
return cg.owners
}
func (cg *ConsumerGroup) watchTopics(topics []string) (<-chan string, []*zk.Watcher, error) {
ch := make(chan string)
cases := make([]reflect.SelectCase, len(topics))
watchers := make([]*zk.Watcher, len(topics))
for i, topic := range topics {
w, err := cg.storage.watchTopic(topic)
if err != nil {
return nil, nil, fmt.Errorf("encounter error when watch topic: %s, err: %s", topic, err)
}
watchers[i] = w
cases[i] = reflect.SelectCase{Dir: reflect.SelectRecv, Chan: reflect.ValueOf(w.EvCh)}
}
go func(cases []reflect.SelectCase, ch chan string, topics []string) {
chosen, _, ok := reflect.Select(cases)
if !ok {
//the chosen channel has been closed.
return
}
topic := topics[chosen]
num, err := cg.storage.getPartitionsNum(topic)
if err != nil {
cg.logger.WithFields(logrus.Fields{
"topic": topic,
"err": err,
}).Error("Failed to get partitions in zookeeper after topic metadata change")
return
}
for {
saramaClient := cg.saramaClients[topic]
saramaClient.RefreshMetadata(topic)
partitions, err := saramaClient.Partitions(topic)
if err != nil {
cg.logger.WithFields(logrus.Fields{
"topic": topic,
"err": err,
}).Error("Failed to get partitions in broker after topic metadata change")
return
}
if len(partitions) == num {
break
}
time.Sleep(100 * time.Millisecond)
}
ch <- topics[chosen]
}(cases, ch, topics)
return ch, watchers, nil
}