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 <chia7712@gmail.com>, Jun Rao <junrao@gmail.com>, David Arthur <mumrah@gmail.com>, Colin P. McCabe <cmccabe@apache.org>, Justine Olshan <jolshan@confluen.io>, Ken Huang <s7133700@gmail.com>
This commit is contained in:
Ismael Juma 2025-02-19 05:35:42 -08:00 committed by GitHub
parent a6a588fbed
commit 3dba3125e9
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
118 changed files with 969 additions and 1337 deletions

View File

@ -18,12 +18,13 @@
"type": "request", "type": "request",
"listeners": ["controller"], "listeners": ["controller"],
"name": "AlterPartitionRequest", "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 1 adds LeaderRecoveryState field (KIP-704).
//
// Version 2 adds TopicId field to replace TopicName field (KIP-841). // Version 2 adds TopicId field to replace TopicName field (KIP-841).
// //
// Version 3 adds the NewIsrEpochs field and deprecates the NewIsr field (KIP-903). // Version 3 adds the NewIsrEpochs field and deprecates the NewIsr field (KIP-903).
"validVersions": "0-3", "validVersions": "2-3",
"flexibleVersions": "0+", "flexibleVersions": "0+",
"fields": [ "fields": [
{ "name": "BrokerId", "type": "int32", "versions": "0+", "entityType": "brokerId", { "name": "BrokerId", "type": "int32", "versions": "0+", "entityType": "brokerId",
@ -32,8 +33,6 @@
"about": "The epoch of the requesting broker." }, "about": "The epoch of the requesting broker." },
{ "name": "Topics", "type": "[]TopicData", "versions": "0+", { "name": "Topics", "type": "[]TopicData", "versions": "0+",
"about": "The topics to alter ISRs for.", "fields": [ "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, { "name": "TopicId", "type": "uuid", "versions": "2+", "ignorable": true,
"about": "The ID of the topic to alter ISRs for." }, "about": "The ID of the topic to alter ISRs for." },
{ "name": "Partitions", "type": "[]PartitionData", "versions": "0+", { "name": "Partitions", "type": "[]PartitionData", "versions": "0+",

View File

@ -17,13 +17,14 @@
"apiKey": 56, "apiKey": 56,
"type": "response", "type": "response",
"name": "AlterPartitionResponse", "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 1 adds LeaderRecoveryState field (KIP-704).
//
// Version 2 adds TopicId field to replace TopicName field, can return the following new errors: // 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). // INELIGIBLE_REPLICA, NEW_LEADER_ELECTED and UNKNOWN_TOPIC_ID (KIP-841).
// //
// Version 3 is the same as version 2 (KIP-903). // Version 3 is the same as version 2 (KIP-903).
"validVersions": "0-3", "validVersions": "2-3",
"flexibleVersions": "0+", "flexibleVersions": "0+",
"fields": [ "fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
@ -32,8 +33,6 @@
"about": "The top level response error code." }, "about": "The top level response error code." },
{ "name": "Topics", "type": "[]TopicData", "versions": "0+", { "name": "Topics", "type": "[]TopicData", "versions": "0+",
"about": "The responses for each topic.", "fields": [ "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, { "name": "TopicId", "type": "uuid", "versions": "2+", "ignorable": true,
"about": "The ID of the topic." }, "about": "The ID of the topic." },
{ "name": "Partitions", "type": "[]PartitionData", "versions": "0+", { "name": "Partitions", "type": "[]PartitionData", "versions": "0+",

View File

@ -34,7 +34,6 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertTrue;
class AlterPartitionRequestTest { class AlterPartitionRequestTest {
String topic = "test-topic";
Uuid topicId = Uuid.randomUuid(); Uuid topicId = Uuid.randomUuid();
@ParameterizedTest @ParameterizedTest
@ -44,9 +43,7 @@ class AlterPartitionRequestTest {
.setBrokerId(1) .setBrokerId(1)
.setBrokerEpoch(1); .setBrokerEpoch(1);
TopicData topicData = new TopicData() TopicData topicData = new TopicData().setTopicId(topicId);
.setTopicId(topicId)
.setTopicName(topic);
List<BrokerState> newIsrWithBrokerEpoch = new LinkedList<>(); List<BrokerState> newIsrWithBrokerEpoch = new LinkedList<>();
newIsrWithBrokerEpoch.add(new BrokerState().setBrokerId(1).setBrokerEpoch(1001)); newIsrWithBrokerEpoch.add(new BrokerState().setBrokerId(1).setBrokerEpoch(1001));

View File

@ -1103,7 +1103,7 @@ public class RequestResponseTest {
case BEGIN_QUORUM_EPOCH: return createBeginQuorumEpochResponse(); case BEGIN_QUORUM_EPOCH: return createBeginQuorumEpochResponse();
case END_QUORUM_EPOCH: return createEndQuorumEpochResponse(); case END_QUORUM_EPOCH: return createEndQuorumEpochResponse();
case DESCRIBE_QUORUM: return createDescribeQuorumResponse(); case DESCRIBE_QUORUM: return createDescribeQuorumResponse();
case ALTER_PARTITION: return createAlterPartitionResponse(version); case ALTER_PARTITION: return createAlterPartitionResponse();
case UPDATE_FEATURES: return createUpdateFeaturesResponse(); case UPDATE_FEATURES: return createUpdateFeaturesResponse();
case ENVELOPE: return createEnvelopeResponse(); case ENVELOPE: return createEnvelopeResponse();
case FETCH_SNAPSHOT: return createFetchSnapshotResponse(); case FETCH_SNAPSHOT: return createFetchSnapshotResponse();
@ -1703,41 +1703,31 @@ public class RequestResponseTest {
.setPartitionIndex(1) .setPartitionIndex(1)
.setPartitionEpoch(2) .setPartitionEpoch(2)
.setLeaderEpoch(3) .setLeaderEpoch(3)
.setNewIsrWithEpochs(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(asList(1, 2))); .setNewIsrWithEpochs(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(asList(1, 2)))
.setLeaderRecoveryState((byte) 1); // non-default value
if (version >= 1) {
// Use the none default value; 1 - RECOVERING
partitionData.setLeaderRecoveryState((byte) 1);
}
AlterPartitionRequestData data = new AlterPartitionRequestData() AlterPartitionRequestData data = new AlterPartitionRequestData()
.setBrokerEpoch(123L) .setBrokerEpoch(123L)
.setBrokerId(1) .setBrokerId(1)
.setTopics(singletonList(new AlterPartitionRequestData.TopicData() .setTopics(singletonList(new AlterPartitionRequestData.TopicData()
.setTopicName("topic1")
.setTopicId(Uuid.randomUuid()) .setTopicId(Uuid.randomUuid())
.setPartitions(singletonList(partitionData)))); .setPartitions(singletonList(partitionData))));
return new AlterPartitionRequest.Builder(data).build(version); return new AlterPartitionRequest.Builder(data).build(version);
} }
private AlterPartitionResponse createAlterPartitionResponse(int version) { private AlterPartitionResponse createAlterPartitionResponse() {
AlterPartitionResponseData.PartitionData partitionData = new AlterPartitionResponseData.PartitionData() AlterPartitionResponseData.PartitionData partitionData = new AlterPartitionResponseData.PartitionData()
.setPartitionEpoch(1) .setPartitionEpoch(1)
.setIsr(asList(0, 1, 2)) .setIsr(asList(0, 1, 2))
.setErrorCode(Errors.NONE.code()) .setErrorCode(Errors.NONE.code())
.setLeaderEpoch(2) .setLeaderEpoch(2)
.setLeaderId(3); .setLeaderId(3)
.setLeaderRecoveryState((byte) 1); // non-default value
if (version >= 1) {
// Use the none default value; 1 - RECOVERING
partitionData.setLeaderRecoveryState((byte) 1);
}
AlterPartitionResponseData data = new AlterPartitionResponseData() AlterPartitionResponseData data = new AlterPartitionResponseData()
.setErrorCode(Errors.NONE.code()) .setErrorCode(Errors.NONE.code())
.setThrottleTimeMs(123) .setThrottleTimeMs(123)
.setTopics(singletonList(new AlterPartitionResponseData.TopicData() .setTopics(singletonList(new AlterPartitionResponseData.TopicData()
.setTopicName("topic1")
.setTopicId(Uuid.randomUuid()) .setTopicId(Uuid.randomUuid())
.setPartitions(singletonList(partitionData)))); .setPartitions(singletonList(partitionData))));
return new AlterPartitionResponse(data); return new AlterPartitionResponse(data);

View File

@ -52,7 +52,7 @@ public class MetadataVersionConfigValidator implements MetadataPublisher {
) { ) {
if (delta.featuresDelta() != null) { if (delta.featuresDelta() != null) {
if (delta.metadataVersionChanged().isPresent()) { if (delta.metadataVersionChanged().isPresent()) {
onMetadataVersionChanged(newImage.features().metadataVersion()); onMetadataVersionChanged(newImage.features().metadataVersionOrThrow());
} }
} }
} }

View File

@ -119,8 +119,8 @@ object Partition {
time: Time, time: Time,
replicaManager: ReplicaManager): Partition = { replicaManager: ReplicaManager): Partition = {
Partition( Partition(
topicPartition = topicIdPartition.topicPartition(), topicPartition = topicIdPartition.topicPartition,
topicId = Option(topicIdPartition.topicId()), topicId = Some(topicIdPartition.topicId),
time = time, time = time,
replicaManager = replicaManager) replicaManager = replicaManager)
} }
@ -1814,7 +1814,7 @@ class Partition(val topicPartition: TopicPartition,
private def submitAlterPartition(proposedIsrState: PendingPartitionChange): CompletableFuture[LeaderAndIsr] = { private def submitAlterPartition(proposedIsrState: PendingPartitionChange): CompletableFuture[LeaderAndIsr] = {
debug(s"Submitting ISR state change $proposedIsrState") debug(s"Submitting ISR state change $proposedIsrState")
val future = alterIsrManager.submit( 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 proposedIsrState.sentLeaderAndIsr
) )
future.whenComplete { (leaderAndIsr, e) => future.whenComplete { (leaderAndIsr, e) =>

View File

@ -16,23 +16,18 @@
*/ */
package kafka.server package kafka.server
import java.util
import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.atomic.AtomicBoolean
import java.util.concurrent.{CompletableFuture, ConcurrentHashMap} import java.util.concurrent.{CompletableFuture, ConcurrentHashMap}
import kafka.utils.Logging import kafka.utils.Logging
import org.apache.kafka.clients.ClientResponse 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.errors.OperationNotAttemptedException
import org.apache.kafka.common.message.AlterPartitionRequestData import org.apache.kafka.common.message.AlterPartitionRequestData
import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.protocol.Errors 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.requests.{AlterPartitionRequest, AlterPartitionResponse}
import org.apache.kafka.common.utils.Time import org.apache.kafka.common.utils.Time
import org.apache.kafka.metadata.{LeaderAndIsr, LeaderRecoveryState} 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 org.apache.kafka.server.util.Scheduler
import scala.collection.mutable import scala.collection.mutable
@ -70,7 +65,6 @@ object AlterPartitionManager {
*/ */
def apply( def apply(
config: KafkaConfig, config: KafkaConfig,
metadataCache: MetadataCache,
scheduler: Scheduler, scheduler: Scheduler,
controllerNodeProvider: ControllerNodeProvider, controllerNodeProvider: ControllerNodeProvider,
time: Time, time: Time,
@ -92,8 +86,7 @@ object AlterPartitionManager {
scheduler = scheduler, scheduler = scheduler,
time = time, time = time,
brokerId = config.brokerId, brokerId = config.brokerId,
brokerEpochSupplier = brokerEpochSupplier, brokerEpochSupplier = brokerEpochSupplier
metadataVersionSupplier = () => metadataCache.metadataVersion()
) )
} }
} }
@ -104,17 +97,10 @@ class DefaultAlterPartitionManager(
val time: Time, val time: Time,
val brokerId: Int, val brokerId: Int,
val brokerEpochSupplier: () => Long, val brokerEpochSupplier: () => Long,
val metadataVersionSupplier: () => MetadataVersion
) extends AlterPartitionManager with Logging { ) extends AlterPartitionManager with Logging {
// Used to allow only one pending ISR update per partition (visible for testing). // Used to allow only one pending ISR update per partition (visible for testing)
// Note that we key items by TopicPartition despite using TopicIdPartition while private[server] val unsentIsrUpdates = new ConcurrentHashMap[TopicIdPartition, AlterPartitionItem]()
// 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 in-flight request at a time // Used to allow only one in-flight request at a time
private val inflightRequest: AtomicBoolean = new AtomicBoolean(false) private val inflightRequest: AtomicBoolean = new AtomicBoolean(false)
@ -133,7 +119,7 @@ class DefaultAlterPartitionManager(
): CompletableFuture[LeaderAndIsr] = { ): CompletableFuture[LeaderAndIsr] = {
val future = new CompletableFuture[LeaderAndIsr]() val future = new CompletableFuture[LeaderAndIsr]()
val alterPartitionItem = AlterPartitionItem(topicIdPartition, leaderAndIsr, future) 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) { if (enqueued) {
maybePropagateIsrChanges() maybePropagateIsrChanges()
} else { } else {
@ -161,7 +147,7 @@ class DefaultAlterPartitionManager(
private def sendRequest(inflightAlterPartitionItems: Seq[AlterPartitionItem]): Unit = { private def sendRequest(inflightAlterPartitionItems: Seq[AlterPartitionItem]): Unit = {
val brokerEpoch = brokerEpochSupplier() val brokerEpoch = brokerEpochSupplier()
val (request, topicNamesByIds) = buildRequest(inflightAlterPartitionItems, brokerEpoch) val request = buildRequest(inflightAlterPartitionItems, brokerEpoch)
debug(s"Sending AlterPartition to controller $request") debug(s"Sending AlterPartition to controller $request")
// We will not timeout AlterPartition request, instead letting it retry indefinitely // We will not timeout AlterPartition request, instead letting it retry indefinitely
@ -182,11 +168,9 @@ class DefaultAlterPartitionManager(
Errors.UNSUPPORTED_VERSION Errors.UNSUPPORTED_VERSION
} else { } else {
handleAlterPartitionResponse( handleAlterPartitionResponse(
response.requestHeader,
response.responseBody.asInstanceOf[AlterPartitionResponse], response.responseBody.asInstanceOf[AlterPartitionResponse],
brokerEpoch, brokerEpoch,
inflightAlterPartitionItems, inflightAlterPartitionItems
topicNamesByIds
) )
} }
} finally { } finally {
@ -218,59 +202,40 @@ class DefaultAlterPartitionManager(
* supported by the controller. The final decision is taken when the AlterPartitionRequest * 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. * 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 * @return an AlterPartitionRequest.Builder with the provided parameters.
* topic id to topic name. This mapping is used in the response handling.
*/ */
private def buildRequest( private def buildRequest(
inflightAlterPartitionItems: Seq[AlterPartitionItem], inflightAlterPartitionItems: Seq[AlterPartitionItem],
brokerEpoch: Long brokerEpoch: Long
): (AlterPartitionRequest.Builder, mutable.Map[Uuid, String]) = { ): AlterPartitionRequest.Builder = {
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]()
val message = new AlterPartitionRequestData() val message = new AlterPartitionRequestData()
.setBrokerId(brokerId) .setBrokerId(brokerId)
.setBrokerEpoch(brokerEpoch) .setBrokerEpoch(brokerEpoch)
inflightAlterPartitionItems.groupBy(_.topicIdPartition.topic).foreach { case (topicName, items) => inflightAlterPartitionItems.groupBy(_.topicIdPartition.topicId).foreach { case (topicId, items) =>
val topicId = items.head.topicIdPartition.topicId val topicData = new AlterPartitionRequestData.TopicData().setTopicId(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)
message.topics.add(topicData) message.topics.add(topicData)
items.foreach { item => items.foreach { item =>
val partitionData = new AlterPartitionRequestData.PartitionData() val partitionData = new AlterPartitionRequestData.PartitionData()
.setPartitionIndex(item.topicIdPartition.partition) .setPartitionIndex(item.topicIdPartition.partitionId)
.setLeaderEpoch(item.leaderAndIsr.leaderEpoch) .setLeaderEpoch(item.leaderAndIsr.leaderEpoch)
.setNewIsrWithEpochs(item.leaderAndIsr.isrWithBrokerEpoch) .setNewIsrWithEpochs(item.leaderAndIsr.isrWithBrokerEpoch)
.setPartitionEpoch(item.leaderAndIsr.partitionEpoch) .setPartitionEpoch(item.leaderAndIsr.partitionEpoch)
if (metadataVersion.isLeaderRecoverySupported) { partitionData.setLeaderRecoveryState(item.leaderAndIsr.leaderRecoveryState.value)
partitionData.setLeaderRecoveryState(item.leaderAndIsr.leaderRecoveryState.value)
}
topicData.partitions.add(partitionData) topicData.partitions.add(partitionData)
} }
} }
(new AlterPartitionRequest.Builder(message), topicNamesByIds) new AlterPartitionRequest.Builder(message)
} }
private def handleAlterPartitionResponse( private def handleAlterPartitionResponse(
requestHeader: RequestHeader,
alterPartitionResp: AlterPartitionResponse, alterPartitionResp: AlterPartitionResponse,
sentBrokerEpoch: Long, sentBrokerEpoch: Long,
inflightAlterPartitionItems: Seq[AlterPartitionItem], inflightAlterPartitionItems: Seq[AlterPartitionItem],
topicNamesByIds: mutable.Map[Uuid, String]
): Errors = { ): Errors = {
val data = alterPartitionResp.data val data = alterPartitionResp.data
@ -284,37 +249,31 @@ class DefaultAlterPartitionManager(
case Errors.NONE => case Errors.NONE =>
// Collect partition-level responses to pass to the callbacks // 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 => data.topics.forEach { topic =>
// Topic IDs are used since version 2 of the AlterPartition API. topic.partitions.forEach { partition =>
val topicName = if (requestHeader.apiVersion > 1) topicNamesByIds.get(topic.topicId).orNull else topic.topicName val tp = new TopicIdPartition(topic.topicId, partition.partitionIndex)
if (topicName == null || topicName.isEmpty) { val apiError = Errors.forCode(partition.errorCode)
error(s"Received an unexpected topic $topic in the alter partition response, ignoring it.") debug(s"Controller successfully handled AlterPartition request for $tp: $partition")
} else { if (apiError == Errors.NONE) {
topic.partitions.forEach { partition => LeaderRecoveryState.optionalOf(partition.leaderRecoveryState).toScala match {
val tp = new TopicPartition(topicName, partition.partitionIndex) case Some(leaderRecoveryState) =>
val apiError = Errors.forCode(partition.errorCode) partitionResponses(tp) = Right(
debug(s"Controller successfully handled AlterPartition request for $tp: $partition") new LeaderAndIsr(
if (apiError == Errors.NONE) { partition.leaderId,
LeaderRecoveryState.optionalOf(partition.leaderRecoveryState).toScala match { partition.leaderEpoch,
case Some(leaderRecoveryState) => partition.isr,
partitionResponses(tp) = Right( leaderRecoveryState,
new LeaderAndIsr( partition.partitionEpoch
partition.leaderId,
partition.leaderEpoch,
partition.isr,
leaderRecoveryState,
partition.partitionEpoch
)
) )
)
case None => case None =>
error(s"Controller returned an invalid leader recovery state (${partition.leaderRecoveryState}) for $tp: $partition") error(s"Controller returned an invalid leader recovery state (${partition.leaderRecoveryState}) for $tp: $partition")
partitionResponses(tp) = Left(Errors.UNKNOWN_SERVER_ERROR) partitionResponses(tp) = Left(Errors.UNKNOWN_SERVER_ERROR)
}
} else {
partitionResponses(tp) = Left(apiError)
} }
} 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 // partition was somehow erroneously excluded from the response. Note that these callbacks are run from
// the leaderIsrUpdateLock write lock in Partition#sendAlterPartitionRequest // the leaderIsrUpdateLock write lock in Partition#sendAlterPartitionRequest
inflightAlterPartitionItems.foreach { inflightAlterPartition => inflightAlterPartitionItems.foreach { inflightAlterPartition =>
partitionResponses.get(inflightAlterPartition.topicIdPartition.topicPartition) match { partitionResponses.get(inflightAlterPartition.topicIdPartition) match {
case Some(leaderAndIsrOrError) => case Some(leaderAndIsrOrError) =>
// Regardless of callback outcome, we need to clear from the unsent updates map to unblock further // 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. // 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 { leaderAndIsrOrError match {
case Left(error) => inflightAlterPartition.future.completeExceptionally(error.exception) case Left(error) => inflightAlterPartition.future.completeExceptionally(error.exception)
case Right(leaderAndIsr) => inflightAlterPartition.future.complete(leaderAndIsr) case Right(leaderAndIsr) => inflightAlterPartition.future.complete(leaderAndIsr)

View File

@ -280,7 +280,6 @@ class BrokerServer(
alterPartitionManager = AlterPartitionManager( alterPartitionManager = AlterPartitionManager(
config, config,
metadataCache,
scheduler = kafkaScheduler, scheduler = kafkaScheduler,
controllerNodeProvider, controllerNodeProvider,
time = time, time = time,

View File

@ -34,6 +34,7 @@ import org.apache.kafka.queue.{EventQueue, KafkaEventQueue}
import org.apache.kafka.server.common.{ControllerRequestCompletionHandler, MetadataVersion, NodeToControllerChannelManager} import org.apache.kafka.server.common.{ControllerRequestCompletionHandler, MetadataVersion, NodeToControllerChannelManager}
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
import scala.jdk.OptionConverters._
/** /**
* The controller registration manager handles registering this controller with the controller * 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. * 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. * True if we're registered. Only read or written from the event queue thread.
@ -172,7 +173,7 @@ class ControllerRegistrationManager(
override def run(): Unit = { override def run(): Unit = {
try { try {
if (delta.featuresDelta() != null) { if (delta.featuresDelta() != null) {
metadataVersion = newImage.features().metadataVersion() metadataVersion = newImage.features().metadataVersion().toScala
} }
if (delta.clusterDelta() != null) { if (delta.clusterDelta() != null) {
if (delta.clusterDelta().changedControllers().containsKey(nodeId)) { if (delta.clusterDelta().changedControllers().containsKey(nodeId)) {
@ -197,12 +198,14 @@ class ControllerRegistrationManager(
} }
private def maybeSendControllerRegistration(): Unit = { private def maybeSendControllerRegistration(): Unit = {
val metadataVersion = this.metadataVersion
if (registeredInLog) { if (registeredInLog) {
debug("maybeSendControllerRegistration: controller is already registered.") debug("maybeSendControllerRegistration: controller is already registered.")
} else if (_channelManager == null) { } else if (_channelManager == null) {
debug("maybeSendControllerRegistration: cannot register yet because the channel manager has " + debug("maybeSendControllerRegistration: cannot register yet because the channel manager has not been initialized.")
"not been initialized.") } else if (metadataVersion.isEmpty) {
} else if (!metadataVersion.isControllerRegistrationSupported) { 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 " + info("maybeSendControllerRegistration: cannot register yet because the metadata.version is " +
s"still $metadataVersion, which does not support KIP-919 controller registration.") s"still $metadataVersion, which does not support KIP-919 controller registration.")
} else if (pendingRpc) { } else if (pendingRpc) {

View File

@ -181,7 +181,7 @@ object KafkaRaftServer {
} }
// Load the BootstrapMetadata. // Load the BootstrapMetadata.
val bootstrapDirectory = new BootstrapDirectory(config.metadataLogDir, Optional.empty()) val bootstrapDirectory = new BootstrapDirectory(config.metadataLogDir)
val bootstrapMetadata = bootstrapDirectory.read() val bootstrapMetadata = bootstrapDirectory.read()
(metaPropsEnsemble, bootstrapMetadata) (metaPropsEnsemble, bootstrapMetadata)
} }

View File

@ -2694,7 +2694,7 @@ class ReplicaManager(val config: KafkaConfig,
def applyDelta(delta: TopicsDelta, newImage: MetadataImage): Unit = { def applyDelta(delta: TopicsDelta, newImage: MetadataImage): Unit = {
// Before taking the lock, compute the local changes // Before taking the lock, compute the local changes
val localChanges = delta.localChanges(config.nodeId) val localChanges = delta.localChanges(config.nodeId)
val metadataVersion = newImage.features().metadataVersion() val metadataVersion = newImage.features().metadataVersionOrThrow()
replicaStateChangeLock.synchronized { replicaStateChangeLock.synchronized {
// Handle deleted partitions. We need to do this first because we might subsequently // Handle deleted partitions. We need to do this first because we might subsequently

View File

@ -119,7 +119,7 @@ class BrokerMetadataPublisher(
// Publish the new metadata image to the metadata cache. // Publish the new metadata image to the metadata cache.
metadataCache.setImage(newImage) metadataCache.setImage(newImage)
val metadataVersionLogMsg = s"metadata.version ${newImage.features().metadataVersion()}" def metadataVersionLogMsg = s"metadata.version ${newImage.features().metadataVersion()}"
if (_firstPublish) { if (_firstPublish) {
info(s"Publishing initial metadata at offset $highestOffsetAndEpoch with $metadataVersionLogMsg.") info(s"Publishing initial metadata at offset $highestOffsetAndEpoch with $metadataVersionLogMsg.")

View File

@ -513,7 +513,7 @@ class KRaftMetadataCache(
_currentImage.scram().describe(request) _currentImage.scram().describe(request)
} }
override def metadataVersion(): MetadataVersion = _currentImage.features().metadataVersion() override def metadataVersion(): MetadataVersion = _currentImage.features().metadataVersionOrThrow()
override def features(): FinalizedFeatures = { override def features(): FinalizedFeatures = {
val image = _currentImage val image = _currentImage
@ -522,7 +522,8 @@ class KRaftMetadataCache(
if (kraftVersionLevel > 0) { if (kraftVersionLevel > 0) {
finalizedFeatures.put(KRaftVersion.FEATURE_NAME, kraftVersionLevel) finalizedFeatures.put(KRaftVersion.FEATURE_NAME, kraftVersionLevel)
} }
new FinalizedFeatures(image.features().metadataVersion(), new FinalizedFeatures(
image.features().metadataVersionOrThrow(),
finalizedFeatures, finalizedFeatures,
image.highestOffsetAndEpoch().offset) image.highestOffsetAndEpoch().offset)
} }

View File

@ -184,7 +184,7 @@ object StorageTool extends Logging {
} catch { } catch {
case e: IllegalArgumentException => case e: IllegalArgumentException =>
throw new TerseFailure(s"Unknown release version '$releaseVersion'. Supported versions are: " + 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") formatParser.addArgument("--release-version", "-r")
.action(store()) .action(store())
.help(s"The release version to use for the initial feature settings. The minimum is " + .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") formatParser.addArgument("--feature", "-f")
.help("The setting to use for a specific feature, in feature=level format. For example: `kraft.version=1`.") .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") versionMappingParser.addArgument("--release-version", "-r")
.action(store()) .action(store())
.help(s"The release version to use for the corresponding feature mapping. The minimum is " + .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 = { private def addFeatureDependenciesParser(subparsers: Subparsers): Unit = {

View File

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

View File

@ -380,7 +380,7 @@ public class ConfigCommandIntegrationTest {
@ClusterTest( @ClusterTest(
// Must be at greater than 1MB per cleaner thread, set to 2M+2 so that we can set 2 cleaner threads. // 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")}, 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() { public void testUnsupportedVersionException() {
try (Admin client = cluster.admin()) { try (Admin client = cluster.admin()) {

View File

@ -44,7 +44,8 @@ public class ReconfigurableQuorumIntegrationTest {
static void checkKRaftVersions(Admin admin, short finalized) throws Exception { static void checkKRaftVersions(Admin admin, short finalized) throws Exception {
FeatureMetadata featureMetadata = admin.describeFeatures().featureMetadata().get(); FeatureMetadata featureMetadata = admin.describeFeatures().featureMetadata().get();
if (finalized > 0) { 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(). assertEquals(finalized, featureMetadata.finalizedFeatures().
get(KRaftVersion.FEATURE_NAME).minVersionLevel()); get(KRaftVersion.FEATURE_NAME).minVersionLevel());
assertEquals(finalized, featureMetadata.finalizedFeatures(). assertEquals(finalized, featureMetadata.finalizedFeatures().
@ -64,13 +65,13 @@ public class ReconfigurableQuorumIntegrationTest {
new TestKitNodes.Builder(). new TestKitNodes.Builder().
setNumBrokerNodes(1). setNumBrokerNodes(1).
setNumControllerNodes(1). setNumControllerNodes(1).
build()).build() build()
) { ).build()) {
cluster.format(); cluster.format();
cluster.startup(); cluster.startup();
try (Admin admin = Admin.create(cluster.clientProperties())) { try (Admin admin = Admin.create(cluster.clientProperties())) {
TestUtils.retryOnExceptionWithTimeout(30_000, () -> { 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(). new TestKitNodes.Builder().
setNumBrokerNodes(1). setNumBrokerNodes(1).
setNumControllerNodes(1). setNumControllerNodes(1).
setFeature(KRaftVersion.FEATURE_NAME, (short) 1). setFeature(KRaftVersion.FEATURE_NAME, KRaftVersion.KRAFT_VERSION_1.featureLevel()).
build()).build() build()
) { ).build()) {
cluster.format(); cluster.format();
cluster.startup(); cluster.startup();
try (Admin admin = Admin.create(cluster.clientProperties())) { try (Admin admin = Admin.create(cluster.clientProperties())) {
TestUtils.retryOnExceptionWithTimeout(30_000, () -> { 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(). new TestKitNodes.Builder().
setNumBrokerNodes(1). setNumBrokerNodes(1).
setNumControllerNodes(3). setNumControllerNodes(3).
setFeature(KRaftVersion.FEATURE_NAME, (short) 1). setFeature(KRaftVersion.FEATURE_NAME, KRaftVersion.KRAFT_VERSION_1.featureLevel()).
build()).build() build()
) { ).build()) {
cluster.format(); cluster.format();
cluster.startup(); cluster.startup();
try (Admin admin = Admin.create(cluster.clientProperties())) { try (Admin admin = Admin.create(cluster.clientProperties())) {
@ -135,7 +136,7 @@ public class ReconfigurableQuorumIntegrationTest {
new TestKitNodes.Builder(). new TestKitNodes.Builder().
setNumBrokerNodes(1). setNumBrokerNodes(1).
setNumControllerNodes(4). setNumControllerNodes(4).
setFeature(KRaftVersion.FEATURE_NAME, (short) 1). setFeature(KRaftVersion.FEATURE_NAME, KRaftVersion.KRAFT_VERSION_1.featureLevel()).
build()).build() build()).build()
) { ) {
cluster.format(); cluster.format();

View File

@ -55,7 +55,7 @@ import scala.jdk.CollectionConverters._
class ProducerIntegrationTest { class ProducerIntegrationTest {
@ClusterTests(Array( @ClusterTests(Array(
new ClusterTest(metadataVersion = MetadataVersion.IBP_3_3_IV0) new ClusterTest(metadataVersion = MetadataVersion.IBP_3_3_IV3)
)) ))
def testUniqueProducerIds(clusterInstance: ClusterInstance): Unit = { def testUniqueProducerIds(clusterInstance: ClusterInstance): Unit = {
verifyUniqueIds(clusterInstance) verifyUniqueIds(clusterInstance)

View File

@ -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) private def doOnStartedKafkaCluster(nodes: TestKitNodes)
(action: KafkaClusterTestKit => Unit): Unit = { (action: KafkaClusterTestKit => Unit): Unit = {
val cluster = new KafkaClusterTestKit.Builder(nodes).build() val cluster = new KafkaClusterTestKit.Builder(nodes).build()
@ -990,7 +978,7 @@ class KRaftClusterTest {
def testUpdateMetadataVersion(): Unit = { def testUpdateMetadataVersion(): Unit = {
val cluster = new KafkaClusterTestKit.Builder( val cluster = new KafkaClusterTestKit.Builder(
new TestKitNodes.Builder(). new TestKitNodes.Builder().
setBootstrapMetadataVersion(MetadataVersion.MINIMUM_BOOTSTRAP_VERSION). setBootstrapMetadataVersion(MetadataVersion.MINIMUM_VERSION).
setNumBrokerNodes(4). setNumBrokerNodes(4).
setNumControllerNodes(3).build()).build() setNumControllerNodes(3).build()).build()
try { try {
@ -1008,8 +996,8 @@ class KRaftClusterTest {
} finally { } finally {
admin.close() admin.close()
} }
TestUtils.waitUntilTrue(() => cluster.brokers().get(0).metadataCache.currentImage().features().metadataVersion().equals(MetadataVersion.latestTesting()), TestUtils.waitUntilTrue(() => cluster.brokers().get(0).metadataCache.currentImage().features().metadataVersion()
"Timed out waiting for metadata.version update") .equals(Optional.of(MetadataVersion.latestTesting())), "Timed out waiting for metadata.version update")
} finally { } finally {
cluster.close() cluster.close()
} }
@ -1143,7 +1131,7 @@ class KRaftClusterTest {
def testSingleControllerSingleBrokerCluster(): Unit = { def testSingleControllerSingleBrokerCluster(): Unit = {
val cluster = new KafkaClusterTestKit.Builder( val cluster = new KafkaClusterTestKit.Builder(
new TestKitNodes.Builder(). new TestKitNodes.Builder().
setBootstrapMetadataVersion(MetadataVersion.MINIMUM_BOOTSTRAP_VERSION). setBootstrapMetadataVersion(MetadataVersion.MINIMUM_VERSION).
setNumBrokerNodes(1). setNumBrokerNodes(1).
setNumControllerNodes(1).build()).build() setNumControllerNodes(1).build()).build()
try { try {

View File

@ -29,12 +29,11 @@ import scala.jdk.CollectionConverters._
class MetadataVersionIntegrationTest { class MetadataVersionIntegrationTest {
@ClusterTests(value = Array( @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_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_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 = { def testBasicMetadataVersionUpgrade(clusterInstance: ClusterInstance): Unit = {
val admin = clusterInstance.admin() val admin = clusterInstance.admin()
@ -45,7 +44,7 @@ class MetadataVersionIntegrationTest {
assertEquals(ff.maxVersionLevel(), clusterInstance.config().metadataVersion().featureLevel()) assertEquals(ff.maxVersionLevel(), clusterInstance.config().metadataVersion().featureLevel())
// Update to new version // 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( val updateResult = admin.updateFeatures(
Map("metadata.version" -> new FeatureUpdate(updateVersion, UpgradeType.UPGRADE)).asJava, new UpdateFeaturesOptions()) Map("metadata.version" -> new FeatureUpdate(updateVersion, UpgradeType.UPGRADE)).asJava, new UpdateFeaturesOptions())
updateResult.all().get() 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 = { def testUpgradeSameVersion(clusterInstance: ClusterInstance): Unit = {
val admin = clusterInstance.admin() val admin = clusterInstance.admin()
try { try {
val updateVersion = MetadataVersion.IBP_3_3_IV0.featureLevel.shortValue val updateVersion = MetadataVersion.IBP_3_9_IV0.featureLevel.shortValue
val updateResult = admin.updateFeatures( val updateResult = admin.updateFeatures(
Map("metadata.version" -> new FeatureUpdate(updateVersion, UpgradeType.UPGRADE)).asJava, new UpdateFeaturesOptions()) Map("metadata.version" -> new FeatureUpdate(updateVersion, UpgradeType.UPGRADE)).asJava, new UpdateFeaturesOptions())
updateResult.all().get() updateResult.all().get()

View File

@ -23,13 +23,13 @@ import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.common.{TopicPartition, Uuid}
import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition
import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset 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.metrics.Metrics
import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.record.{MemoryRecords, SimpleRecord} import org.apache.kafka.common.record.{MemoryRecords, SimpleRecord}
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
import org.apache.kafka.image.{MetadataDelta, MetadataImage, MetadataProvenance} 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.network.BrokerEndPoint
import org.apache.kafka.server.util.{MockScheduler, MockTime} import org.apache.kafka.server.util.{MockScheduler, MockTime}
import org.apache.kafka.storage.internals.log.{AppendOrigin, LogDirFailureChannel} import org.apache.kafka.storage.internals.log.{AppendOrigin, LogDirFailureChannel}
@ -75,6 +75,10 @@ class LocalLeaderEndPointTest extends Logging {
) )
val delta = new MetadataDelta(MetadataImage.EMPTY) val delta = new MetadataDelta(MetadataImage.EMPTY)
delta.replay(new FeatureLevelRecord()
.setName(MetadataVersion.FEATURE_NAME)
.setFeatureLevel(MetadataVersion.MINIMUM_VERSION.featureLevel())
)
delta.replay(new TopicRecord() delta.replay(new TopicRecord()
.setName(topic) .setName(topic)
.setTopicId(topicId) .setTopicId(topicId)

View File

@ -65,7 +65,7 @@ class RemoteLeaderEndPointTest {
blockingSend = new MockBlockingSender(offsets = new util.HashMap[TopicPartition, EpochEndOffset](), blockingSend = new MockBlockingSender(offsets = new util.HashMap[TopicPartition, EpochEndOffset](),
sourceBroker = sourceBroker, time = time) sourceBroker = sourceBroker, time = time)
endPoint = new RemoteLeaderEndPoint(logPrefix, blockingSend, fetchSessionHandler, 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 @Test

View File

@ -24,7 +24,6 @@ import java.util.concurrent.atomic.AtomicBoolean
import kafka.log._ import kafka.log._
import kafka.server._ import kafka.server._
import kafka.utils._ import kafka.utils._
import org.apache.kafka.common.TopicIdPartition
import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.protocol.ApiKeys
import org.apache.kafka.common.record.{MemoryRecords, SimpleRecord} 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.common.{TopicPartition, Uuid}
import org.apache.kafka.coordinator.transaction.TransactionLogConfig import org.apache.kafka.coordinator.transaction.TransactionLogConfig
import org.apache.kafka.metadata.{LeaderAndIsr, MockConfigRepository} 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.config.ReplicationConfigs
import org.apache.kafka.server.storage.log.{FetchIsolation, FetchParams} import org.apache.kafka.server.storage.log.{FetchIsolation, FetchParams}
import org.apache.kafka.server.util.MockTime import org.apache.kafka.server.util.MockTime
@ -74,6 +73,7 @@ class PartitionLockTest extends Logging {
var partition: Partition = _ var partition: Partition = _
private val topicPartition = new TopicPartition("test-topic", 0) private val topicPartition = new TopicPartition("test-topic", 0)
private val topicId = Uuid.randomUuid()
@BeforeEach @BeforeEach
def setUp(): Unit = { 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( when(offsetCheckpoints.fetch(
ArgumentMatchers.anyString, ArgumentMatchers.anyString,
ArgumentMatchers.eq(topicPartition) ArgumentMatchers.eq(topicPartition)
@ -342,7 +342,7 @@ class PartitionLockTest extends Logging {
ArgumentMatchers.any[LeaderAndIsr] ArgumentMatchers.any[LeaderAndIsr]
)).thenReturn(new CompletableFuture[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 controllerEpoch = 0
val replicas = (0 to numReplicaFetchers).map(i => Integer.valueOf(brokerId + i)).toList.asJava val replicas = (0 to numReplicaFetchers).map(i => Integer.valueOf(brokerId + i)).toList.asJava
@ -356,7 +356,7 @@ class PartitionLockTest extends Logging {
.setIsr(isr) .setIsr(isr)
.setPartitionEpoch(1) .setPartitionEpoch(1)
.setReplicas(replicas) .setReplicas(replicas)
.setIsNew(true), offsetCheckpoints, None), "Expected become leader transition to succeed") .setIsNew(true), offsetCheckpoints, Some(topicId)), "Expected become leader transition to succeed")
partition partition
} }

View File

@ -52,7 +52,7 @@ import org.apache.kafka.common.replica.ClientMetadata
import org.apache.kafka.common.replica.ClientMetadata.DefaultClientMetadata import org.apache.kafka.common.replica.ClientMetadata.DefaultClientMetadata
import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol}
import org.apache.kafka.coordinator.transaction.TransactionLogConfig 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.metrics.KafkaYammerMetrics
import org.apache.kafka.server.purgatory.{DelayedOperationPurgatory, TopicPartitionOperationKey} import org.apache.kafka.server.purgatory.{DelayedOperationPurgatory, TopicPartitionOperationKey}
import org.apache.kafka.server.share.fetch.DelayedShareFetchPartitionKey import org.apache.kafka.server.share.fetch.DelayedShareFetchPartitionKey
@ -365,7 +365,7 @@ class PartitionTest extends AbstractPartitionTest {
.setPartitionEpoch(partitionEpoch) .setPartitionEpoch(partitionEpoch)
.setReplicas(replicas.map(Int.box).asJava) .setReplicas(replicas.map(Int.box).asJava)
.setIsNew(true), .setIsNew(true),
offsetCheckpoints, None offsetCheckpoints, topicId
)) ))
assertThrows(classOf[UnknownLeaderEpochException], () => { assertThrows(classOf[UnknownLeaderEpochException], () => {
@ -475,7 +475,7 @@ class PartitionTest extends AbstractPartitionTest {
val localLog = new LocalLog(log.dir, log.config, segments, offsets.recoveryPoint, val localLog = new LocalLog(log.dir, log.config, segments, offsets.recoveryPoint,
offsets.nextOffsetMetadata, mockTime.scheduler, mockTime, log.topicPartition, offsets.nextOffsetMetadata, mockTime.scheduler, mockTime, log.topicPartition,
logDirFailureChannel) 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) .setReplicas(replicas.map(Int.box).asJava)
.setIsNew(true) .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(leaderEpoch, partition.getLeaderEpoch, "Current leader epoch")
assertEquals(Set[Integer](leader, follower2), partition.partitionState.isr, "ISR") assertEquals(Set[Integer](leader, follower2), partition.partitionState.isr, "ISR")
@ -1145,7 +1145,7 @@ class PartitionTest extends AbstractPartitionTest {
.setPartitionEpoch(1) .setPartitionEpoch(1)
.setReplicas(replicas.map(Int.box).asJava) .setReplicas(replicas.map(Int.box).asJava)
.setIsNew(true) .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(leaderEpoch, partition.getLeaderEpoch, "Current leader epoch")
assertEquals(Set[Integer](leader, follower2), partition.partitionState.isr, "ISR") assertEquals(Set[Integer](leader, follower2), partition.partitionState.isr, "ISR")
@ -1182,7 +1182,7 @@ class PartitionTest extends AbstractPartitionTest {
.setPartitionEpoch(1) .setPartitionEpoch(1)
.setReplicas(replicas.map(Int.box).asJava) .setReplicas(replicas.map(Int.box).asJava)
.setIsNew(false) .setIsNew(false)
assertTrue(partition.makeLeader(newLeaderState, offsetCheckpoints, None), assertTrue(partition.makeLeader(newLeaderState, offsetCheckpoints, topicId),
"Expected makeLeader() to return 'leader changed' after makeFollower()") "Expected makeLeader() to return 'leader changed' after makeFollower()")
val currentLeaderEpochStartOffset = partition.localLogOrException.logEndOffset val currentLeaderEpochStartOffset = partition.localLogOrException.logEndOffset
@ -1410,7 +1410,7 @@ class PartitionTest extends AbstractPartitionTest {
@Test @Test
def testInvalidAlterPartitionRequestsAreNotRetried(): Unit = { def testInvalidAlterPartitionRequestsAreNotRetried(): Unit = {
val log = logManager.getOrCreateLog(topicPartition, topicId = None) val log = logManager.getOrCreateLog(topicPartition, topicId = topicId)
seedLogData(log, numRecords = 10, leaderEpoch = 4) seedLogData(log, numRecords = 10, leaderEpoch = 4)
val controllerEpoch = 0 val controllerEpoch = 0
@ -1462,7 +1462,7 @@ class PartitionTest extends AbstractPartitionTest {
@Test @Test
def testIsrExpansion(): Unit = { def testIsrExpansion(): Unit = {
val log = logManager.getOrCreateLog(topicPartition, topicId = None) val log = logManager.getOrCreateLog(topicPartition, topicId = topicId)
seedLogData(log, numRecords = 10, leaderEpoch = 4) seedLogData(log, numRecords = 10, leaderEpoch = 4)
val controllerEpoch = 0 val controllerEpoch = 0
@ -1526,7 +1526,7 @@ class PartitionTest extends AbstractPartitionTest {
@Test @Test
def testIsrNotExpandedIfUpdateFails(): Unit = { def testIsrNotExpandedIfUpdateFails(): Unit = {
val log = logManager.getOrCreateLog(topicPartition, topicId = None) val log = logManager.getOrCreateLog(topicPartition, topicId = topicId)
seedLogData(log, numRecords = 10, leaderEpoch = 4) seedLogData(log, numRecords = 10, leaderEpoch = 4)
val controllerEpoch = 0 val controllerEpoch = 0
@ -1682,7 +1682,7 @@ class PartitionTest extends AbstractPartitionTest {
@ParameterizedTest @ParameterizedTest
@ValueSource(strings = Array("kraft")) @ValueSource(strings = Array("kraft"))
def testIsrNotExpandedIfReplicaIsFencedOrShutdown(quorum: String): Unit = { 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) seedLogData(log, numRecords = 10, leaderEpoch = 4)
val controllerEpoch = 0 val controllerEpoch = 0
@ -1789,7 +1789,7 @@ class PartitionTest extends AbstractPartitionTest {
@Test @Test
def testIsrCanExpandedIfBrokerEpochsMatchWithKraftMetadataCache(): Unit = { def testIsrCanExpandedIfBrokerEpochsMatchWithKraftMetadataCache(): Unit = {
val log = logManager.getOrCreateLog(topicPartition, topicId = None) val log = logManager.getOrCreateLog(topicPartition, topicId = topicId)
seedLogData(log, numRecords = 10, leaderEpoch = 4) seedLogData(log, numRecords = 10, leaderEpoch = 4)
val controllerEpoch = 0 val controllerEpoch = 0
@ -1949,7 +1949,7 @@ class PartitionTest extends AbstractPartitionTest {
@Test @Test
def testIsrNotExpandedIfReplicaIsInControlledShutdown(): Unit = { def testIsrNotExpandedIfReplicaIsInControlledShutdown(): Unit = {
val log = logManager.getOrCreateLog(topicPartition, topicId = None) val log = logManager.getOrCreateLog(topicPartition, topicId = topicId)
seedLogData(log, numRecords = 10, leaderEpoch = 4) seedLogData(log, numRecords = 10, leaderEpoch = 4)
val controllerEpoch = 0 val controllerEpoch = 0
@ -2100,7 +2100,7 @@ class PartitionTest extends AbstractPartitionTest {
@Test @Test
def testMaybeShrinkIsr(): Unit = { def testMaybeShrinkIsr(): Unit = {
val log = logManager.getOrCreateLog(topicPartition, topicId = None) val log = logManager.getOrCreateLog(topicPartition, topicId = topicId)
seedLogData(log, numRecords = 10, leaderEpoch = 4) seedLogData(log, numRecords = 10, leaderEpoch = 4)
val controllerEpoch = 0 val controllerEpoch = 0
@ -2183,7 +2183,7 @@ class PartitionTest extends AbstractPartitionTest {
@Test @Test
def testHighWatermarkAdvanceShouldNotAdvanceWhenUnderMinISR(): Unit = { def testHighWatermarkAdvanceShouldNotAdvanceWhenUnderMinISR(): Unit = {
configRepository.setTopicConfig(topicPartition.topic, TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "3") 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) seedLogData(log, numRecords = 10, leaderEpoch = 4)
val controllerEpoch = 0 val controllerEpoch = 0
@ -2240,7 +2240,7 @@ class PartitionTest extends AbstractPartitionTest {
@Test @Test
def testAlterIsrLeaderAndIsrRace(): Unit = { def testAlterIsrLeaderAndIsrRace(): Unit = {
val log = logManager.getOrCreateLog(topicPartition, topicId = None) val log = logManager.getOrCreateLog(topicPartition, topicId = topicId)
seedLogData(log, numRecords = 10, leaderEpoch = 4) seedLogData(log, numRecords = 10, leaderEpoch = 4)
val controllerEpoch = 0 val controllerEpoch = 0
@ -2251,7 +2251,7 @@ class PartitionTest extends AbstractPartitionTest {
val initializeTimeMs = time.milliseconds() val initializeTimeMs = time.milliseconds()
assertTrue(makeLeader( assertTrue(makeLeader(
topicId = None, topicId = topicId,
controllerEpoch = controllerEpoch, controllerEpoch = controllerEpoch,
leaderEpoch = leaderEpoch, leaderEpoch = leaderEpoch,
isr = isr, isr = isr,
@ -2274,7 +2274,7 @@ class PartitionTest extends AbstractPartitionTest {
// Become leader again, reset the ISR state // Become leader again, reset the ISR state
assertFalse(makeLeader( assertFalse(makeLeader(
topicId = None, topicId = topicId,
controllerEpoch = controllerEpoch, controllerEpoch = controllerEpoch,
leaderEpoch = leaderEpoch, leaderEpoch = leaderEpoch,
isr = isr, isr = isr,
@ -2298,7 +2298,7 @@ class PartitionTest extends AbstractPartitionTest {
@Test @Test
def testShouldNotShrinkIsrIfPreviousFetchIsCaughtUp(): Unit = { def testShouldNotShrinkIsrIfPreviousFetchIsCaughtUp(): Unit = {
val log = logManager.getOrCreateLog(topicPartition, topicId = None) val log = logManager.getOrCreateLog(topicPartition, topicId = topicId)
seedLogData(log, numRecords = 10, leaderEpoch = 4) seedLogData(log, numRecords = 10, leaderEpoch = 4)
val controllerEpoch = 0 val controllerEpoch = 0
@ -2310,7 +2310,7 @@ class PartitionTest extends AbstractPartitionTest {
addBrokerEpochToMockMetadataCache(metadataCache, replicas.toList) addBrokerEpochToMockMetadataCache(metadataCache, replicas.toList)
assertTrue(makeLeader( assertTrue(makeLeader(
topicId = None, topicId = topicId,
controllerEpoch = controllerEpoch, controllerEpoch = controllerEpoch,
leaderEpoch = leaderEpoch, leaderEpoch = leaderEpoch,
isr = isr, isr = isr,
@ -2358,7 +2358,7 @@ class PartitionTest extends AbstractPartitionTest {
@Test @Test
def testShouldNotShrinkIsrIfFollowerCaughtUpToLogEnd(): Unit = { def testShouldNotShrinkIsrIfFollowerCaughtUpToLogEnd(): Unit = {
val log = logManager.getOrCreateLog(topicPartition, topicId = None) val log = logManager.getOrCreateLog(topicPartition, topicId = topicId)
seedLogData(log, numRecords = 10, leaderEpoch = 4) seedLogData(log, numRecords = 10, leaderEpoch = 4)
val controllerEpoch = 0 val controllerEpoch = 0
@ -2370,7 +2370,7 @@ class PartitionTest extends AbstractPartitionTest {
addBrokerEpochToMockMetadataCache(metadataCache, replicas.toList) addBrokerEpochToMockMetadataCache(metadataCache, replicas.toList)
assertTrue(makeLeader( assertTrue(makeLeader(
topicId = None, topicId = topicId,
controllerEpoch = controllerEpoch, controllerEpoch = controllerEpoch,
leaderEpoch = leaderEpoch, leaderEpoch = leaderEpoch,
isr = isr, isr = isr,
@ -2406,7 +2406,7 @@ class PartitionTest extends AbstractPartitionTest {
@Test @Test
def testIsrNotShrunkIfUpdateFails(): Unit = { def testIsrNotShrunkIfUpdateFails(): Unit = {
val log = logManager.getOrCreateLog(topicPartition, topicId = None) val log = logManager.getOrCreateLog(topicPartition, topicId = topicId)
seedLogData(log, numRecords = 10, leaderEpoch = 4) seedLogData(log, numRecords = 10, leaderEpoch = 4)
val controllerEpoch = 0 val controllerEpoch = 0
@ -2417,7 +2417,7 @@ class PartitionTest extends AbstractPartitionTest {
val initializeTimeMs = time.milliseconds() val initializeTimeMs = time.milliseconds()
assertTrue(makeLeader( assertTrue(makeLeader(
topicId = None, topicId = topicId,
controllerEpoch = controllerEpoch, controllerEpoch = controllerEpoch,
leaderEpoch = leaderEpoch, leaderEpoch = leaderEpoch,
isr = isr, isr = isr,
@ -2493,7 +2493,7 @@ class PartitionTest extends AbstractPartitionTest {
} }
def handleAlterIsrFailure(error: Errors, callback: (Int, Int, Partition) => Unit): Unit = { 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) seedLogData(log, numRecords = 10, leaderEpoch = 4)
val controllerEpoch = 0 val controllerEpoch = 0
@ -2504,7 +2504,7 @@ class PartitionTest extends AbstractPartitionTest {
addBrokerEpochToMockMetadataCache(metadataCache, replicas.toList) addBrokerEpochToMockMetadataCache(metadataCache, replicas.toList)
assertTrue(makeLeader( assertTrue(makeLeader(
topicId = None, topicId = topicId,
controllerEpoch = controllerEpoch, controllerEpoch = controllerEpoch,
leaderEpoch = leaderEpoch, leaderEpoch = leaderEpoch,
isr = isr, isr = isr,
@ -2547,8 +2547,7 @@ class PartitionTest extends AbstractPartitionTest {
partitionEpoch: Int = 0 partitionEpoch: Int = 0
): ClientResponse = { ): ClientResponse = {
val alterPartitionResponseData = new AlterPartitionResponseData() val alterPartitionResponseData = new AlterPartitionResponseData()
val topicResponse = new AlterPartitionResponseData.TopicData() val topicResponse = new AlterPartitionResponseData.TopicData().setTopicId(topicId.get)
.setTopicName(topicPartition.topic)
topicResponse.partitions.add(new AlterPartitionResponseData.PartitionData() topicResponse.partitions.add(new AlterPartitionResponseData.PartitionData()
.setPartitionIndex(topicPartition.partition) .setPartitionIndex(topicPartition.partition)
@ -2572,8 +2571,7 @@ class PartitionTest extends AbstractPartitionTest {
scheduler = mock(classOf[KafkaScheduler]), scheduler = mock(classOf[KafkaScheduler]),
time = time, time = time,
brokerId = brokerId, brokerId = brokerId,
brokerEpochSupplier = () => 0, brokerEpochSupplier = () => 0
metadataVersionSupplier = () => MetadataVersion.IBP_3_0_IV1
) )
partition = new Partition(topicPartition, partition = new Partition(topicPartition,
@ -2587,7 +2585,7 @@ class PartitionTest extends AbstractPartitionTest {
logManager, logManager,
alterPartitionManager) alterPartitionManager)
val log = logManager.getOrCreateLog(topicPartition, topicId = None) val log = logManager.getOrCreateLog(topicPartition, topicId = topicId)
seedLogData(log, numRecords = 10, leaderEpoch = 4) seedLogData(log, numRecords = 10, leaderEpoch = 4)
val controllerEpoch = 0 val controllerEpoch = 0
@ -2621,7 +2619,7 @@ class PartitionTest extends AbstractPartitionTest {
} }
assertTrue(makeLeader( assertTrue(makeLeader(
topicId = None, topicId = topicId,
controllerEpoch, controllerEpoch,
leaderEpoch, leaderEpoch,
isr, isr,
@ -2644,7 +2642,7 @@ class PartitionTest extends AbstractPartitionTest {
@Test @Test
def testSingleInFlightAlterIsr(): Unit = { def testSingleInFlightAlterIsr(): Unit = {
val log = logManager.getOrCreateLog(topicPartition, topicId = None) val log = logManager.getOrCreateLog(topicPartition, topicId = topicId)
seedLogData(log, numRecords = 10, leaderEpoch = 4) seedLogData(log, numRecords = 10, leaderEpoch = 4)
val controllerEpoch = 0 val controllerEpoch = 0
@ -2659,7 +2657,7 @@ class PartitionTest extends AbstractPartitionTest {
doNothing().when(delayedOperations).checkAndCompleteAll() doNothing().when(delayedOperations).checkAndCompleteAll()
assertTrue(makeLeader( assertTrue(makeLeader(
topicId = None, topicId = topicId,
controllerEpoch = controllerEpoch, controllerEpoch = controllerEpoch,
leaderEpoch = leaderEpoch, leaderEpoch = leaderEpoch,
isr = isr, isr = isr,
@ -2685,7 +2683,7 @@ class PartitionTest extends AbstractPartitionTest {
@Test @Test
def testUseCheckpointToInitializeHighWatermark(): Unit = { def testUseCheckpointToInitializeHighWatermark(): Unit = {
val log = logManager.getOrCreateLog(topicPartition, topicId = None) val log = logManager.getOrCreateLog(topicPartition, topicId = topicId)
seedLogData(log, numRecords = 6, leaderEpoch = 5) seedLogData(log, numRecords = 6, leaderEpoch = 5)
when(offsetCheckpoints.fetch(logDir1.getAbsolutePath, topicPartition)) when(offsetCheckpoints.fetch(logDir1.getAbsolutePath, topicPartition))
@ -3266,7 +3264,7 @@ class PartitionTest extends AbstractPartitionTest {
@Test @Test
def testAddAndRemoveListeners(): Unit = { 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 replicas = Seq(brokerId, brokerId + 1)
val isr = replicas val isr = replicas
@ -3339,7 +3337,7 @@ class PartitionTest extends AbstractPartitionTest {
@Test @Test
def testAddListenerFailsWhenPartitionIsDeleted(): Unit = { def testAddListenerFailsWhenPartitionIsDeleted(): Unit = {
partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, topicId = None) partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, topicId = topicId)
partition.makeLeader( partition.makeLeader(
new LeaderAndIsrRequest.PartitionState() new LeaderAndIsrRequest.PartitionState()
@ -3360,7 +3358,7 @@ class PartitionTest extends AbstractPartitionTest {
@Test @Test
def testPartitionListenerWhenLogOffsetsChanged(): Unit = { 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 replicas = Seq(brokerId, brokerId + 1)
val isr = Seq(brokerId, brokerId + 1) val isr = Seq(brokerId, brokerId + 1)
@ -3405,7 +3403,7 @@ class PartitionTest extends AbstractPartitionTest {
@Test @Test
def testPartitionListenerWhenPartitionFailed(): Unit = { def testPartitionListenerWhenPartitionFailed(): Unit = {
partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, topicId = None) partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, topicId = topicId)
partition.makeLeader( partition.makeLeader(
new LeaderAndIsrRequest.PartitionState() new LeaderAndIsrRequest.PartitionState()
@ -3429,7 +3427,7 @@ class PartitionTest extends AbstractPartitionTest {
@Test @Test
def testPartitionListenerWhenPartitionIsDeleted(): Unit = { def testPartitionListenerWhenPartitionIsDeleted(): Unit = {
partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, topicId = None) partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, topicId = topicId)
partition.makeLeader( partition.makeLeader(
new LeaderAndIsrRequest.PartitionState() new LeaderAndIsrRequest.PartitionState()
@ -3454,7 +3452,7 @@ class PartitionTest extends AbstractPartitionTest {
@Test @Test
def testPartitionListenerWhenCurrentIsReplacedWithFutureLog(): Unit = { def testPartitionListenerWhenCurrentIsReplacedWithFutureLog(): Unit = {
logManager.maybeUpdatePreferredLogDir(topicPartition, logDir1.getAbsolutePath) 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) assertTrue(partition.log.isDefined)
val replicas = Seq(brokerId, brokerId + 1) val replicas = Seq(brokerId, brokerId + 1)
@ -3627,6 +3625,7 @@ class PartitionTest extends AbstractPartitionTest {
private class SlowLog( private class SlowLog(
log: UnifiedLog, log: UnifiedLog,
topicId: Option[Uuid],
logStartOffset: Long, logStartOffset: Long,
localLog: LocalLog, localLog: LocalLog,
leaderEpochCache: LeaderEpochFileCache, leaderEpochCache: LeaderEpochFileCache,
@ -3639,7 +3638,7 @@ class PartitionTest extends AbstractPartitionTest {
log.producerIdExpirationCheckIntervalMs, log.producerIdExpirationCheckIntervalMs,
leaderEpochCache, leaderEpochCache,
producerStateManager, producerStateManager,
_topicId = None) { _topicId = topicId) {
override def appendAsFollower(records: MemoryRecords): LogAppendInfo = { override def appendAsFollower(records: MemoryRecords): LogAppendInfo = {
appendSemaphore.acquire() appendSemaphore.acquire()

View File

@ -24,6 +24,7 @@ import kafka.network
import org.apache.kafka.common.memory.MemoryPool import org.apache.kafka.common.memory.MemoryPool
import org.apache.kafka.common.message._ import org.apache.kafka.common.message._
import org.apache.kafka.common.network.{ClientInformation, ListenerName, NetworkSend} 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.requests._
import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol}
import org.apache.kafka.network.RequestConvertToJson import org.apache.kafka.network.RequestConvertToJson
@ -38,7 +39,7 @@ class RequestConvertToJsonTest {
@Test @Test
def testRequestHeaderNode(): Unit = { 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 req = request(alterIsrRequest)
val header = req.header val header = req.header
@ -52,7 +53,7 @@ class RequestConvertToJsonTest {
@Test @Test
def testRequestDesc(): Unit = { 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 req = request(alterIsrRequest)
val expectedNode = new ObjectNode(JsonNodeFactory.instance) val expectedNode = new ObjectNode(JsonNodeFactory.instance)
@ -67,7 +68,7 @@ class RequestConvertToJsonTest {
@Test @Test
def testRequestDescMetrics(): Unit = { 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 req = request(alterIsrRequest)
val send = new NetworkSend(req.context.connectionId, alterIsrRequest.toSend(req.header)) val send = new NetworkSend(req.context.connectionId, alterIsrRequest.toSend(req.header))
val headerLog = RequestConvertToJson.requestHeaderNode(req.header) val headerLog = RequestConvertToJson.requestHeaderNode(req.header)

View File

@ -69,7 +69,7 @@ abstract class AbstractApiVersionsRequestTest(cluster: ClusterInstance) {
assertEquals(MetadataVersion.latestTesting().featureLevel(), apiVersionsResponse.data().finalizedFeatures().find(MetadataVersion.FEATURE_NAME).maxVersionLevel()) assertEquals(MetadataVersion.latestTesting().featureLevel(), apiVersionsResponse.data().finalizedFeatures().find(MetadataVersion.FEATURE_NAME).maxVersionLevel())
assertEquals(5, apiVersionsResponse.data().supportedFeatures().size()) 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) { if (apiVersion < 4) {
assertEquals(1, apiVersionsResponse.data().supportedFeatures().find("kraft.version").minVersion()) assertEquals(1, apiVersionsResponse.data().supportedFeatures().find("kraft.version").minVersion())
} else { } else {

View File

@ -18,9 +18,7 @@
package kafka.server package kafka.server
import java.util.Collections import java.util.Collections
import java.util.stream.{Stream => JStream}
import org.apache.kafka.clients.ClientResponse import org.apache.kafka.clients.ClientResponse
import org.apache.kafka.common.TopicIdPartition
import org.apache.kafka.common.Uuid import org.apache.kafka.common.Uuid
import org.apache.kafka.common.errors.{AuthenticationException, OperationNotAttemptedException, UnknownServerException, UnsupportedVersionException} import org.apache.kafka.common.errors.{AuthenticationException, OperationNotAttemptedException, UnknownServerException, UnsupportedVersionException}
import org.apache.kafka.common.message.AlterPartitionRequestData.BrokerState 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.RequestHeader
import org.apache.kafka.common.requests.{AbstractRequest, AlterPartitionRequest, AlterPartitionResponse} import org.apache.kafka.common.requests.{AbstractRequest, AlterPartitionRequest, AlterPartitionResponse}
import org.apache.kafka.metadata.{LeaderAndIsr, LeaderRecoveryState} 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.common.MetadataVersion.{IBP_3_0_IV1, IBP_3_2_IV0, IBP_3_5_IV1}
import org.apache.kafka.server.util.{MockScheduler, MockTime} import org.apache.kafka.server.util.{MockScheduler, MockTime}
import org.apache.kafka.test.TestUtils.assertFutureThrows import org.apache.kafka.test.TestUtils.assertFutureThrows
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.BeforeEach import org.junit.jupiter.api.BeforeEach
import org.junit.jupiter.api.Test import org.junit.jupiter.api.Test
import org.junit.jupiter.params.ParameterizedTest 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.mockito.ArgumentMatcher import org.mockito.ArgumentMatcher
import org.mockito.ArgumentMatchers.any import org.mockito.ArgumentMatchers.any
import org.mockito.Mockito.{mock, reset, times, verify} import org.mockito.Mockito.{mock, reset, times, verify}
@ -51,8 +47,6 @@ import scala.collection.mutable.ListBuffer
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
class AlterPartitionManagerTest { class AlterPartitionManagerTest {
val topic = "test-topic"
val topicId = Uuid.randomUuid() val topicId = Uuid.randomUuid()
val time = new MockTime val time = new MockTime
val metrics = new Metrics val metrics = new Metrics
@ -60,31 +54,29 @@ class AlterPartitionManagerTest {
var brokerToController: NodeToControllerChannelManager = _ var brokerToController: NodeToControllerChannelManager = _
val tp0 = new TopicIdPartition(topicId, 0, topic) val tp0 = new TopicIdPartition(topicId, 0)
val tp1 = new TopicIdPartition(topicId, 1, topic) val tp1 = new TopicIdPartition(topicId, 1)
val tp2 = new TopicIdPartition(topicId, 2, topic) val tp2 = new TopicIdPartition(topicId, 2)
@BeforeEach @BeforeEach
def setup(): Unit = { def setup(): Unit = {
brokerToController = mock(classOf[NodeToControllerChannelManager]) brokerToController = mock(classOf[NodeToControllerChannelManager])
} }
@ParameterizedTest @Test
@MethodSource(Array("provideMetadataVersions")) def testBasic(): Unit = {
def testBasic(metadataVersion: MetadataVersion): Unit = {
val scheduler = new MockScheduler(time) 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.start()
alterPartitionManager.submit(tp0, new LeaderAndIsr(1, 1, List(1, 2, 3).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, 10)) alterPartitionManager.submit(tp0, new LeaderAndIsr(1, 1, List(1, 2, 3).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, 10))
verify(brokerToController).start() verify(brokerToController).start()
verify(brokerToController).sendRequest(any(), any()) verify(brokerToController).sendRequest(any(), any())
} }
@ParameterizedTest @Test
@MethodSource(Array("provideMetadataVersions")) def testBasicWithBrokerEpoch(): Unit = {
def testBasicWithBrokerEpoch(metadataVersion: MetadataVersion): Unit = {
val scheduler = new MockScheduler(time) 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() alterPartitionManager.start()
val isrWithBrokerEpoch = ListBuffer[BrokerState]() val isrWithBrokerEpoch = ListBuffer[BrokerState]()
for (ii <- 1 to 3) { for (ii <- 1 to 3) {
@ -96,7 +88,6 @@ class AlterPartitionManagerTest {
.setBrokerId(brokerId) .setBrokerId(brokerId)
.setBrokerEpoch(101) .setBrokerEpoch(101)
val topicData = new AlterPartitionRequestData.TopicData() val topicData = new AlterPartitionRequestData.TopicData()
.setTopicName(topic)
.setTopicId(topicId) .setTopicId(topicId)
val newIsrWithBrokerEpoch = new ListBuffer[BrokerState]() val newIsrWithBrokerEpoch = new ListBuffer[BrokerState]()
@ -118,33 +109,28 @@ class AlterPartitionManagerTest {
} }
@ParameterizedTest @ParameterizedTest
@MethodSource(Array("provideLeaderRecoveryState")) @EnumSource(classOf[LeaderRecoveryState])
def testBasicSentLeaderRecoveryState( def testBasicSentLeaderRecoveryState(leaderRecoveryState: LeaderRecoveryState): Unit = {
metadataVersion: MetadataVersion,
leaderRecoveryState: LeaderRecoveryState
): Unit = {
val requestCapture = ArgumentCaptor.forClass(classOf[AbstractRequest.Builder[AlterPartitionRequest]]) val requestCapture = ArgumentCaptor.forClass(classOf[AbstractRequest.Builder[AlterPartitionRequest]])
val scheduler = new MockScheduler(time) 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.start()
alterPartitionManager.submit(tp0, new LeaderAndIsr(1, 1, List(1).map(Int.box).asJava, leaderRecoveryState, 10)) alterPartitionManager.submit(tp0, new LeaderAndIsr(1, 1, List(1).map(Int.box).asJava, leaderRecoveryState, 10))
verify(brokerToController).start() verify(brokerToController).start()
verify(brokerToController).sendRequest(requestCapture.capture(), any()) verify(brokerToController).sendRequest(requestCapture.capture(), any())
val request = requestCapture.getValue.build() val request = requestCapture.getValue.build()
val expectedLeaderRecoveryState = if (metadataVersion.isAtLeast(IBP_3_2_IV0)) leaderRecoveryState else LeaderRecoveryState.RECOVERED assertEquals(leaderRecoveryState.value, request.data.topics.get(0).partitions.get(0).leaderRecoveryState())
assertEquals(expectedLeaderRecoveryState.value, request.data.topics.get(0).partitions.get(0).leaderRecoveryState())
} }
@ParameterizedTest @Test
@MethodSource(Array("provideMetadataVersions")) def testOverwriteWithinBatch(): Unit = {
def testOverwriteWithinBatch(metadataVersion: MetadataVersion): Unit = {
val capture: ArgumentCaptor[AbstractRequest.Builder[AlterPartitionRequest]] = ArgumentCaptor.forClass(classOf[AbstractRequest.Builder[AlterPartitionRequest]]) val capture: ArgumentCaptor[AbstractRequest.Builder[AlterPartitionRequest]] = ArgumentCaptor.forClass(classOf[AbstractRequest.Builder[AlterPartitionRequest]])
val callbackCapture: ArgumentCaptor[ControllerRequestCompletionHandler] = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler]) val callbackCapture: ArgumentCaptor[ControllerRequestCompletionHandler] = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler])
val scheduler = new MockScheduler(time) 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.start()
// Only send one ISR update for a given topic+partition // Only send one ISR update for a given topic+partition
@ -181,23 +167,22 @@ class AlterPartitionManagerTest {
} }
} }
@ParameterizedTest @Test
@MethodSource(Array("provideMetadataVersions")) def testSingleBatch(): Unit = {
def testSingleBatch(metadataVersion: MetadataVersion): Unit = {
val capture: ArgumentCaptor[AbstractRequest.Builder[AlterPartitionRequest]] = ArgumentCaptor.forClass(classOf[AbstractRequest.Builder[AlterPartitionRequest]]) val capture: ArgumentCaptor[AbstractRequest.Builder[AlterPartitionRequest]] = ArgumentCaptor.forClass(classOf[AbstractRequest.Builder[AlterPartitionRequest]])
val callbackCapture: ArgumentCaptor[ControllerRequestCompletionHandler] = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler]) val callbackCapture: ArgumentCaptor[ControllerRequestCompletionHandler] = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler])
val scheduler = new MockScheduler(time) 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.start()
// First request will send batch of one // 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)) new LeaderAndIsr(1, 1, List(1, 2, 3).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, 10))
// Other submissions will queue up until a response // Other submissions will queue up until a response
for (i <- 1 to 9) { 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)) 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 callbackCapture = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler])
val scheduler = new MockScheduler(time) 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.start()
val future = alterPartitionManager.submit(tp0, leaderAndIsr) val future = alterPartitionManager.submit(tp0, leaderAndIsr)
val finalFuture = new CompletableFuture[LeaderAndIsr]() val finalFuture = new CompletableFuture[LeaderAndIsr]()
@ -266,7 +251,7 @@ class AlterPartitionManagerTest {
assertEquals(leaderAndIsr, finalFuture.get(200, TimeUnit.MILLISECONDS)) assertEquals(leaderAndIsr, finalFuture.get(200, TimeUnit.MILLISECONDS))
// No more items in unsentIsrUpdates // No more items in unsentIsrUpdates
assertFalse(alterPartitionManager.unsentIsrUpdates.containsKey(tp0.topicPartition)) assertFalse(alterPartitionManager.unsentIsrUpdates.containsKey(tp0))
} }
@Test @Test
@ -307,7 +292,7 @@ class AlterPartitionManagerTest {
val callbackCapture: ArgumentCaptor[ControllerRequestCompletionHandler] = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler]) val callbackCapture: ArgumentCaptor[ControllerRequestCompletionHandler] = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler])
val scheduler = new MockScheduler(time) 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.start()
alterPartitionManager.submit(tp0, leaderAndIsr) alterPartitionManager.submit(tp0, leaderAndIsr)
@ -316,7 +301,7 @@ class AlterPartitionManagerTest {
callbackCapture.getValue.onComplete(response) callbackCapture.getValue.onComplete(response)
// Any top-level error, we want to retry, so we don't clear items from the pending map // 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) reset(brokerToController)
@ -331,7 +316,7 @@ class AlterPartitionManagerTest {
verify(brokerToController).sendRequest(any(), callbackCapture.capture()) verify(brokerToController).sendRequest(any(), callbackCapture.capture())
callbackCapture.getValue.onComplete(retryResponse) callbackCapture.getValue.onComplete(retryResponse)
assertFalse(alterPartitionManager.unsentIsrUpdates.containsKey(tp0.topicPartition)) assertFalse(alterPartitionManager.unsentIsrUpdates.containsKey(tp0))
} }
@Test @Test
@ -366,7 +351,7 @@ class AlterPartitionManagerTest {
reset(brokerToController) reset(brokerToController)
val scheduler = new MockScheduler(time) 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.start()
val future = alterPartitionManager.submit(tp, new LeaderAndIsr(1, 1, List(1, 2, 3).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, 10)) 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 alterPartitionManager
} }
@ParameterizedTest @Test
@MethodSource(Array("provideMetadataVersions")) def testOneInFlight(): Unit = {
def testOneInFlight(metadataVersion: MetadataVersion): Unit = {
val callbackCapture: ArgumentCaptor[ControllerRequestCompletionHandler] = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler]) val callbackCapture: ArgumentCaptor[ControllerRequestCompletionHandler] = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler])
val scheduler = new MockScheduler(time) 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.start()
// First submit will send the request // First submit will send the request
@ -410,9 +394,8 @@ class AlterPartitionManagerTest {
callbackCapture.getValue.onComplete(resp) callbackCapture.getValue.onComplete(resp)
} }
@ParameterizedTest @Test
@MethodSource(Array("provideMetadataVersions")) def testPartitionMissingInResponse(): Unit = {
def testPartitionMissingInResponse(metadataVersion: MetadataVersion): Unit = {
val expectedVersion = ApiKeys.ALTER_PARTITION.latestVersion val expectedVersion = ApiKeys.ALTER_PARTITION.latestVersion
val leaderAndIsr = new LeaderAndIsr(1, 1, List(1, 2, 3).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, 10) val leaderAndIsr = new LeaderAndIsr(1, 1, List(1, 2, 3).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, 10)
val brokerEpoch = 2 val brokerEpoch = 2
@ -423,8 +406,7 @@ class AlterPartitionManagerTest {
scheduler, scheduler,
time, time,
brokerId, brokerId,
() => brokerEpoch, () => brokerEpoch
() => metadataVersion
) )
alterPartitionManager.start() alterPartitionManager.start()
@ -498,13 +480,13 @@ class AlterPartitionManagerTest {
val alterPartitionRequest = request.asInstanceOf[AlterPartitionRequest.Builder].build() val alterPartitionRequest = request.asInstanceOf[AlterPartitionRequest.Builder].build()
assertEquals(expectedVersion, alterPartitionRequest.version) 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 => topicData.partitions.asScala.map { partitionData =>
new TopicIdPartition(topicData.topicId, partitionData.partitionIndex, topicData.topicName) (topicData.topicId, partitionData.partitionIndex)
} }
}.toSet }.toSet
expectedTopicPartitions == requestTopicPartitions expectedTopicPartitions.map(tp => (tp.topicId, tp.partitionId)) == requestTopicIdPartitions
} }
} }
@ -538,11 +520,10 @@ class AlterPartitionManagerTest {
new AlterPartitionResponse(new AlterPartitionResponseData() new AlterPartitionResponse(new AlterPartitionResponseData()
.setTopics(Collections.singletonList( .setTopics(Collections.singletonList(
new AlterPartitionResponseData.TopicData() new AlterPartitionResponseData.TopicData()
.setTopicName(tp.topic)
.setTopicId(tp.topicId) .setTopicId(tp.topicId)
.setPartitions(Collections.singletonList( .setPartitions(Collections.singletonList(
new AlterPartitionResponseData.PartitionData() new AlterPartitionResponseData.PartitionData()
.setPartitionIndex(tp.partition) .setPartitionIndex(tp.partitionId)
.setPartitionEpoch(partitionEpoch) .setPartitionEpoch(partitionEpoch)
.setLeaderEpoch(leaderEpoch) .setLeaderEpoch(leaderEpoch)
.setLeaderId(leaderId) .setLeaderId(leaderId)
@ -550,26 +531,3 @@ class AlterPartitionManagerTest {
.setErrorCode(error.code)))))) .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)
)
}
}
}

View File

@ -18,9 +18,11 @@ package kafka.server
import org.apache.kafka.clients.NodeApiVersions import org.apache.kafka.clients.NodeApiVersions
import org.apache.kafka.common.message.ApiMessageType.ListenerType 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.common.protocol.ApiKeys
import org.apache.kafka.image.{MetadataDelta, MetadataImage, MetadataProvenance}
import org.apache.kafka.server.BrokerFeatures 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.Test
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.ParameterizedTest
@ -31,7 +33,16 @@ import scala.jdk.CollectionConverters._
class ApiVersionManagerTest { class ApiVersionManagerTest {
private val brokerFeatures = BrokerFeatures.createDefault(true) 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 @ParameterizedTest
@EnumSource(classOf[ListenerType]) @EnumSource(classOf[ListenerType])

View File

@ -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.ApiVersionsRequest
import org.apache.kafka.common.requests.ApiVersionsResponse import org.apache.kafka.common.requests.ApiVersionsResponse
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.server.config.ServerConfigs import org.apache.kafka.server.config.ServerConfigs
import org.junit.jupiter.api.Assertions.{assertEquals, assertNotNull, assertNull} import org.junit.jupiter.api.Assertions.{assertEquals, assertNotNull, assertNull}
import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.api.Test
import org.junit.jupiter.params.provider.ValueSource
import java.util.Properties import java.util.Properties
@ -38,28 +38,18 @@ class ApiVersionsResponseIntegrationTest extends BaseRequestTest {
connectAndReceive[ApiVersionsResponse](request) connectAndReceive[ApiVersionsResponse](request)
} }
@ParameterizedTest @Test
@ValueSource(strings = Array("kraft")) def testSendV3ApiVersionsRequest(): Unit = {
def testSendV3ApiVersionsRequest(quorum: String): Unit = {
val response = sendApiVersionsRequest(3) val response = sendApiVersionsRequest(3)
if (quorum.equals("kraft")) { assertFeatureHasMinVersion("metadata.version", response.data().supportedFeatures(), MetadataVersion.MINIMUM_VERSION.featureLevel())
assertFeatureHasMinVersion("metadata.version", response.data().supportedFeatures(), 1) assertFeatureMissing("kraft.version", response.data().supportedFeatures())
assertFeatureMissing("kraft.version", response.data().supportedFeatures())
} else {
assertEquals(0, response.data().supportedFeatures().size())
}
} }
@ParameterizedTest @Test
@ValueSource(strings = Array("kraft")) def testSendV4ApiVersionsRequest(): Unit = {
def testSendV4ApiVersionsRequest(quorum: String): Unit = {
val response = sendApiVersionsRequest(4) val response = sendApiVersionsRequest(4)
if (quorum.equals("kraft")) { assertFeatureHasMinVersion("metadata.version", response.data().supportedFeatures(), MetadataVersion.MINIMUM_VERSION.featureLevel())
assertFeatureHasMinVersion("metadata.version", response.data().supportedFeatures(), 1) assertFeatureHasMinVersion("kraft.version", response.data().supportedFeatures(), 0)
assertFeatureHasMinVersion("kraft.version", response.data().supportedFeatures(), 0)
} else {
assertEquals(0, response.data().supportedFeatures().size())
}
} }
def assertFeatureHasMinVersion( def assertFeatureHasMinVersion(

View File

@ -29,7 +29,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.common.test.ClusterInstance
import org.apache.kafka.common.utils.Time import org.apache.kafka.common.utils.Time
import org.apache.kafka.common.{Node, Uuid} import org.apache.kafka.common.{Node, Uuid}
import org.apache.kafka.server.common.{ControllerRequestCompletionHandler, Feature, MetadataVersion, NodeToControllerChannelManager} import org.apache.kafka.server.common.{ControllerRequestCompletionHandler, Feature, MetadataVersion, MetadataVersionTestUtils, NodeToControllerChannelManager}
import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.Assertions.assertEquals
import java.util import java.util
@ -88,16 +88,15 @@ class BrokerRegistrationRequestTest {
clusterId: String, clusterId: String,
brokerId: Int, brokerId: Int,
zkEpoch: Option[Long], zkEpoch: Option[Long],
ibpToSend: Option[(MetadataVersion, MetadataVersion)] featureLevelToSend: Option[(Short, Short)]
): Errors = { ): Errors = {
val features = new BrokerRegistrationRequestData.FeatureCollection() val features = new BrokerRegistrationRequestData.FeatureCollection()
ibpToSend foreach { featureLevelToSend.foreach { case (min, max) =>
case (min, max) => features.add(new BrokerRegistrationRequestData.Feature()
features.add(new BrokerRegistrationRequestData.Feature() .setName(MetadataVersion.FEATURE_NAME)
.setName(MetadataVersion.FEATURE_NAME) .setMinSupportedVersion(min)
.setMinSupportedVersion(min.featureLevel()) .setMaxSupportedVersion(max)
.setMaxSupportedVersion(max.featureLevel()) )
)
} }
Feature.PRODUCTION_FEATURES.stream().filter(_.featureName != MetadataVersion.FEATURE_NAME).forEach { Feature.PRODUCTION_FEATURES.stream().filter(_.featureName != MetadataVersion.FEATURE_NAME).forEach {
feature => feature =>
@ -150,7 +149,7 @@ class BrokerRegistrationRequestTest {
// Invalid registration (isMigratingZkBroker, but MV does not support migrations) // Invalid registration (isMigratingZkBroker, but MV does not support migrations)
assertEquals( assertEquals(
Errors.BROKER_ID_NOT_REGISTERED, 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 // No features (MV) sent with registration, controller can't verify
assertEquals( assertEquals(
@ -160,12 +159,12 @@ class BrokerRegistrationRequestTest {
// Given MV is too high for controller to support // Given MV is too high for controller to support
assertEquals( assertEquals(
Errors.BROKER_ID_NOT_REGISTERED, 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 // Controller supports this MV and isMigratingZkBroker is false, so this one works
assertEquals( assertEquals(
Errors.NONE, 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 { } finally {
channelManager.shutdown() channelManager.shutdown()
} }

View File

@ -411,7 +411,7 @@ class ControllerApisTest {
assertThrows(classOf[ClusterAuthorizationException], () => { assertThrows(classOf[ClusterAuthorizationException], () => {
controllerApis = createControllerApis(Some(createDenyAllAuthorizer()), new MockController.Builder().build()) controllerApis = createControllerApis(Some(createDenyAllAuthorizer()), new MockController.Builder().build())
controllerApis.handleAlterPartitionRequest(buildRequest(new AlterPartitionRequest.Builder( controllerApis.handleAlterPartitionRequest(buildRequest(new AlterPartitionRequest.Builder(
new AlterPartitionRequestData()).build(0))) new AlterPartitionRequestData()).build(ApiKeys.ALTER_PARTITION.latestVersion)))
}) })
} }

View File

@ -37,7 +37,7 @@ import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource import org.junit.jupiter.params.provider.ValueSource
import java.util import java.util
import java.util.{OptionalInt, Properties} import java.util.{Optional, OptionalInt, Properties}
import java.util.concurrent.{CompletableFuture, TimeUnit} import java.util.concurrent.{CompletableFuture, TimeUnit}
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
@ -62,7 +62,7 @@ class ControllerRegistrationManagerTest {
): java.util.Map[String, VersionRange] = { ): java.util.Map[String, VersionRange] = {
val results = new util.HashMap[String, VersionRange]() val results = new util.HashMap[String, VersionRange]()
results.put(MetadataVersion.FEATURE_NAME, VersionRange.of( results.put(MetadataVersion.FEATURE_NAME, VersionRange.of(
MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel(), MetadataVersion.MINIMUM_VERSION.featureLevel(),
highestSupportedMetadataVersion.featureLevel())) highestSupportedMetadataVersion.featureLevel()))
results results
} }
@ -105,7 +105,7 @@ class ControllerRegistrationManagerTest {
val delta = new MetadataDelta.Builder(). val delta = new MetadataDelta.Builder().
setImage(prevImage). setImage(prevImage).
build() build()
if (!prevImage.features().metadataVersion().equals(metadataVersion)) { if (!prevImage.features().metadataVersion.equals(Optional.of(metadataVersion))) {
delta.replay(new FeatureLevelRecord(). delta.replay(new FeatureLevelRecord().
setName(MetadataVersion.FEATURE_NAME). setName(MetadataVersion.FEATURE_NAME).
setFeatureLevel(metadataVersion.featureLevel())) setFeatureLevel(metadataVersion.featureLevel()))
@ -119,7 +119,7 @@ class ControllerRegistrationManagerTest {
} }
val provenance = new MetadataProvenance(100, 200, 300, true) val provenance = new MetadataProvenance(100, 200, 300, true)
val newImage = delta.apply(provenance) 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) new SnapshotManifest(provenance, 1000)
} else { } else {
new LogDeltaManifest.Builder(). new LogDeltaManifest.Builder().

View File

@ -23,7 +23,7 @@ import kafka.network.RequestChannel
import kafka.server.QuotaFactory.QuotaManagers import kafka.server.QuotaFactory.QuotaManagers
import kafka.server.metadata.KRaftMetadataCache import kafka.server.metadata.KRaftMetadataCache
import kafka.server.share.SharePartitionManager 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.OpType
import org.apache.kafka.clients.admin.{AlterConfigOp, ConfigEntry} import org.apache.kafka.clients.admin.{AlterConfigOp, ConfigEntry}
import org.apache.kafka.common._ 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.CreateTopicsRequestData.CreatableTopic
import org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult import org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult
import org.apache.kafka.common.message.DescribeShareGroupOffsetsRequestData.{DescribeShareGroupOffsetsRequestGroup, DescribeShareGroupOffsetsRequestTopic} 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.IncrementalAlterConfigsRequestData.{AlterConfigsResource => IAlterConfigsResource, AlterConfigsResourceCollection => IAlterConfigsResourceCollection, AlterableConfig => IAlterableConfig, AlterableConfigCollection => IAlterableConfigCollection}
import org.apache.kafka.common.message.IncrementalAlterConfigsResponseData.{AlterConfigsResourceResponse => IAlterConfigsResourceResponse} import org.apache.kafka.common.message.IncrementalAlterConfigsResponseData.{AlterConfigsResourceResponse => IAlterConfigsResourceResponse}
import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity 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.OffsetDeleteResponseData.{OffsetDeleteResponsePartition, OffsetDeleteResponsePartitionCollection, OffsetDeleteResponseTopic, OffsetDeleteResponseTopicCollection}
import org.apache.kafka.common.message.ShareFetchRequestData.{AcknowledgementBatch, ForgottenTopic} import org.apache.kafka.common.message.ShareFetchRequestData.{AcknowledgementBatch, ForgottenTopic}
import org.apache.kafka.common.message.ShareFetchResponseData.{AcquiredRecords, PartitionData, ShareFetchableTopicResponse} 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.metadata.RegisterBrokerRecord.{BrokerEndpoint, BrokerEndpointCollection}
import org.apache.kafka.common.protocol.ApiMessage import org.apache.kafka.common.protocol.ApiMessage
import org.apache.kafka.common.message._ 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.group.{GroupConfig, GroupCoordinator, GroupCoordinatorConfig}
import org.apache.kafka.coordinator.share.{ShareCoordinator, ShareCoordinatorTestConfig} import org.apache.kafka.coordinator.share.{ShareCoordinator, ShareCoordinatorTestConfig}
import org.apache.kafka.coordinator.transaction.TransactionLogConfig 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.metadata.{ConfigRepository, MockConfigRepository}
import org.apache.kafka.network.metrics.{RequestChannelMetrics, RequestMetrics} import org.apache.kafka.network.metrics.{RequestChannelMetrics, RequestMetrics}
import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.raft.QuorumConfig
@ -9781,7 +9782,16 @@ class KafkaApisTest extends Logging {
val consumerGroupHeartbeatRequest = new ConsumerGroupHeartbeatRequestData().setGroupId("group") val consumerGroupHeartbeatRequest = new ConsumerGroupHeartbeatRequestData().setGroupId("group")
val requestChannelRequest = buildRequest(new ConsumerGroupHeartbeatRequest.Builder(consumerGroupHeartbeatRequest).build()) 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 = createKafkaApis()
kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching)
@ -9920,7 +9930,16 @@ class KafkaApisTest extends Logging {
val expectedDescribedGroup = new DescribedGroup().setGroupId(groupId).setErrorCode(errorCode) val expectedDescribedGroup = new DescribedGroup().setGroupId(groupId).setErrorCode(errorCode)
val expectedResponse = new ConsumerGroupDescribeResponseData() val expectedResponse = new ConsumerGroupDescribeResponseData()
expectedResponse.groups.add(expectedDescribedGroup) 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 = createKafkaApis()
kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching)
val response = verifyNoThrottling[ConsumerGroupDescribeResponse](requestChannelRequest) val response = verifyNoThrottling[ConsumerGroupDescribeResponse](requestChannelRequest)

View File

@ -109,7 +109,7 @@ class KafkaRaftServerTest {
} }
private def writeBootstrapMetadata(logDir: File, metadataVersion: MetadataVersion): Unit = { 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")) bootstrapDirectory.writeBinaryFile(BootstrapMetadata.fromVersion(metadataVersion, "test"))
} }
@ -262,7 +262,7 @@ class KafkaRaftServerTest {
} }
@Test @Test
def testKRaftUpdateAt3_3_IV1(): Unit = { def testKRaftUpdateAt3_3_IV3(): Unit = {
val clusterId = clusterIdBase64 val clusterId = clusterIdBase64
val nodeId = 0 val nodeId = 0
val metaProperties = new MetaProperties.Builder(). val metaProperties = new MetaProperties.Builder().
@ -280,12 +280,12 @@ class KafkaRaftServerTest {
configProperties.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL") configProperties.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
val (metaPropertiesEnsemble, bootstrapMetadata) = 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()) assertEquals(metaProperties, metaPropertiesEnsemble.logDirProps().values().iterator().next())
assertTrue(metaPropertiesEnsemble.errorLogDirs().isEmpty) assertTrue(metaPropertiesEnsemble.errorLogDirs().isEmpty)
assertTrue(metaPropertiesEnsemble.emptyLogDirs().isEmpty) assertTrue(metaPropertiesEnsemble.emptyLogDirs().isEmpty)
assertEquals(bootstrapMetadata.metadataVersion(), MetadataVersion.IBP_3_3_IV1) assertEquals(bootstrapMetadata.metadataVersion(), MetadataVersion.IBP_3_3_IV3)
} }
@Test @Test

View File

@ -90,7 +90,7 @@ class ReplicaFetcherThreadTest {
val logContext = new LogContext(s"[ReplicaFetcher replicaId=${brokerConfig.brokerId}, leaderId=${leaderEndpointBlockingSend.brokerEndPoint().id}, fetcherId=$fetcherId] ") val logContext = new LogContext(s"[ReplicaFetcher replicaId=${brokerConfig.brokerId}, leaderId=${leaderEndpointBlockingSend.brokerEndPoint().id}, fetcherId=$fetcherId] ")
val fetchSessionHandler = new FetchSessionHandler(logContext, leaderEndpointBlockingSend.brokerEndPoint().id) val fetchSessionHandler = new FetchSessionHandler(logContext, leaderEndpointBlockingSend.brokerEndPoint().id)
val leader = new RemoteLeaderEndPoint(logContext.logPrefix, leaderEndpointBlockingSend, fetchSessionHandler, 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, new ReplicaFetcherThread(name,
leader, leader,
brokerConfig, brokerConfig,
@ -280,9 +280,9 @@ class ReplicaFetcherThreadTest {
val logContext = new LogContext(s"[ReplicaFetcher replicaId=${config.brokerId}, leaderId=${brokerEndPoint.id}, fetcherId=0] ") val logContext = new LogContext(s"[ReplicaFetcher replicaId=${config.brokerId}, leaderId=${brokerEndPoint.id}, fetcherId=0] ")
val fetchSessionHandler = new FetchSessionHandler(logContext, brokerEndPoint.id) val fetchSessionHandler = new FetchSessionHandler(logContext, brokerEndPoint.id)
val leader = new RemoteLeaderEndPoint(logContext.logPrefix, mockNetwork, fetchSessionHandler, config, 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, 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 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))) thread.addPartitions(Map(t1p0 -> initialFetchState(Some(topicId1), initialLEO), t1p1 -> initialFetchState(Some(topicId1), initialLEO)))
@ -396,7 +396,7 @@ class ReplicaFetcherThreadTest {
config, config,
replicaManager, replicaManager,
quota, quota,
() => MetadataVersion.MINIMUM_KRAFT_VERSION, () => MetadataVersion.MINIMUM_VERSION,
() => 1 () => 1
) )
@ -408,7 +408,7 @@ class ReplicaFetcherThreadTest {
replicaManager, replicaManager,
quota, quota,
logContext.logPrefix, logContext.logPrefix,
() => MetadataVersion.MINIMUM_KRAFT_VERSION () => MetadataVersion.MINIMUM_VERSION
) )
thread.addPartitions(Map( thread.addPartitions(Map(
@ -488,7 +488,7 @@ class ReplicaFetcherThreadTest {
config, config,
replicaManager, replicaManager,
quota, quota,
() => MetadataVersion.MINIMUM_KRAFT_VERSION, () => MetadataVersion.MINIMUM_VERSION,
() => 1 () => 1
) )
@ -500,7 +500,7 @@ class ReplicaFetcherThreadTest {
replicaManager, replicaManager,
quota, quota,
logContext.logPrefix, logContext.logPrefix,
() => MetadataVersion.MINIMUM_KRAFT_VERSION () => MetadataVersion.MINIMUM_VERSION
) )
thread.addPartitions(Map( thread.addPartitions(Map(
@ -597,7 +597,7 @@ class ReplicaFetcherThreadTest {
val logContext = new LogContext(s"[ReplicaFetcher replicaId=${config.brokerId}, leaderId=${brokerEndPoint.id}, fetcherId=0] ") val logContext = new LogContext(s"[ReplicaFetcher replicaId=${config.brokerId}, leaderId=${brokerEndPoint.id}, fetcherId=0] ")
val fetchSessionHandler = new FetchSessionHandler(logContext, brokerEndPoint.id) val fetchSessionHandler = new FetchSessionHandler(logContext, brokerEndPoint.id)
val leader = new RemoteLeaderEndPoint(logContext.logPrefix, mockBlockingSend, fetchSessionHandler, config, 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", val thread = new ReplicaFetcherThread("bob",
leader, leader,
config, config,
@ -605,7 +605,7 @@ class ReplicaFetcherThreadTest {
replicaManager, replicaManager,
replicaQuota, replicaQuota,
logContext.logPrefix, logContext.logPrefix,
() => MetadataVersion.MINIMUM_KRAFT_VERSION) () => MetadataVersion.MINIMUM_VERSION)
val leaderEpoch = 1 val leaderEpoch = 1

View File

@ -24,8 +24,7 @@ import kafka.server.QuotaFactory.QuotaManagers
import kafka.server.metadata.KRaftMetadataCache import kafka.server.metadata.KRaftMetadataCache
import kafka.utils.TestUtils.waitUntilTrue import kafka.utils.TestUtils.waitUntilTrue
import kafka.utils.{CoreUtils, Logging, TestUtils} import kafka.utils.{CoreUtils, Logging, TestUtils}
import org.apache.kafka.common.metadata.RegisterBrokerRecord import org.apache.kafka.common.metadata.{FeatureLevelRecord, PartitionChangeRecord, PartitionRecord, RegisterBrokerRecord, TopicRecord}
import org.apache.kafka.common.metadata.{PartitionChangeRecord, PartitionRecord, TopicRecord}
import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.record.SimpleRecord 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.requests.{FetchRequest, ProduceResponse}
import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.security.auth.KafkaPrincipal
import org.apache.kafka.common.utils.{Time, Utils} 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.image.{MetadataDelta, MetadataImage}
import org.apache.kafka.metadata.{LeaderAndIsr, LeaderRecoveryState, MockConfigRepository} import org.apache.kafka.metadata.{LeaderAndIsr, LeaderRecoveryState, MockConfigRepository}
import org.apache.kafka.metadata.PartitionRegistration import org.apache.kafka.metadata.PartitionRegistration
import org.apache.kafka.metadata.storage.Formatter import org.apache.kafka.metadata.storage.Formatter
import org.apache.kafka.raft.QuorumConfig 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.config.{KRaftConfigs, ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.storage.log.{FetchIsolation, FetchParams, FetchPartitionData} import org.apache.kafka.server.storage.log.{FetchIsolation, FetchParams, FetchPartitionData}
import org.apache.kafka.server.util.{MockTime, ShutdownableThread} 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 topicModel = new TopicModel(Uuid.randomUuid(), "foo", Map(0 -> initialPartitionRegistration))
val topicPartition = new TopicPartition(topicModel.name, 0) 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) val controller = new ControllerModel(Seq(localId, remoteId), topicModel, channel, replicaManager, metadataCache)
submit(new Clock(time)) submit(new Clock(time))
@ -129,6 +128,7 @@ class ReplicaManagerConcurrencyTest extends Logging {
clientId = s"replica-$remoteId", clientId = s"replica-$remoteId",
replicaId = remoteId, replicaId = remoteId,
topicIdPartition, topicIdPartition,
topicPartition.topic,
replicaManager replicaManager
) )
@ -211,6 +211,7 @@ class ReplicaManagerConcurrencyTest extends Logging {
clientId: String, clientId: String,
replicaId: Int, replicaId: Int,
topicIdPartition: TopicIdPartition, topicIdPartition: TopicIdPartition,
topicName: String,
replicaManager: ReplicaManager replicaManager: ReplicaManager
) extends ShutdownableThread(clientId, false) { ) extends ShutdownableThread(clientId, false) {
private val random = new Random() private val random = new Random()
@ -236,11 +237,14 @@ class ReplicaManagerConcurrencyTest extends Logging {
) )
val future = new CompletableFuture[FetchPartitionData]() 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 { try {
assertEquals(1, results.size) assertEquals(1, results.size)
val (topicIdPartition, result) = results.head 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) assertEquals(Errors.NONE, result.error)
future.complete(result) future.complete(result)
} catch { } catch {
@ -261,7 +265,7 @@ class ReplicaManagerConcurrencyTest extends Logging {
replicaManager.fetchMessages( replicaManager.fetchMessages(
params = fetchParams, params = fetchParams,
fetchInfos = Seq(topicIdPartition -> partitionData), fetchInfos = Seq(new org.apache.kafka.common.TopicIdPartition(topicIdPartition.topicId, topicIdPartition.partitionId, topicName) -> partitionData),
quota = QuotaFactory.UNBOUNDED_QUOTA, quota = QuotaFactory.UNBOUNDED_QUOTA,
responseCallback = fetchCallback, responseCallback = fetchCallback,
) )
@ -373,6 +377,10 @@ class ReplicaManagerConcurrencyTest extends Logging {
case InitializeEvent => case InitializeEvent =>
val delta = new MetadataDelta.Builder().setImage(latestImage).build() val delta = new MetadataDelta.Builder().setImage(latestImage).build()
brokerIds.foreach { brokerId => brokerIds.foreach { brokerId =>
delta.replay(new FeatureLevelRecord()
.setName(MetadataVersion.FEATURE_NAME)
.setFeatureLevel(MetadataVersion.MINIMUM_VERSION.featureLevel())
)
delta.replay(new RegisterBrokerRecord() delta.replay(new RegisterBrokerRecord()
.setBrokerId(brokerId) .setBrokerId(brokerId)
.setFenced(false) .setFenced(false)
@ -419,7 +427,7 @@ class ReplicaManagerConcurrencyTest extends Logging {
leaderAndIsr: LeaderAndIsr, leaderAndIsr: LeaderAndIsr,
delta: MetadataDelta delta: MetadataDelta
): LeaderAndIsr = { ): LeaderAndIsr = {
val partitionModel = partitions.getOrElse(topicPartition.partition, val partitionModel = partitions.getOrElse(topicPartition.partitionId,
throw new IllegalStateException(s"Unexpected partition $topicPartition") throw new IllegalStateException(s"Unexpected partition $topicPartition")
) )
partitionModel.alterIsr(leaderAndIsr, delta) partitionModel.alterIsr(leaderAndIsr, delta)

View File

@ -274,7 +274,7 @@ class ReplicaManagerTest {
val logManager = TestUtils.createLogManager(config.logDirs.map(new File(_)), new LogConfig(new Properties())) val logManager = TestUtils.createLogManager(config.logDirs.map(new File(_)), new LogConfig(new Properties()))
val metadataCache: MetadataCache = mock(classOf[MetadataCache]) val metadataCache: MetadataCache = mock(classOf[MetadataCache])
mockGetAliveBrokerFunctions(metadataCache, Seq(new Node(0, "host0", 0))) 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( val rm = new ReplicaManager(
metrics = metrics, metrics = metrics,
config = config, config = config,
@ -336,7 +336,7 @@ class ReplicaManagerTest {
val spyLogManager = spy(logManager) val spyLogManager = spy(logManager)
val metadataCache: MetadataCache = mock(classOf[MetadataCache]) val metadataCache: MetadataCache = mock(classOf[MetadataCache])
mockGetAliveBrokerFunctions(metadataCache, Seq(new Node(0, "host0", 0))) 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 tp0 = new TopicPartition(topic, 0)
val uuid = Uuid.randomUuid() val uuid = Uuid.randomUuid()
val rm = new ReplicaManager( val rm = new ReplicaManager(
@ -409,7 +409,7 @@ class ReplicaManagerTest {
val aliveBrokers = Seq(new Node(0, "host0", 0), new Node(1, "host1", 1)) val aliveBrokers = Seq(new Node(0, "host0", 0), new Node(1, "host1", 1))
val metadataCache: MetadataCache = mock(classOf[MetadataCache]) val metadataCache: MetadataCache = mock(classOf[MetadataCache])
mockGetAliveBrokerFunctions(metadataCache, aliveBrokers) mockGetAliveBrokerFunctions(metadataCache, aliveBrokers)
when(metadataCache.metadataVersion()).thenReturn(MetadataVersion.MINIMUM_KRAFT_VERSION) when(metadataCache.metadataVersion()).thenReturn(MetadataVersion.MINIMUM_VERSION)
val rm = new ReplicaManager( val rm = new ReplicaManager(
metrics = metrics, metrics = metrics,
config = config, config = config,
@ -2841,7 +2841,7 @@ class ReplicaManagerTest {
any[TopicPartition], any[ListenerName])). any[TopicPartition], any[ListenerName])).
thenReturn(Map(leaderBrokerId -> new Node(leaderBrokerId, "host1", 9092, "rack-a"), thenReturn(Map(leaderBrokerId -> new Node(leaderBrokerId, "host1", 9092, "rack-a"),
followerBrokerId -> new Node(followerBrokerId, "host2", 9092, "rack-b")).toMap) 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)) when(metadataCache.getAliveBrokerEpoch(leaderBrokerId)).thenReturn(Some(brokerEpoch))
val mockProducePurgatory = new DelayedOperationPurgatory[DelayedProduce]( val mockProducePurgatory = new DelayedOperationPurgatory[DelayedProduce](
"Produce", timer, 0, false) "Produce", timer, 0, false)
@ -2895,9 +2895,9 @@ class ReplicaManagerTest {
s"fetcherId=$fetcherId] ") s"fetcherId=$fetcherId] ")
val fetchSessionHandler = new FetchSessionHandler(logContext, sourceBroker.id) val fetchSessionHandler = new FetchSessionHandler(logContext, sourceBroker.id)
val leader = new RemoteLeaderEndPoint(logContext.logPrefix, blockingSend, fetchSessionHandler, rm.config, 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, 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 = { override def doWork(): Unit = {
// In case the thread starts before the partition is added by AbstractFetcherManager, // In case the thread starts before the partition is added by AbstractFetcherManager,
// add it here (it's a no-op if already added) // 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.topicIdInfo()).thenReturn((topicIds.asJava, topicNames.asJava))
when(metadataCache.topicNamesToIds()).thenReturn(topicIds.asJava) when(metadataCache.topicNamesToIds()).thenReturn(topicIds.asJava)
when(metadataCache.topicIdsToNames()).thenReturn(topicNames.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) mockGetAliveBrokerFunctions(metadataCache, aliveBrokers)
when(metadataCache.getAliveBrokerEpoch(brokerId+1)).thenReturn(Some(brokerEpoch)) when(metadataCache.getAliveBrokerEpoch(brokerId+1)).thenReturn(Some(brokerEpoch))
val mockProducePurgatory = new DelayedOperationPurgatory[DelayedProduce]( val mockProducePurgatory = new DelayedOperationPurgatory[DelayedProduce](
@ -3361,7 +3361,7 @@ class ReplicaManagerTest {
leader.setReplicaPartitionStateCallback(tp => PartitionState(leaderEpoch = 0)) leader.setReplicaPartitionStateCallback(tp => PartitionState(leaderEpoch = 0))
val fetcher = new ReplicaFetcherThread(threadName, leader, config, failedPartitions, replicaManager, val fetcher = new ReplicaFetcherThread(threadName, leader, config, failedPartitions, replicaManager,
quotaManager, "", () => MetadataVersion.MINIMUM_KRAFT_VERSION) quotaManager, "", () => MetadataVersion.MINIMUM_VERSION)
val initialFetchState = InitialFetchState( val initialFetchState = InitialFetchState(
topicId = Some(Uuid.randomUuid()), topicId = Some(Uuid.randomUuid()),
@ -3608,8 +3608,8 @@ class ReplicaManagerTest {
val aliveBrokers = Seq(new Node(0, "host0", 0), new Node(1, "host1", 1)) val aliveBrokers = Seq(new Node(0, "host0", 0), new Node(1, "host1", 1))
mockGetAliveBrokerFunctions(metadataCache0, aliveBrokers) mockGetAliveBrokerFunctions(metadataCache0, aliveBrokers)
mockGetAliveBrokerFunctions(metadataCache1, aliveBrokers) mockGetAliveBrokerFunctions(metadataCache1, aliveBrokers)
when(metadataCache0.metadataVersion()).thenReturn(MetadataVersion.MINIMUM_KRAFT_VERSION) when(metadataCache0.metadataVersion()).thenReturn(MetadataVersion.MINIMUM_VERSION)
when(metadataCache1.metadataVersion()).thenReturn(MetadataVersion.MINIMUM_KRAFT_VERSION) when(metadataCache1.metadataVersion()).thenReturn(MetadataVersion.MINIMUM_VERSION)
// each replica manager is for a broker // each replica manager is for a broker
val rm0 = new ReplicaManager( val rm0 = new ReplicaManager(

View File

@ -21,7 +21,7 @@ import java.io.{ByteArrayOutputStream, File, PrintStream}
import java.nio.charset.StandardCharsets import java.nio.charset.StandardCharsets
import java.nio.file.Files import java.nio.file.Files
import java.util import java.util
import java.util.{Optional, Properties} import java.util.Properties
import kafka.server.KafkaConfig import kafka.server.KafkaConfig
import kafka.utils.TestUtils import kafka.utils.TestUtils
import net.sourceforge.argparse4j.inf.ArgumentParserException import net.sourceforge.argparse4j.inf.ArgumentParserException
@ -537,10 +537,10 @@ Found problem:
def testVersionMappingWithValidReleaseVersion(): Unit = { def testVersionMappingWithValidReleaseVersion(): Unit = {
val stream = new ByteArrayOutputStream() val stream = new ByteArrayOutputStream()
// Test with a valid release version // 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 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 // Check that the metadata version is correctly included in the output
assertTrue(output.contains(s"metadata.version=${metadataVersion.featureLevel()} (${metadataVersion.version()})"), assertTrue(output.contains(s"metadata.version=${metadataVersion.featureLevel()} (${metadataVersion.version()})"),
s"Output did not contain expected Metadata Version: $output" s"Output did not contain expected Metadata Version: $output"
@ -589,7 +589,7 @@ Found problem:
}) })
assertEquals("Unknown release version '2.9-IV2'." + 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 " to " + MetadataVersion.LATEST_PRODUCTION.version, exception.getMessage
) )
@ -598,7 +598,7 @@ Found problem:
}) })
assertEquals("Unknown release version 'invalid'." + 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 " to " + MetadataVersion.LATEST_PRODUCTION.version, exception2.getMessage
) )
} }
@ -712,7 +712,7 @@ Found problem:
// Not doing full SCRAM record validation since that's covered elsewhere. // Not doing full SCRAM record validation since that's covered elsewhere.
// Just checking that we generate the correct number of records // 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 val scramRecords = bootstrapMetadata.records().asScala
.filter(apiMessageAndVersion => apiMessageAndVersion.message().isInstanceOf[UserScramCredentialRecord]) .filter(apiMessageAndVersion => apiMessageAndVersion.message().isInstanceOf[UserScramCredentialRecord])
.map(apiMessageAndVersion => apiMessageAndVersion.message().asInstanceOf[UserScramCredentialRecord]) .map(apiMessageAndVersion => apiMessageAndVersion.message().asInstanceOf[UserScramCredentialRecord])

View File

@ -51,7 +51,7 @@ import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.network.metrics.RequestChannelMetrics import org.apache.kafka.network.metrics.RequestChannelMetrics
import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.raft.QuorumConfig
import org.apache.kafka.server.authorizer.{AuthorizableRequestContext, Authorizer => JAuthorizer} 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.config.{DelegationTokenManagerConfigs, KRaftConfigs, ReplicationConfigs, ServerConfigs, ServerLogConfigs}
import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.server.metrics.KafkaYammerMetrics
import org.apache.kafka.server.util.MockTime import org.apache.kafka.server.util.MockTime

View File

@ -3878,18 +3878,18 @@ In the replica description 0@controller-0:1234:3Db5QLSqSZieL3rJBUUegA, 0 is the
quorum:<p/> quorum:<p/>
<pre><code class="language-bash"> <pre><code class="language-bash">
Feature: kraft.version SupportedMinVersion: 0 SupportedMaxVersion: 1 FinalizedVersionLevel: 0 Epoch: 5 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
</code></pre><p/> </code></pre><p/>
Here is another example of a static quorum:<p/> Here is another example of a static quorum:<p/>
<pre><code class="language-bash"> <pre><code class="language-bash">
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
</code></pre><p/> </code></pre><p/>
Here is an example of a dynamic quorum:<p/> Here is an example of a dynamic quorum:<p/>
<pre><code class="language-bash"> <pre><code class="language-bash">
Feature: kraft.version SupportedMinVersion: 0 SupportedMaxVersion: 1 FinalizedVersionLevel: 1 Epoch: 5 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
</code></pre><p/> </code></pre><p/>
The static versus dynamic nature of the quorum is determined at the time of formatting. The static versus dynamic nature of the quorum is determined at the time of formatting.

View File

@ -270,7 +270,7 @@ public class ReplicaFetcherThreadBenchmark {
config, config,
replicaManager, replicaManager,
replicaQuota, replicaQuota,
() -> MetadataVersion.MINIMUM_KRAFT_VERSION, () -> MetadataVersion.MINIMUM_VERSION,
() -> -1L () -> -1L
) { ) {
@Override @Override
@ -303,7 +303,7 @@ public class ReplicaFetcherThreadBenchmark {
replicaManager, replicaManager,
replicaQuota, replicaQuota,
String.format("[ReplicaFetcher replicaId=%d, leaderId=%d, fetcherId=%d", config.brokerId(), 3, 3), String.format("[ReplicaFetcher replicaId=%d, leaderId=%d, fetcherId=%d", config.brokerId(), 3, 3),
() -> MetadataVersion.MINIMUM_KRAFT_VERSION () -> MetadataVersion.MINIMUM_VERSION
); );
pool = partitions; pool = partitions;

View File

@ -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()); activationMessageConsumer.accept(logMessageBuilder.toString().trim());
return ControllerResult.atomicOf(records, null); return ControllerResult.atomicOf(records, null);
} }
@ -153,10 +146,9 @@ public class ActivationRecordsGenerator {
* </p> * </p>
* If the log is empty, write the bootstrap records. If the log is not empty, do some validation and * 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 * 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 * 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 * (e.g., lots of SCRAM credentials).
* include a ZkMigrationState record regardless of whether the log was empty or not.
*/ */
static ControllerResult<Void> generate( static ControllerResult<Void> generate(
Consumer<String> activationMessageConsumer, Consumer<String> activationMessageConsumer,

View File

@ -401,16 +401,10 @@ public class ClusterControlManager {
record.features().add(processRegistrationFeature(brokerId, finalizedFeatures, feature)); record.features().add(processRegistrationFeature(brokerId, finalizedFeatures, feature));
unverifiedFeatures.remove(feature.name()); 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)
if (request.features().find(MetadataVersion.FEATURE_NAME) == null) { throw new InvalidRegistrationException("Request features do not contain '" + MetadataVersion.FEATURE_NAME + "'");
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);
}
// We also need to check every controller feature is supported by the broker. // We also need to check every controller feature is supported by the broker.
unverifiedFeatures.forEach((featureName, finalizedVersion) -> { unverifiedFeatures.forEach((featureName, finalizedVersion) -> {
if (finalizedVersion != 0 && request.features().findAll(featureName).isEmpty()) { if (finalizedVersion != 0 && request.features().findAll(featureName).isEmpty()) {
@ -495,8 +489,14 @@ public class ClusterControlManager {
FinalizedControllerFeatures finalizedFeatures, FinalizedControllerFeatures finalizedFeatures,
BrokerRegistrationRequestData.Feature feature BrokerRegistrationRequestData.Feature feature
) { ) {
int defaultVersion = feature.name().equals(MetadataVersion.FEATURE_NAME) ? 1 : 0; // The default value for MetadataVersion is 1 not 0. // MetadataVersion has no default while the other features default to `0`
short finalized = finalizedFeatures.versionOrDefault(feature.name(), (short) defaultVersion); 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)) { if (!VersionRange.of(feature.minSupportedVersion(), feature.maxSupportedVersion()).contains(finalized)) {
throw new UnsupportedVersionException("Unable to register because the broker " + throw new UnsupportedVersionException("Unable to register because the broker " +
"does not support finalized version " + finalized + " of " + feature.name() + "does not support finalized version " + finalized + " of " + feature.name() +

View File

@ -55,7 +55,6 @@ public class FeatureControlManager {
private SnapshotRegistry snapshotRegistry = null; private SnapshotRegistry snapshotRegistry = null;
private QuorumFeatures quorumFeatures = null; private QuorumFeatures quorumFeatures = null;
private MetadataVersion metadataVersion = MetadataVersion.latestProduction(); private MetadataVersion metadataVersion = MetadataVersion.latestProduction();
private MetadataVersion minimumBootstrapVersion = MetadataVersion.MINIMUM_BOOTSTRAP_VERSION;
private ClusterFeatureSupportDescriber clusterSupportDescriber = new ClusterFeatureSupportDescriber() { private ClusterFeatureSupportDescriber clusterSupportDescriber = new ClusterFeatureSupportDescriber() {
@Override @Override
public Iterator<Entry<Integer, Map<String, VersionRange>>> brokerSupported() { public Iterator<Entry<Integer, Map<String, VersionRange>>> brokerSupported() {
@ -88,11 +87,6 @@ public class FeatureControlManager {
return this; return this;
} }
Builder setMinimumBootstrapVersion(MetadataVersion minimumBootstrapVersion) {
this.minimumBootstrapVersion = minimumBootstrapVersion;
return this;
}
Builder setClusterFeatureSupportDescriber(ClusterFeatureSupportDescriber clusterSupportDescriber) { Builder setClusterFeatureSupportDescriber(ClusterFeatureSupportDescriber clusterSupportDescriber) {
this.clusterSupportDescriber = clusterSupportDescriber; this.clusterSupportDescriber = clusterSupportDescriber;
return this; return this;
@ -104,7 +98,7 @@ public class FeatureControlManager {
if (quorumFeatures == null) { if (quorumFeatures == null) {
Map<String, VersionRange> localSupportedFeatures = new HashMap<>(); Map<String, VersionRange> localSupportedFeatures = new HashMap<>();
localSupportedFeatures.put(MetadataVersion.FEATURE_NAME, VersionRange.of( localSupportedFeatures.put(MetadataVersion.FEATURE_NAME, VersionRange.of(
MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel(), MetadataVersion.MINIMUM_VERSION.featureLevel(),
MetadataVersion.latestProduction().featureLevel())); MetadataVersion.latestProduction().featureLevel()));
quorumFeatures = new QuorumFeatures(0, localSupportedFeatures, Collections.singletonList(0)); quorumFeatures = new QuorumFeatures(0, localSupportedFeatures, Collections.singletonList(0));
} }
@ -113,7 +107,6 @@ public class FeatureControlManager {
quorumFeatures, quorumFeatures,
snapshotRegistry, snapshotRegistry,
metadataVersion, metadataVersion,
minimumBootstrapVersion,
clusterSupportDescriber clusterSupportDescriber
); );
} }
@ -136,11 +129,6 @@ public class FeatureControlManager {
*/ */
private final TimelineObject<MetadataVersion> metadataVersion; private final TimelineObject<MetadataVersion> metadataVersion;
/**
* The minimum bootstrap version that we can't downgrade before.
*/
private final MetadataVersion minimumBootstrapVersion;
/** /**
* Gives information about the supported versions in the cluster. * Gives information about the supported versions in the cluster.
*/ */
@ -151,14 +139,12 @@ public class FeatureControlManager {
QuorumFeatures quorumFeatures, QuorumFeatures quorumFeatures,
SnapshotRegistry snapshotRegistry, SnapshotRegistry snapshotRegistry,
MetadataVersion metadataVersion, MetadataVersion metadataVersion,
MetadataVersion minimumBootstrapVersion,
ClusterFeatureSupportDescriber clusterSupportDescriber ClusterFeatureSupportDescriber clusterSupportDescriber
) { ) {
this.log = logContext.logger(FeatureControlManager.class); this.log = logContext.logger(FeatureControlManager.class);
this.quorumFeatures = quorumFeatures; this.quorumFeatures = quorumFeatures;
this.finalizedVersions = new TimelineHashMap<>(snapshotRegistry, 0); this.finalizedVersions = new TimelineHashMap<>(snapshotRegistry, 0);
this.metadataVersion = new TimelineObject<>(snapshotRegistry, metadataVersion); this.metadataVersion = new TimelineObject<>(snapshotRegistry, metadataVersion);
this.minimumBootstrapVersion = minimumBootstrapVersion;
this.clusterSupportDescriber = clusterSupportDescriber; this.clusterSupportDescriber = clusterSupportDescriber;
} }
@ -328,15 +314,10 @@ public class FeatureControlManager {
try { try {
newVersion = MetadataVersion.fromFeatureLevel(newVersionLevel); newVersion = MetadataVersion.fromFeatureLevel(newVersionLevel);
} catch (IllegalArgumentException e) { } 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)) { if (newVersion.isLessThan(currentVersion)) {
// This is a downgrade // This is a downgrade
boolean metadataChanged = MetadataVersion.checkIfMetadataChanged(currentVersion, newVersion); boolean metadataChanged = MetadataVersion.checkIfMetadataChanged(currentVersion, newVersion);

View File

@ -349,8 +349,7 @@ public class PartitionChangeBuilder {
// If the election was unclean, we have to forcibly set the ISR to just the // If the election was unclean, we have to forcibly set the ISR to just the
// new leader. This can result in data loss! // new leader. This can result in data loss!
record.setIsr(Collections.singletonList(electionResult.node)); record.setIsr(Collections.singletonList(electionResult.node));
if (partition.leaderRecoveryState != LeaderRecoveryState.RECOVERING && if (partition.leaderRecoveryState != LeaderRecoveryState.RECOVERING) {
metadataVersion.isLeaderRecoverySupported()) {
// And mark the leader recovery state as RECOVERING // And mark the leader recovery state as RECOVERING
record.setLeaderRecoveryState(LeaderRecoveryState.RECOVERING.value()); record.setLeaderRecoveryState(LeaderRecoveryState.RECOVERING.value());
} }

View File

@ -117,7 +117,6 @@ import org.apache.kafka.server.authorizer.AclCreateResult;
import org.apache.kafka.server.authorizer.AclDeleteResult; import org.apache.kafka.server.authorizer.AclDeleteResult;
import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.server.common.ApiMessageAndVersion;
import org.apache.kafka.server.common.KRaftVersion; import org.apache.kafka.server.common.KRaftVersion;
import org.apache.kafka.server.common.MetadataVersion;
import org.apache.kafka.server.fault.FaultHandler; import org.apache.kafka.server.fault.FaultHandler;
import org.apache.kafka.server.fault.FaultHandlerException; import org.apache.kafka.server.fault.FaultHandlerException;
import org.apache.kafka.server.policy.AlterConfigPolicy; import org.apache.kafka.server.policy.AlterConfigPolicy;
@ -128,7 +127,6 @@ import org.apache.kafka.timeline.SnapshotRegistry;
import org.slf4j.Logger; import org.slf4j.Logger;
import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.EnumSet; import java.util.EnumSet;
@ -1539,12 +1537,6 @@ public final class QuorumController implements Controller {
setLogContext(logContext). setLogContext(logContext).
setQuorumFeatures(quorumFeatures). setQuorumFeatures(quorumFeatures).
setSnapshotRegistry(snapshotRegistry). 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). setClusterFeatureSupportDescriber(clusterSupportDescriber).
build(); build();
this.clusterControl = new ClusterControlManager.Builder(). this.clusterControl = new ClusterControlManager.Builder().
@ -1643,20 +1635,13 @@ public final class QuorumController implements Controller {
/** /**
* Register the writeNoOpRecord task. * Register the writeNoOpRecord task.
* *
* This task periodically writes a NoOpRecord to the metadata log, if the MetadataVersion * This task periodically writes a NoOpRecord to the metadata log.
* supports it.
* *
* @param maxIdleIntervalNs The period at which to write the NoOpRecord. * @param maxIdleIntervalNs The period at which to write the NoOpRecord.
*/ */
private void registerWriteNoOpRecord(long maxIdleIntervalNs) { private void registerWriteNoOpRecord(long maxIdleIntervalNs) {
periodicControl.registerTask(new PeriodicTask("writeNoOpRecord", periodicControl.registerTask(new PeriodicTask("writeNoOpRecord",
() -> { () -> ControllerResult.of(List.of(new ApiMessageAndVersion(new NoOpRecord(), (short) 0)), false),
ArrayList<ApiMessageAndVersion> records = new ArrayList<>(1);
if (featureControl.metadataVersion().isNoOpRecordSupported()) {
records.add(new ApiMessageAndVersion(new NoOpRecord(), (short) 0));
}
return ControllerResult.of(records, false);
},
maxIdleIntervalNs, maxIdleIntervalNs,
EnumSet.noneOf(PeriodicTaskFlag.class))); EnumSet.noneOf(PeriodicTaskFlag.class)));
} }

View File

@ -57,7 +57,7 @@ public final class QuorumFeatures {
public static Map<String, VersionRange> defaultSupportedFeatureMap(boolean enableUnstable) { public static Map<String, VersionRange> defaultSupportedFeatureMap(boolean enableUnstable) {
Map<String, VersionRange> features = new HashMap<>(1); Map<String, VersionRange> features = new HashMap<>(1);
features.put(MetadataVersion.FEATURE_NAME, VersionRange.of( features.put(MetadataVersion.FEATURE_NAME, VersionRange.of(
MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel(), MetadataVersion.MINIMUM_VERSION.featureLevel(),
enableUnstable ? enableUnstable ?
MetadataVersion.latestTesting().featureLevel() : MetadataVersion.latestTesting().featureLevel() :
MetadataVersion.latestProduction().featureLevel())); MetadataVersion.latestProduction().featureLevel()));

View File

@ -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.message.ListPartitionReassignmentsResponseData.OngoingTopicReassignment;
import org.apache.kafka.common.metadata.BrokerRegistrationChangeRecord; import org.apache.kafka.common.metadata.BrokerRegistrationChangeRecord;
import org.apache.kafka.common.metadata.ClearElrRecord; 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.PartitionChangeRecord;
import org.apache.kafka.common.metadata.PartitionRecord; import org.apache.kafka.common.metadata.PartitionRecord;
import org.apache.kafka.common.metadata.RemoveTopicRecord; import org.apache.kafka.common.metadata.RemoveTopicRecord;
import org.apache.kafka.common.metadata.TopicRecord; 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.metadata.UnregisterBrokerRecord;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.requests.AlterPartitionRequest; 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.NONE;
import static org.apache.kafka.common.protocol.Errors.NOT_CONTROLLER; 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.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.TOPIC_AUTHORIZATION_FAILED;
import static org.apache.kafka.common.protocol.Errors.UNKNOWN_TOPIC_ID; import static org.apache.kafka.common.protocol.Errors.UNKNOWN_TOPIC_ID;
import static org.apache.kafka.common.protocol.Errors.UNKNOWN_TOPIC_OR_PARTITION; import static org.apache.kafka.common.protocol.Errors.UNKNOWN_TOPIC_OR_PARTITION;
@ -847,8 +844,7 @@ public class ReplicationControlManager {
for (Entry<Integer, PartitionRegistration> partEntry : newParts.entrySet()) { for (Entry<Integer, PartitionRegistration> partEntry : newParts.entrySet()) {
int partitionIndex = partEntry.getKey(); int partitionIndex = partEntry.getKey();
PartitionRegistration info = partEntry.getValue(); PartitionRegistration info = partEntry.getValue();
records.add(info.toRecord(topicId, partitionIndex, new ImageWriterOptions.Builder(). records.add(info.toRecord(topicId, partitionIndex, new ImageWriterOptions.Builder(featureControl.metadataVersion()).
setMetadataVersion(featureControl.metadataVersion()).
setEligibleLeaderReplicasEnabled(featureControl.isElrFeatureEnabled()). setEligibleLeaderReplicasEnabled(featureControl.isElrFeatureEnabled()).
build())); build()));
} }
@ -1077,20 +1073,17 @@ public class ReplicationControlManager {
for (AlterPartitionRequestData.TopicData topicData : request.topics()) { for (AlterPartitionRequestData.TopicData topicData : request.topics()) {
AlterPartitionResponseData.TopicData responseTopicData = AlterPartitionResponseData.TopicData responseTopicData =
new AlterPartitionResponseData.TopicData(). new AlterPartitionResponseData.TopicData().
setTopicName(topicData.topicName()).
setTopicId(topicData.topicId()); setTopicId(topicData.topicId());
response.topics().add(responseTopicData); 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)) { 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()) { for (AlterPartitionRequestData.PartitionData partitionData : topicData.partitions()) {
responseTopicData.partitions().add(new AlterPartitionResponseData.PartitionData(). responseTopicData.partitions().add(new AlterPartitionResponseData.PartitionData().
setPartitionIndex(partitionData.partitionIndex()). setPartitionIndex(partitionData.partitionIndex()).
setErrorCode(error.code())); setErrorCode(UNKNOWN_TOPIC_ID.code()));
} }
log.info("Rejecting AlterPartition request for unknown topic ID {} or name {}.", log.info("Rejecting AlterPartition request for unknown topic ID {}.", topicData.topicId());
topicData.topicId(), topicData.topicName());
continue; continue;
} }
@ -1161,8 +1154,7 @@ public class ReplicationControlManager {
// fetch new metadata before trying again. This return code is // fetch new metadata before trying again. This return code is
// unusual because we both return an error and generate a new // unusual because we both return an error and generate a new
// metadata record. We usually only do one or the other. // 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 = NEW_LEADER_ELECTED;
Errors error = requestVersion > 1 ? NEW_LEADER_ELECTED : FENCED_LEADER_EPOCH;
log.info("AlterPartition request from node {} for {}-{} completed " + log.info("AlterPartition request from node {} for {}-{} completed " +
"the ongoing partition reassignment and triggered a " + "the ongoing partition reassignment and triggered a " +
"leadership change. Returning {}.", "leadership change. Returning {}.",
@ -1331,12 +1323,7 @@ public class ReplicationControlManager {
log.info("Rejecting AlterPartition request from node {} for {}-{} because " + log.info("Rejecting AlterPartition request from node {} for {}-{} because " +
"it specified ineligible replicas {} in the new ISR {}.", "it specified ineligible replicas {} in the new ISR {}.",
brokerId, topic.name, partitionId, ineligibleReplicas, partitionData.newIsrWithEpochs()); brokerId, topic.name, partitionId, ineligibleReplicas, partitionData.newIsrWithEpochs());
return INELIGIBLE_REPLICA;
if (requestApiVersion > 1) {
return INELIGIBLE_REPLICA;
} else {
return OPERATION_NOT_ATTEMPTED;
}
} }
return Errors.NONE; return Errors.NONE;
@ -1368,7 +1355,7 @@ public class ReplicationControlManager {
* Generate the appropriate records to handle a broker being fenced. * Generate the appropriate records to handle a broker being fenced.
* *
* First, we remove this broker from any ISR. Then we generate a * First, we remove this broker from any ISR. Then we generate a
* FenceBrokerRecord. * BrokerRegistrationChangeRecord.
* *
* @param brokerId The broker id. * @param brokerId The broker id.
* @param records The record list to append to. * @param records The record list to append to.
@ -1380,16 +1367,10 @@ public class ReplicationControlManager {
} }
generateLeaderAndIsrUpdates("handleBrokerFenced", brokerId, NO_LEADER, NO_LEADER, records, generateLeaderAndIsrUpdates("handleBrokerFenced", brokerId, NO_LEADER, NO_LEADER, records,
brokersToIsrs.partitionsWithBrokerInIsr(brokerId)); brokersToIsrs.partitionsWithBrokerInIsr(brokerId));
if (featureControl.metadataVersion().isBrokerRegistrationChangeRecordSupported()) { records.add(new ApiMessageAndVersion(new BrokerRegistrationChangeRecord().
records.add(new ApiMessageAndVersion(new BrokerRegistrationChangeRecord(). setBrokerId(brokerId).setBrokerEpoch(brokerRegistration.epoch()).
setBrokerId(brokerId).setBrokerEpoch(brokerRegistration.epoch()). setFenced(BrokerRegistrationFencingChange.FENCE.value()),
setFenced(BrokerRegistrationFencingChange.FENCE.value()), (short) 0));
(short) 0));
} else {
records.add(new ApiMessageAndVersion(new FenceBrokerRecord().
setId(brokerId).setEpoch(brokerRegistration.epoch()),
(short) 0));
}
} }
/** /**
@ -1416,7 +1397,7 @@ public class ReplicationControlManager {
/** /**
* Generate the appropriate records to handle a broker becoming unfenced. * 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 * partitions that don't currently have a leader that should be led by the newly
* unfenced broker. * unfenced broker.
* *
@ -1425,15 +1406,10 @@ public class ReplicationControlManager {
* @param records The record list to append to. * @param records The record list to append to.
*/ */
void handleBrokerUnfenced(int brokerId, long brokerEpoch, List<ApiMessageAndVersion> records) { void handleBrokerUnfenced(int brokerId, long brokerEpoch, List<ApiMessageAndVersion> records) {
if (featureControl.metadataVersion().isBrokerRegistrationChangeRecordSupported()) { records.add(new ApiMessageAndVersion(new BrokerRegistrationChangeRecord().
records.add(new ApiMessageAndVersion(new BrokerRegistrationChangeRecord(). setBrokerId(brokerId).setBrokerEpoch(brokerEpoch).
setBrokerId(brokerId).setBrokerEpoch(brokerEpoch). setFenced(BrokerRegistrationFencingChange.UNFENCE.value()),
setFenced(BrokerRegistrationFencingChange.UNFENCE.value()), (short) 0));
(short) 0));
} else {
records.add(new ApiMessageAndVersion(new UnfenceBrokerRecord().setId(brokerId).
setEpoch(brokerEpoch), (short) 0));
}
generateLeaderAndIsrUpdates("handleBrokerUnfenced", NO_LEADER, brokerId, NO_LEADER, records, generateLeaderAndIsrUpdates("handleBrokerUnfenced", NO_LEADER, brokerId, NO_LEADER, records,
brokersToIsrs.partitionsWithNoLeader()); brokersToIsrs.partitionsWithNoLeader());
} }
@ -1450,8 +1426,7 @@ public class ReplicationControlManager {
* @param records The record list to append to. * @param records The record list to append to.
*/ */
void handleBrokerInControlledShutdown(int brokerId, long brokerEpoch, List<ApiMessageAndVersion> records) { void handleBrokerInControlledShutdown(int brokerId, long brokerEpoch, List<ApiMessageAndVersion> records) {
if (featureControl.metadataVersion().isInControlledShutdownStateSupported() if (!clusterControl.inControlledShutdown(brokerId)) {
&& !clusterControl.inControlledShutdown(brokerId)) {
records.add(new ApiMessageAndVersion(new BrokerRegistrationChangeRecord(). records.add(new ApiMessageAndVersion(new BrokerRegistrationChangeRecord().
setBrokerId(brokerId).setBrokerEpoch(brokerEpoch). setBrokerId(brokerId).setBrokerEpoch(brokerEpoch).
setInControlledShutdown(BrokerRegistrationInControlledShutdownChange.IN_CONTROLLED_SHUTDOWN.value()), setInControlledShutdown(BrokerRegistrationInControlledShutdownChange.IN_CONTROLLED_SHUTDOWN.value()),
@ -1941,8 +1916,7 @@ public class ReplicationControlManager {
" time(s): All brokers are currently fenced or in controlled shutdown."); " time(s): All brokers are currently fenced or in controlled shutdown.");
} }
records.add(buildPartitionRegistration(partitionAssignment, isr) records.add(buildPartitionRegistration(partitionAssignment, isr)
.toRecord(topicId, partitionId, new ImageWriterOptions.Builder(). .toRecord(topicId, partitionId, new ImageWriterOptions.Builder(featureControl.metadataVersion()).
setMetadataVersion(featureControl.metadataVersion()).
setEligibleLeaderReplicasEnabled(featureControl.isElrFeatureEnabled()). setEligibleLeaderReplicasEnabled(featureControl.isElrFeatureEnabled()).
build())); build()));
partitionId++; partitionId++;

View File

@ -20,7 +20,6 @@ package org.apache.kafka.image;
import org.apache.kafka.common.Uuid; import org.apache.kafka.common.Uuid;
import org.apache.kafka.image.node.AclsImageNode; import org.apache.kafka.image.node.AclsImageNode;
import org.apache.kafka.image.writer.ImageWriter; 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.StandardAcl;
import org.apache.kafka.metadata.authorizer.StandardAclWithId; import org.apache.kafka.metadata.authorizer.StandardAclWithId;
@ -51,11 +50,7 @@ public final class AclsImage {
return acls; return acls;
} }
public void write(ImageWriter writer, ImageWriterOptions options) { public void write(ImageWriter writer) {
// 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.
for (Entry<Uuid, StandardAcl> entry : acls.entrySet()) { for (Entry<Uuid, StandardAcl> entry : acls.entrySet()) {
StandardAclWithId aclWithId = new StandardAclWithId(entry.getKey(), entry.getValue()); StandardAclWithId aclWithId = new StandardAclWithId(entry.getKey(), entry.getValue());
writer.write(0, aclWithId.toRecord()); writer.write(0, aclWithId.toRecord());

View File

@ -30,6 +30,7 @@ import java.util.Optional;
* Represents changes to the cluster in the metadata image. * Represents changes to the cluster in the metadata image.
*/ */
public final class FeaturesDelta { public final class FeaturesDelta {
private static final short MINIMUM_PERSISTED_FEATURE_LEVEL = 4;
private final FeaturesImage image; private final FeaturesImage image;
private final Map<String, Optional<Short>> changes = new HashMap<>(); private final Map<String, Optional<Short>> changes = new HashMap<>();
@ -58,7 +59,13 @@ public final class FeaturesDelta {
public void replay(FeatureLevelRecord record) { public void replay(FeatureLevelRecord record) {
if (record.name().equals(MetadataVersion.FEATURE_NAME)) { 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 { } else {
if (record.featureLevel() == 0) { if (record.featureLevel() == 0) {
changes.put(record.name(), Optional.empty()); changes.put(record.name(), Optional.empty());
@ -90,11 +97,11 @@ public final class FeaturesDelta {
} }
} }
final MetadataVersion metadataVersion; final Optional<MetadataVersion> metadataVersion;
if (metadataVersionChange == null) { if (metadataVersionChange == null) {
metadataVersion = image.metadataVersion(); metadataVersion = image.metadataVersion();
} else { } else {
metadataVersion = metadataVersionChange; metadataVersion = Optional.of(metadataVersionChange);
} }
return new FeaturesImage(newFinalizedVersions, metadataVersion); return new FeaturesImage(newFinalizedVersions, metadataVersion);

View File

@ -24,9 +24,7 @@ import org.apache.kafka.image.writer.ImageWriterOptions;
import org.apache.kafka.server.common.EligibleLeaderReplicasVersion; import org.apache.kafka.server.common.EligibleLeaderReplicasVersion;
import org.apache.kafka.server.common.MetadataVersion; import org.apache.kafka.server.common.MetadataVersion;
import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Map.Entry; import java.util.Map.Entry;
import java.util.Objects; import java.util.Objects;
@ -41,29 +39,40 @@ import java.util.Optional;
public final class FeaturesImage { public final class FeaturesImage {
public static final FeaturesImage EMPTY = new FeaturesImage( public static final FeaturesImage EMPTY = new FeaturesImage(
Collections.emptyMap(), Collections.emptyMap(),
MetadataVersion.MINIMUM_KRAFT_VERSION Optional.empty()
); );
private final Map<String, Short> finalizedVersions; private final Map<String, Short> finalizedVersions;
private final MetadataVersion metadataVersion; private final Optional<MetadataVersion> metadataVersion;
public FeaturesImage( public FeaturesImage(
Map<String, Short> finalizedVersions, Map<String, Short> finalizedVersions,
MetadataVersion metadataVersion) { MetadataVersion metadataVersion) {
this(finalizedVersions, Optional.of(metadataVersion));
}
FeaturesImage(
Map<String, Short> finalizedVersions,
Optional<MetadataVersion> metadataVersion) {
this.finalizedVersions = Collections.unmodifiableMap(finalizedVersions); this.finalizedVersions = Collections.unmodifiableMap(finalizedVersions);
this.metadataVersion = metadataVersion; this.metadataVersion = metadataVersion;
} }
public boolean isEmpty() { public boolean isEmpty() {
return finalizedVersions.isEmpty() && return finalizedVersions.isEmpty() && metadataVersion.isEmpty();
metadataVersion.equals(MetadataVersion.MINIMUM_KRAFT_VERSION);
} }
public MetadataVersion metadataVersion() { public Optional<MetadataVersion> metadataVersion() {
return metadataVersion; return metadataVersion;
} }
public MetadataVersion metadataVersionOrThrow() {
return metadataVersion.orElseThrow(() ->
new IllegalStateException("Unknown metadata version for FeaturesImage: " + this));
}
public Map<String, Short> finalizedVersions() { public Map<String, Short> finalizedVersions() {
return finalizedVersions; return finalizedVersions;
} }
@ -73,26 +82,8 @@ public final class FeaturesImage {
>= EligibleLeaderReplicasVersion.ELRV_1.featureLevel(); >= EligibleLeaderReplicasVersion.ELRV_1.featureLevel();
} }
private Optional<Short> finalizedVersion(String feature) {
return Optional.ofNullable(finalizedVersions.get(feature));
}
public void write(ImageWriter writer, ImageWriterOptions options) { public void write(ImageWriter writer, ImageWriterOptions options) {
if (options.metadataVersion().isLessThan(MetadataVersion.MINIMUM_BOOTSTRAP_VERSION)) { writeFeatureLevels(writer, options);
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<String> features = new ArrayList<>(finalizedVersions.keySet());
features.sort(String::compareTo);
options.handleLoss("feature flag(s): " + String.join(", ", features));
}
} }
private void writeFeatureLevels(ImageWriter writer, ImageWriterOptions options) { private void writeFeatureLevels(ImageWriter writer, ImageWriterOptions options) {

View File

@ -156,7 +156,7 @@ public final class MetadataImage {
configs.write(writer, options); configs.write(writer, options);
clientQuotas.write(writer, options); clientQuotas.write(writer, options);
producerIds.write(writer, options); producerIds.write(writer, options);
acls.write(writer, options); acls.write(writer);
scram.write(writer, options); scram.write(writer, options);
delegationTokens.write(writer, options); delegationTokens.write(writer, options);
writer.close(true); writer.close(true);

View File

@ -100,7 +100,7 @@ public class MetadataBatchLoader {
*/ */
public final void resetToImage(MetadataImage image) { public final void resetToImage(MetadataImage image) {
this.image = image; this.image = image;
this.hasSeenRecord = true; this.hasSeenRecord = !image.isEmpty();
this.delta = new MetadataDelta.Builder().setImage(image).build(); this.delta = new MetadataDelta.Builder().setImage(image).build();
this.transactionState = TransactionState.NO_TRANSACTION; this.transactionState = TransactionState.NO_TRANSACTION;
this.lastOffset = image.provenance().lastContainedOffset(); this.lastOffset = image.provenance().lastContainedOffset();

View File

@ -123,7 +123,6 @@ public class MetadataLoader implements RaftClient.Listener<ApiMessageAndVersion>
return new MetadataLoader( return new MetadataLoader(
time, time,
logContext, logContext,
nodeId,
threadNamePrefix, threadNamePrefix,
faultHandler, faultHandler,
metrics, metrics,
@ -194,7 +193,6 @@ public class MetadataLoader implements RaftClient.Listener<ApiMessageAndVersion>
private MetadataLoader( private MetadataLoader(
Time time, Time time,
LogContext logContext, LogContext logContext,
int nodeId,
String threadNamePrefix, String threadNamePrefix,
FaultHandler faultHandler, FaultHandler faultHandler,
MetadataLoaderMetrics metrics, MetadataLoaderMetrics metrics,
@ -293,8 +291,7 @@ public class MetadataLoader implements RaftClient.Listener<ApiMessageAndVersion>
setImage(MetadataImage.EMPTY). setImage(MetadataImage.EMPTY).
build(); build();
ImageReWriter writer = new ImageReWriter(delta); ImageReWriter writer = new ImageReWriter(delta);
image.write(writer, new ImageWriterOptions.Builder(). image.write(writer, new ImageWriterOptions.Builder(image.features().metadataVersionOrThrow()).
setMetadataVersion(image.features().metadataVersion()).
setEligibleLeaderReplicasEnabled(image.features().isElrEnabled()). setEligibleLeaderReplicasEnabled(image.features().isElrEnabled()).
build()); build());
// ImageReWriter#close invokes finishSnapshot, so we don't need to invoke it here. // ImageReWriter#close invokes finishSnapshot, so we don't need to invoke it here.
@ -348,7 +345,7 @@ public class MetadataLoader implements RaftClient.Listener<ApiMessageAndVersion>
} }
} }
metrics.updateLastAppliedImageProvenance(image.provenance()); metrics.updateLastAppliedImageProvenance(image.provenance());
metrics.setCurrentMetadataVersion(image.features().metadataVersion()); metrics.setCurrentMetadataVersion(image.features().metadataVersionOrThrow());
if (!uninitializedPublishers.isEmpty()) { if (!uninitializedPublishers.isEmpty()) {
scheduleInitializeNewPublishers(0); scheduleInitializeNewPublishers(0);
} }

View File

@ -45,7 +45,7 @@ public final class MetadataLoaderMetrics implements AutoCloseable {
private final Optional<MetricsRegistry> registry; private final Optional<MetricsRegistry> registry;
private final AtomicReference<MetadataVersion> currentMetadataVersion = private final AtomicReference<MetadataVersion> currentMetadataVersion =
new AtomicReference<>(MetadataVersion.MINIMUM_KRAFT_VERSION); new AtomicReference<>(MetadataVersion.MINIMUM_VERSION);
private final AtomicInteger currentControllerId = new AtomicInteger(-1); private final AtomicInteger currentControllerId = new AtomicInteger(-1);
private final AtomicLong handleLoadSnapshotCount = new AtomicLong(0); private final AtomicLong handleLoadSnapshotCount = new AtomicLong(0);
private final Consumer<Long> batchProcessingTimeNsUpdater; private final Consumer<Long> batchProcessingTimeNsUpdater;

View File

@ -53,7 +53,8 @@ public interface MetadataNode {
for (String name : names) { for (String name : names) {
printer.enterNode(name); printer.enterNode(name);
MetadataNode child = child(name); MetadataNode child = child(name);
child.print(printer); if (child != null)
child.print(printer);
printer.leaveNode(); printer.leaveNode();
} }
} }

View File

@ -90,8 +90,7 @@ public class BrokerRegistrationTracker implements MetadataPublisher {
} }
} }
if (checkBrokerRegistration) { if (checkBrokerRegistration) {
if (brokerRegistrationNeedsRefresh(newImage.features().metadataVersion(), if (brokerRegistrationNeedsRefresh(newImage.features().metadataVersionOrThrow(), delta.clusterDelta().broker(id))) {
delta.clusterDelta().broker(id))) {
refreshRegistrationCallback.run(); refreshRegistrationCallback.run();
} }
} }

View File

@ -146,8 +146,7 @@ public class SnapshotEmitter implements SnapshotGenerator.Emitter {
} }
RaftSnapshotWriter writer = new RaftSnapshotWriter(snapshotWriter.get(), batchSize); RaftSnapshotWriter writer = new RaftSnapshotWriter(snapshotWriter.get(), batchSize);
try { try {
image.write(writer, new ImageWriterOptions.Builder(). image.write(writer, new ImageWriterOptions.Builder(image.features().metadataVersionOrThrow()).
setMetadataVersion(image.features().metadataVersion()).
setEligibleLeaderReplicasEnabled(image.features().isElrEnabled()). setEligibleLeaderReplicasEnabled(image.features().isElrEnabled()).
build()); build());
writer.close(true); writer.close(true);

View File

@ -29,30 +29,22 @@ import java.util.function.Consumer;
public final class ImageWriterOptions { public final class ImageWriterOptions {
public static class Builder { public static class Builder {
private MetadataVersion metadataVersion; private MetadataVersion metadataVersion;
private MetadataVersion requestedMetadataVersion;
private boolean isEligibleLeaderReplicasEnabled = false;
private Consumer<UnwritableMetadataException> lossHandler = e -> { private Consumer<UnwritableMetadataException> lossHandler = e -> {
throw e; throw e;
}; };
private boolean isEligibleLeaderReplicasEnabled = false;
public Builder() { public Builder(MetadataVersion metadataVersion) {
this.metadataVersion = MetadataVersion.latestProduction(); this.metadataVersion = metadataVersion;
} }
public Builder(MetadataImage image) { public Builder(MetadataImage image) {
this.metadataVersion = image.features().metadataVersion(); this.metadataVersion = image.features().metadataVersionOrThrow();
this.isEligibleLeaderReplicasEnabled = image.features().isElrEnabled(); this.isEligibleLeaderReplicasEnabled = image.features().isElrEnabled();
} }
public Builder setMetadataVersion(MetadataVersion metadataVersion) { public Builder setMetadataVersion(MetadataVersion metadataVersion) {
this.requestedMetadataVersion = metadataVersion; this.metadataVersion = 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;
}
return this; return this;
} }
@ -65,10 +57,6 @@ public final class ImageWriterOptions {
return metadataVersion; return metadataVersion;
} }
public MetadataVersion requestedMetadataVersion() {
return requestedMetadataVersion;
}
public boolean isEligibleLeaderReplicasEnabled() { public boolean isEligibleLeaderReplicasEnabled() {
return isEligibleLeaderReplicasEnabled; return isEligibleLeaderReplicasEnabled;
} }
@ -79,24 +67,21 @@ public final class ImageWriterOptions {
} }
public ImageWriterOptions build() { public ImageWriterOptions build() {
return new ImageWriterOptions(metadataVersion, lossHandler, requestedMetadataVersion, isEligibleLeaderReplicasEnabled); return new ImageWriterOptions(metadataVersion, lossHandler, isEligibleLeaderReplicasEnabled);
} }
} }
private final MetadataVersion metadataVersion; private final MetadataVersion metadataVersion;
private final MetadataVersion requestedMetadataVersion;
private final Consumer<UnwritableMetadataException> lossHandler; private final Consumer<UnwritableMetadataException> lossHandler;
private final boolean isEligibleLeaderReplicasEnabled; private final boolean isEligibleLeaderReplicasEnabled;
private ImageWriterOptions( private ImageWriterOptions(
MetadataVersion metadataVersion, MetadataVersion metadataVersion,
Consumer<UnwritableMetadataException> lossHandler, Consumer<UnwritableMetadataException> lossHandler,
MetadataVersion orgMetadataVersion,
boolean isEligibleLeaderReplicasEnabled boolean isEligibleLeaderReplicasEnabled
) { ) {
this.metadataVersion = metadataVersion; this.metadataVersion = metadataVersion;
this.lossHandler = lossHandler; this.lossHandler = lossHandler;
this.requestedMetadataVersion = orgMetadataVersion;
this.isEligibleLeaderReplicasEnabled = isEligibleLeaderReplicasEnabled; this.isEligibleLeaderReplicasEnabled = isEligibleLeaderReplicasEnabled;
} }
@ -108,6 +93,6 @@ public final class ImageWriterOptions {
} }
public void handleLoss(String loss) { public void handleLoss(String loss) {
lossHandler.accept(new UnwritableMetadataException(requestedMetadataVersion, loss)); lossHandler.accept(new UnwritableMetadataException(metadataVersion, loss));
} }
} }

View File

@ -294,15 +294,8 @@ public class BrokerRegistration {
setRack(rack.orElse(null)). setRack(rack.orElse(null)).
setBrokerEpoch(epoch). setBrokerEpoch(epoch).
setIncarnationId(incarnationId). setIncarnationId(incarnationId).
setFenced(fenced); setFenced(fenced).
setInControlledShutdown(inControlledShutdown);
if (inControlledShutdown) {
if (options.metadataVersion().isInControlledShutdownStateSupported()) {
registrationRecord.setInControlledShutdown(true);
} else {
options.handleLoss("the inControlledShutdown state of one or more brokers");
}
}
if (isMigratingZkBroker) { if (isMigratingZkBroker) {
if (options.metadataVersion().isMigrationSupported()) { if (options.metadataVersion().isMigrationSupported()) {

View File

@ -111,7 +111,7 @@ public class ControllerRegistration {
if (supportedFeatures == null) { if (supportedFeatures == null) {
supportedFeatures = new HashMap<>(); supportedFeatures = new HashMap<>();
supportedFeatures.put(MetadataVersion.FEATURE_NAME, VersionRange.of( supportedFeatures.put(MetadataVersion.FEATURE_NAME, VersionRange.of(
MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel(), MetadataVersion.MINIMUM_VERSION.featureLevel(),
MetadataVersion.latestProduction().featureLevel())); MetadataVersion.latestProduction().featureLevel()));
} }
return new ControllerRegistration(id, return new ControllerRegistration(id,

View File

@ -31,12 +31,9 @@ import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Objects; import java.util.Objects;
import java.util.Optional;
import static java.nio.file.StandardCopyOption.ATOMIC_MOVE; import static java.nio.file.StandardCopyOption.ATOMIC_MOVE;
import static java.nio.file.StandardCopyOption.REPLACE_EXISTING; 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 * 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"; public static final String BINARY_BOOTSTRAP_FILENAME = "bootstrap.checkpoint";
private final String directoryPath; private final String directoryPath;
private final Optional<String> ibp;
/** /**
* Create a new BootstrapDirectory object. * Create a new BootstrapDirectory object.
* *
* @param directoryPath The path to the directory with the bootstrap file. * @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( public BootstrapDirectory(
String directoryPath, String directoryPath
Optional<String> ibp
) { ) {
this.directoryPath = Objects.requireNonNull(directoryPath); this.directoryPath = Objects.requireNonNull(directoryPath);
this.ibp = Objects.requireNonNull(ibp);
} }
public BootstrapMetadata read() throws Exception { public BootstrapMetadata read() throws Exception {
@ -82,16 +74,7 @@ public class BootstrapDirectory {
} }
BootstrapMetadata readFromConfiguration() { BootstrapMetadata readFromConfiguration() {
if (ibp.isEmpty()) { return BootstrapMetadata.fromVersion(MetadataVersion.latestProduction(), "the default bootstrap");
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);
} }
BootstrapMetadata readFromBinaryFile(String binaryPath) throws Exception { BootstrapMetadata readFromBinaryFile(String binaryPath) throws Exception {

View File

@ -30,9 +30,6 @@ import java.util.Map;
import java.util.Objects; import java.util.Objects;
import java.util.Optional; 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 * 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 * 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 String source
) { ) {
this.records = Objects.requireNonNull(records); 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; this.metadataVersion = metadataVersion;
Objects.requireNonNull(source); Objects.requireNonNull(source);
this.source = source; this.source = source;

View File

@ -26,12 +26,12 @@ import org.apache.kafka.server.common.FinalizedFeatures;
import org.slf4j.Logger; 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 { public class FeaturesPublisher implements MetadataPublisher {
private final Logger log; private final Logger log;
private volatile FinalizedFeatures finalizedFeatures = FinalizedFeatures.fromKRaftVersion(MINIMUM_KRAFT_VERSION); private volatile FinalizedFeatures finalizedFeatures = FinalizedFeatures.fromKRaftVersion(MINIMUM_VERSION);
public FeaturesPublisher( public FeaturesPublisher(
LogContext logContext LogContext logContext
@ -55,7 +55,7 @@ public class FeaturesPublisher implements MetadataPublisher {
LoaderManifest manifest LoaderManifest manifest
) { ) {
if (delta.featuresDelta() != null) { if (delta.featuresDelta() != null) {
FinalizedFeatures newFinalizedFeatures = new FinalizedFeatures(newImage.features().metadataVersion(), FinalizedFeatures newFinalizedFeatures = new FinalizedFeatures(newImage.features().metadataVersionOrThrow(),
newImage.features().finalizedVersions(), newImage.features().finalizedVersions(),
newImage.provenance().lastContainedOffset() newImage.provenance().lastContainedOffset()
); );

View File

@ -284,10 +284,6 @@ public class Formatter {
} }
MetadataVersion verifyReleaseVersion(MetadataVersion metadataVersion) { MetadataVersion verifyReleaseVersion(MetadataVersion metadataVersion) {
if (!metadataVersion.isKRaftSupported()) {
throw new FormatterException(MetadataVersion.FEATURE_NAME + " " + metadataVersion +
" is too old to be supported.");
}
if (!unstableFeatureVersionsEnabled) { if (!unstableFeatureVersionsEnabled) {
if (!metadataVersion.isProduction()) { if (!metadataVersion.isProduction()) {
throw new FormatterException(MetadataVersion.FEATURE_NAME + " " + metadataVersion + throw new FormatterException(MetadataVersion.FEATURE_NAME + " " + metadataVersion +
@ -435,7 +431,7 @@ public class Formatter {
directoryTypes.get(writeLogDir).description(), writeLogDir, directoryTypes.get(writeLogDir).description(), writeLogDir,
MetadataVersion.FEATURE_NAME, releaseVersion); MetadataVersion.FEATURE_NAME, releaseVersion);
Files.createDirectories(Paths.get(writeLogDir)); Files.createDirectories(Paths.get(writeLogDir));
BootstrapDirectory bootstrapDirectory = new BootstrapDirectory(writeLogDir, Optional.empty()); BootstrapDirectory bootstrapDirectory = new BootstrapDirectory(writeLogDir);
bootstrapDirectory.writeBinaryFile(bootstrapMetadata); bootstrapDirectory.writeBinaryFile(bootstrapMetadata);
if (directoryTypes.get(writeLogDir).isDynamicMetadataDirectory()) { if (directoryTypes.get(writeLogDir).isDynamicMetadataDirectory()) {
writeDynamicQuorumSnapshot(writeLogDir, writeDynamicQuorumSnapshot(writeLogDir,

View File

@ -41,10 +41,10 @@ public class ActivationRecordsGeneratorTest {
public void testActivationMessageForEmptyLog() { public void testActivationMessageForEmptyLog() {
ControllerResult<Void> result = ActivationRecordsGenerator.recordsForEmptyLog( ControllerResult<Void> result = ActivationRecordsGenerator.recordsForEmptyLog(
logMsg -> assertEquals("Performing controller activation. The metadata log appears to be empty. " + logMsg -> assertEquals("Performing controller activation. The metadata log appears to be empty. " +
"Appending 1 bootstrap record(s) 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, -1L,
BootstrapMetadata.fromVersion(MetadataVersion.MINIMUM_BOOTSTRAP_VERSION, "test"), BootstrapMetadata.fromVersion(MetadataVersion.MINIMUM_VERSION, "test"),
MetadataVersion.MINIMUM_KRAFT_VERSION, MetadataVersion.MINIMUM_VERSION,
2 2
); );
assertTrue(result.isAtomic()); assertTrue(result.isAtomic());

View File

@ -28,7 +28,6 @@ import org.apache.kafka.common.message.BrokerRegistrationRequestData;
import org.apache.kafka.common.message.ControllerRegistrationRequestData; import org.apache.kafka.common.message.ControllerRegistrationRequestData;
import org.apache.kafka.common.metadata.BrokerRegistrationChangeRecord; import org.apache.kafka.common.metadata.BrokerRegistrationChangeRecord;
import org.apache.kafka.common.metadata.FeatureLevelRecord; 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.PartitionChangeRecord;
import org.apache.kafka.common.metadata.RegisterBrokerRecord; import org.apache.kafka.common.metadata.RegisterBrokerRecord;
import org.apache.kafka.common.metadata.RegisterBrokerRecord.BrokerEndpoint; 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.api.Timeout;
import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.Arguments; 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.MethodSource;
import org.junit.jupiter.params.provider.ValueSource; import org.junit.jupiter.params.provider.ValueSource;
@ -76,7 +74,6 @@ import java.util.OptionalLong;
import java.util.stream.Stream; import java.util.stream.Stream;
import static java.util.Arrays.asList; 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.assertDoesNotThrow;
import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertFalse;
@ -86,9 +83,9 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
@Timeout(value = 40) @Timeout(value = 40)
public class ClusterControlManagerTest { public class ClusterControlManagerTest {
@ParameterizedTest
@EnumSource(value = MetadataVersion.class, names = {"IBP_3_0_IV1", "IBP_3_3_IV2"}) @Test
public void testReplay(MetadataVersion metadataVersion) { public void testReplay() {
MockTime time = new MockTime(0, 0, 0); MockTime time = new MockTime(0, 0, 0);
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
@ -123,27 +120,15 @@ public class ClusterControlManagerTest {
assertFalse(clusterControl.isUnfenced(0)); assertFalse(clusterControl.isUnfenced(0));
assertFalse(clusterControl.isUnfenced(1)); assertFalse(clusterControl.isUnfenced(1));
if (metadataVersion.isLessThan(IBP_3_3_IV2)) { BrokerRegistrationChangeRecord changeRecord =
UnfenceBrokerRecord unfenceBrokerRecord = new BrokerRegistrationChangeRecord().setBrokerId(1).setBrokerEpoch(100).setFenced(BrokerRegistrationFencingChange.UNFENCE.value());
new UnfenceBrokerRecord().setId(1).setEpoch(100); clusterControl.replay(changeRecord);
clusterControl.replay(unfenceBrokerRecord);
} else {
BrokerRegistrationChangeRecord changeRecord =
new BrokerRegistrationChangeRecord().setBrokerId(1).setBrokerEpoch(100).setFenced(BrokerRegistrationFencingChange.UNFENCE.value());
clusterControl.replay(changeRecord);
}
assertFalse(clusterControl.isUnfenced(0)); assertFalse(clusterControl.isUnfenced(0));
assertTrue(clusterControl.isUnfenced(1)); assertTrue(clusterControl.isUnfenced(1));
if (metadataVersion.isLessThan(IBP_3_3_IV2)) { changeRecord =
FenceBrokerRecord fenceBrokerRecord = new BrokerRegistrationChangeRecord().setBrokerId(1).setBrokerEpoch(100).setFenced(BrokerRegistrationFencingChange.FENCE.value());
new FenceBrokerRecord().setId(1).setEpoch(100); clusterControl.replay(changeRecord);
clusterControl.replay(fenceBrokerRecord);
} else {
BrokerRegistrationChangeRecord changeRecord =
new BrokerRegistrationChangeRecord().setBrokerId(1).setBrokerEpoch(100).setFenced(BrokerRegistrationFencingChange.FENCE.value());
clusterControl.replay(changeRecord);
}
assertFalse(clusterControl.isUnfenced(0)); assertFalse(clusterControl.isUnfenced(0));
assertFalse(clusterControl.isUnfenced(1)); assertFalse(clusterControl.isUnfenced(1));
} }
@ -289,8 +274,7 @@ public class ClusterControlManagerTest {
private static Stream<Arguments> metadataVersions() { private static Stream<Arguments> metadataVersions() {
return Stream.of( return Stream.of(
MetadataVersion.IBP_3_3_IV2, MetadataVersion.MINIMUM_VERSION,
MetadataVersion.IBP_3_3_IV3,
MetadataVersion.IBP_3_7_IV2, // introduces directory assignment MetadataVersion.IBP_3_7_IV2, // introduces directory assignment
MetadataVersion.latestTesting() MetadataVersion.latestTesting()
).map(Arguments::of); ).map(Arguments::of);
@ -327,9 +311,14 @@ public class ClusterControlManagerTest {
setBrokerId(0). setBrokerId(0).
setLogDirs(logDirs). setLogDirs(logDirs).
setRack(null). 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")), setIncarnationId(Uuid.fromString("0H4fUu1xQEKXFYwB1aBjhg")),
123L, 123L,
new FinalizedControllerFeatures(Collections.emptyMap(), 456L), new FinalizedControllerFeatures(Map.of(MetadataVersion.FEATURE_NAME, metadataVersion.featureLevel()), 456L),
false); false);
short expectedVersion = metadataVersion.registerBrokerRecordVersion(); short expectedVersion = metadataVersion.registerBrokerRecordVersion();
@ -345,8 +334,8 @@ public class ClusterControlManagerTest {
setFeatures(new RegisterBrokerRecord.BrokerFeatureCollection(Collections.singletonList( setFeatures(new RegisterBrokerRecord.BrokerFeatureCollection(Collections.singletonList(
new RegisterBrokerRecord.BrokerFeature(). new RegisterBrokerRecord.BrokerFeature().
setName(MetadataVersion.FEATURE_NAME). setName(MetadataVersion.FEATURE_NAME).
setMinSupportedVersion((short) 1). setMinSupportedVersion(metadataVersion.featureLevel()).
setMaxSupportedVersion((short) 1)).iterator())). setMaxSupportedVersion(metadataVersion.featureLevel())).iterator())).
setInControlledShutdown(false), expectedVersion)), setInControlledShutdown(false), expectedVersion)),
result.records()); result.records());
} }
@ -461,9 +450,9 @@ public class ClusterControlManagerTest {
} }
} }
@ParameterizedTest @Test
@EnumSource(value = MetadataVersion.class, names = {"IBP_3_3_IV2", "IBP_3_3_IV3"}) public void testRegistrationsToRecords() {
public void testRegistrationsToRecords(MetadataVersion metadataVersion) { MetadataVersion metadataVersion = MetadataVersion.MINIMUM_VERSION;
MockTime time = new MockTime(0, 0, 0); MockTime time = new MockTime(0, 0, 0);
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
FeatureControlManager featureControl = new FeatureControlManager.Builder(). FeatureControlManager featureControl = new FeatureControlManager.Builder().
@ -506,8 +495,7 @@ public class ClusterControlManagerTest {
clusterControl.replay(registrationChangeRecord); clusterControl.replay(registrationChangeRecord);
short expectedVersion = metadataVersion.registerBrokerRecordVersion(); short expectedVersion = metadataVersion.registerBrokerRecordVersion();
ImageWriterOptions options = new ImageWriterOptions.Builder(). ImageWriterOptions options = new ImageWriterOptions.Builder(metadataVersion).
setMetadataVersion(metadataVersion).
setLossHandler(__ -> { }). setLossHandler(__ -> { }).
build(); build();
assertEquals(new ApiMessageAndVersion(new RegisterBrokerRecord(). assertEquals(new ApiMessageAndVersion(new RegisterBrokerRecord().
@ -517,7 +505,7 @@ public class ClusterControlManagerTest {
setPort((short) 9092). setPort((short) 9092).
setName("PLAINTEXT"). setName("PLAINTEXT").
setHost("example.com")).iterator())). setHost("example.com")).iterator())).
setInControlledShutdown(metadataVersion.isInControlledShutdownStateSupported()). setInControlledShutdown(true).
setFenced(false), expectedVersion), setFenced(false), expectedVersion),
clusterControl.brokerRegistrations().get(0).toRecord(options)); clusterControl.brokerRegistrations().get(0).toRecord(options));
assertEquals(new ApiMessageAndVersion(new RegisterBrokerRecord(). assertEquals(new ApiMessageAndVersion(new RegisterBrokerRecord().
@ -545,7 +533,7 @@ public class ClusterControlManagerTest {
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
Map<String, VersionRange> supportedFeatures = new HashMap<>(); Map<String, VersionRange> supportedFeatures = new HashMap<>();
supportedFeatures.put(MetadataVersion.FEATURE_NAME, VersionRange.of( supportedFeatures.put(MetadataVersion.FEATURE_NAME, VersionRange.of(
MetadataVersion.IBP_3_1_IV0.featureLevel(), MetadataVersion.MINIMUM_VERSION.featureLevel(),
MetadataVersion.IBP_3_7_IV0.featureLevel())); MetadataVersion.IBP_3_7_IV0.featureLevel()));
supportedFeatures.put(TestFeatureVersion.FEATURE_NAME, VersionRange.of( supportedFeatures.put(TestFeatureVersion.FEATURE_NAME, VersionRange.of(
TestFeatureVersion.TEST_0.featureLevel(), TestFeatureVersion.TEST_0.featureLevel(),
@ -582,7 +570,7 @@ public class ClusterControlManagerTest {
baseRequest.setFeatures(new BrokerRegistrationRequestData.FeatureCollection( baseRequest.setFeatures(new BrokerRegistrationRequestData.FeatureCollection(
Collections.singleton(new BrokerRegistrationRequestData.Feature(). Collections.singleton(new BrokerRegistrationRequestData.Feature().
setName(MetadataVersion.FEATURE_NAME). setName(MetadataVersion.FEATURE_NAME).
setMinSupportedVersion(MetadataVersion.IBP_3_1_IV0.featureLevel()). setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()).
setMaxSupportedVersion(MetadataVersion.IBP_3_7_IV0.featureLevel())).iterator())), setMaxSupportedVersion(MetadataVersion.IBP_3_7_IV0.featureLevel())).iterator())),
123L, 123L,
featureControl.finalizedFeatures(Long.MAX_VALUE), featureControl.finalizedFeatures(Long.MAX_VALUE),
@ -594,7 +582,7 @@ public class ClusterControlManagerTest {
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
Map<String, VersionRange> supportedFeatures = new HashMap<>(); Map<String, VersionRange> supportedFeatures = new HashMap<>();
supportedFeatures.put(MetadataVersion.FEATURE_NAME, VersionRange.of( supportedFeatures.put(MetadataVersion.FEATURE_NAME, VersionRange.of(
MetadataVersion.IBP_3_1_IV0.featureLevel(), MetadataVersion.MINIMUM_VERSION.featureLevel(),
MetadataVersion.IBP_3_9_IV0.featureLevel())); MetadataVersion.IBP_3_9_IV0.featureLevel()));
supportedFeatures.put(KRaftVersion.FEATURE_NAME, VersionRange.of( supportedFeatures.put(KRaftVersion.FEATURE_NAME, VersionRange.of(
KRaftVersion.KRAFT_VERSION_1.featureLevel(), KRaftVersion.KRAFT_VERSION_1.featureLevel(),
@ -680,10 +668,10 @@ public class ClusterControlManagerTest {
setSnapshotRegistry(snapshotRegistry). setSnapshotRegistry(snapshotRegistry).
setQuorumFeatures(new QuorumFeatures(0, setQuorumFeatures(new QuorumFeatures(0,
Collections.singletonMap(MetadataVersion.FEATURE_NAME, VersionRange.of( Collections.singletonMap(MetadataVersion.FEATURE_NAME, VersionRange.of(
MetadataVersion.IBP_3_1_IV0.featureLevel(), MetadataVersion.IBP_3_5_IV0.featureLevel(),
MetadataVersion.IBP_3_3_IV0.featureLevel())), MetadataVersion.IBP_3_6_IV0.featureLevel())),
Collections.singletonList(0))). Collections.singletonList(0))).
setMetadataVersion(MetadataVersion.IBP_3_3_IV0). setMetadataVersion(MetadataVersion.IBP_3_5_IV0).
build(); build();
ClusterControlManager clusterControl = new ClusterControlManager.Builder(). ClusterControlManager clusterControl = new ClusterControlManager.Builder().
setClusterId("fPZv1VBsRFmnlRvmGcOW9w"). setClusterId("fPZv1VBsRFmnlRvmGcOW9w").
@ -694,21 +682,26 @@ public class ClusterControlManagerTest {
build(); build();
clusterControl.activate(); clusterControl.activate();
assertEquals("Unable to register because the broker does not support finalized version 4 of " + assertEquals("Unable to register because the broker does not support finalized version 9 of " +
"metadata.version. The broker wants a version between 1 and 1, inclusive.", "metadata.version. The broker wants a version between 7 and 7, inclusive.",
assertThrows(UnsupportedVersionException.class, assertThrows(UnsupportedVersionException.class,
() -> clusterControl.registerBroker( () -> clusterControl.registerBroker(
new BrokerRegistrationRequestData(). new BrokerRegistrationRequestData().
setClusterId("fPZv1VBsRFmnlRvmGcOW9w"). setClusterId("fPZv1VBsRFmnlRvmGcOW9w").
setBrokerId(0). setBrokerId(0).
setRack(null). 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")), setIncarnationId(Uuid.fromString("0H4fUu1xQEKXFYwB1aBjhg")),
123L, 123L,
featureControl.finalizedFeatures(Long.MAX_VALUE), featureControl.finalizedFeatures(Long.MAX_VALUE),
false)).getMessage()); false)).getMessage());
assertEquals("Unable to register because the broker does not support finalized version 4 of " + 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.", "metadata.version. The broker wants a version between 8 and 8, inclusive.",
assertThrows(UnsupportedVersionException.class, assertThrows(UnsupportedVersionException.class,
() -> clusterControl.registerBroker( () -> clusterControl.registerBroker(
new BrokerRegistrationRequestData(). new BrokerRegistrationRequestData().
@ -718,8 +711,8 @@ public class ClusterControlManagerTest {
setFeatures(new BrokerRegistrationRequestData.FeatureCollection( setFeatures(new BrokerRegistrationRequestData.FeatureCollection(
Collections.singleton(new BrokerRegistrationRequestData.Feature(). Collections.singleton(new BrokerRegistrationRequestData.Feature().
setName(MetadataVersion.FEATURE_NAME). setName(MetadataVersion.FEATURE_NAME).
setMinSupportedVersion(MetadataVersion.IBP_3_3_IV3.featureLevel()). setMinSupportedVersion(MetadataVersion.IBP_3_4_IV0.featureLevel()).
setMaxSupportedVersion(MetadataVersion.IBP_3_3_IV3.featureLevel())).iterator())). setMaxSupportedVersion(MetadataVersion.IBP_3_4_IV0.featureLevel())).iterator())).
setIncarnationId(Uuid.fromString("0H4fUu1xQEKXFYwB1aBjhg")), setIncarnationId(Uuid.fromString("0H4fUu1xQEKXFYwB1aBjhg")),
123L, 123L,
featureControl.finalizedFeatures(Long.MAX_VALUE), featureControl.finalizedFeatures(Long.MAX_VALUE),
@ -727,19 +720,19 @@ public class ClusterControlManagerTest {
} }
@Test @Test
public void testRegisterControlWithOlderMetadataVersion() { public void testRegisterControlWithUnsupportedMetadataVersion() {
FeatureControlManager featureControl = new FeatureControlManager.Builder(). FeatureControlManager featureControl = new FeatureControlManager.Builder().
setMetadataVersion(MetadataVersion.IBP_3_3_IV0). setMetadataVersion(MetadataVersion.IBP_3_6_IV2).
build(); build();
ClusterControlManager clusterControl = new ClusterControlManager.Builder(). ClusterControlManager clusterControl = new ClusterControlManager.Builder().
setClusterId("fPZv1VBsRFmnlRvmGcOW9w"). setClusterId("fPZv1VBsRFmnlRvmGcOW9w").
setFeatureControlManager(featureControl). setFeatureControlManager(featureControl).
setBrokerShutdownHandler((brokerId, isCleanShutdown, records) -> { }). setBrokerShutdownHandler((brokerId, isCleanShutdown, records) -> { }).
build(); build();
clusterControl.activate(); clusterControl.activate();
assertEquals("The current MetadataVersion is too old to support controller registrations.", assertEquals("The current MetadataVersion is too old to support controller registrations.",
assertThrows(UnsupportedVersionException.class, () -> clusterControl.registerController( assertThrows(UnsupportedVersionException.class, () -> clusterControl.registerController(
new ControllerRegistrationRequestData().setControllerId(1))).getMessage()); new ControllerRegistrationRequestData().setControllerId(1))).getMessage());
} }
@Test @Test
@ -779,8 +772,14 @@ public class ClusterControlManagerTest {
BrokerRegistrationRequestData data = new BrokerRegistrationRequestData().setBrokerId(brokerId) BrokerRegistrationRequestData data = new BrokerRegistrationRequestData().setBrokerId(brokerId)
.setClusterId(clusterControl.clusterId()) .setClusterId(clusterControl.clusterId())
.setIncarnationId(new Uuid(brokerId, brokerId)) .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); .setLogDirs(dirs);
FinalizedControllerFeatures finalizedFeatures = new FinalizedControllerFeatures(Collections.emptyMap(), 456L); FinalizedControllerFeatures finalizedFeatures = new FinalizedControllerFeatures(
Map.of(MetadataVersion.FEATURE_NAME, MetadataVersion.MINIMUM_VERSION.featureLevel()), 456L);
ControllerResult<BrokerRegistrationReply> result = clusterControl.registerBroker(data, 123L, finalizedFeatures, false); ControllerResult<BrokerRegistrationReply> result = clusterControl.registerBroker(data, 123L, finalizedFeatures, false);
RecordTestUtils.replayAll(clusterControl, result.records()); RecordTestUtils.replayAll(clusterControl, result.records());
} }
@ -832,25 +831,37 @@ public class ClusterControlManagerTest {
setBrokerShutdownHandler((brokerId, isCleanShutdown, records) -> { }). setBrokerShutdownHandler((brokerId, isCleanShutdown, records) -> { }).
build(); build();
clusterControl.activate(); clusterControl.activate();
var finalizedFeatures = new FinalizedControllerFeatures(Map.of(MetadataVersion.FEATURE_NAME, MetadataVersion.MINIMUM_VERSION.featureLevel()),
100L);
RecordTestUtils.replayAll(clusterControl, clusterControl.registerBroker( RecordTestUtils.replayAll(clusterControl, clusterControl.registerBroker(
new BrokerRegistrationRequestData(). new BrokerRegistrationRequestData().
setBrokerId(1). setBrokerId(1).
setClusterId(clusterControl.clusterId()). 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")). setIncarnationId(Uuid.fromString("mISEfEFwQIuaD1gKCc5tzQ")).
setLogDirs(Arrays.asList(Uuid.fromString("Vv1gzkM2QpuE-PPrIc6XEw"))), setLogDirs(Arrays.asList(Uuid.fromString("Vv1gzkM2QpuE-PPrIc6XEw"))),
100, 100,
new FinalizedControllerFeatures(Collections.emptyMap(), 100L), finalizedFeatures,
false). false).
records()); records());
RecordTestUtils.replayAll(clusterControl, clusterControl.registerBroker( RecordTestUtils.replayAll(clusterControl, clusterControl.registerBroker(
new BrokerRegistrationRequestData(). new BrokerRegistrationRequestData().
setBrokerId(1). setBrokerId(1).
setClusterId(clusterControl.clusterId()). 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 ? setIncarnationId(newIncarnationId ?
Uuid.fromString("07OOcU7MQFeSmGAFPP2Zww") : Uuid.fromString("mISEfEFwQIuaD1gKCc5tzQ")). Uuid.fromString("07OOcU7MQFeSmGAFPP2Zww") : Uuid.fromString("mISEfEFwQIuaD1gKCc5tzQ")).
setLogDirs(Arrays.asList(Uuid.fromString("Vv1gzkM2QpuE-PPrIc6XEw"))), setLogDirs(Arrays.asList(Uuid.fromString("Vv1gzkM2QpuE-PPrIc6XEw"))),
111, 111,
new FinalizedControllerFeatures(Collections.emptyMap(), 100L), finalizedFeatures,
false). false).
records()); records());
if (newIncarnationId) { if (newIncarnationId) {
@ -879,14 +890,21 @@ public class ClusterControlManagerTest {
}). }).
build(); build();
clusterControl.activate(); clusterControl.activate();
var finalizedFeatures = new FinalizedControllerFeatures(Map.of(MetadataVersion.FEATURE_NAME,
MetadataVersion.MINIMUM_VERSION.featureLevel()), 100L);
List<ApiMessageAndVersion> records = clusterControl.registerBroker( List<ApiMessageAndVersion> records = clusterControl.registerBroker(
new BrokerRegistrationRequestData(). new BrokerRegistrationRequestData().
setBrokerId(1). setBrokerId(1).
setClusterId(clusterControl.clusterId()). setClusterId(clusterControl.clusterId()).
setIncarnationId(Uuid.fromString("mISEfEFwQIuaD1gKCc5tzQ")). 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"))), setLogDirs(Arrays.asList(Uuid.fromString("Vv1gzkM2QpuE-PPrIc6XEw"))),
100, 100,
new FinalizedControllerFeatures(Collections.emptyMap(), 100L), finalizedFeatures,
true). true).
records(); records();
records.add(new ApiMessageAndVersion(new BrokerRegistrationChangeRecord(). records.add(new ApiMessageAndVersion(new BrokerRegistrationChangeRecord().
@ -901,9 +919,14 @@ public class ClusterControlManagerTest {
setClusterId(clusterControl.clusterId()). setClusterId(clusterControl.clusterId()).
setIncarnationId(Uuid.fromString("07OOcU7MQFeSmGAFPP2Zww")). setIncarnationId(Uuid.fromString("07OOcU7MQFeSmGAFPP2Zww")).
setPreviousBrokerEpoch(isCleanShutdown ? 100 : 10). 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"))), setLogDirs(Arrays.asList(Uuid.fromString("Vv1gzkM2QpuE-PPrIc6XEw"))),
111, 111,
new FinalizedControllerFeatures(Collections.emptyMap(), 100L), finalizedFeatures,
true).records(); true).records();
RecordTestUtils.replayAll(clusterControl, records); RecordTestUtils.replayAll(clusterControl, records);
assertEquals(Uuid.fromString("07OOcU7MQFeSmGAFPP2Zww"), assertEquals(Uuid.fromString("07OOcU7MQFeSmGAFPP2Zww"),

View File

@ -28,6 +28,7 @@ import org.apache.kafka.metadata.VersionRange;
import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.server.common.ApiMessageAndVersion;
import org.apache.kafka.server.common.Feature; import org.apache.kafka.server.common.Feature;
import org.apache.kafka.server.common.MetadataVersion; 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.TestFeatureVersion;
import org.apache.kafka.server.common.TransactionVersion; import org.apache.kafka.server.common.TransactionVersion;
import org.apache.kafka.timeline.SnapshotRegistry; import org.apache.kafka.timeline.SnapshotRegistry;
@ -99,10 +100,10 @@ public class FeatureControlManagerTest {
FeatureControlManager manager = new FeatureControlManager.Builder(). FeatureControlManager manager = new FeatureControlManager.Builder().
setQuorumFeatures(features(TestFeatureVersion.FEATURE_NAME, 0, 2)). setQuorumFeatures(features(TestFeatureVersion.FEATURE_NAME, 0, 2)).
setSnapshotRegistry(snapshotRegistry). setSnapshotRegistry(snapshotRegistry).
setMetadataVersion(MetadataVersion.IBP_3_3_IV0). setMetadataVersion(MetadataVersion.MINIMUM_VERSION).
build(); build();
snapshotRegistry.idempotentCreateSnapshot(-1); 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)); manager.finalizedFeatures(-1));
assertEquals(ControllerResult.of(emptyList(), new ApiError(Errors.INVALID_UPDATE_VERSION, 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")), "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). setLogContext(logContext).
setQuorumFeatures(features("foo", 1, 2)). setQuorumFeatures(features("foo", 1, 2)).
setSnapshotRegistry(snapshotRegistry). setSnapshotRegistry(snapshotRegistry).
setMetadataVersion(MetadataVersion.IBP_3_3_IV0). setMetadataVersion(MetadataVersion.MINIMUM_VERSION).
build(); build();
manager.replay(record); manager.replay(record);
snapshotRegistry.idempotentCreateSnapshot(123); 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)); manager.finalizedFeatures(123));
} }
@ -224,12 +226,12 @@ public class FeatureControlManagerTest {
setLogContext(logContext). setLogContext(logContext).
setQuorumFeatures(features(TestFeatureVersion.FEATURE_NAME, 0, 5, TransactionVersion.FEATURE_NAME, 0, 2)). setQuorumFeatures(features(TestFeatureVersion.FEATURE_NAME, 0, 5, TransactionVersion.FEATURE_NAME, 0, 2)).
setSnapshotRegistry(snapshotRegistry). setSnapshotRegistry(snapshotRegistry).
setMetadataVersion(MetadataVersion.IBP_3_3_IV0). setMetadataVersion(MetadataVersion.MINIMUM_VERSION).
build(); build();
ControllerResult<ApiError> result = manager. ControllerResult<ApiError> result = manager.
updateFeatures(updateMap(TestFeatureVersion.FEATURE_NAME, 1, TransactionVersion.FEATURE_NAME, 2), Collections.emptyMap(), false); updateFeatures(updateMap(TestFeatureVersion.FEATURE_NAME, 1, TransactionVersion.FEATURE_NAME, 2), Collections.emptyMap(), false);
RecordTestUtils.replayAll(manager, result.records()); 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) 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(Optional.of((short) 2), manager.finalizedFeatures(Long.MAX_VALUE).get(TransactionVersion.FEATURE_NAME));
assertEquals(new HashSet<>(Arrays.asList( assertEquals(new HashSet<>(Arrays.asList(
@ -240,38 +242,27 @@ public class FeatureControlManagerTest {
private static final FeatureControlManager.Builder TEST_MANAGER_BUILDER1 = private static final FeatureControlManager.Builder TEST_MANAGER_BUILDER1 =
new FeatureControlManager.Builder(). new FeatureControlManager.Builder().
setQuorumFeatures(features(MetadataVersion.FEATURE_NAME, setQuorumFeatures(features(MetadataVersion.FEATURE_NAME,
MetadataVersion.IBP_3_3_IV0.featureLevel(), MetadataVersion.IBP_3_3_IV3.featureLevel())). MetadataVersion.MINIMUM_VERSION.featureLevel(), MetadataVersion.IBP_3_6_IV0.featureLevel())).
setMetadataVersion(MetadataVersion.IBP_3_3_IV2); setMetadataVersion(MetadataVersion.IBP_3_4_IV0);
@Test @Test
public void testApplyMetadataVersionChangeRecord() { public void testApplyMetadataVersionChangeRecord() {
FeatureControlManager manager = TEST_MANAGER_BUILDER1.build(); FeatureControlManager manager = TEST_MANAGER_BUILDER1.build();
MetadataVersion initialMetadataVersion = manager.metadataVersion();
manager.replay(new FeatureLevelRecord(). manager.replay(new FeatureLevelRecord().
setName(MetadataVersion.FEATURE_NAME). setName(MetadataVersion.FEATURE_NAME).
setFeatureLevel(MetadataVersion.IBP_3_3_IV3.featureLevel())); setFeatureLevel((short) (initialMetadataVersion.featureLevel() + 1)));
assertEquals(MetadataVersion.IBP_3_3_IV3, manager.metadataVersion()); assertEquals(MetadataVersion.IBP_3_5_IV0, 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));
} }
@Test @Test
public void testCannotDowngradeToHigherVersion() { public void testCannotDowngradeToHigherVersion() {
FeatureControlManager manager = TEST_MANAGER_BUILDER1.build(); FeatureControlManager manager = TEST_MANAGER_BUILDER1.build();
assertEquals(ControllerResult.of(Collections.emptyList(), new ApiError(Errors.INVALID_UPDATE_VERSION, 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.")), "newer version.")),
manager.updateFeatures( 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), singletonMap(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.SAFE_DOWNGRADE),
true)); true));
} }
@ -280,23 +271,26 @@ public class FeatureControlManagerTest {
public void testCannotUnsafeDowngradeToHigherVersion() { public void testCannotUnsafeDowngradeToHigherVersion() {
FeatureControlManager manager = TEST_MANAGER_BUILDER1.build(); FeatureControlManager manager = TEST_MANAGER_BUILDER1.build();
assertEquals(ControllerResult.of(Collections.emptyList(), new ApiError(Errors.INVALID_UPDATE_VERSION, 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.")), "newer version.")),
manager.updateFeatures( 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), singletonMap(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE),
true)); true));
} }
@Test @Test
public void testCannotUpgradeToLowerVersion() { 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, 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 " + "version of this feature without setting the upgrade type to either safe or " +
"unsafe downgrade.")), "unsafe downgrade.")),
manager.updateFeatures( 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), singletonMap(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UPGRADE),
true)); true));
} }
@ -306,7 +300,7 @@ public class FeatureControlManagerTest {
FeatureControlManager manager = TEST_MANAGER_BUILDER1.build(); FeatureControlManager manager = TEST_MANAGER_BUILDER1.build();
assertEquals(ControllerResult.of(Collections.emptyList(), ApiError.NONE), assertEquals(ControllerResult.of(Collections.emptyList(), ApiError.NONE),
manager.updateFeatures( 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), singletonMap(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UPGRADE),
true)); true));
} }
@ -315,10 +309,10 @@ public class FeatureControlManagerTest {
public void testCannotUseSafeDowngradeIfMetadataChanged() { public void testCannotUseSafeDowngradeIfMetadataChanged() {
FeatureControlManager manager = TEST_MANAGER_BUILDER1.build(); FeatureControlManager manager = TEST_MANAGER_BUILDER1.build();
assertEquals(ControllerResult.of(Collections.emptyList(), new ApiError(Errors.INVALID_UPDATE_VERSION, 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.")), "it might delete metadata information.")),
manager.updateFeatures( 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), singletonMap(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.SAFE_DOWNGRADE),
true)); true));
} }
@ -327,9 +321,9 @@ public class FeatureControlManagerTest {
public void testUnsafeDowngradeIsTemporarilyDisabled() { public void testUnsafeDowngradeIsTemporarilyDisabled() {
FeatureControlManager manager = TEST_MANAGER_BUILDER1.build(); FeatureControlManager manager = TEST_MANAGER_BUILDER1.build();
assertEquals(ControllerResult.of(Collections.emptyList(), new ApiError(Errors.INVALID_UPDATE_VERSION, 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( 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), singletonMap(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE),
true)); true));
} }
@ -340,7 +334,7 @@ public class FeatureControlManagerTest {
FeatureControlManager manager = TEST_MANAGER_BUILDER1.build(); FeatureControlManager manager = TEST_MANAGER_BUILDER1.build();
assertEquals(ControllerResult.of(Collections.emptyList(), ApiError.NONE), assertEquals(ControllerResult.of(Collections.emptyList(), ApiError.NONE),
manager.updateFeatures( 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), singletonMap(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE),
true)); true));
} }
@ -349,28 +343,27 @@ public class FeatureControlManagerTest {
public void testCanUseSafeDowngradeIfMetadataDidNotChange() { public void testCanUseSafeDowngradeIfMetadataDidNotChange() {
FeatureControlManager manager = new FeatureControlManager.Builder(). FeatureControlManager manager = new FeatureControlManager.Builder().
setQuorumFeatures(features(MetadataVersion.FEATURE_NAME, setQuorumFeatures(features(MetadataVersion.FEATURE_NAME,
MetadataVersion.IBP_3_0_IV1.featureLevel(), MetadataVersion.IBP_3_3_IV1.featureLevel())). MetadataVersion.MINIMUM_VERSION.featureLevel(), MetadataVersion.IBP_3_6_IV0.featureLevel())).
setMetadataVersion(MetadataVersion.IBP_3_1_IV0). setMetadataVersion(MetadataVersion.IBP_3_5_IV0).
setMinimumBootstrapVersion(MetadataVersion.IBP_3_0_IV1).
build(); build();
assertEquals(ControllerResult.of(Collections.emptyList(), ApiError.NONE), assertEquals(ControllerResult.of(Collections.emptyList(), ApiError.NONE),
manager.updateFeatures( 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), singletonMap(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.SAFE_DOWNGRADE),
true)); true));
} }
@Test @Test
public void testCannotDowngradeBefore3_3_IV0() { public void testCannotDowngradeBeforeMinimumKraftVersion() {
FeatureControlManager manager = new FeatureControlManager.Builder(). FeatureControlManager manager = new FeatureControlManager.Builder().
setQuorumFeatures(features(MetadataVersion.FEATURE_NAME, setQuorumFeatures(features(MetadataVersion.FEATURE_NAME,
MetadataVersion.IBP_3_0_IV1.featureLevel(), MetadataVersion.IBP_3_3_IV3.featureLevel())). MetadataVersion.MINIMUM_VERSION.featureLevel(), MetadataVersion.latestTesting().featureLevel())).
setMetadataVersion(MetadataVersion.IBP_3_3_IV0). setMetadataVersion(MetadataVersion.MINIMUM_VERSION).
build(); build();
assertEquals(ControllerResult.of(Collections.emptyList(), new ApiError(Errors.INVALID_UPDATE_VERSION, 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( 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), singletonMap(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE),
true)); true));
} }
@ -379,7 +372,7 @@ public class FeatureControlManagerTest {
public void testCreateFeatureLevelRecords() { public void testCreateFeatureLevelRecords() {
Map<String, VersionRange> localSupportedFeatures = new HashMap<>(); Map<String, VersionRange> localSupportedFeatures = new HashMap<>();
localSupportedFeatures.put(MetadataVersion.FEATURE_NAME, VersionRange.of( 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)); localSupportedFeatures.put(Feature.TEST_VERSION.featureName(), VersionRange.of(0, 2));
FeatureControlManager manager = new FeatureControlManager.Builder(). FeatureControlManager manager = new FeatureControlManager.Builder().
setQuorumFeatures(new QuorumFeatures(0, localSupportedFeatures, emptyList())). setQuorumFeatures(new QuorumFeatures(0, localSupportedFeatures, emptyList())).

View File

@ -598,9 +598,8 @@ public class PartitionChangeBuilderTest {
); );
} }
@ParameterizedTest @Test
@ValueSource(booleans = {true, false}) public void testChangeInLeadershipDoesNotChangeRecoveryState() {
public void testChangeInLeadershipDoesNotChangeRecoveryState(boolean isLeaderRecoverySupported) {
final byte noChange = (byte) -1; final byte noChange = (byte) -1;
int leaderId = 1; int leaderId = 1;
LeaderRecoveryState recoveryState = LeaderRecoveryState.RECOVERING; LeaderRecoveryState recoveryState = LeaderRecoveryState.RECOVERING;
@ -618,7 +617,7 @@ public class PartitionChangeBuilderTest {
setPartitionEpoch(200). setPartitionEpoch(200).
build(); build();
MetadataVersion metadataVersion = leaderRecoveryMetadataVersion(isLeaderRecoverySupported); MetadataVersion metadataVersion = MetadataVersion.MINIMUM_VERSION;
// Change the partition so that there is no leader // Change the partition so that there is no leader
PartitionChangeBuilder offlineBuilder = new PartitionChangeBuilder( PartitionChangeBuilder offlineBuilder = new PartitionChangeBuilder(
@ -667,10 +666,8 @@ public class PartitionChangeBuilderTest {
assertEquals(recoveryState, registration.leaderRecoveryState); assertEquals(recoveryState, registration.leaderRecoveryState);
} }
@ParameterizedTest @Test
@ValueSource(booleans = {true, false}) void testUncleanSetsLeaderRecoveringState() {
void testUncleanSetsLeaderRecoveringState(boolean isLeaderRecoverySupported) {
final byte noChange = (byte) -1;
int leaderId = 1; int leaderId = 1;
PartitionRegistration registration = new PartitionRegistration.Builder(). PartitionRegistration registration = new PartitionRegistration.Builder().
setReplicas(new int[] {leaderId, leaderId + 1, leaderId + 2}). setReplicas(new int[] {leaderId, leaderId + 1, leaderId + 2}).
@ -686,7 +683,7 @@ public class PartitionChangeBuilderTest {
setPartitionEpoch(200). setPartitionEpoch(200).
build(); build();
MetadataVersion metadataVersion = leaderRecoveryMetadataVersion(isLeaderRecoverySupported); MetadataVersion metadataVersion = MetadataVersion.MINIMUM_VERSION;
// Change the partition using unclean leader election // Change the partition using unclean leader election
PartitionChangeBuilder onlineBuilder = new PartitionChangeBuilder( PartitionChangeBuilder onlineBuilder = new PartitionChangeBuilder(
@ -703,26 +700,16 @@ public class PartitionChangeBuilderTest {
.get() .get()
.message(); .message();
byte expectedRecoveryChange = noChange; assertEquals(LeaderRecoveryState.RECOVERING.value(), changeRecord.leaderRecoveryState());
if (isLeaderRecoverySupported) {
expectedRecoveryChange = LeaderRecoveryState.RECOVERING.value();
}
assertEquals(expectedRecoveryChange, changeRecord.leaderRecoveryState());
assertEquals(leaderId, changeRecord.leader()); assertEquals(leaderId, changeRecord.leader());
assertEquals(1, changeRecord.isr().size()); assertEquals(1, changeRecord.isr().size());
assertEquals(leaderId, changeRecord.isr().get(0)); assertEquals(leaderId, changeRecord.isr().get(0));
registration = registration.merge(changeRecord); registration = registration.merge(changeRecord);
LeaderRecoveryState expectedRecovery = LeaderRecoveryState.RECOVERED;
if (isLeaderRecoverySupported) {
expectedRecovery = LeaderRecoveryState.RECOVERING;
}
assertEquals(leaderId, registration.leader); assertEquals(leaderId, registration.leader);
assertEquals(leaderId, registration.isr[0]); assertEquals(leaderId, registration.isr[0]);
assertEquals(expectedRecovery, registration.leaderRecoveryState); assertEquals(LeaderRecoveryState.RECOVERING, registration.leaderRecoveryState);
} }
@Test @Test
@ -770,7 +757,7 @@ public class PartitionChangeBuilderTest {
topicId, topicId,
0, 0,
isValidLeader, isValidLeader,
leaderRecoveryMetadataVersion(false), MetadataVersion.MINIMUM_VERSION,
2 2
); );
@ -790,14 +777,6 @@ public class PartitionChangeBuilderTest {
build()); build());
} }
private MetadataVersion leaderRecoveryMetadataVersion(boolean isSupported) {
if (isSupported) {
return MetadataVersion.IBP_3_2_IV0;
} else {
return MetadataVersion.IBP_3_1_IV0;
}
}
@ParameterizedTest @ParameterizedTest
@MethodSource("partitionChangeRecordVersions") @MethodSource("partitionChangeRecordVersions")
public void testEligibleLeaderReplicas_IsrShrinkBelowMinISR(short version) { public void testEligibleLeaderReplicas_IsrShrinkBelowMinISR(short version) {

View File

@ -55,7 +55,7 @@ public class QuorumControllerIntegrationTestUtils {
private static final Logger log = LoggerFactory.getLogger(QuorumControllerIntegrationTestUtils.class); private static final Logger log = LoggerFactory.getLogger(QuorumControllerIntegrationTestUtils.class);
BrokerRegistrationRequestData.FeatureCollection brokerFeatures() { 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) .setBrokerId(brokerId)
.setRack(null) .setRack(null)
.setClusterId(controller.clusterId()) .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()))) Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME, EligibleLeaderReplicasVersion.ELRV_1.featureLevel())))
.setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwB" + brokerId)) .setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwB" + brokerId))
.setLogDirs(Collections.singletonList( .setLogDirs(Collections.singletonList(

View File

@ -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.RegisterBrokerRecord.BrokerEndpointCollection;
import org.apache.kafka.common.metadata.RegisterControllerRecord; import org.apache.kafka.common.metadata.RegisterControllerRecord;
import org.apache.kafka.common.metadata.TopicRecord; 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.ApiKeys;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.requests.AlterPartitionRequest; import org.apache.kafka.common.requests.AlterPartitionRequest;
@ -191,7 +190,7 @@ public class QuorumControllerTest {
) { ) {
controlEnv.activeController().registerBroker(ANONYMOUS_CONTEXT, controlEnv.activeController().registerBroker(ANONYMOUS_CONTEXT,
new BrokerRegistrationRequestData(). 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()))). Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME, EligibleLeaderReplicasVersion.ELRV_1.featureLevel()))).
setBrokerId(0). setBrokerId(0).
setLogDirs(Collections.singletonList(Uuid.fromString("iiaQjkRPQcuMULNII0MUeA"))). setLogDirs(Collections.singletonList(Uuid.fromString("iiaQjkRPQcuMULNII0MUeA"))).
@ -233,7 +232,7 @@ public class QuorumControllerTest {
) { ) {
controlEnv.activeController().registerBroker(ANONYMOUS_CONTEXT, controlEnv.activeController().registerBroker(ANONYMOUS_CONTEXT,
new BrokerRegistrationRequestData(). 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()))). Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME, EligibleLeaderReplicasVersion.ELRV_1.featureLevel()))).
setBrokerId(0). setBrokerId(0).
setLogDirs(Collections.singletonList(Uuid.fromString("sTbzRAMnTpahIyIPNjiLhw"))). setLogDirs(Collections.singletonList(Uuid.fromString("sTbzRAMnTpahIyIPNjiLhw"))).
@ -289,7 +288,7 @@ public class QuorumControllerTest {
new BrokerRegistrationRequestData(). new BrokerRegistrationRequestData().
setBrokerId(brokerId). setBrokerId(brokerId).
setClusterId(active.clusterId()). setClusterId(active.clusterId()).
setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.latestTesting())). setFeatures(brokerFeatures(MetadataVersion.MINIMUM_VERSION, MetadataVersion.latestTesting())).
setIncarnationId(Uuid.randomUuid()). setIncarnationId(Uuid.randomUuid()).
setListeners(listeners)); setListeners(listeners));
brokerEpochs.put(brokerId, reply.get().epoch()); brokerEpochs.put(brokerId, reply.get().epoch());
@ -398,7 +397,7 @@ public class QuorumControllerTest {
QuorumController active = controlEnv.activeController(); QuorumController active = controlEnv.activeController();
Map<Integer, Long> brokerEpochs = new HashMap<>(); Map<Integer, Long> brokerEpochs = new HashMap<>();
BrokerRegistrationRequestData.FeatureCollection features = 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())); Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME, EligibleLeaderReplicasVersion.ELRV_1.featureLevel()));
for (Integer brokerId : allBrokers) { for (Integer brokerId : allBrokers) {
CompletableFuture<BrokerRegistrationReply> reply = active.registerBroker( CompletableFuture<BrokerRegistrationReply> reply = active.registerBroker(
@ -530,7 +529,7 @@ public class QuorumControllerTest {
QuorumController active = controlEnv.activeController(); QuorumController active = controlEnv.activeController();
Map<Integer, Long> brokerEpochs = new HashMap<>(); Map<Integer, Long> brokerEpochs = new HashMap<>();
BrokerRegistrationRequestData.FeatureCollection features = 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())); Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME, EligibleLeaderReplicasVersion.ELRV_0.featureLevel()));
for (Integer brokerId : allBrokers) { for (Integer brokerId : allBrokers) {
CompletableFuture<BrokerRegistrationReply> reply = active.registerBroker( CompletableFuture<BrokerRegistrationReply> reply = active.registerBroker(
@ -629,7 +628,7 @@ public class QuorumControllerTest {
new BrokerRegistrationRequestData(). new BrokerRegistrationRequestData().
setBrokerId(brokerId). setBrokerId(brokerId).
setClusterId(active.clusterId()). 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()))). Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME, EligibleLeaderReplicasVersion.ELRV_1.featureLevel()))).
setIncarnationId(Uuid.randomUuid()). setIncarnationId(Uuid.randomUuid()).
setLogDirs(Collections.singletonList(Uuid.randomUuid())). setLogDirs(Collections.singletonList(Uuid.randomUuid())).
@ -760,7 +759,7 @@ public class QuorumControllerTest {
new BrokerRegistrationRequestData(). new BrokerRegistrationRequestData().
setBrokerId(brokerId). setBrokerId(brokerId).
setClusterId(active.clusterId()). 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()). setIncarnationId(Uuid.randomUuid()).
setListeners(listeners)); setListeners(listeners));
brokerEpochs.put(brokerId, reply.get().epoch()); brokerEpochs.put(brokerId, reply.get().epoch());
@ -818,7 +817,7 @@ public class QuorumControllerTest {
new BrokerRegistrationRequestData(). new BrokerRegistrationRequestData().
setBrokerId(brokerId). setBrokerId(brokerId).
setClusterId(active.clusterId()). 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()). setIncarnationId(Uuid.randomUuid()).
setListeners(listeners)); setListeners(listeners));
brokerEpochs.put(brokerId, reply.get().epoch()); brokerEpochs.put(brokerId, reply.get().epoch());
@ -830,7 +829,8 @@ public class QuorumControllerTest {
// Let the unfenced broker, 3, join the ISR partition 2 // Let the unfenced broker, 3, join the ISR partition 2
Set<TopicIdPartition> imbalancedPartitions = new HashSet<>(active.replicationControl().imbalancedPartitions()); Set<TopicIdPartition> imbalancedPartitions = new HashSet<>(active.replicationControl().imbalancedPartitions());
assertEquals(1, imbalancedPartitions.size()); 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); PartitionRegistration partitionRegistration = active.replicationControl().getPartition(topicIdFoo, imbalancedPartitionId);
AlterPartitionRequestData.PartitionData partitionData = new AlterPartitionRequestData.PartitionData() AlterPartitionRequestData.PartitionData partitionData = new AlterPartitionRequestData.PartitionData()
.setPartitionIndex(imbalancedPartitionId) .setPartitionIndex(imbalancedPartitionId)
@ -839,7 +839,7 @@ public class QuorumControllerTest {
.setNewIsrWithEpochs(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(Arrays.asList(1, 2, 3))); .setNewIsrWithEpochs(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(Arrays.asList(1, 2, 3)));
AlterPartitionRequestData.TopicData topicData = new AlterPartitionRequestData.TopicData() AlterPartitionRequestData.TopicData topicData = new AlterPartitionRequestData.TopicData()
.setTopicName("foo"); .setTopicId(impalancedTp.topicId());
topicData.partitions().add(partitionData); topicData.partitions().add(partitionData);
AlterPartitionRequestData alterPartitionRequest = new AlterPartitionRequestData() AlterPartitionRequestData alterPartitionRequest = new AlterPartitionRequestData()
@ -848,7 +848,7 @@ public class QuorumControllerTest {
alterPartitionRequest.topics().add(topicData); alterPartitionRequest.topics().add(topicData);
active.alterPartition(ANONYMOUS_CONTEXT, new AlterPartitionRequest 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())); AtomicLong lastHeartbeatMs = new AtomicLong(getMonotonicMs(active.time()));
sendBrokerHeartbeatToUnfenceBrokers(active, allBrokers, brokerEpochs); sendBrokerHeartbeatToUnfenceBrokers(active, allBrokers, brokerEpochs);
@ -938,7 +938,7 @@ public class QuorumControllerTest {
BrokerRegistrationRequestData.FeatureCollection brokerFeatures = new BrokerRegistrationRequestData.FeatureCollection(); BrokerRegistrationRequestData.FeatureCollection brokerFeatures = new BrokerRegistrationRequestData.FeatureCollection();
brokerFeatures.add(new BrokerRegistrationRequestData.Feature() brokerFeatures.add(new BrokerRegistrationRequestData.Feature()
.setName(MetadataVersion.FEATURE_NAME) .setName(MetadataVersion.FEATURE_NAME)
.setMinSupportedVersion(MetadataVersion.IBP_3_0_IV1.featureLevel()) .setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel())
.setMaxSupportedVersion(MetadataVersion.latestTesting().featureLevel())); .setMaxSupportedVersion(MetadataVersion.latestTesting().featureLevel()));
// broker registration requests do not include initial versions of features // broker registration requests do not include initial versions of features
if (brokerMaxSupportedKraftVersion != 0) { if (brokerMaxSupportedKraftVersion != 0) {
@ -991,7 +991,7 @@ public class QuorumControllerTest {
setBrokerId(0). setBrokerId(0).
setClusterId(active.clusterId()). setClusterId(active.clusterId()).
setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwBA")). 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()))). Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME, EligibleLeaderReplicasVersion.ELRV_1.featureLevel()))).
setLogDirs(Collections.singletonList(Uuid.fromString("vBpaRsZVSaGsQT53wtYGtg"))). setLogDirs(Collections.singletonList(Uuid.fromString("vBpaRsZVSaGsQT53wtYGtg"))).
setListeners(listeners)); setListeners(listeners));
@ -1082,7 +1082,7 @@ public class QuorumControllerTest {
singletonList( singletonList(
new ControllerRegistrationRequestData.Feature(). new ControllerRegistrationRequestData.Feature().
setName(MetadataVersion.FEATURE_NAME). setName(MetadataVersion.FEATURE_NAME).
setMinSupportedVersion(MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel()). setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()).
setMaxSupportedVersion(MetadataVersion.IBP_3_7_IV0.featureLevel()) setMaxSupportedVersion(MetadataVersion.IBP_3_7_IV0.featureLevel())
).iterator() ).iterator()
))).get(); ))).get();
@ -1093,7 +1093,7 @@ public class QuorumControllerTest {
setBrokerId(i). setBrokerId(i).
setRack(null). setRack(null).
setClusterId(active.clusterId()). 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)). setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwB" + i)).
setListeners(new ListenerCollection(singletonList(new Listener(). setListeners(new ListenerCollection(singletonList(new Listener().
setName("PLAINTEXT").setHost("localhost"). setName("PLAINTEXT").setHost("localhost").
@ -1152,7 +1152,7 @@ public class QuorumControllerTest {
singletonList( singletonList(
new RegisterControllerRecord.ControllerFeature(). new RegisterControllerRecord.ControllerFeature().
setName(MetadataVersion.FEATURE_NAME). setName(MetadataVersion.FEATURE_NAME).
setMinSupportedVersion(MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel()). setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()).
setMaxSupportedVersion(MetadataVersion.IBP_3_7_IV0.featureLevel())).iterator())), setMaxSupportedVersion(MetadataVersion.IBP_3_7_IV0.featureLevel())).iterator())),
(short) 0), (short) 0),
new ApiMessageAndVersion(new RegisterControllerRecord(). new ApiMessageAndVersion(new RegisterControllerRecord().
@ -1169,7 +1169,7 @@ public class QuorumControllerTest {
singletonList( singletonList(
new RegisterControllerRecord.ControllerFeature(). new RegisterControllerRecord.ControllerFeature().
setName(MetadataVersion.FEATURE_NAME). setName(MetadataVersion.FEATURE_NAME).
setMinSupportedVersion(MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel()). setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()).
setMaxSupportedVersion(MetadataVersion.IBP_3_7_IV0.featureLevel())).iterator())), setMaxSupportedVersion(MetadataVersion.IBP_3_7_IV0.featureLevel())).iterator())),
(short) 0), (short) 0),
new ApiMessageAndVersion(new RegisterControllerRecord(). new ApiMessageAndVersion(new RegisterControllerRecord().
@ -1186,7 +1186,7 @@ public class QuorumControllerTest {
singletonList( singletonList(
new RegisterControllerRecord.ControllerFeature(). new RegisterControllerRecord.ControllerFeature().
setName(MetadataVersion.FEATURE_NAME). setName(MetadataVersion.FEATURE_NAME).
setMinSupportedVersion(MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel()). setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()).
setMaxSupportedVersion(MetadataVersion.IBP_3_7_IV0.featureLevel())).iterator())), setMaxSupportedVersion(MetadataVersion.IBP_3_7_IV0.featureLevel())).iterator())),
(short) 0), (short) 0),
new ApiMessageAndVersion(new RegisterBrokerRecord(). new ApiMessageAndVersion(new RegisterBrokerRecord().
@ -1195,7 +1195,7 @@ public class QuorumControllerTest {
setEndPoints(new BrokerEndpointCollection( setEndPoints(new BrokerEndpointCollection(
singletonList(new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost"). singletonList(new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost").
setPort(9092).setSecurityProtocol((short) 0)).iterator())). 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). setRack(null).
setFenced(true), (short) 2), setFenced(true), (short) 2),
new ApiMessageAndVersion(new RegisterBrokerRecord(). new ApiMessageAndVersion(new RegisterBrokerRecord().
@ -1204,7 +1204,7 @@ public class QuorumControllerTest {
setEndPoints(new BrokerEndpointCollection(singletonList( setEndPoints(new BrokerEndpointCollection(singletonList(
new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost"). new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost").
setPort(9093).setSecurityProtocol((short) 0)).iterator())). 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). setRack(null).
setFenced(true), (short) 2), setFenced(true), (short) 2),
new ApiMessageAndVersion(new RegisterBrokerRecord(). new ApiMessageAndVersion(new RegisterBrokerRecord().
@ -1213,7 +1213,7 @@ public class QuorumControllerTest {
setEndPoints(new BrokerEndpointCollection( setEndPoints(new BrokerEndpointCollection(
singletonList(new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost"). singletonList(new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost").
setPort(9094).setSecurityProtocol((short) 0)).iterator())). 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). setRack(null).
setFenced(true), (short) 2), setFenced(true), (short) 2),
new ApiMessageAndVersion(new RegisterBrokerRecord(). new ApiMessageAndVersion(new RegisterBrokerRecord().
@ -1222,7 +1222,7 @@ public class QuorumControllerTest {
setEndPoints(new BrokerEndpointCollection(singletonList( setEndPoints(new BrokerEndpointCollection(singletonList(
new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost"). new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost").
setPort(9095).setSecurityProtocol((short) 0)).iterator())). 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). setRack(null).
setFenced(true), (short) 2), setFenced(true), (short) 2),
new ApiMessageAndVersion(new BrokerRegistrationChangeRecord(). new ApiMessageAndVersion(new BrokerRegistrationChangeRecord().
@ -1493,28 +1493,11 @@ public class QuorumControllerTest {
} }
} }
private static final List<ApiMessageAndVersion> 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( private static final BootstrapMetadata COMPLEX_BOOTSTRAP = BootstrapMetadata.fromRecords(
Arrays.asList( Arrays.asList(
new ApiMessageAndVersion(new FeatureLevelRecord(). new ApiMessageAndVersion(new FeatureLevelRecord().
setName(MetadataVersion.FEATURE_NAME). setName(MetadataVersion.FEATURE_NAME).
setFeatureLevel(MetadataVersion.IBP_3_3_IV1.featureLevel()), setFeatureLevel(MetadataVersion.MINIMUM_VERSION.featureLevel()),
(short) 0), (short) 0),
new ApiMessageAndVersion(new ConfigRecord(). new ApiMessageAndVersion(new ConfigRecord().
setResourceType(BROKER.id()). setResourceType(BROKER.id()).
@ -1524,31 +1507,6 @@ public class QuorumControllerTest {
(short) 0)), (short) 0)),
"test bootstrap"); "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 @Test
public void testInsertBootstrapRecordsToEmptyLog() throws Exception { public void testInsertBootstrapRecordsToEmptyLog() throws Exception {
try ( try (
@ -1566,7 +1524,7 @@ public class QuorumControllerTest {
TestUtils.waitForCondition(() -> { TestUtils.waitForCondition(() -> {
FinalizedControllerFeatures features = active.finalizedFeatures(ctx).get(); FinalizedControllerFeatures features = active.finalizedFeatures(ctx).get();
Optional<Short> metadataVersionOpt = features.get(MetadataVersion.FEATURE_NAME); Optional<Short> 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"); }, "Failed to see expected metadata.version from bootstrap metadata");
TestUtils.waitForCondition(() -> { TestUtils.waitForCondition(() -> {
@ -1651,8 +1609,8 @@ public class QuorumControllerTest {
public void testActivationRecords33() { public void testActivationRecords33() {
FeatureControlManager featureControl; FeatureControlManager featureControl;
featureControl = getActivationRecords(MetadataVersion.IBP_3_3_IV0); featureControl = getActivationRecords(MetadataVersion.IBP_3_3_IV3);
assertEquals(MetadataVersion.IBP_3_3_IV0, featureControl.metadataVersion()); assertEquals(MetadataVersion.IBP_3_3_IV3, featureControl.metadataVersion());
} }
@Test @Test

View File

@ -49,7 +49,7 @@ public class QuorumFeaturesTest {
public void testDefaultFeatureMap() { public void testDefaultFeatureMap() {
Map<String, VersionRange> expectedFeatures = new HashMap<>(1); Map<String, VersionRange> expectedFeatures = new HashMap<>(1);
expectedFeatures.put(MetadataVersion.FEATURE_NAME, VersionRange.of( expectedFeatures.put(MetadataVersion.FEATURE_NAME, VersionRange.of(
MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel(), MetadataVersion.MINIMUM_VERSION.featureLevel(),
MetadataVersion.LATEST_PRODUCTION.featureLevel())); MetadataVersion.LATEST_PRODUCTION.featureLevel()));
for (Feature feature : Feature.PRODUCTION_FEATURES) { for (Feature feature : Feature.PRODUCTION_FEATURES) {
short maxVersion = feature.latestProduction(); short maxVersion = feature.latestProduction();
@ -67,7 +67,7 @@ public class QuorumFeaturesTest {
public void testDefaultFeatureMapWithUnstable() { public void testDefaultFeatureMapWithUnstable() {
Map<String, VersionRange> expectedFeatures = new HashMap<>(1); Map<String, VersionRange> expectedFeatures = new HashMap<>(1);
expectedFeatures.put(MetadataVersion.FEATURE_NAME, VersionRange.of( expectedFeatures.put(MetadataVersion.FEATURE_NAME, VersionRange.of(
MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel(), MetadataVersion.MINIMUM_VERSION.featureLevel(),
MetadataVersion.latestTesting().featureLevel())); MetadataVersion.latestTesting().featureLevel()));
for (Feature feature : Feature.PRODUCTION_FEATURES) { for (Feature feature : Feature.PRODUCTION_FEATURES) {
short maxVersion = feature.defaultLevel(MetadataVersion.latestTesting()); short maxVersion = feature.defaultLevel(MetadataVersion.latestTesting());

View File

@ -105,7 +105,6 @@ import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.Timeout;
import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.CsvSource; import org.junit.jupiter.params.provider.CsvSource;
import org.junit.jupiter.params.provider.EnumSource;
import org.junit.jupiter.params.provider.ValueSource; import org.junit.jupiter.params.provider.ValueSource;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; 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.metadata.MetadataRecordType.CLEAR_ELR_RECORD;
import static org.apache.kafka.common.protocol.Errors.ELECTION_NOT_NEEDED; 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.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.INELIGIBLE_REPLICA;
import static org.apache.kafka.common.protocol.Errors.INVALID_PARTITIONS; import static org.apache.kafka.common.protocol.Errors.INVALID_PARTITIONS;
import static org.apache.kafka.common.protocol.Errors.INVALID_REPLICATION_FACTOR; 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_CONTROLLER;
import static org.apache.kafka.common.protocol.Errors.NOT_LEADER_OR_FOLLOWER; 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.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.POLICY_VIOLATION;
import static org.apache.kafka.common.protocol.Errors.PREFERRED_LEADER_NOT_AVAILABLE; import static org.apache.kafka.common.protocol.Errors.PREFERRED_LEADER_NOT_AVAILABLE;
import static org.apache.kafka.common.protocol.Errors.THROTTLING_QUOTA_EXCEEDED; import static org.apache.kafka.common.protocol.Errors.THROTTLING_QUOTA_EXCEEDED;
@ -443,9 +440,7 @@ public class ReplicationControlManagerTest {
.setLeaderRecoveryState(leaderRecoveryState.value()) .setLeaderRecoveryState(leaderRecoveryState.value())
.setNewIsrWithEpochs(isrWithEpoch); .setNewIsrWithEpochs(isrWithEpoch);
String topicName = replicationControl.getTopic(topicIdPartition.topicId()).name();
TopicData topicData = new TopicData() TopicData topicData = new TopicData()
.setTopicName(topicName)
.setTopicId(topicIdPartition.topicId()) .setTopicId(topicIdPartition.topicId())
.setPartitions(singletonList(partitionData)); .setPartitions(singletonList(partitionData));
@ -1252,15 +1247,13 @@ public class ReplicationControlManagerTest {
ctx.registerBrokers(0, 1, 2); ctx.registerBrokers(0, 1, 2);
ctx.unfenceBrokers(0, 1, 2); ctx.unfenceBrokers(0, 1, 2);
String topicName = "foo";
Uuid topicId = Uuid.randomUuid(); Uuid topicId = Uuid.randomUuid();
AlterPartitionRequestData request = new AlterPartitionRequestData() AlterPartitionRequestData request = new AlterPartitionRequestData()
.setBrokerId(0) .setBrokerId(0)
.setBrokerEpoch(100) .setBrokerEpoch(100)
.setTopics(singletonList(new TopicData() .setTopics(singletonList(new TopicData()
.setTopicName(version <= 1 ? topicName : "") .setTopicId(topicId)
.setTopicId(version > 1 ? topicId : Uuid.ZERO_UUID)
.setPartitions(singletonList(new PartitionData() .setPartitions(singletonList(new PartitionData()
.setPartitionIndex(0))))); .setPartitionIndex(0)))));
@ -1270,11 +1263,10 @@ public class ReplicationControlManagerTest {
ControllerResult<AlterPartitionResponseData> result = ControllerResult<AlterPartitionResponseData> result =
replicationControl.alterPartition(requestContext, request); replicationControl.alterPartition(requestContext, request);
Errors expectedError = version > 1 ? UNKNOWN_TOPIC_ID : UNKNOWN_TOPIC_OR_PARTITION; Errors expectedError = UNKNOWN_TOPIC_ID;
AlterPartitionResponseData expectedResponse = new AlterPartitionResponseData() AlterPartitionResponseData expectedResponse = new AlterPartitionResponseData()
.setTopics(singletonList(new AlterPartitionResponseData.TopicData() .setTopics(singletonList(new AlterPartitionResponseData.TopicData()
.setTopicName(version <= 1 ? topicName : "") .setTopicId(topicId)
.setTopicId(version > 1 ? topicId : Uuid.ZERO_UUID)
.setPartitions(singletonList(new AlterPartitionResponseData.PartitionData() .setPartitions(singletonList(new AlterPartitionResponseData.PartitionData()
.setPartitionIndex(0) .setPartitionIndex(0)
.setErrorCode(expectedError.code()))))); .setErrorCode(expectedError.code())))));
@ -1908,8 +1900,7 @@ public class ReplicationControlManagerTest {
setBrokerId(3). setBrokerId(3).
setBrokerEpoch(103). setBrokerEpoch(103).
setTopics(singletonList(new TopicData(). setTopics(singletonList(new TopicData().
setTopicName(version <= 1 ? "foo" : ""). setTopicId(fooId).
setTopicId(version > 1 ? fooId : Uuid.ZERO_UUID).
setPartitions(singletonList(new PartitionData(). setPartitions(singletonList(new PartitionData().
setPartitionIndex(1). setPartitionIndex(1).
setPartitionEpoch(1). setPartitionEpoch(1).
@ -1918,11 +1909,10 @@ public class ReplicationControlManagerTest {
ControllerResult<AlterPartitionResponseData> alterPartitionResult = replication.alterPartition( ControllerResult<AlterPartitionResponseData> alterPartitionResult = replication.alterPartition(
requestContext, requestContext,
new AlterPartitionRequest.Builder(alterPartitionRequestData).build(version).data()); 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( assertEquals(new AlterPartitionResponseData().setTopics(singletonList(
new AlterPartitionResponseData.TopicData(). new AlterPartitionResponseData.TopicData().
setTopicName(version <= 1 ? "foo" : ""). setTopicId(fooId).
setTopicId(version > 1 ? fooId : Uuid.ZERO_UUID).
setPartitions(singletonList( setPartitions(singletonList(
new AlterPartitionResponseData.PartitionData(). new AlterPartitionResponseData.PartitionData().
setPartitionIndex(1). setPartitionIndex(1).
@ -1969,8 +1959,7 @@ public class ReplicationControlManagerTest {
.setBrokerId(1) .setBrokerId(1)
.setBrokerEpoch(101) .setBrokerEpoch(101)
.setTopics(singletonList(new TopicData() .setTopics(singletonList(new TopicData()
.setTopicName(version <= 1 ? "foo" : "") .setTopicId(fooId)
.setTopicId(version > 1 ? fooId : Uuid.ZERO_UUID)
.setPartitions(singletonList(new PartitionData() .setPartitions(singletonList(new PartitionData()
.setPartitionIndex(0) .setPartitionIndex(0)
.setPartitionEpoch(1) .setPartitionEpoch(1)
@ -1983,12 +1972,11 @@ public class ReplicationControlManagerTest {
ControllerResult<AlterPartitionResponseData> alterPartitionResult = ControllerResult<AlterPartitionResponseData> alterPartitionResult =
replication.alterPartition(requestContext, new AlterPartitionRequest.Builder(alterIsrRequest).build(version).data()); replication.alterPartition(requestContext, new AlterPartitionRequest.Builder(alterIsrRequest).build(version).data());
Errors expectedError = version <= 1 ? OPERATION_NOT_ATTEMPTED : INELIGIBLE_REPLICA; Errors expectedError = INELIGIBLE_REPLICA;
assertEquals( assertEquals(
new AlterPartitionResponseData() new AlterPartitionResponseData()
.setTopics(singletonList(new AlterPartitionResponseData.TopicData() .setTopics(singletonList(new AlterPartitionResponseData.TopicData()
.setTopicName(version <= 1 ? "foo" : "") .setTopicId(fooId)
.setTopicId(version > 1 ? fooId : Uuid.ZERO_UUID)
.setPartitions(singletonList(new AlterPartitionResponseData.PartitionData() .setPartitions(singletonList(new AlterPartitionResponseData.PartitionData()
.setPartitionIndex(0) .setPartitionIndex(0)
.setErrorCode(expectedError.code()))))), .setErrorCode(expectedError.code()))))),
@ -2003,8 +1991,7 @@ public class ReplicationControlManagerTest {
assertEquals( assertEquals(
new AlterPartitionResponseData() new AlterPartitionResponseData()
.setTopics(singletonList(new AlterPartitionResponseData.TopicData() .setTopics(singletonList(new AlterPartitionResponseData.TopicData()
.setTopicName(version <= 1 ? "foo" : "") .setTopicId(fooId)
.setTopicId(version > 1 ? fooId : Uuid.ZERO_UUID)
.setPartitions(singletonList(new AlterPartitionResponseData.PartitionData() .setPartitions(singletonList(new AlterPartitionResponseData.PartitionData()
.setPartitionIndex(0) .setPartitionIndex(0)
.setLeaderId(1) .setLeaderId(1)
@ -2033,8 +2020,7 @@ public class ReplicationControlManagerTest {
setBrokerId(1). setBrokerId(1).
setBrokerEpoch(101). setBrokerEpoch(101).
setTopics(singletonList(new TopicData(). setTopics(singletonList(new TopicData().
setTopicName(version <= 1 ? "foo" : ""). setTopicId(fooId).
setTopicId(version > 1 ? fooId : Uuid.ZERO_UUID).
setPartitions(singletonList(new PartitionData(). setPartitions(singletonList(new PartitionData().
setPartitionIndex(0). setPartitionIndex(0).
setPartitionEpoch(1). setPartitionEpoch(1).
@ -2073,7 +2059,6 @@ public class ReplicationControlManagerTest {
assertEquals( assertEquals(
new AlterPartitionResponseData(). new AlterPartitionResponseData().
setTopics(singletonList(new AlterPartitionResponseData.TopicData(). setTopics(singletonList(new AlterPartitionResponseData.TopicData().
setTopicName("").
setTopicId(fooId). setTopicId(fooId).
setPartitions(singletonList(new AlterPartitionResponseData.PartitionData(). setPartitions(singletonList(new AlterPartitionResponseData.PartitionData().
setPartitionIndex(0). setPartitionIndex(0).
@ -2119,8 +2104,7 @@ public class ReplicationControlManagerTest {
.setBrokerId(1) .setBrokerId(1)
.setBrokerEpoch(101) .setBrokerEpoch(101)
.setTopics(singletonList(new TopicData() .setTopics(singletonList(new TopicData()
.setTopicName(version <= 1 ? "foo" : "") .setTopicId(fooId)
.setTopicId(version > 1 ? fooId : Uuid.ZERO_UUID)
.setPartitions(singletonList(new PartitionData() .setPartitions(singletonList(new PartitionData()
.setPartitionIndex(0) .setPartitionIndex(0)
.setPartitionEpoch(0) .setPartitionEpoch(0)
@ -2133,12 +2117,11 @@ public class ReplicationControlManagerTest {
ControllerResult<AlterPartitionResponseData> alterPartitionResult = ControllerResult<AlterPartitionResponseData> alterPartitionResult =
replication.alterPartition(requestContext, new AlterPartitionRequest.Builder(alterIsrRequest).build(version).data()); replication.alterPartition(requestContext, new AlterPartitionRequest.Builder(alterIsrRequest).build(version).data());
Errors expectedError = version <= 1 ? OPERATION_NOT_ATTEMPTED : INELIGIBLE_REPLICA; Errors expectedError = INELIGIBLE_REPLICA;
assertEquals( assertEquals(
new AlterPartitionResponseData() new AlterPartitionResponseData()
.setTopics(singletonList(new AlterPartitionResponseData.TopicData() .setTopics(singletonList(new AlterPartitionResponseData.TopicData()
.setTopicName(version <= 1 ? "foo" : "") .setTopicId(fooId)
.setTopicId(version > 1 ? fooId : Uuid.ZERO_UUID)
.setPartitions(singletonList(new AlterPartitionResponseData.PartitionData() .setPartitions(singletonList(new AlterPartitionResponseData.PartitionData()
.setPartitionIndex(0) .setPartitionIndex(0)
.setErrorCode(expectedError.code()))))), .setErrorCode(expectedError.code()))))),
@ -2863,11 +2846,10 @@ public class ReplicationControlManagerTest {
assertEquals(DirectoryId.UNASSIGNED, describer.defaultDir(3)); assertEquals(DirectoryId.UNASSIGNED, describer.defaultDir(3));
} }
@ParameterizedTest @Test
@EnumSource(value = MetadataVersion.class, names = {"IBP_3_3_IV2", "IBP_3_3_IV3"}) public void testProcessBrokerHeartbeatInControlledShutdown() {
public void testProcessBrokerHeartbeatInControlledShutdown(MetadataVersion metadataVersion) {
ReplicationControlTestContext ctx = new ReplicationControlTestContext.Builder(). ReplicationControlTestContext ctx = new ReplicationControlTestContext.Builder().
setMetadataVersion(metadataVersion). setMetadataVersion(MetadataVersion.MINIMUM_VERSION).
build(); build();
ctx.registerBrokers(0, 1, 2); ctx.registerBrokers(0, 1, 2);
ctx.unfenceBrokers(0, 1, 2); ctx.unfenceBrokers(0, 1, 2);
@ -2885,15 +2867,13 @@ public class ReplicationControlManagerTest {
List<ApiMessageAndVersion> expectedRecords = new ArrayList<>(); List<ApiMessageAndVersion> expectedRecords = new ArrayList<>();
if (metadataVersion.isInControlledShutdownStateSupported()) { expectedRecords.add(new ApiMessageAndVersion(
expectedRecords.add(new ApiMessageAndVersion( new BrokerRegistrationChangeRecord()
new BrokerRegistrationChangeRecord() .setBrokerEpoch(100)
.setBrokerEpoch(100) .setBrokerId(0)
.setBrokerId(0) .setInControlledShutdown(BrokerRegistrationInControlledShutdownChange
.setInControlledShutdown(BrokerRegistrationInControlledShutdownChange .IN_CONTROLLED_SHUTDOWN.value()),
.IN_CONTROLLED_SHUTDOWN.value()), (short) 1));
(short) 1));
}
expectedRecords.add(new ApiMessageAndVersion( expectedRecords.add(new ApiMessageAndVersion(
new PartitionChangeRecord() new PartitionChangeRecord()

View File

@ -36,6 +36,7 @@ import org.apache.kafka.image.loader.SnapshotManifest;
import org.apache.kafka.image.writer.ImageReWriter; import org.apache.kafka.image.writer.ImageReWriter;
import org.apache.kafka.image.writer.ImageWriterOptions; import org.apache.kafka.image.writer.ImageWriterOptions;
import org.apache.kafka.raft.LeaderAndEpoch; import org.apache.kafka.raft.LeaderAndEpoch;
import org.apache.kafka.server.common.MetadataVersion;
import org.apache.kafka.server.fault.MockFaultHandler; import org.apache.kafka.server.fault.MockFaultHandler;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
@ -145,9 +146,7 @@ public class ControllerMetadataMetricsPublisherTest {
try (TestEnv env = new TestEnv()) { try (TestEnv env = new TestEnv()) {
MetadataDelta delta = new MetadataDelta(MetadataImage.EMPTY); MetadataDelta delta = new MetadataDelta(MetadataImage.EMPTY);
ImageReWriter writer = new ImageReWriter(delta); ImageReWriter writer = new ImageReWriter(delta);
IMAGE1.write(writer, new ImageWriterOptions.Builder(). IMAGE1.write(writer, new ImageWriterOptions.Builder(MetadataVersion.MINIMUM_VERSION).build());
setMetadataVersion(delta.image().features().metadataVersion()).
build());
env.publisher.onMetadataUpdate(delta, IMAGE1, fakeManifest(true)); env.publisher.onMetadataUpdate(delta, IMAGE1, fakeManifest(true));
assertEquals(0, env.metrics.activeBrokerCount()); assertEquals(0, env.metrics.activeBrokerCount());
assertEquals(3, env.metrics.globalTopicCount()); assertEquals(3, env.metrics.globalTopicCount());

View File

@ -130,8 +130,7 @@ public class ControllerMetricsChangesTest {
static final TopicDelta TOPIC_DELTA2; static final TopicDelta TOPIC_DELTA2;
static { static {
ImageWriterOptions options = new ImageWriterOptions.Builder(). ImageWriterOptions options = new ImageWriterOptions.Builder(MetadataVersion.IBP_3_7_IV0).build(); // highest MV for PartitionRecord v0
setMetadataVersion(MetadataVersion.IBP_3_7_IV0).build(); // highest MV for PartitionRecord v0
TOPIC_DELTA1 = new TopicDelta(new TopicImage("foo", FOO_ID, Collections.emptyMap())); TOPIC_DELTA1 = new TopicDelta(new TopicImage("foo", FOO_ID, Collections.emptyMap()));
TOPIC_DELTA1.replay((PartitionRecord) fakePartitionRegistration(NORMAL). TOPIC_DELTA1.replay((PartitionRecord) fakePartitionRegistration(NORMAL).
toRecord(FOO_ID, 0, options).message()); toRecord(FOO_ID, 0, options).message());

View File

@ -19,7 +19,6 @@ package org.apache.kafka.image;
import org.apache.kafka.common.Uuid; import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.metadata.RemoveAccessControlEntryRecord; 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.image.writer.RecordListWriter;
import org.apache.kafka.metadata.RecordTestUtils; import org.apache.kafka.metadata.RecordTestUtils;
import org.apache.kafka.metadata.authorizer.StandardAcl; import org.apache.kafka.metadata.authorizer.StandardAcl;
@ -116,7 +115,7 @@ public class AclsImageTest {
private static List<ApiMessageAndVersion> getImageRecords(AclsImage image) { private static List<ApiMessageAndVersion> getImageRecords(AclsImage image) {
RecordListWriter writer = new RecordListWriter(); RecordListWriter writer = new RecordListWriter();
image.write(writer, new ImageWriterOptions.Builder().build()); image.write(writer);
return writer.records(); return writer.records();
} }
} }

View File

@ -24,6 +24,7 @@ import org.apache.kafka.image.writer.ImageWriterOptions;
import org.apache.kafka.image.writer.RecordListWriter; import org.apache.kafka.image.writer.RecordListWriter;
import org.apache.kafka.metadata.RecordTestUtils; import org.apache.kafka.metadata.RecordTestUtils;
import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.server.common.ApiMessageAndVersion;
import org.apache.kafka.server.common.MetadataVersion;
import org.apache.kafka.server.config.QuotaConfig; import org.apache.kafka.server.config.QuotaConfig;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
@ -137,7 +138,7 @@ public class ClientQuotasImageTest {
private static List<ApiMessageAndVersion> getImageRecords(ClientQuotasImage image) { private static List<ApiMessageAndVersion> getImageRecords(ClientQuotasImage image) {
RecordListWriter writer = new RecordListWriter(); RecordListWriter writer = new RecordListWriter();
image.write(writer, new ImageWriterOptions.Builder().build()); image.write(writer, new ImageWriterOptions.Builder(MetadataVersion.latestProduction()).build());
return writer.records(); return writer.records();
} }
} }

View File

@ -199,7 +199,7 @@ public class ClusterImageTest {
setFeatures(new BrokerFeatureCollection( setFeatures(new BrokerFeatureCollection(
Collections.singleton(new BrokerFeature(). Collections.singleton(new BrokerFeature().
setName(MetadataVersion.FEATURE_NAME). setName(MetadataVersion.FEATURE_NAME).
setMinSupportedVersion(MetadataVersion.IBP_3_3_IV3.featureLevel()). setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()).
setMaxSupportedVersion(MetadataVersion.IBP_3_6_IV0.featureLevel())).iterator())). setMaxSupportedVersion(MetadataVersion.IBP_3_6_IV0.featureLevel())).iterator())).
setRack("rack3"), setRack("rack3"),
REGISTER_BROKER_RECORD.highestSupportedVersion())); REGISTER_BROKER_RECORD.highestSupportedVersion()));
@ -232,7 +232,7 @@ public class ClusterImageTest {
setIncarnationId(Uuid.fromString("Am5Yse7GQxaw0b2alM74bP")). setIncarnationId(Uuid.fromString("Am5Yse7GQxaw0b2alM74bP")).
setListeners(Collections.singletonList(new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 9094))). setListeners(Collections.singletonList(new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 9094))).
setSupportedFeatures(Collections.singletonMap("metadata.version", 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")). setRack(Optional.of("rack3")).
setFenced(true). setFenced(true).
setIsMigratingZkBroker(true).build()); setIsMigratingZkBroker(true).build());
@ -296,7 +296,7 @@ public class ClusterImageTest {
private static List<ApiMessageAndVersion> getImageRecords(ClusterImage image) { private static List<ApiMessageAndVersion> getImageRecords(ClusterImage image) {
RecordListWriter writer = new RecordListWriter(); RecordListWriter writer = new RecordListWriter();
image.write(writer, new ImageWriterOptions.Builder().build()); image.write(writer, new ImageWriterOptions.Builder(MetadataVersion.latestProduction()).build());
return writer.records(); return writer.records();
} }
@ -311,8 +311,7 @@ public class ClusterImageTest {
setSupportedFeatures(Collections.emptyMap()).build())); setSupportedFeatures(Collections.emptyMap()).build()));
RecordListWriter writer = new RecordListWriter(); RecordListWriter writer = new RecordListWriter();
final AtomicReference<String> lossString = new AtomicReference<>(""); final AtomicReference<String> lossString = new AtomicReference<>("");
testImage.write(writer, new ImageWriterOptions.Builder(). testImage.write(writer, new ImageWriterOptions.Builder(MetadataVersion.IBP_3_6_IV2).
setMetadataVersion(MetadataVersion.IBP_3_6_IV2).
setLossHandler(loss -> lossString.compareAndSet("", loss.loss())). setLossHandler(loss -> lossString.compareAndSet("", loss.loss())).
build()); build());
assertEquals("controller registration data", lossString.get()); assertEquals("controller registration data", lossString.get());

View File

@ -23,6 +23,7 @@ import org.apache.kafka.image.writer.ImageWriterOptions;
import org.apache.kafka.image.writer.RecordListWriter; import org.apache.kafka.image.writer.RecordListWriter;
import org.apache.kafka.metadata.RecordTestUtils; import org.apache.kafka.metadata.RecordTestUtils;
import org.apache.kafka.server.common.ApiMessageAndVersion; 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.Test;
import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.Timeout;
@ -136,7 +137,7 @@ public class ConfigurationsImageTest {
private static List<ApiMessageAndVersion> getImageRecords(ConfigurationsImage image) { private static List<ApiMessageAndVersion> getImageRecords(ConfigurationsImage image) {
RecordListWriter writer = new RecordListWriter(); RecordListWriter writer = new RecordListWriter();
image.write(writer, new ImageWriterOptions.Builder().build()); image.write(writer, new ImageWriterOptions.Builder(MetadataVersion.latestProduction()).build());
return writer.records(); return writer.records();
} }
} }

View File

@ -39,7 +39,7 @@ import java.util.Map;
import java.util.Optional; import java.util.Optional;
import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.fail; import static org.junit.jupiter.api.Assertions.assertThrows;
@Timeout(value = 40) @Timeout(value = 40)
@ -134,28 +134,22 @@ public class DelegationTokenImageTest {
private static List<ApiMessageAndVersion> getImageRecords(DelegationTokenImage image) { private static List<ApiMessageAndVersion> getImageRecords(DelegationTokenImage image) {
RecordListWriter writer = new RecordListWriter(); RecordListWriter writer = new RecordListWriter();
image.write(writer, new ImageWriterOptions.Builder().build()); image.write(writer, new ImageWriterOptions.Builder(MetadataVersion.latestProduction()).build());
return writer.records(); return writer.records();
} }
@Test @Test
public void testEmptyWithInvalidIBP() { public void testEmptyWithInvalidIBP() {
ImageWriterOptions imageWriterOptions = new ImageWriterOptions.Builder(). ImageWriterOptions imageWriterOptions = new ImageWriterOptions.Builder(MetadataVersion.IBP_3_5_IV2).build();
setMetadataVersion(MetadataVersion.IBP_3_5_IV2).build();
RecordListWriter writer = new RecordListWriter(); RecordListWriter writer = new RecordListWriter();
DelegationTokenImage.EMPTY.write(writer, imageWriterOptions); DelegationTokenImage.EMPTY.write(writer, imageWriterOptions);
} }
@Test @Test
public void testImage1withInvalidIBP() { public void testImage1WithInvalidIBP() {
ImageWriterOptions imageWriterOptions = new ImageWriterOptions.Builder(). ImageWriterOptions imageWriterOptions = new ImageWriterOptions.Builder(MetadataVersion.IBP_3_5_IV2).build();
setMetadataVersion(MetadataVersion.IBP_3_5_IV2).build();
RecordListWriter writer = new RecordListWriter(); RecordListWriter writer = new RecordListWriter();
try { assertThrows(Exception.class, () -> IMAGE1.write(writer, imageWriterOptions),
IMAGE1.write(writer, imageWriterOptions); "expected exception writing IMAGE with Delegation Token records for MetadataVersion.IBP_3_5_IV2");
fail("expected exception writing IMAGE with Delegation Token records for MetadataVersion.IBP_3_5_IV2");
} catch (Exception expected) {
// ignore, expected
}
} }
} }

View File

@ -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());
}
}

View File

@ -100,7 +100,13 @@ public class FeaturesImageTest {
@Test @Test
public void testEmptyImageRoundTrip() { 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 @Test
@ -154,7 +160,7 @@ public class FeaturesImageTest {
private static List<ApiMessageAndVersion> getImageRecords(FeaturesImage image) { private static List<ApiMessageAndVersion> getImageRecords(FeaturesImage image) {
RecordListWriter writer = new RecordListWriter(); 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(); return writer.records();
} }
@ -162,10 +168,9 @@ public class FeaturesImageTest {
public void testEmpty() { public void testEmpty() {
assertTrue(FeaturesImage.EMPTY.isEmpty()); assertTrue(FeaturesImage.EMPTY.isEmpty());
assertFalse(new FeaturesImage(Collections.singletonMap("foo", (short) 1), assertFalse(new FeaturesImage(Collections.singletonMap("foo", (short) 1),
FeaturesImage.EMPTY.metadataVersion()).isEmpty()); MetadataVersion.MINIMUM_VERSION).isEmpty());
assertFalse(new FeaturesImage(FeaturesImage.EMPTY.finalizedVersions(), assertFalse(new FeaturesImage(FeaturesImage.EMPTY.finalizedVersions(),
MetadataVersion.IBP_3_3_IV0).isEmpty()); MetadataVersion.MINIMUM_VERSION).isEmpty());
assertTrue(new FeaturesImage(FeaturesImage.EMPTY.finalizedVersions(), FeaturesImage.EMPTY.metadataVersion()).isEmpty());
} }
@Test @Test

View File

@ -77,59 +77,6 @@ public class ImageDowngradeTest {
setFeatureLevel(metadataVersion.featureLevel()), (short) 0); 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. * Test downgrading to a MetadataVersion that doesn't support ZK migration.
*/ */
@ -200,8 +147,7 @@ public class ImageDowngradeTest {
RecordTestUtils.replayAll(delta, inputs); RecordTestUtils.replayAll(delta, inputs);
MetadataImage image = delta.apply(MetadataProvenance.EMPTY); MetadataImage image = delta.apply(MetadataProvenance.EMPTY);
RecordListWriter writer = new RecordListWriter(); RecordListWriter writer = new RecordListWriter();
image.write(writer, new ImageWriterOptions.Builder(). image.write(writer, new ImageWriterOptions.Builder(metadataVersion).
setMetadataVersion(metadataVersion).
setLossHandler(lossConsumer). setLossHandler(lossConsumer).
build()); build());
assertEquals(expectedLosses, lossConsumer.losses, "Failed to get expected metadata losses."); assertEquals(expectedLosses, lossConsumer.losses, "Failed to get expected metadata losses.");

View File

@ -21,10 +21,12 @@ import org.apache.kafka.image.writer.ImageWriterOptions;
import org.apache.kafka.image.writer.RecordListWriter; import org.apache.kafka.image.writer.RecordListWriter;
import org.apache.kafka.metadata.RecordTestUtils; import org.apache.kafka.metadata.RecordTestUtils;
import org.apache.kafka.server.common.ApiMessageAndVersion; 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.Test;
import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.Timeout;
import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Optional; import java.util.Optional;
@ -80,7 +82,22 @@ public class MetadataImageTest {
@Test @Test
public void testEmptyImageRoundTrip() { 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 @Test
@ -92,9 +109,7 @@ public class MetadataImageTest {
public void testApplyDelta1() { public void testApplyDelta1() {
assertEquals(IMAGE2, DELTA1.apply(IMAGE2.provenance())); assertEquals(IMAGE2, DELTA1.apply(IMAGE2.provenance()));
// check image1 + delta1 = image2, since records for image1 + delta1 might differ from records from image2 // check image1 + delta1 = image2, since records for image1 + delta1 might differ from records from image2
ImageWriterOptions options = new ImageWriterOptions.Builder() ImageWriterOptions options = new ImageWriterOptions.Builder(IMAGE1.features().metadataVersionOrThrow()).build();
.setMetadataVersion(IMAGE1.features().metadataVersion())
.build();
List<ApiMessageAndVersion> records = getImageRecords(IMAGE1, options); List<ApiMessageAndVersion> records = getImageRecords(IMAGE1, options);
records.addAll(FeaturesImageTest.DELTA1_RECORDS); records.addAll(FeaturesImageTest.DELTA1_RECORDS);
records.addAll(ClusterImageTest.DELTA1_RECORDS); records.addAll(ClusterImageTest.DELTA1_RECORDS);
@ -114,9 +129,7 @@ public class MetadataImageTest {
} }
private static void testToImage(MetadataImage image) { private static void testToImage(MetadataImage image) {
testToImage(image, new ImageWriterOptions.Builder() testToImage(image, new ImageWriterOptions.Builder(image.features().metadataVersionOrThrow()).build(), Optional.empty());
.setMetadataVersion(image.features().metadataVersion())
.build(), Optional.empty());
} }
static void testToImage(MetadataImage image, ImageWriterOptions options, Optional<List<ApiMessageAndVersion>> fromRecords) { static void testToImage(MetadataImage image, ImageWriterOptions options, Optional<List<ApiMessageAndVersion>> fromRecords) {

View File

@ -17,11 +17,11 @@
package org.apache.kafka.image; package org.apache.kafka.image;
import org.apache.kafka.server.common.MetadataVersion;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout; 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.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertTrue;
@ -30,31 +30,31 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
@Timeout(value = 40) @Timeout(value = 40)
public class MetadataVersionChangeTest { public class MetadataVersionChangeTest {
private static final MetadataVersionChange CHANGE_3_0_IV1_TO_3_3_IV0 = private static final MetadataVersionChange CHANGE_MINIMUM_TO_LATEST =
new MetadataVersionChange(IBP_3_0_IV1, IBP_3_3_IV0); new MetadataVersionChange(MetadataVersion.MINIMUM_VERSION, MetadataVersion.latestProduction());
private static final MetadataVersionChange CHANGE_3_3_IV0_TO_3_0_IV1 = private static final MetadataVersionChange CHANGE_LATEST_TO_MINIMUM =
new MetadataVersionChange(IBP_3_3_IV0, IBP_3_0_IV1); new MetadataVersionChange(MetadataVersion.latestProduction(), MetadataVersion.MINIMUM_VERSION);
@Test @Test
public void testIsUpgrade() { public void testIsUpgrade() {
assertTrue(CHANGE_3_0_IV1_TO_3_3_IV0.isUpgrade()); assertTrue(CHANGE_MINIMUM_TO_LATEST.isUpgrade());
assertFalse(CHANGE_3_3_IV0_TO_3_0_IV1.isUpgrade()); assertFalse(CHANGE_LATEST_TO_MINIMUM.isUpgrade());
} }
@Test @Test
public void testIsDowngrade() { public void testIsDowngrade() {
assertFalse(CHANGE_3_0_IV1_TO_3_3_IV0.isDowngrade()); assertFalse(CHANGE_MINIMUM_TO_LATEST.isDowngrade());
assertTrue(CHANGE_3_3_IV0_TO_3_0_IV1.isDowngrade()); assertTrue(CHANGE_LATEST_TO_MINIMUM.isDowngrade());
} }
@Test @Test
public void testMetadataVersionChangeExceptionToString() { public void testMetadataVersionChangeExceptionToString() {
assertEquals("org.apache.kafka.image.MetadataVersionChangeException: The metadata.version " + assertEquals("org.apache.kafka.image.MetadataVersionChangeException: The metadata.version " +
"is changing from 3.0-IV1 to 3.3-IV0", "is changing from " + MetadataVersion.MINIMUM_VERSION + " to " + MetadataVersion.latestProduction(),
new MetadataVersionChangeException(CHANGE_3_0_IV1_TO_3_3_IV0).toString()); new MetadataVersionChangeException(CHANGE_MINIMUM_TO_LATEST).toString());
assertEquals("org.apache.kafka.image.MetadataVersionChangeException: The metadata.version " + assertEquals("org.apache.kafka.image.MetadataVersionChangeException: The metadata.version " +
"is changing from 3.3-IV0 to 3.0-IV1", "is changing from " + MetadataVersion.latestProduction() + " to " + MetadataVersion.MINIMUM_VERSION,
new MetadataVersionChangeException(CHANGE_3_3_IV0_TO_3_0_IV1).toString()); new MetadataVersionChangeException(CHANGE_LATEST_TO_MINIMUM).toString());
} }
} }

View File

@ -22,6 +22,7 @@ import org.apache.kafka.image.writer.ImageWriterOptions;
import org.apache.kafka.image.writer.RecordListWriter; import org.apache.kafka.image.writer.RecordListWriter;
import org.apache.kafka.metadata.RecordTestUtils; import org.apache.kafka.metadata.RecordTestUtils;
import org.apache.kafka.server.common.ApiMessageAndVersion; 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.Test;
import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.Timeout;
@ -112,7 +113,7 @@ public class ProducerIdsImageTest {
private static List<ApiMessageAndVersion> getImageRecords(ProducerIdsImage image) { private static List<ApiMessageAndVersion> getImageRecords(ProducerIdsImage image) {
RecordListWriter writer = new RecordListWriter(); RecordListWriter writer = new RecordListWriter();
image.write(writer, new ImageWriterOptions.Builder().build()); image.write(writer, new ImageWriterOptions.Builder(MetadataVersion.latestProduction()).build());
return writer.records(); return writer.records();
} }
} }

View File

@ -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_256;
import static org.apache.kafka.clients.admin.ScramMechanism.SCRAM_SHA_512; 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.assertEquals;
import static org.junit.jupiter.api.Assertions.fail; import static org.junit.jupiter.api.Assertions.assertThrows;
@Timeout(value = 40) @Timeout(value = 40)
@ -170,28 +170,22 @@ public class ScramImageTest {
private static List<ApiMessageAndVersion> getImageRecords(ScramImage image) { private static List<ApiMessageAndVersion> getImageRecords(ScramImage image) {
RecordListWriter writer = new RecordListWriter(); RecordListWriter writer = new RecordListWriter();
image.write(writer, new ImageWriterOptions.Builder().build()); image.write(writer, new ImageWriterOptions.Builder(MetadataVersion.latestProduction()).build());
return writer.records(); return writer.records();
} }
@Test @Test
public void testEmptyWithInvalidIBP() { public void testEmptyWithInvalidIBP() {
ImageWriterOptions imageWriterOptions = new ImageWriterOptions.Builder(). ImageWriterOptions imageWriterOptions = new ImageWriterOptions.Builder(MetadataVersion.IBP_3_4_IV0).build();
setMetadataVersion(MetadataVersion.IBP_3_4_IV0).build();
RecordListWriter writer = new RecordListWriter(); RecordListWriter writer = new RecordListWriter();
ScramImage.EMPTY.write(writer, imageWriterOptions); ScramImage.EMPTY.write(writer, imageWriterOptions);
} }
@Test @Test
public void testImage1withInvalidIBP() { public void testImage1withInvalidIBP() {
ImageWriterOptions imageWriterOptions = new ImageWriterOptions.Builder(). ImageWriterOptions imageWriterOptions = new ImageWriterOptions.Builder(MetadataVersion.IBP_3_4_IV0).build();
setMetadataVersion(MetadataVersion.IBP_3_4_IV0).build();
RecordListWriter writer = new RecordListWriter(); RecordListWriter writer = new RecordListWriter();
try { assertThrows(Exception.class, () -> IMAGE1.write(writer, imageWriterOptions),
IMAGE1.write(writer, imageWriterOptions); "expected exception writing IMAGE with SCRAM records for MetadataVersion.IBP_3_4_IV0");
fail("expected exception writing IMAGE with SCRAM records for MetadataVersion.IBP_3_4_IV0");
} catch (Exception expected) {
// ignore, expected
}
} }
} }

View File

@ -31,6 +31,7 @@ import org.apache.kafka.metadata.LeaderRecoveryState;
import org.apache.kafka.metadata.PartitionRegistration; import org.apache.kafka.metadata.PartitionRegistration;
import org.apache.kafka.metadata.RecordTestUtils; import org.apache.kafka.metadata.RecordTestUtils;
import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.server.common.ApiMessageAndVersion;
import org.apache.kafka.server.common.MetadataVersion;
import org.apache.kafka.server.immutable.ImmutableMap; import org.apache.kafka.server.immutable.ImmutableMap;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
@ -595,7 +596,7 @@ public class TopicsImageTest {
private static List<ApiMessageAndVersion> getImageRecords(TopicsImage image) { private static List<ApiMessageAndVersion> getImageRecords(TopicsImage image) {
RecordListWriter writer = new RecordListWriter(); RecordListWriter writer = new RecordListWriter();
image.write(writer, new ImageWriterOptions.Builder().build()); image.write(writer, new ImageWriterOptions.Builder(MetadataVersion.latestProduction()).build());
return writer.records(); return writer.records();
} }

View File

@ -18,11 +18,9 @@
package org.apache.kafka.image.loader; package org.apache.kafka.image.loader;
import org.apache.kafka.common.Uuid; 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.message.SnapshotHeaderRecord;
import org.apache.kafka.common.metadata.AbortTransactionRecord; import org.apache.kafka.common.metadata.AbortTransactionRecord;
import org.apache.kafka.common.metadata.BeginTransactionRecord; 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.EndTransactionRecord;
import org.apache.kafka.common.metadata.FeatureLevelRecord; import org.apache.kafka.common.metadata.FeatureLevelRecord;
import org.apache.kafka.common.metadata.PartitionRecord; import org.apache.kafka.common.metadata.PartitionRecord;
@ -55,6 +53,7 @@ import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Optional;
import java.util.OptionalLong; import java.util.OptionalLong;
import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
@ -65,9 +64,8 @@ import java.util.stream.Collectors;
import static java.util.Arrays.asList; import static java.util.Arrays.asList;
import static java.util.Collections.singletonList; 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.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.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNotNull;
@ -243,8 +241,7 @@ public class MetadataLoaderTest {
boolean loadSnapshot, boolean loadSnapshot,
boolean sameObject boolean sameObject
) throws Exception { ) throws Exception {
MockFaultHandler faultHandler = MockFaultHandler faultHandler = new MockFaultHandler("testPublisherCannotBeInstalledMoreThanOnce");
new MockFaultHandler("testPublisherCannotBeInstalledMoreThanOnce");
MockPublisher publisher = new MockPublisher(); MockPublisher publisher = new MockPublisher();
try (MetadataLoader loader = new MetadataLoader.Builder(). try (MetadataLoader loader = new MetadataLoader.Builder().
setFaultHandler(faultHandler). setFaultHandler(faultHandler).
@ -254,13 +251,18 @@ public class MetadataLoaderTest {
if (loadSnapshot) { if (loadSnapshot) {
MockSnapshotReader snapshotReader = new MockSnapshotReader( MockSnapshotReader snapshotReader = new MockSnapshotReader(
new MetadataProvenance(200, 100, 4000, true), new MetadataProvenance(200, 100, 4000, true),
singletonList( List.of(
Batch.control( Batch.control(
200, 200,
100, 100,
4000, 4000,
10, 10,
singletonList(new ControlRecord(ControlRecordType.SNAPSHOT_HEADER, new SnapshotHeaderRecord())) 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( singletonList(singletonList(new ApiMessageAndVersion(
new FeatureLevelRecord(). new FeatureLevelRecord().
setName(MetadataVersion.FEATURE_NAME). setName(MetadataVersion.FEATURE_NAME).
setFeatureLevel(IBP_3_3_IV2.featureLevel()), (short) 0)))); setFeatureLevel(MINIMUM_VERSION.featureLevel()), (short) 0))));
assertFalse(snapshotReader.closed); assertFalse(snapshotReader.closed);
loader.handleLoadSnapshot(snapshotReader); loader.handleLoadSnapshot(snapshotReader);
loader.waitForAllEventsToBeHandled(); loader.waitForAllEventsToBeHandled();
@ -319,7 +321,7 @@ public class MetadataLoaderTest {
loader.removeAndClosePublisher(publishers.get(0)).get(); loader.removeAndClosePublisher(publishers.get(0)).get();
} }
assertTrue(publishers.get(0).closed); assertTrue(publishers.get(0).closed);
assertEquals(IBP_3_3_IV2, assertEquals(Optional.of(MINIMUM_VERSION),
publishers.get(0).latestImage.features().metadataVersion()); publishers.get(0).latestImage.features().metadataVersion());
assertTrue(publishers.get(1).closed); assertTrue(publishers.get(1).closed);
assertNull(publishers.get(1).latestImage); assertNull(publishers.get(1).latestImage);
@ -342,6 +344,12 @@ public class MetadataLoaderTest {
setHighWaterMarkAccessor(() -> OptionalLong.of(0L)). setHighWaterMarkAccessor(() -> OptionalLong.of(0L)).
build()) { build()) {
loader.installPublishers(publishers).get(); 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); loadEmptySnapshot(loader, 200);
publishers.get(0).firstPublish.get(10, TimeUnit.SECONDS); publishers.get(0).firstPublish.get(10, TimeUnit.SECONDS);
assertEquals(200L, loader.lastAppliedOffset()); assertEquals(200L, loader.lastAppliedOffset());
@ -349,13 +357,23 @@ public class MetadataLoaderTest {
assertEquals(300L, loader.lastAppliedOffset()); assertEquals(300L, loader.lastAppliedOffset());
assertEquals(new SnapshotManifest(new MetadataProvenance(300, 100, 4000, true), 3000000L), assertEquals(new SnapshotManifest(new MetadataProvenance(300, 100, 4000, true), 3000000L),
publishers.get(0).latestSnapshotManifest); publishers.get(0).latestSnapshotManifest);
assertEquals(MetadataVersion.MINIMUM_KRAFT_VERSION, assertEquals(MINIMUM_VERSION,
loader.metrics().currentMetadataVersion()); loader.metrics().currentMetadataVersion());
} }
assertTrue(publishers.get(0).closed); assertTrue(publishers.get(0).closed);
assertEquals(MetadataVersion.IBP_3_0_IV1, assertEquals(Optional.of(MINIMUM_VERSION), publishers.get(0).latestImage.features().metadataVersion());
publishers.get(0).latestImage.features().metadataVersion()); var latestImage = publishers.get(0).latestImage;
assertTrue(publishers.get(0).latestImage.isEmpty()); 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(); faultHandler.maybeRethrowFirstException();
} }
@ -489,7 +507,7 @@ public class MetadataLoaderTest {
.numBytes(10) .numBytes(10)
.build(), .build(),
publishers.get(0).latestLogDeltaManifest); publishers.get(0).latestLogDeltaManifest);
assertEquals(MetadataVersion.IBP_3_3_IV1, assertEquals(Optional.of(MINIMUM_VERSION),
publishers.get(0).latestImage.features().metadataVersion()); publishers.get(0).latestImage.features().metadataVersion());
faultHandler.maybeRethrowFirstException(); faultHandler.maybeRethrowFirstException();
} }
@ -511,7 +529,7 @@ public class MetadataLoaderTest {
new MetadataProvenance(200, 100, 4000, true), asList( new MetadataProvenance(200, 100, 4000, true), asList(
singletonList(new ApiMessageAndVersion(new FeatureLevelRecord(). singletonList(new ApiMessageAndVersion(new FeatureLevelRecord().
setName(MetadataVersion.FEATURE_NAME). setName(MetadataVersion.FEATURE_NAME).
setFeatureLevel(IBP_3_3_IV1.featureLevel()), (short) 0)), setFeatureLevel(MINIMUM_VERSION.featureLevel()), (short) 0)),
singletonList(new ApiMessageAndVersion(new TopicRecord(). singletonList(new ApiMessageAndVersion(new TopicRecord().
setName("foo"). setName("foo").
setTopicId(Uuid.fromString("Uum7sfhHQP-obSvfywmNUA")), (short) 0)) setTopicId(Uuid.fromString("Uum7sfhHQP-obSvfywmNUA")), (short) 0))
@ -531,8 +549,7 @@ public class MetadataLoaderTest {
for (int i = 0; i < 2; i++) { for (int i = 0; i < 2; i++) {
assertTrue(publishers.get(i).closed); assertTrue(publishers.get(i).closed);
assertTrue(publishers.get(i).closed); assertTrue(publishers.get(i).closed);
assertEquals(IBP_3_3_IV1, assertEquals(Optional.of(MINIMUM_VERSION), publishers.get(i).latestImage.features().metadataVersion());
publishers.get(i).latestImage.features().metadataVersion());
} }
faultHandler.maybeRethrowFirstException(); faultHandler.maybeRethrowFirstException();
} }
@ -581,7 +598,7 @@ public class MetadataLoaderTest {
new MetadataProvenance(offset, 100, 4000, true), asList( new MetadataProvenance(offset, 100, 4000, true), asList(
singletonList(new ApiMessageAndVersion(new FeatureLevelRecord(). singletonList(new ApiMessageAndVersion(new FeatureLevelRecord().
setName(MetadataVersion.FEATURE_NAME). setName(MetadataVersion.FEATURE_NAME).
setFeatureLevel(IBP_3_3_IV1.featureLevel()), (short) 0)), setFeatureLevel(MINIMUM_VERSION.featureLevel()), (short) 0)),
singletonList(new ApiMessageAndVersion(new TopicRecord(). singletonList(new ApiMessageAndVersion(new TopicRecord().
setName("foo"). setName("foo").
setTopicId(Uuid.fromString("Uum7sfhHQP-obSvfywmNUA")), (short) 0)) setTopicId(Uuid.fromString("Uum7sfhHQP-obSvfywmNUA")), (short) 0))
@ -597,7 +614,7 @@ public class MetadataLoaderTest {
new MetadataProvenance(offset, 100, 4000, true), asList( new MetadataProvenance(offset, 100, 4000, true), asList(
singletonList(new ApiMessageAndVersion(new FeatureLevelRecord(). singletonList(new ApiMessageAndVersion(new FeatureLevelRecord().
setName(MetadataVersion.FEATURE_NAME). setName(MetadataVersion.FEATURE_NAME).
setFeatureLevel(IBP_3_3_IV2.featureLevel()), (short) 0)), setFeatureLevel(MetadataVersion.latestProduction().featureLevel()), (short) 0)),
singletonList(new ApiMessageAndVersion(new TopicRecord(). singletonList(new ApiMessageAndVersion(new TopicRecord().
setName("bar"). setName("bar").
setTopicId(Uuid.fromString("VcL2Mw-cT4aL6XV9VujzoQ")), (short) 0)) setTopicId(Uuid.fromString("VcL2Mw-cT4aL6XV9VujzoQ")), (short) 0))
@ -625,13 +642,13 @@ public class MetadataLoaderTest {
loadTestSnapshot(loader, 200); loadTestSnapshot(loader, 200);
assertEquals(200L, loader.lastAppliedOffset()); assertEquals(200L, loader.lastAppliedOffset());
assertEquals(IBP_3_3_IV1.featureLevel(), assertEquals(MINIMUM_VERSION.featureLevel(),
loader.metrics().currentMetadataVersion().featureLevel()); loader.metrics().currentMetadataVersion().featureLevel());
assertFalse(publishers.get(0).latestDelta.image().isEmpty()); assertFalse(publishers.get(0).latestDelta.image().isEmpty());
loadTestSnapshot2(loader, 400); loadTestSnapshot2(loader, 400);
assertEquals(400L, loader.lastAppliedOffset()); assertEquals(400L, loader.lastAppliedOffset());
assertEquals(IBP_3_3_IV2.featureLevel(), assertEquals(MetadataVersion.latestProduction().featureLevel(),
loader.metrics().currentMetadataVersion().featureLevel()); loader.metrics().currentMetadataVersion().featureLevel());
// Make sure the topic in the initial snapshot was overwritten by loading the new snapshot. // 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)). setHighWaterMarkAccessor(() -> OptionalLong.of(0)).
build()) { build()) {
loader.installPublishers(publishers).get(); 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.waitForAllEventsToBeHandled();
loader.handleCommit( loader.handleCommit(
@ -722,6 +745,12 @@ public class MetadataLoaderTest {
setHighWaterMarkAccessor(() -> OptionalLong.of(0)). setHighWaterMarkAccessor(() -> OptionalLong.of(0)).
build()) { build()) {
loader.installPublishers(publishers).get(); 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.waitForAllEventsToBeHandled();
loader.handleCommit( loader.handleCommit(
@ -753,6 +782,12 @@ public class MetadataLoaderTest {
setHighWaterMarkAccessor(() -> OptionalLong.of(0)). setHighWaterMarkAccessor(() -> OptionalLong.of(0)).
build()) { build()) {
loader.installPublishers(publishers).get(); 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.waitForAllEventsToBeHandled();
loader.handleCommit( loader.handleCommit(
@ -809,12 +844,10 @@ public class MetadataLoaderTest {
loader.handleCommit( loader.handleCommit(
MockBatchReader.newSingleBatchReader(0, 1, singletonList( MockBatchReader.newSingleBatchReader(0, 1, singletonList(
// Any record will work here // Any record will work here
new ApiMessageAndVersion(new ConfigRecord() new ApiMessageAndVersion(new FeatureLevelRecord()
.setResourceType(ConfigResource.Type.BROKER.id()) .setName(MetadataVersion.FEATURE_NAME)
.setResourceName("3000") .setFeatureLevel(MINIMUM_VERSION.featureLevel()), (short) 0)))
.setName("foo") );
.setValue("bar"), (short) 0)
)));
firstPublish.get(30, TimeUnit.SECONDS); firstPublish.get(30, TimeUnit.SECONDS);
assertFalse(capturedImages.isEmpty()); assertFalse(capturedImages.isEmpty());

View File

@ -34,7 +34,7 @@ import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference; 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; import static org.junit.jupiter.api.Assertions.assertEquals;
@ -116,7 +116,7 @@ public class MetadataLoaderMetricsTest {
MetricsRegistry registry = new MetricsRegistry(); MetricsRegistry registry = new MetricsRegistry();
try { try {
try (FakeMetadataLoaderMetrics fakeMetrics = new FakeMetadataLoaderMetrics(registry)) { try (FakeMetadataLoaderMetrics fakeMetrics = new FakeMetadataLoaderMetrics(registry)) {
fakeMetrics.metrics.setCurrentMetadataVersion(IBP_3_3_IV2); fakeMetrics.metrics.setCurrentMetadataVersion(MINIMUM_VERSION);
fakeMetrics.metrics.incrementHandleLoadSnapshotCount(); fakeMetrics.metrics.incrementHandleLoadSnapshotCount();
fakeMetrics.metrics.incrementHandleLoadSnapshotCount(); fakeMetrics.metrics.incrementHandleLoadSnapshotCount();
@ -124,7 +124,7 @@ public class MetadataLoaderMetricsTest {
Gauge<Integer> currentMetadataVersion = (Gauge<Integer>) registry Gauge<Integer> currentMetadataVersion = (Gauge<Integer>) registry
.allMetrics() .allMetrics()
.get(metricName("MetadataLoader", "CurrentMetadataVersion")); .get(metricName("MetadataLoader", "CurrentMetadataVersion"));
assertEquals(IBP_3_3_IV2.featureLevel(), assertEquals(MINIMUM_VERSION.featureLevel(),
currentMetadataVersion.value().shortValue()); currentMetadataVersion.value().shortValue());
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")

View File

@ -99,6 +99,9 @@ public class BrokerRegistrationTrackerTest {
setBrokerId(1). setBrokerId(1).
setIncarnationId(INCARNATION_ID). setIncarnationId(INCARNATION_ID).
setLogDirs(Arrays.asList(A, B, C))); setLogDirs(Arrays.asList(A, B, C)));
delta.replay(new FeatureLevelRecord().
setName(MetadataVersion.FEATURE_NAME).
setFeatureLevel(MetadataVersion.MINIMUM_VERSION.featureLevel()));
ctx.onMetadataUpdate(delta); ctx.onMetadataUpdate(delta);
assertEquals(0, ctx.numCalls.get()); assertEquals(0, ctx.numCalls.get());
} }
@ -133,8 +136,11 @@ public class BrokerRegistrationTrackerTest {
setBrokerId(1). setBrokerId(1).
setIncarnationId(INCARNATION_ID). setIncarnationId(INCARNATION_ID).
setLogDirs(Collections.emptyList())); setLogDirs(Collections.emptyList()));
delta.replay(new FeatureLevelRecord().
setName(MetadataVersion.FEATURE_NAME).
setFeatureLevel(MetadataVersion.IBP_3_7_IV1.featureLevel()));
ctx.onMetadataUpdate(delta); 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()); assertEquals(0, ctx.numCalls.get());
delta = ctx.newDelta(); delta = ctx.newDelta();

View File

@ -48,40 +48,22 @@ import static org.junit.jupiter.api.Assertions.assertThrows;
public class ImageWriterOptionsTest { public class ImageWriterOptionsTest {
@Test @Test
public void testDefaultLossHandler() { public void testDefaultLossHandler() {
ImageWriterOptions options = new ImageWriterOptions.Builder().build(); ImageWriterOptions options = new ImageWriterOptions.Builder(MetadataVersion.latestProduction()).build();
assertEquals("stuff", assertThrows(UnwritableMetadataException.class, assertEquals("stuff", assertThrows(UnwritableMetadataException.class,
() -> options.handleLoss("stuff")).loss()); () -> 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 @Test
public void testHandleLoss() { public void testHandleLoss() {
String expectedMessage = "stuff"; String expectedMessage = "stuff";
for (int i = MetadataVersion.MINIMUM_KRAFT_VERSION.ordinal(); for (int i = MetadataVersion.MINIMUM_VERSION.ordinal();
i < MetadataVersion.VERSIONS.length; i < MetadataVersion.VERSIONS.length;
i++) { i++) {
MetadataVersion version = MetadataVersion.VERSIONS[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); String formattedMessage = String.format("Metadata has been lost because the following could not be represented in metadata.version %s: %s", version, expectedMessage);
Consumer<UnwritableMetadataException> customLossHandler = e -> assertEquals(formattedMessage, e.getMessage()); Consumer<UnwritableMetadataException> customLossHandler = e -> assertEquals(formattedMessage, e.getMessage());
ImageWriterOptions options = new ImageWriterOptions.Builder() ImageWriterOptions options = new ImageWriterOptions.Builder(version)
.setMetadataVersion(version)
.setLossHandler(customLossHandler) .setLossHandler(customLossHandler)
.build(); .build();
options.handleLoss(expectedMessage); options.handleLoss(expectedMessage);
@ -90,14 +72,12 @@ public class ImageWriterOptionsTest {
@Test @Test
public void testSetEligibleLeaderReplicasEnabled() { public void testSetEligibleLeaderReplicasEnabled() {
MetadataVersion version = MetadataVersion.MINIMUM_BOOTSTRAP_VERSION; MetadataVersion version = MetadataVersion.MINIMUM_VERSION;
ImageWriterOptions options = new ImageWriterOptions.Builder(). ImageWriterOptions options = new ImageWriterOptions.Builder(version).
setMetadataVersion(version).
setEligibleLeaderReplicasEnabled(true).build(); setEligibleLeaderReplicasEnabled(true).build();
assertEquals(true, options.isEligibleLeaderReplicasEnabled()); assertEquals(true, options.isEligibleLeaderReplicasEnabled());
options = new ImageWriterOptions.Builder(). options = new ImageWriterOptions.Builder(version).build();
setMetadataVersion(version).build();
assertEquals(false, options.isEligibleLeaderReplicasEnabled()); assertEquals(false, options.isEligibleLeaderReplicasEnabled());
} }

View File

@ -25,6 +25,7 @@ import org.apache.kafka.common.metadata.RegisterBrokerRecord;
import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.common.security.auth.SecurityProtocol;
import org.apache.kafka.image.writer.ImageWriterOptions; import org.apache.kafka.image.writer.ImageWriterOptions;
import org.apache.kafka.server.common.ApiMessageAndVersion; 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.Test;
import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.Timeout;
@ -143,7 +144,7 @@ public class BrokerRegistrationTest {
} }
private void testRoundTrip(BrokerRegistration registration) { private void testRoundTrip(BrokerRegistration registration) {
ImageWriterOptions options = new ImageWriterOptions.Builder().build(); ImageWriterOptions options = new ImageWriterOptions.Builder(MetadataVersion.latestProduction()).build();
ApiMessageAndVersion messageAndVersion = registration. ApiMessageAndVersion messageAndVersion = registration.
toRecord(options); toRecord(options);
BrokerRegistration registration2 = BrokerRegistration.fromRecord( BrokerRegistration registration2 = BrokerRegistration.fromRecord(

View File

@ -118,12 +118,12 @@ public class ControllerRegistrationTest {
private void testRoundTrip(ControllerRegistration registration) { private void testRoundTrip(ControllerRegistration registration) {
ApiMessageAndVersion messageAndVersion = registration. ApiMessageAndVersion messageAndVersion = registration.
toRecord(new ImageWriterOptions.Builder().build()); toRecord(new ImageWriterOptions.Builder(MetadataVersion.latestProduction()).build());
ControllerRegistration registration2 = new ControllerRegistration.Builder( ControllerRegistration registration2 = new ControllerRegistration.Builder(
(RegisterControllerRecord) messageAndVersion.message()).build(); (RegisterControllerRecord) messageAndVersion.message()).build();
assertEquals(registration, registration2); assertEquals(registration, registration2);
ApiMessageAndVersion messageAndVersion2 = registration2. ApiMessageAndVersion messageAndVersion2 = registration2.
toRecord(new ImageWriterOptions.Builder().build()); toRecord(new ImageWriterOptions.Builder(MetadataVersion.latestProduction()).build());
assertEquals(messageAndVersion, messageAndVersion2); assertEquals(messageAndVersion, messageAndVersion2);
} }

View File

@ -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(); 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"); Uuid topicId = Uuid.fromString("OGdAI5nxT_m-ds3rJMqPLA");
int partitionId = 4; int partitionId = 4;
ApiMessageAndVersion record = registrationA.toRecord(topicId, partitionId, new ImageWriterOptions.Builder(). ApiMessageAndVersion record = registrationA.toRecord(topicId, partitionId,
setMetadataVersion(MetadataVersion.IBP_3_7_IV0).build()); // highest MV for PartitionRecord v0 new ImageWriterOptions.Builder(MetadataVersion.IBP_3_7_IV0).build()); // highest MV for PartitionRecord v0
PartitionRegistration registrationB = PartitionRegistration registrationB =
new PartitionRegistration((PartitionRecord) record.message()); new PartitionRegistration((PartitionRecord) record.message());
assertEquals(registrationA, registrationB); assertEquals(registrationA, registrationB);
@ -336,8 +336,7 @@ public class PartitionRegistrationTest {
)); ));
} }
List<UnwritableMetadataException> exceptions = new ArrayList<>(); List<UnwritableMetadataException> exceptions = new ArrayList<>();
ImageWriterOptions options = new ImageWriterOptions.Builder(). ImageWriterOptions options = new ImageWriterOptions.Builder(metadataVersion).
setMetadataVersion(metadataVersion).
setEligibleLeaderReplicasEnabled(metadataVersion.isElrSupported()). setEligibleLeaderReplicasEnabled(metadataVersion.isElrSupported()).
setLossHandler(exceptions::add). setLossHandler(exceptions::add).
build(); build();
@ -373,8 +372,7 @@ public class PartitionRegistrationTest {
setDirectories(Arrays.asList(DirectoryId.migratingArray(5))). setDirectories(Arrays.asList(DirectoryId.migratingArray(5))).
setPartitionEpoch(0); setPartitionEpoch(0);
List<UnwritableMetadataException> exceptions = new ArrayList<>(); List<UnwritableMetadataException> exceptions = new ArrayList<>();
ImageWriterOptions options = new ImageWriterOptions.Builder(). ImageWriterOptions options = new ImageWriterOptions.Builder(MetadataVersion.IBP_4_0_IV1).
setMetadataVersion(MetadataVersion.IBP_4_0_IV1).
setLossHandler(exceptions::add). setLossHandler(exceptions::add).
build(); build();
assertEquals(new ApiMessageAndVersion(expectRecord, (short) 2), partitionRegistration.toRecord(topicID, 0, options)); assertEquals(new ApiMessageAndVersion(expectRecord, (short) 2), partitionRegistration.toRecord(topicID, 0, options));

View File

@ -374,7 +374,7 @@ public class RecordTestUtils {
Collections.singletonList( Collections.singletonList(
new RegisterControllerRecord.ControllerFeature(). new RegisterControllerRecord.ControllerFeature().
setName(MetadataVersion.FEATURE_NAME). setName(MetadataVersion.FEATURE_NAME).
setMinSupportedVersion(MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel()). setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()).
setMaxSupportedVersion(MetadataVersion.IBP_3_6_IV1.featureLevel()) setMaxSupportedVersion(MetadataVersion.IBP_3_6_IV1.featureLevel())
).iterator() ).iterator()
)); ));

View File

@ -29,7 +29,6 @@ import org.junit.jupiter.api.Timeout;
import java.io.File; import java.io.File;
import java.util.List; import java.util.List;
import java.util.Optional;
import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertThrows;
@ -74,25 +73,7 @@ public class BootstrapDirectoryTest {
try (BootstrapTestDirectory testDirectory = new BootstrapTestDirectory().createDirectory()) { try (BootstrapTestDirectory testDirectory = new BootstrapTestDirectory().createDirectory()) {
assertEquals(BootstrapMetadata.fromVersion(MetadataVersion.latestProduction(), assertEquals(BootstrapMetadata.fromVersion(MetadataVersion.latestProduction(),
"the default bootstrap"), "the default bootstrap"),
new BootstrapDirectory(testDirectory.path(), Optional.empty()).read()); new BootstrapDirectory(testDirectory.path()).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());
} }
} }
@ -100,13 +81,13 @@ public class BootstrapDirectoryTest {
public void testMissingDirectory() { public void testMissingDirectory() {
assertEquals("No such directory as ./non/existent/directory", assertEquals("No such directory as ./non/existent/directory",
assertThrows(RuntimeException.class, () -> assertThrows(RuntimeException.class, () ->
new BootstrapDirectory("./non/existent/directory", Optional.empty()).read()).getMessage()); new BootstrapDirectory("./non/existent/directory").read()).getMessage());
} }
@Test @Test
public void testReadFromConfigurationFile() throws Exception { public void testReadFromConfigurationFile() throws Exception {
try (BootstrapTestDirectory testDirectory = new BootstrapTestDirectory().createDirectory()) { 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, BootstrapMetadata metadata = BootstrapMetadata.fromRecords(SAMPLE_RECORDS1,
"the binary bootstrap metadata file: " + testDirectory.binaryBootstrapPath()); "the binary bootstrap metadata file: " + testDirectory.binaryBootstrapPath());
directory.writeBinaryFile(metadata); directory.writeBinaryFile(metadata);

View File

@ -20,6 +20,8 @@ package org.apache.kafka.metadata.bootstrap;
import org.apache.kafka.common.metadata.FeatureLevelRecord; import org.apache.kafka.common.metadata.FeatureLevelRecord;
import org.apache.kafka.common.metadata.NoOpRecord; import org.apache.kafka.common.metadata.NoOpRecord;
import org.apache.kafka.server.common.ApiMessageAndVersion; 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.Test;
import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.Timeout;
@ -30,8 +32,7 @@ import java.util.List;
import static java.util.Collections.emptyList; import static java.util.Collections.emptyList;
import static org.apache.kafka.server.common.MetadataVersion.FEATURE_NAME; 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_IV3;
import static org.apache.kafka.server.common.MetadataVersion.IBP_3_3_IV2;
import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertThrows;
@ -41,25 +42,25 @@ public class BootstrapMetadataTest {
static final List<ApiMessageAndVersion> SAMPLE_RECORDS1 = List.of( static final List<ApiMessageAndVersion> SAMPLE_RECORDS1 = List.of(
new ApiMessageAndVersion(new FeatureLevelRecord(). new ApiMessageAndVersion(new FeatureLevelRecord().
setName(FEATURE_NAME). setName(FEATURE_NAME).
setFeatureLevel((short) 7), (short) 0), setFeatureLevel((short) 8), (short) 0),
new ApiMessageAndVersion(new NoOpRecord(), (short) 0), new ApiMessageAndVersion(new NoOpRecord(), (short) 0),
new ApiMessageAndVersion(new FeatureLevelRecord(). new ApiMessageAndVersion(new FeatureLevelRecord().
setName(FEATURE_NAME). setName(FEATURE_NAME).
setFeatureLevel((short) 6), (short) 0)); setFeatureLevel((short) 7), (short) 0));
@Test @Test
public void testFromVersion() { public void testFromVersion() {
assertEquals(new BootstrapMetadata(Collections.singletonList( assertEquals(new BootstrapMetadata(Collections.singletonList(
new ApiMessageAndVersion(new FeatureLevelRecord(). new ApiMessageAndVersion(new FeatureLevelRecord().
setName(FEATURE_NAME). setName(FEATURE_NAME).
setFeatureLevel((short) 6), (short) 0)), setFeatureLevel((short) 7), (short) 0)),
IBP_3_3_IV2, "foo"), IBP_3_3_IV3, "foo"),
BootstrapMetadata.fromVersion(IBP_3_3_IV2, "foo")); BootstrapMetadata.fromVersion(IBP_3_3_IV3, "foo"));
} }
@Test @Test
public void testFromRecordsList() { 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")); BootstrapMetadata.fromRecords(SAMPLE_RECORDS1, "bar"));
} }
@ -126,13 +127,14 @@ public class BootstrapMetadataTest {
static final List<ApiMessageAndVersion> RECORDS_WITH_OLD_METADATA_VERSION = Collections.singletonList( static final List<ApiMessageAndVersion> RECORDS_WITH_OLD_METADATA_VERSION = Collections.singletonList(
new ApiMessageAndVersion(new FeatureLevelRecord(). new ApiMessageAndVersion(new FeatureLevelRecord().
setName(FEATURE_NAME). setName(FEATURE_NAME).
setFeatureLevel(IBP_3_0_IV1.featureLevel()), (short) 0)); setFeatureLevel(MetadataVersionTestUtils.IBP_3_0_IV1_FEATURE_LEVEL), (short) 0));
@Test @Test
public void testFromRecordsListWithOldMetadataVersion() { public void testFromRecordsListWithOldMetadataVersion() {
RuntimeException exception = assertThrows(RuntimeException.class, RuntimeException exception = assertThrows(RuntimeException.class,
() -> BootstrapMetadata.fromRecords(RECORDS_WITH_OLD_METADATA_VERSION, "quux")); () -> BootstrapMetadata.fromRecords(RECORDS_WITH_OLD_METADATA_VERSION, "quux"));
assertEquals("Bootstrap metadata.version before 3.3-IV0 are not supported. Can't load " + assertEquals("No MetadataVersion with feature level 1. Valid feature levels are from " + MetadataVersion.MINIMUM_VERSION.featureLevel()
"metadata from quux", exception.getMessage()); + " to " + MetadataVersion.latestTesting().featureLevel() + ".",
exception.getMessage());
} }
} }

View File

@ -144,7 +144,7 @@ public class FormatterTest {
assertEquals(Optional.of(DEFAULT_CLUSTER_ID.toString()), ensemble.clusterId()); assertEquals(Optional.of(DEFAULT_CLUSTER_ID.toString()), ensemble.clusterId());
assertEquals(new HashSet<>(testEnv.directories), ensemble.logDirProps().keySet()); assertEquals(new HashSet<>(testEnv.directories), ensemble.logDirProps().keySet());
BootstrapMetadata bootstrapMetadata = BootstrapMetadata bootstrapMetadata =
new BootstrapDirectory(testEnv.directory(0), Optional.empty()).read(); new BootstrapDirectory(testEnv.directory(0)).read();
assertEquals(MetadataVersion.latestProduction(), bootstrapMetadata.metadataVersion()); assertEquals(MetadataVersion.latestProduction(), bootstrapMetadata.metadataVersion());
} }
} }
@ -225,7 +225,7 @@ public class FormatterTest {
" with metadata.version " + MetadataVersion.IBP_3_5_IV0 + ".", " with metadata.version " + MetadataVersion.IBP_3_5_IV0 + ".",
formatter1.output().trim()); formatter1.output().trim());
BootstrapMetadata bootstrapMetadata = 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(MetadataVersion.IBP_3_5_IV0, bootstrapMetadata.metadataVersion());
assertEquals(1, bootstrapMetadata.records().size()); assertEquals(1, bootstrapMetadata.records().size());
} }
@ -252,7 +252,7 @@ public class FormatterTest {
" with metadata.version " + MetadataVersion.latestTesting() + ".", " with metadata.version " + MetadataVersion.latestTesting() + ".",
formatter1.output().trim()); formatter1.output().trim());
BootstrapMetadata bootstrapMetadata = BootstrapMetadata bootstrapMetadata =
new BootstrapDirectory(testEnv.directory(0), Optional.empty()).read(); new BootstrapDirectory(testEnv.directory(0)).read();
assertEquals(MetadataVersion.latestTesting(), bootstrapMetadata.metadataVersion()); assertEquals(MetadataVersion.latestTesting(), bootstrapMetadata.metadataVersion());
} }
} }
@ -302,7 +302,7 @@ public class FormatterTest {
" with metadata.version " + MetadataVersion.IBP_3_8_IV0 + ".", " with metadata.version " + MetadataVersion.IBP_3_8_IV0 + ".",
formatter1.output().trim()); formatter1.output().trim());
BootstrapMetadata bootstrapMetadata = BootstrapMetadata bootstrapMetadata =
new BootstrapDirectory(testEnv.directory(0), Optional.empty()).read(); new BootstrapDirectory(testEnv.directory(0)).read();
assertEquals(MetadataVersion.IBP_3_8_IV0, bootstrapMetadata.metadataVersion()); assertEquals(MetadataVersion.IBP_3_8_IV0, bootstrapMetadata.metadataVersion());
List<ApiMessageAndVersion> scramRecords = bootstrapMetadata.records().stream(). List<ApiMessageAndVersion> scramRecords = bootstrapMetadata.records().stream().
filter(r -> r.message() instanceof UserScramCredentialRecord). filter(r -> r.message() instanceof UserScramCredentialRecord).
@ -337,7 +337,7 @@ public class FormatterTest {
formatter1.formatter.setFeatureLevel(TestFeatureVersion.FEATURE_NAME, version); formatter1.formatter.setFeatureLevel(TestFeatureVersion.FEATURE_NAME, version);
formatter1.formatter.run(); formatter1.formatter.run();
BootstrapMetadata bootstrapMetadata = BootstrapMetadata bootstrapMetadata =
new BootstrapDirectory(testEnv.directory(0), Optional.empty()).read(); new BootstrapDirectory(testEnv.directory(0)).read();
List<ApiMessageAndVersion> expected = new ArrayList<>(); List<ApiMessageAndVersion> expected = new ArrayList<>();
expected.add(new ApiMessageAndVersion(new FeatureLevelRecord(). expected.add(new ApiMessageAndVersion(new FeatureLevelRecord().
setName(MetadataVersion.FEATURE_NAME). setName(MetadataVersion.FEATURE_NAME).

View File

@ -22,7 +22,7 @@ import java.util.Map;
public enum EligibleLeaderReplicasVersion implements FeatureVersion { public enum EligibleLeaderReplicasVersion implements FeatureVersion {
// Version 0 is the version disable ELR. // 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). // Version 1 enables the ELR (KIP-966).
ELRV_1(1, MetadataVersion.IBP_4_0_IV1, Collections.emptyMap()); ELRV_1(1, MetadataVersion.IBP_4_0_IV1, Collections.emptyMap());

Some files were not shown because too many files have changed in this diff Show More