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

View File

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

View File

@ -17,7 +17,6 @@
package kafka.server package kafka.server
import kafka.migration.MigrationPropagator
import kafka.network.{DataPlaneAcceptor, SocketServer} import kafka.network.{DataPlaneAcceptor, SocketServer}
import kafka.raft.KafkaRaftManager import kafka.raft.KafkaRaftManager
import kafka.server.QuotaFactory.QuotaManagers import kafka.server.QuotaFactory.QuotaManagers
@ -25,7 +24,6 @@ import kafka.server.QuotaFactory.QuotaManagers
import scala.collection.immutable import scala.collection.immutable
import kafka.server.metadata.{AclPublisher, ClientQuotaMetadataManager, DelegationTokenPublisher, DynamicClientQuotaPublisher, DynamicConfigPublisher, KRaftMetadataCache, KRaftMetadataCachePublisher, ScramPublisher} import kafka.server.metadata.{AclPublisher, ClientQuotaMetadataManager, DelegationTokenPublisher, DynamicClientQuotaPublisher, DynamicConfigPublisher, KRaftMetadataCache, KRaftMetadataCachePublisher, ScramPublisher}
import kafka.utils.{CoreUtils, Logging} import kafka.utils.{CoreUtils, Logging}
import kafka.zk.{KafkaZkClient, ZkMigrationClient}
import org.apache.kafka.common.message.ApiMessageType.ListenerType import org.apache.kafka.common.message.ApiMessageType.ListenerType
import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.security.scram.internals.ScramMechanism 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.{KafkaConfigSchema, ListenerInfo}
import org.apache.kafka.metadata.authorizer.ClusterMetadataAuthorizer import org.apache.kafka.metadata.authorizer.ClusterMetadataAuthorizer
import org.apache.kafka.metadata.bootstrap.BootstrapMetadata 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.metadata.publisher.FeaturesPublisher
import org.apache.kafka.raft.QuorumConfig 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.NodeToControllerChannelManager
import org.apache.kafka.server.authorizer.Authorizer 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} 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._ 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. * A Kafka controller that runs in KRaft (Kafka Raft) mode.
*/ */
@ -111,7 +94,6 @@ class ControllerServer(
var clientQuotaMetadataManager: ClientQuotaMetadataManager = _ var clientQuotaMetadataManager: ClientQuotaMetadataManager = _
var controllerApis: ControllerApis = _ var controllerApis: ControllerApis = _
var controllerApisHandlerPool: KafkaRequestHandlerPool = _ var controllerApisHandlerPool: KafkaRequestHandlerPool = _
var migrationSupport: Option[ControllerMigrationSupport] = None
def kafkaYammerMetrics: KafkaYammerMetrics = KafkaYammerMetrics.INSTANCE def kafkaYammerMetrics: KafkaYammerMetrics = KafkaYammerMetrics.INSTANCE
val metadataPublishers: util.List[MetadataPublisher] = new util.ArrayList[MetadataPublisher]() val metadataPublishers: util.List[MetadataPublisher] = new util.ArrayList[MetadataPublisher]()
@volatile var metadataCache : KRaftMetadataCache = _ @volatile var metadataCache : KRaftMetadataCache = _
@ -171,7 +153,6 @@ class ControllerServer(
val apiVersionManager = new SimpleApiVersionManager( val apiVersionManager = new SimpleApiVersionManager(
ListenerType.CONTROLLER, ListenerType.CONTROLLER,
config.unstableApiVersionsEnabled, config.unstableApiVersionsEnabled,
config.migrationEnabled,
() => featuresPublisher.features() () => featuresPublisher.features()
) )
@ -233,7 +214,7 @@ class ControllerServer(
val maxIdleIntervalNs = config.metadataMaxIdleIntervalNs.fold(OptionalLong.empty)(OptionalLong.of) 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). new QuorumController.Builder(config.nodeId, sharedServer.clusterId).
setTime(time). setTime(time).
@ -255,7 +236,6 @@ class ControllerServer(
setBootstrapMetadata(bootstrapMetadata). setBootstrapMetadata(bootstrapMetadata).
setFatalFaultHandler(sharedServer.fatalQuorumControllerFaultHandler). setFatalFaultHandler(sharedServer.fatalQuorumControllerFaultHandler).
setNonFatalFaultHandler(sharedServer.nonFatalQuorumControllerFaultHandler). setNonFatalFaultHandler(sharedServer.nonFatalQuorumControllerFaultHandler).
setZkMigrationEnabled(config.migrationEnabled).
setDelegationTokenCache(tokenCache). setDelegationTokenCache(tokenCache).
setDelegationTokenSecretKey(delegationTokenKeyString). setDelegationTokenSecretKey(delegationTokenKeyString).
setDelegationTokenMaxLifeMs(config.delegationTokenMaxLifeMs). setDelegationTokenMaxLifeMs(config.delegationTokenMaxLifeMs).
@ -274,39 +254,6 @@ class ControllerServer(
case _ => 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, quotaManagers = QuotaFactory.instantiate(config,
metrics, metrics,
time, time,
@ -347,7 +294,6 @@ class ControllerServer(
time, time,
s"controller-${config.nodeId}-", s"controller-${config.nodeId}-",
QuorumFeatures.defaultFeatureMap(config.unstableFeatureVersionsEnabled), QuorumFeatures.defaultFeatureMap(config.unstableFeatureVersionsEnabled),
config.migrationEnabled,
incarnationId, incarnationId,
listenerInfo) listenerInfo)
@ -490,7 +436,6 @@ class ControllerServer(
registrationsPublisher = null registrationsPublisher = null
if (socketServer != null) if (socketServer != null)
CoreUtils.swallow(socketServer.stopProcessingRequests(), this) CoreUtils.swallow(socketServer.stopProcessingRequests(), this)
migrationSupport.foreach(_.shutdown(this))
if (controller != null) if (controller != null)
controller.beginShutdown() controller.beginShutdown()
if (socketServer != null) if (socketServer != null)

View File

@ -81,7 +81,6 @@ class TestRaftServer(
val apiVersionManager = new SimpleApiVersionManager( val apiVersionManager = new SimpleApiVersionManager(
ListenerType.CONTROLLER, ListenerType.CONTROLLER,
true, true,
false,
() => FinalizedFeatures.fromKRaftVersion(MetadataVersion.MINIMUM_KRAFT_VERSION)) () => FinalizedFeatures.fromKRaftVersion(MetadataVersion.MINIMUM_KRAFT_VERSION))
socketServer = new SocketServer(config, metrics, time, credentialProvider, apiVersionManager) 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 // Clean-up any metrics left around by previous tests
TestUtils.clearYammerMetrics() 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)) () => new FinalizedFeatures(MetadataVersion.latestTesting(), Collections.emptyMap[String, java.lang.Short], 0, true))
var server: SocketServer = _ var server: SocketServer = _
val sockets = new ArrayBuffer[Socket] val sockets = new ArrayBuffer[Socket]

View File

@ -17,8 +17,7 @@
package kafka.server package kafka.server
import org.apache.kafka.common.test.api.{AutoStart, ClusterConfigProperty, ClusterInstance, ClusterTest, ClusterTestExtensions, ClusterTests, Type} import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterInstance, ClusterTest, ClusterTestExtensions, Type}
import org.apache.kafka.common.test.api.RaftClusterInvocationContext.RaftClusterInstance
import org.apache.kafka.clients.ClientResponse import org.apache.kafka.clients.ClientResponse
import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic
import org.apache.kafka.common.message.{BrokerRegistrationRequestData, CreateTopicsRequestData} 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.common.{Node, Uuid}
import org.apache.kafka.server.{ControllerRequestCompletionHandler, NodeToControllerChannelManager} import org.apache.kafka.server.{ControllerRequestCompletionHandler, NodeToControllerChannelManager}
import org.apache.kafka.server.common.{Features, MetadataVersion} import org.apache.kafka.server.common.{Features, MetadataVersion}
import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows} import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.Disabled
import org.junit.jupiter.api.extension.ExtendWith import org.junit.jupiter.api.extension.ExtendWith
import java.util import java.util
@ -205,91 +203,4 @@ class BrokerRegistrationRequestTest {
channelManager.shutdown() 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( new SimpleApiVersionManager(
ListenerType.CONTROLLER, ListenerType.CONTROLLER,
true, true,
false,
() => FinalizedFeatures.fromKRaftVersion(MetadataVersion.latestTesting())), () => FinalizedFeatures.fromKRaftVersion(MetadataVersion.latestTesting())),
metadataCache metadataCache
) )

