KAFKA-17613: Remove ZK migration code (#17293)

Remove the controller machinery for doing ZK migration in Kafka 4.0.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Arthur <mumrah@gmail.com>
This commit is contained in:
Colin Patrick McCabe 2024-10-03 12:01:14 -07:00 committed by GitHub
parent 8b5d755bf6
commit 85bfdf4127
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
54 changed files with 107 additions and 5925 deletions

View File

@ -1,115 +0,0 @@
/*
* 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 kafka.migration
import kafka.cluster.Broker
import kafka.controller.{ControllerChannelContext, LeaderIsrAndControllerEpoch}
import org.apache.kafka.common.{TopicPartition, Uuid}
import org.apache.kafka.image.MetadataImage
import org.apache.kafka.metadata.LeaderAndIsr
import scala.jdk.CollectionConverters._
object MigrationControllerChannelContext {
def isReplicaOnline(image: MetadataImage, brokerId: Int, replicaAssignment: Set[Int]): Boolean = {
val brokerOnline = image.cluster().containsBroker(brokerId)
brokerOnline && replicaAssignment.contains(brokerId)
}
def partitionReplicaAssignment(image: MetadataImage, tp: TopicPartition): collection.Seq[Int] = {
image.topics().topicsByName().asScala.get(tp.topic()) match {
case Some(topic) => topic.partitions().asScala.get(tp.partition()) match {
case Some(partition) => partition.replicas.toSeq
case None => collection.Seq.empty
}
case None => collection.Seq.empty
}
}
def partitionLeadershipInfo(image: MetadataImage, topicPartition: TopicPartition): Option[LeaderIsrAndControllerEpoch] = {
image.topics().topicsByName().asScala.get(topicPartition.topic()) match {
case Some(topic) => topic.partitions().asScala.get(topicPartition.partition()) match {
case Some(partition) =>
val leaderAndIsr = new LeaderAndIsr(partition.leader, partition.leaderEpoch, partition.isr.toList.map(Integer.valueOf).asJava,
partition.leaderRecoveryState, partition.partitionEpoch)
Some(LeaderIsrAndControllerEpoch(leaderAndIsr, image.highestOffsetAndEpoch().epoch()))
case None => None
}
case None => None
}
}
}
sealed class MigrationControllerChannelContext(
val image: MetadataImage
) extends ControllerChannelContext {
override def isTopicDeletionInProgress(topicName: String): Boolean = {
!image.topics().topicsByName().containsKey(topicName)
}
override val topicIds: collection.Map[String, Uuid] = {
image.topics().topicsByName().asScala.map {
case (name, topic) => name -> topic.id()
}.toMap
}
override val liveBrokerIdAndEpochs: collection.Map[Int, Long] = {
image.cluster().brokers().asScala.map {
case (brokerId, broker) => brokerId.intValue() -> broker.epoch()
}
}
override val liveOrShuttingDownBrokers: collection.Set[Broker] = {
image.cluster().brokers().asScala.values.map { registration =>
Broker.fromBrokerRegistration(registration)
}.toSet
}
override def isTopicQueuedUpForDeletion(topic: String): Boolean = {
!image.topics().topicsByName().containsKey(topic)
}
override def isReplicaOnline(brokerId: Int, partition: TopicPartition): Boolean = {
MigrationControllerChannelContext.isReplicaOnline(
image, brokerId, partitionReplicaAssignment(partition).toSet)
}
override def partitionReplicaAssignment(tp: TopicPartition): collection.Seq[Int] = {
MigrationControllerChannelContext.partitionReplicaAssignment(image, tp)
}
override def leaderEpoch(topicPartition: TopicPartition): Int = {
// Topic is deleted use a special sentinel -2 to the indicate the same.
if (isTopicQueuedUpForDeletion(topicPartition.topic())) {
LeaderAndIsr.EPOCH_DURING_DELETE
} else {
image.topics().topicsByName.asScala.get(topicPartition.topic()) match {
case Some(topic) => topic.partitions().asScala.get(topicPartition.partition()) match {
case Some(partition) => partition.leaderEpoch
case None => LeaderAndIsr.NO_EPOCH
}
case None => LeaderAndIsr.NO_EPOCH
}
}
}
override val liveOrShuttingDownBrokerIds: collection.Set[Int] = liveBrokerIdAndEpochs.keySet
override def partitionLeadershipInfo(topicPartition: TopicPartition): Option[LeaderIsrAndControllerEpoch] = {
MigrationControllerChannelContext.partitionLeadershipInfo(image, topicPartition)
}
}

View File

@ -1,251 +0,0 @@
/*
* 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 kafka.migration
import kafka.cluster.Broker
import kafka.controller.{ControllerChannelContext, ControllerChannelManager, ReplicaAssignment, StateChangeLogger}
import kafka.server.KafkaConfig
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.requests.AbstractControlRequest
import org.apache.kafka.common.utils.Time
import org.apache.kafka.image.{ClusterImage, MetadataDelta, MetadataImage, TopicsImage}
import org.apache.kafka.metadata.{LeaderAndIsr, PartitionRegistration}
import org.apache.kafka.metadata.migration.LegacyPropagator
import java.util
import scala.jdk.CollectionConverters._
import scala.compat.java8.OptionConverters._
object MigrationPropagator {
def calculateBrokerChanges(prevClusterImage: ClusterImage, clusterImage: ClusterImage): (Set[Broker], Set[Broker]) = {
val prevBrokers = prevClusterImage.brokers().values().asScala
.filter(_.isMigratingZkBroker)
.filterNot(_.fenced)
.map(Broker.fromBrokerRegistration)
.toSet
val aliveBrokers = clusterImage.brokers().values().asScala
.filter(_.isMigratingZkBroker)
.filterNot(_.fenced)
.map(Broker.fromBrokerRegistration)
.toSet
val addedBrokers = aliveBrokers -- prevBrokers
val removedBrokers = prevBrokers -- aliveBrokers
(addedBrokers, removedBrokers)
}
}
class MigrationPropagator(
nodeId: Int,
config: KafkaConfig
) extends LegacyPropagator {
@volatile private var _image = MetadataImage.EMPTY
val stateChangeLogger = new StateChangeLogger(nodeId, inControllerContext = true, None)
val channelManager = new ControllerChannelManager(
() => _image.highestOffsetAndEpoch().epoch(),
config,
Time.SYSTEM,
new Metrics(),
stateChangeLogger
)
private val requestBatch = new MigrationPropagatorBatch(
config,
metadataProvider,
() => _image.features().metadataVersion(),
channelManager,
stateChangeLogger
)
private def metadataProvider(): ControllerChannelContext = {
new MigrationControllerChannelContext(_image)
}
def startup(): Unit = {
channelManager.startup(Set.empty)
}
def shutdown(): Unit = {
clear()
channelManager.shutdown()
}
override def publishMetadata(image: MetadataImage): Unit = {
val oldImage = _image
val (addedBrokers, removedBrokers) = MigrationPropagator.calculateBrokerChanges(oldImage.cluster(), image.cluster())
if (addedBrokers.nonEmpty || removedBrokers.nonEmpty) {
stateChangeLogger.logger.info(s"Adding brokers $addedBrokers, removing brokers $removedBrokers.")
}
removedBrokers.foreach(broker => channelManager.removeBroker(broker.id))
addedBrokers.foreach(broker => channelManager.addBroker(broker))
_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.
*/
private 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)
requestBatch.newBatch()
delta.getOrCreateTopicsDelta()
delta.getOrCreateClusterDelta()
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, materializePartitions(image.topics()).asScala.keySet)
}
if (brokersChanged) {
requestBatch.addUpdateMetadataRequestForBrokers(oldZkBrokers.toSeq)
}
requestBatch.sendRequestsToBrokers(zkControllerEpoch)
requestBatch.newBatch()
requestBatch.setUpdateType(AbstractControlRequest.Type.INCREMENTAL)
// Now send LISR, UMR and StopReplica requests for both new zk brokers and existing zk
// brokers based on the topic changes.
if (changedZkBrokers.nonEmpty) {
// For new the brokers, check if there are partition assignments and add LISR appropriately.
materializePartitions(image.topics()).asScala.foreach { case (tp, partitionRegistration) =>
val replicas = partitionRegistration.replicas.toSet
val leaderIsrAndControllerEpochOpt = MigrationControllerChannelContext.partitionLeadershipInfo(image, tp)
val newBrokersWithReplicas = replicas.intersect(changedZkBrokers)
if (newBrokersWithReplicas.nonEmpty) {
leaderIsrAndControllerEpochOpt match {
case Some(leaderIsrAndControllerEpoch) =>
val replicaAssignment = ReplicaAssignment(partitionRegistration.replicas,
partitionRegistration.addingReplicas, partitionRegistration.removingReplicas)
requestBatch.addLeaderAndIsrRequestForBrokers(newBrokersWithReplicas.toSeq, tp,
leaderIsrAndControllerEpoch, replicaAssignment, isNew = true)
case None =>
}
}
}
}
// If there are changes in topic metadata, let's send UMR about the changes to the old Zk brokers.
if (!delta.topicsDelta().deletedTopicIds().isEmpty || !delta.topicsDelta().changedTopics().isEmpty) {
requestBatch.addUpdateMetadataRequestForBrokers(oldZkBrokers.toSeq)
}
// Handle deleted topics by sending appropriate StopReplica and UMR requests to the brokers.
delta.topicsDelta().deletedTopicIds().asScala.foreach { deletedTopicId =>
val deletedTopic = delta.image().topics().getTopic(deletedTopicId)
deletedTopic.partitions().asScala.foreach { case (partition, partitionRegistration) =>
val tp = new TopicPartition(deletedTopic.name(), partition)
val offlineReplicas = partitionRegistration.replicas.filter {
MigrationControllerChannelContext.isReplicaOnline(image, _, partitionRegistration.replicas.toSet)
}
val deletedLeaderAndIsr = LeaderAndIsr.duringDelete(partitionRegistration.isr.toList.map(Integer.valueOf).asJava)
requestBatch.addStopReplicaRequestForBrokers(partitionRegistration.replicas, tp, deletePartition = true)
requestBatch.addUpdateMetadataRequestForBrokers(
oldZkBrokers.toSeq, zkControllerEpoch, tp, deletedLeaderAndIsr.leader, deletedLeaderAndIsr.leaderEpoch,
deletedLeaderAndIsr.partitionEpoch, deletedLeaderAndIsr.isr.asScala.map(_.intValue()).toList, partitionRegistration.replicas, offlineReplicas)
}
}
// Handle changes in other topics and send appropriate LeaderAndIsr and UMR requests to the
// brokers.
delta.topicsDelta().changedTopics().asScala.foreach { case (_, topicDelta) =>
topicDelta.partitionChanges().asScala.foreach { case (partition, partitionRegistration) =>
val tp = new TopicPartition(topicDelta.name(), partition)
// Check for replica leadership changes.
val leaderIsrAndControllerEpochOpt = MigrationControllerChannelContext.partitionLeadershipInfo(image, tp)
leaderIsrAndControllerEpochOpt match {
case Some(leaderIsrAndControllerEpoch) =>
val replicaAssignment = ReplicaAssignment(partitionRegistration.replicas,
partitionRegistration.addingReplicas, partitionRegistration.removingReplicas)
requestBatch.addLeaderAndIsrRequestForBrokers(replicaAssignment.replicas, tp,
leaderIsrAndControllerEpoch, replicaAssignment, isNew = true)
case None =>
}
// Check for removed replicas.
val oldReplicas =
Option(delta.image().topics().getPartition(topicDelta.id(), tp.partition()))
.map(_.replicas.toSet)
.getOrElse(Set.empty)
val newReplicas = partitionRegistration.replicas.toSet
val removedReplicas = oldReplicas -- newReplicas
if (removedReplicas.nonEmpty) {
requestBatch.addStopReplicaRequestForBrokers(removedReplicas.toSeq, tp, deletePartition = true)
}
}
}
// Send all the accumulated requests to the broker.
requestBatch.sendRequestsToBrokers(zkControllerEpoch)
}
override def sendRPCsToBrokersFromMetadataImage(image: MetadataImage, zkControllerEpoch: Int): Unit = {
publishMetadata(image)
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.sendRequestsToBrokers(zkControllerEpoch)
requestBatch.newBatch()
requestBatch.setUpdateType(AbstractControlRequest.Type.FULL)
// When we need to send RPCs from the image, we're sending 'full' requests meaning we let
// every broker know about all the metadata and all the LISR requests it needs to handle.
// Note that we cannot send StopReplica requests from the image. We don't have any state
// about brokers that host a replica but are not part of the replica set known by the Controller.
partitions.asScala.foreach{ case (tp, partitionRegistration) =>
val leaderIsrAndControllerEpochOpt = MigrationControllerChannelContext.partitionLeadershipInfo(image, tp)
leaderIsrAndControllerEpochOpt match {
case Some(leaderIsrAndControllerEpoch) =>
val replicaAssignment = ReplicaAssignment(partitionRegistration.replicas,
partitionRegistration.addingReplicas, partitionRegistration.removingReplicas)
requestBatch.addLeaderAndIsrRequestForBrokers(replicaAssignment.replicas, tp,
leaderIsrAndControllerEpoch, replicaAssignment, isNew = true)
case None => None
}
}
requestBatch.sendRequestsToBrokers(zkControllerEpoch)
}
override def clear(): Unit = {
requestBatch.clear()
}
}

View File

