/
QuorumController.java
2006 lines (1794 loc) · 84.7 KB
/
QuorumController.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.controller;
import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
import org.apache.kafka.clients.admin.FeatureUpdate;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.acl.AclBinding;
import org.apache.kafka.common.acl.AclBindingFilter;
import org.apache.kafka.common.config.ConfigResource;
import org.apache.kafka.common.errors.ApiException;
import org.apache.kafka.common.errors.BrokerIdNotRegisteredException;
import org.apache.kafka.common.errors.InvalidRequestException;
import org.apache.kafka.common.errors.NotControllerException;
import org.apache.kafka.common.errors.UnknownServerException;
import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
import org.apache.kafka.common.message.AllocateProducerIdsRequestData;
import org.apache.kafka.common.message.AllocateProducerIdsResponseData;
import org.apache.kafka.common.message.AlterPartitionRequestData;
import org.apache.kafka.common.message.AlterPartitionResponseData;
import org.apache.kafka.common.message.AlterPartitionReassignmentsRequestData;
import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData;
import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
import org.apache.kafka.common.message.BrokerRegistrationRequestData;
import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic;
import org.apache.kafka.common.message.CreatePartitionsResponseData.CreatePartitionsTopicResult;
import org.apache.kafka.common.message.CreateTopicsRequestData;
import org.apache.kafka.common.message.CreateTopicsResponseData;
import org.apache.kafka.common.message.ElectLeadersRequestData;
import org.apache.kafka.common.message.ElectLeadersResponseData;
import org.apache.kafka.common.message.ListPartitionReassignmentsRequestData;
import org.apache.kafka.common.message.ListPartitionReassignmentsResponseData;
import org.apache.kafka.common.message.UpdateFeaturesRequestData;
import org.apache.kafka.common.message.UpdateFeaturesResponseData;
import org.apache.kafka.common.metadata.AccessControlEntryRecord;
import org.apache.kafka.common.metadata.BrokerRegistrationChangeRecord;
import org.apache.kafka.common.metadata.ConfigRecord;
import org.apache.kafka.common.metadata.ClientQuotaRecord;
import org.apache.kafka.common.metadata.FeatureLevelRecord;
import org.apache.kafka.common.metadata.FenceBrokerRecord;
import org.apache.kafka.common.metadata.MetadataRecordType;
import org.apache.kafka.common.metadata.NoOpRecord;
import org.apache.kafka.common.metadata.PartitionChangeRecord;
import org.apache.kafka.common.metadata.PartitionRecord;
import org.apache.kafka.common.metadata.ProducerIdsRecord;
import org.apache.kafka.common.metadata.RegisterBrokerRecord;
import org.apache.kafka.common.metadata.RemoveAccessControlEntryRecord;
import org.apache.kafka.common.metadata.RemoveTopicRecord;
import org.apache.kafka.common.metadata.TopicRecord;
import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
import org.apache.kafka.common.protocol.ApiMessage;
import org.apache.kafka.common.quota.ClientQuotaAlteration;
import org.apache.kafka.common.quota.ClientQuotaEntity;
import org.apache.kafka.common.requests.ApiError;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.controller.SnapshotGenerator.Section;
import org.apache.kafka.metadata.BrokerHeartbeatReply;
import org.apache.kafka.metadata.BrokerRegistrationReply;
import org.apache.kafka.metadata.FinalizedControllerFeatures;
import org.apache.kafka.metadata.KafkaConfigSchema;
import org.apache.kafka.metadata.authorizer.ClusterMetadataAuthorizer;
import org.apache.kafka.metadata.placement.ReplicaPlacer;
import org.apache.kafka.metadata.placement.StripedReplicaPlacer;
import org.apache.kafka.queue.EventQueue.EarliestDeadlineFunction;
import org.apache.kafka.queue.EventQueue;
import org.apache.kafka.queue.KafkaEventQueue;
import org.apache.kafka.raft.Batch;
import org.apache.kafka.raft.BatchReader;
import org.apache.kafka.raft.LeaderAndEpoch;
import org.apache.kafka.raft.OffsetAndEpoch;
import org.apache.kafka.raft.RaftClient;
import org.apache.kafka.server.authorizer.AclCreateResult;
import org.apache.kafka.server.authorizer.AclDeleteResult;
import org.apache.kafka.server.common.ApiMessageAndVersion;
import org.apache.kafka.server.common.MetadataVersion;
import org.apache.kafka.server.policy.AlterConfigPolicy;
import org.apache.kafka.server.policy.CreateTopicPolicy;
import org.apache.kafka.snapshot.SnapshotReader;
import org.apache.kafka.snapshot.SnapshotWriter;
import org.apache.kafka.timeline.SnapshotRegistry;
import org.slf4j.Logger;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map.Entry;
import java.util.Map;
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.CountDownLatch;
import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.function.Consumer;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import static java.util.concurrent.TimeUnit.MICROSECONDS;
import static java.util.concurrent.TimeUnit.NANOSECONDS;
/**
* QuorumController implements the main logic of the KRaft (Kafka Raft Metadata) mode controller.
*
* The node which is the leader of the metadata log becomes the active controller. All
* other nodes remain in standby mode. Standby controllers cannot create new metadata log
* entries. They just replay the metadata log entries that the current active controller
* has created.
*
* The QuorumController is single-threaded. A single event handler thread performs most
* operations. This avoids the need for complex locking.
*
* The controller exposes an asynchronous, futures-based API to the world. This reflects
* the fact that the controller may have several operations in progress at any given
* point. The future associated with each operation will not be completed until the
* results of the operation have been made durable to the metadata log.
*
* The QuorumController uses the "metadata.version" feature flag as a mechanism to control
* the usage of new log record schemas. Starting with 3.3, this version must be set before
* the controller can fully initialize.
*/
public final class QuorumController implements Controller {
/**
* A builder class which creates the QuorumController.
*/
static public class Builder {
private final int nodeId;
private final String clusterId;
private Time time = Time.SYSTEM;
private String threadNamePrefix = null;
private LogContext logContext = null;
private KafkaConfigSchema configSchema = KafkaConfigSchema.EMPTY;
private RaftClient<ApiMessageAndVersion> raftClient = null;
private QuorumFeatures quorumFeatures = null;
private short defaultReplicationFactor = 3;
private int defaultNumPartitions = 1;
private ReplicaPlacer replicaPlacer = new StripedReplicaPlacer(new Random());
private long snapshotMaxNewRecordBytes = Long.MAX_VALUE;
private OptionalLong leaderImbalanceCheckIntervalNs = OptionalLong.empty();
private OptionalLong maxIdleIntervalNs = OptionalLong.empty();
private long sessionTimeoutNs = ClusterControlManager.DEFAULT_SESSION_TIMEOUT_NS;
private ControllerMetrics controllerMetrics = null;
private Optional<CreateTopicPolicy> createTopicPolicy = Optional.empty();
private Optional<AlterConfigPolicy> alterConfigPolicy = Optional.empty();
private ConfigurationValidator configurationValidator = ConfigurationValidator.NO_OP;
private Optional<ClusterMetadataAuthorizer> authorizer = Optional.empty();
private Map<String, Object> staticConfig = Collections.emptyMap();
private BootstrapMetadata bootstrapMetadata = null;
public Builder(int nodeId, String clusterId) {
this.nodeId = nodeId;
this.clusterId = clusterId;
}
public int nodeId() {
return nodeId;
}
public Builder setTime(Time time) {
this.time = time;
return this;
}
public Builder setThreadNamePrefix(String threadNamePrefix) {
this.threadNamePrefix = threadNamePrefix;
return this;
}
public Builder setLogContext(LogContext logContext) {
this.logContext = logContext;
return this;
}
public Builder setConfigSchema(KafkaConfigSchema configSchema) {
this.configSchema = configSchema;
return this;
}
public Builder setRaftClient(RaftClient<ApiMessageAndVersion> logManager) {
this.raftClient = logManager;
return this;
}
public Builder setQuorumFeatures(QuorumFeatures quorumFeatures) {
this.quorumFeatures = quorumFeatures;
return this;
}
public Builder setDefaultReplicationFactor(short defaultReplicationFactor) {
this.defaultReplicationFactor = defaultReplicationFactor;
return this;
}
public Builder setDefaultNumPartitions(int defaultNumPartitions) {
this.defaultNumPartitions = defaultNumPartitions;
return this;
}
public Builder setReplicaPlacer(ReplicaPlacer replicaPlacer) {
this.replicaPlacer = replicaPlacer;
return this;
}
public Builder setSnapshotMaxNewRecordBytes(long value) {
this.snapshotMaxNewRecordBytes = value;
return this;
}
public Builder setLeaderImbalanceCheckIntervalNs(OptionalLong value) {
this.leaderImbalanceCheckIntervalNs = value;
return this;
}
public Builder setMaxIdleIntervalNs(OptionalLong value) {
this.maxIdleIntervalNs = value;
return this;
}
public Builder setSessionTimeoutNs(long sessionTimeoutNs) {
this.sessionTimeoutNs = sessionTimeoutNs;
return this;
}
public Builder setMetrics(ControllerMetrics controllerMetrics) {
this.controllerMetrics = controllerMetrics;
return this;
}
public Builder setBootstrapMetadata(BootstrapMetadata bootstrapMetadata) {
this.bootstrapMetadata = bootstrapMetadata;
return this;
}
public Builder setCreateTopicPolicy(Optional<CreateTopicPolicy> createTopicPolicy) {
this.createTopicPolicy = createTopicPolicy;
return this;
}
public Builder setAlterConfigPolicy(Optional<AlterConfigPolicy> alterConfigPolicy) {
this.alterConfigPolicy = alterConfigPolicy;
return this;
}
public Builder setConfigurationValidator(ConfigurationValidator configurationValidator) {
this.configurationValidator = configurationValidator;
return this;
}
public Builder setAuthorizer(ClusterMetadataAuthorizer authorizer) {
this.authorizer = Optional.of(authorizer);
return this;
}
public Builder setStaticConfig(Map<String, Object> staticConfig) {
this.staticConfig = staticConfig;
return this;
}
@SuppressWarnings("unchecked")
public QuorumController build() throws Exception {
if (raftClient == null) {
throw new IllegalStateException("You must set a raft client.");
} else if (bootstrapMetadata == null) {
throw new IllegalStateException("You must specify an initial metadata.version using the kafka-storage tool.");
} else if (quorumFeatures == null) {
throw new IllegalStateException("You must specify the quorum features");
}
if (threadNamePrefix == null) {
threadNamePrefix = String.format("Node%d_", nodeId);
}
if (logContext == null) {
logContext = new LogContext(String.format("[Controller %d] ", nodeId));
}
if (controllerMetrics == null) {
controllerMetrics = (ControllerMetrics) Class.forName(
"org.apache.kafka.controller.MockControllerMetrics").getConstructor().newInstance();
}
KafkaEventQueue queue = null;
try {
queue = new KafkaEventQueue(time, logContext, threadNamePrefix + "QuorumController");
return new QuorumController(
logContext,
nodeId,
clusterId,
queue,
time,
configSchema,
raftClient,
quorumFeatures,
defaultReplicationFactor,
defaultNumPartitions,
replicaPlacer,
snapshotMaxNewRecordBytes,
leaderImbalanceCheckIntervalNs,
maxIdleIntervalNs,
sessionTimeoutNs,
controllerMetrics,
createTopicPolicy,
alterConfigPolicy,
configurationValidator,
authorizer,
staticConfig,
bootstrapMetadata
);
} catch (Exception e) {
Utils.closeQuietly(queue, "event queue");
throw e;
}
}
}
/**
* Checks that a configuration resource exists.
* <p>
* This object must be used only from the controller event thread.
*/
class ConfigResourceExistenceChecker implements Consumer<ConfigResource> {
@Override
public void accept(ConfigResource configResource) {
switch (configResource.type()) {
case BROKER_LOGGER:
break;
case BROKER:
// Cluster configs are always allowed.
if (configResource.name().isEmpty()) break;
// Otherwise, check that the broker ID is valid.
int brokerId;
try {
brokerId = Integer.parseInt(configResource.name());
} catch (NumberFormatException e) {
throw new InvalidRequestException("Invalid broker name " +
configResource.name());
}
if (!clusterControl.brokerRegistrations().containsKey(brokerId)) {
throw new BrokerIdNotRegisteredException("No broker with id " +
brokerId + " found.");
}
break;
case TOPIC:
if (replicationControl.getTopicId(configResource.name()) == null) {
throw new UnknownTopicOrPartitionException("The topic '" +
configResource.name() + "' does not exist.");
}
break;
default:
break;
}
}
}
public static final String CONTROLLER_THREAD_SUFFIX = "QuorumControllerEventHandler";
private static final String ACTIVE_CONTROLLER_EXCEPTION_TEXT_PREFIX =
"The active controller appears to be node ";
private NotControllerException newNotControllerException() {
OptionalInt latestController = raftClient.leaderAndEpoch().leaderId();
if (latestController.isPresent()) {
return new NotControllerException(ACTIVE_CONTROLLER_EXCEPTION_TEXT_PREFIX +
latestController.getAsInt());
} else {
return new NotControllerException("No controller appears to be active.");
}
}
public static int exceptionToApparentController(NotControllerException e) {
if (e.getMessage().startsWith(ACTIVE_CONTROLLER_EXCEPTION_TEXT_PREFIX)) {
return Integer.parseInt(e.getMessage().substring(
ACTIVE_CONTROLLER_EXCEPTION_TEXT_PREFIX.length()));
} else {
return -1;
}
}
private void handleEventEnd(String name, long startProcessingTimeNs) {
long endProcessingTime = time.nanoseconds();
long deltaNs = endProcessingTime - startProcessingTimeNs;
log.debug("Processed {} in {} us", name,
MICROSECONDS.convert(deltaNs, NANOSECONDS));
controllerMetrics.updateEventQueueProcessingTime(NANOSECONDS.toMillis(deltaNs));
}
private Throwable handleEventException(String name,
OptionalLong startProcessingTimeNs,
Throwable exception) {
if (!startProcessingTimeNs.isPresent()) {
log.error("{}: unable to start processing because of {}.", name,
exception.getClass().getSimpleName());
if (exception instanceof ApiException) {
return exception;
} else {
return new UnknownServerException(exception);
}
}
long endProcessingTime = time.nanoseconds();
long deltaNs = endProcessingTime - startProcessingTimeNs.getAsLong();
long deltaUs = MICROSECONDS.convert(deltaNs, NANOSECONDS);
if (exception instanceof ApiException) {
log.info("{}: failed with {} in {} us", name,
exception.getClass().getSimpleName(), deltaUs);
return exception;
}
log.warn("{}: failed with unknown server exception {} at epoch {} in {} us. " +
"Renouncing leadership and reverting to the last committed offset {}.",
name, exception.getClass().getSimpleName(), curClaimEpoch, deltaUs,
lastCommittedOffset, exception);
raftClient.resign(curClaimEpoch);
renounce();
return new UnknownServerException(exception);
}
/**
* A controller event for handling internal state changes, such as Raft inputs.
*/
class ControlEvent implements EventQueue.Event {
private final String name;
private final Runnable handler;
private final long eventCreatedTimeNs = time.nanoseconds();
private OptionalLong startProcessingTimeNs = OptionalLong.empty();
ControlEvent(String name, Runnable handler) {
this.name = name;
this.handler = handler;
}
@Override
public void run() throws Exception {
long now = time.nanoseconds();
controllerMetrics.updateEventQueueTime(NANOSECONDS.toMillis(now - eventCreatedTimeNs));
startProcessingTimeNs = OptionalLong.of(now);
log.debug("Executing {}.", this);
handler.run();
handleEventEnd(this.toString(), startProcessingTimeNs.getAsLong());
}
@Override
public void handleException(Throwable exception) {
handleEventException(name, startProcessingTimeNs, exception);
}
@Override
public String toString() {
return name;
}
}
private void appendControlEvent(String name, Runnable handler) {
ControlEvent event = new ControlEvent(name, handler);
queue.append(event);
}
private static final String GENERATE_SNAPSHOT = "generateSnapshot";
private static final int MAX_BATCHES_PER_GENERATE_CALL = 10;
class SnapshotGeneratorManager implements Runnable {
private SnapshotGenerator generator = null;
void createSnapshotGenerator(long committedOffset, int committedEpoch, long committedTimestamp) {
if (generator != null) {
throw new RuntimeException("Snapshot generator already exists.");
}
if (!snapshotRegistry.hasSnapshot(committedOffset)) {
throw new RuntimeException(
String.format(
"Cannot generate a snapshot at committed offset %d because it does not exists in the snapshot registry.",
committedOffset
)
);
}
Optional<SnapshotWriter<ApiMessageAndVersion>> writer = raftClient.createSnapshot(
committedOffset,
committedEpoch,
committedTimestamp
);
if (writer.isPresent()) {
generator = new SnapshotGenerator(
logContext,
writer.get(),
MAX_BATCHES_PER_GENERATE_CALL,
Arrays.asList(
new Section("features", featureControl.iterator(committedOffset)),
new Section("cluster", clusterControl.iterator(committedOffset)),
new Section("replication", replicationControl.iterator(committedOffset)),
new Section("configuration", configurationControl.iterator(committedOffset)),
new Section("clientQuotas", clientQuotaControlManager.iterator(committedOffset)),
new Section("producerIds", producerIdControlManager.iterator(committedOffset)),
new Section("acls", aclControlManager.iterator(committedOffset))
)
);
reschedule(0);
} else {
log.info(
"Skipping generation of snapshot for committed offset {} and epoch {} since it already exists",
committedOffset,
committedEpoch
);
}
}
void cancel() {
if (generator == null) return;
log.error("Cancelling snapshot {}", generator.lastContainedLogOffset());
generator.writer().close();
generator = null;
// Delete every in-memory snapshot up to the committed offset. They are not needed since this
// snapshot generation was canceled.
snapshotRegistry.deleteSnapshotsUpTo(lastCommittedOffset);
queue.cancelDeferred(GENERATE_SNAPSHOT);
}
void reschedule(long delayNs) {
ControlEvent event = new ControlEvent(GENERATE_SNAPSHOT, this);
queue.scheduleDeferred(event.name,
new EarliestDeadlineFunction(time.nanoseconds() + delayNs), event);
}
@Override
public void run() {
if (generator == null) {
log.debug("No snapshot is in progress.");
return;
}
OptionalLong nextDelay;
try {
nextDelay = generator.generateBatches();
} catch (Exception e) {
log.error("Error while generating snapshot {}", generator.lastContainedLogOffset(), e);
generator.writer().close();
generator = null;
return;
}
if (!nextDelay.isPresent()) {
log.info("Finished generating snapshot {}.", generator.lastContainedLogOffset());
generator.writer().close();
generator = null;
// Delete every in-memory snapshot up to the committed offset. They are not needed since this
// snapshot generation finished.
snapshotRegistry.deleteSnapshotsUpTo(lastCommittedOffset);
return;
}
reschedule(nextDelay.getAsLong());
}
OptionalLong snapshotLastOffsetFromLog() {
if (generator == null) {
return OptionalLong.empty();
}
return OptionalLong.of(generator.lastContainedLogOffset());
}
}
/**
* A controller event that reads the committed internal state in order to expose it
* to an API.
*/
class ControllerReadEvent<T> implements EventQueue.Event {
private final String name;
private final CompletableFuture<T> future;
private final Supplier<T> handler;
private final long eventCreatedTimeNs = time.nanoseconds();
private OptionalLong startProcessingTimeNs = OptionalLong.empty();
ControllerReadEvent(String name, Supplier<T> handler) {
this.name = name;
this.future = new CompletableFuture<T>();
this.handler = handler;
}
CompletableFuture<T> future() {
return future;
}
@Override
public void run() throws Exception {
long now = time.nanoseconds();
controllerMetrics.updateEventQueueTime(NANOSECONDS.toMillis(now - eventCreatedTimeNs));
startProcessingTimeNs = OptionalLong.of(now);
T value = handler.get();
handleEventEnd(this.toString(), startProcessingTimeNs.getAsLong());
future.complete(value);
}
@Override
public void handleException(Throwable exception) {
future.completeExceptionally(
handleEventException(name, startProcessingTimeNs, exception));
}
@Override
public String toString() {
return name + "(" + System.identityHashCode(this) + ")";
}
}
// Visible for testing
ReplicationControlManager replicationControl() {
return replicationControl;
}
// Visible for testing
ClusterControlManager clusterControl() {
return clusterControl;
}
<T> CompletableFuture<T> appendReadEvent(
String name,
OptionalLong deadlineNs,
Supplier<T> handler
) {
ControllerReadEvent<T> event = new ControllerReadEvent<T>(name, handler);
if (deadlineNs.isPresent()) {
queue.appendWithDeadline(deadlineNs.getAsLong(), event);
} else {
queue.append(event);
}
return event.future();
}
interface ControllerWriteOperation<T> {
/**
* Generate the metadata records needed to implement this controller write
* operation. In general, this operation should not modify the "hard state" of
* the controller. That modification will happen later on, when we replay the
* records generated by this function.
* <p>
* There are cases where this function modifies the "soft state" of the
* controller. Mainly, this happens when we process cluster heartbeats.
* <p>
* This function also generates an RPC result. In general, if the RPC resulted in
* an error, the RPC result will be an error, and the generated record list will
* be empty. This would happen if we tried to create a topic with incorrect
* parameters, for example. Of course, partial errors are possible for batch
* operations.
*
* @return A result containing a list of records, and the RPC result.
*/
ControllerResult<T> generateRecordsAndResult() throws Exception;
/**
* Once we've passed the records to the Raft layer, we will invoke this function
* with the end offset at which those records were placed. If there were no
* records to write, we'll just pass the last write offset.
*/
default void processBatchEndOffset(long offset) {
}
}
/**
* A controller event that modifies the controller state.
*/
class ControllerWriteEvent<T> implements EventQueue.Event, DeferredEvent {
private final String name;
private final CompletableFuture<T> future;
private final ControllerWriteOperation<T> op;
private final long eventCreatedTimeNs = time.nanoseconds();
private OptionalLong startProcessingTimeNs = OptionalLong.empty();
private ControllerResultAndOffset<T> resultAndOffset;
ControllerWriteEvent(String name, ControllerWriteOperation<T> op) {
this.name = name;
this.future = new CompletableFuture<T>();
this.op = op;
this.resultAndOffset = null;
}
CompletableFuture<T> future() {
return future;
}
@Override
public void run() throws Exception {
long now = time.nanoseconds();
controllerMetrics.updateEventQueueTime(NANOSECONDS.toMillis(now - eventCreatedTimeNs));
int controllerEpoch = curClaimEpoch;
if (controllerEpoch == -1) {
throw newNotControllerException();
}
startProcessingTimeNs = OptionalLong.of(now);
ControllerResult<T> result = op.generateRecordsAndResult();
if (result.records().isEmpty()) {
op.processBatchEndOffset(writeOffset);
// If the operation did not return any records, then it was actually just
// a read after all, and not a read + write. However, this read was done
// from the latest in-memory state, which might contain uncommitted data.
OptionalLong maybeOffset = purgatory.highestPendingOffset();
if (!maybeOffset.isPresent()) {
// If the purgatory is empty, there are no pending operations and no
// uncommitted state. We can complete immediately.
resultAndOffset = ControllerResultAndOffset.of(-1, result);
log.debug("Completing read-only operation {} immediately because " +
"the purgatory is empty.", this);
complete(null);
} else {
// If there are operations in the purgatory, we want to wait for the latest
// one to complete before returning our result to the user.
resultAndOffset = ControllerResultAndOffset.of(maybeOffset.getAsLong(), result);
log.debug("Read-only operation {} will be completed when the log " +
"reaches offset {}", this, resultAndOffset.offset());
}
} else {
// If the operation returned a batch of records, those records need to be
// written before we can return our result to the user. Here, we hand off
// the batch of records to the raft client. They will be written out
// asynchronously.
final long offset;
if (result.isAtomic()) {
offset = raftClient.scheduleAtomicAppend(controllerEpoch, result.records());
} else {
offset = raftClient.scheduleAppend(controllerEpoch, result.records());
}
op.processBatchEndOffset(offset);
updateWriteOffset(offset);
resultAndOffset = ControllerResultAndOffset.of(offset, result);
for (ApiMessageAndVersion message : result.records()) {
replay(message.message(), Optional.empty(), offset);
}
snapshotRegistry.getOrCreateSnapshot(offset);
log.debug("Read-write operation {} will be completed when the log " +
"reaches offset {}.", this, resultAndOffset.offset());
}
// After every controller write event, schedule a leader rebalance if there are any topic partition
// with leader that is not the preferred leader.
maybeScheduleNextBalancePartitionLeaders();
// Remember the latest offset and future if it is not already completed
if (!future.isDone()) {
purgatory.add(resultAndOffset.offset(), this);
}
}
@Override
public void handleException(Throwable exception) {
complete(exception);
}
@Override
public void complete(Throwable exception) {
if (exception == null) {
handleEventEnd(this.toString(), startProcessingTimeNs.getAsLong());
future.complete(resultAndOffset.response());
} else {
future.completeExceptionally(
handleEventException(name, startProcessingTimeNs, exception));
}
}
@Override
public String toString() {
return name + "(" + System.identityHashCode(this) + ")";
}
}
private <T> CompletableFuture<T> prependWriteEvent(String name,
ControllerWriteOperation<T> op) {
ControllerWriteEvent<T> event = new ControllerWriteEvent<>(name, op);
queue.prepend(event);
return event.future();
}
private <T> CompletableFuture<T> appendWriteEvent(String name,
OptionalLong deadlineNs,
ControllerWriteOperation<T> op) {
ControllerWriteEvent<T> event = new ControllerWriteEvent<>(name, op);
if (deadlineNs.isPresent()) {
queue.appendWithDeadline(deadlineNs.getAsLong(), event);
} else {
queue.append(event);
}
return event.future();
}
class QuorumMetaLogListener implements RaftClient.Listener<ApiMessageAndVersion> {
@Override
public void handleCommit(BatchReader<ApiMessageAndVersion> reader) {
appendRaftEvent("handleCommit[baseOffset=" + reader.baseOffset() + "]", () -> {
try {
maybeCompleteAuthorizerInitialLoad();
long processedRecordsSize = 0;
while (reader.hasNext()) {
Batch<ApiMessageAndVersion> batch = reader.next();
long offset = batch.lastOffset();
int epoch = batch.epoch();
List<ApiMessageAndVersion> messages = batch.records();
if (isActiveController()) {
// If the controller is active, the records were already replayed,
// so we don't need to do it here.
log.debug("Completing purgatory items up to offset {} and epoch {}.", offset, epoch);
// Complete any events in the purgatory that were waiting for this offset.
purgatory.completeUpTo(offset);
// Delete all the in-memory snapshots that we no longer need.
// If we are writing a new snapshot, then we need to keep that around;
// otherwise, we should delete up to the current committed offset.
snapshotRegistry.deleteSnapshotsUpTo(
snapshotGeneratorManager.snapshotLastOffsetFromLog().orElse(offset));
} else {
// If the controller is a standby, replay the records that were
// created by the active controller.
if (log.isDebugEnabled()) {
if (log.isTraceEnabled()) {
log.trace("Replaying commits from the active node up to " +
"offset {} and epoch {}: {}.", offset, epoch, messages.stream()
.map(ApiMessageAndVersion::toString)
.collect(Collectors.joining(", ")));
} else {
log.debug("Replaying commits from the active node up to " +
"offset {} and epoch {}.", offset, epoch);
}
}
for (ApiMessageAndVersion messageAndVersion : messages) {
replay(messageAndVersion.message(), Optional.empty(), offset);
}
}
updateLastCommittedState(offset, epoch, batch.appendTimestamp());
processedRecordsSize += batch.sizeInBytes();
}
maybeGenerateSnapshot(processedRecordsSize);
} finally {
reader.close();
}
});
}
@Override
public void handleSnapshot(SnapshotReader<ApiMessageAndVersion> reader) {
appendRaftEvent(String.format("handleSnapshot[snapshotId=%s]", reader.snapshotId()), () -> {
try {
if (isActiveController()) {
throw new IllegalStateException(
String.format(
"Asked to load snapshot (%s) when it is the active controller (%d)",
reader.snapshotId(),
curClaimEpoch
)
);
}
log.info("Starting to replay snapshot ({}), from last commit offset ({}) and epoch ({})",
reader.snapshotId(), lastCommittedOffset, lastCommittedEpoch);
resetState();
while (reader.hasNext()) {
Batch<ApiMessageAndVersion> batch = reader.next();
long offset = batch.lastOffset();
List<ApiMessageAndVersion> messages = batch.records();
if (log.isDebugEnabled()) {
if (log.isTraceEnabled()) {
log.trace(
"Replaying snapshot ({}) batch with last offset of {}: {}",
reader.snapshotId(),
offset,
messages
.stream()
.map(ApiMessageAndVersion::toString)
.collect(Collectors.joining(", "))
);
} else {
log.debug(
"Replaying snapshot ({}) batch with last offset of {}",
reader.snapshotId(),
offset
);
}
}
for (ApiMessageAndVersion messageAndVersion : messages) {
replay(messageAndVersion.message(), Optional.of(reader.snapshotId()), offset);
}
}
updateLastCommittedState(
reader.lastContainedLogOffset(),
reader.lastContainedLogEpoch(),
reader.lastContainedLogTimestamp()
);
snapshotRegistry.getOrCreateSnapshot(lastCommittedOffset);
authorizer.ifPresent(a -> a.loadSnapshot(aclControlManager.idToAcl()));
} finally {
reader.close();
}
});
}
@Override
public void handleLeaderChange(LeaderAndEpoch newLeader) {
if (newLeader.isLeader(nodeId)) {
final int newEpoch = newLeader.epoch();
appendRaftEvent("handleLeaderChange[" + newEpoch + "]", () -> {
int curEpoch = curClaimEpoch;
if (curEpoch != -1) {
throw new RuntimeException("Tried to claim controller epoch " +
newEpoch + ", but we never renounced controller epoch " +
curEpoch);
}
curClaimEpoch = newEpoch;
controllerMetrics.setActive(true);
updateWriteOffset(lastCommittedOffset);
clusterControl.activate();
// Check if we need to bootstrap metadata into the log. This must happen before we can
// write any other records to the log since we need the metadata.version to determine the correct
// record version
final MetadataVersion metadataVersion;
if (!featureControl.sawMetadataVersion()) {
final CompletableFuture<Map<String, ApiError>> future;
if (!bootstrapMetadata.metadataVersion().isKRaftSupported()) {
metadataVersion = MetadataVersion.MINIMUM_KRAFT_VERSION;
future = new CompletableFuture<>();
future.completeExceptionally(
new IllegalStateException("Cannot become leader without a KRaft supported version. " +
"Got " + bootstrapMetadata.metadataVersion()));
} else {
metadataVersion = bootstrapMetadata.metadataVersion();
// This call is here instead of inside the appendWriteEvent for testing purposes.
final List<ApiMessageAndVersion> bootstrapRecords = bootstrapMetadata.records();
// We prepend the bootstrap event in order to ensure the bootstrap metadata is written before
// any external controller write events are processed.
future = prependWriteEvent("bootstrapMetadata", () -> {
if (metadataVersion.isAtLeast(MetadataVersion.IBP_3_3_IV0)) {
log.info("Initializing metadata.version to {}", metadataVersion.featureLevel());
} else {
log.info("Upgrading KRaft cluster and initializing metadata.version to {}",
metadataVersion.featureLevel());
}
return ControllerResult.atomicOf(bootstrapRecords, null);
});
}
future.whenComplete((result, exception) -> {
if (exception != null) {
log.error("Failed to bootstrap metadata.", exception);
appendRaftEvent("bootstrapMetadata[" + curClaimEpoch + "]", () -> {
log.warn("Renouncing the leadership at oldEpoch {} since we could not bootstrap " +
"metadata. Reverting to last committed offset {}.",
curClaimEpoch, lastCommittedOffset);
renounce();
});
}
});
} else {
metadataVersion = featureControl.metadataVersion();
}
log.info(
"Becoming the active controller at epoch {}, committed offset {}, committed epoch {}, and metadata.version {}",
newEpoch, lastCommittedOffset, lastCommittedEpoch, metadataVersion.featureLevel()
);
// Before switching to active, create an in-memory snapshot at the last committed offset. This is
// required because the active controller assumes that there is always an in-memory snapshot at the
// last committed offset.
snapshotRegistry.getOrCreateSnapshot(lastCommittedOffset);
// When becoming the active controller, schedule a leader rebalance if there are any topic partition
// with leader that is not the preferred leader.
maybeScheduleNextBalancePartitionLeaders();
// When becoming leader schedule periodic write of the no op record
maybeScheduleNextWriteNoOpRecord();
});
} else if (isActiveController()) {
appendRaftEvent("handleRenounce[" + curClaimEpoch + "]", () -> {