/
KRaftMigrationDriver.java
899 lines (812 loc) · 40 KB
/
KRaftMigrationDriver.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
/*
* 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.metadata.migration;
import org.apache.kafka.common.utils.ExponentialBackoff;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.controller.QuorumFeatures;
import org.apache.kafka.controller.metrics.QuorumControllerMetrics;
import org.apache.kafka.image.MetadataDelta;
import org.apache.kafka.image.MetadataImage;
import org.apache.kafka.image.MetadataProvenance;
import org.apache.kafka.image.loader.LoaderManifest;
import org.apache.kafka.image.loader.LoaderManifestType;
import org.apache.kafka.image.publisher.MetadataPublisher;
import org.apache.kafka.metadata.BrokerRegistration;
import org.apache.kafka.metadata.KafkaConfigSchema;
import org.apache.kafka.metadata.util.RecordRedactor;
import org.apache.kafka.queue.EventQueue;
import org.apache.kafka.queue.KafkaEventQueue;
import org.apache.kafka.raft.LeaderAndEpoch;
import org.apache.kafka.raft.OffsetAndEpoch;
import org.apache.kafka.server.fault.FaultHandler;
import org.apache.kafka.server.util.Deadline;
import org.apache.kafka.server.util.FutureUtils;
import org.slf4j.Logger;
import java.util.EnumSet;
import java.util.HashSet;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.function.BiConsumer;
import java.util.function.Consumer;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static java.util.concurrent.TimeUnit.NANOSECONDS;
/**
* This class orchestrates and manages the state related to a ZK to KRaft migration. A single event thread is used to
* serialize events coming from various threads and listeners.
*/
public class KRaftMigrationDriver implements MetadataPublisher {
private static class PollTimeSupplier {
private final ExponentialBackoff pollBackoff;
private long pollCount;
PollTimeSupplier() {
this.pollCount = 0;
this.pollBackoff = new ExponentialBackoff(100, 2, 60000, 0.02);
}
void reset() {
this.pollCount = 0;
}
public long nextPollTimeMs() {
long next = pollBackoff.backoff(pollCount);
pollCount++;
return next;
}
}
private final static Consumer<Throwable> NO_OP_HANDLER = ex -> { };
/**
* When waiting for the metadata layer to commit batches, we block the migration driver thread for this
* amount of time. A large value is selected to avoid timeouts in the common case, but prevent us from
* blocking indefinitely.
*/
private final static int METADATA_COMMIT_MAX_WAIT_MS = 300_000;
private final Time time;
private final Logger log;
private final int nodeId;
private final MigrationClient zkMigrationClient;
private final KRaftMigrationZkWriter zkMetadataWriter;
private final LegacyPropagator propagator;
private final ZkRecordConsumer zkRecordConsumer;
private final KafkaEventQueue eventQueue;
private final PollTimeSupplier pollTimeSupplier;
private final QuorumControllerMetrics controllerMetrics;
private final FaultHandler faultHandler;
private final QuorumFeatures quorumFeatures;
private final RecordRedactor recordRedactor;
/**
* A callback for when the migration state has been recovered from ZK. This is used to delay the installation of this
* MetadataPublisher with MetadataLoader.
*/
private final Consumer<MetadataPublisher> initialZkLoadHandler;
private volatile MigrationDriverState migrationState;
private volatile ZkMigrationLeadershipState migrationLeadershipState;
private volatile MetadataImage image;
private volatile boolean firstPublish;
KRaftMigrationDriver(
int nodeId,
ZkRecordConsumer zkRecordConsumer,
MigrationClient zkMigrationClient,
LegacyPropagator propagator,
Consumer<MetadataPublisher> initialZkLoadHandler,
FaultHandler faultHandler,
QuorumFeatures quorumFeatures,
KafkaConfigSchema configSchema,
QuorumControllerMetrics controllerMetrics,
Time time
) {
this.nodeId = nodeId;
this.zkRecordConsumer = zkRecordConsumer;
this.zkMigrationClient = zkMigrationClient;
this.propagator = propagator;
this.time = time;
LogContext logContext = new LogContext("[KRaftMigrationDriver id=" + nodeId + "] ");
this.controllerMetrics = controllerMetrics;
this.log = logContext.logger(KRaftMigrationDriver.class);
this.migrationState = MigrationDriverState.UNINITIALIZED;
this.migrationLeadershipState = ZkMigrationLeadershipState.EMPTY;
this.eventQueue = new KafkaEventQueue(Time.SYSTEM, logContext, "controller-" + nodeId + "-migration-driver-");
this.pollTimeSupplier = new PollTimeSupplier();
this.image = MetadataImage.EMPTY;
this.firstPublish = false;
this.initialZkLoadHandler = initialZkLoadHandler;
this.faultHandler = faultHandler;
this.quorumFeatures = quorumFeatures;
this.zkMetadataWriter = new KRaftMigrationZkWriter(zkMigrationClient);
this.recordRedactor = new RecordRedactor(configSchema);
}
public static Builder newBuilder() {
return new Builder();
}
public void start() {
eventQueue.prepend(new PollEvent());
}
// Visible for testing
public CompletableFuture<MigrationDriverState> migrationState() {
CompletableFuture<MigrationDriverState> stateFuture = new CompletableFuture<>();
eventQueue.append(() -> stateFuture.complete(migrationState));
return stateFuture;
}
private void recoverMigrationStateFromZK() {
applyMigrationOperation("Recovering migration state from ZK", zkMigrationClient::getOrCreateMigrationRecoveryState);
String maybeDone = migrationLeadershipState.initialZkMigrationComplete() ? "done" : "not done";
log.info("Initial migration of ZK metadata is {}.", maybeDone);
// Once we've recovered the migration state from ZK, install this class as a metadata publisher
// by calling the initialZkLoadHandler.
initialZkLoadHandler.accept(this);
// Transition to INACTIVE state and wait for leadership events.
transitionTo(MigrationDriverState.INACTIVE);
}
private boolean isControllerQuorumReadyForMigration() {
Optional<String> notReadyMsg = this.quorumFeatures.reasonAllControllersZkMigrationNotReady();
if (notReadyMsg.isPresent()) {
log.warn("Still waiting for all controller nodes ready to begin the migration. Not ready due to:" + notReadyMsg.get());
return false;
}
return true;
}
private boolean imageDoesNotContainAllBrokers(MetadataImage image, Set<Integer> brokerIds) {
for (BrokerRegistration broker : image.cluster().brokers().values()) {
if (broker.isMigratingZkBroker()) {
brokerIds.remove(broker.id());
}
}
return !brokerIds.isEmpty();
}
private boolean areZkBrokersReadyForMigration() {
if (!firstPublish) {
log.info("Waiting for initial metadata publish before checking if Zk brokers are registered.");
return false;
}
if (image.cluster().isEmpty()) {
// This primarily happens in system tests when we are starting a new ZK cluster and KRaft quorum
// around the same time.
log.info("No brokers are known to KRaft, waiting for brokers to register.");
return false;
}
Set<Integer> zkBrokerRegistrations = zkMigrationClient.readBrokerIds();
if (zkBrokerRegistrations.isEmpty()) {
// Similar to the above empty check
log.info("No brokers are registered in ZK, waiting for brokers to register.");
return false;
}
if (imageDoesNotContainAllBrokers(image, zkBrokerRegistrations)) {
log.info("Still waiting for ZK brokers {} to register with KRaft.", zkBrokerRegistrations);
return false;
}
// Once all of those are found, check the topic assignments. This is much more expensive than listing /brokers
Set<Integer> zkBrokersWithAssignments = new HashSet<>();
zkMigrationClient.topicClient().iterateTopics(
EnumSet.of(TopicMigrationClient.TopicVisitorInterest.TOPICS),
(topicName, topicId, assignments) -> assignments.values().forEach(zkBrokersWithAssignments::addAll)
);
if (imageDoesNotContainAllBrokers(image, zkBrokersWithAssignments)) {
log.info("Still waiting for ZK brokers {} found in metadata to register with KRaft.", zkBrokersWithAssignments);
return false;
}
return true;
}
/**
* Apply a function which transforms our internal migration state.
*
* @param name A descriptive name of the function that is being applied
* @param migrationOp A function which performs some migration operations and possibly transforms our internal state
*/
private void applyMigrationOperation(String name, KRaftMigrationOperation migrationOp) {
ZkMigrationLeadershipState beforeState = this.migrationLeadershipState;
ZkMigrationLeadershipState afterState = migrationOp.apply(beforeState);
if (afterState.loggableChangeSinceState(beforeState)) {
log.info("{}. Transitioned migration state from {} to {}", name, beforeState, afterState);
} else if (afterState.equals(beforeState)) {
log.trace("{}. Kept migration state as {}", name, afterState);
} else {
log.trace("{}. Transitioned migration state from {} to {}", name, beforeState, afterState);
}
this.migrationLeadershipState = afterState;
}
private boolean isValidStateChange(MigrationDriverState newState) {
if (migrationState == newState)
return true;
if (newState == MigrationDriverState.UNINITIALIZED) {
return false;
}
switch (migrationState) {
case UNINITIALIZED:
case DUAL_WRITE:
return newState == MigrationDriverState.INACTIVE;
case INACTIVE:
return newState == MigrationDriverState.WAIT_FOR_CONTROLLER_QUORUM;
case WAIT_FOR_CONTROLLER_QUORUM:
return
newState == MigrationDriverState.INACTIVE ||
newState == MigrationDriverState.BECOME_CONTROLLER ||
newState == MigrationDriverState.WAIT_FOR_BROKERS;
case WAIT_FOR_BROKERS:
return
newState == MigrationDriverState.INACTIVE ||
newState == MigrationDriverState.BECOME_CONTROLLER;
case BECOME_CONTROLLER:
return
newState == MigrationDriverState.INACTIVE ||
newState == MigrationDriverState.ZK_MIGRATION ||
newState == MigrationDriverState.SYNC_KRAFT_TO_ZK;
case ZK_MIGRATION:
return
newState == MigrationDriverState.INACTIVE ||
newState == MigrationDriverState.SYNC_KRAFT_TO_ZK;
case SYNC_KRAFT_TO_ZK:
return
newState == MigrationDriverState.INACTIVE ||
newState == MigrationDriverState.KRAFT_CONTROLLER_TO_BROKER_COMM;
case KRAFT_CONTROLLER_TO_BROKER_COMM:
return
newState == MigrationDriverState.INACTIVE ||
newState == MigrationDriverState.DUAL_WRITE;
default:
log.error("Migration driver trying to transition from an unknown state {}", migrationState);
return false;
}
}
private boolean checkDriverState(MigrationDriverState expectedState) {
if (migrationState.equals(expectedState)) {
return true;
} else {
log.info("Expected driver state {} but found {}. Not running this event {}.",
expectedState, migrationState, this.getClass().getSimpleName());
return false;
}
}
private void transitionTo(MigrationDriverState newState) {
if (!isValidStateChange(newState)) {
throw new IllegalStateException(
String.format("Invalid transition in migration driver from %s to %s", migrationState, newState));
}
if (newState != migrationState) {
log.debug("{} transitioning from {} to {} state", nodeId, migrationState, newState);
pollTimeSupplier.reset();
wakeup();
} else {
log.trace("{} transitioning from {} to {} state", nodeId, migrationState, newState);
}
migrationState = newState;
}
private void wakeup() {
eventQueue.append(new PollEvent());
}
// MetadataPublisher methods
@Override
public String name() {
return "KRaftMigrationDriver";
}
@Override
public void onControllerChange(LeaderAndEpoch newLeaderAndEpoch) {
eventQueue.append(new KRaftLeaderEvent(newLeaderAndEpoch));
}
@Override
public void onMetadataUpdate(
MetadataDelta delta,
MetadataImage newImage,
LoaderManifest manifest
) {
enqueueMetadataChangeEvent(delta,
newImage,
manifest.provenance(),
manifest.type() == LoaderManifestType.SNAPSHOT,
NO_OP_HANDLER);
}
@Override
public void close() throws InterruptedException {
eventQueue.beginShutdown("KRaftMigrationDriver#shutdown");
log.debug("Shutting down KRaftMigrationDriver");
eventQueue.close();
}
/**
* Construct and enqueue a {@link MetadataChangeEvent} with a given completion handler. In production use cases,
* this handler is a no-op. This method exists so that we can add additional logic in our unit tests to wait for the
* enqueued event to finish executing.
*/
void enqueueMetadataChangeEvent(
MetadataDelta delta,
MetadataImage newImage,
MetadataProvenance provenance,
boolean isSnapshot,
Consumer<Throwable> completionHandler
) {
MetadataChangeEvent metadataChangeEvent = new MetadataChangeEvent(
delta,
newImage,
provenance,
isSnapshot,
completionHandler
);
eventQueue.append(metadataChangeEvent);
}
// Events handled by Migration Driver.
abstract class MigrationEvent implements EventQueue.Event {
@SuppressWarnings("ThrowableNotThrown")
@Override
public void handleException(Throwable e) {
if (e instanceof MigrationClientAuthException) {
KRaftMigrationDriver.this.faultHandler.handleFault("Encountered ZooKeeper authentication in " + this, e);
} else if (e instanceof MigrationClientException) {
log.info(String.format("Encountered ZooKeeper error during event %s. Will retry.", this), e.getCause());
} else if (e instanceof RejectedExecutionException) {
log.debug("Not processing {} because the event queue is closed.", this);
} else {
KRaftMigrationDriver.this.faultHandler.handleFault("Unhandled error in " + this, e);
}
}
@Override
public String toString() {
return this.getClass().getSimpleName();
}
}
/**
* An event generated by a call to {@link MetadataPublisher#onControllerChange}. This will not be called until
* this class is registered with {@link org.apache.kafka.image.loader.MetadataLoader}. The registration happens
* after the migration state is loaded from ZooKeeper in {@link #recoverMigrationStateFromZK}.
*/
class KRaftLeaderEvent extends MigrationEvent {
private final LeaderAndEpoch leaderAndEpoch;
KRaftLeaderEvent(LeaderAndEpoch leaderAndEpoch) {
this.leaderAndEpoch = leaderAndEpoch;
}
@Override
public void run() throws Exception {
// We can either be the active controller or just resigned from being the controller.
boolean isActive = leaderAndEpoch.isLeader(KRaftMigrationDriver.this.nodeId);
if (!isActive) {
applyMigrationOperation("Became inactive migration driver", state ->
state.withNewKRaftController(
leaderAndEpoch.leaderId().orElse(ZkMigrationLeadershipState.EMPTY.kraftControllerId()),
leaderAndEpoch.epoch())
);
transitionTo(MigrationDriverState.INACTIVE);
} else {
// Load the existing migration state and apply the new KRaft state
applyMigrationOperation("Became active migration driver", state -> {
ZkMigrationLeadershipState recoveredState = zkMigrationClient.getOrCreateMigrationRecoveryState(state);
return recoveredState.withNewKRaftController(nodeId, leaderAndEpoch.epoch());
});
// Before becoming the controller fo ZkBrokers, we need to make sure the
// Controller Quorum can handle migration.
transitionTo(MigrationDriverState.WAIT_FOR_CONTROLLER_QUORUM);
}
}
}
class MetadataChangeEvent extends MigrationEvent {
private final MetadataDelta delta;
private final MetadataImage image;
private final MetadataProvenance provenance;
private final boolean isSnapshot;
private final Consumer<Throwable> completionHandler;
MetadataChangeEvent(
MetadataDelta delta,
MetadataImage image,
MetadataProvenance provenance,
boolean isSnapshot,
Consumer<Throwable> completionHandler
) {
this.delta = delta;
this.image = image;
this.provenance = provenance;
this.isSnapshot = isSnapshot;
this.completionHandler = completionHandler;
}
@Override
public void run() throws Exception {
if (!firstPublish && image.isEmpty()) {
// KAFKA-15389 When first loading from an empty log, MetadataLoader can publish an empty image
log.debug("Encountered an empty MetadataImage while waiting for the first image to be published. " +
"Ignoring this image since it either does not include bootstrap records or it is a valid " +
"image for an older unsupported metadata version.");
completionHandler.accept(null);
return;
}
KRaftMigrationDriver.this.firstPublish = true;
MetadataImage prevImage = KRaftMigrationDriver.this.image;
KRaftMigrationDriver.this.image = image;
String metadataType = isSnapshot ? "snapshot" : "delta";
if (migrationState.equals(MigrationDriverState.INACTIVE)) {
// No need to log anything if this node is not the active controller
completionHandler.accept(null);
return;
}
if (!migrationState.allowDualWrite()) {
log.trace("Received metadata {}, but the controller is not in dual-write " +
"mode. Ignoring the change to be replicated to Zookeeper", metadataType);
completionHandler.accept(null);
// If the driver is active and dual-write is not yet enabled, then the migration has not yet begun.
// Only wake up the thread if the broker registrations have changed
if (delta.clusterDelta() != null) {
wakeup();
}
return;
}
if (image.highestOffsetAndEpoch().compareTo(migrationLeadershipState.offsetAndEpoch()) < 0) {
log.info("Ignoring {} {} which contains metadata that has already been written to ZK.", metadataType, provenance);
completionHandler.accept(null);
return;
}
Map<String, Integer> dualWriteCounts = new TreeMap<>();
long startTime = time.nanoseconds();
if (isSnapshot) {
zkMetadataWriter.handleSnapshot(image, countingOperationConsumer(
dualWriteCounts, KRaftMigrationDriver.this::applyMigrationOperation));
controllerMetrics.updateZkWriteSnapshotTimeMs(NANOSECONDS.toMillis(time.nanoseconds() - startTime));
} else {
if (zkMetadataWriter.handleDelta(prevImage, image, delta, countingOperationConsumer(
dualWriteCounts, KRaftMigrationDriver.this::applyMigrationOperation))) {
// Only record delta write time if we changed something. Otherwise, no-op records will skew timings.
controllerMetrics.updateZkWriteDeltaTimeMs(NANOSECONDS.toMillis(time.nanoseconds() - startTime));
}
}
if (dualWriteCounts.isEmpty()) {
log.trace("Did not make any ZK writes when handling KRaft {}", isSnapshot ? "snapshot" : "delta");
} else {
log.debug("Made the following ZK writes when handling KRaft {}: {}", isSnapshot ? "snapshot" : "delta", dualWriteCounts);
}
// Persist the offset of the metadata that was written to ZK
ZkMigrationLeadershipState zkStateAfterDualWrite = migrationLeadershipState.withKRaftMetadataOffsetAndEpoch(
image.highestOffsetAndEpoch().offset(), image.highestOffsetAndEpoch().epoch());
//update the dual write offset metric
controllerMetrics.updateDualWriteOffset(image.highestOffsetAndEpoch().offset());
applyMigrationOperation("Updating ZK migration state after " + metadataType,
state -> zkMigrationClient.setMigrationRecoveryState(zkStateAfterDualWrite));
// TODO: Unhappy path: Probably relinquish leadership and let new controller
// retry the write?
if (delta.topicsDelta() != null || delta.clusterDelta() != null) {
log.trace("Sending RPCs to brokers for metadata {}.", metadataType);
propagator.sendRPCsToBrokersFromMetadataDelta(delta, image, migrationLeadershipState.zkControllerEpoch());
} else {
log.trace("Not sending RPCs to brokers for metadata {} since no relevant metadata has changed", metadataType);
}
completionHandler.accept(null);
}
@Override
public void handleException(Throwable e) {
completionHandler.accept(e);
super.handleException(e);
}
}
class WaitForControllerQuorumEvent extends MigrationEvent {
@Override
public void run() throws Exception {
if (checkDriverState(MigrationDriverState.WAIT_FOR_CONTROLLER_QUORUM)) {
if (!firstPublish) {
log.trace("Waiting until we have received metadata before proceeding with migration");
return;
}
ZkMigrationState zkMigrationState = image.features().zkMigrationState();
switch (zkMigrationState) {
case NONE:
// This error message is used in zookeeper_migration_test.py::TestMigration.test_pre_migration_mode_3_4
log.error("The controller's ZkMigrationState is NONE which means this cluster should not be migrated from ZooKeeper. " +
"This controller should not be configured with 'zookeeper.metadata.migration.enable' set to true. " +
"Will not proceed with a migration.");
transitionTo(MigrationDriverState.INACTIVE);
break;
case PRE_MIGRATION:
if (isControllerQuorumReadyForMigration()) {
// Base case when starting the migration
log.info("Controller Quorum is ready for Zk to KRaft migration. Now waiting for ZK brokers.");
transitionTo(MigrationDriverState.WAIT_FOR_BROKERS);
}
break;
case MIGRATION:
if (!migrationLeadershipState.initialZkMigrationComplete()) {
log.error("KRaft controller indicates an active migration, but the ZK state does not.");
transitionTo(MigrationDriverState.INACTIVE);
} else {
// Base case when rebooting a controller during migration
log.info("Migration is in already progress, not waiting on ZK brokers.");
transitionTo(MigrationDriverState.BECOME_CONTROLLER);
}
break;
case POST_MIGRATION:
log.error("KRaft controller indicates a completed migration, but the migration driver is somehow active.");
transitionTo(MigrationDriverState.INACTIVE);
break;
default:
throw new IllegalStateException("Unsupported ZkMigrationState " + zkMigrationState);
}
}
}
}
class WaitForZkBrokersEvent extends MigrationEvent {
@Override
public void run() throws Exception {
if (checkDriverState(MigrationDriverState.WAIT_FOR_BROKERS)) {
if (areZkBrokersReadyForMigration()) {
log.info("Zk brokers are registered and ready for migration");
transitionTo(MigrationDriverState.BECOME_CONTROLLER);
}
}
}
}
class BecomeZkControllerEvent extends MigrationEvent {
@Override
public void run() throws Exception {
if (checkDriverState(MigrationDriverState.BECOME_CONTROLLER)) {
applyMigrationOperation("Claiming ZK controller leadership", zkMigrationClient::claimControllerLeadership);
if (migrationLeadershipState.zkControllerEpochZkVersion() == -1) {
log.info("Unable to claim leadership, will retry until we learn of a different KRaft leader");
} else {
if (!migrationLeadershipState.initialZkMigrationComplete()) {
transitionTo(MigrationDriverState.ZK_MIGRATION);
} else {
transitionTo(MigrationDriverState.SYNC_KRAFT_TO_ZK);
}
}
}
}
}
class MigrateMetadataEvent extends MigrationEvent {
@Override
public void run() throws Exception {
if (!checkDriverState(MigrationDriverState.ZK_MIGRATION)) {
return;
}
Set<Integer> brokersInMetadata = new HashSet<>();
log.info("Starting ZK migration");
MigrationManifest.Builder manifestBuilder = MigrationManifest.newBuilder(time);
try {
FutureUtils.waitWithLogging(KRaftMigrationDriver.this.log, "",
"the metadata layer to begin the migration transaction",
zkRecordConsumer.beginMigration(),
Deadline.fromDelay(time, METADATA_COMMIT_MAX_WAIT_MS, TimeUnit.MILLISECONDS), time);
} catch (Throwable t) {
log.error("Could not start the migration", t);
super.handleException(t);
}
try {
zkMigrationClient.readAllMetadata(batch -> {
try {
log.info("Migrating {} records from ZK", batch.size());
if (log.isTraceEnabled()) {
batch.forEach(apiMessageAndVersion ->
log.trace(recordRedactor.toLoggableString(apiMessageAndVersion.message())));
}
CompletableFuture<?> future = zkRecordConsumer.acceptBatch(batch);
FutureUtils.waitWithLogging(KRaftMigrationDriver.this.log, "",
"the metadata layer to commit migration record batch",
future, Deadline.fromDelay(time, METADATA_COMMIT_MAX_WAIT_MS, TimeUnit.MILLISECONDS), time);
manifestBuilder.acceptBatch(batch);
} catch (Throwable e) {
// This will cause readAllMetadata to throw since this batch consumer is called directly from readAllMetadata
throw new RuntimeException(e);
}
}, brokersInMetadata::add);
CompletableFuture<OffsetAndEpoch> completeMigrationFuture = zkRecordConsumer.completeMigration();
OffsetAndEpoch offsetAndEpochAfterMigration = FutureUtils.waitWithLogging(
KRaftMigrationDriver.this.log, "",
"the metadata layer to complete the migration",
completeMigrationFuture, Deadline.fromDelay(time, METADATA_COMMIT_MAX_WAIT_MS, TimeUnit.MILLISECONDS), time);
MigrationManifest manifest = manifestBuilder.build();
log.info("Completed migration of metadata from ZooKeeper to KRaft. {}. " +
"The current metadata offset is now {} with an epoch of {}. Saw {} brokers in the " +
"migrated metadata {}.",
manifest,
offsetAndEpochAfterMigration.offset(),
offsetAndEpochAfterMigration.epoch(),
brokersInMetadata.size(),
brokersInMetadata);
ZkMigrationLeadershipState newState = migrationLeadershipState.withKRaftMetadataOffsetAndEpoch(
offsetAndEpochAfterMigration.offset(),
offsetAndEpochAfterMigration.epoch());
applyMigrationOperation("Finished initial migration of ZK metadata to KRaft", state -> zkMigrationClient.setMigrationRecoveryState(newState));
// Even though we just migrated everything, we still pass through the SYNC_KRAFT_TO_ZK state. This
// accomplishes two things: ensuring we have consistent metadata state between KRaft and ZK, and
// exercising the snapshot handling code in KRaftMigrationZkWriter.
transitionTo(MigrationDriverState.SYNC_KRAFT_TO_ZK);
} catch (Throwable t) {
MigrationManifest partialManifest = manifestBuilder.build();
log.error("Aborting the metadata migration from ZooKeeper to KRaft. {}.", partialManifest);
zkRecordConsumer.abortMigration(); // This terminates the controller via fatal fault handler
super.handleException(t);
}
}
}
class SyncKRaftMetadataEvent extends MigrationEvent {
@Override
public void run() throws Exception {
if (checkDriverState(MigrationDriverState.SYNC_KRAFT_TO_ZK)) {
log.info("Performing a full metadata sync from KRaft to ZK.");
Map<String, Integer> dualWriteCounts = new TreeMap<>();
long startTime = time.nanoseconds();
zkMetadataWriter.handleSnapshot(image, countingOperationConsumer(
dualWriteCounts, KRaftMigrationDriver.this::applyMigrationOperation));
long endTime = time.nanoseconds();
controllerMetrics.updateZkWriteSnapshotTimeMs(NANOSECONDS.toMillis(startTime - endTime));
log.info("Made the following ZK writes when reconciling with KRaft state: {}", dualWriteCounts);
transitionTo(MigrationDriverState.KRAFT_CONTROLLER_TO_BROKER_COMM);
}
}
}
class SendRPCsToBrokersEvent extends MigrationEvent {
@Override
public void run() throws Exception {
// Ignore sending RPCs to the brokers since we're no longer in the state.
if (checkDriverState(MigrationDriverState.KRAFT_CONTROLLER_TO_BROKER_COMM)) {
if (image.highestOffsetAndEpoch().compareTo(migrationLeadershipState.offsetAndEpoch()) >= 0) {
log.info("Sending RPCs to broker before moving to dual-write mode using " +
"at offset and epoch {}", image.highestOffsetAndEpoch());
propagator.sendRPCsToBrokersFromMetadataImage(image, migrationLeadershipState.zkControllerEpoch());
// Migration leadership state doesn't change since we're not doing any Zk writes.
transitionTo(MigrationDriverState.DUAL_WRITE);
} else {
log.info("Not sending metadata RPCs with current metadata image since does not contain the offset " +
"that was last written to ZK during the migration. Image offset {} is less than migration " +
"leadership state offset {}", image.highestOffsetAndEpoch(), migrationLeadershipState.offsetAndEpoch());
}
}
}
}
class PollEvent extends MigrationEvent {
@Override
public void run() throws Exception {
switch (migrationState) {
case UNINITIALIZED:
recoverMigrationStateFromZK();
break;
case INACTIVE:
// Nothing to do when the driver is inactive. We must wait until a KRaftLeaderEvent
// tells informs us that we are the leader.
break;
case WAIT_FOR_CONTROLLER_QUORUM:
eventQueue.append(new WaitForControllerQuorumEvent());
break;
case WAIT_FOR_BROKERS:
eventQueue.append(new WaitForZkBrokersEvent());
break;
case BECOME_CONTROLLER:
eventQueue.append(new BecomeZkControllerEvent());
break;
case ZK_MIGRATION:
eventQueue.append(new MigrateMetadataEvent());
break;
case SYNC_KRAFT_TO_ZK:
eventQueue.append(new SyncKRaftMetadataEvent());
break;
case KRAFT_CONTROLLER_TO_BROKER_COMM:
eventQueue.append(new SendRPCsToBrokersEvent());
break;
case DUAL_WRITE:
// Nothing to do in the PollEvent. If there's metadata change, we use
// MetadataChange event to drive the writes to Zookeeper.
break;
}
// Poll again after some time
long deadline = time.nanoseconds() + NANOSECONDS.convert(pollTimeSupplier.nextPollTimeMs(), MILLISECONDS);
eventQueue.scheduleDeferred(
"poll",
new EventQueue.DeadlineFunction(deadline),
new PollEvent());
}
}
static KRaftMigrationOperationConsumer countingOperationConsumer(
Map<String, Integer> dualWriteCounts,
BiConsumer<String, KRaftMigrationOperation> operationConsumer
) {
return (opType, logMsg, operation) -> {
dualWriteCounts.compute(opType, (key, value) -> {
if (value == null) {
return 1;
} else {
return value + 1;
}
});
operationConsumer.accept(logMsg, operation);
};
}
public static class Builder {
private Integer nodeId;
private ZkRecordConsumer zkRecordConsumer;
private MigrationClient zkMigrationClient;
private LegacyPropagator propagator;
private Consumer<MetadataPublisher> initialZkLoadHandler;
private FaultHandler faultHandler;
private QuorumFeatures quorumFeatures;
private KafkaConfigSchema configSchema;
private QuorumControllerMetrics controllerMetrics;
private Time time;
public Builder setNodeId(int nodeId) {
this.nodeId = nodeId;
return this;
}
public Builder setZkRecordConsumer(ZkRecordConsumer zkRecordConsumer) {
this.zkRecordConsumer = zkRecordConsumer;
return this;
}
public Builder setZkMigrationClient(MigrationClient zkMigrationClient) {
this.zkMigrationClient = zkMigrationClient;
return this;
}
public Builder setPropagator(LegacyPropagator propagator) {
this.propagator = propagator;
return this;
}
public Builder setInitialZkLoadHandler(Consumer<MetadataPublisher> initialZkLoadHandler) {
this.initialZkLoadHandler = initialZkLoadHandler;
return this;
}
public Builder setFaultHandler(FaultHandler faultHandler) {
this.faultHandler = faultHandler;
return this;
}
public Builder setQuorumFeatures(QuorumFeatures quorumFeatures) {
this.quorumFeatures = quorumFeatures;
return this;
}
public Builder setConfigSchema(KafkaConfigSchema configSchema) {
this.configSchema = configSchema;
return this;
}
public Builder setControllerMetrics(QuorumControllerMetrics controllerMetrics) {
this.controllerMetrics = controllerMetrics;
return this;
}
public Builder setTime(Time time) {
this.time = time;
return this;
}
public KRaftMigrationDriver build() {
if (nodeId == null) {
throw new IllegalStateException("You must specify the node ID of this controller.");
}
if (zkRecordConsumer == null) {
throw new IllegalStateException("You must specify the ZkRecordConsumer.");
}
if (zkMigrationClient == null) {
throw new IllegalStateException("You must specify the MigrationClient.");
}
if (propagator == null) {
throw new IllegalStateException("You must specify the MetadataPropagator.");
}
if (initialZkLoadHandler == null) {
throw new IllegalStateException("You must specify the initial ZK load callback.");
}
if (faultHandler == null) {
throw new IllegalStateException("You must specify the FaultHandler.");
}
if (configSchema == null) {
throw new IllegalStateException("You must specify the KafkaConfigSchema.");
}
if (controllerMetrics == null) {
throw new IllegalStateException("You must specify the QuorumControllerMetrics.");
}
if (time == null) {
throw new IllegalStateException("You must specify the Time.");
}
return new KRaftMigrationDriver(
nodeId,
zkRecordConsumer,
zkMigrationClient,
propagator,
initialZkLoadHandler,
faultHandler,
quorumFeatures,
configSchema,
controllerMetrics,
time
);
}
}
}