forked from lovoo/goka
/
group_consumer.go
272 lines (230 loc) · 5.21 KB
/
group_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
package kafka
import (
"fmt"
"log"
"sync/atomic"
"github.com/Shopify/sarama"
cluster "github.com/bsm/sarama-cluster"
)
type groupConsumer struct {
brokers []string
config *cluster.Config
consumer clusterConsumer
group string
partitionMap map[int32]bool
addPartition chan int32
events chan Event
stop chan bool
done chan bool
running int64
}
func newGroupConsumer(brokers []string, group string, events chan Event, config *cluster.Config) (*groupConsumer, error) {
return &groupConsumer{
group: group,
brokers: brokers,
config: config,
partitionMap: make(map[int32]bool),
addPartition: make(chan int32, 2048),
events: events,
stop: make(chan bool),
done: make(chan bool),
}, nil
}
func (c *groupConsumer) Close() error {
if atomic.LoadInt64(&c.running) == 0 {
// not running
return nil
}
close(c.stop)
<-c.done
if err := c.consumer.Close(); err != nil {
return fmt.Errorf("Failed to close consumer: %v", err)
}
return nil
}
func (c *groupConsumer) Subscribe(topics map[string]int64) error {
var ts []string
for t := range topics {
ts = append(ts, string(t))
}
upConsumer, err := cluster.NewConsumer(c.brokers, c.group, ts, c.config)
if err != nil {
return err
}
c.consumer = upConsumer
go c.run()
return nil
}
func (c *groupConsumer) waitForRebalanceOK() bool {
for {
select {
case n := <-c.consumer.Notifications():
if !c.checkRebalance(cluster.RebalanceStart, n.Type) {
continue
}
select {
case nn := <-c.consumer.Notifications():
if !c.checkRebalance(cluster.RebalanceOK, nn.Type) {
continue
}
return c.handleRebalanceOK(nn)
case <-c.stop:
return false
}
case err := <-c.consumer.Errors():
select {
case c.events <- &Error{err}:
case <-c.stop:
return false
}
case <-c.stop:
return false
}
}
}
func (c *groupConsumer) checkRebalance(expected, actual cluster.NotificationType) bool {
if actual != expected {
select {
case c.events <- &Error{fmt.Errorf("expected %s but received %s", expected, actual)}:
case <-c.stop:
}
return false
}
return true
}
func (c *groupConsumer) handleRebalanceOK(n *cluster.Notification) bool {
if n.Type != cluster.RebalanceOK {
// panic as this is a programming error
log.Panicf("GroupConsumer: unsupported notification type in handleRebalanceOK: %v/%s", n.Type, n.Type)
}
// save partition map
m := c.partitionMap
c.partitionMap = make(map[int32]bool)
// create assignment and update partitionMap
a := make(Assignment)
for _, v := range n.Current {
for _, p := range v {
a[p] = sarama.OffsetNewest
// remember whether partition was added using m[p]
c.partitionMap[p] = m[p]
}
break // copartitioned topics
}
// send assignment
select {
case c.events <- &a:
return true
case <-c.stop:
return false
}
}
// returns true if all partitions are registered. otherwise false
func (c *groupConsumer) partitionsRegistered() bool {
for _, v := range c.partitionMap {
if !v {
return false
}
}
return true
}
func (c *groupConsumer) AddGroupPartition(partition int32) {
select {
case c.addPartition <- partition:
case <-c.stop:
}
}
func (c *groupConsumer) waitForPartitions() bool {
defer c.ensureEmpty()
// if all registered, start consuming
if c.partitionsRegistered() {
return true
}
for {
select {
case par := <-c.addPartition:
c.partitionMap[par] = true
// if all registered, start consuming
if c.partitionsRegistered() {
return true
}
case <-c.stop:
return false
}
}
}
func (c *groupConsumer) ensureEmpty() {
for {
select {
case <-c.addPartition:
default:
return
}
}
}
func (c *groupConsumer) waitForMessages() bool {
for {
select {
case n := <-c.consumer.Notifications():
if !c.checkRebalance(cluster.RebalanceStart, n.Type) {
continue
}
select {
case nn := <-c.consumer.Notifications():
if !c.checkRebalance(cluster.RebalanceOK, nn.Type) {
continue
}
return c.handleRebalanceOK(nn)
case <-c.stop:
return false
}
case msg := <-c.consumer.Messages():
select {
case c.events <- &Message{
Topic: msg.Topic,
Partition: msg.Partition,
Offset: msg.Offset,
Timestamp: msg.Timestamp,
Key: string(msg.Key),
Value: msg.Value,
}:
case <-c.stop:
return false
}
case err := <-c.consumer.Errors():
select {
case c.events <- &Error{err}:
case <-c.stop:
return false
}
case <-c.stop:
return false
}
}
}
func (c *groupConsumer) run() {
atomic.AddInt64(&c.running, 1)
defer close(c.done)
if !c.waitForRebalanceOK() {
return
}
for {
if !c.waitForPartitions() {
return
}
if !c.waitForMessages() {
return
}
}
}
func (c *groupConsumer) Commit(topic string, partition int32, offset int64) error {
c.consumer.MarkPartitionOffset(topic, partition, offset, "")
return nil
}
//go:generate mockgen -package mock -destination=mock/cluster_consumer.go -source=group_consumer.go clusterConsumer
type clusterConsumer interface {
Close() error
MarkPartitionOffset(topic string, partition int32, offset int64, metadata string)
Notifications() <-chan *cluster.Notification
Messages() <-chan *sarama.ConsumerMessage
Errors() <-chan error
}