From 85bfdf4127e135eb85f01be6285dca4094e8ac79 Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Thu, 3 Oct 2024 12:01:14 -0700 Subject: [PATCH] KAFKA-17613: Remove ZK migration code (#17293) Remove the controller machinery for doing ZK migration in Kafka 4.0. Reviewers: Chia-Ping Tsai , David Arthur --- .../MigrationControllerChannelContext.scala | 115 -- .../kafka/migration/MigrationPropagator.scala | 251 ---- .../migration/MigrationPropagatorBatch.scala | 85 -- .../kafka/server/ApiVersionManager.scala | 11 +- .../ControllerRegistrationManager.scala | 3 +- .../scala/kafka/server/ControllerServer.scala | 59 +- .../scala/kafka/tools/TestRaftServer.scala | 1 - .../scala/kafka/zk/ZkMigrationClient.scala | 356 ------ .../zk/migration/ZkAclMigrationClient.scala | 118 -- .../migration/ZkConfigMigrationClient.scala | 394 ------ .../ZkDelegationTokenMigrationClient.scala | 97 -- .../zk/migration/ZkTopicMigrationClient.scala | 358 ------ .../unit/kafka/network/SocketServerTest.scala | 2 +- .../BrokerRegistrationRequestTest.scala | 93 +- .../kafka/server/ControllerApisTest.scala | 1 - .../ControllerRegistrationManagerTest.scala | 1 - .../unit/kafka/server/KafkaApisTest.scala | 1 - .../KRaftMetadataRequestBenchmark.java | 1 - .../metadata/MetadataRequestBenchmark.java | 1 - .../ActivationRecordsGenerator.java | 107 +- .../controller/ClusterControlManager.java | 34 +- .../controller/FeatureControlManager.java | 10 - .../controller/OffsetControlManager.java | 2 +- .../controller/PartitionChangeBuilder.java | 23 +- .../kafka/controller/QuorumController.java | 166 +-- .../controller/ReplicationControlManager.java | 6 - .../metrics/QuorumControllerMetrics.java | 50 +- .../migration/AclMigrationClient.java | 40 - .../migration/BufferingBatchConsumer.java | 58 - .../migration/ConfigMigrationClient.java | 63 - .../DelegationTokenMigrationClient.java | 38 - .../migration/KRaftMigrationDriver.java | 1088 ----------------- .../migration/KRaftMigrationOperation.java | 23 - .../KRaftMigrationOperationConsumer.java | 23 - .../migration/KRaftMigrationZkWriter.java | 684 ----------- .../metadata/migration/LegacyPropagator.java | 37 - .../metadata/migration/MigrationClient.java | 91 -- .../MigrationClientAuthException.java | 26 - .../migration/MigrationClientException.java | 38 - .../migration/MigrationDriverState.java | 61 - .../metadata/migration/MigrationManifest.java | 129 -- .../migration/TopicMigrationClient.java | 85 -- .../metadata/migration/ZkRecordConsumer.java | 30 - .../ActivationRecordsGeneratorTest.java | 200 +-- .../controller/ClusterControlManagerTest.java | 29 - .../PartitionChangeBuilderTest.java | 53 +- ...uorumControllerMetricsIntegrationTest.java | 2 +- .../controller/QuorumControllerTest.java | 311 +---- .../metrics/QuorumControllerMetricsTest.java | 73 +- .../CapturingAclMigrationClient.java | 56 - .../CapturingConfigMigrationClient.java | 76 -- ...pturingDelegationTokenMigrationClient.java | 54 - .../migration/CapturingMigrationClient.java | 204 ---- .../CapturingTopicMigrationClient.java | 114 -- 54 files changed, 107 insertions(+), 5925 deletions(-) delete mode 100644 core/src/main/scala/kafka/migration/MigrationControllerChannelContext.scala delete mode 100644 core/src/main/scala/kafka/migration/MigrationPropagator.scala delete mode 100644 core/src/main/scala/kafka/migration/MigrationPropagatorBatch.scala delete mode 100644 core/src/main/scala/kafka/zk/ZkMigrationClient.scala delete mode 100644 core/src/main/scala/kafka/zk/migration/ZkAclMigrationClient.scala delete mode 100644 core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala delete mode 100644 core/src/main/scala/kafka/zk/migration/ZkDelegationTokenMigrationClient.scala delete mode 100644 core/src/main/scala/kafka/zk/migration/ZkTopicMigrationClient.scala delete mode 100644 metadata/src/main/java/org/apache/kafka/metadata/migration/AclMigrationClient.java delete mode 100644 metadata/src/main/java/org/apache/kafka/metadata/migration/BufferingBatchConsumer.java delete mode 100644 metadata/src/main/java/org/apache/kafka/metadata/migration/ConfigMigrationClient.java delete mode 100644 metadata/src/main/java/org/apache/kafka/metadata/migration/DelegationTokenMigrationClient.java delete mode 100644 metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java delete mode 100644 metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationOperation.java delete mode 100644 metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationOperationConsumer.java delete mode 100644 metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationZkWriter.java delete mode 100644 metadata/src/main/java/org/apache/kafka/metadata/migration/LegacyPropagator.java delete mode 100644 metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationClient.java delete mode 100644 metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationClientAuthException.java delete mode 100644 metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationClientException.java delete mode 100644 metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationDriverState.java delete mode 100644 metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationManifest.java delete mode 100644 metadata/src/main/java/org/apache/kafka/metadata/migration/TopicMigrationClient.java delete mode 100644 metadata/src/main/java/org/apache/kafka/metadata/migration/ZkRecordConsumer.java delete mode 100644 metadata/src/test/java/org/apache/kafka/metadata/migration/CapturingAclMigrationClient.java delete mode 100644 metadata/src/test/java/org/apache/kafka/metadata/migration/CapturingConfigMigrationClient.java delete mode 100644 metadata/src/test/java/org/apache/kafka/metadata/migration/CapturingDelegationTokenMigrationClient.java delete mode 100644 metadata/src/test/java/org/apache/kafka/metadata/migration/CapturingMigrationClient.java delete mode 100644 metadata/src/test/java/org/apache/kafka/metadata/migration/CapturingTopicMigrationClient.java diff --git a/core/src/main/scala/kafka/migration/MigrationControllerChannelContext.scala b/core/src/main/scala/kafka/migration/MigrationControllerChannelContext.scala deleted file mode 100644 index ce592c281a0..00000000000 --- a/core/src/main/scala/kafka/migration/MigrationControllerChannelContext.scala +++ /dev/null @@ -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) - } -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/migration/MigrationPropagator.scala b/core/src/main/scala/kafka/migration/MigrationPropagator.scala deleted file mode 100644 index 8300c3e0f65..00000000000 --- a/core/src/main/scala/kafka/migration/MigrationPropagator.scala +++ /dev/null @@ -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() - } -} diff --git a/core/src/main/scala/kafka/migration/MigrationPropagatorBatch.scala b/core/src/main/scala/kafka/migration/MigrationPropagatorBatch.scala deleted file mode 100644 index 7c4952c9382..00000000000 --- a/core/src/main/scala/kafka/migration/MigrationPropagatorBatch.scala +++ /dev/null @@ -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") - } - } - } -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/ApiVersionManager.scala b/core/src/main/scala/kafka/server/ApiVersionManager.scala index e6820b638da..aa8fc4a35d6 100644 --- a/core/src/main/scala/kafka/server/ApiVersionManager.scala +++ b/core/src/main/scala/kafka/server/ApiVersionManager.scala @@ -57,7 +57,6 @@ object ApiVersionManager { supportedFeatures, metadataCache, config.unstableApiVersionsEnabled, - config.migrationEnabled, clientMetricsManager ) } @@ -71,7 +70,6 @@ object ApiVersionManager { * @param enabledApis the enabled apis, which are computed by the listener type * @param brokerFeatures the broker features * @param enableUnstableLastVersion whether to enable unstable last version, see [[KafkaConfig.unstableApiVersionsEnabled]] - * @param zkMigrationEnabled whether to enable zk migration, see [[KafkaConfig.migrationEnabled]] * @param featuresProvider a provider to the finalized features supported */ class SimpleApiVersionManager( @@ -79,14 +77,12 @@ class SimpleApiVersionManager( val enabledApis: collection.Set[ApiKeys], brokerFeatures: org.apache.kafka.common.feature.Features[SupportedVersionRange], val enableUnstableLastVersion: Boolean, - val zkMigrationEnabled: Boolean, val featuresProvider: () => FinalizedFeatures ) extends ApiVersionManager { def this( listenerType: ListenerType, enableUnstableLastVersion: Boolean, - zkMigrationEnabled: Boolean, featuresProvider: () => FinalizedFeatures ) = { this( @@ -94,7 +90,6 @@ class SimpleApiVersionManager( ApiKeys.apisForListener(listenerType).asScala, BrokerFeatures.defaultSupportedFeatures(enableUnstableLastVersion), enableUnstableLastVersion, - zkMigrationEnabled, featuresProvider ) } @@ -112,7 +107,7 @@ class SimpleApiVersionManager( setSupportedFeatures(brokerFeatures). setFinalizedFeatures(currentFeatures.finalizedFeatures()). setFinalizedFeaturesEpoch(currentFeatures.finalizedFeaturesEpoch()). - setZkMigrationEnabled(zkMigrationEnabled). + setZkMigrationEnabled(false). setAlterFeatureLevel0(alterFeatureLevel0). build() } @@ -130,7 +125,6 @@ class SimpleApiVersionManager( * @param brokerFeatures the broker features * @param metadataCache the metadata cache, used to get the finalized features and the metadata version * @param enableUnstableLastVersion whether to enable unstable last version, see [[KafkaConfig.unstableApiVersionsEnabled]] - * @param zkMigrationEnabled whether to enable zk migration, see [[KafkaConfig.migrationEnabled]] * @param clientMetricsManager the client metrics manager, helps to determine whether client telemetry is enabled */ class DefaultApiVersionManager( @@ -139,7 +133,6 @@ class DefaultApiVersionManager( brokerFeatures: BrokerFeatures, metadataCache: MetadataCache, val enableUnstableLastVersion: Boolean, - val zkMigrationEnabled: Boolean = false, val clientMetricsManager: Option[ClientMetricsManager] = None ) extends ApiVersionManager { @@ -175,7 +168,7 @@ class DefaultApiVersionManager( setSupportedFeatures(brokerFeatures.supportedFeatures). setFinalizedFeatures(finalizedFeatures.finalizedFeatures()). setFinalizedFeaturesEpoch(finalizedFeatures.finalizedFeaturesEpoch()). - setZkMigrationEnabled(zkMigrationEnabled). + setZkMigrationEnabled(false). setAlterFeatureLevel0(alterFeatureLevel0). build() } diff --git a/core/src/main/scala/kafka/server/ControllerRegistrationManager.scala b/core/src/main/scala/kafka/server/ControllerRegistrationManager.scala index 4b5843f8d31..5128e7b68a0 100644 --- a/core/src/main/scala/kafka/server/ControllerRegistrationManager.scala +++ b/core/src/main/scala/kafka/server/ControllerRegistrationManager.scala @@ -50,7 +50,6 @@ class ControllerRegistrationManager( val time: Time, val threadNamePrefix: String, val supportedFeatures: util.Map[String, VersionRange], - val zkMigrationEnabled: Boolean, val incarnationId: Uuid, val listenerInfo: ListenerInfo, val resendExponentialBackoff: ExponentialBackoff = new ExponentialBackoff(100, 2, 120000L, 0.02) @@ -227,7 +226,7 @@ class ControllerRegistrationManager( setFeatures(features). setIncarnationId(incarnationId). setListeners(listenerInfo.toControllerRegistrationRequest). - setZkMigrationReady(zkMigrationEnabled) + setZkMigrationReady(false) info(s"sendControllerRegistration: attempting to send $data") _channelManager.sendRequest(new ControllerRegistrationRequest.Builder(data), diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index 85d6f499285..c03a0b95e61 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -17,7 +17,6 @@ package kafka.server -import kafka.migration.MigrationPropagator import kafka.network.{DataPlaneAcceptor, SocketServer} import kafka.raft.KafkaRaftManager import kafka.server.QuotaFactory.QuotaManagers @@ -25,7 +24,6 @@ import kafka.server.QuotaFactory.QuotaManagers import scala.collection.immutable import kafka.server.metadata.{AclPublisher, ClientQuotaMetadataManager, DelegationTokenPublisher, DynamicClientQuotaPublisher, DynamicConfigPublisher, KRaftMetadataCache, KRaftMetadataCachePublisher, ScramPublisher} import kafka.utils.{CoreUtils, Logging} -import kafka.zk.{KafkaZkClient, ZkMigrationClient} import org.apache.kafka.common.message.ApiMessageType.ListenerType import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.scram.internals.ScramMechanism @@ -38,10 +36,9 @@ import org.apache.kafka.image.publisher.{ControllerRegistrationsPublisher, Metad import org.apache.kafka.metadata.{KafkaConfigSchema, ListenerInfo} import org.apache.kafka.metadata.authorizer.ClusterMetadataAuthorizer import org.apache.kafka.metadata.bootstrap.BootstrapMetadata -import org.apache.kafka.metadata.migration.{KRaftMigrationDriver, LegacyPropagator} import org.apache.kafka.metadata.publisher.FeaturesPublisher import org.apache.kafka.raft.QuorumConfig -import org.apache.kafka.security.{CredentialProvider, PasswordEncoder} +import org.apache.kafka.security.CredentialProvider import org.apache.kafka.server.NodeToControllerChannelManager import org.apache.kafka.server.authorizer.Authorizer import org.apache.kafka.server.config.ServerLogConfigs.{ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG} @@ -60,20 +57,6 @@ import scala.compat.java8.OptionConverters._ import scala.jdk.CollectionConverters._ -case class ControllerMigrationSupport( - zkClient: KafkaZkClient, - migrationDriver: KRaftMigrationDriver, - brokersRpcClient: LegacyPropagator -) { - def shutdown(logging: Logging): Unit = { - Utils.closeQuietly(zkClient, "zk client") - if (brokersRpcClient != null) { - CoreUtils.swallow(brokersRpcClient.shutdown(), logging) - } - Utils.closeQuietly(migrationDriver, "migration driver") - } -} - /** * A Kafka controller that runs in KRaft (Kafka Raft) mode. */ @@ -111,7 +94,6 @@ class ControllerServer( var clientQuotaMetadataManager: ClientQuotaMetadataManager = _ var controllerApis: ControllerApis = _ var controllerApisHandlerPool: KafkaRequestHandlerPool = _ - var migrationSupport: Option[ControllerMigrationSupport] = None def kafkaYammerMetrics: KafkaYammerMetrics = KafkaYammerMetrics.INSTANCE val metadataPublishers: util.List[MetadataPublisher] = new util.ArrayList[MetadataPublisher]() @volatile var metadataCache : KRaftMetadataCache = _ @@ -171,7 +153,6 @@ class ControllerServer( val apiVersionManager = new SimpleApiVersionManager( ListenerType.CONTROLLER, config.unstableApiVersionsEnabled, - config.migrationEnabled, () => featuresPublisher.features() ) @@ -233,7 +214,7 @@ class ControllerServer( val maxIdleIntervalNs = config.metadataMaxIdleIntervalNs.fold(OptionalLong.empty)(OptionalLong.of) - quorumControllerMetrics = new QuorumControllerMetrics(Optional.of(KafkaYammerMetrics.defaultRegistry), time, config.migrationEnabled) + quorumControllerMetrics = new QuorumControllerMetrics(Optional.of(KafkaYammerMetrics.defaultRegistry), time) new QuorumController.Builder(config.nodeId, sharedServer.clusterId). setTime(time). @@ -255,7 +236,6 @@ class ControllerServer( setBootstrapMetadata(bootstrapMetadata). setFatalFaultHandler(sharedServer.fatalQuorumControllerFaultHandler). setNonFatalFaultHandler(sharedServer.nonFatalQuorumControllerFaultHandler). - setZkMigrationEnabled(config.migrationEnabled). setDelegationTokenCache(tokenCache). setDelegationTokenSecretKey(delegationTokenKeyString). setDelegationTokenMaxLifeMs(config.delegationTokenMaxLifeMs). @@ -274,39 +254,6 @@ class ControllerServer( case _ => } - if (config.migrationEnabled) { - val zkClient = KafkaZkClient.createZkClient("KRaft Migration", time, config, KafkaServer.zkClientConfigFromKafkaConfig(config)) - val zkConfigEncoder = config.passwordEncoderSecret match { - case Some(secret) => PasswordEncoder.encrypting(secret, - config.passwordEncoderKeyFactoryAlgorithm, - config.passwordEncoderCipherAlgorithm, - config.passwordEncoderKeyLength, - config.passwordEncoderIterations) - case None => PasswordEncoder.NOOP - } - val migrationClient = ZkMigrationClient(zkClient, zkConfigEncoder) - val propagator: LegacyPropagator = new MigrationPropagator(config.nodeId, config) - val migrationDriver = KRaftMigrationDriver.newBuilder() - .setNodeId(config.nodeId) - .setZkRecordConsumer(controller.asInstanceOf[QuorumController].zkRecordConsumer()) - .setZkMigrationClient(migrationClient) - .setPropagator(propagator) - .setInitialZkLoadHandler(publisher => sharedServer.loader.installPublishers(java.util.Collections.singletonList(publisher))) - .setFaultHandler(sharedServer.faultHandlerFactory.build( - "zk migration", - fatal = false, - () => {} - )) - .setQuorumFeatures(quorumFeatures) - .setConfigSchema(configSchema) - .setControllerMetrics(quorumControllerMetrics) - .setMinMigrationBatchSize(config.migrationMetadataMinBatchSize) - .setTime(time) - .build() - migrationDriver.start() - migrationSupport = Some(ControllerMigrationSupport(zkClient, migrationDriver, propagator)) - } - quotaManagers = QuotaFactory.instantiate(config, metrics, time, @@ -347,7 +294,6 @@ class ControllerServer( time, s"controller-${config.nodeId}-", QuorumFeatures.defaultFeatureMap(config.unstableFeatureVersionsEnabled), - config.migrationEnabled, incarnationId, listenerInfo) @@ -490,7 +436,6 @@ class ControllerServer( registrationsPublisher = null if (socketServer != null) CoreUtils.swallow(socketServer.stopProcessingRequests(), this) - migrationSupport.foreach(_.shutdown(this)) if (controller != null) controller.beginShutdown() if (socketServer != null) diff --git a/core/src/main/scala/kafka/tools/TestRaftServer.scala b/core/src/main/scala/kafka/tools/TestRaftServer.scala index 69025491455..5f43f529401 100644 --- a/core/src/main/scala/kafka/tools/TestRaftServer.scala +++ b/core/src/main/scala/kafka/tools/TestRaftServer.scala @@ -81,7 +81,6 @@ class TestRaftServer( val apiVersionManager = new SimpleApiVersionManager( ListenerType.CONTROLLER, true, - false, () => FinalizedFeatures.fromKRaftVersion(MetadataVersion.MINIMUM_KRAFT_VERSION)) socketServer = new SocketServer(config, metrics, time, credentialProvider, apiVersionManager) diff --git a/core/src/main/scala/kafka/zk/ZkMigrationClient.scala b/core/src/main/scala/kafka/zk/ZkMigrationClient.scala deleted file mode 100644 index 20afc9fb0c4..00000000000 --- a/core/src/main/scala/kafka/zk/ZkMigrationClient.scala +++ /dev/null @@ -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 -} diff --git a/core/src/main/scala/kafka/zk/migration/ZkAclMigrationClient.scala b/core/src/main/scala/kafka/zk/migration/ZkAclMigrationClient.scala deleted file mode 100644 index 7bb4d074dd9..00000000000 --- a/core/src/main/scala/kafka/zk/migration/ZkAclMigrationClient.scala +++ /dev/null @@ -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) - } - } - } -} diff --git a/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala b/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala deleted file mode 100644 index a1f6e1a112f..00000000000 --- a/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala +++ /dev/null @@ -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)) - } - } - } -} diff --git a/core/src/main/scala/kafka/zk/migration/ZkDelegationTokenMigrationClient.scala b/core/src/main/scala/kafka/zk/migration/ZkDelegationTokenMigrationClient.scala deleted file mode 100644 index 77b301a751a..00000000000 --- a/core/src/main/scala/kafka/zk/migration/ZkDelegationTokenMigrationClient.scala +++ /dev/null @@ -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) - } - } -} - diff --git a/core/src/main/scala/kafka/zk/migration/ZkTopicMigrationClient.scala b/core/src/main/scala/kafka/zk/migration/ZkTopicMigrationClient.scala deleted file mode 100644 index 76f4b484819..00000000000 --- a/core/src/main/scala/kafka/zk/migration/ZkTopicMigrationClient.scala +++ /dev/null @@ -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") - } - } -} diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index 2a34d2aea5f..eced2794a41 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -82,7 +82,7 @@ class SocketServerTest { // Clean-up any metrics left around by previous tests TestUtils.clearYammerMetrics() - private val apiVersionManager = new SimpleApiVersionManager(ListenerType.BROKER, true, false, + private val apiVersionManager = new SimpleApiVersionManager(ListenerType.BROKER, true, () => new FinalizedFeatures(MetadataVersion.latestTesting(), Collections.emptyMap[String, java.lang.Short], 0, true)) var server: SocketServer = _ val sockets = new ArrayBuffer[Socket] diff --git a/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala b/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala index 5c0267a2f7b..ad1283f9962 100644 --- a/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala @@ -17,8 +17,7 @@ package kafka.server -import org.apache.kafka.common.test.api.{AutoStart, ClusterConfigProperty, ClusterInstance, ClusterTest, ClusterTestExtensions, ClusterTests, Type} -import org.apache.kafka.common.test.api.RaftClusterInvocationContext.RaftClusterInstance +import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterInstance, ClusterTest, ClusterTestExtensions, Type} import org.apache.kafka.clients.ClientResponse import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic import org.apache.kafka.common.message.{BrokerRegistrationRequestData, CreateTopicsRequestData} @@ -31,8 +30,7 @@ import org.apache.kafka.common.utils.Time import org.apache.kafka.common.{Node, Uuid} import org.apache.kafka.server.{ControllerRequestCompletionHandler, NodeToControllerChannelManager} import org.apache.kafka.server.common.{Features, MetadataVersion} -import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows} -import org.junit.jupiter.api.Disabled +import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.extension.ExtendWith import java.util @@ -205,91 +203,4 @@ class BrokerRegistrationRequestTest { channelManager.shutdown() } } - - @Disabled("zk migration is removed from ClusterTestExtensions") - @ClusterTest( - types = Array(Type.KRAFT), - brokers = 1, - controllers = 1, - metadataVersion = MetadataVersion.IBP_3_4_IV0, - autoStart = AutoStart.NO, - serverProperties = Array(new ClusterConfigProperty(key = "zookeeper.metadata.migration.enable", value = "true"))) - def testRegisterZkWithKRaftMigrationEnabled(clusterInstance: ClusterInstance): Unit = { - clusterInstance.asInstanceOf[RaftClusterInstance].controllers().values().forEach(_.startup()) - - val clusterId = clusterInstance.clusterId() - val channelManager = brokerToControllerChannelManager(clusterInstance) - try { - channelManager.start() - - assertEquals( - Errors.NONE, - registerBroker(channelManager, clusterId, 100, Some(1), Some((MetadataVersion.IBP_3_4_IV0, MetadataVersion.IBP_3_4_IV0)))) - - assertEquals( - Errors.UNSUPPORTED_VERSION, - registerBroker(channelManager, clusterId, 100, Some(1), None)) - - assertEquals( - Errors.UNSUPPORTED_VERSION, - registerBroker(channelManager, clusterId, 100, Some(1), Some((MetadataVersion.IBP_3_3_IV3, MetadataVersion.IBP_3_3_IV3)))) - - // Cannot register KRaft broker when in pre-migration - assertEquals( - Errors.BROKER_ID_NOT_REGISTERED, - registerBroker(channelManager, clusterId, 100, None, Some((MetadataVersion.IBP_3_4_IV0, MetadataVersion.IBP_3_4_IV0)))) - } finally { - channelManager.shutdown() - } - } - - @Disabled("zk migration is removed from ClusterTestExtensions") - @ClusterTest( - types = Array(Type.KRAFT), - brokers = 1, - controllers = 1, - metadataVersion = MetadataVersion.IBP_3_9_IV0, - autoStart = AutoStart.NO, - serverProperties = Array(new ClusterConfigProperty(key = "zookeeper.metadata.migration.enable", value = "true"))) - def testRegisterZkWithKRaftMigrationEnabledKRaftV1(clusterInstance: ClusterInstance): Unit = { - clusterInstance.asInstanceOf[RaftClusterInstance].controllers().values().forEach(_.startup()) - - val clusterId = clusterInstance.clusterId() - val channelManager = brokerToControllerChannelManager(clusterInstance) - try { - channelManager.start() - - assertEquals( - Errors.NONE, - registerBroker(channelManager, clusterId, 100, Some(1), Some((MetadataVersion.IBP_3_9_IV0, MetadataVersion.IBP_3_9_IV0)))) - - // Cannot register KRaft broker when in pre-migration - assertEquals( - Errors.BROKER_ID_NOT_REGISTERED, - registerBroker(channelManager, clusterId, 100, None, Some((MetadataVersion.IBP_3_9_IV0, MetadataVersion.IBP_3_9_IV0)))) - } finally { - channelManager.shutdown() - } - } - - /** - * Start a KRaft cluster with migrations enabled, verify that the controller does not accept metadata changes - * through the RPCs. The migration never proceeds past pre-migration since no ZK brokers are registered. - */ - @Disabled("zk migration is removed from ClusterTestExtensions") - @ClusterTests(Array( - new ClusterTest(types = Array(Type.KRAFT), autoStart = AutoStart.NO, controllers = 1, metadataVersion = MetadataVersion.IBP_3_4_IV0, - serverProperties = Array(new ClusterConfigProperty(key = "zookeeper.metadata.migration.enable", value = "true"))) - )) - def testNoMetadataChangesInPreMigrationMode(clusterInstance: ClusterInstance): Unit = { - clusterInstance.asInstanceOf[RaftClusterInstance].controllers().values().forEach(_.startup()) - - val channelManager = brokerToControllerChannelManager(clusterInstance) - try { - channelManager.start() - assertThrows(classOf[TimeoutException], () => createTopics(channelManager, "test-pre-migration")) - } finally { - channelManager.shutdown() - } - } } diff --git a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala index 50c377b3114..c8ead4aef57 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala @@ -170,7 +170,6 @@ class ControllerApisTest { new SimpleApiVersionManager( ListenerType.CONTROLLER, true, - false, () => FinalizedFeatures.fromKRaftVersion(MetadataVersion.latestTesting())), metadataCache ) diff --git a/core/src/test/scala/unit/kafka/server/ControllerRegistrationManagerTest.scala b/core/src/test/scala/unit/kafka/server/ControllerRegistrationManagerTest.scala index c33622a0038..203f821322a 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerRegistrationManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerRegistrationManagerTest.scala @@ -75,7 +75,6 @@ class ControllerRegistrationManagerTest { Time.SYSTEM, "controller-registration-manager-test-", createSupportedFeatures(MetadataVersion.IBP_3_7_IV0), - false, RecordTestUtils.createTestControllerRegistration(1, false).incarnationId(), ListenerInfo.create(context.config.controllerListeners.map(_.toJava).asJava), new ExponentialBackoff(1, 2, 100, 0.02)) diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 45f0feb9528..2674a314238 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -217,7 +217,6 @@ class KafkaApisTest extends Logging { enabledApis, BrokerFeatures.defaultSupportedFeatures(true), true, - false, () => new FinalizedFeatures(MetadataVersion.latestTesting(), Collections.emptyMap[String, java.lang.Short], 0, raftSupport)) val clientMetricsManagerOpt = if (raftSupport) Some(clientMetricsManager) else None diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/KRaftMetadataRequestBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/KRaftMetadataRequestBenchmark.java index a56ff3400a7..e0c931eeef8 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/KRaftMetadataRequestBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/KRaftMetadataRequestBenchmark.java @@ -208,7 +208,6 @@ public class KRaftMetadataRequestBenchmark { setApiVersionManager(new SimpleApiVersionManager( ApiMessageType.ListenerType.BROKER, false, - false, () -> FinalizedFeatures.fromKRaftVersion(MetadataVersion.latestTesting()))). build(); } diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/MetadataRequestBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/MetadataRequestBenchmark.java index eeebbfaa1a5..0f8d2db4e2f 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/MetadataRequestBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/MetadataRequestBenchmark.java @@ -208,7 +208,6 @@ public class MetadataRequestBenchmark { setApiVersionManager(new SimpleApiVersionManager( ApiMessageType.ListenerType.ZK_BROKER, false, - false, () -> FinalizedFeatures.fromKRaftVersion(MetadataVersion.latestTesting()))). build(); } diff --git a/metadata/src/main/java/org/apache/kafka/controller/ActivationRecordsGenerator.java b/metadata/src/main/java/org/apache/kafka/controller/ActivationRecordsGenerator.java index d81d230984b..b2ef4fe4f11 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ActivationRecordsGenerator.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ActivationRecordsGenerator.java @@ -29,12 +29,14 @@ import java.util.ArrayList; import java.util.List; import java.util.function.Consumer; +import static org.apache.kafka.metadata.migration.ZkMigrationState.NONE; +import static org.apache.kafka.metadata.migration.ZkMigrationState.POST_MIGRATION; + public class ActivationRecordsGenerator { static ControllerResult recordsForEmptyLog( Consumer activationMessageConsumer, long transactionStartOffset, - boolean zkMigrationEnabled, BootstrapMetadata bootstrapMetadata, MetadataVersion metadataVersion ) { @@ -89,20 +91,9 @@ public class ActivationRecordsGenerator { records.addAll(bootstrapMetadata.records()); if (metadataVersion.isMigrationSupported()) { - if (zkMigrationEnabled) { - logMessageBuilder.append("Putting the controller into pre-migration mode. No metadata updates " + - "will be allowed until the ZK metadata has been migrated. "); - records.add(ZkMigrationState.PRE_MIGRATION.toRecord()); - } else { - logMessageBuilder.append("Setting the ZK migration state to NONE since this is a de-novo " + - "KRaft cluster. "); - records.add(ZkMigrationState.NONE.toRecord()); - } - } else { - if (zkMigrationEnabled) { - throw new RuntimeException("The bootstrap metadata.version " + bootstrapMetadata.metadataVersion() + - " does not support ZK migrations. Cannot continue with ZK migrations enabled."); - } + logMessageBuilder.append("Setting the ZK migration state to NONE since this is a de-novo " + + "KRaft cluster. "); + records.add(NONE.toRecord()); } activationMessageConsumer.accept(logMessageBuilder.toString().trim()); @@ -117,9 +108,8 @@ public class ActivationRecordsGenerator { static ControllerResult recordsForNonEmptyLog( Consumer activationMessageConsumer, long transactionStartOffset, - boolean zkMigrationEnabled, - FeatureControlManager featureControl, - MetadataVersion metadataVersion + ZkMigrationState zkMigrationState, + MetadataVersion curMetadataVersion ) { StringBuilder logMessageBuilder = new StringBuilder("Performing controller activation. "); @@ -128,9 +118,9 @@ public class ActivationRecordsGenerator { // Check for in-flight transaction if (transactionStartOffset != -1L) { - if (!metadataVersion.isMetadataTransactionSupported()) { + if (!curMetadataVersion.isMetadataTransactionSupported()) { throw new RuntimeException("Detected in-progress transaction at offset " + transactionStartOffset + - ", but the metadata.version " + metadataVersion + + ", but the metadata.version " + curMetadataVersion + " does not support transactions. Cannot continue."); } else { logMessageBuilder @@ -142,64 +132,29 @@ public class ActivationRecordsGenerator { } } - if (metadataVersion.equals(MetadataVersion.MINIMUM_KRAFT_VERSION)) { + if (curMetadataVersion.equals(MetadataVersion.MINIMUM_KRAFT_VERSION)) { logMessageBuilder.append("No metadata.version feature level record was found in the log. ") .append("Treating the log as version ") .append(MetadataVersion.MINIMUM_KRAFT_VERSION) .append(". "); } - if (zkMigrationEnabled && !metadataVersion.isMigrationSupported()) { - throw new RuntimeException("Should not have ZK migrations enabled on a cluster running " + - "metadata.version " + featureControl.metadataVersion()); - } else if (metadataVersion.isMigrationSupported()) { - logMessageBuilder - .append("Loaded ZK migration state of ") - .append(featureControl.zkMigrationState()) - .append(". "); - switch (featureControl.zkMigrationState()) { - case NONE: - // Since this is the default state there may or may not be an actual NONE in the log. Regardless, - // it will eventually be persisted in a snapshot, so we don't need to explicitly write it here. - if (zkMigrationEnabled) { - throw new RuntimeException("Should not have ZK migrations enabled on a cluster that was " + - "created in KRaft mode."); - } + if (curMetadataVersion.isMigrationSupported()) { + if (zkMigrationState == NONE || zkMigrationState == POST_MIGRATION) { + logMessageBuilder + .append("Loaded ZK migration state of ") + .append(zkMigrationState) + .append(". "); + if (zkMigrationState == NONE) { logMessageBuilder.append("This is expected because this is a de-novo KRaft cluster."); - break; - case PRE_MIGRATION: - if (!metadataVersion.isMetadataTransactionSupported()) { - logMessageBuilder - .append("Activating pre-migration controller without empty log. ") - .append("There may be a partial migration. "); - } - break; - case MIGRATION: - if (!zkMigrationEnabled) { - // This can happen if controller leadership transfers to a controller with migrations enabled - // after another controller had finalized the migration. For example, during a rolling restart - // of the controller quorum during which the migration config is being set to false. - logMessageBuilder - .append("Completing the ZK migration since this controller was configured with ") - .append("'zookeeper.metadata.migration.enable' set to 'false'. "); - records.add(ZkMigrationState.POST_MIGRATION.toRecord()); - } else { - // This log message is used in zookeeper_migration_test.py - logMessageBuilder - .append("Staying in ZK migration mode since 'zookeeper.metadata.migration.enable' ") - .append("is still 'true'. "); - } - break; - case POST_MIGRATION: - if (zkMigrationEnabled) { - logMessageBuilder - .append("Ignoring 'zookeeper.metadata.migration.enable' value of 'true' since ") - .append("the ZK migration has been completed. "); - } - break; - default: - throw new IllegalStateException("Unsupported ZkMigrationState " + featureControl.zkMigrationState()); + } + } else { + throw new RuntimeException("Cannot load ZkMigrationState." + zkMigrationState + + " because ZK migration is no longer supported."); } + } else if (zkMigrationState != NONE) { + throw new RuntimeException("Should not have ZkMigrationState." + zkMigrationState + + " on a cluster running metadata version " + curMetadataVersion + "."); } activationMessageConsumer.accept(logMessageBuilder.toString().trim()); @@ -220,16 +175,16 @@ public class ActivationRecordsGenerator { Consumer activationMessageConsumer, boolean isEmpty, long transactionStartOffset, - boolean zkMigrationEnabled, BootstrapMetadata bootstrapMetadata, - FeatureControlManager featureControl + ZkMigrationState zkMigrationState, + MetadataVersion curMetadataVersion ) { if (isEmpty) { - return recordsForEmptyLog(activationMessageConsumer, transactionStartOffset, zkMigrationEnabled, - bootstrapMetadata, bootstrapMetadata.metadataVersion()); + return recordsForEmptyLog(activationMessageConsumer, transactionStartOffset, + bootstrapMetadata, bootstrapMetadata.metadataVersion()); } else { - return recordsForNonEmptyLog(activationMessageConsumer, transactionStartOffset, zkMigrationEnabled, - featureControl, featureControl.metadataVersion()); + return recordsForNonEmptyLog(activationMessageConsumer, transactionStartOffset, + zkMigrationState, curMetadataVersion); } } } diff --git a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java index b0b43f05a4c..0e263a548c8 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java @@ -91,7 +91,6 @@ public class ClusterControlManager { private long sessionTimeoutNs = DEFAULT_SESSION_TIMEOUT_NS; private ReplicaPlacer replicaPlacer = null; private FeatureControlManager featureControl = null; - private boolean zkMigrationEnabled = false; private BrokerUncleanShutdownHandler brokerUncleanShutdownHandler = null; private String interBrokerListenerName = "PLAINTEXT"; @@ -130,11 +129,6 @@ public class ClusterControlManager { return this; } - Builder setZkMigrationEnabled(boolean zkMigrationEnabled) { - this.zkMigrationEnabled = zkMigrationEnabled; - return this; - } - Builder setBrokerUncleanShutdownHandler(BrokerUncleanShutdownHandler brokerUncleanShutdownHandler) { this.brokerUncleanShutdownHandler = brokerUncleanShutdownHandler; return this; @@ -171,7 +165,6 @@ public class ClusterControlManager { sessionTimeoutNs, replicaPlacer, featureControl, - zkMigrationEnabled, brokerUncleanShutdownHandler, interBrokerListenerName ); @@ -260,11 +253,6 @@ public class ClusterControlManager { */ private final FeatureControlManager featureControl; - /** - * True if migration from ZK is enabled. - */ - private final boolean zkMigrationEnabled; - private final BrokerUncleanShutdownHandler brokerUncleanShutdownHandler; /** @@ -290,7 +278,6 @@ public class ClusterControlManager { long sessionTimeoutNs, ReplicaPlacer replicaPlacer, FeatureControlManager featureControl, - boolean zkMigrationEnabled, BrokerUncleanShutdownHandler brokerUncleanShutdownHandler, String interBrokerListenerName ) { @@ -305,7 +292,6 @@ public class ClusterControlManager { this.heartbeatManager = null; this.readyBrokersFuture = Optional.empty(); this.featureControl = featureControl; - this.zkMigrationEnabled = zkMigrationEnabled; this.controllerRegistrations = new TimelineHashMap<>(snapshotRegistry, 0); this.directoryToBroker = new TimelineHashMap<>(snapshotRegistry, 0); this.brokerUncleanShutdownHandler = brokerUncleanShutdownHandler; @@ -349,10 +335,6 @@ public class ClusterControlManager { .collect(Collectors.toSet()); } - boolean zkRegistrationAllowed() { - return zkMigrationEnabled && featureControl.metadataVersion().isMigrationSupported(); - } - /** * Process an incoming broker registration request. */ @@ -382,20 +364,8 @@ public class ClusterControlManager { } } - if (request.isMigratingZkBroker() && !zkRegistrationAllowed()) { - throw new BrokerIdNotRegisteredException("Controller does not support registering ZK brokers."); - } - - if (!request.isMigratingZkBroker() && featureControl.inPreMigrationMode()) { - throw new BrokerIdNotRegisteredException("Controller is in pre-migration mode and cannot register KRaft " + - "brokers until the metadata migration is complete."); - } - if (request.isMigratingZkBroker()) { - if (request.listeners().find(interBrokerListenerName) == null) { - throw new InvalidRegistrationException("Broker does not have the current inter.broker.listener " + - interBrokerListenerName); - } + throw new BrokerIdNotRegisteredException("Controller does not support registering ZK brokers."); } if (featureControl.metadataVersion().isDirectoryAssignmentSupported()) { @@ -510,7 +480,7 @@ public class ClusterControlManager { records.add(new ApiMessageAndVersion(new RegisterControllerRecord(). setControllerId(request.controllerId()). setIncarnationId(request.incarnationId()). - setZkMigrationReady(request.zkMigrationReady()). + setZkMigrationReady(false). setEndPoints(listenerInfo.toControllerRegistrationRecord()). setFeatures(features), (short) 0)); diff --git a/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java index 2f8b9b0efed..d22c3f1fe44 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java @@ -397,16 +397,6 @@ public class FeatureControlManager { return new FinalizedControllerFeatures(features, epoch); } - /** - * Tests if the controller should be preventing metadata updates due to being in the PRE_MIGRATION - * state. If the controller does not yet support migrations (before 3.4-IV0), then the migration state - * will be NONE and this will return false. Once the controller has been upgraded to a version that supports - * migrations, then this method checks if the migration state is equal to PRE_MIGRATION. - */ - boolean inPreMigrationMode() { - return migrationControlState.get().equals(ZkMigrationState.PRE_MIGRATION); - } - public void replay(FeatureLevelRecord record) { VersionRange range = quorumFeatures.localSupportedFeature(record.name()); if (!range.contains(record.featureLevel())) { diff --git a/metadata/src/main/java/org/apache/kafka/controller/OffsetControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/OffsetControlManager.java index c52f059d7ac..99375af6141 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/OffsetControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/OffsetControlManager.java @@ -75,7 +75,7 @@ class OffsetControlManager { if (logContext == null) logContext = new LogContext(); if (snapshotRegistry == null) snapshotRegistry = new SnapshotRegistry(logContext); if (metrics == null) { - metrics = new QuorumControllerMetrics(Optional.empty(), time, false); + metrics = new QuorumControllerMetrics(Optional.empty(), time); } return new OffsetControlManager(logContext, snapshotRegistry, diff --git a/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java b/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java index 4d4f65125d5..0592847df6f 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java +++ b/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java @@ -97,7 +97,6 @@ public class PartitionChangeBuilder { private List uncleanShutdownReplicas; private Election election = Election.ONLINE; private LeaderRecoveryState targetLeaderRecoveryState; - private boolean zkMigrationEnabled; private boolean eligibleLeaderReplicasEnabled; private DefaultDirProvider defaultDirProvider; @@ -118,7 +117,6 @@ public class PartitionChangeBuilder { this.partitionId = partitionId; this.isAcceptableLeader = isAcceptableLeader; this.metadataVersion = metadataVersion; - this.zkMigrationEnabled = false; this.eligibleLeaderReplicasEnabled = false; this.minISR = minISR; @@ -179,11 +177,6 @@ public class PartitionChangeBuilder { return this; } - public PartitionChangeBuilder setZkMigrationEnabled(boolean zkMigrationEnabled) { - this.zkMigrationEnabled = zkMigrationEnabled; - return this; - } - public PartitionChangeBuilder setEligibleLeaderReplicasEnabled(boolean eligibleLeaderReplicasEnabled) { this.eligibleLeaderReplicasEnabled = eligibleLeaderReplicasEnabled; return this; @@ -392,17 +385,11 @@ public class PartitionChangeBuilder { * the PartitionChangeRecord. */ void triggerLeaderEpochBumpForIsrShrinkIfNeeded(PartitionChangeRecord record) { - if (!(metadataVersion.isLeaderEpochBumpRequiredOnIsrShrink() || zkMigrationEnabled)) { - // We only need to bump the leader epoch on an ISR shrink in two cases: - // - // 1. In older metadata versions before 3.6, there was a bug (KAFKA-15021) in the - // broker replica manager that required that the leader epoch be bumped whenever - // the ISR shrank. (This was never necessary for EXPANSIONS, only SHRINKS.) - // - // 2. During ZK migration, we bump the leader epoch during all ISR shrinks, in order - // to maintain compatibility with migrating brokers that are still in ZK mode. - // - // If we're not in either case, we can exit here. + if (!metadataVersion.isLeaderEpochBumpRequiredOnIsrShrink()) { + // We only need to bump the leader epoch on an ISR shrink in older metadata versions + // before 3.6, where there was a bug (KAFKA-15021) in the broker replica manager that + // required that the leader epoch be bumped whenever the ISR shrank. (This was never + // necessary for EXPANSIONS, only SHRINKS.) return; } if (record.leader() != NO_LEADER_CHANGE) { diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java index d4fb01625f1..e49dbc249e9 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -103,8 +103,6 @@ import org.apache.kafka.metadata.FinalizedControllerFeatures; import org.apache.kafka.metadata.KafkaConfigSchema; import org.apache.kafka.metadata.VersionRange; import org.apache.kafka.metadata.bootstrap.BootstrapMetadata; -import org.apache.kafka.metadata.migration.ZkMigrationState; -import org.apache.kafka.metadata.migration.ZkRecordConsumer; import org.apache.kafka.metadata.placement.ReplicaPlacer; import org.apache.kafka.metadata.placement.StripedReplicaPlacer; import org.apache.kafka.metadata.util.RecordRedactor; @@ -131,7 +129,6 @@ import org.apache.kafka.timeline.SnapshotRegistry; import org.slf4j.Logger; -import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.EnumSet; @@ -154,7 +151,6 @@ import java.util.function.Supplier; import static java.util.concurrent.TimeUnit.MICROSECONDS; import static java.util.concurrent.TimeUnit.NANOSECONDS; import static org.apache.kafka.controller.QuorumController.ControllerOperationFlag.DOES_NOT_UPDATE_QUEUE_TIME; -import static org.apache.kafka.controller.QuorumController.ControllerOperationFlag.RUNS_IN_PREMIGRATION; /** @@ -217,7 +213,6 @@ public final class QuorumController implements Controller { private Map staticConfig = Collections.emptyMap(); private BootstrapMetadata bootstrapMetadata = null; private int maxRecordsPerBatch = MAX_RECORDS_PER_BATCH; - private boolean zkMigrationEnabled = false; private boolean eligibleLeaderReplicasEnabled = false; private DelegationTokenCache tokenCache; private String tokenSecretKeyString; @@ -341,11 +336,6 @@ public final class QuorumController implements Controller { return this; } - public Builder setZkMigrationEnabled(boolean zkMigrationEnabled) { - this.zkMigrationEnabled = zkMigrationEnabled; - return this; - } - public Builder setEligibleLeaderReplicasEnabled(boolean eligibleLeaderReplicasEnabled) { this.eligibleLeaderReplicasEnabled = eligibleLeaderReplicasEnabled; return this; @@ -406,7 +396,7 @@ public final class QuorumController implements Controller { logContext = new LogContext(String.format("[QuorumController id=%d] ", nodeId)); } if (controllerMetrics == null) { - controllerMetrics = new QuorumControllerMetrics(Optional.empty(), time, zkMigrationEnabled); + controllerMetrics = new QuorumControllerMetrics(Optional.empty(), time); } KafkaEventQueue queue = null; @@ -436,7 +426,6 @@ public final class QuorumController implements Controller { staticConfig, bootstrapMetadata, maxRecordsPerBatch, - zkMigrationEnabled, tokenCache, tokenSecretKeyString, delegationTokenMaxLifeMs, @@ -674,10 +663,6 @@ public final class QuorumController implements Controller { return configurationControl; } - public ZkRecordConsumer zkRecordConsumer() { - return zkRecordConsumer; - } - CompletableFuture appendReadEvent( String name, OptionalLong deadlineNs, @@ -697,23 +682,7 @@ public final class QuorumController implements Controller { * A flag that signifies that this operation should not update the event queue time metric. * We use this when the event was not appended to the queue. */ - DOES_NOT_UPDATE_QUEUE_TIME, - - /** - * A flag that signifies that this operation can be processed when in pre-migration mode. - * Operations without this flag will always return NOT_CONTROLLER when invoked in premigration - * mode. - *

- * 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 - *

- * 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. - *

- * In practice, very few operations should use this flag. - */ - RUNS_IN_PREMIGRATION + DOES_NOT_UPDATE_QUEUE_TIME; } interface ControllerWriteOperation { @@ -784,24 +753,13 @@ public final class QuorumController implements Controller { if (!isActiveController(controllerEpoch)) { throw ControllerExceptions.newWrongControllerException(latestController()); } - if (featureControl.inPreMigrationMode() && !flags.contains(RUNS_IN_PREMIGRATION)) { - log.info("Cannot run write operation {} in pre-migration mode. Returning NOT_CONTROLLER.", name); - throw ControllerExceptions.newPreMigrationException(latestController()); - } ControllerResult result = op.generateRecordsAndResult(); if (result.records().isEmpty()) { op.processBatchEndOffset(offsetControl.nextWriteOffset() - 1); // If the operation did not return any records, then it was actually just // a read after all, and not a read + write. However, this read was done // from the latest in-memory state, which might contain uncommitted data. - // If the operation can complete within a transaction, let it use the - // unstable purgatory so that it can complete sooner. - OptionalLong maybeOffset; - if (featureControl.inPreMigrationMode() && flags.contains(RUNS_IN_PREMIGRATION)) { - maybeOffset = deferredUnstableEventQueue.highestPendingOffset(); - } else { - maybeOffset = deferredEventQueue.highestPendingOffset(); - } + OptionalLong maybeOffset = deferredEventQueue.highestPendingOffset(); if (!maybeOffset.isPresent()) { // If the purgatory is empty, there are no pending operations and no // uncommitted state. We can complete immediately. @@ -862,11 +820,7 @@ public final class QuorumController implements Controller { // Remember the latest offset and future if it is not already completed if (!future.isDone()) { - if (featureControl.inPreMigrationMode() && flags.contains(RUNS_IN_PREMIGRATION)) { - deferredUnstableEventQueue.add(resultAndOffset.offset(), this); - } else { - deferredEventQueue.add(resultAndOffset.offset(), this); - } + deferredEventQueue.add(resultAndOffset.offset(), this); } } @@ -981,78 +935,6 @@ public final class QuorumController implements Controller { return event.future(); } - class MigrationRecordConsumer implements ZkRecordConsumer { - private final EnumSet eventFlags = EnumSet.of(RUNS_IN_PREMIGRATION); - - private volatile OffsetAndEpoch highestMigrationRecordOffset; - - class MigrationWriteOperation implements ControllerWriteOperation { - private final List batch; - - MigrationWriteOperation(List batch) { - this.batch = batch; - } - @Override - public ControllerResult 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 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 recordBatch) { - ControllerWriteEvent batchEvent = new ControllerWriteEvent<>( - "ZK Migration Batch", - new MigrationWriteOperation(recordBatch), eventFlags); - queue.append(batchEvent); - return batchEvent.future; - } - - @Override - public CompletableFuture completeMigration() { - log.info("Completing migration of ZooKeeper metadata to KRaft."); - List records = new ArrayList<>(2); - records.add(ZkMigrationState.MIGRATION.toRecord()); - if (featureControl.metadataVersion().isMetadataTransactionSupported()) { - records.add(new ApiMessageAndVersion(new EndTransactionRecord(), (short) 0)); - } - ControllerWriteEvent 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 { @Override public void handleCommit(BatchReader reader) { @@ -1077,7 +959,6 @@ public final class QuorumController implements Controller { // items that were waiting for these offsets. offsetControl.handleCommitBatch(batch); deferredEventQueue.completeUpTo(offsetControl.lastStableOffset()); - deferredUnstableEventQueue.completeUpTo(offsetControl.lastCommittedOffset()); } else { // If the controller is a standby, replay the records that were // created by the active controller. @@ -1228,7 +1109,7 @@ public final class QuorumController implements Controller { ControllerWriteEvent activationEvent = new ControllerWriteEvent<>( "completeActivation[" + epoch + "]", new CompleteActivationEvent(), - EnumSet.of(DOES_NOT_UPDATE_QUEUE_TIME, RUNS_IN_PREMIGRATION) + EnumSet.of(DOES_NOT_UPDATE_QUEUE_TIME) ); queue.prepend(activationEvent); } catch (Throwable e) { @@ -1244,9 +1125,9 @@ public final class QuorumController implements Controller { log::warn, logReplayTracker.empty(), offsetControl.transactionStartOffset(), - zkMigrationEnabled, bootstrapMetadata, - featureControl); + featureControl.zkMigrationState(), + featureControl.metadataVersion()); } catch (Throwable t) { throw fatalFaultHandler.handleFault("exception while completing controller " + "activation", t); @@ -1276,8 +1157,6 @@ public final class QuorumController implements Controller { curClaimEpoch = -1; deferredEventQueue.failAll(ControllerExceptions. newWrongControllerException(OptionalInt.empty())); - deferredUnstableEventQueue.failAll(ControllerExceptions. - newWrongControllerException(OptionalInt.empty())); offsetControl.deactivate(); clusterControl.deactivate(); cancelMaybeFenceReplicas(); @@ -1474,7 +1353,7 @@ public final class QuorumController implements Controller { null ); }, - EnumSet.of(DOES_NOT_UPDATE_QUEUE_TIME, RUNS_IN_PREMIGRATION) + EnumSet.of(DOES_NOT_UPDATE_QUEUE_TIME) ); long delayNs = time.nanoseconds() + maxIdleIntervalNs.getAsLong(); @@ -1683,13 +1562,6 @@ public final class QuorumController implements Controller { */ private final DeferredEventQueue deferredEventQueue; - /** - * The deferred event queue which holds deferred operations which are waiting for the metadata - * log's committed offset to advance. This must be accessed only by the event queue thread and - * can contain records which are part of an incomplete transaction. - */ - private final DeferredEventQueue deferredUnstableEventQueue; - /** * Manages read and write offsets, and in-memory snapshots. */ @@ -1823,10 +1695,6 @@ public final class QuorumController implements Controller { */ private final BootstrapMetadata bootstrapMetadata; - private final ZkRecordConsumer zkRecordConsumer; - - private final boolean zkMigrationEnabled; - private final boolean eligibleLeaderReplicasEnabled; /** @@ -1868,7 +1736,6 @@ public final class QuorumController implements Controller { Map staticConfig, BootstrapMetadata bootstrapMetadata, int maxRecordsPerBatch, - boolean zkMigrationEnabled, DelegationTokenCache tokenCache, String tokenSecretKeyString, long delegationTokenMaxLifeMs, @@ -1888,7 +1755,6 @@ public final class QuorumController implements Controller { this.controllerMetrics = controllerMetrics; this.snapshotRegistry = new SnapshotRegistry(logContext); this.deferredEventQueue = new DeferredEventQueue(logContext); - this.deferredUnstableEventQueue = new DeferredEventQueue(logContext); this.offsetControl = new OffsetControlManager.Builder(). setLogContext(logContext). setSnapshotRegistry(snapshotRegistry). @@ -1931,7 +1797,6 @@ public final class QuorumController implements Controller { setSessionTimeoutNs(sessionTimeoutNs). setReplicaPlacer(replicaPlacer). setFeatureControlManager(featureControl). - setZkMigrationEnabled(zkMigrationEnabled). setBrokerUncleanShutdownHandler(this::handleUncleanBrokerShutdown). setInterBrokerListenerName(interBrokerListenerName). build(); @@ -1978,15 +1843,13 @@ public final class QuorumController implements Controller { this.maxRecordsPerBatch = maxRecordsPerBatch; this.metaLogListener = new QuorumMetaLogListener(); this.curClaimEpoch = -1; - this.zkRecordConsumer = new MigrationRecordConsumer(); - this.zkMigrationEnabled = zkMigrationEnabled; this.recordRedactor = new RecordRedactor(configSchema); this.eligibleLeaderReplicasEnabled = eligibleLeaderReplicasEnabled; this.uncleanLeaderElectionCheckIntervalNs = TimeUnit.MILLISECONDS.toNanos(uncleanLeaderElectionCheckIntervalMs); - log.info("Creating new QuorumController with clusterId {}.{}{}", - clusterId, zkMigrationEnabled ? " ZK migration mode is enabled." : "", + log.info("Creating new QuorumController with clusterId {}.{}", + clusterId, eligibleLeaderReplicasEnabled ? " Eligible leader replicas enabled." : ""); this.raftClient.register(metaLogListener); @@ -2061,7 +1924,8 @@ public final class QuorumController implements Controller { int brokerId ) { return appendWriteEvent("unregisterBroker", context.deadlineNs(), - () -> replicationControl.unregisterBroker(brokerId), EnumSet.of(RUNS_IN_PREMIGRATION)); + () -> replicationControl.unregisterBroker(brokerId), + EnumSet.noneOf(ControllerOperationFlag.class)); } @Override @@ -2238,7 +2102,7 @@ public final class QuorumController implements Controller { } } }, - EnumSet.of(RUNS_IN_PREMIGRATION)).whenComplete((__, t) -> { + EnumSet.noneOf(ControllerOperationFlag.class)).whenComplete((__, t) -> { if (ControllerExceptions.isTimeoutException(t)) { replicationControl.processExpiredBrokerHeartbeat(request); controllerMetrics.incrementTimedOutHeartbeats(); @@ -2262,7 +2126,7 @@ public final class QuorumController implements Controller { rescheduleMaybeFenceStaleBrokers(); return result; }, - EnumSet.of(RUNS_IN_PREMIGRATION)); + EnumSet.noneOf(ControllerOperationFlag.class)); } @Override @@ -2366,7 +2230,7 @@ public final class QuorumController implements Controller { ) { return appendWriteEvent("registerController", context.deadlineNs(), () -> clusterControl.registerController(request), - EnumSet.of(RUNS_IN_PREMIGRATION)); + EnumSet.noneOf(ControllerOperationFlag.class)); } @Override diff --git a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java index e7a4359fb54..25897f74903 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java @@ -1070,7 +1070,6 @@ public class ReplicationControlManager { featureControl.metadataVersion(), getTopicEffectiveMinIsr(topic.name) ) - .setZkMigrationEnabled(clusterControl.zkRegistrationAllowed()) .setEligibleLeaderReplicasEnabled(isElrEnabled()); if (configurationControl.uncleanLeaderElectionEnabledForTopic(topic.name())) { builder.setElection(PartitionChangeBuilder.Election.UNCLEAN); @@ -1517,7 +1516,6 @@ public class ReplicationControlManager { getTopicEffectiveMinIsr(topic) ) .setElection(election) - .setZkMigrationEnabled(clusterControl.zkRegistrationAllowed()) .setEligibleLeaderReplicasEnabled(isElrEnabled()) .setDefaultDirProvider(clusterDescriber) .build(); @@ -1666,7 +1664,6 @@ public class ReplicationControlManager { getTopicEffectiveMinIsr(topic.name) ) .setElection(PartitionChangeBuilder.Election.PREFERRED) - .setZkMigrationEnabled(clusterControl.zkRegistrationAllowed()) .setEligibleLeaderReplicasEnabled(isElrEnabled()) .setDefaultDirProvider(clusterDescriber) .build().ifPresent(records::add); @@ -1936,7 +1933,6 @@ public class ReplicationControlManager { featureControl.metadataVersion(), getTopicEffectiveMinIsr(topic.name) ); - builder.setZkMigrationEnabled(clusterControl.zkRegistrationAllowed()); builder.setEligibleLeaderReplicasEnabled(isElrEnabled()); if (configurationControl.uncleanLeaderElectionEnabledForTopic(topic.name)) { builder.setElection(PartitionChangeBuilder.Election.UNCLEAN); @@ -2055,7 +2051,6 @@ public class ReplicationControlManager { featureControl.metadataVersion(), getTopicEffectiveMinIsr(topicName) ); - builder.setZkMigrationEnabled(clusterControl.zkRegistrationAllowed()); builder.setEligibleLeaderReplicasEnabled(isElrEnabled()); if (configurationControl.uncleanLeaderElectionEnabledForTopic(topicName)) { builder.setElection(PartitionChangeBuilder.Election.UNCLEAN); @@ -2117,7 +2112,6 @@ public class ReplicationControlManager { featureControl.metadataVersion(), getTopicEffectiveMinIsr(topics.get(tp.topicId()).name) ); - builder.setZkMigrationEnabled(clusterControl.zkRegistrationAllowed()); builder.setEligibleLeaderReplicasEnabled(isElrEnabled()); if (!reassignment.replicas().equals(currentReplicas)) { builder.setTargetReplicas(reassignment.replicas()); diff --git a/metadata/src/main/java/org/apache/kafka/controller/metrics/QuorumControllerMetrics.java b/metadata/src/main/java/org/apache/kafka/controller/metrics/QuorumControllerMetrics.java index 483ed65ce86..1e71ee6fc13 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/metrics/QuorumControllerMetrics.java +++ b/metadata/src/main/java/org/apache/kafka/controller/metrics/QuorumControllerMetrics.java @@ -45,12 +45,6 @@ public class QuorumControllerMetrics implements AutoCloseable { "ControllerEventManager", "EventQueueTimeMs"); private static final MetricName EVENT_QUEUE_PROCESSING_TIME_MS = getMetricName( "ControllerEventManager", "EventQueueProcessingTimeMs"); - private static final MetricName ZK_WRITE_BEHIND_LAG = getMetricName( - "KafkaController", "ZkWriteBehindLag"); - private static final MetricName ZK_WRITE_SNAPSHOT_TIME_MS = getMetricName( - "KafkaController", "ZkWriteSnapshotTimeMs"); - private static final MetricName ZK_WRITE_DELTA_TIME_MS = getMetricName( - "KafkaController", "ZkWriteDeltaTimeMs"); private static final MetricName LAST_APPLIED_RECORD_OFFSET = getMetricName( "KafkaController", "LastAppliedRecordOffset"); private static final MetricName LAST_COMMITTED_RECORD_OFFSET = getMetricName( @@ -73,11 +67,8 @@ public class QuorumControllerMetrics implements AutoCloseable { private final AtomicLong lastAppliedRecordOffset = new AtomicLong(0); private final AtomicLong lastCommittedRecordOffset = new AtomicLong(0); private final AtomicLong lastAppliedRecordTimestamp = new AtomicLong(0); - private final AtomicLong dualWriteOffset = new AtomicLong(0); private final Consumer eventQueueTimeUpdater; private final Consumer eventQueueProcessingTimeUpdater; - private final Consumer zkWriteSnapshotTimeHandler; - private final Consumer zkWriteDeltaTimeHandler; private final AtomicLong timedOutHeartbeats = new AtomicLong(0); private final AtomicLong operationsStarted = new AtomicLong(0); @@ -95,8 +86,7 @@ public class QuorumControllerMetrics implements AutoCloseable { public QuorumControllerMetrics( Optional registry, - Time time, - boolean zkMigrationEnabled + Time time ) { this.registry = registry; this.active = false; @@ -156,23 +146,6 @@ public class QuorumControllerMetrics implements AutoCloseable { return newActiveControllers(); } })); - - if (zkMigrationEnabled) { - registry.ifPresent(r -> r.newGauge(ZK_WRITE_BEHIND_LAG, new Gauge() { - @Override - public Long value() { - // not in dual-write mode or not an active controller: set metric value to 0 - if (dualWriteOffset() == 0 || !active()) return 0L; - // in dual write mode - else return lastCommittedRecordOffset() - dualWriteOffset(); - } - })); - this.zkWriteSnapshotTimeHandler = newHistogram(ZK_WRITE_SNAPSHOT_TIME_MS, true); - this.zkWriteDeltaTimeHandler = newHistogram(ZK_WRITE_DELTA_TIME_MS, true); - } else { - this.zkWriteSnapshotTimeHandler = __ -> { }; - this.zkWriteDeltaTimeHandler = __ -> { }; - } } public void setActive(boolean active) { @@ -191,14 +164,6 @@ public class QuorumControllerMetrics implements AutoCloseable { eventQueueProcessingTimeUpdater.accept(durationMs); } - public void updateZkWriteSnapshotTimeMs(long durationMs) { - zkWriteSnapshotTimeHandler.accept(durationMs); - } - - public void updateZkWriteDeltaTimeMs(long durationMs) { - zkWriteDeltaTimeHandler.accept(durationMs); - } - public void setLastAppliedRecordOffset(long offset) { lastAppliedRecordOffset.set(offset); } @@ -223,14 +188,6 @@ public class QuorumControllerMetrics implements AutoCloseable { return lastAppliedRecordTimestamp.get(); } - public void updateDualWriteOffset(long offset) { - dualWriteOffset.set(offset); - } - - public long dualWriteOffset() { - return dualWriteOffset.get(); - } - public void incrementTimedOutHeartbeats() { timedOutHeartbeats.incrementAndGet(); } @@ -276,10 +233,7 @@ public class QuorumControllerMetrics implements AutoCloseable { TIMED_OUT_BROKER_HEARTBEAT_COUNT, EVENT_QUEUE_OPERATIONS_STARTED_COUNT, EVENT_QUEUE_OPERATIONS_TIMED_OUT_COUNT, - NEW_ACTIVE_CONTROLLERS_COUNT, - ZK_WRITE_BEHIND_LAG, - ZK_WRITE_SNAPSHOT_TIME_MS, - ZK_WRITE_DELTA_TIME_MS + NEW_ACTIVE_CONTROLLERS_COUNT ).forEach(r::removeMetric)); } diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/AclMigrationClient.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/AclMigrationClient.java deleted file mode 100644 index 0b2b368bc2d..00000000000 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/AclMigrationClient.java +++ /dev/null @@ -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 aclsToWrite, - ZkMigrationLeadershipState state - ); - - void iterateAcls(BiConsumer> aclConsumer); -} diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/BufferingBatchConsumer.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/BufferingBatchConsumer.java deleted file mode 100644 index ed8a8236aaf..00000000000 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/BufferingBatchConsumer.java +++ /dev/null @@ -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. - *

- * Note that {@link #flush()} must be called after the last batch has been accepted in order to flush any - * buffered records. - */ -public class BufferingBatchConsumer implements Consumer> { - - private final Consumer> delegateConsumer; - private final int minBatchSize; - private List bufferedBatch; - - BufferingBatchConsumer(Consumer> delegateConsumer, int minBatchSize) { - this.delegateConsumer = delegateConsumer; - this.minBatchSize = minBatchSize; - this.bufferedBatch = new ArrayList<>(minBatchSize); - } - - @Override - public void accept(List batch) { - bufferedBatch.addAll(batch); - if (bufferedBatch.size() >= minBatchSize) { - flush(); - } - } - - public void flush() { - if (!bufferedBatch.isEmpty()) { - delegateConsumer.accept(bufferedBatch); - bufferedBatch = new ArrayList<>(minBatchSize); - } - } -} diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/ConfigMigrationClient.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/ConfigMigrationClient.java deleted file mode 100644 index 57f63fb3cce..00000000000 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/ConfigMigrationClient.java +++ /dev/null @@ -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 entityDataList, Map quotas); - - void visitScramCredential(String userName, ScramMechanism scramMechanism, ScramCredential scramCredential); - } - - void iterateClientQuotas(ClientQuotaVisitor visitor); - - void iterateBrokerConfigs(BiConsumer> configConsumer); - - void iterateTopicConfigs(BiConsumer> configConsumer); - - void readTopicConfigs(String topicName, Consumer> configConsumer); - - ZkMigrationLeadershipState writeConfigs( - ConfigResource configResource, - Map configMap, - ZkMigrationLeadershipState state - ); - - ZkMigrationLeadershipState writeClientQuotas( - Map clientQuotaEntity, - Map quotas, - Map scram, - ZkMigrationLeadershipState state - ); - - ZkMigrationLeadershipState deleteConfigs( - ConfigResource configResource, - ZkMigrationLeadershipState state - ); -} diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/DelegationTokenMigrationClient.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/DelegationTokenMigrationClient.java deleted file mode 100644 index 48eee1c1d92..00000000000 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/DelegationTokenMigrationClient.java +++ /dev/null @@ -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 getDelegationTokens(); - - ZkMigrationLeadershipState writeDelegationToken( - String tokenId, - TokenInformation tokenInformation, - ZkMigrationLeadershipState state - ); - - ZkMigrationLeadershipState deleteDelegationToken( - String tokenId, - ZkMigrationLeadershipState state - ); - -} diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java deleted file mode 100644 index c00c25fd158..00000000000 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java +++ /dev/null @@ -1,1088 +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.utils.ExponentialBackoff; -import org.apache.kafka.common.utils.LogContext; -import org.apache.kafka.common.utils.Time; -import org.apache.kafka.controller.QuorumFeatures; -import org.apache.kafka.controller.metrics.QuorumControllerMetrics; -import org.apache.kafka.image.MetadataDelta; -import org.apache.kafka.image.MetadataImage; -import org.apache.kafka.image.MetadataProvenance; -import org.apache.kafka.image.loader.LoaderManifest; -import org.apache.kafka.image.loader.LoaderManifestType; -import org.apache.kafka.image.publisher.MetadataPublisher; -import org.apache.kafka.metadata.BrokerRegistration; -import org.apache.kafka.metadata.KafkaConfigSchema; -import org.apache.kafka.metadata.util.RecordRedactor; -import org.apache.kafka.queue.EventQueue; -import org.apache.kafka.queue.KafkaEventQueue; -import org.apache.kafka.raft.LeaderAndEpoch; -import org.apache.kafka.raft.OffsetAndEpoch; -import org.apache.kafka.server.common.ApiMessageAndVersion; -import org.apache.kafka.server.fault.FaultHandler; -import org.apache.kafka.server.util.Deadline; -import org.apache.kafka.server.util.FutureUtils; - -import org.slf4j.Logger; - -import java.util.EnumSet; -import java.util.HashSet; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.TreeMap; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.RejectedExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.function.BiConsumer; -import java.util.function.Consumer; - -import static java.util.concurrent.TimeUnit.MILLISECONDS; -import static java.util.concurrent.TimeUnit.NANOSECONDS; - -/** - * This class orchestrates and manages the state related to a ZK to KRaft migration. A single event thread is used to - * serialize events coming from various threads and listeners. - */ -public class KRaftMigrationDriver implements MetadataPublisher { - - private static class PollTimeSupplier { - private final ExponentialBackoff pollBackoff; - private long pollCount; - - PollTimeSupplier() { - this.pollCount = 0; - this.pollBackoff = new ExponentialBackoff(100, 2, 60000, 0.02); - } - - void reset() { - this.pollCount = 0; - } - - public long nextPollTimeMs() { - long next = pollBackoff.backoff(pollCount); - pollCount++; - return next; - } - } - - private static final Consumer NO_OP_HANDLER = ex -> { }; - - /** - * When waiting for the metadata layer to commit batches, we block the migration driver thread for this - * amount of time. A large value is selected to avoid timeouts in the common case, but prevent us from - * blocking indefinitely. - */ - static final int METADATA_COMMIT_MAX_WAIT_MS = 300_000; - - private final Time time; - private final Logger log; - private final int nodeId; - private final MigrationClient zkMigrationClient; - private final KRaftMigrationZkWriter zkMetadataWriter; - private final LegacyPropagator propagator; - private final ZkRecordConsumer zkRecordConsumer; - private final KafkaEventQueue eventQueue; - private final PollTimeSupplier pollTimeSupplier; - private final QuorumControllerMetrics controllerMetrics; - private final FaultHandler faultHandler; - private final QuorumFeatures quorumFeatures; - private final RecordRedactor recordRedactor; - /** - * A callback for when the migration state has been recovered from ZK. This is used to delay the installation of this - * MetadataPublisher with MetadataLoader. - */ - private final Consumer initialZkLoadHandler; - private final int minBatchSize; - private volatile MigrationDriverState migrationState; - private volatile ZkMigrationLeadershipState migrationLeadershipState; - private volatile MetadataImage image; - private volatile boolean firstPublish; - - // This is updated by the MetadataPublisher thread. When processing events in the migration driver thread, - // we should check if a newer leader has been seen by examining this variable. - private volatile LeaderAndEpoch curLeaderAndEpoch; - - KRaftMigrationDriver( - int nodeId, - ZkRecordConsumer zkRecordConsumer, - MigrationClient zkMigrationClient, - LegacyPropagator propagator, - Consumer initialZkLoadHandler, - FaultHandler faultHandler, - QuorumFeatures quorumFeatures, - KafkaConfigSchema configSchema, - QuorumControllerMetrics controllerMetrics, - int minBatchSize, - Time time - ) { - this.nodeId = nodeId; - this.zkRecordConsumer = zkRecordConsumer; - this.zkMigrationClient = zkMigrationClient; - this.propagator = propagator; - this.time = time; - LogContext logContext = new LogContext("[KRaftMigrationDriver id=" + nodeId + "] "); - this.controllerMetrics = controllerMetrics; - Logger log = logContext.logger(KRaftMigrationDriver.class); - this.log = log; - this.migrationState = MigrationDriverState.UNINITIALIZED; - this.migrationLeadershipState = ZkMigrationLeadershipState.EMPTY; - this.eventQueue = new KafkaEventQueue(Time.SYSTEM, logContext, "controller-" + nodeId + "-migration-driver-"); - this.pollTimeSupplier = new PollTimeSupplier(); - this.image = MetadataImage.EMPTY; - this.firstPublish = false; - this.initialZkLoadHandler = initialZkLoadHandler; - this.faultHandler = faultHandler; - this.quorumFeatures = quorumFeatures; - this.zkMetadataWriter = new KRaftMigrationZkWriter(zkMigrationClient, log::error); - this.recordRedactor = new RecordRedactor(configSchema); - this.minBatchSize = minBatchSize; - } - - public static Builder newBuilder() { - return new Builder(); - } - - public void start() { - eventQueue.prepend(new PollEvent()); - } - - // Visible for testing - public CompletableFuture migrationState() { - CompletableFuture stateFuture = new CompletableFuture<>(); - eventQueue.append(() -> stateFuture.complete(migrationState)); - return stateFuture; - } - - private boolean isControllerQuorumReadyForMigration() { - Optional notReadyMsg = this.quorumFeatures.reasonAllControllersZkMigrationNotReady( - image.features().metadataVersion(), image.cluster().controllers()); - if (notReadyMsg.isPresent()) { - log.warn("Still waiting for all controller nodes ready to begin the migration. Not ready due to:" + notReadyMsg.get()); - return false; - } - return true; - } - - private boolean imageDoesNotContainAllBrokers(MetadataImage image, Set brokerIds) { - for (BrokerRegistration broker : image.cluster().brokers().values()) { - if (broker.isMigratingZkBroker()) { - brokerIds.remove(broker.id()); - } - } - return !brokerIds.isEmpty(); - } - - private boolean areZkBrokersReadyForMigration() { - if (!firstPublish) { - log.info("Waiting for initial metadata publish before checking if Zk brokers are registered."); - return false; - } - - if (image.cluster().isEmpty()) { - // This primarily happens in system tests when we are starting a new ZK cluster and KRaft quorum - // around the same time. - log.info("No brokers are known to KRaft, waiting for brokers to register."); - return false; - } - - Set zkBrokerRegistrations = zkMigrationClient.readBrokerIds(); - if (zkBrokerRegistrations.isEmpty()) { - // Similar to the above empty check - log.info("No brokers are registered in ZK, waiting for brokers to register."); - return false; - } - - if (imageDoesNotContainAllBrokers(image, zkBrokerRegistrations)) { - log.info("Still waiting for ZK brokers {} to register with KRaft.", zkBrokerRegistrations); - return false; - } - - // Once all of those are found, check the topic assignments. This is much more expensive than listing /brokers - Set zkBrokersWithAssignments = new HashSet<>(); - zkMigrationClient.topicClient().iterateTopics( - EnumSet.of(TopicMigrationClient.TopicVisitorInterest.TOPICS), - (topicName, topicId, assignments) -> assignments.values().forEach(zkBrokersWithAssignments::addAll) - ); - - if (imageDoesNotContainAllBrokers(image, zkBrokersWithAssignments)) { - log.info("Still waiting for ZK brokers {} found in metadata to register with KRaft.", zkBrokersWithAssignments); - return false; - } - - return true; - } - - /** - * Apply a function which transforms our internal migration state. - * - * @param name A descriptive name of the function that is being applied - * @param migrationOp A function which performs some migration operations and possibly transforms our internal state - */ - - private void applyMigrationOperation(String name, KRaftMigrationOperation migrationOp) { - applyMigrationOperation(name, migrationOp, false); - } - - private void applyMigrationOperation(String name, KRaftMigrationOperation migrationOp, boolean alwaysLog) { - ZkMigrationLeadershipState beforeState = this.migrationLeadershipState; - long startTimeNs = time.nanoseconds(); - ZkMigrationLeadershipState afterState = migrationOp.apply(beforeState); - long durationNs = time.nanoseconds() - startTimeNs; - if (afterState.loggableChangeSinceState(beforeState) || alwaysLog) { - log.info("{} in {} ns. Transitioned migration state from {} to {}", - name, durationNs, beforeState, afterState); - } else if (afterState.equals(beforeState)) { - log.trace("{} in {} ns. Kept migration state as {}", name, durationNs, afterState); - } else { - log.trace("{} in {} ns. Transitioned migration state from {} to {}", - name, durationNs, beforeState, afterState); - } - this.migrationLeadershipState = afterState; - } - - private boolean isValidStateChange(MigrationDriverState newState) { - if (migrationState == newState) - return true; - - if (newState == MigrationDriverState.UNINITIALIZED) { - return false; - } - - switch (migrationState) { - case UNINITIALIZED: - case DUAL_WRITE: - return newState == MigrationDriverState.INACTIVE; - case INACTIVE: - return newState == MigrationDriverState.WAIT_FOR_CONTROLLER_QUORUM; - case WAIT_FOR_CONTROLLER_QUORUM: - return - newState == MigrationDriverState.INACTIVE || - newState == MigrationDriverState.BECOME_CONTROLLER || - newState == MigrationDriverState.WAIT_FOR_BROKERS; - case WAIT_FOR_BROKERS: - return - newState == MigrationDriverState.INACTIVE || - newState == MigrationDriverState.BECOME_CONTROLLER; - case BECOME_CONTROLLER: - return - newState == MigrationDriverState.INACTIVE || - newState == MigrationDriverState.ZK_MIGRATION || - newState == MigrationDriverState.SYNC_KRAFT_TO_ZK; - case ZK_MIGRATION: - return - newState == MigrationDriverState.INACTIVE || - newState == MigrationDriverState.SYNC_KRAFT_TO_ZK; - case SYNC_KRAFT_TO_ZK: - return - newState == MigrationDriverState.INACTIVE || - newState == MigrationDriverState.KRAFT_CONTROLLER_TO_BROKER_COMM; - case KRAFT_CONTROLLER_TO_BROKER_COMM: - return - newState == MigrationDriverState.INACTIVE || - newState == MigrationDriverState.DUAL_WRITE; - default: - log.error("Migration driver trying to transition from an unknown state {}", migrationState); - return false; - } - } - - /** - * Check that the migration driver is in the correct state for a given event. If the event causes - * updates (i.e., to ZK or broker RPCs), also check that the event is for the current KRaft controller epoch. - */ - private boolean checkDriverState(MigrationDriverState expectedState, MigrationEvent migrationEvent) { - if (migrationEvent instanceof MigrationWriteEvent) { - LeaderAndEpoch curLeaderAndEpoch = KRaftMigrationDriver.this.curLeaderAndEpoch; - LeaderAndEpoch eventLeaderAndEpoch = ((MigrationWriteEvent) migrationEvent).eventLeaderAndEpoch(); - if (!eventLeaderAndEpoch.equals(curLeaderAndEpoch)) { - log.info("Current leader epoch is {}, but event was created with epoch {}. Not running this event {}.", - curLeaderAndEpoch, eventLeaderAndEpoch, migrationEvent); - return false; - } - } - - if (!migrationState.equals(expectedState)) { - log.info("Expected driver state {} but found {}. Not running this event {}.", - expectedState, migrationState, migrationEvent); - return false; - } - - return true; - } - - // Visible for testing - void transitionTo(MigrationDriverState newState) { - if (!isValidStateChange(newState)) { - throw new IllegalStateException( - String.format("Invalid transition in migration driver from %s to %s", migrationState, newState)); - } - - if (newState != migrationState) { - log.info("{} transitioning from {} to {} state", nodeId, migrationState, newState); - pollTimeSupplier.reset(); - wakeup(); - } else { - log.trace("{} transitioning from {} to {} state", nodeId, migrationState, newState); - } - - migrationState = newState; - } - - private void wakeup() { - eventQueue.append(new PollEvent()); - } - - // MetadataPublisher methods - - @Override - public String name() { - return "KRaftMigrationDriver"; - } - - @Override - public void onControllerChange(LeaderAndEpoch newLeaderAndEpoch) { - curLeaderAndEpoch = newLeaderAndEpoch; - if (migrationState.equals(MigrationDriverState.UNINITIALIZED)) { - eventQueue.append(new RecoverMigrationStateFromZKEvent()); - } - eventQueue.append(new KRaftLeaderEvent(newLeaderAndEpoch)); - } - - @Override - public void onMetadataUpdate( - MetadataDelta delta, - MetadataImage newImage, - LoaderManifest manifest - ) { - enqueueMetadataChangeEvent(delta, - newImage, - manifest.provenance(), - manifest.type() == LoaderManifestType.SNAPSHOT, - NO_OP_HANDLER); - } - - @Override - public void close() throws InterruptedException { - eventQueue.beginShutdown("KRaftMigrationDriver#shutdown"); - log.debug("Shutting down KRaftMigrationDriver"); - eventQueue.close(); - } - - /** - * Construct and enqueue a {@link MetadataChangeEvent} with a given completion handler. In production use cases, - * this handler is a no-op. This method exists so that we can add additional logic in our unit tests to wait for the - * enqueued event to finish executing. - */ - void enqueueMetadataChangeEvent( - MetadataDelta delta, - MetadataImage newImage, - MetadataProvenance provenance, - boolean isSnapshot, - Consumer completionHandler - ) { - LeaderAndEpoch eventLeaderAndEpoch = KRaftMigrationDriver.this.curLeaderAndEpoch; - MetadataChangeEvent metadataChangeEvent = new MetadataChangeEvent( - delta, - newImage, - provenance, - isSnapshot, - completionHandler, - eventLeaderAndEpoch - ); - eventQueue.append(metadataChangeEvent); - } - - // Events handled by Migration Driver. - abstract class MigrationEvent implements EventQueue.Event { - @SuppressWarnings("ThrowableNotThrown") - @Override - public void handleException(Throwable e) { - if (e instanceof MigrationClientAuthException) { - KRaftMigrationDriver.this.faultHandler.handleFault("Encountered ZooKeeper authentication in " + this, e); - } else if (e instanceof MigrationClientException) { - log.info(String.format("Encountered ZooKeeper error during event %s. Will retry.", this), e.getCause()); - } else if (e instanceof RejectedExecutionException) { - log.debug("Not processing {} because the event queue is closed.", this); - } else { - KRaftMigrationDriver.this.faultHandler.handleFault("Unhandled error in " + this, e); - } - } - - @Override - public String toString() { - return this.getClass().getSimpleName(); - } - } - - /** - * An event that has some side effects like updating ZK or sending RPCs to brokers. - * These event should only run if they are for the current KRaft controller epoch. - * See {@link #checkDriverState(MigrationDriverState, MigrationEvent)} - */ - interface MigrationWriteEvent { - /** - * @return The LeaderAndEpoch as seen at the time of event creation. - */ - LeaderAndEpoch eventLeaderAndEpoch(); - } - - /** - * An event generated by a call to {@link MetadataPublisher#onControllerChange}. This will not be called until - * this class is registered with {@link org.apache.kafka.image.loader.MetadataLoader}. The registration happens - * after the migration state is loaded from ZooKeeper in {@link RecoverMigrationStateFromZKEvent}. - */ - class KRaftLeaderEvent extends MigrationEvent { - private final LeaderAndEpoch leaderAndEpoch; - - KRaftLeaderEvent(LeaderAndEpoch leaderAndEpoch) { - this.leaderAndEpoch = leaderAndEpoch; - } - - @Override - public void run() throws Exception { - // We can either be the active controller or just resigned from being the controller. - boolean isActive = leaderAndEpoch.isLeader(KRaftMigrationDriver.this.nodeId); - - if (!isActive) { - applyMigrationOperation("Became inactive migration driver", state -> - state.withNewKRaftController( - leaderAndEpoch.leaderId().orElse(ZkMigrationLeadershipState.EMPTY.kraftControllerId()), - leaderAndEpoch.epoch() - ).withUnknownZkController() - ); - transitionTo(MigrationDriverState.INACTIVE); - } else { - // Load the existing migration state and apply the new KRaft state - applyMigrationOperation("Became active migration driver", state -> { - ZkMigrationLeadershipState recoveredState = zkMigrationClient.getOrCreateMigrationRecoveryState(state); - return recoveredState.withNewKRaftController(nodeId, leaderAndEpoch.epoch()).withUnknownZkController(); - }); - - // Before becoming the controller fo ZkBrokers, we need to make sure the - // Controller Quorum can handle migration. - transitionTo(MigrationDriverState.WAIT_FOR_CONTROLLER_QUORUM); - } - } - } - - class MetadataChangeEvent extends MigrationEvent { - private final MetadataDelta delta; - private final MetadataImage image; - private final MetadataProvenance provenance; - private final boolean isSnapshot; - private final Consumer completionHandler; - private final LeaderAndEpoch leaderAndEpoch; - - MetadataChangeEvent( - MetadataDelta delta, - MetadataImage image, - MetadataProvenance provenance, - boolean isSnapshot, - Consumer completionHandler, - LeaderAndEpoch leaderAndEpoch - ) { - this.delta = delta; - this.image = image; - this.provenance = provenance; - this.isSnapshot = isSnapshot; - this.completionHandler = completionHandler; - this.leaderAndEpoch = leaderAndEpoch; - } - - @Override - public void run() throws Exception { - if (!firstPublish && image.isEmpty()) { - // KAFKA-15389 When first loading from an empty log, MetadataLoader can publish an empty image - log.debug("Encountered an empty MetadataImage while waiting for the first image to be published. " + - "Ignoring this image since it either does not include bootstrap records or it is a valid " + - "image for an older unsupported metadata version."); - completionHandler.accept(null); - return; - } - LeaderAndEpoch curLeaderAndEpoch = KRaftMigrationDriver.this.curLeaderAndEpoch; - KRaftMigrationDriver.this.firstPublish = true; - MetadataImage prevImage = KRaftMigrationDriver.this.image; - KRaftMigrationDriver.this.image = image; - String metadataType = isSnapshot ? "snapshot" : "delta"; - - if (EnumSet.of(MigrationDriverState.UNINITIALIZED, MigrationDriverState.INACTIVE).contains(migrationState)) { - // No need to log anything if this node is not the active controller or the driver has not initialized - completionHandler.accept(null); - return; - } - - if (!migrationState.allowDualWrite()) { - log.trace("Received metadata {}, but the controller is not in dual-write " + - "mode. Ignoring this metadata update.", metadataType); - completionHandler.accept(null); - // If the driver is active and dual-write is not yet enabled, then the migration has not yet begun. - // Only wake up the thread if the broker registrations have changed - if (delta.clusterDelta() != null) { - wakeup(); - } - return; - } - - if (!curLeaderAndEpoch.equals(leaderAndEpoch)) { - log.trace("Received metadata {} with {}, but the current leader and epoch is {}." + - "Ignoring this metadata update.", metadataType, leaderAndEpoch, curLeaderAndEpoch); - completionHandler.accept(null); - return; - } - - // Until the metadata has been migrated, the migrationLeadershipState offset is -1. We need to ignore - // metadata images until we see that the migration has happened and the image exceeds the offset of the - // migration - if (!migrationLeadershipState.initialZkMigrationComplete()) { - log.info("Ignoring {} {} since the migration has not finished.", metadataType, provenance); - completionHandler.accept(null); - return; - } - - // If the migration has finished, the migrationLeadershipState offset will be positive. Ignore any images - // which are older than the offset that has been written to ZK. - if (image.highestOffsetAndEpoch().compareTo(migrationLeadershipState.offsetAndEpoch()) < 0) { - log.info("Ignoring {} {} which contains metadata that has already been written to ZK.", metadataType, provenance); - completionHandler.accept(null); - return; - } - - Map dualWriteCounts = new TreeMap<>(); - long startTime = time.nanoseconds(); - if (isSnapshot) { - zkMetadataWriter.handleSnapshot(image, countingOperationConsumer( - dualWriteCounts, KRaftMigrationDriver.this::applyMigrationOperation)); - controllerMetrics.updateZkWriteSnapshotTimeMs(NANOSECONDS.toMillis(time.nanoseconds() - startTime)); - } else { - if (zkMetadataWriter.handleDelta(prevImage, image, delta, countingOperationConsumer( - dualWriteCounts, KRaftMigrationDriver.this::applyMigrationOperation))) { - // Only record delta write time if we changed something. Otherwise, no-op records will skew timings. - controllerMetrics.updateZkWriteDeltaTimeMs(NANOSECONDS.toMillis(time.nanoseconds() - startTime)); - } - } - if (dualWriteCounts.isEmpty()) { - log.trace("Did not make any ZK writes when handling KRaft {}", isSnapshot ? "snapshot" : "delta"); - } else { - log.debug("Made the following ZK writes when handling KRaft {}: {}", isSnapshot ? "snapshot" : "delta", dualWriteCounts); - } - - // Persist the offset of the metadata that was written to ZK - ZkMigrationLeadershipState zkStateAfterDualWrite = migrationLeadershipState.withKRaftMetadataOffsetAndEpoch( - image.highestOffsetAndEpoch().offset(), image.highestOffsetAndEpoch().epoch()); - //update the dual write offset metric - controllerMetrics.updateDualWriteOffset(image.highestOffsetAndEpoch().offset()); - - applyMigrationOperation("Updated ZK migration state after " + metadataType, - state -> zkMigrationClient.setMigrationRecoveryState(zkStateAfterDualWrite)); - - if (isSnapshot) { - // When we load a snapshot, need to send full metadata updates to the brokers - log.debug("Sending full metadata RPCs to brokers for snapshot."); - propagator.sendRPCsToBrokersFromMetadataImage(image, migrationLeadershipState.zkControllerEpoch()); - } else { - // delta - if (delta.topicsDelta() != null || delta.clusterDelta() != null) { - log.trace("Sending incremental metadata RPCs to brokers for delta."); - propagator.sendRPCsToBrokersFromMetadataDelta(delta, image, migrationLeadershipState.zkControllerEpoch()); - } else { - log.trace("Not sending RPCs to brokers for metadata {} since no relevant metadata has changed", metadataType); - } - } - - completionHandler.accept(null); - } - - @Override - public void handleException(Throwable e) { - completionHandler.accept(e); - super.handleException(e); - } - - @Override - public String toString() { - return "MetadataChangeEvent{" + - "provenance=" + provenance + - ", isSnapshot=" + isSnapshot + - '}'; - } - } - - class WaitForControllerQuorumEvent extends MigrationEvent { - - @Override - public void run() throws Exception { - if (checkDriverState(MigrationDriverState.WAIT_FOR_CONTROLLER_QUORUM, this)) { - if (!firstPublish) { - log.trace("Waiting until we have received metadata before proceeding with migration"); - return; - } - - ZkMigrationState zkMigrationState = image.features().zkMigrationState(); - switch (zkMigrationState) { - case NONE: - // This error message is used in zookeeper_migration_test.py::TestMigration.test_pre_migration_mode_3_4 - log.error("The controller's ZkMigrationState is NONE which means this cluster should not be migrated from ZooKeeper. " + - "This controller should not be configured with 'zookeeper.metadata.migration.enable' set to true. " + - "Will not proceed with a migration."); - transitionTo(MigrationDriverState.INACTIVE); - break; - case PRE_MIGRATION: - if (isControllerQuorumReadyForMigration()) { - // Base case when starting the migration - log.info("Controller Quorum is ready for Zk to KRaft migration. Now waiting for ZK brokers."); - transitionTo(MigrationDriverState.WAIT_FOR_BROKERS); - } - break; - case MIGRATION: - if (!migrationLeadershipState.initialZkMigrationComplete()) { - log.error("KRaft controller indicates an active migration, but the ZK state does not."); - transitionTo(MigrationDriverState.INACTIVE); - } else { - // Base case when rebooting a controller during migration - log.info("Migration is in already progress, not waiting on ZK brokers."); - transitionTo(MigrationDriverState.BECOME_CONTROLLER); - } - break; - case POST_MIGRATION: - log.error("KRaft controller indicates a completed migration, but the migration driver is somehow active."); - transitionTo(MigrationDriverState.INACTIVE); - break; - default: - throw new IllegalStateException("Unsupported ZkMigrationState " + zkMigrationState); - } - } - } - } - - class WaitForZkBrokersEvent extends MigrationEvent { - @Override - public void run() throws Exception { - if (checkDriverState(MigrationDriverState.WAIT_FOR_BROKERS, this)) { - if (areZkBrokersReadyForMigration()) { - log.info("Zk brokers are registered and ready for migration"); - transitionTo(MigrationDriverState.BECOME_CONTROLLER); - } - } - } - } - - class BecomeZkControllerEvent extends MigrationEvent implements MigrationWriteEvent { - private final LeaderAndEpoch leaderAndEpoch; - - BecomeZkControllerEvent(LeaderAndEpoch leaderAndEpoch) { - this.leaderAndEpoch = leaderAndEpoch; - } - - @Override - public void run() throws Exception { - // The leader epoch check in checkDriverState prevents us from getting stuck retrying this event after a - // new leader has been seen. - if (checkDriverState(MigrationDriverState.BECOME_CONTROLLER, this)) { - applyMigrationOperation("Claimed ZK controller leadership", zkMigrationClient::claimControllerLeadership, true); - if (migrationLeadershipState.zkControllerEpochZkVersion() == ZkMigrationLeadershipState.UNKNOWN_ZK_VERSION) { - log.info("Unable to claim leadership, will retry until we learn of a different KRaft leader"); - return; // Stay in BECOME_CONTROLLER state and retry - } - - // KAFKA-16171 and KAFKA-16667: Prior writing to /controller and /controller_epoch ZNodes above, - // the previous controller could have modified the /migration ZNode. Since ZK does grant us linearizability - // between writes and reads on different ZNodes, we need to write something to the /migration ZNode to - // ensure we have the latest /migration zkVersion. - applyMigrationOperation("Updated migration state", state -> { - // ZkVersion of -1 causes an unconditional update on /migration via KafkaZkClient#retryRequestsUntilConnected - state = state.withMigrationZkVersion(-1); - return zkMigrationClient.setMigrationRecoveryState(state); - }); - - if (!migrationLeadershipState.initialZkMigrationComplete()) { - transitionTo(MigrationDriverState.ZK_MIGRATION); - } else { - transitionTo(MigrationDriverState.SYNC_KRAFT_TO_ZK); - } - } - } - - @Override - public LeaderAndEpoch eventLeaderAndEpoch() { - return leaderAndEpoch; - } - } - - private BufferingBatchConsumer buildMigrationBatchConsumer( - MigrationManifest.Builder manifestBuilder - ) { - return new BufferingBatchConsumer<>(batch -> { - try { - if (log.isTraceEnabled()) { - batch.forEach(apiMessageAndVersion -> - log.trace(recordRedactor.toLoggableString(apiMessageAndVersion.message()))); - } - CompletableFuture future = zkRecordConsumer.acceptBatch(batch); - long batchStart = time.nanoseconds(); - FutureUtils.waitWithLogging(KRaftMigrationDriver.this.log, "", - "the metadata layer to commit " + batch.size() + " migration records", - future, Deadline.fromDelay(time, METADATA_COMMIT_MAX_WAIT_MS, TimeUnit.MILLISECONDS), time); - long batchEnd = time.nanoseconds(); - manifestBuilder.acceptBatch(batch, batchEnd - batchStart); - } catch (Throwable e) { - // This will cause readAllMetadata to throw since this batch consumer is called directly from readAllMetadata - throw new RuntimeException(e); - } - }, minBatchSize); - } - - class MigrateMetadataEvent extends MigrationEvent implements MigrationWriteEvent { - - private final LeaderAndEpoch leaderAndEpoch; - - MigrateMetadataEvent(LeaderAndEpoch leaderAndEpoch) { - this.leaderAndEpoch = leaderAndEpoch; - } - - @Override - public void run() throws Exception { - if (!checkDriverState(MigrationDriverState.ZK_MIGRATION, this)) { - return; - } - Set brokersInMetadata = new HashSet<>(); - log.info("Starting ZK migration"); - MigrationManifest.Builder manifestBuilder = MigrationManifest.newBuilder(time); - try { - FutureUtils.waitWithLogging(KRaftMigrationDriver.this.log, "", - "the metadata layer to begin the migration transaction", - zkRecordConsumer.beginMigration(), - Deadline.fromDelay(time, METADATA_COMMIT_MAX_WAIT_MS, TimeUnit.MILLISECONDS), time); - } catch (Throwable t) { - log.error("Could not start the migration", t); - super.handleException(t); - } - try { - BufferingBatchConsumer migrationBatchConsumer = buildMigrationBatchConsumer(manifestBuilder); - zkMigrationClient.readAllMetadata( - migrationBatchConsumer, - brokersInMetadata::add - ); - migrationBatchConsumer.flush(); - CompletableFuture completeMigrationFuture = zkRecordConsumer.completeMigration(); - OffsetAndEpoch offsetAndEpochAfterMigration = FutureUtils.waitWithLogging( - KRaftMigrationDriver.this.log, "", - "the metadata layer to complete the migration", - completeMigrationFuture, Deadline.fromDelay(time, METADATA_COMMIT_MAX_WAIT_MS, TimeUnit.MILLISECONDS), time); - MigrationManifest manifest = manifestBuilder.build(); - log.info("Completed migration of metadata from ZooKeeper to KRaft. {}. " + - "The current metadata offset is now {} with an epoch of {}. Saw {} brokers in the " + - "migrated metadata {}.", - manifest, - offsetAndEpochAfterMigration.offset(), - offsetAndEpochAfterMigration.epoch(), - brokersInMetadata.size(), - brokersInMetadata); - ZkMigrationLeadershipState newState = migrationLeadershipState.withKRaftMetadataOffsetAndEpoch( - offsetAndEpochAfterMigration.offset(), - offsetAndEpochAfterMigration.epoch()); - applyMigrationOperation( - "Finished initial migration of ZK metadata to KRaft", - state -> zkMigrationClient.setMigrationRecoveryState(newState), - true - ); - // Even though we just migrated everything, we still pass through the SYNC_KRAFT_TO_ZK state. This - // accomplishes two things: ensuring we have consistent metadata state between KRaft and ZK, and - // exercising the snapshot handling code in KRaftMigrationZkWriter. - transitionTo(MigrationDriverState.SYNC_KRAFT_TO_ZK); - } catch (Throwable t) { - MigrationManifest partialManifest = manifestBuilder.build(); - log.error("Aborting the metadata migration from ZooKeeper to KRaft. {}.", partialManifest, t); - zkRecordConsumer.abortMigration(); // This terminates the controller via fatal fault handler - super.handleException(t); - } - } - - @Override - public LeaderAndEpoch eventLeaderAndEpoch() { - return leaderAndEpoch; - } - } - - class SyncKRaftMetadataEvent extends MigrationEvent implements MigrationWriteEvent { - private final LeaderAndEpoch leaderAndEpoch; - - SyncKRaftMetadataEvent(LeaderAndEpoch leaderAndEpoch) { - this.leaderAndEpoch = leaderAndEpoch; - } - - @Override - public void run() throws Exception { - if (checkDriverState(MigrationDriverState.SYNC_KRAFT_TO_ZK, this)) { - // The migration offset will be non-negative at this point, so we just need to check that the image - // we have actually includes the migration metadata. - if (image.highestOffsetAndEpoch().compareTo(migrationLeadershipState.offsetAndEpoch()) < 0) { - log.info("Ignoring image {} which does not contain a superset of the metadata in ZK. Staying in " + - "SYNC_KRAFT_TO_ZK until a newer image is loaded", image.provenance()); - return; - } - log.info("Performing a full metadata sync from KRaft to ZK."); - Map dualWriteCounts = new TreeMap<>(); - long startTime = time.nanoseconds(); - zkMetadataWriter.handleSnapshot(image, countingOperationConsumer( - dualWriteCounts, KRaftMigrationDriver.this::applyMigrationOperation)); - long endTime = time.nanoseconds(); - controllerMetrics.updateZkWriteSnapshotTimeMs(NANOSECONDS.toMillis(startTime - endTime)); - if (dualWriteCounts.isEmpty()) { - log.info("Did not make any ZK writes when reconciling with KRaft state."); - } else { - log.info("Made the following ZK writes when reconciling with KRaft state: {}", dualWriteCounts); - } - transitionTo(MigrationDriverState.KRAFT_CONTROLLER_TO_BROKER_COMM); - } - } - - @Override - public LeaderAndEpoch eventLeaderAndEpoch() { - return leaderAndEpoch; - } - } - - class SendRPCsToBrokersEvent extends MigrationEvent implements MigrationWriteEvent { - - private final LeaderAndEpoch leaderAndEpoch; - - SendRPCsToBrokersEvent(LeaderAndEpoch leaderAndEpoch) { - this.leaderAndEpoch = leaderAndEpoch; - } - - @Override - public void run() throws Exception { - // Ignore sending RPCs to the brokers since we're no longer in the state. - if (checkDriverState(MigrationDriverState.KRAFT_CONTROLLER_TO_BROKER_COMM, this)) { - if (image.highestOffsetAndEpoch().compareTo(migrationLeadershipState.offsetAndEpoch()) >= 0) { - log.info("Sending RPCs to broker before moving to dual-write mode using " + - "at offset and epoch {}", image.highestOffsetAndEpoch()); - propagator.sendRPCsToBrokersFromMetadataImage(image, migrationLeadershipState.zkControllerEpoch()); - // Migration leadership state doesn't change since we're not doing any Zk writes. - transitionTo(MigrationDriverState.DUAL_WRITE); - } else { - log.info("Not sending metadata RPCs with current metadata image since does not contain the offset " + - "that was last written to ZK during the migration. Image offset {} is less than migration " + - "leadership state offset {}", image.highestOffsetAndEpoch(), migrationLeadershipState.offsetAndEpoch()); - } - } - } - - @Override - public LeaderAndEpoch eventLeaderAndEpoch() { - return leaderAndEpoch; - } - } - - class RecoverMigrationStateFromZKEvent extends MigrationEvent { - @Override - public void run() throws Exception { - if (checkDriverState(MigrationDriverState.UNINITIALIZED, this)) { - applyMigrationOperation("Recovered migration state from ZK", zkMigrationClient::getOrCreateMigrationRecoveryState); - String maybeDone = migrationLeadershipState.initialZkMigrationComplete() ? "done" : "not done"; - log.info("Initial migration of ZK metadata is {}.", maybeDone); - - // Once we've recovered the migration state from ZK, install this class as a metadata publisher - // by calling the initialZkLoadHandler. - initialZkLoadHandler.accept(KRaftMigrationDriver.this); - - // Transition to INACTIVE state and wait for leadership events. - transitionTo(MigrationDriverState.INACTIVE); - } - } - } - - class PollEvent extends MigrationEvent { - - @Override - public void run() throws Exception { - LeaderAndEpoch eventLeaderAndEpoch = KRaftMigrationDriver.this.curLeaderAndEpoch; - switch (migrationState) { - case UNINITIALIZED: - eventQueue.append(new RecoverMigrationStateFromZKEvent()); - break; - case INACTIVE: - // Nothing to do when the driver is inactive. We must wait until a KRaftLeaderEvent - // tells informs us that we are the leader. - break; - case WAIT_FOR_CONTROLLER_QUORUM: - eventQueue.append(new WaitForControllerQuorumEvent()); - break; - case WAIT_FOR_BROKERS: - eventQueue.append(new WaitForZkBrokersEvent()); - break; - case BECOME_CONTROLLER: - eventQueue.append(new BecomeZkControllerEvent(eventLeaderAndEpoch)); - break; - case ZK_MIGRATION: - eventQueue.append(new MigrateMetadataEvent(eventLeaderAndEpoch)); - break; - case SYNC_KRAFT_TO_ZK: - eventQueue.append(new SyncKRaftMetadataEvent(eventLeaderAndEpoch)); - break; - case KRAFT_CONTROLLER_TO_BROKER_COMM: - eventQueue.append(new SendRPCsToBrokersEvent(eventLeaderAndEpoch)); - break; - case DUAL_WRITE: - // Nothing to do in the PollEvent. If there's metadata change, we use - // MetadataChange event to drive the writes to Zookeeper. - break; - } - - // Poll again after some time - long deadline = time.nanoseconds() + NANOSECONDS.convert(pollTimeSupplier.nextPollTimeMs(), MILLISECONDS); - eventQueue.scheduleDeferred( - "poll", - new EventQueue.DeadlineFunction(deadline), - new PollEvent()); - } - } - - static KRaftMigrationOperationConsumer countingOperationConsumer( - Map dualWriteCounts, - BiConsumer operationConsumer - ) { - return (opType, logMsg, operation) -> { - dualWriteCounts.compute(opType, (key, value) -> { - if (value == null) { - return 1; - } else { - return value + 1; - } - }); - operationConsumer.accept(logMsg, operation); - }; - } - - public static class Builder { - private Integer nodeId; - private ZkRecordConsumer zkRecordConsumer; - private MigrationClient zkMigrationClient; - private LegacyPropagator propagator; - private Consumer initialZkLoadHandler; - private FaultHandler faultHandler; - private QuorumFeatures quorumFeatures; - private KafkaConfigSchema configSchema; - private QuorumControllerMetrics controllerMetrics; - private Integer minBatchSize; - private Time time; - - public Builder setNodeId(int nodeId) { - this.nodeId = nodeId; - return this; - } - - public Builder setZkRecordConsumer(ZkRecordConsumer zkRecordConsumer) { - this.zkRecordConsumer = zkRecordConsumer; - return this; - } - - public Builder setZkMigrationClient(MigrationClient zkMigrationClient) { - this.zkMigrationClient = zkMigrationClient; - return this; - } - - public Builder setPropagator(LegacyPropagator propagator) { - this.propagator = propagator; - return this; - } - - public Builder setInitialZkLoadHandler(Consumer initialZkLoadHandler) { - this.initialZkLoadHandler = initialZkLoadHandler; - return this; - } - - public Builder setFaultHandler(FaultHandler faultHandler) { - this.faultHandler = faultHandler; - return this; - } - - public Builder setQuorumFeatures(QuorumFeatures quorumFeatures) { - this.quorumFeatures = quorumFeatures; - return this; - } - - public Builder setConfigSchema(KafkaConfigSchema configSchema) { - this.configSchema = configSchema; - return this; - } - - public Builder setControllerMetrics(QuorumControllerMetrics controllerMetrics) { - this.controllerMetrics = controllerMetrics; - return this; - } - - public Builder setTime(Time time) { - this.time = time; - return this; - } - - public Builder setMinMigrationBatchSize(int minBatchSize) { - this.minBatchSize = minBatchSize; - return this; - } - - public KRaftMigrationDriver build() { - if (nodeId == null) { - throw new IllegalStateException("You must specify the node ID of this controller."); - } - if (zkRecordConsumer == null) { - throw new IllegalStateException("You must specify the ZkRecordConsumer."); - } - if (zkMigrationClient == null) { - throw new IllegalStateException("You must specify the MigrationClient."); - } - if (propagator == null) { - throw new IllegalStateException("You must specify the MetadataPropagator."); - } - if (initialZkLoadHandler == null) { - throw new IllegalStateException("You must specify the initial ZK load callback."); - } - if (faultHandler == null) { - throw new IllegalStateException("You must specify the FaultHandler."); - } - if (configSchema == null) { - throw new IllegalStateException("You must specify the KafkaConfigSchema."); - } - if (controllerMetrics == null) { - throw new IllegalStateException("You must specify the QuorumControllerMetrics."); - } - if (time == null) { - throw new IllegalStateException("You must specify the Time."); - } - if (minBatchSize == null) { - minBatchSize = 200; - } - return new KRaftMigrationDriver( - nodeId, - zkRecordConsumer, - zkMigrationClient, - propagator, - initialZkLoadHandler, - faultHandler, - quorumFeatures, - configSchema, - controllerMetrics, - minBatchSize, - time - ); - } - } -} diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationOperation.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationOperation.java deleted file mode 100644 index 55764c4cd1c..00000000000 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationOperation.java +++ /dev/null @@ -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); -} diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationOperationConsumer.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationOperationConsumer.java deleted file mode 100644 index c7f322b6ea8..00000000000 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationOperationConsumer.java +++ /dev/null @@ -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); -} diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationZkWriter.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationZkWriter.java deleted file mode 100644 index 702889df48b..00000000000 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationZkWriter.java +++ /dev/null @@ -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 errorLogger; - - public KRaftMigrationZkWriter( - MigrationClient migrationClient, - Consumer 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 deletedTopics = new HashMap<>(); - Set topicsInZk = new HashSet<>(); - Set newTopics = new HashSet<>(topicsImage.topicsById().keySet()); - Set changedTopics = new HashSet<>(); - Map> partitionsInZk = new HashMap<>(); - Map> extraneousPartitionsInZk = new HashMap<>(); - Map> changedPartitions = new HashMap<>(); - Map> newPartitions = new HashMap<>(); - - Set 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> 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 topicPartitionsInZk = partitionsInZk.computeIfAbsent(topicId, __ -> new HashSet<>()); - if (!topicPartitionsInZk.equals(topic.partitions().keySet())) { - Map 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 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 newPartitions = new HashMap<>(topicDelta.newPartitions()); - Map 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 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 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 resourcesToUpdate = new HashSet<>(); - BiConsumer> processConfigsForResource = (ConfigResource resource, Map configs) -> { - newResources.remove(resource); - Map 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 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 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 getScramCredentialStringsForUser(ScramImage image, String userName) { - Map 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 changedNonUserEntities = new HashSet<>(); - Set changedUsers = new HashSet<>(); - - if (clientQuotasImage != null) { - for (Entry 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> mechanismEntry : scramImage.mechanisms().entrySet()) { - for (Entry userEntry : mechanismEntry.getValue().entrySet()) { - changedUsers.add(userEntry.getKey()); - } - } - } - migrationClient.configClient().iterateClientQuotas(new ConfigMigrationClient.ClientQuotaVisitor() { - @Override - public void visitClientQuota(List entityDataList, Map quotas) { - Map 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 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 quotaMap = clientQuotasImage.entities(). - getOrDefault(entity, ClientQuotaImage.EMPTY).quotaMap(); - Map 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 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 updatedResources = configsDelta.changes().keySet(); - updatedResources.forEach(configResource -> { - Map 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 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 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 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 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> 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 newResources = new HashSet<>(allAclsInSnapshot.keySet()); - Set resourcesToDelete = new HashSet<>(); - Map> changedResources = new HashMap<>(); - migrationClient.aclClient().iterateAcls((resourcePattern, accessControlEntries) -> { - newResources.remove(resourcePattern); - if (!allAclsInSnapshot.containsKey(resourcePattern)) { - resourcesToDelete.add(resourcePattern); - } else { - Set 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 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> 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 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 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 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)); - } - }); - } -} diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/LegacyPropagator.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/LegacyPropagator.java deleted file mode 100644 index c2bd4e6f3d0..00000000000 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/LegacyPropagator.java +++ /dev/null @@ -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(); -} diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationClient.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationClient.java deleted file mode 100644 index e7ffda47add..00000000000 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationClient.java +++ /dev/null @@ -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 readProducerId(); - - ZkMigrationLeadershipState writeProducerId( - long nextProducerId, - ZkMigrationLeadershipState state - ); - - void readAllMetadata(Consumer> batchConsumer, Consumer brokerIdConsumer); - - Set readBrokerIds(); -} diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationClientAuthException.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationClientAuthException.java deleted file mode 100644 index a10b8ae3f49..00000000000 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationClientAuthException.java +++ /dev/null @@ -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); - } -} diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationClientException.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationClientException.java deleted file mode 100644 index 6c816423761..00000000000 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationClientException.java +++ /dev/null @@ -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); - } -} diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationDriverState.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationDriverState.java deleted file mode 100644 index 07d318d1209..00000000000 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationDriverState.java +++ /dev/null @@ -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; - } -} diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationManifest.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationManifest.java deleted file mode 100644 index 39dae38d52c..00000000000 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationManifest.java +++ /dev/null @@ -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 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 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 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 recordTypeCounts; - - MigrationManifest( - int totalRecords, - int totalBatches, - long totalBatchDurationsNs, - long durationNanos, - Map 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); - } -} \ No newline at end of file diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/TopicMigrationClient.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/TopicMigrationClient.java deleted file mode 100644 index 5eafd72b298..00000000000 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/TopicMigrationClient.java +++ /dev/null @@ -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> assignments); - default void visitPartition(TopicIdPartition topicIdPartition, PartitionRegistration partitionRegistration) { - - } - } - - void iterateTopics(EnumSet interests, TopicVisitor visitor); - - Set readPendingTopicDeletions(); - - ZkMigrationLeadershipState clearPendingTopicDeletions( - Set pendingTopicDeletions, - ZkMigrationLeadershipState state - ); - - ZkMigrationLeadershipState deleteTopic( - String topicName, - ZkMigrationLeadershipState state - ); - - ZkMigrationLeadershipState createTopic( - String topicName, - Uuid topicId, - Map topicPartitions, - ZkMigrationLeadershipState state - ); - - ZkMigrationLeadershipState updateTopic( - String topicName, - Uuid topicId, - Map topicPartitions, - ZkMigrationLeadershipState state - ); - - ZkMigrationLeadershipState createTopicPartitions( - Map> topicPartitions, - ZkMigrationLeadershipState state - ); - - ZkMigrationLeadershipState updateTopicPartitions( - Map> topicPartitions, - ZkMigrationLeadershipState state - ); - - ZkMigrationLeadershipState deleteTopicPartitions( - Map> topicPartitions, - ZkMigrationLeadershipState state - ); -} diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/ZkRecordConsumer.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/ZkRecordConsumer.java deleted file mode 100644 index 910a58bec79..00000000000 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/ZkRecordConsumer.java +++ /dev/null @@ -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 recordBatch); - CompletableFuture completeMigration(); - void abortMigration(); -} diff --git a/metadata/src/test/java/org/apache/kafka/controller/ActivationRecordsGeneratorTest.java b/metadata/src/test/java/org/apache/kafka/controller/ActivationRecordsGeneratorTest.java index 614908a3daf..5f948a79885 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ActivationRecordsGeneratorTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ActivationRecordsGeneratorTest.java @@ -17,15 +17,12 @@ package org.apache.kafka.controller; -import org.apache.kafka.common.metadata.ZkMigrationStateRecord; import org.apache.kafka.metadata.bootstrap.BootstrapMetadata; import org.apache.kafka.metadata.migration.ZkMigrationState; import org.apache.kafka.server.common.MetadataVersion; import org.junit.jupiter.api.Test; -import java.util.Optional; - import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -45,7 +42,6 @@ public class ActivationRecordsGeneratorTest { logMsg -> assertEquals("Performing controller activation. The metadata log appears to be empty. " + "Appending 1 bootstrap record(s) at metadata.version 3.0-IV1 from bootstrap source 'test'.", logMsg), -1L, - false, BootstrapMetadata.fromVersion(MetadataVersion.MINIMUM_BOOTSTRAP_VERSION, "test"), MetadataVersion.MINIMUM_KRAFT_VERSION ); @@ -57,7 +53,6 @@ public class ActivationRecordsGeneratorTest { "Appending 1 bootstrap record(s) at metadata.version 3.4-IV0 from bootstrap " + "source 'test'. Setting the ZK migration state to NONE since this is a de-novo KRaft cluster.", logMsg), -1L, - false, BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_4_IV0, "test"), MetadataVersion.IBP_3_4_IV0 ); @@ -65,50 +60,11 @@ public class ActivationRecordsGeneratorTest { assertEquals(2, result.records().size()); - result = ActivationRecordsGenerator.recordsForEmptyLog( - logMsg -> assertEquals("Performing controller activation. The metadata log appears to be empty. " + - "Appending 1 bootstrap record(s) at metadata.version 3.4-IV0 from bootstrap " + - "source 'test'. Putting the controller into pre-migration mode. No metadata updates will be allowed " + - "until the ZK metadata has been migrated.", logMsg), - -1L, - true, - BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_4_IV0, "test"), - MetadataVersion.IBP_3_4_IV0 - ); - assertTrue(result.isAtomic()); - assertEquals(2, result.records().size()); - - assertEquals( - "The bootstrap metadata.version 3.3-IV2 does not support ZK migrations. Cannot continue with ZK migrations enabled.", - assertThrows(RuntimeException.class, () -> - ActivationRecordsGenerator.recordsForEmptyLog( - logMsg -> fail(), - -1L, - true, - BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_3_IV2, "test"), - MetadataVersion.IBP_3_3_IV2 - )).getMessage() - ); - result = ActivationRecordsGenerator.recordsForEmptyLog( logMsg -> assertEquals("Performing controller activation. The metadata log appears to be empty. " + "Appending 1 bootstrap record(s) in metadata transaction at metadata.version 3.6-IV1 from bootstrap " + "source 'test'. Setting the ZK migration state to NONE since this is a de-novo KRaft cluster.", logMsg), -1L, - false, - BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_6_IV1, "test"), - MetadataVersion.IBP_3_6_IV1 - ); - assertFalse(result.isAtomic()); - assertEquals(4, result.records().size()); - - result = ActivationRecordsGenerator.recordsForEmptyLog( - logMsg -> assertEquals("Performing controller activation. The metadata log appears to be empty. " + - "Appending 1 bootstrap record(s) in metadata transaction at metadata.version 3.6-IV1 from bootstrap " + - "source 'test'. Putting the controller into pre-migration mode. No metadata updates will be allowed " + - "until the ZK metadata has been migrated.", logMsg), - -1L, - true, BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_6_IV1, "test"), MetadataVersion.IBP_3_6_IV1 ); @@ -121,49 +77,11 @@ public class ActivationRecordsGeneratorTest { "metadata.version 3.6-IV1 from bootstrap source 'test'. Setting the ZK migration state to NONE " + "since this is a de-novo KRaft cluster.", logMsg), 0L, - false, BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_6_IV1, "test"), MetadataVersion.IBP_3_6_IV1 ); assertFalse(result.isAtomic()); assertEquals(5, result.records().size()); - - result = ActivationRecordsGenerator.recordsForEmptyLog( - logMsg -> assertEquals("Performing controller activation. Aborting partial bootstrap records " + - "transaction at offset 0. Re-appending 1 bootstrap record(s) in new metadata transaction at " + - "metadata.version 3.6-IV1 from bootstrap source 'test'. Putting the controller into pre-migration " + - "mode. No metadata updates will be allowed until the ZK metadata has been migrated.", logMsg), - 0L, - true, - BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_6_IV1, "test"), - MetadataVersion.IBP_3_6_IV1 - ); - assertFalse(result.isAtomic()); - assertEquals(5, result.records().size()); - - assertEquals( - "Detected partial bootstrap records transaction at 0, but the metadata.version 3.6-IV0 does not " + - "support transactions. Cannot continue.", - assertThrows(RuntimeException.class, () -> - ActivationRecordsGenerator.recordsForEmptyLog( - logMsg -> assertEquals("", logMsg), - 0L, - true, - BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_6_IV0, "test"), - MetadataVersion.IBP_3_6_IV0 - )).getMessage() - ); - } - - FeatureControlManager buildFeatureControl( - MetadataVersion metadataVersion, - Optional zkMigrationState - ) { - FeatureControlManager featureControl = new FeatureControlManager.Builder() - .setMetadataVersion(metadataVersion).build(); - zkMigrationState.ifPresent(migrationState -> - featureControl.replay((ZkMigrationStateRecord) migrationState.toRecord().message())); - return featureControl; } @Test @@ -174,8 +92,7 @@ public class ActivationRecordsGeneratorTest { logMsg -> assertEquals("Performing controller activation. No metadata.version feature level " + "record was found in the log. Treating the log as version 3.0-IV1.", logMsg), -1L, - false, - buildFeatureControl(MetadataVersion.MINIMUM_KRAFT_VERSION, Optional.empty()), + ZkMigrationState.NONE, MetadataVersion.MINIMUM_KRAFT_VERSION ); assertTrue(result.isAtomic()); @@ -184,8 +101,7 @@ public class ActivationRecordsGeneratorTest { result = ActivationRecordsGenerator.recordsForNonEmptyLog( logMsg -> assertEquals("Performing controller activation.", logMsg), -1L, - false, - buildFeatureControl(MetadataVersion.IBP_3_3_IV0, Optional.empty()), + ZkMigrationState.NONE, MetadataVersion.IBP_3_3_IV0 ); assertTrue(result.isAtomic()); @@ -195,8 +111,7 @@ public class ActivationRecordsGeneratorTest { logMsg -> assertEquals("Performing controller activation. Loaded ZK migration state of NONE. " + "This is expected because this is a de-novo KRaft cluster.", logMsg), -1L, - false, - buildFeatureControl(MetadataVersion.IBP_3_4_IV0, Optional.empty()), + ZkMigrationState.NONE, MetadataVersion.IBP_3_4_IV0 ); assertTrue(result.isAtomic()); @@ -207,8 +122,7 @@ public class ActivationRecordsGeneratorTest { "transaction at offset 42. Loaded ZK migration state of NONE. " + "This is expected because this is a de-novo KRaft cluster.", logMsg), 42L, - false, - buildFeatureControl(MetadataVersion.IBP_3_6_IV1, Optional.empty()), + ZkMigrationState.NONE, MetadataVersion.IBP_3_6_IV1 ); assertTrue(result.isAtomic()); @@ -221,8 +135,7 @@ public class ActivationRecordsGeneratorTest { ActivationRecordsGenerator.recordsForNonEmptyLog( logMsg -> fail(), 42L, - false, - buildFeatureControl(MetadataVersion.IBP_3_6_IV0, Optional.empty()), + ZkMigrationState.NONE, MetadataVersion.IBP_3_6_IV0 )).getMessage() ); @@ -230,97 +143,35 @@ public class ActivationRecordsGeneratorTest { @Test public void testActivationMessageForNonEmptyLogWithMigrations() { - ControllerResult result; - assertEquals( - "Should not have ZK migrations enabled on a cluster running metadata.version 3.3-IV0", + "Should not have ZkMigrationState.MIGRATION on a cluster running metadata version 3.3-IV0.", assertThrows(RuntimeException.class, () -> ActivationRecordsGenerator.recordsForNonEmptyLog( logMsg -> fail(), -1L, - true, - buildFeatureControl(MetadataVersion.IBP_3_3_IV0, Optional.empty()), + ZkMigrationState.MIGRATION, MetadataVersion.IBP_3_3_IV0 )).getMessage() ); assertEquals( - "Should not have ZK migrations enabled on a cluster that was created in KRaft mode.", + "Cannot load ZkMigrationState.MIGRATION because ZK migration is no longer supported.", assertThrows(RuntimeException.class, () -> ActivationRecordsGenerator.recordsForNonEmptyLog( logMsg -> fail(), -1L, - true, - buildFeatureControl(MetadataVersion.IBP_3_4_IV0, Optional.empty()), - MetadataVersion.IBP_3_4_IV0 + ZkMigrationState.MIGRATION, + MetadataVersion.IBP_3_9_IV0 ) ).getMessage() ); - result = ActivationRecordsGenerator.recordsForNonEmptyLog( - logMsg -> assertEquals("Performing controller activation. Loaded ZK migration state of " + - "PRE_MIGRATION. Activating pre-migration controller without empty log. There may be a partial " + - "migration.", logMsg), - -1L, - true, - buildFeatureControl(MetadataVersion.IBP_3_4_IV0, Optional.of(ZkMigrationState.PRE_MIGRATION)), - MetadataVersion.IBP_3_4_IV0 - ); - assertTrue(result.isAtomic()); - assertEquals(0, result.records().size()); - - result = ActivationRecordsGenerator.recordsForNonEmptyLog( - logMsg -> assertEquals("Performing controller activation. Loaded ZK migration state of " + - "PRE_MIGRATION.", logMsg), - -1L, - true, - buildFeatureControl(MetadataVersion.IBP_3_6_IV1, Optional.of(ZkMigrationState.PRE_MIGRATION)), - MetadataVersion.IBP_3_6_IV1 - ); - assertTrue(result.isAtomic()); - assertEquals(0, result.records().size()); - - result = ActivationRecordsGenerator.recordsForNonEmptyLog( - logMsg -> assertEquals("Performing controller activation. Loaded ZK migration state of MIGRATION. " + - "Staying in ZK migration mode since 'zookeeper.metadata.migration.enable' is still 'true'.", logMsg), - -1L, - true, - buildFeatureControl(MetadataVersion.IBP_3_4_IV0, Optional.of(ZkMigrationState.MIGRATION)), - MetadataVersion.IBP_3_4_IV0 - ); - assertTrue(result.isAtomic()); - assertEquals(0, result.records().size()); - - result = ActivationRecordsGenerator.recordsForNonEmptyLog( - logMsg -> assertEquals("Performing controller activation. Loaded ZK migration state of MIGRATION. " + - "Completing the ZK migration since this controller was configured with " + - "'zookeeper.metadata.migration.enable' set to 'false'.", logMsg), - -1L, - false, - buildFeatureControl(MetadataVersion.IBP_3_4_IV0, Optional.of(ZkMigrationState.MIGRATION)), - MetadataVersion.IBP_3_4_IV0 - ); - assertTrue(result.isAtomic()); - assertEquals(1, result.records().size()); - - result = ActivationRecordsGenerator.recordsForNonEmptyLog( - logMsg -> assertEquals("Performing controller activation. Aborting in-progress metadata " + - "transaction at offset 42. Loaded ZK migration state of MIGRATION. Completing the ZK migration " + - "since this controller was configured with 'zookeeper.metadata.migration.enable' set to 'false'.", logMsg), - 42L, - false, - buildFeatureControl(MetadataVersion.IBP_3_6_IV1, Optional.of(ZkMigrationState.MIGRATION)), - MetadataVersion.IBP_3_6_IV1 - ); - assertTrue(result.isAtomic()); - assertEquals(2, result.records().size()); - + ControllerResult result; result = ActivationRecordsGenerator.recordsForNonEmptyLog( logMsg -> assertEquals("Performing controller activation. Loaded ZK migration state of " + "POST_MIGRATION.", logMsg), -1L, - false, - buildFeatureControl(MetadataVersion.IBP_3_4_IV0, Optional.of(ZkMigrationState.POST_MIGRATION)), + ZkMigrationState.POST_MIGRATION, MetadataVersion.IBP_3_4_IV0 ); assertTrue(result.isAtomic()); @@ -330,32 +181,7 @@ public class ActivationRecordsGeneratorTest { logMsg -> assertEquals("Performing controller activation. Aborting in-progress metadata " + "transaction at offset 42. Loaded ZK migration state of POST_MIGRATION.", logMsg), 42L, - false, - buildFeatureControl(MetadataVersion.IBP_3_6_IV1, Optional.of(ZkMigrationState.POST_MIGRATION)), - MetadataVersion.IBP_3_6_IV1 - ); - assertTrue(result.isAtomic()); - assertEquals(1, result.records().size()); - - result = ActivationRecordsGenerator.recordsForNonEmptyLog( - logMsg -> assertEquals("Performing controller activation. Loaded ZK migration state of " + - "POST_MIGRATION. Ignoring 'zookeeper.metadata.migration.enable' value of 'true' since the " + - "ZK migration has been completed.", logMsg), - -1L, - true, - buildFeatureControl(MetadataVersion.IBP_3_4_IV0, Optional.of(ZkMigrationState.POST_MIGRATION)), - MetadataVersion.IBP_3_4_IV0 - ); - assertTrue(result.isAtomic()); - assertEquals(0, result.records().size()); - - result = ActivationRecordsGenerator.recordsForNonEmptyLog( - logMsg -> assertEquals("Performing controller activation. Aborting in-progress metadata " + - "transaction at offset 42. Loaded ZK migration state of POST_MIGRATION. Ignoring " + - "'zookeeper.metadata.migration.enable' value of 'true' since the ZK migration has been completed.", logMsg), - 42L, - true, - buildFeatureControl(MetadataVersion.IBP_3_6_IV1, Optional.of(ZkMigrationState.POST_MIGRATION)), + ZkMigrationState.POST_MIGRATION, MetadataVersion.IBP_3_6_IV1 ); assertTrue(result.isAtomic()); diff --git a/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java index 786891e695b..0646d4aaa56 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java @@ -852,33 +852,4 @@ public class ClusterControlManagerTest { clusterControl.brokerRegistrations().get(1).epoch()); } } - - @Test - public void testRegistrationWithIncorrectInterBrokerListenerName() { - ClusterControlManager clusterControl = new ClusterControlManager.Builder(). - setClusterId("pjvUwj3ZTEeSVQmUiH3IJw"). - setFeatureControlManager(new FeatureControlManager.Builder().build()). - setBrokerUncleanShutdownHandler((brokerId, records) -> { }). - setInterBrokerListenerName("INTERNAL"). - setZkMigrationEnabled(true). - build(); - clusterControl.activate(); - assertEquals("Broker does not have the current inter.broker.listener INTERNAL", - assertThrows(InvalidRegistrationException.class, - () -> clusterControl.registerBroker( - new BrokerRegistrationRequestData(). - setBrokerId(1). - setClusterId(clusterControl.clusterId()). - setIncarnationId(Uuid.fromString("07OOcU7MQFeSmGAFPP2Zww")). - setLogDirs(Arrays.asList(Uuid.fromString("Vv1gzkM2QpuE-PPrIc6XEw"))). - setIsMigratingZkBroker(true). - setListeners(new BrokerRegistrationRequestData.ListenerCollection(Collections.singleton( - new BrokerRegistrationRequestData.Listener(). - setName("PLAINTEXT"). - setHost("example.com"). - setPort(9092). - setSecurityProtocol(SecurityProtocol.PLAINTEXT.id)).iterator())), - 111, - new FinalizedControllerFeatures(Collections.emptyMap(), 100L))).getMessage()); - } } diff --git a/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java b/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java index f7f4cadb3c4..18bf8c80afd 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java @@ -55,7 +55,6 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.junit.jupiter.params.provider.Arguments.arguments; @Timeout(value = 40) @@ -338,22 +337,6 @@ public class PartitionChangeBuilderTest { 1); } - /** - * Test that shrinking the ISR does increase the leader epoch in later MVs when ZK migration is on. - */ - @ParameterizedTest - @ValueSource(strings = {"3.6-IV0", "3.7-IV2", "4.0-IV0"}) - public void testLeaderEpochBumpOnIsrShrinkWithZkMigration(String metadataVersionString) { - MetadataVersion metadataVersion = MetadataVersion.fromVersionString(metadataVersionString); - testTriggerLeaderEpochBumpIfNeeded( - createFooBuilder(metadataVersion). - setZkMigrationEnabled(true). - setTargetIsrWithBrokerStates( - AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(Arrays.asList(2, 1))), - new PartitionChangeRecord(), - 1); - } - /** * Test that expanding the ISR doesn't increase the leader epoch. */ @@ -368,22 +351,6 @@ public class PartitionChangeBuilderTest { NO_LEADER_CHANGE); } - /** - * Test that expanding the ISR doesn't increase the leader epoch during ZK migration. - */ - @ParameterizedTest - @ValueSource(strings = {"3.4-IV0", "3.5-IV2", "3.6-IV0", "3.7-IV2", "4.0-IV0"}) - public void testNoLeaderEpochBumpOnIsrExpansionDuringMigration(String metadataVersionString) { - MetadataVersion metadataVersion = MetadataVersion.fromVersionString(metadataVersionString); - testTriggerLeaderEpochBumpIfNeeded( - createFooBuilder(metadataVersion). - setZkMigrationEnabled(true). - setTargetIsrWithBrokerStates( - AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(Arrays.asList(2, 1, 3, 4))), - new PartitionChangeRecord(), - NO_LEADER_CHANGE); - } - /** * Test that changing the replica set such that not all the old replicas remain * always results in a leader epoch increase. @@ -631,18 +598,9 @@ public class PartitionChangeBuilderTest { ); } - private static Stream leaderRecoveryAndZkMigrationParams() { - return Stream.of( - arguments(true, true), - arguments(true, false), - arguments(false, true), - arguments(false, false) - ); - } - @ParameterizedTest - @MethodSource("leaderRecoveryAndZkMigrationParams") - public void testChangeInLeadershipDoesNotChangeRecoveryState(boolean isLeaderRecoverySupported, boolean zkMigrationsEnabled) { + @ValueSource(booleans = {true, false}) + public void testChangeInLeadershipDoesNotChangeRecoveryState(boolean isLeaderRecoverySupported) { final byte noChange = (byte) -1; int leaderId = 1; LeaderRecoveryState recoveryState = LeaderRecoveryState.RECOVERING; @@ -671,7 +629,6 @@ public class PartitionChangeBuilderTest { metadataVersion, 2 ); - offlineBuilder.setZkMigrationEnabled(zkMigrationsEnabled); // Set the target ISR to empty to indicate that the last leader is offline offlineBuilder.setTargetIsrWithBrokerStates(Collections.emptyList()); @@ -698,7 +655,6 @@ public class PartitionChangeBuilderTest { metadataVersion, 2 ); - onlineBuilder.setZkMigrationEnabled(zkMigrationsEnabled); // The only broker in the ISR is elected leader and stays in the recovering changeRecord = (PartitionChangeRecord) onlineBuilder.build().get().message(); @@ -712,8 +668,8 @@ public class PartitionChangeBuilderTest { } @ParameterizedTest - @MethodSource("leaderRecoveryAndZkMigrationParams") - void testUncleanSetsLeaderRecoveringState(boolean isLeaderRecoverySupported, boolean zkMigrationsEnabled) { + @ValueSource(booleans = {true, false}) + void testUncleanSetsLeaderRecoveringState(boolean isLeaderRecoverySupported) { final byte noChange = (byte) -1; int leaderId = 1; PartitionRegistration registration = new PartitionRegistration.Builder(). @@ -741,7 +697,6 @@ public class PartitionChangeBuilderTest { metadataVersion, 2 ).setElection(Election.UNCLEAN); - onlineBuilder.setZkMigrationEnabled(zkMigrationsEnabled); // The partition should stay as recovering PartitionChangeRecord changeRecord = (PartitionChangeRecord) onlineBuilder .build() diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerMetricsIntegrationTest.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerMetricsIntegrationTest.java index 08ac06d7209..68fa088f44c 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerMetricsIntegrationTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerMetricsIntegrationTest.java @@ -57,7 +57,7 @@ public class QuorumControllerMetricsIntegrationTest { final AtomicBoolean closed = new AtomicBoolean(false); MockControllerMetrics() { - super(Optional.empty(), Time.SYSTEM, true); + super(Optional.empty(), Time.SYSTEM); } @Override diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java index 0dcf8f1ccd0..a8985d880bc 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java @@ -100,7 +100,6 @@ import org.apache.kafka.metadata.RecordTestUtils.TestThroughAllIntermediateImage import org.apache.kafka.metadata.authorizer.StandardAuthorizer; import org.apache.kafka.metadata.bootstrap.BootstrapMetadata; import org.apache.kafka.metadata.migration.ZkMigrationState; -import org.apache.kafka.metadata.migration.ZkRecordConsumer; import org.apache.kafka.metadata.util.BatchFileWriter; import org.apache.kafka.metalog.LocalLogManager; import org.apache.kafka.metalog.LocalLogManagerTestEnv; @@ -111,7 +110,6 @@ import org.apache.kafka.server.common.Features; import org.apache.kafka.server.common.KRaftVersion; import org.apache.kafka.server.common.MetadataVersion; import org.apache.kafka.server.common.TopicIdPartition; -import org.apache.kafka.server.fault.FaultHandlerException; import org.apache.kafka.snapshot.FileRawSnapshotReader; import org.apache.kafka.snapshot.Snapshots; import org.apache.kafka.test.TestUtils; @@ -121,14 +119,11 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.CsvSource; -import org.junit.jupiter.params.provider.EnumSource; -import org.junit.jupiter.params.provider.ValueSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.File; import java.nio.file.Path; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -162,7 +157,6 @@ import static org.apache.kafka.controller.ConfigurationControlManagerTest.entry; import static org.apache.kafka.controller.ControllerRequestContextUtil.ANONYMOUS_CONTEXT; import static org.apache.kafka.controller.ControllerRequestContextUtil.anonymousContextFor; import static org.apache.kafka.controller.QuorumControllerIntegrationTestUtils.brokerFeatures; -import static org.apache.kafka.controller.QuorumControllerIntegrationTestUtils.forceRenounce; import static org.apache.kafka.controller.QuorumControllerIntegrationTestUtils.pause; import static org.apache.kafka.controller.QuorumControllerIntegrationTestUtils.registerBrokersAndUnfence; import static org.apache.kafka.controller.QuorumControllerIntegrationTestUtils.sendBrokerHeartbeatToUnfenceBrokers; @@ -1382,75 +1376,9 @@ public class QuorumControllerTest { appender)).getMessage()); } - @Test - public void testBootstrapZkMigrationRecord() throws Exception { - assertEquals(ZkMigrationState.PRE_MIGRATION, - checkBootstrapZkMigrationRecord(MetadataVersion.IBP_3_4_IV0, true)); - - assertEquals(ZkMigrationState.NONE, - checkBootstrapZkMigrationRecord(MetadataVersion.IBP_3_4_IV0, false)); - - assertEquals(ZkMigrationState.NONE, - checkBootstrapZkMigrationRecord(MetadataVersion.IBP_3_3_IV0, false)); - - assertEquals( - "The bootstrap metadata.version 3.3-IV0 does not support ZK migrations. Cannot continue with ZK migrations enabled.", - assertThrows(FaultHandlerException.class, () -> - checkBootstrapZkMigrationRecord(MetadataVersion.IBP_3_3_IV0, true)).getCause().getMessage() - ); - } - - public ZkMigrationState checkBootstrapZkMigrationRecord( - MetadataVersion metadataVersion, - boolean migrationEnabled - ) throws Exception { - try ( - LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(1).build(); - QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv). - setControllerBuilderInitializer(controllerBuilder -> - controllerBuilder.setZkMigrationEnabled(migrationEnabled) - ). - setBootstrapMetadata(BootstrapMetadata.fromVersion(metadataVersion, "test")). - build() - ) { - QuorumController active = controlEnv.activeController(); - ZkMigrationState zkMigrationState = active.appendReadEvent("read migration state", OptionalLong.empty(), - () -> active.featureControl().zkMigrationState()).get(30, TimeUnit.SECONDS); - testToImages(logEnv.allRecords()); - return zkMigrationState; - } - } - - @Test - public void testUpgradeMigrationStateFrom34() throws Exception { - try (LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(1).build()) { - // In 3.4, we only wrote a PRE_MIGRATION to the log. In that software version, we defined this - // as enum value 1. In 3.5+ software, this enum value is redefined as MIGRATION - BootstrapMetadata bootstrapMetadata = BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_4_IV0, "test"); - List initialRecords = new ArrayList<>(bootstrapMetadata.records()); - initialRecords.add(ZkMigrationState.of((byte) 1).toRecord()); - logEnv.appendInitialRecords(initialRecords); - try ( - QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv). - setControllerBuilderInitializer(controllerBuilder -> - controllerBuilder.setZkMigrationEnabled(true) - ). - setBootstrapMetadata(bootstrapMetadata). - build() - ) { - QuorumController active = controlEnv.activeController(); - assertEquals(active.featureControl().zkMigrationState(), ZkMigrationState.MIGRATION); - assertFalse(active.featureControl().inPreMigrationMode()); - } - - testToImages(logEnv.allRecords()); - } - } - FeatureControlManager getActivationRecords( - MetadataVersion metadataVersion, - Optional stateInLog, - boolean zkMigrationEnabled + MetadataVersion metadataVersion, + Optional stateInLog ) { SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); FeatureControlManager featureControlManager = new FeatureControlManager.Builder() @@ -1458,16 +1386,13 @@ public class QuorumControllerTest { .setMetadataVersion(metadataVersion) .build(); - stateInLog.ifPresent(zkMigrationState -> - featureControlManager.replay((ZkMigrationStateRecord) zkMigrationState.toRecord().message())); - ControllerResult result = ActivationRecordsGenerator.generate( msg -> { }, !stateInLog.isPresent(), -1L, - zkMigrationEnabled, BootstrapMetadata.fromVersion(metadataVersion, "test"), - featureControlManager); + stateInLog.orElseGet(() -> ZkMigrationState.NONE), + metadataVersion); RecordTestUtils.replayAll(featureControlManager, result.records()); return featureControlManager; } @@ -1476,21 +1401,11 @@ public class QuorumControllerTest { public void testActivationRecords33() { FeatureControlManager featureControl; - assertEquals( - "The bootstrap metadata.version 3.3-IV0 does not support ZK migrations. Cannot continue with ZK migrations enabled.", - assertThrows(RuntimeException.class, () -> getActivationRecords(MetadataVersion.IBP_3_3_IV0, Optional.empty(), true)).getMessage() - ); - - featureControl = getActivationRecords(MetadataVersion.IBP_3_3_IV0, Optional.empty(), false); + featureControl = getActivationRecords(MetadataVersion.IBP_3_3_IV0, Optional.empty()); assertEquals(MetadataVersion.IBP_3_3_IV0, featureControl.metadataVersion()); assertEquals(ZkMigrationState.NONE, featureControl.zkMigrationState()); - assertEquals( - "Should not have ZK migrations enabled on a cluster running metadata.version 3.3-IV0", - assertThrows(RuntimeException.class, () -> getActivationRecords(MetadataVersion.IBP_3_3_IV0, Optional.of(ZkMigrationState.NONE), true)).getMessage() - ); - - featureControl = getActivationRecords(MetadataVersion.IBP_3_3_IV0, Optional.of(ZkMigrationState.NONE), false); + featureControl = getActivationRecords(MetadataVersion.IBP_3_3_IV0, Optional.of(ZkMigrationState.NONE)); assertEquals(MetadataVersion.IBP_3_3_IV0, featureControl.metadataVersion()); assertEquals(ZkMigrationState.NONE, featureControl.zkMigrationState()); } @@ -1499,67 +1414,32 @@ public class QuorumControllerTest { public void testActivationRecords34() { FeatureControlManager featureControl; - featureControl = getActivationRecords(MetadataVersion.IBP_3_4_IV0, Optional.empty(), true); - assertEquals(MetadataVersion.IBP_3_4_IV0, featureControl.metadataVersion()); - assertEquals(ZkMigrationState.PRE_MIGRATION, featureControl.zkMigrationState()); - - featureControl = getActivationRecords(MetadataVersion.IBP_3_4_IV0, Optional.empty(), false); + featureControl = getActivationRecords(MetadataVersion.IBP_3_4_IV0, Optional.empty()); assertEquals(MetadataVersion.IBP_3_4_IV0, featureControl.metadataVersion()); assertEquals(ZkMigrationState.NONE, featureControl.zkMigrationState()); - assertEquals( - "Should not have ZK migrations enabled on a cluster that was created in KRaft mode.", - assertThrows(RuntimeException.class, () -> getActivationRecords(MetadataVersion.IBP_3_4_IV0, Optional.of(ZkMigrationState.NONE), true)).getMessage() - ); - - featureControl = getActivationRecords(MetadataVersion.IBP_3_4_IV0, Optional.of(ZkMigrationState.NONE), false); + featureControl = getActivationRecords(MetadataVersion.IBP_3_4_IV0, Optional.of(ZkMigrationState.NONE)); assertEquals(MetadataVersion.IBP_3_4_IV0, featureControl.metadataVersion()); assertEquals(ZkMigrationState.NONE, featureControl.zkMigrationState()); - - featureControl = getActivationRecords(MetadataVersion.IBP_3_4_IV0, Optional.of(ZkMigrationState.PRE_MIGRATION), true); - assertEquals(MetadataVersion.IBP_3_4_IV0, featureControl.metadataVersion()); - assertEquals(ZkMigrationState.PRE_MIGRATION, featureControl.zkMigrationState()); - - featureControl = getActivationRecords(MetadataVersion.IBP_3_4_IV0, Optional.of(ZkMigrationState.MIGRATION), true); - assertEquals(MetadataVersion.IBP_3_4_IV0, featureControl.metadataVersion()); - assertEquals(ZkMigrationState.MIGRATION, featureControl.zkMigrationState()); - - featureControl = getActivationRecords(MetadataVersion.IBP_3_4_IV0, Optional.of(ZkMigrationState.MIGRATION), false); - assertEquals(MetadataVersion.IBP_3_4_IV0, featureControl.metadataVersion()); - assertEquals(ZkMigrationState.POST_MIGRATION, featureControl.zkMigrationState()); - - featureControl = getActivationRecords(MetadataVersion.IBP_3_4_IV0, Optional.of(ZkMigrationState.POST_MIGRATION), true); - assertEquals(MetadataVersion.IBP_3_4_IV0, featureControl.metadataVersion()); - assertEquals(ZkMigrationState.POST_MIGRATION, featureControl.zkMigrationState()); - - featureControl = getActivationRecords(MetadataVersion.IBP_3_4_IV0, Optional.of(ZkMigrationState.POST_MIGRATION), false); - assertEquals(MetadataVersion.IBP_3_4_IV0, featureControl.metadataVersion()); - assertEquals(ZkMigrationState.POST_MIGRATION, featureControl.zkMigrationState()); } @Test public void testActivationRecordsNonEmptyLog() { FeatureControlManager featureControl = getActivationRecords( - MetadataVersion.IBP_3_4_IV0, Optional.empty(), true); - assertEquals(MetadataVersion.IBP_3_4_IV0, featureControl.metadataVersion()); - assertEquals(ZkMigrationState.PRE_MIGRATION, featureControl.zkMigrationState()); + MetadataVersion.IBP_3_9_IV0, Optional.empty()); + assertEquals(MetadataVersion.IBP_3_9_IV0, featureControl.metadataVersion()); + assertEquals(ZkMigrationState.NONE, featureControl.zkMigrationState()); } - @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void testActivationRecordsPartialBootstrap(boolean zkMigrationEnabled) { - FeatureControlManager featureControlManager = new FeatureControlManager.Builder() - .setSnapshotRegistry(new SnapshotRegistry(new LogContext())) - .setMetadataVersion(MetadataVersion.IBP_3_6_IV1) - .build(); - + @Test + public void testActivationRecordsPartialBootstrap() { ControllerResult result = ActivationRecordsGenerator.generate( logMsg -> { }, true, 0L, - zkMigrationEnabled, BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_6_IV1, "test"), - featureControlManager); + ZkMigrationState.NONE, + MetadataVersion.IBP_3_6_IV1); assertFalse(result.isAtomic()); assertTrue(RecordTestUtils.recordAtIndexAs( AbortTransactionRecord.class, result.records(), 0).isPresent()); @@ -1569,27 +1449,6 @@ public class QuorumControllerTest { EndTransactionRecord.class, result.records(), result.records().size() - 1).isPresent()); } - @Test - public void testMigrationsEnabledForOldBootstrapMetadataVersion() throws Exception { - try ( - LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(1).build() - ) { - QuorumControllerTestEnv.Builder controlEnvBuilder = new QuorumControllerTestEnv.Builder(logEnv). - setControllerBuilderInitializer(controllerBuilder -> - controllerBuilder.setZkMigrationEnabled(true) - ). - setBootstrapMetadata(BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_3_IV0, "test")); - - QuorumControllerTestEnv controlEnv = controlEnvBuilder.build(); - QuorumController active = controlEnv.activeController(); - assertEquals(ZkMigrationState.NONE, active.appendReadEvent("read migration state", OptionalLong.empty(), - () -> active.featureControl().zkMigrationState()).get(30, TimeUnit.SECONDS)); - assertThrows(FaultHandlerException.class, controlEnv::close); - - testToImages(logEnv.allRecords()); - } - } - /** * Tests all intermediate images lead to the same final image for each image & delta type. * @param fromRecords @@ -1618,12 +1477,6 @@ public class QuorumControllerTest { @Test public void testActivationRecordsPartialTransaction() { - SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); - FeatureControlManager featureControlManager = new FeatureControlManager.Builder() - .setSnapshotRegistry(snapshotRegistry) - .setMetadataVersion(MetadataVersion.IBP_3_6_IV1) - .build(); - OffsetControlManager offsetControlManager = new OffsetControlManager.Builder().build(); offsetControlManager.replay(new BeginTransactionRecord(), 10); offsetControlManager.handleCommitBatch(Batch.data(20, 1, 1L, 0, @@ -1633,9 +1486,9 @@ public class QuorumControllerTest { logMsg -> { }, false, offsetControlManager.transactionStartOffset(), - false, BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_6_IV1, "test"), - featureControlManager); + ZkMigrationState.NONE, + MetadataVersion.IBP_3_6_IV1); assertTrue(result.isAtomic()); offsetControlManager.replay( @@ -1647,12 +1500,6 @@ public class QuorumControllerTest { @Test public void testActivationRecordsPartialTransactionNoSupport() { - SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); - FeatureControlManager featureControlManager = new FeatureControlManager.Builder() - .setSnapshotRegistry(snapshotRegistry) - .setMetadataVersion(MetadataVersion.IBP_3_6_IV0) - .build(); - OffsetControlManager offsetControlManager = new OffsetControlManager.Builder().build(); offsetControlManager.replay(new BeginTransactionRecord(), 10); offsetControlManager.handleCommitBatch(Batch.data(20, 1, 1L, 0, @@ -1663,129 +1510,9 @@ public class QuorumControllerTest { msg -> { }, false, offsetControlManager.transactionStartOffset(), - false, BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_6_IV0, "test"), - featureControlManager) + ZkMigrationState.NONE, + MetadataVersion.IBP_3_6_IV0) ); } - - private static final List 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>> 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()); - - } - } } diff --git a/metadata/src/test/java/org/apache/kafka/controller/metrics/QuorumControllerMetricsTest.java b/metadata/src/test/java/org/apache/kafka/controller/metrics/QuorumControllerMetricsTest.java index 04dfefa98cd..041f3c8312b 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/metrics/QuorumControllerMetricsTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/metrics/QuorumControllerMetricsTest.java @@ -25,8 +25,6 @@ import com.yammer.metrics.core.MetricName; import com.yammer.metrics.core.MetricsRegistry; import org.junit.jupiter.api.Test; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.ValueSource; import java.util.Arrays; import java.util.Collections; @@ -36,16 +34,14 @@ import java.util.Optional; import static org.junit.jupiter.api.Assertions.assertEquals; public class QuorumControllerMetricsTest { - @ParameterizedTest - @ValueSource(booleans = {false, true}) - public void testMetricNames(boolean inMigration) { + @Test + public void testMetricNames() { MetricsRegistry registry = new MetricsRegistry(); MockTime time = new MockTime(); try { try (QuorumControllerMetrics metrics = new QuorumControllerMetrics( Optional.of(registry), - time, - inMigration)) { + time)) { HashSet expected = new HashSet<>(Arrays.asList( "kafka.controller:type=ControllerEventManager,name=EventQueueProcessingTimeMs", "kafka.controller:type=ControllerEventManager,name=EventQueueTimeMs", @@ -59,11 +55,6 @@ public class QuorumControllerMetricsTest { "kafka.controller:type=KafkaController,name=NewActiveControllersCount", "kafka.controller:type=KafkaController,name=TimedOutBrokerHeartbeatCount" )); - if (inMigration) { - expected.add("kafka.controller:type=KafkaController,name=ZkWriteBehindLag"); - expected.add("kafka.controller:type=KafkaController,name=ZkWriteSnapshotTimeMs"); - expected.add("kafka.controller:type=KafkaController,name=ZkWriteDeltaTimeMs"); - } ControllerMetricsTestUtils.assertMetricsForTypeEqual(registry, "kafka.controller", expected); } ControllerMetricsTestUtils.assertMetricsForTypeEqual(registry, "kafka.controller", @@ -77,7 +68,7 @@ public class QuorumControllerMetricsTest { public void testUpdateEventQueueTime() { MetricsRegistry registry = new MetricsRegistry(); MockTime time = new MockTime(); - try (QuorumControllerMetrics metrics = new QuorumControllerMetrics(Optional.of(registry), time, false)) { + try (QuorumControllerMetrics metrics = new QuorumControllerMetrics(Optional.of(registry), time)) { metrics.updateEventQueueTime(1000); assertMetricHistogram(registry, metricName("ControllerEventManager", "EventQueueTimeMs"), 1, 1000); } finally { @@ -89,7 +80,7 @@ public class QuorumControllerMetricsTest { public void testUpdateEventQueueProcessingTime() { MetricsRegistry registry = new MetricsRegistry(); MockTime time = new MockTime(); - try (QuorumControllerMetrics metrics = new QuorumControllerMetrics(Optional.of(registry), time, false)) { + try (QuorumControllerMetrics metrics = new QuorumControllerMetrics(Optional.of(registry), time)) { metrics.updateEventQueueProcessingTime(1000); assertMetricHistogram(registry, metricName("ControllerEventManager", "EventQueueProcessingTimeMs"), 1, 1000); } finally { @@ -102,11 +93,10 @@ public class QuorumControllerMetricsTest { MetricsRegistry registry = new MetricsRegistry(); MockTime time = new MockTime(); time.sleep(1000); - try (QuorumControllerMetrics metrics = new QuorumControllerMetrics(Optional.of(registry), time, true)) { + try (QuorumControllerMetrics metrics = new QuorumControllerMetrics(Optional.of(registry), time)) { metrics.setLastAppliedRecordOffset(100); metrics.setLastAppliedRecordTimestamp(500); metrics.setLastCommittedRecordOffset(50); - metrics.updateDualWriteOffset(40L); metrics.setActive(true); for (int i = 0; i < 2; i++) { metrics.incrementTimedOutHeartbeats(); @@ -145,12 +135,6 @@ public class QuorumControllerMetricsTest { .get(metricName("KafkaController", "LastCommittedRecordOffset")); assertEquals(50, lastCommittedRecordOffset.value()); - @SuppressWarnings("unchecked") - Gauge zkWriteBehindLag = (Gauge) registry - .allMetrics() - .get(metricName("KafkaController", "ZkWriteBehindLag")); - assertEquals(10L, zkWriteBehindLag.value()); - @SuppressWarnings("unchecked") Gauge timedOutBrokerHeartbeats = (Gauge) registry .allMetrics() @@ -179,51 +163,6 @@ public class QuorumControllerMetricsTest { } } - @Test - public void testUpdateZKWriteBehindLag() { - MetricsRegistry registry = new MetricsRegistry(); - MockTime time = new MockTime(); - // test zkWriteBehindLag metric when NOT in dual-write mode - try (QuorumControllerMetrics metrics = new QuorumControllerMetrics(Optional.of(registry), time, true)) { - metrics.updateDualWriteOffset(0); - @SuppressWarnings("unchecked") - Gauge zkWriteBehindLag = (Gauge) 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 zkWriteBehindLag = (Gauge) 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 zkWriteBehindLag = (Gauge) registry - .allMetrics() - .get(metricName("KafkaController", "ZkWriteBehindLag")); - assertEquals(0, zkWriteBehindLag.value()); - } finally { - registry.shutdown(); - } - } - private static void assertMetricHistogram(MetricsRegistry registry, MetricName metricName, long count, double sum) { Histogram histogram = (Histogram) registry.allMetrics().get(metricName); diff --git a/metadata/src/test/java/org/apache/kafka/metadata/migration/CapturingAclMigrationClient.java b/metadata/src/test/java/org/apache/kafka/metadata/migration/CapturingAclMigrationClient.java deleted file mode 100644 index 717750379a5..00000000000 --- a/metadata/src/test/java/org/apache/kafka/metadata/migration/CapturingAclMigrationClient.java +++ /dev/null @@ -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 deletedResources = new ArrayList<>(); - public LinkedHashMap> 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 aclsToWrite, ZkMigrationLeadershipState state) { - updatedResources.put(resourcePattern, aclsToWrite); - return state; - } - - @Override - public void iterateAcls(BiConsumer> aclConsumer) { - - } -} diff --git a/metadata/src/test/java/org/apache/kafka/metadata/migration/CapturingConfigMigrationClient.java b/metadata/src/test/java/org/apache/kafka/metadata/migration/CapturingConfigMigrationClient.java deleted file mode 100644 index 6c1393dbc3e..00000000000 --- a/metadata/src/test/java/org/apache/kafka/metadata/migration/CapturingConfigMigrationClient.java +++ /dev/null @@ -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 deletedResources = new ArrayList<>(); - public LinkedHashMap> writtenConfigs = new LinkedHashMap<>(); - - public void reset() { - deletedResources.clear(); - writtenConfigs.clear(); - } - - @Override - public void iterateClientQuotas(ClientQuotaVisitor visitor) { - - } - - @Override - public void iterateBrokerConfigs(BiConsumer> configConsumer) { - - } - - @Override - public void iterateTopicConfigs(BiConsumer> configConsumer) { - - } - - @Override - public void readTopicConfigs(String topicName, Consumer> configConsumer) { - - } - - @Override - public ZkMigrationLeadershipState writeConfigs(ConfigResource configResource, Map configMap, ZkMigrationLeadershipState state) { - writtenConfigs.put(configResource, configMap); - return state; - } - - @Override - public ZkMigrationLeadershipState writeClientQuotas(Map clientQuotaEntity, Map quotas, Map scram, ZkMigrationLeadershipState state) { - return null; - } - - - @Override - public ZkMigrationLeadershipState deleteConfigs(ConfigResource configResource, ZkMigrationLeadershipState state) { - deletedResources.add(configResource); - return state; - } - -} diff --git a/metadata/src/test/java/org/apache/kafka/metadata/migration/CapturingDelegationTokenMigrationClient.java b/metadata/src/test/java/org/apache/kafka/metadata/migration/CapturingDelegationTokenMigrationClient.java deleted file mode 100644 index 0ee6168e99a..00000000000 --- a/metadata/src/test/java/org/apache/kafka/metadata/migration/CapturingDelegationTokenMigrationClient.java +++ /dev/null @@ -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 deletedDelegationTokens = new ArrayList<>(); - public HashMap updatedDelegationTokens = new HashMap<>(); - - public void reset() { - deletedDelegationTokens.clear(); - updatedDelegationTokens.clear(); - } - - - @Override - public List 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; - } - -} diff --git a/metadata/src/test/java/org/apache/kafka/metadata/migration/CapturingMigrationClient.java b/metadata/src/test/java/org/apache/kafka/metadata/migration/CapturingMigrationClient.java deleted file mode 100644 index 03579056a58..00000000000 --- a/metadata/src/test/java/org/apache/kafka/metadata/migration/CapturingMigrationClient.java +++ /dev/null @@ -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> recordBatches() { - return Collections.emptyList(); - } - - default List brokerIds() { - return Collections.emptyList(); - } - } - - static Builder newBuilder() { - return new Builder(); - } - - public static class Builder { - Set 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 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 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 readProducerId() { - return Optional.empty(); - } - - @Override - public ZkMigrationLeadershipState writeProducerId( - long nextProducerId, - ZkMigrationLeadershipState state - ) { - this.state = state; - return state; - } - - @Override - public void readAllMetadata( - Consumer> batchConsumer, - Consumer brokerIdConsumer - ) { - batchSupplier.recordBatches().forEach(batchConsumer); - batchSupplier.brokerIds().forEach(brokerIdConsumer); - } - - @Override - public Set readBrokerIds() { - return brokerIds; - } -} diff --git a/metadata/src/test/java/org/apache/kafka/metadata/migration/CapturingTopicMigrationClient.java b/metadata/src/test/java/org/apache/kafka/metadata/migration/CapturingTopicMigrationClient.java deleted file mode 100644 index e3b7cede7fc..00000000000 --- a/metadata/src/test/java/org/apache/kafka/metadata/migration/CapturingTopicMigrationClient.java +++ /dev/null @@ -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 deletedTopics = new ArrayList<>(); - public List createdTopics = new ArrayList<>(); - public LinkedHashMap> updatedTopics = new LinkedHashMap<>(); - public LinkedHashMap> newTopicPartitions = new LinkedHashMap<>(); - public LinkedHashMap> updatedTopicPartitions = new LinkedHashMap<>(); - public LinkedHashMap> deletedTopicPartitions = new LinkedHashMap<>(); - - - public void reset() { - createdTopics.clear(); - updatedTopicPartitions.clear(); - deletedTopics.clear(); - updatedTopics.clear(); - deletedTopicPartitions.clear(); - } - - - @Override - public void iterateTopics(EnumSet interests, TopicVisitor visitor) { - - } - - @Override - public Set readPendingTopicDeletions() { - return Collections.emptySet(); - } - - @Override - public ZkMigrationLeadershipState clearPendingTopicDeletions( - Set 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 topicPartitions, ZkMigrationLeadershipState state) { - createdTopics.add(topicName); - return state; - } - - @Override - public ZkMigrationLeadershipState updateTopic( - String topicName, - Uuid topicId, - Map topicPartitions, - ZkMigrationLeadershipState state - ) { - updatedTopics.put(topicName, topicPartitions); - return state; - } - - @Override - public ZkMigrationLeadershipState createTopicPartitions(Map> topicPartitions, ZkMigrationLeadershipState state) { - topicPartitions.forEach((topicName, partitionMap) -> - newTopicPartitions.put(topicName, partitionMap.keySet()) - ); - return state; - } - - @Override - public ZkMigrationLeadershipState updateTopicPartitions(Map> topicPartitions, ZkMigrationLeadershipState state) { - topicPartitions.forEach((topicName, partitionMap) -> - updatedTopicPartitions.put(topicName, partitionMap.keySet()) - ); - return state; - } - - @Override - public ZkMigrationLeadershipState deleteTopicPartitions(Map> topicPartitions, ZkMigrationLeadershipState state) { - deletedTopicPartitions.putAll(topicPartitions); - return state; - } -}