@ -1,85 +0,0 @@
/*
* 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 kafka.migration
import kafka.controller.{AbstractControllerBrokerRequestBatch, ControllerChannelContext, ControllerChannelManager, StateChangeLogger}
import kafka.server.KafkaConfig
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.requests._
import org.apache.kafka.server.common.MetadataVersion
import scala.collection.mutable.ArrayBuffer
import scala.jdk.CollectionConverters._
sealed class MigrationPropagatorBatch(
config: KafkaConfig,
metadataProvider: () => ControllerChannelContext,
metadataVersionProvider: () => MetadataVersion,
controllerChannelManager: ControllerChannelManager,
stateChangeLogger: StateChangeLogger
) extends AbstractControllerBrokerRequestBatch(
config,
metadataProvider,
metadataVersionProvider,
stateChangeLogger,
kraftController = true,
) {
override def sendRequest(brokerId: Int, request: AbstractControlRequest.Builder[_ <: AbstractControlRequest], callback: AbstractResponse => Unit): Unit = {
controllerChannelManager.sendRequest(brokerId, request, callback)
}
override def handleLeaderAndIsrResponse(response: LeaderAndIsrResponse, broker: Int): Unit = {
if (response.error != Errors.NONE) {
stateChangeLogger.error(s"Received error ${response.error} in LeaderAndIsr " +
s"response $response from broker $broker")
return
}
val partitionErrors = response.partitionErrors(
metadataProvider().topicIds.map { case (id, string) => (string, id) }.asJava)
val offlineReplicas = new ArrayBuffer[TopicPartition]()
partitionErrors.forEach{ case(tp, error) =>
if (error == Errors.KAFKA_STORAGE_ERROR) {
offlineReplicas += tp
}
}
if (offlineReplicas.nonEmpty) {
stateChangeLogger.error(s"Found ${offlineReplicas.mkString(",")} on broker $broker as offline")
}
}
override def handleUpdateMetadataResponse(response: UpdateMetadataResponse, broker: Int): Unit = {
if (response.error != Errors.NONE) {
stateChangeLogger.error(s"Received error ${response.error} in UpdateMetadata " +
s"response $response from broker $broker")
}
}
override def handleStopReplicaResponse(response: StopReplicaResponse, broker: Int,
partitionErrorsForDeletingTopics: Map[TopicPartition, Errors]): Unit = {
if (response.error() != Errors.NONE) {
stateChangeLogger.error(s"Received error ${response.error} in StopReplica " +
s"response $response from broker $broker")
}
partitionErrorsForDeletingTopics.foreach{ case(tp, error) =>
if (error != Errors.NONE) {
stateChangeLogger.error(s"Received error $error in StopReplica request for partition $tp " +
s"from broker $broker")
}
}
}
}

View File

@ -57,7 +57,6 @@ object ApiVersionManager {
supportedFeatures,
metadataCache,
config.unstableApiVersionsEnabled,
config.migrationEnabled,
clientMetricsManager
)
}
@ -71,7 +70,6 @@ object ApiVersionManager {
* @param enabledApis the enabled apis, which are computed by the listener type
* @param brokerFeatures the broker features
* @param enableUnstableLastVersion whether to enable unstable last version, see [[KafkaConfig.unstableApiVersionsEnabled]]
* @param zkMigrationEnabled whether to enable zk migration, see [[KafkaConfig.migrationEnabled]]
* @param featuresProvider a provider to the finalized features supported
*/
class SimpleApiVersionManager(
@ -79,14 +77,12 @@ class SimpleApiVersionManager(
val enabledApis: collection.Set[ApiKeys],
brokerFeatures: org.apache.kafka.common.feature.Features[SupportedVersionRange],
val enableUnstableLastVersion: Boolean,
val zkMigrationEnabled: Boolean,
val featuresProvider: () => FinalizedFeatures
) extends ApiVersionManager {
def this(
listenerType: ListenerType,
enableUnstableLastVersion: Boolean,
zkMigrationEnabled: Boolean,
featuresProvider: () => FinalizedFeatures
) = {
this(
@ -94,7 +90,6 @@ class SimpleApiVersionManager(
ApiKeys.apisForListener(listenerType).asScala,
BrokerFeatures.defaultSupportedFeatures(enableUnstableLastVersion),
enableUnstableLastVersion,
zkMigrationEnabled,
featuresProvider
)
}
@ -112,7 +107,7 @@ class SimpleApiVersionManager(
setSupportedFeatures(brokerFeatures).
setFinalizedFeatures(currentFeatures.finalizedFeatures()).
setFinalizedFeaturesEpoch(currentFeatures.finalizedFeaturesEpoch()).
setZkMigrationEnabled(zkMigrationEnabled).
setZkMigrationEnabled(false).
setAlterFeatureLevel0(alterFeatureLevel0).
build()
}
@ -130,7 +125,6 @@ class SimpleApiVersionManager(
* @param brokerFeatures the broker features
* @param metadataCache the metadata cache, used to get the finalized features and the metadata version
* @param enableUnstableLastVersion whether to enable unstable last version, see [[KafkaConfig.unstableApiVersionsEnabled]]
* @param zkMigrationEnabled whether to enable zk migration, see [[KafkaConfig.migrationEnabled]]
* @param clientMetricsManager the client metrics manager, helps to determine whether client telemetry is enabled
*/
class DefaultApiVersionManager(
@ -139,7 +133,6 @@ class DefaultApiVersionManager(
brokerFeatures: BrokerFeatures,
metadataCache: MetadataCache,
val enableUnstableLastVersion: Boolean,
val zkMigrationEnabled: Boolean = false,
val clientMetricsManager: Option[ClientMetricsManager] = None
) extends ApiVersionManager {
@ -175,7 +168,7 @@ class DefaultApiVersionManager(
setSupportedFeatures(brokerFeatures.supportedFeatures).
setFinalizedFeatures(finalizedFeatures.finalizedFeatures()).
setFinalizedFeaturesEpoch(finalizedFeatures.finalizedFeaturesEpoch()).
setZkMigrationEnabled(zkMigrationEnabled).
setZkMigrationEnabled(false).
setAlterFeatureLevel0(alterFeatureLevel0).
build()
}

View File

@ -50,7 +50,6 @@ class ControllerRegistrationManager(
val time: Time,
val threadNamePrefix: String,
val supportedFeatures: util.Map[String, VersionRange],
val zkMigrationEnabled: Boolean,
val incarnationId: Uuid,
val listenerInfo: ListenerInfo,
val resendExponentialBackoff: ExponentialBackoff = new ExponentialBackoff(100, 2, 120000L, 0.02)
@ -227,7 +226,7 @@ class ControllerRegistrationManager(
setFeatures(features).
setIncarnationId(incarnationId).
setListeners(listenerInfo.toControllerRegistrationRequest).
setZkMigrationReady(zkMigrationEnabled)
setZkMigrationReady(false)
info(s"sendControllerRegistration: attempting to send $data")
_channelManager.sendRequest(new ControllerRegistrationRequest.Builder(data),

View File

@ -17,7 +17,6 @@
package kafka.server
import kafka.migration.MigrationPropagator
import kafka.network.{DataPlaneAcceptor, SocketServer}
import kafka.raft.KafkaRaftManager
import kafka.server.QuotaFactory.QuotaManagers
@ -25,7 +24,6 @@ import kafka.server.QuotaFactory.QuotaManagers
import scala.collection.immutable
import kafka.server.metadata.{AclPublisher, ClientQuotaMetadataManager, DelegationTokenPublisher, DynamicClientQuotaPublisher, DynamicConfigPublisher, KRaftMetadataCache, KRaftMetadataCachePublisher, ScramPublisher}
import kafka.utils.{CoreUtils, Logging}
import kafka.zk.{KafkaZkClient, ZkMigrationClient}
import org.apache.kafka.common.message.ApiMessageType.ListenerType
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.security.scram.internals.ScramMechanism
@ -38,10 +36,9 @@ import org.apache.kafka.image.publisher.{ControllerRegistrationsPublisher, Metad
import org.apache.kafka.metadata.{KafkaConfigSchema, ListenerInfo}
import org.apache.kafka.metadata.authorizer.ClusterMetadataAuthorizer
import org.apache.kafka.metadata.bootstrap.BootstrapMetadata
import org.apache.kafka.metadata.migration.{KRaftMigrationDriver, LegacyPropagator}
import org.apache.kafka.metadata.publisher.FeaturesPublisher
import org.apache.kafka.raft.QuorumConfig
import org.apache.kafka.security.{CredentialProvider, PasswordEncoder}
import org.apache.kafka.security.CredentialProvider
import org.apache.kafka.server.NodeToControllerChannelManager
import org.apache.kafka.server.authorizer.Authorizer
import org.apache.kafka.server.config.ServerLogConfigs.{ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG}
@ -60,20 +57,6 @@ import scala.compat.java8.OptionConverters._
import scala.jdk.CollectionConverters._
case class ControllerMigrationSupport(
zkClient: KafkaZkClient,
migrationDriver: KRaftMigrationDriver,
brokersRpcClient: LegacyPropagator
) {
def shutdown(logging: Logging): Unit = {
Utils.closeQuietly(zkClient, "zk client")
if (brokersRpcClient != null) {
CoreUtils.swallow(brokersRpcClient.shutdown(), logging)
}
Utils.closeQuietly(migrationDriver, "migration driver")
}
}
/**
* A Kafka controller that runs in KRaft (Kafka Raft) mode.
*/
@ -111,7 +94,6 @@ class ControllerServer(
var clientQuotaMetadataManager: ClientQuotaMetadataManager = _
var controllerApis: ControllerApis = _
var controllerApisHandlerPool: KafkaRequestHandlerPool = _
var migrationSupport: Option[ControllerMigrationSupport] = None
def kafkaYammerMetrics: KafkaYammerMetrics = KafkaYammerMetrics.INSTANCE
val metadataPublishers: util.List[MetadataPublisher] = new util.ArrayList[MetadataPublisher]()
@volatile var metadataCache : KRaftMetadataCache = _
@ -171,7 +153,6 @@ class ControllerServer(
val apiVersionManager = new SimpleApiVersionManager(
ListenerType.CONTROLLER,
config.unstableApiVersionsEnabled,
config.migrationEnabled,
() => featuresPublisher.features()
)
@ -233,7 +214,7 @@ class ControllerServer(
val maxIdleIntervalNs = config.metadataMaxIdleIntervalNs.fold(OptionalLong.empty)(OptionalLong.of)
quorumControllerMetrics = new QuorumControllerMetrics(Optional.of(KafkaYammerMetrics.defaultRegistry), time, config.migrationEnabled)
quorumControllerMetrics = new QuorumControllerMetrics(Optional.of(KafkaYammerMetrics.defaultRegistry), time)
new QuorumController.Builder(config.nodeId, sharedServer.clusterId).
setTime(time).
@ -255,7 +236,6 @@ class ControllerServer(
setBootstrapMetadata(bootstrapMetadata).
setFatalFaultHandler(sharedServer.fatalQuorumControllerFaultHandler).
setNonFatalFaultHandler(sharedServer.nonFatalQuorumControllerFaultHandler).
setZkMigrationEnabled(config.migrationEnabled).
setDelegationTokenCache(tokenCache).
setDelegationTokenSecretKey(delegationTokenKeyString).
setDelegationTokenMaxLifeMs(config.delegationTokenMaxLifeMs).
@ -274,39 +254,6 @@ class ControllerServer(
case _ =>
}
if (config.migrationEnabled) {
val zkClient = KafkaZkClient.createZkClient("KRaft Migration", time, config, KafkaServer.zkClientConfigFromKafkaConfig(config))
val zkConfigEncoder = config.passwordEncoderSecret match {
case Some(secret) => PasswordEncoder.encrypting(secret,
config.passwordEncoderKeyFactoryAlgorithm,
config.passwordEncoderCipherAlgorithm,
config.passwordEncoderKeyLength,
config.passwordEncoderIterations)
case None => PasswordEncoder.NOOP
}
val migrationClient = ZkMigrationClient(zkClient, zkConfigEncoder)
val propagator: LegacyPropagator = new MigrationPropagator(config.nodeId, config)
val migrationDriver = KRaftMigrationDriver.newBuilder()
.setNodeId(config.nodeId)
.setZkRecordConsumer(controller.asInstanceOf[QuorumController].zkRecordConsumer())
.setZkMigrationClient(migrationClient)
.setPropagator(propagator)
.setInitialZkLoadHandler(publisher => sharedServer.loader.installPublishers(java.util.Collections.singletonList(publisher)))
.setFaultHandler(sharedServer.faultHandlerFactory.build(
"zk migration",
fatal = false,
() => {}
))
.setQuorumFeatures(quorumFeatures)
.setConfigSchema(configSchema)
.setControllerMetrics(quorumControllerMetrics)
.setMinMigrationBatchSize(config.migrationMetadataMinBatchSize)
.setTime(time)
.build()
migrationDriver.start()
migrationSupport = Some(ControllerMigrationSupport(zkClient, migrationDriver, propagator))
}
quotaManagers = QuotaFactory.instantiate(config,
metrics,
time,
@ -347,7 +294,6 @@ class ControllerServer(
time,
s"controller-${config.nodeId}-",
QuorumFeatures.defaultFeatureMap(config.unstableFeatureVersionsEnabled),
config.migrationEnabled,
incarnationId,
listenerInfo)
@ -490,7 +436,6 @@ class ControllerServer(
registrationsPublisher = null
if (socketServer != null)
CoreUtils.swallow(socketServer.stopProcessingRequests(), this)
migrationSupport.foreach(_.shutdown(this))
if (controller != null)
controller.beginShutdown()
if (socketServer != null)

View File

@ -81,7 +81,6 @@ class TestRaftServer(
val apiVersionManager = new SimpleApiVersionManager(
ListenerType.CONTROLLER,
true,
false,
() => FinalizedFeatures.fromKRaftVersion(MetadataVersion.MINIMUM_KRAFT_VERSION))
socketServer = new SocketServer(config, metrics, time, credentialProvider, apiVersionManager)

View File

@ -1,356 +0,0 @@
/*
* 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 kafka.zk
import kafka.utils.Logging
import kafka.zk.ZkMigrationClient.wrapZkException
import kafka.zk.migration.{ZkAclMigrationClient, ZkConfigMigrationClient, ZkDelegationTokenMigrationClient, ZkTopicMigrationClient}
import kafka.zookeeper._
import org.apache.kafka.clients.admin.ScramMechanism
import org.apache.kafka.common.acl.AccessControlEntry
import org.apache.kafka.common.config.ConfigResource
import org.apache.kafka.common.errors.ControllerMovedException
import org.apache.kafka.common.metadata._
import org.apache.kafka.common.resource.ResourcePattern
import org.apache.kafka.common.security.scram.ScramCredential
import org.apache.kafka.common.{TopicIdPartition, Uuid}
import org.apache.kafka.metadata.DelegationTokenData
import org.apache.kafka.metadata.PartitionRegistration
import org.apache.kafka.metadata.migration.ConfigMigrationClient.ClientQuotaVisitor
import org.apache.kafka.metadata.migration.TopicMigrationClient.{TopicVisitor, TopicVisitorInterest}
import org.apache.kafka.metadata.migration._
import org.apache.kafka.security.PasswordEncoder
import org.apache.kafka.server.common.{ApiMessageAndVersion, ProducerIdsBlock}
import org.apache.zookeeper.KeeperException
import org.apache.zookeeper.KeeperException.{AuthFailedException, NoAuthException, SessionClosedRequireAuthException}
import java.{lang, util}
import java.util.function.Consumer
import scala.collection.Seq
import scala.compat.java8.OptionConverters._
import scala.jdk.CollectionConverters._
object ZkMigrationClient {
private val MaxBatchSize = 100
def apply(
zkClient: KafkaZkClient,
zkConfigEncoder: PasswordEncoder
): ZkMigrationClient = {
val topicClient = new ZkTopicMigrationClient(zkClient)
val configClient = new ZkConfigMigrationClient(zkClient, zkConfigEncoder)
val aclClient = new ZkAclMigrationClient(zkClient)
val delegationTokenClient = new ZkDelegationTokenMigrationClient(zkClient)
new ZkMigrationClient(zkClient, topicClient, configClient, aclClient, delegationTokenClient)
}
/**
* Wrap a function such that any KeeperExceptions is captured and converted to a MigrationClientException.
* Any authentication related exception is converted to a MigrationClientAuthException which may be treated
* differently by the caller.
*/
@throws(classOf[MigrationClientException])
def wrapZkException[T](fn: => T): T = {
try {
fn
} catch {
case e @ (_: MigrationClientException | _: MigrationClientAuthException) => throw e
case e @ (_: AuthFailedException | _: NoAuthException | _: SessionClosedRequireAuthException) =>
// We don't expect authentication errors to be recoverable, so treat them differently
throw new MigrationClientAuthException(e)
case e: KeeperException => throw new MigrationClientException(e)
}
}
@throws(classOf[MigrationClientException])
def logAndRethrow[T](logger: Logging, msg: String)(fn: => T): T = {
try {
fn
} catch {
case e: Throwable =>
logger.error(msg, e)
throw e
}
}
}
/**
* Migration client in KRaft controller responsible for handling communication to Zookeeper and
* the ZkBrokers present in the cluster. Methods that directly use KafkaZkClient should use the wrapZkException
* wrapper function in order to translate KeeperExceptions into something usable by the caller.
*/
class ZkMigrationClient(
zkClient: KafkaZkClient,
topicClient: TopicMigrationClient,
configClient: ConfigMigrationClient,
aclClient: AclMigrationClient,
delegationTokenClient: DelegationTokenMigrationClient
) extends MigrationClient with Logging {
override def getOrCreateMigrationRecoveryState(
initialState: ZkMigrationLeadershipState
): ZkMigrationLeadershipState = wrapZkException {
zkClient.createTopLevelPaths()
zkClient.getOrCreateMigrationState(initialState)
}
override def setMigrationRecoveryState(
state: ZkMigrationLeadershipState
): ZkMigrationLeadershipState = wrapZkException {
zkClient.updateMigrationState(state)
}
override def claimControllerLeadership(
state: ZkMigrationLeadershipState
): ZkMigrationLeadershipState = wrapZkException {
zkClient.tryRegisterKRaftControllerAsActiveController(state.kraftControllerId(), state.kraftControllerEpoch()) match {
case SuccessfulRegistrationResult(controllerEpoch, controllerEpochZkVersion) =>
state.withZkController(controllerEpoch, controllerEpochZkVersion)
case FailedRegistrationResult() => state.withUnknownZkController()
}
}
override def releaseControllerLeadership(
state: ZkMigrationLeadershipState
): ZkMigrationLeadershipState = wrapZkException {
try {
zkClient.deleteController(state.zkControllerEpochZkVersion())
state.withUnknownZkController()
} catch {
case _: ControllerMovedException =>
// If the controller moved, no need to release
state.withUnknownZkController()
case t: Throwable =>
throw new MigrationClientException("Could not release controller leadership due to underlying error", t)
}
}
def migrateTopics(
recordConsumer: Consumer[util.List[ApiMessageAndVersion]],
brokerIdConsumer: Consumer[Integer]
): Unit = wrapZkException {
var topicBatch = new util.ArrayList[ApiMessageAndVersion]()
topicClient.iterateTopics(
util.EnumSet.allOf(classOf[TopicVisitorInterest]),
new TopicVisitor() {
override def visitTopic(topicName: String, topicId: Uuid, assignments: util.Map[Integer, util.List[Integer]]): Unit = {
if (!topicBatch.isEmpty) {
recordConsumer.accept(topicBatch)
topicBatch = new util.ArrayList[ApiMessageAndVersion]()
}
topicBatch.add(new ApiMessageAndVersion(new TopicRecord()
.setName(topicName)
.setTopicId(topicId), 0.toShort))
// This breaks the abstraction a bit, but the topic configs belong in the topic batch
// when migrating topics and the logic for reading configs lives elsewhere
configClient.readTopicConfigs(topicName, (topicConfigs: util.Map[String, String]) => {
topicConfigs.forEach((key: Any, value: Any) => {
topicBatch.add(new ApiMessageAndVersion(new ConfigRecord()
.setResourceType(ConfigResource.Type.TOPIC.id)
.setResourceName(topicName)
.setName(key.toString)
.setValue(value.toString), 0.toShort))
})
})
}
override def visitPartition(topicIdPartition: TopicIdPartition, partitionRegistration: PartitionRegistration): Unit = {
val record = new PartitionRecord()
.setTopicId(topicIdPartition.topicId())
.setPartitionId(topicIdPartition.partition())
.setReplicas(partitionRegistration.replicas.map(Integer.valueOf).toList.asJava)
.setAddingReplicas(partitionRegistration.addingReplicas.map(Integer.valueOf).toList.asJava)
.setRemovingReplicas(partitionRegistration.removingReplicas.map(Integer.valueOf).toList.asJava)
.setIsr(partitionRegistration.isr.map(Integer.valueOf).toList.asJava)
.setLeader(partitionRegistration.leader)
.setLeaderEpoch(partitionRegistration.leaderEpoch)
.setPartitionEpoch(partitionRegistration.partitionEpoch)
.setLeaderRecoveryState(partitionRegistration.leaderRecoveryState.value())
partitionRegistration.replicas.foreach(brokerIdConsumer.accept(_))
partitionRegistration.addingReplicas.foreach(brokerIdConsumer.accept(_))
topicBatch.add(new ApiMessageAndVersion(record, 0.toShort))
}
}
)
if (!topicBatch.isEmpty) {
recordConsumer.accept(topicBatch)
}
}
def migrateBrokerConfigs(
recordConsumer: Consumer[util.List[ApiMessageAndVersion]],
brokerIdConsumer: Consumer[Integer]
): Unit = wrapZkException {
configClient.iterateBrokerConfigs((broker, props) => {
if (broker.nonEmpty) {
brokerIdConsumer.accept(Integer.valueOf(broker))
}
val batch = new util.ArrayList[ApiMessageAndVersion]()
props.forEach((key, value) => {
batch.add(new ApiMessageAndVersion(new ConfigRecord()
.setResourceType(ConfigResource.Type.BROKER.id)
.setResourceName(broker)
.setName(key)
.setValue(value), 0.toShort))
})
if (!batch.isEmpty) {
recordConsumer.accept(batch)
}
})
}
def migrateClientQuotas(
recordConsumer: Consumer[util.List[ApiMessageAndVersion]]
): Unit = wrapZkException {
configClient.iterateClientQuotas(new ClientQuotaVisitor {
override def visitClientQuota(
entityDataList: util.List[ClientQuotaRecord.EntityData],
quotas: util.Map[String, lang.Double]
): Unit = {
val batch = new util.ArrayList[ApiMessageAndVersion]()
quotas.forEach((key, value) => {
batch.add(new ApiMessageAndVersion(new ClientQuotaRecord()
.setEntity(entityDataList)
.setKey(key)
.setValue(value), 0.toShort))
})
recordConsumer.accept(batch)
}
override def visitScramCredential(
userName: String,
scramMechanism: ScramMechanism,
scramCredential: ScramCredential
): Unit = {
val batch = new util.ArrayList[ApiMessageAndVersion]()
batch.add(new ApiMessageAndVersion(new UserScramCredentialRecord()
.setName(userName)
.setMechanism(scramMechanism.`type`)
.setSalt(scramCredential.salt)
.setStoredKey(scramCredential.storedKey)
.setServerKey(scramCredential.serverKey)
.setIterations(scramCredential.iterations), 0.toShort))
recordConsumer.accept(batch)
}
})
}
def migrateProducerId(
recordConsumer: Consumer[util.List[ApiMessageAndVersion]]
): Unit = wrapZkException {
val (dataOpt, _) = zkClient.getDataAndVersion(ProducerIdBlockZNode.path)
dataOpt match {
case Some(data) =>
val producerIdBlock = ProducerIdBlockZNode.parseProducerIdBlockData(data)
recordConsumer.accept(List(new ApiMessageAndVersion(new ProducerIdsRecord()
.setBrokerEpoch(-1)
.setBrokerId(producerIdBlock.assignedBrokerId)
.setNextProducerId(producerIdBlock.nextBlockFirstId()), 0.toShort)).asJava)
case None => // Nothing to migrate
}
}
def migrateAcls(recordConsumer: Consumer[util.List[ApiMessageAndVersion]]): Unit = {
aclClient.iterateAcls(new util.function.BiConsumer[ResourcePattern, util.Set[AccessControlEntry]]() {
override def accept(resourcePattern: ResourcePattern, acls: util.Set[AccessControlEntry]): Unit = {
val batch = new util.ArrayList[ApiMessageAndVersion]()
acls.asScala.foreach { entry =>
batch.add(new ApiMessageAndVersion(new AccessControlEntryRecord()
.setId(Uuid.randomUuid())
.setResourceType(resourcePattern.resourceType().code())
.setResourceName(resourcePattern.name())
.setPatternType(resourcePattern.patternType().code())
.setPrincipal(entry.principal())
.setHost(entry.host())
.setOperation(entry.operation().code())
.setPermissionType(entry.permissionType().code()), AccessControlEntryRecord.HIGHEST_SUPPORTED_VERSION))
if (batch.size() == ZkMigrationClient.MaxBatchSize) {
recordConsumer.accept(batch)
batch.clear()
}
}
if (!batch.isEmpty) {
recordConsumer.accept(batch)
}
}
})
}
private def migrateDelegationTokens(
recordConsumer: Consumer[util.List[ApiMessageAndVersion]]
): Unit = wrapZkException {
val batch = new util.ArrayList[ApiMessageAndVersion]()
val tokens = zkClient.getChildren(DelegationTokensZNode.path)
for (tokenId <- tokens) {
zkClient.getDelegationTokenInfo(tokenId) match {
case Some(tokenInformation) =>
val newDelegationTokenData = new DelegationTokenData(tokenInformation)
batch.add(new ApiMessageAndVersion(newDelegationTokenData.toRecord, 0.toShort))
case None =>
}
}
if (!batch.isEmpty) {
recordConsumer.accept(batch)
}
}
override def readAllMetadata(
batchConsumer: Consumer[util.List[ApiMessageAndVersion]],
brokerIdConsumer: Consumer[Integer]
): Unit = {
migrateTopics(batchConsumer, brokerIdConsumer)
migrateBrokerConfigs(batchConsumer, brokerIdConsumer)
migrateClientQuotas(batchConsumer)
migrateProducerId(batchConsumer)
migrateAcls(batchConsumer)
migrateDelegationTokens(batchConsumer)
}
override def readBrokerIds(): util.Set[Integer] = wrapZkException {
new util.HashSet[Integer](zkClient.getSortedBrokerList.map(Integer.valueOf).toSet.asJava)
}
override def readProducerId(): util.Optional[ProducerIdsBlock] = {
val (dataOpt, _) = zkClient.getDataAndVersion(ProducerIdBlockZNode.path)
dataOpt.map(ProducerIdBlockZNode.parseProducerIdBlockData).asJava
}
override def writeProducerId(
nextProducerId: Long,
state: ZkMigrationLeadershipState
): ZkMigrationLeadershipState = wrapZkException {
val newProducerIdBlockData = ProducerIdBlockZNode.generateProducerIdBlockJson(
new ProducerIdsBlock(-1, nextProducerId, ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE))
val request = SetDataRequest(ProducerIdBlockZNode.path, newProducerIdBlockData, ZkVersion.MatchAnyVersion)
val (migrationZkVersion, _) = zkClient.retryMigrationRequestsUntilConnected(Seq(request), state)
state.withMigrationZkVersion(migrationZkVersion)
}
override def topicClient(): TopicMigrationClient = topicClient
override def configClient(): ConfigMigrationClient = configClient
override def aclClient(): AclMigrationClient = aclClient
override def delegationTokenClient(): DelegationTokenMigrationClient = delegationTokenClient
}

View File

@ -1,118 +0,0 @@
/*
* 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 kafka.zk.migration
import kafka.security.authorizer.AclAuthorizer.{ResourceOrdering, VersionedAcls}
import kafka.security.authorizer.AclAuthorizer
import kafka.utils.Logging
import kafka.zk.ZkMigrationClient.{logAndRethrow, wrapZkException}
import kafka.zk.{KafkaZkClient, ResourceZNode, ZkAclStore, ZkVersion}
import kafka.zookeeper.{CreateRequest, DeleteRequest, SetDataRequest}
import org.apache.kafka.common.acl.AccessControlEntry
import org.apache.kafka.common.resource.ResourcePattern
import org.apache.kafka.metadata.migration.{AclMigrationClient, MigrationClientException, ZkMigrationLeadershipState}
import org.apache.kafka.security.authorizer.AclEntry
import org.apache.zookeeper.CreateMode
import org.apache.zookeeper.KeeperException.Code
import java.util
import java.util.function.BiConsumer
import scala.jdk.CollectionConverters._
class ZkAclMigrationClient(
zkClient: KafkaZkClient
) extends AclMigrationClient with Logging {
private def aclChangeNotificationRequest(resourcePattern: ResourcePattern): CreateRequest = {
// ZK broker needs the ACL change notification znode to be updated in order to process the new ACLs
val aclChange = ZkAclStore(resourcePattern.patternType).changeStore.createChangeNode(resourcePattern)
CreateRequest(aclChange.path, aclChange.bytes, zkClient.defaultAcls(aclChange.path), CreateMode.PERSISTENT_SEQUENTIAL)
}
private def tryWriteAcls(
resourcePattern: ResourcePattern,
aclEntries: Set[AclEntry],
create: Boolean,
state: ZkMigrationLeadershipState
): Option[ZkMigrationLeadershipState] = wrapZkException {
val aclData = ResourceZNode.encode(aclEntries)
val request = if (create) {
val path = ResourceZNode.path(resourcePattern)
CreateRequest(path, aclData, zkClient.defaultAcls(path), CreateMode.PERSISTENT)
} else {
SetDataRequest(ResourceZNode.path(resourcePattern), aclData, ZkVersion.MatchAnyVersion)
}
val (migrationZkVersion, responses) = zkClient.retryMigrationRequestsUntilConnected(Seq(request), state)
if (responses.head.resultCode.equals(Code.NONODE)) {
// Need to call this method again with create=true
None
} else {
// Write the ACL notification outside of a metadata multi-op
zkClient.retryRequestUntilConnected(aclChangeNotificationRequest(resourcePattern))
Some(state.withMigrationZkVersion(migrationZkVersion))
}
}
override def writeResourceAcls(
resourcePattern: ResourcePattern,
aclsToWrite: util.Collection[AccessControlEntry],
state: ZkMigrationLeadershipState
): ZkMigrationLeadershipState = {
val acls = aclsToWrite.asScala.map(new AclEntry(_)).toSet
tryWriteAcls(resourcePattern, acls, create = false, state) match {
case Some(newState) => newState
case None => tryWriteAcls(resourcePattern, acls, create = true, state) match {
case Some(newState) => newState
case None => throw new MigrationClientException(s"Could not write ACLs for resource pattern $resourcePattern")
}
}
}
override def deleteResource(
resourcePattern: ResourcePattern,
state: ZkMigrationLeadershipState
): ZkMigrationLeadershipState = {
val request = DeleteRequest(ResourceZNode.path(resourcePattern), ZkVersion.MatchAnyVersion)
val (migrationZkVersion, responses) = zkClient.retryMigrationRequestsUntilConnected(Seq(request), state)
if (responses.head.resultCode.equals(Code.OK) || responses.head.resultCode.equals(Code.NONODE)) {
// Write the ACL notification outside of a metadata multi-op
zkClient.retryRequestUntilConnected(aclChangeNotificationRequest(resourcePattern))
state.withMigrationZkVersion(migrationZkVersion)
} else {
throw new MigrationClientException(s"Could not delete ACL for resource pattern $resourcePattern")
}
}
override def iterateAcls(
aclConsumer: BiConsumer[ResourcePattern, util.Set[AccessControlEntry]]
): Unit = {
// This is probably fairly inefficient, but it preserves the semantics from AclAuthorizer (which is non-trivial)
var allAcls = new scala.collection.immutable.TreeMap[ResourcePattern, VersionedAcls]()(new ResourceOrdering)
def updateAcls(resourcePattern: ResourcePattern, versionedAcls: VersionedAcls): Unit = {
allAcls = allAcls.updated(resourcePattern, versionedAcls)
}
AclAuthorizer.loadAllAcls(zkClient, this, updateAcls)
allAcls.foreach { case (resourcePattern, versionedAcls) =>
logAndRethrow(this, s"Error in ACL consumer. Resource was $resourcePattern.") {
aclConsumer.accept(resourcePattern, versionedAcls.acls.map(_.ace).asJava)
}
}
}
}

View File

@ -1,394 +0,0 @@
/*
* 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 kafka.zk.migration
import kafka.server.{DynamicBrokerConfig, DynamicConfig, ZkAdminManager}
import kafka.utils.Logging
import kafka.zk.ZkMigrationClient.{logAndRethrow, wrapZkException}
import kafka.zk._
import kafka.zk.migration.ZkConfigMigrationClient.getSanitizedClientQuotaZNodeName
import kafka.zookeeper.{CreateRequest, DeleteRequest, SetDataRequest}
import org.apache.kafka.clients.admin.ScramMechanism
import org.apache.kafka.common.config.types.Password
import org.apache.kafka.common.config.{ConfigDef, ConfigResource}
import org.apache.kafka.common.errors.InvalidRequestException
import org.apache.kafka.common.metadata.ClientQuotaRecord.EntityData
import org.apache.kafka.common.quota.ClientQuotaEntity
import org.apache.kafka.common.security.scram.internals.ScramCredentialUtils
import org.apache.kafka.common.utils.Sanitizer
import org.apache.kafka.metadata.migration.ConfigMigrationClient.ClientQuotaVisitor
import org.apache.kafka.metadata.migration.{ConfigMigrationClient, MigrationClientException, ZkMigrationLeadershipState}
import org.apache.kafka.security.PasswordEncoder
import org.apache.kafka.server.config.{ConfigType, ZooKeeperInternals}
import org.apache.zookeeper.KeeperException.Code
import org.apache.zookeeper.{CreateMode, KeeperException}
import java.{lang, util}
import java.util.Properties
import java.util.function.{BiConsumer, Consumer}
import scala.collection.Seq
import scala.jdk.CollectionConverters._
class ZkConfigMigrationClient(
zkClient: KafkaZkClient,
passwordEncoder: PasswordEncoder
) extends ConfigMigrationClient with Logging {
val adminZkClient = new AdminZkClient(zkClient)
/**
* In ZK, we use the special string "&lt;default&gt;" to represent the default config entity.
* In KRaft, we use an empty string. This method converts the between the two conventions.
*/
private def fromZkConfigfEntityName(entityName: String): String = {
if (entityName.equals(ZooKeeperInternals.DEFAULT_STRING)) {
""
} else {
entityName
}
}
private def toZkConfigEntityName(entityName: String): String = {
if (entityName.isEmpty) {
ZooKeeperInternals.DEFAULT_STRING
} else {
entityName
}
}
private def buildClientQuotaEntityData(
entityType: String,
znodeName: String
): EntityData = {
val result = new EntityData().setEntityType(entityType)
if (znodeName.equals(ZooKeeperInternals.DEFAULT_STRING)) {
// Default __client quota__ entity names are null. This is different than default __configs__,
// which have their names set to the empty string instead.
result.setEntityName(null)
} else {
// ZNode names are sanitized before being stored in ZooKeeper.
// For example, @ is turned into %40. Undo the sanitization here.
result.setEntityName(Sanitizer.desanitize(znodeName))
}
result
}
override def iterateClientQuotas(visitor: ClientQuotaVisitor): Unit = {
def migrateEntityType(zkEntityType: String, entityType: String): Unit = {
adminZkClient.fetchAllEntityConfigs(zkEntityType).foreach { case (znodeName, props) =>
val entity = List(buildClientQuotaEntityData(entityType, znodeName)).asJava
ScramMechanism.values().filter(_ != ScramMechanism.UNKNOWN).foreach { mechanism =>
val propertyValue = props.getProperty(mechanism.mechanismName)
if (propertyValue != null) {
val scramCredentials = ScramCredentialUtils.credentialFromString(propertyValue)
logAndRethrow(this, s"Error in client quota visitor for SCRAM credential. User was $entity.") {
visitor.visitScramCredential(Sanitizer.desanitize(znodeName), mechanism, scramCredentials)
}
props.remove(mechanism.mechanismName)
}
}
val quotaMap = ZkAdminManager.clientQuotaPropsToDoubleMap(props.asScala).map {
case (key, value) => key -> lang.Double.valueOf(value)
}.toMap.asJava
if (!quotaMap.isEmpty) {
logAndRethrow(this, s"Error in client quota visitor. Entity was $entity.") {
visitor.visitClientQuota(entity, quotaMap)
}
}
}
}
migrateEntityType(ConfigType.USER, ClientQuotaEntity.USER)
migrateEntityType(ConfigType.CLIENT, ClientQuotaEntity.CLIENT_ID)
adminZkClient.fetchAllChildEntityConfigs(ConfigType.USER, ConfigType.CLIENT).foreach { case (znodePath, props) =>
// Taken from ZkAdminManager
val components = znodePath.split("/")
if (components.size != 3 || components(1) != "clients")
throw new IllegalArgumentException(s"Unexpected config path: $znodePath")
val entity = List(
buildClientQuotaEntityData(ClientQuotaEntity.USER, components(0)),
buildClientQuotaEntityData(ClientQuotaEntity.CLIENT_ID, components(2))
)
val quotaMap = props.asScala.map { case (key, value) =>
val doubleValue = try lang.Double.valueOf(value) catch {
case _: NumberFormatException =>
throw new IllegalStateException(s"Unexpected client quota configuration value: $key -> $value")
}
key -> doubleValue
}.asJava
logAndRethrow(this, s"Error in client quota entity visitor. Entity was $entity.") {
visitor.visitClientQuota(entity.asJava, quotaMap)
}
}
migrateEntityType(ConfigType.IP, ClientQuotaEntity.IP)
}
override def iterateBrokerConfigs(configConsumer: BiConsumer[String, util.Map[String, String]]): Unit = {
val brokerEntities = zkClient.getAllEntitiesWithConfig(ConfigType.BROKER)
zkClient.getEntitiesConfigs(ConfigType.BROKER, brokerEntities.toSet).foreach { case (broker, props) =>
val brokerResource = fromZkConfigfEntityName(broker)
val decodedProps = props.asScala.map { case (key, value) =>
if (DynamicBrokerConfig.isPasswordConfig(key))
key -> passwordEncoder.decode(value).value
else
key -> value
}.toMap.asJava
logAndRethrow(this, s"Error in broker config consumer. Broker was $brokerResource.") {
configConsumer.accept(brokerResource, decodedProps)
}
}
}
override def iterateTopicConfigs(configConsumer: BiConsumer[String, util.Map[String, String]]): Unit = {
val topicEntities = zkClient.getAllEntitiesWithConfig(ConfigType.TOPIC)
topicEntities.foreach { topic =>
readTopicConfigs(topic, props => configConsumer.accept(topic, props))
}
}
override def readTopicConfigs(topicName: String, configConsumer: Consumer[util.Map[String, String]]): Unit = {
val topicResource = fromZkConfigfEntityName(topicName)
val props = zkClient.getEntityConfigs(ConfigType.TOPIC, topicResource)
val decodedProps = props.asScala.map { case (key, value) =>
if (DynamicBrokerConfig.isPasswordConfig(key))
key -> passwordEncoder.decode(value).value
else
key -> value
}.toMap.asJava
logAndRethrow(this, s"Error in topic config consumer. Topic was $topicResource.") {
configConsumer.accept(decodedProps)
}
}
override def writeConfigs(
configResource: ConfigResource,
configMap: util.Map[String, String],
state: ZkMigrationLeadershipState
): ZkMigrationLeadershipState = wrapZkException {
val configType = configResource.`type`() match {
case ConfigResource.Type.BROKER => Some(ConfigType.BROKER)
case ConfigResource.Type.TOPIC => Some(ConfigType.TOPIC)
case _ => None
}
val configName = toZkConfigEntityName(configResource.name())
if (configType.isDefined) {
val props = new Properties()
configMap.forEach { case (key, value) =>
if (DynamicBrokerConfig.isPasswordConfig(key)) {
props.put(key, passwordEncoder.encode(new Password(value)))
} else
props.put(key, value)
}
tryWriteEntityConfig(configType.get, configName, props, create = false, state) match {
case Some(newState) =>
newState
case None =>
val createPath = ConfigEntityTypeZNode.path(configType.get)
debug(s"Recursively creating ZNode $createPath and attempting to write $configResource configs a second time.")
zkClient.createRecursive(createPath, throwIfPathExists = false)
tryWriteEntityConfig(configType.get, configName, props, create = true, state) match {
case Some(newStateSecondTry) => newStateSecondTry
case None => throw new MigrationClientException(
s"Could not write ${configType.get} configs on second attempt when using Create instead of SetData.")
}
}
} else {
error(s"Not updating ZK for $configResource since it is not a Broker or Topic entity.")
state
}
}
override def deleteConfigs(
configResource: ConfigResource,
state: ZkMigrationLeadershipState
): ZkMigrationLeadershipState = wrapZkException {
val configType = configResource.`type`() match {
case ConfigResource.Type.BROKER => Some(ConfigType.BROKER)
case ConfigResource.Type.TOPIC => Some(ConfigType.TOPIC)
case _ => None
}
val configName = toZkConfigEntityName(configResource.name())
if (configType.isDefined) {
val path = ConfigEntityZNode.path(configType.get, configName)
val requests = Seq(DeleteRequest(path, ZkVersion.MatchAnyVersion))
val (migrationZkVersion, responses) = zkClient.retryMigrationRequestsUntilConnected(requests, state)
if (responses.head.resultCode.equals(Code.NONODE)) {
// Not fatal. This is expected in the case this is a topic config and we delete the topic
debug(s"Did not delete $configResource since the node did not exist.")
state
} else if (responses.head.resultCode.equals(Code.OK)) {
// Write the notification znode if our update was successful
zkClient.createConfigChangeNotification(s"${configType.get}/$configName")
state.withMigrationZkVersion(migrationZkVersion)
} else {
throw KeeperException.create(responses.head.resultCode, path)
}
} else {
error(s"Not updating ZK for $configResource since it is not a Broker or Topic entity.")
state
}
}
override def writeClientQuotas(
entity: util.Map[String, String],
quotas: util.Map[String, java.lang.Double],
scram: util.Map[String, String],
state: ZkMigrationLeadershipState
): ZkMigrationLeadershipState = wrapZkException {
val user: Option[String] = getSanitizedClientQuotaZNodeName(entity, ClientQuotaEntity.USER)
val client: Option[String] = getSanitizedClientQuotaZNodeName(entity, ClientQuotaEntity.CLIENT_ID)
val ip: Option[String] = getSanitizedClientQuotaZNodeName(entity, ClientQuotaEntity.IP)
val props = new Properties()
val (configType, path, configKeys) = if (user.isDefined && client.isEmpty) {
(Some(ConfigType.USER), user, DynamicConfig.User.configKeys)
} else if (user.isDefined && client.isDefined) {
(Some(ConfigType.USER), Some(s"${user.get}/clients/${client.get}"),
DynamicConfig.User.configKeys)
} else if (client.isDefined) {
(Some(ConfigType.CLIENT), client, DynamicConfig.Client.configKeys)
} else if (ip.isDefined) {
(Some(ConfigType.IP), ip, DynamicConfig.Ip.configKeys)
} else {
(None, None, Map.empty.asJava)
}
if (path.isEmpty) {
error(s"Skipping unknown client quota entity $entity")
return state
}
// This logic is duplicated from ZkAdminManager
quotas.forEach { case (key, value) =>
val configKey = configKeys.get(key)
if (configKey == null) {
throw new MigrationClientException(s"Invalid configuration key $key")
} else {
configKey.`type` match {
case ConfigDef.Type.DOUBLE =>
props.setProperty(key, value.toString)
case ConfigDef.Type.LONG | ConfigDef.Type.INT =>
val epsilon = 1e-6
val intValue = if (configKey.`type` == ConfigDef.Type.LONG)
(value + epsilon).toLong
else
(value + epsilon).toInt
if ((intValue.toDouble - value).abs > epsilon)
throw new InvalidRequestException(s"Configuration $key must be a ${configKey.`type`} value")
props.setProperty(key, intValue.toString)
case _ =>
throw new MigrationClientException(s"Unexpected config type ${configKey.`type`}")
}
}
}
scram.forEach { case (key, value) => props.put(key, value) }
// Try to write the client quota configs once with create=false, and again with create=true if the first operation fails
tryWriteEntityConfig(configType.get, path.get, props, create = false, state) match {
case Some(newState) =>
newState
case None =>
// If we didn't update the migration state, we failed to write the client quota. Try again
// after recursively create its parent znodes
val createPath = if (user.isDefined && client.isDefined) {
s"${ConfigEntityTypeZNode.path(configType.get)}/${user.get}/clients"
} else {
ConfigEntityTypeZNode.path(configType.get)
}
zkClient.createRecursive(createPath, throwIfPathExists = false)
debug(s"Recursively creating ZNode $createPath and attempting to write $entity quotas a second time.")
tryWriteEntityConfig(configType.get, path.get, props, create = true, state) match {
case Some(newStateSecondTry) => newStateSecondTry
case None => throw new MigrationClientException(
s"Could not write client quotas for $entity on second attempt when using Create instead of SetData")
}
}
}
// Try to update an entity config and the migration state. If NoNode is encountered, it probably means we
// need to recursively create the parent ZNode. In this case, return None.
private def tryWriteEntityConfig(
entityType: String,
path: String,
props: Properties,
create: Boolean,
state: ZkMigrationLeadershipState
): Option[ZkMigrationLeadershipState] = wrapZkException {
val configData = ConfigEntityZNode.encode(props)
val requests = if (create) {
Seq(CreateRequest(ConfigEntityZNode.path(entityType, path), configData, zkClient.defaultAcls(path), CreateMode.PERSISTENT))
} else {
Seq(SetDataRequest(ConfigEntityZNode.path(entityType, path), configData, ZkVersion.MatchAnyVersion))
}
val (migrationZkVersion, responses) = zkClient.retryMigrationRequestsUntilConnected(requests, state)
if (!create && responses.head.resultCode.equals(Code.NONODE)) {
// Not fatal. Just means we need to Create this node instead of SetData
None
} else if (responses.head.resultCode.equals(Code.OK)) {
// Write the notification znode if our update was successful
zkClient.createConfigChangeNotification(s"$entityType/$path")
Some(state.withMigrationZkVersion(migrationZkVersion))
} else {
throw KeeperException.create(responses.head.resultCode, path)
}
}
}
object ZkConfigMigrationClient {
/**
* Find the znode name to use for a ClientQuotaEntity.
*
* @param entity The client quota entity map. See org.apache.kafka.common.ClientQuotaEntity.
* @param component The component that we want a znode name for.
* @return Some(znodeName) if there is a znode path; None otherwise.
*/
def getSanitizedClientQuotaZNodeName(
entity: util.Map[String, String],
component: String
): Option[String] = {
if (!entity.containsKey(component)) {
// There is no znode path, because the component wasn't found. For example, if the
// entity was (user -> "bob") and our component was "ip", we would return None here.
None
} else {
val rawValue = entity.get(component)
if (rawValue == null) {
// A raw value of null means this is a default entity. For example, (user -> null) means
// the default user. Yes, this means we stored a null value in the map and it did not mean
// "not present." This is an unfortunate API that should be revisited at some point.
Some(ZooKeeperInternals.DEFAULT_STRING)
} else {
// We found a non-null value, and now we need to sanitize it. For example, "c@@ldude" will
// turn into c%40%40ldude, so that we can use it as a znode name in ZooKeeper.
Some(Sanitizer.sanitize(rawValue))
}
}
}
}

View File

@ -1,97 +0,0 @@
/*
* 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 kafka.zk.migration
import kafka.utils.Logging
import kafka.zk.ZkMigrationClient.wrapZkException
import kafka.zk._
import kafka.zookeeper._
import org.apache.kafka.metadata.migration.{DelegationTokenMigrationClient, ZkMigrationLeadershipState}
import org.apache.kafka.common.security.token.delegation.TokenInformation
import org.apache.zookeeper.KeeperException.Code
import org.apache.zookeeper.{CreateMode, KeeperException}
import scala.jdk.CollectionConverters._
import scala.collection._
class ZkDelegationTokenMigrationClient(
zkClient: KafkaZkClient
) extends DelegationTokenMigrationClient with Logging {
val adminZkClient = new AdminZkClient(zkClient)
override def getDelegationTokens: java.util.List[String] = {
zkClient.getChildren(DelegationTokensZNode.path).asJava
}
override def writeDelegationToken(
tokenId: String,
tokenInformation: TokenInformation,
state: ZkMigrationLeadershipState
): ZkMigrationLeadershipState = wrapZkException {
val path = DelegationTokenInfoZNode.path(tokenId)
def set(tokenData: Array[Byte]): (Int, Seq[SetDataResponse]) = {
val setRequest = SetDataRequest(path, tokenData, ZkVersion.MatchAnyVersion)
zkClient.retryMigrationRequestsUntilConnected(Seq(setRequest), state)
}
def create(tokenData: Array[Byte]): (Int, Seq[CreateResponse]) = {
val createRequest = CreateRequest(path, tokenData, zkClient.defaultAcls(path), CreateMode.PERSISTENT)
zkClient.retryMigrationRequestsUntilConnected(Seq(createRequest), state)
}
val tokenInfo = DelegationTokenInfoZNode.encode(tokenInformation)
val (setMigrationZkVersion, setResponses) = set(tokenInfo)
if (setResponses.head.resultCode.equals(Code.NONODE)) {
val (createMigrationZkVersion, createResponses) = create(tokenInfo)
if (createResponses.head.resultCode.equals(Code.OK)) {
state.withMigrationZkVersion(createMigrationZkVersion)
} else {
throw KeeperException.create(createResponses.head.resultCode, path)
}
} else if (setResponses.head.resultCode.equals(Code.OK)) {
state.withMigrationZkVersion(setMigrationZkVersion)
} else {
throw KeeperException.create(setResponses.head.resultCode, path)
}
}
override def deleteDelegationToken(
tokenId: String,
state: ZkMigrationLeadershipState
): ZkMigrationLeadershipState = wrapZkException {
val path = DelegationTokenInfoZNode.path(tokenId)
val requests = Seq(DeleteRequest(path, ZkVersion.MatchAnyVersion))
val (migrationZkVersion, responses) = zkClient.retryMigrationRequestsUntilConnected(requests, state)
if (responses.head.resultCode.equals(Code.NONODE)) {
// Not fatal.
error(s"Did not delete $tokenId since the node did not exist.")
state
} else if (responses.head.resultCode.equals(Code.OK)) {
state.withMigrationZkVersion(migrationZkVersion)
} else {
throw KeeperException.create(responses.head.resultCode, path)
}
}
}

View File

@ -1,358 +0,0 @@
/*
* 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 kafka.zk.migration
import kafka.controller.{LeaderIsrAndControllerEpoch, ReplicaAssignment}
import kafka.utils.Logging
import kafka.zk.TopicZNode.TopicIdReplicaAssignment
import kafka.zk.ZkMigrationClient.{logAndRethrow, wrapZkException}
import kafka.zk._
import kafka.zookeeper.{CreateRequest, DeleteRequest, GetChildrenRequest, SetDataRequest}
import org.apache.kafka.common.metadata.PartitionRecord
import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid}
import org.apache.kafka.metadata.migration.TopicMigrationClient.TopicVisitorInterest
import org.apache.kafka.metadata.migration.{MigrationClientException, TopicMigrationClient, ZkMigrationLeadershipState}
import org.apache.kafka.metadata.{LeaderAndIsr, LeaderRecoveryState, PartitionRegistration}
import org.apache.kafka.server.config.ConfigType
import org.apache.zookeeper.CreateMode
import org.apache.zookeeper.KeeperException.Code
import java.util
import java.util.Properties
import scala.collection.Seq
import scala.collection.mutable.ArrayBuffer
import scala.jdk.CollectionConverters._
class ZkTopicMigrationClient(zkClient: KafkaZkClient) extends TopicMigrationClient with Logging {
override def iterateTopics(
interests: util.EnumSet[TopicVisitorInterest],
visitor: TopicMigrationClient.TopicVisitor,
): Unit = wrapZkException {
if (!interests.contains(TopicVisitorInterest.TOPICS)) {
throw new IllegalArgumentException("Must specify at least TOPICS in topic visitor interests.")
}
val allTopics = zkClient.getAllTopicsInCluster()
val topicDeletions = readPendingTopicDeletions().asScala
val topicsToMigrated = allTopics -- topicDeletions
if (topicDeletions.nonEmpty) {
warn(s"Found ${topicDeletions.size} pending topic deletions. These will be not migrated " +
s"to KRaft. After the migration, the brokers will reconcile their logs with these pending topic deletions.")
}
topicDeletions.foreach {
deletion => logger.info(s"Not migrating pending deleted topic: $deletion")
}
val replicaAssignmentAndTopicIds = zkClient.getReplicaAssignmentAndTopicIdForTopics(topicsToMigrated)
replicaAssignmentAndTopicIds.foreach { case TopicIdReplicaAssignment(topic, topicIdOpt, partitionAssignments) =>
val topicAssignment = partitionAssignments.map { case (partition, assignment) =>
partition.partition().asInstanceOf[Integer] -> assignment.replicas.map(Integer.valueOf).asJava
}.toMap.asJava
logAndRethrow(this, s"Error in topic consumer. Topic was $topic.") {
visitor.visitTopic(topic, topicIdOpt.get, topicAssignment)
}
if (interests.contains(TopicVisitorInterest.PARTITIONS)) {
val partitions = partitionAssignments.keys.toSeq
val leaderIsrAndControllerEpochs = zkClient.getTopicPartitionStates(partitions)
partitionAssignments.foreach { case (topicPartition, replicaAssignment) =>
val replicaList = replicaAssignment.replicas.map(Integer.valueOf).asJava
val record = new PartitionRecord()
.setTopicId(topicIdOpt.get)
.setPartitionId(topicPartition.partition)
.setReplicas(replicaList)
.setAddingReplicas(replicaAssignment.addingReplicas.map(Integer.valueOf).asJava)
.setRemovingReplicas(replicaAssignment.removingReplicas.map(Integer.valueOf).asJava)
leaderIsrAndControllerEpochs.get(topicPartition) match {
case Some(leaderIsrAndEpoch) =>
record
.setIsr(leaderIsrAndEpoch.leaderAndIsr.isr)
.setLeader(leaderIsrAndEpoch.leaderAndIsr.leader)
.setLeaderEpoch(leaderIsrAndEpoch.leaderAndIsr.leaderEpoch)
.setPartitionEpoch(leaderIsrAndEpoch.leaderAndIsr.partitionEpoch)
.setLeaderRecoveryState(leaderIsrAndEpoch.leaderAndIsr.leaderRecoveryState.value())
case None =>
warn(s"Could not find partition state in ZK for $topicPartition. Initializing this partition " +
s"with ISR={$replicaList} and leaderEpoch=0.")
record
.setIsr(replicaList)
.setLeader(replicaList.get(0))
.setLeaderEpoch(0)
.setPartitionEpoch(0)
.setLeaderRecoveryState(LeaderRecoveryState.RECOVERED.value())
}
logAndRethrow(this, s"Error in partition consumer. TopicPartition was $topicPartition.") {
visitor.visitPartition(new TopicIdPartition(topicIdOpt.get, topicPartition), new PartitionRegistration(record))
}
}
}
}
}
override def createTopic(
topicName: String,
topicId: Uuid,
partitions: util.Map[Integer, PartitionRegistration],
state: ZkMigrationLeadershipState
): ZkMigrationLeadershipState = wrapZkException {
val assignments = partitions.asScala.map { case (partitionId, partition) =>
new TopicPartition(topicName, partitionId) ->
ReplicaAssignment(partition.replicas, partition.addingReplicas, partition.removingReplicas)
}
val createTopicZNode = {
val path = TopicZNode.path(topicName)
CreateRequest(
path,
TopicZNode.encode(Some(topicId), assignments),
zkClient.defaultAcls(path),
CreateMode.PERSISTENT)
}
val topicConfigZNode = {
val path = ConfigEntityZNode.path(ConfigType.TOPIC, topicName)
CreateRequest(
path,
ConfigEntityZNode.encode(new Properties()),
zkClient.defaultAcls(path),
CreateMode.PERSISTENT)
}
val createPartitionZNodeReqs = createTopicPartitionZNodesRequests(topicName, partitions, state)
val requests = Seq(createTopicZNode, topicConfigZNode) ++ createPartitionZNodeReqs
val (migrationZkVersion, responses) = zkClient.retryMigrationRequestsUntilConnected(requests, state)
val resultCodes = responses.map { response => response.path -> response.resultCode }.toMap
if (resultCodes(TopicZNode.path(topicName)).equals(Code.NODEEXISTS)) {
// topic already created, just return
state
} else if (resultCodes.forall { case (_, code) => code.equals(Code.OK) }) {
// ok
state.withMigrationZkVersion(migrationZkVersion)
} else {
// not ok
throw new MigrationClientException(s"Failed to create or update topic $topicName. ZK operations had results $resultCodes")
}
}
private def createTopicPartitionZNodesRequests(
topicName: String,
partitions: util.Map[Integer, PartitionRegistration],
state: ZkMigrationLeadershipState
): Seq[CreateRequest] = {
val createPartitionsZNode = {
val path = TopicPartitionsZNode.path(topicName)
CreateRequest(
path,
null,
zkClient.defaultAcls(path),
CreateMode.PERSISTENT)
}
val createPartitionZNodeReqs = partitions.asScala.toSeq.flatMap { case (partitionId, partition) =>
val topicPartition = new TopicPartition(topicName, partitionId)
Seq(
createTopicPartition(topicPartition),
createTopicPartitionState(topicPartition, partition, state.kraftControllerEpoch())
)
}
Seq(createPartitionsZNode) ++ createPartitionZNodeReqs
}
private def recursiveChildren(path: String, acc: ArrayBuffer[String]): Unit = {
val topicChildZNodes = zkClient.retryRequestUntilConnected(GetChildrenRequest(path, registerWatch = false))
topicChildZNodes.children.foreach { child =>
recursiveChildren(s"$path/$child", acc)
acc.append(s"$path/$child")
}
}
private def recursiveChildren(path: String): Seq[String] = {
val buffer = new ArrayBuffer[String]()
recursiveChildren(path, buffer)
buffer.toSeq
}
override def updateTopic(
topicName: String,
topicId: Uuid,
partitions: util.Map[Integer, PartitionRegistration],
state: ZkMigrationLeadershipState
): ZkMigrationLeadershipState = wrapZkException {
val assignments = partitions.asScala.map { case (partitionId, partition) =>
new TopicPartition(topicName, partitionId) ->
ReplicaAssignment(partition.replicas, partition.addingReplicas, partition.removingReplicas)
}
val request = SetDataRequest(
TopicZNode.path(topicName),
TopicZNode.encode(Some(topicId), assignments),
ZkVersion.MatchAnyVersion
)
val (migrationZkVersion, responses) = zkClient.retryMigrationRequestsUntilConnected(Seq(request), state)
val resultCodes = responses.map { response => response.path -> response.resultCode }.toMap
if (resultCodes.forall { case (_, code) => code.equals(Code.OK) } ) {
state.withMigrationZkVersion(migrationZkVersion)
} else {
throw new MigrationClientException(s"Failed to update topic metadata: $topicName. ZK transaction had results $resultCodes")
}
}
override def deleteTopic(
topicName: String,
state: ZkMigrationLeadershipState
): ZkMigrationLeadershipState = wrapZkException {
// Delete the partition state ZNodes recursively, then topic config, and finally the topic znode
val topicPath = TopicZNode.path(topicName)
val topicChildZNodes = recursiveChildren(topicPath)
val deleteRequests = topicChildZNodes.map { childPath =>
DeleteRequest(childPath, ZkVersion.MatchAnyVersion)
} ++ Seq(
DeleteRequest(ConfigEntityZNode.path(ConfigType.TOPIC, topicName), ZkVersion.MatchAnyVersion),
DeleteRequest(TopicZNode.path(topicName), ZkVersion.MatchAnyVersion)
)
val (migrationZkVersion, responses) = zkClient.retryMigrationRequestsUntilConnected(deleteRequests, state)
val resultCodes = responses.map { response => response.path -> response.resultCode }.toMap
if (responses.last.resultCode.equals(Code.OK)) {
state.withMigrationZkVersion(migrationZkVersion)
} else {
throw new MigrationClientException(s"Failed to delete topic $topicName. ZK operations had results $resultCodes")
}
}
override def createTopicPartitions(topicPartitions: util.Map[String, util.Map[Integer, PartitionRegistration]], state: ZkMigrationLeadershipState)
:ZkMigrationLeadershipState = wrapZkException {
val requests = topicPartitions.asScala.toSeq.flatMap { case (topicName, partitions) =>
createTopicPartitionZNodesRequests(topicName, partitions, state)
}
val (migrationZkVersion, responses) = zkClient.retryMigrationRequestsUntilConnected(requests, state)
val resultCodes = responses.map { response => response.path -> response.resultCode }.toMap
if (resultCodes.forall { case (_, code) => code.equals(Code.OK) || code.equals(Code.NODEEXISTS) }) {
state.withMigrationZkVersion(migrationZkVersion)
} else {
throw new MigrationClientException(s"Failed to create partition states: $topicPartitions. ZK transaction had results $resultCodes")
}
}
override def updateTopicPartitions(
topicPartitions: util.Map[String, util.Map[Integer, PartitionRegistration]],
state: ZkMigrationLeadershipState
): ZkMigrationLeadershipState = wrapZkException {
val requests = topicPartitions.asScala.flatMap { case (topicName, partitionRegistrations) =>
partitionRegistrations.asScala.flatMap { case (partitionId, partitionRegistration) =>
val topicPartition = new TopicPartition(topicName, partitionId)
Seq(updateTopicPartitionState(topicPartition, partitionRegistration, state.kraftControllerEpoch()))
}
}
if (requests.isEmpty) {
state
} else {
val (migrationZkVersion, responses) = zkClient.retryMigrationRequestsUntilConnected(requests.toSeq, state)
val resultCodes = responses.map { response => response.path -> response.resultCode }.toMap
if (resultCodes.forall { case (_, code) => code.equals(Code.OK) }) {
state.withMigrationZkVersion(migrationZkVersion)
} else {
throw new MigrationClientException(s"Failed to update partition states: $topicPartitions. ZK transaction had results $resultCodes")
}
}
}
override def deleteTopicPartitions(
topicPartitions: util.Map[String, util.Set[Integer]],
state: ZkMigrationLeadershipState
): ZkMigrationLeadershipState = {
val requests = topicPartitions.asScala.flatMap { case (topicName, partitionIds) =>
partitionIds.asScala.map { partitionId =>
val topicPartition = new TopicPartition(topicName, partitionId)
val path = TopicPartitionZNode.path(topicPartition)
DeleteRequest(path, ZkVersion.MatchAnyVersion)
}
}
if (requests.isEmpty) {
state
} else {
val (migrationZkVersion, responses) = zkClient.retryMigrationRequestsUntilConnected(requests.toSeq, state)
val resultCodes = responses.map { response => response.path -> response.resultCode }.toMap
if (resultCodes.forall { case (_, code) => code.equals(Code.OK) }) {
state.withMigrationZkVersion(migrationZkVersion)
} else {
throw new MigrationClientException(s"Failed to delete partition states: $topicPartitions. ZK transaction had results $resultCodes")
}
}
}
private def createTopicPartition(
topicPartition: TopicPartition
): CreateRequest = wrapZkException {
val path = TopicPartitionZNode.path(topicPartition)
CreateRequest(path, null, zkClient.defaultAcls(path), CreateMode.PERSISTENT, Some(topicPartition))
}
private def partitionStatePathAndData(
topicPartition: TopicPartition,
partitionRegistration: PartitionRegistration,
controllerEpoch: Int
): (String, Array[Byte]) = {
val path = TopicPartitionStateZNode.path(topicPartition)
val data = TopicPartitionStateZNode.encode(LeaderIsrAndControllerEpoch(new LeaderAndIsr(
partitionRegistration.leader,
partitionRegistration.leaderEpoch,
partitionRegistration.isr.toList.map(Integer.valueOf).asJava,
partitionRegistration.leaderRecoveryState,
partitionRegistration.partitionEpoch), controllerEpoch))
(path, data)
}
private def createTopicPartitionState(
topicPartition: TopicPartition,
partitionRegistration: PartitionRegistration,
controllerEpoch: Int
): CreateRequest = {
val (path, data) = partitionStatePathAndData(topicPartition, partitionRegistration, controllerEpoch)
CreateRequest(path, data, zkClient.defaultAcls(path), CreateMode.PERSISTENT, Some(topicPartition))
}
private def updateTopicPartitionState(
topicPartition: TopicPartition,
partitionRegistration: PartitionRegistration,
controllerEpoch: Int
): SetDataRequest = {
val (path, data) = partitionStatePathAndData(topicPartition, partitionRegistration, controllerEpoch)
SetDataRequest(path, data, ZkVersion.MatchAnyVersion, Some(topicPartition))
}
override def readPendingTopicDeletions(): util.Set[String] = {
zkClient.getTopicDeletions.toSet.asJava
}
override def clearPendingTopicDeletions(
pendingTopicDeletions: util.Set[String],
state: ZkMigrationLeadershipState
): ZkMigrationLeadershipState = {
val deleteRequests = pendingTopicDeletions.asScala.map { topicName =>
DeleteRequest(DeleteTopicsTopicZNode.path(topicName), ZkVersion.MatchAnyVersion)
}.toSeq
val (migrationZkVersion, responses) = zkClient.retryMigrationRequestsUntilConnected(deleteRequests, state)
val resultCodes = responses.map { response => response.path -> response.resultCode }.toMap
if (resultCodes.forall { case (_, code) => code.equals(Code.OK) }) {
state.withMigrationZkVersion(migrationZkVersion)
} else {
throw new MigrationClientException(s"Failed to delete pending topic deletions: $pendingTopicDeletions. ZK transaction had results $resultCodes")
}
}
}

View File

@ -82,7 +82,7 @@ class SocketServerTest {
// Clean-up any metrics left around by previous tests
TestUtils.clearYammerMetrics()
private val apiVersionManager = new SimpleApiVersionManager(ListenerType.BROKER, true, false,
private val apiVersionManager = new SimpleApiVersionManager(ListenerType.BROKER, true,
() => new FinalizedFeatures(MetadataVersion.latestTesting(), Collections.emptyMap[String, java.lang.Short], 0, true))
var server: SocketServer = _
val sockets = new ArrayBuffer[Socket]

View File

@ -17,8 +17,7 @@
package kafka.server
import org.apache.kafka.common.test.api.{AutoStart, ClusterConfigProperty, ClusterInstance, ClusterTest, ClusterTestExtensions, ClusterTests, Type}
import org.apache.kafka.common.test.api.RaftClusterInvocationContext.RaftClusterInstance
import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterInstance, ClusterTest, ClusterTestExtensions, Type}
import org.apache.kafka.clients.ClientResponse
import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic
import org.apache.kafka.common.message.{BrokerRegistrationRequestData, CreateTopicsRequestData}
@ -31,8 +30,7 @@ import org.apache.kafka.common.utils.Time
import org.apache.kafka.common.{Node, Uuid}
import org.apache.kafka.server.{ControllerRequestCompletionHandler, NodeToControllerChannelManager}
import org.apache.kafka.server.common.{Features, MetadataVersion}
import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows}
import org.junit.jupiter.api.Disabled
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.extension.ExtendWith
import java.util
@ -205,91 +203,4 @@ class BrokerRegistrationRequestTest {
channelManager.shutdown()
}
}
@Disabled("zk migration is removed from ClusterTestExtensions")
@ClusterTest(
types = Array(Type.KRAFT),
brokers = 1,
controllers = 1,
metadataVersion = MetadataVersion.IBP_3_4_IV0,
autoStart = AutoStart.NO,
serverProperties = Array(new ClusterConfigProperty(key = "zookeeper.metadata.migration.enable", value = "true")))
def testRegisterZkWithKRaftMigrationEnabled(clusterInstance: ClusterInstance): Unit = {
clusterInstance.asInstanceOf[RaftClusterInstance].controllers().values().forEach(_.startup())
val clusterId = clusterInstance.clusterId()
val channelManager = brokerToControllerChannelManager(clusterInstance)
try {
channelManager.start()
assertEquals(
Errors.NONE,
registerBroker(channelManager, clusterId, 100, Some(1), Some((MetadataVersion.IBP_3_4_IV0, MetadataVersion.IBP_3_4_IV0))))
assertEquals(
Errors.UNSUPPORTED_VERSION,
registerBroker(channelManager, clusterId, 100, Some(1), None))
assertEquals(
Errors.UNSUPPORTED_VERSION,
registerBroker(channelManager, clusterId, 100, Some(1), Some((MetadataVersion.IBP_3_3_IV3, MetadataVersion.IBP_3_3_IV3))))
// Cannot register KRaft broker when in pre-migration
assertEquals(
Errors.BROKER_ID_NOT_REGISTERED,
registerBroker(channelManager, clusterId, 100, None, Some((MetadataVersion.IBP_3_4_IV0, MetadataVersion.IBP_3_4_IV0))))
} finally {
channelManager.shutdown()
}
}
@Disabled("zk migration is removed from ClusterTestExtensions")
@ClusterTest(
types = Array(Type.KRAFT),
brokers = 1,
controllers = 1,
metadataVersion = MetadataVersion.IBP_3_9_IV0,
autoStart = AutoStart.NO,
serverProperties = Array(new ClusterConfigProperty(key = "zookeeper.metadata.migration.enable", value = "true")))
def testRegisterZkWithKRaftMigrationEnabledKRaftV1(clusterInstance: ClusterInstance): Unit = {
clusterInstance.asInstanceOf[RaftClusterInstance].controllers().values().forEach(_.startup())
val clusterId = clusterInstance.clusterId()
val channelManager = brokerToControllerChannelManager(clusterInstance)
try {
channelManager.start()
assertEquals(
Errors.NONE,
registerBroker(channelManager, clusterId, 100, Some(1), Some((MetadataVersion.IBP_3_9_IV0, MetadataVersion.IBP_3_9_IV0))))
// Cannot register KRaft broker when in pre-migration
assertEquals(
Errors.BROKER_ID_NOT_REGISTERED,
registerBroker(channelManager, clusterId, 100, None, Some((MetadataVersion.IBP_3_9_IV0, MetadataVersion.IBP_3_9_IV0))))
} finally {
channelManager.shutdown()
}
}
/**
* Start a KRaft cluster with migrations enabled, verify that the controller does not accept metadata changes
* through the RPCs. The migration never proceeds past pre-migration since no ZK brokers are registered.
*/
@Disabled("zk migration is removed from ClusterTestExtensions")
@ClusterTests(Array(
new ClusterTest(types = Array(Type.KRAFT), autoStart = AutoStart.NO, controllers = 1, metadataVersion = MetadataVersion.IBP_3_4_IV0,
serverProperties = Array(new ClusterConfigProperty(key = "zookeeper.metadata.migration.enable", value = "true")))
))
def testNoMetadataChangesInPreMigrationMode(clusterInstance: ClusterInstance): Unit = {
clusterInstance.asInstanceOf[RaftClusterInstance].controllers().values().forEach(_.startup())
val channelManager = brokerToControllerChannelManager(clusterInstance)
try {
channelManager.start()
assertThrows(classOf[TimeoutException], () => createTopics(channelManager, "test-pre-migration"))
} finally {
channelManager.shutdown()
}
}
}

View File

@ -170,7 +170,6 @@ class ControllerApisTest {
new SimpleApiVersionManager(
ListenerType.CONTROLLER,
true,
false,
() => FinalizedFeatures.fromKRaftVersion(MetadataVersion.latestTesting())),
metadataCache
)

View File

@ -75,7 +75,6 @@ class ControllerRegistrationManagerTest {
Time.SYSTEM,
"controller-registration-manager-test-",
createSupportedFeatures(MetadataVersion.IBP_3_7_IV0),
false,
RecordTestUtils.createTestControllerRegistration(1, false).incarnationId(),
ListenerInfo.create(context.config.controllerListeners.map(_.toJava).asJava),
new ExponentialBackoff(1, 2, 100, 0.02))

View File

@ -217,7 +217,6 @@ class KafkaApisTest extends Logging {
enabledApis,
BrokerFeatures.defaultSupportedFeatures(true),
true,
false,
() => new FinalizedFeatures(MetadataVersion.latestTesting(), Collections.emptyMap[String, java.lang.Short], 0, raftSupport))
val clientMetricsManagerOpt = if (raftSupport) Some(clientMetricsManager) else None

View File

@ -208,7 +208,6 @@ public class KRaftMetadataRequestBenchmark {
setApiVersionManager(new SimpleApiVersionManager(
ApiMessageType.ListenerType.BROKER,
false,
false,
() -> FinalizedFeatures.fromKRaftVersion(MetadataVersion.latestTesting()))).
build();
}

View File

@ -208,7 +208,6 @@ public class MetadataRequestBenchmark {
setApiVersionManager(new SimpleApiVersionManager(
ApiMessageType.ListenerType.ZK_BROKER,
false,
false,
() -> FinalizedFeatures.fromKRaftVersion(MetadataVersion.latestTesting()))).
build();
}

View File

@ -29,12 +29,14 @@ import java.util.ArrayList;
import java.util.List;
import java.util.function.Consumer;
import static org.apache.kafka.metadata.migration.ZkMigrationState.NONE;
import static org.apache.kafka.metadata.migration.ZkMigrationState.POST_MIGRATION;
public class ActivationRecordsGenerator {
static ControllerResult<Void> recordsForEmptyLog(
Consumer<String> activationMessageConsumer,
long transactionStartOffset,
boolean zkMigrationEnabled,
BootstrapMetadata bootstrapMetadata,
MetadataVersion metadataVersion
) {
@ -89,20 +91,9 @@ public class ActivationRecordsGenerator {
records.addAll(bootstrapMetadata.records());
if (metadataVersion.isMigrationSupported()) {
if (zkMigrationEnabled) {
logMessageBuilder.append("Putting the controller into pre-migration mode. No metadata updates " +
"will be allowed until the ZK metadata has been migrated. ");
records.add(ZkMigrationState.PRE_MIGRATION.toRecord());
} else {
logMessageBuilder.append("Setting the ZK migration state to NONE since this is a de-novo " +
"KRaft cluster. ");
records.add(ZkMigrationState.NONE.toRecord());
}
} else {
if (zkMigrationEnabled) {
throw new RuntimeException("The bootstrap metadata.version " + bootstrapMetadata.metadataVersion() +
" does not support ZK migrations. Cannot continue with ZK migrations enabled.");
}
records.add(NONE.toRecord());
}
activationMessageConsumer.accept(logMessageBuilder.toString().trim());
@ -117,9 +108,8 @@ public class ActivationRecordsGenerator {
static ControllerResult<Void> recordsForNonEmptyLog(
Consumer<String> activationMessageConsumer,
long transactionStartOffset,
boolean zkMigrationEnabled,
FeatureControlManager featureControl,
MetadataVersion metadataVersion
ZkMigrationState zkMigrationState,
MetadataVersion curMetadataVersion
) {
StringBuilder logMessageBuilder = new StringBuilder("Performing controller activation. ");
@ -128,9 +118,9 @@ public class ActivationRecordsGenerator {
// Check for in-flight transaction
if (transactionStartOffset != -1L) {
if (!metadataVersion.isMetadataTransactionSupported()) {
if (!curMetadataVersion.isMetadataTransactionSupported()) {
throw new RuntimeException("Detected in-progress transaction at offset " + transactionStartOffset +
", but the metadata.version " + metadataVersion +
", but the metadata.version " + curMetadataVersion +
" does not support transactions. Cannot continue.");
} else {
logMessageBuilder
@ -142,64 +132,29 @@ public class ActivationRecordsGenerator {
}
}
if (metadataVersion.equals(MetadataVersion.MINIMUM_KRAFT_VERSION)) {
if (curMetadataVersion.equals(MetadataVersion.MINIMUM_KRAFT_VERSION)) {
logMessageBuilder.append("No metadata.version feature level record was found in the log. ")
.append("Treating the log as version ")
.append(MetadataVersion.MINIMUM_KRAFT_VERSION)
.append(". ");
}
if (zkMigrationEnabled && !metadataVersion.isMigrationSupported()) {
throw new RuntimeException("Should not have ZK migrations enabled on a cluster running " +
"metadata.version " + featureControl.metadataVersion());
} else if (metadataVersion.isMigrationSupported()) {
if (curMetadataVersion.isMigrationSupported()) {
if (zkMigrationState == NONE || zkMigrationState == POST_MIGRATION) {
logMessageBuilder
.append("Loaded ZK migration state of ")
.append(featureControl.zkMigrationState())
.append(zkMigrationState)
.append(". ");
switch (featureControl.zkMigrationState()) {
case NONE:
// Since this is the default state there may or may not be an actual NONE in the log. Regardless,
// it will eventually be persisted in a snapshot, so we don't need to explicitly write it here.
if (zkMigrationEnabled) {
throw new RuntimeException("Should not have ZK migrations enabled on a cluster that was " +
"created in KRaft mode.");
}
if (zkMigrationState == NONE) {
logMessageBuilder.append("This is expected because this is a de-novo KRaft cluster.");
break;
case PRE_MIGRATION:
if (!metadataVersion.isMetadataTransactionSupported()) {
logMessageBuilder
.append("Activating pre-migration controller without empty log. ")
.append("There may be a partial migration. ");
}
break;
case MIGRATION:
if (!zkMigrationEnabled) {
// This can happen if controller leadership transfers to a controller with migrations enabled
// after another controller had finalized the migration. For example, during a rolling restart
// of the controller quorum during which the migration config is being set to false.
logMessageBuilder
.append("Completing the ZK migration since this controller was configured with ")
.append("'zookeeper.metadata.migration.enable' set to 'false'. ");
records.add(ZkMigrationState.POST_MIGRATION.toRecord());
} else {
// This log message is used in zookeeper_migration_test.py
logMessageBuilder
.append("Staying in ZK migration mode since 'zookeeper.metadata.migration.enable' ")
.append("is still 'true'. ");
}
break;
case POST_MIGRATION:
if (zkMigrationEnabled) {
logMessageBuilder
.append("Ignoring 'zookeeper.metadata.migration.enable' value of 'true' since ")
.append("the ZK migration has been completed. ");
}
break;
default:
throw new IllegalStateException("Unsupported ZkMigrationState " + featureControl.zkMigrationState());
throw new RuntimeException("Cannot load ZkMigrationState." + zkMigrationState +
" because ZK migration is no longer supported.");
}
} else if (zkMigrationState != NONE) {
throw new RuntimeException("Should not have ZkMigrationState." + zkMigrationState +
" on a cluster running metadata version " + curMetadataVersion + ".");
}
activationMessageConsumer.accept(logMessageBuilder.toString().trim());
@ -220,16 +175,16 @@ public class ActivationRecordsGenerator {
Consumer<String> activationMessageConsumer,
boolean isEmpty,
long transactionStartOffset,
boolean zkMigrationEnabled,
BootstrapMetadata bootstrapMetadata,
FeatureControlManager featureControl
ZkMigrationState zkMigrationState,
MetadataVersion curMetadataVersion
) {
if (isEmpty) {
return recordsForEmptyLog(activationMessageConsumer, transactionStartOffset, zkMigrationEnabled,
return recordsForEmptyLog(activationMessageConsumer, transactionStartOffset,
bootstrapMetadata, bootstrapMetadata.metadataVersion());
} else {
return recordsForNonEmptyLog(activationMessageConsumer, transactionStartOffset, zkMigrationEnabled,
featureControl, featureControl.metadataVersion());
return recordsForNonEmptyLog(activationMessageConsumer, transactionStartOffset,
zkMigrationState, curMetadataVersion);
}
}
}

View File

@ -91,7 +91,6 @@ public class ClusterControlManager {
private long sessionTimeoutNs = DEFAULT_SESSION_TIMEOUT_NS;
private ReplicaPlacer replicaPlacer = null;
private FeatureControlManager featureControl = null;
private boolean zkMigrationEnabled = false;
private BrokerUncleanShutdownHandler brokerUncleanShutdownHandler = null;
private String interBrokerListenerName = "PLAINTEXT";
@ -130,11 +129,6 @@ public class ClusterControlManager {
return this;
}
Builder setZkMigrationEnabled(boolean zkMigrationEnabled) {
this.zkMigrationEnabled = zkMigrationEnabled;
return this;
}
Builder setBrokerUncleanShutdownHandler(BrokerUncleanShutdownHandler brokerUncleanShutdownHandler) {
this.brokerUncleanShutdownHandler = brokerUncleanShutdownHandler;
return this;
@ -171,7 +165,6 @@ public class ClusterControlManager {
sessionTimeoutNs,
replicaPlacer,
featureControl,
zkMigrationEnabled,
brokerUncleanShutdownHandler,
interBrokerListenerName
);
@ -260,11 +253,6 @@ public class ClusterControlManager {
*/
private final FeatureControlManager featureControl;
/**
* True if migration from ZK is enabled.
*/
private final boolean zkMigrationEnabled;
private final BrokerUncleanShutdownHandler brokerUncleanShutdownHandler;
/**
@ -290,7 +278,6 @@ public class ClusterControlManager {
long sessionTimeoutNs,
ReplicaPlacer replicaPlacer,
FeatureControlManager featureControl,
boolean zkMigrationEnabled,
BrokerUncleanShutdownHandler brokerUncleanShutdownHandler,
String interBrokerListenerName
) {
@ -305,7 +292,6 @@ public class ClusterControlManager {
this.heartbeatManager = null;
this.readyBrokersFuture = Optional.empty();
this.featureControl = featureControl;
this.zkMigrationEnabled = zkMigrationEnabled;
this.controllerRegistrations = new TimelineHashMap<>(snapshotRegistry, 0);
this.directoryToBroker = new TimelineHashMap<>(snapshotRegistry, 0);
this.brokerUncleanShutdownHandler = brokerUncleanShutdownHandler;
@ -349,10 +335,6 @@ public class ClusterControlManager {
.collect(Collectors.toSet());
}
boolean zkRegistrationAllowed() {
return zkMigrationEnabled && featureControl.metadataVersion().isMigrationSupported();
}
/**
* Process an incoming broker registration request.
*/
@ -382,20 +364,8 @@ public class ClusterControlManager {
}
}
if (request.isMigratingZkBroker() && !zkRegistrationAllowed()) {
throw new BrokerIdNotRegisteredException("Controller does not support registering ZK brokers.");
}
if (!request.isMigratingZkBroker() && featureControl.inPreMigrationMode()) {
throw new BrokerIdNotRegisteredException("Controller is in pre-migration mode and cannot register KRaft " +
"brokers until the metadata migration is complete.");
}
if (request.isMigratingZkBroker()) {
if (request.listeners().find(interBrokerListenerName) == null) {
throw new InvalidRegistrationException("Broker does not have the current inter.broker.listener " +
interBrokerListenerName);
}
throw new BrokerIdNotRegisteredException("Controller does not support registering ZK brokers.");
}
if (featureControl.metadataVersion().isDirectoryAssignmentSupported()) {
@ -510,7 +480,7 @@ public class ClusterControlManager {
records.add(new ApiMessageAndVersion(new RegisterControllerRecord().
setControllerId(request.controllerId()).
setIncarnationId(request.incarnationId()).
setZkMigrationReady(request.zkMigrationReady()).
setZkMigrationReady(false).
setEndPoints(listenerInfo.toControllerRegistrationRecord()).
setFeatures(features),
(short) 0));

View File

@ -397,16 +397,6 @@ public class FeatureControlManager {
return new FinalizedControllerFeatures(features, epoch);
}
/**
* Tests if the controller should be preventing metadata updates due to being in the PRE_MIGRATION
* state. If the controller does not yet support migrations (before 3.4-IV0), then the migration state
* will be NONE and this will return false. Once the controller has been upgraded to a version that supports
* migrations, then this method checks if the migration state is equal to PRE_MIGRATION.
*/
boolean inPreMigrationMode() {
return migrationControlState.get().equals(ZkMigrationState.PRE_MIGRATION);
}
public void replay(FeatureLevelRecord record) {
VersionRange range = quorumFeatures.localSupportedFeature(record.name());
if (!range.contains(record.featureLevel())) {

View File

@ -75,7 +75,7 @@ class OffsetControlManager {
if (logContext == null) logContext = new LogContext();
if (snapshotRegistry == null) snapshotRegistry = new SnapshotRegistry(logContext);
if (metrics == null) {
metrics = new QuorumControllerMetrics(Optional.empty(), time, false);
metrics = new QuorumControllerMetrics(Optional.empty(), time);
}
return new OffsetControlManager(logContext,
snapshotRegistry,

View File

@ -97,7 +97,6 @@ public class PartitionChangeBuilder {
private List<Integer> uncleanShutdownReplicas;
private Election election = Election.ONLINE;
private LeaderRecoveryState targetLeaderRecoveryState;
private boolean zkMigrationEnabled;
private boolean eligibleLeaderReplicasEnabled;
private DefaultDirProvider defaultDirProvider;
@ -118,7 +117,6 @@ public class PartitionChangeBuilder {
this.partitionId = partitionId;
this.isAcceptableLeader = isAcceptableLeader;
this.metadataVersion = metadataVersion;
this.zkMigrationEnabled = false;
this.eligibleLeaderReplicasEnabled = false;
this.minISR = minISR;
@ -179,11 +177,6 @@ public class PartitionChangeBuilder {
return this;
}
public PartitionChangeBuilder setZkMigrationEnabled(boolean zkMigrationEnabled) {
this.zkMigrationEnabled = zkMigrationEnabled;
return this;
}
public PartitionChangeBuilder setEligibleLeaderReplicasEnabled(boolean eligibleLeaderReplicasEnabled) {
this.eligibleLeaderReplicasEnabled = eligibleLeaderReplicasEnabled;
return this;
@ -392,17 +385,11 @@ public class PartitionChangeBuilder {
* the PartitionChangeRecord.
*/
void triggerLeaderEpochBumpForIsrShrinkIfNeeded(PartitionChangeRecord record) {
if (!(metadataVersion.isLeaderEpochBumpRequiredOnIsrShrink() || zkMigrationEnabled)) {
// We only need to bump the leader epoch on an ISR shrink in two cases:
//
// 1. In older metadata versions before 3.6, there was a bug (KAFKA-15021) in the
// broker replica manager that required that the leader epoch be bumped whenever
// the ISR shrank. (This was never necessary for EXPANSIONS, only SHRINKS.)
//
// 2. During ZK migration, we bump the leader epoch during all ISR shrinks, in order
// to maintain compatibility with migrating brokers that are still in ZK mode.
//
// If we're not in either case, we can exit here.
if (!metadataVersion.isLeaderEpochBumpRequiredOnIsrShrink()) {
// We only need to bump the leader epoch on an ISR shrink in older metadata versions
// before 3.6, where there was a bug (KAFKA-15021) in the broker replica manager that
// required that the leader epoch be bumped whenever the ISR shrank. (This was never
// necessary for EXPANSIONS, only SHRINKS.)
return;
}
if (record.leader() != NO_LEADER_CHANGE) {

View File

@ -103,8 +103,6 @@ import org.apache.kafka.metadata.FinalizedControllerFeatures;
import org.apache.kafka.metadata.KafkaConfigSchema;
import org.apache.kafka.metadata.VersionRange;
import org.apache.kafka.metadata.bootstrap.BootstrapMetadata;
import org.apache.kafka.metadata.migration.ZkMigrationState;
import org.apache.kafka.metadata.migration.ZkRecordConsumer;
import org.apache.kafka.metadata.placement.ReplicaPlacer;
import org.apache.kafka.metadata.placement.StripedReplicaPlacer;
import org.apache.kafka.metadata.util.RecordRedactor;
@ -131,7 +129,6 @@ import org.apache.kafka.timeline.SnapshotRegistry;
import org.slf4j.Logger;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.EnumSet;
@ -154,7 +151,6 @@ import java.util.function.Supplier;
import static java.util.concurrent.TimeUnit.MICROSECONDS;
import static java.util.concurrent.TimeUnit.NANOSECONDS;
import static org.apache.kafka.controller.QuorumController.ControllerOperationFlag.DOES_NOT_UPDATE_QUEUE_TIME;
import static org.apache.kafka.controller.QuorumController.ControllerOperationFlag.RUNS_IN_PREMIGRATION;
/**
@ -217,7 +213,6 @@ public final class QuorumController implements Controller {
private Map<String, Object> staticConfig = Collections.emptyMap();
private BootstrapMetadata bootstrapMetadata = null;
private int maxRecordsPerBatch = MAX_RECORDS_PER_BATCH;
private boolean zkMigrationEnabled = false;
private boolean eligibleLeaderReplicasEnabled = false;
private DelegationTokenCache tokenCache;
private String tokenSecretKeyString;
@ -341,11 +336,6 @@ public final class QuorumController implements Controller {
return this;
}
public Builder setZkMigrationEnabled(boolean zkMigrationEnabled) {
this.zkMigrationEnabled = zkMigrationEnabled;
return this;
}
public Builder setEligibleLeaderReplicasEnabled(boolean eligibleLeaderReplicasEnabled) {
this.eligibleLeaderReplicasEnabled = eligibleLeaderReplicasEnabled;
return this;
@ -406,7 +396,7 @@ public final class QuorumController implements Controller {
logContext = new LogContext(String.format("[QuorumController id=%d] ", nodeId));
}
if (controllerMetrics == null) {
controllerMetrics = new QuorumControllerMetrics(Optional.empty(), time, zkMigrationEnabled);
controllerMetrics = new QuorumControllerMetrics(Optional.empty(), time);
}
KafkaEventQueue queue = null;
@ -436,7 +426,6 @@ public final class QuorumController implements Controller {
staticConfig,
bootstrapMetadata,
maxRecordsPerBatch,
zkMigrationEnabled,
tokenCache,
tokenSecretKeyString,
delegationTokenMaxLifeMs,
@ -674,10 +663,6 @@ public final class QuorumController implements Controller {
return configurationControl;
}
public ZkRecordConsumer zkRecordConsumer() {
return zkRecordConsumer;
}
<T> CompletableFuture<T> appendReadEvent(
String name,
OptionalLong deadlineNs,
@ -697,23 +682,7 @@ public final class QuorumController implements Controller {
* A flag that signifies that this operation should not update the event queue time metric.
* We use this when the event was not appended to the queue.
*/
DOES_NOT_UPDATE_QUEUE_TIME,
/**
* A flag that signifies that this operation can be processed when in pre-migration mode.
* Operations without this flag will always return NOT_CONTROLLER when invoked in premigration
* mode.
* <p>
* In pre-migration mode, we are still waiting to load the metadata from Apache ZooKeeper into
* the metadata log. Therefore, the metadata log is mostly empty, even though the cluster really
* does have metadata
* <p>
* Events using this flag will be completed even if a transaction is ongoing. Pre-migration
* events will be completed using the unstable (committed) offset rather than the stable offset.
* <p>
* In practice, very few operations should use this flag.
*/
RUNS_IN_PREMIGRATION
DOES_NOT_UPDATE_QUEUE_TIME;
}
interface ControllerWriteOperation<T> {
@ -784,24 +753,13 @@ public final class QuorumController implements Controller {
if (!isActiveController(controllerEpoch)) {
throw ControllerExceptions.newWrongControllerException(latestController());
}
if (featureControl.inPreMigrationMode() && !flags.contains(RUNS_IN_PREMIGRATION)) {
log.info("Cannot run write operation {} in pre-migration mode. Returning NOT_CONTROLLER.", name);
throw ControllerExceptions.newPreMigrationException(latestController());
}
ControllerResult<T> result = op.generateRecordsAndResult();
if (result.records().isEmpty()) {
op.processBatchEndOffset(offsetControl.nextWriteOffset() - 1);
// 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.
// If the operation can complete within a transaction, let it use the
// unstable purgatory so that it can complete sooner.
OptionalLong maybeOffset;
if (featureControl.inPreMigrationMode() && flags.contains(RUNS_IN_PREMIGRATION)) {
maybeOffset = deferredUnstableEventQueue.highestPendingOffset();
} else {
maybeOffset = deferredEventQueue.highestPendingOffset();
}
OptionalLong maybeOffset = deferredEventQueue.highestPendingOffset();
if (!maybeOffset.isPresent()) {
// If the purgatory is empty, there are no pending operations and no
// uncommitted state. We can complete immediately.
@ -862,13 +820,9 @@ public final class QuorumController implements Controller {
// Remember the latest offset and future if it is not already completed
if (!future.isDone()) {
if (featureControl.inPreMigrationMode() && flags.contains(RUNS_IN_PREMIGRATION)) {
deferredUnstableEventQueue.add(resultAndOffset.offset(), this);
} else {
deferredEventQueue.add(resultAndOffset.offset(), this);
}
}
}
@Override
public void handleException(Throwable exception) {
@ -981,78 +935,6 @@ public final class QuorumController implements Controller {
return event.future();
}
class MigrationRecordConsumer implements ZkRecordConsumer {
private final EnumSet<ControllerOperationFlag> eventFlags = EnumSet.of(RUNS_IN_PREMIGRATION);
private volatile OffsetAndEpoch highestMigrationRecordOffset;
class MigrationWriteOperation implements ControllerWriteOperation<Void> {
private final List<ApiMessageAndVersion> batch;
MigrationWriteOperation(List<ApiMessageAndVersion> batch) {
this.batch = batch;
}
@Override
public ControllerResult<Void> generateRecordsAndResult() {
return ControllerResult.of(batch, null);
}
public void processBatchEndOffset(long offset) {
highestMigrationRecordOffset = new OffsetAndEpoch(offset, curClaimEpoch);
}
}
@Override
public CompletableFuture<?> beginMigration() {
if (featureControl.metadataVersion().isMetadataTransactionSupported()) {
log.info("Starting migration of ZooKeeper metadata to KRaft.");
ControllerWriteEvent<Void> batchEvent = new ControllerWriteEvent<>(
"Begin ZK Migration Transaction",
new MigrationWriteOperation(Collections.singletonList(
new ApiMessageAndVersion(
new BeginTransactionRecord().setName("ZK Migration"), (short) 0))
), eventFlags);
queue.append(batchEvent);
return batchEvent.future;
} else {
log.warn("Starting ZK Migration without metadata transactions enabled. This is not safe since " +
"a controller failover or processing error may lead to partially migrated metadata.");
return CompletableFuture.completedFuture(null);
}
}
@Override
public CompletableFuture<?> acceptBatch(List<ApiMessageAndVersion> recordBatch) {
ControllerWriteEvent<Void> batchEvent = new ControllerWriteEvent<>(
"ZK Migration Batch",
new MigrationWriteOperation(recordBatch), eventFlags);
queue.append(batchEvent);
return batchEvent.future;
}
@Override
public CompletableFuture<OffsetAndEpoch> completeMigration() {
log.info("Completing migration of ZooKeeper metadata to KRaft.");
List<ApiMessageAndVersion> records = new ArrayList<>(2);
records.add(ZkMigrationState.MIGRATION.toRecord());
if (featureControl.metadataVersion().isMetadataTransactionSupported()) {
records.add(new ApiMessageAndVersion(new EndTransactionRecord(), (short) 0));
}
ControllerWriteEvent<Void> event = new ControllerWriteEvent<>(
"Complete ZK Migration",
new MigrationWriteOperation(records),
eventFlags);
queue.append(event);
return event.future.thenApply(__ -> highestMigrationRecordOffset);
}
@Override
public void abortMigration() {
// If something goes wrong during the migration, cause the controller to crash and let the
// next controller abort the migration transaction (if in use).
fatalFaultHandler.handleFault("Aborting the ZK migration");
}
}
class QuorumMetaLogListener implements RaftClient.Listener<ApiMessageAndVersion> {
@Override
public void handleCommit(BatchReader<ApiMessageAndVersion> reader) {
@ -1077,7 +959,6 @@ public final class QuorumController implements Controller {
// items that were waiting for these offsets.
offsetControl.handleCommitBatch(batch);
deferredEventQueue.completeUpTo(offsetControl.lastStableOffset());
deferredUnstableEventQueue.completeUpTo(offsetControl.lastCommittedOffset());
} else {
// If the controller is a standby, replay the records that were
// created by the active controller.
@ -1228,7 +1109,7 @@ public final class QuorumController implements Controller {
ControllerWriteEvent<Void> activationEvent = new ControllerWriteEvent<>(
"completeActivation[" + epoch + "]",
new CompleteActivationEvent(),
EnumSet.of(DOES_NOT_UPDATE_QUEUE_TIME, RUNS_IN_PREMIGRATION)
EnumSet.of(DOES_NOT_UPDATE_QUEUE_TIME)
);
queue.prepend(activationEvent);
} catch (Throwable e) {
@ -1244,9 +1125,9 @@ public final class QuorumController implements Controller {
log::warn,
logReplayTracker.empty(),
offsetControl.transactionStartOffset(),
zkMigrationEnabled,
bootstrapMetadata,
featureControl);
featureControl.zkMigrationState(),
featureControl.metadataVersion());
} catch (Throwable t) {
throw fatalFaultHandler.handleFault("exception while completing controller " +
"activation", t);
@ -1276,8 +1157,6 @@ public final class QuorumController implements Controller {
curClaimEpoch = -1;
deferredEventQueue.failAll(ControllerExceptions.
newWrongControllerException(OptionalInt.empty()));
deferredUnstableEventQueue.failAll(ControllerExceptions.
newWrongControllerException(OptionalInt.empty()));
offsetControl.deactivate();
clusterControl.deactivate();
cancelMaybeFenceReplicas();
@ -1474,7 +1353,7 @@ public final class QuorumController implements Controller {
null
);
},
EnumSet.of(DOES_NOT_UPDATE_QUEUE_TIME, RUNS_IN_PREMIGRATION)
EnumSet.of(DOES_NOT_UPDATE_QUEUE_TIME)
);
long delayNs = time.nanoseconds() + maxIdleIntervalNs.getAsLong();
@ -1683,13 +1562,6 @@ public final class QuorumController implements Controller {
*/
private final DeferredEventQueue deferredEventQueue;
/**
* The deferred event queue which holds deferred operations which are waiting for the metadata
* log's committed offset to advance. This must be accessed only by the event queue thread and
* can contain records which are part of an incomplete transaction.
*/
private final DeferredEventQueue deferredUnstableEventQueue;
/**
* Manages read and write offsets, and in-memory snapshots.
*/
@ -1823,10 +1695,6 @@ public final class QuorumController implements Controller {
*/
private final BootstrapMetadata bootstrapMetadata;
private final ZkRecordConsumer zkRecordConsumer;
private final boolean zkMigrationEnabled;
private final boolean eligibleLeaderReplicasEnabled;
/**
@ -1868,7 +1736,6 @@ public final class QuorumController implements Controller {
Map<String, Object> staticConfig,
BootstrapMetadata bootstrapMetadata,
int maxRecordsPerBatch,
boolean zkMigrationEnabled,
DelegationTokenCache tokenCache,
String tokenSecretKeyString,
long delegationTokenMaxLifeMs,
@ -1888,7 +1755,6 @@ public final class QuorumController implements Controller {
this.controllerMetrics = controllerMetrics;
this.snapshotRegistry = new SnapshotRegistry(logContext);
this.deferredEventQueue = new DeferredEventQueue(logContext);
this.deferredUnstableEventQueue = new DeferredEventQueue(logContext);
this.offsetControl = new OffsetControlManager.Builder().
setLogContext(logContext).
setSnapshotRegistry(snapshotRegistry).
@ -1931,7 +1797,6 @@ public final class QuorumController implements Controller {
setSessionTimeoutNs(sessionTimeoutNs).
setReplicaPlacer(replicaPlacer).
setFeatureControlManager(featureControl).
setZkMigrationEnabled(zkMigrationEnabled).
setBrokerUncleanShutdownHandler(this::handleUncleanBrokerShutdown).
setInterBrokerListenerName(interBrokerListenerName).
build();
@ -1978,15 +1843,13 @@ public final class QuorumController implements Controller {
this.maxRecordsPerBatch = maxRecordsPerBatch;
this.metaLogListener = new QuorumMetaLogListener();
this.curClaimEpoch = -1;
this.zkRecordConsumer = new MigrationRecordConsumer();
this.zkMigrationEnabled = zkMigrationEnabled;
this.recordRedactor = new RecordRedactor(configSchema);
this.eligibleLeaderReplicasEnabled = eligibleLeaderReplicasEnabled;
this.uncleanLeaderElectionCheckIntervalNs =
TimeUnit.MILLISECONDS.toNanos(uncleanLeaderElectionCheckIntervalMs);
log.info("Creating new QuorumController with clusterId {}.{}{}",
clusterId, zkMigrationEnabled ? " ZK migration mode is enabled." : "",
log.info("Creating new QuorumController with clusterId {}.{}",
clusterId,
eligibleLeaderReplicasEnabled ? " Eligible leader replicas enabled." : "");
this.raftClient.register(metaLogListener);
@ -2061,7 +1924,8 @@ public final class QuorumController implements Controller {
int brokerId
) {
return appendWriteEvent("unregisterBroker", context.deadlineNs(),
() -> replicationControl.unregisterBroker(brokerId), EnumSet.of(RUNS_IN_PREMIGRATION));
() -> replicationControl.unregisterBroker(brokerId),
EnumSet.noneOf(ControllerOperationFlag.class));
}
@Override
@ -2238,7 +2102,7 @@ public final class QuorumController implements Controller {
}
}
},
EnumSet.of(RUNS_IN_PREMIGRATION)).whenComplete((__, t) -> {
EnumSet.noneOf(ControllerOperationFlag.class)).whenComplete((__, t) -> {
if (ControllerExceptions.isTimeoutException(t)) {
replicationControl.processExpiredBrokerHeartbeat(request);
controllerMetrics.incrementTimedOutHeartbeats();
@ -2262,7 +2126,7 @@ public final class QuorumController implements Controller {
rescheduleMaybeFenceStaleBrokers();
return result;
},
EnumSet.of(RUNS_IN_PREMIGRATION));
EnumSet.noneOf(ControllerOperationFlag.class));
}
@Override
@ -2366,7 +2230,7 @@ public final class QuorumController implements Controller {
) {
return appendWriteEvent("registerController", context.deadlineNs(),
() -> clusterControl.registerController(request),
EnumSet.of(RUNS_IN_PREMIGRATION));
EnumSet.noneOf(ControllerOperationFlag.class));
}
@Override

View File

@ -1070,7 +1070,6 @@ public class ReplicationControlManager {
featureControl.metadataVersion(),
getTopicEffectiveMinIsr(topic.name)
)
.setZkMigrationEnabled(clusterControl.zkRegistrationAllowed())
.setEligibleLeaderReplicasEnabled(isElrEnabled());
if (configurationControl.uncleanLeaderElectionEnabledForTopic(topic.name())) {
builder.setElection(PartitionChangeBuilder.Election.UNCLEAN);
@ -1517,7 +1516,6 @@ public class ReplicationControlManager {
getTopicEffectiveMinIsr(topic)
)
.setElection(election)
.setZkMigrationEnabled(clusterControl.zkRegistrationAllowed())
.setEligibleLeaderReplicasEnabled(isElrEnabled())
.setDefaultDirProvider(clusterDescriber)
.build();
@ -1666,7 +1664,6 @@ public class ReplicationControlManager {
getTopicEffectiveMinIsr(topic.name)
)
.setElection(PartitionChangeBuilder.Election.PREFERRED)
.setZkMigrationEnabled(clusterControl.zkRegistrationAllowed())
.setEligibleLeaderReplicasEnabled(isElrEnabled())
.setDefaultDirProvider(clusterDescriber)
.build().ifPresent(records::add);
@ -1936,7 +1933,6 @@ public class ReplicationControlManager {
featureControl.metadataVersion(),
getTopicEffectiveMinIsr(topic.name)
);
builder.setZkMigrationEnabled(clusterControl.zkRegistrationAllowed());
builder.setEligibleLeaderReplicasEnabled(isElrEnabled());
if (configurationControl.uncleanLeaderElectionEnabledForTopic(topic.name)) {
builder.setElection(PartitionChangeBuilder.Election.UNCLEAN);
@ -2055,7 +2051,6 @@ public class ReplicationControlManager {
featureControl.metadataVersion(),
getTopicEffectiveMinIsr(topicName)
);
builder.setZkMigrationEnabled(clusterControl.zkRegistrationAllowed());
builder.setEligibleLeaderReplicasEnabled(isElrEnabled());
if (configurationControl.uncleanLeaderElectionEnabledForTopic(topicName)) {
builder.setElection(PartitionChangeBuilder.Election.UNCLEAN);
@ -2117,7 +2112,6 @@ public class ReplicationControlManager {
featureControl.metadataVersion(),
getTopicEffectiveMinIsr(topics.get(tp.topicId()).name)
);
builder.setZkMigrationEnabled(clusterControl.zkRegistrationAllowed());
builder.setEligibleLeaderReplicasEnabled(isElrEnabled());
if (!reassignment.replicas().equals(currentReplicas)) {
builder.setTargetReplicas(reassignment.replicas());

View File

@ -45,12 +45,6 @@ public class QuorumControllerMetrics implements AutoCloseable {
"ControllerEventManager", "EventQueueTimeMs");
private static final MetricName EVENT_QUEUE_PROCESSING_TIME_MS = getMetricName(
"ControllerEventManager", "EventQueueProcessingTimeMs");
private static final MetricName ZK_WRITE_BEHIND_LAG = getMetricName(
"KafkaController", "ZkWriteBehindLag");
private static final MetricName ZK_WRITE_SNAPSHOT_TIME_MS = getMetricName(
"KafkaController", "ZkWriteSnapshotTimeMs");
private static final MetricName ZK_WRITE_DELTA_TIME_MS = getMetricName(
"KafkaController", "ZkWriteDeltaTimeMs");
private static final MetricName LAST_APPLIED_RECORD_OFFSET = getMetricName(
"KafkaController", "LastAppliedRecordOffset");
private static final MetricName LAST_COMMITTED_RECORD_OFFSET = getMetricName(
@ -73,11 +67,8 @@ public class QuorumControllerMetrics implements AutoCloseable {
private final AtomicLong lastAppliedRecordOffset = new AtomicLong(0);
private final AtomicLong lastCommittedRecordOffset = new AtomicLong(0);
private final AtomicLong lastAppliedRecordTimestamp = new AtomicLong(0);
private final AtomicLong dualWriteOffset = new AtomicLong(0);
private final Consumer<Long> eventQueueTimeUpdater;
private final Consumer<Long> eventQueueProcessingTimeUpdater;
private final Consumer<Long> zkWriteSnapshotTimeHandler;
private final Consumer<Long> zkWriteDeltaTimeHandler;
private final AtomicLong timedOutHeartbeats = new AtomicLong(0);
private final AtomicLong operationsStarted = new AtomicLong(0);
@ -95,8 +86,7 @@ public class QuorumControllerMetrics implements AutoCloseable {
public QuorumControllerMetrics(
Optional<MetricsRegistry> registry,
Time time,
boolean zkMigrationEnabled
Time time
) {
this.registry = registry;
this.active = false;
@ -156,23 +146,6 @@ public class QuorumControllerMetrics implements AutoCloseable {
return newActiveControllers();
}
}));
if (zkMigrationEnabled) {
registry.ifPresent(r -> r.newGauge(ZK_WRITE_BEHIND_LAG, new Gauge<Long>() {
@Override
public Long value() {
// not in dual-write mode or not an active controller: set metric value to 0
if (dualWriteOffset() == 0 || !active()) return 0L;
// in dual write mode
else return lastCommittedRecordOffset() - dualWriteOffset();
}
}));
this.zkWriteSnapshotTimeHandler = newHistogram(ZK_WRITE_SNAPSHOT_TIME_MS, true);
this.zkWriteDeltaTimeHandler = newHistogram(ZK_WRITE_DELTA_TIME_MS, true);
} else {
this.zkWriteSnapshotTimeHandler = __ -> { };
this.zkWriteDeltaTimeHandler = __ -> { };
}
}
public void setActive(boolean active) {
@ -191,14 +164,6 @@ public class QuorumControllerMetrics implements AutoCloseable {
eventQueueProcessingTimeUpdater.accept(durationMs);
}
public void updateZkWriteSnapshotTimeMs(long durationMs) {
zkWriteSnapshotTimeHandler.accept(durationMs);
}
public void updateZkWriteDeltaTimeMs(long durationMs) {
zkWriteDeltaTimeHandler.accept(durationMs);
}
public void setLastAppliedRecordOffset(long offset) {
lastAppliedRecordOffset.set(offset);
}
@ -223,14 +188,6 @@ public class QuorumControllerMetrics implements AutoCloseable {
return lastAppliedRecordTimestamp.get();
}
public void updateDualWriteOffset(long offset) {
dualWriteOffset.set(offset);
}
public long dualWriteOffset() {
return dualWriteOffset.get();
}
public void incrementTimedOutHeartbeats() {
timedOutHeartbeats.incrementAndGet();
}
@ -276,10 +233,7 @@ public class QuorumControllerMetrics implements AutoCloseable {
TIMED_OUT_BROKER_HEARTBEAT_COUNT,
EVENT_QUEUE_OPERATIONS_STARTED_COUNT,
EVENT_QUEUE_OPERATIONS_TIMED_OUT_COUNT,
NEW_ACTIVE_CONTROLLERS_COUNT,
ZK_WRITE_BEHIND_LAG,
ZK_WRITE_SNAPSHOT_TIME_MS,
ZK_WRITE_DELTA_TIME_MS
NEW_ACTIVE_CONTROLLERS_COUNT
).forEach(r::removeMetric));
}

View File

@ -1,40 +0,0 @@
/*
* 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.acl.AccessControlEntry;
import org.apache.kafka.common.resource.ResourcePattern;
import java.util.Collection;
import java.util.Set;
import java.util.function.BiConsumer;
public interface AclMigrationClient {
ZkMigrationLeadershipState deleteResource(
ResourcePattern resourcePattern,
ZkMigrationLeadershipState state
);
ZkMigrationLeadershipState writeResourceAcls(
ResourcePattern resourcePattern,
Collection<AccessControlEntry> aclsToWrite,
ZkMigrationLeadershipState state
);
void iterateAcls(BiConsumer<ResourcePattern, Set<AccessControlEntry>> aclConsumer);
}

View File

@ -1,58 +0,0 @@
/*
* 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 java.util.ArrayList;
import java.util.List;
import java.util.function.Consumer;
/**
* A record batch consumer that merges incoming batches into batches of a minimum a given size. It does so
* by buffering the records into an array that is later flushed to a downstream consumer. Batches consumed
* by this class will not be broken apart, only combined with other batches to reach the minimum batch size.
* </p>
* Note that {@link #flush()} must be called after the last batch has been accepted in order to flush any
* buffered records.
*/
public class BufferingBatchConsumer<T> implements Consumer<List<T>> {
private final Consumer<List<T>> delegateConsumer;
private final int minBatchSize;
private List<T> bufferedBatch;
BufferingBatchConsumer(Consumer<List<T>> delegateConsumer, int minBatchSize) {
this.delegateConsumer = delegateConsumer;
this.minBatchSize = minBatchSize;
this.bufferedBatch = new ArrayList<>(minBatchSize);
}
@Override
public void accept(List<T> batch) {
bufferedBatch.addAll(batch);
if (bufferedBatch.size() >= minBatchSize) {
flush();
}
}
public void flush() {
if (!bufferedBatch.isEmpty()) {
delegateConsumer.accept(bufferedBatch);
bufferedBatch = new ArrayList<>(minBatchSize);
}
}
}

View File

@ -1,63 +0,0 @@
/*
* 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.clients.admin.ScramMechanism;
import org.apache.kafka.common.config.ConfigResource;
import org.apache.kafka.common.metadata.ClientQuotaRecord;
import org.apache.kafka.common.security.scram.ScramCredential;
import java.util.List;
import java.util.Map;
import java.util.function.BiConsumer;
import java.util.function.Consumer;
public interface ConfigMigrationClient {
interface ClientQuotaVisitor {
void visitClientQuota(List<ClientQuotaRecord.EntityData> entityDataList, Map<String, Double> quotas);
void visitScramCredential(String userName, ScramMechanism scramMechanism, ScramCredential scramCredential);
}
void iterateClientQuotas(ClientQuotaVisitor visitor);
void iterateBrokerConfigs(BiConsumer<String, Map<String, String>> configConsumer);
void iterateTopicConfigs(BiConsumer<String, Map<String, String>> configConsumer);
void readTopicConfigs(String topicName, Consumer<Map<String, String>> configConsumer);
ZkMigrationLeadershipState writeConfigs(
ConfigResource configResource,
Map<String, String> configMap,
ZkMigrationLeadershipState state
);
ZkMigrationLeadershipState writeClientQuotas(
Map<String, String> clientQuotaEntity,
Map<String, Double> quotas,
Map<String, String> scram,
ZkMigrationLeadershipState state
);
ZkMigrationLeadershipState deleteConfigs(
ConfigResource configResource,
ZkMigrationLeadershipState state
);
}

View File

@ -1,38 +0,0 @@
/*
* 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.security.token.delegation.TokenInformation;
import java.util.List;
public interface DelegationTokenMigrationClient {
List<String> getDelegationTokens();
ZkMigrationLeadershipState writeDelegationToken(
String tokenId,
TokenInformation tokenInformation,
ZkMigrationLeadershipState state
);
ZkMigrationLeadershipState deleteDelegationToken(
String tokenId,
ZkMigrationLeadershipState state
);
}

View File

@ -1,23 +0,0 @@
/*
* 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;
@FunctionalInterface
public interface KRaftMigrationOperation {
ZkMigrationLeadershipState apply(ZkMigrationLeadershipState migrationState);
}

View File

@ -1,23 +0,0 @@
/*
* 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;
@FunctionalInterface
public interface KRaftMigrationOperationConsumer {
void accept(String opType, String logMsg, KRaftMigrationOperation operation);
}

View File

@ -1,684 +0,0 @@
/*
* 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.clients.admin.ScramMechanism;
import org.apache.kafka.common.TopicIdPartition;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.acl.AccessControlEntry;
import org.apache.kafka.common.config.ConfigResource;
import org.apache.kafka.common.metadata.ClientQuotaRecord;
import org.apache.kafka.common.quota.ClientQuotaEntity;
import org.apache.kafka.common.resource.ResourcePattern;
import org.apache.kafka.common.security.scram.ScramCredential;
import org.apache.kafka.common.security.scram.internals.ScramCredentialUtils;
import org.apache.kafka.image.AclsDelta;
import org.apache.kafka.image.AclsImage;
import org.apache.kafka.image.ClientQuotaImage;
import org.apache.kafka.image.ClientQuotasImage;
import org.apache.kafka.image.ConfigurationsDelta;
import org.apache.kafka.image.ConfigurationsImage;
import org.apache.kafka.image.DelegationTokenDelta;
import org.apache.kafka.image.DelegationTokenImage;
import org.apache.kafka.image.MetadataDelta;
import org.apache.kafka.image.MetadataImage;
import org.apache.kafka.image.ProducerIdsDelta;
import org.apache.kafka.image.ProducerIdsImage;
import org.apache.kafka.image.ScramImage;
import org.apache.kafka.image.TopicImage;
import org.apache.kafka.image.TopicsDelta;
import org.apache.kafka.image.TopicsImage;
import org.apache.kafka.metadata.DelegationTokenData;
import org.apache.kafka.metadata.PartitionRegistration;
import org.apache.kafka.metadata.ScramCredentialData;
import org.apache.kafka.metadata.authorizer.StandardAcl;
import org.apache.kafka.server.common.ProducerIdsBlock;
import java.util.ArrayList;
import java.util.Collections;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Optional;
import java.util.Set;
import java.util.function.BiConsumer;
import java.util.function.Consumer;
import java.util.function.Function;
public class KRaftMigrationZkWriter {
private static final String UPDATE_PRODUCER_ID = "UpdateProducerId";
private static final String CREATE_TOPIC = "CreateTopic";
private static final String UPDATE_TOPIC = "UpdateTopic";
private static final String DELETE_TOPIC = "DeleteTopic";
private static final String DELETE_PENDING_TOPIC_DELETION = "DeletePendingTopicDeletion";
private static final String UPDATE_PARTITION = "UpdatePartition";
private static final String DELETE_PARTITION = "DeletePartition";
private static final String UPDATE_BROKER_CONFIG = "UpdateBrokerConfig";
private static final String DELETE_BROKER_CONFIG = "DeleteBrokerConfig";
private static final String UPDATE_TOPIC_CONFIG = "UpdateTopicConfig";
private static final String DELETE_TOPIC_CONFIG = "DeleteTopicConfig";
private static final String UPDATE_CLIENT_QUOTA = "UpdateClientQuota";
private static final String UPDATE_ACL = "UpdateAcl";
private static final String DELETE_ACL = "DeleteAcl";
private final MigrationClient migrationClient;
private final Consumer<String> errorLogger;
public KRaftMigrationZkWriter(
MigrationClient migrationClient,
Consumer<String> errorLogger
) {
this.migrationClient = migrationClient;
this.errorLogger = errorLogger;
}
public void handleSnapshot(MetadataImage image, KRaftMigrationOperationConsumer operationConsumer) {
handleTopicsSnapshot(image.topics(), operationConsumer);
handleConfigsSnapshot(image.configs(), operationConsumer);
handleClientQuotasSnapshot(image.clientQuotas(), image.scram(), operationConsumer);
handleProducerIdSnapshot(image.producerIds(), operationConsumer);
handleAclsSnapshot(image.acls(), operationConsumer);
handleDelegationTokenSnapshot(image.delegationTokens(), operationConsumer);
}
public boolean handleDelta(
MetadataImage previousImage,
MetadataImage image,
MetadataDelta delta,
KRaftMigrationOperationConsumer operationConsumer
) {
boolean updated = false;
if (delta.topicsDelta() != null) {
handleTopicsDelta(previousImage.topics().topicIdToNameView()::get, image.topics(), delta.topicsDelta(), operationConsumer);
updated = true;
}
if (delta.configsDelta() != null) {
handleConfigsDelta(image.configs(), delta.configsDelta(), operationConsumer);
updated = true;
}
if ((delta.clientQuotasDelta() != null) || (delta.scramDelta() != null)) {
handleClientQuotasDelta(image, delta, operationConsumer);
updated = true;
}
if (delta.producerIdsDelta() != null) {
handleProducerIdDelta(delta.producerIdsDelta(), operationConsumer);
updated = true;
}
if (delta.aclsDelta() != null) {
handleAclsDelta(previousImage.acls(), image.acls(), delta.aclsDelta(), operationConsumer);
updated = true;
}
if (delta.delegationTokenDelta() != null) {
handleDelegationTokenDelta(image.delegationTokens(), delta.delegationTokenDelta(), operationConsumer);
updated = true;
}
return updated;
}
/**
* Handle a snapshot of the topic metadata. This requires scanning through all the topics and partitions
* in ZooKeeper to determine what has changed. Topic configs are not handled here since they exist in the
* ConfigurationsImage.
*/
void handleTopicsSnapshot(TopicsImage topicsImage, KRaftMigrationOperationConsumer operationConsumer) {
Map<Uuid, String> deletedTopics = new HashMap<>();
Set<Uuid> topicsInZk = new HashSet<>();
Set<Uuid> newTopics = new HashSet<>(topicsImage.topicsById().keySet());
Set<Uuid> changedTopics = new HashSet<>();
Map<Uuid, Set<Integer>> partitionsInZk = new HashMap<>();
Map<String, Set<Integer>> extraneousPartitionsInZk = new HashMap<>();
Map<Uuid, Map<Integer, PartitionRegistration>> changedPartitions = new HashMap<>();
Map<Uuid, Map<Integer, PartitionRegistration>> newPartitions = new HashMap<>();
Set<String> pendingTopicDeletions = migrationClient.topicClient().readPendingTopicDeletions();
if (!pendingTopicDeletions.isEmpty()) {
operationConsumer.accept(
DELETE_PENDING_TOPIC_DELETION,
"Deleted pending topic deletions",
migrationState -> migrationClient.topicClient().clearPendingTopicDeletions(pendingTopicDeletions, migrationState)
);
}
migrationClient.topicClient().iterateTopics(
EnumSet.of(
TopicMigrationClient.TopicVisitorInterest.TOPICS,
TopicMigrationClient.TopicVisitorInterest.PARTITIONS),
new TopicMigrationClient.TopicVisitor() {
@Override
public void visitTopic(String topicName, Uuid topicId, Map<Integer, List<Integer>> assignments) {
TopicImage topic = topicsImage.getTopic(topicId);
if (topic == null) {
// If KRaft does not have this topic, it was deleted
deletedTopics.put(topicId, topicName);
} else {
if (!newTopics.remove(topicId)) return;
topicsInZk.add(topicId);
}
}
@Override
public void visitPartition(TopicIdPartition topicIdPartition, PartitionRegistration partitionRegistration) {
TopicImage topic = topicsImage.getTopic(topicIdPartition.topicId());
if (topic == null) {
return; // The topic was deleted in KRaft. Handled by deletedTopics
}
// If there is failure in previous Zk writes, We could end up with Zookeeper
// containing with partial or without any partitions for existing topics. So
// accumulate the partition ids to check for any missing partitions in Zk.
partitionsInZk
.computeIfAbsent(topic.id(), __ -> new HashSet<>())
.add(topicIdPartition.partition());
// Check if the KRaft partition state changed
PartitionRegistration kraftPartition = topic.partitions().get(topicIdPartition.partition());
if (kraftPartition != null) {
if (!kraftPartition.equals(partitionRegistration)) {
changedPartitions.computeIfAbsent(topicIdPartition.topicId(), __ -> new HashMap<>())
.put(topicIdPartition.partition(), kraftPartition);
}
// Check if partition assignment has changed. This will need topic update.
if (!kraftPartition.hasSameAssignment(partitionRegistration)) {
changedTopics.add(topic.id());
}
}
}
});
// Check for any partition changes in existing topics.
topicsInZk.forEach(topicId -> {
TopicImage topic = topicsImage.getTopic(topicId);
Set<Integer> topicPartitionsInZk = partitionsInZk.computeIfAbsent(topicId, __ -> new HashSet<>());
if (!topicPartitionsInZk.equals(topic.partitions().keySet())) {
Map<Integer, PartitionRegistration> newTopicPartitions = new HashMap<>(topic.partitions());
// Compute KRaft partitions that are not in ZK
topicPartitionsInZk.forEach(newTopicPartitions::remove);
newPartitions.put(topicId, newTopicPartitions);
// Compute ZK partitions that are not in KRaft
topicPartitionsInZk.removeAll(topic.partitions().keySet());
if (!topicPartitionsInZk.isEmpty()) {
extraneousPartitionsInZk.put(topic.name(), topicPartitionsInZk);
}
changedTopics.add(topicId);
}
});
newTopics.forEach(topicId -> {
TopicImage topic = topicsImage.getTopic(topicId);
operationConsumer.accept(
CREATE_TOPIC,
"Created Topic " + topic.name() + ", ID " + topicId,
migrationState -> migrationClient.topicClient().createTopic(topic.name(), topicId, topic.partitions(), migrationState)
);
});
changedTopics.forEach(topicId -> {
TopicImage topic = topicsImage.getTopic(topicId);
operationConsumer.accept(
UPDATE_TOPIC,
"Changed Topic " + topic.name() + ", ID " + topicId,
migrationState -> migrationClient.topicClient().updateTopic(topic.name(), topicId, topic.partitions(), migrationState)
);
});
deletedTopics.forEach((topicId, topicName) ->
operationConsumer.accept(
DELETE_TOPIC,
"Deleted Topic " + topicName + ", ID " + topicId,
migrationState -> migrationClient.topicClient().deleteTopic(topicName, migrationState)
)
);
newPartitions.forEach((topicId, partitionMap) -> {
TopicImage topic = topicsImage.getTopic(topicId);
operationConsumer.accept(
UPDATE_PARTITION,
"Created additional partitions for Topic " + topic.name() + ", ID " + topicId,
migrationState -> migrationClient.topicClient().updateTopicPartitions(
Collections.singletonMap(topic.name(), partitionMap),
migrationState));
});
changedPartitions.forEach((topicId, partitionMap) -> {
TopicImage topic = topicsImage.getTopic(topicId);
operationConsumer.accept(
UPDATE_PARTITION,
"Updated Partitions for Topic " + topic.name() + ", ID " + topicId,
migrationState -> migrationClient.topicClient().updateTopicPartitions(
Collections.singletonMap(topic.name(), partitionMap),
migrationState));
});
extraneousPartitionsInZk.forEach((topicName, partitions) ->
operationConsumer.accept(
DELETE_PARTITION,
"Deleted extraneous Partitions " + partitions + " for Topic " + topicName,
migrationState -> migrationClient.topicClient().deleteTopicPartitions(
Collections.singletonMap(topicName, partitions),
migrationState))
);
}
void handleTopicsDelta(
Function<Uuid, String> deletedTopicNameResolver,
TopicsImage topicsImage,
TopicsDelta topicsDelta,
KRaftMigrationOperationConsumer operationConsumer
) {
topicsDelta.deletedTopicIds().forEach(topicId -> {
String name = deletedTopicNameResolver.apply(topicId);
operationConsumer.accept(DELETE_TOPIC, "Deleted topic " + name + ", ID " + topicId,
migrationState -> migrationClient.topicClient().deleteTopic(name, migrationState));
});
topicsDelta.changedTopics().forEach((topicId, topicDelta) -> {
if (topicsDelta.createdTopicIds().contains(topicId)) {
operationConsumer.accept(
CREATE_TOPIC,
"Created Topic " + topicDelta.name() + ", ID " + topicId,
migrationState -> migrationClient.topicClient().createTopic(
topicDelta.name(),
topicId,
topicDelta.partitionChanges(),
migrationState));
} else {
if (topicDelta.hasPartitionsWithAssignmentChanges())
operationConsumer.accept(
UPDATE_TOPIC,
"Updated Topic " + topicDelta.name() + ", ID " + topicId,
migrationState -> migrationClient.topicClient().updateTopic(
topicDelta.name(),
topicId,
topicsImage.getTopic(topicId).partitions(),
migrationState));
Map<Integer, PartitionRegistration> newPartitions = new HashMap<>(topicDelta.newPartitions());
Map<Integer, PartitionRegistration> changedPartitions = new HashMap<>(topicDelta.partitionChanges());
if (!newPartitions.isEmpty()) {
operationConsumer.accept(
UPDATE_PARTITION,
"Created new partitions for Topic " + topicDelta.name() + ", ID " + topicId,
migrationState -> migrationClient.topicClient().createTopicPartitions(
Collections.singletonMap(topicDelta.name(), newPartitions),
migrationState));
newPartitions.keySet().forEach(changedPartitions::remove);
}
if (!changedPartitions.isEmpty()) {
// Need a final for the lambda
final Map<Integer, PartitionRegistration> finalChangedPartitions = changedPartitions;
operationConsumer.accept(
UPDATE_PARTITION,
"Updated Partitions for Topic " + topicDelta.name() + ", ID " + topicId,
migrationState -> migrationClient.topicClient().updateTopicPartitions(
Collections.singletonMap(topicDelta.name(), finalChangedPartitions),
migrationState));
}
}
});
}
private String brokerOrTopicOpType(ConfigResource resource, String brokerOp, String topicOp) {
if (resource.type().equals(ConfigResource.Type.BROKER)) {
return brokerOp;
} else {
return topicOp;
}
}
void handleConfigsSnapshot(ConfigurationsImage configsImage, KRaftMigrationOperationConsumer operationConsumer) {
Set<ConfigResource> newResources = new HashSet<>();
configsImage.resourceData().keySet().forEach(resource -> {
if (EnumSet.of(ConfigResource.Type.BROKER, ConfigResource.Type.TOPIC).contains(resource.type())) {
newResources.add(resource);
} else {
throw new RuntimeException("Unknown config resource type " + resource.type());
}
});
Set<ConfigResource> resourcesToUpdate = new HashSet<>();
BiConsumer<ConfigResource, Map<String, String>> processConfigsForResource = (ConfigResource resource, Map<String, String> configs) -> {
newResources.remove(resource);
Map<String, String> kraftProps = configsImage.configMapForResource(resource);
if (!kraftProps.equals(configs)) {
resourcesToUpdate.add(resource);
}
};
migrationClient.configClient().iterateBrokerConfigs((broker, configs) -> {
ConfigResource brokerResource = new ConfigResource(ConfigResource.Type.BROKER, broker);
processConfigsForResource.accept(brokerResource, configs);
});
migrationClient.configClient().iterateTopicConfigs((topic, configs) -> {
ConfigResource topicResource = new ConfigResource(ConfigResource.Type.TOPIC, topic);
processConfigsForResource.accept(topicResource, configs);
});
newResources.forEach(resource -> {
Map<String, String> props = configsImage.configMapForResource(resource);
if (!props.isEmpty()) {
String opType = brokerOrTopicOpType(resource, UPDATE_BROKER_CONFIG, UPDATE_TOPIC_CONFIG);
operationConsumer.accept(opType, "Created configs for " + resource.type().name() + " " + resource.name(),
migrationState -> migrationClient.configClient().writeConfigs(resource, props, migrationState));
}
});
resourcesToUpdate.forEach(resource -> {
Map<String, String> props = configsImage.configMapForResource(resource);
if (props.isEmpty()) {
String opType = brokerOrTopicOpType(resource, DELETE_BROKER_CONFIG, DELETE_TOPIC_CONFIG);
operationConsumer.accept(opType, "Deleted configs for " + resource.type().name() + " " + resource.name(),
migrationState -> migrationClient.configClient().deleteConfigs(resource, migrationState));
} else {
String opType = brokerOrTopicOpType(resource, UPDATE_BROKER_CONFIG, UPDATE_TOPIC_CONFIG);
operationConsumer.accept(opType, "Updated configs for " + resource.type().name() + " " + resource.name(),
migrationState -> migrationClient.configClient().writeConfigs(resource, props, migrationState));
}
});
}
private Map<String, String> getScramCredentialStringsForUser(ScramImage image, String userName) {
Map<String, String> userScramCredentialStrings = new HashMap<>();
if (image != null) {
image.mechanisms().forEach((scramMechanism, scramMechanismMap) -> {
ScramCredentialData scramCredentialData = scramMechanismMap.get(userName);
if (scramCredentialData != null) {
userScramCredentialStrings.put(scramMechanism.mechanismName(),
ScramCredentialUtils.credentialToString(scramCredentialData.toCredential(scramMechanism)));
}
});
}
return userScramCredentialStrings;
}
void handleClientQuotasSnapshot(ClientQuotasImage clientQuotasImage, ScramImage scramImage, KRaftMigrationOperationConsumer opConsumer) {
Set<ClientQuotaEntity> changedNonUserEntities = new HashSet<>();
Set<String> changedUsers = new HashSet<>();
if (clientQuotasImage != null) {
for (Entry<ClientQuotaEntity, ClientQuotaImage> entry : clientQuotasImage.entities().entrySet()) {
ClientQuotaEntity entity = entry.getKey();
if (entity.entries().containsKey(ClientQuotaEntity.USER) &&
!entity.entries().containsKey(ClientQuotaEntity.CLIENT_ID)) {
// Track regular user entities separately
// There should only be 1 entry in the list of type ClientQuotaEntity.USER
changedUsers.add(entity.entries().get(ClientQuotaEntity.USER));
} else {
changedNonUserEntities.add(entity);
}
}
}
if (scramImage != null) {
for (Entry<ScramMechanism, Map<String, ScramCredentialData>> mechanismEntry : scramImage.mechanisms().entrySet()) {
for (Entry<String, ScramCredentialData> userEntry : mechanismEntry.getValue().entrySet()) {
changedUsers.add(userEntry.getKey());
}
}
}
migrationClient.configClient().iterateClientQuotas(new ConfigMigrationClient.ClientQuotaVisitor() {
@Override
public void visitClientQuota(List<ClientQuotaRecord.EntityData> entityDataList, Map<String, Double> quotas) {
Map<String, String> entityMap = new HashMap<>(2);
entityDataList.forEach(entityData -> entityMap.put(entityData.entityType(), entityData.entityName()));
ClientQuotaEntity entity = new ClientQuotaEntity(entityMap);
if (!clientQuotasImage.entities().getOrDefault(entity, ClientQuotaImage.EMPTY).quotaMap().equals(quotas)) {
if (entity.entries().containsKey(ClientQuotaEntity.USER) &&
!entity.entries().containsKey(ClientQuotaEntity.CLIENT_ID)) {
// Track regular user entities separately
changedUsers.add(entityMap.get(ClientQuotaEntity.USER));
} else {
changedNonUserEntities.add(entity);
}
}
}
@Override
public void visitScramCredential(String userName, ScramMechanism scramMechanism, ScramCredential scramCredential) {
// For each ZK entity, see if it exists in the image and if it's equal
ScramCredentialData data = scramImage.mechanisms().getOrDefault(scramMechanism, Collections.emptyMap()).get(userName);
if (data == null || !data.toCredential(scramMechanism).equals(scramCredential)) {
changedUsers.add(userName);
}
}
});
changedNonUserEntities.forEach(entity -> {
Map<String, Double> quotaMap = clientQuotasImage.entities().getOrDefault(entity, ClientQuotaImage.EMPTY).quotaMap();
opConsumer.accept(UPDATE_CLIENT_QUOTA, "Updated client quotas for " + entity, migrationState ->
migrationClient.configClient().writeClientQuotas(entity.entries(), quotaMap, Collections.emptyMap(), migrationState));
});
changedUsers.forEach(userName -> {
ClientQuotaEntity entity = new ClientQuotaEntity(Collections.singletonMap(ClientQuotaEntity.USER, userName));
Map<String, Double> quotaMap = clientQuotasImage.entities().
getOrDefault(entity, ClientQuotaImage.EMPTY).quotaMap();
Map<String, String> scramMap = getScramCredentialStringsForUser(scramImage, userName);
opConsumer.accept(UPDATE_CLIENT_QUOTA, "Updated client quotas for " + userName, migrationState ->
migrationClient.configClient().writeClientQuotas(entity.entries(), quotaMap, scramMap, migrationState));
});
}
void handleProducerIdSnapshot(ProducerIdsImage image, KRaftMigrationOperationConsumer operationConsumer) {
if (image.isEmpty()) {
// No producer IDs have been allocated, nothing to dual-write
return;
}
Optional<ProducerIdsBlock> zkProducerId = migrationClient.readProducerId();
if (zkProducerId.isPresent()) {
if (zkProducerId.get().nextBlockFirstId() != image.nextProducerId()) {
operationConsumer.accept(UPDATE_PRODUCER_ID, "Set next producer ID", migrationState ->
migrationClient.writeProducerId(image.nextProducerId(), migrationState));
}
} else {
operationConsumer.accept(UPDATE_PRODUCER_ID, "Set next producer ID", migrationState ->
migrationClient.writeProducerId(image.nextProducerId(), migrationState));
}
}
void handleConfigsDelta(ConfigurationsImage configsImage, ConfigurationsDelta configsDelta, KRaftMigrationOperationConsumer operationConsumer) {
Set<ConfigResource> updatedResources = configsDelta.changes().keySet();
updatedResources.forEach(configResource -> {
Map<String, String> props = configsImage.configMapForResource(configResource);
if (props.isEmpty()) {
operationConsumer.accept("DeleteConfig", "Deleted configs for " + configResource, migrationState ->
migrationClient.configClient().deleteConfigs(configResource, migrationState));
} else {
operationConsumer.accept("UpdateConfig", "Updated configs for " + configResource, migrationState ->
migrationClient.configClient().writeConfigs(configResource, props, migrationState));
}
});
}
void handleClientQuotasDelta(MetadataImage metadataImage, MetadataDelta metadataDelta, KRaftMigrationOperationConsumer operationConsumer) {
if ((metadataDelta.clientQuotasDelta() != null) || (metadataDelta.scramDelta() != null)) {
// A list of users with scram or quota changes
HashSet<String> users = new HashSet<>();
// Populate list with users with scram changes
if (metadataDelta.scramDelta() != null) {
metadataDelta.scramDelta().changes().forEach((scramMechanism, changes) ->
changes.forEach((userName, changeOpt) -> users.add(userName))
);
}
// Populate list with users with quota changes
// and apply quota changes to all non-user quota changes
if (metadataDelta.clientQuotasDelta() != null) {
metadataDelta.clientQuotasDelta().changes().forEach((clientQuotaEntity, clientQuotaDelta) -> {
if ((clientQuotaEntity.entries().containsKey(ClientQuotaEntity.USER)) &&
(!clientQuotaEntity.entries().containsKey(ClientQuotaEntity.CLIENT_ID))) {
String userName = clientQuotaEntity.entries().get(ClientQuotaEntity.USER);
// Add clientQuotaEntity to list to process at the end
users.add(userName);
} else {
Map<String, Double> quotaMap = metadataImage.clientQuotas().entities().get(clientQuotaEntity).quotaMap();
operationConsumer.accept(UPDATE_CLIENT_QUOTA, "Updated client quota " + clientQuotaEntity, migrationState ->
migrationClient.configClient().writeClientQuotas(clientQuotaEntity.entries(), quotaMap, Collections.emptyMap(), migrationState));
}
});
}
// Update user scram and quota data for each user with changes in either.
users.forEach(userName -> {
Map<String, String> userScramMap = getScramCredentialStringsForUser(metadataImage.scram(), userName);
ClientQuotaEntity clientQuotaEntity = new ClientQuotaEntity(Collections.singletonMap(ClientQuotaEntity.USER, userName));
if ((metadataImage.clientQuotas() == null) ||
(metadataImage.clientQuotas().entities().get(clientQuotaEntity) == null)) {
operationConsumer.accept(UPDATE_CLIENT_QUOTA, "Updated scram credentials for " + clientQuotaEntity, migrationState ->
migrationClient.configClient().writeClientQuotas(clientQuotaEntity.entries(), Collections.emptyMap(), userScramMap, migrationState));
} else {
Map<String, Double> quotaMap = metadataImage.clientQuotas().entities().get(clientQuotaEntity).quotaMap();
operationConsumer.accept(UPDATE_CLIENT_QUOTA, "Updated client quota for " + clientQuotaEntity, migrationState ->
migrationClient.configClient().writeClientQuotas(clientQuotaEntity.entries(), quotaMap, userScramMap, migrationState));
}
});
}
}
void handleProducerIdDelta(ProducerIdsDelta delta, KRaftMigrationOperationConsumer operationConsumer) {
operationConsumer.accept(UPDATE_PRODUCER_ID, "Set next producer ID", migrationState ->
migrationClient.writeProducerId(delta.nextProducerId(), migrationState));
}
private ResourcePattern resourcePatternFromAcl(StandardAcl acl) {
return new ResourcePattern(acl.resourceType(), acl.resourceName(), acl.patternType());
}
void handleAclsSnapshot(AclsImage image, KRaftMigrationOperationConsumer operationConsumer) {
// Need to compare contents of image with all ACLs in ZK and issue updates
Map<ResourcePattern, Set<AccessControlEntry>> allAclsInSnapshot = new HashMap<>();
image.acls().values().forEach(standardAcl -> {
ResourcePattern resourcePattern = resourcePatternFromAcl(standardAcl);
allAclsInSnapshot.computeIfAbsent(resourcePattern, __ -> new HashSet<>()).add(
new AccessControlEntry(standardAcl.principal(), standardAcl.host(), standardAcl.operation(), standardAcl.permissionType())
);
});
Set<ResourcePattern> newResources = new HashSet<>(allAclsInSnapshot.keySet());
Set<ResourcePattern> resourcesToDelete = new HashSet<>();
Map<ResourcePattern, Set<AccessControlEntry>> changedResources = new HashMap<>();
migrationClient.aclClient().iterateAcls((resourcePattern, accessControlEntries) -> {
newResources.remove(resourcePattern);
if (!allAclsInSnapshot.containsKey(resourcePattern)) {
resourcesToDelete.add(resourcePattern);
} else {
Set<AccessControlEntry> snapshotEntries = allAclsInSnapshot.get(resourcePattern);
if (!snapshotEntries.equals(accessControlEntries)) {
changedResources.put(resourcePattern, snapshotEntries);
}
}
});
newResources.forEach(resourcePattern -> {
// newResources is generated from allAclsInSnapshot, and we don't remove from that map, so this unguarded .get() is safe
Set<AccessControlEntry> accessControlEntries = allAclsInSnapshot.get(resourcePattern);
String logMsg = "Wrote " + accessControlEntries.size() + " for resource " + resourcePattern;
operationConsumer.accept(UPDATE_ACL, logMsg, migrationState ->
migrationClient.aclClient().writeResourceAcls(resourcePattern, accessControlEntries, migrationState));
});
resourcesToDelete.forEach(deletedResource -> {
String logMsg = "Deleted resource " + deletedResource + " which has no ACLs in snapshot";
operationConsumer.accept(DELETE_ACL, logMsg, migrationState ->
migrationClient.aclClient().deleteResource(deletedResource, migrationState));
});
changedResources.forEach((resourcePattern, accessControlEntries) -> {
String logMsg = "Wrote " + accessControlEntries.size() + " for resource " + resourcePattern;
operationConsumer.accept(UPDATE_ACL, logMsg, migrationState ->
migrationClient.aclClient().writeResourceAcls(resourcePattern, accessControlEntries, migrationState));
});
}
void handleAclsDelta(AclsImage prevImage, AclsImage image, AclsDelta delta, KRaftMigrationOperationConsumer operationConsumer) {
// Need to collect all ACLs for any changed resource pattern
Map<ResourcePattern, List<AccessControlEntry>> aclsToWrite = new HashMap<>();
delta.changes().forEach((aclId, aclChange) -> {
if (aclChange.isPresent()) {
ResourcePattern resourcePattern = resourcePatternFromAcl(aclChange.get());
aclsToWrite.put(resourcePattern, new ArrayList<>());
} else {
// We need to look in the previous image to get deleted ACLs resource pattern
StandardAcl deletedAcl = prevImage.acls().get(aclId);
if (deletedAcl == null) {
errorLogger.accept("Cannot delete ACL " + aclId + " from ZK since it is missing from previous AclImage");
} else {
ResourcePattern resourcePattern = resourcePatternFromAcl(deletedAcl);
aclsToWrite.put(resourcePattern, new ArrayList<>());
}
}
});
// Iterate through the new image to collect any ACLs for these changed resources
image.acls().forEach((uuid, standardAcl) -> {
ResourcePattern resourcePattern = resourcePatternFromAcl(standardAcl);
List<AccessControlEntry> entries = aclsToWrite.get(resourcePattern);
if (entries != null) {
entries.add(new AccessControlEntry(standardAcl.principal(), standardAcl.host(), standardAcl.operation(), standardAcl.permissionType()));
}
});
// If there are no more ACLs for a resource, delete it. Otherwise, update it with the new set of ACLs
aclsToWrite.forEach((resourcePattern, accessControlEntries) -> {
if (accessControlEntries.isEmpty()) {
String logMsg = "Deleted resource " + resourcePattern + " which has no more ACLs";
operationConsumer.accept(DELETE_ACL, logMsg, migrationState ->
migrationClient.aclClient().deleteResource(resourcePattern, migrationState));
} else {
String logMsg = "Wrote " + accessControlEntries.size() + " for resource " + resourcePattern;
operationConsumer.accept(UPDATE_ACL, logMsg, migrationState ->
migrationClient.aclClient().writeResourceAcls(resourcePattern, accessControlEntries, migrationState));
}
});
}
void handleDelegationTokenDelta(DelegationTokenImage image, DelegationTokenDelta delta, KRaftMigrationOperationConsumer operationConsumer) {
Set<String> updatedTokens = delta.changes().keySet();
updatedTokens.forEach(tokenId -> {
DelegationTokenData tokenData = image.tokens().get(tokenId);
if (tokenData == null) {
operationConsumer.accept("DeleteDelegationToken", "Deleted DelegationToken for " + tokenId, migrationState ->
migrationClient.delegationTokenClient().deleteDelegationToken(tokenId, migrationState));
} else {
operationConsumer.accept("UpdateDelegationToken", "Updated DelegationToken for " + tokenId, migrationState ->
migrationClient.delegationTokenClient().writeDelegationToken(tokenId, tokenData.tokenInformation(), migrationState));
}
});
}
void handleDelegationTokenSnapshot(DelegationTokenImage image, KRaftMigrationOperationConsumer operationConsumer) {
image.tokens().keySet().forEach(tokenId -> {
DelegationTokenData tokenData = image.tokens().get(tokenId);
operationConsumer.accept("UpdateDelegationToken", "Updated DelegationToken for " + tokenId, migrationState ->
migrationClient.delegationTokenClient().writeDelegationToken(tokenId, tokenData.tokenInformation(), migrationState));
});
List<String> tokens = migrationClient.delegationTokenClient().getDelegationTokens();
tokens.forEach(tokenId -> {
if (!image.tokens().containsKey(tokenId)) {
operationConsumer.accept("DeleteDelegationToken", "Deleted DelegationToken for " + tokenId, migrationState ->
migrationClient.delegationTokenClient().deleteDelegationToken(tokenId, migrationState));
}
});
}
}

View File

@ -1,37 +0,0 @@
/*
* 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.image.MetadataDelta;
import org.apache.kafka.image.MetadataImage;
public interface LegacyPropagator {
void startup();
void shutdown();
void publishMetadata(MetadataImage image);
void sendRPCsToBrokersFromMetadataDelta(MetadataDelta delta,
MetadataImage image,
int zkControllerEpoch);
void sendRPCsToBrokersFromMetadataImage(MetadataImage image, int zkControllerEpoch);
void clear();
}

View File

@ -1,91 +0,0 @@
/*
* 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.server.common.ApiMessageAndVersion;
import org.apache.kafka.server.common.ProducerIdsBlock;
import java.util.List;
import java.util.Optional;
import java.util.Set;
import java.util.function.Consumer;
/**
* Methods for interacting with ZooKeeper during a KIP-866 migration. The migration leadership state is stored under
* a ZNode /migration. All write operations to ZK during the migration are performed as a multi-op transaction which
* also updates the state of /migration.
*/
public interface MigrationClient {
/**
* Read or initialize the ZK migration leader state in ZK. If the ZNode is absent, the given {@code initialState}
* will be written and subsequently returned with the zkVersion of the node. If the ZNode is present, it will be
* read and returned.
* @param initialState An initial, empty, state to write to ZooKeeper for the migration state.
* @return The existing migration state, or the initial state given.
*/
ZkMigrationLeadershipState getOrCreateMigrationRecoveryState(ZkMigrationLeadershipState initialState);
/**
* Overwrite the migration state in ZK. This is done as a conditional update using
* {@link ZkMigrationLeadershipState#migrationZkVersion()}. If the conditional update fails, an exception is thrown.
* @param state The migration state to persist
* @return The persisted migration state or an exception.
*/
ZkMigrationLeadershipState setMigrationRecoveryState(ZkMigrationLeadershipState state);
/**
* Attempt to claim controller leadership of the cluster in ZooKeeper. This involves overwriting the /controller
* and /controller_epoch ZNodes. The epoch given by {@code state} must be greater than the current epoch in ZooKeeper.
*
*
* @param state The current migration leadership state
* @return An updated migration leadership state including the version of /controller_epoch ZNode, if the
* leadership claim was successful. Otherwise, return the previous state unmodified.
*/
ZkMigrationLeadershipState claimControllerLeadership(ZkMigrationLeadershipState state);
/**
* Release an existing claim on the cluster leadership in ZooKeeper. This involves deleting the /controller ZNode
* so that another controller can claim leadership.
*
* @param state The current migration leadership state.
* @return An updated migration leadership state with controllerZkVersion = 1, or raise an exception if ZooKeeper
*
*
*/
ZkMigrationLeadershipState releaseControllerLeadership(ZkMigrationLeadershipState state);
TopicMigrationClient topicClient();
ConfigMigrationClient configClient();
AclMigrationClient aclClient();
DelegationTokenMigrationClient delegationTokenClient();
Optional<ProducerIdsBlock> readProducerId();
ZkMigrationLeadershipState writeProducerId(
long nextProducerId,
ZkMigrationLeadershipState state
);
void readAllMetadata(Consumer<List<ApiMessageAndVersion>> batchConsumer, Consumer<Integer> brokerIdConsumer);
Set<Integer> readBrokerIds();
}

View File

@ -1,26 +0,0 @@
/*
* 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;
/**
* Wrapped for authentication exceptions in the migration client such as ZooKeeper AuthFailedException
*/
public class MigrationClientAuthException extends MigrationClientException {
public MigrationClientAuthException(Throwable t) {
super(t);
}
}

View File

@ -1,38 +0,0 @@
/*
* 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.KafkaException;
/**
* Unchecked exception that can be thrown by the migration client.
*
* Authentication related errors should use {@link MigrationClientAuthException}.
*/
public class MigrationClientException extends KafkaException {
public MigrationClientException(String message, Throwable t) {
super(message, t);
}
public MigrationClientException(Throwable t) {
super(t);
}
public MigrationClientException(String message) {
super(message);
}
}

View File

@ -1,61 +0,0 @@
/*
* 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;
/**
* 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.
*
* UNINITIALIZEDINACTIVEDUAL_WRITE
*
*
*
*
*
*
* WAIT_FOR_CONTROLLER_QUORUMZK_MIGRATIONKRAFT_CONTROLLER_TO_BROKER_COMM
*
*
*
*
*
*
* BECOME_CONTROLLERWAIT_FOR_BROKERS
*/
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.
WAIT_FOR_BROKERS(false), // Wait for Zk brokers to be ready for migration.
BECOME_CONTROLLER(false), // Become controller for the Zk Brokers.
ZK_MIGRATION(false), // The cluster has satisfied the migration criteria
SYNC_KRAFT_TO_ZK(false), // A full sync of metadata from KRaft to ZK.
KRAFT_CONTROLLER_TO_BROKER_COMM(true), // First communication from Controller to send full RPCs to the Zk brokers.
DUAL_WRITE(true); // The data has been migrated
private final boolean allowDualWrite;
MigrationDriverState(boolean allowDualWrite) {
this.allowDualWrite = allowDualWrite;
}
public boolean allowDualWrite() {
return allowDualWrite;
}
}

View File

@ -1,129 +0,0 @@
/*
* 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.metadata.MetadataRecordType;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.server.common.ApiMessageAndVersion;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.TreeMap;
import java.util.concurrent.TimeUnit;
/**
* Holds the results of a ZK to KRaft metadata migration. The {@link #toString()} can be used to provide a
* human-readable summary of the migration.
*/
public class MigrationManifest {
public static class Builder {
private final Time time;
private final long startTimeNanos;
private final Map<MetadataRecordType, Integer> counts = new HashMap<>();
private int batches = 0;
private int total = 0;
private long batchDurationsNs = 0;
private long endTimeNanos = 0;
Builder(Time time) {
this.time = time;
this.startTimeNanos = time.nanoseconds();
}
public void acceptBatch(List<ApiMessageAndVersion> recordBatch, long durationNs) {
batches++;
batchDurationsNs += durationNs;
recordBatch.forEach(apiMessageAndVersion -> {
MetadataRecordType type = MetadataRecordType.fromId(apiMessageAndVersion.message().apiKey());
counts.merge(type, 1, Integer::sum);
total++;
});
}
public MigrationManifest build() {
if (endTimeNanos == 0) {
endTimeNanos = time.nanoseconds();
}
Map<MetadataRecordType, Integer> orderedCounts = new TreeMap<>(counts);
return new MigrationManifest(total, batches, batchDurationsNs, endTimeNanos - startTimeNanos, orderedCounts);
}
}
private final int totalRecords;
private final int totalBatches;
private final long totalBatchDurationsNs;
private final long durationNanos;
private final Map<MetadataRecordType, Integer> recordTypeCounts;
MigrationManifest(
int totalRecords,
int totalBatches,
long totalBatchDurationsNs,
long durationNanos,
Map<MetadataRecordType, Integer> recordTypeCounts
) {
this.totalRecords = totalRecords;
this.totalBatches = totalBatches;
this.totalBatchDurationsNs = totalBatchDurationsNs;
this.durationNanos = durationNanos;
this.recordTypeCounts = Collections.unmodifiableMap(recordTypeCounts);
}
public static Builder newBuilder(Time time) {
return new Builder(time);
}
public long durationMs() {
return TimeUnit.NANOSECONDS.toMillis(durationNanos);
}
public double avgBatchDurationMs() {
if (totalBatches == 0) {
return -1;
}
return 1.0 * TimeUnit.NANOSECONDS.toMillis(totalBatchDurationsNs) / totalBatches;
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
MigrationManifest that = (MigrationManifest) o;
return totalRecords == that.totalRecords &&
totalBatches == that.totalBatches &&
totalBatchDurationsNs == that.totalBatchDurationsNs &&
durationNanos == that.durationNanos &&
recordTypeCounts.equals(that.recordTypeCounts);
}
@Override
public int hashCode() {
return Objects.hash(totalRecords, totalBatches, totalBatchDurationsNs, durationNanos, recordTypeCounts);
}
public String toString() {
return String.format(
"%d records were generated in %d ms across %d batches. The average time spent waiting on a " +
"batch was %.2f ms. The record types were %s",
totalRecords, durationMs(), totalBatches, avgBatchDurationMs(), recordTypeCounts);
}
}

View File

@ -1,85 +0,0 @@
/*
* 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.TopicIdPartition;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.metadata.PartitionRegistration;
import java.util.EnumSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
public interface TopicMigrationClient {
enum TopicVisitorInterest {
TOPICS,
PARTITIONS
}
interface TopicVisitor {
void visitTopic(String topicName, Uuid topicId, Map<Integer, List<Integer>> assignments);
default void visitPartition(TopicIdPartition topicIdPartition, PartitionRegistration partitionRegistration) {
}
}
void iterateTopics(EnumSet<TopicVisitorInterest> interests, TopicVisitor visitor);
Set<String> readPendingTopicDeletions();
ZkMigrationLeadershipState clearPendingTopicDeletions(
Set<String> pendingTopicDeletions,
ZkMigrationLeadershipState state
);
ZkMigrationLeadershipState deleteTopic(
String topicName,
ZkMigrationLeadershipState state
);
ZkMigrationLeadershipState createTopic(
String topicName,
Uuid topicId,
Map<Integer, PartitionRegistration> topicPartitions,
ZkMigrationLeadershipState state
);
ZkMigrationLeadershipState updateTopic(
String topicName,
Uuid topicId,
Map<Integer, PartitionRegistration> topicPartitions,
ZkMigrationLeadershipState state
);
ZkMigrationLeadershipState createTopicPartitions(
Map<String, Map<Integer, PartitionRegistration>> topicPartitions,
ZkMigrationLeadershipState state
);
ZkMigrationLeadershipState updateTopicPartitions(
Map<String, Map<Integer, PartitionRegistration>> topicPartitions,
ZkMigrationLeadershipState state
);
ZkMigrationLeadershipState deleteTopicPartitions(
Map<String, Set<Integer>> topicPartitions,
ZkMigrationLeadershipState state
);
}

View File

@ -1,30 +0,0 @@
/*
* 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.raft.OffsetAndEpoch;
import org.apache.kafka.server.common.ApiMessageAndVersion;
import java.util.List;
import java.util.concurrent.CompletableFuture;
public interface ZkRecordConsumer {
CompletableFuture<?> beginMigration();
CompletableFuture<?> acceptBatch(List<ApiMessageAndVersion> recordBatch);
CompletableFuture<OffsetAndEpoch> completeMigration();
void abortMigration();
}

View File

@ -17,15 +17,12 @@
package org.apache.kafka.controller;
import org.apache.kafka.common.metadata.ZkMigrationStateRecord;
import org.apache.kafka.metadata.bootstrap.BootstrapMetadata;
import org.apache.kafka.metadata.migration.ZkMigrationState;
import org.apache.kafka.server.common.MetadataVersion;
import org.junit.jupiter.api.Test;
import java.util.Optional;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertThrows;
@ -45,7 +42,6 @@ public class ActivationRecordsGeneratorTest {
logMsg -> assertEquals("Performing controller activation. The metadata log appears to be empty. " +
"Appending 1 bootstrap record(s) at metadata.version 3.0-IV1 from bootstrap source 'test'.", logMsg),
-1L,
false,
BootstrapMetadata.fromVersion(MetadataVersion.MINIMUM_BOOTSTRAP_VERSION, "test"),
MetadataVersion.MINIMUM_KRAFT_VERSION
);
@ -57,7 +53,6 @@ public class ActivationRecordsGeneratorTest {
"Appending 1 bootstrap record(s) at metadata.version 3.4-IV0 from bootstrap " +
"source 'test'. Setting the ZK migration state to NONE since this is a de-novo KRaft cluster.", logMsg),
-1L,
false,
BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_4_IV0, "test"),
MetadataVersion.IBP_3_4_IV0
);
@ -65,50 +60,11 @@ public class ActivationRecordsGeneratorTest {
assertEquals(2, result.records().size());
result = ActivationRecordsGenerator.recordsForEmptyLog(
logMsg -> assertEquals("Performing controller activation. The metadata log appears to be empty. " +
"Appending 1 bootstrap record(s) at metadata.version 3.4-IV0 from bootstrap " +
"source 'test'. Putting the controller into pre-migration mode. No metadata updates will be allowed " +
"until the ZK metadata has been migrated.", logMsg),
-1L,
true,
BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_4_IV0, "test"),
MetadataVersion.IBP_3_4_IV0
);
assertTrue(result.isAtomic());
assertEquals(2, result.records().size());
assertEquals(
"The bootstrap metadata.version 3.3-IV2 does not support ZK migrations. Cannot continue with ZK migrations enabled.",
assertThrows(RuntimeException.class, () ->
ActivationRecordsGenerator.recordsForEmptyLog(
logMsg -> fail(),
-1L,
true,
BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_3_IV2, "test"),
MetadataVersion.IBP_3_3_IV2
)).getMessage()
);
result = ActivationRecordsGenerator.recordsForEmptyLog(
logMsg -> assertEquals("Performing controller activation. The metadata log appears to be empty. " +
"Appending 1 bootstrap record(s) in metadata transaction at metadata.version 3.6-IV1 from bootstrap " +
"source 'test'. Setting the ZK migration state to NONE since this is a de-novo KRaft cluster.", logMsg),
-1L,
false,
BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_6_IV1, "test"),
MetadataVersion.IBP_3_6_IV1
);
assertFalse(result.isAtomic());
assertEquals(4, result.records().size());
result = ActivationRecordsGenerator.recordsForEmptyLog(
logMsg -> assertEquals("Performing controller activation. The metadata log appears to be empty. " +
"Appending 1 bootstrap record(s) in metadata transaction at metadata.version 3.6-IV1 from bootstrap " +
"source 'test'. Putting the controller into pre-migration mode. No metadata updates will be allowed " +
"until the ZK metadata has been migrated.", logMsg),
-1L,
true,
BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_6_IV1, "test"),
MetadataVersion.IBP_3_6_IV1
);
@ -121,49 +77,11 @@ public class ActivationRecordsGeneratorTest {
"metadata.version 3.6-IV1 from bootstrap source 'test'. Setting the ZK migration state to NONE " +
"since this is a de-novo KRaft cluster.", logMsg),
0L,
false,
BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_6_IV1, "test"),
MetadataVersion.IBP_3_6_IV1
);
assertFalse(result.isAtomic());
assertEquals(5, result.records().size());
result = ActivationRecordsGenerator.recordsForEmptyLog(
logMsg -> assertEquals("Performing controller activation. Aborting partial bootstrap records " +
"transaction at offset 0. Re-appending 1 bootstrap record(s) in new metadata transaction at " +
"metadata.version 3.6-IV1 from bootstrap source 'test'. Putting the controller into pre-migration " +
"mode. No metadata updates will be allowed until the ZK metadata has been migrated.", logMsg),
0L,
true,
BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_6_IV1, "test"),
MetadataVersion.IBP_3_6_IV1
);
assertFalse(result.isAtomic());
assertEquals(5, result.records().size());
assertEquals(
"Detected partial bootstrap records transaction at 0, but the metadata.version 3.6-IV0 does not " +
"support transactions. Cannot continue.",
assertThrows(RuntimeException.class, () ->
ActivationRecordsGenerator.recordsForEmptyLog(
logMsg -> assertEquals("", logMsg),
0L,
true,
BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_6_IV0, "test"),
MetadataVersion.IBP_3_6_IV0
)).getMessage()
);
}
FeatureControlManager buildFeatureControl(
MetadataVersion metadataVersion,
Optional<ZkMigrationState> zkMigrationState
) {
FeatureControlManager featureControl = new FeatureControlManager.Builder()
.setMetadataVersion(metadataVersion).build();
zkMigrationState.ifPresent(migrationState ->
featureControl.replay((ZkMigrationStateRecord) migrationState.toRecord().message()));
return featureControl;
}
@Test
@ -174,8 +92,7 @@ public class ActivationRecordsGeneratorTest {
logMsg -> assertEquals("Performing controller activation. No metadata.version feature level " +
"record was found in the log. Treating the log as version 3.0-IV1.", logMsg),
-1L,
false,
buildFeatureControl(MetadataVersion.MINIMUM_KRAFT_VERSION, Optional.empty()),
ZkMigrationState.NONE,
MetadataVersion.MINIMUM_KRAFT_VERSION
);
assertTrue(result.isAtomic());
@ -184,8 +101,7 @@ public class ActivationRecordsGeneratorTest {
result = ActivationRecordsGenerator.recordsForNonEmptyLog(
logMsg -> assertEquals("Performing controller activation.", logMsg),
-1L,
false,
buildFeatureControl(MetadataVersion.IBP_3_3_IV0, Optional.empty()),
ZkMigrationState.NONE,
MetadataVersion.IBP_3_3_IV0
);
assertTrue(result.isAtomic());
@ -195,8 +111,7 @@ public class ActivationRecordsGeneratorTest {
logMsg -> assertEquals("Performing controller activation. Loaded ZK migration state of NONE. "
+ "This is expected because this is a de-novo KRaft cluster.", logMsg),
-1L,
false,
buildFeatureControl(MetadataVersion.IBP_3_4_IV0, Optional.empty()),
ZkMigrationState.NONE,
MetadataVersion.IBP_3_4_IV0
);
assertTrue(result.isAtomic());
@ -207,8 +122,7 @@ public class ActivationRecordsGeneratorTest {
"transaction at offset 42. Loaded ZK migration state of NONE. " +
"This is expected because this is a de-novo KRaft cluster.", logMsg),
42L,
false,
buildFeatureControl(MetadataVersion.IBP_3_6_IV1, Optional.empty()),
ZkMigrationState.NONE,
MetadataVersion.IBP_3_6_IV1
);
assertTrue(result.isAtomic());
@ -221,8 +135,7 @@ public class ActivationRecordsGeneratorTest {
ActivationRecordsGenerator.recordsForNonEmptyLog(
logMsg -> fail(),
42L,
false,
buildFeatureControl(MetadataVersion.IBP_3_6_IV0, Optional.empty()),
ZkMigrationState.NONE,
MetadataVersion.IBP_3_6_IV0
)).getMessage()
);
@ -230,97 +143,35 @@ public class ActivationRecordsGeneratorTest {
@Test
public void testActivationMessageForNonEmptyLogWithMigrations() {
ControllerResult<Void> result;
assertEquals(
"Should not have ZK migrations enabled on a cluster running metadata.version 3.3-IV0",
"Should not have ZkMigrationState.MIGRATION on a cluster running metadata version 3.3-IV0.",
assertThrows(RuntimeException.class, () ->
ActivationRecordsGenerator.recordsForNonEmptyLog(
logMsg -> fail(),
-1L,
true,
buildFeatureControl(MetadataVersion.IBP_3_3_IV0, Optional.empty()),
ZkMigrationState.MIGRATION,
MetadataVersion.IBP_3_3_IV0
)).getMessage()
);
assertEquals(
"Should not have ZK migrations enabled on a cluster that was created in KRaft mode.",
"Cannot load ZkMigrationState.MIGRATION because ZK migration is no longer supported.",
assertThrows(RuntimeException.class, () ->
ActivationRecordsGenerator.recordsForNonEmptyLog(
logMsg -> fail(),
-1L,
true,
buildFeatureControl(MetadataVersion.IBP_3_4_IV0, Optional.empty()),
MetadataVersion.IBP_3_4_IV0
ZkMigrationState.MIGRATION,
MetadataVersion.IBP_3_9_IV0
)
).getMessage()
);
result = ActivationRecordsGenerator.recordsForNonEmptyLog(
logMsg -> assertEquals("Performing controller activation. Loaded ZK migration state of " +
"PRE_MIGRATION. Activating pre-migration controller without empty log. There may be a partial " +
"migration.", logMsg),
-1L,
true,
buildFeatureControl(MetadataVersion.IBP_3_4_IV0, Optional.of(ZkMigrationState.PRE_MIGRATION)),
MetadataVersion.IBP_3_4_IV0
);
assertTrue(result.isAtomic());
assertEquals(0, result.records().size());
result = ActivationRecordsGenerator.recordsForNonEmptyLog(
logMsg -> assertEquals("Performing controller activation. Loaded ZK migration state of " +
"PRE_MIGRATION.", logMsg),
-1L,
true,
buildFeatureControl(MetadataVersion.IBP_3_6_IV1, Optional.of(ZkMigrationState.PRE_MIGRATION)),
MetadataVersion.IBP_3_6_IV1
);
assertTrue(result.isAtomic());
assertEquals(0, result.records().size());
result = ActivationRecordsGenerator.recordsForNonEmptyLog(
logMsg -> assertEquals("Performing controller activation. Loaded ZK migration state of MIGRATION. " +
"Staying in ZK migration mode since 'zookeeper.metadata.migration.enable' is still 'true'.", logMsg),
-1L,
true,
buildFeatureControl(MetadataVersion.IBP_3_4_IV0, Optional.of(ZkMigrationState.MIGRATION)),
MetadataVersion.IBP_3_4_IV0
);
assertTrue(result.isAtomic());
assertEquals(0, result.records().size());
result = ActivationRecordsGenerator.recordsForNonEmptyLog(
logMsg -> assertEquals("Performing controller activation. Loaded ZK migration state of MIGRATION. " +
"Completing the ZK migration since this controller was configured with " +
"'zookeeper.metadata.migration.enable' set to 'false'.", logMsg),
-1L,
false,
buildFeatureControl(MetadataVersion.IBP_3_4_IV0, Optional.of(ZkMigrationState.MIGRATION)),
MetadataVersion.IBP_3_4_IV0
);
assertTrue(result.isAtomic());
assertEquals(1, result.records().size());
result = ActivationRecordsGenerator.recordsForNonEmptyLog(
logMsg -> assertEquals("Performing controller activation. Aborting in-progress metadata " +
"transaction at offset 42. Loaded ZK migration state of MIGRATION. Completing the ZK migration " +
"since this controller was configured with 'zookeeper.metadata.migration.enable' set to 'false'.", logMsg),
42L,
false,
buildFeatureControl(MetadataVersion.IBP_3_6_IV1, Optional.of(ZkMigrationState.MIGRATION)),
MetadataVersion.IBP_3_6_IV1
);
assertTrue(result.isAtomic());
assertEquals(2, result.records().size());
ControllerResult<Void> result;
result = ActivationRecordsGenerator.recordsForNonEmptyLog(
logMsg -> assertEquals("Performing controller activation. Loaded ZK migration state of " +
"POST_MIGRATION.", logMsg),
-1L,
false,
buildFeatureControl(MetadataVersion.IBP_3_4_IV0, Optional.of(ZkMigrationState.POST_MIGRATION)),
ZkMigrationState.POST_MIGRATION,
MetadataVersion.IBP_3_4_IV0
);
assertTrue(result.isAtomic());
@ -330,32 +181,7 @@ public class ActivationRecordsGeneratorTest {
logMsg -> assertEquals("Performing controller activation. Aborting in-progress metadata " +
"transaction at offset 42. Loaded ZK migration state of POST_MIGRATION.", logMsg),
42L,
false,
buildFeatureControl(MetadataVersion.IBP_3_6_IV1, Optional.of(ZkMigrationState.POST_MIGRATION)),
MetadataVersion.IBP_3_6_IV1
);
assertTrue(result.isAtomic());
assertEquals(1, result.records().size());
result = ActivationRecordsGenerator.recordsForNonEmptyLog(
logMsg -> assertEquals("Performing controller activation. Loaded ZK migration state of " +
"POST_MIGRATION. Ignoring 'zookeeper.metadata.migration.enable' value of 'true' since the " +
"ZK migration has been completed.", logMsg),
-1L,
true,
buildFeatureControl(MetadataVersion.IBP_3_4_IV0, Optional.of(ZkMigrationState.POST_MIGRATION)),
MetadataVersion.IBP_3_4_IV0
);
assertTrue(result.isAtomic());
assertEquals(0, result.records().size());
result = ActivationRecordsGenerator.recordsForNonEmptyLog(
logMsg -> assertEquals("Performing controller activation. Aborting in-progress metadata " +
"transaction at offset 42. Loaded ZK migration state of POST_MIGRATION. Ignoring " +
"'zookeeper.metadata.migration.enable' value of 'true' since the ZK migration has been completed.", logMsg),
42L,
true,
buildFeatureControl(MetadataVersion.IBP_3_6_IV1, Optional.of(ZkMigrationState.POST_MIGRATION)),
ZkMigrationState.POST_MIGRATION,
MetadataVersion.IBP_3_6_IV1
);
assertTrue(result.isAtomic());

View File

@ -852,33 +852,4 @@ public class ClusterControlManagerTest {
clusterControl.brokerRegistrations().get(1).epoch());
}
}
@Test
public void testRegistrationWithIncorrectInterBrokerListenerName() {
ClusterControlManager clusterControl = new ClusterControlManager.Builder().
setClusterId("pjvUwj3ZTEeSVQmUiH3IJw").
setFeatureControlManager(new FeatureControlManager.Builder().build()).
setBrokerUncleanShutdownHandler((brokerId, records) -> { }).
setInterBrokerListenerName("INTERNAL").
setZkMigrationEnabled(true).
build();
clusterControl.activate();
assertEquals("Broker does not have the current inter.broker.listener INTERNAL",
assertThrows(InvalidRegistrationException.class,
() -> clusterControl.registerBroker(
new BrokerRegistrationRequestData().
setBrokerId(1).
setClusterId(clusterControl.clusterId()).
setIncarnationId(Uuid.fromString("07OOcU7MQFeSmGAFPP2Zww")).
setLogDirs(Arrays.asList(Uuid.fromString("Vv1gzkM2QpuE-PPrIc6XEw"))).
setIsMigratingZkBroker(true).
setListeners(new BrokerRegistrationRequestData.ListenerCollection(Collections.singleton(
new BrokerRegistrationRequestData.Listener().
setName("PLAINTEXT").
setHost("example.com").
setPort(9092).
setSecurityProtocol(SecurityProtocol.PLAINTEXT.id)).iterator())),
111,
new FinalizedControllerFeatures(Collections.emptyMap(), 100L))).getMessage());
}
}

View File

@ -55,7 +55,6 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.params.provider.Arguments.arguments;
@Timeout(value = 40)
@ -338,22 +337,6 @@ public class PartitionChangeBuilderTest {
1);
}
/**
* Test that shrinking the ISR does increase the leader epoch in later MVs when ZK migration is on.
*/
@ParameterizedTest
@ValueSource(strings = {"3.6-IV0", "3.7-IV2", "4.0-IV0"})
public void testLeaderEpochBumpOnIsrShrinkWithZkMigration(String metadataVersionString) {
MetadataVersion metadataVersion = MetadataVersion.fromVersionString(metadataVersionString);
testTriggerLeaderEpochBumpIfNeeded(
createFooBuilder(metadataVersion).
setZkMigrationEnabled(true).
setTargetIsrWithBrokerStates(
AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(Arrays.asList(2, 1))),
new PartitionChangeRecord(),
1);
}
/**
* Test that expanding the ISR doesn't increase the leader epoch.
*/
@ -368,22 +351,6 @@ public class PartitionChangeBuilderTest {
NO_LEADER_CHANGE);
}
/**
* Test that expanding the ISR doesn't increase the leader epoch during ZK migration.
*/
@ParameterizedTest
@ValueSource(strings = {"3.4-IV0", "3.5-IV2", "3.6-IV0", "3.7-IV2", "4.0-IV0"})
public void testNoLeaderEpochBumpOnIsrExpansionDuringMigration(String metadataVersionString) {
MetadataVersion metadataVersion = MetadataVersion.fromVersionString(metadataVersionString);
testTriggerLeaderEpochBumpIfNeeded(
createFooBuilder(metadataVersion).
setZkMigrationEnabled(true).
setTargetIsrWithBrokerStates(
AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(Arrays.asList(2, 1, 3, 4))),
new PartitionChangeRecord(),
NO_LEADER_CHANGE);
}
/**
* Test that changing the replica set such that not all the old replicas remain
* always results in a leader epoch increase.
@ -631,18 +598,9 @@ public class PartitionChangeBuilderTest {
);
}
private static Stream<Arguments> leaderRecoveryAndZkMigrationParams() {
return Stream.of(
arguments(true, true),
arguments(true, false),
arguments(false, true),
arguments(false, false)
);
}
@ParameterizedTest
@MethodSource("leaderRecoveryAndZkMigrationParams")
public void testChangeInLeadershipDoesNotChangeRecoveryState(boolean isLeaderRecoverySupported, boolean zkMigrationsEnabled) {
@ValueSource(booleans = {true, false})
public void testChangeInLeadershipDoesNotChangeRecoveryState(boolean isLeaderRecoverySupported) {
final byte noChange = (byte) -1;
int leaderId = 1;
LeaderRecoveryState recoveryState = LeaderRecoveryState.RECOVERING;
@ -671,7 +629,6 @@ public class PartitionChangeBuilderTest {
metadataVersion,
2
);
offlineBuilder.setZkMigrationEnabled(zkMigrationsEnabled);
// Set the target ISR to empty to indicate that the last leader is offline
offlineBuilder.setTargetIsrWithBrokerStates(Collections.emptyList());
@ -698,7 +655,6 @@ public class PartitionChangeBuilderTest {
metadataVersion,
2
);
onlineBuilder.setZkMigrationEnabled(zkMigrationsEnabled);
// The only broker in the ISR is elected leader and stays in the recovering
changeRecord = (PartitionChangeRecord) onlineBuilder.build().get().message();
@ -712,8 +668,8 @@ public class PartitionChangeBuilderTest {
}
@ParameterizedTest
@MethodSource("leaderRecoveryAndZkMigrationParams")
void testUncleanSetsLeaderRecoveringState(boolean isLeaderRecoverySupported, boolean zkMigrationsEnabled) {
@ValueSource(booleans = {true, false})
void testUncleanSetsLeaderRecoveringState(boolean isLeaderRecoverySupported) {
final byte noChange = (byte) -1;
int leaderId = 1;
PartitionRegistration registration = new PartitionRegistration.Builder().
@ -741,7 +697,6 @@ public class PartitionChangeBuilderTest {
metadataVersion,
2
).setElection(Election.UNCLEAN);
onlineBuilder.setZkMigrationEnabled(zkMigrationsEnabled);
// The partition should stay as recovering
PartitionChangeRecord changeRecord = (PartitionChangeRecord) onlineBuilder
.build()

View File

@ -57,7 +57,7 @@ public class QuorumControllerMetricsIntegrationTest {
final AtomicBoolean closed = new AtomicBoolean(false);
MockControllerMetrics() {
super(Optional.empty(), Time.SYSTEM, true);
super(Optional.empty(), Time.SYSTEM);
}
@Override

View File

@ -100,7 +100,6 @@ import org.apache.kafka.metadata.RecordTestUtils.TestThroughAllIntermediateImage
import org.apache.kafka.metadata.authorizer.StandardAuthorizer;
import org.apache.kafka.metadata.bootstrap.BootstrapMetadata;
import org.apache.kafka.metadata.migration.ZkMigrationState;
import org.apache.kafka.metadata.migration.ZkRecordConsumer;
import org.apache.kafka.metadata.util.BatchFileWriter;
import org.apache.kafka.metalog.LocalLogManager;
import org.apache.kafka.metalog.LocalLogManagerTestEnv;
@ -111,7 +110,6 @@ import org.apache.kafka.server.common.Features;
import org.apache.kafka.server.common.KRaftVersion;
import org.apache.kafka.server.common.MetadataVersion;
import org.apache.kafka.server.common.TopicIdPartition;
import org.apache.kafka.server.fault.FaultHandlerException;
import org.apache.kafka.snapshot.FileRawSnapshotReader;
import org.apache.kafka.snapshot.Snapshots;
import org.apache.kafka.test.TestUtils;
@ -121,14 +119,11 @@ import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.CsvSource;
import org.junit.jupiter.params.provider.EnumSource;
import org.junit.jupiter.params.provider.ValueSource;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
@ -162,7 +157,6 @@ import static org.apache.kafka.controller.ConfigurationControlManagerTest.entry;
import static org.apache.kafka.controller.ControllerRequestContextUtil.ANONYMOUS_CONTEXT;
import static org.apache.kafka.controller.ControllerRequestContextUtil.anonymousContextFor;
import static org.apache.kafka.controller.QuorumControllerIntegrationTestUtils.brokerFeatures;
import static org.apache.kafka.controller.QuorumControllerIntegrationTestUtils.forceRenounce;
import static org.apache.kafka.controller.QuorumControllerIntegrationTestUtils.pause;
import static org.apache.kafka.controller.QuorumControllerIntegrationTestUtils.registerBrokersAndUnfence;
import static org.apache.kafka.controller.QuorumControllerIntegrationTestUtils.sendBrokerHeartbeatToUnfenceBrokers;
@ -1382,75 +1376,9 @@ public class QuorumControllerTest {
appender)).getMessage());
}
@Test
public void testBootstrapZkMigrationRecord() throws Exception {
assertEquals(ZkMigrationState.PRE_MIGRATION,
checkBootstrapZkMigrationRecord(MetadataVersion.IBP_3_4_IV0, true));
assertEquals(ZkMigrationState.NONE,
checkBootstrapZkMigrationRecord(MetadataVersion.IBP_3_4_IV0, false));
assertEquals(ZkMigrationState.NONE,
checkBootstrapZkMigrationRecord(MetadataVersion.IBP_3_3_IV0, false));
assertEquals(
"The bootstrap metadata.version 3.3-IV0 does not support ZK migrations. Cannot continue with ZK migrations enabled.",
assertThrows(FaultHandlerException.class, () ->
checkBootstrapZkMigrationRecord(MetadataVersion.IBP_3_3_IV0, true)).getCause().getMessage()
);
}
public ZkMigrationState checkBootstrapZkMigrationRecord(
MetadataVersion metadataVersion,
boolean migrationEnabled
) throws Exception {
try (
LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(1).build();
QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv).
setControllerBuilderInitializer(controllerBuilder ->
controllerBuilder.setZkMigrationEnabled(migrationEnabled)
).
setBootstrapMetadata(BootstrapMetadata.fromVersion(metadataVersion, "test")).
build()
) {
QuorumController active = controlEnv.activeController();
ZkMigrationState zkMigrationState = active.appendReadEvent("read migration state", OptionalLong.empty(),
() -> active.featureControl().zkMigrationState()).get(30, TimeUnit.SECONDS);
testToImages(logEnv.allRecords());
return zkMigrationState;
}
}
@Test
public void testUpgradeMigrationStateFrom34() throws Exception {
try (LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(1).build()) {
// In 3.4, we only wrote a PRE_MIGRATION to the log. In that software version, we defined this
// as enum value 1. In 3.5+ software, this enum value is redefined as MIGRATION
BootstrapMetadata bootstrapMetadata = BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_4_IV0, "test");
List<ApiMessageAndVersion> initialRecords = new ArrayList<>(bootstrapMetadata.records());
initialRecords.add(ZkMigrationState.of((byte) 1).toRecord());
logEnv.appendInitialRecords(initialRecords);
try (
QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv).
setControllerBuilderInitializer(controllerBuilder ->
controllerBuilder.setZkMigrationEnabled(true)
).
setBootstrapMetadata(bootstrapMetadata).
build()
) {
QuorumController active = controlEnv.activeController();
assertEquals(active.featureControl().zkMigrationState(), ZkMigrationState.MIGRATION);
assertFalse(active.featureControl().inPreMigrationMode());
}
testToImages(logEnv.allRecords());
}
}
FeatureControlManager getActivationRecords(
MetadataVersion metadataVersion,
Optional<ZkMigrationState> stateInLog,
boolean zkMigrationEnabled
Optional<ZkMigrationState> stateInLog
) {
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
FeatureControlManager featureControlManager = new FeatureControlManager.Builder()
@ -1458,16 +1386,13 @@ public class QuorumControllerTest {
.setMetadataVersion(metadataVersion)
.build();
stateInLog.ifPresent(zkMigrationState ->
featureControlManager.replay((ZkMigrationStateRecord) zkMigrationState.toRecord().message()));
ControllerResult<Void> result = ActivationRecordsGenerator.generate(
msg -> { },
!stateInLog.isPresent(),
-1L,
zkMigrationEnabled,
BootstrapMetadata.fromVersion(metadataVersion, "test"),
featureControlManager);
stateInLog.orElseGet(() -> ZkMigrationState.NONE),
metadataVersion);
RecordTestUtils.replayAll(featureControlManager, result.records());
return featureControlManager;
}
@ -1476,21 +1401,11 @@ public class QuorumControllerTest {
public void testActivationRecords33() {
FeatureControlManager featureControl;
assertEquals(
"The bootstrap metadata.version 3.3-IV0 does not support ZK migrations. Cannot continue with ZK migrations enabled.",
assertThrows(RuntimeException.class, () -> getActivationRecords(MetadataVersion.IBP_3_3_IV0, Optional.empty(), true)).getMessage()
);
featureControl = getActivationRecords(MetadataVersion.IBP_3_3_IV0, Optional.empty(), false);
featureControl = getActivationRecords(MetadataVersion.IBP_3_3_IV0, Optional.empty());
assertEquals(MetadataVersion.IBP_3_3_IV0, featureControl.metadataVersion());
assertEquals(ZkMigrationState.NONE, featureControl.zkMigrationState());
assertEquals(
"Should not have ZK migrations enabled on a cluster running metadata.version 3.3-IV0",
assertThrows(RuntimeException.class, () -> getActivationRecords(MetadataVersion.IBP_3_3_IV0, Optional.of(ZkMigrationState.NONE), true)).getMessage()
);
featureControl = getActivationRecords(MetadataVersion.IBP_3_3_IV0, Optional.of(ZkMigrationState.NONE), false);
featureControl = getActivationRecords(MetadataVersion.IBP_3_3_IV0, Optional.of(ZkMigrationState.NONE));
assertEquals(MetadataVersion.IBP_3_3_IV0, featureControl.metadataVersion());
assertEquals(ZkMigrationState.NONE, featureControl.zkMigrationState());
}
@ -1499,67 +1414,32 @@ public class QuorumControllerTest {
public void testActivationRecords34() {
FeatureControlManager featureControl;
featureControl = getActivationRecords(MetadataVersion.IBP_3_4_IV0, Optional.empty(), true);
assertEquals(MetadataVersion.IBP_3_4_IV0, featureControl.metadataVersion());
assertEquals(ZkMigrationState.PRE_MIGRATION, featureControl.zkMigrationState());
featureControl = getActivationRecords(MetadataVersion.IBP_3_4_IV0, Optional.empty(), false);
featureControl = getActivationRecords(MetadataVersion.IBP_3_4_IV0, Optional.empty());
assertEquals(MetadataVersion.IBP_3_4_IV0, featureControl.metadataVersion());
assertEquals(ZkMigrationState.NONE, featureControl.zkMigrationState());
assertEquals(
"Should not have ZK migrations enabled on a cluster that was created in KRaft mode.",
assertThrows(RuntimeException.class, () -> getActivationRecords(MetadataVersion.IBP_3_4_IV0, Optional.of(ZkMigrationState.NONE), true)).getMessage()
);
featureControl = getActivationRecords(MetadataVersion.IBP_3_4_IV0, Optional.of(ZkMigrationState.NONE), false);
featureControl = getActivationRecords(MetadataVersion.IBP_3_4_IV0, Optional.of(ZkMigrationState.NONE));
assertEquals(MetadataVersion.IBP_3_4_IV0, featureControl.metadataVersion());
assertEquals(ZkMigrationState.NONE, featureControl.zkMigrationState());
featureControl = getActivationRecords(MetadataVersion.IBP_3_4_IV0, Optional.of(ZkMigrationState.PRE_MIGRATION), true);
assertEquals(MetadataVersion.IBP_3_4_IV0, featureControl.metadataVersion());
assertEquals(ZkMigrationState.PRE_MIGRATION, featureControl.zkMigrationState());
featureControl = getActivationRecords(MetadataVersion.IBP_3_4_IV0, Optional.of(ZkMigrationState.MIGRATION), true);
assertEquals(MetadataVersion.IBP_3_4_IV0, featureControl.metadataVersion());
assertEquals(ZkMigrationState.MIGRATION, featureControl.zkMigrationState());
featureControl = getActivationRecords(MetadataVersion.IBP_3_4_IV0, Optional.of(ZkMigrationState.MIGRATION), false);
assertEquals(MetadataVersion.IBP_3_4_IV0, featureControl.metadataVersion());
assertEquals(ZkMigrationState.POST_MIGRATION, featureControl.zkMigrationState());
featureControl = getActivationRecords(MetadataVersion.IBP_3_4_IV0, Optional.of(ZkMigrationState.POST_MIGRATION), true);
assertEquals(MetadataVersion.IBP_3_4_IV0, featureControl.metadataVersion());
assertEquals(ZkMigrationState.POST_MIGRATION, featureControl.zkMigrationState());
featureControl = getActivationRecords(MetadataVersion.IBP_3_4_IV0, Optional.of(ZkMigrationState.POST_MIGRATION), false);
assertEquals(MetadataVersion.IBP_3_4_IV0, featureControl.metadataVersion());
assertEquals(ZkMigrationState.POST_MIGRATION, featureControl.zkMigrationState());
}
@Test
public void testActivationRecordsNonEmptyLog() {
FeatureControlManager featureControl = getActivationRecords(
MetadataVersion.IBP_3_4_IV0, Optional.empty(), true);
assertEquals(MetadataVersion.IBP_3_4_IV0, featureControl.metadataVersion());
assertEquals(ZkMigrationState.PRE_MIGRATION, featureControl.zkMigrationState());
MetadataVersion.IBP_3_9_IV0, Optional.empty());
assertEquals(MetadataVersion.IBP_3_9_IV0, featureControl.metadataVersion());
assertEquals(ZkMigrationState.NONE, featureControl.zkMigrationState());
}
@ParameterizedTest
@ValueSource(booleans = {true, false})
public void testActivationRecordsPartialBootstrap(boolean zkMigrationEnabled) {
FeatureControlManager featureControlManager = new FeatureControlManager.Builder()
.setSnapshotRegistry(new SnapshotRegistry(new LogContext()))
.setMetadataVersion(MetadataVersion.IBP_3_6_IV1)
.build();
@Test
public void testActivationRecordsPartialBootstrap() {
ControllerResult<Void> result = ActivationRecordsGenerator.generate(
logMsg -> { },
true,
0L,
zkMigrationEnabled,
BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_6_IV1, "test"),
featureControlManager);
ZkMigrationState.NONE,
MetadataVersion.IBP_3_6_IV1);
assertFalse(result.isAtomic());
assertTrue(RecordTestUtils.recordAtIndexAs(
AbortTransactionRecord.class, result.records(), 0).isPresent());
@ -1569,27 +1449,6 @@ public class QuorumControllerTest {
EndTransactionRecord.class, result.records(), result.records().size() - 1).isPresent());
}
@Test
public void testMigrationsEnabledForOldBootstrapMetadataVersion() throws Exception {
try (
LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(1).build()
) {
QuorumControllerTestEnv.Builder controlEnvBuilder = new QuorumControllerTestEnv.Builder(logEnv).
setControllerBuilderInitializer(controllerBuilder ->
controllerBuilder.setZkMigrationEnabled(true)
).
setBootstrapMetadata(BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_3_IV0, "test"));
QuorumControllerTestEnv controlEnv = controlEnvBuilder.build();
QuorumController active = controlEnv.activeController();
assertEquals(ZkMigrationState.NONE, active.appendReadEvent("read migration state", OptionalLong.empty(),
() -> active.featureControl().zkMigrationState()).get(30, TimeUnit.SECONDS));
assertThrows(FaultHandlerException.class, controlEnv::close);
testToImages(logEnv.allRecords());
}
}
/**
* Tests all intermediate images lead to the same final image for each image & delta type.
* @param fromRecords
@ -1618,12 +1477,6 @@ public class QuorumControllerTest {
@Test
public void testActivationRecordsPartialTransaction() {
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
FeatureControlManager featureControlManager = new FeatureControlManager.Builder()
.setSnapshotRegistry(snapshotRegistry)
.setMetadataVersion(MetadataVersion.IBP_3_6_IV1)
.build();
OffsetControlManager offsetControlManager = new OffsetControlManager.Builder().build();
offsetControlManager.replay(new BeginTransactionRecord(), 10);
offsetControlManager.handleCommitBatch(Batch.data(20, 1, 1L, 0,
@ -1633,9 +1486,9 @@ public class QuorumControllerTest {
logMsg -> { },
false,
offsetControlManager.transactionStartOffset(),
false,
BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_6_IV1, "test"),
featureControlManager);
ZkMigrationState.NONE,
MetadataVersion.IBP_3_6_IV1);
assertTrue(result.isAtomic());
offsetControlManager.replay(
@ -1647,12 +1500,6 @@ public class QuorumControllerTest {
@Test
public void testActivationRecordsPartialTransactionNoSupport() {
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
FeatureControlManager featureControlManager = new FeatureControlManager.Builder()
.setSnapshotRegistry(snapshotRegistry)
.setMetadataVersion(MetadataVersion.IBP_3_6_IV0)
.build();
OffsetControlManager offsetControlManager = new OffsetControlManager.Builder().build();
offsetControlManager.replay(new BeginTransactionRecord(), 10);
offsetControlManager.handleCommitBatch(Batch.data(20, 1, 1L, 0,
@ -1663,129 +1510,9 @@ public class QuorumControllerTest {
msg -> { },
false,
offsetControlManager.transactionStartOffset(),
false,
BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_6_IV0, "test"),
featureControlManager)
ZkMigrationState.NONE,
MetadataVersion.IBP_3_6_IV0)
);
}
private static final List<ApiMessageAndVersion> ZK_MIGRATION_RECORDS =
Collections.unmodifiableList(Arrays.asList(
new ApiMessageAndVersion(new TopicRecord().
setName("spam").
setTopicId(Uuid.fromString("qvRJLpDYRHmgEi8_TPBYTQ")),
(short) 0),
new ApiMessageAndVersion(new PartitionRecord().setPartitionId(0).
setTopicId(Uuid.fromString("qvRJLpDYRHmgEi8_TPBYTQ")).setReplicas(Arrays.asList(0, 1, 2)).
setIsr(Arrays.asList(0, 1, 2)).setRemovingReplicas(Collections.emptyList()).
setAddingReplicas(Collections.emptyList()).setLeader(0).setLeaderEpoch(0).
setPartitionEpoch(0), (short) 0),
new ApiMessageAndVersion(new PartitionRecord().setPartitionId(1).
setTopicId(Uuid.fromString("qvRJLpDYRHmgEi8_TPBYTQ")).setReplicas(Arrays.asList(1, 2, 0)).
setIsr(Arrays.asList(1, 2, 0)).setRemovingReplicas(Collections.emptyList()).
setAddingReplicas(Collections.emptyList()).setLeader(1).setLeaderEpoch(0).
setPartitionEpoch(0), (short) 0)
));
@Test
public void testFailoverDuringMigrationTransaction() throws Exception {
try (
LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(3).build()
) {
QuorumControllerTestEnv.Builder controlEnvBuilder = new QuorumControllerTestEnv.Builder(logEnv).
setControllerBuilderInitializer(controllerBuilder -> controllerBuilder.setZkMigrationEnabled(true)).
setBootstrapMetadata(BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_6_IV1, "test"));
QuorumControllerTestEnv controlEnv = controlEnvBuilder.build();
QuorumController active = controlEnv.activeController(true);
ZkRecordConsumer migrationConsumer = active.zkRecordConsumer();
migrationConsumer.beginMigration().get(30, TimeUnit.SECONDS);
migrationConsumer.acceptBatch(ZK_MIGRATION_RECORDS).get(30, TimeUnit.SECONDS);
forceRenounce(active);
// Ensure next controller doesn't see the topic from partial migration
QuorumController newActive = controlEnv.activeController(true);
CompletableFuture<Map<String, ResultOrError<Uuid>>> results =
newActive.findTopicIds(ANONYMOUS_CONTEXT, Collections.singleton("spam"));
assertEquals(
Errors.UNKNOWN_TOPIC_OR_PARTITION,
results.get(30, TimeUnit.SECONDS).get("spam").error().error());
assertEquals(
ZkMigrationState.PRE_MIGRATION,
newActive.appendReadEvent("read migration state", OptionalLong.empty(),
() -> newActive.featureControl().zkMigrationState()
).get(30, TimeUnit.SECONDS)
);
// Ensure the migration can happen on new active controller
migrationConsumer = newActive.zkRecordConsumer();
migrationConsumer.beginMigration().get(30, TimeUnit.SECONDS);
migrationConsumer.acceptBatch(ZK_MIGRATION_RECORDS).get(30, TimeUnit.SECONDS);
migrationConsumer.completeMigration().get(30, TimeUnit.SECONDS);
results = newActive.findTopicIds(ANONYMOUS_CONTEXT, Collections.singleton("spam"));
assertTrue(results.get(30, TimeUnit.SECONDS).get("spam").isResult());
assertEquals(ZkMigrationState.MIGRATION, newActive.appendReadEvent("read migration state", OptionalLong.empty(),
() -> newActive.featureControl().zkMigrationState()).get(30, TimeUnit.SECONDS));
}
}
@ParameterizedTest
@EnumSource(value = MetadataVersion.class, names = {"IBP_3_4_IV0", "IBP_3_5_IV0", "IBP_3_6_IV0", "IBP_3_6_IV1"})
public void testBrokerHeartbeatDuringMigration(MetadataVersion metadataVersion) throws Exception {
try (
LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(1).build()
) {
QuorumControllerTestEnv.Builder controlEnvBuilder = new QuorumControllerTestEnv.Builder(logEnv).
setControllerBuilderInitializer(controllerBuilder ->
controllerBuilder
.setZkMigrationEnabled(true)
.setMaxIdleIntervalNs(OptionalLong.of(TimeUnit.MILLISECONDS.toNanos(100)))
).
setBootstrapMetadata(BootstrapMetadata.fromVersion(metadataVersion, "test"));
QuorumControllerTestEnv controlEnv = controlEnvBuilder.build();
QuorumController active = controlEnv.activeController(true);
// Register a ZK broker
BrokerRegistrationReply reply = active.registerBroker(ANONYMOUS_CONTEXT,
new BrokerRegistrationRequestData().
setBrokerId(0).
setRack(null).
setClusterId(active.clusterId()).
setIsMigratingZkBroker(true).
setFeatures(brokerFeatures(metadataVersion, metadataVersion)).
setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwB0")).
setListeners(new ListenerCollection(singletonList(new Listener().
setName("PLAINTEXT").setHost("localhost").
setPort(9092)).iterator()))).get();
// Start migration
ZkRecordConsumer migrationConsumer = active.zkRecordConsumer();
migrationConsumer.beginMigration().get(30, TimeUnit.SECONDS);
// Interleave migration batches with heartbeats. Ensure the heartbeat events use the correct
// offset when adding to the purgatory. Otherwise, we get errors like:
// There is already a deferred event with offset 292. We should not add one with an offset of 241 which is lower than that.
for (int i = 0; i < 100; i++) {
Uuid topicId = Uuid.randomUuid();
String topicName = "testBrokerHeartbeatDuringMigration" + i;
Future<?> migrationFuture = migrationConsumer.acceptBatch(
Arrays.asList(
new ApiMessageAndVersion(new TopicRecord().setTopicId(topicId).setName(topicName), (short) 0),
new ApiMessageAndVersion(new PartitionRecord().setTopicId(topicId).setPartitionId(0).setIsr(Arrays.asList(0, 1, 2)), (short) 0)));
active.processBrokerHeartbeat(ANONYMOUS_CONTEXT, new BrokerHeartbeatRequestData().
setWantFence(false).setBrokerEpoch(reply.epoch()).setBrokerId(0).
setCurrentMetadataOffset(100000L + i));
migrationFuture.get();
}
// Ensure that we can complete a heartbeat even though we leave migration transaction hanging
assertEquals(new BrokerHeartbeatReply(true, false, false, false),
active.processBrokerHeartbeat(ANONYMOUS_CONTEXT, new BrokerHeartbeatRequestData().
setWantFence(false).setBrokerEpoch(reply.epoch()).setBrokerId(0).
setCurrentMetadataOffset(100100L)).get());
}
}
}

View File

@ -25,8 +25,6 @@ import com.yammer.metrics.core.MetricName;
import com.yammer.metrics.core.MetricsRegistry;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.ValueSource;
import java.util.Arrays;
import java.util.Collections;
@ -36,16 +34,14 @@ import java.util.Optional;
import static org.junit.jupiter.api.Assertions.assertEquals;
public class QuorumControllerMetricsTest {
@ParameterizedTest
@ValueSource(booleans = {false, true})
public void testMetricNames(boolean inMigration) {
@Test
public void testMetricNames() {
MetricsRegistry registry = new MetricsRegistry();
MockTime time = new MockTime();
try {
try (QuorumControllerMetrics metrics = new QuorumControllerMetrics(
Optional.of(registry),
time,
inMigration)) {
time)) {
HashSet<String> expected = new HashSet<>(Arrays.asList(
"kafka.controller:type=ControllerEventManager,name=EventQueueProcessingTimeMs",
"kafka.controller:type=ControllerEventManager,name=EventQueueTimeMs",
@ -59,11 +55,6 @@ public class QuorumControllerMetricsTest {
"kafka.controller:type=KafkaController,name=NewActiveControllersCount",
"kafka.controller:type=KafkaController,name=TimedOutBrokerHeartbeatCount"
));
if (inMigration) {
expected.add("kafka.controller:type=KafkaController,name=ZkWriteBehindLag");
expected.add("kafka.controller:type=KafkaController,name=ZkWriteSnapshotTimeMs");
expected.add("kafka.controller:type=KafkaController,name=ZkWriteDeltaTimeMs");
}
ControllerMetricsTestUtils.assertMetricsForTypeEqual(registry, "kafka.controller", expected);
}
ControllerMetricsTestUtils.assertMetricsForTypeEqual(registry, "kafka.controller",
@ -77,7 +68,7 @@ public class QuorumControllerMetricsTest {
public void testUpdateEventQueueTime() {
MetricsRegistry registry = new MetricsRegistry();
MockTime time = new MockTime();
try (QuorumControllerMetrics metrics = new QuorumControllerMetrics(Optional.of(registry), time, false)) {
try (QuorumControllerMetrics metrics = new QuorumControllerMetrics(Optional.of(registry), time)) {
metrics.updateEventQueueTime(1000);
assertMetricHistogram(registry, metricName("ControllerEventManager", "EventQueueTimeMs"), 1, 1000);
} finally {
@ -89,7 +80,7 @@ public class QuorumControllerMetricsTest {
public void testUpdateEventQueueProcessingTime() {
MetricsRegistry registry = new MetricsRegistry();
MockTime time = new MockTime();
try (QuorumControllerMetrics metrics = new QuorumControllerMetrics(Optional.of(registry), time, false)) {
try (QuorumControllerMetrics metrics = new QuorumControllerMetrics(Optional.of(registry), time)) {
metrics.updateEventQueueProcessingTime(1000);
assertMetricHistogram(registry, metricName("ControllerEventManager", "EventQueueProcessingTimeMs"), 1, 1000);
} finally {
@ -102,11 +93,10 @@ public class QuorumControllerMetricsTest {
MetricsRegistry registry = new MetricsRegistry();
MockTime time = new MockTime();
time.sleep(1000);
try (QuorumControllerMetrics metrics = new QuorumControllerMetrics(Optional.of(registry), time, true)) {
try (QuorumControllerMetrics metrics = new QuorumControllerMetrics(Optional.of(registry), time)) {
metrics.setLastAppliedRecordOffset(100);
metrics.setLastAppliedRecordTimestamp(500);
metrics.setLastCommittedRecordOffset(50);
metrics.updateDualWriteOffset(40L);
metrics.setActive(true);
for (int i = 0; i < 2; i++) {
metrics.incrementTimedOutHeartbeats();
@ -145,12 +135,6 @@ public class QuorumControllerMetricsTest {
.get(metricName("KafkaController", "LastCommittedRecordOffset"));
assertEquals(50, lastCommittedRecordOffset.value());
@SuppressWarnings("unchecked")
Gauge<Long> zkWriteBehindLag = (Gauge<Long>) registry
.allMetrics()
.get(metricName("KafkaController", "ZkWriteBehindLag"));
assertEquals(10L, zkWriteBehindLag.value());
@SuppressWarnings("unchecked")
Gauge<Long> timedOutBrokerHeartbeats = (Gauge<Long>) registry
.allMetrics()
@ -179,51 +163,6 @@ public class QuorumControllerMetricsTest {
}
}
@Test
public void testUpdateZKWriteBehindLag() {
MetricsRegistry registry = new MetricsRegistry();
MockTime time = new MockTime();
// test zkWriteBehindLag metric when NOT in dual-write mode
try (QuorumControllerMetrics metrics = new QuorumControllerMetrics(Optional.of(registry), time, true)) {
metrics.updateDualWriteOffset(0);
@SuppressWarnings("unchecked")
Gauge<Long> zkWriteBehindLag = (Gauge<Long>) registry
.allMetrics()
.get(metricName("KafkaController", "ZkWriteBehindLag"));
assertEquals(0, zkWriteBehindLag.value());
} finally {
registry.shutdown();
}
// test zkWriteBehindLag metric when in dual-write mode
try (QuorumControllerMetrics metrics = new QuorumControllerMetrics(Optional.of(registry), time, true)) {
metrics.updateDualWriteOffset(90);
metrics.setLastCommittedRecordOffset(100);
metrics.setActive(true);
@SuppressWarnings("unchecked")
Gauge<Long> zkWriteBehindLag = (Gauge<Long>) registry
.allMetrics()
.get(metricName("KafkaController", "ZkWriteBehindLag"));
assertEquals(10, zkWriteBehindLag.value());
} finally {
registry.shutdown();
}
// test zkWriteBehindLag metric when in dual-write mode and not active
try (QuorumControllerMetrics metrics = new QuorumControllerMetrics(Optional.of(registry), time, true)) {
metrics.updateDualWriteOffset(90);
metrics.setLastCommittedRecordOffset(100);
metrics.setActive(false);
@SuppressWarnings("unchecked")
Gauge<Long> zkWriteBehindLag = (Gauge<Long>) registry
.allMetrics()
.get(metricName("KafkaController", "ZkWriteBehindLag"));
assertEquals(0, zkWriteBehindLag.value());
} finally {
registry.shutdown();
}
}
private static void assertMetricHistogram(MetricsRegistry registry, MetricName metricName, long count, double sum) {
Histogram histogram = (Histogram) registry.allMetrics().get(metricName);

View File

@ -1,56 +0,0 @@
/*
* 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.acl.AccessControlEntry;
import org.apache.kafka.common.resource.ResourcePattern;
import java.util.ArrayList;
import java.util.Collection;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Set;
import java.util.function.BiConsumer;
public class CapturingAclMigrationClient implements AclMigrationClient {
public List<ResourcePattern> deletedResources = new ArrayList<>();
public LinkedHashMap<ResourcePattern, Collection<AccessControlEntry>> updatedResources = new LinkedHashMap<>();
public void reset() {
deletedResources.clear();
updatedResources.clear();
}
@Override
public ZkMigrationLeadershipState deleteResource(ResourcePattern resourcePattern, ZkMigrationLeadershipState state) {
deletedResources.add(resourcePattern);
return state;
}
@Override
public ZkMigrationLeadershipState writeResourceAcls(ResourcePattern resourcePattern, Collection<AccessControlEntry> aclsToWrite, ZkMigrationLeadershipState state) {
updatedResources.put(resourcePattern, aclsToWrite);
return state;
}
@Override
public void iterateAcls(BiConsumer<ResourcePattern, Set<AccessControlEntry>> aclConsumer) {
}
}

View File

@ -1,76 +0,0 @@
/*
* 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.config.ConfigResource;
import java.util.ArrayList;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.function.BiConsumer;
import java.util.function.Consumer;
public class CapturingConfigMigrationClient implements ConfigMigrationClient {
public List<ConfigResource> deletedResources = new ArrayList<>();
public LinkedHashMap<ConfigResource, Map<String, String>> writtenConfigs = new LinkedHashMap<>();
public void reset() {
deletedResources.clear();
writtenConfigs.clear();
}
@Override
public void iterateClientQuotas(ClientQuotaVisitor visitor) {
}
@Override
public void iterateBrokerConfigs(BiConsumer<String, Map<String, String>> configConsumer) {
}
@Override
public void iterateTopicConfigs(BiConsumer<String, Map<String, String>> configConsumer) {
}
@Override
public void readTopicConfigs(String topicName, Consumer<Map<String, String>> configConsumer) {
}
@Override
public ZkMigrationLeadershipState writeConfigs(ConfigResource configResource, Map<String, String> configMap, ZkMigrationLeadershipState state) {
writtenConfigs.put(configResource, configMap);
return state;
}
@Override
public ZkMigrationLeadershipState writeClientQuotas(Map<String, String> clientQuotaEntity, Map<String, Double> quotas, Map<String, String> scram, ZkMigrationLeadershipState state) {
return null;
}
@Override
public ZkMigrationLeadershipState deleteConfigs(ConfigResource configResource, ZkMigrationLeadershipState state) {
deletedResources.add(configResource);
return state;
}
}

View File

@ -1,54 +0,0 @@
/*
* 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.security.token.delegation.TokenInformation;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
public class CapturingDelegationTokenMigrationClient implements DelegationTokenMigrationClient {
public List<String> deletedDelegationTokens = new ArrayList<>();
public HashMap<String, TokenInformation> updatedDelegationTokens = new HashMap<>();
public void reset() {
deletedDelegationTokens.clear();
updatedDelegationTokens.clear();
}
@Override
public List<String> getDelegationTokens() {
return new ArrayList<>();
}
@Override
public ZkMigrationLeadershipState deleteDelegationToken(String tokenId, ZkMigrationLeadershipState state) {
deletedDelegationTokens.add(tokenId);
return state;
}
@Override
public ZkMigrationLeadershipState writeDelegationToken(String tokenId, TokenInformation tokenInformation, ZkMigrationLeadershipState state) {
updatedDelegationTokens.put(tokenId, tokenInformation);
return state;
}
}

View File

@ -1,204 +0,0 @@
/*
* 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.server.common.ApiMessageAndVersion;
import org.apache.kafka.server.common.ProducerIdsBlock;
import java.util.Collections;
import java.util.List;
import java.util.Optional;
import java.util.Set;
import java.util.function.Consumer;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
class CapturingMigrationClient implements MigrationClient {
static final MigrationBatchSupplier EMPTY_BATCH_SUPPLIER = new MigrationBatchSupplier() {
};
interface MigrationBatchSupplier {
default List<List<ApiMessageAndVersion>> recordBatches() {
return Collections.emptyList();
}
default List<Integer> brokerIds() {
return Collections.emptyList();
}
}
static Builder newBuilder() {
return new Builder();
}
public static class Builder {
Set<Integer> brokersInZk = Collections.emptySet();
TopicMigrationClient topicMigrationClient = new CapturingTopicMigrationClient();
ConfigMigrationClient configMigrationClient = new CapturingConfigMigrationClient();
AclMigrationClient aclMigrationClient = new CapturingAclMigrationClient();
DelegationTokenMigrationClient delegationTokenMigrationClient = new CapturingDelegationTokenMigrationClient();
MigrationBatchSupplier batchSupplier = EMPTY_BATCH_SUPPLIER;
public Builder setBrokersInZk(int... brokerIds) {
brokersInZk = IntStream.of(brokerIds).boxed().collect(Collectors.toSet());
return this;
}
public Builder setTopicMigrationClient(TopicMigrationClient topicMigrationClient) {
this.topicMigrationClient = topicMigrationClient;
return this;
}
public Builder setConfigMigrationClient(ConfigMigrationClient configMigrationClient) {
this.configMigrationClient = configMigrationClient;
return this;
}
public Builder setAclMigrationClient(AclMigrationClient aclMigrationClient) {
this.aclMigrationClient = aclMigrationClient;
return this;
}
public Builder setDelegationTokenMigrationClient(DelegationTokenMigrationClient delegationTokenMigrationClient) {
this.delegationTokenMigrationClient = delegationTokenMigrationClient;
return this;
}
public Builder setBatchSupplier(MigrationBatchSupplier batchSupplier) {
this.batchSupplier = batchSupplier;
return this;
}
public CapturingMigrationClient build() {
return new CapturingMigrationClient(
brokersInZk,
topicMigrationClient,
configMigrationClient,
aclMigrationClient,
delegationTokenMigrationClient,
batchSupplier
);
}
}
private final Set<Integer> brokerIds;
private final TopicMigrationClient topicMigrationClient;
private final ConfigMigrationClient configMigrationClient;
private final AclMigrationClient aclMigrationClient;
private final DelegationTokenMigrationClient delegationTokenMigrationClient;
private final MigrationBatchSupplier batchSupplier;
private ZkMigrationLeadershipState state = null;
CapturingMigrationClient(
Set<Integer> brokerIdsInZk,
TopicMigrationClient topicMigrationClient,
ConfigMigrationClient configMigrationClient,
AclMigrationClient aclMigrationClient,
DelegationTokenMigrationClient delegationTokenMigrationClient,
MigrationBatchSupplier batchSupplier
) {
this.brokerIds = brokerIdsInZk;
this.topicMigrationClient = topicMigrationClient;
this.configMigrationClient = configMigrationClient;
this.aclMigrationClient = aclMigrationClient;
this.delegationTokenMigrationClient = delegationTokenMigrationClient;
this.batchSupplier = batchSupplier;
}
@Override
public ZkMigrationLeadershipState getOrCreateMigrationRecoveryState(ZkMigrationLeadershipState initialState) {
if (this.state == null) {
this.state = initialState;
}
return this.state;
}
@Override
public ZkMigrationLeadershipState setMigrationRecoveryState(ZkMigrationLeadershipState state) {
this.state = state;
return state;
}
@Override
public ZkMigrationLeadershipState claimControllerLeadership(ZkMigrationLeadershipState state) {
if (state.zkControllerEpochZkVersion() == ZkMigrationLeadershipState.UNKNOWN_ZK_VERSION) {
this.state = state.withZkController(0, 0);
} else {
this.state = state.withZkController(state.zkControllerEpoch() + 1, state.zkControllerEpochZkVersion() + 1);
}
return this.state;
}
@Override
public ZkMigrationLeadershipState releaseControllerLeadership(ZkMigrationLeadershipState state) {
this.state = state.withUnknownZkController();
return this.state;
}
@Override
public TopicMigrationClient topicClient() {
return topicMigrationClient;
}
@Override
public ConfigMigrationClient configClient() {
return configMigrationClient;
}
@Override
public AclMigrationClient aclClient() {
return aclMigrationClient;
}
@Override
public DelegationTokenMigrationClient delegationTokenClient() {
return delegationTokenMigrationClient;
}
@Override
public Optional<ProducerIdsBlock> readProducerId() {
return Optional.empty();
}
@Override
public ZkMigrationLeadershipState writeProducerId(
long nextProducerId,
ZkMigrationLeadershipState state
) {
this.state = state;
return state;
}
@Override
public void readAllMetadata(
Consumer<List<ApiMessageAndVersion>> batchConsumer,
Consumer<Integer> brokerIdConsumer
) {
batchSupplier.recordBatches().forEach(batchConsumer);
batchSupplier.brokerIds().forEach(brokerIdConsumer);
}
@Override
public Set<Integer> readBrokerIds() {
return brokerIds;
}
}

View File

@ -1,114 +0,0 @@
/*
* 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.Uuid;
import org.apache.kafka.metadata.PartitionRegistration;
import java.util.ArrayList;
import java.util.Collections;
import java.util.EnumSet;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
public class CapturingTopicMigrationClient implements TopicMigrationClient {
public List<String> deletedTopics = new ArrayList<>();
public List<String> createdTopics = new ArrayList<>();
public LinkedHashMap<String, Map<Integer, PartitionRegistration>> updatedTopics = new LinkedHashMap<>();
public LinkedHashMap<String, Set<Integer>> newTopicPartitions = new LinkedHashMap<>();
public LinkedHashMap<String, Set<Integer>> updatedTopicPartitions = new LinkedHashMap<>();
public LinkedHashMap<String, Set<Integer>> deletedTopicPartitions = new LinkedHashMap<>();
public void reset() {
createdTopics.clear();
updatedTopicPartitions.clear();
deletedTopics.clear();
updatedTopics.clear();
deletedTopicPartitions.clear();
}
@Override
public void iterateTopics(EnumSet<TopicVisitorInterest> interests, TopicVisitor visitor) {
}
@Override
public Set<String> readPendingTopicDeletions() {
return Collections.emptySet();
}
@Override
public ZkMigrationLeadershipState clearPendingTopicDeletions(
Set<String> pendingTopicDeletions,
ZkMigrationLeadershipState state
) {
return state;
}
@Override
public ZkMigrationLeadershipState deleteTopic(
String topicName,
ZkMigrationLeadershipState state
) {
deletedTopics.add(topicName);
return state;
}
@Override
public ZkMigrationLeadershipState createTopic(String topicName, Uuid topicId, Map<Integer, PartitionRegistration> topicPartitions, ZkMigrationLeadershipState state) {
createdTopics.add(topicName);
return state;
}
@Override
public ZkMigrationLeadershipState updateTopic(
String topicName,
Uuid topicId,
Map<Integer, PartitionRegistration> topicPartitions,
ZkMigrationLeadershipState state
) {
updatedTopics.put(topicName, topicPartitions);
return state;
}
@Override
public ZkMigrationLeadershipState createTopicPartitions(Map<String, Map<Integer, PartitionRegistration>> topicPartitions, ZkMigrationLeadershipState state) {
topicPartitions.forEach((topicName, partitionMap) ->
newTopicPartitions.put(topicName, partitionMap.keySet())
);
return state;
}
@Override
public ZkMigrationLeadershipState updateTopicPartitions(Map<String, Map<Integer, PartitionRegistration>> topicPartitions, ZkMigrationLeadershipState state) {
topicPartitions.forEach((topicName, partitionMap) ->
updatedTopicPartitions.put(topicName, partitionMap.keySet())
);
return state;
}
@Override
public ZkMigrationLeadershipState deleteTopicPartitions(Map<String, Set<Integer>> topicPartitions, ZkMigrationLeadershipState state) {
deletedTopicPartitions.putAll(topicPartitions);
return state;
}
}