forked from lovoo/goka
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathpartition_processor.go
591 lines (508 loc) · 14.8 KB
/
partition_processor.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
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
package goka
import (
"context"
"fmt"
"strings"
"sync"
"time"
"github.com/Shopify/sarama"
"github.com/lovoo/goka/multierr"
)
const (
// PPStateIdle marks the partition processor as idling (not started yet)
PPStateIdle State = iota
// PPStateRecovering indicates a recovering partition processor
PPStateRecovering
// PPStateRunning indicates a running partition processor
PPStateRunning
// PPStateStopping indicates a stopped partition processor
PPStateStopping
)
// PPRunMode configures how the partition processor participates as part of the processor
type PPRunMode int
const (
// default mode: the processor recovers once and consumes messages
runModeActive PPRunMode = iota
// the processor keeps recovering. This is used for hot standby.
runModePassive
// the processor only recovers once and then stops. This is used for recover-ahead-option
runModeRecoverOnly
)
type commitCallback func(msg *sarama.ConsumerMessage, meta string)
// PartitionProcessor handles message processing of one partition by serializing
// messages from different input topics.
// It also handles joined tables as well as lookup views (managed by `Processor`).
type PartitionProcessor struct {
callbacks map[string]ProcessCallback
log logger
table *PartitionTable
joins map[string]*PartitionTable
lookups map[string]*View
graph *GroupGraph
state *Signal
partition int32
input chan *sarama.ConsumerMessage
inputTopics []string
runnerGroup *multierr.ErrGroup
cancelRunnerGroup func()
// runnerErrors store the errors occuring during runtime of the
// partition processor. It is created in Setup and after the runnerGroup
// finishes.
runnerErrors chan error
runMode PPRunMode
consumer sarama.Consumer
tmgr TopicManager
stats *PartitionProcStats
requestStats chan bool
responseStats chan *PartitionProcStats
updateStats chan func()
cancelStatsLoop context.CancelFunc
commit commitCallback
producer Producer
opts *poptions
}
func newPartitionProcessor(partition int32,
graph *GroupGraph,
commit commitCallback,
logger logger,
opts *poptions,
runMode PPRunMode,
lookupTables map[string]*View,
consumer sarama.Consumer,
producer Producer,
tmgr TopicManager,
backoff Backoff,
backoffResetTime time.Duration) *PartitionProcessor {
// collect all topics I am responsible for
topicMap := make(map[string]bool)
for _, stream := range graph.InputStreams() {
topicMap[stream.Topic()] = true
}
if loop := graph.LoopStream(); loop != nil {
topicMap[loop.Topic()] = true
}
var (
topicList []string
outputList []string
callbacks = make(map[string]ProcessCallback)
)
for t := range topicMap {
topicList = append(topicList, t)
callbacks[t] = graph.callback(t)
}
for _, output := range graph.OutputStreams() {
outputList = append(outputList, output.Topic())
}
if graph.LoopStream() != nil {
outputList = append(outputList, graph.LoopStream().Topic())
}
if graph.GroupTable() != nil {
outputList = append(outputList, graph.GroupTable().Topic())
}
log := logger.Prefix(fmt.Sprintf("PartitionProcessor (%d)", partition))
statsLoopCtx, cancel := context.WithCancel(context.Background())
partProc := &PartitionProcessor{
log: log,
opts: opts,
partition: partition,
state: NewSignal(PPStateIdle, PPStateRecovering, PPStateRunning, PPStateStopping).SetState(PPStateIdle),
callbacks: callbacks,
lookups: lookupTables,
consumer: consumer,
producer: producer,
tmgr: tmgr,
joins: make(map[string]*PartitionTable),
input: make(chan *sarama.ConsumerMessage, opts.partitionChannelSize),
inputTopics: topicList,
graph: graph,
stats: newPartitionProcStats(topicList, outputList),
requestStats: make(chan bool),
responseStats: make(chan *PartitionProcStats, 1),
updateStats: make(chan func(), 10),
cancelStatsLoop: cancel,
commit: commit,
runMode: runMode,
}
go partProc.runStatsLoop(statsLoopCtx)
if graph.GroupTable() != nil {
partProc.table = newPartitionTable(graph.GroupTable().Topic(),
partition,
consumer,
tmgr,
opts.updateCallback,
opts.builders.storage,
log.Prefix("PartTable"),
backoff,
backoffResetTime,
)
}
return partProc
}
// EnqueueMessage enqueues a message in the partition processor's event channel for processing
func (pp *PartitionProcessor) EnqueueMessage(msg *sarama.ConsumerMessage) {
pp.input <- msg
}
// Recovered returns whether the processor is running (i.e. all joins, lookups and the table is recovered and it's consuming messages)
func (pp *PartitionProcessor) Recovered() bool {
return pp.state.IsState(PPStateRunning)
}
// Errors returns a channel of errors during consumption
func (pp *PartitionProcessor) Errors() <-chan error {
return pp.runnerErrors
}
// Start initializes the partition processor
// * recover the table
// * recover all join tables
// * run the join-tables in catchup mode
// * start the processor processing loop to receive messages
func (pp *PartitionProcessor) Start(ctx context.Context) error {
ctx, pp.cancelRunnerGroup = context.WithCancel(ctx)
var runnerCtx context.Context
pp.runnerGroup, runnerCtx = multierr.NewErrGroup(ctx)
pp.runnerErrors = make(chan error, 1)
defer func() {
go func() {
defer close(pp.runnerErrors)
err := pp.runnerGroup.Wait().NilOrError()
if err != nil {
pp.runnerErrors <- err
}
}()
}()
setupErrg, setupCtx := multierr.NewErrGroup(ctx)
pp.state.SetState(PPStateRecovering)
defer pp.state.SetState(PPStateRunning)
if pp.table != nil {
go pp.table.RunStatsLoop(runnerCtx)
setupErrg.Go(func() error {
pp.log.Debugf("catching up table")
defer pp.log.Debugf("catching up table done")
return pp.table.SetupAndRecover(setupCtx, false)
})
}
for _, join := range pp.graph.JointTables() {
table := newPartitionTable(join.Topic(),
pp.partition,
pp.consumer,
pp.tmgr,
pp.opts.updateCallback,
pp.opts.builders.storage,
pp.log.Prefix(fmt.Sprintf("Join %s", join.Topic())),
NewSimpleBackoff(time.Second*10),
time.Minute,
)
pp.joins[join.Topic()] = table
go table.RunStatsLoop(runnerCtx)
setupErrg.Go(func() error {
return table.SetupAndRecover(setupCtx, false)
})
}
// here we wait for our table and the joins to recover
err := setupErrg.Wait().NilOrError()
if err != nil {
return fmt.Errorf("Setup failed. Cannot start processor for partition %d: %v", pp.partition, err)
}
select {
case <-ctx.Done():
return nil
default:
}
// at this point, we have successfully recovered all joins and the table of the partition-processor.
// If the partition-processor was started to do only that (e.g. for group-recover-ahead), we
// will return here
if pp.runMode == runModeRecoverOnly {
return nil
}
for _, join := range pp.joins {
join := join
pp.runnerGroup.Go(func() error {
return join.CatchupForever(runnerCtx, false)
})
}
// now run the processor in a runner-group
pp.runnerGroup.Go(func() error {
var err error
// depending on the run mode, we'll do
switch pp.runMode {
// (a) start the processor's message run loop so it is ready to receive and process messages
case runModeActive:
err = pp.run(runnerCtx)
// (b) run the processor table in catchup mode so it keeps updating it's state.
case runModePassive:
if pp.table != nil {
err = pp.table.CatchupForever(runnerCtx, false)
}
default:
return fmt.Errorf("processor has invalid run mode")
}
if err != nil {
pp.log.Debugf("Run failed with error: %v", err)
}
return err
})
return nil
}
// Stop stops the partition processor
func (pp *PartitionProcessor) Stop() error {
pp.log.Debugf("Stopping")
defer pp.log.Debugf("... Stopping done")
pp.state.SetState(PPStateStopping)
defer pp.state.SetState(PPStateIdle)
errs := new(multierr.Errors)
if pp.cancelRunnerGroup != nil {
pp.cancelRunnerGroup()
}
if pp.runnerGroup != nil {
errs.Collect(<-pp.Errors())
}
// stop the stats updating/serving loop
pp.cancelStatsLoop()
errg, _ := multierr.NewErrGroup(context.Background())
for _, join := range pp.joins {
join := join
errg.Go(func() error {
return join.Close()
})
}
if pp.table != nil {
errg.Go(func() error {
return pp.table.Close()
})
}
errs.Collect(errg.Wait().NilOrError())
return errs.NilOrError()
}
func (pp *PartitionProcessor) run(ctx context.Context) (rerr error) {
pp.log.Debugf("starting")
defer pp.log.Debugf("stopped")
errs := new(multierr.Errors)
defer func() {
errs.Collect(rerr)
rerr = errs.NilOrError()
}()
var (
// syncFailer is called synchronously from the callback within *this*
// goroutine
syncFailer = func(err error) {
// only fail processor if context not already Done
select {
case <-ctx.Done():
rerr = err
return
default:
}
panic(err)
}
closeOnce = new(sync.Once)
asyncErrs = make(chan struct{})
// asyncFailer is called asynchronously from other goroutines, e.g.
// when the promise of a Emit (using a producer internally) fails
asyncFailer = func(err error) {
errs.Collect(err)
closeOnce.Do(func() { close(asyncErrs) })
}
wg sync.WaitGroup
)
defer func() {
if r := recover(); r != nil {
rerr = fmt.Errorf("%v\n%v", r, strings.Join(userStacktrace(), "\n"))
return
}
done := make(chan struct{})
go func() {
wg.Wait()
close(done)
}()
select {
case <-done:
case <-time.NewTimer(60 * time.Second).C:
pp.log.Printf("partition processor did not shutdown in time. Will stop waiting")
}
}()
for {
select {
case ev, isOpen := <-pp.input:
// channel already closed, ev will be nil
if !isOpen {
return nil
}
err := pp.processMessage(ctx, &wg, ev, syncFailer, asyncFailer)
if err != nil {
return fmt.Errorf("error processing message: from %s %v", ev.Value, err)
}
pp.enqueueStatsUpdate(ctx, func() { pp.updateStatsWithMessage(ev) })
case <-ctx.Done():
pp.log.Debugf("exiting, context is cancelled")
return
case <-asyncErrs:
pp.log.Debugf("Errors occurred asynchronously. Will exit partition processor")
return
}
}
}
func (pp *PartitionProcessor) enqueueStatsUpdate(ctx context.Context, updater func()) {
select {
case pp.updateStats <- updater:
case <-ctx.Done():
default:
// going to default indicates the updateStats channel is not read, so so the stats
// loop is not actually running.
// We must not block here, so we'll skip the update
}
}
func (pp *PartitionProcessor) runStatsLoop(ctx context.Context) {
updateHwmStatsTicker := time.NewTicker(statsHwmUpdateInterval)
defer updateHwmStatsTicker.Stop()
for {
select {
case <-pp.requestStats:
stats := pp.collectStats(ctx)
select {
case pp.responseStats <- stats:
case <-ctx.Done():
pp.log.Debugf("exiting, context is cancelled")
return
}
case update := <-pp.updateStats:
update()
case <-updateHwmStatsTicker.C:
pp.updateHwmStats()
case <-ctx.Done():
return
}
}
}
// updateStatsWithMessage updates the stats with a received message
func (pp *PartitionProcessor) updateStatsWithMessage(ev *sarama.ConsumerMessage) {
ip := pp.stats.Input[ev.Topic]
ip.Bytes += len(ev.Value)
ip.LastOffset = ev.Offset
if !ev.Timestamp.IsZero() {
ip.Delay = time.Since(ev.Timestamp)
}
ip.Count++
}
// updateHwmStats updates the offset lag for all input topics based on the
// highwatermarks obtained by the consumer.
func (pp *PartitionProcessor) updateHwmStats() {
hwms := pp.consumer.HighWaterMarks()
for input, inputStats := range pp.stats.Input {
hwm := hwms[input][pp.partition]
if hwm != 0 && inputStats.LastOffset != 0 {
inputStats.OffsetLag = hwm - inputStats.LastOffset
}
}
}
func (pp *PartitionProcessor) collectStats(ctx context.Context) *PartitionProcStats {
var (
stats = pp.stats.clone()
m sync.Mutex
)
errg, ctx := multierr.NewErrGroup(ctx)
for topic, join := range pp.joins {
topic, join := topic, join
errg.Go(func() error {
joinStats := join.fetchStats(ctx)
if joinStats != nil {
joinStats.RunMode = pp.runMode
}
m.Lock()
defer m.Unlock()
stats.Joined[topic] = joinStats
return nil
})
}
if pp.table != nil {
errg.Go(func() error {
stats.TableStats = pp.table.fetchStats(ctx)
if stats.TableStats != nil {
stats.TableStats.RunMode = pp.runMode
}
return nil
})
}
err := errg.Wait().NilOrError()
if err != nil {
pp.log.Printf("Error retrieving stats: %v", err)
}
return stats
}
func (pp *PartitionProcessor) fetchStats(ctx context.Context) *PartitionProcStats {
select {
case <-ctx.Done():
return nil
case <-time.After(fetchStatsTimeout):
pp.log.Printf("requesting stats timed out")
return nil
case pp.requestStats <- true:
}
// retrieve from response-channel
select {
case <-ctx.Done():
return nil
case <-time.After(fetchStatsTimeout):
pp.log.Printf("Fetching stats timed out")
return nil
case stats := <-pp.responseStats:
return stats
}
}
func (pp *PartitionProcessor) enqueueTrackOutputStats(ctx context.Context, topic string, size int) {
pp.enqueueStatsUpdate(ctx, func() {
pp.stats.trackOutput(topic, size)
})
}
func (pp *PartitionProcessor) processMessage(ctx context.Context, wg *sync.WaitGroup, msg *sarama.ConsumerMessage, syncFailer func(err error), asyncFailer func(err error)) error {
msgContext := &cbContext{
ctx: ctx,
graph: pp.graph,
trackOutputStats: pp.enqueueTrackOutputStats,
pviews: pp.joins,
views: pp.lookups,
commit: func() { pp.commit(msg, "") },
wg: wg,
msg: msg,
syncFailer: syncFailer,
asyncFailer: asyncFailer,
emitter: pp.producer.EmitWithHeaders,
emitterDefaultHeaders: pp.opts.producerDefaultHeaders,
table: pp.table,
}
var (
m interface{}
err error
)
// decide whether to decode or ignore message
switch {
case msg.Value == nil && pp.opts.nilHandling == NilIgnore:
// mark the message upstream so we don't receive it again.
// this is usually only an edge case in unit tests, as kafka probably never sends us nil messages
pp.commit(msg, "")
// otherwise drop it.
return nil
case msg.Value == nil && pp.opts.nilHandling == NilProcess:
// process nil messages without decoding them
m = nil
default:
// get stream subcription
codec := pp.graph.codec(msg.Topic)
if codec == nil {
return fmt.Errorf("cannot handle topic %s", msg.Topic)
}
// decode message
m, err = codec.Decode(msg.Value)
if err != nil {
return fmt.Errorf("error decoding message for key %s from %s/%d: %v", msg.Key, msg.Topic, msg.Partition, err)
}
}
cb := pp.callbacks[msg.Topic]
if cb == nil {
return fmt.Errorf("error processing message for key %s from %s/%d: %v", string(msg.Key), msg.Topic, msg.Partition, err)
}
// start context and call the ProcessorCallback cb
msgContext.start()
// now call cb
cb(msgContext, m)
msgContext.finish(nil)
return nil
}