From 3dba3125e9c81fc9ba301416d708205a53e22aaa Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Wed, 19 Feb 2025 05:35:42 -0800 Subject: [PATCH] KAFKA-18601: Assume a baseline of 3.3 for server protocol versions (#18845) 3.3.0 was the first KRaft release that was deemed production-ready and also when KIP-778 (KRaft to KRaft upgrades) landed. Given that, it's reasonable for 4.x to only support upgrades from 3.3.0 or newer (the metadata version also needs to be set to "3.3" or newer before upgrading). Noteworthy changes: 1. `AlterPartition` no longer includes topic names, which makes it possible to simplify `AlterParitionManager` logic. 2. Metadata versions older than `IBP_3_3_IV3` have been removed and `IBP_3_3_IV3` is now the minimum version. 3. `MINIMUM_BOOTSTRAP_VERSION` has been removed. 4. Removed `isLeaderRecoverySupported`, `isNoOpsRecordSupported`, `isKRaftSupported`, `isBrokerRegistrationChangeRecordSupported` and `isInControlledShutdownStateSupported` - these are always `true` now. Also removed related conditional code. 5. Removed default metadata version or metadata version fallbacks in multiple places - we now fail-fast instead of potentially using an incorrect metadata version. 6. Update `MetadataBatchLoader.resetToImage` to set `hasSeenRecord` based on whether image is empty - this was a previously existing issue that became more apparent after the changes in this PR. 7. Remove `ibp` parameter from `BootstrapDirectory` 8. A number of tests were not useful anymore and have been removed. I will update the upgrade notes via a separate PR as there are a few things that need changing and it would be easier to do so that way. Reviewers: Chia-Ping Tsai , Jun Rao , David Arthur , Colin P. McCabe , Justine Olshan , Ken Huang --- .../common/message/AlterPartitionRequest.json | 7 +- .../message/AlterPartitionResponse.json | 7 +- .../requests/AlterPartitionRequestTest.java | 5 +- .../common/requests/RequestResponseTest.java | 22 +-- .../MetadataVersionConfigValidator.java | 2 +- .../main/scala/kafka/cluster/Partition.scala | 6 +- .../kafka/server/AlterPartitionManager.scala | 115 +++++--------- .../scala/kafka/server/BrokerServer.scala | 1 - .../ControllerRegistrationManager.scala | 13 +- .../scala/kafka/server/KafkaRaftServer.scala | 2 +- .../scala/kafka/server/ReplicaManager.scala | 2 +- .../metadata/BrokerMetadataPublisher.scala | 2 +- .../server/metadata/KRaftMetadataCache.scala | 5 +- .../main/scala/kafka/tools/StorageTool.scala | 6 +- .../scala/kafka/tools/TestRaftServer.scala | 2 +- .../admin/ConfigCommandIntegrationTest.java | 2 +- .../ReconfigurableQuorumIntegrationTest.java | 25 +-- .../transaction/ProducerIntegrationTest.scala | 2 +- .../kafka/server/KRaftClusterTest.scala | 20 +-- .../MetadataVersionIntegrationTest.scala | 13 +- .../server/LocalLeaderEndPointTest.scala | 8 +- .../server/RemoteLeaderEndPointTest.scala | 4 +- .../kafka/cluster/PartitionLockTest.scala | 10 +- .../unit/kafka/cluster/PartitionTest.scala | 81 +++++----- .../network/RequestConvertToJsonTest.scala | 7 +- .../AbstractApiVersionsRequestTest.scala | 2 +- .../server/AlterPartitionManagerTest.scala | 120 +++++---------- .../kafka/server/ApiVersionManagerTest.scala | 15 +- .../ApiVersionsResponseIntegrationTest.scala | 30 ++-- .../BrokerRegistrationRequestTest.scala | 23 ++- .../kafka/server/ControllerApisTest.scala | 2 +- .../ControllerRegistrationManagerTest.scala | 8 +- .../unit/kafka/server/KafkaApisTest.scala | 29 +++- .../kafka/server/KafkaRaftServerTest.scala | 8 +- .../server/ReplicaFetcherThreadTest.scala | 18 +-- .../ReplicaManagerConcurrencyTest.scala | 26 ++-- .../kafka/server/ReplicaManagerTest.scala | 20 +-- .../unit/kafka/tools/StorageToolTest.scala | 12 +- .../scala/unit/kafka/utils/TestUtils.scala | 2 +- docs/ops.html | 6 +- .../ReplicaFetcherThreadBenchmark.java | 4 +- .../ActivationRecordsGenerator.java | 12 +- .../controller/ClusterControlManager.java | 24 +-- .../controller/FeatureControlManager.java | 25 +-- .../controller/PartitionChangeBuilder.java | 3 +- .../kafka/controller/QuorumController.java | 19 +-- .../kafka/controller/QuorumFeatures.java | 2 +- .../controller/ReplicationControlManager.java | 62 +++----- .../org/apache/kafka/image/AclsImage.java | 7 +- .../org/apache/kafka/image/FeaturesDelta.java | 13 +- .../org/apache/kafka/image/FeaturesImage.java | 47 +++--- .../org/apache/kafka/image/MetadataImage.java | 2 +- .../image/loader/MetadataBatchLoader.java | 2 +- .../kafka/image/loader/MetadataLoader.java | 7 +- .../loader/metrics/MetadataLoaderMetrics.java | 2 +- .../apache/kafka/image/node/MetadataNode.java | 3 +- .../publisher/BrokerRegistrationTracker.java | 3 +- .../image/publisher/SnapshotEmitter.java | 3 +- .../image/writer/ImageWriterOptions.java | 29 +--- .../kafka/metadata/BrokerRegistration.java | 11 +- .../metadata/ControllerRegistration.java | 2 +- .../bootstrap/BootstrapDirectory.java | 21 +-- .../metadata/bootstrap/BootstrapMetadata.java | 8 - .../metadata/publisher/FeaturesPublisher.java | 6 +- .../kafka/metadata/storage/Formatter.java | 6 +- .../ActivationRecordsGeneratorTest.java | 6 +- .../controller/ClusterControlManagerTest.java | 145 ++++++++++-------- .../controller/FeatureControlManagerTest.java | 83 +++++----- .../PartitionChangeBuilderTest.java | 39 ++--- .../QuorumControllerIntegrationTestUtils.java | 4 +- .../controller/QuorumControllerTest.java | 96 ++++-------- .../kafka/controller/QuorumFeaturesTest.java | 4 +- .../ReplicationControlManagerTest.java | 68 +++----- ...ontrollerMetadataMetricsPublisherTest.java | 5 +- .../metrics/ControllerMetricsChangesTest.java | 3 +- .../org/apache/kafka/image/AclsImageTest.java | 3 +- .../kafka/image/ClientQuotasImageTest.java | 3 +- .../apache/kafka/image/ClusterImageTest.java | 9 +- .../kafka/image/ConfigurationsImageTest.java | 3 +- .../kafka/image/DelegationTokenImageTest.java | 20 +-- .../apache/kafka/image/FeaturesDeltaTest.java | 43 ++++++ .../apache/kafka/image/FeaturesImageTest.java | 15 +- .../kafka/image/ImageDowngradeTest.java | 56 +------ .../apache/kafka/image/MetadataImageTest.java | 27 +++- .../image/MetadataVersionChangeTest.java | 28 ++-- .../kafka/image/ProducerIdsImageTest.java | 3 +- .../apache/kafka/image/ScramImageTest.java | 18 +-- .../apache/kafka/image/TopicsImageTest.java | 3 +- .../image/loader/MetadataLoaderTest.java | 87 +++++++---- .../metrics/MetadataLoaderMetricsTest.java | 6 +- .../BrokerRegistrationTrackerTest.java | 8 +- .../image/writer/ImageWriterOptionsTest.java | 32 +--- .../metadata/BrokerRegistrationTest.java | 3 +- .../metadata/ControllerRegistrationTest.java | 4 +- .../metadata/PartitionRegistrationTest.java | 10 +- .../kafka/metadata/RecordTestUtils.java | 2 +- .../bootstrap/BootstrapDirectoryTest.java | 25 +-- .../bootstrap/BootstrapMetadataTest.java | 24 +-- .../kafka/metadata/storage/FormatterTest.java | 10 +- .../common/EligibleLeaderReplicasVersion.java | 2 +- .../apache/kafka/server/common/Feature.java | 11 +- .../kafka/server/common/GroupVersion.java | 2 +- .../kafka/server/common/KRaftVersion.java | 2 +- .../kafka/server/common/MetadataVersion.java | 90 +++-------- .../server/common/TestFeatureVersion.java | 2 +- .../server/common/TransactionVersion.java | 2 +- .../server/common/UnitTestFeatureVersion.java | 16 +- .../kafka/server/common/FeatureTest.java | 23 +-- .../server/common/FinalizedFeaturesTest.java | 6 +- .../kafka/server/common/KRaftVersionTest.java | 2 +- .../server/common/MetadataVersionTest.java | 80 ++-------- .../common/MetadataVersionTestUtils.java | 25 +++ .../apache/kafka/server/BrokerFeatures.java | 2 +- .../storage/internals/log/LogConfig.java | 6 - .../common/test/api/ClusterConfigTest.java | 2 +- .../apache/kafka/tools/FeatureCommand.java | 8 +- .../kafka/tools/FeatureCommandTest.java | 103 +++++++------ .../ReassignPartitionsCommandTest.java | 14 -- 118 files changed, 969 insertions(+), 1337 deletions(-) create mode 100644 metadata/src/test/java/org/apache/kafka/image/FeaturesDeltaTest.java create mode 100644 server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTestUtils.java diff --git a/clients/src/main/resources/common/message/AlterPartitionRequest.json b/clients/src/main/resources/common/message/AlterPartitionRequest.json index d22f3eb13ad..fa8d318e5c9 100644 --- a/clients/src/main/resources/common/message/AlterPartitionRequest.json +++ b/clients/src/main/resources/common/message/AlterPartitionRequest.json @@ -18,12 +18,13 @@ "type": "request", "listeners": ["controller"], "name": "AlterPartitionRequest", + // Versions 0-1 were removed in Apache Kafka 4.0, version 2 is the new baseline. + // Version 1 adds LeaderRecoveryState field (KIP-704). - // // Version 2 adds TopicId field to replace TopicName field (KIP-841). // // Version 3 adds the NewIsrEpochs field and deprecates the NewIsr field (KIP-903). - "validVersions": "0-3", + "validVersions": "2-3", "flexibleVersions": "0+", "fields": [ { "name": "BrokerId", "type": "int32", "versions": "0+", "entityType": "brokerId", @@ -32,8 +33,6 @@ "about": "The epoch of the requesting broker." }, { "name": "Topics", "type": "[]TopicData", "versions": "0+", "about": "The topics to alter ISRs for.", "fields": [ - { "name": "TopicName", "type": "string", "versions": "0-1", "ignorable": true, "entityType": "topicName", - "about": "The name of the topic to alter ISRs for." }, { "name": "TopicId", "type": "uuid", "versions": "2+", "ignorable": true, "about": "The ID of the topic to alter ISRs for." }, { "name": "Partitions", "type": "[]PartitionData", "versions": "0+", diff --git a/clients/src/main/resources/common/message/AlterPartitionResponse.json b/clients/src/main/resources/common/message/AlterPartitionResponse.json index 0df8f8c9232..94daf85cb3b 100644 --- a/clients/src/main/resources/common/message/AlterPartitionResponse.json +++ b/clients/src/main/resources/common/message/AlterPartitionResponse.json @@ -17,13 +17,14 @@ "apiKey": 56, "type": "response", "name": "AlterPartitionResponse", + // Versions 0-1 were removed in Apache Kafka 4.0, version 2 is the new baseline. + // Version 1 adds LeaderRecoveryState field (KIP-704). - // // Version 2 adds TopicId field to replace TopicName field, can return the following new errors: // INELIGIBLE_REPLICA, NEW_LEADER_ELECTED and UNKNOWN_TOPIC_ID (KIP-841). // // Version 3 is the same as version 2 (KIP-903). - "validVersions": "0-3", + "validVersions": "2-3", "flexibleVersions": "0+", "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", @@ -32,8 +33,6 @@ "about": "The top level response error code." }, { "name": "Topics", "type": "[]TopicData", "versions": "0+", "about": "The responses for each topic.", "fields": [ - { "name": "TopicName", "type": "string", "versions": "0-1", "ignorable": true, "entityType": "topicName", - "about": "The name of the topic." }, { "name": "TopicId", "type": "uuid", "versions": "2+", "ignorable": true, "about": "The ID of the topic." }, { "name": "Partitions", "type": "[]PartitionData", "versions": "0+", diff --git a/clients/src/test/java/org/apache/kafka/common/requests/AlterPartitionRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/AlterPartitionRequestTest.java index 16900fc5dac..9590d232c55 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/AlterPartitionRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/AlterPartitionRequestTest.java @@ -34,7 +34,6 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; class AlterPartitionRequestTest { - String topic = "test-topic"; Uuid topicId = Uuid.randomUuid(); @ParameterizedTest @@ -44,9 +43,7 @@ class AlterPartitionRequestTest { .setBrokerId(1) .setBrokerEpoch(1); - TopicData topicData = new TopicData() - .setTopicId(topicId) - .setTopicName(topic); + TopicData topicData = new TopicData().setTopicId(topicId); List newIsrWithBrokerEpoch = new LinkedList<>(); newIsrWithBrokerEpoch.add(new BrokerState().setBrokerId(1).setBrokerEpoch(1001)); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index f629e9bf721..9018b8c8f27 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -1103,7 +1103,7 @@ public class RequestResponseTest { case BEGIN_QUORUM_EPOCH: return createBeginQuorumEpochResponse(); case END_QUORUM_EPOCH: return createEndQuorumEpochResponse(); case DESCRIBE_QUORUM: return createDescribeQuorumResponse(); - case ALTER_PARTITION: return createAlterPartitionResponse(version); + case ALTER_PARTITION: return createAlterPartitionResponse(); case UPDATE_FEATURES: return createUpdateFeaturesResponse(); case ENVELOPE: return createEnvelopeResponse(); case FETCH_SNAPSHOT: return createFetchSnapshotResponse(); @@ -1703,41 +1703,31 @@ public class RequestResponseTest { .setPartitionIndex(1) .setPartitionEpoch(2) .setLeaderEpoch(3) - .setNewIsrWithEpochs(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(asList(1, 2))); - - if (version >= 1) { - // Use the none default value; 1 - RECOVERING - partitionData.setLeaderRecoveryState((byte) 1); - } + .setNewIsrWithEpochs(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(asList(1, 2))) + .setLeaderRecoveryState((byte) 1); // non-default value AlterPartitionRequestData data = new AlterPartitionRequestData() .setBrokerEpoch(123L) .setBrokerId(1) .setTopics(singletonList(new AlterPartitionRequestData.TopicData() - .setTopicName("topic1") .setTopicId(Uuid.randomUuid()) .setPartitions(singletonList(partitionData)))); return new AlterPartitionRequest.Builder(data).build(version); } - private AlterPartitionResponse createAlterPartitionResponse(int version) { + private AlterPartitionResponse createAlterPartitionResponse() { AlterPartitionResponseData.PartitionData partitionData = new AlterPartitionResponseData.PartitionData() .setPartitionEpoch(1) .setIsr(asList(0, 1, 2)) .setErrorCode(Errors.NONE.code()) .setLeaderEpoch(2) - .setLeaderId(3); - - if (version >= 1) { - // Use the none default value; 1 - RECOVERING - partitionData.setLeaderRecoveryState((byte) 1); - } + .setLeaderId(3) + .setLeaderRecoveryState((byte) 1); // non-default value AlterPartitionResponseData data = new AlterPartitionResponseData() .setErrorCode(Errors.NONE.code()) .setThrottleTimeMs(123) .setTopics(singletonList(new AlterPartitionResponseData.TopicData() - .setTopicName("topic1") .setTopicId(Uuid.randomUuid()) .setPartitions(singletonList(partitionData)))); return new AlterPartitionResponse(data); diff --git a/core/src/main/java/kafka/server/MetadataVersionConfigValidator.java b/core/src/main/java/kafka/server/MetadataVersionConfigValidator.java index a62987be203..6f53ec092f6 100644 --- a/core/src/main/java/kafka/server/MetadataVersionConfigValidator.java +++ b/core/src/main/java/kafka/server/MetadataVersionConfigValidator.java @@ -52,7 +52,7 @@ public class MetadataVersionConfigValidator implements MetadataPublisher { ) { if (delta.featuresDelta() != null) { if (delta.metadataVersionChanged().isPresent()) { - onMetadataVersionChanged(newImage.features().metadataVersion()); + onMetadataVersionChanged(newImage.features().metadataVersionOrThrow()); } } } diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 22074e8da24..2a10afb3b5d 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -119,8 +119,8 @@ object Partition { time: Time, replicaManager: ReplicaManager): Partition = { Partition( - topicPartition = topicIdPartition.topicPartition(), - topicId = Option(topicIdPartition.topicId()), + topicPartition = topicIdPartition.topicPartition, + topicId = Some(topicIdPartition.topicId), time = time, replicaManager = replicaManager) } @@ -1814,7 +1814,7 @@ class Partition(val topicPartition: TopicPartition, private def submitAlterPartition(proposedIsrState: PendingPartitionChange): CompletableFuture[LeaderAndIsr] = { debug(s"Submitting ISR state change $proposedIsrState") val future = alterIsrManager.submit( - new TopicIdPartition(topicId.getOrElse(Uuid.ZERO_UUID), topicPartition), + new org.apache.kafka.server.common.TopicIdPartition(topicId.getOrElse(throw new IllegalStateException("Topic id not set for " + topicPartition)), topicPartition.partition), proposedIsrState.sentLeaderAndIsr ) future.whenComplete { (leaderAndIsr, e) => diff --git a/core/src/main/scala/kafka/server/AlterPartitionManager.scala b/core/src/main/scala/kafka/server/AlterPartitionManager.scala index fa1c3602bee..9f5052349c8 100644 --- a/core/src/main/scala/kafka/server/AlterPartitionManager.scala +++ b/core/src/main/scala/kafka/server/AlterPartitionManager.scala @@ -16,23 +16,18 @@ */ package kafka.server -import java.util import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.{CompletableFuture, ConcurrentHashMap} import kafka.utils.Logging import org.apache.kafka.clients.ClientResponse -import org.apache.kafka.common.TopicIdPartition -import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.Uuid import org.apache.kafka.common.errors.OperationNotAttemptedException import org.apache.kafka.common.message.AlterPartitionRequestData import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.requests.RequestHeader import org.apache.kafka.common.requests.{AlterPartitionRequest, AlterPartitionResponse} import org.apache.kafka.common.utils.Time import org.apache.kafka.metadata.{LeaderAndIsr, LeaderRecoveryState} -import org.apache.kafka.server.common.{ControllerRequestCompletionHandler, MetadataVersion, NodeToControllerChannelManager} +import org.apache.kafka.server.common.{ControllerRequestCompletionHandler, NodeToControllerChannelManager, TopicIdPartition} import org.apache.kafka.server.util.Scheduler import scala.collection.mutable @@ -70,7 +65,6 @@ object AlterPartitionManager { */ def apply( config: KafkaConfig, - metadataCache: MetadataCache, scheduler: Scheduler, controllerNodeProvider: ControllerNodeProvider, time: Time, @@ -92,8 +86,7 @@ object AlterPartitionManager { scheduler = scheduler, time = time, brokerId = config.brokerId, - brokerEpochSupplier = brokerEpochSupplier, - metadataVersionSupplier = () => metadataCache.metadataVersion() + brokerEpochSupplier = brokerEpochSupplier ) } } @@ -104,17 +97,10 @@ class DefaultAlterPartitionManager( val time: Time, val brokerId: Int, val brokerEpochSupplier: () => Long, - val metadataVersionSupplier: () => MetadataVersion ) extends AlterPartitionManager with Logging { - // Used to allow only one pending ISR update per partition (visible for testing). - // Note that we key items by TopicPartition despite using TopicIdPartition while - // submitting changes. This is done to ensure that topics with the same name but - // with a different topic id or no topic id collide here. When a topic is deleted - // and re-created, we cannot have two entries in this Map especially if we cannot - // use an AlterPartition request version which supports topic ids in the end because - // the two updates with the same name would be merged together. - private[server] val unsentIsrUpdates: util.Map[TopicPartition, AlterPartitionItem] = new ConcurrentHashMap[TopicPartition, AlterPartitionItem]() + // Used to allow only one pending ISR update per partition (visible for testing) + private[server] val unsentIsrUpdates = new ConcurrentHashMap[TopicIdPartition, AlterPartitionItem]() // Used to allow only one in-flight request at a time private val inflightRequest: AtomicBoolean = new AtomicBoolean(false) @@ -133,7 +119,7 @@ class DefaultAlterPartitionManager( ): CompletableFuture[LeaderAndIsr] = { val future = new CompletableFuture[LeaderAndIsr]() val alterPartitionItem = AlterPartitionItem(topicIdPartition, leaderAndIsr, future) - val enqueued = unsentIsrUpdates.putIfAbsent(alterPartitionItem.topicIdPartition.topicPartition, alterPartitionItem) == null + val enqueued = unsentIsrUpdates.putIfAbsent(alterPartitionItem.topicIdPartition, alterPartitionItem) == null if (enqueued) { maybePropagateIsrChanges() } else { @@ -161,7 +147,7 @@ class DefaultAlterPartitionManager( private def sendRequest(inflightAlterPartitionItems: Seq[AlterPartitionItem]): Unit = { val brokerEpoch = brokerEpochSupplier() - val (request, topicNamesByIds) = buildRequest(inflightAlterPartitionItems, brokerEpoch) + val request = buildRequest(inflightAlterPartitionItems, brokerEpoch) debug(s"Sending AlterPartition to controller $request") // We will not timeout AlterPartition request, instead letting it retry indefinitely @@ -182,11 +168,9 @@ class DefaultAlterPartitionManager( Errors.UNSUPPORTED_VERSION } else { handleAlterPartitionResponse( - response.requestHeader, response.responseBody.asInstanceOf[AlterPartitionResponse], brokerEpoch, - inflightAlterPartitionItems, - topicNamesByIds + inflightAlterPartitionItems ) } } finally { @@ -218,59 +202,40 @@ class DefaultAlterPartitionManager( * supported by the controller. The final decision is taken when the AlterPartitionRequest * is built in the network client based on the advertised api versions of the controller. * - * @return A tuple containing the AlterPartitionRequest.Builder and a mapping from - * topic id to topic name. This mapping is used in the response handling. + * @return an AlterPartitionRequest.Builder with the provided parameters. */ private def buildRequest( inflightAlterPartitionItems: Seq[AlterPartitionItem], brokerEpoch: Long - ): (AlterPartitionRequest.Builder, mutable.Map[Uuid, String]) = { - val metadataVersion = metadataVersionSupplier() - // We build this mapping in order to map topic id back to their name when we - // receive the response. We cannot rely on the metadata cache for this because - // the metadata cache is updated after the partition state so it might not know - // yet about a topic id already used here. - val topicNamesByIds = mutable.HashMap[Uuid, String]() - + ): AlterPartitionRequest.Builder = { val message = new AlterPartitionRequestData() .setBrokerId(brokerId) .setBrokerEpoch(brokerEpoch) - inflightAlterPartitionItems.groupBy(_.topicIdPartition.topic).foreach { case (topicName, items) => - val topicId = items.head.topicIdPartition.topicId - topicNamesByIds(topicId) = topicName - - // Both the topic name and the topic id are set here because at this stage - // we don't know which version of the request will be used. - val topicData = new AlterPartitionRequestData.TopicData() - .setTopicName(topicName) - .setTopicId(topicId) + inflightAlterPartitionItems.groupBy(_.topicIdPartition.topicId).foreach { case (topicId, items) => + val topicData = new AlterPartitionRequestData.TopicData().setTopicId(topicId) message.topics.add(topicData) items.foreach { item => val partitionData = new AlterPartitionRequestData.PartitionData() - .setPartitionIndex(item.topicIdPartition.partition) + .setPartitionIndex(item.topicIdPartition.partitionId) .setLeaderEpoch(item.leaderAndIsr.leaderEpoch) .setNewIsrWithEpochs(item.leaderAndIsr.isrWithBrokerEpoch) .setPartitionEpoch(item.leaderAndIsr.partitionEpoch) - if (metadataVersion.isLeaderRecoverySupported) { - partitionData.setLeaderRecoveryState(item.leaderAndIsr.leaderRecoveryState.value) - } + partitionData.setLeaderRecoveryState(item.leaderAndIsr.leaderRecoveryState.value) topicData.partitions.add(partitionData) } } - (new AlterPartitionRequest.Builder(message), topicNamesByIds) + new AlterPartitionRequest.Builder(message) } private def handleAlterPartitionResponse( - requestHeader: RequestHeader, alterPartitionResp: AlterPartitionResponse, sentBrokerEpoch: Long, inflightAlterPartitionItems: Seq[AlterPartitionItem], - topicNamesByIds: mutable.Map[Uuid, String] ): Errors = { val data = alterPartitionResp.data @@ -284,37 +249,31 @@ class DefaultAlterPartitionManager( case Errors.NONE => // Collect partition-level responses to pass to the callbacks - val partitionResponses = new mutable.HashMap[TopicPartition, Either[Errors, LeaderAndIsr]]() + val partitionResponses = new mutable.HashMap[TopicIdPartition, Either[Errors, LeaderAndIsr]]() data.topics.forEach { topic => - // Topic IDs are used since version 2 of the AlterPartition API. - val topicName = if (requestHeader.apiVersion > 1) topicNamesByIds.get(topic.topicId).orNull else topic.topicName - if (topicName == null || topicName.isEmpty) { - error(s"Received an unexpected topic $topic in the alter partition response, ignoring it.") - } else { - topic.partitions.forEach { partition => - val tp = new TopicPartition(topicName, partition.partitionIndex) - val apiError = Errors.forCode(partition.errorCode) - debug(s"Controller successfully handled AlterPartition request for $tp: $partition") - if (apiError == Errors.NONE) { - LeaderRecoveryState.optionalOf(partition.leaderRecoveryState).toScala match { - case Some(leaderRecoveryState) => - partitionResponses(tp) = Right( - new LeaderAndIsr( - partition.leaderId, - partition.leaderEpoch, - partition.isr, - leaderRecoveryState, - partition.partitionEpoch - ) + topic.partitions.forEach { partition => + val tp = new TopicIdPartition(topic.topicId, partition.partitionIndex) + val apiError = Errors.forCode(partition.errorCode) + debug(s"Controller successfully handled AlterPartition request for $tp: $partition") + if (apiError == Errors.NONE) { + LeaderRecoveryState.optionalOf(partition.leaderRecoveryState).toScala match { + case Some(leaderRecoveryState) => + partitionResponses(tp) = Right( + new LeaderAndIsr( + partition.leaderId, + partition.leaderEpoch, + partition.isr, + leaderRecoveryState, + partition.partitionEpoch ) + ) - case None => - error(s"Controller returned an invalid leader recovery state (${partition.leaderRecoveryState}) for $tp: $partition") - partitionResponses(tp) = Left(Errors.UNKNOWN_SERVER_ERROR) - } - } else { - partitionResponses(tp) = Left(apiError) + case None => + error(s"Controller returned an invalid leader recovery state (${partition.leaderRecoveryState}) for $tp: $partition") + partitionResponses(tp) = Left(Errors.UNKNOWN_SERVER_ERROR) } + } else { + partitionResponses(tp) = Left(apiError) } } } @@ -323,11 +282,11 @@ class DefaultAlterPartitionManager( // partition was somehow erroneously excluded from the response. Note that these callbacks are run from // the leaderIsrUpdateLock write lock in Partition#sendAlterPartitionRequest inflightAlterPartitionItems.foreach { inflightAlterPartition => - partitionResponses.get(inflightAlterPartition.topicIdPartition.topicPartition) match { + partitionResponses.get(inflightAlterPartition.topicIdPartition) match { case Some(leaderAndIsrOrError) => // Regardless of callback outcome, we need to clear from the unsent updates map to unblock further // updates. We clear it now to allow the callback to submit a new update if needed. - unsentIsrUpdates.remove(inflightAlterPartition.topicIdPartition.topicPartition) + unsentIsrUpdates.remove(inflightAlterPartition.topicIdPartition) leaderAndIsrOrError match { case Left(error) => inflightAlterPartition.future.completeExceptionally(error.exception) case Right(leaderAndIsr) => inflightAlterPartition.future.complete(leaderAndIsr) diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 4bd7eba6be8..34be6b9c128 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -280,7 +280,6 @@ class BrokerServer( alterPartitionManager = AlterPartitionManager( config, - metadataCache, scheduler = kafkaScheduler, controllerNodeProvider, time = time, diff --git a/core/src/main/scala/kafka/server/ControllerRegistrationManager.scala b/core/src/main/scala/kafka/server/ControllerRegistrationManager.scala index f15e3e50cfc..9ee73e1c509 100644 --- a/core/src/main/scala/kafka/server/ControllerRegistrationManager.scala +++ b/core/src/main/scala/kafka/server/ControllerRegistrationManager.scala @@ -34,6 +34,7 @@ import org.apache.kafka.queue.{EventQueue, KafkaEventQueue} import org.apache.kafka.server.common.{ControllerRequestCompletionHandler, MetadataVersion, NodeToControllerChannelManager} import scala.jdk.CollectionConverters._ +import scala.jdk.OptionConverters._ /** * The controller registration manager handles registering this controller with the controller @@ -87,7 +88,7 @@ class ControllerRegistrationManager( /** * The current metadata version that is in effect. Only read or written from the event queue thread. */ - private var metadataVersion: MetadataVersion = MetadataVersion.MINIMUM_KRAFT_VERSION + private var metadataVersion: Option[MetadataVersion] = None /** * True if we're registered. Only read or written from the event queue thread. @@ -172,7 +173,7 @@ class ControllerRegistrationManager( override def run(): Unit = { try { if (delta.featuresDelta() != null) { - metadataVersion = newImage.features().metadataVersion() + metadataVersion = newImage.features().metadataVersion().toScala } if (delta.clusterDelta() != null) { if (delta.clusterDelta().changedControllers().containsKey(nodeId)) { @@ -197,12 +198,14 @@ class ControllerRegistrationManager( } private def maybeSendControllerRegistration(): Unit = { + val metadataVersion = this.metadataVersion if (registeredInLog) { debug("maybeSendControllerRegistration: controller is already registered.") } else if (_channelManager == null) { - debug("maybeSendControllerRegistration: cannot register yet because the channel manager has " + - "not been initialized.") - } else if (!metadataVersion.isControllerRegistrationSupported) { + debug("maybeSendControllerRegistration: cannot register yet because the channel manager has not been initialized.") + } else if (metadataVersion.isEmpty) { + info("maybeSendControllerRegistration: cannot register yet because the metadata.version is not known yet.") + } else if (!metadataVersion.get.isControllerRegistrationSupported) { info("maybeSendControllerRegistration: cannot register yet because the metadata.version is " + s"still $metadataVersion, which does not support KIP-919 controller registration.") } else if (pendingRpc) { diff --git a/core/src/main/scala/kafka/server/KafkaRaftServer.scala b/core/src/main/scala/kafka/server/KafkaRaftServer.scala index 998fa4410c5..34ee4a725f2 100644 --- a/core/src/main/scala/kafka/server/KafkaRaftServer.scala +++ b/core/src/main/scala/kafka/server/KafkaRaftServer.scala @@ -181,7 +181,7 @@ object KafkaRaftServer { } // Load the BootstrapMetadata. - val bootstrapDirectory = new BootstrapDirectory(config.metadataLogDir, Optional.empty()) + val bootstrapDirectory = new BootstrapDirectory(config.metadataLogDir) val bootstrapMetadata = bootstrapDirectory.read() (metaPropsEnsemble, bootstrapMetadata) } diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index f2e030b3e1f..885cfc82be5 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -2694,7 +2694,7 @@ class ReplicaManager(val config: KafkaConfig, def applyDelta(delta: TopicsDelta, newImage: MetadataImage): Unit = { // Before taking the lock, compute the local changes val localChanges = delta.localChanges(config.nodeId) - val metadataVersion = newImage.features().metadataVersion() + val metadataVersion = newImage.features().metadataVersionOrThrow() replicaStateChangeLock.synchronized { // Handle deleted partitions. We need to do this first because we might subsequently diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala index 08738215b21..1985f04348f 100644 --- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala +++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala @@ -119,7 +119,7 @@ class BrokerMetadataPublisher( // Publish the new metadata image to the metadata cache. metadataCache.setImage(newImage) - val metadataVersionLogMsg = s"metadata.version ${newImage.features().metadataVersion()}" + def metadataVersionLogMsg = s"metadata.version ${newImage.features().metadataVersion()}" if (_firstPublish) { info(s"Publishing initial metadata at offset $highestOffsetAndEpoch with $metadataVersionLogMsg.") diff --git a/core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala b/core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala index 4b6fa5f3a5f..75165ac7af9 100644 --- a/core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala +++ b/core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala @@ -513,7 +513,7 @@ class KRaftMetadataCache( _currentImage.scram().describe(request) } - override def metadataVersion(): MetadataVersion = _currentImage.features().metadataVersion() + override def metadataVersion(): MetadataVersion = _currentImage.features().metadataVersionOrThrow() override def features(): FinalizedFeatures = { val image = _currentImage @@ -522,7 +522,8 @@ class KRaftMetadataCache( if (kraftVersionLevel > 0) { finalizedFeatures.put(KRaftVersion.FEATURE_NAME, kraftVersionLevel) } - new FinalizedFeatures(image.features().metadataVersion(), + new FinalizedFeatures( + image.features().metadataVersionOrThrow(), finalizedFeatures, image.highestOffsetAndEpoch().offset) } diff --git a/core/src/main/scala/kafka/tools/StorageTool.scala b/core/src/main/scala/kafka/tools/StorageTool.scala index 89c3cdbca20..40892bca38c 100644 --- a/core/src/main/scala/kafka/tools/StorageTool.scala +++ b/core/src/main/scala/kafka/tools/StorageTool.scala @@ -184,7 +184,7 @@ object StorageTool extends Logging { } catch { case e: IllegalArgumentException => throw new TerseFailure(s"Unknown release version '$releaseVersion'. Supported versions are: " + - s"${MetadataVersion.MINIMUM_BOOTSTRAP_VERSION.version} to ${MetadataVersion.LATEST_PRODUCTION.version}") + s"${MetadataVersion.MINIMUM_VERSION.version} to ${MetadataVersion.LATEST_PRODUCTION.version}") } } @@ -311,7 +311,7 @@ object StorageTool extends Logging { formatParser.addArgument("--release-version", "-r") .action(store()) .help(s"The release version to use for the initial feature settings. The minimum is " + - s"${MetadataVersion.IBP_3_0_IV1}; the default is ${MetadataVersion.LATEST_PRODUCTION}") + s"${MetadataVersion.MINIMUM_VERSION}; the default is ${MetadataVersion.LATEST_PRODUCTION}") formatParser.addArgument("--feature", "-f") .help("The setting to use for a specific feature, in feature=level format. For example: `kraft.version=1`.") @@ -344,7 +344,7 @@ object StorageTool extends Logging { versionMappingParser.addArgument("--release-version", "-r") .action(store()) .help(s"The release version to use for the corresponding feature mapping. The minimum is " + - s"${MetadataVersion.IBP_3_0_IV1}; the default is ${MetadataVersion.LATEST_PRODUCTION}") + s"${MetadataVersion.MINIMUM_VERSION}; the default is ${MetadataVersion.LATEST_PRODUCTION}") } private def addFeatureDependenciesParser(subparsers: Subparsers): Unit = { diff --git a/core/src/main/scala/kafka/tools/TestRaftServer.scala b/core/src/main/scala/kafka/tools/TestRaftServer.scala index 60ffbf7d1b2..418a276bbd1 100644 --- a/core/src/main/scala/kafka/tools/TestRaftServer.scala +++ b/core/src/main/scala/kafka/tools/TestRaftServer.scala @@ -81,7 +81,7 @@ class TestRaftServer( val apiVersionManager = new SimpleApiVersionManager( ListenerType.CONTROLLER, true, - () => FinalizedFeatures.fromKRaftVersion(MetadataVersion.MINIMUM_KRAFT_VERSION)) + () => FinalizedFeatures.fromKRaftVersion(MetadataVersion.MINIMUM_VERSION)) socketServer = new SocketServer(config, metrics, time, credentialProvider, apiVersionManager) val endpoints = Endpoints.fromInetSocketAddresses( diff --git a/core/src/test/java/kafka/admin/ConfigCommandIntegrationTest.java b/core/src/test/java/kafka/admin/ConfigCommandIntegrationTest.java index 9aa7c9da2fa..474b8c58fd5 100644 --- a/core/src/test/java/kafka/admin/ConfigCommandIntegrationTest.java +++ b/core/src/test/java/kafka/admin/ConfigCommandIntegrationTest.java @@ -380,7 +380,7 @@ public class ConfigCommandIntegrationTest { @ClusterTest( // Must be at greater than 1MB per cleaner thread, set to 2M+2 so that we can set 2 cleaner threads. serverProperties = {@ClusterConfigProperty(key = "log.cleaner.dedupe.buffer.size", value = "2097154")}, - metadataVersion = MetadataVersion.IBP_3_3_IV0 + metadataVersion = MetadataVersion.IBP_3_9_IV0 ) public void testUnsupportedVersionException() { try (Admin client = cluster.admin()) { diff --git a/core/src/test/java/kafka/server/ReconfigurableQuorumIntegrationTest.java b/core/src/test/java/kafka/server/ReconfigurableQuorumIntegrationTest.java index 539c11b7c7e..981217ce287 100644 --- a/core/src/test/java/kafka/server/ReconfigurableQuorumIntegrationTest.java +++ b/core/src/test/java/kafka/server/ReconfigurableQuorumIntegrationTest.java @@ -44,7 +44,8 @@ public class ReconfigurableQuorumIntegrationTest { static void checkKRaftVersions(Admin admin, short finalized) throws Exception { FeatureMetadata featureMetadata = admin.describeFeatures().featureMetadata().get(); if (finalized > 0) { - assertTrue(featureMetadata.finalizedFeatures().containsKey(KRaftVersion.FEATURE_NAME)); + assertTrue(featureMetadata.finalizedFeatures().containsKey(KRaftVersion.FEATURE_NAME), + "finalizedFeatures does not contain " + KRaftVersion.FEATURE_NAME + ", finalizedFeatures: " + featureMetadata.finalizedFeatures()); assertEquals(finalized, featureMetadata.finalizedFeatures(). get(KRaftVersion.FEATURE_NAME).minVersionLevel()); assertEquals(finalized, featureMetadata.finalizedFeatures(). @@ -64,13 +65,13 @@ public class ReconfigurableQuorumIntegrationTest { new TestKitNodes.Builder(). setNumBrokerNodes(1). setNumControllerNodes(1). - build()).build() - ) { + build() + ).build()) { cluster.format(); cluster.startup(); try (Admin admin = Admin.create(cluster.clientProperties())) { TestUtils.retryOnExceptionWithTimeout(30_000, () -> { - checkKRaftVersions(admin, (short) 0); + checkKRaftVersions(admin, KRaftVersion.KRAFT_VERSION_0.featureLevel()); }); } } @@ -82,14 +83,14 @@ public class ReconfigurableQuorumIntegrationTest { new TestKitNodes.Builder(). setNumBrokerNodes(1). setNumControllerNodes(1). - setFeature(KRaftVersion.FEATURE_NAME, (short) 1). - build()).build() - ) { + setFeature(KRaftVersion.FEATURE_NAME, KRaftVersion.KRAFT_VERSION_1.featureLevel()). + build() + ).build()) { cluster.format(); cluster.startup(); try (Admin admin = Admin.create(cluster.clientProperties())) { TestUtils.retryOnExceptionWithTimeout(30_000, () -> { - checkKRaftVersions(admin, (short) 1); + checkKRaftVersions(admin, KRaftVersion.KRAFT_VERSION_1.featureLevel()); }); } } @@ -110,9 +111,9 @@ public class ReconfigurableQuorumIntegrationTest { new TestKitNodes.Builder(). setNumBrokerNodes(1). setNumControllerNodes(3). - setFeature(KRaftVersion.FEATURE_NAME, (short) 1). - build()).build() - ) { + setFeature(KRaftVersion.FEATURE_NAME, KRaftVersion.KRAFT_VERSION_1.featureLevel()). + build() + ).build()) { cluster.format(); cluster.startup(); try (Admin admin = Admin.create(cluster.clientProperties())) { @@ -135,7 +136,7 @@ public class ReconfigurableQuorumIntegrationTest { new TestKitNodes.Builder(). setNumBrokerNodes(1). setNumControllerNodes(4). - setFeature(KRaftVersion.FEATURE_NAME, (short) 1). + setFeature(KRaftVersion.FEATURE_NAME, KRaftVersion.KRAFT_VERSION_1.featureLevel()). build()).build() ) { cluster.format(); diff --git a/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIntegrationTest.scala b/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIntegrationTest.scala index e1fa03403b3..674a379cfea 100644 --- a/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIntegrationTest.scala @@ -55,7 +55,7 @@ import scala.jdk.CollectionConverters._ class ProducerIntegrationTest { @ClusterTests(Array( - new ClusterTest(metadataVersion = MetadataVersion.IBP_3_3_IV0) + new ClusterTest(metadataVersion = MetadataVersion.IBP_3_3_IV3) )) def testUniqueProducerIds(clusterInstance: ClusterInstance): Unit = { verifyUniqueIds(clusterInstance) diff --git a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala index 0f3f39ff1db..06cb2e2eebb 100644 --- a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala +++ b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala @@ -437,18 +437,6 @@ class KRaftClusterTest { } } - @Test - def testCreateClusterInvalidMetadataVersion(): Unit = { - assertEquals("Bootstrap metadata.version before 3.3-IV0 are not supported. Can't load " + - "metadata from testkit", assertThrows(classOf[RuntimeException], () => { - new KafkaClusterTestKit.Builder( - new TestKitNodes.Builder(). - setBootstrapMetadataVersion(MetadataVersion.IBP_3_0_IV1). - setNumBrokerNodes(1). - setNumControllerNodes(1).build()).build() - }).getMessage) - } - private def doOnStartedKafkaCluster(nodes: TestKitNodes) (action: KafkaClusterTestKit => Unit): Unit = { val cluster = new KafkaClusterTestKit.Builder(nodes).build() @@ -990,7 +978,7 @@ class KRaftClusterTest { def testUpdateMetadataVersion(): Unit = { val cluster = new KafkaClusterTestKit.Builder( new TestKitNodes.Builder(). - setBootstrapMetadataVersion(MetadataVersion.MINIMUM_BOOTSTRAP_VERSION). + setBootstrapMetadataVersion(MetadataVersion.MINIMUM_VERSION). setNumBrokerNodes(4). setNumControllerNodes(3).build()).build() try { @@ -1008,8 +996,8 @@ class KRaftClusterTest { } finally { admin.close() } - TestUtils.waitUntilTrue(() => cluster.brokers().get(0).metadataCache.currentImage().features().metadataVersion().equals(MetadataVersion.latestTesting()), - "Timed out waiting for metadata.version update") + TestUtils.waitUntilTrue(() => cluster.brokers().get(0).metadataCache.currentImage().features().metadataVersion() + .equals(Optional.of(MetadataVersion.latestTesting())), "Timed out waiting for metadata.version update") } finally { cluster.close() } @@ -1143,7 +1131,7 @@ class KRaftClusterTest { def testSingleControllerSingleBrokerCluster(): Unit = { val cluster = new KafkaClusterTestKit.Builder( new TestKitNodes.Builder(). - setBootstrapMetadataVersion(MetadataVersion.MINIMUM_BOOTSTRAP_VERSION). + setBootstrapMetadataVersion(MetadataVersion.MINIMUM_VERSION). setNumBrokerNodes(1). setNumControllerNodes(1).build()).build() try { diff --git a/core/src/test/scala/integration/kafka/server/MetadataVersionIntegrationTest.scala b/core/src/test/scala/integration/kafka/server/MetadataVersionIntegrationTest.scala index a0637a5a4bf..490ebc48c16 100644 --- a/core/src/test/scala/integration/kafka/server/MetadataVersionIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/server/MetadataVersionIntegrationTest.scala @@ -29,12 +29,11 @@ import scala.jdk.CollectionConverters._ class MetadataVersionIntegrationTest { @ClusterTests(value = Array( - new ClusterTest(types = Array(Type.KRAFT), metadataVersion = MetadataVersion.IBP_3_3_IV0), - new ClusterTest(types = Array(Type.KRAFT), metadataVersion = MetadataVersion.IBP_3_3_IV1), - new ClusterTest(types = Array(Type.KRAFT), metadataVersion = MetadataVersion.IBP_3_3_IV2), new ClusterTest(types = Array(Type.KRAFT), metadataVersion = MetadataVersion.IBP_3_3_IV3), new ClusterTest(types = Array(Type.KRAFT), metadataVersion = MetadataVersion.IBP_3_4_IV0), - new ClusterTest(types = Array(Type.KRAFT), metadataVersion = MetadataVersion.IBP_3_4_IV0) + new ClusterTest(types = Array(Type.KRAFT), metadataVersion = MetadataVersion.IBP_3_5_IV0), + new ClusterTest(types = Array(Type.KRAFT), metadataVersion = MetadataVersion.IBP_3_6_IV0), + new ClusterTest(types = Array(Type.KRAFT), metadataVersion = MetadataVersion.IBP_3_6_IV1) )) def testBasicMetadataVersionUpgrade(clusterInstance: ClusterInstance): Unit = { val admin = clusterInstance.admin() @@ -45,7 +44,7 @@ class MetadataVersionIntegrationTest { assertEquals(ff.maxVersionLevel(), clusterInstance.config().metadataVersion().featureLevel()) // Update to new version - val updateVersion = MetadataVersion.IBP_3_5_IV1.featureLevel.shortValue + val updateVersion = MetadataVersion.IBP_3_7_IV1.featureLevel.shortValue val updateResult = admin.updateFeatures( Map("metadata.version" -> new FeatureUpdate(updateVersion, UpgradeType.UPGRADE)).asJava, new UpdateFeaturesOptions()) updateResult.all().get() @@ -61,11 +60,11 @@ class MetadataVersionIntegrationTest { } } - @ClusterTest(types = Array(Type.KRAFT), metadataVersion = MetadataVersion.IBP_3_3_IV0) + @ClusterTest(types = Array(Type.KRAFT), metadataVersion = MetadataVersion.IBP_3_9_IV0) def testUpgradeSameVersion(clusterInstance: ClusterInstance): Unit = { val admin = clusterInstance.admin() try { - val updateVersion = MetadataVersion.IBP_3_3_IV0.featureLevel.shortValue + val updateVersion = MetadataVersion.IBP_3_9_IV0.featureLevel.shortValue val updateResult = admin.updateFeatures( Map("metadata.version" -> new FeatureUpdate(updateVersion, UpgradeType.UPGRADE)).asJava, new UpdateFeaturesOptions()) updateResult.all().get() diff --git a/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala b/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala index 6e9eeeb5cd7..4b9e7569b6e 100644 --- a/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala +++ b/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala @@ -23,13 +23,13 @@ import org.apache.kafka.common.compress.Compression import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset -import org.apache.kafka.common.metadata.{PartitionChangeRecord, PartitionRecord, TopicRecord} +import org.apache.kafka.common.metadata.{FeatureLevelRecord, PartitionChangeRecord, PartitionRecord, TopicRecord} import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record.{MemoryRecords, SimpleRecord} import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.image.{MetadataDelta, MetadataImage, MetadataProvenance} -import org.apache.kafka.server.common.{KRaftVersion, OffsetAndEpoch} +import org.apache.kafka.server.common.{KRaftVersion, MetadataVersion, OffsetAndEpoch} import org.apache.kafka.server.network.BrokerEndPoint import org.apache.kafka.server.util.{MockScheduler, MockTime} import org.apache.kafka.storage.internals.log.{AppendOrigin, LogDirFailureChannel} @@ -75,6 +75,10 @@ class LocalLeaderEndPointTest extends Logging { ) val delta = new MetadataDelta(MetadataImage.EMPTY) + delta.replay(new FeatureLevelRecord() + .setName(MetadataVersion.FEATURE_NAME) + .setFeatureLevel(MetadataVersion.MINIMUM_VERSION.featureLevel()) + ) delta.replay(new TopicRecord() .setName(topic) .setTopicId(topicId) diff --git a/core/src/test/scala/kafka/server/RemoteLeaderEndPointTest.scala b/core/src/test/scala/kafka/server/RemoteLeaderEndPointTest.scala index 0b1198a64c3..90edb4e306b 100644 --- a/core/src/test/scala/kafka/server/RemoteLeaderEndPointTest.scala +++ b/core/src/test/scala/kafka/server/RemoteLeaderEndPointTest.scala @@ -65,7 +65,7 @@ class RemoteLeaderEndPointTest { blockingSend = new MockBlockingSender(offsets = new util.HashMap[TopicPartition, EpochEndOffset](), sourceBroker = sourceBroker, time = time) endPoint = new RemoteLeaderEndPoint(logPrefix, blockingSend, fetchSessionHandler, - config, replicaManager, QuotaFactory.UNBOUNDED_QUOTA, () => MetadataVersion.MINIMUM_KRAFT_VERSION, () => currentBrokerEpoch) + config, replicaManager, QuotaFactory.UNBOUNDED_QUOTA, () => MetadataVersion.MINIMUM_VERSION, () => currentBrokerEpoch) } @Test @@ -144,4 +144,4 @@ class RemoteLeaderEndPointTest { assertTrue(newPartitionsWithError.isEmpty) assertEquals(if (version < 15) -1L else 2L, newFetchRequestOpt.get.fetchRequest.build(version).replicaEpoch) } -} \ No newline at end of file +} diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala index 86b72fab047..356496fe9d4 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala @@ -24,7 +24,6 @@ import java.util.concurrent.atomic.AtomicBoolean import kafka.log._ import kafka.server._ import kafka.utils._ -import org.apache.kafka.common.TopicIdPartition import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.record.{MemoryRecords, SimpleRecord} @@ -33,7 +32,7 @@ import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.coordinator.transaction.TransactionLogConfig import org.apache.kafka.metadata.{LeaderAndIsr, MockConfigRepository} -import org.apache.kafka.server.common.RequestLocal +import org.apache.kafka.server.common.{RequestLocal, TopicIdPartition} import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.storage.log.{FetchIsolation, FetchParams} import org.apache.kafka.server.util.MockTime @@ -74,6 +73,7 @@ class PartitionLockTest extends Logging { var partition: Partition = _ private val topicPartition = new TopicPartition("test-topic", 0) + private val topicId = Uuid.randomUuid() @BeforeEach def setUp(): Unit = { @@ -332,7 +332,7 @@ class PartitionLockTest extends Logging { } } - val topicIdPartition = new TopicIdPartition(partition.topicId.getOrElse(Uuid.ZERO_UUID), topicPartition) + val topicIdPartition = new TopicIdPartition(topicId, topicPartition.partition) when(offsetCheckpoints.fetch( ArgumentMatchers.anyString, ArgumentMatchers.eq(topicPartition) @@ -342,7 +342,7 @@ class PartitionLockTest extends Logging { ArgumentMatchers.any[LeaderAndIsr] )).thenReturn(new CompletableFuture[LeaderAndIsr]()) - partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, None) + partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, Some(topicId)) val controllerEpoch = 0 val replicas = (0 to numReplicaFetchers).map(i => Integer.valueOf(brokerId + i)).toList.asJava @@ -356,7 +356,7 @@ class PartitionLockTest extends Logging { .setIsr(isr) .setPartitionEpoch(1) .setReplicas(replicas) - .setIsNew(true), offsetCheckpoints, None), "Expected become leader transition to succeed") + .setIsNew(true), offsetCheckpoints, Some(topicId)), "Expected become leader transition to succeed") partition } diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala index 5d028c7199e..9aaaa4a64da 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala @@ -52,7 +52,7 @@ import org.apache.kafka.common.replica.ClientMetadata import org.apache.kafka.common.replica.ClientMetadata.DefaultClientMetadata import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.coordinator.transaction.TransactionLogConfig -import org.apache.kafka.server.common.{ControllerRequestCompletionHandler, MetadataVersion, NodeToControllerChannelManager, RequestLocal} +import org.apache.kafka.server.common.{ControllerRequestCompletionHandler, NodeToControllerChannelManager, RequestLocal} import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.server.purgatory.{DelayedOperationPurgatory, TopicPartitionOperationKey} import org.apache.kafka.server.share.fetch.DelayedShareFetchPartitionKey @@ -365,7 +365,7 @@ class PartitionTest extends AbstractPartitionTest { .setPartitionEpoch(partitionEpoch) .setReplicas(replicas.map(Int.box).asJava) .setIsNew(true), - offsetCheckpoints, None + offsetCheckpoints, topicId )) assertThrows(classOf[UnknownLeaderEpochException], () => { @@ -475,7 +475,7 @@ class PartitionTest extends AbstractPartitionTest { val localLog = new LocalLog(log.dir, log.config, segments, offsets.recoveryPoint, offsets.nextOffsetMetadata, mockTime.scheduler, mockTime, log.topicPartition, logDirFailureChannel) - new SlowLog(log, offsets.logStartOffset, localLog, leaderEpochCache, producerStateManager, appendSemaphore) + new SlowLog(log, topicId, offsets.logStartOffset, localLog, leaderEpochCache, producerStateManager, appendSemaphore) } } @@ -802,7 +802,7 @@ class PartitionTest extends AbstractPartitionTest { .setReplicas(replicas.map(Int.box).asJava) .setIsNew(true) - assertTrue(partition.makeLeader(leaderState, offsetCheckpoints, None), "Expected first makeLeader() to return 'leader changed'") + assertTrue(partition.makeLeader(leaderState, offsetCheckpoints, topicId), "Expected first makeLeader() to return 'leader changed'") assertEquals(leaderEpoch, partition.getLeaderEpoch, "Current leader epoch") assertEquals(Set[Integer](leader, follower2), partition.partitionState.isr, "ISR") @@ -1145,7 +1145,7 @@ class PartitionTest extends AbstractPartitionTest { .setPartitionEpoch(1) .setReplicas(replicas.map(Int.box).asJava) .setIsNew(true) - assertTrue(partition.makeLeader(leaderState, offsetCheckpoints, None), "Expected first makeLeader() to return 'leader changed'") + assertTrue(partition.makeLeader(leaderState, offsetCheckpoints, topicId), "Expected first makeLeader() to return 'leader changed'") assertEquals(leaderEpoch, partition.getLeaderEpoch, "Current leader epoch") assertEquals(Set[Integer](leader, follower2), partition.partitionState.isr, "ISR") @@ -1182,7 +1182,7 @@ class PartitionTest extends AbstractPartitionTest { .setPartitionEpoch(1) .setReplicas(replicas.map(Int.box).asJava) .setIsNew(false) - assertTrue(partition.makeLeader(newLeaderState, offsetCheckpoints, None), + assertTrue(partition.makeLeader(newLeaderState, offsetCheckpoints, topicId), "Expected makeLeader() to return 'leader changed' after makeFollower()") val currentLeaderEpochStartOffset = partition.localLogOrException.logEndOffset @@ -1410,7 +1410,7 @@ class PartitionTest extends AbstractPartitionTest { @Test def testInvalidAlterPartitionRequestsAreNotRetried(): Unit = { - val log = logManager.getOrCreateLog(topicPartition, topicId = None) + val log = logManager.getOrCreateLog(topicPartition, topicId = topicId) seedLogData(log, numRecords = 10, leaderEpoch = 4) val controllerEpoch = 0 @@ -1462,7 +1462,7 @@ class PartitionTest extends AbstractPartitionTest { @Test def testIsrExpansion(): Unit = { - val log = logManager.getOrCreateLog(topicPartition, topicId = None) + val log = logManager.getOrCreateLog(topicPartition, topicId = topicId) seedLogData(log, numRecords = 10, leaderEpoch = 4) val controllerEpoch = 0 @@ -1526,7 +1526,7 @@ class PartitionTest extends AbstractPartitionTest { @Test def testIsrNotExpandedIfUpdateFails(): Unit = { - val log = logManager.getOrCreateLog(topicPartition, topicId = None) + val log = logManager.getOrCreateLog(topicPartition, topicId = topicId) seedLogData(log, numRecords = 10, leaderEpoch = 4) val controllerEpoch = 0 @@ -1682,7 +1682,7 @@ class PartitionTest extends AbstractPartitionTest { @ParameterizedTest @ValueSource(strings = Array("kraft")) def testIsrNotExpandedIfReplicaIsFencedOrShutdown(quorum: String): Unit = { - val log = logManager.getOrCreateLog(topicPartition, topicId = None) + val log = logManager.getOrCreateLog(topicPartition, topicId = topicId) seedLogData(log, numRecords = 10, leaderEpoch = 4) val controllerEpoch = 0 @@ -1789,7 +1789,7 @@ class PartitionTest extends AbstractPartitionTest { @Test def testIsrCanExpandedIfBrokerEpochsMatchWithKraftMetadataCache(): Unit = { - val log = logManager.getOrCreateLog(topicPartition, topicId = None) + val log = logManager.getOrCreateLog(topicPartition, topicId = topicId) seedLogData(log, numRecords = 10, leaderEpoch = 4) val controllerEpoch = 0 @@ -1949,7 +1949,7 @@ class PartitionTest extends AbstractPartitionTest { @Test def testIsrNotExpandedIfReplicaIsInControlledShutdown(): Unit = { - val log = logManager.getOrCreateLog(topicPartition, topicId = None) + val log = logManager.getOrCreateLog(topicPartition, topicId = topicId) seedLogData(log, numRecords = 10, leaderEpoch = 4) val controllerEpoch = 0 @@ -2100,7 +2100,7 @@ class PartitionTest extends AbstractPartitionTest { @Test def testMaybeShrinkIsr(): Unit = { - val log = logManager.getOrCreateLog(topicPartition, topicId = None) + val log = logManager.getOrCreateLog(topicPartition, topicId = topicId) seedLogData(log, numRecords = 10, leaderEpoch = 4) val controllerEpoch = 0 @@ -2183,7 +2183,7 @@ class PartitionTest extends AbstractPartitionTest { @Test def testHighWatermarkAdvanceShouldNotAdvanceWhenUnderMinISR(): Unit = { configRepository.setTopicConfig(topicPartition.topic, TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "3") - val log = logManager.getOrCreateLog(topicPartition, topicId = None) + val log = logManager.getOrCreateLog(topicPartition, topicId = topicId) seedLogData(log, numRecords = 10, leaderEpoch = 4) val controllerEpoch = 0 @@ -2240,7 +2240,7 @@ class PartitionTest extends AbstractPartitionTest { @Test def testAlterIsrLeaderAndIsrRace(): Unit = { - val log = logManager.getOrCreateLog(topicPartition, topicId = None) + val log = logManager.getOrCreateLog(topicPartition, topicId = topicId) seedLogData(log, numRecords = 10, leaderEpoch = 4) val controllerEpoch = 0 @@ -2251,7 +2251,7 @@ class PartitionTest extends AbstractPartitionTest { val initializeTimeMs = time.milliseconds() assertTrue(makeLeader( - topicId = None, + topicId = topicId, controllerEpoch = controllerEpoch, leaderEpoch = leaderEpoch, isr = isr, @@ -2274,7 +2274,7 @@ class PartitionTest extends AbstractPartitionTest { // Become leader again, reset the ISR state assertFalse(makeLeader( - topicId = None, + topicId = topicId, controllerEpoch = controllerEpoch, leaderEpoch = leaderEpoch, isr = isr, @@ -2298,7 +2298,7 @@ class PartitionTest extends AbstractPartitionTest { @Test def testShouldNotShrinkIsrIfPreviousFetchIsCaughtUp(): Unit = { - val log = logManager.getOrCreateLog(topicPartition, topicId = None) + val log = logManager.getOrCreateLog(topicPartition, topicId = topicId) seedLogData(log, numRecords = 10, leaderEpoch = 4) val controllerEpoch = 0 @@ -2310,7 +2310,7 @@ class PartitionTest extends AbstractPartitionTest { addBrokerEpochToMockMetadataCache(metadataCache, replicas.toList) assertTrue(makeLeader( - topicId = None, + topicId = topicId, controllerEpoch = controllerEpoch, leaderEpoch = leaderEpoch, isr = isr, @@ -2358,7 +2358,7 @@ class PartitionTest extends AbstractPartitionTest { @Test def testShouldNotShrinkIsrIfFollowerCaughtUpToLogEnd(): Unit = { - val log = logManager.getOrCreateLog(topicPartition, topicId = None) + val log = logManager.getOrCreateLog(topicPartition, topicId = topicId) seedLogData(log, numRecords = 10, leaderEpoch = 4) val controllerEpoch = 0 @@ -2370,7 +2370,7 @@ class PartitionTest extends AbstractPartitionTest { addBrokerEpochToMockMetadataCache(metadataCache, replicas.toList) assertTrue(makeLeader( - topicId = None, + topicId = topicId, controllerEpoch = controllerEpoch, leaderEpoch = leaderEpoch, isr = isr, @@ -2406,7 +2406,7 @@ class PartitionTest extends AbstractPartitionTest { @Test def testIsrNotShrunkIfUpdateFails(): Unit = { - val log = logManager.getOrCreateLog(topicPartition, topicId = None) + val log = logManager.getOrCreateLog(topicPartition, topicId = topicId) seedLogData(log, numRecords = 10, leaderEpoch = 4) val controllerEpoch = 0 @@ -2417,7 +2417,7 @@ class PartitionTest extends AbstractPartitionTest { val initializeTimeMs = time.milliseconds() assertTrue(makeLeader( - topicId = None, + topicId = topicId, controllerEpoch = controllerEpoch, leaderEpoch = leaderEpoch, isr = isr, @@ -2493,7 +2493,7 @@ class PartitionTest extends AbstractPartitionTest { } def handleAlterIsrFailure(error: Errors, callback: (Int, Int, Partition) => Unit): Unit = { - val log = logManager.getOrCreateLog(topicPartition, topicId = None) + val log = logManager.getOrCreateLog(topicPartition, topicId = topicId) seedLogData(log, numRecords = 10, leaderEpoch = 4) val controllerEpoch = 0 @@ -2504,7 +2504,7 @@ class PartitionTest extends AbstractPartitionTest { addBrokerEpochToMockMetadataCache(metadataCache, replicas.toList) assertTrue(makeLeader( - topicId = None, + topicId = topicId, controllerEpoch = controllerEpoch, leaderEpoch = leaderEpoch, isr = isr, @@ -2547,8 +2547,7 @@ class PartitionTest extends AbstractPartitionTest { partitionEpoch: Int = 0 ): ClientResponse = { val alterPartitionResponseData = new AlterPartitionResponseData() - val topicResponse = new AlterPartitionResponseData.TopicData() - .setTopicName(topicPartition.topic) + val topicResponse = new AlterPartitionResponseData.TopicData().setTopicId(topicId.get) topicResponse.partitions.add(new AlterPartitionResponseData.PartitionData() .setPartitionIndex(topicPartition.partition) @@ -2572,8 +2571,7 @@ class PartitionTest extends AbstractPartitionTest { scheduler = mock(classOf[KafkaScheduler]), time = time, brokerId = brokerId, - brokerEpochSupplier = () => 0, - metadataVersionSupplier = () => MetadataVersion.IBP_3_0_IV1 + brokerEpochSupplier = () => 0 ) partition = new Partition(topicPartition, @@ -2587,7 +2585,7 @@ class PartitionTest extends AbstractPartitionTest { logManager, alterPartitionManager) - val log = logManager.getOrCreateLog(topicPartition, topicId = None) + val log = logManager.getOrCreateLog(topicPartition, topicId = topicId) seedLogData(log, numRecords = 10, leaderEpoch = 4) val controllerEpoch = 0 @@ -2621,7 +2619,7 @@ class PartitionTest extends AbstractPartitionTest { } assertTrue(makeLeader( - topicId = None, + topicId = topicId, controllerEpoch, leaderEpoch, isr, @@ -2644,7 +2642,7 @@ class PartitionTest extends AbstractPartitionTest { @Test def testSingleInFlightAlterIsr(): Unit = { - val log = logManager.getOrCreateLog(topicPartition, topicId = None) + val log = logManager.getOrCreateLog(topicPartition, topicId = topicId) seedLogData(log, numRecords = 10, leaderEpoch = 4) val controllerEpoch = 0 @@ -2659,7 +2657,7 @@ class PartitionTest extends AbstractPartitionTest { doNothing().when(delayedOperations).checkAndCompleteAll() assertTrue(makeLeader( - topicId = None, + topicId = topicId, controllerEpoch = controllerEpoch, leaderEpoch = leaderEpoch, isr = isr, @@ -2685,7 +2683,7 @@ class PartitionTest extends AbstractPartitionTest { @Test def testUseCheckpointToInitializeHighWatermark(): Unit = { - val log = logManager.getOrCreateLog(topicPartition, topicId = None) + val log = logManager.getOrCreateLog(topicPartition, topicId = topicId) seedLogData(log, numRecords = 6, leaderEpoch = 5) when(offsetCheckpoints.fetch(logDir1.getAbsolutePath, topicPartition)) @@ -3266,7 +3264,7 @@ class PartitionTest extends AbstractPartitionTest { @Test def testAddAndRemoveListeners(): Unit = { - partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, topicId = None) + partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, topicId = topicId) val replicas = Seq(brokerId, brokerId + 1) val isr = replicas @@ -3339,7 +3337,7 @@ class PartitionTest extends AbstractPartitionTest { @Test def testAddListenerFailsWhenPartitionIsDeleted(): Unit = { - partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, topicId = None) + partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, topicId = topicId) partition.makeLeader( new LeaderAndIsrRequest.PartitionState() @@ -3360,7 +3358,7 @@ class PartitionTest extends AbstractPartitionTest { @Test def testPartitionListenerWhenLogOffsetsChanged(): Unit = { - partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, topicId = None) + partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, topicId = topicId) val replicas = Seq(brokerId, brokerId + 1) val isr = Seq(brokerId, brokerId + 1) @@ -3405,7 +3403,7 @@ class PartitionTest extends AbstractPartitionTest { @Test def testPartitionListenerWhenPartitionFailed(): Unit = { - partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, topicId = None) + partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, topicId = topicId) partition.makeLeader( new LeaderAndIsrRequest.PartitionState() @@ -3429,7 +3427,7 @@ class PartitionTest extends AbstractPartitionTest { @Test def testPartitionListenerWhenPartitionIsDeleted(): Unit = { - partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, topicId = None) + partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, topicId = topicId) partition.makeLeader( new LeaderAndIsrRequest.PartitionState() @@ -3454,7 +3452,7 @@ class PartitionTest extends AbstractPartitionTest { @Test def testPartitionListenerWhenCurrentIsReplacedWithFutureLog(): Unit = { logManager.maybeUpdatePreferredLogDir(topicPartition, logDir1.getAbsolutePath) - partition.createLogIfNotExists(isNew = true, isFutureReplica = false, offsetCheckpoints, topicId = None) + partition.createLogIfNotExists(isNew = true, isFutureReplica = false, offsetCheckpoints, topicId = topicId) assertTrue(partition.log.isDefined) val replicas = Seq(brokerId, brokerId + 1) @@ -3627,6 +3625,7 @@ class PartitionTest extends AbstractPartitionTest { private class SlowLog( log: UnifiedLog, + topicId: Option[Uuid], logStartOffset: Long, localLog: LocalLog, leaderEpochCache: LeaderEpochFileCache, @@ -3639,7 +3638,7 @@ class PartitionTest extends AbstractPartitionTest { log.producerIdExpirationCheckIntervalMs, leaderEpochCache, producerStateManager, - _topicId = None) { + _topicId = topicId) { override def appendAsFollower(records: MemoryRecords): LogAppendInfo = { appendSemaphore.acquire() diff --git a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala index e6f62a52f95..400e13dc6be 100644 --- a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala +++ b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala @@ -24,6 +24,7 @@ import kafka.network import org.apache.kafka.common.memory.MemoryPool import org.apache.kafka.common.message._ import org.apache.kafka.common.network.{ClientInformation, ListenerName, NetworkSend} +import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.requests._ import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.network.RequestConvertToJson @@ -38,7 +39,7 @@ class RequestConvertToJsonTest { @Test def testRequestHeaderNode(): Unit = { - val alterIsrRequest = new AlterPartitionRequest(new AlterPartitionRequestData(), 0) + val alterIsrRequest = new AlterPartitionRequest(new AlterPartitionRequestData(), ApiKeys.ALTER_PARTITION.latestVersion) val req = request(alterIsrRequest) val header = req.header @@ -52,7 +53,7 @@ class RequestConvertToJsonTest { @Test def testRequestDesc(): Unit = { - val alterIsrRequest = new AlterPartitionRequest(new AlterPartitionRequestData(), 0) + val alterIsrRequest = new AlterPartitionRequest(new AlterPartitionRequestData(), ApiKeys.ALTER_PARTITION.latestVersion) val req = request(alterIsrRequest) val expectedNode = new ObjectNode(JsonNodeFactory.instance) @@ -67,7 +68,7 @@ class RequestConvertToJsonTest { @Test def testRequestDescMetrics(): Unit = { - val alterIsrRequest = new AlterPartitionRequest(new AlterPartitionRequestData(), 0) + val alterIsrRequest = new AlterPartitionRequest(new AlterPartitionRequestData(), ApiKeys.ALTER_PARTITION.latestVersion) val req = request(alterIsrRequest) val send = new NetworkSend(req.context.connectionId, alterIsrRequest.toSend(req.header)) val headerLog = RequestConvertToJson.requestHeaderNode(req.header) diff --git a/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala index 900fb0f66fb..dcbfbcb3497 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala @@ -69,7 +69,7 @@ abstract class AbstractApiVersionsRequestTest(cluster: ClusterInstance) { assertEquals(MetadataVersion.latestTesting().featureLevel(), apiVersionsResponse.data().finalizedFeatures().find(MetadataVersion.FEATURE_NAME).maxVersionLevel()) assertEquals(5, apiVersionsResponse.data().supportedFeatures().size()) - assertEquals(MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel(), apiVersionsResponse.data().supportedFeatures().find(MetadataVersion.FEATURE_NAME).minVersion()) + assertEquals(MetadataVersion.MINIMUM_VERSION.featureLevel(), apiVersionsResponse.data().supportedFeatures().find(MetadataVersion.FEATURE_NAME).minVersion()) if (apiVersion < 4) { assertEquals(1, apiVersionsResponse.data().supportedFeatures().find("kraft.version").minVersion()) } else { diff --git a/core/src/test/scala/unit/kafka/server/AlterPartitionManagerTest.scala b/core/src/test/scala/unit/kafka/server/AlterPartitionManagerTest.scala index 9ce80d44752..eb2c2fdf547 100644 --- a/core/src/test/scala/unit/kafka/server/AlterPartitionManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AlterPartitionManagerTest.scala @@ -18,9 +18,7 @@ package kafka.server import java.util.Collections -import java.util.stream.{Stream => JStream} import org.apache.kafka.clients.ClientResponse -import org.apache.kafka.common.TopicIdPartition import org.apache.kafka.common.Uuid import org.apache.kafka.common.errors.{AuthenticationException, OperationNotAttemptedException, UnknownServerException, UnsupportedVersionException} import org.apache.kafka.common.message.AlterPartitionRequestData.BrokerState @@ -31,16 +29,14 @@ import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.RequestHeader import org.apache.kafka.common.requests.{AbstractRequest, AlterPartitionRequest, AlterPartitionResponse} import org.apache.kafka.metadata.{LeaderAndIsr, LeaderRecoveryState} -import org.apache.kafka.server.common.{ControllerRequestCompletionHandler, MetadataVersion, NodeToControllerChannelManager} -import org.apache.kafka.server.common.MetadataVersion.{IBP_3_0_IV1, IBP_3_2_IV0, IBP_3_5_IV1} +import org.apache.kafka.server.common.{ControllerRequestCompletionHandler, NodeToControllerChannelManager, TopicIdPartition} import org.apache.kafka.server.util.{MockScheduler, MockTime} import org.apache.kafka.test.TestUtils.assertFutureThrows import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.BeforeEach import org.junit.jupiter.api.Test import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.Arguments -import org.junit.jupiter.params.provider.MethodSource +import org.junit.jupiter.params.provider.EnumSource import org.mockito.ArgumentMatcher import org.mockito.ArgumentMatchers.any import org.mockito.Mockito.{mock, reset, times, verify} @@ -51,8 +47,6 @@ import scala.collection.mutable.ListBuffer import scala.jdk.CollectionConverters._ class AlterPartitionManagerTest { - - val topic = "test-topic" val topicId = Uuid.randomUuid() val time = new MockTime val metrics = new Metrics @@ -60,31 +54,29 @@ class AlterPartitionManagerTest { var brokerToController: NodeToControllerChannelManager = _ - val tp0 = new TopicIdPartition(topicId, 0, topic) - val tp1 = new TopicIdPartition(topicId, 1, topic) - val tp2 = new TopicIdPartition(topicId, 2, topic) + val tp0 = new TopicIdPartition(topicId, 0) + val tp1 = new TopicIdPartition(topicId, 1) + val tp2 = new TopicIdPartition(topicId, 2) @BeforeEach def setup(): Unit = { brokerToController = mock(classOf[NodeToControllerChannelManager]) } - @ParameterizedTest - @MethodSource(Array("provideMetadataVersions")) - def testBasic(metadataVersion: MetadataVersion): Unit = { + @Test + def testBasic(): Unit = { val scheduler = new MockScheduler(time) - val alterPartitionManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, () => metadataVersion) + val alterPartitionManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2) alterPartitionManager.start() alterPartitionManager.submit(tp0, new LeaderAndIsr(1, 1, List(1, 2, 3).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, 10)) verify(brokerToController).start() verify(brokerToController).sendRequest(any(), any()) } - @ParameterizedTest - @MethodSource(Array("provideMetadataVersions")) - def testBasicWithBrokerEpoch(metadataVersion: MetadataVersion): Unit = { + @Test + def testBasicWithBrokerEpoch(): Unit = { val scheduler = new MockScheduler(time) - val alterPartitionManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 101, () => metadataVersion) + val alterPartitionManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 101) alterPartitionManager.start() val isrWithBrokerEpoch = ListBuffer[BrokerState]() for (ii <- 1 to 3) { @@ -96,7 +88,6 @@ class AlterPartitionManagerTest { .setBrokerId(brokerId) .setBrokerEpoch(101) val topicData = new AlterPartitionRequestData.TopicData() - .setTopicName(topic) .setTopicId(topicId) val newIsrWithBrokerEpoch = new ListBuffer[BrokerState]() @@ -118,33 +109,28 @@ class AlterPartitionManagerTest { } @ParameterizedTest - @MethodSource(Array("provideLeaderRecoveryState")) - def testBasicSentLeaderRecoveryState( - metadataVersion: MetadataVersion, - leaderRecoveryState: LeaderRecoveryState - ): Unit = { + @EnumSource(classOf[LeaderRecoveryState]) + def testBasicSentLeaderRecoveryState(leaderRecoveryState: LeaderRecoveryState): Unit = { val requestCapture = ArgumentCaptor.forClass(classOf[AbstractRequest.Builder[AlterPartitionRequest]]) val scheduler = new MockScheduler(time) - val alterPartitionManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, () => metadataVersion) + val alterPartitionManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2) alterPartitionManager.start() alterPartitionManager.submit(tp0, new LeaderAndIsr(1, 1, List(1).map(Int.box).asJava, leaderRecoveryState, 10)) verify(brokerToController).start() verify(brokerToController).sendRequest(requestCapture.capture(), any()) val request = requestCapture.getValue.build() - val expectedLeaderRecoveryState = if (metadataVersion.isAtLeast(IBP_3_2_IV0)) leaderRecoveryState else LeaderRecoveryState.RECOVERED - assertEquals(expectedLeaderRecoveryState.value, request.data.topics.get(0).partitions.get(0).leaderRecoveryState()) + assertEquals(leaderRecoveryState.value, request.data.topics.get(0).partitions.get(0).leaderRecoveryState()) } - @ParameterizedTest - @MethodSource(Array("provideMetadataVersions")) - def testOverwriteWithinBatch(metadataVersion: MetadataVersion): Unit = { + @Test + def testOverwriteWithinBatch(): Unit = { val capture: ArgumentCaptor[AbstractRequest.Builder[AlterPartitionRequest]] = ArgumentCaptor.forClass(classOf[AbstractRequest.Builder[AlterPartitionRequest]]) val callbackCapture: ArgumentCaptor[ControllerRequestCompletionHandler] = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler]) val scheduler = new MockScheduler(time) - val alterPartitionManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, () => metadataVersion) + val alterPartitionManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2) alterPartitionManager.start() // Only send one ISR update for a given topic+partition @@ -181,23 +167,22 @@ class AlterPartitionManagerTest { } } - @ParameterizedTest - @MethodSource(Array("provideMetadataVersions")) - def testSingleBatch(metadataVersion: MetadataVersion): Unit = { + @Test + def testSingleBatch(): Unit = { val capture: ArgumentCaptor[AbstractRequest.Builder[AlterPartitionRequest]] = ArgumentCaptor.forClass(classOf[AbstractRequest.Builder[AlterPartitionRequest]]) val callbackCapture: ArgumentCaptor[ControllerRequestCompletionHandler] = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler]) val scheduler = new MockScheduler(time) - val alterPartitionManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, () => metadataVersion) + val alterPartitionManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2) alterPartitionManager.start() // First request will send batch of one - alterPartitionManager.submit(new TopicIdPartition(topicId, 0, topic), + alterPartitionManager.submit(new TopicIdPartition(topicId, 0), new LeaderAndIsr(1, 1, List(1, 2, 3).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, 10)) // Other submissions will queue up until a response for (i <- 1 to 9) { - alterPartitionManager.submit(new TopicIdPartition(topicId, i, topic), + alterPartitionManager.submit(new TopicIdPartition(topicId, i), new LeaderAndIsr(1, 1, List(1, 2, 3).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, 10)) } @@ -233,7 +218,7 @@ class AlterPartitionManagerTest { val callbackCapture = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler]) val scheduler = new MockScheduler(time) - val alterPartitionManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, () => IBP_3_2_IV0) + val alterPartitionManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2) alterPartitionManager.start() val future = alterPartitionManager.submit(tp0, leaderAndIsr) val finalFuture = new CompletableFuture[LeaderAndIsr]() @@ -266,7 +251,7 @@ class AlterPartitionManagerTest { assertEquals(leaderAndIsr, finalFuture.get(200, TimeUnit.MILLISECONDS)) // No more items in unsentIsrUpdates - assertFalse(alterPartitionManager.unsentIsrUpdates.containsKey(tp0.topicPartition)) + assertFalse(alterPartitionManager.unsentIsrUpdates.containsKey(tp0)) } @Test @@ -307,7 +292,7 @@ class AlterPartitionManagerTest { val callbackCapture: ArgumentCaptor[ControllerRequestCompletionHandler] = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler]) val scheduler = new MockScheduler(time) - val alterPartitionManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, () => IBP_3_2_IV0) + val alterPartitionManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2) alterPartitionManager.start() alterPartitionManager.submit(tp0, leaderAndIsr) @@ -316,7 +301,7 @@ class AlterPartitionManagerTest { callbackCapture.getValue.onComplete(response) // Any top-level error, we want to retry, so we don't clear items from the pending map - assertTrue(alterPartitionManager.unsentIsrUpdates.containsKey(tp0.topicPartition)) + assertTrue(alterPartitionManager.unsentIsrUpdates.containsKey(tp0)) reset(brokerToController) @@ -331,7 +316,7 @@ class AlterPartitionManagerTest { verify(brokerToController).sendRequest(any(), callbackCapture.capture()) callbackCapture.getValue.onComplete(retryResponse) - assertFalse(alterPartitionManager.unsentIsrUpdates.containsKey(tp0.topicPartition)) + assertFalse(alterPartitionManager.unsentIsrUpdates.containsKey(tp0)) } @Test @@ -366,7 +351,7 @@ class AlterPartitionManagerTest { reset(brokerToController) val scheduler = new MockScheduler(time) - val alterPartitionManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, () => IBP_3_2_IV0) + val alterPartitionManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2) alterPartitionManager.start() val future = alterPartitionManager.submit(tp, new LeaderAndIsr(1, 1, List(1, 2, 3).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, 10)) @@ -383,13 +368,12 @@ class AlterPartitionManagerTest { alterPartitionManager } - @ParameterizedTest - @MethodSource(Array("provideMetadataVersions")) - def testOneInFlight(metadataVersion: MetadataVersion): Unit = { + @Test + def testOneInFlight(): Unit = { val callbackCapture: ArgumentCaptor[ControllerRequestCompletionHandler] = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler]) val scheduler = new MockScheduler(time) - val alterPartitionManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, () => metadataVersion) + val alterPartitionManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2) alterPartitionManager.start() // First submit will send the request @@ -410,9 +394,8 @@ class AlterPartitionManagerTest { callbackCapture.getValue.onComplete(resp) } - @ParameterizedTest - @MethodSource(Array("provideMetadataVersions")) - def testPartitionMissingInResponse(metadataVersion: MetadataVersion): Unit = { + @Test + def testPartitionMissingInResponse(): Unit = { val expectedVersion = ApiKeys.ALTER_PARTITION.latestVersion val leaderAndIsr = new LeaderAndIsr(1, 1, List(1, 2, 3).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, 10) val brokerEpoch = 2 @@ -423,8 +406,7 @@ class AlterPartitionManagerTest { scheduler, time, brokerId, - () => brokerEpoch, - () => metadataVersion + () => brokerEpoch ) alterPartitionManager.start() @@ -498,13 +480,13 @@ class AlterPartitionManagerTest { val alterPartitionRequest = request.asInstanceOf[AlterPartitionRequest.Builder].build() assertEquals(expectedVersion, alterPartitionRequest.version) - val requestTopicPartitions = alterPartitionRequest.data.topics.asScala.flatMap { topicData => + val requestTopicIdPartitions = alterPartitionRequest.data.topics.asScala.flatMap { topicData => topicData.partitions.asScala.map { partitionData => - new TopicIdPartition(topicData.topicId, partitionData.partitionIndex, topicData.topicName) + (topicData.topicId, partitionData.partitionIndex) } }.toSet - expectedTopicPartitions == requestTopicPartitions + expectedTopicPartitions.map(tp => (tp.topicId, tp.partitionId)) == requestTopicIdPartitions } } @@ -538,11 +520,10 @@ class AlterPartitionManagerTest { new AlterPartitionResponse(new AlterPartitionResponseData() .setTopics(Collections.singletonList( new AlterPartitionResponseData.TopicData() - .setTopicName(tp.topic) .setTopicId(tp.topicId) .setPartitions(Collections.singletonList( new AlterPartitionResponseData.PartitionData() - .setPartitionIndex(tp.partition) + .setPartitionIndex(tp.partitionId) .setPartitionEpoch(partitionEpoch) .setLeaderEpoch(leaderEpoch) .setLeaderId(leaderId) @@ -550,26 +531,3 @@ class AlterPartitionManagerTest { .setErrorCode(error.code)))))) } } - -object AlterPartitionManagerTest { - def provideMetadataVersions(): JStream[MetadataVersion] = { - JStream.of( - // Supports KIP-903: include broker epoch in AlterPartition request - IBP_3_5_IV1, - // Supports KIP-704: unclean leader recovery - IBP_3_2_IV0, - // Supports KIP-497: alter partition - IBP_3_0_IV1 - ) - } - - def provideLeaderRecoveryState(): JStream[Arguments] = { - // Multiply metadataVersions by leaderRecoveryState - provideMetadataVersions().flatMap { metadataVersion => - JStream.of( - Arguments.of(metadataVersion, LeaderRecoveryState.RECOVERED), - Arguments.of(metadataVersion, LeaderRecoveryState.RECOVERING) - ) - } - } -} diff --git a/core/src/test/scala/unit/kafka/server/ApiVersionManagerTest.scala b/core/src/test/scala/unit/kafka/server/ApiVersionManagerTest.scala index 1d506b3b7c2..19a95ca9450 100644 --- a/core/src/test/scala/unit/kafka/server/ApiVersionManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ApiVersionManagerTest.scala @@ -18,9 +18,11 @@ package kafka.server import org.apache.kafka.clients.NodeApiVersions import org.apache.kafka.common.message.ApiMessageType.ListenerType +import org.apache.kafka.common.metadata.FeatureLevelRecord import org.apache.kafka.common.protocol.ApiKeys +import org.apache.kafka.image.{MetadataDelta, MetadataImage, MetadataProvenance} import org.apache.kafka.server.BrokerFeatures -import org.apache.kafka.server.common.KRaftVersion +import org.apache.kafka.server.common.{KRaftVersion, MetadataVersion} import org.junit.jupiter.api.Test import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest @@ -31,7 +33,16 @@ import scala.jdk.CollectionConverters._ class ApiVersionManagerTest { private val brokerFeatures = BrokerFeatures.createDefault(true) - private val metadataCache = MetadataCache.kRaftMetadataCache(1, () => KRaftVersion.LATEST_PRODUCTION) + private val metadataCache = { + val cache = MetadataCache.kRaftMetadataCache(1, () => KRaftVersion.LATEST_PRODUCTION) + val delta = new MetadataDelta(MetadataImage.EMPTY); + delta.replay(new FeatureLevelRecord() + .setName(MetadataVersion.FEATURE_NAME) + .setFeatureLevel(MetadataVersion.latestProduction().featureLevel()) + ) + cache.setImage(delta.apply(MetadataProvenance.EMPTY)) + cache + } @ParameterizedTest @EnumSource(classOf[ListenerType]) diff --git a/core/src/test/scala/unit/kafka/server/ApiVersionsResponseIntegrationTest.scala b/core/src/test/scala/unit/kafka/server/ApiVersionsResponseIntegrationTest.scala index 0d9d070c982..0e482165c44 100644 --- a/core/src/test/scala/unit/kafka/server/ApiVersionsResponseIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/server/ApiVersionsResponseIntegrationTest.scala @@ -17,10 +17,10 @@ import org.apache.kafka.common.message.ApiVersionsResponseData.SupportedFeatureK import org.apache.kafka.common.requests.ApiVersionsRequest import org.apache.kafka.common.requests.ApiVersionsResponse import org.apache.kafka.coordinator.group.GroupCoordinatorConfig +import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.config.ServerConfigs import org.junit.jupiter.api.Assertions.{assertEquals, assertNotNull, assertNull} -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.api.Test import java.util.Properties @@ -38,28 +38,18 @@ class ApiVersionsResponseIntegrationTest extends BaseRequestTest { connectAndReceive[ApiVersionsResponse](request) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testSendV3ApiVersionsRequest(quorum: String): Unit = { + @Test + def testSendV3ApiVersionsRequest(): Unit = { val response = sendApiVersionsRequest(3) - if (quorum.equals("kraft")) { - assertFeatureHasMinVersion("metadata.version", response.data().supportedFeatures(), 1) - assertFeatureMissing("kraft.version", response.data().supportedFeatures()) - } else { - assertEquals(0, response.data().supportedFeatures().size()) - } + assertFeatureHasMinVersion("metadata.version", response.data().supportedFeatures(), MetadataVersion.MINIMUM_VERSION.featureLevel()) + assertFeatureMissing("kraft.version", response.data().supportedFeatures()) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testSendV4ApiVersionsRequest(quorum: String): Unit = { + @Test + def testSendV4ApiVersionsRequest(): Unit = { val response = sendApiVersionsRequest(4) - if (quorum.equals("kraft")) { - assertFeatureHasMinVersion("metadata.version", response.data().supportedFeatures(), 1) - assertFeatureHasMinVersion("kraft.version", response.data().supportedFeatures(), 0) - } else { - assertEquals(0, response.data().supportedFeatures().size()) - } + assertFeatureHasMinVersion("metadata.version", response.data().supportedFeatures(), MetadataVersion.MINIMUM_VERSION.featureLevel()) + assertFeatureHasMinVersion("kraft.version", response.data().supportedFeatures(), 0) } def assertFeatureHasMinVersion( diff --git a/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala b/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala index 1bec9f7f430..c7a4bd45f78 100644 --- a/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala @@ -29,7 +29,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.common.utils.Time import org.apache.kafka.common.{Node, Uuid} -import org.apache.kafka.server.common.{ControllerRequestCompletionHandler, Feature, MetadataVersion, NodeToControllerChannelManager} +import org.apache.kafka.server.common.{ControllerRequestCompletionHandler, Feature, MetadataVersion, MetadataVersionTestUtils, NodeToControllerChannelManager} import org.junit.jupiter.api.Assertions.assertEquals import java.util @@ -88,16 +88,15 @@ class BrokerRegistrationRequestTest { clusterId: String, brokerId: Int, zkEpoch: Option[Long], - ibpToSend: Option[(MetadataVersion, MetadataVersion)] + featureLevelToSend: Option[(Short, Short)] ): Errors = { val features = new BrokerRegistrationRequestData.FeatureCollection() - ibpToSend foreach { - case (min, max) => - features.add(new BrokerRegistrationRequestData.Feature() - .setName(MetadataVersion.FEATURE_NAME) - .setMinSupportedVersion(min.featureLevel()) - .setMaxSupportedVersion(max.featureLevel()) - ) + featureLevelToSend.foreach { case (min, max) => + features.add(new BrokerRegistrationRequestData.Feature() + .setName(MetadataVersion.FEATURE_NAME) + .setMinSupportedVersion(min) + .setMaxSupportedVersion(max) + ) } Feature.PRODUCTION_FEATURES.stream().filter(_.featureName != MetadataVersion.FEATURE_NAME).forEach { feature => @@ -150,7 +149,7 @@ class BrokerRegistrationRequestTest { // Invalid registration (isMigratingZkBroker, but MV does not support migrations) assertEquals( Errors.BROKER_ID_NOT_REGISTERED, - registerBroker(channelManager, clusterId, 100, Some(1), Some((MetadataVersion.IBP_3_3_IV0, MetadataVersion.IBP_3_3_IV3)))) + registerBroker(channelManager, clusterId, 100, Some(1), Some((MetadataVersionTestUtils.IBP_3_3_IV0_FEATURE_LEVEL, MetadataVersion.IBP_3_3_IV3.featureLevel)))) // No features (MV) sent with registration, controller can't verify assertEquals( @@ -160,12 +159,12 @@ class BrokerRegistrationRequestTest { // Given MV is too high for controller to support assertEquals( Errors.BROKER_ID_NOT_REGISTERED, - registerBroker(channelManager, clusterId, 100, Some(1), Some((MetadataVersion.IBP_3_4_IV0, MetadataVersion.IBP_3_4_IV0)))) + registerBroker(channelManager, clusterId, 100, Some(1), Some((MetadataVersion.IBP_3_4_IV0.featureLevel, MetadataVersion.IBP_3_4_IV0.featureLevel)))) // Controller supports this MV and isMigratingZkBroker is false, so this one works assertEquals( Errors.NONE, - registerBroker(channelManager, clusterId, 100, None, Some((MetadataVersion.IBP_3_3_IV3, MetadataVersion.IBP_3_4_IV0)))) + registerBroker(channelManager, clusterId, 100, None, Some((MetadataVersion.IBP_3_3_IV3.featureLevel, MetadataVersion.IBP_3_4_IV0.featureLevel)))) } 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 8a51652a7bb..054189d4050 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala @@ -411,7 +411,7 @@ class ControllerApisTest { assertThrows(classOf[ClusterAuthorizationException], () => { controllerApis = createControllerApis(Some(createDenyAllAuthorizer()), new MockController.Builder().build()) controllerApis.handleAlterPartitionRequest(buildRequest(new AlterPartitionRequest.Builder( - new AlterPartitionRequestData()).build(0))) + new AlterPartitionRequestData()).build(ApiKeys.ALTER_PARTITION.latestVersion))) }) } diff --git a/core/src/test/scala/unit/kafka/server/ControllerRegistrationManagerTest.scala b/core/src/test/scala/unit/kafka/server/ControllerRegistrationManagerTest.scala index 52c90cc93ab..61cc1363027 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerRegistrationManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerRegistrationManagerTest.scala @@ -37,7 +37,7 @@ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource import java.util -import java.util.{OptionalInt, Properties} +import java.util.{Optional, OptionalInt, Properties} import java.util.concurrent.{CompletableFuture, TimeUnit} import scala.jdk.CollectionConverters._ @@ -62,7 +62,7 @@ class ControllerRegistrationManagerTest { ): java.util.Map[String, VersionRange] = { val results = new util.HashMap[String, VersionRange]() results.put(MetadataVersion.FEATURE_NAME, VersionRange.of( - MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel(), + MetadataVersion.MINIMUM_VERSION.featureLevel(), highestSupportedMetadataVersion.featureLevel())) results } @@ -105,7 +105,7 @@ class ControllerRegistrationManagerTest { val delta = new MetadataDelta.Builder(). setImage(prevImage). build() - if (!prevImage.features().metadataVersion().equals(metadataVersion)) { + if (!prevImage.features().metadataVersion.equals(Optional.of(metadataVersion))) { delta.replay(new FeatureLevelRecord(). setName(MetadataVersion.FEATURE_NAME). setFeatureLevel(metadataVersion.featureLevel())) @@ -119,7 +119,7 @@ class ControllerRegistrationManagerTest { } val provenance = new MetadataProvenance(100, 200, 300, true) val newImage = delta.apply(provenance) - val manifest = if (!prevImage.features().metadataVersion().equals(metadataVersion)) { + val manifest = if (!prevImage.features().metadataVersion().equals(Optional.of(metadataVersion))) { new SnapshotManifest(provenance, 1000) } else { new LogDeltaManifest.Builder(). diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 0e82227dd9e..0670bf0b36d 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -23,7 +23,7 @@ import kafka.network.RequestChannel import kafka.server.QuotaFactory.QuotaManagers import kafka.server.metadata.KRaftMetadataCache import kafka.server.share.SharePartitionManager -import kafka.utils.{CoreUtils, LoggingController, Logging, TestUtils} +import kafka.utils.{CoreUtils, Logging, LoggingController, TestUtils} import org.apache.kafka.clients.admin.AlterConfigOp.OpType import org.apache.kafka.clients.admin.{AlterConfigOp, ConfigEntry} import org.apache.kafka.common._ @@ -43,7 +43,7 @@ import org.apache.kafka.common.message.ConsumerGroupDescribeResponseData.Describ import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic import org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult import org.apache.kafka.common.message.DescribeShareGroupOffsetsRequestData.{DescribeShareGroupOffsetsRequestGroup, DescribeShareGroupOffsetsRequestTopic} -import org.apache.kafka.common.message.DescribeShareGroupOffsetsResponseData.{DescribeShareGroupOffsetsResponsePartition, DescribeShareGroupOffsetsResponseGroup, DescribeShareGroupOffsetsResponseTopic} +import org.apache.kafka.common.message.DescribeShareGroupOffsetsResponseData.{DescribeShareGroupOffsetsResponseGroup, DescribeShareGroupOffsetsResponsePartition, DescribeShareGroupOffsetsResponseTopic} import org.apache.kafka.common.message.IncrementalAlterConfigsRequestData.{AlterConfigsResource => IAlterConfigsResource, AlterConfigsResourceCollection => IAlterConfigsResourceCollection, AlterableConfig => IAlterableConfig, AlterableConfigCollection => IAlterableConfigCollection} import org.apache.kafka.common.message.IncrementalAlterConfigsResponseData.{AlterConfigsResourceResponse => IAlterConfigsResourceResponse} import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity @@ -55,7 +55,7 @@ import org.apache.kafka.common.message.OffsetDeleteRequestData.{OffsetDeleteRequ import org.apache.kafka.common.message.OffsetDeleteResponseData.{OffsetDeleteResponsePartition, OffsetDeleteResponsePartitionCollection, OffsetDeleteResponseTopic, OffsetDeleteResponseTopicCollection} import org.apache.kafka.common.message.ShareFetchRequestData.{AcknowledgementBatch, ForgottenTopic} import org.apache.kafka.common.message.ShareFetchResponseData.{AcquiredRecords, PartitionData, ShareFetchableTopicResponse} -import org.apache.kafka.common.metadata.{PartitionRecord, RegisterBrokerRecord, TopicRecord} +import org.apache.kafka.common.metadata.{FeatureLevelRecord, PartitionRecord, RegisterBrokerRecord, TopicRecord} import org.apache.kafka.common.metadata.RegisterBrokerRecord.{BrokerEndpoint, BrokerEndpointCollection} import org.apache.kafka.common.protocol.ApiMessage import org.apache.kafka.common.message._ @@ -77,6 +77,7 @@ import org.apache.kafka.coordinator.group.modern.share.ShareGroupConfig import org.apache.kafka.coordinator.group.{GroupConfig, GroupCoordinator, GroupCoordinatorConfig} import org.apache.kafka.coordinator.share.{ShareCoordinator, ShareCoordinatorTestConfig} import org.apache.kafka.coordinator.transaction.TransactionLogConfig +import org.apache.kafka.image.{MetadataDelta, MetadataImage, MetadataProvenance} import org.apache.kafka.metadata.{ConfigRepository, MockConfigRepository} import org.apache.kafka.network.metrics.{RequestChannelMetrics, RequestMetrics} import org.apache.kafka.raft.QuorumConfig @@ -9781,7 +9782,16 @@ class KafkaApisTest extends Logging { val consumerGroupHeartbeatRequest = new ConsumerGroupHeartbeatRequestData().setGroupId("group") val requestChannelRequest = buildRequest(new ConsumerGroupHeartbeatRequest.Builder(consumerGroupHeartbeatRequest).build()) - metadataCache = MetadataCache.kRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_1) + metadataCache = { + val cache = MetadataCache.kRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_1) + val delta = new MetadataDelta(MetadataImage.EMPTY); + delta.replay(new FeatureLevelRecord() + .setName(MetadataVersion.FEATURE_NAME) + .setFeatureLevel(MetadataVersion.MINIMUM_VERSION.featureLevel()) + ) + cache.setImage(delta.apply(MetadataProvenance.EMPTY)) + cache + } kafkaApis = createKafkaApis() kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) @@ -9920,7 +9930,16 @@ class KafkaApisTest extends Logging { val expectedDescribedGroup = new DescribedGroup().setGroupId(groupId).setErrorCode(errorCode) val expectedResponse = new ConsumerGroupDescribeResponseData() expectedResponse.groups.add(expectedDescribedGroup) - metadataCache = MetadataCache.kRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_1) + metadataCache = { + val cache = MetadataCache.kRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_1) + val delta = new MetadataDelta(MetadataImage.EMPTY); + delta.replay(new FeatureLevelRecord() + .setName(MetadataVersion.FEATURE_NAME) + .setFeatureLevel(MetadataVersion.MINIMUM_VERSION.featureLevel()) + ) + cache.setImage(delta.apply(MetadataProvenance.EMPTY)) + cache + } kafkaApis = createKafkaApis() kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) val response = verifyNoThrottling[ConsumerGroupDescribeResponse](requestChannelRequest) diff --git a/core/src/test/scala/unit/kafka/server/KafkaRaftServerTest.scala b/core/src/test/scala/unit/kafka/server/KafkaRaftServerTest.scala index 1006b7c1fef..48fb8081129 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaRaftServerTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaRaftServerTest.scala @@ -109,7 +109,7 @@ class KafkaRaftServerTest { } private def writeBootstrapMetadata(logDir: File, metadataVersion: MetadataVersion): Unit = { - val bootstrapDirectory = new BootstrapDirectory(logDir.toString, Optional.empty()) + val bootstrapDirectory = new BootstrapDirectory(logDir.toString) bootstrapDirectory.writeBinaryFile(BootstrapMetadata.fromVersion(metadataVersion, "test")) } @@ -262,7 +262,7 @@ class KafkaRaftServerTest { } @Test - def testKRaftUpdateAt3_3_IV1(): Unit = { + def testKRaftUpdateAt3_3_IV3(): Unit = { val clusterId = clusterIdBase64 val nodeId = 0 val metaProperties = new MetaProperties.Builder(). @@ -280,12 +280,12 @@ class KafkaRaftServerTest { configProperties.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL") val (metaPropertiesEnsemble, bootstrapMetadata) = - invokeLoadMetaProperties(metaProperties, configProperties, Some(MetadataVersion.IBP_3_3_IV1)) + invokeLoadMetaProperties(metaProperties, configProperties, Some(MetadataVersion.IBP_3_3_IV3)) assertEquals(metaProperties, metaPropertiesEnsemble.logDirProps().values().iterator().next()) assertTrue(metaPropertiesEnsemble.errorLogDirs().isEmpty) assertTrue(metaPropertiesEnsemble.emptyLogDirs().isEmpty) - assertEquals(bootstrapMetadata.metadataVersion(), MetadataVersion.IBP_3_3_IV1) + assertEquals(bootstrapMetadata.metadataVersion(), MetadataVersion.IBP_3_3_IV3) } @Test diff --git a/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala index e9fe6904a05..6526d6628c3 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala @@ -90,7 +90,7 @@ class ReplicaFetcherThreadTest { val logContext = new LogContext(s"[ReplicaFetcher replicaId=${brokerConfig.brokerId}, leaderId=${leaderEndpointBlockingSend.brokerEndPoint().id}, fetcherId=$fetcherId] ") val fetchSessionHandler = new FetchSessionHandler(logContext, leaderEndpointBlockingSend.brokerEndPoint().id) val leader = new RemoteLeaderEndPoint(logContext.logPrefix, leaderEndpointBlockingSend, fetchSessionHandler, - brokerConfig, replicaMgr, quota, () => MetadataVersion.MINIMUM_KRAFT_VERSION, () => 1) + brokerConfig, replicaMgr, quota, () => MetadataVersion.MINIMUM_VERSION, () => 1) new ReplicaFetcherThread(name, leader, brokerConfig, @@ -280,9 +280,9 @@ class ReplicaFetcherThreadTest { val logContext = new LogContext(s"[ReplicaFetcher replicaId=${config.brokerId}, leaderId=${brokerEndPoint.id}, fetcherId=0] ") val fetchSessionHandler = new FetchSessionHandler(logContext, brokerEndPoint.id) val leader = new RemoteLeaderEndPoint(logContext.logPrefix, mockNetwork, fetchSessionHandler, config, - replicaManager, quota, () => MetadataVersion.MINIMUM_KRAFT_VERSION, () => 1) + replicaManager, quota, () => MetadataVersion.MINIMUM_VERSION, () => 1) val thread = new ReplicaFetcherThread("bob", leader, config, failedPartitions, - replicaManager, quota, logContext.logPrefix, () => MetadataVersion.MINIMUM_KRAFT_VERSION) { + replicaManager, quota, logContext.logPrefix, () => MetadataVersion.MINIMUM_VERSION) { override def processPartitionData(topicPartition: TopicPartition, fetchOffset: Long, partitionData: FetchData): Option[LogAppendInfo] = None } thread.addPartitions(Map(t1p0 -> initialFetchState(Some(topicId1), initialLEO), t1p1 -> initialFetchState(Some(topicId1), initialLEO))) @@ -396,7 +396,7 @@ class ReplicaFetcherThreadTest { config, replicaManager, quota, - () => MetadataVersion.MINIMUM_KRAFT_VERSION, + () => MetadataVersion.MINIMUM_VERSION, () => 1 ) @@ -408,7 +408,7 @@ class ReplicaFetcherThreadTest { replicaManager, quota, logContext.logPrefix, - () => MetadataVersion.MINIMUM_KRAFT_VERSION + () => MetadataVersion.MINIMUM_VERSION ) thread.addPartitions(Map( @@ -488,7 +488,7 @@ class ReplicaFetcherThreadTest { config, replicaManager, quota, - () => MetadataVersion.MINIMUM_KRAFT_VERSION, + () => MetadataVersion.MINIMUM_VERSION, () => 1 ) @@ -500,7 +500,7 @@ class ReplicaFetcherThreadTest { replicaManager, quota, logContext.logPrefix, - () => MetadataVersion.MINIMUM_KRAFT_VERSION + () => MetadataVersion.MINIMUM_VERSION ) thread.addPartitions(Map( @@ -597,7 +597,7 @@ class ReplicaFetcherThreadTest { val logContext = new LogContext(s"[ReplicaFetcher replicaId=${config.brokerId}, leaderId=${brokerEndPoint.id}, fetcherId=0] ") val fetchSessionHandler = new FetchSessionHandler(logContext, brokerEndPoint.id) val leader = new RemoteLeaderEndPoint(logContext.logPrefix, mockBlockingSend, fetchSessionHandler, config, - replicaManager, replicaQuota, () => MetadataVersion.MINIMUM_KRAFT_VERSION, () => 1) + replicaManager, replicaQuota, () => MetadataVersion.MINIMUM_VERSION, () => 1) val thread = new ReplicaFetcherThread("bob", leader, config, @@ -605,7 +605,7 @@ class ReplicaFetcherThreadTest { replicaManager, replicaQuota, logContext.logPrefix, - () => MetadataVersion.MINIMUM_KRAFT_VERSION) + () => MetadataVersion.MINIMUM_VERSION) val leaderEpoch = 1 diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala index 3ba92d7c1ae..98e872d331d 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala @@ -24,8 +24,7 @@ import kafka.server.QuotaFactory.QuotaManagers import kafka.server.metadata.KRaftMetadataCache import kafka.utils.TestUtils.waitUntilTrue import kafka.utils.{CoreUtils, Logging, TestUtils} -import org.apache.kafka.common.metadata.RegisterBrokerRecord -import org.apache.kafka.common.metadata.{PartitionChangeRecord, PartitionRecord, TopicRecord} +import org.apache.kafka.common.metadata.{FeatureLevelRecord, PartitionChangeRecord, PartitionRecord, RegisterBrokerRecord, TopicRecord} import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.record.SimpleRecord @@ -33,13 +32,13 @@ import org.apache.kafka.common.replica.ClientMetadata.DefaultClientMetadata import org.apache.kafka.common.requests.{FetchRequest, ProduceResponse} import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.utils.{Time, Utils} -import org.apache.kafka.common.{DirectoryId, IsolationLevel, TopicIdPartition, TopicPartition, Uuid} +import org.apache.kafka.common.{DirectoryId, IsolationLevel, TopicPartition, Uuid} import org.apache.kafka.image.{MetadataDelta, MetadataImage} import org.apache.kafka.metadata.{LeaderAndIsr, LeaderRecoveryState, MockConfigRepository} import org.apache.kafka.metadata.PartitionRegistration import org.apache.kafka.metadata.storage.Formatter import org.apache.kafka.raft.QuorumConfig -import org.apache.kafka.server.common.KRaftVersion +import org.apache.kafka.server.common.{KRaftVersion, MetadataVersion, TopicIdPartition} import org.apache.kafka.server.config.{KRaftConfigs, ReplicationConfigs, ServerLogConfigs} import org.apache.kafka.server.storage.log.{FetchIsolation, FetchParams, FetchPartitionData} import org.apache.kafka.server.util.{MockTime, ShutdownableThread} @@ -97,7 +96,7 @@ class ReplicaManagerConcurrencyTest extends Logging { val topicModel = new TopicModel(Uuid.randomUuid(), "foo", Map(0 -> initialPartitionRegistration)) val topicPartition = new TopicPartition(topicModel.name, 0) - val topicIdPartition = new TopicIdPartition(topicModel.topicId, topicPartition) + val topicIdPartition = new TopicIdPartition(topicModel.topicId, topicPartition.partition) val controller = new ControllerModel(Seq(localId, remoteId), topicModel, channel, replicaManager, metadataCache) submit(new Clock(time)) @@ -129,6 +128,7 @@ class ReplicaManagerConcurrencyTest extends Logging { clientId = s"replica-$remoteId", replicaId = remoteId, topicIdPartition, + topicPartition.topic, replicaManager ) @@ -211,6 +211,7 @@ class ReplicaManagerConcurrencyTest extends Logging { clientId: String, replicaId: Int, topicIdPartition: TopicIdPartition, + topicName: String, replicaManager: ReplicaManager ) extends ShutdownableThread(clientId, false) { private val random = new Random() @@ -236,11 +237,14 @@ class ReplicaManagerConcurrencyTest extends Logging { ) val future = new CompletableFuture[FetchPartitionData]() - def fetchCallback(results: collection.Seq[(TopicIdPartition, FetchPartitionData)]): Unit = { + def fetchCallback(results: collection.Seq[(org.apache.kafka.common.TopicIdPartition, FetchPartitionData)]): Unit = { try { assertEquals(1, results.size) val (topicIdPartition, result) = results.head - assertEquals(this.topicIdPartition, topicIdPartition) + assertEquals(this.topicIdPartition.topicId, topicIdPartition.topicId) + assertEquals(this.topicIdPartition.partitionId, topicIdPartition.partition) + assertEquals(this.topicIdPartition.topicId, topicIdPartition.topicId) + assertEquals(topicName, topicIdPartition.topic) assertEquals(Errors.NONE, result.error) future.complete(result) } catch { @@ -261,7 +265,7 @@ class ReplicaManagerConcurrencyTest extends Logging { replicaManager.fetchMessages( params = fetchParams, - fetchInfos = Seq(topicIdPartition -> partitionData), + fetchInfos = Seq(new org.apache.kafka.common.TopicIdPartition(topicIdPartition.topicId, topicIdPartition.partitionId, topicName) -> partitionData), quota = QuotaFactory.UNBOUNDED_QUOTA, responseCallback = fetchCallback, ) @@ -373,6 +377,10 @@ class ReplicaManagerConcurrencyTest extends Logging { case InitializeEvent => val delta = new MetadataDelta.Builder().setImage(latestImage).build() brokerIds.foreach { brokerId => + delta.replay(new FeatureLevelRecord() + .setName(MetadataVersion.FEATURE_NAME) + .setFeatureLevel(MetadataVersion.MINIMUM_VERSION.featureLevel()) + ) delta.replay(new RegisterBrokerRecord() .setBrokerId(brokerId) .setFenced(false) @@ -419,7 +427,7 @@ class ReplicaManagerConcurrencyTest extends Logging { leaderAndIsr: LeaderAndIsr, delta: MetadataDelta ): LeaderAndIsr = { - val partitionModel = partitions.getOrElse(topicPartition.partition, + val partitionModel = partitions.getOrElse(topicPartition.partitionId, throw new IllegalStateException(s"Unexpected partition $topicPartition") ) partitionModel.alterIsr(leaderAndIsr, delta) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 616850ebc3e..a631e1c0b03 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -274,7 +274,7 @@ class ReplicaManagerTest { val logManager = TestUtils.createLogManager(config.logDirs.map(new File(_)), new LogConfig(new Properties())) val metadataCache: MetadataCache = mock(classOf[MetadataCache]) mockGetAliveBrokerFunctions(metadataCache, Seq(new Node(0, "host0", 0))) - when(metadataCache.metadataVersion()).thenReturn(MetadataVersion.MINIMUM_KRAFT_VERSION) + when(metadataCache.metadataVersion()).thenReturn(MetadataVersion.MINIMUM_VERSION) val rm = new ReplicaManager( metrics = metrics, config = config, @@ -336,7 +336,7 @@ class ReplicaManagerTest { val spyLogManager = spy(logManager) val metadataCache: MetadataCache = mock(classOf[MetadataCache]) mockGetAliveBrokerFunctions(metadataCache, Seq(new Node(0, "host0", 0))) - when(metadataCache.metadataVersion()).thenReturn(MetadataVersion.MINIMUM_KRAFT_VERSION) + when(metadataCache.metadataVersion()).thenReturn(MetadataVersion.MINIMUM_VERSION) val tp0 = new TopicPartition(topic, 0) val uuid = Uuid.randomUuid() val rm = new ReplicaManager( @@ -409,7 +409,7 @@ class ReplicaManagerTest { val aliveBrokers = Seq(new Node(0, "host0", 0), new Node(1, "host1", 1)) val metadataCache: MetadataCache = mock(classOf[MetadataCache]) mockGetAliveBrokerFunctions(metadataCache, aliveBrokers) - when(metadataCache.metadataVersion()).thenReturn(MetadataVersion.MINIMUM_KRAFT_VERSION) + when(metadataCache.metadataVersion()).thenReturn(MetadataVersion.MINIMUM_VERSION) val rm = new ReplicaManager( metrics = metrics, config = config, @@ -2841,7 +2841,7 @@ class ReplicaManagerTest { any[TopicPartition], any[ListenerName])). thenReturn(Map(leaderBrokerId -> new Node(leaderBrokerId, "host1", 9092, "rack-a"), followerBrokerId -> new Node(followerBrokerId, "host2", 9092, "rack-b")).toMap) - when(metadataCache.metadataVersion()).thenReturn(MetadataVersion.MINIMUM_KRAFT_VERSION) + when(metadataCache.metadataVersion()).thenReturn(MetadataVersion.MINIMUM_VERSION) when(metadataCache.getAliveBrokerEpoch(leaderBrokerId)).thenReturn(Some(brokerEpoch)) val mockProducePurgatory = new DelayedOperationPurgatory[DelayedProduce]( "Produce", timer, 0, false) @@ -2895,9 +2895,9 @@ class ReplicaManagerTest { s"fetcherId=$fetcherId] ") val fetchSessionHandler = new FetchSessionHandler(logContext, sourceBroker.id) val leader = new RemoteLeaderEndPoint(logContext.logPrefix, blockingSend, fetchSessionHandler, rm.config, - rm, quotaManager.follower, () => MetadataVersion.MINIMUM_KRAFT_VERSION, () => 1) + rm, quotaManager.follower, () => MetadataVersion.MINIMUM_VERSION, () => 1) new ReplicaFetcherThread(s"ReplicaFetcherThread-$fetcherId", leader, rm.config, failedPartitions, rm, - quotaManager.follower, logContext.logPrefix, () => MetadataVersion.MINIMUM_KRAFT_VERSION) { + quotaManager.follower, logContext.logPrefix, () => MetadataVersion.MINIMUM_VERSION) { override def doWork(): Unit = { // In case the thread starts before the partition is added by AbstractFetcherManager, // add it here (it's a no-op if already added) @@ -3272,7 +3272,7 @@ class ReplicaManagerTest { when(metadataCache.topicIdInfo()).thenReturn((topicIds.asJava, topicNames.asJava)) when(metadataCache.topicNamesToIds()).thenReturn(topicIds.asJava) when(metadataCache.topicIdsToNames()).thenReturn(topicNames.asJava) - when(metadataCache.metadataVersion()).thenReturn(MetadataVersion.MINIMUM_KRAFT_VERSION) + when(metadataCache.metadataVersion()).thenReturn(MetadataVersion.MINIMUM_VERSION) mockGetAliveBrokerFunctions(metadataCache, aliveBrokers) when(metadataCache.getAliveBrokerEpoch(brokerId+1)).thenReturn(Some(brokerEpoch)) val mockProducePurgatory = new DelayedOperationPurgatory[DelayedProduce]( @@ -3361,7 +3361,7 @@ class ReplicaManagerTest { leader.setReplicaPartitionStateCallback(tp => PartitionState(leaderEpoch = 0)) val fetcher = new ReplicaFetcherThread(threadName, leader, config, failedPartitions, replicaManager, - quotaManager, "", () => MetadataVersion.MINIMUM_KRAFT_VERSION) + quotaManager, "", () => MetadataVersion.MINIMUM_VERSION) val initialFetchState = InitialFetchState( topicId = Some(Uuid.randomUuid()), @@ -3608,8 +3608,8 @@ class ReplicaManagerTest { val aliveBrokers = Seq(new Node(0, "host0", 0), new Node(1, "host1", 1)) mockGetAliveBrokerFunctions(metadataCache0, aliveBrokers) mockGetAliveBrokerFunctions(metadataCache1, aliveBrokers) - when(metadataCache0.metadataVersion()).thenReturn(MetadataVersion.MINIMUM_KRAFT_VERSION) - when(metadataCache1.metadataVersion()).thenReturn(MetadataVersion.MINIMUM_KRAFT_VERSION) + when(metadataCache0.metadataVersion()).thenReturn(MetadataVersion.MINIMUM_VERSION) + when(metadataCache1.metadataVersion()).thenReturn(MetadataVersion.MINIMUM_VERSION) // each replica manager is for a broker val rm0 = new ReplicaManager( diff --git a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala index b53916fc477..9fde243ec19 100644 --- a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala +++ b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala @@ -21,7 +21,7 @@ import java.io.{ByteArrayOutputStream, File, PrintStream} import java.nio.charset.StandardCharsets import java.nio.file.Files import java.util -import java.util.{Optional, Properties} +import java.util.Properties import kafka.server.KafkaConfig import kafka.utils.TestUtils import net.sourceforge.argparse4j.inf.ArgumentParserException @@ -537,10 +537,10 @@ Found problem: def testVersionMappingWithValidReleaseVersion(): Unit = { val stream = new ByteArrayOutputStream() // Test with a valid release version - assertEquals(0, runVersionMappingCommand(stream, "3.3-IV3")) + assertEquals(0, runVersionMappingCommand(stream, MetadataVersion.MINIMUM_VERSION.toString)) val output = stream.toString() - val metadataVersion = MetadataVersion.IBP_3_3_IV3 + val metadataVersion = MetadataVersion.MINIMUM_VERSION // Check that the metadata version is correctly included in the output assertTrue(output.contains(s"metadata.version=${metadataVersion.featureLevel()} (${metadataVersion.version()})"), s"Output did not contain expected Metadata Version: $output" @@ -589,7 +589,7 @@ Found problem: }) assertEquals("Unknown release version '2.9-IV2'." + - " Supported versions are: " + MetadataVersion.MINIMUM_BOOTSTRAP_VERSION.version + + " Supported versions are: " + MetadataVersion.MINIMUM_VERSION.version + " to " + MetadataVersion.LATEST_PRODUCTION.version, exception.getMessage ) @@ -598,7 +598,7 @@ Found problem: }) assertEquals("Unknown release version 'invalid'." + - " Supported versions are: " + MetadataVersion.MINIMUM_BOOTSTRAP_VERSION.version + + " Supported versions are: " + MetadataVersion.MINIMUM_VERSION.version + " to " + MetadataVersion.LATEST_PRODUCTION.version, exception2.getMessage ) } @@ -712,7 +712,7 @@ Found problem: // Not doing full SCRAM record validation since that's covered elsewhere. // Just checking that we generate the correct number of records - val bootstrapMetadata = new BootstrapDirectory(availableDirs.head.toString, Optional.empty).read + val bootstrapMetadata = new BootstrapDirectory(availableDirs.head.toString).read val scramRecords = bootstrapMetadata.records().asScala .filter(apiMessageAndVersion => apiMessageAndVersion.message().isInstanceOf[UserScramCredentialRecord]) .map(apiMessageAndVersion => apiMessageAndVersion.message().asInstanceOf[UserScramCredentialRecord]) diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index a8d99191a20..edffa3698e7 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -51,7 +51,7 @@ import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.network.metrics.RequestChannelMetrics import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.server.authorizer.{AuthorizableRequestContext, Authorizer => JAuthorizer} -import org.apache.kafka.server.common.ControllerRequestCompletionHandler +import org.apache.kafka.server.common.{ControllerRequestCompletionHandler, TopicIdPartition} import org.apache.kafka.server.config.{DelegationTokenManagerConfigs, KRaftConfigs, ReplicationConfigs, ServerConfigs, ServerLogConfigs} import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.server.util.MockTime diff --git a/docs/ops.html b/docs/ops.html index 968844ce570..ffaca8bf3b8 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -3878,18 +3878,18 @@ In the replica description 0@controller-0:1234:3Db5QLSqSZieL3rJBUUegA, 0 is the quorum:


 Feature: kraft.version  SupportedMinVersion: 0  SupportedMaxVersion: 1  FinalizedVersionLevel: 0 Epoch: 5
-Feature: metadata.version       SupportedMinVersion: 3.0-IV1    SupportedMaxVersion: 3.9-IV0 FinalizedVersionLevel: 3.9-IV0  Epoch: 5
+Feature: metadata.version       SupportedMinVersion: 3.3-IV3    SupportedMaxVersion: 3.9-IV0 FinalizedVersionLevel: 3.9-IV0  Epoch: 5
 

Here is another example of a static quorum:


-Feature: metadata.version       SupportedMinVersion: 3.0-IV1    SupportedMaxVersion: 3.8-IV0 FinalizedVersionLevel: 3.8-IV0  Epoch: 5
+Feature: metadata.version       SupportedMinVersion: 3.3-IV3    SupportedMaxVersion: 3.8-IV0 FinalizedVersionLevel: 3.8-IV0  Epoch: 5
 

Here is an example of a dynamic quorum:


 Feature: kraft.version  SupportedMinVersion: 0  SupportedMaxVersion: 1  FinalizedVersionLevel: 1 Epoch: 5
-Feature: metadata.version       SupportedMinVersion: 3.0-IV1    SupportedMaxVersion: 3.9-IV0 FinalizedVersionLevel: 3.9-IV0  Epoch: 5
+Feature: metadata.version       SupportedMinVersion: 3.3-IV3    SupportedMaxVersion: 3.9-IV0 FinalizedVersionLevel: 3.9-IV0  Epoch: 5
 

The static versus dynamic nature of the quorum is determined at the time of formatting. diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java index 954c67fa8ed..d6bec0c8016 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java @@ -270,7 +270,7 @@ public class ReplicaFetcherThreadBenchmark { config, replicaManager, replicaQuota, - () -> MetadataVersion.MINIMUM_KRAFT_VERSION, + () -> MetadataVersion.MINIMUM_VERSION, () -> -1L ) { @Override @@ -303,7 +303,7 @@ public class ReplicaFetcherThreadBenchmark { replicaManager, replicaQuota, String.format("[ReplicaFetcher replicaId=%d, leaderId=%d, fetcherId=%d", config.brokerId(), 3, 3), - () -> MetadataVersion.MINIMUM_KRAFT_VERSION + () -> MetadataVersion.MINIMUM_VERSION ); pool = partitions; 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 a9ea13d40e0..a535f9f19d7 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ActivationRecordsGenerator.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ActivationRecordsGenerator.java @@ -137,13 +137,6 @@ public class ActivationRecordsGenerator { } } - 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(". "); - } - activationMessageConsumer.accept(logMessageBuilder.toString().trim()); return ControllerResult.atomicOf(records, null); } @@ -153,10 +146,9 @@ public class ActivationRecordsGenerator { *

* If the log is empty, write the bootstrap records. If the log is not empty, do some validation and * possibly write some records to put the log into a valid state. For bootstrap records, if KIP-868 - * metadata transactions are supported, ues them. Otherwise, write the bootstrap records as an + * metadata transactions are supported, use them. Otherwise, write the bootstrap records as an * atomic batch. The single atomic batch can be problematic if the bootstrap records are too large - * (e.g., lots of SCRAM credentials). If ZK migrations are enabled, the activation records will - * include a ZkMigrationState record regardless of whether the log was empty or not. + * (e.g., lots of SCRAM credentials). */ static ControllerResult generate( Consumer activationMessageConsumer, 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 9659eb4945c..590eb703fff 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java @@ -401,16 +401,10 @@ public class ClusterControlManager { record.features().add(processRegistrationFeature(brokerId, finalizedFeatures, feature)); unverifiedFeatures.remove(feature.name()); } - // Brokers that don't send a supported metadata.version range are assumed to only - // support the original metadata.version. - if (request.features().find(MetadataVersion.FEATURE_NAME) == null) { - record.features().add(processRegistrationFeature(brokerId, finalizedFeatures, - new BrokerRegistrationRequestData.Feature(). - setName(MetadataVersion.FEATURE_NAME). - setMinSupportedVersion(MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel()). - setMaxSupportedVersion(MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel()))); - unverifiedFeatures.remove(MetadataVersion.FEATURE_NAME); - } + + if (request.features().find(MetadataVersion.FEATURE_NAME) == null) + throw new InvalidRegistrationException("Request features do not contain '" + MetadataVersion.FEATURE_NAME + "'"); + // We also need to check every controller feature is supported by the broker. unverifiedFeatures.forEach((featureName, finalizedVersion) -> { if (finalizedVersion != 0 && request.features().findAll(featureName).isEmpty()) { @@ -495,8 +489,14 @@ public class ClusterControlManager { FinalizedControllerFeatures finalizedFeatures, BrokerRegistrationRequestData.Feature feature ) { - int defaultVersion = feature.name().equals(MetadataVersion.FEATURE_NAME) ? 1 : 0; // The default value for MetadataVersion is 1 not 0. - short finalized = finalizedFeatures.versionOrDefault(feature.name(), (short) defaultVersion); + // MetadataVersion has no default while the other features default to `0` + short finalized; + if (feature.name().equals(MetadataVersion.FEATURE_NAME)) + finalized = finalizedFeatures.get(feature.name()).orElseThrow(() -> + new IllegalArgumentException("Feature with name '" + MetadataVersion.FEATURE_NAME + "' not found in finalizedFeatures " + finalizedFeatures)); + else + finalized = finalizedFeatures.versionOrDefault(feature.name(), (short) 0); + if (!VersionRange.of(feature.minSupportedVersion(), feature.maxSupportedVersion()).contains(finalized)) { throw new UnsupportedVersionException("Unable to register because the broker " + "does not support finalized version " + finalized + " of " + feature.name() + 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 b5cbc386658..40eb23ce639 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java @@ -55,7 +55,6 @@ public class FeatureControlManager { private SnapshotRegistry snapshotRegistry = null; private QuorumFeatures quorumFeatures = null; private MetadataVersion metadataVersion = MetadataVersion.latestProduction(); - private MetadataVersion minimumBootstrapVersion = MetadataVersion.MINIMUM_BOOTSTRAP_VERSION; private ClusterFeatureSupportDescriber clusterSupportDescriber = new ClusterFeatureSupportDescriber() { @Override public Iterator>> brokerSupported() { @@ -88,11 +87,6 @@ public class FeatureControlManager { return this; } - Builder setMinimumBootstrapVersion(MetadataVersion minimumBootstrapVersion) { - this.minimumBootstrapVersion = minimumBootstrapVersion; - return this; - } - Builder setClusterFeatureSupportDescriber(ClusterFeatureSupportDescriber clusterSupportDescriber) { this.clusterSupportDescriber = clusterSupportDescriber; return this; @@ -104,7 +98,7 @@ public class FeatureControlManager { if (quorumFeatures == null) { Map localSupportedFeatures = new HashMap<>(); localSupportedFeatures.put(MetadataVersion.FEATURE_NAME, VersionRange.of( - MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel(), + MetadataVersion.MINIMUM_VERSION.featureLevel(), MetadataVersion.latestProduction().featureLevel())); quorumFeatures = new QuorumFeatures(0, localSupportedFeatures, Collections.singletonList(0)); } @@ -113,7 +107,6 @@ public class FeatureControlManager { quorumFeatures, snapshotRegistry, metadataVersion, - minimumBootstrapVersion, clusterSupportDescriber ); } @@ -136,11 +129,6 @@ public class FeatureControlManager { */ private final TimelineObject metadataVersion; - /** - * The minimum bootstrap version that we can't downgrade before. - */ - private final MetadataVersion minimumBootstrapVersion; - /** * Gives information about the supported versions in the cluster. */ @@ -151,14 +139,12 @@ public class FeatureControlManager { QuorumFeatures quorumFeatures, SnapshotRegistry snapshotRegistry, MetadataVersion metadataVersion, - MetadataVersion minimumBootstrapVersion, ClusterFeatureSupportDescriber clusterSupportDescriber ) { this.log = logContext.logger(FeatureControlManager.class); this.quorumFeatures = quorumFeatures; this.finalizedVersions = new TimelineHashMap<>(snapshotRegistry, 0); this.metadataVersion = new TimelineObject<>(snapshotRegistry, metadataVersion); - this.minimumBootstrapVersion = minimumBootstrapVersion; this.clusterSupportDescriber = clusterSupportDescriber; } @@ -328,15 +314,10 @@ public class FeatureControlManager { try { newVersion = MetadataVersion.fromFeatureLevel(newVersionLevel); } catch (IllegalArgumentException e) { - return invalidMetadataVersion(newVersionLevel, "Unknown metadata.version."); + return invalidMetadataVersion(newVersionLevel, "Valid versions are from " + + MetadataVersion.MINIMUM_VERSION.featureLevel() + " to " + MetadataVersion.latestTesting().featureLevel() + "."); } - // We cannot set a version earlier than IBP_3_3_IV0, since that was the first version that contained - // FeatureLevelRecord itself. - if (newVersion.isLessThan(minimumBootstrapVersion)) { - return invalidMetadataVersion(newVersionLevel, "Unable to set a metadata.version less than " + - minimumBootstrapVersion); - } if (newVersion.isLessThan(currentVersion)) { // This is a downgrade boolean metadataChanged = MetadataVersion.checkIfMetadataChanged(currentVersion, newVersion); 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 c793ed3427e..fe3a55f7593 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java +++ b/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java @@ -349,8 +349,7 @@ public class PartitionChangeBuilder { // If the election was unclean, we have to forcibly set the ISR to just the // new leader. This can result in data loss! record.setIsr(Collections.singletonList(electionResult.node)); - if (partition.leaderRecoveryState != LeaderRecoveryState.RECOVERING && - metadataVersion.isLeaderRecoverySupported()) { + if (partition.leaderRecoveryState != LeaderRecoveryState.RECOVERING) { // And mark the leader recovery state as RECOVERING record.setLeaderRecoveryState(LeaderRecoveryState.RECOVERING.value()); } 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 4ab5c340fa3..bad7c8fbb84 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -117,7 +117,6 @@ import org.apache.kafka.server.authorizer.AclCreateResult; import org.apache.kafka.server.authorizer.AclDeleteResult; import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.server.common.KRaftVersion; -import org.apache.kafka.server.common.MetadataVersion; import org.apache.kafka.server.fault.FaultHandler; import org.apache.kafka.server.fault.FaultHandlerException; import org.apache.kafka.server.policy.AlterConfigPolicy; @@ -128,7 +127,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; @@ -1539,12 +1537,6 @@ public final class QuorumController implements Controller { setLogContext(logContext). setQuorumFeatures(quorumFeatures). setSnapshotRegistry(snapshotRegistry). - // Set the default metadata version to the minimum KRaft version. This only really - // matters if we are upgrading from a version that didn't store metadata.version in - // the log, such as one of the pre-production 3.0, 3.1, or 3.2 versions. Those versions - // are all treated as 3.0IV1. In newer versions the metadata.version will be specified - // by the log. - setMetadataVersion(MetadataVersion.MINIMUM_KRAFT_VERSION). setClusterFeatureSupportDescriber(clusterSupportDescriber). build(); this.clusterControl = new ClusterControlManager.Builder(). @@ -1643,20 +1635,13 @@ public final class QuorumController implements Controller { /** * Register the writeNoOpRecord task. * - * This task periodically writes a NoOpRecord to the metadata log, if the MetadataVersion - * supports it. + * This task periodically writes a NoOpRecord to the metadata log. * * @param maxIdleIntervalNs The period at which to write the NoOpRecord. */ private void registerWriteNoOpRecord(long maxIdleIntervalNs) { periodicControl.registerTask(new PeriodicTask("writeNoOpRecord", - () -> { - ArrayList records = new ArrayList<>(1); - if (featureControl.metadataVersion().isNoOpRecordSupported()) { - records.add(new ApiMessageAndVersion(new NoOpRecord(), (short) 0)); - } - return ControllerResult.of(records, false); - }, + () -> ControllerResult.of(List.of(new ApiMessageAndVersion(new NoOpRecord(), (short) 0)), false), maxIdleIntervalNs, EnumSet.noneOf(PeriodicTaskFlag.class))); } diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumFeatures.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumFeatures.java index 90017d7b75e..7817fdc3ee0 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumFeatures.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumFeatures.java @@ -57,7 +57,7 @@ public final class QuorumFeatures { public static Map defaultSupportedFeatureMap(boolean enableUnstable) { Map features = new HashMap<>(1); features.put(MetadataVersion.FEATURE_NAME, VersionRange.of( - MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel(), + MetadataVersion.MINIMUM_VERSION.featureLevel(), enableUnstable ? MetadataVersion.latestTesting().featureLevel() : MetadataVersion.latestProduction().featureLevel())); 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 cbb8c6e1b9d..b5d4993f90b 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java @@ -70,12 +70,10 @@ import org.apache.kafka.common.message.ListPartitionReassignmentsResponseData.On import org.apache.kafka.common.message.ListPartitionReassignmentsResponseData.OngoingTopicReassignment; import org.apache.kafka.common.metadata.BrokerRegistrationChangeRecord; import org.apache.kafka.common.metadata.ClearElrRecord; -import org.apache.kafka.common.metadata.FenceBrokerRecord; import org.apache.kafka.common.metadata.PartitionChangeRecord; import org.apache.kafka.common.metadata.PartitionRecord; import org.apache.kafka.common.metadata.RemoveTopicRecord; import org.apache.kafka.common.metadata.TopicRecord; -import org.apache.kafka.common.metadata.UnfenceBrokerRecord; import org.apache.kafka.common.metadata.UnregisterBrokerRecord; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.AlterPartitionRequest; @@ -136,7 +134,6 @@ import static org.apache.kafka.common.protocol.Errors.NEW_LEADER_ELECTED; import static org.apache.kafka.common.protocol.Errors.NONE; import static org.apache.kafka.common.protocol.Errors.NOT_CONTROLLER; import static org.apache.kafka.common.protocol.Errors.NO_REASSIGNMENT_IN_PROGRESS; -import static org.apache.kafka.common.protocol.Errors.OPERATION_NOT_ATTEMPTED; import static org.apache.kafka.common.protocol.Errors.TOPIC_AUTHORIZATION_FAILED; import static org.apache.kafka.common.protocol.Errors.UNKNOWN_TOPIC_ID; import static org.apache.kafka.common.protocol.Errors.UNKNOWN_TOPIC_OR_PARTITION; @@ -847,8 +844,7 @@ public class ReplicationControlManager { for (Entry partEntry : newParts.entrySet()) { int partitionIndex = partEntry.getKey(); PartitionRegistration info = partEntry.getValue(); - records.add(info.toRecord(topicId, partitionIndex, new ImageWriterOptions.Builder(). - setMetadataVersion(featureControl.metadataVersion()). + records.add(info.toRecord(topicId, partitionIndex, new ImageWriterOptions.Builder(featureControl.metadataVersion()). setEligibleLeaderReplicasEnabled(featureControl.isElrFeatureEnabled()). build())); } @@ -1077,20 +1073,17 @@ public class ReplicationControlManager { for (AlterPartitionRequestData.TopicData topicData : request.topics()) { AlterPartitionResponseData.TopicData responseTopicData = new AlterPartitionResponseData.TopicData(). - setTopicName(topicData.topicName()). setTopicId(topicData.topicId()); response.topics().add(responseTopicData); - Uuid topicId = requestVersion > 1 ? topicData.topicId() : topicsByName.get(topicData.topicName()); + Uuid topicId = topicData.topicId(); if (topicId == null || topicId.equals(Uuid.ZERO_UUID) || !topics.containsKey(topicId)) { - Errors error = requestVersion > 1 ? UNKNOWN_TOPIC_ID : UNKNOWN_TOPIC_OR_PARTITION; for (AlterPartitionRequestData.PartitionData partitionData : topicData.partitions()) { responseTopicData.partitions().add(new AlterPartitionResponseData.PartitionData(). setPartitionIndex(partitionData.partitionIndex()). - setErrorCode(error.code())); + setErrorCode(UNKNOWN_TOPIC_ID.code())); } - log.info("Rejecting AlterPartition request for unknown topic ID {} or name {}.", - topicData.topicId(), topicData.topicName()); + log.info("Rejecting AlterPartition request for unknown topic ID {}.", topicData.topicId()); continue; } @@ -1161,8 +1154,7 @@ public class ReplicationControlManager { // fetch new metadata before trying again. This return code is // unusual because we both return an error and generate a new // metadata record. We usually only do one or the other. - // FENCED_LEADER_EPOCH is used for request version below or equal to 1. - Errors error = requestVersion > 1 ? NEW_LEADER_ELECTED : FENCED_LEADER_EPOCH; + Errors error = NEW_LEADER_ELECTED; log.info("AlterPartition request from node {} for {}-{} completed " + "the ongoing partition reassignment and triggered a " + "leadership change. Returning {}.", @@ -1331,12 +1323,7 @@ public class ReplicationControlManager { log.info("Rejecting AlterPartition request from node {} for {}-{} because " + "it specified ineligible replicas {} in the new ISR {}.", brokerId, topic.name, partitionId, ineligibleReplicas, partitionData.newIsrWithEpochs()); - - if (requestApiVersion > 1) { - return INELIGIBLE_REPLICA; - } else { - return OPERATION_NOT_ATTEMPTED; - } + return INELIGIBLE_REPLICA; } return Errors.NONE; @@ -1368,7 +1355,7 @@ public class ReplicationControlManager { * Generate the appropriate records to handle a broker being fenced. * * First, we remove this broker from any ISR. Then we generate a - * FenceBrokerRecord. + * BrokerRegistrationChangeRecord. * * @param brokerId The broker id. * @param records The record list to append to. @@ -1380,16 +1367,10 @@ public class ReplicationControlManager { } generateLeaderAndIsrUpdates("handleBrokerFenced", brokerId, NO_LEADER, NO_LEADER, records, brokersToIsrs.partitionsWithBrokerInIsr(brokerId)); - if (featureControl.metadataVersion().isBrokerRegistrationChangeRecordSupported()) { - records.add(new ApiMessageAndVersion(new BrokerRegistrationChangeRecord(). - setBrokerId(brokerId).setBrokerEpoch(brokerRegistration.epoch()). - setFenced(BrokerRegistrationFencingChange.FENCE.value()), - (short) 0)); - } else { - records.add(new ApiMessageAndVersion(new FenceBrokerRecord(). - setId(brokerId).setEpoch(brokerRegistration.epoch()), - (short) 0)); - } + records.add(new ApiMessageAndVersion(new BrokerRegistrationChangeRecord(). + setBrokerId(brokerId).setBrokerEpoch(brokerRegistration.epoch()). + setFenced(BrokerRegistrationFencingChange.FENCE.value()), + (short) 0)); } /** @@ -1416,7 +1397,7 @@ public class ReplicationControlManager { /** * Generate the appropriate records to handle a broker becoming unfenced. * - * First, we create an UnfenceBrokerRecord. Then, we check if there are any + * First, we create a BrokerRegistrationChangeRecord. Then, we check if there are any * partitions that don't currently have a leader that should be led by the newly * unfenced broker. * @@ -1425,15 +1406,10 @@ public class ReplicationControlManager { * @param records The record list to append to. */ void handleBrokerUnfenced(int brokerId, long brokerEpoch, List records) { - if (featureControl.metadataVersion().isBrokerRegistrationChangeRecordSupported()) { - records.add(new ApiMessageAndVersion(new BrokerRegistrationChangeRecord(). - setBrokerId(brokerId).setBrokerEpoch(brokerEpoch). - setFenced(BrokerRegistrationFencingChange.UNFENCE.value()), - (short) 0)); - } else { - records.add(new ApiMessageAndVersion(new UnfenceBrokerRecord().setId(brokerId). - setEpoch(brokerEpoch), (short) 0)); - } + records.add(new ApiMessageAndVersion(new BrokerRegistrationChangeRecord(). + setBrokerId(brokerId).setBrokerEpoch(brokerEpoch). + setFenced(BrokerRegistrationFencingChange.UNFENCE.value()), + (short) 0)); generateLeaderAndIsrUpdates("handleBrokerUnfenced", NO_LEADER, brokerId, NO_LEADER, records, brokersToIsrs.partitionsWithNoLeader()); } @@ -1450,8 +1426,7 @@ public class ReplicationControlManager { * @param records The record list to append to. */ void handleBrokerInControlledShutdown(int brokerId, long brokerEpoch, List records) { - if (featureControl.metadataVersion().isInControlledShutdownStateSupported() - && !clusterControl.inControlledShutdown(brokerId)) { + if (!clusterControl.inControlledShutdown(brokerId)) { records.add(new ApiMessageAndVersion(new BrokerRegistrationChangeRecord(). setBrokerId(brokerId).setBrokerEpoch(brokerEpoch). setInControlledShutdown(BrokerRegistrationInControlledShutdownChange.IN_CONTROLLED_SHUTDOWN.value()), @@ -1941,8 +1916,7 @@ public class ReplicationControlManager { " time(s): All brokers are currently fenced or in controlled shutdown."); } records.add(buildPartitionRegistration(partitionAssignment, isr) - .toRecord(topicId, partitionId, new ImageWriterOptions.Builder(). - setMetadataVersion(featureControl.metadataVersion()). + .toRecord(topicId, partitionId, new ImageWriterOptions.Builder(featureControl.metadataVersion()). setEligibleLeaderReplicasEnabled(featureControl.isElrFeatureEnabled()). build())); partitionId++; diff --git a/metadata/src/main/java/org/apache/kafka/image/AclsImage.java b/metadata/src/main/java/org/apache/kafka/image/AclsImage.java index b9bb42678a7..c2fe142d71f 100644 --- a/metadata/src/main/java/org/apache/kafka/image/AclsImage.java +++ b/metadata/src/main/java/org/apache/kafka/image/AclsImage.java @@ -20,7 +20,6 @@ package org.apache.kafka.image; import org.apache.kafka.common.Uuid; import org.apache.kafka.image.node.AclsImageNode; import org.apache.kafka.image.writer.ImageWriter; -import org.apache.kafka.image.writer.ImageWriterOptions; import org.apache.kafka.metadata.authorizer.StandardAcl; import org.apache.kafka.metadata.authorizer.StandardAclWithId; @@ -51,11 +50,7 @@ public final class AclsImage { return acls; } - public void write(ImageWriter writer, ImageWriterOptions options) { - // Technically, AccessControlEntryRecord appeared in 3.2-IV0, so we should not write it if - // the output version is less than that. However, there is a problem: pre-production KRaft - // images didn't support FeatureLevelRecord, so we can't distinguish 3.2-IV0 from 3.0-IV1. - // The least bad way to resolve this is just to pretend that ACLs were in 3.0-IV1. + public void write(ImageWriter writer) { for (Entry entry : acls.entrySet()) { StandardAclWithId aclWithId = new StandardAclWithId(entry.getKey(), entry.getValue()); writer.write(0, aclWithId.toRecord()); diff --git a/metadata/src/main/java/org/apache/kafka/image/FeaturesDelta.java b/metadata/src/main/java/org/apache/kafka/image/FeaturesDelta.java index 587e42d7c98..3a3cbefe448 100644 --- a/metadata/src/main/java/org/apache/kafka/image/FeaturesDelta.java +++ b/metadata/src/main/java/org/apache/kafka/image/FeaturesDelta.java @@ -30,6 +30,7 @@ import java.util.Optional; * Represents changes to the cluster in the metadata image. */ public final class FeaturesDelta { + private static final short MINIMUM_PERSISTED_FEATURE_LEVEL = 4; private final FeaturesImage image; private final Map> changes = new HashMap<>(); @@ -58,7 +59,13 @@ public final class FeaturesDelta { public void replay(FeatureLevelRecord record) { if (record.name().equals(MetadataVersion.FEATURE_NAME)) { - metadataVersionChange = MetadataVersion.fromFeatureLevel(record.featureLevel()); + try { + metadataVersionChange = MetadataVersion.fromFeatureLevel(record.featureLevel()); + } catch (IllegalArgumentException e) { + throw new IllegalArgumentException("Unsupported metadata version - if you are currently upgrading your cluster, " + + "please ensure the metadata version is set to " + MetadataVersion.MINIMUM_VERSION + " (or higher) before " + + "updating the software version. The metadata version can be updated via the `kafka-features` command-line tool.", e); + } } else { if (record.featureLevel() == 0) { changes.put(record.name(), Optional.empty()); @@ -90,11 +97,11 @@ public final class FeaturesDelta { } } - final MetadataVersion metadataVersion; + final Optional metadataVersion; if (metadataVersionChange == null) { metadataVersion = image.metadataVersion(); } else { - metadataVersion = metadataVersionChange; + metadataVersion = Optional.of(metadataVersionChange); } return new FeaturesImage(newFinalizedVersions, metadataVersion); diff --git a/metadata/src/main/java/org/apache/kafka/image/FeaturesImage.java b/metadata/src/main/java/org/apache/kafka/image/FeaturesImage.java index b4a7c4f30e2..a8ca48ad731 100644 --- a/metadata/src/main/java/org/apache/kafka/image/FeaturesImage.java +++ b/metadata/src/main/java/org/apache/kafka/image/FeaturesImage.java @@ -24,9 +24,7 @@ import org.apache.kafka.image.writer.ImageWriterOptions; import org.apache.kafka.server.common.EligibleLeaderReplicasVersion; import org.apache.kafka.server.common.MetadataVersion; -import java.util.ArrayList; import java.util.Collections; -import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Objects; @@ -41,29 +39,40 @@ import java.util.Optional; public final class FeaturesImage { public static final FeaturesImage EMPTY = new FeaturesImage( Collections.emptyMap(), - MetadataVersion.MINIMUM_KRAFT_VERSION + Optional.empty() ); private final Map finalizedVersions; - private final MetadataVersion metadataVersion; + private final Optional metadataVersion; public FeaturesImage( - Map finalizedVersions, - MetadataVersion metadataVersion) { + Map finalizedVersions, + MetadataVersion metadataVersion) { + this(finalizedVersions, Optional.of(metadataVersion)); + } + + FeaturesImage( + Map finalizedVersions, + Optional metadataVersion) { this.finalizedVersions = Collections.unmodifiableMap(finalizedVersions); this.metadataVersion = metadataVersion; } public boolean isEmpty() { - return finalizedVersions.isEmpty() && - metadataVersion.equals(MetadataVersion.MINIMUM_KRAFT_VERSION); + return finalizedVersions.isEmpty() && metadataVersion.isEmpty(); } - public MetadataVersion metadataVersion() { + public Optional metadataVersion() { return metadataVersion; } + + public MetadataVersion metadataVersionOrThrow() { + return metadataVersion.orElseThrow(() -> + new IllegalStateException("Unknown metadata version for FeaturesImage: " + this)); + } + public Map finalizedVersions() { return finalizedVersions; } @@ -73,26 +82,8 @@ public final class FeaturesImage { >= EligibleLeaderReplicasVersion.ELRV_1.featureLevel(); } - private Optional finalizedVersion(String feature) { - return Optional.ofNullable(finalizedVersions.get(feature)); - } - public void write(ImageWriter writer, ImageWriterOptions options) { - if (options.metadataVersion().isLessThan(MetadataVersion.MINIMUM_BOOTSTRAP_VERSION)) { - handleFeatureLevelNotSupported(options); - } else { - writeFeatureLevels(writer, options); - } - } - - private void handleFeatureLevelNotSupported(ImageWriterOptions options) { - // If the metadata version is older than 3.3-IV0, we can't represent any feature flags, - // because the FeatureLevel record is not supported. - if (!finalizedVersions.isEmpty()) { - List features = new ArrayList<>(finalizedVersions.keySet()); - features.sort(String::compareTo); - options.handleLoss("feature flag(s): " + String.join(", ", features)); - } + writeFeatureLevels(writer, options); } private void writeFeatureLevels(ImageWriter writer, ImageWriterOptions options) { diff --git a/metadata/src/main/java/org/apache/kafka/image/MetadataImage.java b/metadata/src/main/java/org/apache/kafka/image/MetadataImage.java index dca085ae02d..fceca8f564e 100644 --- a/metadata/src/main/java/org/apache/kafka/image/MetadataImage.java +++ b/metadata/src/main/java/org/apache/kafka/image/MetadataImage.java @@ -156,7 +156,7 @@ public final class MetadataImage { configs.write(writer, options); clientQuotas.write(writer, options); producerIds.write(writer, options); - acls.write(writer, options); + acls.write(writer); scram.write(writer, options); delegationTokens.write(writer, options); writer.close(true); diff --git a/metadata/src/main/java/org/apache/kafka/image/loader/MetadataBatchLoader.java b/metadata/src/main/java/org/apache/kafka/image/loader/MetadataBatchLoader.java index 6dca7caf4d5..c4b6286f2a9 100644 --- a/metadata/src/main/java/org/apache/kafka/image/loader/MetadataBatchLoader.java +++ b/metadata/src/main/java/org/apache/kafka/image/loader/MetadataBatchLoader.java @@ -100,7 +100,7 @@ public class MetadataBatchLoader { */ public final void resetToImage(MetadataImage image) { this.image = image; - this.hasSeenRecord = true; + this.hasSeenRecord = !image.isEmpty(); this.delta = new MetadataDelta.Builder().setImage(image).build(); this.transactionState = TransactionState.NO_TRANSACTION; this.lastOffset = image.provenance().lastContainedOffset(); diff --git a/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java b/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java index 703123fe395..4fabd1863b6 100644 --- a/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java +++ b/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java @@ -123,7 +123,6 @@ public class MetadataLoader implements RaftClient.Listener return new MetadataLoader( time, logContext, - nodeId, threadNamePrefix, faultHandler, metrics, @@ -194,7 +193,6 @@ public class MetadataLoader implements RaftClient.Listener private MetadataLoader( Time time, LogContext logContext, - int nodeId, String threadNamePrefix, FaultHandler faultHandler, MetadataLoaderMetrics metrics, @@ -293,8 +291,7 @@ public class MetadataLoader implements RaftClient.Listener setImage(MetadataImage.EMPTY). build(); ImageReWriter writer = new ImageReWriter(delta); - image.write(writer, new ImageWriterOptions.Builder(). - setMetadataVersion(image.features().metadataVersion()). + image.write(writer, new ImageWriterOptions.Builder(image.features().metadataVersionOrThrow()). setEligibleLeaderReplicasEnabled(image.features().isElrEnabled()). build()); // ImageReWriter#close invokes finishSnapshot, so we don't need to invoke it here. @@ -348,7 +345,7 @@ public class MetadataLoader implements RaftClient.Listener } } metrics.updateLastAppliedImageProvenance(image.provenance()); - metrics.setCurrentMetadataVersion(image.features().metadataVersion()); + metrics.setCurrentMetadataVersion(image.features().metadataVersionOrThrow()); if (!uninitializedPublishers.isEmpty()) { scheduleInitializeNewPublishers(0); } diff --git a/metadata/src/main/java/org/apache/kafka/image/loader/metrics/MetadataLoaderMetrics.java b/metadata/src/main/java/org/apache/kafka/image/loader/metrics/MetadataLoaderMetrics.java index d30b3918c4e..feaf387d4c3 100644 --- a/metadata/src/main/java/org/apache/kafka/image/loader/metrics/MetadataLoaderMetrics.java +++ b/metadata/src/main/java/org/apache/kafka/image/loader/metrics/MetadataLoaderMetrics.java @@ -45,7 +45,7 @@ public final class MetadataLoaderMetrics implements AutoCloseable { private final Optional registry; private final AtomicReference currentMetadataVersion = - new AtomicReference<>(MetadataVersion.MINIMUM_KRAFT_VERSION); + new AtomicReference<>(MetadataVersion.MINIMUM_VERSION); private final AtomicInteger currentControllerId = new AtomicInteger(-1); private final AtomicLong handleLoadSnapshotCount = new AtomicLong(0); private final Consumer batchProcessingTimeNsUpdater; diff --git a/metadata/src/main/java/org/apache/kafka/image/node/MetadataNode.java b/metadata/src/main/java/org/apache/kafka/image/node/MetadataNode.java index 7b7eba78789..6e00c214d0a 100644 --- a/metadata/src/main/java/org/apache/kafka/image/node/MetadataNode.java +++ b/metadata/src/main/java/org/apache/kafka/image/node/MetadataNode.java @@ -53,7 +53,8 @@ public interface MetadataNode { for (String name : names) { printer.enterNode(name); MetadataNode child = child(name); - child.print(printer); + if (child != null) + child.print(printer); printer.leaveNode(); } } diff --git a/metadata/src/main/java/org/apache/kafka/image/publisher/BrokerRegistrationTracker.java b/metadata/src/main/java/org/apache/kafka/image/publisher/BrokerRegistrationTracker.java index dfb97934cea..3028b8b3108 100644 --- a/metadata/src/main/java/org/apache/kafka/image/publisher/BrokerRegistrationTracker.java +++ b/metadata/src/main/java/org/apache/kafka/image/publisher/BrokerRegistrationTracker.java @@ -90,8 +90,7 @@ public class BrokerRegistrationTracker implements MetadataPublisher { } } if (checkBrokerRegistration) { - if (brokerRegistrationNeedsRefresh(newImage.features().metadataVersion(), - delta.clusterDelta().broker(id))) { + if (brokerRegistrationNeedsRefresh(newImage.features().metadataVersionOrThrow(), delta.clusterDelta().broker(id))) { refreshRegistrationCallback.run(); } } diff --git a/metadata/src/main/java/org/apache/kafka/image/publisher/SnapshotEmitter.java b/metadata/src/main/java/org/apache/kafka/image/publisher/SnapshotEmitter.java index 18d7be0ea6d..398e3fdae8a 100644 --- a/metadata/src/main/java/org/apache/kafka/image/publisher/SnapshotEmitter.java +++ b/metadata/src/main/java/org/apache/kafka/image/publisher/SnapshotEmitter.java @@ -146,8 +146,7 @@ public class SnapshotEmitter implements SnapshotGenerator.Emitter { } RaftSnapshotWriter writer = new RaftSnapshotWriter(snapshotWriter.get(), batchSize); try { - image.write(writer, new ImageWriterOptions.Builder(). - setMetadataVersion(image.features().metadataVersion()). + image.write(writer, new ImageWriterOptions.Builder(image.features().metadataVersionOrThrow()). setEligibleLeaderReplicasEnabled(image.features().isElrEnabled()). build()); writer.close(true); diff --git a/metadata/src/main/java/org/apache/kafka/image/writer/ImageWriterOptions.java b/metadata/src/main/java/org/apache/kafka/image/writer/ImageWriterOptions.java index 457229d6364..b6d676b3744 100644 --- a/metadata/src/main/java/org/apache/kafka/image/writer/ImageWriterOptions.java +++ b/metadata/src/main/java/org/apache/kafka/image/writer/ImageWriterOptions.java @@ -29,30 +29,22 @@ import java.util.function.Consumer; public final class ImageWriterOptions { public static class Builder { private MetadataVersion metadataVersion; - private MetadataVersion requestedMetadataVersion; - private boolean isEligibleLeaderReplicasEnabled = false; private Consumer lossHandler = e -> { throw e; }; + private boolean isEligibleLeaderReplicasEnabled = false; - public Builder() { - this.metadataVersion = MetadataVersion.latestProduction(); + public Builder(MetadataVersion metadataVersion) { + this.metadataVersion = metadataVersion; } public Builder(MetadataImage image) { - this.metadataVersion = image.features().metadataVersion(); + this.metadataVersion = image.features().metadataVersionOrThrow(); this.isEligibleLeaderReplicasEnabled = image.features().isElrEnabled(); } public Builder setMetadataVersion(MetadataVersion metadataVersion) { - this.requestedMetadataVersion = metadataVersion; - if (metadataVersion.isLessThan(MetadataVersion.MINIMUM_BOOTSTRAP_VERSION)) { - // When writing an image, all versions less than 3.3-IV0 are treated as 3.0-IV1. - // This is because those versions don't support FeatureLevelRecord. - this.metadataVersion = MetadataVersion.MINIMUM_KRAFT_VERSION; - } else { - this.metadataVersion = metadataVersion; - } + this.metadataVersion = metadataVersion; return this; } @@ -65,10 +57,6 @@ public final class ImageWriterOptions { return metadataVersion; } - public MetadataVersion requestedMetadataVersion() { - return requestedMetadataVersion; - } - public boolean isEligibleLeaderReplicasEnabled() { return isEligibleLeaderReplicasEnabled; } @@ -79,24 +67,21 @@ public final class ImageWriterOptions { } public ImageWriterOptions build() { - return new ImageWriterOptions(metadataVersion, lossHandler, requestedMetadataVersion, isEligibleLeaderReplicasEnabled); + return new ImageWriterOptions(metadataVersion, lossHandler, isEligibleLeaderReplicasEnabled); } } private final MetadataVersion metadataVersion; - private final MetadataVersion requestedMetadataVersion; private final Consumer lossHandler; private final boolean isEligibleLeaderReplicasEnabled; private ImageWriterOptions( MetadataVersion metadataVersion, Consumer lossHandler, - MetadataVersion orgMetadataVersion, boolean isEligibleLeaderReplicasEnabled ) { this.metadataVersion = metadataVersion; this.lossHandler = lossHandler; - this.requestedMetadataVersion = orgMetadataVersion; this.isEligibleLeaderReplicasEnabled = isEligibleLeaderReplicasEnabled; } @@ -108,6 +93,6 @@ public final class ImageWriterOptions { } public void handleLoss(String loss) { - lossHandler.accept(new UnwritableMetadataException(requestedMetadataVersion, loss)); + lossHandler.accept(new UnwritableMetadataException(metadataVersion, loss)); } } diff --git a/metadata/src/main/java/org/apache/kafka/metadata/BrokerRegistration.java b/metadata/src/main/java/org/apache/kafka/metadata/BrokerRegistration.java index fbaf8a85510..1a473ad5599 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/BrokerRegistration.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/BrokerRegistration.java @@ -294,15 +294,8 @@ public class BrokerRegistration { setRack(rack.orElse(null)). setBrokerEpoch(epoch). setIncarnationId(incarnationId). - setFenced(fenced); - - if (inControlledShutdown) { - if (options.metadataVersion().isInControlledShutdownStateSupported()) { - registrationRecord.setInControlledShutdown(true); - } else { - options.handleLoss("the inControlledShutdown state of one or more brokers"); - } - } + setFenced(fenced). + setInControlledShutdown(inControlledShutdown); if (isMigratingZkBroker) { if (options.metadataVersion().isMigrationSupported()) { diff --git a/metadata/src/main/java/org/apache/kafka/metadata/ControllerRegistration.java b/metadata/src/main/java/org/apache/kafka/metadata/ControllerRegistration.java index 2e2c4889dff..6aa53652216 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/ControllerRegistration.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/ControllerRegistration.java @@ -111,7 +111,7 @@ public class ControllerRegistration { if (supportedFeatures == null) { supportedFeatures = new HashMap<>(); supportedFeatures.put(MetadataVersion.FEATURE_NAME, VersionRange.of( - MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel(), + MetadataVersion.MINIMUM_VERSION.featureLevel(), MetadataVersion.latestProduction().featureLevel())); } return new ControllerRegistration(id, diff --git a/metadata/src/main/java/org/apache/kafka/metadata/bootstrap/BootstrapDirectory.java b/metadata/src/main/java/org/apache/kafka/metadata/bootstrap/BootstrapDirectory.java index 3a4c3ae8fc5..dbeaeaa6524 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/bootstrap/BootstrapDirectory.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/bootstrap/BootstrapDirectory.java @@ -31,12 +31,9 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Objects; -import java.util.Optional; import static java.nio.file.StandardCopyOption.ATOMIC_MOVE; import static java.nio.file.StandardCopyOption.REPLACE_EXISTING; -import static org.apache.kafka.server.common.MetadataVersion.MINIMUM_BOOTSTRAP_VERSION; - /** * A read-only class that holds the controller bootstrap metadata. A file named "bootstrap.checkpoint" is used and the @@ -46,21 +43,16 @@ public class BootstrapDirectory { public static final String BINARY_BOOTSTRAP_FILENAME = "bootstrap.checkpoint"; private final String directoryPath; - private final Optional ibp; /** * Create a new BootstrapDirectory object. * * @param directoryPath The path to the directory with the bootstrap file. - * @param ibp The configured value of inter.broker.protocol, or the empty string - * if it is not configured. */ public BootstrapDirectory( - String directoryPath, - Optional ibp + String directoryPath ) { this.directoryPath = Objects.requireNonNull(directoryPath); - this.ibp = Objects.requireNonNull(ibp); } public BootstrapMetadata read() throws Exception { @@ -82,16 +74,7 @@ public class BootstrapDirectory { } BootstrapMetadata readFromConfiguration() { - if (ibp.isEmpty()) { - return BootstrapMetadata.fromVersion(MetadataVersion.latestProduction(), "the default bootstrap"); - } - MetadataVersion version = MetadataVersion.fromVersionString(ibp.get()); - if (version.isLessThan(MINIMUM_BOOTSTRAP_VERSION)) { - return BootstrapMetadata.fromVersion(MINIMUM_BOOTSTRAP_VERSION, - "the minimum version bootstrap with metadata.version " + MINIMUM_BOOTSTRAP_VERSION); - } - return BootstrapMetadata.fromVersion(version, - "the configured bootstrap with metadata.version " + version); + return BootstrapMetadata.fromVersion(MetadataVersion.latestProduction(), "the default bootstrap"); } BootstrapMetadata readFromBinaryFile(String binaryPath) throws Exception { diff --git a/metadata/src/main/java/org/apache/kafka/metadata/bootstrap/BootstrapMetadata.java b/metadata/src/main/java/org/apache/kafka/metadata/bootstrap/BootstrapMetadata.java index 1dd6beedeaf..3cb1aed1794 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/bootstrap/BootstrapMetadata.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/bootstrap/BootstrapMetadata.java @@ -30,9 +30,6 @@ import java.util.Map; import java.util.Objects; import java.util.Optional; -import static org.apache.kafka.server.common.MetadataVersion.MINIMUM_BOOTSTRAP_VERSION; - - /** * The bootstrap metadata. On startup, if the metadata log is empty, we will populate the log with * these records. Alternately, if log is not empty, but the metadata version is not set, we will @@ -111,11 +108,6 @@ public class BootstrapMetadata { String source ) { this.records = Objects.requireNonNull(records); - if (metadataVersion.isLessThan(MINIMUM_BOOTSTRAP_VERSION)) { - throw new RuntimeException("Bootstrap metadata.version before " + - MINIMUM_BOOTSTRAP_VERSION + " are not supported. Can't load metadata from " + - source); - } this.metadataVersion = metadataVersion; Objects.requireNonNull(source); this.source = source; diff --git a/metadata/src/main/java/org/apache/kafka/metadata/publisher/FeaturesPublisher.java b/metadata/src/main/java/org/apache/kafka/metadata/publisher/FeaturesPublisher.java index 01572dd9411..60fa202ee42 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/publisher/FeaturesPublisher.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/publisher/FeaturesPublisher.java @@ -26,12 +26,12 @@ import org.apache.kafka.server.common.FinalizedFeatures; import org.slf4j.Logger; -import static org.apache.kafka.server.common.MetadataVersion.MINIMUM_KRAFT_VERSION; +import static org.apache.kafka.server.common.MetadataVersion.MINIMUM_VERSION; public class FeaturesPublisher implements MetadataPublisher { private final Logger log; - private volatile FinalizedFeatures finalizedFeatures = FinalizedFeatures.fromKRaftVersion(MINIMUM_KRAFT_VERSION); + private volatile FinalizedFeatures finalizedFeatures = FinalizedFeatures.fromKRaftVersion(MINIMUM_VERSION); public FeaturesPublisher( LogContext logContext @@ -55,7 +55,7 @@ public class FeaturesPublisher implements MetadataPublisher { LoaderManifest manifest ) { if (delta.featuresDelta() != null) { - FinalizedFeatures newFinalizedFeatures = new FinalizedFeatures(newImage.features().metadataVersion(), + FinalizedFeatures newFinalizedFeatures = new FinalizedFeatures(newImage.features().metadataVersionOrThrow(), newImage.features().finalizedVersions(), newImage.provenance().lastContainedOffset() ); diff --git a/metadata/src/main/java/org/apache/kafka/metadata/storage/Formatter.java b/metadata/src/main/java/org/apache/kafka/metadata/storage/Formatter.java index 79437d4da6d..acba0f7a04b 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/storage/Formatter.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/storage/Formatter.java @@ -284,10 +284,6 @@ public class Formatter { } MetadataVersion verifyReleaseVersion(MetadataVersion metadataVersion) { - if (!metadataVersion.isKRaftSupported()) { - throw new FormatterException(MetadataVersion.FEATURE_NAME + " " + metadataVersion + - " is too old to be supported."); - } if (!unstableFeatureVersionsEnabled) { if (!metadataVersion.isProduction()) { throw new FormatterException(MetadataVersion.FEATURE_NAME + " " + metadataVersion + @@ -435,7 +431,7 @@ public class Formatter { directoryTypes.get(writeLogDir).description(), writeLogDir, MetadataVersion.FEATURE_NAME, releaseVersion); Files.createDirectories(Paths.get(writeLogDir)); - BootstrapDirectory bootstrapDirectory = new BootstrapDirectory(writeLogDir, Optional.empty()); + BootstrapDirectory bootstrapDirectory = new BootstrapDirectory(writeLogDir); bootstrapDirectory.writeBinaryFile(bootstrapMetadata); if (directoryTypes.get(writeLogDir).isDynamicMetadataDirectory()) { writeDynamicQuorumSnapshot(writeLogDir, 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 1668b6bebe3..2e670fa4933 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ActivationRecordsGeneratorTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ActivationRecordsGeneratorTest.java @@ -41,10 +41,10 @@ public class ActivationRecordsGeneratorTest { public void testActivationMessageForEmptyLog() { ControllerResult result = ActivationRecordsGenerator.recordsForEmptyLog( logMsg -> assertEquals("Performing controller activation. The metadata log appears to be empty. " + - "Appending 1 bootstrap record(s) at metadata.version 3.0-IV1 from bootstrap source 'test'.", logMsg), + "Appending 1 bootstrap record(s) at metadata.version 3.3-IV3 from bootstrap source 'test'.", logMsg), -1L, - BootstrapMetadata.fromVersion(MetadataVersion.MINIMUM_BOOTSTRAP_VERSION, "test"), - MetadataVersion.MINIMUM_KRAFT_VERSION, + BootstrapMetadata.fromVersion(MetadataVersion.MINIMUM_VERSION, "test"), + MetadataVersion.MINIMUM_VERSION, 2 ); 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 28b22d5a77d..9298da8df3a 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java @@ -28,7 +28,6 @@ import org.apache.kafka.common.message.BrokerRegistrationRequestData; import org.apache.kafka.common.message.ControllerRegistrationRequestData; import org.apache.kafka.common.metadata.BrokerRegistrationChangeRecord; import org.apache.kafka.common.metadata.FeatureLevelRecord; -import org.apache.kafka.common.metadata.FenceBrokerRecord; import org.apache.kafka.common.metadata.PartitionChangeRecord; import org.apache.kafka.common.metadata.RegisterBrokerRecord; import org.apache.kafka.common.metadata.RegisterBrokerRecord.BrokerEndpoint; @@ -60,7 +59,6 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; -import org.junit.jupiter.params.provider.EnumSource; import org.junit.jupiter.params.provider.MethodSource; import org.junit.jupiter.params.provider.ValueSource; @@ -76,7 +74,6 @@ import java.util.OptionalLong; import java.util.stream.Stream; import static java.util.Arrays.asList; -import static org.apache.kafka.server.common.MetadataVersion.IBP_3_3_IV2; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -86,9 +83,9 @@ import static org.junit.jupiter.api.Assertions.assertTrue; @Timeout(value = 40) public class ClusterControlManagerTest { - @ParameterizedTest - @EnumSource(value = MetadataVersion.class, names = {"IBP_3_0_IV1", "IBP_3_3_IV2"}) - public void testReplay(MetadataVersion metadataVersion) { + + @Test + public void testReplay() { MockTime time = new MockTime(0, 0, 0); SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); @@ -123,27 +120,15 @@ public class ClusterControlManagerTest { assertFalse(clusterControl.isUnfenced(0)); assertFalse(clusterControl.isUnfenced(1)); - if (metadataVersion.isLessThan(IBP_3_3_IV2)) { - UnfenceBrokerRecord unfenceBrokerRecord = - new UnfenceBrokerRecord().setId(1).setEpoch(100); - clusterControl.replay(unfenceBrokerRecord); - } else { - BrokerRegistrationChangeRecord changeRecord = - new BrokerRegistrationChangeRecord().setBrokerId(1).setBrokerEpoch(100).setFenced(BrokerRegistrationFencingChange.UNFENCE.value()); - clusterControl.replay(changeRecord); - } + BrokerRegistrationChangeRecord changeRecord = + new BrokerRegistrationChangeRecord().setBrokerId(1).setBrokerEpoch(100).setFenced(BrokerRegistrationFencingChange.UNFENCE.value()); + clusterControl.replay(changeRecord); assertFalse(clusterControl.isUnfenced(0)); assertTrue(clusterControl.isUnfenced(1)); - if (metadataVersion.isLessThan(IBP_3_3_IV2)) { - FenceBrokerRecord fenceBrokerRecord = - new FenceBrokerRecord().setId(1).setEpoch(100); - clusterControl.replay(fenceBrokerRecord); - } else { - BrokerRegistrationChangeRecord changeRecord = - new BrokerRegistrationChangeRecord().setBrokerId(1).setBrokerEpoch(100).setFenced(BrokerRegistrationFencingChange.FENCE.value()); - clusterControl.replay(changeRecord); - } + changeRecord = + new BrokerRegistrationChangeRecord().setBrokerId(1).setBrokerEpoch(100).setFenced(BrokerRegistrationFencingChange.FENCE.value()); + clusterControl.replay(changeRecord); assertFalse(clusterControl.isUnfenced(0)); assertFalse(clusterControl.isUnfenced(1)); } @@ -289,8 +274,7 @@ public class ClusterControlManagerTest { private static Stream metadataVersions() { return Stream.of( - MetadataVersion.IBP_3_3_IV2, - MetadataVersion.IBP_3_3_IV3, + MetadataVersion.MINIMUM_VERSION, MetadataVersion.IBP_3_7_IV2, // introduces directory assignment MetadataVersion.latestTesting() ).map(Arguments::of); @@ -327,9 +311,14 @@ public class ClusterControlManagerTest { setBrokerId(0). setLogDirs(logDirs). setRack(null). + setFeatures(new BrokerRegistrationRequestData.FeatureCollection( + Collections.singleton(new BrokerRegistrationRequestData.Feature(). + setName(MetadataVersion.FEATURE_NAME). + setMinSupportedVersion(metadataVersion.featureLevel()). + setMaxSupportedVersion(metadataVersion.featureLevel())).iterator())). setIncarnationId(Uuid.fromString("0H4fUu1xQEKXFYwB1aBjhg")), 123L, - new FinalizedControllerFeatures(Collections.emptyMap(), 456L), + new FinalizedControllerFeatures(Map.of(MetadataVersion.FEATURE_NAME, metadataVersion.featureLevel()), 456L), false); short expectedVersion = metadataVersion.registerBrokerRecordVersion(); @@ -345,8 +334,8 @@ public class ClusterControlManagerTest { setFeatures(new RegisterBrokerRecord.BrokerFeatureCollection(Collections.singletonList( new RegisterBrokerRecord.BrokerFeature(). setName(MetadataVersion.FEATURE_NAME). - setMinSupportedVersion((short) 1). - setMaxSupportedVersion((short) 1)).iterator())). + setMinSupportedVersion(metadataVersion.featureLevel()). + setMaxSupportedVersion(metadataVersion.featureLevel())).iterator())). setInControlledShutdown(false), expectedVersion)), result.records()); } @@ -461,9 +450,9 @@ public class ClusterControlManagerTest { } } - @ParameterizedTest - @EnumSource(value = MetadataVersion.class, names = {"IBP_3_3_IV2", "IBP_3_3_IV3"}) - public void testRegistrationsToRecords(MetadataVersion metadataVersion) { + @Test + public void testRegistrationsToRecords() { + MetadataVersion metadataVersion = MetadataVersion.MINIMUM_VERSION; MockTime time = new MockTime(0, 0, 0); SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); FeatureControlManager featureControl = new FeatureControlManager.Builder(). @@ -506,8 +495,7 @@ public class ClusterControlManagerTest { clusterControl.replay(registrationChangeRecord); short expectedVersion = metadataVersion.registerBrokerRecordVersion(); - ImageWriterOptions options = new ImageWriterOptions.Builder(). - setMetadataVersion(metadataVersion). + ImageWriterOptions options = new ImageWriterOptions.Builder(metadataVersion). setLossHandler(__ -> { }). build(); assertEquals(new ApiMessageAndVersion(new RegisterBrokerRecord(). @@ -517,7 +505,7 @@ public class ClusterControlManagerTest { setPort((short) 9092). setName("PLAINTEXT"). setHost("example.com")).iterator())). - setInControlledShutdown(metadataVersion.isInControlledShutdownStateSupported()). + setInControlledShutdown(true). setFenced(false), expectedVersion), clusterControl.brokerRegistrations().get(0).toRecord(options)); assertEquals(new ApiMessageAndVersion(new RegisterBrokerRecord(). @@ -545,7 +533,7 @@ public class ClusterControlManagerTest { SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); Map supportedFeatures = new HashMap<>(); supportedFeatures.put(MetadataVersion.FEATURE_NAME, VersionRange.of( - MetadataVersion.IBP_3_1_IV0.featureLevel(), + MetadataVersion.MINIMUM_VERSION.featureLevel(), MetadataVersion.IBP_3_7_IV0.featureLevel())); supportedFeatures.put(TestFeatureVersion.FEATURE_NAME, VersionRange.of( TestFeatureVersion.TEST_0.featureLevel(), @@ -582,7 +570,7 @@ public class ClusterControlManagerTest { baseRequest.setFeatures(new BrokerRegistrationRequestData.FeatureCollection( Collections.singleton(new BrokerRegistrationRequestData.Feature(). setName(MetadataVersion.FEATURE_NAME). - setMinSupportedVersion(MetadataVersion.IBP_3_1_IV0.featureLevel()). + setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()). setMaxSupportedVersion(MetadataVersion.IBP_3_7_IV0.featureLevel())).iterator())), 123L, featureControl.finalizedFeatures(Long.MAX_VALUE), @@ -594,7 +582,7 @@ public class ClusterControlManagerTest { SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); Map supportedFeatures = new HashMap<>(); supportedFeatures.put(MetadataVersion.FEATURE_NAME, VersionRange.of( - MetadataVersion.IBP_3_1_IV0.featureLevel(), + MetadataVersion.MINIMUM_VERSION.featureLevel(), MetadataVersion.IBP_3_9_IV0.featureLevel())); supportedFeatures.put(KRaftVersion.FEATURE_NAME, VersionRange.of( KRaftVersion.KRAFT_VERSION_1.featureLevel(), @@ -680,10 +668,10 @@ public class ClusterControlManagerTest { setSnapshotRegistry(snapshotRegistry). setQuorumFeatures(new QuorumFeatures(0, Collections.singletonMap(MetadataVersion.FEATURE_NAME, VersionRange.of( - MetadataVersion.IBP_3_1_IV0.featureLevel(), - MetadataVersion.IBP_3_3_IV0.featureLevel())), + MetadataVersion.IBP_3_5_IV0.featureLevel(), + MetadataVersion.IBP_3_6_IV0.featureLevel())), Collections.singletonList(0))). - setMetadataVersion(MetadataVersion.IBP_3_3_IV0). + setMetadataVersion(MetadataVersion.IBP_3_5_IV0). build(); ClusterControlManager clusterControl = new ClusterControlManager.Builder(). setClusterId("fPZv1VBsRFmnlRvmGcOW9w"). @@ -694,21 +682,26 @@ public class ClusterControlManagerTest { build(); clusterControl.activate(); - assertEquals("Unable to register because the broker does not support finalized version 4 of " + - "metadata.version. The broker wants a version between 1 and 1, inclusive.", + assertEquals("Unable to register because the broker does not support finalized version 9 of " + + "metadata.version. The broker wants a version between 7 and 7, inclusive.", assertThrows(UnsupportedVersionException.class, () -> clusterControl.registerBroker( new BrokerRegistrationRequestData(). setClusterId("fPZv1VBsRFmnlRvmGcOW9w"). setBrokerId(0). setRack(null). + setFeatures(new BrokerRegistrationRequestData.FeatureCollection( + Collections.singleton(new BrokerRegistrationRequestData.Feature(). + setName(MetadataVersion.FEATURE_NAME). + setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()). + setMaxSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel())).iterator())). setIncarnationId(Uuid.fromString("0H4fUu1xQEKXFYwB1aBjhg")), 123L, featureControl.finalizedFeatures(Long.MAX_VALUE), false)).getMessage()); - assertEquals("Unable to register because the broker does not support finalized version 4 of " + - "metadata.version. The broker wants a version between 7 and 7, inclusive.", + assertEquals("Unable to register because the broker does not support finalized version 9 of " + + "metadata.version. The broker wants a version between 8 and 8, inclusive.", assertThrows(UnsupportedVersionException.class, () -> clusterControl.registerBroker( new BrokerRegistrationRequestData(). @@ -718,8 +711,8 @@ public class ClusterControlManagerTest { setFeatures(new BrokerRegistrationRequestData.FeatureCollection( Collections.singleton(new BrokerRegistrationRequestData.Feature(). setName(MetadataVersion.FEATURE_NAME). - setMinSupportedVersion(MetadataVersion.IBP_3_3_IV3.featureLevel()). - setMaxSupportedVersion(MetadataVersion.IBP_3_3_IV3.featureLevel())).iterator())). + setMinSupportedVersion(MetadataVersion.IBP_3_4_IV0.featureLevel()). + setMaxSupportedVersion(MetadataVersion.IBP_3_4_IV0.featureLevel())).iterator())). setIncarnationId(Uuid.fromString("0H4fUu1xQEKXFYwB1aBjhg")), 123L, featureControl.finalizedFeatures(Long.MAX_VALUE), @@ -727,19 +720,19 @@ public class ClusterControlManagerTest { } @Test - public void testRegisterControlWithOlderMetadataVersion() { + public void testRegisterControlWithUnsupportedMetadataVersion() { FeatureControlManager featureControl = new FeatureControlManager.Builder(). - setMetadataVersion(MetadataVersion.IBP_3_3_IV0). - build(); + setMetadataVersion(MetadataVersion.IBP_3_6_IV2). + build(); ClusterControlManager clusterControl = new ClusterControlManager.Builder(). - setClusterId("fPZv1VBsRFmnlRvmGcOW9w"). - setFeatureControlManager(featureControl). - setBrokerShutdownHandler((brokerId, isCleanShutdown, records) -> { }). - build(); + setClusterId("fPZv1VBsRFmnlRvmGcOW9w"). + setFeatureControlManager(featureControl). + setBrokerShutdownHandler((brokerId, isCleanShutdown, records) -> { }). + build(); clusterControl.activate(); assertEquals("The current MetadataVersion is too old to support controller registrations.", - assertThrows(UnsupportedVersionException.class, () -> clusterControl.registerController( - new ControllerRegistrationRequestData().setControllerId(1))).getMessage()); + assertThrows(UnsupportedVersionException.class, () -> clusterControl.registerController( + new ControllerRegistrationRequestData().setControllerId(1))).getMessage()); } @Test @@ -779,8 +772,14 @@ public class ClusterControlManagerTest { BrokerRegistrationRequestData data = new BrokerRegistrationRequestData().setBrokerId(brokerId) .setClusterId(clusterControl.clusterId()) .setIncarnationId(new Uuid(brokerId, brokerId)) + .setFeatures(new BrokerRegistrationRequestData.FeatureCollection( + Collections.singleton(new BrokerRegistrationRequestData.Feature(). + setName(MetadataVersion.FEATURE_NAME). + setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()). + setMaxSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel())).iterator())) .setLogDirs(dirs); - FinalizedControllerFeatures finalizedFeatures = new FinalizedControllerFeatures(Collections.emptyMap(), 456L); + FinalizedControllerFeatures finalizedFeatures = new FinalizedControllerFeatures( + Map.of(MetadataVersion.FEATURE_NAME, MetadataVersion.MINIMUM_VERSION.featureLevel()), 456L); ControllerResult result = clusterControl.registerBroker(data, 123L, finalizedFeatures, false); RecordTestUtils.replayAll(clusterControl, result.records()); } @@ -832,25 +831,37 @@ public class ClusterControlManagerTest { setBrokerShutdownHandler((brokerId, isCleanShutdown, records) -> { }). build(); clusterControl.activate(); + var finalizedFeatures = new FinalizedControllerFeatures(Map.of(MetadataVersion.FEATURE_NAME, MetadataVersion.MINIMUM_VERSION.featureLevel()), + 100L); RecordTestUtils.replayAll(clusterControl, clusterControl.registerBroker( new BrokerRegistrationRequestData(). setBrokerId(1). setClusterId(clusterControl.clusterId()). + setFeatures(new BrokerRegistrationRequestData.FeatureCollection( + Collections.singleton(new BrokerRegistrationRequestData.Feature(). + setName(MetadataVersion.FEATURE_NAME). + setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()). + setMaxSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel())).iterator())). setIncarnationId(Uuid.fromString("mISEfEFwQIuaD1gKCc5tzQ")). setLogDirs(Arrays.asList(Uuid.fromString("Vv1gzkM2QpuE-PPrIc6XEw"))), 100, - new FinalizedControllerFeatures(Collections.emptyMap(), 100L), + finalizedFeatures, false). records()); RecordTestUtils.replayAll(clusterControl, clusterControl.registerBroker( new BrokerRegistrationRequestData(). setBrokerId(1). setClusterId(clusterControl.clusterId()). + setFeatures(new BrokerRegistrationRequestData.FeatureCollection( + Collections.singleton(new BrokerRegistrationRequestData.Feature(). + setName(MetadataVersion.FEATURE_NAME). + setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()). + setMaxSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel())).iterator())). setIncarnationId(newIncarnationId ? Uuid.fromString("07OOcU7MQFeSmGAFPP2Zww") : Uuid.fromString("mISEfEFwQIuaD1gKCc5tzQ")). setLogDirs(Arrays.asList(Uuid.fromString("Vv1gzkM2QpuE-PPrIc6XEw"))), 111, - new FinalizedControllerFeatures(Collections.emptyMap(), 100L), + finalizedFeatures, false). records()); if (newIncarnationId) { @@ -879,14 +890,21 @@ public class ClusterControlManagerTest { }). build(); clusterControl.activate(); + var finalizedFeatures = new FinalizedControllerFeatures(Map.of(MetadataVersion.FEATURE_NAME, + MetadataVersion.MINIMUM_VERSION.featureLevel()), 100L); List records = clusterControl.registerBroker( new BrokerRegistrationRequestData(). setBrokerId(1). setClusterId(clusterControl.clusterId()). setIncarnationId(Uuid.fromString("mISEfEFwQIuaD1gKCc5tzQ")). + setFeatures(new BrokerRegistrationRequestData.FeatureCollection( + Collections.singleton(new BrokerRegistrationRequestData.Feature(). + setName(MetadataVersion.FEATURE_NAME). + setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()). + setMaxSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel())).iterator())). setLogDirs(Arrays.asList(Uuid.fromString("Vv1gzkM2QpuE-PPrIc6XEw"))), 100, - new FinalizedControllerFeatures(Collections.emptyMap(), 100L), + finalizedFeatures, true). records(); records.add(new ApiMessageAndVersion(new BrokerRegistrationChangeRecord(). @@ -901,9 +919,14 @@ public class ClusterControlManagerTest { setClusterId(clusterControl.clusterId()). setIncarnationId(Uuid.fromString("07OOcU7MQFeSmGAFPP2Zww")). setPreviousBrokerEpoch(isCleanShutdown ? 100 : 10). + setFeatures(new BrokerRegistrationRequestData.FeatureCollection( + Collections.singleton(new BrokerRegistrationRequestData.Feature(). + setName(MetadataVersion.FEATURE_NAME). + setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()). + setMaxSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel())).iterator())). setLogDirs(Arrays.asList(Uuid.fromString("Vv1gzkM2QpuE-PPrIc6XEw"))), 111, - new FinalizedControllerFeatures(Collections.emptyMap(), 100L), + finalizedFeatures, true).records(); RecordTestUtils.replayAll(clusterControl, records); assertEquals(Uuid.fromString("07OOcU7MQFeSmGAFPP2Zww"), diff --git a/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java index 87ae118ad84..2d0bc4c860e 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java @@ -28,6 +28,7 @@ import org.apache.kafka.metadata.VersionRange; import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.server.common.Feature; import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.common.MetadataVersionTestUtils; import org.apache.kafka.server.common.TestFeatureVersion; import org.apache.kafka.server.common.TransactionVersion; import org.apache.kafka.timeline.SnapshotRegistry; @@ -99,10 +100,10 @@ public class FeatureControlManagerTest { FeatureControlManager manager = new FeatureControlManager.Builder(). setQuorumFeatures(features(TestFeatureVersion.FEATURE_NAME, 0, 2)). setSnapshotRegistry(snapshotRegistry). - setMetadataVersion(MetadataVersion.IBP_3_3_IV0). + setMetadataVersion(MetadataVersion.MINIMUM_VERSION). build(); snapshotRegistry.idempotentCreateSnapshot(-1); - assertEquals(new FinalizedControllerFeatures(Collections.singletonMap("metadata.version", (short) 4), -1), + assertEquals(new FinalizedControllerFeatures(Collections.singletonMap("metadata.version", MetadataVersion.MINIMUM_VERSION.featureLevel()), -1), manager.finalizedFeatures(-1)); assertEquals(ControllerResult.of(emptyList(), new ApiError(Errors.INVALID_UPDATE_VERSION, "Invalid update version 3 for feature " + TestFeatureVersion.FEATURE_NAME + ". Local controller 0 only supports versions 0-2")), @@ -142,11 +143,12 @@ public class FeatureControlManagerTest { setLogContext(logContext). setQuorumFeatures(features("foo", 1, 2)). setSnapshotRegistry(snapshotRegistry). - setMetadataVersion(MetadataVersion.IBP_3_3_IV0). + setMetadataVersion(MetadataVersion.MINIMUM_VERSION). build(); manager.replay(record); snapshotRegistry.idempotentCreateSnapshot(123); - assertEquals(new FinalizedControllerFeatures(versionMap("metadata.version", 4, "foo", 2), 123), + assertEquals( + new FinalizedControllerFeatures(versionMap("metadata.version", MetadataVersion.MINIMUM_VERSION.featureLevel(), "foo", 2), 123), manager.finalizedFeatures(123)); } @@ -224,12 +226,12 @@ public class FeatureControlManagerTest { setLogContext(logContext). setQuorumFeatures(features(TestFeatureVersion.FEATURE_NAME, 0, 5, TransactionVersion.FEATURE_NAME, 0, 2)). setSnapshotRegistry(snapshotRegistry). - setMetadataVersion(MetadataVersion.IBP_3_3_IV0). + setMetadataVersion(MetadataVersion.MINIMUM_VERSION). build(); ControllerResult result = manager. updateFeatures(updateMap(TestFeatureVersion.FEATURE_NAME, 1, TransactionVersion.FEATURE_NAME, 2), Collections.emptyMap(), false); RecordTestUtils.replayAll(manager, result.records()); - assertEquals(MetadataVersion.IBP_3_3_IV0, manager.metadataVersion()); + assertEquals(MetadataVersion.MINIMUM_VERSION, manager.metadataVersion()); assertEquals(Optional.of((short) 1), manager.finalizedFeatures(Long.MAX_VALUE).get(TestFeatureVersion.FEATURE_NAME)); assertEquals(Optional.of((short) 2), manager.finalizedFeatures(Long.MAX_VALUE).get(TransactionVersion.FEATURE_NAME)); assertEquals(new HashSet<>(Arrays.asList( @@ -240,38 +242,27 @@ public class FeatureControlManagerTest { private static final FeatureControlManager.Builder TEST_MANAGER_BUILDER1 = new FeatureControlManager.Builder(). setQuorumFeatures(features(MetadataVersion.FEATURE_NAME, - MetadataVersion.IBP_3_3_IV0.featureLevel(), MetadataVersion.IBP_3_3_IV3.featureLevel())). - setMetadataVersion(MetadataVersion.IBP_3_3_IV2); + MetadataVersion.MINIMUM_VERSION.featureLevel(), MetadataVersion.IBP_3_6_IV0.featureLevel())). + setMetadataVersion(MetadataVersion.IBP_3_4_IV0); @Test public void testApplyMetadataVersionChangeRecord() { FeatureControlManager manager = TEST_MANAGER_BUILDER1.build(); + MetadataVersion initialMetadataVersion = manager.metadataVersion(); manager.replay(new FeatureLevelRecord(). setName(MetadataVersion.FEATURE_NAME). - setFeatureLevel(MetadataVersion.IBP_3_3_IV3.featureLevel())); - assertEquals(MetadataVersion.IBP_3_3_IV3, manager.metadataVersion()); - } - - @Test - public void testCannotDowngradeToVersionBeforeMinimumSupportedKraftVersion() { - FeatureControlManager manager = TEST_MANAGER_BUILDER1.build(); - assertEquals(ControllerResult.of(Collections.emptyList(), new ApiError(Errors.INVALID_UPDATE_VERSION, - "Invalid update version 3 for feature metadata.version. Local controller 0 only " + - "supports versions 4-7")), - manager.updateFeatures( - singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_2_IV0.featureLevel()), - singletonMap(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE), - true)); + setFeatureLevel((short) (initialMetadataVersion.featureLevel() + 1))); + assertEquals(MetadataVersion.IBP_3_5_IV0, manager.metadataVersion()); } @Test public void testCannotDowngradeToHigherVersion() { FeatureControlManager manager = TEST_MANAGER_BUILDER1.build(); assertEquals(ControllerResult.of(Collections.emptyList(), new ApiError(Errors.INVALID_UPDATE_VERSION, - "Invalid update version 7 for feature metadata.version. Can't downgrade to a " + + "Invalid update version 9 for feature metadata.version. Can't downgrade to a " + "newer version.")), manager.updateFeatures( - singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_3_IV3.featureLevel()), + singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_5_IV0.featureLevel()), singletonMap(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.SAFE_DOWNGRADE), true)); } @@ -280,23 +271,26 @@ public class FeatureControlManagerTest { public void testCannotUnsafeDowngradeToHigherVersion() { FeatureControlManager manager = TEST_MANAGER_BUILDER1.build(); assertEquals(ControllerResult.of(Collections.emptyList(), new ApiError(Errors.INVALID_UPDATE_VERSION, - "Invalid update version 7 for feature metadata.version. Can't downgrade to a " + + "Invalid update version 9 for feature metadata.version. Can't downgrade to a " + "newer version.")), manager.updateFeatures( - singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_3_IV3.featureLevel()), + singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_5_IV0.featureLevel()), singletonMap(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE), true)); } @Test public void testCannotUpgradeToLowerVersion() { - FeatureControlManager manager = TEST_MANAGER_BUILDER1.build(); + FeatureControlManager manager = new FeatureControlManager.Builder(). + setQuorumFeatures(features(MetadataVersion.FEATURE_NAME, + MetadataVersion.MINIMUM_VERSION.featureLevel(), MetadataVersion.IBP_3_6_IV0.featureLevel())). + setMetadataVersion(MetadataVersion.IBP_3_5_IV1).build(); assertEquals(ControllerResult.of(Collections.emptyList(), new ApiError(Errors.INVALID_UPDATE_VERSION, - "Invalid update version 4 for feature metadata.version. Can't downgrade the " + + "Invalid update version 9 for feature metadata.version. Can't downgrade the " + "version of this feature without setting the upgrade type to either safe or " + "unsafe downgrade.")), manager.updateFeatures( - singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_3_IV0.featureLevel()), + singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_5_IV0.featureLevel()), singletonMap(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UPGRADE), true)); } @@ -306,7 +300,7 @@ public class FeatureControlManagerTest { FeatureControlManager manager = TEST_MANAGER_BUILDER1.build(); assertEquals(ControllerResult.of(Collections.emptyList(), ApiError.NONE), manager.updateFeatures( - singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_3_IV3.featureLevel()), + singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_5_IV0.featureLevel()), singletonMap(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UPGRADE), true)); } @@ -315,10 +309,10 @@ public class FeatureControlManagerTest { public void testCannotUseSafeDowngradeIfMetadataChanged() { FeatureControlManager manager = TEST_MANAGER_BUILDER1.build(); assertEquals(ControllerResult.of(Collections.emptyList(), new ApiError(Errors.INVALID_UPDATE_VERSION, - "Invalid metadata.version 4. Refusing to perform the requested downgrade because " + + "Invalid metadata.version 7. Refusing to perform the requested downgrade because " + "it might delete metadata information.")), manager.updateFeatures( - singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_3_IV0.featureLevel()), + singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_3_IV3.featureLevel()), singletonMap(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.SAFE_DOWNGRADE), true)); } @@ -327,9 +321,9 @@ public class FeatureControlManagerTest { public void testUnsafeDowngradeIsTemporarilyDisabled() { FeatureControlManager manager = TEST_MANAGER_BUILDER1.build(); assertEquals(ControllerResult.of(Collections.emptyList(), new ApiError(Errors.INVALID_UPDATE_VERSION, - "Invalid metadata.version 4. Unsafe metadata downgrade is not supported in this version.")), + "Invalid metadata.version 7. Unsafe metadata downgrade is not supported in this version.")), manager.updateFeatures( - singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_3_IV0.featureLevel()), + singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_3_IV3.featureLevel()), singletonMap(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE), true)); } @@ -340,7 +334,7 @@ public class FeatureControlManagerTest { FeatureControlManager manager = TEST_MANAGER_BUILDER1.build(); assertEquals(ControllerResult.of(Collections.emptyList(), ApiError.NONE), manager.updateFeatures( - singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_3_IV0.featureLevel()), + singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_3_IV3.featureLevel()), singletonMap(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE), true)); } @@ -349,28 +343,27 @@ public class FeatureControlManagerTest { public void testCanUseSafeDowngradeIfMetadataDidNotChange() { FeatureControlManager manager = new FeatureControlManager.Builder(). setQuorumFeatures(features(MetadataVersion.FEATURE_NAME, - MetadataVersion.IBP_3_0_IV1.featureLevel(), MetadataVersion.IBP_3_3_IV1.featureLevel())). - setMetadataVersion(MetadataVersion.IBP_3_1_IV0). - setMinimumBootstrapVersion(MetadataVersion.IBP_3_0_IV1). + MetadataVersion.MINIMUM_VERSION.featureLevel(), MetadataVersion.IBP_3_6_IV0.featureLevel())). + setMetadataVersion(MetadataVersion.IBP_3_5_IV0). build(); assertEquals(ControllerResult.of(Collections.emptyList(), ApiError.NONE), manager.updateFeatures( - singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_0_IV1.featureLevel()), + singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_4_IV0.featureLevel()), singletonMap(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.SAFE_DOWNGRADE), true)); } @Test - public void testCannotDowngradeBefore3_3_IV0() { + public void testCannotDowngradeBeforeMinimumKraftVersion() { FeatureControlManager manager = new FeatureControlManager.Builder(). setQuorumFeatures(features(MetadataVersion.FEATURE_NAME, - MetadataVersion.IBP_3_0_IV1.featureLevel(), MetadataVersion.IBP_3_3_IV3.featureLevel())). - setMetadataVersion(MetadataVersion.IBP_3_3_IV0). + MetadataVersion.MINIMUM_VERSION.featureLevel(), MetadataVersion.latestTesting().featureLevel())). + setMetadataVersion(MetadataVersion.MINIMUM_VERSION). build(); assertEquals(ControllerResult.of(Collections.emptyList(), new ApiError(Errors.INVALID_UPDATE_VERSION, - "Invalid metadata.version 3. Unable to set a metadata.version less than 3.3-IV0")), + "Invalid update version 6 for feature metadata.version. Local controller 0 only supports versions 7-26")), manager.updateFeatures( - singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_2_IV0.featureLevel()), + singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersionTestUtils.IBP_3_3_IV2_FEATURE_LEVEL), singletonMap(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE), true)); } @@ -379,7 +372,7 @@ public class FeatureControlManagerTest { public void testCreateFeatureLevelRecords() { Map localSupportedFeatures = new HashMap<>(); localSupportedFeatures.put(MetadataVersion.FEATURE_NAME, VersionRange.of( - MetadataVersion.IBP_3_0_IV1.featureLevel(), MetadataVersion.latestTesting().featureLevel())); + MetadataVersion.MINIMUM_VERSION.featureLevel(), MetadataVersion.latestTesting().featureLevel())); localSupportedFeatures.put(Feature.TEST_VERSION.featureName(), VersionRange.of(0, 2)); FeatureControlManager manager = new FeatureControlManager.Builder(). setQuorumFeatures(new QuorumFeatures(0, localSupportedFeatures, emptyList())). 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 18bf8c80afd..bea3acc1c00 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java @@ -598,9 +598,8 @@ public class PartitionChangeBuilderTest { ); } - @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void testChangeInLeadershipDoesNotChangeRecoveryState(boolean isLeaderRecoverySupported) { + @Test + public void testChangeInLeadershipDoesNotChangeRecoveryState() { final byte noChange = (byte) -1; int leaderId = 1; LeaderRecoveryState recoveryState = LeaderRecoveryState.RECOVERING; @@ -618,7 +617,7 @@ public class PartitionChangeBuilderTest { setPartitionEpoch(200). build(); - MetadataVersion metadataVersion = leaderRecoveryMetadataVersion(isLeaderRecoverySupported); + MetadataVersion metadataVersion = MetadataVersion.MINIMUM_VERSION; // Change the partition so that there is no leader PartitionChangeBuilder offlineBuilder = new PartitionChangeBuilder( @@ -667,10 +666,8 @@ public class PartitionChangeBuilderTest { assertEquals(recoveryState, registration.leaderRecoveryState); } - @ParameterizedTest - @ValueSource(booleans = {true, false}) - void testUncleanSetsLeaderRecoveringState(boolean isLeaderRecoverySupported) { - final byte noChange = (byte) -1; + @Test + void testUncleanSetsLeaderRecoveringState() { int leaderId = 1; PartitionRegistration registration = new PartitionRegistration.Builder(). setReplicas(new int[] {leaderId, leaderId + 1, leaderId + 2}). @@ -686,7 +683,7 @@ public class PartitionChangeBuilderTest { setPartitionEpoch(200). build(); - MetadataVersion metadataVersion = leaderRecoveryMetadataVersion(isLeaderRecoverySupported); + MetadataVersion metadataVersion = MetadataVersion.MINIMUM_VERSION; // Change the partition using unclean leader election PartitionChangeBuilder onlineBuilder = new PartitionChangeBuilder( @@ -703,26 +700,16 @@ public class PartitionChangeBuilderTest { .get() .message(); - byte expectedRecoveryChange = noChange; - if (isLeaderRecoverySupported) { - expectedRecoveryChange = LeaderRecoveryState.RECOVERING.value(); - } - - assertEquals(expectedRecoveryChange, changeRecord.leaderRecoveryState()); + assertEquals(LeaderRecoveryState.RECOVERING.value(), changeRecord.leaderRecoveryState()); assertEquals(leaderId, changeRecord.leader()); assertEquals(1, changeRecord.isr().size()); assertEquals(leaderId, changeRecord.isr().get(0)); registration = registration.merge(changeRecord); - LeaderRecoveryState expectedRecovery = LeaderRecoveryState.RECOVERED; - if (isLeaderRecoverySupported) { - expectedRecovery = LeaderRecoveryState.RECOVERING; - } - assertEquals(leaderId, registration.leader); assertEquals(leaderId, registration.isr[0]); - assertEquals(expectedRecovery, registration.leaderRecoveryState); + assertEquals(LeaderRecoveryState.RECOVERING, registration.leaderRecoveryState); } @Test @@ -770,7 +757,7 @@ public class PartitionChangeBuilderTest { topicId, 0, isValidLeader, - leaderRecoveryMetadataVersion(false), + MetadataVersion.MINIMUM_VERSION, 2 ); @@ -790,14 +777,6 @@ public class PartitionChangeBuilderTest { build()); } - private MetadataVersion leaderRecoveryMetadataVersion(boolean isSupported) { - if (isSupported) { - return MetadataVersion.IBP_3_2_IV0; - } else { - return MetadataVersion.IBP_3_1_IV0; - } - } - @ParameterizedTest @MethodSource("partitionChangeRecordVersions") public void testEligibleLeaderReplicas_IsrShrinkBelowMinISR(short version) { diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerIntegrationTestUtils.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerIntegrationTestUtils.java index 2cca9ef7cc6..6057df09698 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerIntegrationTestUtils.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerIntegrationTestUtils.java @@ -55,7 +55,7 @@ public class QuorumControllerIntegrationTestUtils { private static final Logger log = LoggerFactory.getLogger(QuorumControllerIntegrationTestUtils.class); BrokerRegistrationRequestData.FeatureCollection brokerFeatures() { - return brokerFeatures(MetadataVersion.MINIMUM_KRAFT_VERSION, MetadataVersion.latestTesting()); + return brokerFeatures(MetadataVersion.MINIMUM_VERSION, MetadataVersion.latestTesting()); } /** @@ -121,7 +121,7 @@ public class QuorumControllerIntegrationTestUtils { .setBrokerId(brokerId) .setRack(null) .setClusterId(controller.clusterId()) - .setFeatures(brokerFeaturesPlusFeatureVersions(MetadataVersion.IBP_3_0_IV1, MetadataVersion.latestTesting(), + .setFeatures(brokerFeaturesPlusFeatureVersions(MetadataVersion.MINIMUM_VERSION, MetadataVersion.latestTesting(), Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME, EligibleLeaderReplicasVersion.ELRV_1.featureLevel()))) .setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwB" + brokerId)) .setLogDirs(Collections.singletonList( 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 31cde1eda18..f50ba5a9110 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java @@ -60,7 +60,6 @@ import org.apache.kafka.common.metadata.RegisterBrokerRecord.BrokerEndpoint; import org.apache.kafka.common.metadata.RegisterBrokerRecord.BrokerEndpointCollection; import org.apache.kafka.common.metadata.RegisterControllerRecord; import org.apache.kafka.common.metadata.TopicRecord; -import org.apache.kafka.common.metadata.UnfenceBrokerRecord; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.AlterPartitionRequest; @@ -191,7 +190,7 @@ public class QuorumControllerTest { ) { controlEnv.activeController().registerBroker(ANONYMOUS_CONTEXT, new BrokerRegistrationRequestData(). - setFeatures(brokerFeaturesPlusFeatureVersions(MetadataVersion.IBP_3_0_IV1, MetadataVersion.latestTesting(), + setFeatures(brokerFeaturesPlusFeatureVersions(MetadataVersion.MINIMUM_VERSION, MetadataVersion.latestTesting(), Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME, EligibleLeaderReplicasVersion.ELRV_1.featureLevel()))). setBrokerId(0). setLogDirs(Collections.singletonList(Uuid.fromString("iiaQjkRPQcuMULNII0MUeA"))). @@ -233,7 +232,7 @@ public class QuorumControllerTest { ) { controlEnv.activeController().registerBroker(ANONYMOUS_CONTEXT, new BrokerRegistrationRequestData(). - setFeatures(brokerFeaturesPlusFeatureVersions(MetadataVersion.IBP_3_0_IV1, MetadataVersion.latestTesting(), + setFeatures(brokerFeaturesPlusFeatureVersions(MetadataVersion.MINIMUM_VERSION, MetadataVersion.latestTesting(), Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME, EligibleLeaderReplicasVersion.ELRV_1.featureLevel()))). setBrokerId(0). setLogDirs(Collections.singletonList(Uuid.fromString("sTbzRAMnTpahIyIPNjiLhw"))). @@ -289,7 +288,7 @@ public class QuorumControllerTest { new BrokerRegistrationRequestData(). setBrokerId(brokerId). setClusterId(active.clusterId()). - setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.latestTesting())). + setFeatures(brokerFeatures(MetadataVersion.MINIMUM_VERSION, MetadataVersion.latestTesting())). setIncarnationId(Uuid.randomUuid()). setListeners(listeners)); brokerEpochs.put(brokerId, reply.get().epoch()); @@ -398,7 +397,7 @@ public class QuorumControllerTest { QuorumController active = controlEnv.activeController(); Map brokerEpochs = new HashMap<>(); BrokerRegistrationRequestData.FeatureCollection features = - brokerFeaturesPlusFeatureVersions(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_4_0_IV1, + brokerFeaturesPlusFeatureVersions(MetadataVersion.MINIMUM_VERSION, MetadataVersion.IBP_4_0_IV1, Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME, EligibleLeaderReplicasVersion.ELRV_1.featureLevel())); for (Integer brokerId : allBrokers) { CompletableFuture reply = active.registerBroker( @@ -530,7 +529,7 @@ public class QuorumControllerTest { QuorumController active = controlEnv.activeController(); Map brokerEpochs = new HashMap<>(); BrokerRegistrationRequestData.FeatureCollection features = - brokerFeaturesPlusFeatureVersions(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_4_0_IV0, + brokerFeaturesPlusFeatureVersions(MetadataVersion.MINIMUM_VERSION, MetadataVersion.IBP_4_0_IV0, Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME, EligibleLeaderReplicasVersion.ELRV_0.featureLevel())); for (Integer brokerId : allBrokers) { CompletableFuture reply = active.registerBroker( @@ -629,7 +628,7 @@ public class QuorumControllerTest { new BrokerRegistrationRequestData(). setBrokerId(brokerId). setClusterId(active.clusterId()). - setFeatures(brokerFeaturesPlusFeatureVersions(MetadataVersion.IBP_3_0_IV1, MetadataVersion.latestTesting(), + setFeatures(brokerFeaturesPlusFeatureVersions(MetadataVersion.MINIMUM_VERSION, MetadataVersion.latestTesting(), Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME, EligibleLeaderReplicasVersion.ELRV_1.featureLevel()))). setIncarnationId(Uuid.randomUuid()). setLogDirs(Collections.singletonList(Uuid.randomUuid())). @@ -760,7 +759,7 @@ public class QuorumControllerTest { new BrokerRegistrationRequestData(). setBrokerId(brokerId). setClusterId(active.clusterId()). - setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_3_7_IV0)). + setFeatures(brokerFeatures(MetadataVersion.MINIMUM_VERSION, MetadataVersion.IBP_3_7_IV0)). setIncarnationId(Uuid.randomUuid()). setListeners(listeners)); brokerEpochs.put(brokerId, reply.get().epoch()); @@ -818,7 +817,7 @@ public class QuorumControllerTest { new BrokerRegistrationRequestData(). setBrokerId(brokerId). setClusterId(active.clusterId()). - setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_3_7_IV0)). + setFeatures(brokerFeatures(MetadataVersion.MINIMUM_VERSION, MetadataVersion.IBP_3_7_IV0)). setIncarnationId(Uuid.randomUuid()). setListeners(listeners)); brokerEpochs.put(brokerId, reply.get().epoch()); @@ -830,7 +829,8 @@ public class QuorumControllerTest { // Let the unfenced broker, 3, join the ISR partition 2 Set imbalancedPartitions = new HashSet<>(active.replicationControl().imbalancedPartitions()); assertEquals(1, imbalancedPartitions.size()); - int imbalancedPartitionId = imbalancedPartitions.iterator().next().partitionId(); + TopicIdPartition impalancedTp = imbalancedPartitions.iterator().next(); + int imbalancedPartitionId = impalancedTp.partitionId(); PartitionRegistration partitionRegistration = active.replicationControl().getPartition(topicIdFoo, imbalancedPartitionId); AlterPartitionRequestData.PartitionData partitionData = new AlterPartitionRequestData.PartitionData() .setPartitionIndex(imbalancedPartitionId) @@ -839,7 +839,7 @@ public class QuorumControllerTest { .setNewIsrWithEpochs(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(Arrays.asList(1, 2, 3))); AlterPartitionRequestData.TopicData topicData = new AlterPartitionRequestData.TopicData() - .setTopicName("foo"); + .setTopicId(impalancedTp.topicId()); topicData.partitions().add(partitionData); AlterPartitionRequestData alterPartitionRequest = new AlterPartitionRequestData() @@ -848,7 +848,7 @@ public class QuorumControllerTest { alterPartitionRequest.topics().add(topicData); active.alterPartition(ANONYMOUS_CONTEXT, new AlterPartitionRequest - .Builder(alterPartitionRequest).build((short) 0).data()).get(); + .Builder(alterPartitionRequest).build(ApiKeys.ALTER_PARTITION.oldestVersion()).data()).get(); AtomicLong lastHeartbeatMs = new AtomicLong(getMonotonicMs(active.time())); sendBrokerHeartbeatToUnfenceBrokers(active, allBrokers, brokerEpochs); @@ -938,7 +938,7 @@ public class QuorumControllerTest { BrokerRegistrationRequestData.FeatureCollection brokerFeatures = new BrokerRegistrationRequestData.FeatureCollection(); brokerFeatures.add(new BrokerRegistrationRequestData.Feature() .setName(MetadataVersion.FEATURE_NAME) - .setMinSupportedVersion(MetadataVersion.IBP_3_0_IV1.featureLevel()) + .setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()) .setMaxSupportedVersion(MetadataVersion.latestTesting().featureLevel())); // broker registration requests do not include initial versions of features if (brokerMaxSupportedKraftVersion != 0) { @@ -991,7 +991,7 @@ public class QuorumControllerTest { setBrokerId(0). setClusterId(active.clusterId()). setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwBA")). - setFeatures(brokerFeaturesPlusFeatureVersions(MetadataVersion.IBP_3_0_IV1, MetadataVersion.latestTesting(), + setFeatures(brokerFeaturesPlusFeatureVersions(MetadataVersion.MINIMUM_VERSION, MetadataVersion.latestTesting(), Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME, EligibleLeaderReplicasVersion.ELRV_1.featureLevel()))). setLogDirs(Collections.singletonList(Uuid.fromString("vBpaRsZVSaGsQT53wtYGtg"))). setListeners(listeners)); @@ -1082,7 +1082,7 @@ public class QuorumControllerTest { singletonList( new ControllerRegistrationRequestData.Feature(). setName(MetadataVersion.FEATURE_NAME). - setMinSupportedVersion(MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel()). + setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()). setMaxSupportedVersion(MetadataVersion.IBP_3_7_IV0.featureLevel()) ).iterator() ))).get(); @@ -1093,7 +1093,7 @@ public class QuorumControllerTest { setBrokerId(i). setRack(null). setClusterId(active.clusterId()). - setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_3_7_IV0)). + setFeatures(brokerFeatures(MetadataVersion.MINIMUM_VERSION, MetadataVersion.IBP_3_7_IV0)). setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwB" + i)). setListeners(new ListenerCollection(singletonList(new Listener(). setName("PLAINTEXT").setHost("localhost"). @@ -1152,7 +1152,7 @@ public class QuorumControllerTest { singletonList( new RegisterControllerRecord.ControllerFeature(). setName(MetadataVersion.FEATURE_NAME). - setMinSupportedVersion(MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel()). + setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()). setMaxSupportedVersion(MetadataVersion.IBP_3_7_IV0.featureLevel())).iterator())), (short) 0), new ApiMessageAndVersion(new RegisterControllerRecord(). @@ -1169,7 +1169,7 @@ public class QuorumControllerTest { singletonList( new RegisterControllerRecord.ControllerFeature(). setName(MetadataVersion.FEATURE_NAME). - setMinSupportedVersion(MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel()). + setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()). setMaxSupportedVersion(MetadataVersion.IBP_3_7_IV0.featureLevel())).iterator())), (short) 0), new ApiMessageAndVersion(new RegisterControllerRecord(). @@ -1186,7 +1186,7 @@ public class QuorumControllerTest { singletonList( new RegisterControllerRecord.ControllerFeature(). setName(MetadataVersion.FEATURE_NAME). - setMinSupportedVersion(MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel()). + setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()). setMaxSupportedVersion(MetadataVersion.IBP_3_7_IV0.featureLevel())).iterator())), (short) 0), new ApiMessageAndVersion(new RegisterBrokerRecord(). @@ -1195,7 +1195,7 @@ public class QuorumControllerTest { setEndPoints(new BrokerEndpointCollection( singletonList(new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost"). setPort(9092).setSecurityProtocol((short) 0)).iterator())). - setFeatures(registrationFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_3_7_IV0)). + setFeatures(registrationFeatures(MetadataVersion.MINIMUM_VERSION, MetadataVersion.IBP_3_7_IV0)). setRack(null). setFenced(true), (short) 2), new ApiMessageAndVersion(new RegisterBrokerRecord(). @@ -1204,7 +1204,7 @@ public class QuorumControllerTest { setEndPoints(new BrokerEndpointCollection(singletonList( new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost"). setPort(9093).setSecurityProtocol((short) 0)).iterator())). - setFeatures(registrationFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_3_7_IV0)). + setFeatures(registrationFeatures(MetadataVersion.MINIMUM_VERSION, MetadataVersion.IBP_3_7_IV0)). setRack(null). setFenced(true), (short) 2), new ApiMessageAndVersion(new RegisterBrokerRecord(). @@ -1213,7 +1213,7 @@ public class QuorumControllerTest { setEndPoints(new BrokerEndpointCollection( singletonList(new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost"). setPort(9094).setSecurityProtocol((short) 0)).iterator())). - setFeatures(registrationFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_3_7_IV0)). + setFeatures(registrationFeatures(MetadataVersion.MINIMUM_VERSION, MetadataVersion.IBP_3_7_IV0)). setRack(null). setFenced(true), (short) 2), new ApiMessageAndVersion(new RegisterBrokerRecord(). @@ -1222,7 +1222,7 @@ public class QuorumControllerTest { setEndPoints(new BrokerEndpointCollection(singletonList( new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost"). setPort(9095).setSecurityProtocol((short) 0)).iterator())). - setFeatures(registrationFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_3_7_IV0)). + setFeatures(registrationFeatures(MetadataVersion.MINIMUM_VERSION, MetadataVersion.IBP_3_7_IV0)). setRack(null). setFenced(true), (short) 2), new ApiMessageAndVersion(new BrokerRegistrationChangeRecord(). @@ -1493,28 +1493,11 @@ public class QuorumControllerTest { } } - private static final List PRE_PRODUCTION_RECORDS = - List.of( - new ApiMessageAndVersion(new RegisterBrokerRecord(). - setBrokerEpoch(42). - setBrokerId(123). - setIncarnationId(Uuid.fromString("v78Gbc6sQXK0y5qqRxiryw")). - setRack(null), - (short) 0), - new ApiMessageAndVersion(new UnfenceBrokerRecord(). - setEpoch(42). - setId(123), - (short) 0), - new ApiMessageAndVersion(new TopicRecord(). - setName("bar"). - setTopicId(Uuid.fromString("cxBT72dK4si8Ied1iP4wBA")), - (short) 0)); - private static final BootstrapMetadata COMPLEX_BOOTSTRAP = BootstrapMetadata.fromRecords( Arrays.asList( new ApiMessageAndVersion(new FeatureLevelRecord(). setName(MetadataVersion.FEATURE_NAME). - setFeatureLevel(MetadataVersion.IBP_3_3_IV1.featureLevel()), + setFeatureLevel(MetadataVersion.MINIMUM_VERSION.featureLevel()), (short) 0), new ApiMessageAndVersion(new ConfigRecord(). setResourceType(BROKER.id()). @@ -1524,31 +1507,6 @@ public class QuorumControllerTest { (short) 0)), "test bootstrap"); - @Test - public void testUpgradeFromPreProductionVersion() throws Exception { - try ( - InitialSnapshot initialSnapshot = new InitialSnapshot(PRE_PRODUCTION_RECORDS); - LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(3). - setSnapshotReader(FileRawSnapshotReader.open( - initialSnapshot.tempDir.toPath(), new OffsetAndEpoch(0, 0))). - build(); - QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv). - setBootstrapMetadata(COMPLEX_BOOTSTRAP). - build() - ) { - QuorumController active = controlEnv.activeController(); - TestUtils.waitForCondition(() -> - active.featureControl().metadataVersion().equals(MetadataVersion.IBP_3_0_IV1), - "Failed to get a metadata.version of " + MetadataVersion.IBP_3_0_IV1); - // The ConfigRecord in our bootstrap should not have been applied, since there - // were already records present. - assertEquals(Collections.emptyMap(), active.configurationControl(). - getConfigs(new ConfigResource(BROKER, ""))); - - testToImages(logEnv.allRecords()); - } - } - @Test public void testInsertBootstrapRecordsToEmptyLog() throws Exception { try ( @@ -1566,7 +1524,7 @@ public class QuorumControllerTest { TestUtils.waitForCondition(() -> { FinalizedControllerFeatures features = active.finalizedFeatures(ctx).get(); Optional metadataVersionOpt = features.get(MetadataVersion.FEATURE_NAME); - return Optional.of(MetadataVersion.IBP_3_3_IV1.featureLevel()).equals(metadataVersionOpt); + return Optional.of(MetadataVersion.MINIMUM_VERSION.featureLevel()).equals(metadataVersionOpt); }, "Failed to see expected metadata.version from bootstrap metadata"); TestUtils.waitForCondition(() -> { @@ -1651,8 +1609,8 @@ public class QuorumControllerTest { public void testActivationRecords33() { FeatureControlManager featureControl; - featureControl = getActivationRecords(MetadataVersion.IBP_3_3_IV0); - assertEquals(MetadataVersion.IBP_3_3_IV0, featureControl.metadataVersion()); + featureControl = getActivationRecords(MetadataVersion.IBP_3_3_IV3); + assertEquals(MetadataVersion.IBP_3_3_IV3, featureControl.metadataVersion()); } @Test diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumFeaturesTest.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumFeaturesTest.java index cfe320bb38d..24e83a34cc5 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumFeaturesTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumFeaturesTest.java @@ -49,7 +49,7 @@ public class QuorumFeaturesTest { public void testDefaultFeatureMap() { Map expectedFeatures = new HashMap<>(1); expectedFeatures.put(MetadataVersion.FEATURE_NAME, VersionRange.of( - MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel(), + MetadataVersion.MINIMUM_VERSION.featureLevel(), MetadataVersion.LATEST_PRODUCTION.featureLevel())); for (Feature feature : Feature.PRODUCTION_FEATURES) { short maxVersion = feature.latestProduction(); @@ -67,7 +67,7 @@ public class QuorumFeaturesTest { public void testDefaultFeatureMapWithUnstable() { Map expectedFeatures = new HashMap<>(1); expectedFeatures.put(MetadataVersion.FEATURE_NAME, VersionRange.of( - MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel(), + MetadataVersion.MINIMUM_VERSION.featureLevel(), MetadataVersion.latestTesting().featureLevel())); for (Feature feature : Feature.PRODUCTION_FEATURES) { short maxVersion = feature.defaultLevel(MetadataVersion.latestTesting()); diff --git a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java index a8b001c6389..ef58e2bc7f5 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java @@ -105,7 +105,6 @@ 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; @@ -135,7 +134,6 @@ import static org.apache.kafka.common.config.TopicConfig.SEGMENT_BYTES_CONFIG; import static org.apache.kafka.common.metadata.MetadataRecordType.CLEAR_ELR_RECORD; import static org.apache.kafka.common.protocol.Errors.ELECTION_NOT_NEEDED; import static org.apache.kafka.common.protocol.Errors.ELIGIBLE_LEADERS_NOT_AVAILABLE; -import static org.apache.kafka.common.protocol.Errors.FENCED_LEADER_EPOCH; import static org.apache.kafka.common.protocol.Errors.INELIGIBLE_REPLICA; import static org.apache.kafka.common.protocol.Errors.INVALID_PARTITIONS; import static org.apache.kafka.common.protocol.Errors.INVALID_REPLICATION_FACTOR; @@ -146,7 +144,6 @@ import static org.apache.kafka.common.protocol.Errors.NONE; import static org.apache.kafka.common.protocol.Errors.NOT_CONTROLLER; import static org.apache.kafka.common.protocol.Errors.NOT_LEADER_OR_FOLLOWER; import static org.apache.kafka.common.protocol.Errors.NO_REASSIGNMENT_IN_PROGRESS; -import static org.apache.kafka.common.protocol.Errors.OPERATION_NOT_ATTEMPTED; import static org.apache.kafka.common.protocol.Errors.POLICY_VIOLATION; import static org.apache.kafka.common.protocol.Errors.PREFERRED_LEADER_NOT_AVAILABLE; import static org.apache.kafka.common.protocol.Errors.THROTTLING_QUOTA_EXCEEDED; @@ -443,9 +440,7 @@ public class ReplicationControlManagerTest { .setLeaderRecoveryState(leaderRecoveryState.value()) .setNewIsrWithEpochs(isrWithEpoch); - String topicName = replicationControl.getTopic(topicIdPartition.topicId()).name(); TopicData topicData = new TopicData() - .setTopicName(topicName) .setTopicId(topicIdPartition.topicId()) .setPartitions(singletonList(partitionData)); @@ -1252,15 +1247,13 @@ public class ReplicationControlManagerTest { ctx.registerBrokers(0, 1, 2); ctx.unfenceBrokers(0, 1, 2); - String topicName = "foo"; Uuid topicId = Uuid.randomUuid(); AlterPartitionRequestData request = new AlterPartitionRequestData() .setBrokerId(0) .setBrokerEpoch(100) .setTopics(singletonList(new TopicData() - .setTopicName(version <= 1 ? topicName : "") - .setTopicId(version > 1 ? topicId : Uuid.ZERO_UUID) + .setTopicId(topicId) .setPartitions(singletonList(new PartitionData() .setPartitionIndex(0))))); @@ -1270,11 +1263,10 @@ public class ReplicationControlManagerTest { ControllerResult result = replicationControl.alterPartition(requestContext, request); - Errors expectedError = version > 1 ? UNKNOWN_TOPIC_ID : UNKNOWN_TOPIC_OR_PARTITION; + Errors expectedError = UNKNOWN_TOPIC_ID; AlterPartitionResponseData expectedResponse = new AlterPartitionResponseData() .setTopics(singletonList(new AlterPartitionResponseData.TopicData() - .setTopicName(version <= 1 ? topicName : "") - .setTopicId(version > 1 ? topicId : Uuid.ZERO_UUID) + .setTopicId(topicId) .setPartitions(singletonList(new AlterPartitionResponseData.PartitionData() .setPartitionIndex(0) .setErrorCode(expectedError.code()))))); @@ -1908,8 +1900,7 @@ public class ReplicationControlManagerTest { setBrokerId(3). setBrokerEpoch(103). setTopics(singletonList(new TopicData(). - setTopicName(version <= 1 ? "foo" : ""). - setTopicId(version > 1 ? fooId : Uuid.ZERO_UUID). + setTopicId(fooId). setPartitions(singletonList(new PartitionData(). setPartitionIndex(1). setPartitionEpoch(1). @@ -1918,11 +1909,10 @@ public class ReplicationControlManagerTest { ControllerResult alterPartitionResult = replication.alterPartition( requestContext, new AlterPartitionRequest.Builder(alterPartitionRequestData).build(version).data()); - Errors expectedError = version > 1 ? NEW_LEADER_ELECTED : FENCED_LEADER_EPOCH; + Errors expectedError = NEW_LEADER_ELECTED; assertEquals(new AlterPartitionResponseData().setTopics(singletonList( new AlterPartitionResponseData.TopicData(). - setTopicName(version <= 1 ? "foo" : ""). - setTopicId(version > 1 ? fooId : Uuid.ZERO_UUID). + setTopicId(fooId). setPartitions(singletonList( new AlterPartitionResponseData.PartitionData(). setPartitionIndex(1). @@ -1969,8 +1959,7 @@ public class ReplicationControlManagerTest { .setBrokerId(1) .setBrokerEpoch(101) .setTopics(singletonList(new TopicData() - .setTopicName(version <= 1 ? "foo" : "") - .setTopicId(version > 1 ? fooId : Uuid.ZERO_UUID) + .setTopicId(fooId) .setPartitions(singletonList(new PartitionData() .setPartitionIndex(0) .setPartitionEpoch(1) @@ -1983,12 +1972,11 @@ public class ReplicationControlManagerTest { ControllerResult alterPartitionResult = replication.alterPartition(requestContext, new AlterPartitionRequest.Builder(alterIsrRequest).build(version).data()); - Errors expectedError = version <= 1 ? OPERATION_NOT_ATTEMPTED : INELIGIBLE_REPLICA; + Errors expectedError = INELIGIBLE_REPLICA; assertEquals( new AlterPartitionResponseData() .setTopics(singletonList(new AlterPartitionResponseData.TopicData() - .setTopicName(version <= 1 ? "foo" : "") - .setTopicId(version > 1 ? fooId : Uuid.ZERO_UUID) + .setTopicId(fooId) .setPartitions(singletonList(new AlterPartitionResponseData.PartitionData() .setPartitionIndex(0) .setErrorCode(expectedError.code()))))), @@ -2003,8 +1991,7 @@ public class ReplicationControlManagerTest { assertEquals( new AlterPartitionResponseData() .setTopics(singletonList(new AlterPartitionResponseData.TopicData() - .setTopicName(version <= 1 ? "foo" : "") - .setTopicId(version > 1 ? fooId : Uuid.ZERO_UUID) + .setTopicId(fooId) .setPartitions(singletonList(new AlterPartitionResponseData.PartitionData() .setPartitionIndex(0) .setLeaderId(1) @@ -2033,8 +2020,7 @@ public class ReplicationControlManagerTest { setBrokerId(1). setBrokerEpoch(101). setTopics(singletonList(new TopicData(). - setTopicName(version <= 1 ? "foo" : ""). - setTopicId(version > 1 ? fooId : Uuid.ZERO_UUID). + setTopicId(fooId). setPartitions(singletonList(new PartitionData(). setPartitionIndex(0). setPartitionEpoch(1). @@ -2073,7 +2059,6 @@ public class ReplicationControlManagerTest { assertEquals( new AlterPartitionResponseData(). setTopics(singletonList(new AlterPartitionResponseData.TopicData(). - setTopicName(""). setTopicId(fooId). setPartitions(singletonList(new AlterPartitionResponseData.PartitionData(). setPartitionIndex(0). @@ -2119,8 +2104,7 @@ public class ReplicationControlManagerTest { .setBrokerId(1) .setBrokerEpoch(101) .setTopics(singletonList(new TopicData() - .setTopicName(version <= 1 ? "foo" : "") - .setTopicId(version > 1 ? fooId : Uuid.ZERO_UUID) + .setTopicId(fooId) .setPartitions(singletonList(new PartitionData() .setPartitionIndex(0) .setPartitionEpoch(0) @@ -2133,12 +2117,11 @@ public class ReplicationControlManagerTest { ControllerResult alterPartitionResult = replication.alterPartition(requestContext, new AlterPartitionRequest.Builder(alterIsrRequest).build(version).data()); - Errors expectedError = version <= 1 ? OPERATION_NOT_ATTEMPTED : INELIGIBLE_REPLICA; + Errors expectedError = INELIGIBLE_REPLICA; assertEquals( new AlterPartitionResponseData() .setTopics(singletonList(new AlterPartitionResponseData.TopicData() - .setTopicName(version <= 1 ? "foo" : "") - .setTopicId(version > 1 ? fooId : Uuid.ZERO_UUID) + .setTopicId(fooId) .setPartitions(singletonList(new AlterPartitionResponseData.PartitionData() .setPartitionIndex(0) .setErrorCode(expectedError.code()))))), @@ -2863,11 +2846,10 @@ public class ReplicationControlManagerTest { assertEquals(DirectoryId.UNASSIGNED, describer.defaultDir(3)); } - @ParameterizedTest - @EnumSource(value = MetadataVersion.class, names = {"IBP_3_3_IV2", "IBP_3_3_IV3"}) - public void testProcessBrokerHeartbeatInControlledShutdown(MetadataVersion metadataVersion) { + @Test + public void testProcessBrokerHeartbeatInControlledShutdown() { ReplicationControlTestContext ctx = new ReplicationControlTestContext.Builder(). - setMetadataVersion(metadataVersion). + setMetadataVersion(MetadataVersion.MINIMUM_VERSION). build(); ctx.registerBrokers(0, 1, 2); ctx.unfenceBrokers(0, 1, 2); @@ -2885,15 +2867,13 @@ public class ReplicationControlManagerTest { List expectedRecords = new ArrayList<>(); - if (metadataVersion.isInControlledShutdownStateSupported()) { - expectedRecords.add(new ApiMessageAndVersion( - new BrokerRegistrationChangeRecord() - .setBrokerEpoch(100) - .setBrokerId(0) - .setInControlledShutdown(BrokerRegistrationInControlledShutdownChange - .IN_CONTROLLED_SHUTDOWN.value()), - (short) 1)); - } + expectedRecords.add(new ApiMessageAndVersion( + new BrokerRegistrationChangeRecord() + .setBrokerEpoch(100) + .setBrokerId(0) + .setInControlledShutdown(BrokerRegistrationInControlledShutdownChange + .IN_CONTROLLED_SHUTDOWN.value()), + (short) 1)); expectedRecords.add(new ApiMessageAndVersion( new PartitionChangeRecord() diff --git a/metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetadataMetricsPublisherTest.java b/metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetadataMetricsPublisherTest.java index 2d230d60512..ffde7b86be9 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetadataMetricsPublisherTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetadataMetricsPublisherTest.java @@ -36,6 +36,7 @@ import org.apache.kafka.image.loader.SnapshotManifest; import org.apache.kafka.image.writer.ImageReWriter; import org.apache.kafka.image.writer.ImageWriterOptions; import org.apache.kafka.raft.LeaderAndEpoch; +import org.apache.kafka.server.common.MetadataVersion; import org.apache.kafka.server.fault.MockFaultHandler; import org.junit.jupiter.api.Test; @@ -145,9 +146,7 @@ public class ControllerMetadataMetricsPublisherTest { try (TestEnv env = new TestEnv()) { MetadataDelta delta = new MetadataDelta(MetadataImage.EMPTY); ImageReWriter writer = new ImageReWriter(delta); - IMAGE1.write(writer, new ImageWriterOptions.Builder(). - setMetadataVersion(delta.image().features().metadataVersion()). - build()); + IMAGE1.write(writer, new ImageWriterOptions.Builder(MetadataVersion.MINIMUM_VERSION).build()); env.publisher.onMetadataUpdate(delta, IMAGE1, fakeManifest(true)); assertEquals(0, env.metrics.activeBrokerCount()); assertEquals(3, env.metrics.globalTopicCount()); diff --git a/metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetricsChangesTest.java b/metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetricsChangesTest.java index 80f54daf27b..c172725c67a 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetricsChangesTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetricsChangesTest.java @@ -130,8 +130,7 @@ public class ControllerMetricsChangesTest { static final TopicDelta TOPIC_DELTA2; static { - ImageWriterOptions options = new ImageWriterOptions.Builder(). - setMetadataVersion(MetadataVersion.IBP_3_7_IV0).build(); // highest MV for PartitionRecord v0 + ImageWriterOptions options = new ImageWriterOptions.Builder(MetadataVersion.IBP_3_7_IV0).build(); // highest MV for PartitionRecord v0 TOPIC_DELTA1 = new TopicDelta(new TopicImage("foo", FOO_ID, Collections.emptyMap())); TOPIC_DELTA1.replay((PartitionRecord) fakePartitionRegistration(NORMAL). toRecord(FOO_ID, 0, options).message()); diff --git a/metadata/src/test/java/org/apache/kafka/image/AclsImageTest.java b/metadata/src/test/java/org/apache/kafka/image/AclsImageTest.java index 021e3c52e53..38d395a740b 100644 --- a/metadata/src/test/java/org/apache/kafka/image/AclsImageTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/AclsImageTest.java @@ -19,7 +19,6 @@ package org.apache.kafka.image; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.metadata.RemoveAccessControlEntryRecord; -import org.apache.kafka.image.writer.ImageWriterOptions; import org.apache.kafka.image.writer.RecordListWriter; import org.apache.kafka.metadata.RecordTestUtils; import org.apache.kafka.metadata.authorizer.StandardAcl; @@ -116,7 +115,7 @@ public class AclsImageTest { private static List getImageRecords(AclsImage image) { RecordListWriter writer = new RecordListWriter(); - image.write(writer, new ImageWriterOptions.Builder().build()); + image.write(writer); return writer.records(); } } diff --git a/metadata/src/test/java/org/apache/kafka/image/ClientQuotasImageTest.java b/metadata/src/test/java/org/apache/kafka/image/ClientQuotasImageTest.java index 5b527bd51b8..d1d33444989 100644 --- a/metadata/src/test/java/org/apache/kafka/image/ClientQuotasImageTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/ClientQuotasImageTest.java @@ -24,6 +24,7 @@ import org.apache.kafka.image.writer.ImageWriterOptions; import org.apache.kafka.image.writer.RecordListWriter; import org.apache.kafka.metadata.RecordTestUtils; import org.apache.kafka.server.common.ApiMessageAndVersion; +import org.apache.kafka.server.common.MetadataVersion; import org.apache.kafka.server.config.QuotaConfig; import org.junit.jupiter.api.Test; @@ -137,7 +138,7 @@ public class ClientQuotasImageTest { private static List getImageRecords(ClientQuotasImage image) { RecordListWriter writer = new RecordListWriter(); - image.write(writer, new ImageWriterOptions.Builder().build()); + image.write(writer, new ImageWriterOptions.Builder(MetadataVersion.latestProduction()).build()); return writer.records(); } } diff --git a/metadata/src/test/java/org/apache/kafka/image/ClusterImageTest.java b/metadata/src/test/java/org/apache/kafka/image/ClusterImageTest.java index cf79d276b84..7e0cb9905fc 100644 --- a/metadata/src/test/java/org/apache/kafka/image/ClusterImageTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/ClusterImageTest.java @@ -199,7 +199,7 @@ public class ClusterImageTest { setFeatures(new BrokerFeatureCollection( Collections.singleton(new BrokerFeature(). setName(MetadataVersion.FEATURE_NAME). - setMinSupportedVersion(MetadataVersion.IBP_3_3_IV3.featureLevel()). + setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()). setMaxSupportedVersion(MetadataVersion.IBP_3_6_IV0.featureLevel())).iterator())). setRack("rack3"), REGISTER_BROKER_RECORD.highestSupportedVersion())); @@ -232,7 +232,7 @@ public class ClusterImageTest { setIncarnationId(Uuid.fromString("Am5Yse7GQxaw0b2alM74bP")). setListeners(Collections.singletonList(new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 9094))). setSupportedFeatures(Collections.singletonMap("metadata.version", - VersionRange.of(MetadataVersion.IBP_3_3_IV3.featureLevel(), MetadataVersion.IBP_3_6_IV0.featureLevel()))). + VersionRange.of(MetadataVersion.MINIMUM_VERSION.featureLevel(), MetadataVersion.IBP_3_6_IV0.featureLevel()))). setRack(Optional.of("rack3")). setFenced(true). setIsMigratingZkBroker(true).build()); @@ -296,7 +296,7 @@ public class ClusterImageTest { private static List getImageRecords(ClusterImage image) { RecordListWriter writer = new RecordListWriter(); - image.write(writer, new ImageWriterOptions.Builder().build()); + image.write(writer, new ImageWriterOptions.Builder(MetadataVersion.latestProduction()).build()); return writer.records(); } @@ -311,8 +311,7 @@ public class ClusterImageTest { setSupportedFeatures(Collections.emptyMap()).build())); RecordListWriter writer = new RecordListWriter(); final AtomicReference lossString = new AtomicReference<>(""); - testImage.write(writer, new ImageWriterOptions.Builder(). - setMetadataVersion(MetadataVersion.IBP_3_6_IV2). + testImage.write(writer, new ImageWriterOptions.Builder(MetadataVersion.IBP_3_6_IV2). setLossHandler(loss -> lossString.compareAndSet("", loss.loss())). build()); assertEquals("controller registration data", lossString.get()); diff --git a/metadata/src/test/java/org/apache/kafka/image/ConfigurationsImageTest.java b/metadata/src/test/java/org/apache/kafka/image/ConfigurationsImageTest.java index 3b49ec33946..2135a498bf4 100644 --- a/metadata/src/test/java/org/apache/kafka/image/ConfigurationsImageTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/ConfigurationsImageTest.java @@ -23,6 +23,7 @@ import org.apache.kafka.image.writer.ImageWriterOptions; import org.apache.kafka.image.writer.RecordListWriter; import org.apache.kafka.metadata.RecordTestUtils; import org.apache.kafka.server.common.ApiMessageAndVersion; +import org.apache.kafka.server.common.MetadataVersion; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -136,7 +137,7 @@ public class ConfigurationsImageTest { private static List getImageRecords(ConfigurationsImage image) { RecordListWriter writer = new RecordListWriter(); - image.write(writer, new ImageWriterOptions.Builder().build()); + image.write(writer, new ImageWriterOptions.Builder(MetadataVersion.latestProduction()).build()); return writer.records(); } } diff --git a/metadata/src/test/java/org/apache/kafka/image/DelegationTokenImageTest.java b/metadata/src/test/java/org/apache/kafka/image/DelegationTokenImageTest.java index 7de0237068d..d3aec2de472 100644 --- a/metadata/src/test/java/org/apache/kafka/image/DelegationTokenImageTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/DelegationTokenImageTest.java @@ -39,7 +39,7 @@ import java.util.Map; import java.util.Optional; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.fail; +import static org.junit.jupiter.api.Assertions.assertThrows; @Timeout(value = 40) @@ -134,28 +134,22 @@ public class DelegationTokenImageTest { private static List getImageRecords(DelegationTokenImage image) { RecordListWriter writer = new RecordListWriter(); - image.write(writer, new ImageWriterOptions.Builder().build()); + image.write(writer, new ImageWriterOptions.Builder(MetadataVersion.latestProduction()).build()); return writer.records(); } @Test public void testEmptyWithInvalidIBP() { - ImageWriterOptions imageWriterOptions = new ImageWriterOptions.Builder(). - setMetadataVersion(MetadataVersion.IBP_3_5_IV2).build(); + ImageWriterOptions imageWriterOptions = new ImageWriterOptions.Builder(MetadataVersion.IBP_3_5_IV2).build(); RecordListWriter writer = new RecordListWriter(); DelegationTokenImage.EMPTY.write(writer, imageWriterOptions); } @Test - public void testImage1withInvalidIBP() { - ImageWriterOptions imageWriterOptions = new ImageWriterOptions.Builder(). - setMetadataVersion(MetadataVersion.IBP_3_5_IV2).build(); + public void testImage1WithInvalidIBP() { + ImageWriterOptions imageWriterOptions = new ImageWriterOptions.Builder(MetadataVersion.IBP_3_5_IV2).build(); RecordListWriter writer = new RecordListWriter(); - try { - IMAGE1.write(writer, imageWriterOptions); - fail("expected exception writing IMAGE with Delegation Token records for MetadataVersion.IBP_3_5_IV2"); - } catch (Exception expected) { - // ignore, expected - } + assertThrows(Exception.class, () -> IMAGE1.write(writer, imageWriterOptions), + "expected exception writing IMAGE with Delegation Token records for MetadataVersion.IBP_3_5_IV2"); } } diff --git a/metadata/src/test/java/org/apache/kafka/image/FeaturesDeltaTest.java b/metadata/src/test/java/org/apache/kafka/image/FeaturesDeltaTest.java new file mode 100644 index 00000000000..7eb1e5d4fc6 --- /dev/null +++ b/metadata/src/test/java/org/apache/kafka/image/FeaturesDeltaTest.java @@ -0,0 +1,43 @@ +/* + * 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.image; + +import org.apache.kafka.common.metadata.FeatureLevelRecord; +import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.common.MetadataVersionTestUtils; + +import org.junit.jupiter.api.Test; + +import static java.util.Collections.emptyMap; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +class FeaturesDeltaTest { + + @Test + public void testReplayWithUnsupportedFeatureLevel() { + var featuresDelta = new FeaturesDelta(new FeaturesImage(emptyMap(), MetadataVersion.MINIMUM_VERSION)); + var exception = assertThrows(IllegalArgumentException.class, () -> featuresDelta.replay(new FeatureLevelRecord() + .setName(MetadataVersion.FEATURE_NAME) + .setFeatureLevel(MetadataVersionTestUtils.IBP_3_3_IV2_FEATURE_LEVEL))); + assertTrue(exception.getMessage().contains("Unsupported metadata version - if you are currently upgrading your " + + "cluster, please ensure the metadata version is set to " + MetadataVersion.MINIMUM_VERSION), + "Expected substring missing from exception message: " + exception.getMessage()); + } + +} diff --git a/metadata/src/test/java/org/apache/kafka/image/FeaturesImageTest.java b/metadata/src/test/java/org/apache/kafka/image/FeaturesImageTest.java index cb83467e7fb..39e11ae03ea 100644 --- a/metadata/src/test/java/org/apache/kafka/image/FeaturesImageTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/FeaturesImageTest.java @@ -100,7 +100,13 @@ public class FeaturesImageTest { @Test public void testEmptyImageRoundTrip() { - testToImage(FeaturesImage.EMPTY); + var image = FeaturesImage.EMPTY; + var metadataVersion = MetadataVersion.MINIMUM_VERSION; + RecordListWriter writer = new RecordListWriter(); + image.write(writer, new ImageWriterOptions.Builder(metadataVersion).build()); + // A metadata version is required for writing, so the expected image is not actually empty + var expectedImage = new FeaturesImage(Collections.emptyMap(), metadataVersion); + testToImage(expectedImage, writer.records()); } @Test @@ -154,7 +160,7 @@ public class FeaturesImageTest { private static List getImageRecords(FeaturesImage image) { RecordListWriter writer = new RecordListWriter(); - image.write(writer, new ImageWriterOptions.Builder().setMetadataVersion(image.metadataVersion()).build()); + image.write(writer, new ImageWriterOptions.Builder(image.metadataVersionOrThrow()).build()); return writer.records(); } @@ -162,10 +168,9 @@ public class FeaturesImageTest { public void testEmpty() { assertTrue(FeaturesImage.EMPTY.isEmpty()); assertFalse(new FeaturesImage(Collections.singletonMap("foo", (short) 1), - FeaturesImage.EMPTY.metadataVersion()).isEmpty()); + MetadataVersion.MINIMUM_VERSION).isEmpty()); assertFalse(new FeaturesImage(FeaturesImage.EMPTY.finalizedVersions(), - MetadataVersion.IBP_3_3_IV0).isEmpty()); - assertTrue(new FeaturesImage(FeaturesImage.EMPTY.finalizedVersions(), FeaturesImage.EMPTY.metadataVersion()).isEmpty()); + MetadataVersion.MINIMUM_VERSION).isEmpty()); } @Test diff --git a/metadata/src/test/java/org/apache/kafka/image/ImageDowngradeTest.java b/metadata/src/test/java/org/apache/kafka/image/ImageDowngradeTest.java index ba636f3bdba..37bfa3462da 100644 --- a/metadata/src/test/java/org/apache/kafka/image/ImageDowngradeTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/ImageDowngradeTest.java @@ -77,59 +77,6 @@ public class ImageDowngradeTest { setFeatureLevel(metadataVersion.featureLevel()), (short) 0); } - /** - * Test downgrading to a MetadataVersion that doesn't support FeatureLevelRecord. - */ - @Test - public void testPremodernVersion() { - writeWithExpectedLosses(MetadataVersion.IBP_3_2_IV0, - Collections.singletonList( - "feature flag(s): foo.feature"), - Arrays.asList( - metadataVersionRecord(MetadataVersion.IBP_3_3_IV0), - TEST_RECORDS.get(0), - TEST_RECORDS.get(1), - new ApiMessageAndVersion(new FeatureLevelRecord(). - setName("foo.feature"). - setFeatureLevel((short) 4), (short) 0)), - Arrays.asList( - TEST_RECORDS.get(0), - TEST_RECORDS.get(1)) - ); - } - - /** - * Test downgrading to a MetadataVersion that doesn't support inControlledShutdown. - */ - @Test - public void testPreControlledShutdownStateVersion() { - writeWithExpectedLosses(MetadataVersion.IBP_3_3_IV2, - Collections.singletonList( - "the inControlledShutdown state of one or more brokers"), - Arrays.asList( - metadataVersionRecord(MetadataVersion.IBP_3_3_IV3), - new ApiMessageAndVersion(new RegisterBrokerRecord(). - setBrokerId(123). - setIncarnationId(Uuid.fromString("XgjKo16hRWeWrTui0iR5Nw")). - setBrokerEpoch(456). - setRack(null). - setFenced(false). - setInControlledShutdown(true), (short) 1), - TEST_RECORDS.get(0), - TEST_RECORDS.get(1)), - Arrays.asList( - metadataVersionRecord(MetadataVersion.IBP_3_3_IV2), - new ApiMessageAndVersion(new RegisterBrokerRecord(). - setBrokerId(123). - setIncarnationId(Uuid.fromString("XgjKo16hRWeWrTui0iR5Nw")). - setBrokerEpoch(456). - setRack(null). - setFenced(false), (short) 0), - TEST_RECORDS.get(0), - TEST_RECORDS.get(1)) - ); - } - /** * Test downgrading to a MetadataVersion that doesn't support ZK migration. */ @@ -200,8 +147,7 @@ public class ImageDowngradeTest { RecordTestUtils.replayAll(delta, inputs); MetadataImage image = delta.apply(MetadataProvenance.EMPTY); RecordListWriter writer = new RecordListWriter(); - image.write(writer, new ImageWriterOptions.Builder(). - setMetadataVersion(metadataVersion). + image.write(writer, new ImageWriterOptions.Builder(metadataVersion). setLossHandler(lossConsumer). build()); assertEquals(expectedLosses, lossConsumer.losses, "Failed to get expected metadata losses."); diff --git a/metadata/src/test/java/org/apache/kafka/image/MetadataImageTest.java b/metadata/src/test/java/org/apache/kafka/image/MetadataImageTest.java index dea6fbab710..715ac0beed4 100644 --- a/metadata/src/test/java/org/apache/kafka/image/MetadataImageTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/MetadataImageTest.java @@ -21,10 +21,12 @@ import org.apache.kafka.image.writer.ImageWriterOptions; import org.apache.kafka.image.writer.RecordListWriter; import org.apache.kafka.metadata.RecordTestUtils; import org.apache.kafka.server.common.ApiMessageAndVersion; +import org.apache.kafka.server.common.MetadataVersion; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; +import java.util.Collections; import java.util.List; import java.util.Optional; @@ -80,7 +82,22 @@ public class MetadataImageTest { @Test public void testEmptyImageRoundTrip() { - testToImage(MetadataImage.EMPTY); + var image = MetadataImage.EMPTY; + var metadataVersion = MetadataVersion.MINIMUM_VERSION; + ImageWriterOptions options = new ImageWriterOptions.Builder(metadataVersion).build(); + // A metadata version is required for writing, so the expected image is not actually empty + var expectedImage = new MetadataImage( + MetadataProvenance.EMPTY, + new FeaturesImage(Collections.emptyMap(), metadataVersion), + ClusterImage.EMPTY, + TopicsImage.EMPTY, + ConfigurationsImage.EMPTY, + ClientQuotasImage.EMPTY, + ProducerIdsImage.EMPTY, + AclsImage.EMPTY, + ScramImage.EMPTY, + DelegationTokenImage.EMPTY); + testToImage(expectedImage, getImageRecords(image, options)); } @Test @@ -92,9 +109,7 @@ public class MetadataImageTest { public void testApplyDelta1() { assertEquals(IMAGE2, DELTA1.apply(IMAGE2.provenance())); // check image1 + delta1 = image2, since records for image1 + delta1 might differ from records from image2 - ImageWriterOptions options = new ImageWriterOptions.Builder() - .setMetadataVersion(IMAGE1.features().metadataVersion()) - .build(); + ImageWriterOptions options = new ImageWriterOptions.Builder(IMAGE1.features().metadataVersionOrThrow()).build(); List records = getImageRecords(IMAGE1, options); records.addAll(FeaturesImageTest.DELTA1_RECORDS); records.addAll(ClusterImageTest.DELTA1_RECORDS); @@ -114,9 +129,7 @@ public class MetadataImageTest { } private static void testToImage(MetadataImage image) { - testToImage(image, new ImageWriterOptions.Builder() - .setMetadataVersion(image.features().metadataVersion()) - .build(), Optional.empty()); + testToImage(image, new ImageWriterOptions.Builder(image.features().metadataVersionOrThrow()).build(), Optional.empty()); } static void testToImage(MetadataImage image, ImageWriterOptions options, Optional> fromRecords) { diff --git a/metadata/src/test/java/org/apache/kafka/image/MetadataVersionChangeTest.java b/metadata/src/test/java/org/apache/kafka/image/MetadataVersionChangeTest.java index 2fd869d7152..323cc5abf9a 100644 --- a/metadata/src/test/java/org/apache/kafka/image/MetadataVersionChangeTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/MetadataVersionChangeTest.java @@ -17,11 +17,11 @@ package org.apache.kafka.image; +import org.apache.kafka.server.common.MetadataVersion; + import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import static org.apache.kafka.server.common.MetadataVersion.IBP_3_0_IV1; -import static org.apache.kafka.server.common.MetadataVersion.IBP_3_3_IV0; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -30,31 +30,31 @@ import static org.junit.jupiter.api.Assertions.assertTrue; @Timeout(value = 40) public class MetadataVersionChangeTest { - private static final MetadataVersionChange CHANGE_3_0_IV1_TO_3_3_IV0 = - new MetadataVersionChange(IBP_3_0_IV1, IBP_3_3_IV0); + private static final MetadataVersionChange CHANGE_MINIMUM_TO_LATEST = + new MetadataVersionChange(MetadataVersion.MINIMUM_VERSION, MetadataVersion.latestProduction()); - private static final MetadataVersionChange CHANGE_3_3_IV0_TO_3_0_IV1 = - new MetadataVersionChange(IBP_3_3_IV0, IBP_3_0_IV1); + private static final MetadataVersionChange CHANGE_LATEST_TO_MINIMUM = + new MetadataVersionChange(MetadataVersion.latestProduction(), MetadataVersion.MINIMUM_VERSION); @Test public void testIsUpgrade() { - assertTrue(CHANGE_3_0_IV1_TO_3_3_IV0.isUpgrade()); - assertFalse(CHANGE_3_3_IV0_TO_3_0_IV1.isUpgrade()); + assertTrue(CHANGE_MINIMUM_TO_LATEST.isUpgrade()); + assertFalse(CHANGE_LATEST_TO_MINIMUM.isUpgrade()); } @Test public void testIsDowngrade() { - assertFalse(CHANGE_3_0_IV1_TO_3_3_IV0.isDowngrade()); - assertTrue(CHANGE_3_3_IV0_TO_3_0_IV1.isDowngrade()); + assertFalse(CHANGE_MINIMUM_TO_LATEST.isDowngrade()); + assertTrue(CHANGE_LATEST_TO_MINIMUM.isDowngrade()); } @Test public void testMetadataVersionChangeExceptionToString() { assertEquals("org.apache.kafka.image.MetadataVersionChangeException: The metadata.version " + - "is changing from 3.0-IV1 to 3.3-IV0", - new MetadataVersionChangeException(CHANGE_3_0_IV1_TO_3_3_IV0).toString()); + "is changing from " + MetadataVersion.MINIMUM_VERSION + " to " + MetadataVersion.latestProduction(), + new MetadataVersionChangeException(CHANGE_MINIMUM_TO_LATEST).toString()); assertEquals("org.apache.kafka.image.MetadataVersionChangeException: The metadata.version " + - "is changing from 3.3-IV0 to 3.0-IV1", - new MetadataVersionChangeException(CHANGE_3_3_IV0_TO_3_0_IV1).toString()); + "is changing from " + MetadataVersion.latestProduction() + " to " + MetadataVersion.MINIMUM_VERSION, + new MetadataVersionChangeException(CHANGE_LATEST_TO_MINIMUM).toString()); } } diff --git a/metadata/src/test/java/org/apache/kafka/image/ProducerIdsImageTest.java b/metadata/src/test/java/org/apache/kafka/image/ProducerIdsImageTest.java index 4412d16deb4..e34b1f3d1a6 100644 --- a/metadata/src/test/java/org/apache/kafka/image/ProducerIdsImageTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/ProducerIdsImageTest.java @@ -22,6 +22,7 @@ import org.apache.kafka.image.writer.ImageWriterOptions; import org.apache.kafka.image.writer.RecordListWriter; import org.apache.kafka.metadata.RecordTestUtils; import org.apache.kafka.server.common.ApiMessageAndVersion; +import org.apache.kafka.server.common.MetadataVersion; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -112,7 +113,7 @@ public class ProducerIdsImageTest { private static List getImageRecords(ProducerIdsImage image) { RecordListWriter writer = new RecordListWriter(); - image.write(writer, new ImageWriterOptions.Builder().build()); + image.write(writer, new ImageWriterOptions.Builder(MetadataVersion.latestProduction()).build()); return writer.records(); } } diff --git a/metadata/src/test/java/org/apache/kafka/image/ScramImageTest.java b/metadata/src/test/java/org/apache/kafka/image/ScramImageTest.java index 528b75fbe03..acc8e426eaf 100644 --- a/metadata/src/test/java/org/apache/kafka/image/ScramImageTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/ScramImageTest.java @@ -41,7 +41,7 @@ import java.util.Random; import static org.apache.kafka.clients.admin.ScramMechanism.SCRAM_SHA_256; import static org.apache.kafka.clients.admin.ScramMechanism.SCRAM_SHA_512; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.fail; +import static org.junit.jupiter.api.Assertions.assertThrows; @Timeout(value = 40) @@ -170,28 +170,22 @@ public class ScramImageTest { private static List getImageRecords(ScramImage image) { RecordListWriter writer = new RecordListWriter(); - image.write(writer, new ImageWriterOptions.Builder().build()); + image.write(writer, new ImageWriterOptions.Builder(MetadataVersion.latestProduction()).build()); return writer.records(); } @Test public void testEmptyWithInvalidIBP() { - ImageWriterOptions imageWriterOptions = new ImageWriterOptions.Builder(). - setMetadataVersion(MetadataVersion.IBP_3_4_IV0).build(); + ImageWriterOptions imageWriterOptions = new ImageWriterOptions.Builder(MetadataVersion.IBP_3_4_IV0).build(); RecordListWriter writer = new RecordListWriter(); ScramImage.EMPTY.write(writer, imageWriterOptions); } @Test public void testImage1withInvalidIBP() { - ImageWriterOptions imageWriterOptions = new ImageWriterOptions.Builder(). - setMetadataVersion(MetadataVersion.IBP_3_4_IV0).build(); + ImageWriterOptions imageWriterOptions = new ImageWriterOptions.Builder(MetadataVersion.IBP_3_4_IV0).build(); RecordListWriter writer = new RecordListWriter(); - try { - IMAGE1.write(writer, imageWriterOptions); - fail("expected exception writing IMAGE with SCRAM records for MetadataVersion.IBP_3_4_IV0"); - } catch (Exception expected) { - // ignore, expected - } + assertThrows(Exception.class, () -> IMAGE1.write(writer, imageWriterOptions), + "expected exception writing IMAGE with SCRAM records for MetadataVersion.IBP_3_4_IV0"); } } diff --git a/metadata/src/test/java/org/apache/kafka/image/TopicsImageTest.java b/metadata/src/test/java/org/apache/kafka/image/TopicsImageTest.java index 398cb84b5aa..c868ebdfee7 100644 --- a/metadata/src/test/java/org/apache/kafka/image/TopicsImageTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/TopicsImageTest.java @@ -31,6 +31,7 @@ import org.apache.kafka.metadata.LeaderRecoveryState; import org.apache.kafka.metadata.PartitionRegistration; import org.apache.kafka.metadata.RecordTestUtils; import org.apache.kafka.server.common.ApiMessageAndVersion; +import org.apache.kafka.server.common.MetadataVersion; import org.apache.kafka.server.immutable.ImmutableMap; import org.junit.jupiter.api.Test; @@ -595,7 +596,7 @@ public class TopicsImageTest { private static List getImageRecords(TopicsImage image) { RecordListWriter writer = new RecordListWriter(); - image.write(writer, new ImageWriterOptions.Builder().build()); + image.write(writer, new ImageWriterOptions.Builder(MetadataVersion.latestProduction()).build()); return writer.records(); } diff --git a/metadata/src/test/java/org/apache/kafka/image/loader/MetadataLoaderTest.java b/metadata/src/test/java/org/apache/kafka/image/loader/MetadataLoaderTest.java index ce87c57a910..7b2a6e82bc8 100644 --- a/metadata/src/test/java/org/apache/kafka/image/loader/MetadataLoaderTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/loader/MetadataLoaderTest.java @@ -18,11 +18,9 @@ package org.apache.kafka.image.loader; import org.apache.kafka.common.Uuid; -import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.message.SnapshotHeaderRecord; import org.apache.kafka.common.metadata.AbortTransactionRecord; import org.apache.kafka.common.metadata.BeginTransactionRecord; -import org.apache.kafka.common.metadata.ConfigRecord; import org.apache.kafka.common.metadata.EndTransactionRecord; import org.apache.kafka.common.metadata.FeatureLevelRecord; import org.apache.kafka.common.metadata.PartitionRecord; @@ -55,6 +53,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Iterator; import java.util.List; +import java.util.Optional; import java.util.OptionalLong; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; @@ -65,9 +64,8 @@ import java.util.stream.Collectors; import static java.util.Arrays.asList; import static java.util.Collections.singletonList; -import static org.apache.kafka.server.common.MetadataVersion.IBP_3_3_IV1; -import static org.apache.kafka.server.common.MetadataVersion.IBP_3_3_IV2; import static org.apache.kafka.server.common.MetadataVersion.IBP_3_5_IV0; +import static org.apache.kafka.server.common.MetadataVersion.MINIMUM_VERSION; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotNull; @@ -243,8 +241,7 @@ public class MetadataLoaderTest { boolean loadSnapshot, boolean sameObject ) throws Exception { - MockFaultHandler faultHandler = - new MockFaultHandler("testPublisherCannotBeInstalledMoreThanOnce"); + MockFaultHandler faultHandler = new MockFaultHandler("testPublisherCannotBeInstalledMoreThanOnce"); MockPublisher publisher = new MockPublisher(); try (MetadataLoader loader = new MetadataLoader.Builder(). setFaultHandler(faultHandler). @@ -254,13 +251,18 @@ public class MetadataLoaderTest { if (loadSnapshot) { MockSnapshotReader snapshotReader = new MockSnapshotReader( new MetadataProvenance(200, 100, 4000, true), - singletonList( + List.of( Batch.control( 200, 100, 4000, 10, singletonList(new ControlRecord(ControlRecordType.SNAPSHOT_HEADER, new SnapshotHeaderRecord())) + ), + Batch.data(0, 0, 0, 0, + singletonList(new ApiMessageAndVersion(new FeatureLevelRecord(). + setName(MetadataVersion.FEATURE_NAME). + setFeatureLevel(MINIMUM_VERSION.featureLevel()), (short) 0)) ) ) ); @@ -310,7 +312,7 @@ public class MetadataLoaderTest { singletonList(singletonList(new ApiMessageAndVersion( new FeatureLevelRecord(). setName(MetadataVersion.FEATURE_NAME). - setFeatureLevel(IBP_3_3_IV2.featureLevel()), (short) 0)))); + setFeatureLevel(MINIMUM_VERSION.featureLevel()), (short) 0)))); assertFalse(snapshotReader.closed); loader.handleLoadSnapshot(snapshotReader); loader.waitForAllEventsToBeHandled(); @@ -319,7 +321,7 @@ public class MetadataLoaderTest { loader.removeAndClosePublisher(publishers.get(0)).get(); } assertTrue(publishers.get(0).closed); - assertEquals(IBP_3_3_IV2, + assertEquals(Optional.of(MINIMUM_VERSION), publishers.get(0).latestImage.features().metadataVersion()); assertTrue(publishers.get(1).closed); assertNull(publishers.get(1).latestImage); @@ -342,6 +344,12 @@ public class MetadataLoaderTest { setHighWaterMarkAccessor(() -> OptionalLong.of(0L)). build()) { loader.installPublishers(publishers).get(); + loader.handleCommit( + MockBatchReader.newSingleBatchReader(400, 50, List.of( + new ApiMessageAndVersion(new FeatureLevelRecord() + .setName(MetadataVersion.FEATURE_NAME) + .setFeatureLevel(MINIMUM_VERSION.featureLevel()), (short) 0))) + ); loadEmptySnapshot(loader, 200); publishers.get(0).firstPublish.get(10, TimeUnit.SECONDS); assertEquals(200L, loader.lastAppliedOffset()); @@ -349,13 +357,23 @@ public class MetadataLoaderTest { assertEquals(300L, loader.lastAppliedOffset()); assertEquals(new SnapshotManifest(new MetadataProvenance(300, 100, 4000, true), 3000000L), publishers.get(0).latestSnapshotManifest); - assertEquals(MetadataVersion.MINIMUM_KRAFT_VERSION, + assertEquals(MINIMUM_VERSION, loader.metrics().currentMetadataVersion()); } assertTrue(publishers.get(0).closed); - assertEquals(MetadataVersion.IBP_3_0_IV1, - publishers.get(0).latestImage.features().metadataVersion()); - assertTrue(publishers.get(0).latestImage.isEmpty()); + assertEquals(Optional.of(MINIMUM_VERSION), publishers.get(0).latestImage.features().metadataVersion()); + var latestImage = publishers.get(0).latestImage; + assertFalse(latestImage.isEmpty()); + assertFalse(latestImage.features().isEmpty()); + assertTrue(latestImage.features().finalizedVersions().isEmpty()); + assertTrue(latestImage.cluster().isEmpty()); + assertTrue(latestImage.topics().isEmpty()); + assertTrue(latestImage.cluster().isEmpty()); + assertTrue(latestImage.configs().isEmpty()); + assertTrue(latestImage.producerIds().isEmpty()); + assertTrue(latestImage.acls().isEmpty()); + assertTrue(latestImage.scram().isEmpty()); + assertTrue(latestImage.delegationTokens().isEmpty()); faultHandler.maybeRethrowFirstException(); } @@ -489,7 +507,7 @@ public class MetadataLoaderTest { .numBytes(10) .build(), publishers.get(0).latestLogDeltaManifest); - assertEquals(MetadataVersion.IBP_3_3_IV1, + assertEquals(Optional.of(MINIMUM_VERSION), publishers.get(0).latestImage.features().metadataVersion()); faultHandler.maybeRethrowFirstException(); } @@ -511,7 +529,7 @@ public class MetadataLoaderTest { new MetadataProvenance(200, 100, 4000, true), asList( singletonList(new ApiMessageAndVersion(new FeatureLevelRecord(). setName(MetadataVersion.FEATURE_NAME). - setFeatureLevel(IBP_3_3_IV1.featureLevel()), (short) 0)), + setFeatureLevel(MINIMUM_VERSION.featureLevel()), (short) 0)), singletonList(new ApiMessageAndVersion(new TopicRecord(). setName("foo"). setTopicId(Uuid.fromString("Uum7sfhHQP-obSvfywmNUA")), (short) 0)) @@ -531,8 +549,7 @@ public class MetadataLoaderTest { for (int i = 0; i < 2; i++) { assertTrue(publishers.get(i).closed); assertTrue(publishers.get(i).closed); - assertEquals(IBP_3_3_IV1, - publishers.get(i).latestImage.features().metadataVersion()); + assertEquals(Optional.of(MINIMUM_VERSION), publishers.get(i).latestImage.features().metadataVersion()); } faultHandler.maybeRethrowFirstException(); } @@ -581,7 +598,7 @@ public class MetadataLoaderTest { new MetadataProvenance(offset, 100, 4000, true), asList( singletonList(new ApiMessageAndVersion(new FeatureLevelRecord(). setName(MetadataVersion.FEATURE_NAME). - setFeatureLevel(IBP_3_3_IV1.featureLevel()), (short) 0)), + setFeatureLevel(MINIMUM_VERSION.featureLevel()), (short) 0)), singletonList(new ApiMessageAndVersion(new TopicRecord(). setName("foo"). setTopicId(Uuid.fromString("Uum7sfhHQP-obSvfywmNUA")), (short) 0)) @@ -597,7 +614,7 @@ public class MetadataLoaderTest { new MetadataProvenance(offset, 100, 4000, true), asList( singletonList(new ApiMessageAndVersion(new FeatureLevelRecord(). setName(MetadataVersion.FEATURE_NAME). - setFeatureLevel(IBP_3_3_IV2.featureLevel()), (short) 0)), + setFeatureLevel(MetadataVersion.latestProduction().featureLevel()), (short) 0)), singletonList(new ApiMessageAndVersion(new TopicRecord(). setName("bar"). setTopicId(Uuid.fromString("VcL2Mw-cT4aL6XV9VujzoQ")), (short) 0)) @@ -625,13 +642,13 @@ public class MetadataLoaderTest { loadTestSnapshot(loader, 200); assertEquals(200L, loader.lastAppliedOffset()); - assertEquals(IBP_3_3_IV1.featureLevel(), + assertEquals(MINIMUM_VERSION.featureLevel(), loader.metrics().currentMetadataVersion().featureLevel()); assertFalse(publishers.get(0).latestDelta.image().isEmpty()); loadTestSnapshot2(loader, 400); assertEquals(400L, loader.lastAppliedOffset()); - assertEquals(IBP_3_3_IV2.featureLevel(), + assertEquals(MetadataVersion.latestProduction().featureLevel(), loader.metrics().currentMetadataVersion().featureLevel()); // Make sure the topic in the initial snapshot was overwritten by loading the new snapshot. @@ -661,6 +678,12 @@ public class MetadataLoaderTest { setHighWaterMarkAccessor(() -> OptionalLong.of(0)). build()) { loader.installPublishers(publishers).get(); + loader.handleCommit( + MockBatchReader.newSingleBatchReader(400, 50, List.of( + new ApiMessageAndVersion(new FeatureLevelRecord() + .setName(MetadataVersion.FEATURE_NAME) + .setFeatureLevel(MINIMUM_VERSION.featureLevel()), (short) 0))) + ); loader.waitForAllEventsToBeHandled(); loader.handleCommit( @@ -722,6 +745,12 @@ public class MetadataLoaderTest { setHighWaterMarkAccessor(() -> OptionalLong.of(0)). build()) { loader.installPublishers(publishers).get(); + loader.handleCommit( + MockBatchReader.newSingleBatchReader(400, 50, List.of( + new ApiMessageAndVersion(new FeatureLevelRecord(). + setName(MetadataVersion.FEATURE_NAME). + setFeatureLevel(MINIMUM_VERSION.featureLevel()), (short) 0))) + ); loader.waitForAllEventsToBeHandled(); loader.handleCommit( @@ -753,6 +782,12 @@ public class MetadataLoaderTest { setHighWaterMarkAccessor(() -> OptionalLong.of(0)). build()) { loader.installPublishers(publishers).get(); + loader.handleCommit( + MockBatchReader.newSingleBatchReader(400, 50, List.of( + new ApiMessageAndVersion(new FeatureLevelRecord(). + setName(MetadataVersion.FEATURE_NAME). + setFeatureLevel(MINIMUM_VERSION.featureLevel()), (short) 0))) + ); loader.waitForAllEventsToBeHandled(); loader.handleCommit( @@ -809,12 +844,10 @@ public class MetadataLoaderTest { loader.handleCommit( MockBatchReader.newSingleBatchReader(0, 1, singletonList( // Any record will work here - new ApiMessageAndVersion(new ConfigRecord() - .setResourceType(ConfigResource.Type.BROKER.id()) - .setResourceName("3000") - .setName("foo") - .setValue("bar"), (short) 0) - ))); + new ApiMessageAndVersion(new FeatureLevelRecord() + .setName(MetadataVersion.FEATURE_NAME) + .setFeatureLevel(MINIMUM_VERSION.featureLevel()), (short) 0))) + ); firstPublish.get(30, TimeUnit.SECONDS); assertFalse(capturedImages.isEmpty()); diff --git a/metadata/src/test/java/org/apache/kafka/image/loader/metrics/MetadataLoaderMetricsTest.java b/metadata/src/test/java/org/apache/kafka/image/loader/metrics/MetadataLoaderMetricsTest.java index e26087951d0..13062c98169 100644 --- a/metadata/src/test/java/org/apache/kafka/image/loader/metrics/MetadataLoaderMetricsTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/loader/metrics/MetadataLoaderMetricsTest.java @@ -34,7 +34,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; -import static org.apache.kafka.server.common.MetadataVersion.IBP_3_3_IV2; +import static org.apache.kafka.server.common.MetadataVersion.MINIMUM_VERSION; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -116,7 +116,7 @@ public class MetadataLoaderMetricsTest { MetricsRegistry registry = new MetricsRegistry(); try { try (FakeMetadataLoaderMetrics fakeMetrics = new FakeMetadataLoaderMetrics(registry)) { - fakeMetrics.metrics.setCurrentMetadataVersion(IBP_3_3_IV2); + fakeMetrics.metrics.setCurrentMetadataVersion(MINIMUM_VERSION); fakeMetrics.metrics.incrementHandleLoadSnapshotCount(); fakeMetrics.metrics.incrementHandleLoadSnapshotCount(); @@ -124,7 +124,7 @@ public class MetadataLoaderMetricsTest { Gauge currentMetadataVersion = (Gauge) registry .allMetrics() .get(metricName("MetadataLoader", "CurrentMetadataVersion")); - assertEquals(IBP_3_3_IV2.featureLevel(), + assertEquals(MINIMUM_VERSION.featureLevel(), currentMetadataVersion.value().shortValue()); @SuppressWarnings("unchecked") diff --git a/metadata/src/test/java/org/apache/kafka/image/publisher/BrokerRegistrationTrackerTest.java b/metadata/src/test/java/org/apache/kafka/image/publisher/BrokerRegistrationTrackerTest.java index 2e03587a7c2..5c3bc5a1230 100644 --- a/metadata/src/test/java/org/apache/kafka/image/publisher/BrokerRegistrationTrackerTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/publisher/BrokerRegistrationTrackerTest.java @@ -99,6 +99,9 @@ public class BrokerRegistrationTrackerTest { setBrokerId(1). setIncarnationId(INCARNATION_ID). setLogDirs(Arrays.asList(A, B, C))); + delta.replay(new FeatureLevelRecord(). + setName(MetadataVersion.FEATURE_NAME). + setFeatureLevel(MetadataVersion.MINIMUM_VERSION.featureLevel())); ctx.onMetadataUpdate(delta); assertEquals(0, ctx.numCalls.get()); } @@ -133,8 +136,11 @@ public class BrokerRegistrationTrackerTest { setBrokerId(1). setIncarnationId(INCARNATION_ID). setLogDirs(Collections.emptyList())); + delta.replay(new FeatureLevelRecord(). + setName(MetadataVersion.FEATURE_NAME). + setFeatureLevel(MetadataVersion.IBP_3_7_IV1.featureLevel())); ctx.onMetadataUpdate(delta); - // No calls are made because MetadataVersion is 3.0-IV1 initially + // No calls are made because MetadataVersion is older than IBP_3_7_IV2 initially assertEquals(0, ctx.numCalls.get()); delta = ctx.newDelta(); diff --git a/metadata/src/test/java/org/apache/kafka/image/writer/ImageWriterOptionsTest.java b/metadata/src/test/java/org/apache/kafka/image/writer/ImageWriterOptionsTest.java index 8ed78258f93..0070d41bdf9 100644 --- a/metadata/src/test/java/org/apache/kafka/image/writer/ImageWriterOptionsTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/writer/ImageWriterOptionsTest.java @@ -48,40 +48,22 @@ import static org.junit.jupiter.api.Assertions.assertThrows; public class ImageWriterOptionsTest { @Test public void testDefaultLossHandler() { - ImageWriterOptions options = new ImageWriterOptions.Builder().build(); + ImageWriterOptions options = new ImageWriterOptions.Builder(MetadataVersion.latestProduction()).build(); assertEquals("stuff", assertThrows(UnwritableMetadataException.class, () -> options.handleLoss("stuff")).loss()); } - @Test - public void testSetMetadataVersion() { - for (int i = MetadataVersion.MINIMUM_KRAFT_VERSION.ordinal(); - i < MetadataVersion.VERSIONS.length; - i++) { - MetadataVersion version = MetadataVersion.VERSIONS[i]; - ImageWriterOptions.Builder options = new ImageWriterOptions.Builder(). - setMetadataVersion(version); - if (i < MetadataVersion.MINIMUM_BOOTSTRAP_VERSION.ordinal()) { - assertEquals(MetadataVersion.MINIMUM_KRAFT_VERSION, options.metadataVersion()); - assertEquals(version, options.requestedMetadataVersion()); - } else { - assertEquals(version, options.metadataVersion()); - } - } - } - @Test public void testHandleLoss() { String expectedMessage = "stuff"; - for (int i = MetadataVersion.MINIMUM_KRAFT_VERSION.ordinal(); + for (int i = MetadataVersion.MINIMUM_VERSION.ordinal(); i < MetadataVersion.VERSIONS.length; i++) { MetadataVersion version = MetadataVersion.VERSIONS[i]; String formattedMessage = String.format("Metadata has been lost because the following could not be represented in metadata.version %s: %s", version, expectedMessage); Consumer customLossHandler = e -> assertEquals(formattedMessage, e.getMessage()); - ImageWriterOptions options = new ImageWriterOptions.Builder() - .setMetadataVersion(version) + ImageWriterOptions options = new ImageWriterOptions.Builder(version) .setLossHandler(customLossHandler) .build(); options.handleLoss(expectedMessage); @@ -90,14 +72,12 @@ public class ImageWriterOptionsTest { @Test public void testSetEligibleLeaderReplicasEnabled() { - MetadataVersion version = MetadataVersion.MINIMUM_BOOTSTRAP_VERSION; - ImageWriterOptions options = new ImageWriterOptions.Builder(). - setMetadataVersion(version). + MetadataVersion version = MetadataVersion.MINIMUM_VERSION; + ImageWriterOptions options = new ImageWriterOptions.Builder(version). setEligibleLeaderReplicasEnabled(true).build(); assertEquals(true, options.isEligibleLeaderReplicasEnabled()); - options = new ImageWriterOptions.Builder(). - setMetadataVersion(version).build(); + options = new ImageWriterOptions.Builder(version).build(); assertEquals(false, options.isEligibleLeaderReplicasEnabled()); } diff --git a/metadata/src/test/java/org/apache/kafka/metadata/BrokerRegistrationTest.java b/metadata/src/test/java/org/apache/kafka/metadata/BrokerRegistrationTest.java index e45234d225e..f415e506055 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/BrokerRegistrationTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/BrokerRegistrationTest.java @@ -25,6 +25,7 @@ import org.apache.kafka.common.metadata.RegisterBrokerRecord; import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.image.writer.ImageWriterOptions; import org.apache.kafka.server.common.ApiMessageAndVersion; +import org.apache.kafka.server.common.MetadataVersion; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -143,7 +144,7 @@ public class BrokerRegistrationTest { } private void testRoundTrip(BrokerRegistration registration) { - ImageWriterOptions options = new ImageWriterOptions.Builder().build(); + ImageWriterOptions options = new ImageWriterOptions.Builder(MetadataVersion.latestProduction()).build(); ApiMessageAndVersion messageAndVersion = registration. toRecord(options); BrokerRegistration registration2 = BrokerRegistration.fromRecord( diff --git a/metadata/src/test/java/org/apache/kafka/metadata/ControllerRegistrationTest.java b/metadata/src/test/java/org/apache/kafka/metadata/ControllerRegistrationTest.java index fc8786a2028..a24de3092bb 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/ControllerRegistrationTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/ControllerRegistrationTest.java @@ -118,12 +118,12 @@ public class ControllerRegistrationTest { private void testRoundTrip(ControllerRegistration registration) { ApiMessageAndVersion messageAndVersion = registration. - toRecord(new ImageWriterOptions.Builder().build()); + toRecord(new ImageWriterOptions.Builder(MetadataVersion.latestProduction()).build()); ControllerRegistration registration2 = new ControllerRegistration.Builder( (RegisterControllerRecord) messageAndVersion.message()).build(); assertEquals(registration, registration2); ApiMessageAndVersion messageAndVersion2 = registration2. - toRecord(new ImageWriterOptions.Builder().build()); + toRecord(new ImageWriterOptions.Builder(MetadataVersion.latestProduction()).build()); assertEquals(messageAndVersion, messageAndVersion2); } diff --git a/metadata/src/test/java/org/apache/kafka/metadata/PartitionRegistrationTest.java b/metadata/src/test/java/org/apache/kafka/metadata/PartitionRegistrationTest.java index 2ef04c83f01..8dc817f2621 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/PartitionRegistrationTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/PartitionRegistrationTest.java @@ -91,8 +91,8 @@ public class PartitionRegistrationTest { setIsr(new int[]{1, 2}).setRemovingReplicas(new int[]{1}).setLeader(1).setLeaderRecoveryState(LeaderRecoveryState.RECOVERED).setLeaderEpoch(0).setPartitionEpoch(0).build(); Uuid topicId = Uuid.fromString("OGdAI5nxT_m-ds3rJMqPLA"); int partitionId = 4; - ApiMessageAndVersion record = registrationA.toRecord(topicId, partitionId, new ImageWriterOptions.Builder(). - setMetadataVersion(MetadataVersion.IBP_3_7_IV0).build()); // highest MV for PartitionRecord v0 + ApiMessageAndVersion record = registrationA.toRecord(topicId, partitionId, + new ImageWriterOptions.Builder(MetadataVersion.IBP_3_7_IV0).build()); // highest MV for PartitionRecord v0 PartitionRegistration registrationB = new PartitionRegistration((PartitionRecord) record.message()); assertEquals(registrationA, registrationB); @@ -336,8 +336,7 @@ public class PartitionRegistrationTest { )); } List exceptions = new ArrayList<>(); - ImageWriterOptions options = new ImageWriterOptions.Builder(). - setMetadataVersion(metadataVersion). + ImageWriterOptions options = new ImageWriterOptions.Builder(metadataVersion). setEligibleLeaderReplicasEnabled(metadataVersion.isElrSupported()). setLossHandler(exceptions::add). build(); @@ -373,8 +372,7 @@ public class PartitionRegistrationTest { setDirectories(Arrays.asList(DirectoryId.migratingArray(5))). setPartitionEpoch(0); List exceptions = new ArrayList<>(); - ImageWriterOptions options = new ImageWriterOptions.Builder(). - setMetadataVersion(MetadataVersion.IBP_4_0_IV1). + ImageWriterOptions options = new ImageWriterOptions.Builder(MetadataVersion.IBP_4_0_IV1). setLossHandler(exceptions::add). build(); assertEquals(new ApiMessageAndVersion(expectRecord, (short) 2), partitionRegistration.toRecord(topicID, 0, options)); diff --git a/metadata/src/test/java/org/apache/kafka/metadata/RecordTestUtils.java b/metadata/src/test/java/org/apache/kafka/metadata/RecordTestUtils.java index 2005fab92a5..70ec1425308 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/RecordTestUtils.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/RecordTestUtils.java @@ -374,7 +374,7 @@ public class RecordTestUtils { Collections.singletonList( new RegisterControllerRecord.ControllerFeature(). setName(MetadataVersion.FEATURE_NAME). - setMinSupportedVersion(MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel()). + setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()). setMaxSupportedVersion(MetadataVersion.IBP_3_6_IV1.featureLevel()) ).iterator() )); diff --git a/metadata/src/test/java/org/apache/kafka/metadata/bootstrap/BootstrapDirectoryTest.java b/metadata/src/test/java/org/apache/kafka/metadata/bootstrap/BootstrapDirectoryTest.java index 4eabc02f518..97240b227bb 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/bootstrap/BootstrapDirectoryTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/bootstrap/BootstrapDirectoryTest.java @@ -29,7 +29,6 @@ import org.junit.jupiter.api.Timeout; import java.io.File; import java.util.List; -import java.util.Optional; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -74,25 +73,7 @@ public class BootstrapDirectoryTest { try (BootstrapTestDirectory testDirectory = new BootstrapTestDirectory().createDirectory()) { assertEquals(BootstrapMetadata.fromVersion(MetadataVersion.latestProduction(), "the default bootstrap"), - new BootstrapDirectory(testDirectory.path(), Optional.empty()).read()); - } - } - - @Test - public void testReadFromConfigurationWithAncientVersion() throws Exception { - try (BootstrapTestDirectory testDirectory = new BootstrapTestDirectory().createDirectory()) { - assertEquals(BootstrapMetadata.fromVersion(MetadataVersion.MINIMUM_BOOTSTRAP_VERSION, - "the minimum version bootstrap with metadata.version 3.3-IV0"), - new BootstrapDirectory(testDirectory.path(), Optional.of("3.0")).read()); - } - } - - @Test - public void testReadFromConfiguration() throws Exception { - try (BootstrapTestDirectory testDirectory = new BootstrapTestDirectory().createDirectory()) { - assertEquals(BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_3_IV2, - "the configured bootstrap with metadata.version 3.3-IV2"), - new BootstrapDirectory(testDirectory.path(), Optional.of("3.3-IV2")).read()); + new BootstrapDirectory(testDirectory.path()).read()); } } @@ -100,13 +81,13 @@ public class BootstrapDirectoryTest { public void testMissingDirectory() { assertEquals("No such directory as ./non/existent/directory", assertThrows(RuntimeException.class, () -> - new BootstrapDirectory("./non/existent/directory", Optional.empty()).read()).getMessage()); + new BootstrapDirectory("./non/existent/directory").read()).getMessage()); } @Test public void testReadFromConfigurationFile() throws Exception { try (BootstrapTestDirectory testDirectory = new BootstrapTestDirectory().createDirectory()) { - BootstrapDirectory directory = new BootstrapDirectory(testDirectory.path(), Optional.of("3.0-IV0")); + BootstrapDirectory directory = new BootstrapDirectory(testDirectory.path()); BootstrapMetadata metadata = BootstrapMetadata.fromRecords(SAMPLE_RECORDS1, "the binary bootstrap metadata file: " + testDirectory.binaryBootstrapPath()); directory.writeBinaryFile(metadata); diff --git a/metadata/src/test/java/org/apache/kafka/metadata/bootstrap/BootstrapMetadataTest.java b/metadata/src/test/java/org/apache/kafka/metadata/bootstrap/BootstrapMetadataTest.java index 550f6ed966a..edee755eee3 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/bootstrap/BootstrapMetadataTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/bootstrap/BootstrapMetadataTest.java @@ -20,6 +20,8 @@ package org.apache.kafka.metadata.bootstrap; import org.apache.kafka.common.metadata.FeatureLevelRecord; import org.apache.kafka.common.metadata.NoOpRecord; import org.apache.kafka.server.common.ApiMessageAndVersion; +import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.common.MetadataVersionTestUtils; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -30,8 +32,7 @@ import java.util.List; import static java.util.Collections.emptyList; import static org.apache.kafka.server.common.MetadataVersion.FEATURE_NAME; -import static org.apache.kafka.server.common.MetadataVersion.IBP_3_0_IV1; -import static org.apache.kafka.server.common.MetadataVersion.IBP_3_3_IV2; +import static org.apache.kafka.server.common.MetadataVersion.IBP_3_3_IV3; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -41,25 +42,25 @@ public class BootstrapMetadataTest { static final List SAMPLE_RECORDS1 = List.of( new ApiMessageAndVersion(new FeatureLevelRecord(). setName(FEATURE_NAME). - setFeatureLevel((short) 7), (short) 0), + setFeatureLevel((short) 8), (short) 0), new ApiMessageAndVersion(new NoOpRecord(), (short) 0), new ApiMessageAndVersion(new FeatureLevelRecord(). setName(FEATURE_NAME). - setFeatureLevel((short) 6), (short) 0)); + setFeatureLevel((short) 7), (short) 0)); @Test public void testFromVersion() { assertEquals(new BootstrapMetadata(Collections.singletonList( new ApiMessageAndVersion(new FeatureLevelRecord(). setName(FEATURE_NAME). - setFeatureLevel((short) 6), (short) 0)), - IBP_3_3_IV2, "foo"), - BootstrapMetadata.fromVersion(IBP_3_3_IV2, "foo")); + setFeatureLevel((short) 7), (short) 0)), + IBP_3_3_IV3, "foo"), + BootstrapMetadata.fromVersion(IBP_3_3_IV3, "foo")); } @Test public void testFromRecordsList() { - assertEquals(new BootstrapMetadata(SAMPLE_RECORDS1, IBP_3_3_IV2, "bar"), + assertEquals(new BootstrapMetadata(SAMPLE_RECORDS1, IBP_3_3_IV3, "bar"), BootstrapMetadata.fromRecords(SAMPLE_RECORDS1, "bar")); } @@ -126,13 +127,14 @@ public class BootstrapMetadataTest { static final List RECORDS_WITH_OLD_METADATA_VERSION = Collections.singletonList( new ApiMessageAndVersion(new FeatureLevelRecord(). setName(FEATURE_NAME). - setFeatureLevel(IBP_3_0_IV1.featureLevel()), (short) 0)); + setFeatureLevel(MetadataVersionTestUtils.IBP_3_0_IV1_FEATURE_LEVEL), (short) 0)); @Test public void testFromRecordsListWithOldMetadataVersion() { RuntimeException exception = assertThrows(RuntimeException.class, () -> BootstrapMetadata.fromRecords(RECORDS_WITH_OLD_METADATA_VERSION, "quux")); - assertEquals("Bootstrap metadata.version before 3.3-IV0 are not supported. Can't load " + - "metadata from quux", exception.getMessage()); + assertEquals("No MetadataVersion with feature level 1. Valid feature levels are from " + MetadataVersion.MINIMUM_VERSION.featureLevel() + + " to " + MetadataVersion.latestTesting().featureLevel() + ".", + exception.getMessage()); } } diff --git a/metadata/src/test/java/org/apache/kafka/metadata/storage/FormatterTest.java b/metadata/src/test/java/org/apache/kafka/metadata/storage/FormatterTest.java index ae34a55f30f..d6ee95c8ccf 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/storage/FormatterTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/storage/FormatterTest.java @@ -144,7 +144,7 @@ public class FormatterTest { assertEquals(Optional.of(DEFAULT_CLUSTER_ID.toString()), ensemble.clusterId()); assertEquals(new HashSet<>(testEnv.directories), ensemble.logDirProps().keySet()); BootstrapMetadata bootstrapMetadata = - new BootstrapDirectory(testEnv.directory(0), Optional.empty()).read(); + new BootstrapDirectory(testEnv.directory(0)).read(); assertEquals(MetadataVersion.latestProduction(), bootstrapMetadata.metadataVersion()); } } @@ -225,7 +225,7 @@ public class FormatterTest { " with metadata.version " + MetadataVersion.IBP_3_5_IV0 + ".", formatter1.output().trim()); BootstrapMetadata bootstrapMetadata = - new BootstrapDirectory(testEnv.directory(0), Optional.empty()).read(); + new BootstrapDirectory(testEnv.directory(0)).read(); assertEquals(MetadataVersion.IBP_3_5_IV0, bootstrapMetadata.metadataVersion()); assertEquals(1, bootstrapMetadata.records().size()); } @@ -252,7 +252,7 @@ public class FormatterTest { " with metadata.version " + MetadataVersion.latestTesting() + ".", formatter1.output().trim()); BootstrapMetadata bootstrapMetadata = - new BootstrapDirectory(testEnv.directory(0), Optional.empty()).read(); + new BootstrapDirectory(testEnv.directory(0)).read(); assertEquals(MetadataVersion.latestTesting(), bootstrapMetadata.metadataVersion()); } } @@ -302,7 +302,7 @@ public class FormatterTest { " with metadata.version " + MetadataVersion.IBP_3_8_IV0 + ".", formatter1.output().trim()); BootstrapMetadata bootstrapMetadata = - new BootstrapDirectory(testEnv.directory(0), Optional.empty()).read(); + new BootstrapDirectory(testEnv.directory(0)).read(); assertEquals(MetadataVersion.IBP_3_8_IV0, bootstrapMetadata.metadataVersion()); List scramRecords = bootstrapMetadata.records().stream(). filter(r -> r.message() instanceof UserScramCredentialRecord). @@ -337,7 +337,7 @@ public class FormatterTest { formatter1.formatter.setFeatureLevel(TestFeatureVersion.FEATURE_NAME, version); formatter1.formatter.run(); BootstrapMetadata bootstrapMetadata = - new BootstrapDirectory(testEnv.directory(0), Optional.empty()).read(); + new BootstrapDirectory(testEnv.directory(0)).read(); List expected = new ArrayList<>(); expected.add(new ApiMessageAndVersion(new FeatureLevelRecord(). setName(MetadataVersion.FEATURE_NAME). diff --git a/server-common/src/main/java/org/apache/kafka/server/common/EligibleLeaderReplicasVersion.java b/server-common/src/main/java/org/apache/kafka/server/common/EligibleLeaderReplicasVersion.java index ae3dab0fb31..b9049b636f9 100644 --- a/server-common/src/main/java/org/apache/kafka/server/common/EligibleLeaderReplicasVersion.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/EligibleLeaderReplicasVersion.java @@ -22,7 +22,7 @@ import java.util.Map; public enum EligibleLeaderReplicasVersion implements FeatureVersion { // Version 0 is the version disable ELR. - ELRV_0(0, MetadataVersion.MINIMUM_KRAFT_VERSION, Collections.emptyMap()), + ELRV_0(0, MetadataVersion.MINIMUM_VERSION, Collections.emptyMap()), // Version 1 enables the ELR (KIP-966). ELRV_1(1, MetadataVersion.IBP_4_0_IV1, Collections.emptyMap()); diff --git a/server-common/src/main/java/org/apache/kafka/server/common/Feature.java b/server-common/src/main/java/org/apache/kafka/server/common/Feature.java index 8a812fe521d..af71bc63080 100644 --- a/server-common/src/main/java/org/apache/kafka/server/common/Feature.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/Feature.java @@ -155,8 +155,6 @@ public enum Feature { * For example, say feature X level x relies on feature Y level y: * if feature X >= x then throw an error if feature Y < y. * - * All feature levels above 0 in kraft require metadata.version=4 (IBP_3_3_IV0) in order to write the feature records to the cluster. - * * @param feature the feature we are validating * @param features the feature versions we have (or want to set) * @throws IllegalArgumentException if the feature is not valid @@ -164,9 +162,9 @@ public enum Feature { public static void validateVersion(FeatureVersion feature, Map features) { Short metadataVersion = features.get(MetadataVersion.FEATURE_NAME); - if (feature.featureLevel() >= 1 && (metadataVersion == null || metadataVersion < MetadataVersion.IBP_3_3_IV0.featureLevel())) + if (feature.featureLevel() >= 1 && (metadataVersion == null || metadataVersion < MetadataVersion.MINIMUM_VERSION.featureLevel())) throw new IllegalArgumentException(feature.featureName() + " could not be set to " + feature.featureLevel() + - " because it depends on metadata.version=4 (" + MetadataVersion.IBP_3_3_IV0 + ")"); + " because it depends on metadata.version=" + MetadataVersion.MINIMUM_VERSION.featureLevel() + " (" + MetadataVersion.MINIMUM_VERSION + ")"); for (Map.Entry dependency: feature.dependencies().entrySet()) { Short featureLevel = features.get(dependency.getKey()); @@ -297,11 +295,6 @@ public enum Feature { } for (MetadataVersion metadataVersion: MetadataVersion.values()) { - // Only checking the kraft metadata versions. - if (metadataVersion.compareTo(MetadataVersion.MINIMUM_KRAFT_VERSION) < 0) { - continue; - } - defaultVersion = feature.defaultVersion(metadataVersion); for (Map.Entry dependency: defaultVersion.dependencies().entrySet()) { String dependencyFeatureName = dependency.getKey(); diff --git a/server-common/src/main/java/org/apache/kafka/server/common/GroupVersion.java b/server-common/src/main/java/org/apache/kafka/server/common/GroupVersion.java index 881031e6ecf..92976fac77d 100644 --- a/server-common/src/main/java/org/apache/kafka/server/common/GroupVersion.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/GroupVersion.java @@ -22,7 +22,7 @@ import java.util.Map; public enum GroupVersion implements FeatureVersion { // Version 0 is the original group coordinator prior to KIP-848. - GV_0(0, MetadataVersion.MINIMUM_KRAFT_VERSION, Collections.emptyMap()), + GV_0(0, MetadataVersion.MINIMUM_VERSION, Collections.emptyMap()), // Version 1 enables the consumer rebalance protocol (KIP-848). GV_1(1, MetadataVersion.IBP_4_0_IV0, Collections.emptyMap()); diff --git a/server-common/src/main/java/org/apache/kafka/server/common/KRaftVersion.java b/server-common/src/main/java/org/apache/kafka/server/common/KRaftVersion.java index 734b515b5a8..211f6dcac44 100644 --- a/server-common/src/main/java/org/apache/kafka/server/common/KRaftVersion.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/KRaftVersion.java @@ -21,7 +21,7 @@ import java.util.Map; public enum KRaftVersion implements FeatureVersion { // Version 0 is the initial version of KRaft. - KRAFT_VERSION_0(0, MetadataVersion.MINIMUM_KRAFT_VERSION), + KRAFT_VERSION_0(0, MetadataVersion.MINIMUM_VERSION), // Version 1 enables KIP-853. KRAFT_VERSION_1(1, MetadataVersion.IBP_3_9_IV0); diff --git a/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java b/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java index 631e6cbea2d..9207d08e3e7 100644 --- a/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java @@ -32,37 +32,18 @@ import java.util.regex.Pattern; * We consider a version newer than another if it is lower in the enum list (to avoid depending on lexicographic order) *
* Since the api protocol may change more than once within the same release and to facilitate people deploying code from - * trunk, we have the concept of internal versions (first introduced during the 0.10.0 development cycle). For example, - * the first time we introduce a version change in a release, say 0.10.0, we will add a config value "0.10.0-IV0" and a - * corresponding enum constant IBP_0_10_0-IV0. We will also add a config value "0.10.0" that will be mapped to the - * latest internal version object, which is IBP_0_10_0-IV0. When we change the protocol a second time while developing - * 0.10.0, we will add a new config value "0.10.0-IV1" and a corresponding enum constant IBP_0_10_0-IV1. We will change - * the config value "0.10.0" to map to the latest internal version IBP_0_10_0-IV1. The config value of - * "0.10.0-IV0" is still mapped to IBP_0_10_0-IV0. This way, if people are deploying from trunk, they can use - * "0.10.0-IV0" and "0.10.0-IV1" to upgrade one internal version at a time. For most people who just want to use - * released version, they can use "0.10.0" when upgrading to the 0.10.0 release. + * trunk, we have the concept of internal versions (first introduced during the 1.0 development cycle). For example, + * the first time we introduce a version change in a release, say 1.0, we will add a config value "1.0-IV0" and a + * corresponding enum constant IBP_1_0-IV0. We will also add a config value "1.0" that will be mapped to the + * latest internal version object, which is IBP_1_0-IV0. When we change the protocol a second time while developing + * 1.0, we will add a new config value "1.0-IV1" and a corresponding enum constant IBP_1_0-IV1. We will change + * the config value "1.0" to map to the latest internal version IBP_1_0-IV1. The config value of + * "1.0-IV0" is still mapped to IBP_1_0-IV0. This way, if people are deploying from trunk, they can use + * "1.0-IV0" and "1.0-IV1" to upgrade one internal version at a time. For most people who just want to use + * released version, they can use "1.0" when upgrading to the 1.0 release. */ public enum MetadataVersion { - // Introduce ListOffsets V7 which supports listing offsets by max timestamp (KIP-734) - // Assume message format version is 3.0 (KIP-724) - IBP_3_0_IV1(1, "3.0", "IV1", true), - - // Adds topic IDs to Fetch requests/responses (KIP-516) - IBP_3_1_IV0(2, "3.1", "IV0", false), - - // Support for leader recovery for unclean leader election (KIP-704) - IBP_3_2_IV0(3, "3.2", "IV0", true), - - // Support for metadata.version feature flag and Removes min_version_level from the finalized version range that is written to ZooKeeper (KIP-778) - IBP_3_3_IV0(4, "3.3", "IV0", false), - - // Support NoopRecord for the cluster metadata log (KIP-835) - IBP_3_3_IV1(5, "3.3", "IV1", true), - - // In KRaft mode, use BrokerRegistrationChangeRecord instead of UnfenceBrokerRecord and FenceBrokerRecord. - IBP_3_3_IV2(6, "3.3", "IV2", true), - // Adds InControlledShutdown state to RegisterBrokerRecord and BrokerRegistrationChangeRecord (KIP-841). IBP_3_3_IV3(7, "3.3", "IV3", true), @@ -139,15 +120,9 @@ public enum MetadataVersion { public static final String FEATURE_NAME = "metadata.version"; /** - * The first version we currently support in KRaft. + * Minimum supported version. */ - public static final MetadataVersion MINIMUM_KRAFT_VERSION = IBP_3_0_IV1; - - /** - * The first version we currently support in the bootstrap metadata. We chose 3.3IV0 since it - * is the first version that supports storing the metadata.version in the log. - */ - public static final MetadataVersion MINIMUM_BOOTSTRAP_VERSION = IBP_3_3_IV0; + public static final MetadataVersion MINIMUM_VERSION = IBP_3_3_IV3; /** * The latest production-ready MetadataVersion. This is the latest version that is stable @@ -193,14 +168,6 @@ public enum MetadataVersion { return featureLevel; } - public boolean isLeaderRecoverySupported() { - return this.isAtLeast(IBP_3_2_IV0); - } - - public boolean isNoOpRecordSupported() { - return this.isAtLeast(IBP_3_3_IV1); - } - public boolean isScramSupported() { return this.isAtLeast(IBP_3_5_IV2); } @@ -225,18 +192,6 @@ public enum MetadataVersion { return this.isAtLeast(IBP_4_0_IV1); } - public boolean isKRaftSupported() { - return this.featureLevel > 0; - } - - public boolean isBrokerRegistrationChangeRecordSupported() { - return this.isAtLeast(IBP_3_3_IV2); - } - - public boolean isInControlledShutdownStateSupported() { - return this.isAtLeast(IBP_3_3_IV3); - } - public boolean isMigrationSupported() { return this.isAtLeast(MetadataVersion.IBP_3_4_IV0); } @@ -248,10 +203,8 @@ public enum MetadataVersion { } else if (isMigrationSupported()) { // new isMigrationZkBroker field return (short) 2; - } else if (isInControlledShutdownStateSupported()) { - return (short) 1; } else { - return (short) 0; + return (short) 1; } } @@ -297,10 +250,9 @@ public enum MetadataVersion { return 15; } else if (this.isAtLeast(IBP_3_5_IV0)) { return 14; - } else if (this.isAtLeast(IBP_3_1_IV0)) { + } else { return 13; } - return 12; } public short listOffsetRequestVersion() { @@ -358,14 +310,14 @@ public enum MetadataVersion { } /** - * Return an `MetadataVersion` instance for `versionString`, which can be in a variety of formats (e.g. "0.8.0", "0.8.0.x", - * "0.10.0", "0.10.0-IV1"). `IllegalArgumentException` is thrown if `versionString` cannot be mapped to an `MetadataVersion`. - * Note that 'misconfigured' values such as "1.0.1" will be parsed to `IBP_1_0_IV0` as we ignore anything after the first - * two digits for versions that don't start with "0." + * Return an `MetadataVersion` instance for `versionString`, which can be in a variety of formats (e.g. "3.8", "3.8.x", + * "3.8.0", "3.8-IV0"). `IllegalArgumentException` is thrown if `versionString` cannot be mapped to an `MetadataVersion`. + * Note that 'misconfigured' values such as "3.8.1" will be parsed to `IBP_3_8_IV0` as we ignore anything after the first + * two segments. */ public static MetadataVersion fromVersionString(String versionString) { String[] versionSegments = versionString.split(Pattern.quote(".")); - int numSegments = (versionString.startsWith("0.")) ? 3 : 2; + int numSegments = 2; String key; if (numSegments >= versionSegments.length) { key = versionString; @@ -373,7 +325,8 @@ public enum MetadataVersion { key = String.join(".", Arrays.copyOfRange(versionSegments, 0, numSegments)); } return Optional.ofNullable(IBP_VERSIONS.get(key)).orElseThrow(() -> - new IllegalArgumentException("Version " + versionString + " is not a valid version") + new IllegalArgumentException("Version " + versionString + " is not a valid version. The minimum version is " + MINIMUM_VERSION + + " and the maximum version is " + latestTesting()) ); } @@ -383,7 +336,8 @@ public enum MetadataVersion { return metadataVersion; } } - throw new IllegalArgumentException("No MetadataVersion with feature level " + version); + throw new IllegalArgumentException("No MetadataVersion with feature level " + version + ". Valid feature levels are from " + + MINIMUM_VERSION.featureLevel + " to " + latestTesting().featureLevel + "."); } // Testing only diff --git a/server-common/src/main/java/org/apache/kafka/server/common/TestFeatureVersion.java b/server-common/src/main/java/org/apache/kafka/server/common/TestFeatureVersion.java index e9d54d0f211..032f5995f5e 100644 --- a/server-common/src/main/java/org/apache/kafka/server/common/TestFeatureVersion.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/TestFeatureVersion.java @@ -20,7 +20,7 @@ import java.util.Collections; import java.util.Map; public enum TestFeatureVersion implements FeatureVersion { - TEST_0(0, MetadataVersion.MINIMUM_KRAFT_VERSION, Collections.emptyMap()), + TEST_0(0, MetadataVersion.MINIMUM_VERSION, Collections.emptyMap()), // TEST_1 released right before MV 3.7-IVO was released, and it has no dependencies TEST_1(1, MetadataVersion.IBP_3_7_IV0, Collections.emptyMap()), // TEST_2 is not yet set to be the default version and maps to the latest testing version, and it depends on this metadata version diff --git a/server-common/src/main/java/org/apache/kafka/server/common/TransactionVersion.java b/server-common/src/main/java/org/apache/kafka/server/common/TransactionVersion.java index fc85f55606f..05bbd821801 100644 --- a/server-common/src/main/java/org/apache/kafka/server/common/TransactionVersion.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/TransactionVersion.java @@ -25,7 +25,7 @@ import java.util.Map; public enum TransactionVersion implements FeatureVersion { // Version 0 is the original transaction coordinator with no extra features enabled. - TV_0(0, MetadataVersion.MINIMUM_KRAFT_VERSION, Collections.emptyMap()), + TV_0(0, MetadataVersion.MINIMUM_VERSION, Collections.emptyMap()), // Version 1 enables flexible transactional state records. (KIP-890) TV_1(1, MetadataVersion.IBP_4_0_IV2, Collections.emptyMap()), // Version 2 enables epoch bump per transaction and optimizations. (KIP-890) diff --git a/server-common/src/main/java/org/apache/kafka/server/common/UnitTestFeatureVersion.java b/server-common/src/main/java/org/apache/kafka/server/common/UnitTestFeatureVersion.java index ea107998e76..6aaa8ae7456 100644 --- a/server-common/src/main/java/org/apache/kafka/server/common/UnitTestFeatureVersion.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/UnitTestFeatureVersion.java @@ -27,7 +27,7 @@ public class UnitTestFeatureVersion { * The feature is used for testing latest production is not one of the feature versions. */ public enum FV0 implements FeatureVersion { - UT_FV0_0(0, MetadataVersion.MINIMUM_KRAFT_VERSION, Collections.emptyMap()), + UT_FV0_0(0, MetadataVersion.MINIMUM_VERSION, Collections.emptyMap()), UT_FV0_1(1, MetadataVersion.IBP_3_7_IV0, Collections.emptyMap()); private final short featureLevel; @@ -69,7 +69,7 @@ public class UnitTestFeatureVersion { * The feature is used to test latest production lags behind the default value. */ public enum FV1 implements FeatureVersion { - UT_FV1_0(0, MetadataVersion.MINIMUM_KRAFT_VERSION, Collections.emptyMap()), + UT_FV1_0(0, MetadataVersion.MINIMUM_VERSION, Collections.emptyMap()), UT_FV1_1(1, MetadataVersion.IBP_3_7_IV0, Collections.emptyMap()); private final short featureLevel; @@ -111,7 +111,7 @@ public class UnitTestFeatureVersion { * The feature is used to test the dependency of the latest production that is not yet production ready. */ public enum FV2 implements FeatureVersion { - UT_FV2_0(0, MetadataVersion.MINIMUM_KRAFT_VERSION, Collections.emptyMap()), + UT_FV2_0(0, MetadataVersion.MINIMUM_VERSION, Collections.emptyMap()), UT_FV2_1(1, MetadataVersion.IBP_3_7_IV0, Collections.emptyMap()); private final short featureLevel; @@ -153,7 +153,7 @@ public class UnitTestFeatureVersion { * The feature is used to test the dependency of the latest production that is not yet production ready. */ public enum FV3 implements FeatureVersion { - UT_FV3_0(0, MetadataVersion.MINIMUM_KRAFT_VERSION, Collections.emptyMap()), + UT_FV3_0(0, MetadataVersion.MINIMUM_VERSION, Collections.emptyMap()), UT_FV3_1(1, MetadataVersion.IBP_3_7_IV0, Collections.singletonMap(FV2.FEATURE_NAME, (short) 1)); private final short featureLevel; @@ -195,7 +195,7 @@ public class UnitTestFeatureVersion { * The feature is used to test the dependency of the default value that is not yet default ready. */ public enum FV4 implements FeatureVersion { - UT_FV4_0(0, MetadataVersion.MINIMUM_KRAFT_VERSION, Collections.emptyMap()), + UT_FV4_0(0, MetadataVersion.MINIMUM_VERSION, Collections.emptyMap()), UT_FV4_1(1, MetadataVersion.latestTesting(), Collections.emptyMap()); private final short featureLevel; @@ -237,7 +237,7 @@ public class UnitTestFeatureVersion { * The feature is used to test the dependency of the default value that is not yet default ready. */ public enum FV5 implements FeatureVersion { - UT_FV5_0(0, MetadataVersion.MINIMUM_KRAFT_VERSION, Collections.emptyMap()), + UT_FV5_0(0, MetadataVersion.MINIMUM_VERSION, Collections.emptyMap()), UT_FV5_1(1, MetadataVersion.IBP_3_7_IV0, Collections.singletonMap(FV4.FEATURE_NAME, (short) 1)); private final short featureLevel; @@ -279,7 +279,7 @@ public class UnitTestFeatureVersion { * The feature is used to test the latest production has MV dependency that is not yet production ready. */ public enum FV6 implements FeatureVersion { - UT_FV6_0(0, MetadataVersion.MINIMUM_KRAFT_VERSION, Collections.emptyMap()), + UT_FV6_0(0, MetadataVersion.MINIMUM_VERSION, Collections.emptyMap()), UT_FV6_1(1, MetadataVersion.latestTesting(), Collections.singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.latestTesting().featureLevel())); private final short featureLevel; @@ -321,7 +321,7 @@ public class UnitTestFeatureVersion { * The feature is used to test the default value has MV dependency that is behind the bootstrap MV. */ public enum FV7 implements FeatureVersion { - UT_FV7_0(0, MetadataVersion.MINIMUM_KRAFT_VERSION, Collections.singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_7_IV0.featureLevel())), + UT_FV7_0(0, MetadataVersion.MINIMUM_VERSION, Collections.singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_7_IV0.featureLevel())), UT_FV7_1(1, MetadataVersion.IBP_3_8_IV0, Collections.singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_8_IV0.featureLevel())); private final short featureLevel; diff --git a/server-common/src/test/java/org/apache/kafka/server/common/FeatureTest.java b/server-common/src/test/java/org/apache/kafka/server/common/FeatureTest.java index 31e842435e3..1e50a8b58c4 100644 --- a/server-common/src/test/java/org/apache/kafka/server/common/FeatureTest.java +++ b/server-common/src/test/java/org/apache/kafka/server/common/FeatureTest.java @@ -42,7 +42,7 @@ public class FeatureTest { "UNIT_TEST_VERSION_7"}, mode = EnumSource.Mode.EXCLUDE) public void testV0SupportedInEarliestMV(Feature feature) { assertTrue(feature.featureVersions().length >= 1); - assertEquals(MetadataVersion.MINIMUM_KRAFT_VERSION, + assertEquals(MetadataVersion.MINIMUM_VERSION, feature.featureVersions()[0].bootstrapMetadataVersion()); } @@ -88,7 +88,7 @@ public class FeatureTest { Map deps = new HashMap<>(); deps.putAll(featureImpl.dependencies()); if (!deps.containsKey(MetadataVersion.FEATURE_NAME)) { - deps.put(MetadataVersion.FEATURE_NAME, MetadataVersion.MINIMUM_BOOTSTRAP_VERSION.featureLevel()); + deps.put(MetadataVersion.FEATURE_NAME, MetadataVersion.MINIMUM_VERSION.featureLevel()); } // Ensure that the feature is valid given the typical metadataVersionMapping and the dependencies. @@ -111,7 +111,7 @@ public class FeatureTest { assertThrows(IllegalArgumentException.class, () -> Feature.validateVersion( TestFeatureVersion.TEST_1, - Collections.singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_0_IV1.featureLevel()) + Collections.singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersionTestUtils.IBP_3_0_IV1_FEATURE_LEVEL) ) ); @@ -276,18 +276,19 @@ public class FeatureTest { @Test public void testValidateWithMVDependencyNotProductionReady() { if (MetadataVersion.latestProduction().isLessThan(MetadataVersion.latestTesting())) { - assertThrows(IllegalArgumentException.class, () -> - validateDefaultValueAndLatestProductionValue(Feature.UNIT_TEST_VERSION_6), - "Feature UNIT_TEST_VERSION_6 has latest production FeatureVersion UT_FV6_1 with " + - "MV dependency 4.0-IV3 that is not production ready. (MV latest production: 4.0-IV0)"); + IllegalArgumentException exception = assertThrows(IllegalArgumentException.class, () -> + validateDefaultValueAndLatestProductionValue(Feature.UNIT_TEST_VERSION_6)); + assertEquals("Feature UNIT_TEST_VERSION_6 has latest production FeatureVersion UT_FV6_1 with MV dependency " + + MetadataVersion.latestTesting() + " that is not production ready. (MV latest production: " + MetadataVersion.latestProduction() + ")", + exception.getMessage()); } } @Test public void testValidateWithMVDependencyAheadOfBootstrapMV() { - assertThrows(IllegalArgumentException.class, () -> - validateDefaultValueAndLatestProductionValue(Feature.UNIT_TEST_VERSION_7), - "Feature UNIT_TEST_VERSION_7 has default FeatureVersion UT_FV7_0 when MV=3.0-IV1 with " + - "MV dependency 3.7-IV0 that is behind its bootstrap MV 3.0-IV1."); + IllegalArgumentException exception = assertThrows(IllegalArgumentException.class, () -> + validateDefaultValueAndLatestProductionValue(Feature.UNIT_TEST_VERSION_7)); + assertEquals("Feature UNIT_TEST_VERSION_7 has default FeatureVersion UT_FV7_0 when MV=" + MetadataVersion.MINIMUM_VERSION + + " with MV dependency 3.7-IV0 that is behind its bootstrap MV " + MetadataVersion.MINIMUM_VERSION + ".", exception.getMessage()); } } diff --git a/server-common/src/test/java/org/apache/kafka/server/common/FinalizedFeaturesTest.java b/server-common/src/test/java/org/apache/kafka/server/common/FinalizedFeaturesTest.java index 31d57bedfe5..e0021155135 100644 --- a/server-common/src/test/java/org/apache/kafka/server/common/FinalizedFeaturesTest.java +++ b/server-common/src/test/java/org/apache/kafka/server/common/FinalizedFeaturesTest.java @@ -22,15 +22,15 @@ import org.junit.jupiter.api.Test; import java.util.Collections; import static org.apache.kafka.server.common.MetadataVersion.FEATURE_NAME; -import static org.apache.kafka.server.common.MetadataVersion.MINIMUM_KRAFT_VERSION; +import static org.apache.kafka.server.common.MetadataVersion.MINIMUM_VERSION; import static org.junit.jupiter.api.Assertions.assertEquals; class FinalizedFeaturesTest { @Test public void testKRaftModeFeatures() { - FinalizedFeatures finalizedFeatures = new FinalizedFeatures(MINIMUM_KRAFT_VERSION, + FinalizedFeatures finalizedFeatures = new FinalizedFeatures(MINIMUM_VERSION, Collections.singletonMap("foo", (short) 2), 123); - assertEquals(MINIMUM_KRAFT_VERSION.featureLevel(), + assertEquals(MINIMUM_VERSION.featureLevel(), finalizedFeatures.finalizedFeatures().get(FEATURE_NAME)); assertEquals((short) 2, finalizedFeatures.finalizedFeatures().get("foo")); diff --git a/server-common/src/test/java/org/apache/kafka/server/common/KRaftVersionTest.java b/server-common/src/test/java/org/apache/kafka/server/common/KRaftVersionTest.java index d8309be01b5..f850d370322 100644 --- a/server-common/src/test/java/org/apache/kafka/server/common/KRaftVersionTest.java +++ b/server-common/src/test/java/org/apache/kafka/server/common/KRaftVersionTest.java @@ -52,7 +52,7 @@ public final class KRaftVersionTest { MetadataVersion metadataVersion = KRaftVersion.values()[i].bootstrapMetadataVersion(); switch (i) { case 0: - assertEquals(MetadataVersion.MINIMUM_KRAFT_VERSION, metadataVersion); + assertEquals(MetadataVersion.MINIMUM_VERSION, metadataVersion); break; case 1: assertEquals(MetadataVersion.IBP_3_9_IV0, metadataVersion); diff --git a/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java b/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java index 09f8564fc95..0414f7cd1cc 100644 --- a/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java +++ b/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java @@ -29,17 +29,11 @@ import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; class MetadataVersionTest { - @Test - public void testKRaftFeatureLevelsBefore3_0_IV1() { - for (int i = 0; i < MetadataVersion.IBP_3_0_IV1.ordinal(); i++) { - assertEquals(-1, MetadataVersion.VERSIONS[i].featureLevel()); - } - } @Test - public void testKRaftFeatureLevelsAtAndAfter3_0_IV1() { - for (int i = MetadataVersion.IBP_3_0_IV1.ordinal(); i < MetadataVersion.VERSIONS.length; i++) { - int expectedLevel = i - MetadataVersion.IBP_3_0_IV1.ordinal() + 1; + public void testFeatureLevels() { + for (int i = MINIMUM_VERSION.ordinal(); i < MetadataVersion.VERSIONS.length; i++) { + int expectedLevel = i + MINIMUM_VERSION.featureLevel(); assertEquals(expectedLevel, MetadataVersion.VERSIONS[i].featureLevel()); } } @@ -47,20 +41,8 @@ class MetadataVersionTest { @Test @SuppressWarnings("checkstyle:JavaNCSS") public void testFromVersionString() { - assertEquals(IBP_3_0_IV1, MetadataVersion.fromVersionString("3.0")); - assertEquals(IBP_3_0_IV1, MetadataVersion.fromVersionString("3.0-IV1")); - - assertEquals(IBP_3_1_IV0, MetadataVersion.fromVersionString("3.1")); - assertEquals(IBP_3_1_IV0, MetadataVersion.fromVersionString("3.1-IV0")); - - assertEquals(IBP_3_2_IV0, MetadataVersion.fromVersionString("3.2")); - assertEquals(IBP_3_2_IV0, MetadataVersion.fromVersionString("3.2-IV0")); - // 3.3-IV3 is the latest production version in the 3.3 line assertEquals(IBP_3_3_IV3, MetadataVersion.fromVersionString("3.3")); - assertEquals(IBP_3_3_IV0, MetadataVersion.fromVersionString("3.3-IV0")); - assertEquals(IBP_3_3_IV1, MetadataVersion.fromVersionString("3.3-IV1")); - assertEquals(IBP_3_3_IV2, MetadataVersion.fromVersionString("3.3-IV2")); assertEquals(IBP_3_3_IV3, MetadataVersion.fromVersionString("3.3-IV3")); // 3.4-IV0 is the latest production version in the 3.4 line @@ -106,12 +88,6 @@ class MetadataVersionTest { @Test public void testShortVersion() { - assertEquals("3.0", IBP_3_0_IV1.shortVersion()); - assertEquals("3.1", IBP_3_1_IV0.shortVersion()); - assertEquals("3.2", IBP_3_2_IV0.shortVersion()); - assertEquals("3.3", IBP_3_3_IV0.shortVersion()); - assertEquals("3.3", IBP_3_3_IV1.shortVersion()); - assertEquals("3.3", IBP_3_3_IV2.shortVersion()); assertEquals("3.3", IBP_3_3_IV3.shortVersion()); assertEquals("3.4", IBP_3_4_IV0.shortVersion()); assertEquals("3.5", IBP_3_5_IV0.shortVersion()); @@ -136,12 +112,6 @@ class MetadataVersionTest { @Test public void testVersion() { - assertEquals("3.0-IV1", IBP_3_0_IV1.version()); - assertEquals("3.1-IV0", IBP_3_1_IV0.version()); - assertEquals("3.2-IV0", IBP_3_2_IV0.version()); - assertEquals("3.3-IV0", IBP_3_3_IV0.version()); - assertEquals("3.3-IV1", IBP_3_3_IV1.version()); - assertEquals("3.3-IV2", IBP_3_3_IV2.version()); assertEquals("3.3-IV3", IBP_3_3_IV3.version()); assertEquals("3.4-IV0", IBP_3_4_IV0.version()); assertEquals("3.5-IV0", IBP_3_5_IV0.version()); @@ -175,41 +145,17 @@ class MetadataVersionTest { @Test public void testMetadataChanged() { - assertFalse(MetadataVersion.checkIfMetadataChanged(IBP_3_2_IV0, IBP_3_2_IV0)); - assertTrue(MetadataVersion.checkIfMetadataChanged(IBP_3_2_IV0, IBP_3_1_IV0)); - assertTrue(MetadataVersion.checkIfMetadataChanged(IBP_3_2_IV0, IBP_3_0_IV1)); - assertTrue(MetadataVersion.checkIfMetadataChanged(IBP_3_2_IV0, IBP_3_0_IV1)); - assertTrue(MetadataVersion.checkIfMetadataChanged(IBP_3_3_IV1, IBP_3_3_IV0)); + assertTrue(MetadataVersion.checkIfMetadataChanged(IBP_3_4_IV0, IBP_3_3_IV3)); + assertFalse(MetadataVersion.checkIfMetadataChanged(IBP_3_5_IV0, IBP_3_4_IV0)); + assertFalse(MetadataVersion.checkIfMetadataChanged(IBP_3_5_IV1, IBP_3_5_IV0)); + assertTrue(MetadataVersion.checkIfMetadataChanged(IBP_3_6_IV0, IBP_3_5_IV1)); + assertTrue(MetadataVersion.checkIfMetadataChanged(IBP_3_6_IV1, IBP_3_6_IV0)); // Check that argument order doesn't matter - assertTrue(MetadataVersion.checkIfMetadataChanged(IBP_3_1_IV0, IBP_3_2_IV0)); - assertTrue(MetadataVersion.checkIfMetadataChanged(IBP_3_0_IV1, IBP_3_2_IV0)); - } - - @Test - public void testKRaftVersions() { - for (MetadataVersion metadataVersion : MetadataVersion.VERSIONS) { - if (metadataVersion.isKRaftSupported()) { - assertTrue(metadataVersion.featureLevel() > 0); - } else { - assertEquals(-1, metadataVersion.featureLevel()); - } - } - - for (MetadataVersion metadataVersion : MetadataVersion.VERSIONS) { - if (metadataVersion.isAtLeast(IBP_3_0_IV1)) { - assertTrue(metadataVersion.isKRaftSupported(), metadataVersion.toString()); - } else { - assertFalse(metadataVersion.isKRaftSupported()); - } - } - } - - @ParameterizedTest - @EnumSource(value = MetadataVersion.class) - public void testIsInControlledShutdownStateSupported(MetadataVersion metadataVersion) { - assertEquals(metadataVersion.isAtLeast(IBP_3_3_IV3), - metadataVersion.isInControlledShutdownStateSupported()); + assertTrue(MetadataVersion.checkIfMetadataChanged(IBP_3_3_IV3, IBP_3_4_IV0)); + assertFalse(MetadataVersion.checkIfMetadataChanged(IBP_3_4_IV0, IBP_3_5_IV0)); + assertFalse(MetadataVersion.checkIfMetadataChanged(IBP_3_5_IV0, IBP_3_5_IV1)); + assertTrue(MetadataVersion.checkIfMetadataChanged(IBP_3_5_IV1, IBP_3_6_IV0)); } @ParameterizedTest @@ -284,7 +230,7 @@ class MetadataVersionTest { /** * We need to ensure that the latest production MV doesn't inadvertently rely on an unstable - * request version. Currently, the broker selects the version for some inter-broker RPCs based on the MV + * request version. Currently, the broker selects the version for some inter-broker RPCs based on the MV * rather than using the supported version from the ApiResponse. */ @Test diff --git a/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTestUtils.java b/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTestUtils.java new file mode 100644 index 00000000000..73027f0bd33 --- /dev/null +++ b/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTestUtils.java @@ -0,0 +1,25 @@ +/* + * 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.server.common; + +public class MetadataVersionTestUtils { + // References to feature levels that are no longer supported in production, but still useful in tests + public static final short IBP_3_0_IV1_FEATURE_LEVEL = 1; + public static final short IBP_3_3_IV0_FEATURE_LEVEL = 4; + public static final short IBP_3_3_IV2_FEATURE_LEVEL = 6; +} diff --git a/server/src/main/java/org/apache/kafka/server/BrokerFeatures.java b/server/src/main/java/org/apache/kafka/server/BrokerFeatures.java index 197ebea427a..f611411f9b8 100644 --- a/server/src/main/java/org/apache/kafka/server/BrokerFeatures.java +++ b/server/src/main/java/org/apache/kafka/server/BrokerFeatures.java @@ -68,7 +68,7 @@ public class BrokerFeatures { Map features = new HashMap<>(); features.put(MetadataVersion.FEATURE_NAME, new SupportedVersionRange( - MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel(), + MetadataVersion.MINIMUM_VERSION.featureLevel(), unstableFeatureVersionsEnabled ? MetadataVersion.latestTesting().featureLevel() : MetadataVersion.latestProduction().featureLevel())); PRODUCTION_FEATURES.forEach(feature -> { diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index a74a10a2219..46e5f69fc67 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -31,7 +31,6 @@ import org.apache.kafka.common.record.Records; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.utils.ConfigUtils; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.server.common.MetadataVersion; import org.apache.kafka.server.config.QuotaConfig; import org.apache.kafka.server.config.ServerLogConfigs; import org.apache.kafka.server.config.ServerTopicConfigSynonyms; @@ -63,7 +62,6 @@ import static org.apache.kafka.common.config.ConfigDef.Type.LIST; import static org.apache.kafka.common.config.ConfigDef.Type.LONG; import static org.apache.kafka.common.config.ConfigDef.Type.STRING; import static org.apache.kafka.common.config.ConfigDef.ValidString.in; -import static org.apache.kafka.server.common.MetadataVersion.IBP_3_0_IV1; public class LogConfig extends AbstractConfig { @@ -444,10 +442,6 @@ public class LogConfig extends AbstractConfig { return new LogConfigDef(CONFIG); } - public static boolean shouldIgnoreMessageFormatVersion(MetadataVersion interBrokerProtocolVersion) { - return interBrokerProtocolVersion.isAtLeast(IBP_3_0_IV1); - } - public static Optional configType(String configName) { return Optional.ofNullable(CONFIG.configKeys().get(configName)).map(c -> c.type); } diff --git a/test-common/test-common-internal-api/src/test/java/org/apache/kafka/common/test/api/ClusterConfigTest.java b/test-common/test-common-internal-api/src/test/java/org/apache/kafka/common/test/api/ClusterConfigTest.java index 0eeb8ede394..bdd68e6b6a5 100644 --- a/test-common/test-common-internal-api/src/test/java/org/apache/kafka/common/test/api/ClusterConfigTest.java +++ b/test-common/test-common-internal-api/src/test/java/org/apache/kafka/common/test/api/ClusterConfigTest.java @@ -66,7 +66,7 @@ public class ClusterConfigTest { .setControllerSecurityProtocol(SecurityProtocol.SASL_PLAINTEXT) .setControllerListenerName(ListenerName.normalised("CONTROLLER")) .setTrustStoreFile(trustStoreFile) - .setMetadataVersion(MetadataVersion.IBP_3_0_IV1) + .setMetadataVersion(MetadataVersion.MINIMUM_VERSION) .setServerProperties(Collections.singletonMap("broker", "broker_value")) .setConsumerProperties(Collections.singletonMap("consumer", "consumer_value")) .setProducerProperties(Collections.singletonMap("producer", "producer_value")) diff --git a/tools/src/main/java/org/apache/kafka/tools/FeatureCommand.java b/tools/src/main/java/org/apache/kafka/tools/FeatureCommand.java index fd01bccec8f..b3427b6a909 100644 --- a/tools/src/main/java/org/apache/kafka/tools/FeatureCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/FeatureCommand.java @@ -196,7 +196,7 @@ public class FeatureCommand { ); versionMappingParser.addArgument("--release-version") .help("The release version to use for the corresponding feature mapping. The minimum is " + - MetadataVersion.IBP_3_0_IV1 + "; the default is " + MetadataVersion.LATEST_PRODUCTION) + MetadataVersion.MINIMUM_VERSION + "; the default is " + MetadataVersion.LATEST_PRODUCTION) .action(store()); } @@ -298,7 +298,7 @@ public class FeatureCommand { } catch (Throwable e) { throw new TerseException("Unknown metadata.version " + releaseVersion + ". Supported metadata.version are " + metadataVersionsToString( - MetadataVersion.MINIMUM_BOOTSTRAP_VERSION, MetadataVersion.latestProduction())); + MetadataVersion.MINIMUM_VERSION, MetadataVersion.latestProduction())); } try { for (Feature feature : Feature.PRODUCTION_FEATURES) { @@ -320,7 +320,7 @@ public class FeatureCommand { } catch (Throwable e) { throw new TerseException("Unknown metadata.version " + metadata + ". Supported metadata.version are " + metadataVersionsToString( - MetadataVersion.MINIMUM_BOOTSTRAP_VERSION, MetadataVersion.latestProduction())); + MetadataVersion.MINIMUM_VERSION, MetadataVersion.latestProduction())); } updates.put(MetadataVersion.FEATURE_NAME, new FeatureUpdate(metadataVersion.featureLevel(), upgradeType)); } @@ -373,7 +373,7 @@ public class FeatureCommand { } } catch (IllegalArgumentException e) { throw new TerseException("Unknown release version '" + releaseVersion + "'." + - " Supported versions are: " + MetadataVersion.MINIMUM_BOOTSTRAP_VERSION + + " Supported versions are: " + MetadataVersion.MINIMUM_VERSION + " to " + MetadataVersion.LATEST_PRODUCTION); } } diff --git a/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java index 5d3a9ecabbb..340889d8765 100644 --- a/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java @@ -48,7 +48,7 @@ public class FeatureCommandTest { private final List testingFeatures = Arrays.stream(Feature.FEATURES).collect(Collectors.toList()); - @ClusterTest(types = {Type.KRAFT}, metadataVersion = MetadataVersion.IBP_3_3_IV1) + @ClusterTest(types = {Type.KRAFT}, metadataVersion = MetadataVersion.IBP_3_3_IV3) public void testDescribeWithKRaft(ClusterInstance cluster) { String commandOutput = ToolsTestUtils.captureStandardOut(() -> assertEquals(0, FeatureCommand.mainNoExit("--bootstrap-server", cluster.bootstrapServers(), "describe")) @@ -63,8 +63,8 @@ public class FeatureCommandTest { "SupportedMaxVersion: 1\tFinalizedVersionLevel: 0\t", outputWithoutEpoch(features.get(1))); assertEquals("Feature: kraft.version\tSupportedMinVersion: 0\t" + "SupportedMaxVersion: 1\tFinalizedVersionLevel: 0\t", outputWithoutEpoch(features.get(2))); - assertEquals("Feature: metadata.version\tSupportedMinVersion: 3.0-IV1\t" + - "SupportedMaxVersion: 4.1-IV0\tFinalizedVersionLevel: 3.3-IV1\t", outputWithoutEpoch(features.get(3))); + assertEquals("Feature: metadata.version\tSupportedMinVersion: 3.3-IV3\t" + + "SupportedMaxVersion: 4.1-IV0\tFinalizedVersionLevel: 3.3-IV3\t", outputWithoutEpoch(features.get(3))); assertEquals("Feature: transaction.version\tSupportedMinVersion: 0\t" + "SupportedMaxVersion: 2\tFinalizedVersionLevel: 0\t", outputWithoutEpoch(features.get(4))); } @@ -85,28 +85,28 @@ public class FeatureCommandTest { "SupportedMaxVersion: 1\tFinalizedVersionLevel: 0\t", outputWithoutEpoch(features.get(1))); assertEquals("Feature: kraft.version\tSupportedMinVersion: 0\t" + "SupportedMaxVersion: 1\tFinalizedVersionLevel: 0\t", outputWithoutEpoch(features.get(2))); - assertEquals("Feature: metadata.version\tSupportedMinVersion: 3.0-IV1\t" + + assertEquals("Feature: metadata.version\tSupportedMinVersion: 3.3-IV3\t" + "SupportedMaxVersion: 4.1-IV0\tFinalizedVersionLevel: 3.7-IV0\t", outputWithoutEpoch(features.get(3))); assertEquals("Feature: transaction.version\tSupportedMinVersion: 0\t" + "SupportedMaxVersion: 2\tFinalizedVersionLevel: 0\t", outputWithoutEpoch(features.get(4))); } - @ClusterTest(types = {Type.KRAFT}, metadataVersion = MetadataVersion.IBP_3_3_IV1) + @ClusterTest(types = {Type.KRAFT}, metadataVersion = MetadataVersion.IBP_3_3_IV3) public void testUpgradeMetadataVersionWithKraft(ClusterInstance cluster) { String commandOutput = ToolsTestUtils.captureStandardOut(() -> assertEquals(0, FeatureCommand.mainNoExit("--bootstrap-server", cluster.bootstrapServers(), - "upgrade", "--feature", "metadata.version=5")) + "upgrade", "--feature", "metadata.version=7")) ); - assertEquals("metadata.version was upgraded to 5.", commandOutput); + assertEquals("metadata.version was upgraded to 7.", commandOutput); commandOutput = ToolsTestUtils.captureStandardOut(() -> assertEquals(0, FeatureCommand.mainNoExit("--bootstrap-server", cluster.bootstrapServers(), - "upgrade", "--metadata", "3.3-IV2")) + "upgrade", "--metadata", "3.4-IV0")) ); - assertEquals(format("`metadata` flag is deprecated and may be removed in a future release.%nmetadata.version was upgraded to 6."), commandOutput); + assertEquals(format("`metadata` flag is deprecated and may be removed in a future release.%nmetadata.version was upgraded to 8."), commandOutput); } - @ClusterTest(types = {Type.KRAFT}, metadataVersion = MetadataVersion.IBP_3_3_IV1) + @ClusterTest(types = {Type.KRAFT}, metadataVersion = MetadataVersion.IBP_3_4_IV0) public void testDowngradeMetadataVersionWithKRaft(ClusterInstance cluster) { String commandOutput = ToolsTestUtils.captureStandardOut(() -> assertEquals(1, FeatureCommand.mainNoExit("--bootstrap-server", cluster.bootstrapServers(), @@ -114,24 +114,24 @@ public class FeatureCommandTest { ); // Change expected message to reflect possible MetadataVersion range 1-N (N increases when adding a new version) assertEquals("Could not disable metadata.version. The update failed for all features since the following " + - "feature had an error: Invalid update version 0 for feature metadata.version. Local controller 3000 only supports versions 1-26", commandOutput); + "feature had an error: Invalid update version 0 for feature metadata.version. Local controller 3000 only supports versions 7-26", commandOutput); commandOutput = ToolsTestUtils.captureStandardOut(() -> assertEquals(1, FeatureCommand.mainNoExit("--bootstrap-server", cluster.bootstrapServers(), - "downgrade", "--metadata", "3.3-IV0")) + "downgrade", "--metadata", "3.3-IV3")) ); - assertEquals(format("`metadata` flag is deprecated and may be removed in a future release.%nCould not downgrade metadata.version to 4." + - " The update failed for all features since the following feature had an error: Invalid metadata.version 4." + + assertEquals(format("`metadata` flag is deprecated and may be removed in a future release.%nCould not downgrade metadata.version to 7." + + " The update failed for all features since the following feature had an error: Invalid metadata.version 7." + " Refusing to perform the requested downgrade because it might delete metadata information."), commandOutput); commandOutput = ToolsTestUtils.captureStandardOut(() -> assertEquals(1, FeatureCommand.mainNoExit("--bootstrap-server", cluster.bootstrapServers(), - "downgrade", "--unsafe", "--metadata", "3.3-IV0")) + "downgrade", "--unsafe", "--metadata", "3.3-IV3")) ); - assertEquals(format("`metadata` flag is deprecated and may be removed in a future release.%nCould not downgrade metadata.version to 4." + - " The update failed for all features since the following feature had an error: Invalid metadata.version 4." + + assertEquals(format("`metadata` flag is deprecated and may be removed in a future release.%nCould not downgrade metadata.version to 7." + + " The update failed for all features since the following feature had an error: Invalid metadata.version 7." + " Unsafe metadata downgrade is not supported in this version."), commandOutput); } @@ -192,14 +192,14 @@ public class FeatureCommandTest { @Test public void testLevelToString() { assertEquals("5", FeatureCommand.levelToString("foo.bar", (short) 5)); - assertEquals("3.3-IV0", - FeatureCommand.levelToString(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_3_IV0.featureLevel())); + assertEquals("3.9-IV0", + FeatureCommand.levelToString(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_9_IV0.featureLevel())); } @Test public void testMetadataVersionsToString() { - assertEquals("3.3-IV0, 3.3-IV1, 3.3-IV2, 3.3-IV3", - FeatureCommand.metadataVersionsToString(MetadataVersion.IBP_3_3_IV0, MetadataVersion.IBP_3_3_IV3)); + assertEquals("3.5-IV0, 3.5-IV1, 3.5-IV2, 3.6-IV0", + FeatureCommand.metadataVersionsToString(MetadataVersion.IBP_3_5_IV0, MetadataVersion.IBP_3_6_IV0)); } @Test @@ -223,15 +223,15 @@ public class FeatureCommandTest { private static MockAdminClient buildAdminClient() { Map minSupportedFeatureLevels = new HashMap<>(); - minSupportedFeatureLevels.put(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_3_IV0.featureLevel()); + minSupportedFeatureLevels.put(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_3_IV3.featureLevel()); minSupportedFeatureLevels.put("foo.bar", (short) 0); Map featureLevels = new HashMap<>(); - featureLevels.put(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_3_IV2.featureLevel()); + featureLevels.put(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_4_IV0.featureLevel()); featureLevels.put("foo.bar", (short) 5); Map maxSupportedFeatureLevels = new HashMap<>(); - maxSupportedFeatureLevels.put(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_3_IV3.featureLevel()); + maxSupportedFeatureLevels.put(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_5_IV0.featureLevel()); maxSupportedFeatureLevels.put("foo.bar", (short) 10); return new MockAdminClient.Builder(). @@ -250,28 +250,38 @@ public class FeatureCommandTest { } }); assertEquals(format("Feature: foo.bar\tSupportedMinVersion: 0\tSupportedMaxVersion: 10\tFinalizedVersionLevel: 5\tEpoch: 123%n" + - "Feature: metadata.version\tSupportedMinVersion: 3.3-IV0\tSupportedMaxVersion: 3.3-IV3\tFinalizedVersionLevel: 3.3-IV2\tEpoch: 123"), describeResult); + "Feature: metadata.version\tSupportedMinVersion: 3.3-IV3\tSupportedMaxVersion: 3.5-IV0\tFinalizedVersionLevel: 3.4-IV0\tEpoch: 123"), describeResult); } @Test - public void testHandleUpgrade() { + public void testHandleUpgradeToUnsupportedMetadataVersion() { Map namespace = new HashMap<>(); namespace.put("metadata", "3.3-IV1"); namespace.put("feature", Collections.singletonList("foo.bar=6")); namespace.put("dry_run", false); + Throwable t = assertThrows(TerseException.class, () -> FeatureCommand.handleUpgrade(new Namespace(namespace), buildAdminClient())); + assertTrue(t.getMessage().contains("Unknown metadata.version 3.3-IV1")); + } + + @Test + public void testHandleUpgradeToLowerVersion() { + Map namespace = new HashMap<>(); + namespace.put("metadata", "3.3-IV3"); + namespace.put("feature", Collections.singletonList("foo.bar=6")); + namespace.put("dry_run", false); String upgradeOutput = ToolsTestUtils.captureStandardOut(() -> { Throwable t = assertThrows(TerseException.class, () -> FeatureCommand.handleUpgrade(new Namespace(namespace), buildAdminClient())); assertTrue(t.getMessage().contains("2 out of 2 operation(s) failed.")); }); assertEquals(format("`metadata` flag is deprecated and may be removed in a future release.%nCould not upgrade foo.bar to 6." + - " Invalid update version 5 for feature metadata.version. Can't upgrade to lower version.%n" + - "Could not upgrade metadata.version to 5. Invalid update version 5 for feature metadata.version. Can't upgrade to lower version."), upgradeOutput); + " Invalid update version 7 for feature metadata.version. Can't upgrade to lower version.%n" + + "Could not upgrade metadata.version to 7. Invalid update version 7 for feature metadata.version. Can't upgrade to lower version."), upgradeOutput); } @Test - public void testHandleUpgradeDryRun() { + public void testHandleUpgradeToLowerVersionDryRun() { Map namespace = new HashMap<>(); - namespace.put("metadata", "3.3-IV1"); + namespace.put("metadata", "3.3-IV3"); namespace.put("feature", Collections.singletonList("foo.bar=6")); namespace.put("dry_run", true); String upgradeOutput = ToolsTestUtils.captureStandardOut(() -> { @@ -279,14 +289,14 @@ public class FeatureCommandTest { assertTrue(t.getMessage().contains("2 out of 2 operation(s) failed.")); }); assertEquals(format("`metadata` flag is deprecated and may be removed in a future release.%nCan not upgrade foo.bar to 6." + - " Invalid update version 5 for feature metadata.version. Can't upgrade to lower version.%n" + - "Can not upgrade metadata.version to 5. Invalid update version 5 for feature metadata.version. Can't upgrade to lower version."), upgradeOutput); + " Invalid update version 7 for feature metadata.version. Can't upgrade to lower version.%n" + + "Can not upgrade metadata.version to 7. Invalid update version 7 for feature metadata.version. Can't upgrade to lower version."), upgradeOutput); } @Test public void testHandleDowngrade() { Map namespace = new HashMap<>(); - namespace.put("metadata", "3.3-IV3"); + namespace.put("metadata", "3.7-IV0"); namespace.put("feature", Collections.singletonList("foo.bar=1")); namespace.put("dry_run", false); String downgradeOutput = ToolsTestUtils.captureStandardOut(() -> { @@ -294,22 +304,23 @@ public class FeatureCommandTest { assertTrue(t.getMessage().contains("2 out of 2 operation(s) failed.")); }); assertEquals(format("`metadata` flag is deprecated and may be removed in a future release.%nCould not downgrade foo.bar to 1." + - " Invalid update version 7 for feature metadata.version. Can't downgrade to newer version.%n" + - "Could not downgrade metadata.version to 7. Invalid update version 7 for feature metadata.version. Can't downgrade to newer version."), downgradeOutput); + " Invalid update version 15 for feature metadata.version. Can't downgrade to newer version.%n" + + "Could not downgrade metadata.version to 15. Invalid update version 15 for feature metadata.version. Can't downgrade to newer version."), downgradeOutput); } @Test public void testHandleDowngradeDryRun() { Map namespace = new HashMap<>(); - namespace.put("metadata", "3.3-IV3"); + namespace.put("metadata", "3.7-IV0"); namespace.put("feature", Collections.singletonList("foo.bar=1")); namespace.put("dry_run", true); String downgradeOutput = ToolsTestUtils.captureStandardOut(() -> { Throwable t = assertThrows(TerseException.class, () -> FeatureCommand.handleDowngrade(new Namespace(namespace), buildAdminClient())); assertTrue(t.getMessage().contains("2 out of 2 operation(s) failed.")); }); - assertEquals(format("`metadata` flag is deprecated and may be removed in a future release.%nCan not downgrade foo.bar to 1. Invalid update version 7 for feature metadata.version." + - " Can't downgrade to newer version.%nCan not downgrade metadata.version to 7. Invalid update version 7 for feature metadata.version. Can't downgrade to newer version."), downgradeOutput); + assertEquals(format("`metadata` flag is deprecated and may be removed in a future release.%nCan not downgrade foo.bar to 1. Invalid update " + + "version 15 for feature metadata.version. Can't downgrade to newer version.%nCan not downgrade metadata.version to 15. Invalid update " + + "version 15 for feature metadata.version. Can't downgrade to newer version."), downgradeOutput); } @Test @@ -321,9 +332,9 @@ public class FeatureCommandTest { Throwable t = assertThrows(TerseException.class, () -> FeatureCommand.handleDisable(new Namespace(namespace), buildAdminClient())); assertTrue(t.getMessage().contains("3 out of 3 operation(s) failed.")); }); - assertEquals(format("Could not disable foo.bar. Invalid update version 0 for feature metadata.version. Can't downgrade below 4%n" + - "Could not disable metadata.version. Invalid update version 0 for feature metadata.version. Can't downgrade below 4%n" + - "Could not disable quux. Invalid update version 0 for feature metadata.version. Can't downgrade below 4"), disableOutput); + assertEquals(format("Could not disable foo.bar. Invalid update version 0 for feature metadata.version. Can't downgrade below 7%n" + + "Could not disable metadata.version. Invalid update version 0 for feature metadata.version. Can't downgrade below 7%n" + + "Could not disable quux. Invalid update version 0 for feature metadata.version. Can't downgrade below 7"), disableOutput); } @Test @@ -335,9 +346,9 @@ public class FeatureCommandTest { Throwable t = assertThrows(TerseException.class, () -> FeatureCommand.handleDisable(new Namespace(namespace), buildAdminClient())); assertTrue(t.getMessage().contains("3 out of 3 operation(s) failed.")); }); - assertEquals(format("Can not disable foo.bar. Invalid update version 0 for feature metadata.version. Can't downgrade below 4%n" + - "Can not disable metadata.version. Invalid update version 0 for feature metadata.version. Can't downgrade below 4%n" + - "Can not disable quux. Invalid update version 0 for feature metadata.version. Can't downgrade below 4"), disableOutput); + assertEquals(format("Can not disable foo.bar. Invalid update version 0 for feature metadata.version. Can't downgrade below 7%n" + + "Can not disable metadata.version. Invalid update version 0 for feature metadata.version. Can't downgrade below 7%n" + + "Can not disable quux. Invalid update version 0 for feature metadata.version. Can't downgrade below 7"), disableOutput); } @Test @@ -428,7 +439,7 @@ public class FeatureCommandTest { ); assertEquals("Unknown release version '2.9-IV2'." + - " Supported versions are: " + MetadataVersion.MINIMUM_BOOTSTRAP_VERSION + + " Supported versions are: " + MetadataVersion.MINIMUM_VERSION + " to " + MetadataVersion.LATEST_PRODUCTION, exception1.getMessage()); namespace.put("release_version", "invalid"); @@ -438,7 +449,7 @@ public class FeatureCommandTest { ); assertEquals("Unknown release version 'invalid'." + - " Supported versions are: " + MetadataVersion.MINIMUM_BOOTSTRAP_VERSION + + " Supported versions are: " + MetadataVersion.MINIMUM_VERSION + " to " + MetadataVersion.LATEST_PRODUCTION, exception2.getMessage()); } diff --git a/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandTest.java index 2c30cb38fb4..d7f6f937d97 100644 --- a/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandTest.java @@ -44,7 +44,6 @@ import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfigProperty; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.ClusterTestDefaults; -import org.apache.kafka.common.test.api.ClusterTests; import org.apache.kafka.common.test.api.Type; import org.apache.kafka.common.utils.Time; import org.apache.kafka.server.config.QuotaConfig; @@ -79,7 +78,6 @@ import static java.util.Collections.singleton; import static java.util.Collections.singletonList; import static java.util.Collections.singletonMap; import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC; -import static org.apache.kafka.server.common.MetadataVersion.IBP_3_3_IV0; import static org.apache.kafka.server.config.QuotaConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG; import static org.apache.kafka.server.config.QuotaConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG; import static org.apache.kafka.server.config.ReplicationConfigs.AUTO_LEADER_REBALANCE_ENABLE_CONFIG; @@ -128,18 +126,6 @@ public class ReassignPartitionsCommandTest { executeAndVerifyReassignment(); } - @ClusterTests({ - @ClusterTest(types = {Type.KRAFT, Type.CO_KRAFT}, metadataVersion = IBP_3_3_IV0) - }) - public void testReassignmentWithAlterPartitionDisabled() throws Exception { - // Test reassignment when the IBP is on an older version which does not use - // the `AlterPartition` API. In this case, the controller will register individual - // watches for each reassigning partition so that the reassignment can be - // completed as soon as the ISR is expanded. - createTopics(); - executeAndVerifyReassignment(); - } - @ClusterTest public void testHighWaterMarkAfterPartitionReassignment() throws Exception { createTopics();