mirror of https://github.com/apache/kafka.git
KAFKA-13854 Refactor ApiVersion to MetadataVersion (#12072)
Refactoring ApiVersion to MetadataVersion to support both old IBP versioning and new KRaft versioning (feature flags) for KIP-778. IBP versions are now encoded as enum constants and explicitly prefixed w/ IBP_ instead of KAFKA_, and having a LegacyApiVersion vs DefaultApiVersion was not necessary and replaced with appropriate parsing rules for extracting the correct shortVersions/versions. Co-authored-by: David Arthur <mumrah@gmail.com> Reviewers: Ismael Juma <ismael@juma.me.uk>, David Arthur <mumrah@gmail.com>, dengziming <dengziming1993@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
This commit is contained in:
parent
4a6287e832
commit
8245c9a3d5
|
@ -2301,6 +2301,7 @@ project(':jmh-benchmarks') {
|
|||
// jmh requires jopt 4.x while `core` depends on 5.0, they are not binary compatible
|
||||
exclude group: 'net.sf.jopt-simple', module: 'jopt-simple'
|
||||
}
|
||||
implementation project(':server-common')
|
||||
implementation project(':clients')
|
||||
implementation project(':metadata')
|
||||
implementation project(':streams')
|
||||
|
|
|
@ -301,6 +301,10 @@
|
|||
files="metadata[\\/]src[\\/](generated|generated-test)[\\/].+.java$"/>
|
||||
<suppress checks="BooleanExpressionComplexity"
|
||||
files="(MetadataImage).java"/>
|
||||
<suppress checks="ImportControl"
|
||||
files="ApiVersionsResponse.java"/>
|
||||
<suppress checks="AvoidStarImport"
|
||||
files="MetadataVersionTest.java"/>
|
||||
|
||||
<!-- Storage -->
|
||||
<suppress checks="(CyclomaticComplexity|ParameterNumber)"
|
||||
|
|
|
@ -16,10 +16,12 @@
|
|||
*/
|
||||
package org.apache.kafka.common.requests;
|
||||
|
||||
import org.apache.kafka.clients.NodeApiVersions;
|
||||
import org.apache.kafka.common.feature.Features;
|
||||
import org.apache.kafka.common.feature.FinalizedVersionRange;
|
||||
import org.apache.kafka.common.feature.SupportedVersionRange;
|
||||
import org.apache.kafka.common.message.ApiMessageType;
|
||||
import org.apache.kafka.common.message.ApiMessageType.ListenerType;
|
||||
import org.apache.kafka.common.message.ApiVersionsResponseData;
|
||||
import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersion;
|
||||
import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersionCollection;
|
||||
|
@ -121,6 +123,32 @@ public class ApiVersionsResponse extends AbstractResponse {
|
|||
);
|
||||
}
|
||||
|
||||
public static ApiVersionsResponse createApiVersionsResponse(
|
||||
int throttleTimeMs,
|
||||
RecordVersion minRecordVersion,
|
||||
Features<SupportedVersionRange> latestSupportedFeatures,
|
||||
Features<FinalizedVersionRange> finalizedFeatures,
|
||||
long finalizedFeaturesEpoch,
|
||||
NodeApiVersions controllerApiVersions,
|
||||
ListenerType listenerType
|
||||
) {
|
||||
ApiVersionCollection apiKeys;
|
||||
if (controllerApiVersions != null) {
|
||||
apiKeys = intersectForwardableApis(
|
||||
listenerType, minRecordVersion, controllerApiVersions.allSupportedApiVersions());
|
||||
} else {
|
||||
apiKeys = filterApis(minRecordVersion, listenerType);
|
||||
}
|
||||
|
||||
return createApiVersionsResponse(
|
||||
throttleTimeMs,
|
||||
apiKeys,
|
||||
latestSupportedFeatures,
|
||||
finalizedFeatures,
|
||||
finalizedFeaturesEpoch
|
||||
);
|
||||
}
|
||||
|
||||
public static ApiVersionsResponse createApiVersionsResponse(
|
||||
int throttleTimeMs,
|
||||
ApiVersionCollection apiVersions,
|
||||
|
|
|
@ -17,10 +17,18 @@
|
|||
|
||||
package org.apache.kafka.common.requests;
|
||||
|
||||
import java.util.HashSet;
|
||||
import org.apache.kafka.common.feature.Features;
|
||||
import org.apache.kafka.common.feature.FinalizedVersionRange;
|
||||
import org.apache.kafka.common.feature.SupportedVersionRange;
|
||||
import org.apache.kafka.common.message.ApiMessageType;
|
||||
import org.apache.kafka.common.message.ApiMessageType.ListenerType;
|
||||
import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersion;
|
||||
import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersionCollection;
|
||||
import org.apache.kafka.common.message.ApiVersionsResponseData.FinalizedFeatureKey;
|
||||
import org.apache.kafka.common.message.ApiVersionsResponseData.SupportedFeatureKey;
|
||||
import org.apache.kafka.common.protocol.ApiKeys;
|
||||
import org.apache.kafka.common.record.RecordBatch;
|
||||
import org.apache.kafka.common.record.RecordVersion;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
@ -102,6 +110,92 @@ public class ApiVersionsResponseTest {
|
|||
ApiKeys.JOIN_GROUP.latestVersion(), commonResponse);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldCreateApiResponseOnlyWithKeysSupportedByMagicValue() {
|
||||
ApiVersionsResponse response = ApiVersionsResponse.createApiVersionsResponse(
|
||||
10,
|
||||
RecordVersion.V1,
|
||||
Features.emptySupportedFeatures(),
|
||||
Features.emptyFinalizedFeatures(),
|
||||
ApiVersionsResponse.UNKNOWN_FINALIZED_FEATURES_EPOCH,
|
||||
null,
|
||||
ListenerType.ZK_BROKER
|
||||
);
|
||||
verifyApiKeysForMagic(response, RecordBatch.MAGIC_VALUE_V1);
|
||||
assertEquals(10, response.throttleTimeMs());
|
||||
assertTrue(response.data().supportedFeatures().isEmpty());
|
||||
assertTrue(response.data().finalizedFeatures().isEmpty());
|
||||
assertEquals(ApiVersionsResponse.UNKNOWN_FINALIZED_FEATURES_EPOCH, response.data().finalizedFeaturesEpoch());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldReturnFeatureKeysWhenMagicIsCurrentValueAndThrottleMsIsDefaultThrottle() {
|
||||
ApiVersionsResponse response = ApiVersionsResponse.createApiVersionsResponse(
|
||||
10,
|
||||
RecordVersion.V1,
|
||||
Features.supportedFeatures(
|
||||
Utils.mkMap(Utils.mkEntry("feature", new SupportedVersionRange((short) 1, (short) 4)))),
|
||||
Features.finalizedFeatures(
|
||||
Utils.mkMap(Utils.mkEntry("feature", new FinalizedVersionRange((short) 2, (short) 3)))),
|
||||
10L,
|
||||
null,
|
||||
ListenerType.ZK_BROKER
|
||||
);
|
||||
|
||||
verifyApiKeysForMagic(response, RecordBatch.MAGIC_VALUE_V1);
|
||||
assertEquals(10, response.throttleTimeMs());
|
||||
assertEquals(1, response.data().supportedFeatures().size());
|
||||
SupportedFeatureKey sKey = response.data().supportedFeatures().find("feature");
|
||||
assertNotNull(sKey);
|
||||
assertEquals(1, sKey.minVersion());
|
||||
assertEquals(4, sKey.maxVersion());
|
||||
assertEquals(1, response.data().finalizedFeatures().size());
|
||||
FinalizedFeatureKey fKey = response.data().finalizedFeatures().find("feature");
|
||||
assertNotNull(fKey);
|
||||
assertEquals(2, fKey.minVersionLevel());
|
||||
assertEquals(3, fKey.maxVersionLevel());
|
||||
assertEquals(10, response.data().finalizedFeaturesEpoch());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldReturnAllKeysWhenMagicIsCurrentValueAndThrottleMsIsDefaultThrottle() {
|
||||
ApiVersionsResponse response = ApiVersionsResponse.createApiVersionsResponse(
|
||||
AbstractResponse.DEFAULT_THROTTLE_TIME,
|
||||
RecordVersion.current(),
|
||||
Features.emptySupportedFeatures(),
|
||||
Features.emptyFinalizedFeatures(),
|
||||
ApiVersionsResponse.UNKNOWN_FINALIZED_FEATURES_EPOCH,
|
||||
null,
|
||||
ListenerType.ZK_BROKER
|
||||
);
|
||||
assertEquals(new HashSet<ApiKeys>(ApiKeys.zkBrokerApis()), apiKeysInResponse(response));
|
||||
assertEquals(AbstractResponse.DEFAULT_THROTTLE_TIME, response.throttleTimeMs());
|
||||
assertTrue(response.data().supportedFeatures().isEmpty());
|
||||
assertTrue(response.data().finalizedFeatures().isEmpty());
|
||||
assertEquals(ApiVersionsResponse.UNKNOWN_FINALIZED_FEATURES_EPOCH, response.data().finalizedFeaturesEpoch());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMetadataQuorumApisAreDisabled() {
|
||||
ApiVersionsResponse response = ApiVersionsResponse.createApiVersionsResponse(
|
||||
AbstractResponse.DEFAULT_THROTTLE_TIME,
|
||||
RecordVersion.current(),
|
||||
Features.emptySupportedFeatures(),
|
||||
Features.emptyFinalizedFeatures(),
|
||||
ApiVersionsResponse.UNKNOWN_FINALIZED_FEATURES_EPOCH,
|
||||
null,
|
||||
ListenerType.ZK_BROKER
|
||||
);
|
||||
|
||||
// Ensure that APIs needed for the KRaft mode are not exposed through ApiVersions until we are ready for them
|
||||
HashSet<ApiKeys> exposedApis = apiKeysInResponse(response);
|
||||
assertFalse(exposedApis.contains(ApiKeys.ENVELOPE));
|
||||
assertFalse(exposedApis.contains(ApiKeys.VOTE));
|
||||
assertFalse(exposedApis.contains(ApiKeys.BEGIN_QUORUM_EPOCH));
|
||||
assertFalse(exposedApis.contains(ApiKeys.END_QUORUM_EPOCH));
|
||||
assertFalse(exposedApis.contains(ApiKeys.DESCRIBE_QUORUM));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIntersect() {
|
||||
assertFalse(ApiVersionsResponse.intersect(null, null).isPresent());
|
||||
|
@ -145,4 +239,18 @@ public class ApiVersionsResponseTest {
|
|||
assertEquals(expectedVersionsForForwardableAPI, commonResponse.find(forwardableAPIKey));
|
||||
}
|
||||
|
||||
private void verifyApiKeysForMagic(ApiVersionsResponse response, Byte maxMagic) {
|
||||
for (ApiVersion version : response.data().apiKeys()) {
|
||||
assertTrue(ApiKeys.forId(version.apiKey()).minRequiredInterBrokerMagic <= maxMagic);
|
||||
}
|
||||
}
|
||||
|
||||
private HashSet<ApiKeys> apiKeysInResponse(ApiVersionsResponse apiVersions) {
|
||||
HashSet<ApiKeys> apiKeys = new HashSet<>();
|
||||
for (ApiVersion version : apiVersions.data().apiKeys()) {
|
||||
apiKeys.add(ApiKeys.forId(version.apiKey()));
|
||||
}
|
||||
return apiKeys;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -17,7 +17,6 @@
|
|||
|
||||
package kafka.server.builders;
|
||||
|
||||
import kafka.api.ApiVersion;
|
||||
import kafka.log.CleanerConfig;
|
||||
import kafka.log.LogConfig;
|
||||
import kafka.log.LogManager;
|
||||
|
@ -26,6 +25,7 @@ import kafka.server.LogDirFailureChannel;
|
|||
import kafka.server.metadata.ConfigRepository;
|
||||
import kafka.utils.Scheduler;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
import org.apache.kafka.server.common.MetadataVersion;
|
||||
import scala.collection.JavaConverters;
|
||||
|
||||
import java.io.File;
|
||||
|
@ -46,7 +46,7 @@ public class LogManagerBuilder {
|
|||
private long retentionCheckMs = 1000L;
|
||||
private int maxTransactionTimeoutMs = 15 * 60 * 1000;
|
||||
private int maxPidExpirationMs = 60000;
|
||||
private ApiVersion interBrokerProtocolVersion = ApiVersion.latestVersion();
|
||||
private MetadataVersion interBrokerProtocolVersion = MetadataVersion.latest();
|
||||
private Scheduler scheduler = null;
|
||||
private BrokerTopicStats brokerTopicStats = null;
|
||||
private LogDirFailureChannel logDirFailureChannel = null;
|
||||
|
@ -113,7 +113,7 @@ public class LogManagerBuilder {
|
|||
return this;
|
||||
}
|
||||
|
||||
public LogManagerBuilder setInterBrokerProtocolVersion(ApiVersion interBrokerProtocolVersion) {
|
||||
public LogManagerBuilder setInterBrokerProtocolVersion(MetadataVersion interBrokerProtocolVersion) {
|
||||
this.interBrokerProtocolVersion = interBrokerProtocolVersion;
|
||||
return this;
|
||||
}
|
||||
|
|
|
@ -1,500 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package kafka.api
|
||||
|
||||
import org.apache.kafka.clients.NodeApiVersions
|
||||
import org.apache.kafka.common.config.ConfigDef.Validator
|
||||
import org.apache.kafka.common.config.ConfigException
|
||||
import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
|
||||
import org.apache.kafka.common.message.ApiMessageType.ListenerType
|
||||
import org.apache.kafka.common.record.RecordVersion
|
||||
import org.apache.kafka.common.requests.ApiVersionsResponse
|
||||
|
||||
/**
|
||||
* This class contains the different Kafka versions.
|
||||
* Right now, we use them for upgrades - users can configure the version of the API brokers will use to communicate between themselves.
|
||||
* This is only for inter-broker communications - when communicating with clients, the client decides on the API version.
|
||||
*
|
||||
* Note that the ID we initialize for each version is important.
|
||||
* We consider a version newer than another, if it has a higher ID (to avoid depending on lexicographic order)
|
||||
*
|
||||
* Since the api protocol may change more than once within the same release and to facilitate people deploying code from
|
||||
* trunk, we have the concept of internal versions (first introduced during the 0.10.0 development cycle). For example,
|
||||
* the first time we introduce a version change in a release, say 0.10.0, we will add a config value "0.10.0-IV0" and a
|
||||
* corresponding case object KAFKA_0_10_0-IV0. We will also add a config value "0.10.0" that will be mapped to the
|
||||
* latest internal version object, which is KAFKA_0_10_0-IV0. When we change the protocol a second time while developing
|
||||
* 0.10.0, we will add a new config value "0.10.0-IV1" and a corresponding case object KAFKA_0_10_0-IV1. We will change
|
||||
* the config value "0.10.0" to map to the latest internal version object KAFKA_0_10_0-IV1. The config value of
|
||||
* "0.10.0-IV0" is still mapped to KAFKA_0_10_0-IV0. This way, if people are deploying from trunk, they can use
|
||||
* "0.10.0-IV0" and "0.10.0-IV1" to upgrade one internal version at a time. For most people who just want to use
|
||||
* released version, they can use "0.10.0" when upgrading to the 0.10.0 release.
|
||||
*/
|
||||
object ApiVersion {
|
||||
// This implicit is necessary due to: https://issues.scala-lang.org/browse/SI-8541
|
||||
implicit def orderingByVersion[A <: ApiVersion]: Ordering[A] = Ordering.by(_.id)
|
||||
|
||||
val allVersions: Seq[ApiVersion] = Seq(
|
||||
KAFKA_0_8_0,
|
||||
KAFKA_0_8_1,
|
||||
KAFKA_0_8_2,
|
||||
KAFKA_0_9_0,
|
||||
// 0.10.0-IV0 is introduced for KIP-31/32 which changes the message format.
|
||||
KAFKA_0_10_0_IV0,
|
||||
// 0.10.0-IV1 is introduced for KIP-36(rack awareness) and KIP-43(SASL handshake).
|
||||
KAFKA_0_10_0_IV1,
|
||||
// introduced for JoinGroup protocol change in KIP-62
|
||||
KAFKA_0_10_1_IV0,
|
||||
// 0.10.1-IV1 is introduced for KIP-74(fetch response size limit).
|
||||
KAFKA_0_10_1_IV1,
|
||||
// introduced ListOffsetRequest v1 in KIP-79
|
||||
KAFKA_0_10_1_IV2,
|
||||
// introduced UpdateMetadataRequest v3 in KIP-103
|
||||
KAFKA_0_10_2_IV0,
|
||||
// KIP-98 (idempotent and transactional producer support)
|
||||
KAFKA_0_11_0_IV0,
|
||||
// introduced DeleteRecordsRequest v0 and FetchRequest v4 in KIP-107
|
||||
KAFKA_0_11_0_IV1,
|
||||
// Introduced leader epoch fetches to the replica fetcher via KIP-101
|
||||
KAFKA_0_11_0_IV2,
|
||||
// Introduced LeaderAndIsrRequest V1, UpdateMetadataRequest V4 and FetchRequest V6 via KIP-112
|
||||
KAFKA_1_0_IV0,
|
||||
// Introduced DeleteGroupsRequest V0 via KIP-229, plus KIP-227 incremental fetch requests,
|
||||
// and KafkaStorageException for fetch requests.
|
||||
KAFKA_1_1_IV0,
|
||||
// Introduced OffsetsForLeaderEpochRequest V1 via KIP-279 (Fix log divergence between leader and follower after fast leader fail over)
|
||||
KAFKA_2_0_IV0,
|
||||
// Several request versions were bumped due to KIP-219 (Improve quota communication)
|
||||
KAFKA_2_0_IV1,
|
||||
// Introduced new schemas for group offset (v2) and group metadata (v2) (KIP-211)
|
||||
KAFKA_2_1_IV0,
|
||||
// New Fetch, OffsetsForLeaderEpoch, and ListOffsets schemas (KIP-320)
|
||||
KAFKA_2_1_IV1,
|
||||
// Support ZStandard Compression Codec (KIP-110)
|
||||
KAFKA_2_1_IV2,
|
||||
// Introduced broker generation (KIP-380), and
|
||||
// LeaderAdnIsrRequest V2, UpdateMetadataRequest V5, StopReplicaRequest V1
|
||||
KAFKA_2_2_IV0,
|
||||
// New error code for ListOffsets when a new leader is lagging behind former HW (KIP-207)
|
||||
KAFKA_2_2_IV1,
|
||||
// Introduced static membership.
|
||||
KAFKA_2_3_IV0,
|
||||
// Add rack_id to FetchRequest, preferred_read_replica to FetchResponse, and replica_id to OffsetsForLeaderRequest
|
||||
KAFKA_2_3_IV1,
|
||||
// Add adding_replicas and removing_replicas fields to LeaderAndIsrRequest
|
||||
KAFKA_2_4_IV0,
|
||||
// Flexible version support in inter-broker APIs
|
||||
KAFKA_2_4_IV1,
|
||||
// No new APIs, equivalent to 2.4-IV1
|
||||
KAFKA_2_5_IV0,
|
||||
// Introduced StopReplicaRequest V3 containing the leader epoch for each partition (KIP-570)
|
||||
KAFKA_2_6_IV0,
|
||||
// Introduced feature versioning support (KIP-584)
|
||||
KAFKA_2_7_IV0,
|
||||
// Bup Fetch protocol for Raft protocol (KIP-595)
|
||||
KAFKA_2_7_IV1,
|
||||
// Introduced AlterPartition (KIP-497)
|
||||
KAFKA_2_7_IV2,
|
||||
// Flexible versioning on ListOffsets, WriteTxnMarkers and OffsetsForLeaderEpoch. Also adds topic IDs (KIP-516)
|
||||
KAFKA_2_8_IV0,
|
||||
// Introduced topic IDs to LeaderAndIsr and UpdateMetadata requests/responses (KIP-516)
|
||||
KAFKA_2_8_IV1,
|
||||
// Introduce AllocateProducerIds (KIP-730)
|
||||
KAFKA_3_0_IV0,
|
||||
// Introduce ListOffsets V7 which supports listing offsets by max timestamp (KIP-734)
|
||||
// Assume message format version is 3.0 (KIP-724)
|
||||
KAFKA_3_0_IV1,
|
||||
// Adds topic IDs to Fetch requests/responses (KIP-516)
|
||||
KAFKA_3_1_IV0,
|
||||
// Support for leader recovery for unclean leader election (KIP-704)
|
||||
KAFKA_3_2_IV0
|
||||
)
|
||||
|
||||
// Map keys are the union of the short and full versions
|
||||
private val versionMap: Map[String, ApiVersion] =
|
||||
allVersions.map(v => v.version -> v).toMap ++ allVersions.groupBy(_.shortVersion).map { case (k, v) => k -> v.last }
|
||||
|
||||
/**
|
||||
* Return an `ApiVersion` instance for `versionString`, which can be in a variety of formats (e.g. "0.8.0", "0.8.0.x",
|
||||
* "0.10.0", "0.10.0-IV1"). `IllegalArgumentException` is thrown if `versionString` cannot be mapped to an `ApiVersion`.
|
||||
*/
|
||||
def apply(versionString: String): ApiVersion = {
|
||||
val versionSegments = versionString.split('.').toSeq
|
||||
val numSegments = if (versionString.startsWith("0.")) 3 else 2
|
||||
val key = versionSegments.take(numSegments).mkString(".")
|
||||
versionMap.getOrElse(key, throw new IllegalArgumentException(s"Version `$versionString` is not a valid version"))
|
||||
}
|
||||
|
||||
val latestVersion: ApiVersion = allVersions.last
|
||||
|
||||
def isTruncationOnFetchSupported(version: ApiVersion): Boolean = version >= KAFKA_2_7_IV1
|
||||
|
||||
/**
|
||||
* Return the minimum `ApiVersion` that supports `RecordVersion`.
|
||||
*/
|
||||
def minSupportedFor(recordVersion: RecordVersion): ApiVersion = {
|
||||
recordVersion match {
|
||||
case RecordVersion.V0 => KAFKA_0_8_0
|
||||
case RecordVersion.V1 => KAFKA_0_10_0_IV0
|
||||
case RecordVersion.V2 => KAFKA_0_11_0_IV0
|
||||
case _ => throw new IllegalArgumentException(s"Invalid message format version $recordVersion")
|
||||
}
|
||||
}
|
||||
|
||||
def apiVersionsResponse(
|
||||
throttleTimeMs: Int,
|
||||
minRecordVersion: RecordVersion,
|
||||
latestSupportedFeatures: Features[SupportedVersionRange],
|
||||
controllerApiVersions: Option[NodeApiVersions],
|
||||
listenerType: ListenerType
|
||||
): ApiVersionsResponse = {
|
||||
apiVersionsResponse(
|
||||
throttleTimeMs,
|
||||
minRecordVersion,
|
||||
latestSupportedFeatures,
|
||||
Features.emptyFinalizedFeatures,
|
||||
ApiVersionsResponse.UNKNOWN_FINALIZED_FEATURES_EPOCH,
|
||||
controllerApiVersions,
|
||||
listenerType
|
||||
)
|
||||
}
|
||||
|
||||
def apiVersionsResponse(
|
||||
throttleTimeMs: Int,
|
||||
minRecordVersion: RecordVersion,
|
||||
latestSupportedFeatures: Features[SupportedVersionRange],
|
||||
finalizedFeatures: Features[FinalizedVersionRange],
|
||||
finalizedFeaturesEpoch: Long,
|
||||
controllerApiVersions: Option[NodeApiVersions],
|
||||
listenerType: ListenerType
|
||||
): ApiVersionsResponse = {
|
||||
val apiKeys = controllerApiVersions match {
|
||||
case None => ApiVersionsResponse.filterApis(minRecordVersion, listenerType)
|
||||
case Some(controllerApiVersion) => ApiVersionsResponse.intersectForwardableApis(
|
||||
listenerType, minRecordVersion, controllerApiVersion.allSupportedApiVersions())
|
||||
}
|
||||
|
||||
ApiVersionsResponse.createApiVersionsResponse(
|
||||
throttleTimeMs,
|
||||
apiKeys,
|
||||
latestSupportedFeatures,
|
||||
finalizedFeatures,
|
||||
finalizedFeaturesEpoch
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
sealed trait ApiVersion extends Ordered[ApiVersion] {
|
||||
def version: String
|
||||
def shortVersion: String
|
||||
def recordVersion: RecordVersion
|
||||
def id: Int
|
||||
|
||||
def isAlterIsrSupported: Boolean = this >= KAFKA_2_7_IV2
|
||||
|
||||
def isAllocateProducerIdsSupported: Boolean = this >= KAFKA_3_0_IV0
|
||||
|
||||
override def compare(that: ApiVersion): Int =
|
||||
ApiVersion.orderingByVersion.compare(this, that)
|
||||
|
||||
override def toString: String = version
|
||||
}
|
||||
|
||||
/**
|
||||
* For versions before 0.10.0, `version` and `shortVersion` were the same.
|
||||
*/
|
||||
sealed trait LegacyApiVersion extends ApiVersion {
|
||||
def version = shortVersion
|
||||
}
|
||||
|
||||
/**
|
||||
* From 0.10.0 onwards, each version has a sub-version. For example, IV0 is the sub-version of 0.10.0-IV0.
|
||||
*/
|
||||
sealed trait DefaultApiVersion extends ApiVersion {
|
||||
lazy val version = shortVersion + "-" + subVersion
|
||||
protected def subVersion: String
|
||||
}
|
||||
|
||||
// Keep the IDs in order of versions
|
||||
case object KAFKA_0_8_0 extends LegacyApiVersion {
|
||||
val shortVersion = "0.8.0"
|
||||
val recordVersion = RecordVersion.V0
|
||||
val id: Int = 0
|
||||
}
|
||||
|
||||
case object KAFKA_0_8_1 extends LegacyApiVersion {
|
||||
val shortVersion = "0.8.1"
|
||||
val recordVersion = RecordVersion.V0
|
||||
val id: Int = 1
|
||||
}
|
||||
|
||||
case object KAFKA_0_8_2 extends LegacyApiVersion {
|
||||
val shortVersion = "0.8.2"
|
||||
val recordVersion = RecordVersion.V0
|
||||
val id: Int = 2
|
||||
}
|
||||
|
||||
case object KAFKA_0_9_0 extends LegacyApiVersion {
|
||||
val shortVersion = "0.9.0"
|
||||
val subVersion = ""
|
||||
val recordVersion = RecordVersion.V0
|
||||
val id: Int = 3
|
||||
}
|
||||
|
||||
case object KAFKA_0_10_0_IV0 extends DefaultApiVersion {
|
||||
val shortVersion = "0.10.0"
|
||||
val subVersion = "IV0"
|
||||
val recordVersion = RecordVersion.V1
|
||||
val id: Int = 4
|
||||
}
|
||||
|
||||
case object KAFKA_0_10_0_IV1 extends DefaultApiVersion {
|
||||
val shortVersion = "0.10.0"
|
||||
val subVersion = "IV1"
|
||||
val recordVersion = RecordVersion.V1
|
||||
val id: Int = 5
|
||||
}
|
||||
|
||||
case object KAFKA_0_10_1_IV0 extends DefaultApiVersion {
|
||||
val shortVersion = "0.10.1"
|
||||
val subVersion = "IV0"
|
||||
val recordVersion = RecordVersion.V1
|
||||
val id: Int = 6
|
||||
}
|
||||
|
||||
case object KAFKA_0_10_1_IV1 extends DefaultApiVersion {
|
||||
val shortVersion = "0.10.1"
|
||||
val subVersion = "IV1"
|
||||
val recordVersion = RecordVersion.V1
|
||||
val id: Int = 7
|
||||
}
|
||||
|
||||
case object KAFKA_0_10_1_IV2 extends DefaultApiVersion {
|
||||
val shortVersion = "0.10.1"
|
||||
val subVersion = "IV2"
|
||||
val recordVersion = RecordVersion.V1
|
||||
val id: Int = 8
|
||||
}
|
||||
|
||||
case object KAFKA_0_10_2_IV0 extends DefaultApiVersion {
|
||||
val shortVersion = "0.10.2"
|
||||
val subVersion = "IV0"
|
||||
val recordVersion = RecordVersion.V1
|
||||
val id: Int = 9
|
||||
}
|
||||
|
||||
case object KAFKA_0_11_0_IV0 extends DefaultApiVersion {
|
||||
val shortVersion = "0.11.0"
|
||||
val subVersion = "IV0"
|
||||
val recordVersion = RecordVersion.V2
|
||||
val id: Int = 10
|
||||
}
|
||||
|
||||
case object KAFKA_0_11_0_IV1 extends DefaultApiVersion {
|
||||
val shortVersion = "0.11.0"
|
||||
val subVersion = "IV1"
|
||||
val recordVersion = RecordVersion.V2
|
||||
val id: Int = 11
|
||||
}
|
||||
|
||||
case object KAFKA_0_11_0_IV2 extends DefaultApiVersion {
|
||||
val shortVersion = "0.11.0"
|
||||
val subVersion = "IV2"
|
||||
val recordVersion = RecordVersion.V2
|
||||
val id: Int = 12
|
||||
}
|
||||
|
||||
case object KAFKA_1_0_IV0 extends DefaultApiVersion {
|
||||
val shortVersion = "1.0"
|
||||
val subVersion = "IV0"
|
||||
val recordVersion = RecordVersion.V2
|
||||
val id: Int = 13
|
||||
}
|
||||
|
||||
case object KAFKA_1_1_IV0 extends DefaultApiVersion {
|
||||
val shortVersion = "1.1"
|
||||
val subVersion = "IV0"
|
||||
val recordVersion = RecordVersion.V2
|
||||
val id: Int = 14
|
||||
}
|
||||
|
||||
case object KAFKA_2_0_IV0 extends DefaultApiVersion {
|
||||
val shortVersion: String = "2.0"
|
||||
val subVersion = "IV0"
|
||||
val recordVersion = RecordVersion.V2
|
||||
val id: Int = 15
|
||||
}
|
||||
|
||||
case object KAFKA_2_0_IV1 extends DefaultApiVersion {
|
||||
val shortVersion: String = "2.0"
|
||||
val subVersion = "IV1"
|
||||
val recordVersion = RecordVersion.V2
|
||||
val id: Int = 16
|
||||
}
|
||||
|
||||
case object KAFKA_2_1_IV0 extends DefaultApiVersion {
|
||||
val shortVersion: String = "2.1"
|
||||
val subVersion = "IV0"
|
||||
val recordVersion = RecordVersion.V2
|
||||
val id: Int = 17
|
||||
}
|
||||
|
||||
case object KAFKA_2_1_IV1 extends DefaultApiVersion {
|
||||
val shortVersion: String = "2.1"
|
||||
val subVersion = "IV1"
|
||||
val recordVersion = RecordVersion.V2
|
||||
val id: Int = 18
|
||||
}
|
||||
|
||||
case object KAFKA_2_1_IV2 extends DefaultApiVersion {
|
||||
val shortVersion: String = "2.1"
|
||||
val subVersion = "IV2"
|
||||
val recordVersion = RecordVersion.V2
|
||||
val id: Int = 19
|
||||
}
|
||||
|
||||
case object KAFKA_2_2_IV0 extends DefaultApiVersion {
|
||||
val shortVersion: String = "2.2"
|
||||
val subVersion = "IV0"
|
||||
val recordVersion = RecordVersion.V2
|
||||
val id: Int = 20
|
||||
}
|
||||
|
||||
case object KAFKA_2_2_IV1 extends DefaultApiVersion {
|
||||
val shortVersion: String = "2.2"
|
||||
val subVersion = "IV1"
|
||||
val recordVersion = RecordVersion.V2
|
||||
val id: Int = 21
|
||||
}
|
||||
|
||||
case object KAFKA_2_3_IV0 extends DefaultApiVersion {
|
||||
val shortVersion: String = "2.3"
|
||||
val subVersion = "IV0"
|
||||
val recordVersion = RecordVersion.V2
|
||||
val id: Int = 22
|
||||
}
|
||||
|
||||
case object KAFKA_2_3_IV1 extends DefaultApiVersion {
|
||||
val shortVersion: String = "2.3"
|
||||
val subVersion = "IV1"
|
||||
val recordVersion = RecordVersion.V2
|
||||
val id: Int = 23
|
||||
}
|
||||
|
||||
case object KAFKA_2_4_IV0 extends DefaultApiVersion {
|
||||
val shortVersion: String = "2.4"
|
||||
val subVersion = "IV0"
|
||||
val recordVersion = RecordVersion.V2
|
||||
val id: Int = 24
|
||||
}
|
||||
|
||||
case object KAFKA_2_4_IV1 extends DefaultApiVersion {
|
||||
val shortVersion: String = "2.4"
|
||||
val subVersion = "IV1"
|
||||
val recordVersion = RecordVersion.V2
|
||||
val id: Int = 25
|
||||
}
|
||||
|
||||
case object KAFKA_2_5_IV0 extends DefaultApiVersion {
|
||||
val shortVersion: String = "2.5"
|
||||
val subVersion = "IV0"
|
||||
val recordVersion = RecordVersion.V2
|
||||
val id: Int = 26
|
||||
}
|
||||
|
||||
case object KAFKA_2_6_IV0 extends DefaultApiVersion {
|
||||
val shortVersion: String = "2.6"
|
||||
val subVersion = "IV0"
|
||||
val recordVersion = RecordVersion.V2
|
||||
val id: Int = 27
|
||||
}
|
||||
|
||||
case object KAFKA_2_7_IV0 extends DefaultApiVersion {
|
||||
val shortVersion: String = "2.7"
|
||||
val subVersion = "IV0"
|
||||
val recordVersion = RecordVersion.V2
|
||||
val id: Int = 28
|
||||
}
|
||||
|
||||
case object KAFKA_2_7_IV1 extends DefaultApiVersion {
|
||||
val shortVersion: String = "2.7"
|
||||
val subVersion = "IV1"
|
||||
val recordVersion = RecordVersion.V2
|
||||
val id: Int = 29
|
||||
}
|
||||
|
||||
case object KAFKA_2_7_IV2 extends DefaultApiVersion {
|
||||
val shortVersion: String = "2.7"
|
||||
val subVersion = "IV2"
|
||||
val recordVersion = RecordVersion.V2
|
||||
val id: Int = 30
|
||||
}
|
||||
|
||||
case object KAFKA_2_8_IV0 extends DefaultApiVersion {
|
||||
val shortVersion: String = "2.8"
|
||||
val subVersion = "IV0"
|
||||
val recordVersion = RecordVersion.V2
|
||||
val id: Int = 31
|
||||
}
|
||||
|
||||
case object KAFKA_2_8_IV1 extends DefaultApiVersion {
|
||||
val shortVersion: String = "2.8"
|
||||
val subVersion = "IV1"
|
||||
val recordVersion = RecordVersion.V2
|
||||
val id: Int = 32
|
||||
}
|
||||
|
||||
case object KAFKA_3_0_IV0 extends DefaultApiVersion {
|
||||
val shortVersion: String = "3.0"
|
||||
val subVersion = "IV0"
|
||||
val recordVersion = RecordVersion.V2
|
||||
val id: Int = 33
|
||||
}
|
||||
|
||||
case object KAFKA_3_0_IV1 extends DefaultApiVersion {
|
||||
val shortVersion: String = "3.0"
|
||||
val subVersion = "IV1"
|
||||
val recordVersion = RecordVersion.V2
|
||||
val id: Int = 34
|
||||
}
|
||||
|
||||
case object KAFKA_3_1_IV0 extends DefaultApiVersion {
|
||||
val shortVersion: String = "3.1"
|
||||
val subVersion = "IV0"
|
||||
val recordVersion = RecordVersion.V2
|
||||
val id: Int = 35
|
||||
}
|
||||
|
||||
case object KAFKA_3_2_IV0 extends DefaultApiVersion {
|
||||
val shortVersion: String = "3.2"
|
||||
val subVersion = "IV0"
|
||||
val recordVersion = RecordVersion.V2
|
||||
val id: Int = 36
|
||||
}
|
||||
|
||||
object ApiVersionValidator extends Validator {
|
||||
|
||||
override def ensureValid(name: String, value: Any): Unit = {
|
||||
try {
|
||||
ApiVersion(value.toString)
|
||||
} catch {
|
||||
case e: IllegalArgumentException => throw new ConfigException(name, value.toString, e.getMessage)
|
||||
}
|
||||
}
|
||||
|
||||
override def toString: String = "[" + ApiVersion.allVersions.map(_.version).distinct.mkString(", ") + "]"
|
||||
}
|
|
@ -20,7 +20,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock
|
|||
import java.util.Optional
|
||||
import java.util.concurrent.CompletableFuture
|
||||
|
||||
import kafka.api.{ApiVersion, LeaderAndIsr}
|
||||
import kafka.api.LeaderAndIsr
|
||||
import kafka.common.UnexpectedAppendOffsetException
|
||||
import kafka.controller.{KafkaController, StateChangeLogger}
|
||||
import kafka.log._
|
||||
|
@ -42,6 +42,7 @@ import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.{UNDEFINED
|
|||
import org.apache.kafka.common.utils.Time
|
||||
import org.apache.kafka.common.{IsolationLevel, TopicPartition, Uuid}
|
||||
import org.apache.kafka.metadata.LeaderRecoveryState
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
|
||||
import scala.collection.{Map, Seq}
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
@ -232,7 +233,7 @@ case class CommittedPartitionState(
|
|||
*/
|
||||
class Partition(val topicPartition: TopicPartition,
|
||||
val replicaLagTimeMaxMs: Long,
|
||||
interBrokerProtocolVersion: ApiVersion,
|
||||
interBrokerProtocolVersion: MetadataVersion,
|
||||
localBrokerId: Int,
|
||||
time: Time,
|
||||
alterPartitionListener: AlterPartitionListener,
|
||||
|
|
|
@ -38,6 +38,7 @@ import org.apache.kafka.common.security.JaasContext
|
|||
import org.apache.kafka.common.security.auth.SecurityProtocol
|
||||
import org.apache.kafka.common.utils.{LogContext, Time}
|
||||
import org.apache.kafka.common.{KafkaException, Node, Reconfigurable, TopicPartition, Uuid}
|
||||
import org.apache.kafka.server.common.MetadataVersion._
|
||||
|
||||
import scala.jdk.CollectionConverters._
|
||||
import scala.collection.mutable.HashMap
|
||||
|
@ -390,7 +391,7 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig,
|
|||
.setRemovingReplicas(replicaAssignment.removingReplicas.map(Integer.valueOf).asJava)
|
||||
.setIsNew(isNew || alreadyNew)
|
||||
|
||||
if (config.interBrokerProtocolVersion >= KAFKA_3_2_IV0) {
|
||||
if (config.interBrokerProtocolVersion.isAtLeast(IBP_3_2_IV0)) {
|
||||
partitionState.setLeaderRecoveryState(leaderAndIsr.leaderRecoveryState.value)
|
||||
}
|
||||
|
||||
|
@ -460,12 +461,12 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig,
|
|||
|
||||
private def sendLeaderAndIsrRequest(controllerEpoch: Int, stateChangeLog: StateChangeLogger): Unit = {
|
||||
val leaderAndIsrRequestVersion: Short =
|
||||
if (config.interBrokerProtocolVersion >= KAFKA_3_2_IV0) 6
|
||||
else if (config.interBrokerProtocolVersion >= KAFKA_2_8_IV1) 5
|
||||
else if (config.interBrokerProtocolVersion >= KAFKA_2_4_IV1) 4
|
||||
else if (config.interBrokerProtocolVersion >= KAFKA_2_4_IV0) 3
|
||||
else if (config.interBrokerProtocolVersion >= KAFKA_2_2_IV0) 2
|
||||
else if (config.interBrokerProtocolVersion >= KAFKA_1_0_IV0) 1
|
||||
if (config.interBrokerProtocolVersion.isAtLeast(IBP_3_2_IV0)) 6
|
||||
else if (config.interBrokerProtocolVersion.isAtLeast(IBP_2_8_IV1)) 5
|
||||
else if (config.interBrokerProtocolVersion.isAtLeast(IBP_2_4_IV1)) 4
|
||||
else if (config.interBrokerProtocolVersion.isAtLeast(IBP_2_4_IV0)) 3
|
||||
else if (config.interBrokerProtocolVersion.isAtLeast(IBP_2_2_IV0)) 2
|
||||
else if (config.interBrokerProtocolVersion.isAtLeast(IBP_1_0_IV0)) 1
|
||||
else 0
|
||||
|
||||
leaderAndIsrRequestMap.forKeyValue { (broker, leaderAndIsrPartitionStates) =>
|
||||
|
@ -511,13 +512,13 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig,
|
|||
|
||||
val partitionStates = updateMetadataRequestPartitionInfoMap.values.toBuffer
|
||||
val updateMetadataRequestVersion: Short =
|
||||
if (config.interBrokerProtocolVersion >= KAFKA_2_8_IV1) 7
|
||||
else if (config.interBrokerProtocolVersion >= KAFKA_2_4_IV1) 6
|
||||
else if (config.interBrokerProtocolVersion >= KAFKA_2_2_IV0) 5
|
||||
else if (config.interBrokerProtocolVersion >= KAFKA_1_0_IV0) 4
|
||||
else if (config.interBrokerProtocolVersion >= KAFKA_0_10_2_IV0) 3
|
||||
else if (config.interBrokerProtocolVersion >= KAFKA_0_10_0_IV1) 2
|
||||
else if (config.interBrokerProtocolVersion >= KAFKA_0_9_0) 1
|
||||
if (config.interBrokerProtocolVersion.isAtLeast(IBP_2_8_IV1)) 7
|
||||
else if (config.interBrokerProtocolVersion.isAtLeast(IBP_2_4_IV1)) 6
|
||||
else if (config.interBrokerProtocolVersion.isAtLeast(IBP_2_2_IV0)) 5
|
||||
else if (config.interBrokerProtocolVersion.isAtLeast(IBP_1_0_IV0)) 4
|
||||
else if (config.interBrokerProtocolVersion.isAtLeast(IBP_0_10_2_IV0)) 3
|
||||
else if (config.interBrokerProtocolVersion.isAtLeast(IBP_0_10_0_IV1)) 2
|
||||
else if (config.interBrokerProtocolVersion.isAtLeast(IBP_0_9_0)) 1
|
||||
else 0
|
||||
|
||||
val liveBrokers = controllerContext.liveOrShuttingDownBrokers.iterator.map { broker =>
|
||||
|
@ -567,9 +568,9 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig,
|
|||
private def sendStopReplicaRequests(controllerEpoch: Int, stateChangeLog: StateChangeLogger): Unit = {
|
||||
val traceEnabled = stateChangeLog.isTraceEnabled
|
||||
val stopReplicaRequestVersion: Short =
|
||||
if (config.interBrokerProtocolVersion >= KAFKA_2_6_IV0) 3
|
||||
else if (config.interBrokerProtocolVersion >= KAFKA_2_4_IV1) 2
|
||||
else if (config.interBrokerProtocolVersion >= KAFKA_2_2_IV0) 1
|
||||
if (config.interBrokerProtocolVersion.isAtLeast(IBP_2_6_IV0)) 3
|
||||
else if (config.interBrokerProtocolVersion.isAtLeast(IBP_2_4_IV1)) 2
|
||||
else if (config.interBrokerProtocolVersion.isAtLeast(IBP_2_2_IV0)) 1
|
||||
else 0
|
||||
|
||||
def responseCallback(brokerId: Int, isPartitionDeleted: TopicPartition => Boolean)
|
||||
|
|
|
@ -331,36 +331,36 @@ class KafkaController(val config: KafkaConfig,
|
|||
* This method sets up the FeatureZNode with enabled status, which means that the finalized
|
||||
* features stored in the FeatureZNode are active. The enabled status should be written by the
|
||||
* controller to the FeatureZNode only when the broker IBP config is greater than or equal to
|
||||
* KAFKA_2_7_IV0.
|
||||
* IBP_2_7_IV0.
|
||||
*
|
||||
* There are multiple cases handled here:
|
||||
*
|
||||
* 1. New cluster bootstrap:
|
||||
* A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config
|
||||
* setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all
|
||||
* setting greater than or equal to IBP_2_7_IV0. We would like to start the cluster with all
|
||||
* the possible supported features finalized immediately. Assuming this is the case, the
|
||||
* controller will start up and notice that the FeatureZNode is absent in the new cluster,
|
||||
* it will then create a FeatureZNode (with enabled status) containing the entire list of
|
||||
* supported features as its finalized features.
|
||||
*
|
||||
* 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0:
|
||||
* Imagine there was an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the
|
||||
* 2. Broker binary upgraded, but IBP config set to lower than IBP_2_7_IV0:
|
||||
* Imagine there was an existing Kafka cluster with IBP config less than IBP_2_7_IV0, and the
|
||||
* broker binary has now been upgraded to a newer version that supports the feature versioning
|
||||
* system (KIP-584). But the IBP config is still set to lower than KAFKA_2_7_IV0, and may be
|
||||
* system (KIP-584). But the IBP config is still set to lower than IBP_2_7_IV0, and may be
|
||||
* set to a higher value later. In this case, we want to start with no finalized features and
|
||||
* allow the user to finalize them whenever they are ready i.e. in the future whenever the
|
||||
* user sets IBP config to be greater than or equal to KAFKA_2_7_IV0, then the user could start
|
||||
* user sets IBP config to be greater than or equal to IBP_2_7_IV0, then the user could start
|
||||
* finalizing the features. This process ensures we do not enable all the possible features
|
||||
* immediately after an upgrade, which could be harmful to Kafka.
|
||||
* This is how we handle such a case:
|
||||
* - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the
|
||||
* - Before the IBP config upgrade (i.e. IBP config set to less than IBP_2_7_IV0), the
|
||||
* controller will start up and check if the FeatureZNode is absent.
|
||||
* - If the node is absent, it will react by creating a FeatureZNode with disabled status
|
||||
* and empty finalized features.
|
||||
* - Otherwise, if a node already exists in enabled status then the controller will just
|
||||
* flip the status to disabled and clear the finalized features.
|
||||
* - After the IBP config upgrade (i.e. IBP config set to greater than or equal to
|
||||
* KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
|
||||
* IBP_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists
|
||||
* and whether it is disabled.
|
||||
* - If the node is in disabled status, the controller won’t upgrade all features immediately.
|
||||
* Instead it will just switch the FeatureZNode status to enabled status. This lets the
|
||||
|
@ -368,17 +368,17 @@ class KafkaController(val config: KafkaConfig,
|
|||
* - Otherwise, if a node already exists in enabled status then the controller will leave
|
||||
* the node umodified.
|
||||
*
|
||||
* 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0:
|
||||
* Imagine there was an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker
|
||||
* binary has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and
|
||||
* 3. Broker binary upgraded, with existing cluster IBP config >= IBP_2_7_IV0:
|
||||
* Imagine there was an existing Kafka cluster with IBP config >= IBP_2_7_IV0, and the broker
|
||||
* binary has just been upgraded to a newer version (that supports IBP config IBP_2_7_IV0 and
|
||||
* higher). The controller will start up and find that a FeatureZNode is already present with
|
||||
* enabled status and existing finalized features. In such a case, the controller leaves the node
|
||||
* unmodified.
|
||||
*
|
||||
* 4. Broker downgrade:
|
||||
* Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to
|
||||
* KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
|
||||
* value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning
|
||||
* IBP_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a
|
||||
* value less than IBP_2_7_IV0. This means the user is also disabling the feature versioning
|
||||
* system (KIP-584). In this case, when the controller starts up with the lower IBP config, it
|
||||
* will switch the FeatureZNode status to disabled with empty features.
|
||||
*/
|
||||
|
@ -413,14 +413,14 @@ class KafkaController(val config: KafkaConfig,
|
|||
* Sets up the FeatureZNode with disabled status. This status means the feature versioning system
|
||||
* (KIP-584) is disabled, and, the finalized features stored in the FeatureZNode are not relevant.
|
||||
* This status should be written by the controller to the FeatureZNode only when the broker
|
||||
* IBP config is less than KAFKA_2_7_IV0.
|
||||
* IBP config is less than IBP_2_7_IV0.
|
||||
*
|
||||
* NOTE:
|
||||
* 1. When this method returns, existing finalized features (if any) will be cleared from the
|
||||
* FeatureZNode.
|
||||
* 2. This method, unlike enableFeatureVersioning() need not wait for the FinalizedFeatureCache
|
||||
* to be updated, because, such updates to the cache (via FinalizedFeatureChangeListener)
|
||||
* are disabled when IBP config is < than KAFKA_2_7_IV0.
|
||||
* are disabled when IBP config is < than IBP_2_7_IV0.
|
||||
*/
|
||||
private def disableFeatureVersioning(): Unit = {
|
||||
val newNode = FeatureZNode(FeatureZNodeStatus.Disabled, Features.emptyFinalizedFeatures())
|
||||
|
|
|
@ -16,7 +16,6 @@
|
|||
*/
|
||||
package kafka.controller
|
||||
|
||||
import kafka.api.KAFKA_3_2_IV0
|
||||
import kafka.api.LeaderAndIsr
|
||||
import kafka.common.StateChangeFailedException
|
||||
import kafka.controller.Election._
|
||||
|
@ -28,8 +27,10 @@ import kafka.zk.KafkaZkClient.UpdateLeaderAndIsrResult
|
|||
import kafka.zk.TopicPartitionStateZNode
|
||||
import org.apache.kafka.common.TopicPartition
|
||||
import org.apache.kafka.common.errors.ControllerMovedException
|
||||
import org.apache.kafka.server.common.MetadataVersion.IBP_3_2_IV0
|
||||
import org.apache.zookeeper.KeeperException
|
||||
import org.apache.zookeeper.KeeperException.Code
|
||||
|
||||
import scala.collection.{Map, Seq, mutable}
|
||||
|
||||
abstract class PartitionStateMachine(controllerContext: ControllerContext) extends Logging {
|
||||
|
@ -132,7 +133,7 @@ class ZkPartitionStateMachine(config: KafkaConfig,
|
|||
controllerBrokerRequestBatch: ControllerBrokerRequestBatch)
|
||||
extends PartitionStateMachine(controllerContext) {
|
||||
|
||||
private val isLeaderRecoverySupported = config.interBrokerProtocolVersion >= KAFKA_3_2_IV0
|
||||
private val isLeaderRecoverySupported = config.interBrokerProtocolVersion.isAtLeast(IBP_3_2_IV0)
|
||||
|
||||
private val controllerId = config.brokerId
|
||||
this.logIdent = s"[PartitionStateMachine controllerId=$controllerId] "
|
||||
|
|
|
@ -26,7 +26,6 @@ import java.util.concurrent.locks.ReentrantLock
|
|||
import java.util.concurrent.{ConcurrentHashMap, TimeUnit}
|
||||
|
||||
import com.yammer.metrics.core.Gauge
|
||||
import kafka.api.{ApiVersion, KAFKA_0_10_1_IV0, KAFKA_2_1_IV0, KAFKA_2_1_IV1, KAFKA_2_3_IV0}
|
||||
import kafka.common.OffsetAndMetadata
|
||||
import kafka.internals.generated.{GroupMetadataValue, OffsetCommitKey, OffsetCommitValue, GroupMetadataKey => GroupMetadataKeyData}
|
||||
import kafka.log.AppendOrigin
|
||||
|
@ -47,13 +46,15 @@ import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
|
|||
import org.apache.kafka.common.requests.{OffsetCommitRequest, OffsetFetchResponse}
|
||||
import org.apache.kafka.common.utils.{Time, Utils}
|
||||
import org.apache.kafka.common.{KafkaException, MessageFormatter, TopicPartition}
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.apache.kafka.server.common.MetadataVersion.{IBP_0_10_1_IV0, IBP_2_1_IV0, IBP_2_1_IV1, IBP_2_3_IV0}
|
||||
|
||||
import scala.collection._
|
||||
import scala.collection.mutable.ArrayBuffer
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
||||
class GroupMetadataManager(brokerId: Int,
|
||||
interBrokerProtocolVersion: ApiVersion,
|
||||
interBrokerProtocolVersion: MetadataVersion,
|
||||
config: OffsetConfig,
|
||||
val replicaManager: ReplicaManager,
|
||||
time: Time,
|
||||
|
@ -1074,14 +1075,14 @@ object GroupMetadataManager {
|
|||
* Generates the payload for offset commit message from given offset and metadata
|
||||
*
|
||||
* @param offsetAndMetadata consumer's current offset and metadata
|
||||
* @param apiVersion the api version
|
||||
* @param metadataVersion the api version
|
||||
* @return payload for offset commit message
|
||||
*/
|
||||
def offsetCommitValue(offsetAndMetadata: OffsetAndMetadata,
|
||||
apiVersion: ApiVersion): Array[Byte] = {
|
||||
metadataVersion: MetadataVersion): Array[Byte] = {
|
||||
val version =
|
||||
if (apiVersion < KAFKA_2_1_IV0 || offsetAndMetadata.expireTimestamp.nonEmpty) 1.toShort
|
||||
else if (apiVersion < KAFKA_2_1_IV1) 2.toShort
|
||||
if (metadataVersion.isLessThan(IBP_2_1_IV0) || offsetAndMetadata.expireTimestamp.nonEmpty) 1.toShort
|
||||
else if (metadataVersion.isLessThan(IBP_2_1_IV1)) 2.toShort
|
||||
else 3.toShort
|
||||
MessageUtil.toVersionPrefixedBytes(version, new OffsetCommitValue()
|
||||
.setOffset(offsetAndMetadata.offset)
|
||||
|
@ -1099,17 +1100,17 @@ object GroupMetadataManager {
|
|||
*
|
||||
* @param groupMetadata current group metadata
|
||||
* @param assignment the assignment for the rebalancing generation
|
||||
* @param apiVersion the api version
|
||||
* @param metadataVersion the api version
|
||||
* @return payload for offset commit message
|
||||
*/
|
||||
def groupMetadataValue(groupMetadata: GroupMetadata,
|
||||
assignment: Map[String, Array[Byte]],
|
||||
apiVersion: ApiVersion): Array[Byte] = {
|
||||
metadataVersion: MetadataVersion): Array[Byte] = {
|
||||
|
||||
val version =
|
||||
if (apiVersion < KAFKA_0_10_1_IV0) 0.toShort
|
||||
else if (apiVersion < KAFKA_2_1_IV0) 1.toShort
|
||||
else if (apiVersion < KAFKA_2_3_IV0) 2.toShort
|
||||
if (metadataVersion.isLessThan(IBP_0_10_1_IV0)) 0.toShort
|
||||
else if (metadataVersion.isLessThan(IBP_2_1_IV0)) 1.toShort
|
||||
else if (metadataVersion.isLessThan(IBP_2_3_IV0)) 2.toShort
|
||||
else 3.toShort
|
||||
|
||||
MessageUtil.toVersionPrefixedBytes(version, new GroupMetadataValue()
|
||||
|
|
|
@ -19,7 +19,7 @@ package kafka.coordinator.transaction
|
|||
|
||||
import java.util
|
||||
import java.util.concurrent.{BlockingQueue, ConcurrentHashMap, LinkedBlockingQueue}
|
||||
import kafka.api.KAFKA_2_8_IV0
|
||||
|
||||
import kafka.common.{InterBrokerSendThread, RequestAndCompletionHandler}
|
||||
import kafka.metrics.KafkaMetricsGroup
|
||||
import kafka.server.{KafkaConfig, MetadataCache, RequestLocal}
|
||||
|
@ -34,6 +34,7 @@ import org.apache.kafka.common.requests.{TransactionResult, WriteTxnMarkersReque
|
|||
import org.apache.kafka.common.security.JaasContext
|
||||
import org.apache.kafka.common.utils.{LogContext, Time}
|
||||
import org.apache.kafka.common.{Node, Reconfigurable, TopicPartition}
|
||||
import org.apache.kafka.server.common.MetadataVersion.IBP_2_8_IV0
|
||||
|
||||
import scala.collection.{concurrent, immutable}
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
@ -147,7 +148,7 @@ class TransactionMarkerChannelManager(
|
|||
private val transactionsWithPendingMarkers = new ConcurrentHashMap[String, PendingCompleteTxn]
|
||||
|
||||
val writeTxnMarkersRequestVersion: Short =
|
||||
if (config.interBrokerProtocolVersion >= KAFKA_2_8_IV0) 1
|
||||
if (config.interBrokerProtocolVersion.isAtLeast(IBP_2_8_IV0)) 1
|
||||
else 0
|
||||
|
||||
newGauge("UnknownDestinationQueueSize", () => markersQueueForUnknownBroker.totalNumMarkers)
|
||||
|
|
|
@ -17,7 +17,6 @@
|
|||
|
||||
package kafka.log
|
||||
|
||||
import kafka.api.{ApiVersion, ApiVersionValidator, KAFKA_3_0_IV1}
|
||||
import kafka.log.LogConfig.configDef
|
||||
import kafka.message.BrokerCompressionCodec
|
||||
import kafka.server.{KafkaConfig, ThrottledReplicaListValidator}
|
||||
|
@ -29,9 +28,12 @@ import org.apache.kafka.common.record.{LegacyRecord, RecordVersion, TimestampTyp
|
|||
import org.apache.kafka.common.utils.{ConfigUtils, Utils}
|
||||
import org.apache.kafka.metadata.ConfigSynonym
|
||||
import org.apache.kafka.metadata.ConfigSynonym.{HOURS_TO_MILLISECONDS, MINUTES_TO_MILLISECONDS}
|
||||
|
||||
import java.util.Arrays.asList
|
||||
import java.util.{Collections, Locale, Properties}
|
||||
|
||||
import org.apache.kafka.server.common.{MetadataVersion, MetadataVersionValidator}
|
||||
import org.apache.kafka.server.common.MetadataVersion._
|
||||
|
||||
import scala.annotation.nowarn
|
||||
import scala.collection.{Map, mutable}
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
@ -103,7 +105,7 @@ case class LogConfig(props: java.util.Map[_, _], overriddenConfigs: Set[String]
|
|||
|
||||
/* See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for details */
|
||||
@deprecated("3.0")
|
||||
val messageFormatVersion = ApiVersion(getString(LogConfig.MessageFormatVersionProp))
|
||||
val messageFormatVersion = MetadataVersion.fromVersionString(getString(LogConfig.MessageFormatVersionProp))
|
||||
|
||||
val messageTimestampType = TimestampType.forName(getString(LogConfig.MessageTimestampTypeProp))
|
||||
val messageTimestampDifferenceMaxMs = getLong(LogConfig.MessageTimestampDifferenceMaxMsProp).longValue
|
||||
|
@ -157,7 +159,7 @@ case class LogConfig(props: java.util.Map[_, _], overriddenConfigs: Set[String]
|
|||
def remoteLogConfig = _remoteLogConfig
|
||||
|
||||
@nowarn("cat=deprecation")
|
||||
def recordVersion = messageFormatVersion.recordVersion
|
||||
def recordVersion = messageFormatVersion.highestSupportedRecordVersion
|
||||
|
||||
def randomSegmentJitter: Long =
|
||||
if (segmentJitterMs == 0) 0 else Utils.abs(scala.util.Random.nextInt()) % math.min(segmentJitterMs, segmentMs)
|
||||
|
@ -367,7 +369,7 @@ object LogConfig {
|
|||
MEDIUM, CompressionTypeDoc, KafkaConfig.CompressionTypeProp)
|
||||
.define(PreAllocateEnableProp, BOOLEAN, Defaults.PreAllocateEnable, MEDIUM, PreAllocateEnableDoc,
|
||||
KafkaConfig.LogPreAllocateProp)
|
||||
.define(MessageFormatVersionProp, STRING, Defaults.MessageFormatVersion, ApiVersionValidator, MEDIUM, MessageFormatVersionDoc,
|
||||
.define(MessageFormatVersionProp, STRING, Defaults.MessageFormatVersion, new MetadataVersionValidator(), MEDIUM, MessageFormatVersionDoc,
|
||||
KafkaConfig.LogMessageFormatVersionProp)
|
||||
.define(MessageTimestampTypeProp, STRING, Defaults.MessageTimestampType, in("CreateTime", "LogAppendTime"), MEDIUM, MessageTimestampTypeDoc,
|
||||
KafkaConfig.LogMessageTimestampTypeProp)
|
||||
|
@ -560,17 +562,17 @@ object LogConfig {
|
|||
logProps
|
||||
}
|
||||
|
||||
def shouldIgnoreMessageFormatVersion(interBrokerProtocolVersion: ApiVersion): Boolean =
|
||||
interBrokerProtocolVersion >= KAFKA_3_0_IV1
|
||||
def shouldIgnoreMessageFormatVersion(interBrokerProtocolVersion: MetadataVersion): Boolean =
|
||||
interBrokerProtocolVersion.isAtLeast(IBP_3_0_IV1)
|
||||
|
||||
class MessageFormatVersion(messageFormatVersionString: String, interBrokerProtocolVersionString: String) {
|
||||
val messageFormatVersion = ApiVersion(messageFormatVersionString)
|
||||
private val interBrokerProtocolVersion = ApiVersion(interBrokerProtocolVersionString)
|
||||
val messageFormatVersion = MetadataVersion.fromVersionString(messageFormatVersionString)
|
||||
private val interBrokerProtocolVersion = MetadataVersion.fromVersionString(interBrokerProtocolVersionString)
|
||||
|
||||
def shouldIgnore: Boolean = shouldIgnoreMessageFormatVersion(interBrokerProtocolVersion)
|
||||
|
||||
def shouldWarn: Boolean =
|
||||
interBrokerProtocolVersion >= KAFKA_3_0_IV1 && messageFormatVersion.recordVersion.precedes(RecordVersion.V2)
|
||||
interBrokerProtocolVersion.isAtLeast(IBP_3_0_IV1) && messageFormatVersion.highestSupportedRecordVersion.precedes(RecordVersion.V2)
|
||||
|
||||
@nowarn("cat=deprecation")
|
||||
def topicWarningMessage(topicName: String): String = {
|
||||
|
|
|
@ -17,13 +17,12 @@
|
|||
|
||||
package kafka.log
|
||||
|
||||
import kafka.api.ApiVersion
|
||||
import kafka.log.LogConfig.MessageFormatVersion
|
||||
|
||||
import java.io._
|
||||
import java.nio.file.Files
|
||||
import java.util.concurrent._
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
|
||||
import kafka.metrics.KafkaMetricsGroup
|
||||
import kafka.server.checkpoints.OffsetCheckpointFile
|
||||
import kafka.server.metadata.ConfigRepository
|
||||
|
@ -38,8 +37,10 @@ import scala.collection._
|
|||
import scala.collection.mutable.ArrayBuffer
|
||||
import scala.util.{Failure, Success, Try}
|
||||
import kafka.utils.Implicits._
|
||||
|
||||
import java.util.Properties
|
||||
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
|
||||
import scala.annotation.nowarn
|
||||
|
||||
/**
|
||||
|
@ -65,7 +66,7 @@ class LogManager(logDirs: Seq[File],
|
|||
val retentionCheckMs: Long,
|
||||
val maxTransactionTimeoutMs: Int,
|
||||
val maxPidExpirationMs: Int,
|
||||
interBrokerProtocolVersion: ApiVersion,
|
||||
interBrokerProtocolVersion: MetadataVersion,
|
||||
scheduler: Scheduler,
|
||||
brokerTopicStats: BrokerTopicStats,
|
||||
logDirFailureChannel: LogDirFailureChannel,
|
||||
|
|
|
@ -17,7 +17,7 @@
|
|||
package kafka.log
|
||||
|
||||
import java.nio.ByteBuffer
|
||||
import kafka.api.{ApiVersion, KAFKA_2_1_IV0}
|
||||
|
||||
import kafka.common.{LongRef, RecordValidationException}
|
||||
import kafka.message.{CompressionCodec, NoCompressionCodec, ZStdCompressionCodec}
|
||||
import kafka.server.{BrokerTopicStats, RequestLocal}
|
||||
|
@ -29,6 +29,8 @@ import org.apache.kafka.common.TopicPartition
|
|||
import org.apache.kafka.common.protocol.Errors
|
||||
import org.apache.kafka.common.requests.ProduceResponse.RecordError
|
||||
import org.apache.kafka.common.utils.Time
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.apache.kafka.server.common.MetadataVersion.IBP_2_1_IV0
|
||||
|
||||
import scala.collection.{Seq, mutable}
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
@ -94,7 +96,7 @@ private[log] object LogValidator extends Logging {
|
|||
timestampDiffMaxMs: Long,
|
||||
partitionLeaderEpoch: Int,
|
||||
origin: AppendOrigin,
|
||||
interBrokerProtocolVersion: ApiVersion,
|
||||
interBrokerProtocolVersion: MetadataVersion,
|
||||
brokerTopicStats: BrokerTopicStats,
|
||||
requestLocal: RequestLocal): ValidationAndOffsetAssignResult = {
|
||||
if (sourceCodec == NoCompressionCodec && targetCodec == NoCompressionCodec) {
|
||||
|
@ -365,11 +367,11 @@ private[log] object LogValidator extends Logging {
|
|||
timestampDiffMaxMs: Long,
|
||||
partitionLeaderEpoch: Int,
|
||||
origin: AppendOrigin,
|
||||
interBrokerProtocolVersion: ApiVersion,
|
||||
interBrokerProtocolVersion: MetadataVersion,
|
||||
brokerTopicStats: BrokerTopicStats,
|
||||
requestLocal: RequestLocal): ValidationAndOffsetAssignResult = {
|
||||
|
||||
if (targetCodec == ZStdCompressionCodec && interBrokerProtocolVersion < KAFKA_2_1_IV0)
|
||||
if (targetCodec == ZStdCompressionCodec && interBrokerProtocolVersion.isLessThan(IBP_2_1_IV0))
|
||||
throw new UnsupportedCompressionTypeException("Produce requests to inter.broker.protocol.version < 2.1 broker " +
|
||||
"are not allowed to use ZStandard compression")
|
||||
|
||||
|
|
|
@ -18,12 +18,11 @@
|
|||
package kafka.log
|
||||
|
||||
import com.yammer.metrics.core.MetricName
|
||||
|
||||
import java.io.{File, IOException}
|
||||
import java.nio.file.Files
|
||||
import java.util.Optional
|
||||
import java.util.concurrent.TimeUnit
|
||||
import kafka.api.{ApiVersion, KAFKA_0_10_0_IV0}
|
||||
|
||||
import kafka.common.{LongRef, OffsetsOutOfOrderException, UnexpectedAppendOffsetException}
|
||||
import kafka.log.AppendOrigin.RaftLeader
|
||||
import kafka.message.{BrokerCompressionCodec, CompressionCodec, NoCompressionCodec}
|
||||
|
@ -41,6 +40,8 @@ import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_
|
|||
import org.apache.kafka.common.requests.ProduceResponse.RecordError
|
||||
import org.apache.kafka.common.utils.{Time, Utils}
|
||||
import org.apache.kafka.common.{InvalidRecordException, KafkaException, TopicPartition, Uuid}
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.apache.kafka.server.common.MetadataVersion.IBP_0_10_0_IV0
|
||||
|
||||
import scala.annotation.nowarn
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
@ -717,7 +718,7 @@ class UnifiedLog(@volatile var logStartOffset: Long,
|
|||
def appendAsLeader(records: MemoryRecords,
|
||||
leaderEpoch: Int,
|
||||
origin: AppendOrigin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion: ApiVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion: MetadataVersion = MetadataVersion.latest,
|
||||
requestLocal: RequestLocal = RequestLocal.NoCaching): LogAppendInfo = {
|
||||
val validateAndAssignOffsets = origin != AppendOrigin.RaftLeader
|
||||
append(records, origin, interBrokerProtocolVersion, validateAndAssignOffsets, leaderEpoch, Some(requestLocal), ignoreRecordSize = false)
|
||||
|
@ -733,7 +734,7 @@ class UnifiedLog(@volatile var logStartOffset: Long,
|
|||
def appendAsFollower(records: MemoryRecords): LogAppendInfo = {
|
||||
append(records,
|
||||
origin = AppendOrigin.Replication,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
validateAndAssignOffsets = false,
|
||||
leaderEpoch = -1,
|
||||
None,
|
||||
|
@ -761,7 +762,7 @@ class UnifiedLog(@volatile var logStartOffset: Long,
|
|||
*/
|
||||
private def append(records: MemoryRecords,
|
||||
origin: AppendOrigin,
|
||||
interBrokerProtocolVersion: ApiVersion,
|
||||
interBrokerProtocolVersion: MetadataVersion,
|
||||
validateAndAssignOffsets: Boolean,
|
||||
leaderEpoch: Int,
|
||||
requestLocal: Option[RequestLocal],
|
||||
|
@ -1225,12 +1226,12 @@ class UnifiedLog(@volatile var logStartOffset: Long,
|
|||
maybeHandleIOException(s"Error while fetching offset by timestamp for $topicPartition in dir ${dir.getParent}") {
|
||||
debug(s"Searching offset for timestamp $targetTimestamp")
|
||||
|
||||
if (config.messageFormatVersion < KAFKA_0_10_0_IV0 &&
|
||||
if (config.messageFormatVersion.isLessThan(IBP_0_10_0_IV0) &&
|
||||
targetTimestamp != ListOffsetsRequest.EARLIEST_TIMESTAMP &&
|
||||
targetTimestamp != ListOffsetsRequest.LATEST_TIMESTAMP)
|
||||
throw new UnsupportedForMessageFormatException(s"Cannot search offsets based on timestamp because message format version " +
|
||||
s"for partition $topicPartition is ${config.messageFormatVersion} which is earlier than the minimum " +
|
||||
s"required version $KAFKA_0_10_0_IV0")
|
||||
s"required version $IBP_0_10_0_IV0")
|
||||
|
||||
// For the earliest and latest, we do not need to return the timestamp.
|
||||
if (targetTimestamp == ListOffsetsRequest.EARLIEST_TIMESTAMP) {
|
||||
|
|
|
@ -20,7 +20,6 @@ import java.{lang, util}
|
|||
import java.util.concurrent.{CompletableFuture, CompletionStage}
|
||||
|
||||
import com.typesafe.scalalogging.Logger
|
||||
import kafka.api.KAFKA_2_0_IV1
|
||||
import kafka.security.authorizer.AclEntry.ResourceSeparator
|
||||
import kafka.server.{KafkaConfig, KafkaServer}
|
||||
import kafka.utils._
|
||||
|
@ -37,6 +36,7 @@ import org.apache.kafka.common.security.auth.KafkaPrincipal
|
|||
import org.apache.kafka.common.utils.{SecurityUtils, Time}
|
||||
import org.apache.kafka.server.authorizer.AclDeleteResult.AclBindingDeleteResult
|
||||
import org.apache.kafka.server.authorizer._
|
||||
import org.apache.kafka.server.common.MetadataVersion.IBP_2_0_IV1
|
||||
import org.apache.zookeeper.client.ZKClientConfig
|
||||
|
||||
import scala.annotation.nowarn
|
||||
|
@ -182,7 +182,7 @@ class AclAuthorizer extends Authorizer with Logging {
|
|||
metricGroup = "kafka.security", metricType = "AclAuthorizer", createChrootIfNecessary = true)
|
||||
zkClient.createAclPaths()
|
||||
|
||||
extendedAclSupport = kafkaConfig.interBrokerProtocolVersion >= KAFKA_2_0_IV1
|
||||
extendedAclSupport = kafkaConfig.interBrokerProtocolVersion.isAtLeast(IBP_2_0_IV1)
|
||||
|
||||
// Start change listeners first and then populate the cache so that there is no timing window
|
||||
// between loading cache and processing change notifications.
|
||||
|
@ -207,7 +207,7 @@ class AclAuthorizer extends Authorizer with Logging {
|
|||
try {
|
||||
if (!extendedAclSupport && aclBinding.pattern.patternType == PatternType.PREFIXED) {
|
||||
throw new UnsupportedVersionException(s"Adding ACLs on prefixed resource patterns requires " +
|
||||
s"${KafkaConfig.InterBrokerProtocolVersionProp} of $KAFKA_2_0_IV1 or greater")
|
||||
s"${KafkaConfig.InterBrokerProtocolVersionProp} of $IBP_2_0_IV1 or greater")
|
||||
}
|
||||
validateAclBinding(aclBinding)
|
||||
true
|
||||
|
|
|
@ -557,11 +557,11 @@ abstract class AbstractFetcherThread(name: String,
|
|||
* For each topic partition, the offset to truncate to is calculated based on leader's returned
|
||||
* epoch and offset:
|
||||
* -- If the leader replied with undefined epoch offset, we must use the high watermark. This can
|
||||
* happen if 1) the leader is still using message format older than KAFKA_0_11_0; 2) the follower
|
||||
* happen if 1) the leader is still using message format older than IBP_0_11_0; 2) the follower
|
||||
* requested leader epoch < the first leader epoch known to the leader.
|
||||
* -- If the leader replied with the valid offset but undefined leader epoch, we truncate to
|
||||
* leader's offset if it is lower than follower's Log End Offset. This may happen if the
|
||||
* leader is on the inter-broker protocol version < KAFKA_2_0_IV0
|
||||
* leader is on the inter-broker protocol version < IBP_2_0_IV0
|
||||
* -- If the leader replied with leader epoch not known to the follower, we truncate to the
|
||||
* end offset of the largest epoch that is smaller than the epoch the leader replied with, and
|
||||
* send OffsetsForLeaderEpochRequest with that leader epoch. In a more rare case, where the
|
||||
|
@ -584,7 +584,7 @@ abstract class AbstractFetcherThread(name: String,
|
|||
s"The initial fetch offset ${partitionStates.stateValue(tp).fetchOffset} will be used for truncation.")
|
||||
OffsetTruncationState(partitionStates.stateValue(tp).fetchOffset, truncationCompleted = true)
|
||||
} else if (leaderEpochOffset.leaderEpoch == UNDEFINED_EPOCH) {
|
||||
// either leader or follower or both use inter-broker protocol version < KAFKA_2_0_IV0
|
||||
// either leader or follower or both use inter-broker protocol version < IBP_2_0_IV0
|
||||
// (version 0 of OffsetForLeaderEpoch request/response)
|
||||
warn(s"Leader or replica is on protocol version where leader epoch is not considered in the OffsetsForLeaderEpoch response. " +
|
||||
s"The leader's offset ${leaderEpochOffset.endOffset} will be used for truncation in $tp.")
|
||||
|
|
|
@ -20,8 +20,6 @@ import java.util
|
|||
import java.util.concurrent.atomic.AtomicBoolean
|
||||
import java.util.concurrent.{CompletableFuture, ConcurrentHashMap, TimeUnit}
|
||||
|
||||
import kafka.api.ApiVersion
|
||||
import kafka.api.KAFKA_3_2_IV0
|
||||
import kafka.api.LeaderAndIsr
|
||||
import kafka.metrics.KafkaMetricsGroup
|
||||
import kafka.utils.{KafkaScheduler, Logging, Scheduler}
|
||||
|
@ -35,6 +33,7 @@ import org.apache.kafka.common.protocol.Errors
|
|||
import org.apache.kafka.common.requests.{AlterPartitionRequest, AlterPartitionResponse}
|
||||
import org.apache.kafka.common.utils.Time
|
||||
import org.apache.kafka.metadata.LeaderRecoveryState
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
|
||||
import scala.collection.mutable
|
||||
import scala.collection.mutable.ListBuffer
|
||||
|
@ -122,7 +121,7 @@ class DefaultAlterPartitionManager(
|
|||
val time: Time,
|
||||
val brokerId: Int,
|
||||
val brokerEpochSupplier: () => Long,
|
||||
ibpVersion: ApiVersion
|
||||
ibpVersion: MetadataVersion
|
||||
) extends AlterPartitionManager with Logging with KafkaMetricsGroup {
|
||||
|
||||
// Used to allow only one pending ISR update per partition (visible for testing)
|
||||
|
@ -234,7 +233,7 @@ class DefaultAlterPartitionManager(
|
|||
.setNewIsr(item.leaderAndIsr.isr.map(Integer.valueOf).asJava)
|
||||
.setPartitionEpoch(item.leaderAndIsr.partitionEpoch)
|
||||
|
||||
if (ibpVersion >= KAFKA_3_2_IV0) {
|
||||
if (ibpVersion.isAtLeast(MetadataVersion.IBP_3_2_IV0)) {
|
||||
partitionData.setLeaderRecoveryState(item.leaderAndIsr.leaderRecoveryState.value)
|
||||
}
|
||||
|
||||
|
|
|
@ -16,12 +16,13 @@
|
|||
*/
|
||||
package kafka.server
|
||||
|
||||
import kafka.api.ApiVersion
|
||||
import kafka.network
|
||||
import kafka.network.RequestChannel
|
||||
import org.apache.kafka.common.feature.Features
|
||||
import org.apache.kafka.common.message.ApiMessageType.ListenerType
|
||||
import org.apache.kafka.common.protocol.ApiKeys
|
||||
import org.apache.kafka.common.requests.ApiVersionsResponse
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
||||
|
@ -69,7 +70,7 @@ class SimpleApiVersionManager(
|
|||
|
||||
class DefaultApiVersionManager(
|
||||
val listenerType: ListenerType,
|
||||
interBrokerProtocolVersion: ApiVersion,
|
||||
interBrokerProtocolVersion: MetadataVersion,
|
||||
forwardingManager: Option[ForwardingManager],
|
||||
features: BrokerFeatures,
|
||||
featureCache: FinalizedFeatureCache
|
||||
|
@ -81,19 +82,21 @@ class DefaultApiVersionManager(
|
|||
val controllerApiVersions = forwardingManager.flatMap(_.controllerApiVersions)
|
||||
|
||||
finalizedFeaturesOpt match {
|
||||
case Some(finalizedFeatures) => ApiVersion.apiVersionsResponse(
|
||||
case Some(finalizedFeatures) => ApiVersionsResponse.createApiVersionsResponse(
|
||||
throttleTimeMs,
|
||||
interBrokerProtocolVersion.recordVersion,
|
||||
interBrokerProtocolVersion.highestSupportedRecordVersion,
|
||||
supportedFeatures,
|
||||
finalizedFeatures.features,
|
||||
finalizedFeatures.epoch,
|
||||
controllerApiVersions,
|
||||
controllerApiVersions.orNull,
|
||||
listenerType)
|
||||
case None => ApiVersion.apiVersionsResponse(
|
||||
case None => ApiVersionsResponse.createApiVersionsResponse(
|
||||
throttleTimeMs,
|
||||
interBrokerProtocolVersion.recordVersion,
|
||||
interBrokerProtocolVersion.highestSupportedRecordVersion,
|
||||
supportedFeatures,
|
||||
controllerApiVersions,
|
||||
Features.emptyFinalizedFeatures,
|
||||
ApiVersionsResponse.UNKNOWN_FINALIZED_FEATURES_EPOCH,
|
||||
controllerApiVersions.orNull,
|
||||
listenerType)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -105,7 +105,7 @@ class TopicConfigHandler(private val logManager: LogManager, kafkaConfig: KafkaC
|
|||
if (messageFormatVersion.shouldWarn)
|
||||
warn(messageFormatVersion.topicWarningMessage(topic))
|
||||
Some(LogConfig.MessageFormatVersionProp)
|
||||
} else if (kafkaConfig.interBrokerProtocolVersion < messageFormatVersion.messageFormatVersion) {
|
||||
} else if (kafkaConfig.interBrokerProtocolVersion.isLessThan(messageFormatVersion.messageFormatVersion)) {
|
||||
warn(s"Topic configuration ${LogConfig.MessageFormatVersionProp} is ignored for `$topic` because `$versionString` " +
|
||||
s"is higher than what is allowed by the inter-broker protocol version `${kafkaConfig.interBrokerProtocolVersionString}`")
|
||||
Some(LogConfig.MessageFormatVersionProp)
|
||||
|
|
|
@ -22,7 +22,6 @@ import java.util.OptionalLong
|
|||
import java.util.concurrent.locks.ReentrantLock
|
||||
import java.util.concurrent.{CompletableFuture, TimeUnit}
|
||||
|
||||
import kafka.api.KAFKA_3_2_IV0
|
||||
import kafka.cluster.Broker.ServerInfo
|
||||
import kafka.metrics.{KafkaMetricsGroup, LinuxIoMetricsCollector}
|
||||
import kafka.network.{DataPlaneAcceptor, SocketServer}
|
||||
|
@ -43,6 +42,7 @@ import org.apache.kafka.raft.RaftConfig
|
|||
import org.apache.kafka.raft.RaftConfig.AddressSpec
|
||||
import org.apache.kafka.server.authorizer.Authorizer
|
||||
import org.apache.kafka.server.common.ApiMessageAndVersion
|
||||
import org.apache.kafka.server.common.MetadataVersion.IBP_3_2_IV0
|
||||
import org.apache.kafka.common.config.ConfigException
|
||||
import org.apache.kafka.metadata.authorizer.ClusterMetadataAuthorizer
|
||||
import org.apache.kafka.server.metrics.KafkaYammerMetrics
|
||||
|
@ -177,7 +177,7 @@ class ControllerServer(
|
|||
setQuorumFeatures(quorumFeatures).
|
||||
setDefaultReplicationFactor(config.defaultReplicationFactor.toShort).
|
||||
setDefaultNumPartitions(config.numPartitions.intValue()).
|
||||
setIsLeaderRecoverySupported(config.interBrokerProtocolVersion >= KAFKA_3_2_IV0).
|
||||
setIsLeaderRecoverySupported(config.interBrokerProtocolVersion.isAtLeast(IBP_3_2_IV0)).
|
||||
setSessionTimeoutNs(TimeUnit.NANOSECONDS.convert(config.brokerSessionTimeoutMs.longValue(),
|
||||
TimeUnit.MILLISECONDS)).
|
||||
setSnapshotMaxNewRecordBytes(config.metadataSnapshotMaxNewRecordBytes).
|
||||
|
|
|
@ -18,7 +18,7 @@
|
|||
package kafka.server
|
||||
|
||||
import kafka.admin.AdminUtils
|
||||
import kafka.api.{ApiVersion, ElectLeadersRequestOps, KAFKA_0_11_0_IV0, KAFKA_2_3_IV0}
|
||||
import kafka.api.ElectLeadersRequestOps
|
||||
import kafka.common.OffsetAndMetadata
|
||||
import kafka.controller.ReplicaAssignment
|
||||
import kafka.coordinator.group._
|
||||
|
@ -79,6 +79,9 @@ import java.util.concurrent.ConcurrentHashMap
|
|||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import java.util.{Collections, Optional}
|
||||
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.apache.kafka.server.common.MetadataVersion.{IBP_0_11_0_IV0, IBP_2_3_IV0}
|
||||
|
||||
import scala.annotation.nowarn
|
||||
import scala.collection.{Map, Seq, Set, immutable, mutable}
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
@ -430,7 +433,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
.setTopics(responseTopicList)
|
||||
.setThrottleTimeMs(requestThrottleMs)
|
||||
))
|
||||
} else if (offsetCommitRequest.data.groupInstanceId != null && config.interBrokerProtocolVersion < KAFKA_2_3_IV0) {
|
||||
} else if (offsetCommitRequest.data.groupInstanceId != null && config.interBrokerProtocolVersion.isLessThan(IBP_2_3_IV0)) {
|
||||
// Only enable static membership when IBP >= 2.3, because it is not safe for the broker to use the static member logic
|
||||
// until we are sure that all brokers support it. If static group being loaded by an older coordinator, it will discard
|
||||
// the group.instance.id field, so static members could accidentally become "dynamic", which leads to wrong states.
|
||||
|
@ -1664,7 +1667,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
requestHelper.sendResponseMaybeThrottle(request, createResponse)
|
||||
}
|
||||
|
||||
if (joinGroupRequest.data.groupInstanceId != null && config.interBrokerProtocolVersion < KAFKA_2_3_IV0) {
|
||||
if (joinGroupRequest.data.groupInstanceId != null && config.interBrokerProtocolVersion.isLessThan(IBP_2_3_IV0)) {
|
||||
// Only enable static membership when IBP >= 2.3, because it is not safe for the broker to use the static member logic
|
||||
// until we are sure that all brokers support it. If static group being loaded by an older coordinator, it will discard
|
||||
// the group.instance.id field, so static members could accidentally become "dynamic", which leads to wrong states.
|
||||
|
@ -1718,7 +1721,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
))
|
||||
}
|
||||
|
||||
if (syncGroupRequest.data.groupInstanceId != null && config.interBrokerProtocolVersion < KAFKA_2_3_IV0) {
|
||||
if (syncGroupRequest.data.groupInstanceId != null && config.interBrokerProtocolVersion.isLessThan(IBP_2_3_IV0)) {
|
||||
// Only enable static membership when IBP >= 2.3, because it is not safe for the broker to use the static member logic
|
||||
// until we are sure that all brokers support it. If static group being loaded by an older coordinator, it will discard
|
||||
// the group.instance.id field, so static members could accidentally become "dynamic", which leads to wrong states.
|
||||
|
@ -1791,7 +1794,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
requestHelper.sendResponseMaybeThrottle(request, createResponse)
|
||||
}
|
||||
|
||||
if (heartbeatRequest.data.groupInstanceId != null && config.interBrokerProtocolVersion < KAFKA_2_3_IV0) {
|
||||
if (heartbeatRequest.data.groupInstanceId != null && config.interBrokerProtocolVersion.isLessThan(IBP_2_3_IV0)) {
|
||||
// Only enable static membership when IBP >= 2.3, because it is not safe for the broker to use the static member logic
|
||||
// until we are sure that all brokers support it. If static group being loaded by an older coordinator, it will discard
|
||||
// the group.instance.id field, so static members could accidentally become "dynamic", which leads to wrong states.
|
||||
|
@ -2229,7 +2232,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
}
|
||||
|
||||
def handleEndTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
|
||||
ensureInterBrokerVersion(KAFKA_0_11_0_IV0)
|
||||
ensureInterBrokerVersion(IBP_0_11_0_IV0)
|
||||
val endTxnRequest = request.body[EndTxnRequest]
|
||||
val transactionalId = endTxnRequest.data.transactionalId
|
||||
|
||||
|
@ -2270,7 +2273,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
}
|
||||
|
||||
def handleWriteTxnMarkersRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
|
||||
ensureInterBrokerVersion(KAFKA_0_11_0_IV0)
|
||||
ensureInterBrokerVersion(IBP_0_11_0_IV0)
|
||||
authHelper.authorizeClusterOperation(request, CLUSTER_ACTION)
|
||||
val writeTxnMarkersRequest = request.body[WriteTxnMarkersRequest]
|
||||
val errors = new ConcurrentHashMap[java.lang.Long, util.Map[TopicPartition, Errors]]()
|
||||
|
@ -2375,13 +2378,13 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
requestHelper.sendResponseExemptThrottle(request, new WriteTxnMarkersResponse(errors))
|
||||
}
|
||||
|
||||
def ensureInterBrokerVersion(version: ApiVersion): Unit = {
|
||||
if (config.interBrokerProtocolVersion < version)
|
||||
def ensureInterBrokerVersion(version: MetadataVersion): Unit = {
|
||||
if (config.interBrokerProtocolVersion.isLessThan(version))
|
||||
throw new UnsupportedVersionException(s"inter.broker.protocol.version: ${config.interBrokerProtocolVersion.version} is less than the required version: ${version.version}")
|
||||
}
|
||||
|
||||
def handleAddPartitionToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
|
||||
ensureInterBrokerVersion(KAFKA_0_11_0_IV0)
|
||||
ensureInterBrokerVersion(IBP_0_11_0_IV0)
|
||||
val addPartitionsToTxnRequest = request.body[AddPartitionsToTxnRequest]
|
||||
val transactionalId = addPartitionsToTxnRequest.data.transactionalId
|
||||
val partitionsToAdd = addPartitionsToTxnRequest.partitions.asScala
|
||||
|
@ -2444,7 +2447,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
}
|
||||
|
||||
def handleAddOffsetsToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
|
||||
ensureInterBrokerVersion(KAFKA_0_11_0_IV0)
|
||||
ensureInterBrokerVersion(IBP_0_11_0_IV0)
|
||||
val addOffsetsToTxnRequest = request.body[AddOffsetsToTxnRequest]
|
||||
val transactionalId = addOffsetsToTxnRequest.data.transactionalId
|
||||
val groupId = addOffsetsToTxnRequest.data.groupId
|
||||
|
@ -2494,7 +2497,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
}
|
||||
|
||||
def handleTxnOffsetCommitRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
|
||||
ensureInterBrokerVersion(KAFKA_0_11_0_IV0)
|
||||
ensureInterBrokerVersion(IBP_0_11_0_IV0)
|
||||
val header = request.header
|
||||
val txnOffsetCommitRequest = request.body[TxnOffsetCommitRequest]
|
||||
|
||||
|
|
|
@ -19,7 +19,7 @@ package kafka.server
|
|||
|
||||
import java.util
|
||||
import java.util.{Collections, Locale, Properties}
|
||||
import kafka.api.{ApiVersion, ApiVersionValidator, KAFKA_0_10_0_IV1, KAFKA_2_1_IV0, KAFKA_2_7_IV0, KAFKA_2_8_IV0, KAFKA_3_0_IV1}
|
||||
|
||||
import kafka.cluster.EndPoint
|
||||
import kafka.coordinator.group.OffsetConfig
|
||||
import kafka.coordinator.transaction.{TransactionLog, TransactionStateManager}
|
||||
|
@ -47,6 +47,8 @@ import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuild
|
|||
import org.apache.kafka.common.utils.Utils
|
||||
import org.apache.kafka.raft.RaftConfig
|
||||
import org.apache.kafka.server.authorizer.Authorizer
|
||||
import org.apache.kafka.server.common.{MetadataVersion, MetadataVersionValidator}
|
||||
import org.apache.kafka.server.common.MetadataVersion._
|
||||
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
|
||||
import org.apache.zookeeper.client.ZKClientConfig
|
||||
|
||||
|
@ -141,7 +143,7 @@ object Defaults {
|
|||
|
||||
/* See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for details */
|
||||
@deprecated("3.0")
|
||||
val LogMessageFormatVersion = KAFKA_3_0_IV1.version
|
||||
val LogMessageFormatVersion = IBP_3_0_IV1.version
|
||||
|
||||
val LogMessageTimestampType = "CreateTime"
|
||||
val LogMessageTimestampDifferenceMaxMs = Long.MaxValue
|
||||
|
@ -172,7 +174,7 @@ object Defaults {
|
|||
val LeaderImbalanceCheckIntervalSeconds = 300
|
||||
val UncleanLeaderElectionEnable = false
|
||||
val InterBrokerSecurityProtocol = SecurityProtocol.PLAINTEXT.toString
|
||||
val InterBrokerProtocolVersion = ApiVersion.latestVersion.toString
|
||||
val InterBrokerProtocolVersion = MetadataVersion.latest.version
|
||||
|
||||
/** ********* Controlled shutdown configuration ***********/
|
||||
val ControlledShutdownMaxRetries = 3
|
||||
|
@ -841,8 +843,8 @@ object KafkaConfig {
|
|||
val LogFlushOffsetCheckpointIntervalMsDoc = "The frequency with which we update the persistent record of the last flush which acts as the log recovery point"
|
||||
val LogFlushStartOffsetCheckpointIntervalMsDoc = "The frequency with which we update the persistent record of log start offset"
|
||||
val LogPreAllocateEnableDoc = "Should pre allocate file when create new segment? If you are using Kafka on Windows, you probably need to set it to true."
|
||||
val LogMessageFormatVersionDoc = "Specify the message format version the broker will use to append messages to the logs. The value should be a valid ApiVersion. " +
|
||||
"Some examples are: 0.8.2, 0.9.0.0, 0.10.0, check ApiVersion for more details. By setting a particular message format version, the " +
|
||||
val LogMessageFormatVersionDoc = "Specify the message format version the broker will use to append messages to the logs. The value should be a valid MetadataVersion. " +
|
||||
"Some examples are: 0.8.2, 0.9.0.0, 0.10.0, check MetadataVersion for more details. By setting a particular message format version, the " +
|
||||
"user is certifying that all the existing messages on disk are smaller or equal than the specified version. Setting this value incorrectly " +
|
||||
"will cause consumers with older versions to break as they will receive messages with a format that they don't understand."
|
||||
|
||||
|
@ -907,7 +909,7 @@ object KafkaConfig {
|
|||
"properties at the same time."
|
||||
val InterBrokerProtocolVersionDoc = "Specify which version of the inter-broker protocol will be used.\n" +
|
||||
" This is typically bumped after all brokers were upgraded to a new version.\n" +
|
||||
" Example of some valid values are: 0.8.0, 0.8.1, 0.8.1.1, 0.8.2, 0.8.2.0, 0.8.2.1, 0.9.0.0, 0.9.0.1 Check ApiVersion for the full list."
|
||||
" Example of some valid values are: 0.8.0, 0.8.1, 0.8.1.1, 0.8.2, 0.8.2.0, 0.8.2.1, 0.9.0.0, 0.9.0.1 Check MetadataVersion for the full list."
|
||||
val InterBrokerListenerNameDoc = s"Name of listener used for communication between brokers. If this is unset, the listener name is defined by $InterBrokerSecurityProtocolProp. " +
|
||||
s"It is an error to set this and $InterBrokerSecurityProtocolProp properties at the same time."
|
||||
val ReplicaSelectorClassDoc = "The fully qualified class name that implements ReplicaSelector. This is used by the broker to find the preferred read replica. By default, we use an implementation that returns the leader."
|
||||
|
@ -1202,7 +1204,7 @@ object KafkaConfig {
|
|||
.define(NumRecoveryThreadsPerDataDirProp, INT, Defaults.NumRecoveryThreadsPerDataDir, atLeast(1), HIGH, NumRecoveryThreadsPerDataDirDoc)
|
||||
.define(AutoCreateTopicsEnableProp, BOOLEAN, Defaults.AutoCreateTopicsEnable, HIGH, AutoCreateTopicsEnableDoc)
|
||||
.define(MinInSyncReplicasProp, INT, Defaults.MinInSyncReplicas, atLeast(1), HIGH, MinInSyncReplicasDoc)
|
||||
.define(LogMessageFormatVersionProp, STRING, Defaults.LogMessageFormatVersion, ApiVersionValidator, MEDIUM, LogMessageFormatVersionDoc)
|
||||
.define(LogMessageFormatVersionProp, STRING, Defaults.LogMessageFormatVersion, new MetadataVersionValidator(), MEDIUM, LogMessageFormatVersionDoc)
|
||||
.define(LogMessageTimestampTypeProp, STRING, Defaults.LogMessageTimestampType, in("CreateTime", "LogAppendTime"), MEDIUM, LogMessageTimestampTypeDoc)
|
||||
.define(LogMessageTimestampDifferenceMaxMsProp, LONG, Defaults.LogMessageTimestampDifferenceMaxMs, atLeast(0), MEDIUM, LogMessageTimestampDifferenceMaxMsDoc)
|
||||
.define(CreateTopicPolicyClassNameProp, CLASS, null, LOW, CreateTopicPolicyClassNameDoc)
|
||||
|
@ -1230,7 +1232,7 @@ object KafkaConfig {
|
|||
.define(LeaderImbalanceCheckIntervalSecondsProp, LONG, Defaults.LeaderImbalanceCheckIntervalSeconds, atLeast(1), HIGH, LeaderImbalanceCheckIntervalSecondsDoc)
|
||||
.define(UncleanLeaderElectionEnableProp, BOOLEAN, Defaults.UncleanLeaderElectionEnable, HIGH, UncleanLeaderElectionEnableDoc)
|
||||
.define(InterBrokerSecurityProtocolProp, STRING, Defaults.InterBrokerSecurityProtocol, MEDIUM, InterBrokerSecurityProtocolDoc)
|
||||
.define(InterBrokerProtocolVersionProp, STRING, Defaults.InterBrokerProtocolVersion, ApiVersionValidator, MEDIUM, InterBrokerProtocolVersionDoc)
|
||||
.define(InterBrokerProtocolVersionProp, STRING, Defaults.InterBrokerProtocolVersion, new MetadataVersionValidator(), MEDIUM, InterBrokerProtocolVersionDoc)
|
||||
.define(InterBrokerListenerNameProp, STRING, null, MEDIUM, InterBrokerListenerNameDoc)
|
||||
.define(ReplicaSelectorClassProp, STRING, null, MEDIUM, ReplicaSelectorClassDoc)
|
||||
|
||||
|
@ -1712,7 +1714,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
|
|||
def minInSyncReplicas = getInt(KafkaConfig.MinInSyncReplicasProp)
|
||||
def logPreAllocateEnable: java.lang.Boolean = getBoolean(KafkaConfig.LogPreAllocateProp)
|
||||
|
||||
// We keep the user-provided String as `ApiVersion.apply` can choose a slightly different version (eg if `0.10.0`
|
||||
// We keep the user-provided String as `MetadataVersion.fromVersionString` can choose a slightly different version (eg if `0.10.0`
|
||||
// is passed, `0.10.0-IV0` may be picked)
|
||||
@nowarn("cat=deprecation")
|
||||
private val logMessageFormatVersionString = getString(KafkaConfig.LogMessageFormatVersionProp)
|
||||
|
@ -1721,8 +1723,8 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
|
|||
@deprecated("3.0")
|
||||
lazy val logMessageFormatVersion =
|
||||
if (LogConfig.shouldIgnoreMessageFormatVersion(interBrokerProtocolVersion))
|
||||
ApiVersion(Defaults.LogMessageFormatVersion)
|
||||
else ApiVersion(logMessageFormatVersionString)
|
||||
MetadataVersion.fromVersionString(Defaults.LogMessageFormatVersion)
|
||||
else MetadataVersion.fromVersionString(logMessageFormatVersionString)
|
||||
|
||||
def logMessageTimestampType = TimestampType.forName(getString(KafkaConfig.LogMessageTimestampTypeProp))
|
||||
def logMessageTimestampDifferenceMaxMs: Long = getLong(KafkaConfig.LogMessageTimestampDifferenceMaxMsProp)
|
||||
|
@ -1749,10 +1751,10 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
|
|||
val leaderImbalanceCheckIntervalSeconds: Long = getLong(KafkaConfig.LeaderImbalanceCheckIntervalSecondsProp)
|
||||
def uncleanLeaderElectionEnable: java.lang.Boolean = getBoolean(KafkaConfig.UncleanLeaderElectionEnableProp)
|
||||
|
||||
// We keep the user-provided String as `ApiVersion.apply` can choose a slightly different version (eg if `0.10.0`
|
||||
// We keep the user-provided String as `MetadataVersion.fromVersionString` can choose a slightly different version (eg if `0.10.0`
|
||||
// is passed, `0.10.0-IV0` may be picked)
|
||||
val interBrokerProtocolVersionString = getString(KafkaConfig.InterBrokerProtocolVersionProp)
|
||||
val interBrokerProtocolVersion = ApiVersion(interBrokerProtocolVersionString)
|
||||
val interBrokerProtocolVersion = MetadataVersion.fromVersionString(interBrokerProtocolVersionString)
|
||||
|
||||
/** ********* Controlled shutdown configuration ***********/
|
||||
val controlledShutdownMaxRetries = getInt(KafkaConfig.ControlledShutdownMaxRetriesProp)
|
||||
|
@ -1760,7 +1762,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
|
|||
val controlledShutdownEnable = getBoolean(KafkaConfig.ControlledShutdownEnableProp)
|
||||
|
||||
/** ********* Feature configuration ***********/
|
||||
def isFeatureVersioningSupported = interBrokerProtocolVersion >= KAFKA_2_7_IV0
|
||||
def isFeatureVersioningSupported = interBrokerProtocolVersion.isFeatureVersioningSupported()
|
||||
|
||||
/** ********* Group coordinator configuration ***********/
|
||||
val groupMinSessionTimeoutMs = getInt(KafkaConfig.GroupMinSessionTimeoutMsProp)
|
||||
|
@ -1812,7 +1814,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
|
|||
def controlPlaneListenerName = getControlPlaneListenerNameAndSecurityProtocol.map { case (listenerName, _) => listenerName }
|
||||
def controlPlaneSecurityProtocol = getControlPlaneListenerNameAndSecurityProtocol.map { case (_, securityProtocol) => securityProtocol }
|
||||
def saslMechanismInterBrokerProtocol = getString(KafkaConfig.SaslMechanismInterBrokerProtocolProp)
|
||||
val saslInterBrokerHandshakeRequestEnable = interBrokerProtocolVersion >= KAFKA_0_10_0_IV1
|
||||
val saslInterBrokerHandshakeRequestEnable = interBrokerProtocolVersion.isSaslInterBrokerHandshakeRequestEnabled()
|
||||
|
||||
/** ********* DelegationToken Configuration **************/
|
||||
val delegationTokenSecretKey = Option(getPassword(KafkaConfig.DelegationTokenSecretKeyProp))
|
||||
|
@ -1993,7 +1995,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
|
|||
|
||||
// Topic IDs are used with all self-managed quorum clusters and ZK cluster with IBP greater than or equal to 2.8
|
||||
def usesTopicId: Boolean =
|
||||
usesSelfManagedQuorum || interBrokerProtocolVersion >= KAFKA_2_8_IV0
|
||||
usesSelfManagedQuorum || interBrokerProtocolVersion.isTopicIdsSupported()
|
||||
|
||||
validateValues()
|
||||
|
||||
|
@ -2157,15 +2159,15 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
|
|||
if (messageFormatVersion.shouldWarn)
|
||||
warn(messageFormatVersion.brokerWarningMessage)
|
||||
|
||||
val recordVersion = logMessageFormatVersion.recordVersion
|
||||
require(interBrokerProtocolVersion.recordVersion.value >= recordVersion.value,
|
||||
val recordVersion = logMessageFormatVersion.highestSupportedRecordVersion
|
||||
require(interBrokerProtocolVersion.highestSupportedRecordVersion().value >= recordVersion.value,
|
||||
s"log.message.format.version $logMessageFormatVersionString can only be used when inter.broker.protocol.version " +
|
||||
s"is set to version ${ApiVersion.minSupportedFor(recordVersion).shortVersion} or higher")
|
||||
s"is set to version ${MetadataVersion.minSupportedFor(recordVersion).shortVersion} or higher")
|
||||
|
||||
if (offsetsTopicCompressionCodec == ZStdCompressionCodec)
|
||||
require(interBrokerProtocolVersion.recordVersion.value >= KAFKA_2_1_IV0.recordVersion.value,
|
||||
require(interBrokerProtocolVersion.highestSupportedRecordVersion().value >= IBP_2_1_IV0.highestSupportedRecordVersion().value,
|
||||
"offsets.topic.compression.codec zstd can only be used when inter.broker.protocol.version " +
|
||||
s"is set to version ${KAFKA_2_1_IV0.shortVersion} or higher")
|
||||
s"is set to version ${IBP_2_1_IV0.shortVersion} or higher")
|
||||
|
||||
val interBrokerUsesSasl = interBrokerSecurityProtocol == SecurityProtocol.SASL_PLAINTEXT || interBrokerSecurityProtocol == SecurityProtocol.SASL_SSL
|
||||
require(!interBrokerUsesSasl || saslInterBrokerHandshakeRequestEnable || saslMechanismInterBrokerProtocol == SaslConfigs.GSSAPI_MECHANISM,
|
||||
|
|
|
@ -22,7 +22,6 @@ import java.net.{InetAddress, SocketTimeoutException}
|
|||
import java.util.concurrent._
|
||||
import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger}
|
||||
|
||||
import kafka.api.{KAFKA_0_9_0, KAFKA_2_2_IV0, KAFKA_2_4_IV1}
|
||||
import kafka.cluster.{Broker, EndPoint}
|
||||
import kafka.common.{GenerateBrokerIdException, InconsistentBrokerIdException, InconsistentClusterIdException}
|
||||
import kafka.controller.KafkaController
|
||||
|
@ -50,6 +49,7 @@ import org.apache.kafka.common.utils.{AppInfoParser, LogContext, Time, Utils}
|
|||
import org.apache.kafka.common.{Endpoint, Node}
|
||||
import org.apache.kafka.metadata.BrokerState
|
||||
import org.apache.kafka.server.authorizer.Authorizer
|
||||
import org.apache.kafka.server.common.MetadataVersion._
|
||||
import org.apache.kafka.server.metrics.KafkaYammerMetrics
|
||||
import org.apache.zookeeper.client.ZKClientConfig
|
||||
|
||||
|
@ -627,9 +627,9 @@ class KafkaServer(
|
|||
|
||||
// send the controlled shutdown request
|
||||
val controlledShutdownApiVersion: Short =
|
||||
if (config.interBrokerProtocolVersion < KAFKA_0_9_0) 0
|
||||
else if (config.interBrokerProtocolVersion < KAFKA_2_2_IV0) 1
|
||||
else if (config.interBrokerProtocolVersion < KAFKA_2_4_IV1) 2
|
||||
if (config.interBrokerProtocolVersion.isLessThan(IBP_0_9_0)) 0
|
||||
else if (config.interBrokerProtocolVersion.isLessThan(IBP_2_2_IV0)) 1
|
||||
else if (config.interBrokerProtocolVersion.isLessThan(IBP_2_4_IV1)) 2
|
||||
else 3
|
||||
|
||||
val controlledShutdownRequest = new ControlledShutdownRequest.Builder(
|
||||
|
|
|
@ -20,7 +20,6 @@ package kafka.server
|
|||
import java.util.Collections
|
||||
import java.util.Optional
|
||||
|
||||
import kafka.api._
|
||||
import kafka.cluster.BrokerEndPoint
|
||||
import kafka.log.{LeaderOffsetIncremented, LogAppendInfo}
|
||||
import kafka.server.AbstractFetcherThread.ReplicaFetch
|
||||
|
@ -38,6 +37,7 @@ import org.apache.kafka.common.protocol.Errors
|
|||
import org.apache.kafka.common.record.MemoryRecords
|
||||
import org.apache.kafka.common.requests._
|
||||
import org.apache.kafka.common.utils.{LogContext, Time}
|
||||
import org.apache.kafka.server.common.MetadataVersion._
|
||||
|
||||
import scala.jdk.CollectionConverters._
|
||||
import scala.collection.{Map, mutable}
|
||||
|
@ -72,44 +72,44 @@ class ReplicaFetcherThread(name: String,
|
|||
|
||||
// Visible for testing
|
||||
private[server] val fetchRequestVersion: Short =
|
||||
if (brokerConfig.interBrokerProtocolVersion >= KAFKA_3_1_IV0) 13
|
||||
else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_2_7_IV1) 12
|
||||
else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_2_3_IV1) 11
|
||||
else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_2_1_IV2) 10
|
||||
else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_2_0_IV1) 8
|
||||
else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_1_1_IV0) 7
|
||||
else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_11_0_IV1) 5
|
||||
else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_11_0_IV0) 4
|
||||
else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_10_1_IV1) 3
|
||||
else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_10_0_IV0) 2
|
||||
else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_9_0) 1
|
||||
if (brokerConfig.interBrokerProtocolVersion.isAtLeast(IBP_3_1_IV0)) 13
|
||||
else if (brokerConfig.interBrokerProtocolVersion.isAtLeast(IBP_2_7_IV1)) 12
|
||||
else if (brokerConfig.interBrokerProtocolVersion.isAtLeast(IBP_2_3_IV1)) 11
|
||||
else if (brokerConfig.interBrokerProtocolVersion.isAtLeast(IBP_2_1_IV2)) 10
|
||||
else if (brokerConfig.interBrokerProtocolVersion.isAtLeast(IBP_2_0_IV1)) 8
|
||||
else if (brokerConfig.interBrokerProtocolVersion.isAtLeast(IBP_1_1_IV0)) 7
|
||||
else if (brokerConfig.interBrokerProtocolVersion.isAtLeast(IBP_0_11_0_IV1)) 5
|
||||
else if (brokerConfig.interBrokerProtocolVersion.isAtLeast(IBP_0_11_0_IV0)) 4
|
||||
else if (brokerConfig.interBrokerProtocolVersion.isAtLeast(IBP_0_10_1_IV1)) 3
|
||||
else if (brokerConfig.interBrokerProtocolVersion.isAtLeast(IBP_0_10_0_IV0)) 2
|
||||
else if (brokerConfig.interBrokerProtocolVersion.isAtLeast(IBP_0_9_0)) 1
|
||||
else 0
|
||||
|
||||
// Visible for testing
|
||||
private[server] val offsetForLeaderEpochRequestVersion: Short =
|
||||
if (brokerConfig.interBrokerProtocolVersion >= KAFKA_2_8_IV0) 4
|
||||
else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_2_3_IV1) 3
|
||||
else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_2_1_IV1) 2
|
||||
else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_2_0_IV0) 1
|
||||
if (brokerConfig.interBrokerProtocolVersion.isAtLeast(IBP_2_8_IV0)) 4
|
||||
else if (brokerConfig.interBrokerProtocolVersion.isAtLeast(IBP_2_3_IV1)) 3
|
||||
else if (brokerConfig.interBrokerProtocolVersion.isAtLeast(IBP_2_1_IV1)) 2
|
||||
else if (brokerConfig.interBrokerProtocolVersion.isAtLeast(IBP_2_0_IV0)) 1
|
||||
else 0
|
||||
|
||||
// Visible for testing
|
||||
private[server] val listOffsetRequestVersion: Short =
|
||||
if (brokerConfig.interBrokerProtocolVersion >= KAFKA_3_0_IV1) 7
|
||||
else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_2_8_IV0) 6
|
||||
else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_2_2_IV1) 5
|
||||
else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_2_1_IV1) 4
|
||||
else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_2_0_IV1) 3
|
||||
else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_11_0_IV0) 2
|
||||
else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_10_1_IV2) 1
|
||||
if (brokerConfig.interBrokerProtocolVersion.isAtLeast(IBP_3_0_IV1)) 7
|
||||
else if (brokerConfig.interBrokerProtocolVersion.isAtLeast(IBP_2_8_IV0)) 6
|
||||
else if (brokerConfig.interBrokerProtocolVersion.isAtLeast(IBP_2_2_IV1)) 5
|
||||
else if (brokerConfig.interBrokerProtocolVersion.isAtLeast(IBP_2_1_IV1)) 4
|
||||
else if (brokerConfig.interBrokerProtocolVersion.isAtLeast(IBP_2_0_IV1)) 3
|
||||
else if (brokerConfig.interBrokerProtocolVersion.isAtLeast(IBP_0_11_0_IV0)) 2
|
||||
else if (brokerConfig.interBrokerProtocolVersion.isAtLeast(IBP_0_10_1_IV2)) 1
|
||||
else 0
|
||||
|
||||
private val maxWait = brokerConfig.replicaFetchWaitMaxMs
|
||||
private val minBytes = brokerConfig.replicaFetchMinBytes
|
||||
private val maxBytes = brokerConfig.replicaFetchResponseMaxBytes
|
||||
private val fetchSize = brokerConfig.replicaFetchMaxBytes
|
||||
override protected val isOffsetForLeaderEpochSupported: Boolean = brokerConfig.interBrokerProtocolVersion >= KAFKA_0_11_0_IV2
|
||||
override protected val isTruncationOnFetchSupported = ApiVersion.isTruncationOnFetchSupported(brokerConfig.interBrokerProtocolVersion)
|
||||
override protected val isOffsetForLeaderEpochSupported: Boolean = brokerConfig.interBrokerProtocolVersion.isOffsetForLeaderEpochSupported
|
||||
override protected val isTruncationOnFetchSupported = brokerConfig.interBrokerProtocolVersion.isTruncationOnFetchSupported
|
||||
val fetchSessionHandler = new FetchSessionHandler(logContext, sourceBroker.id)
|
||||
|
||||
override protected def latestEpoch(topicPartition: TopicPartition): Option[Int] = {
|
||||
|
@ -261,7 +261,7 @@ class ReplicaFetcherThread(name: String,
|
|||
|
||||
Errors.forCode(responsePartition.errorCode) match {
|
||||
case Errors.NONE =>
|
||||
if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_10_1_IV2)
|
||||
if (brokerConfig.interBrokerProtocolVersion.isAtLeast(IBP_0_10_1_IV2))
|
||||
responsePartition.offset
|
||||
else
|
||||
responsePartition.oldStyleOffsets.get(0)
|
||||
|
|
|
@ -21,6 +21,7 @@ import java.util.Optional
|
|||
import java.util.concurrent.TimeUnit
|
||||
import java.util.concurrent.atomic.AtomicBoolean
|
||||
import java.util.concurrent.locks.Lock
|
||||
|
||||
import com.yammer.metrics.core.Meter
|
||||
import kafka.api._
|
||||
import kafka.cluster.{BrokerEndPoint, Partition}
|
||||
|
@ -60,6 +61,7 @@ import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
|
|||
import org.apache.kafka.common.requests._
|
||||
import org.apache.kafka.common.utils.Time
|
||||
import org.apache.kafka.image.{LocalReplicaChanges, MetadataImage, TopicsDelta}
|
||||
import org.apache.kafka.server.common.MetadataVersion._
|
||||
|
||||
import scala.jdk.CollectionConverters._
|
||||
import scala.collection.{Map, Seq, Set, mutable}
|
||||
|
@ -307,7 +309,7 @@ class ReplicaManager(val config: KafkaConfig,
|
|||
// If inter-broker protocol (IBP) < 1.0, the controller will send LeaderAndIsrRequest V0 which does not include isNew field.
|
||||
// In this case, the broker receiving the request cannot determine whether it is safe to create a partition if a log directory has failed.
|
||||
// Thus, we choose to halt the broker on any log directory failure if IBP < 1.0
|
||||
val haltBrokerOnFailure = config.interBrokerProtocolVersion < KAFKA_1_0_IV0
|
||||
val haltBrokerOnFailure = config.interBrokerProtocolVersion.isLessThan(IBP_1_0_IV0)
|
||||
logDirFailureHandler = new LogDirFailureHandler("LogDirFailureHandler", haltBrokerOnFailure)
|
||||
logDirFailureHandler.start()
|
||||
}
|
||||
|
@ -1806,7 +1808,7 @@ class ReplicaManager(val config: KafkaConfig,
|
|||
* OffsetForLeaderEpoch request.
|
||||
*/
|
||||
protected def initialFetchOffset(log: UnifiedLog): Long = {
|
||||
if (ApiVersion.isTruncationOnFetchSupported(config.interBrokerProtocolVersion) && log.latestEpoch.nonEmpty)
|
||||
if (config.interBrokerProtocolVersion.isTruncationOnFetchSupported() && log.latestEpoch.nonEmpty)
|
||||
log.logEndOffset
|
||||
else
|
||||
log.highWatermark
|
||||
|
|
|
@ -19,9 +19,10 @@ package kafka.zk
|
|||
import java.nio.charset.StandardCharsets.UTF_8
|
||||
import java.util
|
||||
import java.util.Properties
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonProperty
|
||||
import com.fasterxml.jackson.core.JsonProcessingException
|
||||
import kafka.api.{ApiVersion, KAFKA_0_10_0_IV1, KAFKA_2_7_IV0, LeaderAndIsr}
|
||||
import kafka.api.LeaderAndIsr
|
||||
import kafka.cluster.{Broker, EndPoint}
|
||||
import kafka.common.{NotificationHandler, ZkNodeChangeNotificationListener}
|
||||
import kafka.controller.{IsrChangeNotificationHandler, LeaderIsrAndControllerEpoch, ReplicaAssignment}
|
||||
|
@ -40,7 +41,8 @@ import org.apache.kafka.common.security.token.delegation.{DelegationToken, Token
|
|||
import org.apache.kafka.common.utils.{SecurityUtils, Time}
|
||||
import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid}
|
||||
import org.apache.kafka.metadata.LeaderRecoveryState
|
||||
import org.apache.kafka.server.common.ProducerIdsBlock
|
||||
import org.apache.kafka.server.common.{MetadataVersion, ProducerIdsBlock}
|
||||
import org.apache.kafka.server.common.MetadataVersion.{IBP_0_10_0_IV1, IBP_2_7_IV0}
|
||||
import org.apache.zookeeper.ZooDefs
|
||||
import org.apache.zookeeper.data.{ACL, Stat}
|
||||
|
||||
|
@ -84,9 +86,9 @@ object BrokerIdsZNode {
|
|||
object BrokerInfo {
|
||||
|
||||
/**
|
||||
* - Create a broker info with v5 json format if the apiVersion is 2.7.x or above.
|
||||
* - Create a broker info with v5 json format if the metadataVersion is 2.7.x or above.
|
||||
* - Create a broker info with v4 json format (which includes multiple endpoints and rack) if
|
||||
* the apiVersion is 0.10.0.X or above but lesser than 2.7.x.
|
||||
* the metadataVersion is 0.10.0.X or above but lesser than 2.7.x.
|
||||
* - Register the broker with v2 json format otherwise.
|
||||
*
|
||||
* Due to KAFKA-3100, 0.9.0.0 broker and old clients will break if JSON version is above 2.
|
||||
|
@ -95,11 +97,11 @@ object BrokerInfo {
|
|||
* without having to upgrade to 0.9.0.1 first (clients have to be upgraded to 0.9.0.1 in
|
||||
* any case).
|
||||
*/
|
||||
def apply(broker: Broker, apiVersion: ApiVersion, jmxPort: Int): BrokerInfo = {
|
||||
def apply(broker: Broker, metadataVersion: MetadataVersion, jmxPort: Int): BrokerInfo = {
|
||||
val version = {
|
||||
if (apiVersion >= KAFKA_2_7_IV0)
|
||||
if (metadataVersion.isAtLeast(IBP_2_7_IV0))
|
||||
5
|
||||
else if (apiVersion >= KAFKA_0_10_0_IV1)
|
||||
else if (metadataVersion.isAtLeast(IBP_0_10_0_IV1))
|
||||
4
|
||||
else
|
||||
2
|
||||
|
@ -846,12 +848,12 @@ object DelegationTokenInfoZNode {
|
|||
* Enabled -> This status means the feature versioning system (KIP-584) is enabled, and, the
|
||||
* finalized features stored in the FeatureZNode are active. This status is written by
|
||||
* the controller to the FeatureZNode only when the broker IBP config is greater than
|
||||
* or equal to KAFKA_2_7_IV0.
|
||||
* or equal to IBP_2_7_IV0.
|
||||
*
|
||||
* Disabled -> This status means the feature versioning system (KIP-584) is disabled, and, the
|
||||
* the finalized features stored in the FeatureZNode is not relevant. This status is
|
||||
* written by the controller to the FeatureZNode only when the broker IBP config
|
||||
* is less than KAFKA_2_7_IV0.
|
||||
* is less than IBP_2_7_IV0.
|
||||
*/
|
||||
sealed trait FeatureZNodeStatus {
|
||||
def id: Int
|
||||
|
|
|
@ -17,7 +17,6 @@
|
|||
package kafka.admin
|
||||
|
||||
import kafka.admin.ConfigCommand.ConfigCommandOptions
|
||||
import kafka.api.ApiVersion
|
||||
import kafka.cluster.{Broker, EndPoint}
|
||||
import kafka.server.{ConfigEntityName, KafkaConfig, QuorumTestHarness}
|
||||
import kafka.utils.{Exit, Logging, TestInfoUtils}
|
||||
|
@ -25,6 +24,7 @@ import kafka.zk.{AdminZkClient, BrokerInfo}
|
|||
import org.apache.kafka.common.config.ConfigException
|
||||
import org.apache.kafka.common.network.ListenerName
|
||||
import org.apache.kafka.common.security.auth.SecurityProtocol
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
import org.junit.jupiter.params.provider.ValueSource
|
||||
|
@ -173,7 +173,7 @@ class ConfigCommandIntegrationTest extends QuorumTestHarness with Logging {
|
|||
zkClient.createTopLevelPaths()
|
||||
val securityProtocol = SecurityProtocol.PLAINTEXT
|
||||
val endpoint = new EndPoint("localhost", 9092, ListenerName.forSecurityProtocol(securityProtocol), securityProtocol)
|
||||
val brokerInfo = BrokerInfo(Broker(id, Seq(endpoint), rack = None), ApiVersion.latestVersion, jmxPort = 9192)
|
||||
val brokerInfo = BrokerInfo(Broker(id, Seq(endpoint), rack = None), MetadataVersion.latest, jmxPort = 9192)
|
||||
zkClient.registerBroker(brokerInfo)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,8 +19,8 @@ package kafka.admin
|
|||
|
||||
import java.io.Closeable
|
||||
import java.util.{Collections, HashMap, List}
|
||||
|
||||
import kafka.admin.ReassignPartitionsCommand._
|
||||
import kafka.api.KAFKA_2_7_IV1
|
||||
import kafka.server.{IsrChangePropagationConfig, KafkaConfig, KafkaServer, ZkAlterPartitionManager}
|
||||
import kafka.utils.Implicits._
|
||||
import kafka.utils.TestUtils
|
||||
|
@ -30,6 +30,7 @@ import org.apache.kafka.clients.producer.ProducerRecord
|
|||
import org.apache.kafka.common.config.ConfigResource
|
||||
import org.apache.kafka.common.utils.Utils
|
||||
import org.apache.kafka.common.{TopicPartition, TopicPartitionReplica}
|
||||
import org.apache.kafka.server.common.MetadataVersion.IBP_2_7_IV1
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue}
|
||||
import org.junit.jupiter.api.{AfterEach, Test, Timeout}
|
||||
|
||||
|
@ -66,7 +67,7 @@ class ReassignPartitionsIntegrationTest extends QuorumTestHarness {
|
|||
// the `AlterIsr` API. In this case, the controller will register individual
|
||||
// watches for each reassigning partition so that the reassignment can be
|
||||
// completed as soon as the ISR is expanded.
|
||||
val configOverrides = Map(KafkaConfig.InterBrokerProtocolVersionProp -> KAFKA_2_7_IV1.version)
|
||||
val configOverrides = Map(KafkaConfig.InterBrokerProtocolVersionProp -> IBP_2_7_IV1.version)
|
||||
cluster = new ReassignPartitionsTestCluster(zkConnect, configOverrides = configOverrides)
|
||||
cluster.setup()
|
||||
executeAndVerifyReassignment()
|
||||
|
@ -89,7 +90,7 @@ class ReassignPartitionsIntegrationTest extends QuorumTestHarness {
|
|||
maxDelayMs = 500
|
||||
)
|
||||
|
||||
val oldIbpConfig = Map(KafkaConfig.InterBrokerProtocolVersionProp -> KAFKA_2_7_IV1.version)
|
||||
val oldIbpConfig = Map(KafkaConfig.InterBrokerProtocolVersionProp -> IBP_2_7_IV1.version)
|
||||
val brokerConfigOverrides = Map(1 -> oldIbpConfig, 2 -> oldIbpConfig, 3 -> oldIbpConfig)
|
||||
|
||||
cluster = new ReassignPartitionsTestCluster(zkConnect, brokerConfigOverrides = brokerConfigOverrides)
|
||||
|
|
|
@ -19,11 +19,13 @@ package integration.kafka.server
|
|||
|
||||
import java.time.Duration
|
||||
import java.util.Arrays.asList
|
||||
import kafka.api.{ApiVersion, DefaultApiVersion, KAFKA_2_7_IV0, KAFKA_2_8_IV1, KAFKA_3_1_IV0}
|
||||
|
||||
import kafka.server.{BaseRequestTest, KafkaConfig}
|
||||
import kafka.utils.TestUtils
|
||||
import org.apache.kafka.clients.producer.ProducerRecord
|
||||
import org.apache.kafka.common.TopicPartition
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.apache.kafka.server.common.MetadataVersion.{IBP_2_7_IV0, IBP_2_8_IV1, IBP_3_1_IV0}
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
|
@ -35,26 +37,26 @@ class FetchRequestBetweenDifferentIbpTest extends BaseRequestTest {
|
|||
override def generateConfigs: Seq[KafkaConfig] = {
|
||||
// Brokers should be at most 2 different IBP versions, but for more test coverage, three are used here.
|
||||
Seq(
|
||||
createConfig(0, KAFKA_2_7_IV0),
|
||||
createConfig(1, KAFKA_2_8_IV1),
|
||||
createConfig(2, KAFKA_3_1_IV0)
|
||||
createConfig(0, IBP_2_7_IV0),
|
||||
createConfig(1, IBP_2_8_IV1),
|
||||
createConfig(2, IBP_3_1_IV0)
|
||||
)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testControllerOldIBP(): Unit = {
|
||||
// Ensure controller version < KAFKA_2_8_IV1, and then create a topic where leader of partition 0 is not the controller,
|
||||
// Ensure controller version < IBP_2_8_IV1, and then create a topic where leader of partition 0 is not the controller,
|
||||
// leader of partition 1 is.
|
||||
testControllerWithGivenIBP(KAFKA_2_7_IV0, 0)
|
||||
testControllerWithGivenIBP(IBP_2_7_IV0, 0)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testControllerNewIBP(): Unit = {
|
||||
// Ensure controller version = KAFKA_3_1_IV0, and then create a topic where leader of partition 1 is the old version.
|
||||
testControllerWithGivenIBP(KAFKA_3_1_IV0, 2)
|
||||
// Ensure controller version = IBP_3_1_IV0, and then create a topic where leader of partition 1 is the old version.
|
||||
testControllerWithGivenIBP(IBP_3_1_IV0, 2)
|
||||
}
|
||||
|
||||
def testControllerWithGivenIBP(version: DefaultApiVersion, controllerBroker: Int): Unit = {
|
||||
def testControllerWithGivenIBP(version: MetadataVersion, controllerBroker: Int): Unit = {
|
||||
val topic = "topic"
|
||||
val producer = createProducer()
|
||||
val consumer = createConsumer()
|
||||
|
@ -79,16 +81,16 @@ class FetchRequestBetweenDifferentIbpTest extends BaseRequestTest {
|
|||
|
||||
@Test
|
||||
def testControllerNewToOldIBP(): Unit = {
|
||||
testControllerSwitchingIBP(KAFKA_3_1_IV0, 2, KAFKA_2_7_IV0, 0)
|
||||
testControllerSwitchingIBP(IBP_3_1_IV0, 2, IBP_2_7_IV0, 0)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testControllerOldToNewIBP(): Unit = {
|
||||
testControllerSwitchingIBP(KAFKA_2_7_IV0, 0, KAFKA_3_1_IV0, 2)
|
||||
testControllerSwitchingIBP(IBP_2_7_IV0, 0, IBP_3_1_IV0, 2)
|
||||
}
|
||||
|
||||
|
||||
def testControllerSwitchingIBP(version1: DefaultApiVersion, broker1: Int, version2: DefaultApiVersion, broker2: Int): Unit = {
|
||||
def testControllerSwitchingIBP(version1: MetadataVersion, broker1: Int, version2: MetadataVersion, broker2: Int): Unit = {
|
||||
val topic = "topic"
|
||||
val topic2 = "topic2"
|
||||
val producer = createProducer()
|
||||
|
@ -132,7 +134,7 @@ class FetchRequestBetweenDifferentIbpTest extends BaseRequestTest {
|
|||
assertEquals(2, count2)
|
||||
}
|
||||
|
||||
private def ensureControllerWithIBP(version: DefaultApiVersion): Unit = {
|
||||
private def ensureControllerWithIBP(version: MetadataVersion): Unit = {
|
||||
val nonControllerServers = servers.filter(_.config.interBrokerProtocolVersion != version)
|
||||
nonControllerServers.iterator.foreach(server => {
|
||||
server.shutdown()
|
||||
|
@ -143,7 +145,7 @@ class FetchRequestBetweenDifferentIbpTest extends BaseRequestTest {
|
|||
})
|
||||
}
|
||||
|
||||
private def createConfig(nodeId: Int, interBrokerVersion: ApiVersion): KafkaConfig = {
|
||||
private def createConfig(nodeId: Int, interBrokerVersion: MetadataVersion): KafkaConfig = {
|
||||
val props = TestUtils.createBrokerConfig(nodeId, zkConnect)
|
||||
props.put(KafkaConfig.InterBrokerProtocolVersionProp, interBrokerVersion.version)
|
||||
KafkaConfig.fromProps(props)
|
||||
|
|
|
@ -20,12 +20,13 @@ package integration.kafka.server
|
|||
import java.time.Duration
|
||||
import java.util.Arrays.asList
|
||||
|
||||
import kafka.api.{ApiVersion, KAFKA_2_7_IV0, KAFKA_3_1_IV0}
|
||||
import kafka.server.{BaseRequestTest, KafkaConfig}
|
||||
import kafka.utils.TestUtils
|
||||
import kafka.zk.ZkVersion
|
||||
import org.apache.kafka.clients.producer.ProducerRecord
|
||||
import org.apache.kafka.common.TopicPartition
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.apache.kafka.server.common.MetadataVersion.{IBP_2_7_IV0, IBP_3_1_IV0}
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
|
@ -37,8 +38,8 @@ class FetchRequestTestDowngrade extends BaseRequestTest {
|
|||
override def generateConfigs: Seq[KafkaConfig] = {
|
||||
// Controller should start with newer IBP and downgrade to the older one.
|
||||
Seq(
|
||||
createConfig(0, KAFKA_3_1_IV0),
|
||||
createConfig(1, KAFKA_2_7_IV0)
|
||||
createConfig(0, IBP_3_1_IV0),
|
||||
createConfig(1, IBP_2_7_IV0)
|
||||
)
|
||||
}
|
||||
|
||||
|
@ -72,7 +73,7 @@ class FetchRequestTestDowngrade extends BaseRequestTest {
|
|||
}
|
||||
}
|
||||
|
||||
private def createConfig(nodeId: Int, interBrokerVersion: ApiVersion): KafkaConfig = {
|
||||
private def createConfig(nodeId: Int, interBrokerVersion: MetadataVersion): KafkaConfig = {
|
||||
val props = TestUtils.createBrokerConfig(nodeId, zkConnect)
|
||||
props.put(KafkaConfig.InterBrokerProtocolVersionProp, interBrokerVersion.version)
|
||||
KafkaConfig.fromProps(props)
|
||||
|
|
|
@ -17,7 +17,6 @@
|
|||
|
||||
package kafka.server
|
||||
|
||||
import kafka.api.{ApiVersion, KAFKA_2_8_IV0}
|
||||
import kafka.network.SocketServer
|
||||
import kafka.utils.TestUtils
|
||||
import kafka.zk.ZkVersion
|
||||
|
@ -25,6 +24,8 @@ import org.apache.kafka.common.Uuid
|
|||
import org.apache.kafka.common.message.MetadataRequestData
|
||||
import org.apache.kafka.common.protocol.Errors
|
||||
import org.apache.kafka.common.requests.{MetadataRequest, MetadataResponse}
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.apache.kafka.server.common.MetadataVersion.IBP_2_8_IV0
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
|
@ -35,9 +36,9 @@ class MetadataRequestBetweenDifferentIbpTest extends BaseRequestTest {
|
|||
override def brokerCount: Int = 3
|
||||
override def generateConfigs: Seq[KafkaConfig] = {
|
||||
Seq(
|
||||
createConfig(0, KAFKA_2_8_IV0),
|
||||
createConfig(1, ApiVersion.latestVersion),
|
||||
createConfig(2, ApiVersion.latestVersion)
|
||||
createConfig(0, IBP_2_8_IV0),
|
||||
createConfig(1, MetadataVersion.latest),
|
||||
createConfig(2, MetadataVersion.latest)
|
||||
)
|
||||
}
|
||||
|
||||
|
@ -58,7 +59,7 @@ class MetadataRequestBetweenDifferentIbpTest extends BaseRequestTest {
|
|||
assertEquals(topicId, topicMetadata.topicId())
|
||||
assertEquals(topic, topicMetadata.topic())
|
||||
|
||||
// Make the broker whose version=KAFKA_2_8_IV0 controller
|
||||
// Make the broker whose version=IBP_2_8_IV0 controller
|
||||
ensureControllerIn(Seq(0))
|
||||
|
||||
// Restart the broker whose ibp is higher, and the controller will send metadata request to it
|
||||
|
@ -77,7 +78,7 @@ class MetadataRequestBetweenDifferentIbpTest extends BaseRequestTest {
|
|||
}
|
||||
}
|
||||
|
||||
private def createConfig(nodeId: Int,interBrokerVersion: ApiVersion): KafkaConfig = {
|
||||
private def createConfig(nodeId: Int, interBrokerVersion: MetadataVersion): KafkaConfig = {
|
||||
val props = TestUtils.createBrokerConfig(nodeId, zkConnect)
|
||||
props.put(KafkaConfig.InterBrokerProtocolVersionProp, interBrokerVersion.version)
|
||||
KafkaConfig.fromProps(props)
|
||||
|
|
|
@ -17,15 +17,14 @@
|
|||
|
||||
package kafka.admin
|
||||
|
||||
import kafka.api.KAFKA_2_7_IV0
|
||||
import kafka.server.{BaseRequestTest, KafkaConfig, KafkaServer}
|
||||
import kafka.utils.TestUtils
|
||||
import kafka.utils.TestUtils.waitUntilTrue
|
||||
import org.apache.kafka.common.feature.{Features, SupportedVersionRange}
|
||||
import org.apache.kafka.common.utils.Utils
|
||||
|
||||
import java.util.Properties
|
||||
|
||||
import org.apache.kafka.server.common.MetadataVersion.IBP_2_7_IV0
|
||||
import org.junit.jupiter.api.Assertions.assertTrue
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
|
@ -33,7 +32,7 @@ class FeatureCommandTest extends BaseRequestTest {
|
|||
override def brokerCount: Int = 3
|
||||
|
||||
override def brokerPropertyOverrides(props: Properties): Unit = {
|
||||
props.put(KafkaConfig.InterBrokerProtocolVersionProp, KAFKA_2_7_IV0.toString)
|
||||
props.put(KafkaConfig.InterBrokerProtocolVersionProp, IBP_2_7_IV0.toString)
|
||||
}
|
||||
|
||||
private val defaultSupportedFeatures: Features[SupportedVersionRange] =
|
||||
|
|
|
@ -1,287 +0,0 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package kafka.api
|
||||
|
||||
import java.util
|
||||
|
||||
import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange}
|
||||
import org.apache.kafka.common.message.ApiMessageType.ListenerType
|
||||
import org.apache.kafka.common.protocol.ApiKeys
|
||||
import org.apache.kafka.common.record.{RecordBatch, RecordVersion}
|
||||
import org.apache.kafka.common.requests.{AbstractResponse, ApiVersionsResponse}
|
||||
import org.apache.kafka.common.utils.Utils
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
||||
class ApiVersionTest {
|
||||
|
||||
@Test
|
||||
def testApply(): Unit = {
|
||||
assertEquals(KAFKA_0_8_0, ApiVersion("0.8.0"))
|
||||
assertEquals(KAFKA_0_8_0, ApiVersion("0.8.0.0"))
|
||||
assertEquals(KAFKA_0_8_0, ApiVersion("0.8.0.1"))
|
||||
|
||||
assertEquals(KAFKA_0_8_1, ApiVersion("0.8.1"))
|
||||
assertEquals(KAFKA_0_8_1, ApiVersion("0.8.1.0"))
|
||||
assertEquals(KAFKA_0_8_1, ApiVersion("0.8.1.1"))
|
||||
|
||||
assertEquals(KAFKA_0_8_2, ApiVersion("0.8.2"))
|
||||
assertEquals(KAFKA_0_8_2, ApiVersion("0.8.2.0"))
|
||||
assertEquals(KAFKA_0_8_2, ApiVersion("0.8.2.1"))
|
||||
|
||||
assertEquals(KAFKA_0_9_0, ApiVersion("0.9.0"))
|
||||
assertEquals(KAFKA_0_9_0, ApiVersion("0.9.0.0"))
|
||||
assertEquals(KAFKA_0_9_0, ApiVersion("0.9.0.1"))
|
||||
|
||||
assertEquals(KAFKA_0_10_0_IV0, ApiVersion("0.10.0-IV0"))
|
||||
|
||||
assertEquals(KAFKA_0_10_0_IV1, ApiVersion("0.10.0"))
|
||||
assertEquals(KAFKA_0_10_0_IV1, ApiVersion("0.10.0.0"))
|
||||
assertEquals(KAFKA_0_10_0_IV1, ApiVersion("0.10.0.0-IV0"))
|
||||
assertEquals(KAFKA_0_10_0_IV1, ApiVersion("0.10.0.1"))
|
||||
|
||||
assertEquals(KAFKA_0_10_1_IV0, ApiVersion("0.10.1-IV0"))
|
||||
assertEquals(KAFKA_0_10_1_IV1, ApiVersion("0.10.1-IV1"))
|
||||
|
||||
assertEquals(KAFKA_0_10_1_IV2, ApiVersion("0.10.1"))
|
||||
assertEquals(KAFKA_0_10_1_IV2, ApiVersion("0.10.1.0"))
|
||||
assertEquals(KAFKA_0_10_1_IV2, ApiVersion("0.10.1-IV2"))
|
||||
assertEquals(KAFKA_0_10_1_IV2, ApiVersion("0.10.1.1"))
|
||||
|
||||
assertEquals(KAFKA_0_10_2_IV0, ApiVersion("0.10.2"))
|
||||
assertEquals(KAFKA_0_10_2_IV0, ApiVersion("0.10.2.0"))
|
||||
assertEquals(KAFKA_0_10_2_IV0, ApiVersion("0.10.2-IV0"))
|
||||
assertEquals(KAFKA_0_10_2_IV0, ApiVersion("0.10.2.1"))
|
||||
|
||||
assertEquals(KAFKA_0_11_0_IV0, ApiVersion("0.11.0-IV0"))
|
||||
assertEquals(KAFKA_0_11_0_IV1, ApiVersion("0.11.0-IV1"))
|
||||
|
||||
assertEquals(KAFKA_0_11_0_IV2, ApiVersion("0.11.0"))
|
||||
assertEquals(KAFKA_0_11_0_IV2, ApiVersion("0.11.0.0"))
|
||||
assertEquals(KAFKA_0_11_0_IV2, ApiVersion("0.11.0-IV2"))
|
||||
assertEquals(KAFKA_0_11_0_IV2, ApiVersion("0.11.0.1"))
|
||||
|
||||
assertEquals(KAFKA_1_0_IV0, ApiVersion("1.0"))
|
||||
assertEquals(KAFKA_1_0_IV0, ApiVersion("1.0.0"))
|
||||
assertEquals(KAFKA_1_0_IV0, ApiVersion("1.0.0-IV0"))
|
||||
assertEquals(KAFKA_1_0_IV0, ApiVersion("1.0.1"))
|
||||
|
||||
assertEquals(KAFKA_1_1_IV0, ApiVersion("1.1-IV0"))
|
||||
|
||||
assertEquals(KAFKA_2_0_IV1, ApiVersion("2.0"))
|
||||
assertEquals(KAFKA_2_0_IV0, ApiVersion("2.0-IV0"))
|
||||
assertEquals(KAFKA_2_0_IV1, ApiVersion("2.0-IV1"))
|
||||
|
||||
assertEquals(KAFKA_2_1_IV2, ApiVersion("2.1"))
|
||||
assertEquals(KAFKA_2_1_IV0, ApiVersion("2.1-IV0"))
|
||||
assertEquals(KAFKA_2_1_IV1, ApiVersion("2.1-IV1"))
|
||||
assertEquals(KAFKA_2_1_IV2, ApiVersion("2.1-IV2"))
|
||||
|
||||
assertEquals(KAFKA_2_2_IV1, ApiVersion("2.2"))
|
||||
assertEquals(KAFKA_2_2_IV0, ApiVersion("2.2-IV0"))
|
||||
assertEquals(KAFKA_2_2_IV1, ApiVersion("2.2-IV1"))
|
||||
|
||||
assertEquals(KAFKA_2_3_IV1, ApiVersion("2.3"))
|
||||
assertEquals(KAFKA_2_3_IV0, ApiVersion("2.3-IV0"))
|
||||
assertEquals(KAFKA_2_3_IV1, ApiVersion("2.3-IV1"))
|
||||
|
||||
assertEquals(KAFKA_2_4_IV1, ApiVersion("2.4"))
|
||||
assertEquals(KAFKA_2_4_IV0, ApiVersion("2.4-IV0"))
|
||||
assertEquals(KAFKA_2_4_IV1, ApiVersion("2.4-IV1"))
|
||||
|
||||
assertEquals(KAFKA_2_5_IV0, ApiVersion("2.5"))
|
||||
assertEquals(KAFKA_2_5_IV0, ApiVersion("2.5-IV0"))
|
||||
|
||||
assertEquals(KAFKA_2_6_IV0, ApiVersion("2.6"))
|
||||
assertEquals(KAFKA_2_6_IV0, ApiVersion("2.6-IV0"))
|
||||
|
||||
assertEquals(KAFKA_2_7_IV0, ApiVersion("2.7-IV0"))
|
||||
assertEquals(KAFKA_2_7_IV1, ApiVersion("2.7-IV1"))
|
||||
assertEquals(KAFKA_2_7_IV2, ApiVersion("2.7-IV2"))
|
||||
|
||||
assertEquals(KAFKA_2_8_IV1, ApiVersion("2.8"))
|
||||
assertEquals(KAFKA_2_8_IV0, ApiVersion("2.8-IV0"))
|
||||
assertEquals(KAFKA_2_8_IV1, ApiVersion("2.8-IV1"))
|
||||
|
||||
assertEquals(KAFKA_3_0_IV1, ApiVersion("3.0"))
|
||||
assertEquals(KAFKA_3_0_IV0, ApiVersion("3.0-IV0"))
|
||||
assertEquals(KAFKA_3_0_IV1, ApiVersion("3.0-IV1"))
|
||||
|
||||
assertEquals(KAFKA_3_1_IV0, ApiVersion("3.1"))
|
||||
assertEquals(KAFKA_3_1_IV0, ApiVersion("3.1-IV0"))
|
||||
|
||||
assertEquals(KAFKA_3_2_IV0, ApiVersion("3.2"))
|
||||
assertEquals(KAFKA_3_2_IV0, ApiVersion("3.2-IV0"))
|
||||
}
|
||||
|
||||
@Test
|
||||
def testApiVersionUniqueIds(): Unit = {
|
||||
val allIds: Seq[Int] = ApiVersion.allVersions.map(apiVersion => {
|
||||
apiVersion.id
|
||||
})
|
||||
|
||||
val uniqueIds: Set[Int] = allIds.toSet
|
||||
|
||||
assertEquals(allIds.size, uniqueIds.size)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testMinSupportedVersionFor(): Unit = {
|
||||
assertEquals(KAFKA_0_8_0, ApiVersion.minSupportedFor(RecordVersion.V0))
|
||||
assertEquals(KAFKA_0_10_0_IV0, ApiVersion.minSupportedFor(RecordVersion.V1))
|
||||
assertEquals(KAFKA_0_11_0_IV0, ApiVersion.minSupportedFor(RecordVersion.V2))
|
||||
|
||||
// Ensure that all record versions have a defined min version so that we remember to update the method
|
||||
for (recordVersion <- RecordVersion.values)
|
||||
assertNotNull(ApiVersion.minSupportedFor(recordVersion))
|
||||
}
|
||||
|
||||
@Test
|
||||
def testShortVersion(): Unit = {
|
||||
assertEquals("0.8.0", KAFKA_0_8_0.shortVersion)
|
||||
assertEquals("0.10.0", KAFKA_0_10_0_IV0.shortVersion)
|
||||
assertEquals("0.10.0", KAFKA_0_10_0_IV1.shortVersion)
|
||||
assertEquals("0.11.0", KAFKA_0_11_0_IV0.shortVersion)
|
||||
assertEquals("0.11.0", KAFKA_0_11_0_IV1.shortVersion)
|
||||
assertEquals("0.11.0", KAFKA_0_11_0_IV2.shortVersion)
|
||||
assertEquals("1.0", KAFKA_1_0_IV0.shortVersion)
|
||||
assertEquals("1.1", KAFKA_1_1_IV0.shortVersion)
|
||||
assertEquals("2.0", KAFKA_2_0_IV0.shortVersion)
|
||||
assertEquals("2.0", KAFKA_2_0_IV1.shortVersion)
|
||||
assertEquals("2.1", KAFKA_2_1_IV0.shortVersion)
|
||||
assertEquals("2.1", KAFKA_2_1_IV1.shortVersion)
|
||||
assertEquals("2.1", KAFKA_2_1_IV2.shortVersion)
|
||||
assertEquals("2.2", KAFKA_2_2_IV0.shortVersion)
|
||||
assertEquals("2.2", KAFKA_2_2_IV1.shortVersion)
|
||||
assertEquals("2.3", KAFKA_2_3_IV0.shortVersion)
|
||||
assertEquals("2.3", KAFKA_2_3_IV1.shortVersion)
|
||||
assertEquals("2.4", KAFKA_2_4_IV0.shortVersion)
|
||||
assertEquals("2.5", KAFKA_2_5_IV0.shortVersion)
|
||||
assertEquals("2.6", KAFKA_2_6_IV0.shortVersion)
|
||||
assertEquals("2.7", KAFKA_2_7_IV2.shortVersion)
|
||||
assertEquals("2.8", KAFKA_2_8_IV0.shortVersion)
|
||||
assertEquals("2.8", KAFKA_2_8_IV1.shortVersion)
|
||||
assertEquals("3.0", KAFKA_3_0_IV0.shortVersion)
|
||||
assertEquals("3.0", KAFKA_3_0_IV1.shortVersion)
|
||||
assertEquals("3.1", KAFKA_3_1_IV0.shortVersion)
|
||||
assertEquals("3.2", KAFKA_3_2_IV0.shortVersion)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testApiVersionValidator(): Unit = {
|
||||
val str = ApiVersionValidator.toString
|
||||
val apiVersions = str.slice(1, str.length).split(",")
|
||||
assertEquals(ApiVersion.allVersions.size, apiVersions.length)
|
||||
}
|
||||
|
||||
@Test
|
||||
def shouldCreateApiResponseOnlyWithKeysSupportedByMagicValue(): Unit = {
|
||||
val response = ApiVersion.apiVersionsResponse(
|
||||
10,
|
||||
RecordVersion.V1,
|
||||
Features.emptySupportedFeatures,
|
||||
None,
|
||||
ListenerType.ZK_BROKER
|
||||
)
|
||||
verifyApiKeysForMagic(response, RecordBatch.MAGIC_VALUE_V1)
|
||||
assertEquals(10, response.throttleTimeMs)
|
||||
assertTrue(response.data.supportedFeatures.isEmpty)
|
||||
assertTrue(response.data.finalizedFeatures.isEmpty)
|
||||
assertEquals(ApiVersionsResponse.UNKNOWN_FINALIZED_FEATURES_EPOCH, response.data.finalizedFeaturesEpoch)
|
||||
}
|
||||
|
||||
@Test
|
||||
def shouldReturnFeatureKeysWhenMagicIsCurrentValueAndThrottleMsIsDefaultThrottle(): Unit = {
|
||||
val response = ApiVersion.apiVersionsResponse(
|
||||
10,
|
||||
RecordVersion.V1,
|
||||
Features.supportedFeatures(
|
||||
Utils.mkMap(Utils.mkEntry("feature", new SupportedVersionRange(1.toShort, 4.toShort)))),
|
||||
Features.finalizedFeatures(
|
||||
Utils.mkMap(Utils.mkEntry("feature", new FinalizedVersionRange(2.toShort, 3.toShort)))),
|
||||
10,
|
||||
None,
|
||||
ListenerType.ZK_BROKER
|
||||
)
|
||||
|
||||
verifyApiKeysForMagic(response, RecordBatch.MAGIC_VALUE_V1)
|
||||
assertEquals(10, response.throttleTimeMs)
|
||||
assertEquals(1, response.data.supportedFeatures.size)
|
||||
val sKey = response.data.supportedFeatures.find("feature")
|
||||
assertNotNull(sKey)
|
||||
assertEquals(1, sKey.minVersion)
|
||||
assertEquals(4, sKey.maxVersion)
|
||||
assertEquals(1, response.data.finalizedFeatures.size)
|
||||
val fKey = response.data.finalizedFeatures.find("feature")
|
||||
assertNotNull(fKey)
|
||||
assertEquals(2, fKey.minVersionLevel)
|
||||
assertEquals(3, fKey.maxVersionLevel)
|
||||
assertEquals(10, response.data.finalizedFeaturesEpoch)
|
||||
}
|
||||
|
||||
private def verifyApiKeysForMagic(response: ApiVersionsResponse, maxMagic: Byte): Unit = {
|
||||
for (version <- response.data.apiKeys.asScala) {
|
||||
assertTrue(ApiKeys.forId(version.apiKey).minRequiredInterBrokerMagic <= maxMagic)
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
def shouldReturnAllKeysWhenMagicIsCurrentValueAndThrottleMsIsDefaultThrottle(): Unit = {
|
||||
val response = ApiVersion.apiVersionsResponse(
|
||||
AbstractResponse.DEFAULT_THROTTLE_TIME,
|
||||
RecordVersion.current(),
|
||||
Features.emptySupportedFeatures,
|
||||
None,
|
||||
ListenerType.ZK_BROKER
|
||||
)
|
||||
assertEquals(new util.HashSet[ApiKeys](ApiKeys.zkBrokerApis), apiKeysInResponse(response))
|
||||
assertEquals(AbstractResponse.DEFAULT_THROTTLE_TIME, response.throttleTimeMs)
|
||||
assertTrue(response.data.supportedFeatures.isEmpty)
|
||||
assertTrue(response.data.finalizedFeatures.isEmpty)
|
||||
assertEquals(ApiVersionsResponse.UNKNOWN_FINALIZED_FEATURES_EPOCH, response.data.finalizedFeaturesEpoch)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testMetadataQuorumApisAreDisabled(): Unit = {
|
||||
val response = ApiVersion.apiVersionsResponse(
|
||||
AbstractResponse.DEFAULT_THROTTLE_TIME,
|
||||
RecordVersion.current(),
|
||||
Features.emptySupportedFeatures,
|
||||
None,
|
||||
ListenerType.ZK_BROKER
|
||||
)
|
||||
|
||||
// Ensure that APIs needed for the KRaft mode are not exposed through ApiVersions until we are ready for them
|
||||
val exposedApis = apiKeysInResponse(response)
|
||||
assertFalse(exposedApis.contains(ApiKeys.ENVELOPE))
|
||||
assertFalse(exposedApis.contains(ApiKeys.VOTE))
|
||||
assertFalse(exposedApis.contains(ApiKeys.BEGIN_QUORUM_EPOCH))
|
||||
assertFalse(exposedApis.contains(ApiKeys.END_QUORUM_EPOCH))
|
||||
assertFalse(exposedApis.contains(ApiKeys.DESCRIBE_QUORUM))
|
||||
}
|
||||
|
||||
private def apiKeysInResponse(apiVersions: ApiVersionsResponse) = {
|
||||
val apiKeys = new util.HashSet[ApiKeys]
|
||||
for (version <- apiVersions.data.apiKeys.asScala) {
|
||||
apiKeys.add(ApiKeys.forId(version.apiKey))
|
||||
}
|
||||
apiKeys
|
||||
}
|
||||
}
|
|
@ -16,7 +16,6 @@
|
|||
*/
|
||||
package kafka.cluster
|
||||
|
||||
import kafka.api.ApiVersion
|
||||
import kafka.log.{CleanerConfig, LogConfig, LogManager}
|
||||
import kafka.server.{Defaults, MetadataCache}
|
||||
import kafka.server.checkpoints.OffsetCheckpoints
|
||||
|
@ -30,10 +29,11 @@ import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
|
|||
import org.junit.jupiter.api.{AfterEach, BeforeEach}
|
||||
import org.mockito.ArgumentMatchers
|
||||
import org.mockito.Mockito.{mock, when}
|
||||
|
||||
import java.io.File
|
||||
import java.util.Properties
|
||||
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
||||
object AbstractPartitionTest {
|
||||
|
@ -90,7 +90,7 @@ class AbstractPartitionTest {
|
|||
.thenReturn(None)
|
||||
}
|
||||
|
||||
protected def interBrokerProtocolVersion: ApiVersion = ApiVersion.latestVersion
|
||||
protected def interBrokerProtocolVersion: MetadataVersion = MetadataVersion.latest
|
||||
|
||||
def createLogProperties(overrides: Map[String, String]): Properties = {
|
||||
val logProps = new Properties()
|
||||
|
|
|
@ -21,7 +21,7 @@ import java.util.Properties
|
|||
import java.util.concurrent._
|
||||
import java.util.concurrent.atomic.AtomicBoolean
|
||||
|
||||
import kafka.api.{ApiVersion, LeaderAndIsr}
|
||||
import kafka.api.LeaderAndIsr
|
||||
import kafka.log._
|
||||
import kafka.server._
|
||||
import kafka.server.checkpoints.OffsetCheckpoints
|
||||
|
@ -32,6 +32,7 @@ import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrParti
|
|||
import org.apache.kafka.common.record.{MemoryRecords, SimpleRecord}
|
||||
import org.apache.kafka.common.utils.Utils
|
||||
import org.apache.kafka.common.{TopicPartition, Uuid}
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue}
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
||||
import org.mockito.ArgumentMatchers
|
||||
|
@ -262,7 +263,7 @@ class PartitionLockTest extends Logging {
|
|||
logManager.startup(Set.empty)
|
||||
val partition = new Partition(topicPartition,
|
||||
replicaLagTimeMaxMs = kafka.server.Defaults.ReplicaLagTimeMaxMs,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
localBrokerId = brokerId,
|
||||
mockTime,
|
||||
isrChangeListener,
|
||||
|
@ -396,7 +397,7 @@ class PartitionLockTest extends Logging {
|
|||
keepPartitionMetadataFile = true) {
|
||||
|
||||
override def appendAsLeader(records: MemoryRecords, leaderEpoch: Int, origin: AppendOrigin,
|
||||
interBrokerProtocolVersion: ApiVersion, requestLocal: RequestLocal): LogAppendInfo = {
|
||||
interBrokerProtocolVersion: MetadataVersion, requestLocal: RequestLocal): LogAppendInfo = {
|
||||
val appendInfo = super.appendAsLeader(records, leaderEpoch, origin, interBrokerProtocolVersion, requestLocal)
|
||||
appendSemaphore.acquire()
|
||||
appendInfo
|
||||
|
|
|
@ -17,7 +17,6 @@
|
|||
package kafka.cluster
|
||||
|
||||
import com.yammer.metrics.core.Metric
|
||||
import kafka.api.{ApiVersion, KAFKA_2_6_IV0}
|
||||
import kafka.common.UnexpectedAppendOffsetException
|
||||
import kafka.log.{Defaults => _, _}
|
||||
import kafka.server._
|
||||
|
@ -45,6 +44,8 @@ import java.util.Optional
|
|||
import java.util.concurrent.{CountDownLatch, Semaphore}
|
||||
|
||||
import kafka.server.epoch.LeaderEpochFileCache
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.apache.kafka.server.common.MetadataVersion.IBP_2_6_IV0
|
||||
import org.apache.kafka.server.metrics.KafkaYammerMetrics
|
||||
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
@ -199,7 +200,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
partition = new Partition(
|
||||
topicPartition,
|
||||
replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
localBrokerId = brokerId,
|
||||
time,
|
||||
isrChangeListener,
|
||||
|
@ -1739,7 +1740,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
|
||||
val partition = new Partition(topicPartition,
|
||||
replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs,
|
||||
interBrokerProtocolVersion = KAFKA_2_6_IV0, // shouldn't matter, but set this to a ZK isr version
|
||||
interBrokerProtocolVersion = IBP_2_6_IV0, // shouldn't matter, but set this to a ZK isr version
|
||||
localBrokerId = brokerId,
|
||||
time,
|
||||
isrChangeListener,
|
||||
|
@ -1836,7 +1837,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
// Create new Partition object for same topicPartition
|
||||
val partition2 = new Partition(topicPartition,
|
||||
replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
localBrokerId = brokerId,
|
||||
time,
|
||||
isrChangeListener,
|
||||
|
@ -1880,7 +1881,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
// Create new Partition object for same topicPartition
|
||||
val partition2 = new Partition(topicPartition,
|
||||
replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
localBrokerId = brokerId,
|
||||
time,
|
||||
isrChangeListener,
|
||||
|
@ -1962,7 +1963,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
def testUpdateAssignmentAndIsr(): Unit = {
|
||||
val topicPartition = new TopicPartition("test", 1)
|
||||
val partition = new Partition(
|
||||
topicPartition, 1000, ApiVersion.latestVersion, 0,
|
||||
topicPartition, 1000, MetadataVersion.latest, 0,
|
||||
new SystemTime(), mock(classOf[AlterPartitionListener]), mock(classOf[DelayedOperations]),
|
||||
mock(classOf[MetadataCache]), mock(classOf[LogManager]), mock(classOf[AlterPartitionManager]))
|
||||
|
||||
|
@ -2005,7 +2006,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
val spyLogManager = spy(logManager)
|
||||
val partition = new Partition(topicPartition,
|
||||
replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
localBrokerId = brokerId,
|
||||
time,
|
||||
isrChangeListener,
|
||||
|
@ -2043,7 +2044,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
|
||||
val partition = new Partition(topicPartition,
|
||||
replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
localBrokerId = brokerId,
|
||||
time,
|
||||
isrChangeListener,
|
||||
|
@ -2084,7 +2085,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
|
||||
val partition = new Partition(topicPartition,
|
||||
replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
localBrokerId = brokerId,
|
||||
time,
|
||||
isrChangeListener,
|
||||
|
|
|
@ -16,28 +16,30 @@
|
|||
*/
|
||||
package kafka.cluster
|
||||
|
||||
import kafka.api.{ApiVersion, KAFKA_2_8_IV1}
|
||||
import kafka.log.LogConfig
|
||||
import kafka.utils.TestUtils
|
||||
import org.apache.kafka.common.record.{RecordVersion, SimpleRecord}
|
||||
import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.{UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET}
|
||||
import org.junit.jupiter.api.Assertions.assertEquals
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
import java.util.Optional
|
||||
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.apache.kafka.server.common.MetadataVersion.IBP_2_8_IV1
|
||||
|
||||
import scala.annotation.nowarn
|
||||
|
||||
class PartitionWithLegacyMessageFormatTest extends AbstractPartitionTest {
|
||||
|
||||
// legacy message formats are only supported with IBP < 3.0
|
||||
override protected def interBrokerProtocolVersion: ApiVersion = KAFKA_2_8_IV1
|
||||
override protected def interBrokerProtocolVersion: MetadataVersion = IBP_2_8_IV1
|
||||
|
||||
@nowarn("cat=deprecation")
|
||||
@Test
|
||||
def testMakeLeaderDoesNotUpdateEpochCacheForOldFormats(): Unit = {
|
||||
val leaderEpoch = 8
|
||||
configRepository.setTopicConfig(topicPartition.topic(),
|
||||
LogConfig.MessageFormatVersionProp, kafka.api.KAFKA_0_10_2_IV0.shortVersion)
|
||||
LogConfig.MessageFormatVersionProp, MetadataVersion.IBP_0_10_2_IV0.shortVersion)
|
||||
val log = logManager.getOrCreateLog(topicPartition, topicId = None)
|
||||
log.appendAsLeader(TestUtils.records(List(
|
||||
new SimpleRecord("k1".getBytes, "v1".getBytes),
|
||||
|
|
|
@ -17,7 +17,8 @@
|
|||
package kafka.controller
|
||||
|
||||
import java.util.Properties
|
||||
import kafka.api.{ApiVersion, KAFKA_0_10_0_IV1, KAFKA_0_10_2_IV0, KAFKA_0_9_0, KAFKA_1_0_IV0, KAFKA_2_2_IV0, KAFKA_2_4_IV0, KAFKA_2_4_IV1, KAFKA_2_6_IV0, KAFKA_2_8_IV1, KAFKA_3_2_IV0, LeaderAndIsr}
|
||||
|
||||
import kafka.api.LeaderAndIsr
|
||||
import kafka.cluster.{Broker, EndPoint}
|
||||
import kafka.server.KafkaConfig
|
||||
import kafka.utils.TestUtils
|
||||
|
@ -32,8 +33,11 @@ import org.apache.kafka.common.requests.{AbstractControlRequest, AbstractRespons
|
|||
import org.apache.kafka.common.security.auth.SecurityProtocol
|
||||
import org.apache.kafka.common.{TopicPartition, Uuid}
|
||||
import org.apache.kafka.metadata.LeaderRecoveryState
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.apache.kafka.server.common.MetadataVersion.{IBP_0_10_0_IV1, IBP_0_10_2_IV0, IBP_0_9_0, IBP_1_0_IV0, IBP_2_2_IV0, IBP_2_4_IV0, IBP_2_4_IV1, IBP_2_6_IV0, IBP_2_8_IV1, IBP_3_2_IV0}
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
import scala.collection.mutable
|
||||
import scala.collection.mutable.ListBuffer
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
@ -157,23 +161,23 @@ class ControllerChannelManagerTest {
|
|||
|
||||
@Test
|
||||
def testLeaderAndIsrInterBrokerProtocolVersion(): Unit = {
|
||||
testLeaderAndIsrRequestFollowsInterBrokerProtocolVersion(ApiVersion.latestVersion, ApiKeys.LEADER_AND_ISR.latestVersion)
|
||||
testLeaderAndIsrRequestFollowsInterBrokerProtocolVersion(MetadataVersion.latest, ApiKeys.LEADER_AND_ISR.latestVersion)
|
||||
|
||||
for (apiVersion <- ApiVersion.allVersions) {
|
||||
for (metadataVersion <- MetadataVersion.VALUES) {
|
||||
val leaderAndIsrRequestVersion: Short =
|
||||
if (apiVersion >= KAFKA_3_2_IV0) 6
|
||||
else if (apiVersion >= KAFKA_2_8_IV1) 5
|
||||
else if (apiVersion >= KAFKA_2_4_IV1) 4
|
||||
else if (apiVersion >= KAFKA_2_4_IV0) 3
|
||||
else if (apiVersion >= KAFKA_2_2_IV0) 2
|
||||
else if (apiVersion >= KAFKA_1_0_IV0) 1
|
||||
if (metadataVersion.isAtLeast(IBP_3_2_IV0)) 6
|
||||
else if (metadataVersion.isAtLeast(IBP_2_8_IV1)) 5
|
||||
else if (metadataVersion.isAtLeast(IBP_2_4_IV1)) 4
|
||||
else if (metadataVersion.isAtLeast(IBP_2_4_IV0)) 3
|
||||
else if (metadataVersion.isAtLeast(IBP_2_2_IV0)) 2
|
||||
else if (metadataVersion.isAtLeast(IBP_1_0_IV0)) 1
|
||||
else 0
|
||||
|
||||
testLeaderAndIsrRequestFollowsInterBrokerProtocolVersion(apiVersion, leaderAndIsrRequestVersion)
|
||||
testLeaderAndIsrRequestFollowsInterBrokerProtocolVersion(metadataVersion, leaderAndIsrRequestVersion)
|
||||
}
|
||||
}
|
||||
|
||||
private def testLeaderAndIsrRequestFollowsInterBrokerProtocolVersion(interBrokerProtocolVersion: ApiVersion,
|
||||
private def testLeaderAndIsrRequestFollowsInterBrokerProtocolVersion(interBrokerProtocolVersion: MetadataVersion,
|
||||
expectedLeaderAndIsrVersion: Short): Unit = {
|
||||
val context = initContext(Seq(1, 2, 3), 2, 3, Set("foo", "bar"))
|
||||
val config = createConfig(interBrokerProtocolVersion)
|
||||
|
@ -181,7 +185,7 @@ class ControllerChannelManagerTest {
|
|||
|
||||
val partition = new TopicPartition("foo", 0)
|
||||
var leaderAndIsr = LeaderAndIsr(1, List(1, 2))
|
||||
if (interBrokerProtocolVersion >= KAFKA_3_2_IV0) {
|
||||
if (interBrokerProtocolVersion.isAtLeast(IBP_3_2_IV0)) {
|
||||
leaderAndIsr = leaderAndIsr.copy(leaderRecoveryState = LeaderRecoveryState.RECOVERING)
|
||||
}
|
||||
|
||||
|
@ -201,7 +205,7 @@ class ControllerChannelManagerTest {
|
|||
val byteBuffer = request.serialize
|
||||
val deserializedRequest = LeaderAndIsrRequest.parse(byteBuffer, expectedLeaderAndIsrVersion)
|
||||
|
||||
val expectedRecovery = if (interBrokerProtocolVersion >= KAFKA_3_2_IV0) {
|
||||
val expectedRecovery = if (interBrokerProtocolVersion.isAtLeast(IBP_3_2_IV0)) {
|
||||
LeaderRecoveryState.RECOVERING
|
||||
} else {
|
||||
LeaderRecoveryState.RECOVERED
|
||||
|
@ -213,10 +217,10 @@ class ControllerChannelManagerTest {
|
|||
}
|
||||
}
|
||||
|
||||
if (interBrokerProtocolVersion >= KAFKA_2_8_IV1) {
|
||||
if (interBrokerProtocolVersion.isAtLeast(IBP_2_8_IV1)) {
|
||||
assertFalse(request.topicIds().get("foo").equals(Uuid.ZERO_UUID))
|
||||
assertFalse(deserializedRequest.topicIds().get("foo").equals(Uuid.ZERO_UUID))
|
||||
} else if (interBrokerProtocolVersion >= KAFKA_2_2_IV0) {
|
||||
} else if (interBrokerProtocolVersion.isAtLeast(IBP_2_2_IV0)) {
|
||||
assertFalse(request.topicIds().get("foo").equals(Uuid.ZERO_UUID))
|
||||
assertTrue(deserializedRequest.topicIds().get("foo").equals(Uuid.ZERO_UUID))
|
||||
} else {
|
||||
|
@ -374,24 +378,24 @@ class ControllerChannelManagerTest {
|
|||
|
||||
@Test
|
||||
def testUpdateMetadataInterBrokerProtocolVersion(): Unit = {
|
||||
testUpdateMetadataFollowsInterBrokerProtocolVersion(ApiVersion.latestVersion, ApiKeys.UPDATE_METADATA.latestVersion)
|
||||
testUpdateMetadataFollowsInterBrokerProtocolVersion(MetadataVersion.latest, ApiKeys.UPDATE_METADATA.latestVersion)
|
||||
|
||||
for (apiVersion <- ApiVersion.allVersions) {
|
||||
for (metadataVersion <- MetadataVersion.VALUES) {
|
||||
val updateMetadataRequestVersion: Short =
|
||||
if (apiVersion >= KAFKA_2_8_IV1) 7
|
||||
else if (apiVersion >= KAFKA_2_4_IV1) 6
|
||||
else if (apiVersion >= KAFKA_2_2_IV0) 5
|
||||
else if (apiVersion >= KAFKA_1_0_IV0) 4
|
||||
else if (apiVersion >= KAFKA_0_10_2_IV0) 3
|
||||
else if (apiVersion >= KAFKA_0_10_0_IV1) 2
|
||||
else if (apiVersion >= KAFKA_0_9_0) 1
|
||||
if (metadataVersion.isAtLeast(IBP_2_8_IV1)) 7
|
||||
else if (metadataVersion.isAtLeast(IBP_2_4_IV1)) 6
|
||||
else if (metadataVersion.isAtLeast(IBP_2_2_IV0)) 5
|
||||
else if (metadataVersion.isAtLeast(IBP_1_0_IV0)) 4
|
||||
else if (metadataVersion.isAtLeast(IBP_0_10_2_IV0)) 3
|
||||
else if (metadataVersion.isAtLeast(IBP_0_10_0_IV1)) 2
|
||||
else if (metadataVersion.isAtLeast(IBP_0_9_0)) 1
|
||||
else 0
|
||||
|
||||
testUpdateMetadataFollowsInterBrokerProtocolVersion(apiVersion, updateMetadataRequestVersion)
|
||||
testUpdateMetadataFollowsInterBrokerProtocolVersion(metadataVersion, updateMetadataRequestVersion)
|
||||
}
|
||||
}
|
||||
|
||||
private def testUpdateMetadataFollowsInterBrokerProtocolVersion(interBrokerProtocolVersion: ApiVersion,
|
||||
private def testUpdateMetadataFollowsInterBrokerProtocolVersion(interBrokerProtocolVersion: MetadataVersion,
|
||||
expectedUpdateMetadataVersion: Short): Unit = {
|
||||
val context = initContext(Seq(1, 2, 3), 2, 3, Set("foo", "bar"))
|
||||
val config = createConfig(interBrokerProtocolVersion)
|
||||
|
@ -470,12 +474,12 @@ class ControllerChannelManagerTest {
|
|||
|
||||
@Test
|
||||
def testStopReplicaRequestsWhileTopicQueuedForDeletion(): Unit = {
|
||||
for (apiVersion <- ApiVersion.allVersions) {
|
||||
testStopReplicaRequestsWhileTopicQueuedForDeletion(apiVersion)
|
||||
for (metadataVersion <- MetadataVersion.VALUES) {
|
||||
testStopReplicaRequestsWhileTopicQueuedForDeletion(metadataVersion)
|
||||
}
|
||||
}
|
||||
|
||||
private def testStopReplicaRequestsWhileTopicQueuedForDeletion(interBrokerProtocolVersion: ApiVersion): Unit = {
|
||||
private def testStopReplicaRequestsWhileTopicQueuedForDeletion(interBrokerProtocolVersion: MetadataVersion): Unit = {
|
||||
val context = initContext(Seq(1, 2, 3), 2, 3, Set("foo", "bar"))
|
||||
val config = createConfig(interBrokerProtocolVersion)
|
||||
val batch = new MockControllerBrokerRequestBatch(context, config)
|
||||
|
@ -517,12 +521,12 @@ class ControllerChannelManagerTest {
|
|||
|
||||
@Test
|
||||
def testStopReplicaRequestsWhileTopicDeletionStarted(): Unit = {
|
||||
for (apiVersion <- ApiVersion.allVersions) {
|
||||
testStopReplicaRequestsWhileTopicDeletionStarted(apiVersion)
|
||||
for (metadataVersion <- MetadataVersion.VALUES) {
|
||||
testStopReplicaRequestsWhileTopicDeletionStarted(metadataVersion)
|
||||
}
|
||||
}
|
||||
|
||||
private def testStopReplicaRequestsWhileTopicDeletionStarted(interBrokerProtocolVersion: ApiVersion): Unit = {
|
||||
private def testStopReplicaRequestsWhileTopicDeletionStarted(interBrokerProtocolVersion: MetadataVersion): Unit = {
|
||||
val context = initContext(Seq(1, 2, 3), 2, 3, Set("foo", "bar"))
|
||||
val config = createConfig(interBrokerProtocolVersion)
|
||||
val batch = new MockControllerBrokerRequestBatch(context, config)
|
||||
|
@ -572,12 +576,12 @@ class ControllerChannelManagerTest {
|
|||
|
||||
@Test
|
||||
def testStopReplicaRequestWithoutDeletePartitionWhileTopicDeletionStarted(): Unit = {
|
||||
for (apiVersion <- ApiVersion.allVersions) {
|
||||
testStopReplicaRequestWithoutDeletePartitionWhileTopicDeletionStarted(apiVersion)
|
||||
for (metadataVersion <- MetadataVersion.VALUES) {
|
||||
testStopReplicaRequestWithoutDeletePartitionWhileTopicDeletionStarted(metadataVersion)
|
||||
}
|
||||
}
|
||||
|
||||
private def testStopReplicaRequestWithoutDeletePartitionWhileTopicDeletionStarted(interBrokerProtocolVersion: ApiVersion): Unit = {
|
||||
private def testStopReplicaRequestWithoutDeletePartitionWhileTopicDeletionStarted(interBrokerProtocolVersion: MetadataVersion): Unit = {
|
||||
val context = initContext(Seq(1, 2, 3), 2, 3, Set("foo", "bar"))
|
||||
val config = createConfig(interBrokerProtocolVersion)
|
||||
val batch = new MockControllerBrokerRequestBatch(context, config)
|
||||
|
@ -619,22 +623,22 @@ class ControllerChannelManagerTest {
|
|||
|
||||
@Test
|
||||
def testMixedDeleteAndNotDeleteStopReplicaRequests(): Unit = {
|
||||
testMixedDeleteAndNotDeleteStopReplicaRequests(ApiVersion.latestVersion,
|
||||
testMixedDeleteAndNotDeleteStopReplicaRequests(MetadataVersion.latest,
|
||||
ApiKeys.STOP_REPLICA.latestVersion)
|
||||
|
||||
for (apiVersion <- ApiVersion.allVersions) {
|
||||
if (apiVersion < KAFKA_2_2_IV0)
|
||||
testMixedDeleteAndNotDeleteStopReplicaRequests(apiVersion, 0.toShort)
|
||||
else if (apiVersion < KAFKA_2_4_IV1)
|
||||
testMixedDeleteAndNotDeleteStopReplicaRequests(apiVersion, 1.toShort)
|
||||
else if (apiVersion < KAFKA_2_6_IV0)
|
||||
testMixedDeleteAndNotDeleteStopReplicaRequests(apiVersion, 2.toShort)
|
||||
for (metadataVersion <- MetadataVersion.VALUES) {
|
||||
if (metadataVersion.isLessThan(IBP_2_2_IV0))
|
||||
testMixedDeleteAndNotDeleteStopReplicaRequests(metadataVersion, 0.toShort)
|
||||
else if (metadataVersion.isLessThan(IBP_2_4_IV1))
|
||||
testMixedDeleteAndNotDeleteStopReplicaRequests(metadataVersion, 1.toShort)
|
||||
else if (metadataVersion.isLessThan(IBP_2_6_IV0))
|
||||
testMixedDeleteAndNotDeleteStopReplicaRequests(metadataVersion, 2.toShort)
|
||||
else
|
||||
testMixedDeleteAndNotDeleteStopReplicaRequests(apiVersion, 3.toShort)
|
||||
testMixedDeleteAndNotDeleteStopReplicaRequests(metadataVersion, 3.toShort)
|
||||
}
|
||||
}
|
||||
|
||||
private def testMixedDeleteAndNotDeleteStopReplicaRequests(interBrokerProtocolVersion: ApiVersion,
|
||||
private def testMixedDeleteAndNotDeleteStopReplicaRequests(interBrokerProtocolVersion: MetadataVersion,
|
||||
expectedStopReplicaRequestVersion: Short): Unit = {
|
||||
val context = initContext(Seq(1, 2, 3), 2, 3, Set("foo", "bar"))
|
||||
val config = createConfig(interBrokerProtocolVersion)
|
||||
|
@ -665,8 +669,8 @@ class ControllerChannelManagerTest {
|
|||
assertEquals(1, batch.sentRequests.size)
|
||||
assertTrue(batch.sentRequests.contains(2))
|
||||
|
||||
// Since KAFKA_2_6_IV0, only one StopReplicaRequest is sent out
|
||||
if (interBrokerProtocolVersion >= KAFKA_2_6_IV0) {
|
||||
// Since IBP_2_6_IV0, only one StopReplicaRequest is sent out
|
||||
if (interBrokerProtocolVersion.isAtLeast(IBP_2_6_IV0)) {
|
||||
val sentRequests = batch.sentRequests(2)
|
||||
assertEquals(1, sentRequests.size)
|
||||
|
||||
|
@ -769,21 +773,21 @@ class ControllerChannelManagerTest {
|
|||
|
||||
@Test
|
||||
def testStopReplicaInterBrokerProtocolVersion(): Unit = {
|
||||
testStopReplicaFollowsInterBrokerProtocolVersion(ApiVersion.latestVersion, ApiKeys.STOP_REPLICA.latestVersion)
|
||||
testStopReplicaFollowsInterBrokerProtocolVersion(MetadataVersion.latest, ApiKeys.STOP_REPLICA.latestVersion)
|
||||
|
||||
for (apiVersion <- ApiVersion.allVersions) {
|
||||
if (apiVersion < KAFKA_2_2_IV0)
|
||||
testStopReplicaFollowsInterBrokerProtocolVersion(apiVersion, 0.toShort)
|
||||
else if (apiVersion < KAFKA_2_4_IV1)
|
||||
testStopReplicaFollowsInterBrokerProtocolVersion(apiVersion, 1.toShort)
|
||||
else if (apiVersion < KAFKA_2_6_IV0)
|
||||
testStopReplicaFollowsInterBrokerProtocolVersion(apiVersion, 2.toShort)
|
||||
for (metadataVersion <- MetadataVersion.VALUES) {
|
||||
if (metadataVersion.isLessThan(IBP_2_2_IV0))
|
||||
testStopReplicaFollowsInterBrokerProtocolVersion(metadataVersion, 0.toShort)
|
||||
else if (metadataVersion.isLessThan(IBP_2_4_IV1))
|
||||
testStopReplicaFollowsInterBrokerProtocolVersion(metadataVersion, 1.toShort)
|
||||
else if (metadataVersion.isLessThan(IBP_2_6_IV0))
|
||||
testStopReplicaFollowsInterBrokerProtocolVersion(metadataVersion, 2.toShort)
|
||||
else
|
||||
testStopReplicaFollowsInterBrokerProtocolVersion(apiVersion, 3.toShort)
|
||||
testStopReplicaFollowsInterBrokerProtocolVersion(metadataVersion, 3.toShort)
|
||||
}
|
||||
}
|
||||
|
||||
private def testStopReplicaFollowsInterBrokerProtocolVersion(interBrokerProtocolVersion: ApiVersion,
|
||||
private def testStopReplicaFollowsInterBrokerProtocolVersion(interBrokerProtocolVersion: MetadataVersion,
|
||||
expectedStopReplicaRequestVersion: Short): Unit = {
|
||||
val context = initContext(Seq(1, 2, 3), 2, 3, Set("foo"))
|
||||
val config = createConfig(interBrokerProtocolVersion)
|
||||
|
@ -884,7 +888,7 @@ class ControllerChannelManagerTest {
|
|||
}
|
||||
}
|
||||
|
||||
private def createConfig(interBrokerVersion: ApiVersion): KafkaConfig = {
|
||||
private def createConfig(interBrokerVersion: MetadataVersion): KafkaConfig = {
|
||||
val props = new Properties()
|
||||
props.put(KafkaConfig.BrokerIdProp, controllerId.toString)
|
||||
props.put(KafkaConfig.ZkConnectProp, "zkConnect")
|
||||
|
|
|
@ -21,7 +21,7 @@ import java.util.Properties
|
|||
import java.util.concurrent.{CompletableFuture, CountDownLatch, LinkedBlockingQueue, TimeUnit}
|
||||
|
||||
import com.yammer.metrics.core.Timer
|
||||
import kafka.api.{ApiVersion, KAFKA_2_6_IV0, KAFKA_2_7_IV0, LeaderAndIsr}
|
||||
import kafka.api.LeaderAndIsr
|
||||
import kafka.controller.KafkaController.AlterPartitionCallback
|
||||
import kafka.server.{KafkaConfig, KafkaServer, QuorumTestHarness}
|
||||
import kafka.utils.{LogCaptureAppender, TestUtils}
|
||||
|
@ -32,6 +32,8 @@ import org.apache.kafka.common.metrics.KafkaMetric
|
|||
import org.apache.kafka.common.protocol.Errors
|
||||
import org.apache.kafka.common.{ElectionType, TopicPartition, Uuid}
|
||||
import org.apache.kafka.metadata.LeaderRecoveryState
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.apache.kafka.server.common.MetadataVersion.{IBP_2_6_IV0, IBP_2_7_IV0}
|
||||
import org.apache.kafka.server.metrics.KafkaYammerMetrics
|
||||
import org.apache.log4j.Level
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertNotEquals, assertTrue}
|
||||
|
@ -630,32 +632,32 @@ class ControllerIntegrationTest extends QuorumTestHarness {
|
|||
|
||||
@Test
|
||||
def testControllerFeatureZNodeSetupWhenFeatureVersioningIsEnabledWithNonExistingFeatureZNode(): Unit = {
|
||||
testControllerFeatureZNodeSetup(Option.empty, KAFKA_2_7_IV0)
|
||||
testControllerFeatureZNodeSetup(Option.empty, IBP_2_7_IV0)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testControllerFeatureZNodeSetupWhenFeatureVersioningIsEnabledWithDisabledExistingFeatureZNode(): Unit = {
|
||||
testControllerFeatureZNodeSetup(Some(new FeatureZNode(FeatureZNodeStatus.Disabled, Features.emptyFinalizedFeatures())), KAFKA_2_7_IV0)
|
||||
testControllerFeatureZNodeSetup(Some(new FeatureZNode(FeatureZNodeStatus.Disabled, Features.emptyFinalizedFeatures())), IBP_2_7_IV0)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testControllerFeatureZNodeSetupWhenFeatureVersioningIsEnabledWithEnabledExistingFeatureZNode(): Unit = {
|
||||
testControllerFeatureZNodeSetup(Some(new FeatureZNode(FeatureZNodeStatus.Enabled, Features.emptyFinalizedFeatures())), KAFKA_2_7_IV0)
|
||||
testControllerFeatureZNodeSetup(Some(new FeatureZNode(FeatureZNodeStatus.Enabled, Features.emptyFinalizedFeatures())), IBP_2_7_IV0)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testControllerFeatureZNodeSetupWhenFeatureVersioningIsDisabledWithNonExistingFeatureZNode(): Unit = {
|
||||
testControllerFeatureZNodeSetup(Option.empty, KAFKA_2_6_IV0)
|
||||
testControllerFeatureZNodeSetup(Option.empty, IBP_2_6_IV0)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testControllerFeatureZNodeSetupWhenFeatureVersioningIsDisabledWithDisabledExistingFeatureZNode(): Unit = {
|
||||
testControllerFeatureZNodeSetup(Some(new FeatureZNode(FeatureZNodeStatus.Disabled, Features.emptyFinalizedFeatures())), KAFKA_2_6_IV0)
|
||||
testControllerFeatureZNodeSetup(Some(new FeatureZNode(FeatureZNodeStatus.Disabled, Features.emptyFinalizedFeatures())), IBP_2_6_IV0)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testControllerFeatureZNodeSetupWhenFeatureVersioningIsDisabledWithEnabledExistingFeatureZNode(): Unit = {
|
||||
testControllerFeatureZNodeSetup(Some(new FeatureZNode(FeatureZNodeStatus.Enabled, Features.emptyFinalizedFeatures())), KAFKA_2_6_IV0)
|
||||
testControllerFeatureZNodeSetup(Some(new FeatureZNode(FeatureZNodeStatus.Enabled, Features.emptyFinalizedFeatures())), IBP_2_6_IV0)
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -782,7 +784,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
|
|||
}
|
||||
|
||||
private def testControllerFeatureZNodeSetup(initialZNode: Option[FeatureZNode],
|
||||
interBrokerProtocolVersion: ApiVersion): Unit = {
|
||||
interBrokerProtocolVersion: MetadataVersion): Unit = {
|
||||
val versionBeforeOpt = initialZNode match {
|
||||
case Some(node) =>
|
||||
zkClient.createFeatureZNode(node)
|
||||
|
@ -809,7 +811,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
|
|||
|
||||
val (mayBeFeatureZNodeBytes, versionAfter) = zkClient.getDataAndVersion(FeatureZNode.path)
|
||||
val newZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
|
||||
if (interBrokerProtocolVersion >= KAFKA_2_7_IV0) {
|
||||
if (interBrokerProtocolVersion.isAtLeast(IBP_2_7_IV0)) {
|
||||
val emptyZNode = new FeatureZNode(FeatureZNodeStatus.Enabled, Features.emptyFinalizedFeatures)
|
||||
initialZNode match {
|
||||
case Some(node) => {
|
||||
|
@ -1213,7 +1215,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
|
|||
|
||||
@Test
|
||||
def testTopicIdsAreNotAdded(): Unit = {
|
||||
servers = makeServers(1, interBrokerProtocolVersion = Some(KAFKA_2_7_IV0))
|
||||
servers = makeServers(1, interBrokerProtocolVersion = Some(IBP_2_7_IV0))
|
||||
TestUtils.waitUntilControllerElected(zkClient)
|
||||
val controller = getController().kafkaController
|
||||
val tp1 = new TopicPartition("t1", 0)
|
||||
|
@ -1275,7 +1277,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
|
|||
val assignment = Map(tp.partition -> ReplicaAssignment(Seq(0), List(), List()))
|
||||
val adminZkClient = new AdminZkClient(zkClient)
|
||||
|
||||
servers = makeServers(1, interBrokerProtocolVersion = Some(KAFKA_2_7_IV0))
|
||||
servers = makeServers(1, interBrokerProtocolVersion = Some(IBP_2_7_IV0))
|
||||
adminZkClient.createTopic(tp.topic, 1, 1)
|
||||
waitForPartitionState(tp, firstControllerEpoch, 0, LeaderAndIsr.InitialLeaderEpoch,
|
||||
"failed to get expected partition state upon topic creation")
|
||||
|
@ -1316,7 +1318,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
|
|||
|
||||
@Test
|
||||
def testNoTopicIdPersistsThroughControllerReelection(): Unit = {
|
||||
servers = makeServers(2, interBrokerProtocolVersion = Some(KAFKA_2_7_IV0))
|
||||
servers = makeServers(2, interBrokerProtocolVersion = Some(IBP_2_7_IV0))
|
||||
val controllerId = TestUtils.waitUntilControllerElected(zkClient)
|
||||
val controller = getController().kafkaController
|
||||
val tp = new TopicPartition("t", 0)
|
||||
|
@ -1356,7 +1358,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
|
|||
|
||||
@Test
|
||||
def testTopicIdCreatedOnUpgrade(): Unit = {
|
||||
servers = makeServers(1, interBrokerProtocolVersion = Some(KAFKA_2_7_IV0))
|
||||
servers = makeServers(1, interBrokerProtocolVersion = Some(IBP_2_7_IV0))
|
||||
val controllerId = TestUtils.waitUntilControllerElected(zkClient)
|
||||
val controller = getController().kafkaController
|
||||
val tp = new TopicPartition("t", 0)
|
||||
|
@ -1393,12 +1395,12 @@ class ControllerIntegrationTest extends QuorumTestHarness {
|
|||
@Test
|
||||
def testTopicIdCreatedOnUpgradeMultiBrokerScenario(): Unit = {
|
||||
// Simulate an upgrade scenario where the controller is still on a pre-topic ID IBP, but the other two brokers are upgraded.
|
||||
servers = makeServers(1, interBrokerProtocolVersion = Some(KAFKA_2_7_IV0))
|
||||
servers = makeServers(1, interBrokerProtocolVersion = Some(MetadataVersion.IBP_2_7_IV0))
|
||||
servers = servers ++ makeServers(3, startingIdNumber = 1)
|
||||
val originalControllerId = TestUtils.waitUntilControllerElected(zkClient)
|
||||
assertEquals(0, originalControllerId)
|
||||
val controller = getController().kafkaController
|
||||
assertEquals(KAFKA_2_7_IV0, servers(originalControllerId).config.interBrokerProtocolVersion)
|
||||
assertEquals(IBP_2_7_IV0, servers(originalControllerId).config.interBrokerProtocolVersion)
|
||||
val remainingBrokers = servers.filter(_.config.brokerId != originalControllerId)
|
||||
val tp = new TopicPartition("t", 0)
|
||||
// Only the remaining brokers will have the replicas for the partition
|
||||
|
@ -1452,7 +1454,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
|
|||
val adminZkClient = new AdminZkClient(zkClient)
|
||||
|
||||
// start server with old IBP
|
||||
servers = makeServers(1, interBrokerProtocolVersion = Some(KAFKA_2_7_IV0))
|
||||
servers = makeServers(1, interBrokerProtocolVersion = Some(IBP_2_7_IV0))
|
||||
// use create topic with ZK client directly, without topic ID
|
||||
adminZkClient.createTopic(tp.topic, 1, 1)
|
||||
waitForPartitionState(tp, firstControllerEpoch, 0, LeaderAndIsr.InitialLeaderEpoch,
|
||||
|
@ -1478,7 +1480,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
|
|||
// Downgrade back to 2.7
|
||||
servers(0).shutdown()
|
||||
servers(0).awaitShutdown()
|
||||
servers = makeServers(1, interBrokerProtocolVersion = Some(KAFKA_2_7_IV0))
|
||||
servers = makeServers(1, interBrokerProtocolVersion = Some(IBP_2_7_IV0))
|
||||
waitForPartitionState(tp, firstControllerEpoch, 0, LeaderAndIsr.InitialLeaderEpoch,
|
||||
"failed to get expected partition state upon topic creation")
|
||||
val topicIdAfterDowngrade = zkClient.getTopicIdsForTopics(Set(tp.topic())).get(tp.topic())
|
||||
|
@ -1611,7 +1613,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
|
|||
listeners : Option[String] = None,
|
||||
listenerSecurityProtocolMap : Option[String] = None,
|
||||
controlPlaneListenerName : Option[String] = None,
|
||||
interBrokerProtocolVersion: Option[ApiVersion] = None,
|
||||
interBrokerProtocolVersion: Option[MetadataVersion] = None,
|
||||
logDirCount: Int = 1,
|
||||
startingIdNumber: Int = 0): Seq[KafkaServer] = {
|
||||
val configs = TestUtils.createBrokerConfigs(numConfigs, zkConnect, enableControlledShutdown = enableControlledShutdown, logDirCount = logDirCount, startingIdNumber = startingIdNumber)
|
||||
|
|
|
@ -16,8 +16,6 @@
|
|||
*/
|
||||
package kafka.controller
|
||||
|
||||
import kafka.api.KAFKA_3_1_IV0
|
||||
import kafka.api.KAFKA_3_2_IV0
|
||||
import kafka.api.LeaderAndIsr
|
||||
import kafka.log.LogConfig
|
||||
import kafka.server.KafkaConfig
|
||||
|
@ -26,6 +24,7 @@ import kafka.zk.KafkaZkClient.UpdateLeaderAndIsrResult
|
|||
import kafka.zk.{KafkaZkClient, TopicPartitionStateZNode}
|
||||
import kafka.zookeeper._
|
||||
import org.apache.kafka.common.TopicPartition
|
||||
import org.apache.kafka.server.common.MetadataVersion.{IBP_3_1_IV0, IBP_3_2_IV0}
|
||||
import org.apache.zookeeper.KeeperException.Code
|
||||
import org.apache.zookeeper.data.Stat
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
|
@ -294,7 +293,7 @@ class PartitionStateMachineTest {
|
|||
|
||||
|
||||
val partitionStateMachine = {
|
||||
val apiVersion = if (isLeaderRecoverySupported) KAFKA_3_2_IV0 else KAFKA_3_1_IV0
|
||||
val apiVersion = if (isLeaderRecoverySupported) IBP_3_2_IV0 else IBP_3_1_IV0
|
||||
val properties = TestUtils.createBrokerConfig(brokerId, "zkConnect")
|
||||
|
||||
properties.setProperty(KafkaConfig.InterBrokerProtocolVersionProp, apiVersion.toString)
|
||||
|
|
|
@ -24,7 +24,6 @@ import java.util.{Collections, Optional}
|
|||
|
||||
import com.yammer.metrics.core.Gauge
|
||||
import javax.management.ObjectName
|
||||
import kafka.api._
|
||||
import kafka.cluster.Partition
|
||||
import kafka.common.OffsetAndMetadata
|
||||
import kafka.log.{AppendOrigin, LogAppendInfo, UnifiedLog}
|
||||
|
@ -41,6 +40,8 @@ import org.apache.kafka.common.record._
|
|||
import org.apache.kafka.common.requests.OffsetFetchResponse
|
||||
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
|
||||
import org.apache.kafka.common.utils.Utils
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.apache.kafka.server.common.MetadataVersion._
|
||||
import org.apache.kafka.server.metrics.KafkaYammerMetrics
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
||||
|
@ -91,7 +92,7 @@ class GroupMetadataManagerTest {
|
|||
metrics = new kMetrics()
|
||||
time = new MockTime
|
||||
replicaManager = mock(classOf[ReplicaManager])
|
||||
groupMetadataManager = new GroupMetadataManager(0, ApiVersion.latestVersion, offsetConfig, replicaManager,
|
||||
groupMetadataManager = new GroupMetadataManager(0, MetadataVersion.latest, offsetConfig, replicaManager,
|
||||
time, metrics)
|
||||
groupMetadataManager.startup(() => numOffsetsPartitions, false)
|
||||
partition = mock(classOf[Partition])
|
||||
|
@ -106,7 +107,7 @@ class GroupMetadataManagerTest {
|
|||
def testLogInfoFromCleanupGroupMetadata(): Unit = {
|
||||
var expiredOffsets: Int = 0
|
||||
var infoCount = 0
|
||||
val gmm = new GroupMetadataManager(0, ApiVersion.latestVersion, offsetConfig, replicaManager, time, metrics) {
|
||||
val gmm = new GroupMetadataManager(0, MetadataVersion.latest, offsetConfig, replicaManager, time, metrics) {
|
||||
override def cleanupGroupMetadata(groups: Iterable[GroupMetadata], requestLocal: RequestLocal,
|
||||
selector: GroupMetadata => Map[TopicPartition, OffsetAndMetadata]): Int = expiredOffsets
|
||||
|
||||
|
@ -1054,17 +1055,17 @@ class GroupMetadataManagerTest {
|
|||
val protocol = "range"
|
||||
val memberId = "memberId"
|
||||
|
||||
for (apiVersion <- ApiVersion.allVersions) {
|
||||
val groupMetadataRecord = buildStableGroupRecordWithMember(generation, protocolType, protocol, memberId, apiVersion = apiVersion)
|
||||
for (metadataVersion <- MetadataVersion.VALUES) {
|
||||
val groupMetadataRecord = buildStableGroupRecordWithMember(generation, protocolType, protocol, memberId, metadataVersion = metadataVersion)
|
||||
|
||||
val deserializedGroupMetadata = GroupMetadataManager.readGroupMessageValue(groupId, groupMetadataRecord.value(), time)
|
||||
// GROUP_METADATA_VALUE_SCHEMA_V2 or higher should correctly set the currentStateTimestamp
|
||||
if (apiVersion >= KAFKA_2_1_IV0)
|
||||
if (metadataVersion.isAtLeast(IBP_2_1_IV0))
|
||||
assertEquals(Some(time.milliseconds()), deserializedGroupMetadata.currentStateTimestamp,
|
||||
s"the apiVersion $apiVersion doesn't set the currentStateTimestamp correctly.")
|
||||
s"the metadataVersion $metadataVersion doesn't set the currentStateTimestamp correctly.")
|
||||
else
|
||||
assertTrue(deserializedGroupMetadata.currentStateTimestamp.isEmpty,
|
||||
s"the apiVersion $apiVersion should not set the currentStateTimestamp.")
|
||||
s"the metadataVersion $metadataVersion should not set the currentStateTimestamp.")
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1073,10 +1074,10 @@ class GroupMetadataManagerTest {
|
|||
val generation = 1
|
||||
val protocol = "range"
|
||||
val memberId = "memberId"
|
||||
val oldApiVersions = Array(KAFKA_0_9_0, KAFKA_0_10_1_IV0, KAFKA_2_1_IV0)
|
||||
val oldMetadataVersions = Array(IBP_0_9_0, IBP_0_10_1_IV0, IBP_2_1_IV0)
|
||||
|
||||
for (apiVersion <- oldApiVersions) {
|
||||
val groupMetadataRecord = buildStableGroupRecordWithMember(generation, protocolType, protocol, memberId, apiVersion = apiVersion)
|
||||
for (metadataVersion <- oldMetadataVersions) {
|
||||
val groupMetadataRecord = buildStableGroupRecordWithMember(generation, protocolType, protocol, memberId, metadataVersion = metadataVersion)
|
||||
|
||||
val deserializedGroupMetadata = GroupMetadataManager.readGroupMessageValue(groupId, groupMetadataRecord.value(), time)
|
||||
assertEquals(groupId, deserializedGroupMetadata.groupId)
|
||||
|
@ -2181,10 +2182,10 @@ class GroupMetadataManagerTest {
|
|||
new TopicPartition("bar", 0) -> 8992L
|
||||
)
|
||||
|
||||
val apiVersion = KAFKA_1_1_IV0
|
||||
val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets, apiVersion = apiVersion, retentionTimeOpt = Some(100))
|
||||
val metadataVersion = IBP_1_1_IV0
|
||||
val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets, metadataVersion = metadataVersion, retentionTimeOpt = Some(100))
|
||||
val memberId = "98098230493"
|
||||
val groupMetadataRecord = buildStableGroupRecordWithMember(generation, protocolType, protocol, memberId, apiVersion = apiVersion)
|
||||
val groupMetadataRecord = buildStableGroupRecordWithMember(generation, protocolType, protocol, memberId, metadataVersion = metadataVersion)
|
||||
val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE,
|
||||
(offsetCommitRecords ++ Seq(groupMetadataRecord)).toArray: _*)
|
||||
|
||||
|
@ -2255,8 +2256,8 @@ class GroupMetadataManagerTest {
|
|||
commitTimestamp = time.milliseconds(),
|
||||
expireTimestamp = None)
|
||||
|
||||
def verifySerde(apiVersion: ApiVersion, expectedOffsetCommitValueVersion: Int): Unit = {
|
||||
val bytes = GroupMetadataManager.offsetCommitValue(offsetAndMetadata, apiVersion)
|
||||
def verifySerde(metadataVersion: MetadataVersion, expectedOffsetCommitValueVersion: Int): Unit = {
|
||||
val bytes = GroupMetadataManager.offsetCommitValue(offsetAndMetadata, metadataVersion)
|
||||
val buffer = ByteBuffer.wrap(bytes)
|
||||
|
||||
assertEquals(expectedOffsetCommitValueVersion, buffer.getShort(0).toInt)
|
||||
|
@ -2275,10 +2276,10 @@ class GroupMetadataManagerTest {
|
|||
assertEquals(expectedLeaderEpoch, deserializedOffsetAndMetadata.leaderEpoch)
|
||||
}
|
||||
|
||||
for (version <- ApiVersion.allVersions) {
|
||||
for (version <- MetadataVersion.VALUES) {
|
||||
val expectedSchemaVersion = version match {
|
||||
case v if v < KAFKA_2_1_IV0 => 1
|
||||
case v if v < KAFKA_2_1_IV1 => 2
|
||||
case v if v.isLessThan(IBP_2_1_IV0) => 1
|
||||
case v if v.isLessThan(IBP_2_1_IV1) => 2
|
||||
case _ => 3
|
||||
}
|
||||
verifySerde(version, expectedSchemaVersion)
|
||||
|
@ -2297,8 +2298,8 @@ class GroupMetadataManagerTest {
|
|||
commitTimestamp = time.milliseconds(),
|
||||
expireTimestamp = Some(time.milliseconds() + 1000))
|
||||
|
||||
def verifySerde(apiVersion: ApiVersion): Unit = {
|
||||
val bytes = GroupMetadataManager.offsetCommitValue(offsetAndMetadata, apiVersion)
|
||||
def verifySerde(metadataVersion: MetadataVersion): Unit = {
|
||||
val bytes = GroupMetadataManager.offsetCommitValue(offsetAndMetadata, metadataVersion)
|
||||
val buffer = ByteBuffer.wrap(bytes)
|
||||
assertEquals(1, buffer.getShort(0).toInt)
|
||||
|
||||
|
@ -2306,7 +2307,7 @@ class GroupMetadataManagerTest {
|
|||
assertEquals(offsetAndMetadata, deserializedOffsetAndMetadata)
|
||||
}
|
||||
|
||||
for (version <- ApiVersion.allVersions)
|
||||
for (version <- MetadataVersion.VALUES)
|
||||
verifySerde(version)
|
||||
}
|
||||
|
||||
|
@ -2319,13 +2320,13 @@ class GroupMetadataManagerTest {
|
|||
commitTimestamp = time.milliseconds(),
|
||||
expireTimestamp = None)
|
||||
|
||||
def verifySerde(apiVersion: ApiVersion): Unit = {
|
||||
val bytes = GroupMetadataManager.offsetCommitValue(offsetAndMetadata, apiVersion)
|
||||
def verifySerde(metadataVersion: MetadataVersion): Unit = {
|
||||
val bytes = GroupMetadataManager.offsetCommitValue(offsetAndMetadata, metadataVersion)
|
||||
val buffer = ByteBuffer.wrap(bytes)
|
||||
val version = buffer.getShort(0).toInt
|
||||
if (apiVersion < KAFKA_2_1_IV0)
|
||||
if (metadataVersion.isLessThan(IBP_2_1_IV0))
|
||||
assertEquals(1, version)
|
||||
else if (apiVersion < KAFKA_2_1_IV1)
|
||||
else if (metadataVersion.isLessThan(IBP_2_1_IV1))
|
||||
assertEquals(2, version)
|
||||
else
|
||||
assertEquals(3, version)
|
||||
|
@ -2334,7 +2335,7 @@ class GroupMetadataManagerTest {
|
|||
assertEquals(offsetAndMetadata, deserializedOffsetAndMetadata)
|
||||
}
|
||||
|
||||
for (version <- ApiVersion.allVersions)
|
||||
for (version <- MetadataVersion.VALUES)
|
||||
verifySerde(version)
|
||||
}
|
||||
|
||||
|
@ -2397,7 +2398,7 @@ class GroupMetadataManagerTest {
|
|||
val offsetCommitRecord = TestUtils.records(Seq(
|
||||
new SimpleRecord(
|
||||
GroupMetadataManager.offsetCommitKey(groupId, topicPartition),
|
||||
GroupMetadataManager.offsetCommitValue(OffsetAndMetadata(35L, "", time.milliseconds()), ApiVersion.latestVersion)
|
||||
GroupMetadataManager.offsetCommitValue(OffsetAndMetadata(35L, "", time.milliseconds()), MetadataVersion.latest)
|
||||
)
|
||||
)).records.asScala.head
|
||||
val (keyStringOpt, valueStringOpt) = GroupMetadataManager.formatRecordKeyAndValue(offsetCommitRecord)
|
||||
|
@ -2487,20 +2488,20 @@ class GroupMetadataManagerTest {
|
|||
protocol: String,
|
||||
memberId: String,
|
||||
assignmentBytes: Array[Byte] = Array.emptyByteArray,
|
||||
apiVersion: ApiVersion = ApiVersion.latestVersion): SimpleRecord = {
|
||||
metadataVersion: MetadataVersion = MetadataVersion.latest): SimpleRecord = {
|
||||
val memberProtocols = List((protocol, Array.emptyByteArray))
|
||||
val member = new MemberMetadata(memberId, Some(groupInstanceId), "clientId", "clientHost", 30000, 10000, protocolType, memberProtocols)
|
||||
val group = GroupMetadata.loadGroup(groupId, Stable, generation, protocolType, protocol, memberId,
|
||||
if (apiVersion >= KAFKA_2_1_IV0) Some(time.milliseconds()) else None, Seq(member), time)
|
||||
if (metadataVersion.isAtLeast(IBP_2_1_IV0)) Some(time.milliseconds()) else None, Seq(member), time)
|
||||
val groupMetadataKey = GroupMetadataManager.groupMetadataKey(groupId)
|
||||
val groupMetadataValue = GroupMetadataManager.groupMetadataValue(group, Map(memberId -> assignmentBytes), apiVersion)
|
||||
val groupMetadataValue = GroupMetadataManager.groupMetadataValue(group, Map(memberId -> assignmentBytes), metadataVersion)
|
||||
new SimpleRecord(groupMetadataKey, groupMetadataValue)
|
||||
}
|
||||
|
||||
private def buildEmptyGroupRecord(generation: Int, protocolType: String): SimpleRecord = {
|
||||
val group = GroupMetadata.loadGroup(groupId, Empty, generation, protocolType, null, null, None, Seq.empty, time)
|
||||
val groupMetadataKey = GroupMetadataManager.groupMetadataKey(groupId)
|
||||
val groupMetadataValue = GroupMetadataManager.groupMetadataValue(group, Map.empty, ApiVersion.latestVersion)
|
||||
val groupMetadataValue = GroupMetadataManager.groupMetadataValue(group, Map.empty, MetadataVersion.latest)
|
||||
new SimpleRecord(groupMetadataKey, groupMetadataValue)
|
||||
}
|
||||
|
||||
|
@ -2544,7 +2545,7 @@ class GroupMetadataManagerTest {
|
|||
|
||||
private def createCommittedOffsetRecords(committedOffsets: Map[TopicPartition, Long],
|
||||
groupId: String = groupId,
|
||||
apiVersion: ApiVersion = ApiVersion.latestVersion,
|
||||
metadataVersion: MetadataVersion = MetadataVersion.latest,
|
||||
retentionTimeOpt: Option[Long] = None): Seq[SimpleRecord] = {
|
||||
committedOffsets.map { case (topicPartition, offset) =>
|
||||
val commitTimestamp = time.milliseconds()
|
||||
|
@ -2556,7 +2557,7 @@ class GroupMetadataManagerTest {
|
|||
OffsetAndMetadata(offset, "", commitTimestamp)
|
||||
}
|
||||
val offsetCommitKey = GroupMetadataManager.offsetCommitKey(groupId, topicPartition)
|
||||
val offsetCommitValue = GroupMetadataManager.offsetCommitValue(offsetAndMetadata, apiVersion)
|
||||
val offsetCommitValue = GroupMetadataManager.offsetCommitValue(offsetAndMetadata, metadataVersion)
|
||||
new SimpleRecord(offsetCommitKey, offsetCommitValue)
|
||||
}.toSeq
|
||||
}
|
||||
|
|
|
@ -19,13 +19,13 @@ package kafka.log
|
|||
|
||||
import java.io.File
|
||||
import java.util.Properties
|
||||
import kafka.api.KAFKA_0_11_0_IV0
|
||||
import kafka.api.{KAFKA_0_10_0_IV1, KAFKA_0_9_0}
|
||||
|
||||
import kafka.server.KafkaConfig
|
||||
import kafka.server.checkpoints.OffsetCheckpointFile
|
||||
import kafka.utils._
|
||||
import org.apache.kafka.common.TopicPartition
|
||||
import org.apache.kafka.common.record._
|
||||
import org.apache.kafka.server.common.MetadataVersion.{IBP_0_9_0, IBP_0_10_0_IV1, IBP_0_11_0_IV0}
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api.extension.ExtensionContext
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
|
@ -151,7 +151,7 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati
|
|||
|
||||
val log = cleaner.logs.get(topicPartitions(0))
|
||||
val props = logConfigProperties(maxMessageSize = maxMessageSize)
|
||||
props.put(LogConfig.MessageFormatVersionProp, KAFKA_0_9_0.version)
|
||||
props.put(LogConfig.MessageFormatVersionProp, IBP_0_9_0.version)
|
||||
log.updateConfig(new LogConfig(props))
|
||||
|
||||
val appends = writeDups(numKeys = 100, numDups = 3, log = log, codec = codec, magicValue = RecordBatch.MAGIC_VALUE_V0)
|
||||
|
@ -173,7 +173,7 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati
|
|||
val largeMessageOffset = appendInfo.firstOffset.map(_.messageOffset).get
|
||||
|
||||
// also add some messages with version 1 and version 2 to check that we handle mixed format versions correctly
|
||||
props.put(LogConfig.MessageFormatVersionProp, KAFKA_0_11_0_IV0.version)
|
||||
props.put(LogConfig.MessageFormatVersionProp, IBP_0_11_0_IV0.version)
|
||||
log.updateConfig(new LogConfig(props))
|
||||
val dupsV1 = writeDups(startKey = 30, numKeys = 40, numDups = 3, log = log, codec = codec, magicValue = RecordBatch.MAGIC_VALUE_V1)
|
||||
val dupsV2 = writeDups(startKey = 15, numKeys = 5, numDups = 3, log = log, codec = codec, magicValue = RecordBatch.MAGIC_VALUE_V2)
|
||||
|
@ -194,7 +194,7 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati
|
|||
|
||||
val log = cleaner.logs.get(topicPartitions(0))
|
||||
val props = logConfigProperties(maxMessageSize = maxMessageSize, segmentSize = 256)
|
||||
props.put(LogConfig.MessageFormatVersionProp, KAFKA_0_9_0.version)
|
||||
props.put(LogConfig.MessageFormatVersionProp, IBP_0_9_0.version)
|
||||
log.updateConfig(new LogConfig(props))
|
||||
|
||||
// with compression enabled, these messages will be written as a single message containing
|
||||
|
@ -202,7 +202,7 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati
|
|||
var appendsV0 = writeDupsSingleMessageSet(numKeys = 2, numDups = 3, log = log, codec = codec, magicValue = RecordBatch.MAGIC_VALUE_V0)
|
||||
appendsV0 ++= writeDupsSingleMessageSet(numKeys = 2, startKey = 3, numDups = 2, log = log, codec = codec, magicValue = RecordBatch.MAGIC_VALUE_V0)
|
||||
|
||||
props.put(LogConfig.MessageFormatVersionProp, KAFKA_0_10_0_IV1.version)
|
||||
props.put(LogConfig.MessageFormatVersionProp, IBP_0_10_0_IV1.version)
|
||||
log.updateConfig(new LogConfig(props))
|
||||
|
||||
var appendsV1 = writeDupsSingleMessageSet(startKey = 4, numKeys = 2, numDups = 2, log = log, codec = codec, magicValue = RecordBatch.MAGIC_VALUE_V1)
|
||||
|
|
|
@ -17,7 +17,6 @@
|
|||
|
||||
package kafka.log
|
||||
|
||||
import kafka.api.KAFKA_3_0_IV1
|
||||
import kafka.server.{KafkaConfig, ThrottledReplicaListValidator}
|
||||
import kafka.utils.TestUtils
|
||||
import org.apache.kafka.common.config.ConfigDef.Importance.MEDIUM
|
||||
|
@ -25,8 +24,10 @@ import org.apache.kafka.common.config.ConfigDef.Type.INT
|
|||
import org.apache.kafka.common.config.{ConfigException, TopicConfig}
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
import java.util.{Collections, Properties}
|
||||
|
||||
import org.apache.kafka.server.common.MetadataVersion.IBP_3_0_IV1
|
||||
|
||||
import scala.annotation.nowarn
|
||||
|
||||
class LogConfigTest {
|
||||
|
@ -65,7 +66,7 @@ class LogConfigTest {
|
|||
assertEquals(2 * millisInHour, logProps.get(LogConfig.SegmentJitterMsProp))
|
||||
assertEquals(2 * millisInHour, logProps.get(LogConfig.RetentionMsProp))
|
||||
// The message format version should always be 3.0 if the inter-broker protocol version is 3.0 or higher
|
||||
assertEquals(KAFKA_3_0_IV1.version, logProps.get(LogConfig.MessageFormatVersionProp))
|
||||
assertEquals(IBP_3_0_IV1.version, logProps.get(LogConfig.MessageFormatVersionProp))
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -21,7 +21,7 @@ import java.io.{BufferedWriter, File, FileWriter}
|
|||
import java.nio.ByteBuffer
|
||||
import java.nio.file.{Files, NoSuchFileException, Paths}
|
||||
import java.util.Properties
|
||||
import kafka.api.{ApiVersion, KAFKA_0_11_0_IV0}
|
||||
|
||||
import kafka.server.epoch.{EpochEntry, LeaderEpochFileCache}
|
||||
import kafka.server.{BrokerTopicStats, FetchDataInfo, KafkaConfig, LogDirFailureChannel}
|
||||
import kafka.server.metadata.MockConfigRepository
|
||||
|
@ -29,6 +29,8 @@ import kafka.utils.{CoreUtils, MockTime, Scheduler, TestUtils}
|
|||
import org.apache.kafka.common.TopicPartition
|
||||
import org.apache.kafka.common.record.{CompressionType, ControlRecordType, DefaultRecordBatch, MemoryRecords, RecordBatch, RecordVersion, SimpleRecord, TimestampType}
|
||||
import org.apache.kafka.common.utils.{Time, Utils}
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.apache.kafka.server.common.MetadataVersion.IBP_0_11_0_IV0
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertNotEquals, assertThrows, assertTrue}
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
||||
import org.mockito.ArgumentMatchers
|
||||
|
@ -177,12 +179,12 @@ class LogLoaderTest {
|
|||
|
||||
@Test
|
||||
def testProducerSnapshotsRecoveryAfterUncleanShutdownV1(): Unit = {
|
||||
testProducerSnapshotsRecoveryAfterUncleanShutdown(ApiVersion.minSupportedFor(RecordVersion.V1).version)
|
||||
testProducerSnapshotsRecoveryAfterUncleanShutdown(MetadataVersion.minSupportedFor(RecordVersion.V1).version)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testProducerSnapshotsRecoveryAfterUncleanShutdownCurrentMessageFormat(): Unit = {
|
||||
testProducerSnapshotsRecoveryAfterUncleanShutdown(ApiVersion.latestVersion.version)
|
||||
testProducerSnapshotsRecoveryAfterUncleanShutdown(MetadataVersion.latest.version)
|
||||
}
|
||||
|
||||
private def createLog(dir: File,
|
||||
|
@ -267,7 +269,7 @@ class LogLoaderTest {
|
|||
val expectedSegmentsWithReads = mutable.Set[Long]()
|
||||
val expectedSnapshotOffsets = mutable.Set[Long]()
|
||||
|
||||
if (logConfig.messageFormatVersion < KAFKA_0_11_0_IV0) {
|
||||
if (logConfig.messageFormatVersion.isLessThan(IBP_0_11_0_IV0)) {
|
||||
expectedSegmentsWithReads += activeSegmentOffset
|
||||
expectedSnapshotOffsets ++= log.logSegments.map(_.baseOffset).toVector.takeRight(2) :+ log.logEndOffset
|
||||
} else {
|
||||
|
|
|
@ -19,7 +19,6 @@ package kafka.log
|
|||
import java.nio.ByteBuffer
|
||||
import java.util.concurrent.TimeUnit
|
||||
|
||||
import kafka.api.{ApiVersion, KAFKA_2_0_IV1, KAFKA_2_3_IV1}
|
||||
import kafka.common.{LongRef, RecordValidationException}
|
||||
import kafka.log.LogValidator.ValidationAndOffsetAssignResult
|
||||
import kafka.message._
|
||||
|
@ -29,6 +28,7 @@ import org.apache.kafka.common.errors.{InvalidTimestampException, UnsupportedCom
|
|||
import org.apache.kafka.common.record._
|
||||
import org.apache.kafka.common.utils.Time
|
||||
import org.apache.kafka.common.{InvalidRecordException, TopicPartition}
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.apache.kafka.server.metrics.KafkaYammerMetrics
|
||||
import org.apache.kafka.test.TestUtils
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
|
@ -127,7 +127,7 @@ class LogValidatorTest {
|
|||
1000L,
|
||||
RecordBatch.NO_PRODUCER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
KAFKA_2_3_IV1,
|
||||
MetadataVersion.IBP_2_3_IV1,
|
||||
brokerTopicStats,
|
||||
RequestLocal.withThreadConfinedCaching)
|
||||
}
|
||||
|
@ -159,7 +159,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 1000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching)
|
||||
val validatedRecords = validatedResults.validatedRecords
|
||||
|
@ -199,7 +199,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 1000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching)
|
||||
val validatedRecords = validatedResults.validatedRecords
|
||||
|
@ -248,7 +248,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 1000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching)
|
||||
val validatedRecords = validatedResults.validatedRecords
|
||||
|
@ -311,7 +311,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 1000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching)
|
||||
}
|
||||
|
@ -356,7 +356,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 1000L,
|
||||
partitionLeaderEpoch = partitionLeaderEpoch,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching)
|
||||
val validatedRecords = validatingResults.validatedRecords
|
||||
|
@ -429,7 +429,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 1000L,
|
||||
partitionLeaderEpoch = partitionLeaderEpoch,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching)
|
||||
val validatedRecords = validatingResults.validatedRecords
|
||||
|
@ -486,7 +486,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 1000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching)
|
||||
val validatedRecords = validatedResults.validatedRecords
|
||||
|
@ -532,7 +532,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 1000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching)
|
||||
val validatedRecords = validatedResults.validatedRecords
|
||||
|
@ -590,7 +590,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 1000L,
|
||||
partitionLeaderEpoch = partitionLeaderEpoch,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching)
|
||||
val validatedRecords = validatedResults.validatedRecords
|
||||
|
@ -644,7 +644,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 1000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching))
|
||||
}
|
||||
|
@ -668,7 +668,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 1000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching))
|
||||
}
|
||||
|
@ -692,7 +692,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 1000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching))
|
||||
}
|
||||
|
@ -716,7 +716,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 1000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching))
|
||||
}
|
||||
|
@ -739,7 +739,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 1000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords, offset)
|
||||
}
|
||||
|
@ -762,7 +762,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 1000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords, offset)
|
||||
}
|
||||
|
@ -786,7 +786,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 5000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords
|
||||
checkOffsets(messageWithOffset, offset)
|
||||
|
@ -811,7 +811,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 5000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords
|
||||
checkOffsets(messageWithOffset, offset)
|
||||
|
@ -837,7 +837,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 5000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords
|
||||
checkOffsets(compressedMessagesWithOffset, offset)
|
||||
|
@ -863,7 +863,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 5000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords
|
||||
checkOffsets(compressedMessagesWithOffset, offset)
|
||||
|
@ -887,7 +887,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 1000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching)
|
||||
checkOffsets(validatedResults.validatedRecords, offset)
|
||||
|
@ -913,7 +913,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 1000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching)
|
||||
checkOffsets(validatedResults.validatedRecords, offset)
|
||||
|
@ -939,7 +939,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 1000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching)
|
||||
checkOffsets(validatedResults.validatedRecords, offset)
|
||||
|
@ -965,7 +965,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 1000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching)
|
||||
checkOffsets(validatedResults.validatedRecords, offset)
|
||||
|
@ -991,7 +991,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 5000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching))
|
||||
}
|
||||
|
@ -1014,7 +1014,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 5000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Coordinator,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching)
|
||||
val batches = TestUtils.toList(result.validatedRecords.batches)
|
||||
|
@ -1042,7 +1042,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 5000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords, offset)
|
||||
}
|
||||
|
@ -1066,7 +1066,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 5000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords, offset)
|
||||
}
|
||||
|
@ -1089,7 +1089,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 1000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords, offset)
|
||||
}
|
||||
|
@ -1112,7 +1112,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 1000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords, offset)
|
||||
}
|
||||
|
@ -1136,7 +1136,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 5000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords, offset)
|
||||
}
|
||||
|
@ -1160,7 +1160,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 5000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords, offset)
|
||||
}
|
||||
|
@ -1186,7 +1186,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 5000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching))
|
||||
}
|
||||
|
@ -1212,7 +1212,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 5000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching))
|
||||
}
|
||||
|
@ -1236,7 +1236,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 5000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords, offset)
|
||||
}
|
||||
|
@ -1260,7 +1260,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 5000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords, offset)
|
||||
}
|
||||
|
@ -1282,7 +1282,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 5000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching)
|
||||
)
|
||||
|
@ -1313,7 +1313,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 1000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = KAFKA_2_0_IV1,
|
||||
interBrokerProtocolVersion = MetadataVersion.IBP_2_0_IV1,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching))
|
||||
}
|
||||
|
@ -1348,7 +1348,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 1000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching)
|
||||
)
|
||||
|
@ -1427,7 +1427,7 @@ class LogValidatorTest {
|
|||
timestampDiffMaxMs = 5000L,
|
||||
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
origin = AppendOrigin.Client,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
requestLocal = RequestLocal.withThreadConfinedCaching))
|
||||
}
|
||||
|
|
|
@ -18,6 +18,7 @@
|
|||
package kafka.security.auth
|
||||
|
||||
import java.nio.charset.StandardCharsets
|
||||
|
||||
import kafka.admin.ZkSecurityMigrator
|
||||
import kafka.server.QuorumTestHarness
|
||||
import kafka.utils.{Logging, TestUtils}
|
||||
|
@ -30,12 +31,12 @@ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
|
|||
|
||||
import scala.util.{Failure, Success, Try}
|
||||
import javax.security.auth.login.Configuration
|
||||
import kafka.api.ApiVersion
|
||||
import kafka.cluster.{Broker, EndPoint}
|
||||
import kafka.controller.ReplicaAssignment
|
||||
import org.apache.kafka.common.network.ListenerName
|
||||
import org.apache.kafka.common.security.auth.SecurityProtocol
|
||||
import org.apache.kafka.common.utils.Time
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.apache.zookeeper.client.ZKClientConfig
|
||||
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
@ -136,7 +137,7 @@ class ZkAuthorizationTest extends QuorumTestHarness with Logging {
|
|||
private def createBrokerInfo(id: Int, host: String, port: Int, securityProtocol: SecurityProtocol,
|
||||
rack: Option[String] = None): BrokerInfo =
|
||||
BrokerInfo(Broker(id, Seq(new EndPoint(host, port, ListenerName.forSecurityProtocol
|
||||
(securityProtocol), securityProtocol)), rack = rack), ApiVersion.latestVersion, jmxPort = port + 10)
|
||||
(securityProtocol), securityProtocol)), rack = rack), MetadataVersion.latest, jmxPort = port + 10)
|
||||
|
||||
private def newKafkaZkClient(connectionString: String, isSecure: Boolean) =
|
||||
KafkaZkClient(connectionString, isSecure, 6000, 6000, Int.MaxValue, Time.SYSTEM, "ZkAuthorizationTest",
|
||||
|
|
|
@ -22,8 +22,8 @@ import java.nio.charset.StandardCharsets.UTF_8
|
|||
import java.nio.file.Files
|
||||
import java.util.{Collections, UUID}
|
||||
import java.util.concurrent.{Executors, Semaphore, TimeUnit}
|
||||
|
||||
import kafka.Kafka
|
||||
import kafka.api.{ApiVersion, KAFKA_2_0_IV0, KAFKA_2_0_IV1}
|
||||
import kafka.security.authorizer.AclEntry.{WildcardHost, WildcardPrincipalString}
|
||||
import kafka.server.{KafkaConfig, QuorumTestHarness}
|
||||
import kafka.utils.TestUtils
|
||||
|
@ -42,6 +42,8 @@ import org.apache.kafka.common.resource.PatternType.{LITERAL, MATCH, PREFIXED}
|
|||
import org.apache.kafka.common.security.auth.KafkaPrincipal
|
||||
import org.apache.kafka.server.authorizer._
|
||||
import org.apache.kafka.common.utils.{Time, SecurityUtils => JSecurityUtils}
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.apache.kafka.server.common.MetadataVersion.{IBP_2_0_IV0, IBP_2_0_IV1}
|
||||
import org.apache.zookeeper.client.ZKClientConfig
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
|
||||
|
@ -714,7 +716,7 @@ class AclAuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
|
|||
|
||||
@Test
|
||||
def testThrowsOnAddPrefixedAclIfInterBrokerProtocolVersionTooLow(): Unit = {
|
||||
givenAuthorizerWithProtocolVersion(Option(KAFKA_2_0_IV0))
|
||||
givenAuthorizerWithProtocolVersion(Option(IBP_2_0_IV0))
|
||||
val e = assertThrows(classOf[ApiException],
|
||||
() => addAcls(aclAuthorizer, Set(denyReadAcl), new ResourcePattern(TOPIC, "z_other", PREFIXED)))
|
||||
assertTrue(e.getCause.isInstanceOf[UnsupportedVersionException], s"Unexpected exception $e")
|
||||
|
@ -736,7 +738,7 @@ class AclAuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
|
|||
|
||||
@Test
|
||||
def testWritesExtendedAclChangeEventWhenInterBrokerProtocolAtLeastKafkaV2(): Unit = {
|
||||
givenAuthorizerWithProtocolVersion(Option(KAFKA_2_0_IV1))
|
||||
givenAuthorizerWithProtocolVersion(Option(IBP_2_0_IV1))
|
||||
val resource = new ResourcePattern(TOPIC, "z_other", PREFIXED)
|
||||
val expected = new String(ZkAclStore(PREFIXED).changeStore
|
||||
.createChangeNode(resource).bytes, UTF_8)
|
||||
|
@ -750,7 +752,7 @@ class AclAuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
|
|||
|
||||
@Test
|
||||
def testWritesLiteralWritesLiteralAclChangeEventWhenInterBrokerProtocolLessThanKafkaV2eralAclChangesForOlderProtocolVersions(): Unit = {
|
||||
givenAuthorizerWithProtocolVersion(Option(KAFKA_2_0_IV0))
|
||||
givenAuthorizerWithProtocolVersion(Option(IBP_2_0_IV0))
|
||||
val resource = new ResourcePattern(TOPIC, "z_other", LITERAL)
|
||||
val expected = new String(ZkAclStore(LITERAL).changeStore
|
||||
.createChangeNode(resource).bytes, UTF_8)
|
||||
|
@ -764,7 +766,7 @@ class AclAuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
|
|||
|
||||
@Test
|
||||
def testWritesLiteralAclChangeEventWhenInterBrokerProtocolIsKafkaV2(): Unit = {
|
||||
givenAuthorizerWithProtocolVersion(Option(KAFKA_2_0_IV1))
|
||||
givenAuthorizerWithProtocolVersion(Option(IBP_2_0_IV1))
|
||||
val resource = new ResourcePattern(TOPIC, "z_other", LITERAL)
|
||||
val expected = new String(ZkAclStore(LITERAL).changeStore
|
||||
.createChangeNode(resource).bytes, UTF_8)
|
||||
|
@ -994,7 +996,7 @@ class AclAuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
|
|||
}
|
||||
}
|
||||
|
||||
private def givenAuthorizerWithProtocolVersion(protocolVersion: Option[ApiVersion]): Unit = {
|
||||
private def givenAuthorizerWithProtocolVersion(protocolVersion: Option[MetadataVersion]): Unit = {
|
||||
aclAuthorizer.close()
|
||||
|
||||
val props = TestUtils.createBrokerConfig(0, zkConnect)
|
||||
|
|
|
@ -19,9 +19,7 @@ package kafka.server
|
|||
|
||||
import java.util.Collections
|
||||
import java.util.stream.{Stream => JStream}
|
||||
import kafka.api.ApiVersion
|
||||
import kafka.api.KAFKA_2_7_IV2
|
||||
import kafka.api.KAFKA_3_2_IV0
|
||||
|
||||
import kafka.api.LeaderAndIsr
|
||||
import kafka.utils.{MockScheduler, MockTime}
|
||||
import kafka.zk.KafkaZkClient
|
||||
|
@ -33,6 +31,8 @@ import org.apache.kafka.common.metrics.Metrics
|
|||
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
|
||||
import org.apache.kafka.common.requests.{AbstractRequest, AlterPartitionRequest, AlterPartitionResponse}
|
||||
import org.apache.kafka.metadata.LeaderRecoveryState
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.apache.kafka.server.common.MetadataVersion.{IBP_2_7_IV2, IBP_3_2_IV0}
|
||||
import org.apache.kafka.test.TestUtils.assertFutureThrows
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api.BeforeEach
|
||||
|
@ -43,6 +43,7 @@ import org.junit.jupiter.params.provider.MethodSource
|
|||
import org.mockito.ArgumentMatchers.{any, anyString}
|
||||
import org.mockito.Mockito.{mock, reset, times, verify}
|
||||
import org.mockito.{ArgumentCaptor, ArgumentMatchers, Mockito}
|
||||
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
||||
class AlterPartitionManagerTest {
|
||||
|
@ -64,10 +65,10 @@ class AlterPartitionManagerTest {
|
|||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource(Array("provideApiVersions"))
|
||||
def testBasic(apiVersion: ApiVersion): Unit = {
|
||||
@MethodSource(Array("provideMetadataVersions"))
|
||||
def testBasic(metadataVersion: MetadataVersion): Unit = {
|
||||
val scheduler = new MockScheduler(time)
|
||||
val alterIsrManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, apiVersion)
|
||||
val alterIsrManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, metadataVersion)
|
||||
alterIsrManager.start()
|
||||
alterIsrManager.submit(tp0, new LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10), 0)
|
||||
verify(brokerToController).start()
|
||||
|
@ -77,31 +78,31 @@ class AlterPartitionManagerTest {
|
|||
@ParameterizedTest
|
||||
@MethodSource(Array("provideLeaderRecoveryState"))
|
||||
def testBasicSentLeaderRecoveryState(
|
||||
apiVersion: ApiVersion,
|
||||
metadataVersion: MetadataVersion,
|
||||
leaderRecoveryState: LeaderRecoveryState
|
||||
): Unit = {
|
||||
val requestCapture = ArgumentCaptor.forClass(classOf[AbstractRequest.Builder[AlterPartitionRequest]])
|
||||
|
||||
val scheduler = new MockScheduler(time)
|
||||
val alterIsrManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, apiVersion)
|
||||
val alterIsrManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, metadataVersion)
|
||||
alterIsrManager.start()
|
||||
alterIsrManager.submit(tp0, new LeaderAndIsr(1, 1, List(1), leaderRecoveryState, 10), 0)
|
||||
verify(brokerToController).start()
|
||||
verify(brokerToController).sendRequest(requestCapture.capture(), any())
|
||||
|
||||
val request = requestCapture.getValue.build()
|
||||
val expectedLeaderRecoveryState = if (apiVersion >= KAFKA_3_2_IV0) leaderRecoveryState else LeaderRecoveryState.RECOVERED
|
||||
val expectedLeaderRecoveryState = if (metadataVersion.isAtLeast(IBP_3_2_IV0)) leaderRecoveryState else LeaderRecoveryState.RECOVERED
|
||||
assertEquals(expectedLeaderRecoveryState.value, request.data.topics.get(0).partitions.get(0).leaderRecoveryState())
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource(Array("provideApiVersions"))
|
||||
def testOverwriteWithinBatch(apiVersion: ApiVersion): Unit = {
|
||||
@MethodSource(Array("provideMetadataVersions"))
|
||||
def testOverwriteWithinBatch(metadataVersion: MetadataVersion): Unit = {
|
||||
val capture: ArgumentCaptor[AbstractRequest.Builder[AlterPartitionRequest]] = ArgumentCaptor.forClass(classOf[AbstractRequest.Builder[AlterPartitionRequest]])
|
||||
val callbackCapture: ArgumentCaptor[ControllerRequestCompletionHandler] = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler])
|
||||
|
||||
val scheduler = new MockScheduler(time)
|
||||
val alterIsrManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, apiVersion)
|
||||
val alterIsrManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, metadataVersion)
|
||||
alterIsrManager.start()
|
||||
|
||||
// Only send one ISR update for a given topic+partition
|
||||
|
@ -133,13 +134,13 @@ class AlterPartitionManagerTest {
|
|||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource(Array("provideApiVersions"))
|
||||
def testSingleBatch(apiVersion: ApiVersion): Unit = {
|
||||
@MethodSource(Array("provideMetadataVersions"))
|
||||
def testSingleBatch(metadataVersion: MetadataVersion): Unit = {
|
||||
val capture: ArgumentCaptor[AbstractRequest.Builder[AlterPartitionRequest]] = ArgumentCaptor.forClass(classOf[AbstractRequest.Builder[AlterPartitionRequest]])
|
||||
val callbackCapture: ArgumentCaptor[ControllerRequestCompletionHandler] = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler])
|
||||
|
||||
val scheduler = new MockScheduler(time)
|
||||
val alterIsrManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, apiVersion)
|
||||
val alterIsrManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, metadataVersion)
|
||||
alterIsrManager.start()
|
||||
|
||||
// First request will send batch of one
|
||||
|
@ -209,7 +210,7 @@ class AlterPartitionManagerTest {
|
|||
val callbackCapture: ArgumentCaptor[ControllerRequestCompletionHandler] = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler])
|
||||
|
||||
val scheduler = new MockScheduler(time)
|
||||
val alterIsrManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, KAFKA_3_2_IV0)
|
||||
val alterIsrManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, IBP_3_2_IV0)
|
||||
alterIsrManager.start()
|
||||
alterIsrManager.submit(tp0, leaderAndIsr, 0)
|
||||
|
||||
|
@ -269,7 +270,7 @@ class AlterPartitionManagerTest {
|
|||
reset(brokerToController)
|
||||
|
||||
val scheduler = new MockScheduler(time)
|
||||
val alterIsrManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, KAFKA_3_2_IV0)
|
||||
val alterIsrManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, IBP_3_2_IV0)
|
||||
alterIsrManager.start()
|
||||
|
||||
val future = alterIsrManager.submit(tp, LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10), 0)
|
||||
|
@ -288,12 +289,12 @@ class AlterPartitionManagerTest {
|
|||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource(Array("provideApiVersions"))
|
||||
def testOneInFlight(apiVersion: ApiVersion): Unit = {
|
||||
@MethodSource(Array("provideMetadataVersions"))
|
||||
def testOneInFlight(metadataVersion: MetadataVersion): Unit = {
|
||||
val callbackCapture: ArgumentCaptor[ControllerRequestCompletionHandler] = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler])
|
||||
|
||||
val scheduler = new MockScheduler(time)
|
||||
val alterIsrManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, apiVersion)
|
||||
val alterIsrManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, metadataVersion)
|
||||
alterIsrManager.start()
|
||||
|
||||
// First submit will send the request
|
||||
|
@ -316,13 +317,13 @@ class AlterPartitionManagerTest {
|
|||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource(Array("provideApiVersions"))
|
||||
def testPartitionMissingInResponse(apiVersion: ApiVersion): Unit = {
|
||||
@MethodSource(Array("provideMetadataVersions"))
|
||||
def testPartitionMissingInResponse(metadataVersion: MetadataVersion): Unit = {
|
||||
brokerToController = Mockito.mock(classOf[BrokerToControllerChannelManager])
|
||||
|
||||
val brokerEpoch = 2
|
||||
val scheduler = new MockScheduler(time)
|
||||
val alterIsrManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => brokerEpoch, apiVersion)
|
||||
val alterIsrManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => brokerEpoch, metadataVersion)
|
||||
alterIsrManager.start()
|
||||
|
||||
def matchesAlterIsr(topicPartitions: Set[TopicPartition]): AbstractRequest.Builder[_ <: AbstractRequest] = {
|
||||
|
@ -422,21 +423,21 @@ class AlterPartitionManagerTest {
|
|||
}
|
||||
|
||||
object AlterPartitionManagerTest {
|
||||
def provideApiVersions(): JStream[ApiVersion] = {
|
||||
def provideMetadataVersions(): JStream[MetadataVersion] = {
|
||||
JStream.of(
|
||||
// Supports KIP-704: unclean leader recovery
|
||||
KAFKA_3_2_IV0,
|
||||
IBP_3_2_IV0,
|
||||
// Supports KIP-497: alter partition
|
||||
KAFKA_2_7_IV2
|
||||
IBP_2_7_IV2
|
||||
)
|
||||
}
|
||||
|
||||
def provideLeaderRecoveryState(): JStream[Arguments] = {
|
||||
// Multiply apiVersions by leaderRecoveryState
|
||||
provideApiVersions().flatMap { apiVersion =>
|
||||
// Multiply metadataVersions by leaderRecoveryState
|
||||
provideMetadataVersions().flatMap { metadataVersion =>
|
||||
JStream.of(
|
||||
Arguments.of(apiVersion, LeaderRecoveryState.RECOVERED),
|
||||
Arguments.of(apiVersion, LeaderRecoveryState.RECOVERING)
|
||||
Arguments.of(metadataVersion, LeaderRecoveryState.RECOVERED),
|
||||
Arguments.of(metadataVersion, LeaderRecoveryState.RECOVERING)
|
||||
)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -16,10 +16,10 @@
|
|||
*/
|
||||
package kafka.server
|
||||
|
||||
import kafka.api.ApiVersion
|
||||
import org.apache.kafka.clients.NodeApiVersions
|
||||
import org.apache.kafka.common.message.ApiMessageType.ListenerType
|
||||
import org.apache.kafka.common.protocol.ApiKeys
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.junit.jupiter.api.{Disabled, Test}
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
|
@ -37,7 +37,7 @@ class ApiVersionManagerTest {
|
|||
def testApiScope(apiScope: ListenerType): Unit = {
|
||||
val versionManager = new DefaultApiVersionManager(
|
||||
listenerType = apiScope,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
forwardingManager = None,
|
||||
features = brokerFeatures,
|
||||
featureCache = featureCache
|
||||
|
@ -61,7 +61,7 @@ class ApiVersionManagerTest {
|
|||
|
||||
val versionManager = new DefaultApiVersionManager(
|
||||
listenerType = ListenerType.ZK_BROKER,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
forwardingManager = Some(forwardingManager),
|
||||
features = brokerFeatures,
|
||||
featureCache = featureCache
|
||||
|
@ -82,7 +82,7 @@ class ApiVersionManagerTest {
|
|||
for (forwardingManagerOpt <- Seq(Some(forwardingManager), None)) {
|
||||
val versionManager = new DefaultApiVersionManager(
|
||||
listenerType = ListenerType.BROKER,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
forwardingManager = forwardingManagerOpt,
|
||||
features = brokerFeatures,
|
||||
featureCache = featureCache
|
||||
|
@ -104,7 +104,7 @@ class ApiVersionManagerTest {
|
|||
|
||||
val versionManager = new DefaultApiVersionManager(
|
||||
listenerType = ListenerType.ZK_BROKER,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
forwardingManager = Some(forwardingManager),
|
||||
features = brokerFeatures,
|
||||
featureCache = featureCache
|
||||
|
@ -123,7 +123,7 @@ class ApiVersionManagerTest {
|
|||
def testEnvelopeDisabledWhenForwardingManagerEmpty(): Unit = {
|
||||
val versionManager = new DefaultApiVersionManager(
|
||||
listenerType = ListenerType.ZK_BROKER,
|
||||
interBrokerProtocolVersion = ApiVersion.latestVersion,
|
||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||
forwardingManager = None,
|
||||
features = brokerFeatures,
|
||||
featureCache = featureCache
|
||||
|
|
|
@ -16,7 +16,6 @@
|
|||
*/
|
||||
package kafka.server
|
||||
|
||||
import kafka.api.KAFKA_3_0_IV1
|
||||
import java.net.InetAddress
|
||||
import java.nio.charset.StandardCharsets
|
||||
import java.util
|
||||
|
@ -42,6 +41,7 @@ import org.apache.kafka.common.quota.ClientQuotaEntity.{CLIENT_ID, IP, USER}
|
|||
import org.apache.kafka.common.quota.{ClientQuotaAlteration, ClientQuotaEntity}
|
||||
import org.apache.kafka.common.record.{CompressionType, RecordVersion}
|
||||
import org.apache.kafka.common.security.auth.KafkaPrincipal
|
||||
import org.apache.kafka.server.common.MetadataVersion.IBP_3_0_IV1
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api.Test
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
|
@ -154,7 +154,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
|
|||
"Topic metadata propagation failed")
|
||||
val log = server.logManager.getLog(tp).get
|
||||
// message format version should always be 3.0 if inter-broker protocol is 3.0 or higher
|
||||
assertEquals(KAFKA_3_0_IV1, log.config.messageFormatVersion)
|
||||
assertEquals(IBP_3_0_IV1, log.config.messageFormatVersion)
|
||||
assertEquals(RecordVersion.V2, log.config.recordVersion)
|
||||
|
||||
val compressionType = CompressionType.LZ4.name
|
||||
|
@ -165,7 +165,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
|
|||
TestUtils.waitUntilTrue(() =>
|
||||
server.logManager.getLog(tp).get.config.compressionType == compressionType,
|
||||
"Topic config change propagation failed")
|
||||
assertEquals(KAFKA_3_0_IV1, log.config.messageFormatVersion)
|
||||
assertEquals(IBP_3_0_IV1, log.config.messageFormatVersion)
|
||||
assertEquals(RecordVersion.V2, log.config.recordVersion)
|
||||
}
|
||||
|
||||
|
|
|
@ -16,16 +16,16 @@
|
|||
*/
|
||||
package kafka.server
|
||||
|
||||
import kafka.api.KAFKA_0_10_2_IV0
|
||||
import kafka.log.LogConfig
|
||||
import org.apache.kafka.clients.producer.ProducerRecord
|
||||
import org.apache.kafka.common.protocol.Errors
|
||||
import org.apache.kafka.common.requests.{FetchRequest, FetchResponse}
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
import java.util.Properties
|
||||
|
||||
import org.apache.kafka.server.common.MetadataVersion.IBP_0_10_2_IV0
|
||||
|
||||
import scala.annotation.nowarn
|
||||
import scala.collection.Seq
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
@ -49,7 +49,7 @@ class FetchRequestWithLegacyMessageFormatTest extends BaseFetchRequestTest {
|
|||
val maxPartitionBytes = 200
|
||||
// Fetch v2 down-converts if the message format is >= 0.11 and we want to avoid
|
||||
// that as it affects the size of the returned buffer
|
||||
val topicConfig = Map(LogConfig.MessageFormatVersionProp -> KAFKA_0_10_2_IV0.version)
|
||||
val topicConfig = Map(LogConfig.MessageFormatVersionProp -> IBP_0_10_2_IV0.version)
|
||||
val (topicPartition, leaderId) = createTopics(numTopics = 1, numPartitions = 1, topicConfig).head
|
||||
val topicIds = getTopicIds().asJava
|
||||
val topicNames = topicIds.asScala.map(_.swap).asJava
|
||||
|
|
|
@ -23,7 +23,8 @@ import java.util
|
|||
import java.util.Arrays.asList
|
||||
import java.util.concurrent.TimeUnit
|
||||
import java.util.{Collections, Optional, Properties, Random}
|
||||
import kafka.api.{ApiVersion, KAFKA_0_10_2_IV0, KAFKA_2_2_IV1, LeaderAndIsr}
|
||||
|
||||
import kafka.api.LeaderAndIsr
|
||||
import kafka.cluster.Broker
|
||||
import kafka.controller.{ControllerContext, KafkaController}
|
||||
import kafka.coordinator.group.GroupCoordinatorConcurrencyTest.{JoinGroupCallback, SyncGroupCallback}
|
||||
|
@ -91,8 +92,9 @@ import org.mockito.{ArgumentCaptor, ArgumentMatchers, Mockito}
|
|||
|
||||
import scala.collection.{Map, Seq, mutable}
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
||||
import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.apache.kafka.server.common.MetadataVersion.{IBP_0_10_2_IV0, IBP_2_2_IV1}
|
||||
|
||||
class KafkaApisTest {
|
||||
private val requestChannel: RequestChannel = mock(classOf[RequestChannel])
|
||||
|
@ -132,7 +134,7 @@ class KafkaApisTest {
|
|||
metrics.close()
|
||||
}
|
||||
|
||||
def createKafkaApis(interBrokerProtocolVersion: ApiVersion = ApiVersion.latestVersion,
|
||||
def createKafkaApis(interBrokerProtocolVersion: MetadataVersion = MetadataVersion.latest,
|
||||
authorizer: Option[Authorizer] = None,
|
||||
enableForwarding: Boolean = false,
|
||||
configRepository: ConfigRepository = new MockConfigRepository(),
|
||||
|
@ -1649,31 +1651,31 @@ class KafkaApisTest {
|
|||
@Test
|
||||
def shouldThrowUnsupportedVersionExceptionOnHandleAddOffsetToTxnRequestWhenInterBrokerProtocolNotSupported(): Unit = {
|
||||
assertThrows(classOf[UnsupportedVersionException],
|
||||
() => createKafkaApis(KAFKA_0_10_2_IV0).handleAddOffsetsToTxnRequest(null, RequestLocal.withThreadConfinedCaching))
|
||||
() => createKafkaApis(IBP_0_10_2_IV0).handleAddOffsetsToTxnRequest(null, RequestLocal.withThreadConfinedCaching))
|
||||
}
|
||||
|
||||
@Test
|
||||
def shouldThrowUnsupportedVersionExceptionOnHandleAddPartitionsToTxnRequestWhenInterBrokerProtocolNotSupported(): Unit = {
|
||||
assertThrows(classOf[UnsupportedVersionException],
|
||||
() => createKafkaApis(KAFKA_0_10_2_IV0).handleAddPartitionToTxnRequest(null, RequestLocal.withThreadConfinedCaching))
|
||||
() => createKafkaApis(IBP_0_10_2_IV0).handleAddPartitionToTxnRequest(null, RequestLocal.withThreadConfinedCaching))
|
||||
}
|
||||
|
||||
@Test
|
||||
def shouldThrowUnsupportedVersionExceptionOnHandleTxnOffsetCommitRequestWhenInterBrokerProtocolNotSupported(): Unit = {
|
||||
assertThrows(classOf[UnsupportedVersionException],
|
||||
() => createKafkaApis(KAFKA_0_10_2_IV0).handleAddPartitionToTxnRequest(null, RequestLocal.withThreadConfinedCaching))
|
||||
() => createKafkaApis(IBP_0_10_2_IV0).handleAddPartitionToTxnRequest(null, RequestLocal.withThreadConfinedCaching))
|
||||
}
|
||||
|
||||
@Test
|
||||
def shouldThrowUnsupportedVersionExceptionOnHandleEndTxnRequestWhenInterBrokerProtocolNotSupported(): Unit = {
|
||||
assertThrows(classOf[UnsupportedVersionException],
|
||||
() => createKafkaApis(KAFKA_0_10_2_IV0).handleEndTxnRequest(null, RequestLocal.withThreadConfinedCaching))
|
||||
() => createKafkaApis(IBP_0_10_2_IV0).handleEndTxnRequest(null, RequestLocal.withThreadConfinedCaching))
|
||||
}
|
||||
|
||||
@Test
|
||||
def shouldThrowUnsupportedVersionExceptionOnHandleWriteTxnMarkersRequestWhenInterBrokerProtocolNotSupported(): Unit = {
|
||||
assertThrows(classOf[UnsupportedVersionException],
|
||||
() => createKafkaApis(KAFKA_0_10_2_IV0).handleWriteTxnMarkersRequest(null, RequestLocal.withThreadConfinedCaching))
|
||||
() => createKafkaApis(IBP_0_10_2_IV0).handleWriteTxnMarkersRequest(null, RequestLocal.withThreadConfinedCaching))
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -2782,7 +2784,7 @@ class KafkaApisTest {
|
|||
|
||||
val requestChannelRequest = buildRequest(joinGroupRequest)
|
||||
|
||||
createKafkaApis(KAFKA_2_2_IV1).handleJoinGroupRequest(requestChannelRequest, RequestLocal.withThreadConfinedCaching)
|
||||
createKafkaApis(IBP_2_2_IV1).handleJoinGroupRequest(requestChannelRequest, RequestLocal.withThreadConfinedCaching)
|
||||
|
||||
val capturedResponse = verifyNoThrottling(requestChannelRequest)
|
||||
val response = capturedResponse.getValue.asInstanceOf[JoinGroupResponse]
|
||||
|
@ -2801,7 +2803,7 @@ class KafkaApisTest {
|
|||
|
||||
val requestChannelRequest = buildRequest(syncGroupRequest)
|
||||
|
||||
createKafkaApis(KAFKA_2_2_IV1).handleSyncGroupRequest(requestChannelRequest, RequestLocal.withThreadConfinedCaching)
|
||||
createKafkaApis(IBP_2_2_IV1).handleSyncGroupRequest(requestChannelRequest, RequestLocal.withThreadConfinedCaching)
|
||||
|
||||
val capturedResponse = verifyNoThrottling(requestChannelRequest)
|
||||
val response = capturedResponse.getValue.asInstanceOf[SyncGroupResponse]
|
||||
|
@ -2819,7 +2821,7 @@ class KafkaApisTest {
|
|||
).build()
|
||||
val requestChannelRequest = buildRequest(heartbeatRequest)
|
||||
|
||||
createKafkaApis(KAFKA_2_2_IV1).handleHeartbeatRequest(requestChannelRequest)
|
||||
createKafkaApis(IBP_2_2_IV1).handleHeartbeatRequest(requestChannelRequest)
|
||||
|
||||
val capturedResponse = verifyNoThrottling(requestChannelRequest)
|
||||
val response = capturedResponse.getValue.asInstanceOf[HeartbeatResponse]
|
||||
|
@ -2849,7 +2851,7 @@ class KafkaApisTest {
|
|||
|
||||
val requestChannelRequest = buildRequest(offsetCommitRequest)
|
||||
|
||||
createKafkaApis(KAFKA_2_2_IV1).handleOffsetCommitRequest(requestChannelRequest, RequestLocal.withThreadConfinedCaching)
|
||||
createKafkaApis(IBP_2_2_IV1).handleOffsetCommitRequest(requestChannelRequest, RequestLocal.withThreadConfinedCaching)
|
||||
|
||||
val expectedTopicErrors = Collections.singletonList(
|
||||
new OffsetCommitResponseData.OffsetCommitResponseTopic()
|
||||
|
@ -2991,7 +2993,7 @@ class KafkaApisTest {
|
|||
|
||||
val requestChannelRequest = buildRequest(initProducerIdRequest)
|
||||
|
||||
createKafkaApis(KAFKA_2_2_IV1).handleInitProducerIdRequest(requestChannelRequest, RequestLocal.withThreadConfinedCaching)
|
||||
createKafkaApis(IBP_2_2_IV1).handleInitProducerIdRequest(requestChannelRequest, RequestLocal.withThreadConfinedCaching)
|
||||
|
||||
val capturedResponse = verifyNoThrottling(requestChannelRequest)
|
||||
val response = capturedResponse.getValue.asInstanceOf[InitProducerIdResponse]
|
||||
|
@ -3009,7 +3011,7 @@ class KafkaApisTest {
|
|||
).build()
|
||||
val requestChannelRequest = buildRequest(initProducerIdRequest)
|
||||
|
||||
createKafkaApis(KAFKA_2_2_IV1).handleInitProducerIdRequest(requestChannelRequest, RequestLocal.withThreadConfinedCaching)
|
||||
createKafkaApis(IBP_2_2_IV1).handleInitProducerIdRequest(requestChannelRequest, RequestLocal.withThreadConfinedCaching)
|
||||
|
||||
val capturedResponse = verifyNoThrottling(requestChannelRequest)
|
||||
val response = capturedResponse.getValue.asInstanceOf[InitProducerIdResponse]
|
||||
|
|
|
@ -17,7 +17,6 @@
|
|||
|
||||
package kafka.server
|
||||
|
||||
import kafka.api.{ApiVersion, KAFKA_0_8_2, KAFKA_3_0_IV1}
|
||||
import kafka.cluster.EndPoint
|
||||
import kafka.log.LogConfig
|
||||
import kafka.message._
|
||||
|
@ -32,11 +31,13 @@ import org.apache.kafka.raft.RaftConfig
|
|||
import org.apache.kafka.raft.RaftConfig.{AddressSpec, InetAddressSpec, UNKNOWN_ADDRESS_SPEC_INSTANCE}
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
import java.net.InetSocketAddress
|
||||
import java.util
|
||||
import java.util.{Collections, Properties}
|
||||
|
||||
import org.apache.kafka.common.Node
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.apache.kafka.server.common.MetadataVersion.{IBP_0_8_2, IBP_3_0_IV1}
|
||||
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
|
||||
import org.junit.jupiter.api.function.Executable
|
||||
|
||||
|
@ -547,23 +548,23 @@ class KafkaConfigTest {
|
|||
props.put(KafkaConfig.BrokerIdProp, "1")
|
||||
props.put(KafkaConfig.ZkConnectProp, "localhost:2181")
|
||||
val conf = KafkaConfig.fromProps(props)
|
||||
assertEquals(ApiVersion.latestVersion, conf.interBrokerProtocolVersion)
|
||||
assertEquals(MetadataVersion.latest, conf.interBrokerProtocolVersion)
|
||||
|
||||
props.put(KafkaConfig.InterBrokerProtocolVersionProp, "0.8.2.0")
|
||||
// We need to set the message format version to make the configuration valid.
|
||||
props.put(KafkaConfig.LogMessageFormatVersionProp, "0.8.2.0")
|
||||
val conf2 = KafkaConfig.fromProps(props)
|
||||
assertEquals(KAFKA_0_8_2, conf2.interBrokerProtocolVersion)
|
||||
assertEquals(IBP_0_8_2, conf2.interBrokerProtocolVersion)
|
||||
|
||||
// check that 0.8.2.0 is the same as 0.8.2.1
|
||||
props.put(KafkaConfig.InterBrokerProtocolVersionProp, "0.8.2.1")
|
||||
// We need to set the message format version to make the configuration valid
|
||||
props.put(KafkaConfig.LogMessageFormatVersionProp, "0.8.2.1")
|
||||
val conf3 = KafkaConfig.fromProps(props)
|
||||
assertEquals(KAFKA_0_8_2, conf3.interBrokerProtocolVersion)
|
||||
assertEquals(IBP_0_8_2, conf3.interBrokerProtocolVersion)
|
||||
|
||||
//check that latest is newer than 0.8.2
|
||||
assertTrue(ApiVersion.latestVersion >= conf3.interBrokerProtocolVersion)
|
||||
assertTrue(MetadataVersion.latest.isAtLeast(conf3.interBrokerProtocolVersion))
|
||||
}
|
||||
|
||||
private def isValidKafkaConfig(props: Properties): Boolean = {
|
||||
|
@ -690,20 +691,20 @@ class KafkaConfigTest {
|
|||
@nowarn("cat=deprecation")
|
||||
@Test
|
||||
def testInterBrokerVersionMessageFormatCompatibility(): Unit = {
|
||||
def buildConfig(interBrokerProtocol: ApiVersion, messageFormat: ApiVersion): KafkaConfig = {
|
||||
def buildConfig(interBrokerProtocol: MetadataVersion, messageFormat: MetadataVersion): KafkaConfig = {
|
||||
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
|
||||
props.put(KafkaConfig.InterBrokerProtocolVersionProp, interBrokerProtocol.version)
|
||||
props.put(KafkaConfig.LogMessageFormatVersionProp, messageFormat.version)
|
||||
KafkaConfig.fromProps(props)
|
||||
}
|
||||
|
||||
ApiVersion.allVersions.foreach { interBrokerVersion =>
|
||||
ApiVersion.allVersions.foreach { messageFormatVersion =>
|
||||
if (interBrokerVersion.recordVersion.value >= messageFormatVersion.recordVersion.value) {
|
||||
MetadataVersion.VALUES.foreach { interBrokerVersion =>
|
||||
MetadataVersion.VALUES.foreach { messageFormatVersion =>
|
||||
if (interBrokerVersion.highestSupportedRecordVersion.value >= messageFormatVersion.highestSupportedRecordVersion.value) {
|
||||
val config = buildConfig(interBrokerVersion, messageFormatVersion)
|
||||
assertEquals(interBrokerVersion, config.interBrokerProtocolVersion)
|
||||
if (interBrokerVersion >= KAFKA_3_0_IV1)
|
||||
assertEquals(KAFKA_3_0_IV1, config.logMessageFormatVersion)
|
||||
if (interBrokerVersion.isAtLeast(IBP_3_0_IV1))
|
||||
assertEquals(IBP_3_0_IV1, config.logMessageFormatVersion)
|
||||
else
|
||||
assertEquals(messageFormatVersion, config.logMessageFormatVersion)
|
||||
} else {
|
||||
|
|
|
@ -17,14 +17,14 @@
|
|||
|
||||
package kafka.server
|
||||
|
||||
import kafka.api.ApiVersion
|
||||
import kafka.utils.TestUtils
|
||||
import org.apache.kafka.common.security.JaasUtils
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertNull, assertThrows, fail}
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
import java.util.Properties
|
||||
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
|
||||
class KafkaServerTest extends QuorumTestHarness {
|
||||
|
||||
@Test
|
||||
|
@ -126,7 +126,7 @@ class KafkaServerTest extends QuorumTestHarness {
|
|||
@Test
|
||||
def testAlterIsrManager(): Unit = {
|
||||
val props = TestUtils.createBrokerConfigs(1, zkConnect).head
|
||||
props.put(KafkaConfig.InterBrokerProtocolVersionProp, ApiVersion.latestVersion.toString)
|
||||
props.put(KafkaConfig.InterBrokerProtocolVersionProp, MetadataVersion.latest.toString)
|
||||
|
||||
val server = TestUtils.createServer(KafkaConfig.fromProps(props))
|
||||
server.replicaManager.alterPartitionManager match {
|
||||
|
|
|
@ -16,7 +16,6 @@
|
|||
*/
|
||||
package kafka.server
|
||||
|
||||
import kafka.api.{ApiVersion, KAFKA_2_6_IV0}
|
||||
import kafka.cluster.{BrokerEndPoint, Partition}
|
||||
import kafka.log.{LogAppendInfo, LogManager, UnifiedLog}
|
||||
import kafka.server.AbstractFetcherThread.ResultWithPartitions
|
||||
|
@ -40,10 +39,13 @@ import org.junit.jupiter.api.{AfterEach, Test}
|
|||
import org.mockito.ArgumentCaptor
|
||||
import org.mockito.ArgumentMatchers.{any, anyBoolean, anyLong}
|
||||
import org.mockito.Mockito.{mock, never, times, verify, when}
|
||||
|
||||
import java.nio.charset.StandardCharsets
|
||||
import java.util
|
||||
import java.util.{Collections, Optional}
|
||||
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.apache.kafka.server.common.MetadataVersion.IBP_2_6_IV0
|
||||
|
||||
import scala.collection.{Map, mutable}
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
||||
|
@ -250,15 +252,15 @@ class ReplicaFetcherThreadTest {
|
|||
|
||||
@Test
|
||||
def shouldFetchLeaderEpochOnFirstFetchOnlyIfLeaderEpochKnownToBothIbp26(): Unit = {
|
||||
verifyFetchLeaderEpochOnFirstFetch(KAFKA_2_6_IV0)
|
||||
verifyFetchLeaderEpochOnFirstFetch(IBP_2_6_IV0)
|
||||
}
|
||||
|
||||
@Test
|
||||
def shouldNotFetchLeaderEpochOnFirstFetchWithTruncateOnFetch(): Unit = {
|
||||
verifyFetchLeaderEpochOnFirstFetch(ApiVersion.latestVersion, epochFetchCount = 0)
|
||||
verifyFetchLeaderEpochOnFirstFetch(MetadataVersion.latest, epochFetchCount = 0)
|
||||
}
|
||||
|
||||
private def verifyFetchLeaderEpochOnFirstFetch(ibp: ApiVersion, epochFetchCount: Int = 1): Unit = {
|
||||
private def verifyFetchLeaderEpochOnFirstFetch(ibp: MetadataVersion, epochFetchCount: Int = 1): Unit = {
|
||||
val props = TestUtils.createBrokerConfig(1, "localhost:1234")
|
||||
props.setProperty(KafkaConfig.InterBrokerProtocolVersionProp, ibp.version)
|
||||
val config = KafkaConfig.fromProps(props)
|
||||
|
@ -1074,7 +1076,7 @@ class ReplicaFetcherThreadTest {
|
|||
|
||||
private def kafkaConfigNoTruncateOnFetch: KafkaConfig = {
|
||||
val props = TestUtils.createBrokerConfig(1, "localhost:1234")
|
||||
props.setProperty(KafkaConfig.InterBrokerProtocolVersionProp, KAFKA_2_6_IV0.version)
|
||||
props.setProperty(KafkaConfig.InterBrokerProtocolVersionProp, IBP_2_6_IV0.version)
|
||||
KafkaConfig.fromProps(props)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -54,6 +54,7 @@ import org.apache.kafka.common.utils.{Time, Utils}
|
|||
import org.apache.kafka.common.{IsolationLevel, Node, TopicIdPartition, TopicPartition, Uuid}
|
||||
import org.apache.kafka.image.{AclsImage, ClientQuotasImage, ClusterImageTest, ConfigurationsImage, FeaturesImage, MetadataImage, ProducerIdsImage, TopicsDelta, TopicsImage}
|
||||
import org.apache.kafka.raft.{OffsetAndEpoch => RaftOffsetAndEpoch}
|
||||
import org.apache.kafka.server.common.MetadataVersion.IBP_2_6_IV0
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
|
@ -1084,7 +1085,7 @@ class ReplicaManagerTest {
|
|||
@Test
|
||||
def testBecomeFollowerWhenLeaderIsUnchangedButMissedLeaderUpdateIbp26(): Unit = {
|
||||
val extraProps = new Properties
|
||||
extraProps.put(KafkaConfig.InterBrokerProtocolVersionProp, KAFKA_2_6_IV0.version)
|
||||
extraProps.put(KafkaConfig.InterBrokerProtocolVersionProp, IBP_2_6_IV0.version)
|
||||
verifyBecomeFollowerWhenLeaderIsUnchangedButMissedLeaderUpdate(extraProps, expectTruncation = true)
|
||||
}
|
||||
|
||||
|
|
|
@ -19,7 +19,6 @@ package kafka.server
|
|||
|
||||
import java.util.{Arrays, LinkedHashMap, Optional, Properties}
|
||||
|
||||
import kafka.api.KAFKA_2_7_IV0
|
||||
import kafka.network.SocketServer
|
||||
import kafka.utils.TestUtils
|
||||
import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid}
|
||||
|
@ -27,6 +26,7 @@ import org.apache.kafka.common.message.DeleteTopicsRequestData
|
|||
import org.apache.kafka.common.message.DeleteTopicsRequestData.DeleteTopicState
|
||||
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
|
||||
import org.apache.kafka.common.requests.{DeleteTopicsRequest, DeleteTopicsResponse, FetchRequest, FetchResponse, MetadataRequest, MetadataResponse}
|
||||
import org.apache.kafka.server.common.MetadataVersion.{IBP_2_7_IV0}
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
|
||||
import org.junit.jupiter.api.{BeforeEach, Test, TestInfo}
|
||||
|
||||
|
@ -36,7 +36,7 @@ import scala.jdk.CollectionConverters._
|
|||
class TopicIdWithOldInterBrokerProtocolTest extends BaseRequestTest {
|
||||
|
||||
override def brokerPropertyOverrides(properties: Properties): Unit = {
|
||||
properties.setProperty(KafkaConfig.InterBrokerProtocolVersionProp, KAFKA_2_7_IV0.toString)
|
||||
properties.setProperty(KafkaConfig.InterBrokerProtocolVersionProp, IBP_2_7_IV0.toString)
|
||||
properties.setProperty(KafkaConfig.OffsetsTopicPartitionsProp, "1")
|
||||
properties.setProperty(KafkaConfig.DefaultReplicationFactorProp, "2")
|
||||
properties.setProperty(KafkaConfig.RackProp, s"rack/${properties.getProperty(KafkaConfig.BrokerIdProp)}")
|
||||
|
|
|
@ -20,7 +20,6 @@ package kafka.server
|
|||
import java.util.{Optional, Properties}
|
||||
import java.util.concurrent.ExecutionException
|
||||
|
||||
import kafka.api.KAFKA_2_7_IV0
|
||||
import kafka.utils.TestUtils
|
||||
import kafka.zk.{FeatureZNode, FeatureZNodeStatus, ZkVersion}
|
||||
import kafka.utils.TestUtils.waitUntilTrue
|
||||
|
@ -33,8 +32,9 @@ import org.apache.kafka.common.message.UpdateFeaturesRequestData.FeatureUpdateKe
|
|||
import org.apache.kafka.common.protocol.Errors
|
||||
import org.apache.kafka.common.requests.{UpdateFeaturesRequest, UpdateFeaturesResponse}
|
||||
import org.apache.kafka.common.utils.Utils
|
||||
import org.apache.kafka.server.common.MetadataVersion.IBP_2_7_IV0
|
||||
import org.junit.jupiter.api.Test
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertNotEquals, assertNotNull, assertTrue, assertThrows}
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertNotEquals, assertNotNull, assertThrows, assertTrue}
|
||||
|
||||
import scala.jdk.CollectionConverters._
|
||||
import scala.reflect.ClassTag
|
||||
|
@ -45,7 +45,7 @@ class UpdateFeaturesTest extends BaseRequestTest {
|
|||
override def brokerCount = 3
|
||||
|
||||
override def brokerPropertyOverrides(props: Properties): Unit = {
|
||||
props.put(KafkaConfig.InterBrokerProtocolVersionProp, KAFKA_2_7_IV0.toString)
|
||||
props.put(KafkaConfig.InterBrokerProtocolVersionProp, IBP_2_7_IV0.toString)
|
||||
}
|
||||
|
||||
private def defaultSupportedFeatures(): Features[SupportedVersionRange] = {
|
||||
|
|
|
@ -19,7 +19,6 @@ package kafka.server.epoch
|
|||
|
||||
import java.io.{File, RandomAccessFile}
|
||||
import java.util.Properties
|
||||
import kafka.api.ApiVersion
|
||||
import kafka.log.{UnifiedLog, LogLoader}
|
||||
import kafka.server.KafkaConfig._
|
||||
import kafka.server.{KafkaConfig, KafkaServer}
|
||||
|
@ -32,6 +31,7 @@ import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
|
|||
import org.apache.kafka.common.TopicPartition
|
||||
import org.apache.kafka.common.record.RecordBatch
|
||||
import org.apache.kafka.common.serialization.ByteArrayDeserializer
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
|
||||
|
||||
|
@ -49,8 +49,8 @@ import scala.collection.Seq
|
|||
*/
|
||||
class EpochDrivenReplicationProtocolAcceptanceTest extends QuorumTestHarness with Logging {
|
||||
|
||||
// Set this to KAFKA_0_11_0_IV1 to demonstrate the tests failing in the pre-KIP-101 case
|
||||
val apiVersion = ApiVersion.latestVersion
|
||||
// Set this to IBP_0_11_0_IV1 to demonstrate the tests failing in the pre-KIP-101 case
|
||||
val metadataVersion = MetadataVersion.latest
|
||||
val topic = "topic1"
|
||||
val msg = new Array[Byte](1000)
|
||||
val msgBigger = new Array[Byte](10000)
|
||||
|
@ -178,7 +178,7 @@ class EpochDrivenReplicationProtocolAcceptanceTest extends QuorumTestHarness wit
|
|||
assertEquals(getLogFile(brokers(0), 0).length, getLogFile(brokers(1), 0).length, "Log files should match Broker0 vs Broker 1")
|
||||
}
|
||||
|
||||
//We can reproduce the pre-KIP-101 failure of this test by setting KafkaConfig.InterBrokerProtocolVersionProp = KAFKA_0_11_0_IV1
|
||||
//We can reproduce the pre-KIP-101 failure of this test by setting KafkaConfig.InterBrokerProtocolVersionProp = IBP_0_11_0_IV1
|
||||
@Test
|
||||
def offsetsShouldNotGoBackwards(): Unit = {
|
||||
|
||||
|
@ -465,7 +465,7 @@ class EpochDrivenReplicationProtocolAcceptanceTest extends QuorumTestHarness wit
|
|||
|
||||
private def createBrokerForId(id: Int, enableUncleanLeaderElection: Boolean = false): KafkaServer = {
|
||||
val config = createBrokerConfig(id, zkConnect)
|
||||
TestUtils.setIbpAndMessageFormatVersions(config, apiVersion)
|
||||
TestUtils.setIbpAndMessageFormatVersions(config, metadataVersion)
|
||||
config.setProperty(KafkaConfig.UncleanLeaderElectionEnableProp, enableUncleanLeaderElection.toString)
|
||||
createServer(fromProps(config))
|
||||
}
|
||||
|
|
|
@ -17,7 +17,7 @@
|
|||
|
||||
package kafka.server.epoch
|
||||
|
||||
import kafka.api.KAFKA_2_6_IV0
|
||||
import org.apache.kafka.server.common.MetadataVersion.IBP_2_6_IV0
|
||||
|
||||
/**
|
||||
* With IBP 2.7 onwards, we truncate based on diverging epochs returned in fetch responses.
|
||||
|
@ -25,5 +25,5 @@ import kafka.api.KAFKA_2_6_IV0
|
|||
* verifies that we handle older IBP versions with truncation on leader/follower change correctly.
|
||||
*/
|
||||
class EpochDrivenReplicationProtocolAcceptanceWithIbp26Test extends EpochDrivenReplicationProtocolAcceptanceTest {
|
||||
override val apiVersion = KAFKA_2_6_IV0
|
||||
override val metadataVersion = IBP_2_6_IV0
|
||||
}
|
||||
|
|
|
@ -69,6 +69,7 @@ import org.apache.kafka.common.utils.{Time, Utils}
|
|||
import org.apache.kafka.common.{KafkaFuture, TopicPartition}
|
||||
import org.apache.kafka.controller.QuorumController
|
||||
import org.apache.kafka.server.authorizer.{AuthorizableRequestContext, Authorizer => JAuthorizer}
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.apache.kafka.server.metrics.KafkaYammerMetrics
|
||||
import org.apache.kafka.test.{TestSslUtils, TestUtils => JTestUtils}
|
||||
import org.apache.zookeeper.KeeperException.SessionExpiredException
|
||||
|
@ -361,7 +362,7 @@ object TestUtils extends Logging {
|
|||
}
|
||||
|
||||
@nowarn("cat=deprecation")
|
||||
def setIbpAndMessageFormatVersions(config: Properties, version: ApiVersion): Unit = {
|
||||
def setIbpAndMessageFormatVersions(config: Properties, version: MetadataVersion): Unit = {
|
||||
config.setProperty(KafkaConfig.InterBrokerProtocolVersionProp, version.version)
|
||||
// for clarity, only set the log message format version if it's not ignored
|
||||
if (!LogConfig.shouldIgnoreMessageFormatVersion(version))
|
||||
|
@ -808,7 +809,7 @@ object TestUtils extends Logging {
|
|||
Broker(b.id, Seq(EndPoint("localhost", 6667, listenerName, protocol)), b.rack)
|
||||
}
|
||||
brokers.foreach(b => zkClient.registerBroker(BrokerInfo(Broker(b.id, b.endPoints, rack = b.rack),
|
||||
ApiVersion.latestVersion, jmxPort = -1)))
|
||||
MetadataVersion.latest, jmxPort = -1)))
|
||||
brokers
|
||||
}
|
||||
|
||||
|
@ -1233,7 +1234,7 @@ object TestUtils extends Logging {
|
|||
configRepository: ConfigRepository = new MockConfigRepository,
|
||||
cleanerConfig: CleanerConfig = CleanerConfig(enableCleaner = false),
|
||||
time: MockTime = new MockTime(),
|
||||
interBrokerProtocolVersion: ApiVersion = ApiVersion.latestVersion): LogManager = {
|
||||
interBrokerProtocolVersion: MetadataVersion = MetadataVersion.latest): LogManager = {
|
||||
new LogManager(logDirs = logDirs.map(_.getAbsoluteFile),
|
||||
initialOfflineDirs = Array.empty[File],
|
||||
configRepository = configRepository,
|
||||
|
|
|
@ -19,7 +19,8 @@ package kafka.zk
|
|||
import java.nio.charset.StandardCharsets.UTF_8
|
||||
import java.util.concurrent.{CountDownLatch, TimeUnit}
|
||||
import java.util.{Collections, Properties}
|
||||
import kafka.api.{ApiVersion, LeaderAndIsr}
|
||||
|
||||
import kafka.api.LeaderAndIsr
|
||||
import kafka.cluster.{Broker, EndPoint}
|
||||
import kafka.controller.{LeaderIsrAndControllerEpoch, ReplicaAssignment}
|
||||
import kafka.log.LogConfig
|
||||
|
@ -42,6 +43,7 @@ import org.apache.kafka.common.security.token.delegation.TokenInformation
|
|||
import org.apache.kafka.common.utils.{SecurityUtils, Time}
|
||||
import org.apache.kafka.common.{TopicPartition, Uuid}
|
||||
import org.apache.kafka.metadata.LeaderRecoveryState
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.apache.zookeeper.KeeperException.{Code, NoAuthException, NoNodeException, NodeExistsException}
|
||||
import org.apache.zookeeper.ZooDefs
|
||||
import org.apache.zookeeper.client.ZKClientConfig
|
||||
|
@ -51,6 +53,7 @@ import org.junit.jupiter.api.Assertions._
|
|||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
import org.junit.jupiter.params.provider.ValueSource
|
||||
|
||||
import scala.collection.mutable.ArrayBuffer
|
||||
import scala.collection.{Seq, mutable}
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
@ -808,7 +811,7 @@ class KafkaZkClientTest extends QuorumTestHarness {
|
|||
Seq(new EndPoint(host, port, ListenerName.forSecurityProtocol(securityProtocol), securityProtocol)),
|
||||
rack = rack,
|
||||
features = features),
|
||||
ApiVersion.latestVersion, jmxPort = port + 10)
|
||||
MetadataVersion.latest, jmxPort = port + 10)
|
||||
|
||||
@Test
|
||||
def testRegisterBrokerInfo(): Unit = {
|
||||
|
|
|
@ -17,8 +17,6 @@
|
|||
|
||||
package org.apache.kafka.jmh.fetcher;
|
||||
|
||||
import kafka.api.ApiVersion;
|
||||
import kafka.api.ApiVersion$;
|
||||
import kafka.cluster.BrokerEndPoint;
|
||||
import kafka.cluster.DelayedOperations;
|
||||
import kafka.cluster.AlterPartitionListener;
|
||||
|
@ -69,6 +67,7 @@ import org.apache.kafka.common.requests.FetchResponse;
|
|||
import org.apache.kafka.common.requests.UpdateMetadataRequest;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.server.common.MetadataVersion;
|
||||
import org.mockito.Mockito;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
import org.openjdk.jmh.annotations.BenchmarkMode;
|
||||
|
@ -145,7 +144,7 @@ public class ReplicaFetcherThreadBenchmark {
|
|||
setFlushStartOffsetCheckpointMs(10000L).
|
||||
setRetentionCheckMs(1000L).
|
||||
setMaxPidExpirationMs(60000).
|
||||
setInterBrokerProtocolVersion(ApiVersion.latestVersion()).
|
||||
setInterBrokerProtocolVersion(MetadataVersion.latest()).
|
||||
setScheduler(scheduler).
|
||||
setBrokerTopicStats(brokerTopicStats).
|
||||
setLogDirFailureChannel(logDirFailureChannel).
|
||||
|
@ -174,7 +173,7 @@ public class ReplicaFetcherThreadBenchmark {
|
|||
OffsetCheckpoints offsetCheckpoints = Mockito.mock(OffsetCheckpoints.class);
|
||||
Mockito.when(offsetCheckpoints.fetch(logDir.getAbsolutePath(), tp)).thenReturn(Option.apply(0L));
|
||||
AlterPartitionManager isrChannelManager = Mockito.mock(AlterPartitionManager.class);
|
||||
Partition partition = new Partition(tp, 100, ApiVersion$.MODULE$.latestVersion(),
|
||||
Partition partition = new Partition(tp, 100, MetadataVersion.latest(),
|
||||
0, Time.SYSTEM, alterPartitionListener, new DelayedOperationsMock(tp),
|
||||
Mockito.mock(MetadataCache.class), logManager, isrChannelManager);
|
||||
|
||||
|
|
|
@ -17,8 +17,6 @@
|
|||
|
||||
package org.apache.kafka.jmh.partition;
|
||||
|
||||
import kafka.api.ApiVersion;
|
||||
import kafka.api.ApiVersion$;
|
||||
import kafka.cluster.DelayedOperations;
|
||||
import kafka.cluster.AlterPartitionListener;
|
||||
import kafka.cluster.Partition;
|
||||
|
@ -42,6 +40,7 @@ import org.apache.kafka.common.record.MemoryRecords;
|
|||
import org.apache.kafka.common.record.SimpleRecord;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.server.common.MetadataVersion;
|
||||
import org.mockito.Mockito;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
import org.openjdk.jmh.annotations.BenchmarkMode;
|
||||
|
@ -111,7 +110,7 @@ public class PartitionMakeFollowerBenchmark {
|
|||
setFlushStartOffsetCheckpointMs(10000L).
|
||||
setRetentionCheckMs(1000L).
|
||||
setMaxPidExpirationMs(60000).
|
||||
setInterBrokerProtocolVersion(ApiVersion.latestVersion()).
|
||||
setInterBrokerProtocolVersion(MetadataVersion.latest()).
|
||||
setScheduler(scheduler).
|
||||
setBrokerTopicStats(brokerTopicStats).
|
||||
setLogDirFailureChannel(logDirFailureChannel).
|
||||
|
@ -125,7 +124,7 @@ public class PartitionMakeFollowerBenchmark {
|
|||
AlterPartitionListener alterPartitionListener = Mockito.mock(AlterPartitionListener.class);
|
||||
AlterPartitionManager alterPartitionManager = Mockito.mock(AlterPartitionManager.class);
|
||||
partition = new Partition(tp, 100,
|
||||
ApiVersion$.MODULE$.latestVersion(), 0, Time.SYSTEM,
|
||||
MetadataVersion.latest(), 0, Time.SYSTEM,
|
||||
alterPartitionListener, delayedOperations,
|
||||
Mockito.mock(MetadataCache.class), logManager, alterPartitionManager);
|
||||
partition.createLogIfNotExists(true, false, offsetCheckpoints, topicId);
|
||||
|
|
|
@ -17,8 +17,6 @@
|
|||
|
||||
package org.apache.kafka.jmh.partition;
|
||||
|
||||
import kafka.api.ApiVersion;
|
||||
import kafka.api.ApiVersion$;
|
||||
import kafka.cluster.DelayedOperations;
|
||||
import kafka.cluster.AlterPartitionListener;
|
||||
import kafka.cluster.Partition;
|
||||
|
@ -39,6 +37,7 @@ import org.apache.kafka.common.TopicPartition;
|
|||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
import org.apache.kafka.server.common.MetadataVersion;
|
||||
import org.mockito.Mockito;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
import org.openjdk.jmh.annotations.BenchmarkMode;
|
||||
|
@ -97,7 +96,7 @@ public class UpdateFollowerFetchStateBenchmark {
|
|||
setFlushStartOffsetCheckpointMs(10000L).
|
||||
setRetentionCheckMs(1000L).
|
||||
setMaxPidExpirationMs(60000).
|
||||
setInterBrokerProtocolVersion(ApiVersion.latestVersion()).
|
||||
setInterBrokerProtocolVersion(MetadataVersion.latest()).
|
||||
setScheduler(scheduler).
|
||||
setBrokerTopicStats(brokerTopicStats).
|
||||
setLogDirFailureChannel(logDirFailureChannel).
|
||||
|
@ -124,7 +123,7 @@ public class UpdateFollowerFetchStateBenchmark {
|
|||
AlterPartitionListener alterPartitionListener = Mockito.mock(AlterPartitionListener.class);
|
||||
AlterPartitionManager alterPartitionManager = Mockito.mock(AlterPartitionManager.class);
|
||||
partition = new Partition(topicPartition, 100,
|
||||
ApiVersion$.MODULE$.latestVersion(), 0, Time.SYSTEM,
|
||||
MetadataVersion.latest(), 0, Time.SYSTEM,
|
||||
alterPartitionListener, delayedOperations,
|
||||
Mockito.mock(MetadataCache.class), logManager, alterPartitionManager);
|
||||
partition.makeLeader(partitionState, offsetCheckpoints, topicId);
|
||||
|
|
|
@ -16,7 +16,6 @@
|
|||
*/
|
||||
package org.apache.kafka.jmh.record;
|
||||
|
||||
import kafka.api.ApiVersion;
|
||||
import kafka.common.LongRef;
|
||||
import kafka.log.AppendOrigin;
|
||||
import kafka.log.LogValidator;
|
||||
|
@ -26,6 +25,7 @@ import org.apache.kafka.common.record.CompressionType;
|
|||
import org.apache.kafka.common.record.MemoryRecords;
|
||||
import org.apache.kafka.common.record.TimestampType;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
import org.apache.kafka.server.common.MetadataVersion;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
import org.openjdk.jmh.annotations.Fork;
|
||||
import org.openjdk.jmh.annotations.Measurement;
|
||||
|
@ -58,7 +58,7 @@ public class CompressedRecordBatchValidationBenchmark extends BaseRecordBatchBen
|
|||
CompressionCodec.getCompressionCodec(compressionType.id),
|
||||
false, messageVersion, TimestampType.CREATE_TIME, Long.MAX_VALUE, 0,
|
||||
new AppendOrigin.Client$(),
|
||||
ApiVersion.latestVersion(),
|
||||
MetadataVersion.latest(),
|
||||
brokerTopicStats,
|
||||
requestLocal);
|
||||
}
|
||||
|
|
|
@ -16,7 +16,6 @@
|
|||
*/
|
||||
package org.apache.kafka.jmh.server;
|
||||
|
||||
import kafka.api.ApiVersion;
|
||||
import kafka.cluster.Partition;
|
||||
import kafka.log.CleanerConfig;
|
||||
import kafka.log.LogConfig;
|
||||
|
@ -39,6 +38,7 @@ import org.apache.kafka.common.Uuid;
|
|||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.common.metrics.Metrics;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.server.common.MetadataVersion;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
import org.openjdk.jmh.annotations.Fork;
|
||||
import org.openjdk.jmh.annotations.Level;
|
||||
|
@ -107,7 +107,7 @@ public class CheckpointBench {
|
|||
this.logManager = TestUtils.createLogManager(JavaConverters.asScalaBuffer(files),
|
||||
LogConfig.apply(), new MockConfigRepository(), CleanerConfig.apply(1, 4 * 1024 * 1024L, 0.9d,
|
||||
1024 * 1024, 32 * 1024 * 1024,
|
||||
Double.MAX_VALUE, 15 * 1000, true, "MD5"), time, ApiVersion.latestVersion());
|
||||
Double.MAX_VALUE, 15 * 1000, true, "MD5"), time, MetadataVersion.latest());
|
||||
scheduler.startup();
|
||||
final BrokerTopicStats brokerTopicStats = new BrokerTopicStats();
|
||||
final MetadataCache metadataCache =
|
||||
|
|
|
@ -16,7 +16,6 @@
|
|||
*/
|
||||
package org.apache.kafka.jmh.server;
|
||||
|
||||
import kafka.api.ApiVersion;
|
||||
import kafka.cluster.Partition;
|
||||
import kafka.log.CleanerConfig;
|
||||
import kafka.log.Defaults;
|
||||
|
@ -44,6 +43,7 @@ import org.apache.kafka.common.message.LeaderAndIsrRequestData;
|
|||
import org.apache.kafka.common.metrics.Metrics;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.server.common.MetadataVersion;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
import org.openjdk.jmh.annotations.BenchmarkMode;
|
||||
import org.openjdk.jmh.annotations.Fork;
|
||||
|
@ -134,7 +134,7 @@ public class PartitionCreationBench {
|
|||
setFlushStartOffsetCheckpointMs(10000L).
|
||||
setRetentionCheckMs(1000L).
|
||||
setMaxPidExpirationMs(60000).
|
||||
setInterBrokerProtocolVersion(ApiVersion.latestVersion()).
|
||||
setInterBrokerProtocolVersion(MetadataVersion.latest()).
|
||||
setScheduler(scheduler).
|
||||
setBrokerTopicStats(brokerTopicStats).
|
||||
setLogDirFailureChannel(failureChannel).
|
||||
|
|
|
@ -0,0 +1,288 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.kafka.server.common;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.regex.Pattern;
|
||||
import org.apache.kafka.common.record.RecordVersion;
|
||||
|
||||
/**
|
||||
* This class contains the different Kafka versions.
|
||||
* Right now, we use them for upgrades - users can configure the version of the API brokers will use to communicate between themselves.
|
||||
* This is only for inter-broker communications - when communicating with clients, the client decides on the API version.
|
||||
*
|
||||
* Note that the ID we initialize for each version is important.
|
||||
* We consider a version newer than another if it is lower in the enum list (to avoid depending on lexicographic order)
|
||||
*
|
||||
* Since the api protocol may change more than once within the same release and to facilitate people deploying code from
|
||||
* trunk, we have the concept of internal versions (first introduced during the 0.10.0 development cycle). For example,
|
||||
* the first time we introduce a version change in a release, say 0.10.0, we will add a config value "0.10.0-IV0" and a
|
||||
* corresponding enum constant IBP_0_10_0-IV0. We will also add a config value "0.10.0" that will be mapped to the
|
||||
* latest internal version object, which is IBP_0_10_0-IV0. When we change the protocol a second time while developing
|
||||
* 0.10.0, we will add a new config value "0.10.0-IV1" and a corresponding enum constant IBP_0_10_0-IV1. We will change
|
||||
* the config value "0.10.0" to map to the latest internal version IBP_0_10_0-IV1. The config value of
|
||||
* "0.10.0-IV0" is still mapped to IBP_0_10_0-IV0. This way, if people are deploying from trunk, they can use
|
||||
* "0.10.0-IV0" and "0.10.0-IV1" to upgrade one internal version at a time. For most people who just want to use
|
||||
* released version, they can use "0.10.0" when upgrading to the 0.10.0 release.
|
||||
*/
|
||||
public enum MetadataVersion {
|
||||
IBP_0_8_0(-1, "0.8.0", ""),
|
||||
IBP_0_8_1(-1, "0.8.1", ""),
|
||||
IBP_0_8_2(-1, "0.8.2", ""),
|
||||
IBP_0_9_0(-1, "0.9.0", ""),
|
||||
|
||||
// 0.10.0-IV0 is introduced for KIP-31/32 which changes the message format.
|
||||
IBP_0_10_0_IV0(-1, "0.10.0", "IV0"),
|
||||
|
||||
// 0.10.0-IV1 is introduced for KIP-36(rack awareness) and KIP-43(SASL handshake).
|
||||
IBP_0_10_0_IV1(-1, "0.10.0", "IV1"),
|
||||
|
||||
// introduced for JoinGroup protocol change in KIP-62
|
||||
IBP_0_10_1_IV0(-1, "0.10.1", "IV0"),
|
||||
|
||||
// 0.10.1-IV1 is introduced for KIP-74(fetch response size limit).
|
||||
IBP_0_10_1_IV1(-1, "0.10.1", "IV1"),
|
||||
|
||||
// introduced ListOffsetRequest v1 in KIP-79
|
||||
IBP_0_10_1_IV2(-1, "0.10.1", "IV2"),
|
||||
|
||||
// introduced UpdateMetadataRequest v3 in KIP-103
|
||||
IBP_0_10_2_IV0(-1, "0.10.2", "IV0"),
|
||||
|
||||
// KIP-98 (idempotent and transactional producer support)
|
||||
IBP_0_11_0_IV0(-1, "0.11.0", "IV0"),
|
||||
|
||||
// introduced DeleteRecordsRequest v0 and FetchRequest v4 in KIP-107
|
||||
IBP_0_11_0_IV1(-1, "0.11.0", "IV1"),
|
||||
|
||||
// Introduced leader epoch fetches to the replica fetcher via KIP-101
|
||||
IBP_0_11_0_IV2(-1, "0.11.0", "IV2"),
|
||||
|
||||
// Introduced LeaderAndIsrRequest V1, UpdateMetadataRequest V4 and FetchRequest V6 via KIP-112
|
||||
IBP_1_0_IV0(-1, "1.0", "IV0"),
|
||||
|
||||
// Introduced DeleteGroupsRequest V0 via KIP-229, plus KIP-227 incremental fetch requests,
|
||||
// and KafkaStorageException for fetch requests.
|
||||
IBP_1_1_IV0(-1, "1.1", "IV0"),
|
||||
|
||||
// Introduced OffsetsForLeaderEpochRequest V1 via KIP-279 (Fix log divergence between leader and follower after fast leader fail over)
|
||||
IBP_2_0_IV0(-1, "2.0", "IV0"),
|
||||
|
||||
// Several request versions were bumped due to KIP-219 (Improve quota communication)
|
||||
IBP_2_0_IV1(-1, "2.0", "IV1"),
|
||||
|
||||
// Introduced new schemas for group offset (v2) and group metadata (v2) (KIP-211)
|
||||
IBP_2_1_IV0(-1, "2.1", "IV0"),
|
||||
|
||||
// New Fetch, OffsetsForLeaderEpoch, and ListOffsets schemas (KIP-320)
|
||||
IBP_2_1_IV1(-1, "2.1", "IV1"),
|
||||
|
||||
// Support ZStandard Compression Codec (KIP-110)
|
||||
IBP_2_1_IV2(-1, "2.1", "IV2"),
|
||||
|
||||
// Introduced broker generation (KIP-380), and
|
||||
// LeaderAdnIsrRequest V2, UpdateMetadataRequest V5, StopReplicaRequest V1
|
||||
IBP_2_2_IV0(-1, "2.2", "IV0"),
|
||||
|
||||
// New error code for ListOffsets when a new leader is lagging behind former HW (KIP-207)
|
||||
IBP_2_2_IV1(-1, "2.2", "IV1"),
|
||||
|
||||
// Introduced static membership.
|
||||
IBP_2_3_IV0(-1, "2.3", "IV0"),
|
||||
|
||||
// Add rack_id to FetchRequest, preferred_read_replica to FetchResponse, and replica_id to OffsetsForLeaderRequest
|
||||
IBP_2_3_IV1(-1, "2.3", "IV1"),
|
||||
|
||||
// Add adding_replicas and removing_replicas fields to LeaderAndIsrRequest
|
||||
IBP_2_4_IV0(-1, "2.4", "IV0"),
|
||||
|
||||
// Flexible version support in inter-broker APIs
|
||||
IBP_2_4_IV1(-1, "2.4", "IV1"),
|
||||
|
||||
// No new APIs, equivalent to 2.4-IV1
|
||||
IBP_2_5_IV0(-1, "2.5", "IV0"),
|
||||
|
||||
// Introduced StopReplicaRequest V3 containing the leader epoch for each partition (KIP-570)
|
||||
IBP_2_6_IV0(-1, "2.6", "IV0"),
|
||||
|
||||
// Introduced feature versioning support (KIP-584)
|
||||
IBP_2_7_IV0(-1, "2.7", "IV0"),
|
||||
|
||||
// Bup Fetch protocol for Raft protocol (KIP-595)
|
||||
IBP_2_7_IV1(-1, "2.7", "IV1"),
|
||||
|
||||
// Introduced AlterPartition (KIP-497)
|
||||
IBP_2_7_IV2(-1, "2.7", "IV2"),
|
||||
|
||||
// Flexible versioning on ListOffsets, WriteTxnMarkers and OffsetsForLeaderEpoch. Also adds topic IDs (KIP-516)
|
||||
IBP_2_8_IV0(-1, "2.8", "IV0"),
|
||||
|
||||
// Introduced topic IDs to LeaderAndIsr and UpdateMetadata requests/responses (KIP-516)
|
||||
IBP_2_8_IV1(-1, "2.8", "IV1"),
|
||||
|
||||
// Introduce AllocateProducerIds (KIP-730)
|
||||
IBP_3_0_IV0(1, "3.0", "IV0"),
|
||||
|
||||
// Introduce ListOffsets V7 which supports listing offsets by max timestamp (KIP-734)
|
||||
// Assume message format version is 3.0 (KIP-724)
|
||||
IBP_3_0_IV1(2, "3.0", "IV1"),
|
||||
|
||||
// Adds topic IDs to Fetch requests/responses (KIP-516)
|
||||
IBP_3_1_IV0(3, "3.1", "IV0"),
|
||||
|
||||
// Support for leader recovery for unclean leader election (KIP-704)
|
||||
IBP_3_2_IV0(4, "3.2", "IV0");
|
||||
|
||||
public static final MetadataVersion[] VALUES = MetadataVersion.values();
|
||||
private final Optional<Short> featureLevel;
|
||||
private final String release;
|
||||
private final String ibpVersion;
|
||||
|
||||
MetadataVersion(int featureLevel, String release, String subVersion) {
|
||||
if (featureLevel > 0) {
|
||||
this.featureLevel = Optional.of((short) featureLevel);
|
||||
} else {
|
||||
this.featureLevel = Optional.empty();
|
||||
}
|
||||
this.release = release;
|
||||
if (subVersion.isEmpty()) {
|
||||
this.ibpVersion = release;
|
||||
} else {
|
||||
this.ibpVersion = String.format("%s-%s", release, subVersion);
|
||||
}
|
||||
}
|
||||
|
||||
public Optional<Short> featureLevel() {
|
||||
return featureLevel;
|
||||
}
|
||||
|
||||
public boolean isSaslInterBrokerHandshakeRequestEnabled() {
|
||||
return this.isAtLeast(IBP_0_10_0_IV1);
|
||||
}
|
||||
|
||||
public boolean isOffsetForLeaderEpochSupported() {
|
||||
return this.isAtLeast(IBP_0_11_0_IV2);
|
||||
}
|
||||
|
||||
public boolean isFeatureVersioningSupported() {
|
||||
return this.isAtLeast(IBP_2_7_IV0);
|
||||
}
|
||||
|
||||
public boolean isTruncationOnFetchSupported() {
|
||||
return this.isAtLeast(IBP_2_7_IV1);
|
||||
}
|
||||
|
||||
public boolean isAlterIsrSupported() {
|
||||
return this.isAtLeast(IBP_2_7_IV2);
|
||||
}
|
||||
|
||||
public boolean isTopicIdsSupported() {
|
||||
return this.isAtLeast(IBP_2_8_IV0);
|
||||
}
|
||||
|
||||
public boolean isAllocateProducerIdsSupported() {
|
||||
return this.isAtLeast(IBP_3_0_IV0);
|
||||
}
|
||||
|
||||
|
||||
public RecordVersion highestSupportedRecordVersion() {
|
||||
if (this.isLessThan(IBP_0_10_0_IV0)) {
|
||||
return RecordVersion.V0;
|
||||
} else if (this.isLessThan(IBP_0_11_0_IV0)) {
|
||||
return RecordVersion.V1;
|
||||
} else {
|
||||
return RecordVersion.V2;
|
||||
}
|
||||
}
|
||||
|
||||
private static final Map<String, MetadataVersion> IBP_VERSIONS;
|
||||
static {
|
||||
{
|
||||
IBP_VERSIONS = new HashMap<>();
|
||||
Map<String, MetadataVersion> maxInterVersion = new HashMap<>();
|
||||
for (MetadataVersion metadataVersion : VALUES) {
|
||||
maxInterVersion.put(metadataVersion.release, metadataVersion);
|
||||
IBP_VERSIONS.put(metadataVersion.ibpVersion, metadataVersion);
|
||||
}
|
||||
IBP_VERSIONS.putAll(maxInterVersion);
|
||||
}
|
||||
}
|
||||
|
||||
public String shortVersion() {
|
||||
return release;
|
||||
}
|
||||
|
||||
public String version() {
|
||||
return ibpVersion;
|
||||
}
|
||||
|
||||
/**
|
||||
* Return an `MetadataVersion` instance for `versionString`, which can be in a variety of formats (e.g. "0.8.0", "0.8.0.x",
|
||||
* "0.10.0", "0.10.0-IV1"). `IllegalArgumentException` is thrown if `versionString` cannot be mapped to an `MetadataVersion`.
|
||||
* Note that 'misconfigured' values such as "1.0.1" will be parsed to `IBP_1_0_IV0` as we ignore anything after the first
|
||||
* two digits for versions that don't start with "0."
|
||||
*/
|
||||
public static MetadataVersion fromVersionString(String versionString) {
|
||||
String[] versionSegments = versionString.split(Pattern.quote("."));
|
||||
int numSegments = (versionString.startsWith("0.")) ? 3 : 2;
|
||||
String key;
|
||||
if (numSegments >= versionSegments.length) {
|
||||
key = versionString;
|
||||
} else {
|
||||
key = String.join(".", Arrays.copyOfRange(versionSegments, 0, numSegments));
|
||||
}
|
||||
return Optional.ofNullable(IBP_VERSIONS.get(key)).orElseThrow(() ->
|
||||
new IllegalArgumentException("Version " + versionString + " is not a valid version")
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the minimum `MetadataVersion` that supports `RecordVersion`.
|
||||
*/
|
||||
public static MetadataVersion minSupportedFor(RecordVersion recordVersion) {
|
||||
switch (recordVersion) {
|
||||
case V0:
|
||||
return IBP_0_8_0;
|
||||
case V1:
|
||||
return IBP_0_10_0_IV0;
|
||||
case V2:
|
||||
return IBP_0_11_0_IV0;
|
||||
default:
|
||||
throw new IllegalArgumentException("Invalid message format version " + recordVersion);
|
||||
}
|
||||
}
|
||||
|
||||
public static MetadataVersion latest() {
|
||||
return VALUES[VALUES.length - 1];
|
||||
}
|
||||
|
||||
public boolean isAtLeast(MetadataVersion otherVersion) {
|
||||
return this.compareTo(otherVersion) >= 0;
|
||||
}
|
||||
|
||||
public boolean isLessThan(MetadataVersion otherVersion) {
|
||||
return this.compareTo(otherVersion) < 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return ibpVersion;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,40 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.kafka.server.common;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.kafka.common.config.ConfigDef.Validator;
|
||||
import org.apache.kafka.common.config.ConfigException;
|
||||
|
||||
public class MetadataVersionValidator implements Validator {
|
||||
|
||||
@Override
|
||||
public void ensureValid(String name, Object value) {
|
||||
try {
|
||||
MetadataVersion.fromVersionString(value.toString());
|
||||
} catch (IllegalArgumentException e) {
|
||||
throw new ConfigException(name, value.toString(), e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "[" + Arrays.stream(MetadataVersion.VALUES).map(MetadataVersion::version).collect(
|
||||
Collectors.joining(", ")) + "]";
|
||||
}
|
||||
}
|
|
@ -0,0 +1,343 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.kafka.server.common;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import org.apache.kafka.common.feature.Features;
|
||||
import org.apache.kafka.common.feature.FinalizedVersionRange;
|
||||
import org.apache.kafka.common.feature.SupportedVersionRange;
|
||||
import org.apache.kafka.common.message.ApiMessageType.ListenerType;
|
||||
import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersion;
|
||||
import org.apache.kafka.common.message.ApiVersionsResponseData.FinalizedFeatureKey;
|
||||
import org.apache.kafka.common.message.ApiVersionsResponseData.SupportedFeatureKey;
|
||||
import org.apache.kafka.common.protocol.ApiKeys;
|
||||
import org.apache.kafka.common.record.RecordBatch;
|
||||
import org.apache.kafka.common.record.RecordVersion;
|
||||
import org.apache.kafka.common.requests.AbstractResponse;
|
||||
import org.apache.kafka.common.requests.ApiVersionsResponse;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
|
||||
import static org.apache.kafka.server.common.MetadataVersion.*;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
class MetadataVersionTest {
|
||||
|
||||
@Test
|
||||
public void testFeatureLevel() {
|
||||
int firstFeatureLevelIndex = Arrays.asList(MetadataVersion.VALUES).indexOf(IBP_3_0_IV0);
|
||||
for (int i = 0; i < firstFeatureLevelIndex; i++) {
|
||||
assertFalse(MetadataVersion.VALUES[i].featureLevel().isPresent());
|
||||
}
|
||||
short expectedFeatureLevel = 1;
|
||||
for (int i = firstFeatureLevelIndex; i < MetadataVersion.VALUES.length; i++) {
|
||||
MetadataVersion metadataVersion = MetadataVersion.VALUES[i];
|
||||
short featureLevel = metadataVersion.featureLevel().orElseThrow(() ->
|
||||
new IllegalArgumentException(
|
||||
String.format("Metadata version %s must have a non-null feature level", metadataVersion.version())));
|
||||
assertEquals(expectedFeatureLevel, featureLevel,
|
||||
String.format("Metadata version %s should have feature level %s", metadataVersion.version(), expectedFeatureLevel));
|
||||
expectedFeatureLevel += 1;
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFromVersionString() {
|
||||
assertEquals(IBP_0_8_0, MetadataVersion.fromVersionString("0.8.0"));
|
||||
assertEquals(IBP_0_8_0, MetadataVersion.fromVersionString("0.8.0.0"));
|
||||
assertEquals(IBP_0_8_0, MetadataVersion.fromVersionString("0.8.0.1"));
|
||||
// should throw an exception as long as IBP_8_0_IV0 is not defined
|
||||
assertThrows(IllegalArgumentException.class, () -> MetadataVersion.fromVersionString("8.0"));
|
||||
|
||||
assertEquals(IBP_0_8_1, MetadataVersion.fromVersionString("0.8.1"));
|
||||
assertEquals(IBP_0_8_1, MetadataVersion.fromVersionString("0.8.1.0"));
|
||||
assertEquals(IBP_0_8_1, MetadataVersion.fromVersionString("0.8.1.1"));
|
||||
|
||||
assertEquals(IBP_0_8_2, MetadataVersion.fromVersionString("0.8.2"));
|
||||
assertEquals(IBP_0_8_2, MetadataVersion.fromVersionString("0.8.2.0"));
|
||||
assertEquals(IBP_0_8_2, MetadataVersion.fromVersionString("0.8.2.1"));
|
||||
|
||||
assertEquals(IBP_0_9_0, MetadataVersion.fromVersionString("0.9.0"));
|
||||
assertEquals(IBP_0_9_0, MetadataVersion.fromVersionString("0.9.0.0"));
|
||||
assertEquals(IBP_0_9_0, MetadataVersion.fromVersionString("0.9.0.1"));
|
||||
|
||||
assertEquals(IBP_0_10_0_IV0, MetadataVersion.fromVersionString("0.10.0-IV0"));
|
||||
|
||||
assertEquals(IBP_0_10_0_IV1, MetadataVersion.fromVersionString("0.10.0"));
|
||||
assertEquals(IBP_0_10_0_IV1, MetadataVersion.fromVersionString("0.10.0.0"));
|
||||
assertEquals(IBP_0_10_0_IV1, MetadataVersion.fromVersionString("0.10.0.0-IV0"));
|
||||
assertEquals(IBP_0_10_0_IV1, MetadataVersion.fromVersionString("0.10.0.1"));
|
||||
|
||||
assertEquals(IBP_0_10_1_IV0, MetadataVersion.fromVersionString("0.10.1-IV0"));
|
||||
assertEquals(IBP_0_10_1_IV1, MetadataVersion.fromVersionString("0.10.1-IV1"));
|
||||
|
||||
assertEquals(IBP_0_10_1_IV2, MetadataVersion.fromVersionString("0.10.1"));
|
||||
assertEquals(IBP_0_10_1_IV2, MetadataVersion.fromVersionString("0.10.1.0"));
|
||||
assertEquals(IBP_0_10_1_IV2, MetadataVersion.fromVersionString("0.10.1-IV2"));
|
||||
assertEquals(IBP_0_10_1_IV2, MetadataVersion.fromVersionString("0.10.1.1"));
|
||||
|
||||
assertEquals(IBP_0_10_2_IV0, MetadataVersion.fromVersionString("0.10.2"));
|
||||
assertEquals(IBP_0_10_2_IV0, MetadataVersion.fromVersionString("0.10.2.0"));
|
||||
assertEquals(IBP_0_10_2_IV0, MetadataVersion.fromVersionString("0.10.2-IV0"));
|
||||
assertEquals(IBP_0_10_2_IV0, MetadataVersion.fromVersionString("0.10.2.1"));
|
||||
|
||||
assertEquals(IBP_0_11_0_IV0, MetadataVersion.fromVersionString("0.11.0-IV0"));
|
||||
assertEquals(IBP_0_11_0_IV1, MetadataVersion.fromVersionString("0.11.0-IV1"));
|
||||
|
||||
assertEquals(IBP_0_11_0_IV2, MetadataVersion.fromVersionString("0.11.0"));
|
||||
assertEquals(IBP_0_11_0_IV2, MetadataVersion.fromVersionString("0.11.0.0"));
|
||||
assertEquals(IBP_0_11_0_IV2, MetadataVersion.fromVersionString("0.11.0-IV2"));
|
||||
assertEquals(IBP_0_11_0_IV2, MetadataVersion.fromVersionString("0.11.0.1"));
|
||||
|
||||
assertEquals(IBP_1_0_IV0, MetadataVersion.fromVersionString("1.0"));
|
||||
assertEquals(IBP_1_0_IV0, MetadataVersion.fromVersionString("1.0.0"));
|
||||
assertEquals(IBP_1_0_IV0, MetadataVersion.fromVersionString("1.0.0-IV0"));
|
||||
assertEquals(IBP_1_0_IV0, MetadataVersion.fromVersionString("1.0.1"));
|
||||
assertThrows(IllegalArgumentException.class, () -> MetadataVersion.fromVersionString("0.1.0"));
|
||||
assertThrows(IllegalArgumentException.class, () -> MetadataVersion.fromVersionString("0.1.0.0"));
|
||||
assertThrows(IllegalArgumentException.class, () -> MetadataVersion.fromVersionString("0.1.0-IV0"));
|
||||
assertThrows(IllegalArgumentException.class, () -> MetadataVersion.fromVersionString("0.1.0.0-IV0"));
|
||||
|
||||
assertEquals(IBP_1_1_IV0, MetadataVersion.fromVersionString("1.1-IV0"));
|
||||
|
||||
assertEquals(IBP_2_0_IV1, MetadataVersion.fromVersionString("2.0"));
|
||||
assertEquals(IBP_2_0_IV0, MetadataVersion.fromVersionString("2.0-IV0"));
|
||||
assertEquals(IBP_2_0_IV1, MetadataVersion.fromVersionString("2.0-IV1"));
|
||||
|
||||
assertEquals(IBP_2_1_IV2, MetadataVersion.fromVersionString("2.1"));
|
||||
assertEquals(IBP_2_1_IV0, MetadataVersion.fromVersionString("2.1-IV0"));
|
||||
assertEquals(IBP_2_1_IV1, MetadataVersion.fromVersionString("2.1-IV1"));
|
||||
assertEquals(IBP_2_1_IV2, MetadataVersion.fromVersionString("2.1-IV2"));
|
||||
|
||||
assertEquals(IBP_2_2_IV1, MetadataVersion.fromVersionString("2.2"));
|
||||
assertEquals(IBP_2_2_IV0, MetadataVersion.fromVersionString("2.2-IV0"));
|
||||
assertEquals(IBP_2_2_IV1, MetadataVersion.fromVersionString("2.2-IV1"));
|
||||
|
||||
assertEquals(IBP_2_3_IV1, MetadataVersion.fromVersionString("2.3"));
|
||||
assertEquals(IBP_2_3_IV0, MetadataVersion.fromVersionString("2.3-IV0"));
|
||||
assertEquals(IBP_2_3_IV1, MetadataVersion.fromVersionString("2.3-IV1"));
|
||||
|
||||
assertEquals(IBP_2_4_IV1, MetadataVersion.fromVersionString("2.4"));
|
||||
assertEquals(IBP_2_4_IV0, MetadataVersion.fromVersionString("2.4-IV0"));
|
||||
assertEquals(IBP_2_4_IV1, MetadataVersion.fromVersionString("2.4-IV1"));
|
||||
|
||||
assertEquals(IBP_2_5_IV0, MetadataVersion.fromVersionString("2.5"));
|
||||
assertEquals(IBP_2_5_IV0, MetadataVersion.fromVersionString("2.5-IV0"));
|
||||
|
||||
assertEquals(IBP_2_6_IV0, MetadataVersion.fromVersionString("2.6"));
|
||||
assertEquals(IBP_2_6_IV0, MetadataVersion.fromVersionString("2.6-IV0"));
|
||||
|
||||
assertEquals(IBP_2_7_IV0, MetadataVersion.fromVersionString("2.7-IV0"));
|
||||
assertEquals(IBP_2_7_IV1, MetadataVersion.fromVersionString("2.7-IV1"));
|
||||
assertEquals(IBP_2_7_IV2, MetadataVersion.fromVersionString("2.7-IV2"));
|
||||
|
||||
assertEquals(IBP_2_8_IV1, MetadataVersion.fromVersionString("2.8"));
|
||||
assertEquals(IBP_2_8_IV0, MetadataVersion.fromVersionString("2.8-IV0"));
|
||||
assertEquals(IBP_2_8_IV1, MetadataVersion.fromVersionString("2.8-IV1"));
|
||||
|
||||
assertEquals(IBP_3_0_IV1, MetadataVersion.fromVersionString("3.0"));
|
||||
assertEquals(IBP_3_0_IV0, MetadataVersion.fromVersionString("3.0-IV0"));
|
||||
assertEquals(IBP_3_0_IV1, MetadataVersion.fromVersionString("3.0-IV1"));
|
||||
|
||||
assertEquals(IBP_3_1_IV0, MetadataVersion.fromVersionString("3.1"));
|
||||
assertEquals(IBP_3_1_IV0, MetadataVersion.fromVersionString("3.1-IV0"));
|
||||
|
||||
assertEquals(IBP_3_2_IV0, MetadataVersion.fromVersionString("3.2"));
|
||||
assertEquals(IBP_3_2_IV0, MetadataVersion.fromVersionString("3.2-IV0"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMinSupportedVersionFor() {
|
||||
assertEquals(IBP_0_8_0, MetadataVersion.minSupportedFor(RecordVersion.V0));
|
||||
assertEquals(IBP_0_10_0_IV0, MetadataVersion.minSupportedFor(RecordVersion.V1));
|
||||
assertEquals(IBP_0_11_0_IV0, MetadataVersion.minSupportedFor(RecordVersion.V2));
|
||||
|
||||
// Ensure that all record versions have a defined min version so that we remember to update the method
|
||||
for (RecordVersion recordVersion : RecordVersion.values()) {
|
||||
assertNotNull(MetadataVersion.minSupportedFor(recordVersion));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testShortVersion() {
|
||||
assertEquals("0.8.0", IBP_0_8_0.shortVersion());
|
||||
assertEquals("0.10.0", IBP_0_10_0_IV0.shortVersion());
|
||||
assertEquals("0.10.0", IBP_0_10_0_IV1.shortVersion());
|
||||
assertEquals("0.11.0", IBP_0_11_0_IV0.shortVersion());
|
||||
assertEquals("0.11.0", IBP_0_11_0_IV1.shortVersion());
|
||||
assertEquals("0.11.0", IBP_0_11_0_IV2.shortVersion());
|
||||
assertEquals("1.0", IBP_1_0_IV0.shortVersion());
|
||||
assertEquals("1.1", IBP_1_1_IV0.shortVersion());
|
||||
assertEquals("2.0", IBP_2_0_IV0.shortVersion());
|
||||
assertEquals("2.0", IBP_2_0_IV1.shortVersion());
|
||||
assertEquals("2.1", IBP_2_1_IV0.shortVersion());
|
||||
assertEquals("2.1", IBP_2_1_IV1.shortVersion());
|
||||
assertEquals("2.1", IBP_2_1_IV2.shortVersion());
|
||||
assertEquals("2.2", IBP_2_2_IV0.shortVersion());
|
||||
assertEquals("2.2", IBP_2_2_IV1.shortVersion());
|
||||
assertEquals("2.3", IBP_2_3_IV0.shortVersion());
|
||||
assertEquals("2.3", IBP_2_3_IV1.shortVersion());
|
||||
assertEquals("2.4", IBP_2_4_IV0.shortVersion());
|
||||
assertEquals("2.5", IBP_2_5_IV0.shortVersion());
|
||||
assertEquals("2.6", IBP_2_6_IV0.shortVersion());
|
||||
assertEquals("2.7", IBP_2_7_IV2.shortVersion());
|
||||
assertEquals("2.8", IBP_2_8_IV0.shortVersion());
|
||||
assertEquals("2.8", IBP_2_8_IV1.shortVersion());
|
||||
assertEquals("3.0", IBP_3_0_IV0.shortVersion());
|
||||
assertEquals("3.0", IBP_3_0_IV1.shortVersion());
|
||||
assertEquals("3.1", IBP_3_1_IV0.shortVersion());
|
||||
assertEquals("3.2", IBP_3_2_IV0.shortVersion());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testVersion() {
|
||||
assertEquals("0.8.0", IBP_0_8_0.version());
|
||||
assertEquals("0.8.2", IBP_0_8_2.version());
|
||||
assertEquals("0.10.0-IV0", IBP_0_10_0_IV0.version());
|
||||
assertEquals("0.10.0-IV1", IBP_0_10_0_IV1.version());
|
||||
assertEquals("0.11.0-IV0", IBP_0_11_0_IV0.version());
|
||||
assertEquals("0.11.0-IV1", IBP_0_11_0_IV1.version());
|
||||
assertEquals("0.11.0-IV2", IBP_0_11_0_IV2.version());
|
||||
assertEquals("1.0-IV0", IBP_1_0_IV0.version());
|
||||
assertEquals("1.1-IV0", IBP_1_1_IV0.version());
|
||||
assertEquals("2.0-IV0", IBP_2_0_IV0.version());
|
||||
assertEquals("2.0-IV1", IBP_2_0_IV1.version());
|
||||
assertEquals("2.1-IV0", IBP_2_1_IV0.version());
|
||||
assertEquals("2.1-IV1", IBP_2_1_IV1.version());
|
||||
assertEquals("2.1-IV2", IBP_2_1_IV2.version());
|
||||
assertEquals("2.2-IV0", IBP_2_2_IV0.version());
|
||||
assertEquals("2.2-IV1", IBP_2_2_IV1.version());
|
||||
assertEquals("2.3-IV0", IBP_2_3_IV0.version());
|
||||
assertEquals("2.3-IV1", IBP_2_3_IV1.version());
|
||||
assertEquals("2.4-IV0", IBP_2_4_IV0.version());
|
||||
assertEquals("2.5-IV0", IBP_2_5_IV0.version());
|
||||
assertEquals("2.6-IV0", IBP_2_6_IV0.version());
|
||||
assertEquals("2.7-IV2", IBP_2_7_IV2.version());
|
||||
assertEquals("2.8-IV0", IBP_2_8_IV0.version());
|
||||
assertEquals("2.8-IV1", IBP_2_8_IV1.version());
|
||||
assertEquals("3.0-IV0", IBP_3_0_IV0.version());
|
||||
assertEquals("3.0-IV1", IBP_3_0_IV1.version());
|
||||
assertEquals("3.1-IV0", IBP_3_1_IV0.version());
|
||||
assertEquals("3.2-IV0", IBP_3_2_IV0.version());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldCreateApiResponseOnlyWithKeysSupportedByMagicValue() {
|
||||
ApiVersionsResponse response = ApiVersionsResponse.createApiVersionsResponse(
|
||||
10,
|
||||
RecordVersion.V1,
|
||||
Features.emptySupportedFeatures(),
|
||||
Features.emptyFinalizedFeatures(),
|
||||
ApiVersionsResponse.UNKNOWN_FINALIZED_FEATURES_EPOCH,
|
||||
null,
|
||||
ListenerType.ZK_BROKER
|
||||
);
|
||||
verifyApiKeysForMagic(response, RecordBatch.MAGIC_VALUE_V1);
|
||||
assertEquals(10, response.throttleTimeMs());
|
||||
assertTrue(response.data().supportedFeatures().isEmpty());
|
||||
assertTrue(response.data().finalizedFeatures().isEmpty());
|
||||
assertEquals(ApiVersionsResponse.UNKNOWN_FINALIZED_FEATURES_EPOCH, response.data().finalizedFeaturesEpoch());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldReturnFeatureKeysWhenMagicIsCurrentValueAndThrottleMsIsDefaultThrottle() {
|
||||
ApiVersionsResponse response = ApiVersionsResponse.createApiVersionsResponse(
|
||||
10,
|
||||
RecordVersion.V1,
|
||||
Features.supportedFeatures(
|
||||
Utils.mkMap(Utils.mkEntry("feature", new SupportedVersionRange((short) 1, (short) 4)))),
|
||||
Features.finalizedFeatures(
|
||||
Utils.mkMap(Utils.mkEntry("feature", new FinalizedVersionRange((short) 2, (short) 3)))),
|
||||
10L,
|
||||
null,
|
||||
ListenerType.ZK_BROKER
|
||||
);
|
||||
|
||||
verifyApiKeysForMagic(response, RecordBatch.MAGIC_VALUE_V1);
|
||||
assertEquals(10, response.throttleTimeMs());
|
||||
assertEquals(1, response.data().supportedFeatures().size());
|
||||
SupportedFeatureKey sKey = response.data().supportedFeatures().find("feature");
|
||||
assertNotNull(sKey);
|
||||
assertEquals(1, sKey.minVersion());
|
||||
assertEquals(4, sKey.maxVersion());
|
||||
assertEquals(1, response.data().finalizedFeatures().size());
|
||||
FinalizedFeatureKey fKey = response.data().finalizedFeatures().find("feature");
|
||||
assertNotNull(fKey);
|
||||
assertEquals(2, fKey.minVersionLevel());
|
||||
assertEquals(3, fKey.maxVersionLevel());
|
||||
assertEquals(10, response.data().finalizedFeaturesEpoch());
|
||||
}
|
||||
|
||||
private void verifyApiKeysForMagic(ApiVersionsResponse response, Byte maxMagic) {
|
||||
for (ApiVersion version : response.data().apiKeys()) {
|
||||
assertTrue(ApiKeys.forId(version.apiKey()).minRequiredInterBrokerMagic <= maxMagic);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldReturnAllKeysWhenMagicIsCurrentValueAndThrottleMsIsDefaultThrottle() {
|
||||
ApiVersionsResponse response = ApiVersionsResponse.createApiVersionsResponse(
|
||||
AbstractResponse.DEFAULT_THROTTLE_TIME,
|
||||
RecordVersion.current(),
|
||||
Features.emptySupportedFeatures(),
|
||||
Features.emptyFinalizedFeatures(),
|
||||
ApiVersionsResponse.UNKNOWN_FINALIZED_FEATURES_EPOCH,
|
||||
null,
|
||||
ListenerType.ZK_BROKER
|
||||
);
|
||||
assertEquals(new HashSet<ApiKeys>(ApiKeys.zkBrokerApis()), apiKeysInResponse(response));
|
||||
assertEquals(AbstractResponse.DEFAULT_THROTTLE_TIME, response.throttleTimeMs());
|
||||
assertTrue(response.data().supportedFeatures().isEmpty());
|
||||
assertTrue(response.data().finalizedFeatures().isEmpty());
|
||||
assertEquals(ApiVersionsResponse.UNKNOWN_FINALIZED_FEATURES_EPOCH, response.data().finalizedFeaturesEpoch());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMetadataQuorumApisAreDisabled() {
|
||||
ApiVersionsResponse response = ApiVersionsResponse.createApiVersionsResponse(
|
||||
AbstractResponse.DEFAULT_THROTTLE_TIME,
|
||||
RecordVersion.current(),
|
||||
Features.emptySupportedFeatures(),
|
||||
Features.emptyFinalizedFeatures(),
|
||||
ApiVersionsResponse.UNKNOWN_FINALIZED_FEATURES_EPOCH,
|
||||
null,
|
||||
ListenerType.ZK_BROKER
|
||||
);
|
||||
|
||||
// Ensure that APIs needed for the KRaft mode are not exposed through ApiVersions until we are ready for them
|
||||
HashSet<ApiKeys> exposedApis = apiKeysInResponse(response);
|
||||
assertFalse(exposedApis.contains(ApiKeys.ENVELOPE));
|
||||
assertFalse(exposedApis.contains(ApiKeys.VOTE));
|
||||
assertFalse(exposedApis.contains(ApiKeys.BEGIN_QUORUM_EPOCH));
|
||||
assertFalse(exposedApis.contains(ApiKeys.END_QUORUM_EPOCH));
|
||||
assertFalse(exposedApis.contains(ApiKeys.DESCRIBE_QUORUM));
|
||||
}
|
||||
|
||||
private HashSet<ApiKeys> apiKeysInResponse(ApiVersionsResponse apiVersions) {
|
||||
HashSet<ApiKeys> apiKeys = new HashSet<>();
|
||||
for (ApiVersion version : apiVersions.data().apiKeys()) {
|
||||
apiKeys.add(ApiKeys.forId(version.apiKey()));
|
||||
}
|
||||
return apiKeys;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,33 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.kafka.server.common;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
public class MetadataVersionValidatorTest {
|
||||
|
||||
@Test
|
||||
public void testMetadataVersionValidator() {
|
||||
String str = new MetadataVersionValidator().toString();
|
||||
String[] apiVersions = str.substring(1).split(",");
|
||||
assertEquals(MetadataVersion.VALUES.length, apiVersions.length);
|
||||
}
|
||||
|
||||
}
|
Loading…
Reference in New Issue