-
Notifications
You must be signed in to change notification settings - Fork 13.6k
/
KafkaRaftClient.java
2602 lines (2280 loc) · 106 KB
/
KafkaRaftClient.java
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
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.raft;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.ClusterAuthorizationException;
import org.apache.kafka.common.errors.NotLeaderOrFollowerException;
import org.apache.kafka.common.memory.MemoryPool;
import org.apache.kafka.common.message.BeginQuorumEpochRequestData;
import org.apache.kafka.common.message.BeginQuorumEpochResponseData;
import org.apache.kafka.common.message.DescribeQuorumRequestData;
import org.apache.kafka.common.message.DescribeQuorumResponseData;
import org.apache.kafka.common.message.EndQuorumEpochRequestData;
import org.apache.kafka.common.message.EndQuorumEpochResponseData;
import org.apache.kafka.common.message.FetchRequestData;
import org.apache.kafka.common.message.FetchResponseData;
import org.apache.kafka.common.message.FetchSnapshotRequestData;
import org.apache.kafka.common.message.FetchSnapshotResponseData;
import org.apache.kafka.common.message.VoteRequestData;
import org.apache.kafka.common.message.VoteResponseData;
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ApiMessage;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.record.Records;
import org.apache.kafka.common.record.UnalignedMemoryRecords;
import org.apache.kafka.common.record.UnalignedRecords;
import org.apache.kafka.common.requests.BeginQuorumEpochRequest;
import org.apache.kafka.common.requests.BeginQuorumEpochResponse;
import org.apache.kafka.common.requests.DescribeQuorumRequest;
import org.apache.kafka.common.requests.DescribeQuorumResponse;
import org.apache.kafka.common.requests.EndQuorumEpochRequest;
import org.apache.kafka.common.requests.EndQuorumEpochResponse;
import org.apache.kafka.common.requests.FetchResponse;
import org.apache.kafka.common.requests.FetchSnapshotRequest;
import org.apache.kafka.common.requests.FetchSnapshotResponse;
import org.apache.kafka.common.requests.VoteRequest;
import org.apache.kafka.common.requests.VoteResponse;
import org.apache.kafka.common.utils.BufferSupplier;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Timer;
import org.apache.kafka.raft.RequestManager.ConnectionState;
import org.apache.kafka.raft.errors.NotLeaderException;
import org.apache.kafka.raft.internals.BatchAccumulator;
import org.apache.kafka.raft.internals.BatchMemoryPool;
import org.apache.kafka.raft.internals.BlockingMessageQueue;
import org.apache.kafka.raft.internals.CloseListener;
import org.apache.kafka.raft.internals.FuturePurgatory;
import org.apache.kafka.raft.internals.KafkaRaftMetrics;
import org.apache.kafka.raft.internals.MemoryBatchReader;
import org.apache.kafka.raft.internals.RecordsBatchReader;
import org.apache.kafka.raft.internals.ThresholdPurgatory;
import org.apache.kafka.server.common.serialization.RecordSerde;
import org.apache.kafka.snapshot.RawSnapshotReader;
import org.apache.kafka.snapshot.RawSnapshotWriter;
import org.apache.kafka.snapshot.RecordsSnapshotReader;
import org.apache.kafka.snapshot.RecordsSnapshotWriter;
import org.apache.kafka.snapshot.SnapshotWriter;
import org.apache.kafka.snapshot.SnapshotReader;
import org.slf4j.Logger;
import java.util.Collections;
import java.util.IdentityHashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.OptionalInt;
import java.util.OptionalLong;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Supplier;
import static java.util.concurrent.CompletableFuture.completedFuture;
import static org.apache.kafka.raft.RaftUtil.hasValidTopicPartition;
/**
* This class implements a Kafkaesque version of the Raft protocol. Leader election
* is more or less pure Raft, but replication is driven by replica fetching and we use Kafka's
* log reconciliation protocol to truncate the log to a common point following each leader
* election.
*
* Like Zookeeper, this protocol distinguishes between voters and observers. Voters are
* the only ones who are eligible to handle protocol requests and they are the only ones
* who take part in elections. The protocol does not yet support dynamic quorum changes.
*
* These are the APIs in this protocol:
*
* 1) {@link VoteRequestData}: Sent by valid voters when their election timeout expires and they
* become a candidate. This request includes the last offset in the log which electors use
* to tell whether or not to grant the vote.
*
* 2) {@link BeginQuorumEpochRequestData}: Sent by the leader of an epoch only to valid voters to
* assert its leadership of the new epoch. This request will be retried indefinitely for
* each voter until it acknowledges the request or a new election occurs.
*
* This is not needed in usual Raft because the leader can use an empty data push
* to achieve the same purpose. The Kafka Raft implementation, however, is driven by
* fetch requests from followers, so there must be a way to find the new leader after
* an election has completed.
*
* 3) {@link EndQuorumEpochRequestData}: Sent by the leader of an epoch to valid voters in order to
* gracefully resign from the current epoch. This causes remaining voters to immediately
* begin a new election.
*
* 4) {@link FetchRequestData}: This is the same as the usual Fetch API in Kafka, but we add snapshot
* check before responding, and we also piggyback some additional metadata on responses (i.e. current
* leader and epoch). Unlike partition replication, we also piggyback truncation detection on this API
* rather than through a separate truncation state.
*
* 5) {@link FetchSnapshotRequestData}: Sent by the follower to the epoch leader in order to fetch a snapshot.
* This happens when a FetchResponse includes a snapshot ID due to the follower's log end offset being less
* than the leader's log start offset. This API is similar to the Fetch API since the snapshot is stored
* as FileRecords, but we use {@link UnalignedRecords} in FetchSnapshotResponse because the records
* are not necessarily offset-aligned.
*/
public class KafkaRaftClient<T> implements RaftClient<T> {
private static final int RETRY_BACKOFF_BASE_MS = 100;
public static final int MAX_FETCH_WAIT_MS = 500;
public static final int MAX_BATCH_SIZE_BYTES = 8 * 1024 * 1024;
public static final int MAX_FETCH_SIZE_BYTES = MAX_BATCH_SIZE_BYTES;
private final AtomicReference<GracefulShutdown> shutdown = new AtomicReference<>();
private final Logger logger;
private final Time time;
private final int fetchMaxWaitMs;
private final String clusterId;
private final NetworkChannel channel;
private final ReplicatedLog log;
private final Random random;
private final FuturePurgatory<Long> appendPurgatory;
private final FuturePurgatory<Long> fetchPurgatory;
private final RecordSerde<T> serde;
private final MemoryPool memoryPool;
private final RaftMessageQueue messageQueue;
private final RaftConfig raftConfig;
private final KafkaRaftMetrics kafkaRaftMetrics;
private final QuorumState quorum;
private final RequestManager requestManager;
private final RaftMetadataLogCleanerManager snapshotCleaner;
private final Map<Listener<T>, ListenerContext> listenerContexts = new IdentityHashMap<>();
private final ConcurrentLinkedQueue<Registration<T>> pendingRegistrations = new ConcurrentLinkedQueue<>();
/**
* Create a new instance.
*
* Note that if the node ID is empty, then the client will behave as a
* non-participating observer.
*/
public KafkaRaftClient(
RecordSerde<T> serde,
NetworkChannel channel,
ReplicatedLog log,
QuorumStateStore quorumStateStore,
Time time,
Metrics metrics,
ExpirationService expirationService,
LogContext logContext,
String clusterId,
OptionalInt nodeId,
RaftConfig raftConfig
) {
this(serde,
channel,
new BlockingMessageQueue(),
log,
quorumStateStore,
new BatchMemoryPool(5, MAX_BATCH_SIZE_BYTES),
time,
metrics,
expirationService,
MAX_FETCH_WAIT_MS,
clusterId,
nodeId,
logContext,
new Random(),
raftConfig);
}
KafkaRaftClient(
RecordSerde<T> serde,
NetworkChannel channel,
RaftMessageQueue messageQueue,
ReplicatedLog log,
QuorumStateStore quorumStateStore,
MemoryPool memoryPool,
Time time,
Metrics metrics,
ExpirationService expirationService,
int fetchMaxWaitMs,
String clusterId,
OptionalInt nodeId,
LogContext logContext,
Random random,
RaftConfig raftConfig
) {
this.serde = serde;
this.channel = channel;
this.messageQueue = messageQueue;
this.log = log;
this.memoryPool = memoryPool;
this.fetchPurgatory = new ThresholdPurgatory<>(expirationService);
this.appendPurgatory = new ThresholdPurgatory<>(expirationService);
this.time = time;
this.clusterId = clusterId;
this.fetchMaxWaitMs = fetchMaxWaitMs;
this.logger = logContext.logger(KafkaRaftClient.class);
this.random = random;
this.raftConfig = raftConfig;
this.snapshotCleaner = new RaftMetadataLogCleanerManager(logger, time, 60000, log::maybeClean);
Set<Integer> quorumVoterIds = raftConfig.quorumVoterIds();
this.requestManager = new RequestManager(quorumVoterIds, raftConfig.retryBackoffMs(),
raftConfig.requestTimeoutMs(), random);
this.quorum = new QuorumState(
nodeId,
quorumVoterIds,
raftConfig.electionTimeoutMs(),
raftConfig.fetchTimeoutMs(),
quorumStateStore,
time,
logContext,
random);
this.kafkaRaftMetrics = new KafkaRaftMetrics(metrics, "raft", quorum);
kafkaRaftMetrics.updateNumUnknownVoterConnections(quorum.remoteVoters().size());
// Update the voter endpoints with what's in RaftConfig
Map<Integer, RaftConfig.AddressSpec> voterAddresses = raftConfig.quorumVoterConnections();
voterAddresses.entrySet().stream()
.filter(e -> e.getValue() instanceof RaftConfig.InetAddressSpec)
.forEach(e -> this.channel.updateEndpoint(e.getKey(), (RaftConfig.InetAddressSpec) e.getValue()));
}
private void updateFollowerHighWatermark(
FollowerState state,
OptionalLong highWatermarkOpt
) {
highWatermarkOpt.ifPresent(highWatermark -> {
long newHighWatermark = Math.min(endOffset().offset, highWatermark);
if (state.updateHighWatermark(OptionalLong.of(newHighWatermark))) {
logger.debug("Follower high watermark updated to {}", newHighWatermark);
log.updateHighWatermark(new LogOffsetMetadata(newHighWatermark));
updateListenersProgress(newHighWatermark);
}
});
}
private void updateLeaderEndOffsetAndTimestamp(
LeaderState<T> state,
long currentTimeMs
) {
final LogOffsetMetadata endOffsetMetadata = log.endOffset();
if (state.updateLocalState(currentTimeMs, endOffsetMetadata)) {
onUpdateLeaderHighWatermark(state, currentTimeMs);
}
fetchPurgatory.maybeComplete(endOffsetMetadata.offset, currentTimeMs);
}
private void onUpdateLeaderHighWatermark(
LeaderState<T> state,
long currentTimeMs
) {
state.highWatermark().ifPresent(highWatermark -> {
logger.debug("Leader high watermark updated to {}", highWatermark);
log.updateHighWatermark(highWatermark);
// After updating the high watermark, we first clear the append
// purgatory so that we have an opportunity to route the pending
// records still held in memory directly to the listener
appendPurgatory.maybeComplete(highWatermark.offset, currentTimeMs);
// It is also possible that the high watermark is being updated
// for the first time following the leader election, so we need
// to give lagging listeners an opportunity to catch up as well
updateListenersProgress(highWatermark.offset);
});
}
private void updateListenersProgress(long highWatermark) {
for (ListenerContext listenerContext : listenerContexts.values()) {
listenerContext.nextExpectedOffset().ifPresent(nextExpectedOffset -> {
if (nextExpectedOffset < log.startOffset() && nextExpectedOffset < highWatermark) {
SnapshotReader<T> snapshot = latestSnapshot().orElseThrow(() -> new IllegalStateException(
String.format(
"Snapshot expected since next offset of %s is %d, log start offset is %d and high-watermark is %d",
listenerContext.listenerName(),
nextExpectedOffset,
log.startOffset(),
highWatermark
)
));
listenerContext.fireHandleSnapshot(snapshot);
}
});
// Re-read the expected offset in case the snapshot had to be reloaded
listenerContext.nextExpectedOffset().ifPresent(nextExpectedOffset -> {
if (nextExpectedOffset < highWatermark) {
LogFetchInfo readInfo = log.read(nextExpectedOffset, Isolation.COMMITTED);
listenerContext.fireHandleCommit(nextExpectedOffset, readInfo.records);
}
});
}
}
private Optional<SnapshotReader<T>> latestSnapshot() {
return log.latestSnapshot().map(reader ->
RecordsSnapshotReader.of(reader,
serde,
BufferSupplier.create(),
MAX_BATCH_SIZE_BYTES,
true /* Validate batch CRC*/
)
);
}
private void maybeFireHandleCommit(long baseOffset, int epoch, long appendTimestamp, int sizeInBytes, List<T> records) {
for (ListenerContext listenerContext : listenerContexts.values()) {
listenerContext.nextExpectedOffset().ifPresent(nextOffset -> {
if (nextOffset == baseOffset) {
listenerContext.fireHandleCommit(baseOffset, epoch, appendTimestamp, sizeInBytes, records);
}
});
}
}
private void maybeFireLeaderChange(LeaderState<T> state) {
for (ListenerContext listenerContext : listenerContexts.values()) {
listenerContext.maybeFireLeaderChange(quorum.leaderAndEpoch(), state.epochStartOffset());
}
}
private void maybeFireLeaderChange() {
for (ListenerContext listenerContext : listenerContexts.values()) {
listenerContext.maybeFireLeaderChange(quorum.leaderAndEpoch());
}
}
@Override
public void initialize() {
quorum.initialize(new OffsetAndEpoch(log.endOffset().offset, log.lastFetchedEpoch()));
long currentTimeMs = time.milliseconds();
if (quorum.isLeader()) {
throw new IllegalStateException("Voter cannot initialize as a Leader");
} else if (quorum.isCandidate()) {
onBecomeCandidate(currentTimeMs);
} else if (quorum.isFollower()) {
onBecomeFollower(currentTimeMs);
}
// When there is only a single voter, become candidate immediately
if (quorum.isVoter()
&& quorum.remoteVoters().isEmpty()
&& !quorum.isCandidate()) {
transitionToCandidate(currentTimeMs);
}
}
@Override
public void register(Listener<T> listener) {
pendingRegistrations.add(Registration.register(listener));
wakeup();
}
@Override
public void unregister(Listener<T> listener) {
pendingRegistrations.add(Registration.unregister(listener));
// No need to wakeup the polling thread. It is a removal so the updates can be
// delayed until the polling thread wakes up for other reasons.
}
@Override
public LeaderAndEpoch leaderAndEpoch() {
return quorum.leaderAndEpoch();
}
@Override
public OptionalInt nodeId() {
return quorum.localId();
}
private OffsetAndEpoch endOffset() {
return new OffsetAndEpoch(log.endOffset().offset, log.lastFetchedEpoch());
}
private void resetConnections() {
requestManager.resetAll();
}
private void onBecomeLeader(long currentTimeMs) {
long endOffset = log.endOffset().offset;
BatchAccumulator<T> accumulator = new BatchAccumulator<>(
quorum.epoch(),
endOffset,
raftConfig.appendLingerMs(),
MAX_BATCH_SIZE_BYTES,
memoryPool,
time,
CompressionType.NONE,
serde
);
LeaderState<T> state = quorum.transitionToLeader(endOffset, accumulator);
maybeFireLeaderChange(state);
log.initializeLeaderEpoch(quorum.epoch());
// The high watermark can only be advanced once we have written a record
// from the new leader's epoch. Hence we write a control message immediately
// to ensure there is no delay committing pending data.
state.appendLeaderChangeMessage(currentTimeMs);
resetConnections();
kafkaRaftMetrics.maybeUpdateElectionLatency(currentTimeMs);
}
private void flushLeaderLog(LeaderState<T> state, long currentTimeMs) {
// We update the end offset before flushing so that parked fetches can return sooner.
updateLeaderEndOffsetAndTimestamp(state, currentTimeMs);
log.flush(false);
}
private boolean maybeTransitionToLeader(CandidateState state, long currentTimeMs) {
if (state.isVoteGranted()) {
onBecomeLeader(currentTimeMs);
return true;
} else {
return false;
}
}
private void onBecomeCandidate(long currentTimeMs) {
CandidateState state = quorum.candidateStateOrThrow();
if (!maybeTransitionToLeader(state, currentTimeMs)) {
resetConnections();
kafkaRaftMetrics.updateElectionStartMs(currentTimeMs);
}
}
private void transitionToCandidate(long currentTimeMs) {
quorum.transitionToCandidate();
maybeFireLeaderChange();
onBecomeCandidate(currentTimeMs);
}
private void transitionToUnattached(int epoch) {
quorum.transitionToUnattached(epoch);
maybeFireLeaderChange();
resetConnections();
}
private void transitionToResigned(List<Integer> preferredSuccessors) {
fetchPurgatory.completeAllExceptionally(
Errors.NOT_LEADER_OR_FOLLOWER.exception("Not handling request since this node is resigning"));
quorum.transitionToResigned(preferredSuccessors);
maybeFireLeaderChange();
resetConnections();
}
private void transitionToVoted(int candidateId, int epoch) {
quorum.transitionToVoted(epoch, candidateId);
maybeFireLeaderChange();
resetConnections();
}
private void onBecomeFollower(long currentTimeMs) {
kafkaRaftMetrics.maybeUpdateElectionLatency(currentTimeMs);
resetConnections();
// After becoming a follower, we need to complete all pending fetches so that
// they can be re-sent to the leader without waiting for their expirations
fetchPurgatory.completeAllExceptionally(new NotLeaderOrFollowerException(
"Cannot process the fetch request because the node is no longer the leader."));
// Clearing the append purgatory should complete all futures exceptionally since this node is no longer the leader
appendPurgatory.completeAllExceptionally(new NotLeaderOrFollowerException(
"Failed to receive sufficient acknowledgments for this append before leader change."));
}
private void transitionToFollower(
int epoch,
int leaderId,
long currentTimeMs
) {
quorum.transitionToFollower(epoch, leaderId);
maybeFireLeaderChange();
onBecomeFollower(currentTimeMs);
}
private VoteResponseData buildVoteResponse(Errors partitionLevelError, boolean voteGranted) {
return VoteResponse.singletonResponse(
Errors.NONE,
log.topicPartition(),
partitionLevelError,
quorum.epoch(),
quorum.leaderIdOrSentinel(),
voteGranted);
}
/**
* Handle a Vote request. This API may return the following errors:
*
* - {@link Errors#INCONSISTENT_CLUSTER_ID} if the cluster id is presented in request
* but different from this node
* - {@link Errors#BROKER_NOT_AVAILABLE} if this node is currently shutting down
* - {@link Errors#FENCED_LEADER_EPOCH} if the epoch is smaller than this node's epoch
* - {@link Errors#INCONSISTENT_VOTER_SET} if the request suggests inconsistent voter membership (e.g.
* if this node or the sender is not one of the current known voters)
* - {@link Errors#INVALID_REQUEST} if the last epoch or offset are invalid
*/
private VoteResponseData handleVoteRequest(
RaftRequest.Inbound requestMetadata
) {
VoteRequestData request = (VoteRequestData) requestMetadata.data;
if (!hasValidClusterId(request.clusterId())) {
return new VoteResponseData().setErrorCode(Errors.INCONSISTENT_CLUSTER_ID.code());
}
if (!hasValidTopicPartition(request, log.topicPartition())) {
// Until we support multi-raft, we treat individual topic partition mismatches as invalid requests
return new VoteResponseData().setErrorCode(Errors.INVALID_REQUEST.code());
}
VoteRequestData.PartitionData partitionRequest =
request.topics().get(0).partitions().get(0);
int candidateId = partitionRequest.candidateId();
int candidateEpoch = partitionRequest.candidateEpoch();
int lastEpoch = partitionRequest.lastOffsetEpoch();
long lastEpochEndOffset = partitionRequest.lastOffset();
if (lastEpochEndOffset < 0 || lastEpoch < 0 || lastEpoch >= candidateEpoch) {
return buildVoteResponse(Errors.INVALID_REQUEST, false);
}
Optional<Errors> errorOpt = validateVoterOnlyRequest(candidateId, candidateEpoch);
if (errorOpt.isPresent()) {
return buildVoteResponse(errorOpt.get(), false);
}
if (candidateEpoch > quorum.epoch()) {
transitionToUnattached(candidateEpoch);
}
OffsetAndEpoch lastEpochEndOffsetAndEpoch = new OffsetAndEpoch(lastEpochEndOffset, lastEpoch);
boolean voteGranted = quorum.canGrantVote(candidateId, lastEpochEndOffsetAndEpoch.compareTo(endOffset()) >= 0);
if (voteGranted && quorum.isUnattached()) {
transitionToVoted(candidateId, candidateEpoch);
}
logger.info("Vote request {} with epoch {} is {}", request, candidateEpoch, voteGranted ? "granted" : "rejected");
return buildVoteResponse(Errors.NONE, voteGranted);
}
private boolean handleVoteResponse(
RaftResponse.Inbound responseMetadata,
long currentTimeMs
) {
int remoteNodeId = responseMetadata.sourceId();
VoteResponseData response = (VoteResponseData) responseMetadata.data;
Errors topLevelError = Errors.forCode(response.errorCode());
if (topLevelError != Errors.NONE) {
return handleTopLevelError(topLevelError, responseMetadata);
}
if (!hasValidTopicPartition(response, log.topicPartition())) {
return false;
}
VoteResponseData.PartitionData partitionResponse =
response.topics().get(0).partitions().get(0);
Errors error = Errors.forCode(partitionResponse.errorCode());
OptionalInt responseLeaderId = optionalLeaderId(partitionResponse.leaderId());
int responseEpoch = partitionResponse.leaderEpoch();
Optional<Boolean> handled = maybeHandleCommonResponse(
error, responseLeaderId, responseEpoch, currentTimeMs);
if (handled.isPresent()) {
return handled.get();
} else if (error == Errors.NONE) {
if (quorum.isLeader()) {
logger.debug("Ignoring vote response {} since we already became leader for epoch {}",
partitionResponse, quorum.epoch());
} else if (quorum.isCandidate()) {
CandidateState state = quorum.candidateStateOrThrow();
if (partitionResponse.voteGranted()) {
state.recordGrantedVote(remoteNodeId);
maybeTransitionToLeader(state, currentTimeMs);
} else {
state.recordRejectedVote(remoteNodeId);
// If our vote is rejected, we go immediately to the random backoff. This
// ensures that we are not stuck waiting for the election timeout when the
// vote has become gridlocked.
if (state.isVoteRejected() && !state.isBackingOff()) {
logger.info("Insufficient remaining votes to become leader (rejected by {}). " +
"We will backoff before retrying election again", state.rejectingVoters());
state.startBackingOff(
currentTimeMs,
binaryExponentialElectionBackoffMs(state.retries())
);
}
}
} else {
logger.debug("Ignoring vote response {} since we are no longer a candidate in epoch {}",
partitionResponse, quorum.epoch());
}
return true;
} else {
return handleUnexpectedError(error, responseMetadata);
}
}
private int binaryExponentialElectionBackoffMs(int retries) {
if (retries <= 0) {
throw new IllegalArgumentException("Retries " + retries + " should be larger than zero");
}
// upper limit exponential co-efficients at 20 to avoid overflow
return Math.min(RETRY_BACKOFF_BASE_MS * random.nextInt(2 << Math.min(20, retries - 1)),
raftConfig.electionBackoffMaxMs());
}
private int strictExponentialElectionBackoffMs(int positionInSuccessors, int totalNumSuccessors) {
if (positionInSuccessors <= 0 || positionInSuccessors >= totalNumSuccessors) {
throw new IllegalArgumentException("Position " + positionInSuccessors + " should be larger than zero" +
" and smaller than total number of successors " + totalNumSuccessors);
}
int retryBackOffBaseMs = raftConfig.electionBackoffMaxMs() >> (totalNumSuccessors - 1);
return Math.min(raftConfig.electionBackoffMaxMs(), retryBackOffBaseMs << (positionInSuccessors - 1));
}
private BeginQuorumEpochResponseData buildBeginQuorumEpochResponse(Errors partitionLevelError) {
return BeginQuorumEpochResponse.singletonResponse(
Errors.NONE,
log.topicPartition(),
partitionLevelError,
quorum.epoch(),
quorum.leaderIdOrSentinel());
}
/**
* Handle a BeginEpoch request. This API may return the following errors:
*
* - {@link Errors#INCONSISTENT_CLUSTER_ID} if the cluster id is presented in request
* but different from this node
* - {@link Errors#BROKER_NOT_AVAILABLE} if this node is currently shutting down
* - {@link Errors#INCONSISTENT_VOTER_SET} if the request suggests inconsistent voter membership (e.g.
* if this node or the sender is not one of the current known voters)
* - {@link Errors#FENCED_LEADER_EPOCH} if the epoch is smaller than this node's epoch
*/
private BeginQuorumEpochResponseData handleBeginQuorumEpochRequest(
RaftRequest.Inbound requestMetadata,
long currentTimeMs
) {
BeginQuorumEpochRequestData request = (BeginQuorumEpochRequestData) requestMetadata.data;
if (!hasValidClusterId(request.clusterId())) {
return new BeginQuorumEpochResponseData().setErrorCode(Errors.INCONSISTENT_CLUSTER_ID.code());
}
if (!hasValidTopicPartition(request, log.topicPartition())) {
// Until we support multi-raft, we treat topic partition mismatches as invalid requests
return new BeginQuorumEpochResponseData().setErrorCode(Errors.INVALID_REQUEST.code());
}
BeginQuorumEpochRequestData.PartitionData partitionRequest =
request.topics().get(0).partitions().get(0);
int requestLeaderId = partitionRequest.leaderId();
int requestEpoch = partitionRequest.leaderEpoch();
Optional<Errors> errorOpt = validateVoterOnlyRequest(requestLeaderId, requestEpoch);
if (errorOpt.isPresent()) {
return buildBeginQuorumEpochResponse(errorOpt.get());
}
maybeTransition(OptionalInt.of(requestLeaderId), requestEpoch, currentTimeMs);
return buildBeginQuorumEpochResponse(Errors.NONE);
}
private boolean handleBeginQuorumEpochResponse(
RaftResponse.Inbound responseMetadata,
long currentTimeMs
) {
int remoteNodeId = responseMetadata.sourceId();
BeginQuorumEpochResponseData response = (BeginQuorumEpochResponseData) responseMetadata.data;
Errors topLevelError = Errors.forCode(response.errorCode());
if (topLevelError != Errors.NONE) {
return handleTopLevelError(topLevelError, responseMetadata);
}
if (!hasValidTopicPartition(response, log.topicPartition())) {
return false;
}
BeginQuorumEpochResponseData.PartitionData partitionResponse =
response.topics().get(0).partitions().get(0);
Errors partitionError = Errors.forCode(partitionResponse.errorCode());
OptionalInt responseLeaderId = optionalLeaderId(partitionResponse.leaderId());
int responseEpoch = partitionResponse.leaderEpoch();
Optional<Boolean> handled = maybeHandleCommonResponse(
partitionError, responseLeaderId, responseEpoch, currentTimeMs);
if (handled.isPresent()) {
return handled.get();
} else if (partitionError == Errors.NONE) {
if (quorum.isLeader()) {
LeaderState<T> state = quorum.leaderStateOrThrow();
state.addAcknowledgementFrom(remoteNodeId);
} else {
logger.debug("Ignoring BeginQuorumEpoch response {} since " +
"this node is not the leader anymore", response);
}
return true;
} else {
return handleUnexpectedError(partitionError, responseMetadata);
}
}
private EndQuorumEpochResponseData buildEndQuorumEpochResponse(Errors partitionLevelError) {
return EndQuorumEpochResponse.singletonResponse(
Errors.NONE,
log.topicPartition(),
partitionLevelError,
quorum.epoch(),
quorum.leaderIdOrSentinel());
}
/**
* Handle an EndEpoch request. This API may return the following errors:
*
* - {@link Errors#INCONSISTENT_CLUSTER_ID} if the cluster id is presented in request
* but different from this node
* - {@link Errors#BROKER_NOT_AVAILABLE} if this node is currently shutting down
* - {@link Errors#INCONSISTENT_VOTER_SET} if the request suggests inconsistent voter membership (e.g.
* if this node or the sender is not one of the current known voters)
* - {@link Errors#FENCED_LEADER_EPOCH} if the epoch is smaller than this node's epoch
*/
private EndQuorumEpochResponseData handleEndQuorumEpochRequest(
RaftRequest.Inbound requestMetadata,
long currentTimeMs
) {
EndQuorumEpochRequestData request = (EndQuorumEpochRequestData) requestMetadata.data;
if (!hasValidClusterId(request.clusterId())) {
return new EndQuorumEpochResponseData().setErrorCode(Errors.INCONSISTENT_CLUSTER_ID.code());
}
if (!hasValidTopicPartition(request, log.topicPartition())) {
// Until we support multi-raft, we treat topic partition mismatches as invalid requests
return new EndQuorumEpochResponseData().setErrorCode(Errors.INVALID_REQUEST.code());
}
EndQuorumEpochRequestData.PartitionData partitionRequest =
request.topics().get(0).partitions().get(0);
int requestEpoch = partitionRequest.leaderEpoch();
int requestLeaderId = partitionRequest.leaderId();
Optional<Errors> errorOpt = validateVoterOnlyRequest(requestLeaderId, requestEpoch);
if (errorOpt.isPresent()) {
return buildEndQuorumEpochResponse(errorOpt.get());
}
maybeTransition(OptionalInt.of(requestLeaderId), requestEpoch, currentTimeMs);
if (quorum.isFollower()) {
FollowerState state = quorum.followerStateOrThrow();
if (state.leaderId() == requestLeaderId) {
List<Integer> preferredSuccessors = partitionRequest.preferredSuccessors();
long electionBackoffMs = endEpochElectionBackoff(preferredSuccessors);
logger.debug("Overriding follower fetch timeout to {} after receiving " +
"EndQuorumEpoch request from leader {} in epoch {}", electionBackoffMs,
requestLeaderId, requestEpoch);
state.overrideFetchTimeout(currentTimeMs, electionBackoffMs);
}
}
return buildEndQuorumEpochResponse(Errors.NONE);
}
private long endEpochElectionBackoff(List<Integer> preferredSuccessors) {
// Based on the priority inside the preferred successors, choose the corresponding delayed
// election backoff time based on strict exponential mechanism so that the most up-to-date
// voter has a higher chance to be elected. If the node's priority is highest, become
// candidate immediately instead of waiting for next poll.
int position = preferredSuccessors.indexOf(quorum.localIdOrThrow());
if (position <= 0) {
return 0;
} else {
return strictExponentialElectionBackoffMs(position, preferredSuccessors.size());
}
}
private boolean handleEndQuorumEpochResponse(
RaftResponse.Inbound responseMetadata,
long currentTimeMs
) {
EndQuorumEpochResponseData response = (EndQuorumEpochResponseData) responseMetadata.data;
Errors topLevelError = Errors.forCode(response.errorCode());
if (topLevelError != Errors.NONE) {
return handleTopLevelError(topLevelError, responseMetadata);
}
if (!hasValidTopicPartition(response, log.topicPartition())) {
return false;
}
EndQuorumEpochResponseData.PartitionData partitionResponse =
response.topics().get(0).partitions().get(0);
Errors partitionError = Errors.forCode(partitionResponse.errorCode());
OptionalInt responseLeaderId = optionalLeaderId(partitionResponse.leaderId());
int responseEpoch = partitionResponse.leaderEpoch();
Optional<Boolean> handled = maybeHandleCommonResponse(
partitionError, responseLeaderId, responseEpoch, currentTimeMs);
if (handled.isPresent()) {
return handled.get();
} else if (partitionError == Errors.NONE) {
ResignedState resignedState = quorum.resignedStateOrThrow();
resignedState.acknowledgeResignation(responseMetadata.sourceId());
return true;
} else {
return handleUnexpectedError(partitionError, responseMetadata);
}
}
private FetchResponseData buildFetchResponse(
Errors error,
Records records,
ValidOffsetAndEpoch validOffsetAndEpoch,
Optional<LogOffsetMetadata> highWatermark
) {
return RaftUtil.singletonFetchResponse(log.topicPartition(), log.topicId(), Errors.NONE, partitionData -> {
partitionData
.setRecords(records)
.setErrorCode(error.code())
.setLogStartOffset(log.startOffset())
.setHighWatermark(highWatermark
.map(offsetMetadata -> offsetMetadata.offset)
.orElse(-1L));
partitionData.currentLeader()
.setLeaderEpoch(quorum.epoch())
.setLeaderId(quorum.leaderIdOrSentinel());
switch (validOffsetAndEpoch.kind()) {
case DIVERGING:
partitionData.divergingEpoch()
.setEpoch(validOffsetAndEpoch.offsetAndEpoch().epoch)
.setEndOffset(validOffsetAndEpoch.offsetAndEpoch().offset);
break;
case SNAPSHOT:
partitionData.snapshotId()
.setEpoch(validOffsetAndEpoch.offsetAndEpoch().epoch)
.setEndOffset(validOffsetAndEpoch.offsetAndEpoch().offset);
break;
default:
}
});
}
private FetchResponseData buildEmptyFetchResponse(
Errors error,
Optional<LogOffsetMetadata> highWatermark
) {
return buildFetchResponse(
error,
MemoryRecords.EMPTY,
ValidOffsetAndEpoch.valid(),
highWatermark
);
}
private boolean hasValidClusterId(String requestClusterId) {
// We don't enforce the cluster id if it is not provided.
if (requestClusterId == null) {
return true;
}
return clusterId.equals(requestClusterId);
}
/**
* Handle a Fetch request. The fetch offset and last fetched epoch are always
* validated against the current log. In the case that they do not match, the response will
* indicate the diverging offset/epoch. A follower is expected to truncate its log in this
* case and resend the fetch.
*
* This API may return the following errors:
*
* - {@link Errors#INCONSISTENT_CLUSTER_ID} if the cluster id is presented in request
* but different from this node
* - {@link Errors#BROKER_NOT_AVAILABLE} if this node is currently shutting down
* - {@link Errors#FENCED_LEADER_EPOCH} if the epoch is smaller than this node's epoch
* - {@link Errors#INVALID_REQUEST} if the request epoch is larger than the leader's current epoch
* or if either the fetch offset or the last fetched epoch is invalid
*/
private CompletableFuture<FetchResponseData> handleFetchRequest(
RaftRequest.Inbound requestMetadata,
long currentTimeMs
) {
FetchRequestData request = (FetchRequestData) requestMetadata.data;
if (!hasValidClusterId(request.clusterId())) {
return completedFuture(new FetchResponseData().setErrorCode(Errors.INCONSISTENT_CLUSTER_ID.code()));
}
if (!hasValidTopicPartition(request, log.topicPartition(), log.topicId())) {
// Until we support multi-raft, we treat topic partition mismatches as invalid requests
return completedFuture(new FetchResponseData().setErrorCode(Errors.INVALID_REQUEST.code()));
}
// If the ID is valid, we can set the topic name.
request.topics().get(0).setTopic(log.topicPartition().topic());
FetchRequestData.FetchPartition fetchPartition = request.topics().get(0).partitions().get(0);
if (request.maxWaitMs() < 0
|| fetchPartition.fetchOffset() < 0
|| fetchPartition.lastFetchedEpoch() < 0
|| fetchPartition.lastFetchedEpoch() > fetchPartition.currentLeaderEpoch()) {
return completedFuture(buildEmptyFetchResponse(
Errors.INVALID_REQUEST, Optional.empty()));
}
FetchResponseData response = tryCompleteFetchRequest(request.replicaId(), fetchPartition, currentTimeMs);
FetchResponseData.PartitionData partitionResponse =
response.responses().get(0).partitions().get(0);
if (partitionResponse.errorCode() != Errors.NONE.code()
|| FetchResponse.recordsSize(partitionResponse) > 0
|| request.maxWaitMs() == 0) {
return completedFuture(response);
}
CompletableFuture<Long> future = fetchPurgatory.await(
fetchPartition.fetchOffset(),
request.maxWaitMs());
return future.handle((completionTimeMs, exception) -> {
if (exception != null) {
Throwable cause = exception instanceof ExecutionException ?
exception.getCause() : exception;
// If the fetch timed out in purgatory, it means no new data is available,
// and we will complete the fetch successfully. Otherwise, if there was
// any other error, we need to return it.
Errors error = Errors.forException(cause);
if (error != Errors.REQUEST_TIMED_OUT) {
logger.debug("Failed to handle fetch from {} at {} due to {}",
request.replicaId(), fetchPartition.fetchOffset(), error);
return buildEmptyFetchResponse(error, Optional.empty());
}
}
// FIXME: `completionTimeMs`, which can be null
logger.trace("Completing delayed fetch from {} starting at offset {} at {}",
request.replicaId(), fetchPartition.fetchOffset(), completionTimeMs);
return tryCompleteFetchRequest(request.replicaId(), fetchPartition, time.milliseconds());
});
}
private FetchResponseData tryCompleteFetchRequest(
int replicaId,
FetchRequestData.FetchPartition request,
long currentTimeMs