View File

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

View File

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

View File

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

View File

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

View File

@ -29,12 +29,14 @@ import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.function.Consumer; 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 { public class ActivationRecordsGenerator {
static ControllerResult<Void> recordsForEmptyLog( static ControllerResult<Void> recordsForEmptyLog(
Consumer<String> activationMessageConsumer, Consumer<String> activationMessageConsumer,
long transactionStartOffset, long transactionStartOffset,
boolean zkMigrationEnabled,
BootstrapMetadata bootstrapMetadata, BootstrapMetadata bootstrapMetadata,
MetadataVersion metadataVersion MetadataVersion metadataVersion
) { ) {
@ -89,20 +91,9 @@ public class ActivationRecordsGenerator {
records.addAll(bootstrapMetadata.records()); records.addAll(bootstrapMetadata.records());
if (metadataVersion.isMigrationSupported()) { 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 " + logMessageBuilder.append("Setting the ZK migration state to NONE since this is a de-novo " +
"KRaft cluster. "); "KRaft cluster. ");
records.add(ZkMigrationState.NONE.toRecord()); records.add(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.");
}
} }
activationMessageConsumer.accept(logMessageBuilder.toString().trim()); activationMessageConsumer.accept(logMessageBuilder.toString().trim());
@ -117,9 +108,8 @@ public class ActivationRecordsGenerator {
static ControllerResult<Void> recordsForNonEmptyLog( static ControllerResult<Void> recordsForNonEmptyLog(
Consumer<String> activationMessageConsumer, Consumer<String> activationMessageConsumer,
long transactionStartOffset, long transactionStartOffset,
boolean zkMigrationEnabled, ZkMigrationState zkMigrationState,
FeatureControlManager featureControl, MetadataVersion curMetadataVersion
MetadataVersion metadataVersion
) { ) {
StringBuilder logMessageBuilder = new StringBuilder("Performing controller activation. "); StringBuilder logMessageBuilder = new StringBuilder("Performing controller activation. ");
@ -128,9 +118,9 @@ public class ActivationRecordsGenerator {
// Check for in-flight transaction // Check for in-flight transaction
if (transactionStartOffset != -1L) { if (transactionStartOffset != -1L) {
if (!metadataVersion.isMetadataTransactionSupported()) { if (!curMetadataVersion.isMetadataTransactionSupported()) {
throw new RuntimeException("Detected in-progress transaction at offset " + transactionStartOffset + 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."); " does not support transactions. Cannot continue.");
} else { } else {
logMessageBuilder 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. ") logMessageBuilder.append("No metadata.version feature level record was found in the log. ")
.append("Treating the log as version ") .append("Treating the log as version ")
.append(MetadataVersion.MINIMUM_KRAFT_VERSION) .append(MetadataVersion.MINIMUM_KRAFT_VERSION)
.append(". "); .append(". ");
} }
if (zkMigrationEnabled && !metadataVersion.isMigrationSupported()) { if (curMetadataVersion.isMigrationSupported()) {
throw new RuntimeException("Should not have ZK migrations enabled on a cluster running " + if (zkMigrationState == NONE || zkMigrationState == POST_MIGRATION) {
"metadata.version " + featureControl.metadataVersion());
} else if (metadataVersion.isMigrationSupported()) {
logMessageBuilder logMessageBuilder
.append("Loaded ZK migration state of ") .append("Loaded ZK migration state of ")
.append(featureControl.zkMigrationState()) .append(zkMigrationState)
.append(". "); .append(". ");
switch (featureControl.zkMigrationState()) { if (zkMigrationState == NONE) {
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.");
}
logMessageBuilder.append("This is expected because this is a de-novo KRaft cluster."); 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 { } else {
// This log message is used in zookeeper_migration_test.py throw new RuntimeException("Cannot load ZkMigrationState." + zkMigrationState +
logMessageBuilder " because ZK migration is no longer supported.");
.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());
} }
} else if (zkMigrationState != NONE) {
throw new RuntimeException("Should not have ZkMigrationState." + zkMigrationState +
" on a cluster running metadata version " + curMetadataVersion + ".");
} }
activationMessageConsumer.accept(logMessageBuilder.toString().trim()); activationMessageConsumer.accept(logMessageBuilder.toString().trim());
@ -220,16 +175,16 @@ public class ActivationRecordsGenerator {
Consumer<String> activationMessageConsumer, Consumer<String> activationMessageConsumer,
boolean isEmpty, boolean isEmpty,
long transactionStartOffset, long transactionStartOffset,
boolean zkMigrationEnabled,
BootstrapMetadata bootstrapMetadata, BootstrapMetadata bootstrapMetadata,
FeatureControlManager featureControl ZkMigrationState zkMigrationState,
MetadataVersion curMetadataVersion
) { ) {
if (isEmpty) { if (isEmpty) {
return recordsForEmptyLog(activationMessageConsumer, transactionStartOffset, zkMigrationEnabled, return recordsForEmptyLog(activationMessageConsumer, transactionStartOffset,
bootstrapMetadata, bootstrapMetadata.metadataVersion()); bootstrapMetadata, bootstrapMetadata.metadataVersion());
} else { } else {
return recordsForNonEmptyLog(activationMessageConsumer, transactionStartOffset, zkMigrationEnabled, return recordsForNonEmptyLog(activationMessageConsumer, transactionStartOffset,
featureControl, featureControl.metadataVersion()); zkMigrationState, curMetadataVersion);
} }
} }
} }

View File

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

View File

@ -397,16 +397,6 @@ public class FeatureControlManager {
return new FinalizedControllerFeatures(features, epoch); 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) { public void replay(FeatureLevelRecord record) {
VersionRange range = quorumFeatures.localSupportedFeature(record.name()); VersionRange range = quorumFeatures.localSupportedFeature(record.name());
if (!range.contains(record.featureLevel())) { if (!range.contains(record.featureLevel())) {

View File

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

View File

@ -97,7 +97,6 @@ public class PartitionChangeBuilder {
private List<Integer> uncleanShutdownReplicas; private List<Integer> uncleanShutdownReplicas;
private Election election = Election.ONLINE; private Election election = Election.ONLINE;
private LeaderRecoveryState targetLeaderRecoveryState; private LeaderRecoveryState targetLeaderRecoveryState;
private boolean zkMigrationEnabled;
private boolean eligibleLeaderReplicasEnabled; private boolean eligibleLeaderReplicasEnabled;
private DefaultDirProvider defaultDirProvider; private DefaultDirProvider defaultDirProvider;
@ -118,7 +117,6 @@ public class PartitionChangeBuilder {
this.partitionId = partitionId; this.partitionId = partitionId;
this.isAcceptableLeader = isAcceptableLeader; this.isAcceptableLeader = isAcceptableLeader;
this.metadataVersion = metadataVersion; this.metadataVersion = metadataVersion;
this.zkMigrationEnabled = false;
this.eligibleLeaderReplicasEnabled = false; this.eligibleLeaderReplicasEnabled = false;
this.minISR = minISR; this.minISR = minISR;
@ -179,11 +177,6 @@ public class PartitionChangeBuilder {
return this; return this;
} }
public PartitionChangeBuilder setZkMigrationEnabled(boolean zkMigrationEnabled) {
this.zkMigrationEnabled = zkMigrationEnabled;
return this;
}
public PartitionChangeBuilder setEligibleLeaderReplicasEnabled(boolean eligibleLeaderReplicasEnabled) { public PartitionChangeBuilder setEligibleLeaderReplicasEnabled(boolean eligibleLeaderReplicasEnabled) {
this.eligibleLeaderReplicasEnabled = eligibleLeaderReplicasEnabled; this.eligibleLeaderReplicasEnabled = eligibleLeaderReplicasEnabled;
return this; return this;
@ -392,17 +385,11 @@ public class PartitionChangeBuilder {
* the PartitionChangeRecord. * the PartitionChangeRecord.
*/ */
void triggerLeaderEpochBumpForIsrShrinkIfNeeded(PartitionChangeRecord record) { void triggerLeaderEpochBumpForIsrShrinkIfNeeded(PartitionChangeRecord record) {
if (!(metadataVersion.isLeaderEpochBumpRequiredOnIsrShrink() || zkMigrationEnabled)) { if (!metadataVersion.isLeaderEpochBumpRequiredOnIsrShrink()) {
// We only need to bump the leader epoch on an ISR shrink in two cases: // 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
// 1. In older metadata versions before 3.6, there was a bug (KAFKA-15021) in the // required that the leader epoch be bumped whenever the ISR shrank. (This was never
// broker replica manager that required that the leader epoch be bumped whenever // necessary for EXPANSIONS, only SHRINKS.)
// 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.
return; return;
} }
if (record.leader() != NO_LEADER_CHANGE) { 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.KafkaConfigSchema;
import org.apache.kafka.metadata.VersionRange; import org.apache.kafka.metadata.VersionRange;
import org.apache.kafka.metadata.bootstrap.BootstrapMetadata; 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.ReplicaPlacer;
import org.apache.kafka.metadata.placement.StripedReplicaPlacer; import org.apache.kafka.metadata.placement.StripedReplicaPlacer;
import org.apache.kafka.metadata.util.RecordRedactor; import org.apache.kafka.metadata.util.RecordRedactor;
@ -131,7 +129,6 @@ import org.apache.kafka.timeline.SnapshotRegistry;
import org.slf4j.Logger; import org.slf4j.Logger;
import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.EnumSet; 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.MICROSECONDS;
import static java.util.concurrent.TimeUnit.NANOSECONDS; 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.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 Map<String, Object> staticConfig = Collections.emptyMap();
private BootstrapMetadata bootstrapMetadata = null; private BootstrapMetadata bootstrapMetadata = null;
private int maxRecordsPerBatch = MAX_RECORDS_PER_BATCH; private int maxRecordsPerBatch = MAX_RECORDS_PER_BATCH;
private boolean zkMigrationEnabled = false;
private boolean eligibleLeaderReplicasEnabled = false; private boolean eligibleLeaderReplicasEnabled = false;
private DelegationTokenCache tokenCache; private DelegationTokenCache tokenCache;
private String tokenSecretKeyString; private String tokenSecretKeyString;
@ -341,11 +336,6 @@ public final class QuorumController implements Controller {
return this; return this;
} }
public Builder setZkMigrationEnabled(boolean zkMigrationEnabled) {
this.zkMigrationEnabled = zkMigrationEnabled;
return this;
}
public Builder setEligibleLeaderReplicasEnabled(boolean eligibleLeaderReplicasEnabled) { public Builder setEligibleLeaderReplicasEnabled(boolean eligibleLeaderReplicasEnabled) {
this.eligibleLeaderReplicasEnabled = eligibleLeaderReplicasEnabled; this.eligibleLeaderReplicasEnabled = eligibleLeaderReplicasEnabled;
return this; return this;
@ -406,7 +396,7 @@ public final class QuorumController implements Controller {
logContext = new LogContext(String.format("[QuorumController id=%d] ", nodeId)); logContext = new LogContext(String.format("[QuorumController id=%d] ", nodeId));
} }
if (controllerMetrics == null) { if (controllerMetrics == null) {
controllerMetrics = new QuorumControllerMetrics(Optional.empty(), time, zkMigrationEnabled); controllerMetrics = new QuorumControllerMetrics(Optional.empty(), time);
} }
KafkaEventQueue queue = null; KafkaEventQueue queue = null;
@ -436,7 +426,6 @@ public final class QuorumController implements Controller {
staticConfig, staticConfig,
bootstrapMetadata, bootstrapMetadata,
maxRecordsPerBatch, maxRecordsPerBatch,
zkMigrationEnabled,
tokenCache, tokenCache,
tokenSecretKeyString, tokenSecretKeyString,
delegationTokenMaxLifeMs, delegationTokenMaxLifeMs,
@ -674,10 +663,6 @@ public final class QuorumController implements Controller {
return configurationControl; return configurationControl;
} }
public ZkRecordConsumer zkRecordConsumer() {
return zkRecordConsumer;
}
<T> CompletableFuture<T> appendReadEvent( <T> CompletableFuture<T> appendReadEvent(
String name, String name,
OptionalLong deadlineNs, 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. * 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. * We use this when the event was not appended to the queue.
*/ */
DOES_NOT_UPDATE_QUEUE_TIME, 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
} }
interface ControllerWriteOperation<T> { interface ControllerWriteOperation<T> {
@ -784,24 +753,13 @@ public final class QuorumController implements Controller {
if (!isActiveController(controllerEpoch)) { if (!isActiveController(controllerEpoch)) {
throw ControllerExceptions.newWrongControllerException(latestController()); 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(); ControllerResult<T> result = op.generateRecordsAndResult();
if (result.records().isEmpty()) { if (result.records().isEmpty()) {
op.processBatchEndOffset(offsetControl.nextWriteOffset() - 1); op.processBatchEndOffset(offsetControl.nextWriteOffset() - 1);
// If the operation did not return any records, then it was actually just // 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 // 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. // from the latest in-memory state, which might contain uncommitted data.
// If the operation can complete within a transaction, let it use the OptionalLong maybeOffset = deferredEventQueue.highestPendingOffset();
// 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();
}
if (!maybeOffset.isPresent()) { if (!maybeOffset.isPresent()) {
// If the purgatory is empty, there are no pending operations and no // If the purgatory is empty, there are no pending operations and no
// uncommitted state. We can complete immediately. // 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 // Remember the latest offset and future if it is not already completed
if (!future.isDone()) { if (!future.isDone()) {
if (featureControl.inPreMigrationMode() && flags.contains(RUNS_IN_PREMIGRATION)) {
deferredUnstableEventQueue.add(resultAndOffset.offset(), this);
} else {
deferredEventQueue.add(resultAndOffset.offset(), this); deferredEventQueue.add(resultAndOffset.offset(), this);
} }
} }
}
@Override @Override
public void handleException(Throwable exception) { public void handleException(Throwable exception) {
@ -981,78 +935,6 @@ public final class QuorumController implements Controller {
return event.future(); 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> { class QuorumMetaLogListener implements RaftClient.Listener<ApiMessageAndVersion> {
@Override @Override
public void handleCommit(BatchReader<ApiMessageAndVersion> reader) { public void handleCommit(BatchReader<ApiMessageAndVersion> reader) {
@ -1077,7 +959,6 @@ public final class QuorumController implements Controller {
// items that were waiting for these offsets. // items that were waiting for these offsets.
offsetControl.handleCommitBatch(batch); offsetControl.handleCommitBatch(batch);
deferredEventQueue.completeUpTo(offsetControl.lastStableOffset()); deferredEventQueue.completeUpTo(offsetControl.lastStableOffset());
deferredUnstableEventQueue.completeUpTo(offsetControl.lastCommittedOffset());
} else { } else {
// If the controller is a standby, replay the records that were // If the controller is a standby, replay the records that were
// created by the active controller. // created by the active controller.
@ -1228,7 +1109,7 @@ public final class QuorumController implements Controller {
ControllerWriteEvent<Void> activationEvent = new ControllerWriteEvent<>( ControllerWriteEvent<Void> activationEvent = new ControllerWriteEvent<>(
"completeActivation[" + epoch + "]", "completeActivation[" + epoch + "]",
new CompleteActivationEvent(), new CompleteActivationEvent(),
EnumSet.of(DOES_NOT_UPDATE_QUEUE_TIME, RUNS_IN_PREMIGRATION) EnumSet.of(DOES_NOT_UPDATE_QUEUE_TIME)
); );
queue.prepend(activationEvent); queue.prepend(activationEvent);
} catch (Throwable e) { } catch (Throwable e) {
@ -1244,9 +1125,9 @@ public final class QuorumController implements Controller {
log::warn, log::warn,
logReplayTracker.empty(), logReplayTracker.empty(),
offsetControl.transactionStartOffset(), offsetControl.transactionStartOffset(),
zkMigrationEnabled,
bootstrapMetadata, bootstrapMetadata,
featureControl); featureControl.zkMigrationState(),
featureControl.metadataVersion());
} catch (Throwable t) { } catch (Throwable t) {
throw fatalFaultHandler.handleFault("exception while completing controller " + throw fatalFaultHandler.handleFault("exception while completing controller " +
"activation", t); "activation", t);
@ -1276,8 +1157,6 @@ public final class QuorumController implements Controller {
curClaimEpoch = -1; curClaimEpoch = -1;
deferredEventQueue.failAll(ControllerExceptions. deferredEventQueue.failAll(ControllerExceptions.
newWrongControllerException(OptionalInt.empty())); newWrongControllerException(OptionalInt.empty()));
deferredUnstableEventQueue.failAll(ControllerExceptions.
newWrongControllerException(OptionalInt.empty()));
offsetControl.deactivate(); offsetControl.deactivate();
clusterControl.deactivate(); clusterControl.deactivate();
cancelMaybeFenceReplicas(); cancelMaybeFenceReplicas();
@ -1474,7 +1353,7 @@ public final class QuorumController implements Controller {
null null
); );
}, },
EnumSet.of(DOES_NOT_UPDATE_QUEUE_TIME, RUNS_IN_PREMIGRATION) EnumSet.of(DOES_NOT_UPDATE_QUEUE_TIME)
); );
long delayNs = time.nanoseconds() + maxIdleIntervalNs.getAsLong(); long delayNs = time.nanoseconds() + maxIdleIntervalNs.getAsLong();
@ -1683,13 +1562,6 @@ public final class QuorumController implements Controller {
*/ */
private final DeferredEventQueue deferredEventQueue; 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. * 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 BootstrapMetadata bootstrapMetadata;
private final ZkRecordConsumer zkRecordConsumer;
private final boolean zkMigrationEnabled;
private final boolean eligibleLeaderReplicasEnabled; private final boolean eligibleLeaderReplicasEnabled;
/** /**
@ -1868,7 +1736,6 @@ public final class QuorumController implements Controller {
Map<String, Object> staticConfig, Map<String, Object> staticConfig,
BootstrapMetadata bootstrapMetadata, BootstrapMetadata bootstrapMetadata,
int maxRecordsPerBatch, int maxRecordsPerBatch,
boolean zkMigrationEnabled,
DelegationTokenCache tokenCache, DelegationTokenCache tokenCache,
String tokenSecretKeyString, String tokenSecretKeyString,
long delegationTokenMaxLifeMs, long delegationTokenMaxLifeMs,
@ -1888,7 +1755,6 @@ public final class QuorumController implements Controller {
this.controllerMetrics = controllerMetrics; this.controllerMetrics = controllerMetrics;
this.snapshotRegistry = new SnapshotRegistry(logContext); this.snapshotRegistry = new SnapshotRegistry(logContext);
this.deferredEventQueue = new DeferredEventQueue(logContext); this.deferredEventQueue = new DeferredEventQueue(logContext);
this.deferredUnstableEventQueue = new DeferredEventQueue(logContext);
this.offsetControl = new OffsetControlManager.Builder(). this.offsetControl = new OffsetControlManager.Builder().
setLogContext(logContext). setLogContext(logContext).
setSnapshotRegistry(snapshotRegistry). setSnapshotRegistry(snapshotRegistry).
@ -1931,7 +1797,6 @@ public final class QuorumController implements Controller {
setSessionTimeoutNs(sessionTimeoutNs). setSessionTimeoutNs(sessionTimeoutNs).
setReplicaPlacer(replicaPlacer). setReplicaPlacer(replicaPlacer).
setFeatureControlManager(featureControl). setFeatureControlManager(featureControl).
setZkMigrationEnabled(zkMigrationEnabled).
setBrokerUncleanShutdownHandler(this::handleUncleanBrokerShutdown). setBrokerUncleanShutdownHandler(this::handleUncleanBrokerShutdown).
setInterBrokerListenerName(interBrokerListenerName). setInterBrokerListenerName(interBrokerListenerName).
build(); build();
@ -1978,15 +1843,13 @@ public final class QuorumController implements Controller {
this.maxRecordsPerBatch = maxRecordsPerBatch; this.maxRecordsPerBatch = maxRecordsPerBatch;
this.metaLogListener = new QuorumMetaLogListener(); this.metaLogListener = new QuorumMetaLogListener();
this.curClaimEpoch = -1; this.curClaimEpoch = -1;
this.zkRecordConsumer = new MigrationRecordConsumer();
this.zkMigrationEnabled = zkMigrationEnabled;
this.recordRedactor = new RecordRedactor(configSchema); this.recordRedactor = new RecordRedactor(configSchema);
this.eligibleLeaderReplicasEnabled = eligibleLeaderReplicasEnabled; this.eligibleLeaderReplicasEnabled = eligibleLeaderReplicasEnabled;
this.uncleanLeaderElectionCheckIntervalNs = this.uncleanLeaderElectionCheckIntervalNs =
TimeUnit.MILLISECONDS.toNanos(uncleanLeaderElectionCheckIntervalMs); TimeUnit.MILLISECONDS.toNanos(uncleanLeaderElectionCheckIntervalMs);
log.info("Creating new QuorumController with clusterId {}.{}{}", log.info("Creating new QuorumController with clusterId {}.{}",
clusterId, zkMigrationEnabled ? " ZK migration mode is enabled." : "", clusterId,
eligibleLeaderReplicasEnabled ? " Eligible leader replicas enabled." : ""); eligibleLeaderReplicasEnabled ? " Eligible leader replicas enabled." : "");
this.raftClient.register(metaLogListener); this.raftClient.register(metaLogListener);
@ -2061,7 +1924,8 @@ public final class QuorumController implements Controller {
int brokerId int brokerId
) { ) {
return appendWriteEvent("unregisterBroker", context.deadlineNs(), return appendWriteEvent("unregisterBroker", context.deadlineNs(),
() -> replicationControl.unregisterBroker(brokerId), EnumSet.of(RUNS_IN_PREMIGRATION)); () -> replicationControl.unregisterBroker(brokerId),
EnumSet.noneOf(ControllerOperationFlag.class));
} }
@Override @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)) { if (ControllerExceptions.isTimeoutException(t)) {
replicationControl.processExpiredBrokerHeartbeat(request); replicationControl.processExpiredBrokerHeartbeat(request);
controllerMetrics.incrementTimedOutHeartbeats(); controllerMetrics.incrementTimedOutHeartbeats();
@ -2262,7 +2126,7 @@ public final class QuorumController implements Controller {
rescheduleMaybeFenceStaleBrokers(); rescheduleMaybeFenceStaleBrokers();
return result; return result;
}, },
EnumSet.of(RUNS_IN_PREMIGRATION)); EnumSet.noneOf(ControllerOperationFlag.class));
} }
@Override @Override
@ -2366,7 +2230,7 @@ public final class QuorumController implements Controller {
) { ) {
return appendWriteEvent("registerController", context.deadlineNs(), return appendWriteEvent("registerController", context.deadlineNs(),
() -> clusterControl.registerController(request), () -> clusterControl.registerController(request),
EnumSet.of(RUNS_IN_PREMIGRATION)); EnumSet.noneOf(ControllerOperationFlag.class));
} }
@Override @Override

View File

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

View File

@ -45,12 +45,6 @@ public class QuorumControllerMetrics implements AutoCloseable {
"ControllerEventManager", "EventQueueTimeMs"); "ControllerEventManager", "EventQueueTimeMs");
private static final MetricName EVENT_QUEUE_PROCESSING_TIME_MS = getMetricName( private static final MetricName EVENT_QUEUE_PROCESSING_TIME_MS = getMetricName(
"ControllerEventManager", "EventQueueProcessingTimeMs"); "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( private static final MetricName LAST_APPLIED_RECORD_OFFSET = getMetricName(
"KafkaController", "LastAppliedRecordOffset"); "KafkaController", "LastAppliedRecordOffset");
private static final MetricName LAST_COMMITTED_RECORD_OFFSET = getMetricName( 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 lastAppliedRecordOffset = new AtomicLong(0);
private final AtomicLong lastCommittedRecordOffset = new AtomicLong(0); private final AtomicLong lastCommittedRecordOffset = new AtomicLong(0);
private final AtomicLong lastAppliedRecordTimestamp = 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> eventQueueTimeUpdater;
private final Consumer<Long> eventQueueProcessingTimeUpdater; 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 timedOutHeartbeats = new AtomicLong(0);
private final AtomicLong operationsStarted = new AtomicLong(0); private final AtomicLong operationsStarted = new AtomicLong(0);
@ -95,8 +86,7 @@ public class QuorumControllerMetrics implements AutoCloseable {
public QuorumControllerMetrics( public QuorumControllerMetrics(
Optional<MetricsRegistry> registry, Optional<MetricsRegistry> registry,
Time time, Time time
boolean zkMigrationEnabled
) { ) {
this.registry = registry; this.registry = registry;
this.active = false; this.active = false;
@ -156,23 +146,6 @@ public class QuorumControllerMetrics implements AutoCloseable {
return newActiveControllers(); 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) { public void setActive(boolean active) {
@ -191,14 +164,6 @@ public class QuorumControllerMetrics implements AutoCloseable {
eventQueueProcessingTimeUpdater.accept(durationMs); eventQueueProcessingTimeUpdater.accept(durationMs);
} }
public void updateZkWriteSnapshotTimeMs(long durationMs) {
zkWriteSnapshotTimeHandler.accept(durationMs);
}
public void updateZkWriteDeltaTimeMs(long durationMs) {
zkWriteDeltaTimeHandler.accept(durationMs);
}
public void setLastAppliedRecordOffset(long offset) { public void setLastAppliedRecordOffset(long offset) {
lastAppliedRecordOffset.set(offset); lastAppliedRecordOffset.set(offset);
} }
@ -223,14 +188,6 @@ public class QuorumControllerMetrics implements AutoCloseable {
return lastAppliedRecordTimestamp.get(); return lastAppliedRecordTimestamp.get();
} }
public void updateDualWriteOffset(long offset) {
dualWriteOffset.set(offset);
}
public long dualWriteOffset() {
return dualWriteOffset.get();
}
public void incrementTimedOutHeartbeats() { public void incrementTimedOutHeartbeats() {
timedOutHeartbeats.incrementAndGet(); timedOutHeartbeats.incrementAndGet();
} }
@ -276,10 +233,7 @@ public class QuorumControllerMetrics implements AutoCloseable {
TIMED_OUT_BROKER_HEARTBEAT_COUNT, TIMED_OUT_BROKER_HEARTBEAT_COUNT,
EVENT_QUEUE_OPERATIONS_STARTED_COUNT, EVENT_QUEUE_OPERATIONS_STARTED_COUNT,
EVENT_QUEUE_OPERATIONS_TIMED_OUT_COUNT, EVENT_QUEUE_OPERATIONS_TIMED_OUT_COUNT,
NEW_ACTIVE_CONTROLLERS_COUNT, NEW_ACTIVE_CONTROLLERS_COUNT
ZK_WRITE_BEHIND_LAG,
ZK_WRITE_SNAPSHOT_TIME_MS,
ZK_WRITE_DELTA_TIME_MS
).forEach(r::removeMetric)); ).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; package org.apache.kafka.controller;
import org.apache.kafka.common.metadata.ZkMigrationStateRecord;
import org.apache.kafka.metadata.bootstrap.BootstrapMetadata; import org.apache.kafka.metadata.bootstrap.BootstrapMetadata;
import org.apache.kafka.metadata.migration.ZkMigrationState; import org.apache.kafka.metadata.migration.ZkMigrationState;
import org.apache.kafka.server.common.MetadataVersion; import org.apache.kafka.server.common.MetadataVersion;
import org.junit.jupiter.api.Test; 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.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertThrows; 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. " + 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), "Appending 1 bootstrap record(s) at metadata.version 3.0-IV1 from bootstrap source 'test'.", logMsg),
-1L, -1L,
false,
BootstrapMetadata.fromVersion(MetadataVersion.MINIMUM_BOOTSTRAP_VERSION, "test"), BootstrapMetadata.fromVersion(MetadataVersion.MINIMUM_BOOTSTRAP_VERSION, "test"),
MetadataVersion.MINIMUM_KRAFT_VERSION MetadataVersion.MINIMUM_KRAFT_VERSION
); );
@ -57,7 +53,6 @@ public class ActivationRecordsGeneratorTest {
"Appending 1 bootstrap record(s) at metadata.version 3.4-IV0 from bootstrap " + "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), "source 'test'. Setting the ZK migration state to NONE since this is a de-novo KRaft cluster.", logMsg),
-1L, -1L,
false,
BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_4_IV0, "test"), BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_4_IV0, "test"),
MetadataVersion.IBP_3_4_IV0 MetadataVersion.IBP_3_4_IV0
); );
@ -65,50 +60,11 @@ public class ActivationRecordsGeneratorTest {
assertEquals(2, result.records().size()); 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( result = ActivationRecordsGenerator.recordsForEmptyLog(
logMsg -> assertEquals("Performing controller activation. The metadata log appears to be empty. " + 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 " + "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), "source 'test'. Setting the ZK migration state to NONE since this is a de-novo KRaft cluster.", logMsg),
-1L, -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"), BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_6_IV1, "test"),
MetadataVersion.IBP_3_6_IV1 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 " + "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), "since this is a de-novo KRaft cluster.", logMsg),
0L, 0L,
false,
BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_6_IV1, "test"), BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_6_IV1, "test"),
MetadataVersion.IBP_3_6_IV1 MetadataVersion.IBP_3_6_IV1
); );
assertFalse(result.isAtomic()); assertFalse(result.isAtomic());
assertEquals(5, result.records().size()); 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 @Test
@ -174,8 +92,7 @@ public class ActivationRecordsGeneratorTest {
logMsg -> assertEquals("Performing controller activation. No metadata.version feature level " + 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), "record was found in the log. Treating the log as version 3.0-IV1.", logMsg),
-1L, -1L,
false, ZkMigrationState.NONE,
buildFeatureControl(MetadataVersion.MINIMUM_KRAFT_VERSION, Optional.empty()),
MetadataVersion.MINIMUM_KRAFT_VERSION MetadataVersion.MINIMUM_KRAFT_VERSION
); );
assertTrue(result.isAtomic()); assertTrue(result.isAtomic());
@ -184,8 +101,7 @@ public class ActivationRecordsGeneratorTest {
result = ActivationRecordsGenerator.recordsForNonEmptyLog( result = ActivationRecordsGenerator.recordsForNonEmptyLog(
logMsg -> assertEquals("Performing controller activation.", logMsg), logMsg -> assertEquals("Performing controller activation.", logMsg),
-1L, -1L,
false, ZkMigrationState.NONE,
buildFeatureControl(MetadataVersion.IBP_3_3_IV0, Optional.empty()),
MetadataVersion.IBP_3_3_IV0 MetadataVersion.IBP_3_3_IV0
); );
assertTrue(result.isAtomic()); assertTrue(result.isAtomic());
@ -195,8 +111,7 @@ public class ActivationRecordsGeneratorTest {
logMsg -> assertEquals("Performing controller activation. Loaded ZK migration state of NONE. " logMsg -> assertEquals("Performing controller activation. Loaded ZK migration state of NONE. "
+ "This is expected because this is a de-novo KRaft cluster.", logMsg), + "This is expected because this is a de-novo KRaft cluster.", logMsg),
-1L, -1L,
false, ZkMigrationState.NONE,
buildFeatureControl(MetadataVersion.IBP_3_4_IV0, Optional.empty()),
MetadataVersion.IBP_3_4_IV0 MetadataVersion.IBP_3_4_IV0
); );
assertTrue(result.isAtomic()); assertTrue(result.isAtomic());
@ -207,8 +122,7 @@ public class ActivationRecordsGeneratorTest {
"transaction at offset 42. Loaded ZK migration state of NONE. " + "transaction at offset 42. Loaded ZK migration state of NONE. " +
"This is expected because this is a de-novo KRaft cluster.", logMsg), "This is expected because this is a de-novo KRaft cluster.", logMsg),
42L, 42L,
false, ZkMigrationState.NONE,
buildFeatureControl(MetadataVersion.IBP_3_6_IV1, Optional.empty()),
MetadataVersion.IBP_3_6_IV1 MetadataVersion.IBP_3_6_IV1
); );
assertTrue(result.isAtomic()); assertTrue(result.isAtomic());
@ -221,8 +135,7 @@ public class ActivationRecordsGeneratorTest {
ActivationRecordsGenerator.recordsForNonEmptyLog( ActivationRecordsGenerator.recordsForNonEmptyLog(
logMsg -> fail(), logMsg -> fail(),
42L, 42L,
false, ZkMigrationState.NONE,
buildFeatureControl(MetadataVersion.IBP_3_6_IV0, Optional.empty()),
MetadataVersion.IBP_3_6_IV0 MetadataVersion.IBP_3_6_IV0
)).getMessage() )).getMessage()
); );
@ -230,97 +143,35 @@ public class ActivationRecordsGeneratorTest {
@Test @Test
public void testActivationMessageForNonEmptyLogWithMigrations() { public void testActivationMessageForNonEmptyLogWithMigrations() {
ControllerResult<Void> result;
assertEquals( 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, () -> assertThrows(RuntimeException.class, () ->
ActivationRecordsGenerator.recordsForNonEmptyLog( ActivationRecordsGenerator.recordsForNonEmptyLog(
logMsg -> fail(), logMsg -> fail(),
-1L, -1L,
true, ZkMigrationState.MIGRATION,
buildFeatureControl(MetadataVersion.IBP_3_3_IV0, Optional.empty()),
MetadataVersion.IBP_3_3_IV0 MetadataVersion.IBP_3_3_IV0
)).getMessage() )).getMessage()
); );
assertEquals( 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, () -> assertThrows(RuntimeException.class, () ->
ActivationRecordsGenerator.recordsForNonEmptyLog( ActivationRecordsGenerator.recordsForNonEmptyLog(
logMsg -> fail(), logMsg -> fail(),
-1L, -1L,
true, ZkMigrationState.MIGRATION,
buildFeatureControl(MetadataVersion.IBP_3_4_IV0, Optional.empty()), MetadataVersion.IBP_3_9_IV0
MetadataVersion.IBP_3_4_IV0
) )
).getMessage() ).getMessage()
); );
result = ActivationRecordsGenerator.recordsForNonEmptyLog( ControllerResult<Void> result;
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());
result = ActivationRecordsGenerator.recordsForNonEmptyLog( result = ActivationRecordsGenerator.recordsForNonEmptyLog(
logMsg -> assertEquals("Performing controller activation. Loaded ZK migration state of " + logMsg -> assertEquals("Performing controller activation. Loaded ZK migration state of " +
"POST_MIGRATION.", logMsg), "POST_MIGRATION.", logMsg),
-1L, -1L,
false, ZkMigrationState.POST_MIGRATION,
buildFeatureControl(MetadataVersion.IBP_3_4_IV0, Optional.of(ZkMigrationState.POST_MIGRATION)),
MetadataVersion.IBP_3_4_IV0 MetadataVersion.IBP_3_4_IV0
); );
assertTrue(result.isAtomic()); assertTrue(result.isAtomic());
@ -330,32 +181,7 @@ public class ActivationRecordsGeneratorTest {
logMsg -> assertEquals("Performing controller activation. Aborting in-progress metadata " + logMsg -> assertEquals("Performing controller activation. Aborting in-progress metadata " +
"transaction at offset 42. Loaded ZK migration state of POST_MIGRATION.", logMsg), "transaction at offset 42. Loaded ZK migration state of POST_MIGRATION.", logMsg),
42L, 42L,
false, ZkMigrationState.POST_MIGRATION,
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)),
MetadataVersion.IBP_3_6_IV1 MetadataVersion.IBP_3_6_IV1
); );
assertTrue(result.isAtomic()); assertTrue(result.isAtomic());

View File

@ -852,33 +852,4 @@ public class ClusterControlManagerTest {
clusterControl.brokerRegistrations().get(1).epoch()); 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.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.params.provider.Arguments.arguments;
@Timeout(value = 40) @Timeout(value = 40)
@ -338,22 +337,6 @@ public class PartitionChangeBuilderTest {
1); 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. * Test that expanding the ISR doesn't increase the leader epoch.
*/ */
@ -368,22 +351,6 @@ public class PartitionChangeBuilderTest {
NO_LEADER_CHANGE); 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 * Test that changing the replica set such that not all the old replicas remain
* always results in a leader epoch increase. * 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 @ParameterizedTest
@MethodSource("leaderRecoveryAndZkMigrationParams") @ValueSource(booleans = {true, false})
public void testChangeInLeadershipDoesNotChangeRecoveryState(boolean isLeaderRecoverySupported, boolean zkMigrationsEnabled) { public void testChangeInLeadershipDoesNotChangeRecoveryState(boolean isLeaderRecoverySupported) {
final byte noChange = (byte) -1; final byte noChange = (byte) -1;
int leaderId = 1; int leaderId = 1;
LeaderRecoveryState recoveryState = LeaderRecoveryState.RECOVERING; LeaderRecoveryState recoveryState = LeaderRecoveryState.RECOVERING;
@ -671,7 +629,6 @@ public class PartitionChangeBuilderTest {
metadataVersion, metadataVersion,
2 2
); );
offlineBuilder.setZkMigrationEnabled(zkMigrationsEnabled);
// Set the target ISR to empty to indicate that the last leader is offline // Set the target ISR to empty to indicate that the last leader is offline
offlineBuilder.setTargetIsrWithBrokerStates(Collections.emptyList()); offlineBuilder.setTargetIsrWithBrokerStates(Collections.emptyList());
@ -698,7 +655,6 @@ public class PartitionChangeBuilderTest {
metadataVersion, metadataVersion,
2 2
); );
onlineBuilder.setZkMigrationEnabled(zkMigrationsEnabled);
// The only broker in the ISR is elected leader and stays in the recovering // The only broker in the ISR is elected leader and stays in the recovering
changeRecord = (PartitionChangeRecord) onlineBuilder.build().get().message(); changeRecord = (PartitionChangeRecord) onlineBuilder.build().get().message();
@ -712,8 +668,8 @@ public class PartitionChangeBuilderTest {
} }
@ParameterizedTest @ParameterizedTest
@MethodSource("leaderRecoveryAndZkMigrationParams") @ValueSource(booleans = {true, false})
void testUncleanSetsLeaderRecoveringState(boolean isLeaderRecoverySupported, boolean zkMigrationsEnabled) { void testUncleanSetsLeaderRecoveringState(boolean isLeaderRecoverySupported) {
final byte noChange = (byte) -1; final byte noChange = (byte) -1;
int leaderId = 1; int leaderId = 1;
PartitionRegistration registration = new PartitionRegistration.Builder(). PartitionRegistration registration = new PartitionRegistration.Builder().
@ -741,7 +697,6 @@ public class PartitionChangeBuilderTest {
metadataVersion, metadataVersion,
2 2
).setElection(Election.UNCLEAN); ).setElection(Election.UNCLEAN);
onlineBuilder.setZkMigrationEnabled(zkMigrationsEnabled);
// The partition should stay as recovering // The partition should stay as recovering
PartitionChangeRecord changeRecord = (PartitionChangeRecord) onlineBuilder PartitionChangeRecord changeRecord = (PartitionChangeRecord) onlineBuilder
.build() .build()

View File

@ -57,7 +57,7 @@ public class QuorumControllerMetricsIntegrationTest {
final AtomicBoolean closed = new AtomicBoolean(false); final AtomicBoolean closed = new AtomicBoolean(false);
MockControllerMetrics() { MockControllerMetrics() {
super(Optional.empty(), Time.SYSTEM, true); super(Optional.empty(), Time.SYSTEM);
} }
@Override @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.authorizer.StandardAuthorizer;
import org.apache.kafka.metadata.bootstrap.BootstrapMetadata; import org.apache.kafka.metadata.bootstrap.BootstrapMetadata;
import org.apache.kafka.metadata.migration.ZkMigrationState; 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.metadata.util.BatchFileWriter;
import org.apache.kafka.metalog.LocalLogManager; import org.apache.kafka.metalog.LocalLogManager;
import org.apache.kafka.metalog.LocalLogManagerTestEnv; 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.KRaftVersion;
import org.apache.kafka.server.common.MetadataVersion; import org.apache.kafka.server.common.MetadataVersion;
import org.apache.kafka.server.common.TopicIdPartition; 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.FileRawSnapshotReader;
import org.apache.kafka.snapshot.Snapshots; import org.apache.kafka.snapshot.Snapshots;
import org.apache.kafka.test.TestUtils; 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.api.Timeout;
import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.CsvSource; 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.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.io.File; import java.io.File;
import java.nio.file.Path; import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; 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.ANONYMOUS_CONTEXT;
import static org.apache.kafka.controller.ControllerRequestContextUtil.anonymousContextFor; import static org.apache.kafka.controller.ControllerRequestContextUtil.anonymousContextFor;
import static org.apache.kafka.controller.QuorumControllerIntegrationTestUtils.brokerFeatures; 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.pause;
import static org.apache.kafka.controller.QuorumControllerIntegrationTestUtils.registerBrokersAndUnfence; import static org.apache.kafka.controller.QuorumControllerIntegrationTestUtils.registerBrokersAndUnfence;
import static org.apache.kafka.controller.QuorumControllerIntegrationTestUtils.sendBrokerHeartbeatToUnfenceBrokers; import static org.apache.kafka.controller.QuorumControllerIntegrationTestUtils.sendBrokerHeartbeatToUnfenceBrokers;
@ -1382,75 +1376,9 @@ public class QuorumControllerTest {
appender)).getMessage()); 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( FeatureControlManager getActivationRecords(
MetadataVersion metadataVersion, MetadataVersion metadataVersion,
Optional<ZkMigrationState> stateInLog, Optional<ZkMigrationState> stateInLog
boolean zkMigrationEnabled
) { ) {
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
FeatureControlManager featureControlManager = new FeatureControlManager.Builder() FeatureControlManager featureControlManager = new FeatureControlManager.Builder()
@ -1458,16 +1386,13 @@ public class QuorumControllerTest {
.setMetadataVersion(metadataVersion) .setMetadataVersion(metadataVersion)
.build(); .build();
stateInLog.ifPresent(zkMigrationState ->
featureControlManager.replay((ZkMigrationStateRecord) zkMigrationState.toRecord().message()));
ControllerResult<Void> result = ActivationRecordsGenerator.generate( ControllerResult<Void> result = ActivationRecordsGenerator.generate(
msg -> { }, msg -> { },
!stateInLog.isPresent(), !stateInLog.isPresent(),
-1L, -1L,
zkMigrationEnabled,
BootstrapMetadata.fromVersion(metadataVersion, "test"), BootstrapMetadata.fromVersion(metadataVersion, "test"),
featureControlManager); stateInLog.orElseGet(() -> ZkMigrationState.NONE),
metadataVersion);
RecordTestUtils.replayAll(featureControlManager, result.records()); RecordTestUtils.replayAll(featureControlManager, result.records());
return featureControlManager; return featureControlManager;
} }
@ -1476,21 +1401,11 @@ public class QuorumControllerTest {
public void testActivationRecords33() { public void testActivationRecords33() {
FeatureControlManager featureControl; FeatureControlManager featureControl;
assertEquals( featureControl = getActivationRecords(MetadataVersion.IBP_3_3_IV0, Optional.empty());
"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);
assertEquals(MetadataVersion.IBP_3_3_IV0, featureControl.metadataVersion()); assertEquals(MetadataVersion.IBP_3_3_IV0, featureControl.metadataVersion());
assertEquals(ZkMigrationState.NONE, featureControl.zkMigrationState()); assertEquals(ZkMigrationState.NONE, featureControl.zkMigrationState());
assertEquals( featureControl = getActivationRecords(MetadataVersion.IBP_3_3_IV0, Optional.of(ZkMigrationState.NONE));
"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);
assertEquals(MetadataVersion.IBP_3_3_IV0, featureControl.metadataVersion()); assertEquals(MetadataVersion.IBP_3_3_IV0, featureControl.metadataVersion());
assertEquals(ZkMigrationState.NONE, featureControl.zkMigrationState()); assertEquals(ZkMigrationState.NONE, featureControl.zkMigrationState());
} }
@ -1499,67 +1414,32 @@ public class QuorumControllerTest {
public void testActivationRecords34() { public void testActivationRecords34() {
FeatureControlManager featureControl; FeatureControlManager featureControl;
featureControl = getActivationRecords(MetadataVersion.IBP_3_4_IV0, Optional.empty(), true); featureControl = getActivationRecords(MetadataVersion.IBP_3_4_IV0, Optional.empty());
assertEquals(MetadataVersion.IBP_3_4_IV0, featureControl.metadataVersion());
assertEquals(ZkMigrationState.PRE_MIGRATION, featureControl.zkMigrationState());
featureControl = getActivationRecords(MetadataVersion.IBP_3_4_IV0, Optional.empty(), false);
assertEquals(MetadataVersion.IBP_3_4_IV0, featureControl.metadataVersion()); assertEquals(MetadataVersion.IBP_3_4_IV0, featureControl.metadataVersion());
assertEquals(ZkMigrationState.NONE, featureControl.zkMigrationState()); assertEquals(ZkMigrationState.NONE, featureControl.zkMigrationState());
assertEquals( featureControl = getActivationRecords(MetadataVersion.IBP_3_4_IV0, Optional.of(ZkMigrationState.NONE));
"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);
assertEquals(MetadataVersion.IBP_3_4_IV0, featureControl.metadataVersion()); assertEquals(MetadataVersion.IBP_3_4_IV0, featureControl.metadataVersion());
assertEquals(ZkMigrationState.NONE, featureControl.zkMigrationState()); 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 @Test
public void testActivationRecordsNonEmptyLog() { public void testActivationRecordsNonEmptyLog() {
FeatureControlManager featureControl = getActivationRecords( FeatureControlManager featureControl = getActivationRecords(
MetadataVersion.IBP_3_4_IV0, Optional.empty(), true); MetadataVersion.IBP_3_9_IV0, Optional.empty());
assertEquals(MetadataVersion.IBP_3_4_IV0, featureControl.metadataVersion()); assertEquals(MetadataVersion.IBP_3_9_IV0, featureControl.metadataVersion());
assertEquals(ZkMigrationState.PRE_MIGRATION, featureControl.zkMigrationState()); assertEquals(ZkMigrationState.NONE, featureControl.zkMigrationState());
} }
@ParameterizedTest @Test
@ValueSource(booleans = {true, false}) public void testActivationRecordsPartialBootstrap() {
public void testActivationRecordsPartialBootstrap(boolean zkMigrationEnabled) {
FeatureControlManager featureControlManager = new FeatureControlManager.Builder()
.setSnapshotRegistry(new SnapshotRegistry(new LogContext()))
.setMetadataVersion(MetadataVersion.IBP_3_6_IV1)
.build();
ControllerResult<Void> result = ActivationRecordsGenerator.generate( ControllerResult<Void> result = ActivationRecordsGenerator.generate(
logMsg -> { }, logMsg -> { },
true, true,
0L, 0L,
zkMigrationEnabled,
BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_6_IV1, "test"), BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_6_IV1, "test"),
featureControlManager); ZkMigrationState.NONE,
MetadataVersion.IBP_3_6_IV1);
assertFalse(result.isAtomic()); assertFalse(result.isAtomic());
assertTrue(RecordTestUtils.recordAtIndexAs( assertTrue(RecordTestUtils.recordAtIndexAs(
AbortTransactionRecord.class, result.records(), 0).isPresent()); AbortTransactionRecord.class, result.records(), 0).isPresent());
@ -1569,27 +1449,6 @@ public class QuorumControllerTest {
EndTransactionRecord.class, result.records(), result.records().size() - 1).isPresent()); 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. * Tests all intermediate images lead to the same final image for each image & delta type.
* @param fromRecords * @param fromRecords
@ -1618,12 +1477,6 @@ public class QuorumControllerTest {
@Test @Test
public void testActivationRecordsPartialTransaction() { 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 offsetControlManager = new OffsetControlManager.Builder().build();
offsetControlManager.replay(new BeginTransactionRecord(), 10); offsetControlManager.replay(new BeginTransactionRecord(), 10);
offsetControlManager.handleCommitBatch(Batch.data(20, 1, 1L, 0, offsetControlManager.handleCommitBatch(Batch.data(20, 1, 1L, 0,
@ -1633,9 +1486,9 @@ public class QuorumControllerTest {
logMsg -> { }, logMsg -> { },
false, false,
offsetControlManager.transactionStartOffset(), offsetControlManager.transactionStartOffset(),
false,
BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_6_IV1, "test"), BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_6_IV1, "test"),
featureControlManager); ZkMigrationState.NONE,
MetadataVersion.IBP_3_6_IV1);
assertTrue(result.isAtomic()); assertTrue(result.isAtomic());
offsetControlManager.replay( offsetControlManager.replay(
@ -1647,12 +1500,6 @@ public class QuorumControllerTest {
@Test @Test
public void testActivationRecordsPartialTransactionNoSupport() { 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 offsetControlManager = new OffsetControlManager.Builder().build();
offsetControlManager.replay(new BeginTransactionRecord(), 10); offsetControlManager.replay(new BeginTransactionRecord(), 10);
offsetControlManager.handleCommitBatch(Batch.data(20, 1, 1L, 0, offsetControlManager.handleCommitBatch(Batch.data(20, 1, 1L, 0,
@ -1663,129 +1510,9 @@ public class QuorumControllerTest {
msg -> { }, msg -> { },
false, false,
offsetControlManager.transactionStartOffset(), offsetControlManager.transactionStartOffset(),
false,
BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_6_IV0, "test"), 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 com.yammer.metrics.core.MetricsRegistry;
import org.junit.jupiter.api.Test; 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.Arrays;
import java.util.Collections; import java.util.Collections;
@ -36,16 +34,14 @@ import java.util.Optional;
import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertEquals;
public class QuorumControllerMetricsTest { public class QuorumControllerMetricsTest {
@ParameterizedTest @Test
@ValueSource(booleans = {false, true}) public void testMetricNames() {
public void testMetricNames(boolean inMigration) {
MetricsRegistry registry = new MetricsRegistry(); MetricsRegistry registry = new MetricsRegistry();
MockTime time = new MockTime(); MockTime time = new MockTime();
try { try {
try (QuorumControllerMetrics metrics = new QuorumControllerMetrics( try (QuorumControllerMetrics metrics = new QuorumControllerMetrics(
Optional.of(registry), Optional.of(registry),
time, time)) {
inMigration)) {
HashSet<String> expected = new HashSet<>(Arrays.asList( HashSet<String> expected = new HashSet<>(Arrays.asList(
"kafka.controller:type=ControllerEventManager,name=EventQueueProcessingTimeMs", "kafka.controller:type=ControllerEventManager,name=EventQueueProcessingTimeMs",
"kafka.controller:type=ControllerEventManager,name=EventQueueTimeMs", "kafka.controller:type=ControllerEventManager,name=EventQueueTimeMs",
@ -59,11 +55,6 @@ public class QuorumControllerMetricsTest {
"kafka.controller:type=KafkaController,name=NewActiveControllersCount", "kafka.controller:type=KafkaController,name=NewActiveControllersCount",
"kafka.controller:type=KafkaController,name=TimedOutBrokerHeartbeatCount" "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", expected);
} }
ControllerMetricsTestUtils.assertMetricsForTypeEqual(registry, "kafka.controller", ControllerMetricsTestUtils.assertMetricsForTypeEqual(registry, "kafka.controller",
@ -77,7 +68,7 @@ public class QuorumControllerMetricsTest {
public void testUpdateEventQueueTime() { public void testUpdateEventQueueTime() {
MetricsRegistry registry = new MetricsRegistry(); MetricsRegistry registry = new MetricsRegistry();
MockTime time = new MockTime(); 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); metrics.updateEventQueueTime(1000);
assertMetricHistogram(registry, metricName("ControllerEventManager", "EventQueueTimeMs"), 1, 1000); assertMetricHistogram(registry, metricName("ControllerEventManager", "EventQueueTimeMs"), 1, 1000);
} finally { } finally {
@ -89,7 +80,7 @@ public class QuorumControllerMetricsTest {
public void testUpdateEventQueueProcessingTime() { public void testUpdateEventQueueProcessingTime() {
MetricsRegistry registry = new MetricsRegistry(); MetricsRegistry registry = new MetricsRegistry();
MockTime time = new MockTime(); 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); metrics.updateEventQueueProcessingTime(1000);
assertMetricHistogram(registry, metricName("ControllerEventManager", "EventQueueProcessingTimeMs"), 1, 1000); assertMetricHistogram(registry, metricName("ControllerEventManager", "EventQueueProcessingTimeMs"), 1, 1000);
} finally { } finally {
@ -102,11 +93,10 @@ public class QuorumControllerMetricsTest {
MetricsRegistry registry = new MetricsRegistry(); MetricsRegistry registry = new MetricsRegistry();
MockTime time = new MockTime(); MockTime time = new MockTime();
time.sleep(1000); 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.setLastAppliedRecordOffset(100);
metrics.setLastAppliedRecordTimestamp(500); metrics.setLastAppliedRecordTimestamp(500);
metrics.setLastCommittedRecordOffset(50); metrics.setLastCommittedRecordOffset(50);
metrics.updateDualWriteOffset(40L);
metrics.setActive(true); metrics.setActive(true);
for (int i = 0; i < 2; i++) { for (int i = 0; i < 2; i++) {
metrics.incrementTimedOutHeartbeats(); metrics.incrementTimedOutHeartbeats();
@ -145,12 +135,6 @@ public class QuorumControllerMetricsTest {
.get(metricName("KafkaController", "LastCommittedRecordOffset")); .get(metricName("KafkaController", "LastCommittedRecordOffset"));
assertEquals(50, lastCommittedRecordOffset.value()); assertEquals(50, lastCommittedRecordOffset.value());
@SuppressWarnings("unchecked")
Gauge<Long> zkWriteBehindLag = (Gauge<Long>) registry
.allMetrics()
.get(metricName("KafkaController", "ZkWriteBehindLag"));
assertEquals(10L, zkWriteBehindLag.value());
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
Gauge<Long> timedOutBrokerHeartbeats = (Gauge<Long>) registry Gauge<Long> timedOutBrokerHeartbeats = (Gauge<Long>) registry
.allMetrics() .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) { private static void assertMetricHistogram(MetricsRegistry registry, MetricName metricName, long count, double sum) {
Histogram histogram = (Histogram) registry.allMetrics().get(metricName); 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;
}
}