forked from lovoo/goka
/
queueconsumer.go
351 lines (306 loc) · 9.4 KB
/
queueconsumer.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
package tester
import (
"fmt"
"log"
"sync"
"time"
"github.com/lovoo/goka/kafka"
)
type consumer struct {
tester *Tester
events chan kafka.Event
subscribedTopics map[string]*queueConsumer
simpleConsumers map[string]*queueConsumer
closeOnce sync.Once
sync.Mutex
}
const (
unbound State = iota
bound
running
stopping
stopped
killed
)
const (
eventBufferQueueSize = 100000
)
type queueConsumer struct {
queue *queue
nextOffset int64
waitEventBuffer sync.WaitGroup
state *Signal
eventBuffer chan kafka.Event
events chan kafka.Event
consumer *consumer
}
func newQueueConsumer(topic string, queue *queue) *queueConsumer {
qc := &queueConsumer{
queue: queue,
eventBuffer: make(chan kafka.Event, eventBufferQueueSize),
state: NewSignal(unbound, bound, stopped, stopping, running, killed).SetState(unbound),
}
return qc
}
func (qc *queueConsumer) bindToConsumer(cons *consumer) {
logger.Printf("binding consumer to topic %s", qc.queue.topic)
if !qc.state.IsState(unbound) {
panic(fmt.Errorf("error binding %s to consumer. Already bound", qc.queue.topic))
}
qc.state.SetState(bound)
qc.consumer = cons
qc.events = cons.events
}
func (qc *queueConsumer) isBound() bool {
return !qc.state.IsState(unbound)
}
func (qc *queueConsumer) isRunning() bool {
return qc.state.IsState(running)
}
func (qc *queueConsumer) setRunning() {
qc.state.SetState(running)
}
func (qc *queueConsumer) stop() {
logger.Printf("closing the queueConsumer for topic %s", qc.queue.topic)
if !qc.state.IsState(running) {
panic(fmt.Sprintf("trying to stop consumer %s which is not running (state=%d)", qc.queue.topic, qc.state.State()))
}
qc.state.SetState(stopping)
logger.Printf("[consumer %s]waiting for stopped", qc.queue.topic)
<-qc.state.WaitForState(stopped)
logger.Printf("[consumer %s] stopped", qc.queue.topic)
}
func (qc *queueConsumer) kill() {
qc.stop()
qc.state.SetState(killed)
}
func (qc *queueConsumer) startLoop(setRunning bool) {
logger.Printf("starting queue consumer %s (set-running=%t)", qc.queue.topic, setRunning)
// not bound or already running
if qc.state.IsState(unbound) || qc.state.IsState(running) || qc.state.IsState(stopping) {
panic(fmt.Errorf("the queue consumer %s is in state %v. Cannot start", qc.queue.topic, qc.state.State()))
}
if setRunning {
qc.state.SetState(running)
}
go qc.consumeBuffer()
}
func (qc *queueConsumer) consumeBuffer() {
defer func() {
err := recover()
if err != nil {
logger.Printf("Error consuming the buffer: %v", err)
}
qc.state.SetState(stopped)
}()
for {
select {
case event, ok := <-qc.eventBuffer:
if !ok {
return
}
logger.Printf("[consumer %s]: From Buffer %#v", qc.queue.topic, event)
select {
case qc.events <- event:
qc.waitEventBuffer.Done()
logger.Printf("[consumer %s]: Buffer->Events %#v", qc.queue.topic, event)
case <-qc.state.WaitForState(stopping):
logger.Printf("[consumer %s] received stopping signal", qc.queue.topic)
logger.Printf("[consumer %s] DROPPING MESSAGE (%#v) because the consumer is closed", qc.queue.topic, event)
qc.waitEventBuffer.Done()
return
}
case <-qc.state.WaitForState(stopping):
logger.Printf("[consumer %s] received stopping signal", qc.queue.topic)
return
}
}
}
func (qc *queueConsumer) catchupAndSync() int {
logger.Printf("[consumer %s] catching up", qc.queue.topic)
numMessages := qc.catchupQueue(-1)
logger.Printf("[consumer %s] catching up DONE (%d messages)", qc.queue.topic, numMessages)
eventsProcessed := make(chan struct{})
go func() {
logger.Printf("[consumer %s] wait for all events to be processed", qc.queue.topic)
qc.waitEventBuffer.Wait()
logger.Printf("[consumer %s] done processing events", qc.queue.topic)
close(eventsProcessed)
}()
select {
case <-eventsProcessed:
case <-qc.state.WaitForState(killed):
// The consumer was killed, so we assume the test is done already.
return 0
case <-qc.state.WaitForState(stopped):
}
return numMessages
}
func (qc *queueConsumer) startGroupConsumer() {
logger.Printf("[consumer %s] starting group consumer", qc.queue.topic)
qc.catchupQueue(-1)
}
func (qc *queueConsumer) addToBuffer(event kafka.Event) {
qc.waitEventBuffer.Add(1)
qc.eventBuffer <- event
if len(qc.eventBuffer) > eventBufferQueueSize*0.9 {
logger.Printf("buffer nearly full: %d, %s. Will drop event.", len(qc.eventBuffer), qc.queue.topic)
<-qc.eventBuffer
}
}
func (qc *queueConsumer) startSimpleConsumer(offset int64, firstStart bool) {
logger.Printf("[consumer %s] starting simple consumer (offset=%d)", qc.queue.topic, offset)
if firstStart {
qc.addToBuffer(&kafka.BOF{
Hwm: qc.queue.hwm,
Offset: 0,
Partition: 0,
Topic: qc.queue.topic,
})
qc.catchupQueue(offset)
qc.addToBuffer(&kafka.EOF{
Hwm: qc.queue.hwm,
Partition: 0,
Topic: qc.queue.topic,
})
}
qc.startLoop(true)
}
func (qc *queueConsumer) catchupQueue(fromOffset int64) int {
// we'll always get from the beginning when the consumer
// requests -1 or -2 (for end or beginning resp)
if fromOffset < 0 {
fromOffset = qc.nextOffset
}
// count how many messages we had to catch up on
var forwardedMessages int
for _, msg := range qc.queue.messagesFromOffset(fromOffset) {
qc.addToBuffer(&kafka.Message{
Key: string(msg.key),
Offset: msg.offset,
Partition: 0,
Timestamp: time.Unix(msg.offset, 0),
Topic: qc.queue.topic,
Value: msg.value,
})
forwardedMessages++
// mark the next offset to consume in case we stop here
qc.nextOffset = msg.offset + 1
}
qc.addToBuffer(&kafka.EOF{
Hwm: qc.queue.hwm,
Partition: 0,
Topic: qc.queue.topic,
})
// push some more NOPs
for i := 0; i < 2; i++ {
qc.addToBuffer(&kafka.NOP{
Partition: 0,
Topic: qc.queue.topic,
})
}
return forwardedMessages
}
func (qc *queueConsumer) rebalance() {
qc.addToBuffer(&kafka.Assignment{
0: -1,
})
}
func newConsumer(tester *Tester) *consumer {
return &consumer{
tester: tester,
events: make(chan kafka.Event, 0),
simpleConsumers: make(map[string]*queueConsumer),
subscribedTopics: make(map[string]*queueConsumer),
}
}
// Events returns the event channel of the consumer mock
func (tc *consumer) Events() <-chan kafka.Event {
return tc.events
}
// Subscribe marks the consumer to subscribe to passed topics.
// The consumerMock simply marks the topics as handled to make sure to
// pass emitted messages back to the processor.
func (tc *consumer) Subscribe(topics map[string]int64) error {
log.Printf("consumer: subscribing to topics: %v", topics)
var anyTopic string
for topic := range topics {
anyTopic = topic
if _, exists := tc.subscribedTopics[topic]; exists {
logger.Printf("consumer for %s already exists. This is strange", topic)
}
logger.Printf("Subscribe %s", topic)
tc.subscribedTopics[topic] = tc.tester.getOrCreateQueue(topic).bindConsumer(tc, true)
tc.subscribedTopics[topic].startLoop(false)
}
tc.subscribedTopics[anyTopic].rebalance()
return nil
}
// AddGroupPartition adds a partition for group consumption.
// No action required in the mock.
func (tc *consumer) AddGroupPartition(partition int32) {
for _, consumer := range tc.subscribedTopics {
logger.Printf("AddGroupPartition %s", consumer.queue.topic)
consumer.startGroupConsumer()
consumer.setRunning()
}
}
// Commit commits an offest.
// No action required in the mock.
func (tc *consumer) Commit(topic string, partition int32, offset int64) error {
return nil
}
// AddPartition marks the topic as a table topic.
// The mock has to know the group table topic to ignore emit calls (which would never be consumed)
func (tc *consumer) AddPartition(topic string, partition int32, initialOffset int64) error {
tc.Lock()
defer tc.Unlock()
logger.Printf("AddPartition %s", topic)
var firstStart bool
if _, exists := tc.simpleConsumers[topic]; !exists {
firstStart = true
tc.simpleConsumers[topic] = tc.tester.getOrCreateQueue(topic).bindConsumer(tc, false)
} else {
logger.Printf("AddPartition %s: consumer already existed. Will reuse the one", topic)
}
if tc.simpleConsumers[topic].isRunning() {
panic(fmt.Errorf("simple consumer for %s already running. RemovePartition not called or race condition", topic))
}
tc.simpleConsumers[topic].startSimpleConsumer(initialOffset, firstStart)
return nil
}
// RemovePartition removes a partition from a topic.
// No action required in the mock.
func (tc *consumer) RemovePartition(topic string, partition int32) error {
logger.Printf("consumer RemovePartition %s", topic)
if cons, exists := tc.simpleConsumers[topic]; exists {
cons.stop()
} else {
logger.Printf("consumer for topic %s did not exist. Cannot Remove partition", topic)
}
return nil
}
// Close closes the consumer.
func (tc *consumer) Close() error {
tc.closeOnce.Do(func() {
logger.Printf("closing tester consumer. Will close all subscribed topics")
for _, cons := range tc.subscribedTopics {
if cons.isRunning() {
logger.Printf("closing queue consumer for %s", cons.queue.topic)
cons.kill()
} else {
logger.Printf("queue consumer for %s is not running", cons.queue.topic)
}
}
for _, cons := range tc.simpleConsumers {
if cons.isRunning() {
logger.Printf("closing simple consumer for %s", cons.queue.topic)
cons.kill()
} else {
logger.Printf("queue consumer for %s is not running", cons.queue.topic)
}
}
close(tc.events)
})
return nil
}