-
Notifications
You must be signed in to change notification settings - Fork 172
/
partition_processor.go
714 lines (615 loc) · 18.6 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
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
package goka
import (
"context"
"fmt"
"strings"
"sync"
"sync/atomic"
"time"
"github.com/IBM/sarama"
"github.com/hashicorp/go-multierror"
"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 stopping partition processor
PPStateStopping
// PPStateStopped indicates a stopped partition processor
PPStateStopped
)
// 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 visit struct {
key string
name string
meta interface{}
done func()
}
type commitCallback func(msg *message, meta string)
type ContextWrapper func(ctx Context) Context
// 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 *message
inputTopics []string
visitInput chan *visit
visitCallbacks map[string]ProcessCallback
runnerGroup *multierr.ErrGroup
cancelRunnerGroup func()
runMode PPRunMode
consumer sarama.Consumer
tmgr TopicManager
mStats sync.RWMutex
stats *PartitionProcStats
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)
visitCallbacks = 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))
for _, v := range graph.visitors {
visitCallbacks[v.(*visitor).name] = v.(*visitor).cb
}
partProc := &PartitionProcessor{
log: log,
opts: opts,
partition: partition,
state: NewSignal(PPStateIdle, PPStateRecovering, PPStateRunning, PPStateStopping, PPStateStopped).SetState(PPStateIdle),
callbacks: callbacks,
lookups: lookupTables,
consumer: consumer,
producer: producer,
tmgr: tmgr,
joins: make(map[string]*PartitionTable),
input: make(chan *message, opts.partitionChannelSize),
inputTopics: topicList,
visitInput: make(chan *visit, defaultPPVisitChannelSize),
visitCallbacks: visitCallbacks,
graph: graph,
stats: newPartitionProcStats(topicList, outputList),
commit: commit,
runMode: runMode,
}
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
}
// 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)
}
// 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
// This method takes two contexts, as it does two distinct phases:
// - setting up the partition (loading table, joins etc.), after which it returns.
// This needs a separate context to allow terminatin the setup phase
// - starting the message-processing-loop of the actual processor. This will keep running
// after `Start` returns, so it uses the second context.
func (pp *PartitionProcessor) Start(setupCtx, ctx context.Context) error {
if state := pp.state.State(); state != PPStateIdle {
return fmt.Errorf("partitionprocessor is not idle (but %v), cannot start", state)
}
// runner context
ctx, pp.cancelRunnerGroup = context.WithCancel(ctx)
var runnerCtx context.Context
pp.runnerGroup, runnerCtx = multierr.NewErrGroup(ctx)
setupErrg, setupCtx := multierr.NewErrGroup(setupCtx)
pp.state.SetState(PPStateRecovering)
defer pp.state.SetState(PPStateRunning)
if pp.table != nil {
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(defaultBackoffStep, defaultBackoffMax),
time.Minute,
)
pp.joins[join.Topic()] = table
setupErrg.Go(func() error {
return table.SetupAndRecover(setupCtx, false)
})
}
// here we wait for our table and the joins to recover
err := setupErrg.Wait().ErrorOrNil()
if err != nil {
return fmt.Errorf("Setup failed. Cannot start processor for partition %d: %v", pp.partition, err)
}
// check if one of the contexts might have been closed in the meantime
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 {
defer pp.state.SetState(PPStateStopping)
return join.CatchupForever(runnerCtx, false)
})
}
// now run the processor in a runner-group
pp.runnerGroup.Go(func() error {
defer pp.state.SetState(PPStateStopping)
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:
err = fmt.Errorf("processor has invalid run mode")
}
if err != nil {
pp.log.Debugf("Run failed with error: %v", err)
}
return err
})
return nil
}
func (pp *PartitionProcessor) stopping() (<-chan struct{}, func()) {
return pp.state.WaitForStateMinWithCleanup(PPStateStopping)
}
// 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(PPStateStopped)
close(pp.input)
close(pp.visitInput)
if pp.cancelRunnerGroup != nil {
pp.cancelRunnerGroup()
}
// wait for the runner to be done
runningErrs := multierror.Append(pp.runnerGroup.Wait().ErrorOrNil())
// close all the tables
stopErrg, _ := multierr.NewErrGroup(context.Background())
for _, join := range pp.joins {
join := join
stopErrg.Go(func() error {
return join.Close()
})
}
// close processor table, if there is one
if pp.table != nil {
stopErrg.Go(func() error {
return pp.table.Close()
})
}
// return stopping errors and running errors
return multierror.Append(stopErrg.Wait().ErrorOrNil(), runningErrs).ErrorOrNil()
}
func (pp *PartitionProcessor) run(ctx context.Context) (rerr error) {
pp.log.Debugf("starting")
defer pp.log.Debugf("stopped")
// protect the errors-collection
var mutexErr sync.Mutex
defer func() {
mutexErr.Lock()
defer mutexErr.Unlock()
rerr = multierror.Append(rerr).ErrorOrNil()
}()
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():
mutexErr.Lock()
rerr = multierror.Append(rerr,
newErrProcessing(pp.partition, fmt.Errorf("synchronous error in callback: %w", err)),
)
mutexErr.Unlock()
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 an Emit (using a producer internally) fails
asyncFailer = func(err error) {
mutexErr.Lock()
rerr = multierror.Append(rerr, newErrProcessing(pp.partition, fmt.Errorf("asynchronous error from callback: %w", err)))
mutexErr.Unlock()
closeOnce.Do(func() { close(asyncErrs) })
}
wg sync.WaitGroup
)
defer func() {
if r := recover(); r != nil {
mutexErr.Lock()
rerr = multierror.Append(rerr,
newErrProcessing(pp.partition, fmt.Errorf("panic in callback: %v\n%v", r, strings.Join(userStacktrace(), "\n"))),
)
mutexErr.Unlock()
wg.Done()
}
done := make(chan struct{})
go func() {
defer close(done)
wg.Wait()
}()
timeout := time.NewTimer(60 * time.Second)
defer timeout.Stop()
select {
case <-done:
case <-timeout.C:
pp.log.Printf("partition processor did not shutdown in time. Will stop waiting")
}
}()
updateHwmStatsTicker := time.NewTicker(statsHwmUpdateInterval)
defer updateHwmStatsTicker.Stop()
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 newErrProcessing(pp.partition, err)
}
pp.updateStats(func(stats *PartitionProcStats) {
ip := 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++
})
case <-ctx.Done():
pp.log.Debugf("exiting, context is cancelled")
return
case visit, open := <-pp.visitInput:
if !open {
return nil
}
err := pp.processVisit(ctx, &wg, visit, syncFailer, asyncFailer)
if err != nil {
return newErrProcessing(pp.partition, fmt.Errorf("Error visiting %s for %s: %v", visit.name, visit.key, err))
}
case <-asyncErrs:
pp.log.Debugf("Errors occurred asynchronously. Will exit partition processor")
return
case <-updateHwmStatsTicker.C:
pp.updateStats(pp.updateHwmStats)
}
}
}
func (pp *PartitionProcessor) updateStats(updater func(stats *PartitionProcStats)) {
pp.mStats.Lock()
defer pp.mStats.Unlock()
updater(pp.stats)
}
// updateHwmStats updates the offset lag for all input topics based on the
// highwatermarks obtained by the consumer.
func (pp *PartitionProcessor) updateHwmStats(stats *PartitionProcStats) {
hwms := pp.consumer.HighWaterMarks()
for input, inputStats := range stats.Input {
hwm := hwms[input][pp.partition]
if hwm != 0 && inputStats.LastOffset != 0 {
inputStats.OffsetLag = hwm - inputStats.LastOffset
}
}
}
func (pp *PartitionProcessor) fetchStats(ctx context.Context) *PartitionProcStats {
pp.mStats.RLock()
stats := pp.stats.clone()
pp.mStats.RUnlock()
// mutex for the local stats-clone so the
// error group below doesn't get a concurrent-map-access error
var m sync.Mutex
errg, ctx := multierr.NewErrGroup(ctx)
// fetch join table stats
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 we have processor state, get those stats
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().ErrorOrNil()
if err != nil {
pp.log.Printf("Error retrieving stats: %v", err)
}
return stats
}
func (pp *PartitionProcessor) enqueueTrackOutputStats(ctx context.Context, topic string, size int) {
pp.updateStats(func(stats *PartitionProcStats) {
stats.trackOutput(topic, size)
})
}
func (pp *PartitionProcessor) processVisit(ctx context.Context, wg *sync.WaitGroup, v *visit, syncFailer func(err error), asyncFailer func(err error)) (rerr error) {
cb, ok := pp.visitCallbacks[v.name]
// no callback registered for visit
if !ok {
return fmt.Errorf("no callback registered for visit named '%s'", v.name)
}
msgContext := &cbContext{
ctx: ctx,
graph: pp.graph,
trackOutputStats: pp.enqueueTrackOutputStats,
pviews: pp.joins,
views: pp.lookups,
commit: v.done,
wg: wg,
msg: &message{
key: v.key,
topic: v.name,
partition: pp.partition,
timestamp: time.Now(),
},
syncFailer: syncFailer,
asyncFailer: asyncFailer,
emitter: pp.producer.EmitWithHeaders,
emitterDefaultHeaders: pp.opts.producerDefaultHeaders,
table: pp.table,
}
// start context and call the ProcessorCallback cb
msgContext.start()
defer func() {
if r := recover(); r != nil {
rerr = fmt.Errorf("panic in visit: %v", r)
// mark the visit done, otherwise the processor gets stuck and the caller of `visit` never returns.
v.done()
// throw the panic upwards so the caller can handle the wg.Done
panic(r)
}
}()
// now call cb, wrap the context
cb(pp.opts.contextWrapper(msgContext), v.meta)
msgContext.finish(nil)
return
}
func (pp *PartitionProcessor) processMessage(ctx context.Context, wg *sync.WaitGroup, msg *message, 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", msg.key, msg.topic, msg.partition, err)
}
// start context and call the ProcessorCallback cb
msgContext.start()
// now call cb
cb(pp.opts.contextWrapper(msgContext), m)
msgContext.finish(nil)
return nil
}
// VisitValues iterates over all values in the table and calls the "visit"-callback for the passed name.
// Optional parameter value can be set, which will just be forwarded to the visitor-function
// the function returns after all items of the table have been added to the channel.
func (pp *PartitionProcessor) VisitValues(ctx context.Context, name string, meta interface{}, visited *int64) error {
if _, ok := pp.visitCallbacks[name]; !ok {
return fmt.Errorf("unconfigured visit callback. Did you initialize the processor with DefineGroup(..., Visit(%s, ...), ...)?", name)
}
it, err := pp.table.Iterator()
if err != nil {
return fmt.Errorf("error creating storage iterator")
}
var wg sync.WaitGroup
// drains the channel and drops out when closed.
// This is done when the processor shuts down during visit
// and makes sure the waitgroup is fully counted down.
drainUntilClose := func() {
for range pp.visitInput {
wg.Done()
}
}
// drains the input channel until there are no more items.
// does not wait for close, because the channel stays open for the next visit
drainUntilEmpty := func() {
for {
select {
case _, ok := <-pp.visitInput:
if !ok {
return
}
wg.Done()
default:
return
}
}
}
defer it.Release()
stopping, doneWaitingForStop := pp.stopping()
defer doneWaitingForStop()
for it.Next() {
// add one that we were able to be put into the queue.
// wg.Done will be called by the visit handler as commit
wg.Add(1)
select {
case <-stopping:
drainUntilClose()
wg.Done()
return ErrVisitAborted
case <-ctx.Done():
drainUntilEmpty()
wg.Done()
return ctx.Err()
// enqueue the visit
case pp.visitInput <- &visit{
key: string(it.Key()),
name: name,
meta: meta,
done: func() {
atomic.AddInt64(visited, 1)
wg.Done()
},
}:
}
}
// wait for all visits. We have to wrap it into an extra goroutine using a closing
// channel, since on an explicit stop, the waitgroup will never finish, so we would wait
// forever here.
wgDone := make(chan struct{})
go func() {
defer close(wgDone)
wg.Wait()
}()
select {
case <-stopping:
drainUntilClose()
return ErrVisitAborted
case <-ctx.Done():
drainUntilEmpty()
return ctx.Err()
case <-wgDone:
}
return nil
}