-
Notifications
You must be signed in to change notification settings - Fork 107
/
partition.go
1864 lines (1650 loc) · 55.1 KB
/
partition.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 server
import (
"context"
"fmt"
"math/rand"
"path/filepath"
"strconv"
"sync"
"time"
"github.com/Workiva/go-datastructures/queue"
client "github.com/liftbridge-io/liftbridge-api/go"
"github.com/nats-io/nats.go"
"github.com/pkg/errors"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
"github.com/liftbridge-io/liftbridge/server/commitlog"
encryption "github.com/liftbridge-io/liftbridge/server/encryption"
proto "github.com/liftbridge-io/liftbridge/server/protocol"
)
// recvChannelSize specifies the size of the channel that feeds the leader
// message processing loop.
const recvChannelSize = 64 * 1024
// timestamp returns the current time in Unix nanoseconds. This function exists
// for mocking purposes.
var timestamp = func() int64 { return time.Now().UnixNano() }
// subscription tracks state for a partition subscription.
type subscription struct {
mu sync.Mutex
closed chan struct{}
msgs chan *client.Message
errors chan *status.Status
}
func (s *subscription) Close() {
s.mu.Lock()
defer s.mu.Unlock()
select {
case <-s.closed:
return
default:
}
close(s.closed)
}
func (s *subscription) Messages() <-chan *client.Message {
return s.msgs
}
func (s *subscription) Errors() <-chan *status.Status {
return s.errors
}
func (s *subscription) Closed() <-chan struct{} {
return s.closed
}
// replica tracks the latest log offset for a particular partition replica.
type replica struct {
mu sync.RWMutex
offset int64
}
// updateLatestOffset sets the replica's latest log offset if the given offset
// is greater than the current offset. It returns a bool indicating if the
// offset was updated or not.
func (r *replica) updateLatestOffset(offset int64) (updated bool) {
r.mu.Lock()
if offset > r.offset {
r.offset = offset
updated = true
}
r.mu.Unlock()
return
}
// getLatestOffset returns the replica's latest log offset.
func (r *replica) getLatestOffset() int64 {
r.mu.RLock()
defer r.mu.RUnlock()
return r.offset
}
// EventTimestamps contains the first and latest times when an event has
// occurred.
type EventTimestamps struct {
firstTime time.Time // Time when the first event occurred.
latestTime time.Time // Time when the latest event occurred.
}
// update should be called when an event has occurred. It updates the first and
// latest timestamps.
func (e *EventTimestamps) update() {
timestamp := time.Now()
if e.firstTime.IsZero() {
e.firstTime = timestamp
}
e.latestTime = timestamp
}
// groupMember tracks state for a consumer group member.
type groupMember struct {
consumerID string
groupEpoch uint64
sub *subscription
}
// partition represents a replicated message stream partition backed by a
// durable commit log. A partition is attached to a NATS subject and stores
// messages on that subject in a file-backed log. A partition has a set of
// replicas assigned to it, which are the brokers responsible for replicating
// the partition. The ISR, or in-sync replicas set, is the set of replicas
// which are currently caught up with the partition leader's log. If a replica
// falls behind, it will be removed from the ISR. Followers replicate the
// leader's log by fetching messages from it. All partition access should go
// through exported methods.
type partition struct {
mu sync.RWMutex
closeMu sync.Mutex
sub *nats.Subscription // Subscription to partition NATS subject
leaderReplSub *nats.Subscription // Subscription for replication requests from followers
leaderOffsetSub *nats.Subscription // Subscription for leader epoch offset requests from followers
log commitlog.CommitLog
srv *Server
isLeading bool
isFollowing bool
isClosed bool
replicas map[string]struct{}
isr map[string]*replica
minISR int
replicators map[string]*replicator
commitQueue *queue.Queue
commitCheck chan struct{}
recovered bool
stopFollower chan struct{}
stopLeader chan struct{}
notify chan struct{}
belowMinISR bool
pause bool // Pause replication on the leader (for unit testing)
shutdown sync.WaitGroup
paused bool
autoPauseTime time.Duration
autoPauseDisableIfSubscribers bool
subscriberCount int64
messagesReceivedTimestamps EventTimestamps // First and latest time a message was received on this partition
pauseTimestamps EventTimestamps // First and latest time this partition was paused or resumed
readonlyTimestamps EventTimestamps // First and latest time this partition had its read-only status changed
encryptionHandler encryption.Codec
consumersMu sync.Mutex
consumers map[string]*groupMember // Maps consumer groups to consumers
*proto.Partition
}
// newPartition creates a new stream partition. If the partition is recovered,
// it should not be started until the recovery process has completed to avoid
// starting it in an intermediate state. This call will initialize or recover
// the partition's backing commit log or return an error if it fails to do so.
//
// A partitioned stream maps to separate NATS subjects: subject, subject.1,
// subject.2, etc.
func (s *Server) newPartition(protoPartition *proto.Partition, recovered bool, config *proto.StreamConfig) (*partition, error) {
streamsConfig := &StreamsConfig{
SegmentMaxBytes: s.config.Streams.SegmentMaxBytes,
SegmentMaxAge: s.config.Streams.SegmentMaxAge,
RetentionMaxBytes: s.config.Streams.RetentionMaxBytes,
RetentionMaxMessages: s.config.Streams.RetentionMaxMessages,
RetentionMaxAge: s.config.Streams.RetentionMaxAge,
CleanerInterval: s.config.Streams.CleanerInterval,
Compact: s.config.Streams.Compact,
CompactMaxGoroutines: s.config.Streams.CompactMaxGoroutines,
AutoPauseTime: s.config.Streams.AutoPauseTime,
AutoPauseDisableIfSubscribers: s.config.Streams.AutoPauseDisableIfSubscribers,
MinISR: s.config.Clustering.MinISR,
Encryption: s.config.Streams.Encryption,
}
streamsConfig.ApplyOverrides(config)
var (
file = filepath.Join(s.config.DataDir, "streams", protoPartition.Stream,
strconv.FormatInt(int64(protoPartition.Id), 10))
name = fmt.Sprintf("[subject=%s, stream=%s, partition=%d]",
protoPartition.Subject, protoPartition.Stream, protoPartition.Id)
log, err = commitlog.New(commitlog.Options{
Name: name,
Path: file,
MaxSegmentBytes: streamsConfig.SegmentMaxBytes,
MaxSegmentAge: streamsConfig.SegmentMaxAge,
MaxLogBytes: streamsConfig.RetentionMaxBytes,
MaxLogMessages: streamsConfig.RetentionMaxMessages,
MaxLogAge: streamsConfig.RetentionMaxAge,
CleanerInterval: streamsConfig.CleanerInterval,
Compact: streamsConfig.Compact,
CompactMaxGoroutines: streamsConfig.CompactMaxGoroutines,
Logger: s.logger,
ConcurrencyControl: streamsConfig.ConcurrencyControl,
})
)
if err != nil {
return nil, errors.Wrap(err, "failed to create commit log")
}
replicas := make(map[string]struct{}, len(protoPartition.Replicas))
for _, replica := range protoPartition.Replicas {
replicas[replica] = struct{}{}
}
isr := make(map[string]*replica, len(protoPartition.Isr))
for _, rep := range protoPartition.Isr {
offset := int64(-1)
// For this server, initialize the replica offset to the newest offset.
if rep == s.config.Clustering.ServerID {
offset = log.NewestOffset()
}
isr[rep] = &replica{offset: offset}
}
st := &partition{
Partition: protoPartition,
log: log,
srv: s,
replicas: replicas,
isr: isr,
minISR: streamsConfig.MinISR,
commitCheck: make(chan struct{}, len(protoPartition.Replicas)),
notify: make(chan struct{}, 1),
recovered: recovered,
autoPauseTime: streamsConfig.AutoPauseTime,
autoPauseDisableIfSubscribers: streamsConfig.AutoPauseDisableIfSubscribers,
consumers: make(map[string]*groupMember),
}
if streamsConfig.Encryption {
// Init handler for Encryption-at-Rest
encryptionHandler, err := encryption.NewLocalEncryptionHandler()
if err != nil {
return nil, errors.Wrap(err, "Failed to initialize encryption handler on partition")
}
st.encryptionHandler = encryptionHandler
}
return st, nil
}
// replacePartition creates a new stream partition to replace another one. The
// old partition's events timestamps are kept.
func (s *Server) replacePartition(oldPartition *partition, recovered bool, config *proto.StreamConfig) (*partition, error) {
st, err := s.newPartition(oldPartition.Partition, recovered, config)
if err == nil {
st.messagesReceivedTimestamps = oldPartition.MessagesReceivedTimestamps()
st.pauseTimestamps = oldPartition.PauseTimestamps()
st.readonlyTimestamps = oldPartition.ReadonlyTimestamps()
}
return st, err
}
// String returns a human-readable string representation of the partition.
func (p *partition) String() string {
return fmt.Sprintf("[subject=%s, stream=%s, partition=%d]", p.Subject, p.Stream, p.Id)
}
// close stops the partition if it is running and closes the commit log. Must
// be called within the scope of the partition mutex.
func (p *partition) close() error {
p.closeMu.Lock()
defer p.closeMu.Unlock()
if p.isClosed {
return nil
}
if err := p.log.Close(); err != nil {
return err
}
if err := p.stopLeadingOrFollowing(); err != nil {
return err
}
p.isClosed = true
return nil
}
// Close stops the partition if it is running and closes the commit log.
func (p *partition) Close() error {
p.mu.Lock()
defer p.mu.Unlock()
return p.close()
}
// Pause stops the partition if it is running, closes the commit log and sets
// the paused flag.
func (p *partition) Pause() error {
p.mu.Lock()
defer p.mu.Unlock()
p.paused = true
p.Paused = true // Also set the protobuf value (used for snapshotting)
p.pauseTimestamps.update()
return p.close()
}
// IsPaused indicates if the partition is currently paused.
func (p *partition) IsPaused() bool {
p.mu.RLock()
defer p.mu.RUnlock()
return p.paused
}
// SetReadonly enables or disables readonly for the partition. When enabled,
// new messages cannot be written to the log and consumers will not block once
// they reach the end of the log. This does not affect replication.
func (p *partition) SetReadonly(readonly bool) {
p.log.SetReadonly(readonly)
p.mu.Lock()
defer p.mu.Unlock()
p.Readonly = readonly // Also set the protobuf value (used for snapshotting)
p.readonlyTimestamps.update()
}
// IsReadonly indicates if the partition is currently readonly.
func (p *partition) IsReadonly() bool {
return p.log.IsReadonly()
}
// GetGroupConsumer returns the consumer for the given group or nil if no
// consumer is subscribed.
func (p *partition) GetGroupConsumer(groupID string) *groupMember {
p.consumersMu.Lock()
defer p.consumersMu.Unlock()
return p.consumers[groupID]
}
// Delete stops the partition if it is running, closes, and deletes the commit
// log.
func (p *partition) Delete() error {
p.mu.Lock()
defer p.mu.Unlock()
if err := p.log.Delete(); err != nil {
return err
}
return p.stopLeadingOrFollowing()
}
// Subscribe sets up a subscription on the partition and begins sending
// messages on the returned channel. The subscription will run until the cancel
// channel is closed, the context is canceled, or an error is returned
// asynchronously on the status channel. If the subscriber is part of a
// consumer group, this will ensure only one member of the group is subscribed
// to the partition at a time.
func (p *partition) Subscribe(ctx context.Context, req *client.SubscribeRequest) (
*subscription, *status.Status) {
var (
previousSubscriber *groupMember
groupID string
consumerID string
groupEpoch uint64
)
if req.Consumer != nil {
groupID = req.Consumer.GroupId
consumerID = req.Consumer.ConsumerId
groupEpoch = req.Consumer.GroupEpoch
}
if groupID != "" {
// Grab the consumers mutex if this subscriber is part of a consumer
// group.
p.consumersMu.Lock()
defer p.consumersMu.Unlock()
// If there is an existing member of the group subscribed to the
// partition, check if the new subscriber has a more recent group
// epoch. If it does, it will replace the existing consumer.
existing, ok := p.consumers[groupID]
if ok {
if existing.groupEpoch > groupEpoch {
return nil, status.New(codes.FailedPrecondition,
"Consumer is not currently assigned this partition")
}
previousSubscriber = existing
}
}
startOffset, st := p.getStartOffset(req)
if st != nil {
return nil, st
}
stopOffset, st := p.getStopOffset(req)
if st != nil {
return nil, st
}
if stopOffset != waitForNewMessages && stopOffset < startOffset {
return nil, status.New(
codes.InvalidArgument, fmt.Sprintf("Stop offset is before start offset: %d < %d",
stopOffset, startOffset))
}
// Cancel previous group subscriber if there was one.
if previousSubscriber != nil {
p.srv.logger.Debugf("Replacing group %s consumer %s with consumer %s for partition %s",
groupID, previousSubscriber.consumerID, consumerID, p)
previousSubscriber.sub.Close()
}
var (
ch = make(chan *client.Message)
errCh = make(chan *status.Status)
reader, err = p.log.NewReader(startOffset, false)
)
if err != nil {
return nil, status.New(
codes.Internal, fmt.Sprintf("Failed to create stream reader: %v", err))
}
cancel := make(chan struct{})
p.srv.startGoroutine(p.newSubscribeLoop(ctx, groupID, consumerID, reader,
stopOffset, ch, errCh, cancel))
sub := &subscription{
closed: cancel,
msgs: ch,
errors: errCh,
}
if groupID != "" {
p.consumers[groupID] = &groupMember{
consumerID: consumerID,
groupEpoch: groupEpoch,
sub: sub,
}
}
return sub, nil
}
// newSubscribeLoop returns a function to be called in a goroutine which starts
// the subscription loop.
func (p *partition) newSubscribeLoop(ctx context.Context, groupID, consumerID string,
reader *commitlog.Reader, stopOffset int64, ch chan<- *client.Message, errCh chan<- *status.Status,
cancel <-chan struct{}) func() {
return func() {
// Update the active subscriber count.
p.increaseSubscriberCount()
defer p.decreaseSubscriberCount()
if groupID != "" {
defer p.removeGroupSubscriber(groupID, consumerID)
}
headersBuf := make([]byte, 28)
for {
// TODO: this could be more efficient.
m, offset, timestamp, _, err := reader.ReadMessage(ctx, headersBuf)
if err != nil {
var s *status.Status
if err == commitlog.ErrCommitLogDeleted {
// Partition was deleted while subscribed.
s = status.New(codes.NotFound, err.Error())
} else if err == commitlog.ErrCommitLogClosed {
// Partition was closed while subscribed (likely paused).
code := codes.Internal
if p.IsPaused() {
code = codes.FailedPrecondition
}
s = status.New(code, err.Error())
} else if err == commitlog.ErrCommitLogReadonly {
// Partition was set to readonly while subscribed.
s = status.New(codes.ResourceExhausted, "End of readonly partition")
} else {
s = status.Convert(err)
}
select {
case errCh <- s:
case <-cancel:
}
return
}
msgValue := m.Value()
headers := m.Headers()
// Data decryption
if p.encryptionHandler != nil {
// Decryption of data on server side
decryptedMsg, err := p.encryptionHandler.Read(msgValue)
if err != nil {
s := status.Convert(err)
select {
case errCh <- s:
case <-cancel:
}
return
}
msgValue = decryptedMsg
}
var (
msg = &client.Message{
Stream: p.Stream,
Partition: p.Id,
Offset: offset,
Key: m.Key(),
Value: msgValue,
Timestamp: timestamp,
Headers: headers,
Subject: string(headers["subject"]),
ReplySubject: string(headers["reply"]),
}
)
select {
case ch <- msg:
case <-cancel:
return
}
if offset == stopOffset {
s := status.New(codes.ResourceExhausted, "Stop offset reached")
select {
case errCh <- s:
case <-cancel:
}
return
}
}
}
}
func (p *partition) removeGroupSubscriber(groupID, consumerID string) {
p.consumersMu.Lock()
defer p.consumersMu.Unlock()
sub, ok := p.consumers[groupID]
if !ok {
return
}
if sub.consumerID == consumerID {
delete(p.consumers, groupID)
}
}
func (p *partition) getStartOffset(req *client.SubscribeRequest) (int64, *status.Status) {
var startOffset int64
switch req.StartPosition {
case client.StartPosition_OFFSET:
startOffset = req.StartOffset
case client.StartPosition_TIMESTAMP:
offset, err := p.log.EarliestOffsetAfterTimestamp(req.StartTimestamp)
if err != nil {
return startOffset, status.New(
codes.Internal, fmt.Sprintf("Failed to lookup offset for timestamp: %v", err))
}
startOffset = offset
case client.StartPosition_EARLIEST:
startOffset = p.log.OldestOffset()
case client.StartPosition_LATEST:
startOffset = p.log.NewestOffset()
case client.StartPosition_NEW_ONLY:
startOffset = p.log.NewestOffset() + 1
default:
return startOffset, status.New(
codes.InvalidArgument,
fmt.Sprintf("Unknown StartPosition %s", req.StartPosition))
}
// If log is empty, next offset will be 0.
if startOffset < 0 {
startOffset = 0
}
return startOffset, nil
}
func (p *partition) getStopOffset(req *client.SubscribeRequest) (int64, *status.Status) {
var stopOffset int64
switch req.StopPosition {
case client.StopPosition_STOP_ON_CANCEL:
stopOffset = waitForNewMessages
if p.log.IsReadonly() {
stopOffset = p.log.NewestOffset()
}
case client.StopPosition_STOP_OFFSET:
stopOffset = req.StopOffset
case client.StopPosition_STOP_TIMESTAMP:
var err error
stopOffset, err = p.log.LatestOffsetBeforeTimestamp(req.StopTimestamp)
if err != nil {
return stopOffset, status.New(
codes.Internal, fmt.Sprintf("Failed to lookup offset for timestamp: %v", err))
}
case client.StopPosition_STOP_LATEST:
stopOffset = p.log.NewestOffset()
if stopOffset == -1 {
return stopOffset, status.New(codes.ResourceExhausted, "Stream is empty")
}
default:
return stopOffset, status.New(
codes.InvalidArgument,
fmt.Sprintf("Unknown StopPosition %s", req.StopPosition))
}
return stopOffset, nil
}
// increaseSubscriberCount increases the number of subscribers. Partitions with
// a subscriber count greater than zero will not be auto-paused if the
// partition is idle, and the corresponding configuration option is set.
func (p *partition) increaseSubscriberCount() {
p.mu.Lock()
defer p.mu.Unlock()
p.subscriberCount++
}
// decreaseSubscriberCount decreases the number of subscribers. Partitions with
// a subscriber count greater than zero will not be auto-paused if the
// partition is idle, and the corresponding configuration option is set.
func (p *partition) decreaseSubscriberCount() {
p.mu.Lock()
defer p.mu.Unlock()
p.subscriberCount--
if p.subscriberCount < 0 {
p.subscriberCount = 0
p.srv.logger.Errorf("Negative partition subscriber count for partition %s: %d",
p, p.subscriberCount)
}
}
// MessagesReceivedTimestamps returns the first and latest times a message was
// received on this partition.
func (p *partition) MessagesReceivedTimestamps() EventTimestamps {
p.mu.RLock()
defer p.mu.RUnlock()
return p.messagesReceivedTimestamps
}
// PauseTimestamps returns the first and latest time this partition was paused
// or resumed.
func (p *partition) PauseTimestamps() EventTimestamps {
p.mu.RLock()
defer p.mu.RUnlock()
return p.pauseTimestamps
}
// ReadonlyTimestamps returns the first and latest time this partition had its
// read-only status changed.
func (p *partition) ReadonlyTimestamps() EventTimestamps {
p.mu.RLock()
defer p.mu.RUnlock()
return p.readonlyTimestamps
}
// Notify is used to short circuit the sleep backoff a partition uses when it
// has replicated to the end of the leader's log (i.e. the log end offset).
// When a follower reaches the end of the log, it starts to sleep in between
// replication requests to avoid overloading the leader. However, this causes
// added commit latency when new messages are published to the log since the
// follower is idle. As a result, the leader will note when a follower is
// caught up and send a notification in order to wake an idle follower back up
// when new data is written to the log.
func (p *partition) Notify() {
if p.IsLeader() {
// If we are now the leader, do nothing.
return
}
select {
case p.notify <- struct{}{}:
default:
}
}
// SetLeader sets the leader for the partition to the given replica and leader
// epoch. If the partition's current leader epoch is greater than the given
// epoch, this returns an error. This will also start the partition as a leader
// or follower, if applicable, unless the partition is in recovery mode or
// paused.
func (p *partition) SetLeader(leader string, epoch uint64) error {
p.mu.Lock()
defer p.mu.Unlock()
if epoch < p.LeaderEpoch {
return fmt.Errorf("proposed leader epoch %d is less than current epoch %d",
epoch, p.LeaderEpoch)
}
p.Leader = leader
p.LeaderEpoch = epoch
if p.recovered || p.paused {
// If this partition is being recovered, we will start the
// leader/follower loop later. If it's paused, we won't start it til
// it's resumed.
return nil
}
return p.startLeadingOrFollowing()
}
// StartRecovered starts the partition as a leader or follower, if applicable,
// if it's in recovery mode. This should be called for each partition after the
// recovery process completes. If the partition is paused, this will be a
// no-op.
func (p *partition) StartRecovered() (bool, error) {
p.mu.Lock()
defer p.mu.Unlock()
if !p.recovered {
return false, nil
}
if p.paused {
return true, nil
}
if err := p.startLeadingOrFollowing(); err != nil {
return false, err
}
p.recovered = false
return true, nil
}
// startLeadingOrFollowing starts the partition as a leader or follower, if
// applicable.
func (p *partition) startLeadingOrFollowing() error {
if p.Leader == p.srv.config.Clustering.ServerID {
p.srv.logger.Debugf("Server becoming leader for partition %s, epoch: %d", p, p.LeaderEpoch)
if err := p.becomeLeader(p.LeaderEpoch); err != nil {
p.srv.logger.Errorf("Server failed becoming leader for partition %s: %v", p, err)
return err
}
} else if p.inReplicas(p.srv.config.Clustering.ServerID) {
p.srv.logger.Debugf("Server becoming follower for partition %s, epoch: %d", p, p.LeaderEpoch)
if err := p.becomeFollower(); err != nil {
p.srv.logger.Errorf("Server failed becoming follower for partition %s: %v", p, err)
return err
}
}
return nil
}
// stopLeadingOrFollowing stops the partition as a leader or follower, if
// applicable. Must be called within the scope of the partition mutex.
func (p *partition) stopLeadingOrFollowing() error {
if p.isFollowing {
// Stop following if previously a follower.
if err := p.stopFollowing(); err != nil {
return err
}
} else if p.isLeading {
// If previously a leader, we need to reset.
if err := p.stopLeading(); err != nil {
return err
}
}
return nil
}
// GetLeader returns the replica that is the partition leader and the leader
// epoch.
func (p *partition) GetLeader() (string, uint64) {
p.mu.RLock()
defer p.mu.RUnlock()
return p.Leader, p.LeaderEpoch
}
// IsLeader indicates if this server is the partition leader.
func (p *partition) IsLeader() bool {
p.mu.RLock()
defer p.mu.RUnlock()
return p.isLeading
}
// becomeLeader is called when the server has become the leader for this
// partition.
func (p *partition) becomeLeader(epoch uint64) error {
if err := p.stopLeadingOrFollowing(); err != nil {
return err
}
if !p.recovered {
// Update leader epoch on log if this isn't a recovered partition. A
// recovered partition indicates we were the previous leader and are
// continuing a leader epoch.
if err := p.log.NewLeaderEpoch(epoch); err != nil {
return errors.Wrap(err, "failed to update leader epoch on log")
}
}
// Update this replica's latest offset to ensure it's up to date.
rep := p.isr[p.srv.config.Clustering.ServerID]
rep.updateLatestOffset(p.log.NewestOffset())
// Start message processing loop.
recvChan := make(chan *nats.Msg, recvChannelSize)
p.stopLeader = make(chan struct{})
p.srv.startGoroutineWithArgsWG(func(args ...interface{}) {
stop := args[0].(chan struct{})
p.messageProcessingLoop(recvChan, stop, epoch)
}, p.shutdown, p.stopLeader)
// Start replicating to followers.
p.startReplicating(epoch, p.stopLeader)
// Subscribe to the NATS subject and begin sequencing messages.
// TODO: This should be drained on shutdown.
sub, err := p.srv.nc.QueueSubscribe(p.getSubject(), p.Group, func(m *nats.Msg) {
recvChan <- m
})
if err != nil {
return errors.Wrap(err, "failed to subscribe to NATS")
}
sub.SetPendingLimits(-1, -1)
p.sub = sub
p.srv.nc.Flush()
// Subscribe to the partition replication subject.
sub, err = p.srv.ncRepl.Subscribe(p.getReplicationRequestInbox(), p.handleReplicationRequest)
if err != nil {
return errors.Wrap(err, "failed to subscribe to replication inbox")
}
sub.SetPendingLimits(-1, -1)
p.leaderReplSub = sub
// Also subscribe to leader epoch offset requests subject.
sub, err = p.srv.ncRepl.Subscribe(p.getLeaderOffsetRequestInbox(), p.handleLeaderOffsetRequest)
if err != nil {
return errors.Wrap(err, "failed to subscribe to replication inbox")
}
sub.SetPendingLimits(-1, -1)
p.leaderOffsetSub = sub
p.srv.ncRepl.Flush()
// Start auto-pause timer if enabled.
if p.autoPauseTime > 0 {
p.srv.startGoroutine(func() {
p.autoPauseLoop(p.stopLeader)
})
}
p.isLeading = true
p.isFollowing = false
// Notify the cursor manager if we've become leader for a cursor partition.
if p.Stream == cursorsStream {
p.srv.cursors.BecomePartitionLeader()
}
return nil
}
// stopLeading causes the partition to step down as leader by unsubscribing
// from the NATS subject and replication subject, stopping message processing
// and replication, and disposing the commit queue. Must be called within the
// scope of the partition mutex.
func (p *partition) stopLeading() error {
// Unsubscribe from NATS subject.
if err := p.sub.Unsubscribe(); err != nil {
return err
}
// Unsubscribe from replication subject.
if err := p.leaderReplSub.Unsubscribe(); err != nil {
return err
}
// Unsubscribe from leader epoch offset subject.
if err := p.leaderOffsetSub.Unsubscribe(); err != nil {
return err
}
// Stop processing messages and replicating.
close(p.stopLeader)
// Wait for loops to shutdown. Release mutex while we wait to avoid
// deadlocks.
p.mu.Unlock()
p.shutdown.Wait()
p.mu.Lock()
p.commitQueue.Dispose()
p.isLeading = false
return nil
}
// becomeFollower is called when the server has become a follower for this
// partition.
func (p *partition) becomeFollower() error {
if err := p.stopLeadingOrFollowing(); err != nil {
return err
}
// Truncate potentially uncommitted messages from the log.
if err := p.truncateUncommitted(); err != nil {
return errors.Wrap(err, "failed to truncate log")
}
// Start fetching messages from the leader's log starting at the HW.
p.stopFollower = make(chan struct{})
p.srv.logger.Debugf("Replicating partition %s from leader %s", p, p.Leader)
p.srv.startGoroutine(func() {
p.replicationRequestLoop(p.Leader, p.LeaderEpoch, p.stopFollower)
})
p.isFollowing = true
p.isLeading = false
return nil
}
// stopFollowing causes the partition to step down as a follower by stopping
// replication requests and the leader failure detector.
func (p *partition) stopFollowing() error {
// Stop replication request and leader failure detector loop.
// TODO: Do graceful shutdown similar to stopLeading().
close(p.stopFollower)
p.isFollowing = false
return nil
}
// handleLeaderOffsetRequest is a NATS handler that's invoked when the leader
// receives a leader epoch offset request from a follower. The request will
// contain the latest leader epoch in the follower's leader epoch sequence.
// This will send the last offset for the requested leader epoch, i.e. the
// start offset of the first leader epoch larger than the requested leader
// epoch or the log end offset if the leader's current epoch is equal to the
// one requested.
func (p *partition) handleLeaderOffsetRequest(msg *nats.Msg) {
req, err := proto.UnmarshalLeaderEpochOffsetRequest(msg.Data)
if err != nil {
p.srv.logger.Errorf("Invalid leader epoch offset request for partition %s: %v", p, err)
return
}
resp, err := proto.MarshalLeaderEpochOffsetResponse(&proto.LeaderEpochOffsetResponse{
EndOffset: p.log.LastOffsetForLeaderEpoch(req.LeaderEpoch),
})
if err != nil {
panic(err)
}
if err := msg.Respond(resp); err != nil {
p.srv.logger.Errorf("Failed to respond to leader offset request: %v", err)
}
}
// handleReplicationRequest is a NATS handler that's invoked when the leader
// receives a replication request from a follower. It will send messages to the
// NATS subject specified on the request.
func (p *partition) handleReplicationRequest(msg *nats.Msg) {
received := time.Now()
req, err := proto.UnmarshalReplicationRequest(msg.Data)
if err != nil {
p.srv.logger.Errorf("Invalid replication request for partition %s: %v", p, err)
return
}
p.mu.Lock()
defer p.mu.Unlock()
if p.pause {
return
}
if req.LeaderEpoch != 0 && req.LeaderEpoch != p.LeaderEpoch {
// This could indicate either another leader was elected (e.g. if this
// node was somehow partitioned from the rest of the ISR) or the
// follower is still trying to replicate from a previous leader. In
// either case, drop the request.
p.srv.logger.Warnf("Received replication request for partition %s from replica %s "+
"in leader epoch %d, but current leader epoch is %d",
p, req.ReplicaID, req.LeaderEpoch, p.LeaderEpoch)
return
}
if _, ok := p.replicas[req.ReplicaID]; !ok {
p.srv.logger.Warnf("Received replication request for partition %s from non-replica %s",
p, req.ReplicaID)
return
}
replicator, ok := p.replicators[req.ReplicaID]
if !ok {
panic(fmt.Sprintf("No replicator for partition %s and replica %s", p, req.ReplicaID))
}