-
Notifications
You must be signed in to change notification settings - Fork 3.6k
/
Copy pathreader_test.go
234 lines (192 loc) · 7.45 KB
/
reader_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
package partition
import (
"context"
"fmt"
"sync"
"testing"
"time"
"github.com/go-kit/log"
"github.com/grafana/dskit/services"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/prometheus/model/labels"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/require"
"github.com/twmb/franz-go/pkg/kgo"
"github.com/grafana/loki/v3/pkg/kafka"
"github.com/grafana/loki/v3/pkg/kafka/client"
"github.com/grafana/loki/v3/pkg/kafka/testkafka"
"github.com/grafana/loki/v3/pkg/logproto"
)
type mockConsumer struct {
mock.Mock
recordsChan chan []Record
wg sync.WaitGroup
}
func newMockConsumer() *mockConsumer {
return &mockConsumer{
recordsChan: make(chan []Record, 100),
}
}
func (m *mockConsumer) Start(ctx context.Context, recordsChan <-chan []Record) func() {
m.wg.Add(1)
go func() {
defer m.wg.Done()
for {
select {
case <-ctx.Done():
return
case records, ok := <-recordsChan:
if !ok {
return
}
m.recordsChan <- records
}
}
}()
return m.wg.Wait
}
func (m *mockConsumer) Flush(ctx context.Context) error {
args := m.Called(ctx)
return args.Error(0)
}
func TestPartitionReader_BasicFunctionality(t *testing.T) {
_, kafkaCfg := testkafka.CreateCluster(t, 1, "test")
consumer := newMockConsumer()
consumerFactory := func(_ Committer) (Consumer, error) {
return consumer, nil
}
partitionReader, err := NewReader(kafkaCfg, 0, "test-consumer-group", consumerFactory, log.NewNopLogger(), prometheus.NewRegistry())
require.NoError(t, err)
producer, err := client.NewWriterClient(kafkaCfg, 100, log.NewNopLogger(), prometheus.NewRegistry())
require.NoError(t, err)
err = services.StartAndAwaitRunning(context.Background(), partitionReader)
require.NoError(t, err)
stream := logproto.Stream{
Labels: labels.FromStrings("foo", "bar").String(),
Entries: []logproto.Entry{{Timestamp: time.Now(), Line: "test"}},
}
records, err := kafka.Encode(0, "test-tenant", stream, 10<<20)
require.NoError(t, err)
require.Len(t, records, 1)
require.NoError(t, producer.ProduceSync(context.Background(), records...).FirstErr())
require.NoError(t, producer.ProduceSync(context.Background(), records...).FirstErr())
// Wait for records to be processed
assert.Eventually(t, func() bool {
return len(consumer.recordsChan) == 2
}, 10*time.Second, 100*time.Millisecond)
// Verify the records
for i := 0; i < 2; i++ {
select {
case receivedRecords := <-consumer.recordsChan:
require.Len(t, receivedRecords, 1)
assert.Equal(t, "test-tenant", receivedRecords[0].TenantID)
assert.Equal(t, records[0].Value, receivedRecords[0].Content)
case <-time.After(1 * time.Second):
t.Fatal("Timeout waiting for records")
}
}
err = services.StopAndAwaitTerminated(context.Background(), partitionReader)
require.NoError(t, err)
}
func TestPartitionReader_ProcessCatchUpAtStartup(t *testing.T) {
_, kafkaCfg := testkafka.CreateCluster(t, 1, "test-topic")
var consumerStarting *mockConsumer
consumerFactory := func(_ Committer) (Consumer, error) {
// Return two consumers to ensure we are processing requests during service `start()` and not during `run()`.
if consumerStarting == nil {
consumerStarting = newMockConsumer()
return consumerStarting, nil
}
return newMockConsumer(), nil
}
partitionReader, err := NewReader(kafkaCfg, 0, "test-consumer-group", consumerFactory, log.NewNopLogger(), prometheus.NewRegistry())
require.NoError(t, err)
producer, err := client.NewWriterClient(kafkaCfg, 100, log.NewNopLogger(), prometheus.NewRegistry())
require.NoError(t, err)
stream := logproto.Stream{
Labels: labels.FromStrings("foo", "bar").String(),
Entries: []logproto.Entry{{Timestamp: time.Now(), Line: "test"}},
}
records, err := kafka.Encode(0, "test-tenant", stream, 10<<20)
require.NoError(t, err)
require.Len(t, records, 1)
producer.ProduceSync(context.Background(), records...)
producer.ProduceSync(context.Background(), records...)
// Enable the catch up logic so starting the reader will read any existing records.
kafkaCfg.TargetConsumerLagAtStartup = time.Second * 1
kafkaCfg.MaxConsumerLagAtStartup = time.Second * 2
err = services.StartAndAwaitRunning(context.Background(), partitionReader)
require.NoError(t, err)
// This message should not be processed by the startingConsumer
producer.ProduceSync(context.Background(), records...)
// Wait for records to be processed
require.Eventually(t, func() bool {
return len(consumerStarting.recordsChan) == 1 // All pending messages will be received in one batch
}, 10*time.Second, 10*time.Millisecond)
receivedRecords := <-consumerStarting.recordsChan
require.Len(t, receivedRecords, 2)
assert.Equal(t, "test-tenant", receivedRecords[0].TenantID)
assert.Equal(t, records[0].Value, receivedRecords[0].Content)
assert.Equal(t, "test-tenant", receivedRecords[1].TenantID)
assert.Equal(t, records[0].Value, receivedRecords[1].Content)
assert.Equal(t, 0, len(consumerStarting.recordsChan))
err = services.StopAndAwaitTerminated(context.Background(), partitionReader)
require.NoError(t, err)
}
func TestPartitionReader_ProcessCommits(t *testing.T) {
_, kafkaCfg := testkafka.CreateCluster(t, 1, "test-topic")
consumer := newMockConsumer()
consumerFactory := func(_ Committer) (Consumer, error) {
return consumer, nil
}
partitionID := int32(0)
partitionReader, err := NewReader(kafkaCfg, partitionID, "test-consumer-group", consumerFactory, log.NewNopLogger(), prometheus.NewRegistry())
require.NoError(t, err)
producer, err := client.NewWriterClient(kafkaCfg, 100, log.NewNopLogger(), prometheus.NewRegistry())
require.NoError(t, err)
// Init the client: This usually happens in "start" but we want to manage our own lifecycle for this test.
partitionReader.client, err = client.NewReaderClient(kafkaCfg, nil, log.NewNopLogger(),
kgo.ConsumePartitions(map[string]map[int32]kgo.Offset{
kafkaCfg.Topic: {partitionID: kgo.NewOffset().AtStart()},
}),
)
require.NoError(t, err)
stream := logproto.Stream{
Labels: labels.FromStrings("foo", "bar").String(),
Entries: []logproto.Entry{{Timestamp: time.Now(), Line: "test"}},
}
records, err := kafka.Encode(partitionID, "test-tenant", stream, 10<<20)
require.NoError(t, err)
require.Len(t, records, 1)
ctx, cancel := context.WithDeadlineCause(context.Background(), time.Now().Add(10*time.Second), fmt.Errorf("test unexpectedly deadlocked"))
recordsChan := make(chan []Record)
wait := consumer.Start(ctx, recordsChan)
targetLag := time.Second
i := -1
iterations := 5
producer.ProduceSync(context.Background(), records...)
// timeSince acts as a hook for when we check if we've honoured the lag or not. We modify it to respond "no" initially, to force a re-loop, and then "yes" after `iterations`.
// We also inject a new kafka record each time so there is more to consume.
timeSince := func(time.Time) time.Duration {
i++
if i < iterations {
producer.ProduceSync(context.Background(), records...)
return targetLag + 1
}
return targetLag - 1
}
_, err = partitionReader.processNextFetchesUntilLagHonored(ctx, targetLag, log.NewNopLogger(), recordsChan, timeSince)
assert.NoError(t, err)
// Wait to process all the records
cancel()
wait()
close(recordsChan)
close(consumer.recordsChan)
recordsCount := 0
for receivedRecords := range consumer.recordsChan {
recordsCount += len(receivedRecords)
}
// We expect to have processed all the records, including initial + one per iteration.
assert.Equal(t, iterations+1, recordsCount)
}