mirror of https://github.com/apache/kafka.git
MINOR: some ZK migration code cleanups.
Some minor improvements to the JavaDoc for ZkMigrationState. Rename MigrationState to MigrationDriverState to avoid confusion with ZkMigrationState. Remove ClusterImage#zkBrokers. This costs O(num_brokers) time to calculate, but is only ever used when in migration state. It should just be calculated in the migration code. (Additionally, the function ClusterImage.zkBrokers() returns something other than ClusterImage#zkBrokers, which is confusing.) Also remove ClusterDelta#liveZkBrokerIdChanges. This is only used in one place, and it's easy to calculate it there. In general we should avoid providing expensive accessors unless absolutely necessary. Expensive code should look expensive: if people want to iterate over all brokers, they can write a loop to do that rather than hiding it inside an accessor.
This commit is contained in:
parent
7322f4cd55
commit
6b89672b5e
|
@ -23,12 +23,14 @@ import kafka.server.KafkaConfig
|
|||
import org.apache.kafka.common.TopicPartition
|
||||
import org.apache.kafka.common.metrics.Metrics
|
||||
import org.apache.kafka.common.utils.Time
|
||||
import org.apache.kafka.image.{MetadataDelta, MetadataImage}
|
||||
import org.apache.kafka.image.{MetadataDelta, MetadataImage, TopicsImage}
|
||||
import org.apache.kafka.metadata.PartitionRegistration
|
||||
import org.apache.kafka.metadata.migration.LegacyPropagator
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
|
||||
import java.util
|
||||
import scala.jdk.CollectionConverters._
|
||||
import scala.compat.java8.OptionConverters._
|
||||
|
||||
class MigrationPropagator(
|
||||
nodeId: Int,
|
||||
|
@ -79,6 +81,18 @@ class MigrationPropagator(
|
|||
_image = image
|
||||
}
|
||||
|
||||
/**
|
||||
* A very expensive function that creates a map with an entry for every partition that exists, from
|
||||
* (topic name, partition index) to partition registration.
|
||||
*/
|
||||
def materializePartitions(topicsImage: TopicsImage): util.Map[TopicPartition, PartitionRegistration] = {
|
||||
val result = new util.HashMap[TopicPartition, PartitionRegistration]()
|
||||
topicsImage.topicsById().values().forEach(topic => {
|
||||
topic.partitions().forEach((key, value) => result.put(new TopicPartition(topic.name(), key), value));
|
||||
})
|
||||
result
|
||||
}
|
||||
|
||||
override def sendRPCsToBrokersFromMetadataDelta(delta: MetadataDelta, image: MetadataImage,
|
||||
zkControllerEpoch: Int): Unit = {
|
||||
publishMetadata(image)
|
||||
|
@ -87,15 +101,19 @@ class MigrationPropagator(
|
|||
delta.getOrCreateTopicsDelta()
|
||||
delta.getOrCreateClusterDelta()
|
||||
|
||||
val changedZkBrokers = delta.clusterDelta().liveZkBrokerIdChanges().asScala.map(_.toInt).toSet
|
||||
val zkBrokers = image.cluster().zkBrokers().keySet().asScala.map(_.toInt).toSet
|
||||
val changedZkBrokers = delta.clusterDelta().changedBrokers().values().asScala.map(_.asScala).filter {
|
||||
case None => false
|
||||
case Some(registration) => registration.isMigratingZkBroker && !registration.fenced()
|
||||
}.map(_.get.id()).toSet
|
||||
|
||||
val zkBrokers = image.cluster().brokers().values().asScala.filter(_.isMigratingZkBroker).map(_.id()).toSet
|
||||
val oldZkBrokers = zkBrokers -- changedZkBrokers
|
||||
val brokersChanged = !delta.clusterDelta().changedBrokers().isEmpty
|
||||
|
||||
// First send metadata about the live/dead brokers to all the zk brokers.
|
||||
if (changedZkBrokers.nonEmpty) {
|
||||
// Update new Zk brokers about all the metadata.
|
||||
requestBatch.addUpdateMetadataRequestForBrokers(changedZkBrokers.toSeq, image.topics().partitions().keySet().asScala)
|
||||
requestBatch.addUpdateMetadataRequestForBrokers(changedZkBrokers.toSeq, materializePartitions(image.topics()).asScala.keySet)
|
||||
}
|
||||
if (brokersChanged) {
|
||||
requestBatch.addUpdateMetadataRequestForBrokers(oldZkBrokers.toSeq)
|
||||
|
@ -107,7 +125,7 @@ class MigrationPropagator(
|
|||
// brokers based on the topic changes.
|
||||
if (changedZkBrokers.nonEmpty) {
|
||||
// For new the brokers, check if there are partition assignments and add LISR appropriately.
|
||||
image.topics().partitions().asScala.foreach { case (tp, partitionRegistration) =>
|
||||
materializePartitions(image.topics()).asScala.foreach { case (tp, partitionRegistration) =>
|
||||
val replicas = partitionRegistration.replicas.toSet
|
||||
val leaderIsrAndControllerEpochOpt = MigrationControllerChannelContext.partitionLeadershipInfo(image, tp)
|
||||
val newBrokersWithReplicas = replicas.intersect(changedZkBrokers)
|
||||
|
@ -181,12 +199,12 @@ class MigrationPropagator(
|
|||
override def sendRPCsToBrokersFromMetadataImage(image: MetadataImage, zkControllerEpoch: Int): Unit = {
|
||||
publishMetadata(image)
|
||||
|
||||
val zkBrokers = image.cluster().zkBrokers().keySet().asScala.map(_.toInt).toSeq
|
||||
val partitions = image.topics().partitions()
|
||||
val zkBrokers = image.cluster().brokers().values().asScala.filter(_.isMigratingZkBroker).map(_.id()).toSeq
|
||||
val partitions = materializePartitions(image.topics())
|
||||
// First send all the metadata before sending any other requests to make sure subsequent
|
||||
// requests are handled correctly.
|
||||
requestBatch.newBatch()
|
||||
requestBatch.addUpdateMetadataRequestForBrokers(zkBrokers, partitions.keySet().asScala)
|
||||
requestBatch.addUpdateMetadataRequestForBrokers(zkBrokers, partitions.keySet.asScala)
|
||||
requestBatch.sendRequestsToBrokers(zkControllerEpoch)
|
||||
|
||||
requestBatch.newBatch()
|
||||
|
|
|
@ -31,8 +31,6 @@ import java.util.HashMap;
|
|||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
|
||||
/**
|
||||
|
@ -58,17 +56,6 @@ public final class ClusterDelta {
|
|||
return image.broker(nodeId);
|
||||
}
|
||||
|
||||
public Set<Integer> liveZkBrokerIdChanges() {
|
||||
return changedBrokers
|
||||
.values()
|
||||
.stream()
|
||||
.filter(Optional::isPresent)
|
||||
.map(Optional::get)
|
||||
.filter(registration -> registration.isMigratingZkBroker() && !registration.fenced())
|
||||
.map(BrokerRegistration::id)
|
||||
.collect(Collectors.toSet());
|
||||
}
|
||||
|
||||
public void finishSnapshot() {
|
||||
for (Integer brokerId : image.brokers().keySet()) {
|
||||
if (!changedBrokers.containsKey(brokerId)) {
|
||||
|
|
|
@ -34,15 +34,9 @@ public final class ClusterImage {
|
|||
public static final ClusterImage EMPTY = new ClusterImage(Collections.emptyMap());
|
||||
|
||||
private final Map<Integer, BrokerRegistration> brokers;
|
||||
private final Map<Integer, BrokerRegistration> zkBrokers;
|
||||
|
||||
public ClusterImage(Map<Integer, BrokerRegistration> brokers) {
|
||||
this.brokers = Collections.unmodifiableMap(brokers);
|
||||
this.zkBrokers = Collections.unmodifiableMap(brokers
|
||||
.entrySet()
|
||||
.stream()
|
||||
.filter(entry -> entry.getValue().isMigratingZkBroker())
|
||||
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)));
|
||||
}
|
||||
|
||||
public boolean isEmpty() {
|
||||
|
@ -53,21 +47,10 @@ public final class ClusterImage {
|
|||
return brokers;
|
||||
}
|
||||
|
||||
public Map<Integer, BrokerRegistration> zkBrokers() {
|
||||
return Collections.unmodifiableMap(
|
||||
brokers
|
||||
.entrySet()
|
||||
.stream()
|
||||
.filter(x -> x.getValue().isMigratingZkBroker() && !x.getValue().fenced())
|
||||
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)));
|
||||
}
|
||||
|
||||
public BrokerRegistration broker(int nodeId) {
|
||||
return brokers.get(nodeId);
|
||||
}
|
||||
|
||||
|
||||
|
||||
public boolean containsBroker(int brokerId) {
|
||||
return brokers.containsKey(brokerId);
|
||||
}
|
||||
|
|
|
@ -17,7 +17,6 @@
|
|||
|
||||
package org.apache.kafka.image;
|
||||
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.image.writer.ImageWriter;
|
||||
import org.apache.kafka.image.writer.ImageWriterOptions;
|
||||
|
@ -25,7 +24,6 @@ import org.apache.kafka.metadata.PartitionRegistration;
|
|||
import org.apache.kafka.server.util.TranslatedValueMapView;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.stream.Collectors;
|
||||
|
@ -112,14 +110,6 @@ public final class TopicsImage {
|
|||
return new TranslatedValueMapView<>(topicsById, image -> image.name());
|
||||
}
|
||||
|
||||
public Map<TopicPartition, PartitionRegistration> partitions() {
|
||||
Map<TopicPartition, PartitionRegistration> partitions = new HashMap<>();
|
||||
topicsById.values().forEach(topic -> {
|
||||
topic.partitions().forEach((key, value) -> partitions.put(new TopicPartition(topic.name(), key), value));
|
||||
});
|
||||
return partitions;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "TopicsImage(topicsById=" + topicsById.entrySet().stream().
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.kafka.image.MetadataProvenance;
|
|||
import org.apache.kafka.image.loader.LogDeltaManifest;
|
||||
import org.apache.kafka.image.loader.SnapshotManifest;
|
||||
import org.apache.kafka.image.publisher.MetadataPublisher;
|
||||
import org.apache.kafka.metadata.BrokerRegistration;
|
||||
import org.apache.kafka.queue.EventQueue;
|
||||
import org.apache.kafka.queue.KafkaEventQueue;
|
||||
import org.apache.kafka.raft.LeaderAndEpoch;
|
||||
|
@ -72,7 +73,7 @@ public class KRaftMigrationDriver implements MetadataPublisher {
|
|||
*/
|
||||
private final Consumer<MetadataPublisher> initialZkLoadHandler;
|
||||
private volatile LeaderAndEpoch leaderAndEpoch;
|
||||
private volatile MigrationState migrationState;
|
||||
private volatile MigrationDriverState migrationState;
|
||||
private volatile ZkMigrationLeadershipState migrationLeadershipState;
|
||||
private volatile MetadataImage image;
|
||||
|
||||
|
@ -90,7 +91,7 @@ public class KRaftMigrationDriver implements MetadataPublisher {
|
|||
this.propagator = propagator;
|
||||
this.time = Time.SYSTEM;
|
||||
this.log = LoggerFactory.getLogger(KRaftMigrationDriver.class);
|
||||
this.migrationState = MigrationState.UNINITIALIZED;
|
||||
this.migrationState = MigrationDriverState.UNINITIALIZED;
|
||||
this.migrationLeadershipState = ZkMigrationLeadershipState.EMPTY;
|
||||
this.eventQueue = new KafkaEventQueue(Time.SYSTEM, new LogContext("KRaftMigrationDriver"), "kraft-migration");
|
||||
this.image = MetadataImage.EMPTY;
|
||||
|
@ -110,8 +111,8 @@ public class KRaftMigrationDriver implements MetadataPublisher {
|
|||
}
|
||||
|
||||
// Visible for testing
|
||||
CompletableFuture<MigrationState> migrationState() {
|
||||
CompletableFuture<MigrationState> stateFuture = new CompletableFuture<>();
|
||||
CompletableFuture<MigrationDriverState> migrationState() {
|
||||
CompletableFuture<MigrationDriverState> stateFuture = new CompletableFuture<>();
|
||||
eventQueue.append(() -> stateFuture.complete(migrationState));
|
||||
return stateFuture;
|
||||
}
|
||||
|
@ -123,7 +124,7 @@ public class KRaftMigrationDriver implements MetadataPublisher {
|
|||
log.info("Recovered migration state {}. ZK migration is {}.", migrationLeadershipState, maybeDone);
|
||||
initialZkLoadHandler.accept(this);
|
||||
// Let's transition to INACTIVE state and wait for leadership events.
|
||||
transitionTo(MigrationState.INACTIVE);
|
||||
transitionTo(MigrationDriverState.INACTIVE);
|
||||
}
|
||||
|
||||
private boolean isControllerQuorumReadyForMigration() {
|
||||
|
@ -137,9 +138,12 @@ public class KRaftMigrationDriver implements MetadataPublisher {
|
|||
log.info("Waiting for initial metadata publish before checking if Zk brokers are registered.");
|
||||
return false;
|
||||
}
|
||||
Set<Integer> kraftRegisteredZkBrokers = image.cluster().zkBrokers().keySet();
|
||||
Set<Integer> zkRegisteredZkBrokers = zkMigrationClient.readBrokerIdsFromTopicAssignments();
|
||||
zkRegisteredZkBrokers.removeAll(kraftRegisteredZkBrokers);
|
||||
for (BrokerRegistration broker : image.cluster().brokers().values()) {
|
||||
if (broker.isMigratingZkBroker()) {
|
||||
zkRegisteredZkBrokers.remove(broker.id());
|
||||
}
|
||||
}
|
||||
if (zkRegisteredZkBrokers.isEmpty()) {
|
||||
return true;
|
||||
} else {
|
||||
|
@ -155,43 +159,43 @@ public class KRaftMigrationDriver implements MetadataPublisher {
|
|||
this.migrationLeadershipState = afterState;
|
||||
}
|
||||
|
||||
private boolean isValidStateChange(MigrationState newState) {
|
||||
private boolean isValidStateChange(MigrationDriverState newState) {
|
||||
if (migrationState == newState)
|
||||
return true;
|
||||
switch (migrationState) {
|
||||
case UNINITIALIZED:
|
||||
case DUAL_WRITE:
|
||||
return newState == MigrationState.INACTIVE;
|
||||
return newState == MigrationDriverState.INACTIVE;
|
||||
case INACTIVE:
|
||||
return newState == MigrationState.WAIT_FOR_CONTROLLER_QUORUM;
|
||||
return newState == MigrationDriverState.WAIT_FOR_CONTROLLER_QUORUM;
|
||||
case WAIT_FOR_CONTROLLER_QUORUM:
|
||||
return
|
||||
newState == MigrationState.INACTIVE ||
|
||||
newState == MigrationState.WAIT_FOR_BROKERS;
|
||||
newState == MigrationDriverState.INACTIVE ||
|
||||
newState == MigrationDriverState.WAIT_FOR_BROKERS;
|
||||
case WAIT_FOR_BROKERS:
|
||||
return
|
||||
newState == MigrationState.INACTIVE ||
|
||||
newState == MigrationState.BECOME_CONTROLLER;
|
||||
newState == MigrationDriverState.INACTIVE ||
|
||||
newState == MigrationDriverState.BECOME_CONTROLLER;
|
||||
case BECOME_CONTROLLER:
|
||||
return
|
||||
newState == MigrationState.INACTIVE ||
|
||||
newState == MigrationState.ZK_MIGRATION ||
|
||||
newState == MigrationState.KRAFT_CONTROLLER_TO_BROKER_COMM;
|
||||
newState == MigrationDriverState.INACTIVE ||
|
||||
newState == MigrationDriverState.ZK_MIGRATION ||
|
||||
newState == MigrationDriverState.KRAFT_CONTROLLER_TO_BROKER_COMM;
|
||||
case ZK_MIGRATION:
|
||||
return
|
||||
newState == MigrationState.INACTIVE ||
|
||||
newState == MigrationState.KRAFT_CONTROLLER_TO_BROKER_COMM;
|
||||
newState == MigrationDriverState.INACTIVE ||
|
||||
newState == MigrationDriverState.KRAFT_CONTROLLER_TO_BROKER_COMM;
|
||||
case KRAFT_CONTROLLER_TO_BROKER_COMM:
|
||||
return
|
||||
newState == MigrationState.INACTIVE ||
|
||||
newState == MigrationState.DUAL_WRITE;
|
||||
newState == MigrationDriverState.INACTIVE ||
|
||||
newState == MigrationDriverState.DUAL_WRITE;
|
||||
default:
|
||||
log.error("Migration driver trying to transition from an unknown state {}", migrationState);
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
private void transitionTo(MigrationState newState) {
|
||||
private void transitionTo(MigrationDriverState newState) {
|
||||
if (!isValidStateChange(newState)) {
|
||||
log.error("Error transition in migration driver from {} to {}", migrationState, newState);
|
||||
return;
|
||||
|
@ -336,13 +340,13 @@ public class KRaftMigrationDriver implements MetadataPublisher {
|
|||
default:
|
||||
if (!isActive) {
|
||||
apply("KRaftLeaderEvent is not active", state -> ZkMigrationLeadershipState.EMPTY);
|
||||
transitionTo(MigrationState.INACTIVE);
|
||||
transitionTo(MigrationDriverState.INACTIVE);
|
||||
} else {
|
||||
// Apply the new KRaft state
|
||||
apply("KRaftLeaderEvent is active", state -> state.withNewKRaftController(nodeId, leaderAndEpoch.epoch()));
|
||||
// Before becoming the controller fo ZkBrokers, we need to make sure the
|
||||
// Controller Quorum can handle migration.
|
||||
transitionTo(MigrationState.WAIT_FOR_CONTROLLER_QUORUM);
|
||||
transitionTo(MigrationDriverState.WAIT_FOR_CONTROLLER_QUORUM);
|
||||
}
|
||||
break;
|
||||
}
|
||||
|
@ -359,7 +363,7 @@ public class KRaftMigrationDriver implements MetadataPublisher {
|
|||
log.debug("Controller Quorum is ready for Zk to KRaft migration");
|
||||
// Note that leadership would not change here. Hence we do not need to
|
||||
// `apply` any leadership state change.
|
||||
transitionTo(MigrationState.WAIT_FOR_BROKERS);
|
||||
transitionTo(MigrationDriverState.WAIT_FOR_BROKERS);
|
||||
}
|
||||
break;
|
||||
default:
|
||||
|
@ -380,9 +384,9 @@ public class KRaftMigrationDriver implements MetadataPublisher {
|
|||
// We could not claim leadership, stay in BECOME_CONTROLLER to retry
|
||||
} else {
|
||||
if (!migrationLeadershipState.zkMigrationComplete()) {
|
||||
transitionTo(MigrationState.ZK_MIGRATION);
|
||||
transitionTo(MigrationDriverState.ZK_MIGRATION);
|
||||
} else {
|
||||
transitionTo(MigrationState.KRAFT_CONTROLLER_TO_BROKER_COMM);
|
||||
transitionTo(MigrationDriverState.KRAFT_CONTROLLER_TO_BROKER_COMM);
|
||||
}
|
||||
}
|
||||
break;
|
||||
|
@ -400,7 +404,7 @@ public class KRaftMigrationDriver implements MetadataPublisher {
|
|||
case WAIT_FOR_BROKERS:
|
||||
if (areZkBrokersReadyForMigration()) {
|
||||
log.debug("Zk brokers are registered and ready for migration");
|
||||
transitionTo(MigrationState.BECOME_CONTROLLER);
|
||||
transitionTo(MigrationDriverState.BECOME_CONTROLLER);
|
||||
}
|
||||
break;
|
||||
default:
|
||||
|
@ -447,7 +451,7 @@ public class KRaftMigrationDriver implements MetadataPublisher {
|
|||
offsetAndEpochAfterMigration.offset(),
|
||||
offsetAndEpochAfterMigration.epoch());
|
||||
apply("Migrate metadata from Zk", state -> zkMigrationClient.setMigrationRecoveryState(newState));
|
||||
transitionTo(MigrationState.KRAFT_CONTROLLER_TO_BROKER_COMM);
|
||||
transitionTo(MigrationDriverState.KRAFT_CONTROLLER_TO_BROKER_COMM);
|
||||
} catch (Throwable t) {
|
||||
zkRecordConsumer.abortMigration();
|
||||
super.handleException(t);
|
||||
|
@ -460,13 +464,13 @@ public class KRaftMigrationDriver implements MetadataPublisher {
|
|||
@Override
|
||||
public void run() throws Exception {
|
||||
// Ignore sending RPCs to the brokers since we're no longer in the state.
|
||||
if (migrationState == MigrationState.KRAFT_CONTROLLER_TO_BROKER_COMM) {
|
||||
if (migrationState == MigrationDriverState.KRAFT_CONTROLLER_TO_BROKER_COMM) {
|
||||
if (image.highestOffsetAndEpoch().compareTo(migrationLeadershipState.offsetAndEpoch()) >= 0) {
|
||||
log.trace("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(MigrationState.DUAL_WRITE);
|
||||
transitionTo(MigrationDriverState.DUAL_WRITE);
|
||||
} else {
|
||||
log.trace("Ignoring using metadata image since migration leadership state is at a greater offset and epoch {}",
|
||||
migrationLeadershipState.offsetAndEpoch());
|
||||
|
@ -501,7 +505,7 @@ public class KRaftMigrationDriver implements MetadataPublisher {
|
|||
KRaftMigrationDriver.this.image = image;
|
||||
String metadataType = isSnapshot ? "snapshot" : "delta";
|
||||
|
||||
if (migrationState != MigrationState.DUAL_WRITE) {
|
||||
if (migrationState != MigrationDriverState.DUAL_WRITE) {
|
||||
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);
|
||||
|
|
|
@ -18,6 +18,10 @@
|
|||
package org.apache.kafka.metadata.migration;
|
||||
|
||||
/**
|
||||
* This is the internal state of the KRaftMigrationDriver class on a particular controller node.
|
||||
* Unlike the ZkMigrationState, which is persisted in the metadata log and image, this is soft
|
||||
* state which is stored in memory only.
|
||||
*
|
||||
* UNINITIALIZED───────────────►INACTIVE◄────────────────DUAL_WRITE◄────────────────────────┐
|
||||
* │ ▲ │
|
||||
* │ │ │
|
||||
|
@ -34,7 +38,7 @@ package org.apache.kafka.metadata.migration;
|
|||
* ▼ │ │ │
|
||||
* BECOME_CONTROLLER───────────────────►└────────────────────►WAIT_FOR_BROKERS───────────────────┘
|
||||
*/
|
||||
public enum MigrationState {
|
||||
public enum MigrationDriverState {
|
||||
UNINITIALIZED(false), // Initial state.
|
||||
INACTIVE(false), // State when not the active controller.
|
||||
WAIT_FOR_CONTROLLER_QUORUM(false), // Ensure all the quorum nodes are ready for migration.
|
||||
|
@ -46,7 +50,7 @@ public enum MigrationState {
|
|||
|
||||
private final boolean isActiveController;
|
||||
|
||||
MigrationState(boolean isActiveController) {
|
||||
MigrationDriverState(boolean isActiveController) {
|
||||
this.isActiveController = isActiveController;
|
||||
}
|
||||
|
|
@ -19,32 +19,39 @@ package org.apache.kafka.metadata.migration;
|
|||
import java.util.Optional;
|
||||
|
||||
/**
|
||||
* The cluster-wide ZooKeeper migration state.
|
||||
*
|
||||
* An enumeration of the possible states of the ZkMigrationState field in ZkMigrationStateRecord.
|
||||
* This information is persisted in the metadata log and image.
|
||||
*
|
||||
* @see org.apache.kafka.common.metadata.ZkMigrationStateRecord
|
||||
*/
|
||||
public enum ZkMigrationState {
|
||||
/**
|
||||
* No migration has been started by the controller. The controller is in regular KRaft mode
|
||||
* The cluster was created in KRaft mode. A cluster that was created in ZK mode can never attain
|
||||
* this state; the endpoint of migration is POST_MIGRATION, instead.
|
||||
*/
|
||||
NONE((byte) 0),
|
||||
|
||||
/**
|
||||
* A KRaft controller has been elected with "zookeeper.metadata.migration.enable" set to "true".
|
||||
* The controller is now awaiting the pre-conditions for starting the migration.
|
||||
* The controller is now awaiting the preconditions for starting the migration to KRaft. In this
|
||||
* state, the metadata log does not yet contain the cluster's data. There is a metadata quorum,
|
||||
* but it is not doing anything useful yet.
|
||||
*/
|
||||
PRE_MIGRATION((byte) 1),
|
||||
|
||||
/**
|
||||
* The ZK data has been migrated and the KRaft controller is now writing metadata to both ZK and the
|
||||
* metadata log. The controller will remain in this state until all of the brokers have been restarted
|
||||
* in KRaft mode
|
||||
* The ZK data has been migrated, and the KRaft controller is now writing metadata to both ZK
|
||||
* and the metadata log. The controller will remain in this state until all of the brokers have
|
||||
* been restarted in KRaft mode.
|
||||
*/
|
||||
MIGRATION((byte) 2),
|
||||
|
||||
/**
|
||||
* The migration has been fully completed. The cluster is running in KRaft mode. This state will persist
|
||||
* indefinitely after the migration.
|
||||
* The migration from ZK has been fully completed. The cluster is running in KRaft mode. This state
|
||||
* will persist indefinitely after the migration. In operational terms, this is the same as the NONE
|
||||
* state.
|
||||
*/
|
||||
POST_MIGRATION((byte) 3);
|
||||
|
||||
|
|
|
@ -280,7 +280,7 @@ public class KRaftMigrationDriverTest {
|
|||
driver.publishLogDelta(delta, image, new LogDeltaManifest(provenance,
|
||||
new LeaderAndEpoch(OptionalInt.of(3000), 1), 1, 100, 42));
|
||||
|
||||
TestUtils.waitForCondition(() -> driver.migrationState().get(1, TimeUnit.MINUTES).equals(MigrationState.DUAL_WRITE),
|
||||
TestUtils.waitForCondition(() -> driver.migrationState().get(1, TimeUnit.MINUTES).equals(MigrationDriverState.DUAL_WRITE),
|
||||
"Waiting for KRaftMigrationDriver to enter DUAL_WRITE state");
|
||||
|
||||
Assertions.assertEquals(1, metadataPropagator.images);
|
||||
|
|
Loading…
Reference in New Issue