mirror of https://github.com/apache/kafka.git
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:
parent
8b5d755bf6
commit
85bfdf4127
|
@ -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)
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -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()
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -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")
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -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()
|
||||||
}
|
}
|
||||||
|
|
|
@ -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),
|
||||||
|
|
|
@ -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)
|
||||||
|
|
|
@ -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)
|
||||||
|
|
||||||
|
|
|
@ -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
|
|
||||||
}
|
|
|
@ -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)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -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 "<default>" 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))
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -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)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
|
@ -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")
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -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]
|
||||||
|
|
|
@ -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()
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -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
|
||||||
)
|
)
|
||||||
|
|
|
@ -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))
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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();
|
||||||
}
|
}
|
||||||
|
|
|
@ -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();
|
||||||
}
|
}
|
||||||
|
|
|
@ -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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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));
|
||||||
|
|
|
@ -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())) {
|
||||||
|
|
|
@ -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,
|
||||||
|
|
|
@ -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) {
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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());
|
||||||
|
|
|
@ -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));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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);
|
|
||||||
}
|
|
|
@ -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);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -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
|
|
||||||
);
|
|
||||||
}
|
|
|
@ -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
|
|
||||||
);
|
|
||||||
|
|
||||||
}
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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);
|
|
||||||
}
|
|
|
@ -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);
|
|
||||||
}
|
|
|
@ -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));
|
|
||||||
}
|
|
||||||
});
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -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();
|
|
||||||
}
|
|
|
@ -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();
|
|
||||||
}
|
|
|
@ -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);
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -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);
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -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.
|
|
||||||
*
|
|
||||||
* UNINITIALIZED───────────────►INACTIVE◄────────────────DUAL_WRITE◄────────────────────────┐
|
|
||||||
* │ ▲ │
|
|
||||||
* │ │ │
|
|
||||||
* │ │ │
|
|
||||||
* │ │◄───────────────────────────────────────────────────────┤
|
|
||||||
* │ │ │
|
|
||||||
* ▼ │ │
|
|
||||||
* WAIT_FOR_CONTROLLER_QUORUM───────────┘◄────────────────────ZK_MIGRATION────────────►KRAFT_CONTROLLER_TO_BROKER_COMM
|
|
||||||
* │ ▲ ▲ ▲
|
|
||||||
* │ │ │ │
|
|
||||||
* │ │ │ │
|
|
||||||
* │ │◄────────────────────────┤ │
|
|
||||||
* │ │ │ │
|
|
||||||
* ▼ │ │ │
|
|
||||||
* BECOME_CONTROLLER───────────────────►└────────────────────►WAIT_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;
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -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);
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -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
|
|
||||||
);
|
|
||||||
}
|
|
|
@ -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();
|
|
||||||
}
|
|
|
@ -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());
|
||||||
|
|
|
@ -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());
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -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()
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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());
|
|
||||||
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -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);
|
||||||
|
|
||||||
|
|
|
@ -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) {
|
|
||||||
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -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;
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
|
@ -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;
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
|
@ -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;
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -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;
|
|
||||||
}
|
|
||||||
}
|
|
Loading…
Reference in New Issue