-
Notifications
You must be signed in to change notification settings - Fork 173
/
processor.go
1067 lines (917 loc) · 30.2 KB
/
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
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
package goka
import (
"context"
"errors"
"fmt"
"strings"
"sync"
"time"
"github.com/IBM/sarama"
"github.com/hashicorp/go-multierror"
"github.com/lovoo/goka/multierr"
"github.com/lovoo/goka/storage"
)
const (
// ProcStateIdle indicates an idling partition processor (not started yet)
ProcStateIdle State = iota
// ProcStateStarting indicates a starting partition processor, i.e. before rebalance
ProcStateStarting
// ProcStateSetup indicates a partition processor during setup of a rebalance round
ProcStateSetup
// ProcStateRunning indicates a running partition processor
ProcStateRunning
// ProcStateStopping indicates a stopping partition processor
ProcStateStopping
// ProcStateStopped indicates a stopped partition processor
ProcStateStopped
)
// ProcessCallback function is called for every message received by the
// processor.
type ProcessCallback func(ctx Context, msg interface{})
// Processor is a set of stateful callback functions that, on the arrival of
// messages, modify the content of a table (the group table) and emit messages into other
// topics. Messages as well as rows in the group table are key-value pairs.
// A group is composed by multiple processor instances.
type Processor struct {
opts *poptions
log logger
brokers []string
// hook used to be notified whenever the processor has rebalanced to a new assignment
rebalanceCallback RebalanceCallback
// rwmutex protecting read/write of partitions and lookuptables.
mTables sync.RWMutex
// Partition processors
partitions map[int32]*PartitionProcessor
// lookup tables
lookupTables map[string]*View
partitionCount int
graph *GroupGraph
saramaConsumer sarama.Consumer
producer Producer
tmgr TopicManager
state *Signal
errMux sync.Mutex
err error
done chan struct{}
cancel context.CancelFunc
}
// NewProcessor creates a processor instance in a group given the address of
// Kafka brokers, the consumer group name, a list of subscriptions (topics,
// codecs, and callbacks), and series of options.
func NewProcessor(brokers []string, gg *GroupGraph, options ...ProcessorOption) (*Processor, error) {
options = append(
// default options comes first
[]ProcessorOption{
WithClientID(fmt.Sprintf("goka-processor-%s", gg.Group())),
WithUpdateCallback(DefaultUpdate),
WithPartitionChannelSize(defaultPartitionChannelSize),
WithStorageBuilder(storage.DefaultBuilder(DefaultProcessorStoragePath(gg.Group()))),
WithRebalanceCallback(DefaultRebalance),
},
// user-defined options (may overwrite default ones)
options...,
)
if err := gg.Validate(); err != nil {
return nil, err
}
opts := new(poptions)
err := opts.applyOptions(gg, options...)
if err != nil {
return nil, fmt.Errorf(errApplyOptions, err)
}
npar, err := prepareTopics(brokers, gg, opts)
if err != nil {
return nil, err
}
// create views
lookupTables := make(map[string]*View)
for _, t := range gg.LookupTables() {
view, err := NewView(brokers, Table(t.Topic()), t.Codec(),
WithViewLogger(opts.log),
WithViewHasher(opts.hasher),
WithViewClientID(opts.clientID),
WithViewTopicManagerBuilder(opts.builders.topicmgr),
WithViewStorageBuilder(opts.builders.storage),
WithViewConsumerSaramaBuilder(opts.builders.consumerSarama),
)
if err != nil {
return nil, fmt.Errorf("error creating view: %v", err)
}
lookupTables[t.Topic()] = view
}
// combine things together
processor := &Processor{
opts: opts,
log: opts.log.Prefix(fmt.Sprintf("Processor %s", gg.Group())),
brokers: brokers,
rebalanceCallback: opts.rebalanceCallback,
partitions: make(map[int32]*PartitionProcessor),
partitionCount: npar,
lookupTables: lookupTables,
graph: gg,
state: NewSignal(ProcStateIdle, ProcStateStarting, ProcStateSetup, ProcStateRunning, ProcStateStopping, ProcStateStopped).SetState(ProcStateIdle),
done: make(chan struct{}),
}
return processor, nil
}
// Graph returns the group graph of the processor.
func (g *Processor) Graph() *GroupGraph {
return g.graph
}
// isStateless returns whether the processor is a stateless one.
func (g *Processor) isStateless() bool {
return g.graph.GroupTable() == nil
}
///////////////////////////////////////////////////////////////////////////////
// value getter
///////////////////////////////////////////////////////////////////////////////
// Get returns a read-only copy of a value from the group table if the
// respective partition is owned by the processor instace.
// Get can be called by multiple goroutines concurrently.
// Get can be only used with stateful processors (ie, when group table is
// enabled) and after Recovered returns true.
func (g *Processor) Get(key string) (interface{}, error) {
if g.isStateless() {
return nil, fmt.Errorf("can't get a value from stateless processor")
}
// find partition where key is located
s, err := g.find(key)
if err != nil {
return nil, err
}
// get key and return
val, err := s.Get(key)
if err != nil {
return nil, fmt.Errorf("error getting %s: %v", key, err)
} else if val == nil {
// if the key does not exist the return value is nil
return nil, nil
}
// since we don't know what the codec does, make copy of the object
data := make([]byte, len(val))
copy(data, val)
value, err := g.graph.GroupTable().Codec().Decode(data)
if err != nil {
return nil, fmt.Errorf("error decoding %s: %v", key, err)
}
return value, nil
}
func (g *Processor) find(key string) (storage.Storage, error) {
p, err := g.hash(key)
if err != nil {
return nil, err
}
pproc, ok := g.getPartProc(p)
if !ok {
return nil, fmt.Errorf("this processor does not contain partition %v", p)
}
return pproc.table.st, nil
}
func (g *Processor) getPartProc(partition int32) (*PartitionProcessor, bool) {
g.mTables.RLock()
defer g.mTables.RUnlock()
pproc, ok := g.partitions[partition]
return pproc, ok
}
func (g *Processor) setPartProc(partition int32, pproc *PartitionProcessor) {
g.mTables.Lock()
defer g.mTables.Unlock()
if pproc == nil {
delete(g.partitions, partition)
} else {
g.partitions[partition] = pproc
}
}
func (g *Processor) hash(key string) (int32, error) {
// create a new hasher every time. Alternative would be to store the hash in
// view and every time reset the hasher (ie, hasher.Reset()). But that would
// also require us to protect the access of the hasher with a mutex.
hasher := g.opts.hasher()
_, err := hasher.Write([]byte(key))
if err != nil {
return -1, err
}
hash := int32(hasher.Sum32())
if hash < 0 {
hash = -hash
}
if g.partitionCount == 0 {
return 0, errors.New("can't hash with 0 partitions")
}
return hash % int32(g.partitionCount), nil
}
// Run starts the processor using passed context.
// The processor stops in case of errors or if the context is cancelled
func (g *Processor) Run(ctx context.Context) (rerr error) {
g.log.Debugf("starting")
defer g.log.Debugf("stopped")
// check if the processor was done already
select {
case <-g.done:
return fmt.Errorf("error running processor: it was already run and terminated. Run can only be called once")
default:
}
// create errorgroup
ctx, g.cancel = context.WithCancel(ctx)
errg, ctx := multierr.NewErrGroup(ctx)
defer close(g.done)
defer g.cancel()
// set a starting state. From this point on we know that there's a cancel and a valid context set
// in the processor which we can use for waiting
g.state.SetState(ProcStateStarting)
// collect all errors before leaving
var errs *multierror.Error
defer func() {
g.errMux.Lock()
defer g.errMux.Unlock()
g.err = multierror.Append(errs, rerr).ErrorOrNil()
rerr = g.err
}()
var err error
g.saramaConsumer, err = g.opts.builders.consumerSarama(g.brokers, g.opts.clientID)
if err != nil {
return fmt.Errorf("Error creating consumer for brokers [%s]: %v", strings.Join(g.brokers, ","), err)
}
// close sarama consume after we're done
defer func() {
if err := g.saramaConsumer.Close(); err != nil {
errs = multierror.Append(errs, fmt.Errorf("error closing sarama consumer: %w", err))
}
}()
g.tmgr, err = g.opts.builders.topicmgr(g.brokers)
if err != nil {
return fmt.Errorf("Error creating topic manager for brokers [%s]: %v", strings.Join(g.brokers, ","), err)
}
// create kafka producer
g.log.Debugf("creating producer")
producer, err := g.opts.builders.producer(g.brokers, g.opts.clientID, g.opts.hasher)
if err != nil {
return fmt.Errorf(errBuildProducer, err)
}
g.producer = producer
defer func() {
g.log.Debugf("closing producer")
defer g.log.Debugf("producer ... closed")
if err := g.producer.Close(); err != nil {
errs = multierror.Append(errs, fmt.Errorf("error closing producer: %w", err))
}
}()
// start all lookup tables
g.mTables.RLock()
for topic, view := range g.lookupTables {
g.log.Debugf("Starting lookup table for %s", topic)
// make local copies
topic, view := topic, view
errg.Go(func() error {
if err := view.Run(ctx); err != nil {
return fmt.Errorf("error running lookup table %s: %v", topic, err)
}
return nil
})
}
g.mTables.RUnlock()
if err := g.waitForStartupTables(ctx); err != nil {
return fmt.Errorf("error waiting for start up tables: %w", err)
}
// run the main rebalance-consume-loop
errg.Go(func() error {
return g.rebalanceLoop(ctx)
})
return errg.Wait().ErrorOrNil()
}
func (g *Processor) rebalanceLoop(ctx context.Context) (rerr error) {
// create kafka consumer
consumerGroup, err := g.opts.builders.consumerGroup(g.brokers, string(g.graph.Group()), g.opts.clientID)
if err != nil {
return fmt.Errorf(errBuildConsumer, err)
}
var topics []string
for _, e := range g.graph.InputStreams() {
topics = append(topics, e.Topic())
}
if g.graph.LoopStream() != nil {
topics = append(topics, g.graph.LoopStream().Topic())
}
var errs *multierror.Error
defer func() {
rerr = multierror.Append(errs, rerr).ErrorOrNil()
}()
defer func() {
g.log.Debugf("closing consumer group")
closeErr := consumerGroup.Close()
if closeErr != nil {
g.log.Printf("Error closing consumer group: %v", closeErr)
errs = multierror.Append(
errs,
fmt.Errorf("Error closing consumer group: %w", closeErr),
)
}
g.log.Debugf("closing consumer group ... done")
}()
for {
sessionCtx, sessionCtxCancel := context.WithCancel(ctx)
go func() {
g.handleSessionErrors(ctx, sessionCtx, sessionCtxCancel, consumerGroup)
}()
err := consumerGroup.Consume(ctx, topics, g)
sessionCtxCancel()
// error consuming, no way to recover so we have to kill the processor
if err != nil {
return fmt.Errorf("error consuming from group consumer: %w", err)
}
select {
case <-time.After(5 * time.Second):
g.log.Printf("Consumer group returned, Rebalancing.")
case <-ctx.Done():
g.log.Printf("Consumer group cancelled. Stopping")
return nil
}
}
}
func (g *Processor) handleSessionErrors(ctx, sessionCtx context.Context, sessionCtxCancel context.CancelFunc, consumerGroup sarama.ConsumerGroup) {
errs := consumerGroup.Errors()
for {
select {
case <-ctx.Done():
return
case <-sessionCtx.Done():
return
case err, ok := <-errs:
if !ok {
return
}
if err != nil {
g.log.Printf("error during execution of consumer group: %v", err)
}
var (
errProc *errProcessing
errSetup *errSetup
)
if errors.As(err, &errProc) {
g.log.Debugf("error processing message (non-transient), shutting down processor: %v", err)
sessionCtxCancel()
}
if errors.As(err, &errSetup) {
g.log.Debugf("setup error (non-transient), shutting down processor: %v", err)
sessionCtxCancel()
}
}
}
}
// waits for all tables that are supposed to start up
func (g *Processor) waitForStartupTables(ctx context.Context) error {
errg, startupCtx := multierr.NewErrGroup(ctx)
var (
waitMap = make(map[string]struct{})
mWaitMap sync.Mutex
)
// we'll wait for all lookup tables to have recovered.
// For this we're looping through all tables and start
// a new goroutine that terminates when the table is done (or ctx is closed).
// The extra code adds and removes the table to a map used for logging
// the items that the processor is still waiting to recover before ready to go.
g.mTables.RLock()
for _, view := range g.lookupTables {
view := view
errg.Go(func() error {
name := fmt.Sprintf("lookup-table-%s", view.topic)
mWaitMap.Lock()
waitMap[name] = struct{}{}
mWaitMap.Unlock()
defer func() {
mWaitMap.Lock()
defer mWaitMap.Unlock()
delete(waitMap, name)
}()
select {
case <-startupCtx.Done():
case <-view.WaitRunning():
}
return nil
})
}
g.mTables.RUnlock()
// If we recover ahead, we'll also start all partition processors once in recover-only-mode
// and do the same boilerplate to keep the waitmap up to date.
if g.opts.recoverAhead {
partitions, err := g.findStatefulPartitions()
if err != nil {
return fmt.Errorf("error finding dependent partitions: %w", err)
}
for _, part := range partitions {
part := part
pproc, err := g.createPartitionProcessor(ctx, part, runModeRecoverOnly, func(msg *message, meta string) {
panic("a partition processor in recover-only-mode never commits a message")
})
if err != nil {
return fmt.Errorf("Error creating partition processor for recover-ahead %s/%d: %v", g.Graph().Group(), part, err)
}
errg.Go(func() error {
name := fmt.Sprintf("partition-processor-%d", part)
mWaitMap.Lock()
waitMap[name] = struct{}{}
mWaitMap.Unlock()
defer func() {
mWaitMap.Lock()
defer mWaitMap.Unlock()
delete(waitMap, name)
}()
g.setPartProc(part, pproc)
defer g.setPartProc(part, nil)
err := pproc.Start(ctx, ctx)
if err != nil {
return err
}
return pproc.Stop()
})
}
}
var (
start = time.Now()
logTicker = time.NewTicker(1 * time.Minute)
)
// Now run through
defer logTicker.Stop()
errgWaiter := errg.WaitChan()
for {
select {
// the context has closed, no point in waiting
case <-ctx.Done():
g.log.Debugf("Stopping to wait for views to get up, context closed")
return fmt.Errorf("context closed while waiting for startup tables to become ready")
// the error group is done, which means
// * err==nil --> it's done
// * err!=nil --> it failed, let's return the error
case errs := <-errgWaiter:
err := errs.ErrorOrNil()
if err == nil {
g.log.Debugf("View catchup finished")
}
return err
// log the things we're still waiting for
case <-logTicker.C:
var tablesWaiting []string
mWaitMap.Lock()
for table := range waitMap {
tablesWaiting = append(tablesWaiting, table)
}
mWaitMap.Unlock()
g.log.Printf("Waiting for [%s] to start up since %.2f minutes",
strings.Join(tablesWaiting, ", "),
time.Since(start).Minutes())
}
}
}
// find partitions that will any type of local state for this processor.
// This includes joins and the group-table. If neither are present, it returns an empty list, because
// it means that the processor is stateless and has only streaming-input.
// It returns the list of partitions as an error, or empty if there are no such partitions.
//
// Supports to pass optional map of excluded partitions if the function is used to determine partitions
// that are not part of the current assignment
func (g *Processor) findStatefulPartitions() ([]int32, error) {
var (
err error
allPartitions []int32
)
for _, edge := range chainEdges(g.graph.groupTable, g.graph.inputTables) {
allPartitions, err = g.tmgr.Partitions(edge.Topic())
if err != nil && err != errTopicNotFound {
return nil, err
}
if len(allPartitions) > 0 {
break
}
}
return allPartitions, err
}
// Recovered returns whether the processor is running, i.e. if the processor
// has recovered all lookups/joins/tables and is running
func (g *Processor) Recovered() bool {
return g.state.IsState(ProcStateRunning)
}
// StateReader returns a read only interface of the processors state.
func (g *Processor) StateReader() StateReader {
return g.state
}
func (g *Processor) assignmentFromSession(session sarama.ConsumerGroupSession) (Assignment, error) {
var assignment Assignment
// get the partitions this processor is assigned to.
// We use that loop to verify copartitioning and fail otherwise
for _, claim := range session.Claims() {
// for first claim, generate the assignment
if assignment == nil {
assignment = Assignment{}
for _, part := range claim {
assignment[part] = sarama.OffsetNewest
}
} else {
// for all others, verify the assignment is the same
// first check length
if len(claim) != len(assignment) {
return nil, fmt.Errorf("session claims are not copartitioned: %#v", session.Claims())
}
// then check the partitions are exactly the same
for _, part := range claim {
if _, exists := assignment[part]; !exists {
return nil, fmt.Errorf("session claims are not copartitioned: %#v", session.Claims())
}
}
}
}
return assignment, nil
}
// Setup is run at the beginning of a new session, before ConsumeClaim.
func (g *Processor) Setup(session sarama.ConsumerGroupSession) error {
g.state.SetState(ProcStateSetup)
defer g.state.SetState(ProcStateRunning)
g.log.Printf("setup generation %d, claims=%#v", session.GenerationID(), session.Claims())
defer g.log.Debugf("setup generation %d ... done", session.GenerationID())
if err := g.createAssignedPartitions(session); err != nil {
return err
}
if err := g.createHotStandbyPartitions(session); err != nil {
return err
}
// setup all processors
setupErrg, setupCtx := multierr.NewErrGroup(session.Context())
g.mTables.RLock()
for partID, partition := range g.partitions {
pproc := partition
setupErrg.Go(func() error {
// the partition processors need two contexts:
// setupCtx --> for this setup, which we'll wait for
// the runner ctx, which is active during a session
err := pproc.Start(setupCtx, session.Context())
if err != nil {
return newErrSetup(partID, err)
}
return nil
})
}
g.mTables.RUnlock()
return setupErrg.Wait().ErrorOrNil()
}
func (g *Processor) createAssignedPartitions(session sarama.ConsumerGroupSession) error {
assignment, err := g.assignmentFromSession(session)
if err != nil {
return fmt.Errorf("Error verifying assignment from session: %v", err)
}
if g.rebalanceCallback != nil {
g.rebalanceCallback(assignment)
}
// no partitions configured, just print a log but continue
// in case we have configured standby, we should still start the standby-processors
if len(assignment) == 0 {
g.log.Printf("No partitions assigned. Claims were: %#v. Will probably sleep this generation", session.Claims())
}
// create partition views for all partitions
for partition := range assignment {
// create partition processor for our partition
pproc, err := g.createPartitionProcessor(session.Context(), partition, runModeActive,
createMessageCommitter(session),
)
if err != nil {
return fmt.Errorf("Error creating partition processor for %s/%d: %v", g.Graph().Group(), partition, err)
}
g.setPartProc(partition, pproc)
}
return nil
}
// if hot standby is configured, we find the partitions that are missing
// from the current assignment, and create those processors in standby mode
func (g *Processor) createHotStandbyPartitions(session sarama.ConsumerGroupSession) error {
if !g.opts.hotStandby {
return nil
}
allPartitions, err := g.findStatefulPartitions()
if err != nil {
return fmt.Errorf("Error finding partitions for standby")
}
for _, standby := range allPartitions {
// if the partition already exists, it means it is part of the assignment, so we don't
// need to keep it on hot-standby and can ignore it.
// we check if the partition processor exists and if it does, ignore it
// since it's part of the assignment
if _, exists := g.getPartProc(standby); exists {
continue
}
// otherwise, let's start the partition processor in passive mode
pproc, err := g.createPartitionProcessor(session.Context(), standby, runModePassive,
func(msg *message, metadata string) {
panic("a passive partition processor should never receive input messages, thus never commit any messages")
})
if err != nil {
return fmt.Errorf("Error creating partition processor for %s/%d: %v", g.Graph().Group(), standby, err)
}
g.setPartProc(standby, pproc)
}
return nil
}
// Cleanup is run at the end of a session, once all ConsumeClaim goroutines have exited
// but before the offsets are committed for the very last time.
func (g *Processor) Cleanup(session sarama.ConsumerGroupSession) error {
g.log.Debugf("Cleaning up for %d", session.GenerationID())
defer g.log.Debugf("Cleaning up for %d ... done", session.GenerationID())
g.state.SetState(ProcStateStopping)
defer g.state.SetState(ProcStateStopped)
errg, _ := multierr.NewErrGroup(session.Context())
g.mTables.RLock()
for part, partition := range g.partitions {
partID, pproc := part, partition
errg.Go(func() error {
err := pproc.Stop()
if err != nil {
g.log.Printf("Error running/stopping partition processor %d: %v", partID, err)
return err
}
return nil
})
}
g.mTables.RUnlock()
err := errg.Wait().ErrorOrNil()
// reset the partitions after the session ends
g.mTables.Lock()
defer g.mTables.Unlock()
g.partitions = make(map[int32]*PartitionProcessor)
return err
}
// WaitForReady waits until the processor is ready to consume messages
// (or is actually consuming messages)
// i.e., it is done catching up all partition tables, joins and lookup tables
func (g *Processor) WaitForReady() error {
return g.waitForReady(context.Background())
}
// WaitForReadyContext is context aware option of WaitForReady.
// It either waits until the processor is ready or until context is canceled.
// If the return was caused by context it will return context reported error.
func (g *Processor) WaitForReadyContext(ctx context.Context) error {
return g.waitForReady(ctx)
}
func (g *Processor) waitForReady(ctx context.Context) error {
// wait for the processor to be started (or stopped)
select {
case <-g.state.WaitForStateMin(ProcStateStarting):
case <-g.done:
return nil
case <-ctx.Done():
return ctx.Err()
}
// wait that the processor is actually running
select {
case <-g.state.WaitForState(ProcStateRunning):
case <-g.done:
case <-ctx.Done():
return ctx.Err()
}
// wait for all PartitionProcessors to be running
// copy them first with the mutex so we don't run into a deadlock
g.mTables.RLock()
parts := make([]*PartitionProcessor, 0, len(g.partitions))
for _, part := range g.partitions {
parts = append(parts, part)
}
g.mTables.RUnlock()
for _, part := range parts {
select {
case <-part.state.WaitForState(PPStateRunning):
case <-g.done:
case <-ctx.Done():
return ctx.Err()
}
}
return nil
}
// ConsumeClaim must start a consumer loop of ConsumerGroupClaim's Messages().
// Once the Messages() channel is closed, the Handler must finish its processing
// loop and exit.
func (g *Processor) ConsumeClaim(session sarama.ConsumerGroupSession, claim sarama.ConsumerGroupClaim) error {
g.log.Debugf("ConsumeClaim for topic/partition %s/%d, initialOffset=%d", claim.Topic(), claim.Partition(), claim.InitialOffset())
defer g.log.Debugf("ConsumeClaim done for topic/partition %s/%d", claim.Topic(), claim.Partition())
part, has := g.getPartProc(claim.Partition())
if !has {
return fmt.Errorf("no partition (%d) to handle input in topic %s", claim.Partition(), claim.Topic())
}
messages := claim.Messages()
stopping, doneWaitingForStop := part.stopping()
defer doneWaitingForStop()
for {
select {
case msg, ok := <-messages:
if !ok {
return nil
}
select {
case part.input <- &message{
key: string(msg.Key),
topic: msg.Topic,
offset: msg.Offset,
partition: msg.Partition,
timestamp: msg.Timestamp,
headers: msg.Headers,
value: msg.Value,
}:
case <-stopping:
return nil
case <-session.Context().Done():
return nil
}
case <-stopping:
return nil
case <-session.Context().Done():
return nil
}
}
}
// Stats returns the aggregated stats for the processor including all partitions, tables, lookups and joins
func (g *Processor) Stats() *ProcessorStats {
return g.StatsWithContext(context.Background())
}
// StatsWithContext returns stats for the processor, see #Processor.Stats()
func (g *Processor) StatsWithContext(ctx context.Context) *ProcessorStats {
var (
m sync.Mutex
stats *ProcessorStats
)
errg, ctx := multierr.NewErrGroup(ctx)
// get partition-processor stats
g.mTables.RLock()
stats = newProcessorStats(len(g.partitions))
for partID, proc := range g.partitions {
partID, proc := partID, proc
errg.Go(func() error {
partStats := proc.fetchStats(ctx)
m.Lock()
defer m.Unlock()
if partStats != nil {
stats.Group[partID] = partStats
}
return nil
})
}
// get the lookup table stats
for topic, view := range g.lookupTables {
topic, view := topic, view
errg.Go(func() error {
m.Lock()
defer m.Unlock()
viewStats := view.Stats(ctx)
if viewStats != nil {
stats.Lookup[topic] = viewStats
}
return nil
})
}
g.mTables.RUnlock()
err := errg.Wait().ErrorOrNil()
if err != nil {
g.log.Printf("Error retrieving stats: %v", err)
}
return stats
}
// creates the partition that is responsible for the group processor's table
func (g *Processor) createPartitionProcessor(ctx context.Context, partition int32, runMode PPRunMode, commit commitCallback) (*PartitionProcessor, error) {
g.log.Debugf("Creating partition processor for partition %d", partition)
if _, has := g.getPartProc(partition); has {
return nil, fmt.Errorf("processor [%s]: partition %d already exists", g.graph.Group(), partition)
}
backoff, err := g.opts.builders.backoff()
if err != nil {
return nil, fmt.Errorf("processor [%s]: could not build backoff handler: %v", g.graph.Group(), err)
}
return newPartitionProcessor(partition,
g.graph,
commit,
g.log,
g.opts,
runMode,
g.lookupTables,
g.saramaConsumer,
g.producer,
g.tmgr,
backoff,
g.opts.backoffResetTime), nil
}
// Stop stops the processor.
// This is semantically equivalent of closing the Context
// that was passed to Processor.Run(..).
// This method will return immediately, errors during running
// will be returned from Processor.Run(..)
func (g *Processor) Stop() {
g.cancel()
}
// Done returns a channel that is closed when the processor is stopped.
func (g *Processor) Done() <-chan struct{} {
return g.done
}
// Error returns the error that caused the processor to stop.
func (g *Processor) Error() error {
g.errMux.Lock()
defer g.errMux.Unlock()
return g.err
}
// VisitAllWithStats visits all keys in parallel by passing the visit request
// to all partitions.
// The optional argument "meta" will be forwarded to the visit-function of each key of the table.
// The function returns when
// * all values have been visited or
// * the context is cancelled or
// * the processor rebalances/shuts down
// Return parameters:
// * number of visited items
// * error
func (g *Processor) VisitAllWithStats(ctx context.Context, name string, meta interface{}) (int64, error) {
g.mTables.RLock()
if g.isStateless() {
return 0, fmt.Errorf("cannot visit values in stateless processor")
}
errg, visitCtx := multierr.NewErrGroup(ctx)
var visited int64
for _, part := range g.partitions {
// we'll only do the visit for active mode partitions
// because visit essentially provides write access, which passive partitions don't have
if part.runMode != runModeActive {
continue
}
part := part
errg.Go(func() error {
return part.VisitValues(visitCtx, name, meta, &visited)
})
}
g.mTables.RUnlock()
// wait for the visitors
err := errg.Wait().ErrorOrNil()
return visited, err
}
// VisitAll visits all values from the processor table.
func (g *Processor) VisitAll(ctx context.Context, name string, meta interface{}) error {
_, err := g.VisitAllWithStats(ctx, name, meta)
return err
}
func prepareTopics(brokers []string, gg *GroupGraph, opts *poptions) (npar int, err error) {
// create topic manager
tm, err := opts.builders.topicmgr(brokers)
if err != nil {
return 0, fmt.Errorf("Error creating topic manager: %v", err)
}
defer func() {
e := tm.Close()