-
Notifications
You must be signed in to change notification settings - Fork 2
/
worker.go
326 lines (269 loc) · 7.44 KB
/
worker.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
package kstreams
import (
"context"
"errors"
"fmt"
"log/slog"
"sync"
"time"
"github.com/twmb/franz-go/pkg/kadm"
"github.com/twmb/franz-go/pkg/kgo"
)
type RoutineState string
const (
StateCreated = "CREATED"
StatePartitionsAssigned = "PARTITIONS_ASSIGNED"
StateRunning = "RUNNING"
StateCloseRequested = "CLOSE_REQUESTED"
StateClosed = "CLOSED"
)
// Nice read https://jaceklaskowski.gitbooks.io/mastering-kafka-streams/content/kafka-streams-internals-StreamThread.html
type Worker struct {
client *kgo.Client
adminClient *kadm.Client
log *slog.Logger
group string
t *Topology
state RoutineState
assignedOrRevoked chan AssignedOrRevoked
newlyAssigned map[string][]int32
newlyRevoked map[string][]int32
closeRequested chan struct{}
name string
cancelPollMtx sync.Mutex
cancelPoll func()
closed sync.WaitGroup
maxPollRecords int
taskManager *TaskManager
lastSuccessfulCommit time.Time
commitInterval time.Duration
err error
}
type AssignedOrRevoked struct {
Assigned map[string][]int32
Revoked map[string][]int32
}
// Config
func NewWorker(log *slog.Logger, name string, t *Topology, group string, brokers []string, commitInterval time.Duration) (*Worker, error) {
tm := &TaskManager{
tasks: []*Task{},
log: log,
topology: t,
pgs: t.partitionGroups(),
}
par := make(chan AssignedOrRevoked)
topics := t.GetTopics()
client, err := kgo.NewClient(
kgo.SeedBrokers(brokers...),
kgo.ConsumerGroup(group),
kgo.Balancers(NewPartitionGroupBalancer(log, t.partitionGroups())),
kgo.DisableAutoCommit(),
kgo.ConsumeTopics(topics...),
kgo.OnPartitionsAssigned(func(c1 context.Context, c2 *kgo.Client, m map[string][]int32) {
par <- AssignedOrRevoked{Assigned: m}
}),
kgo.OnPartitionsRevoked(func(c1 context.Context, c2 *kgo.Client, m map[string][]int32) {
par <- AssignedOrRevoked{Revoked: m}
}),
)
if err != nil {
return nil, err
}
tm.client = client
w := &Worker{
name: name,
log: log.With("worker", name),
group: group,
client: client,
adminClient: kadm.NewClient(client),
t: t,
state: StateCreated,
assignedOrRevoked: par,
closeRequested: make(chan struct{}, 1),
maxPollRecords: 10000,
taskManager: tm,
commitInterval: commitInterval,
}
w.closed.Add(1)
return w, nil
}
func (r *Worker) changeState(newState RoutineState) {
r.log.Info("Change state", "from", r.state, "to", newState)
r.state = newState
}
func (r *Worker) Run() error {
return r.Loop()
}
func (r *Worker) handleRunning() {
r.cancelPollMtx.Lock()
select {
case ev := <-r.assignedOrRevoked:
r.newlyAssigned = ev.Assigned
r.newlyRevoked = ev.Revoked
r.changeState(StatePartitionsAssigned)
r.cancelPollMtx.Unlock()
return
default:
}
select {
case <-r.closeRequested:
r.changeState(StateCloseRequested)
r.cancelPollMtx.Unlock()
return
default:
}
pollCtx, cancel := context.WithTimeout(context.Background(), time.Second*10)
defer cancel()
r.cancelPoll = cancel
r.cancelPollMtx.Unlock()
r.log.Debug("Polling Records")
f := r.client.PollRecords(pollCtx, r.maxPollRecords)
r.log.Debug("Polled Records")
if f.IsClientClosed() {
r.changeState(StateCloseRequested)
return
}
if errors.Is(f.Err(), context.Canceled) {
return
}
if !errors.Is(f.Err(), context.DeadlineExceeded) {
for _, fetchError := range f.Errors() {
if errors.Is(fetchError.Err, context.DeadlineExceeded) {
continue
}
r.log.Error("fetch error", fetchError.Err, "topic", fetchError.Topic, "partition", fetchError.Partition)
if fetchError.Err != nil {
r.err = fmt.Errorf("fetch error on topic %s, partition %d: %w", fetchError.Topic, fetchError.Partition, fetchError.Err)
r.changeState(StateCloseRequested)
return
}
}
f.EachPartition(func(fetch kgo.FetchTopicPartition) {
r.log.Info("Processing", "topic", fetch.Topic, "partition", fetch.Partition)
task, err := r.taskManager.TaskFor(fetch.Topic, fetch.Partition)
if err != nil {
r.log.Error("failed to lookup task", err, "topic", fetch.Topic, "partition", fetch.Partition)
r.changeState(StateCloseRequested)
return
}
count := 0
for _, record := range fetch.Records {
count++
recordCtx, cancel := context.WithTimeout(context.Background(), time.Second*60) // TODO make this configurable
err := task.Process(recordCtx, record)
cancel()
if err != nil {
r.log.Error("Failed to process record", err) // TODO - provide middlewares to handle this error. is it always a user code error?
r.changeState(StateCloseRequested)
r.err = err
return
}
}
r.log.Info("Processed", "topic", fetch.Topic, "partition", fetch.Partition)
})
}
commitCtx, cancel := context.WithTimeout(context.Background(), time.Second*10)
defer cancel()
if err := r.maybeCommit(commitCtx); err != nil {
r.log.Error("failed to commit", err)
r.changeState(StateCloseRequested)
r.err = err
return
}
r.log.Info("Committed offests")
}
func (r *Worker) maybeCommit(ctx context.Context) error {
if time.Since(r.lastSuccessfulCommit) >= r.commitInterval {
if err := r.client.Flush(ctx); err != nil {
return fmt.Errorf("failed to flush: %w", err)
}
if err := r.taskManager.Commit(ctx); err != nil {
return fmt.Errorf("failed to commit: %w", err)
}
r.lastSuccessfulCommit = time.Now()
}
return nil
}
func (r *Worker) handleClosed() {
r.closed.Done()
}
func (r *Worker) handlePartitionsAssigned() {
if err := r.taskManager.Revoked(r.newlyRevoked); err != nil {
r.log.Error("revoked failed", err)
}
if err := r.taskManager.Assigned(r.newlyAssigned); err != nil {
r.log.Error("assigned failed", err)
r.changeState(StateCloseRequested)
return
}
r.newlyAssigned = nil
r.newlyRevoked = nil
if len(r.taskManager.tasks) > 0 {
r.changeState(StateRunning)
} else {
r.changeState(StateCreated)
}
}
func (r *Worker) handleCloseRequested() {
wg := sync.WaitGroup{}
wg.Add(1)
if err := r.client.Flush(context.TODO()); err != nil {
r.log.Error("Failed to flush client", err)
}
if err := r.taskManager.Commit(context.TODO()); err != nil {
r.log.Error("Failed to commit", err)
}
go func() {
// Wait until this is closed
for range r.assignedOrRevoked {
// TODO: OnRevoke: do commit here if needed
}
wg.Done()
}()
err := r.taskManager.Close(context.TODO())
if err != nil {
r.log.Error("Failed to close tasks", err)
}
r.client.Close()
close(r.assignedOrRevoked) // Can close this only after client is closed, as the client writer to that channel
wg.Wait()
r.changeState(StateClosed)
}
func (r *Worker) handleCreated() {
select {
case assignments := <-r.assignedOrRevoked:
r.newlyAssigned = assignments.Assigned
r.newlyRevoked = assignments.Revoked
r.changeState(StatePartitionsAssigned)
case <-r.closeRequested:
r.changeState(StateCloseRequested)
}
}
// State transitions may only be done from within the loop
func (r *Worker) Loop() error {
for {
switch r.state {
case StateCreated:
r.handleCreated()
case StateCloseRequested:
r.handleCloseRequested()
case StatePartitionsAssigned:
r.handlePartitionsAssigned()
case StateRunning:
r.handleRunning()
case StateClosed:
r.handleClosed()
return r.err
}
}
}
func (r *Worker) Close() error {
r.cancelPollMtx.Lock()
r.closeRequested <- struct{}{}
if r.cancelPoll != nil {
r.cancelPoll()
}
r.cancelPollMtx.Unlock()
r.closed.Wait()
return nil
}