forked from lovoo/goka
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathprocessor_test.go
315 lines (258 loc) · 7.92 KB
/
processor_test.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
package goka
import (
"context"
"fmt"
"strconv"
"strings"
"testing"
"time"
"github.com/Shopify/sarama"
"github.com/golang/mock/gomock"
"github.com/lovoo/goka/codec"
"github.com/lovoo/goka/internal/test"
"github.com/lovoo/goka/storage"
)
func createMockBuilder(t *testing.T) (*gomock.Controller, *builderMock) {
ctrl := NewMockController(t)
bm := newBuilderMock(ctrl)
bm.st = storage.NewMemory()
return ctrl, bm
}
func createTestConsumerGroupBuilder(t *testing.T) (ConsumerGroupBuilder, *MockConsumerGroup) {
mock := NewMockConsumerGroup(t)
return func(brokers []string, group, clientID string) (sarama.ConsumerGroup, error) {
return mock, nil
}, mock
}
func createTestConsumerBuilder(t *testing.T) (SaramaConsumerBuilder, *MockAutoConsumer) {
cons := NewMockAutoConsumer(t, nil)
return func(brokers []string, clientID string) (sarama.Consumer, error) {
return cons, nil
}, cons
}
func expectCGEmit(bm *builderMock, table string, msgs []*sarama.ConsumerMessage) {
for _, msg := range msgs {
bm.producer.EXPECT().EmitWithHeaders(table, string(msg.Key), msg.Value, nil).Return(NewPromise().finish(nil, nil))
}
}
func expectCGLoop(bm *builderMock, loop string, msgs []*sarama.ConsumerMessage) {
bm.tmgr.EXPECT().EnsureStreamExists(loop, 1).AnyTimes()
for _, msg := range msgs {
bm.producer.EXPECT().EmitWithHeaders(loop, string(msg.Key), gomock.Any(), nil).Return(NewPromise().finish(nil, nil))
}
}
func expectCGConsume(bm *builderMock, table string, msgs []*sarama.ConsumerMessage) {
var (
current int64
)
bm.producer.EXPECT().Close().Return(nil).AnyTimes()
bm.tmgr.EXPECT().Close().Return(nil).AnyTimes()
bm.tmgr.EXPECT().EnsureTableExists(table, gomock.Any()).Return(nil)
bm.tmgr.EXPECT().Partitions(gomock.Any()).Return([]int32{0}, nil).AnyTimes()
bm.tmgr.EXPECT().GetOffset(table, gomock.Any(), sarama.OffsetNewest).Return(func() int64 {
defer func() {
current++
}()
return current
}(), nil)
bm.tmgr.EXPECT().GetOffset(table, gomock.Any(), sarama.OffsetOldest).Return(func() int64 {
return 0
}(), nil)
}
// accumulate is a callback that increments the
// table value by the incoming message.
// Persist and incoming codecs must be codec.Int64
func accumulate(ctx Context, msg interface{}) {
inc := msg.(int64)
val := ctx.Value()
if val == nil {
ctx.SetValue(inc)
} else {
ctx.SetValue(val.(int64) + inc)
}
}
func TestProcessor_Run(t *testing.T) {
t.Run("input-persist", func(t *testing.T) {
ctrl, bm := createMockBuilder(t)
defer ctrl.Finish()
var (
topic = "test-table"
toEmit = []*sarama.ConsumerMessage{
{
Topic: "input",
Value: []byte(strconv.FormatInt(3, 10)),
Key: []byte("test-key-1"),
},
{
Topic: "input",
Value: []byte(strconv.FormatInt(3, 10)),
Key: []byte("test-key-2"),
},
}
)
expectCGConsume(bm, topic, toEmit)
expectCGEmit(bm, topic, toEmit)
groupBuilder, cg := createTestConsumerGroupBuilder(t)
consBuilder, cons := createTestConsumerBuilder(t)
graph := DefineGroup("test",
Input("input", new(codec.Int64), accumulate),
Persist(new(codec.Int64)),
)
ctx, cancel := context.WithTimeout(context.Background(), time.Second*1000)
defer cancel()
newProc, err := NewProcessor([]string{"localhost:9092"}, graph,
bm.createProcessorOptions(consBuilder, groupBuilder)...,
)
test.AssertNil(t, err)
var (
procErr error
done = make(chan struct{})
)
cons.ExpectConsumePartition(topic, 0, 0)
go func() {
defer close(done)
procErr = newProc.Run(ctx)
}()
newProc.WaitForReady()
// if there was an error during startup, no point in sending messages
// and waiting for them to be delivered
test.AssertNil(t, procErr)
for _, msg := range toEmit {
cg.SendMessageWait(msg)
}
val, err := newProc.Get("test-key-1")
test.AssertNil(t, err)
test.AssertEqual(t, val.(int64), int64(3))
val, err = newProc.Get("test-key-2")
test.AssertNil(t, err)
test.AssertEqual(t, val.(int64), int64(3))
// shutdown
newProc.Stop()
<-done
test.AssertNil(t, procErr)
})
t.Run("loopback", func(t *testing.T) {
ctrl, bm := createMockBuilder(t)
defer ctrl.Finish()
var (
topic = "test-table"
loop = "test-loop"
toEmit = []*sarama.ConsumerMessage{
{
Topic: "input",
Value: []byte(strconv.FormatInt(23, 10)),
Key: []byte("test-key"),
},
}
)
expectCGConsume(bm, topic, toEmit)
expectCGLoop(bm, loop, toEmit)
groupBuilder, cg := createTestConsumerGroupBuilder(t)
consBuilder, cons := createTestConsumerBuilder(t)
graph := DefineGroup("test",
// input passes to loopback
Input("input", new(codec.Int64), func(ctx Context, msg interface{}) {
ctx.Loopback(ctx.Key(), msg)
}),
// this will not be called in the test but we define it, otherwise the context will raise an error
Loop(new(codec.Int64), accumulate),
Persist(new(codec.Int64)),
)
ctx, cancel := context.WithTimeout(context.Background(), time.Second)
defer cancel()
newProc, err := NewProcessor([]string{"localhost:9092"}, graph,
bm.createProcessorOptions(consBuilder, groupBuilder)...,
)
test.AssertNil(t, err)
var (
procErr error
done = make(chan struct{})
)
cons.ExpectConsumePartition(topic, 0, 0)
go func() {
defer close(done)
procErr = newProc.Run(ctx)
}()
newProc.WaitForReady()
// if there was an error during startup, no point in sending messages
// and waiting for them to be delivered
test.AssertNil(t, procErr)
for _, msg := range toEmit {
cg.SendMessageWait(msg)
}
// shutdown
newProc.Stop()
<-done
test.AssertNil(t, procErr)
})
t.Run("consume-error", func(t *testing.T) {
ctrl, bm := createMockBuilder(t)
defer ctrl.Finish()
bm.tmgr.EXPECT().Close().Times(1)
bm.tmgr.EXPECT().Partitions(gomock.Any()).Return([]int32{0}, nil).Times(1)
bm.producer.EXPECT().Close().Times(1)
groupBuilder, cg := createTestConsumerGroupBuilder(t)
consBuilder, _ := createTestConsumerBuilder(t)
graph := DefineGroup("test",
// not really used, we're failing anyway
Input("input", new(codec.Int64), accumulate),
)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
newProc, err := NewProcessor([]string{"localhost:9092"}, graph,
bm.createProcessorOptions(consBuilder, groupBuilder)...,
)
test.AssertNil(t, err)
var (
procErr error
done = make(chan struct{})
)
go func() {
defer close(done)
procErr = newProc.Run(ctx)
}()
newProc.WaitForReady()
// if there was an error during startup, no point in sending messages
// and waiting for them to be delivered
test.AssertNil(t, procErr)
cg.SendError(fmt.Errorf("test-error"))
cancel()
<-done
// the errors sent back by the consumergroup do not lead to a failure of the processor
test.AssertNil(t, procErr)
})
t.Run("consgroup-error", func(t *testing.T) {
ctrl, bm := createMockBuilder(t)
defer ctrl.Finish()
bm.tmgr.EXPECT().Close().Times(1)
bm.tmgr.EXPECT().Partitions(gomock.Any()).Return([]int32{0}, nil).Times(1)
bm.producer.EXPECT().Close().Times(1)
groupBuilder, cg := createTestConsumerGroupBuilder(t)
consBuilder, _ := createTestConsumerBuilder(t)
graph := DefineGroup("test",
// not really used, we're failing anyway
Input("input", new(codec.Int64), accumulate),
)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
newProc, err := NewProcessor([]string{"localhost:9092"}, graph,
bm.createProcessorOptions(consBuilder, groupBuilder)...,
)
test.AssertNil(t, err)
var (
procErr error
done = make(chan struct{})
)
cg.FailOnConsume(fmt.Errorf("consume-error"))
go func() {
defer close(done)
procErr = newProc.Run(ctx)
}()
newProc.WaitForReady()
// if there was an error during startup, no point in sending messages
// and waiting for them to be delivered
<-done
// the errors sent back by the consumergroup do not lead to a failure of the processor
test.AssertTrue(t, strings.Contains(procErr.Error(), "consume-error"))
})
}