KAFKA-18465: Remove MetadataVersions older than 3.0-IV1 (#18468)

Apache Kafka 4.0 will only support KRaft and 3.0-IV1 is the minimum version supported by KRaft. So, we can assume that Apache Kafka 4.0 will only communicate with brokers that are 3.0-IV1 or newer.

Note that KRaft was only marked as production-ready in 3.3, so we could go further and set the baseline to 3.3. I think we should have that discussion, but it made sense to start with the non controversial parts.

Reviewers: Jun Rao <junrao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, David Jacot <david.jacot@gmail.com>
This commit is contained in:
Ismael Juma 2025-01-11 09:42:39 -08:00 committed by GitHub
parent f54cfff1dc
commit d4aee71e36
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
87 changed files with 431 additions and 2181 deletions

View File

@ -118,7 +118,7 @@ public final class ClientUtils {
SecurityProtocol securityProtocol = SecurityProtocol.forName(config.getString(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG));
String clientSaslMechanism = config.getString(SaslConfigs.SASL_MECHANISM);
return ChannelBuilders.clientChannelBuilder(securityProtocol, JaasContext.Type.CLIENT, config, null,
clientSaslMechanism, time, true, logContext);
clientSaslMechanism, time, logContext);
}
static List<InetAddress> resolve(String host, HostResolver hostResolver) throws UnknownHostException {
@ -275,4 +275,4 @@ public final class ClientUtils {
return clusterResourceListeners;
}
}
}

View File

@ -56,8 +56,6 @@ public class ChannelBuilders {
* @param listenerName the listenerName if contextType is SERVER or null otherwise
* @param clientSaslMechanism SASL mechanism if mode is CLIENT, ignored otherwise
* @param time the time instance
* @param saslHandshakeRequestEnable flag to enable Sasl handshake requests; disabled only for SASL
* inter-broker connections with inter-broker protocol version < 0.10
* @param logContext the log context instance
*
* @return the configured `ChannelBuilder`
@ -70,7 +68,6 @@ public class ChannelBuilders {
ListenerName listenerName,
String clientSaslMechanism,
Time time,
boolean saslHandshakeRequestEnable,
LogContext logContext) {
if (securityProtocol == SecurityProtocol.SASL_PLAINTEXT || securityProtocol == SecurityProtocol.SASL_SSL) {
@ -80,7 +77,7 @@ public class ChannelBuilders {
throw new IllegalArgumentException("`clientSaslMechanism` must be non-null in client mode if `securityProtocol` is `" + securityProtocol + "`");
}
return create(securityProtocol, ConnectionMode.CLIENT, contextType, config, listenerName, false, clientSaslMechanism,
saslHandshakeRequestEnable, null, null, time, logContext, null);
null, null, time, logContext, null);
}
/**
@ -106,8 +103,8 @@ public class ChannelBuilders {
LogContext logContext,
Function<Short, ApiVersionsResponse> apiVersionSupplier) {
return create(securityProtocol, ConnectionMode.SERVER, JaasContext.Type.SERVER, config, listenerName,
isInterBrokerListener, null, true, credentialCache,
tokenCache, time, logContext, apiVersionSupplier);
isInterBrokerListener, null, credentialCache, tokenCache, time, logContext,
apiVersionSupplier);
}
private static ChannelBuilder create(SecurityProtocol securityProtocol,
@ -117,7 +114,6 @@ public class ChannelBuilders {
ListenerName listenerName,
boolean isInterBrokerListener,
String clientSaslMechanism,
boolean saslHandshakeRequestEnable,
CredentialCache credentialCache,
DelegationTokenCache tokenCache,
Time time,
@ -175,7 +171,6 @@ public class ChannelBuilders {
listenerName,
isInterBrokerListener,
clientSaslMechanism,
saslHandshakeRequestEnable,
credentialCache,
tokenCache,
sslClientAuthOverride,

View File

@ -85,7 +85,6 @@ public class SaslChannelBuilder implements ChannelBuilder, ListenerReconfigurabl
private final String clientSaslMechanism;
private final ConnectionMode connectionMode;
private final Map<String, JaasContext> jaasContexts;
private final boolean handshakeRequestEnable;
private final CredentialCache credentialCache;
private final DelegationTokenCache tokenCache;
private final Map<String, LoginManager> loginManagers;
@ -108,7 +107,6 @@ public class SaslChannelBuilder implements ChannelBuilder, ListenerReconfigurabl
ListenerName listenerName,
boolean isInterBrokerListener,
String clientSaslMechanism,
boolean handshakeRequestEnable,
CredentialCache credentialCache,
DelegationTokenCache tokenCache,
String sslClientAuthOverride,
@ -122,7 +120,6 @@ public class SaslChannelBuilder implements ChannelBuilder, ListenerReconfigurabl
this.securityProtocol = securityProtocol;
this.listenerName = listenerName;
this.isInterBrokerListener = isInterBrokerListener;
this.handshakeRequestEnable = handshakeRequestEnable;
this.clientSaslMechanism = clientSaslMechanism;
this.credentialCache = credentialCache;
this.tokenCache = tokenCache;
@ -295,7 +292,7 @@ public class SaslChannelBuilder implements ChannelBuilder, ListenerReconfigurabl
String servicePrincipal,
TransportLayer transportLayer, Subject subject) {
return new SaslClientAuthenticator(configs, callbackHandler, id, subject, servicePrincipal,
serverHost, clientSaslMechanism, handshakeRequestEnable, transportLayer, time, logContext);
serverHost, clientSaslMechanism, transportLayer, time, logContext);
}
// Package private for testing

View File

@ -68,14 +68,12 @@ public class AlterPartitionRequest extends AbstractRequest {
* @param data The data to be sent. Note that because the version of the
* request is not known at this time, it is expected that all
* topics have a topic id and a topic name set.
* @param canUseTopicIds True if version 2 and above can be used.
*/
public Builder(AlterPartitionRequestData data, boolean canUseTopicIds) {
public Builder(AlterPartitionRequestData data) {
super(
ApiKeys.ALTER_PARTITION,
ApiKeys.ALTER_PARTITION.oldestVersion(),
// Version 1 is the maximum version that can be used without topic ids.
canUseTopicIds ? ApiKeys.ALTER_PARTITION.latestVersion() : 1
ApiKeys.ALTER_PARTITION.latestVersion()
);
this.data = data;
}

View File

@ -31,7 +31,6 @@ import org.apache.kafka.common.message.ApiVersionsResponseData.SupportedFeatureK
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.record.RecordVersion;
import java.nio.ByteBuffer;
import java.util.Map;
@ -172,7 +171,6 @@ public class ApiVersionsResponse extends AbstractResponse {
}
public static ApiVersionCollection controllerApiVersions(
RecordVersion minRecordVersion,
NodeApiVersions controllerApiVersions,
ListenerType listenerType,
boolean enableUnstableLastVersion,
@ -180,27 +178,23 @@ public class ApiVersionsResponse extends AbstractResponse {
) {
return intersectForwardableApis(
listenerType,
minRecordVersion,
controllerApiVersions.allSupportedApiVersions(),
enableUnstableLastVersion,
clientTelemetryEnabled);
}
public static ApiVersionCollection brokerApiVersions(
RecordVersion minRecordVersion,
ListenerType listenerType,
boolean enableUnstableLastVersion,
boolean clientTelemetryEnabled
) {
return filterApis(
minRecordVersion,
listenerType,
enableUnstableLastVersion,
clientTelemetryEnabled);
}
public static ApiVersionCollection filterApis(
RecordVersion minRecordVersion,
ApiMessageType.ListenerType listenerType,
boolean enableUnstableLastVersion,
boolean clientTelemetryEnabled
@ -210,10 +204,7 @@ public class ApiVersionsResponse extends AbstractResponse {
// Skip telemetry APIs if client telemetry is disabled.
if ((apiKey == ApiKeys.GET_TELEMETRY_SUBSCRIPTIONS || apiKey == ApiKeys.PUSH_TELEMETRY) && !clientTelemetryEnabled)
continue;
if (apiKey.minRequiredInterBrokerMagic <= minRecordVersion.value) {
apiKey.toApiVersion(enableUnstableLastVersion).ifPresent(apiKeys::add);
}
apiKey.toApiVersion(enableUnstableLastVersion).ifPresent(apiKeys::add);
}
return apiKeys;
}
@ -234,7 +225,6 @@ public class ApiVersionsResponse extends AbstractResponse {
* known range and that of another set.
*
* @param listenerType the listener type which constrains the set of exposed APIs
* @param minRecordVersion min inter broker magic
* @param activeControllerApiVersions controller ApiVersions
* @param enableUnstableLastVersion whether unstable versions should be advertised or not
* @param clientTelemetryEnabled whether client telemetry is enabled or not
@ -242,42 +232,39 @@ public class ApiVersionsResponse extends AbstractResponse {
*/
public static ApiVersionCollection intersectForwardableApis(
final ApiMessageType.ListenerType listenerType,
final RecordVersion minRecordVersion,
final Map<ApiKeys, ApiVersion> activeControllerApiVersions,
boolean enableUnstableLastVersion,
boolean clientTelemetryEnabled
) {
ApiVersionCollection apiKeys = new ApiVersionCollection();
for (ApiKeys apiKey : ApiKeys.apisForListener(listenerType)) {
if (apiKey.minRequiredInterBrokerMagic <= minRecordVersion.value) {
final Optional<ApiVersion> brokerApiVersion = apiKey.toApiVersion(enableUnstableLastVersion);
if (brokerApiVersion.isEmpty()) {
// Broker does not support this API key.
continue;
}
// Skip telemetry APIs if client telemetry is disabled.
if ((apiKey == ApiKeys.GET_TELEMETRY_SUBSCRIPTIONS || apiKey == ApiKeys.PUSH_TELEMETRY) && !clientTelemetryEnabled)
continue;
final ApiVersion finalApiVersion;
if (!apiKey.forwardable) {
finalApiVersion = brokerApiVersion.get();
} else {
Optional<ApiVersion> intersectVersion = intersect(
brokerApiVersion.get(),
activeControllerApiVersions.getOrDefault(apiKey, null)
);
if (intersectVersion.isPresent()) {
finalApiVersion = intersectVersion.get();
} else {
// Controller doesn't support this API key, or there is no intersection.
continue;
}
}
apiKeys.add(finalApiVersion.duplicate());
final Optional<ApiVersion> brokerApiVersion = apiKey.toApiVersion(enableUnstableLastVersion);
if (brokerApiVersion.isEmpty()) {
// Broker does not support this API key.
continue;
}
// Skip telemetry APIs if client telemetry is disabled.
if ((apiKey == ApiKeys.GET_TELEMETRY_SUBSCRIPTIONS || apiKey == ApiKeys.PUSH_TELEMETRY) && !clientTelemetryEnabled)
continue;
final ApiVersion finalApiVersion;
if (!apiKey.forwardable) {
finalApiVersion = brokerApiVersion.get();
} else {
Optional<ApiVersion> intersectVersion = intersect(
brokerApiVersion.get(),
activeControllerApiVersions.getOrDefault(apiKey, null)
);
if (intersectVersion.isPresent()) {
finalApiVersion = intersectVersion.get();
} else {
// Controller doesn't support this API key, or there is no intersection.
continue;
}
}
apiKeys.add(finalApiVersion.duplicate());
}
return apiKeys;
}

View File

@ -65,11 +65,12 @@ public class OffsetsForLeaderEpochRequest extends AbstractRequest {
return new Builder((short) 3, ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion(), data);
}
public static Builder forFollower(short version, OffsetForLeaderTopicCollection epochsByPartition, int replicaId) {
public static Builder forFollower(OffsetForLeaderTopicCollection epochsByPartition, int replicaId) {
OffsetForLeaderEpochRequestData data = new OffsetForLeaderEpochRequestData();
data.setReplicaId(replicaId);
data.setTopics(epochsByPartition);
return new Builder(version, version, data);
// If we introduce new versions, we should gate them behind the appropriate metadata version
return new Builder((short) 4, (short) 4, data);
}
@Override

View File

@ -110,8 +110,8 @@ public class WriteTxnMarkersRequest extends AbstractRequest {
this.data = data;
}
public Builder(short version, final List<TxnMarkerEntry> markers) {
super(ApiKeys.WRITE_TXN_MARKERS, version);
public Builder(final List<TxnMarkerEntry> markers) {
super(ApiKeys.WRITE_TXN_MARKERS, (short) 1); // if we add new versions, gate them behind metadata version
List<WritableTxnMarker> dataMarkers = new ArrayList<>();
for (TxnMarkerEntry marker : markers) {
final Map<String, WritableTxnMarkerTopic> topicMap = new HashMap<>();

View File

@ -177,7 +177,6 @@ public class SaslClientAuthenticator implements Authenticator {
String servicePrincipal,
String host,
String mechanism,
boolean handshakeRequestEnable,
TransportLayer transportLayer,
Time time,
LogContext logContext) {
@ -196,7 +195,7 @@ public class SaslClientAuthenticator implements Authenticator {
this.reauthInfo = new ReauthInfo();
try {
setSaslState(handshakeRequestEnable ? SaslState.SEND_APIVERSIONS_REQUEST : SaslState.INITIAL);
setSaslState(SaslState.SEND_APIVERSIONS_REQUEST);
// determine client principal from subject for Kerberos to use as authorization id for the SaslClient.
// For other mechanisms, the authenticated principal (username for PLAIN and SCRAM) is used as

View File

@ -161,7 +161,6 @@ import org.apache.kafka.common.quota.ClientQuotaAlteration;
import org.apache.kafka.common.quota.ClientQuotaEntity;
import org.apache.kafka.common.quota.ClientQuotaFilter;
import org.apache.kafka.common.quota.ClientQuotaFilterComponent;
import org.apache.kafka.common.record.RecordVersion;
import org.apache.kafka.common.requests.AddRaftVoterRequest;
import org.apache.kafka.common.requests.AddRaftVoterResponse;
import org.apache.kafka.common.requests.AlterClientQuotasResponse;
@ -777,7 +776,7 @@ public class KafkaAdminClientTest {
if (error == Errors.NONE) {
return new ApiVersionsResponse.Builder().
setApiVersions(ApiVersionsResponse.filterApis(
RecordVersion.current(), ApiMessageType.ListenerType.ZK_BROKER, false, false)).
ApiMessageType.ListenerType.ZK_BROKER, false, false)).
setSupportedFeatures(
convertSupportedFeaturesMap(defaultFeatureMetadata().supportedFeatures())).
setFinalizedFeatures(

View File

@ -166,7 +166,7 @@ public class SaslChannelBuilderTest {
private SaslChannelBuilder createGssapiChannelBuilder(Map<String, JaasContext> jaasContexts, GSSManager gssManager) {
SaslChannelBuilder channelBuilder = new SaslChannelBuilder(ConnectionMode.SERVER, jaasContexts,
SecurityProtocol.SASL_PLAINTEXT, new ListenerName("GSSAPI"), false, "GSSAPI",
true, null, null, null, Time.SYSTEM, new LogContext(), defaultApiVersionsSupplier()) {
null, null, null, Time.SYSTEM, new LogContext(), defaultApiVersionsSupplier()) {
@Override
protected GSSManager gssManager() {
@ -205,7 +205,7 @@ public class SaslChannelBuilderTest {
JaasContext jaasContext = new JaasContext("jaasContext", JaasContext.Type.SERVER, jaasConfig, null);
Map<String, JaasContext> jaasContexts = Collections.singletonMap(saslMechanism, jaasContext);
return new SaslChannelBuilder(ConnectionMode.CLIENT, jaasContexts, securityProtocol, new ListenerName(saslMechanism),
false, saslMechanism, true, null,
false, saslMechanism, null,
null, null, Time.SYSTEM, new LogContext(), defaultApiVersionsSupplier());
}

View File

@ -61,7 +61,7 @@ class AlterPartitionRequestTest {
request.topics().add(topicData);
AlterPartitionRequest.Builder builder = new AlterPartitionRequest.Builder(request, version > 1);
AlterPartitionRequest.Builder builder = new AlterPartitionRequest.Builder(request);
AlterPartitionRequest alterPartitionRequest = builder.build(version);
assertEquals(1, alterPartitionRequest.data().topics().size());
assertEquals(1, alterPartitionRequest.data().topics().get(0).partitions().size());

View File

@ -23,11 +23,8 @@ 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.apache.kafka.test.TestUtils;
@ -103,7 +100,6 @@ public class ApiVersionsResponseTest {
ApiVersionCollection commonResponse = ApiVersionsResponse.intersectForwardableApis(
ApiMessageType.ListenerType.ZK_BROKER,
RecordVersion.current(),
activeControllerApiVersions,
true,
false
@ -115,63 +111,12 @@ public class ApiVersionsResponseTest {
ApiKeys.JOIN_GROUP.latestVersion(), commonResponse);
}
@Test
public void shouldCreateApiResponseOnlyWithKeysSupportedByMagicValue() {
ApiVersionsResponse response = new ApiVersionsResponse.Builder().
setThrottleTimeMs(10).
setApiVersions(ApiVersionsResponse.filterApis(
RecordVersion.V1,
ListenerType.ZK_BROKER,
true,
true)).
setSupportedFeatures(Features.emptySupportedFeatures()).
setFinalizedFeatures(Collections.emptyMap()).
setFinalizedFeaturesEpoch(ApiVersionsResponse.UNKNOWN_FINALIZED_FEATURES_EPOCH).
build();
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 = new ApiVersionsResponse.Builder().
setThrottleTimeMs(10).
setApiVersions(ApiVersionsResponse.filterApis(
RecordVersion.V1,
ListenerType.ZK_BROKER,
true,
true)).
setSupportedFeatures(Features.supportedFeatures(
Utils.mkMap(Utils.mkEntry("feature", new SupportedVersionRange((short) 1, (short) 4))))).
setFinalizedFeatures(Utils.mkMap(Utils.mkEntry("feature", (short) 3))).
setFinalizedFeaturesEpoch(10L).
build();
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(3, fKey.minVersionLevel());
assertEquals(3, fKey.maxVersionLevel());
assertEquals(10, response.data().finalizedFeaturesEpoch());
}
@ParameterizedTest
@EnumSource(names = {"ZK_BROKER", "BROKER"})
public void shouldReturnAllKeysWhenMagicIsCurrentValueAndThrottleMsIsDefaultThrottle(ListenerType listenerType) {
public void shouldReturnAllKeysWhenThrottleMsIsDefaultThrottle(ListenerType listenerType) {
ApiVersionsResponse response = new ApiVersionsResponse.Builder().
setThrottleTimeMs(AbstractResponse.DEFAULT_THROTTLE_TIME).
setApiVersions(ApiVersionsResponse.filterApis(
RecordVersion.current(),
listenerType,
true,
true)).
@ -191,7 +136,6 @@ public class ApiVersionsResponseTest {
ApiVersionsResponse response = new ApiVersionsResponse.Builder().
setThrottleTimeMs(10).
setApiVersions(ApiVersionsResponse.filterApis(
RecordVersion.V1,
ListenerType.BROKER,
true,
true)).
@ -207,7 +151,6 @@ public class ApiVersionsResponseTest {
ApiVersionsResponse response = new ApiVersionsResponse.Builder().
setThrottleTimeMs(10).
setApiVersions(ApiVersionsResponse.filterApis(
RecordVersion.V1,
ListenerType.BROKER,
true,
false)).
@ -223,7 +166,6 @@ public class ApiVersionsResponseTest {
ApiVersionsResponse response = new ApiVersionsResponse.Builder().
setThrottleTimeMs(AbstractResponse.DEFAULT_THROTTLE_TIME).
setApiVersions(ApiVersionsResponse.filterApis(
RecordVersion.current(),
ListenerType.ZK_BROKER,
true,
true)).
@ -278,7 +220,6 @@ public class ApiVersionsResponseTest {
new SupportedVersionRange((short) 0, (short) 1)));
ApiVersionsResponse response = new ApiVersionsResponse.Builder().
setApiVersions(ApiVersionsResponse.filterApis(
RecordVersion.current(),
ListenerType.BROKER,
true,
true)).

View File

@ -42,18 +42,13 @@ public class OffsetsForLeaderEpochRequestTest {
}
@Test
public void testDefaultReplicaId() {
for (short version : ApiKeys.OFFSET_FOR_LEADER_EPOCH.allVersions()) {
int replicaId = 1;
OffsetsForLeaderEpochRequest.Builder builder = OffsetsForLeaderEpochRequest.Builder.forFollower(
version, new OffsetForLeaderTopicCollection(), replicaId);
OffsetsForLeaderEpochRequest request = builder.build();
OffsetsForLeaderEpochRequest parsed = OffsetsForLeaderEpochRequest.parse(request.serialize(), version);
if (version < 3)
assertEquals(OffsetsForLeaderEpochRequest.DEBUGGING_REPLICA_ID, parsed.replicaId());
else
assertEquals(replicaId, parsed.replicaId());
}
public void testForFollower() {
short version = 4;
int replicaId = 1;
OffsetsForLeaderEpochRequest.Builder builder = OffsetsForLeaderEpochRequest.Builder.forFollower(
new OffsetForLeaderTopicCollection(), replicaId);
OffsetsForLeaderEpochRequest request = builder.build();
OffsetsForLeaderEpochRequest parsed = OffsetsForLeaderEpochRequest.parse(request.serialize(), version);
assertEquals(replicaId, parsed.replicaId());
}
}

View File

@ -1007,7 +1007,7 @@ public class RequestResponseTest {
case DELETE_TOPICS: return createDeleteTopicsRequest(version);
case DELETE_RECORDS: return createDeleteRecordsRequest(version);
case INIT_PRODUCER_ID: return createInitPidRequest(version);
case OFFSET_FOR_LEADER_EPOCH: return createLeaderEpochRequestForReplica(version, 1);
case OFFSET_FOR_LEADER_EPOCH: return createLeaderEpochRequestForReplica(1);
case ADD_PARTITIONS_TO_TXN: return createAddPartitionsToTxnRequest(version);
case ADD_OFFSETS_TO_TXN: return createAddOffsetsToTxnRequest(version);
case END_TXN: return createEndTxnRequest(version);
@ -1747,7 +1747,7 @@ public class RequestResponseTest {
.setTopicName("topic1")
.setTopicId(Uuid.randomUuid())
.setPartitions(singletonList(partitionData))));
return new AlterPartitionRequest.Builder(data, version >= 1).build(version);
return new AlterPartitionRequest.Builder(data).build(version);
}
private AlterPartitionResponse createAlterPartitionResponse(int version) {
@ -2905,9 +2905,9 @@ public class RequestResponseTest {
return OffsetsForLeaderEpochRequest.Builder.forConsumer(epochs).build();
}
private OffsetsForLeaderEpochRequest createLeaderEpochRequestForReplica(short version, int replicaId) {
private OffsetsForLeaderEpochRequest createLeaderEpochRequestForReplica(int replicaId) {
OffsetForLeaderTopicCollection epochs = createOffsetForLeaderTopicCollection();
return OffsetsForLeaderEpochRequest.Builder.forFollower(version, epochs, replicaId).build();
return OffsetsForLeaderEpochRequest.Builder.forFollower(epochs, replicaId).build();
}
private OffsetsForLeaderEpochResponse createLeaderEpochResponse() {
@ -3010,7 +3010,7 @@ public class RequestResponseTest {
private WriteTxnMarkersRequest createWriteTxnMarkersRequest(short version) {
List<TopicPartition> partitions = singletonList(new TopicPartition("topic", 73));
WriteTxnMarkersRequest.TxnMarkerEntry txnMarkerEntry = new WriteTxnMarkersRequest.TxnMarkerEntry(21L, (short) 42, 73, TransactionResult.ABORT, partitions);
return new WriteTxnMarkersRequest.Builder(WRITE_TXN_MARKERS.latestVersion(), singletonList(txnMarkerEntry)).build(version);
return new WriteTxnMarkersRequest.Builder(singletonList(txnMarkerEntry)).build(version);
}
private WriteTxnMarkersResponse createWriteTxnMarkersResponse() {

View File

@ -51,7 +51,7 @@ public class WriteTxnMarkersRequestTest {
@Test
public void testConstructor() {
WriteTxnMarkersRequest.Builder builder = new WriteTxnMarkersRequest.Builder(ApiKeys.WRITE_TXN_MARKERS.latestVersion(), markers);
WriteTxnMarkersRequest.Builder builder = new WriteTxnMarkersRequest.Builder(markers);
for (short version : ApiKeys.WRITE_TXN_MARKERS.allVersions()) {
WriteTxnMarkersRequest request = builder.build(version);
assertEquals(1, request.markers().size());
@ -66,7 +66,7 @@ public class WriteTxnMarkersRequestTest {
@Test
public void testGetErrorResponse() {
WriteTxnMarkersRequest.Builder builder = new WriteTxnMarkersRequest.Builder(ApiKeys.WRITE_TXN_MARKERS.latestVersion(), markers);
WriteTxnMarkersRequest.Builder builder = new WriteTxnMarkersRequest.Builder(markers);
for (short version : ApiKeys.WRITE_TXN_MARKERS.allVersions()) {
WriteTxnMarkersRequest request = builder.build(version);
WriteTxnMarkersResponse errorResponse =

View File

@ -201,8 +201,7 @@ public abstract class SaslAuthenticatorFailureDelayTest {
String saslMechanism = (String) saslClientConfigs.get(SaslConfigs.SASL_MECHANISM);
ChannelBuilder channelBuilder = ChannelBuilders.clientChannelBuilder(securityProtocol, JaasContext.Type.CLIENT,
new TestSecurityConfig(clientConfigs), null, saslMechanism, time, true,
new LogContext());
new TestSecurityConfig(clientConfigs), null, saslMechanism, time, new LogContext());
this.selector = NetworkTestUtils.createSelector(channelBuilder, time);
}

View File

@ -1617,7 +1617,6 @@ public class SaslAuthenticatorTest {
null,
null,
"plain",
false,
null,
null,
new LogContext()
@ -1673,7 +1672,7 @@ public class SaslAuthenticatorTest {
Map<String, ?> configs = new TestSecurityConfig(saslClientConfigs).values();
this.channelBuilder = new AlternateSaslChannelBuilder(ConnectionMode.CLIENT,
Collections.singletonMap(saslMechanism, JaasContext.loadClientContext(configs)), securityProtocol, null,
false, saslMechanism, true, credentialCache, null, time);
false, saslMechanism, credentialCache, null, time);
this.channelBuilder.configure(configs);
// initial authentication must succeed
this.selector = NetworkTestUtils.createSelector(channelBuilder, time);
@ -1958,7 +1957,7 @@ public class SaslAuthenticatorTest {
};
SaslChannelBuilder serverChannelBuilder = new SaslChannelBuilder(ConnectionMode.SERVER, jaasContexts,
securityProtocol, listenerName, false, saslMechanism, true,
securityProtocol, listenerName, false, saslMechanism,
credentialCache, null, null, time, new LogContext(), apiVersionSupplier);
serverChannelBuilder.configure(saslServerConfigs);
@ -1999,7 +1998,7 @@ public class SaslAuthenticatorTest {
};
SaslChannelBuilder serverChannelBuilder = new SaslChannelBuilder(ConnectionMode.SERVER, jaasContexts,
securityProtocol, listenerName, false, saslMechanism, true,
securityProtocol, listenerName, false, saslMechanism,
credentialCache, null, null, time, new LogContext(), apiVersionSupplier) {
@Override
protected SaslServerAuthenticator buildServerAuthenticator(Map<String, ?> configs,
@ -2034,7 +2033,7 @@ public class SaslAuthenticatorTest {
final Map<String, JaasContext> jaasContexts = Collections.singletonMap(saslMechanism, jaasContext);
SaslChannelBuilder clientChannelBuilder = new SaslChannelBuilder(ConnectionMode.CLIENT, jaasContexts,
securityProtocol, listenerName, false, saslMechanism, true,
securityProtocol, listenerName, false, saslMechanism,
null, null, null, time, new LogContext(), null) {
@Override
@ -2047,7 +2046,7 @@ public class SaslAuthenticatorTest {
Subject subject) {
return new SaslClientAuthenticator(configs, callbackHandler, id, subject,
servicePrincipal, serverHost, saslMechanism, true,
servicePrincipal, serverHost, saslMechanism,
transportLayer, time, new LogContext()) {
@Override
protected SaslHandshakeRequest createSaslHandshakeRequest(short version) {
@ -2167,8 +2166,7 @@ public class SaslAuthenticatorTest {
String saslMechanism = (String) saslClientConfigs.get(SaslConfigs.SASL_MECHANISM);
this.channelBuilder = ChannelBuilders.clientChannelBuilder(securityProtocol, JaasContext.Type.CLIENT,
new TestSecurityConfig(clientConfigs), null, saslMechanism, time,
true, new LogContext());
new TestSecurityConfig(clientConfigs), null, saslMechanism, time, new LogContext());
this.selector = NetworkTestUtils.createSelector(channelBuilder, time);
}
@ -2572,10 +2570,10 @@ public class SaslAuthenticatorTest {
public AlternateSaslChannelBuilder(ConnectionMode connectionMode, Map<String, JaasContext> jaasContexts,
SecurityProtocol securityProtocol, ListenerName listenerName, boolean isInterBrokerListener,
String clientSaslMechanism, boolean handshakeRequestEnable, CredentialCache credentialCache,
String clientSaslMechanism, CredentialCache credentialCache,
DelegationTokenCache tokenCache, Time time) {
super(connectionMode, jaasContexts, securityProtocol, listenerName, isInterBrokerListener, clientSaslMechanism,
handshakeRequestEnable, credentialCache, tokenCache, null, time, new LogContext(),
credentialCache, tokenCache, null, time, new LogContext(),
version -> TestUtils.defaultApiVersionsResponse(ApiMessageType.ListenerType.ZK_BROKER));
}
@ -2585,10 +2583,10 @@ public class SaslAuthenticatorTest {
TransportLayer transportLayer, Subject subject) {
if (++numInvocations == 1)
return new SaslClientAuthenticator(configs, callbackHandler, id, subject, servicePrincipal, serverHost,
"DIGEST-MD5", true, transportLayer, time, new LogContext());
"DIGEST-MD5", transportLayer, time, new LogContext());
else
return new SaslClientAuthenticator(configs, callbackHandler, id, subject, servicePrincipal, serverHost,
"PLAIN", true, transportLayer, time, new LogContext()) {
"PLAIN", transportLayer, time, new LogContext()) {
@Override
protected SaslHandshakeRequest createSaslHandshakeRequest(short version) {
return new SaslHandshakeRequest.Builder(

View File

@ -31,7 +31,6 @@ import org.apache.kafka.common.network.NetworkReceive;
import org.apache.kafka.common.network.Send;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.record.RecordVersion;
import org.apache.kafka.common.record.UnalignedRecords;
import org.apache.kafka.common.requests.ApiVersionsResponse;
import org.apache.kafka.common.requests.ByteBufferChannel;
@ -665,7 +664,7 @@ public class TestUtils {
) {
return createApiVersionsResponse(
throttleTimeMs,
ApiVersionsResponse.filterApis(RecordVersion.current(), listenerType, true, true),
ApiVersionsResponse.filterApis(listenerType, true, true),
Features.emptySupportedFeatures(),
false
);
@ -678,7 +677,7 @@ public class TestUtils {
) {
return createApiVersionsResponse(
throttleTimeMs,
ApiVersionsResponse.filterApis(RecordVersion.current(), listenerType, enableUnstableLastVersion, true),
ApiVersionsResponse.filterApis(listenerType, enableUnstableLastVersion, true),
Features.emptySupportedFeatures(),
false
);

View File

@ -47,7 +47,6 @@ public class NetworkUtils {
config.interBrokerListenerName(),
config.saslMechanismInterBrokerProtocol(),
time,
config.saslInterBrokerHandshakeRequestEnable(),
logContext
);
@ -89,4 +88,4 @@ public class NetworkUtils {
MetadataRecoveryStrategy.NONE
);
}
}
}

View File

@ -41,7 +41,7 @@ import org.apache.kafka.common.requests._
import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.{UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET}
import org.apache.kafka.common.utils.Time
import org.apache.kafka.metadata.{LeaderAndIsr, LeaderRecoveryState}
import org.apache.kafka.server.common.{MetadataVersion, RequestLocal}
import org.apache.kafka.server.common.RequestLocal
import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchDataInfo, LeaderHwChange, LogAppendInfo, LogOffsetMetadata, LogOffsetSnapshot, LogOffsetsListener, LogReadInfo, LogStartOffsetIncrementReason, OffsetResultHolder, VerificationGuard}
import org.apache.kafka.server.metrics.KafkaMetricsGroup
import org.apache.kafka.server.purgatory.{DelayedOperationPurgatory, TopicPartitionOperationKey}
@ -154,7 +154,6 @@ object Partition {
new Partition(topicPartition,
_topicId = topicId,
replicaLagTimeMaxMs = replicaManager.config.replicaLagTimeMaxMs,
interBrokerProtocolVersion = replicaManager.metadataCache.metadataVersion(),
localBrokerId = replicaManager.config.brokerId,
localBrokerEpochSupplier = replicaManager.brokerEpochSupplier,
time = time,
@ -307,7 +306,6 @@ case class CommittedPartitionState(
*/
class Partition(val topicPartition: TopicPartition,
val replicaLagTimeMaxMs: Long,
interBrokerProtocolVersion: MetadataVersion,
localBrokerId: Int,
localBrokerEpochSupplier: () => Long,
time: Time,
@ -1404,7 +1402,7 @@ class Partition(val topicPartition: TopicPartition,
}
val info = leaderLog.appendAsLeader(records, leaderEpoch = this.leaderEpoch, origin,
interBrokerProtocolVersion, requestLocal, verificationGuard)
requestLocal, verificationGuard)
// we may need to increment high watermark since ISR could be down to 1
(info, maybeIncrementLeaderHW(leaderLog))

View File

@ -128,7 +128,6 @@ class ControllerChannelManager(controllerEpoch: () => Int,
controllerToBrokerListenerName,
config.saslMechanismInterBrokerProtocol,
time,
config.saslInterBrokerHandshakeRequestEnable,
logContext
)
val reconfigurableChannelBuilder = channelBuilder match {
@ -516,12 +515,7 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig,
val leaderAndIsrRequestVersion: Short =
if (metadataVersion.isAtLeast(IBP_3_4_IV0)) 7
else 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
else 5
leaderAndIsrRequestMap.foreachEntry { (broker, leaderAndIsrPartitionStates) =>
if (metadataInstance.liveOrShuttingDownBrokerIds.contains(broker)) {
@ -579,14 +573,7 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig,
val metadataVersion = metadataVersionProvider.apply()
val updateMetadataRequestVersion: Short =
if (metadataVersion.isAtLeast(IBP_3_4_IV0)) 8
else 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
else 7
val liveBrokers = metadataInstance.liveOrShuttingDownBrokers.iterator.map { broker =>
val endpoints = if (updateMetadataRequestVersion == 0) {
@ -648,10 +635,7 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig,
val metadataVersion = metadataVersionProvider.apply()
val stopReplicaRequestVersion: Short =
if (metadataVersion.isAtLeast(IBP_3_4_IV0)) 4
else if (metadataVersion.isAtLeast(IBP_2_6_IV0)) 3
else if (metadataVersion.isAtLeast(IBP_2_4_IV1)) 2
else if (metadataVersion.isAtLeast(IBP_2_2_IV0)) 1
else 0
else 3
def responseCallback(brokerId: Int, isPartitionDeleted: TopicPartition => Boolean)
(response: AbstractResponse): Unit = {

View File

@ -113,7 +113,6 @@ class KafkaController(val config: KafkaConfig,
private val brokerInfo = initialBrokerInfo
@volatile private var _brokerEpoch = initialBrokerEpoch
private val isAlterPartitionEnabled = config.interBrokerProtocolVersion.isAlterPartitionSupported
private val stateChangeLogger = new StateChangeLogger(config.brokerId, inControllerContext = true, None)
val controllerContext = new ControllerContext
var controllerChannelManager = new ControllerChannelManager(
@ -265,7 +264,7 @@ class KafkaController(val config: KafkaConfig,
* This ensures another controller election will be triggered and there will always be an actively serving controller
*/
private def onControllerFailover(): Unit = {
maybeSetupFeatureVersioning()
enableFeatureVersioning()
info("Registering handlers")
@ -437,47 +436,6 @@ class KafkaController(val config: KafkaConfig,
}
}
/**
* Disables the feature versioning system (KIP-584).
*
* 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 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 IBP_2_7_IV0.
*/
private def disableFeatureVersioning(): Unit = {
val newNode = FeatureZNode(config.interBrokerProtocolVersion, FeatureZNodeStatus.Disabled, Map.empty[String, Short])
val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path)
if (version == ZkVersion.UnknownVersion) {
createFeatureZNode(newNode)
} else {
val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get)
if (existingFeatureZNode.status == FeatureZNodeStatus.Disabled &&
existingFeatureZNode.features.nonEmpty) {
warn(s"FeatureZNode at path: ${FeatureZNode.path} with disabled status" +
s" contains non-empty features: ${existingFeatureZNode.features}")
}
if (!newNode.equals(existingFeatureZNode)) {
updateFeatureZNode(newNode)
}
}
}
private def maybeSetupFeatureVersioning(): Unit = {
if (config.isFeatureVersioningSupported) {
enableFeatureVersioning()
} else {
disableFeatureVersioning()
}
}
private def scheduleAutoLeaderRebalanceTask(delay: Long, unit: TimeUnit): Unit = {
kafkaScheduler.scheduleOnce("auto-leader-rebalance-task",
() => eventManager.put(AutoPreferredReplicaLeaderElection),
@ -503,8 +461,6 @@ class KafkaController(val config: KafkaConfig,
// stop token expiry check scheduler
tokenCleanScheduler.shutdown()
// de-register partition ISR listener for on-going partition reassignment task
unregisterPartitionReassignmentIsrChangeHandlers()
// shutdown partition state machine
partitionStateMachine.shutdown()
zkClient.unregisterZNodeChildChangeHandler(topicChangeHandler.path)
@ -828,11 +784,6 @@ class KafkaController(val config: KafkaConfig,
stopRemovedReplicasOfReassignedPartition(topicPartition, unneededReplicas)
}
if (!isAlterPartitionEnabled) {
val reassignIsrChangeHandler = new PartitionReassignmentIsrChangeHandler(eventManager, topicPartition)
zkClient.registerZNodeChangeHandler(reassignIsrChangeHandler)
}
controllerContext.partitionsBeingReassigned.add(topicPartition)
}
@ -1121,22 +1072,9 @@ class KafkaController(val config: KafkaConfig,
}
}
private def unregisterPartitionReassignmentIsrChangeHandlers(): Unit = {
if (!isAlterPartitionEnabled) {
controllerContext.partitionsBeingReassigned.foreach { tp =>
val path = TopicPartitionStateZNode.path(tp)
zkClient.unregisterZNodeChangeHandler(path)
}
}
}
private def removePartitionFromReassigningPartitions(topicPartition: TopicPartition,
assignment: ReplicaAssignment): Unit = {
if (controllerContext.partitionsBeingReassigned.contains(topicPartition)) {
if (!isAlterPartitionEnabled) {
val path = TopicPartitionStateZNode.path(topicPartition)
zkClient.unregisterZNodeChangeHandler(path)
}
maybeRemoveFromZkReassignment((tp, replicas) => tp == topicPartition && replicas == assignment.replicas)
controllerContext.partitionsBeingReassigned.remove(topicPartition)
} else {
@ -1566,7 +1504,6 @@ class KafkaController(val config: KafkaConfig,
// of the cache are compatible with the supported features of each broker.
brokersAndEpochs.partition {
case (broker, _) =>
!config.isFeatureVersioningSupported ||
!featureCache.getFeatureOption.exists(
latestFinalizedFeatures =>
BrokerFeatures.hasIncompatibleFeatures(broker.features,
@ -1677,12 +1614,9 @@ class KafkaController(val config: KafkaConfig,
private def processTopicIds(topicIdAssignments: Set[TopicIdReplicaAssignment]): Unit = {
// Create topic IDs for topics missing them if we are using topic IDs
// Otherwise, maintain what we have in the topicZNode
val updatedTopicIdAssignments = if (config.usesTopicId) {
val updatedTopicIdAssignments = {
val (withTopicIds, withoutTopicIds) = topicIdAssignments.partition(_.topicId.isDefined)
withTopicIds ++ zkClient.setTopicIds(withoutTopicIds, controllerContext.epochZkVersion)
} else {
topicIdAssignments
}
// Add topic IDs to controller context

View File

@ -44,7 +44,6 @@ import org.apache.kafka.common.{TopicIdPartition, TopicPartition}
import org.apache.kafka.coordinator.group.{OffsetAndMetadata, OffsetConfig}
import org.apache.kafka.coordinator.group.generated.{CoordinatorRecordType, GroupMetadataValue, LegacyOffsetCommitKey, OffsetCommitKey, OffsetCommitValue, GroupMetadataKey => GroupMetadataKeyData}
import org.apache.kafka.server.common.{MetadataVersion, RequestLocal}
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 org.apache.kafka.server.metrics.KafkaMetricsGroup
import org.apache.kafka.server.storage.log.FetchIsolation
import org.apache.kafka.server.util.KafkaScheduler
@ -246,7 +245,7 @@ class GroupMetadataManager(brokerId: Int,
val timestampType = TimestampType.CREATE_TIME
val timestamp = time.milliseconds()
val key = GroupMetadataManager.groupMetadataKey(group.groupId)
val value = GroupMetadataManager.groupMetadataValue(group, groupAssignment, interBrokerProtocolVersion)
val value = GroupMetadataManager.groupMetadataValue(group, groupAssignment)
val records = {
val buffer = ByteBuffer.allocate(AbstractRecords.estimateSizeInBytes(RecordBatch.CURRENT_MAGIC_VALUE, compression.`type`(),
@ -351,7 +350,7 @@ class GroupMetadataManager(brokerId: Int,
val records = filteredOffsetMetadata.map { case (topicIdPartition, offsetAndMetadata) =>
val key = GroupMetadataManager.offsetCommitKey(groupId, topicIdPartition.topicPartition)
val value = GroupMetadataManager.offsetCommitValue(offsetAndMetadata, interBrokerProtocolVersion)
val value = GroupMetadataManager.offsetCommitValue(offsetAndMetadata)
new SimpleRecord(timestamp, key, value)
}
val buffer = ByteBuffer.allocate(AbstractRecords.estimateSizeInBytes(magicValue, compression.`type`(), records.asJava))
@ -1085,17 +1084,15 @@ object GroupMetadataManager {
* Generates the payload for offset commit message from given offset and metadata
*
* @param offsetAndMetadata consumer's current offset and metadata
* @param metadataVersion the api version
* @param maxVersion the highest version allowed, we may use a lower version for compatibility reasons
* we serialize with the highest supported non-flexible version until a tagged field is introduced
* or the version is bumped.
* @return payload for offset commit message
*/
def offsetCommitValue(offsetAndMetadata: OffsetAndMetadata,
metadataVersion: MetadataVersion): Array[Byte] = {
def offsetCommitValue(offsetAndMetadata: OffsetAndMetadata, maxVersion: Short = 3): Array[Byte] = {
val version =
if (metadataVersion.isLessThan(IBP_2_1_IV0) || offsetAndMetadata.expireTimestampMs.isPresent) 1.toShort
else if (metadataVersion.isLessThan(IBP_2_1_IV1)) 2.toShort
// Serialize with the highest supported non-flexible version
// until a tagged field is introduced or the version is bumped.
else 3.toShort
if (offsetAndMetadata.expireTimestampMs.isPresent) Math.min(1, maxVersion).toShort
else maxVersion
MessageUtil.toVersionPrefixedBytes(version, new OffsetCommitValue()
.setOffset(offsetAndMetadata.committedOffset)
.setMetadata(offsetAndMetadata.metadata)
@ -1112,21 +1109,14 @@ object GroupMetadataManager {
*
* @param groupMetadata current group metadata
* @param assignment the assignment for the rebalancing generation
* @param metadataVersion the api version
* @param version the version to serialize it with, the default is `3`, the highest supported non-flexible version
* until a tagged field is introduced or the version is bumped. The default should always be used
* outside of tests
* @return payload for offset commit message
*/
def groupMetadataValue(groupMetadata: GroupMetadata,
assignment: Map[String, Array[Byte]],
metadataVersion: MetadataVersion): Array[Byte] = {
val version =
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
// Serialize with the highest supported non-flexible version
// until a tagged field is introduced or the version is bumped.
else 3.toShort
version: Short = 3): Array[Byte] = {
MessageUtil.toVersionPrefixedBytes(version, new GroupMetadataValue()
.setProtocolType(groupMetadata.protocolType.getOrElse(""))
.setGeneration(groupMetadata.generationId)

View File

@ -62,7 +62,6 @@ object TransactionMarkerChannelManager {
config.interBrokerListenerName,
config.saslMechanismInterBrokerProtocol,
time,
config.saslInterBrokerHandshakeRequestEnable,
logContext
)
channelBuilder match {
@ -256,9 +255,7 @@ class TransactionMarkerChannelManager(
}.filter { case (_, entries) => !entries.isEmpty }.map { case (node, entries) =>
val markersToSend = entries.asScala.map(_.txnMarkerEntry).asJava
val requestCompletionHandler = new TransactionMarkerRequestCompletionHandler(node.id, txnStateManager, this, entries)
val request = new WriteTxnMarkersRequest.Builder(
metadataCache.metadataVersion().writeTxnMarkersRequestVersion(), markersToSend
)
val request = new WriteTxnMarkersRequest.Builder(markersToSend)
new RequestAndCompletionHandler(
currentTimeMs,

View File

@ -29,7 +29,7 @@ import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_
import org.apache.kafka.common.requests.ProduceResponse.RecordError
import org.apache.kafka.common.utils.{PrimitiveRef, Time, Utils}
import org.apache.kafka.common.{InvalidRecordException, KafkaException, TopicPartition, Uuid}
import org.apache.kafka.server.common.{MetadataVersion, OffsetAndEpoch, RequestLocal}
import org.apache.kafka.server.common.{OffsetAndEpoch, RequestLocal}
import org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig
import org.apache.kafka.server.metrics.KafkaMetricsGroup
import org.apache.kafka.server.record.BrokerCompressionType
@ -698,7 +698,6 @@ class UnifiedLog(@volatile var logStartOffset: Long,
*
* @param records The records to append
* @param origin Declares the origin of the append which affects required validations
* @param interBrokerProtocolVersion Inter-broker message protocol version
* @param requestLocal request local instance
* @throws KafkaStorageException If the append fails due to an I/O error.
* @return Information about the appended messages including the first and last offset.
@ -706,11 +705,10 @@ class UnifiedLog(@volatile var logStartOffset: Long,
def appendAsLeader(records: MemoryRecords,
leaderEpoch: Int,
origin: AppendOrigin = AppendOrigin.CLIENT,
interBrokerProtocolVersion: MetadataVersion = MetadataVersion.latestProduction,
requestLocal: RequestLocal = RequestLocal.noCaching,
verificationGuard: VerificationGuard = VerificationGuard.SENTINEL): LogAppendInfo = {
val validateAndAssignOffsets = origin != AppendOrigin.RAFT_LEADER
append(records, origin, interBrokerProtocolVersion, validateAndAssignOffsets, leaderEpoch, Some(requestLocal), verificationGuard, ignoreRecordSize = false)
append(records, origin, validateAndAssignOffsets, leaderEpoch, Some(requestLocal), verificationGuard, ignoreRecordSize = false)
}
/**
@ -721,7 +719,7 @@ class UnifiedLog(@volatile var logStartOffset: Long,
* Also see #appendAsLeader.
*/
private[log] def appendAsLeaderWithRecordVersion(records: MemoryRecords, leaderEpoch: Int, recordVersion: RecordVersion): LogAppendInfo = {
append(records, AppendOrigin.CLIENT, MetadataVersion.latestProduction, true, leaderEpoch, Some(RequestLocal.noCaching),
append(records, AppendOrigin.CLIENT, true, leaderEpoch, Some(RequestLocal.noCaching),
VerificationGuard.SENTINEL, ignoreRecordSize = false, recordVersion.value)
}
@ -735,7 +733,6 @@ class UnifiedLog(@volatile var logStartOffset: Long,
def appendAsFollower(records: MemoryRecords): LogAppendInfo = {
append(records,
origin = AppendOrigin.REPLICATION,
interBrokerProtocolVersion = MetadataVersion.latestProduction,
validateAndAssignOffsets = false,
leaderEpoch = -1,
requestLocal = None,
@ -752,7 +749,6 @@ class UnifiedLog(@volatile var logStartOffset: Long,
*
* @param records The log records to append
* @param origin Declares the origin of the append which affects required validations
* @param interBrokerProtocolVersion Inter-broker message protocol version
* @param validateAndAssignOffsets Should the log assign offsets to this message set or blindly apply what it is given
* @param leaderEpoch The partition's leader epoch which will be applied to messages when offsets are assigned on the leader
* @param requestLocal The request local instance if validateAndAssignOffsets is true
@ -764,7 +760,6 @@ class UnifiedLog(@volatile var logStartOffset: Long,
*/
private def append(records: MemoryRecords,
origin: AppendOrigin,
interBrokerProtocolVersion: MetadataVersion,
validateAndAssignOffsets: Boolean,
leaderEpoch: Int,
requestLocal: Option[RequestLocal],
@ -805,8 +800,7 @@ class UnifiedLog(@volatile var logStartOffset: Long,
config.messageTimestampBeforeMaxMs,
config.messageTimestampAfterMaxMs,
leaderEpoch,
origin,
interBrokerProtocolVersion
origin
)
validator.validateMessagesAndAssignOffsets(offset,
validatorMetricsRecorder,

View File

@ -243,7 +243,6 @@ class KafkaRaftManager[T](
controllerListenerName,
config.saslMechanismControllerProtocol,
time,
config.saslInterBrokerHandshakeRequestEnable,
logContext
)

View File

@ -94,8 +94,6 @@ abstract class AbstractFetcherThread(name: String,
protected def endOffsetForEpoch(topicPartition: TopicPartition, epoch: Int): Option[OffsetAndEpoch]
protected val isOffsetForLeaderEpochSupported: Boolean
override def shutdown(): Unit = {
initiateShutdown()
inLock(partitionMapLock) {
@ -151,7 +149,7 @@ abstract class AbstractFetcherThread(name: String,
partitionStates.partitionStateMap.forEach { (tp, state) =>
if (state.isTruncating) {
latestEpoch(tp) match {
case Some(epoch) if isOffsetForLeaderEpochSupported =>
case Some(epoch) =>
partitionsWithEpochs += tp -> new EpochData()
.setPartition(tp.partition)
.setCurrentLeaderEpoch(state.currentLeaderEpoch)

View File

@ -229,9 +229,6 @@ class DefaultAlterPartitionManager(
* supported by the controller. The final decision is taken when the AlterPartitionRequest
* is built in the network client based on the advertised api versions of the controller.
*
* We could use version 2 or above if all the pending changes have an topic id defined;
* otherwise we must use version 1 or below.
*
* @return A tuple containing the AlterPartitionRequest.Builder and a mapping from
* topic id to topic name. This mapping is used in the response handling.
*/
@ -245,9 +242,6 @@ class DefaultAlterPartitionManager(
// the metadata cache is updated after the partition state so it might not know
// yet about a topic id already used here.
val topicNamesByIds = mutable.HashMap[Uuid, String]()
// We can use topic ids only if all the pending changed have one defined and
// we use IBP 2.8 or above.
var canUseTopicIds = metadataVersion.isTopicIdsSupported
val message = new AlterPartitionRequestData()
.setBrokerId(brokerId)
@ -255,7 +249,6 @@ class DefaultAlterPartitionManager(
inflightAlterPartitionItems.groupBy(_.topicIdPartition.topic).foreach { case (topicName, items) =>
val topicId = items.head.topicIdPartition.topicId
canUseTopicIds &= topicId != Uuid.ZERO_UUID
topicNamesByIds(topicId) = topicName
// Both the topic name and the topic id are set here because at this stage
@ -280,8 +273,7 @@ class DefaultAlterPartitionManager(
}
}
// If we cannot use topic ids, the builder will ensure that no version higher than 1 is used.
(new AlterPartitionRequest.Builder(message, canUseTopicIds), topicNamesByIds)
(new AlterPartitionRequest.Builder(message), topicNamesByIds)
}
private def handleAlterPartitionResponse(

View File

@ -150,14 +150,12 @@ class DefaultApiVersionManager(
}
val apiVersions = if (controllerApiVersions.isDefined) {
ApiVersionsResponse.controllerApiVersions(
finalizedFeatures.metadataVersion().highestSupportedRecordVersion,
controllerApiVersions.get,
listenerType,
enableUnstableLastVersion,
clientTelemetryEnabled)
} else {
ApiVersionsResponse.brokerApiVersions(
finalizedFeatures.metadataVersion().highestSupportedRecordVersion,
listenerType,
enableUnstableLastVersion,
clientTelemetryEnabled)

View File

@ -60,7 +60,6 @@ class BrokerBlockingSender(sourceBroker: BrokerEndPoint,
brokerConfig.interBrokerListenerName,
brokerConfig.saslMechanismInterBrokerProtocol,
time,
brokerConfig.saslInterBrokerHandshakeRequestEnable,
logContext
)
val reconfigurableChannelBuilder = channelBuilder match {

View File

@ -71,8 +71,7 @@ import org.apache.kafka.coordinator.group.{Group, GroupCoordinator}
import org.apache.kafka.coordinator.share.ShareCoordinator
import org.apache.kafka.server.ClientMetricsManager
import org.apache.kafka.server.authorizer._
import org.apache.kafka.server.common.{GroupVersion, MetadataVersion, RequestLocal, TransactionVersion}
import org.apache.kafka.server.common.MetadataVersion.{IBP_0_11_0_IV0, IBP_2_3_IV0}
import org.apache.kafka.server.common.{GroupVersion, RequestLocal, TransactionVersion}
import org.apache.kafka.server.share.context.ShareFetchContext
import org.apache.kafka.server.share.{ErroneousAndValidPartitionData, SharePartitionKey}
import org.apache.kafka.server.share.acknowledge.ShareAcknowledgementBatch
@ -297,12 +296,6 @@ class KafkaApis(val requestChannel: RequestChannel,
if (!authHelper.authorize(request.context, READ, GROUP, offsetCommitRequest.data.groupId)) {
requestHelper.sendMaybeThrottle(request, offsetCommitRequest.getErrorResponse(Errors.GROUP_AUTHORIZATION_FAILED.exception))
CompletableFuture.completedFuture[Unit](())
} else if (offsetCommitRequest.data.groupInstanceId != null && metadataCache.metadataVersion().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.
requestHelper.sendMaybeThrottle(request, offsetCommitRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))
CompletableFuture.completedFuture[Unit](())
} else {
val authorizedTopics = authHelper.filterByAuthorized(
request.context,
@ -1437,13 +1430,7 @@ class KafkaApis(val requestChannel: RequestChannel,
): CompletableFuture[Unit] = {
val joinGroupRequest = request.body[JoinGroupRequest]
if (joinGroupRequest.data.groupInstanceId != null && metadataCache.metadataVersion().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.
requestHelper.sendMaybeThrottle(request, joinGroupRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))
CompletableFuture.completedFuture[Unit](())
} else if (!authHelper.authorize(request.context, READ, GROUP, joinGroupRequest.data.groupId)) {
if (!authHelper.authorize(request.context, READ, GROUP, joinGroupRequest.data.groupId)) {
requestHelper.sendMaybeThrottle(request, joinGroupRequest.getErrorResponse(Errors.GROUP_AUTHORIZATION_FAILED.exception))
CompletableFuture.completedFuture[Unit](())
} else {
@ -1467,13 +1454,7 @@ class KafkaApis(val requestChannel: RequestChannel,
): CompletableFuture[Unit] = {
val syncGroupRequest = request.body[SyncGroupRequest]
if (syncGroupRequest.data.groupInstanceId != null && metadataCache.metadataVersion().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.
requestHelper.sendMaybeThrottle(request, syncGroupRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))
CompletableFuture.completedFuture[Unit](())
} else if (!syncGroupRequest.areMandatoryProtocolTypeAndNamePresent()) {
if (!syncGroupRequest.areMandatoryProtocolTypeAndNamePresent()) {
// Starting from version 5, ProtocolType and ProtocolName fields are mandatory.
requestHelper.sendMaybeThrottle(request, syncGroupRequest.getErrorResponse(Errors.INCONSISTENT_GROUP_PROTOCOL.exception))
CompletableFuture.completedFuture[Unit](())
@ -1536,13 +1517,7 @@ class KafkaApis(val requestChannel: RequestChannel,
def handleHeartbeatRequest(request: RequestChannel.Request): CompletableFuture[Unit] = {
val heartbeatRequest = request.body[HeartbeatRequest]
if (heartbeatRequest.data.groupInstanceId != null && metadataCache.metadataVersion().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.
requestHelper.sendMaybeThrottle(request, heartbeatRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))
CompletableFuture.completedFuture[Unit](())
} else if (!authHelper.authorize(request.context, READ, GROUP, heartbeatRequest.data.groupId)) {
if (!authHelper.authorize(request.context, READ, GROUP, heartbeatRequest.data.groupId)) {
requestHelper.sendMaybeThrottle(request, heartbeatRequest.getErrorResponse(Errors.GROUP_AUTHORIZATION_FAILED.exception))
CompletableFuture.completedFuture[Unit](())
} else {
@ -1966,7 +1941,6 @@ class KafkaApis(val requestChannel: RequestChannel,
}
def handleEndTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
ensureInterBrokerVersion(IBP_0_11_0_IV0)
val endTxnRequest = request.body[EndTxnRequest]
val transactionalId = endTxnRequest.data.transactionalId
@ -2010,7 +1984,6 @@ class KafkaApis(val requestChannel: RequestChannel,
}
def handleWriteTxnMarkersRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
ensureInterBrokerVersion(IBP_0_11_0_IV0)
// We are checking for AlterCluster permissions first. If it is not present, we are authorizing cluster operation
// The latter will throw an exception if it is denied.
if (!authHelper.authorize(request.context, ALTER, CLUSTER, CLUSTER_NAME, logIfDenied = false)) {
@ -2183,13 +2156,7 @@ class KafkaApis(val requestChannel: RequestChannel,
requestHelper.sendResponseExemptThrottle(request, new WriteTxnMarkersResponse(errors))
}
def ensureInterBrokerVersion(version: MetadataVersion): Unit = {
if (metadataCache.metadataVersion().isLessThan(version))
throw new UnsupportedVersionException(s"metadata.version: ${metadataCache.metadataVersion()} is less than the required version: ${version}")
}
def handleAddPartitionsToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
ensureInterBrokerVersion(IBP_0_11_0_IV0)
val addPartitionsToTxnRequest =
if (request.context.apiVersion() < 4)
request.body[AddPartitionsToTxnRequest].normalizeRequest()
@ -2302,7 +2269,6 @@ class KafkaApis(val requestChannel: RequestChannel,
}
def handleAddOffsetsToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
ensureInterBrokerVersion(IBP_0_11_0_IV0)
val addOffsetsToTxnRequest = request.body[AddOffsetsToTxnRequest]
val transactionalId = addOffsetsToTxnRequest.data.transactionalId
val groupId = addOffsetsToTxnRequest.data.groupId
@ -2356,7 +2322,6 @@ class KafkaApis(val requestChannel: RequestChannel,
request: RequestChannel.Request,
requestLocal: RequestLocal
): CompletableFuture[Unit] = {
ensureInterBrokerVersion(IBP_0_11_0_IV0)
val txnOffsetCommitRequest = request.body[TxnOffsetCommitRequest]
def sendResponse(response: TxnOffsetCommitResponse): Unit = {
@ -3279,8 +3244,6 @@ class KafkaApis(val requestChannel: RequestChannel,
sendResponseCallback(Left(new ApiError(Errors.CLUSTER_AUTHORIZATION_FAILED)))
} else if (!zkSupport.controller.isActive) {
sendResponseCallback(Left(new ApiError(Errors.NOT_CONTROLLER)))
} else if (!config.isFeatureVersioningSupported) {
sendResponseCallback(Left(new ApiError(Errors.INVALID_REQUEST, "Feature versioning system is disabled.")))
} else {
zkSupport.controller.updateFeatures(updateFeaturesRequest, sendResponseCallback)
}

View File

@ -24,12 +24,12 @@ import kafka.cluster.EndPoint
import kafka.utils.{CoreUtils, Logging}
import kafka.utils.Implicits._
import org.apache.kafka.common.Reconfigurable
import org.apache.kafka.common.config.{ConfigDef, ConfigException, ConfigResource, SaslConfigs, TopicConfig}
import org.apache.kafka.common.config.{ConfigDef, ConfigException, ConfigResource, TopicConfig}
import org.apache.kafka.common.config.ConfigDef.ConfigKey
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs
import org.apache.kafka.common.config.types.Password
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.record.{CompressionType, TimestampType}
import org.apache.kafka.common.record.TimestampType
import org.apache.kafka.common.security.auth.KafkaPrincipalSerde
import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.utils.Utils
@ -43,7 +43,6 @@ import org.apache.kafka.security.authorizer.AuthorizerUtils
import org.apache.kafka.server.ProcessRole
import org.apache.kafka.server.authorizer.Authorizer
import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.server.common.MetadataVersion._
import org.apache.kafka.server.config.{AbstractKafkaConfig, DelegationTokenManagerConfigs, KRaftConfigs, QuotaConfig, ReplicationConfigs, ServerConfigs, ServerLogConfigs, ShareCoordinatorConfig, ZkConfigs}
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
import org.apache.kafka.server.metrics.MetricConfigs
@ -428,9 +427,6 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _])
/** ********* Controlled shutdown configuration ***********/
val controlledShutdownEnable = getBoolean(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG)
/** ********* Feature configuration ***********/
def isFeatureVersioningSupported = interBrokerProtocolVersion.isFeatureVersioningSupported
/** New group coordinator configs */
val isNewGroupCoordinatorEnabled = getBoolean(GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG)
val groupCoordinatorRebalanceProtocols = {
@ -477,7 +473,6 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _])
def interBrokerListenerName = getInterBrokerListenerNameAndSecurityProtocol._1
def interBrokerSecurityProtocol = getInterBrokerListenerNameAndSecurityProtocol._2
def saslMechanismInterBrokerProtocol = getString(BrokerSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG)
val saslInterBrokerHandshakeRequestEnable = interBrokerProtocolVersion.isSaslInterBrokerHandshakeRequestEnabled
/** ********* DelegationToken Configuration **************/
val delegationTokenSecretKey = getPassword(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG)
@ -636,10 +631,6 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _])
}
}
// 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.isTopicIdsSupported
validateValues()
private def validateValues(): Unit = {
@ -780,14 +771,7 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _])
s"${SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG} cannot use the nonroutable meta-address 0.0.0.0. "+
s"Use a routable IP address.")
if (groupCoordinatorConfig.offsetTopicCompressionType == CompressionType.ZSTD)
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 ${IBP_2_1_IV0.shortVersion} or higher")
val interBrokerUsesSasl = interBrokerSecurityProtocol == SecurityProtocol.SASL_PLAINTEXT || interBrokerSecurityProtocol == SecurityProtocol.SASL_SSL
require(!interBrokerUsesSasl || saslInterBrokerHandshakeRequestEnable || saslMechanismInterBrokerProtocol == SaslConfigs.GSSAPI_MECHANISM,
s"Only GSSAPI mechanism is supported for inter-broker communication with SASL when inter.broker.protocol.version is set to $interBrokerProtocolVersionString")
require(!interBrokerUsesSasl || saslEnabledMechanisms(interBrokerListenerName).contains(saslMechanismInterBrokerProtocol),
s"${BrokerSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG} must be included in ${BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG} when SASL is used for inter-broker communication")
require(queuedMaxBytes <= 0 || queuedMaxBytes >= socketRequestMaxBytes,

View File

@ -157,7 +157,6 @@ class NodeToControllerChannelManagerImpl(
controllerInfo.listenerName,
controllerInfo.saslMechanism,
time,
config.saslInterBrokerHandshakeRequestEnable,
logContext
)
channelBuilder match {

View File

@ -63,7 +63,7 @@ class RemoteLeaderEndPoint(logPrefix: String,
private val maxBytes = brokerConfig.replicaFetchResponseMaxBytes
private val fetchSize = brokerConfig.replicaFetchMaxBytes
override def isTruncationOnFetchSupported: Boolean = metadataVersionSupplier().isTruncationOnFetchSupported
override def isTruncationOnFetchSupported: Boolean = true
override def initiateClose(): Unit = blockingSender.initiateClose()
@ -143,8 +143,7 @@ class RemoteLeaderEndPoint(logPrefix: String,
topic.partitions.add(epochData)
}
val epochRequest = OffsetsForLeaderEpochRequest.Builder.forFollower(
metadataVersionSupplier().offsetForLeaderEpochRequestVersion, topics, brokerConfig.brokerId)
val epochRequest = OffsetsForLeaderEpochRequest.Builder.forFollower(topics, brokerConfig.brokerId)
debug(s"Sending offset for leader epoch request $epochRequest")
try {

View File

@ -169,8 +169,6 @@ class ReplicaAlterLogDirsThread(name: String,
}
}
override protected val isOffsetForLeaderEpochSupported: Boolean = true
/**
* Truncate the log for each partition based on current replica's returned epoch and offset.
*

View File

@ -47,8 +47,6 @@ class ReplicaFetcherThread(name: String,
// Visible for testing
private[server] val partitionsWithNewHighWatermark = mutable.Buffer[TopicPartition]()
override protected val isOffsetForLeaderEpochSupported: Boolean = metadataVersionSupplier().isOffsetForLeaderEpochSupported
override protected def latestEpoch(topicPartition: TopicPartition): Option[Int] = {
replicaMgr.localLogOrException(topicPartition).latestEpoch
}

View File

@ -57,7 +57,6 @@ import org.apache.kafka.metadata.LeaderAndIsr
import org.apache.kafka.metadata.LeaderConstants.NO_LEADER
import org.apache.kafka.server.{ActionQueue, DelayedActionQueue, common}
import org.apache.kafka.server.common.{DirectoryEventHandler, RequestLocal, StopPartition, TopicOptionalIdPartition}
import org.apache.kafka.server.common.MetadataVersion._
import org.apache.kafka.server.metrics.KafkaMetricsGroup
import org.apache.kafka.server.network.BrokerEndPoint
import org.apache.kafka.server.purgatory.{DelayedOperationKey, DelayedOperationPurgatory, TopicPartitionOperationKey}
@ -340,13 +339,9 @@ class ReplicaManager(val config: KafkaConfig,
private var logDirFailureHandler: LogDirFailureHandler = _
private class LogDirFailureHandler(name: String, haltBrokerOnDirFailure: Boolean) extends ShutdownableThread(name) {
private class LogDirFailureHandler(name: String) extends ShutdownableThread(name) {
override def doWork(): Unit = {
val newOfflineLogDir = logDirFailureChannel.takeNextOfflineLogDir()
if (haltBrokerOnDirFailure) {
fatal(s"Halting broker because dir $newOfflineLogDir is offline")
Exit.halt(1)
}
handleLogDirFailure(newOfflineLogDir)
}
}
@ -412,11 +407,7 @@ class ReplicaManager(val config: KafkaConfig,
scheduler.schedule("isr-expiration", () => maybeShrinkIsr(), 0L, config.replicaLagTimeMaxMs / 2)
scheduler.schedule("shutdown-idle-replica-alter-log-dirs-thread", () => shutdownIdleReplicaAlterLogDirsThread(), 0L, 10000L)
// 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 = metadataCache.metadataVersion().isLessThan(IBP_1_0_IV0)
logDirFailureHandler = new LogDirFailureHandler("LogDirFailureHandler", haltBrokerOnFailure)
logDirFailureHandler = new LogDirFailureHandler("LogDirFailureHandler")
logDirFailureHandler.start()
addPartitionsToTxnManager.foreach(_.start())
remoteLogManager.foreach(rlm => rlm.setDelayedOperationPurgatory(delayedRemoteListOffsetsPurgatory))
@ -2562,7 +2553,7 @@ class ReplicaManager(val config: KafkaConfig,
* OffsetForLeaderEpoch request.
*/
protected def initialFetchOffset(log: UnifiedLog): Long = {
if (metadataCache.metadataVersion().isTruncationOnFetchSupported && log.latestEpoch.nonEmpty)
if (log.latestEpoch.nonEmpty)
log.logEndOffset
else
log.highWatermark

View File

@ -213,7 +213,7 @@ class ZkAdminManager(val config: KafkaConfig,
CreatePartitionsMetadata(topic.name, assignments.keySet)
} else {
controllerMutationQuota.record(assignments.size)
adminZkClient.createTopicWithAssignment(topic.name, configs, assignments, validate = false, config.usesTopicId)
adminZkClient.createTopicWithAssignment(topic.name, configs, assignments, validate = false)
populateIds(includeConfigsAndMetadata, topic.name)
CreatePartitionsMetadata(topic.name, assignments.keySet)
}

View File

@ -314,7 +314,7 @@ object StorageTool extends Logging {
formatParser.addArgument("--release-version", "-r")
.action(store())
.help(s"The release version to use for the initial feature settings. The minimum is " +
s"${MetadataVersion.IBP_3_0_IV0}; the default is ${MetadataVersion.LATEST_PRODUCTION}")
s"${MetadataVersion.IBP_3_0_IV1}; the default is ${MetadataVersion.LATEST_PRODUCTION}")
formatParser.addArgument("--feature", "-f")
.help("The setting to use for a specific feature, in feature=level format. For example: `kraft.version=1`.")
@ -347,7 +347,7 @@ object StorageTool extends Logging {
versionMappingParser.addArgument("--release-version", "-r")
.action(store())
.help(s"The release version to use for the corresponding feature mapping. The minimum is " +
s"${MetadataVersion.IBP_3_0_IV0}; the default is ${MetadataVersion.LATEST_PRODUCTION}")
s"${MetadataVersion.IBP_3_0_IV1}; the default is ${MetadataVersion.LATEST_PRODUCTION}")
}
private def addFeatureDependenciesParser(subparsers: Subparsers): Unit = {

View File

@ -39,7 +39,6 @@ import org.apache.kafka.common.test.api.ClusterTestExtensions;
import org.apache.kafka.common.test.api.Type;
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig;
import org.apache.kafka.metadata.BrokerState;
import org.apache.kafka.server.common.MetadataVersion;
import org.apache.kafka.server.common.RequestLocal;
import org.apache.kafka.server.config.ServerConfigs;
import org.apache.kafka.storage.internals.log.AppendOrigin;
@ -374,7 +373,6 @@ public class DeleteTopicTest {
),
0,
AppendOrigin.CLIENT,
MetadataVersion.LATEST_PRODUCTION,
RequestLocal.noCaching(),
VerificationGuard.SENTINEL
);
@ -384,4 +382,4 @@ public class DeleteTopicTest {
}
return result;
}
}
}

View File

@ -260,8 +260,7 @@ class GssapiAuthenticationTest extends IntegrationTestHarness with SaslSetup {
private def createSelector(): Selector = {
val channelBuilder = ChannelBuilders.clientChannelBuilder(securityProtocol,
JaasContext.Type.CLIENT, new TestSecurityConfig(clientConfig), null, kafkaClientSaslMechanism,
time, true, new LogContext())
JaasContext.Type.CLIENT, new TestSecurityConfig(clientConfig), null, kafkaClientSaslMechanism, time, new LogContext())
NetworkTestUtils.createSelector(channelBuilder, time)
}
@ -270,7 +269,7 @@ class GssapiAuthenticationTest extends IntegrationTestHarness with SaslSetup {
val config = new TestSecurityConfig(clientConfig)
val jaasContexts = Collections.singletonMap("GSSAPI", JaasContext.loadClientContext(config.values()))
val channelBuilder = new SaslChannelBuilder(ConnectionMode.CLIENT, jaasContexts, securityProtocol,
null, false, kafkaClientSaslMechanism, true, null, null, null, time, new LogContext(),
null, false, kafkaClientSaslMechanism, null, null, null, time, new LogContext(),
_ => org.apache.kafka.test.TestUtils.defaultApiVersionsResponse(ListenerType.ZK_BROKER)) {
override protected def defaultLoginClass(): Class[_ <: Login] = classOf[TestableKerberosLogin]
}

View File

@ -443,7 +443,7 @@ class KRaftClusterTest {
"metadata from testkit", assertThrows(classOf[RuntimeException], () => {
new KafkaClusterTestKit.Builder(
new TestKitNodes.Builder().
setBootstrapMetadataVersion(MetadataVersion.IBP_2_7_IV0).
setBootstrapMetadataVersion(MetadataVersion.IBP_3_0_IV1).
setNumBrokerNodes(1).
setNumControllerNodes(1).build()).build()
}).getMessage)

View File

@ -84,7 +84,6 @@ class AbstractPartitionTest {
alterPartitionListener = createIsrChangeListener()
partition = new Partition(topicPartition,
replicaLagTimeMaxMs = ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_DEFAULT,
interBrokerProtocolVersion = interBrokerProtocolVersion,
localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId),
time,

View File

@ -35,7 +35,7 @@ import org.apache.kafka.common.utils.Utils
import org.apache.kafka.common.{TopicPartition, Uuid}
import org.apache.kafka.coordinator.transaction.TransactionLogConfig
import org.apache.kafka.metadata.LeaderAndIsr
import org.apache.kafka.server.common.{MetadataVersion, RequestLocal}
import org.apache.kafka.server.common.RequestLocal
import org.apache.kafka.server.config.ReplicationConfigs
import org.apache.kafka.server.storage.log.{FetchIsolation, FetchParams}
import org.apache.kafka.server.util.MockTime
@ -275,7 +275,6 @@ class PartitionLockTest extends Logging {
logManager.startup(Set.empty)
val partition = new Partition(topicPartition,
replicaLagTimeMaxMs = ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_DEFAULT,
interBrokerProtocolVersion = MetadataVersion.latestTesting,
localBrokerId = brokerId,
() => 1L,
mockTime,
@ -457,8 +456,8 @@ class PartitionLockTest extends Logging {
keepPartitionMetadataFile = true) {
override def appendAsLeader(records: MemoryRecords, leaderEpoch: Int, origin: AppendOrigin,
interBrokerProtocolVersion: MetadataVersion, requestLocal: RequestLocal, verificationGuard: VerificationGuard): LogAppendInfo = {
val appendInfo = super.appendAsLeader(records, leaderEpoch, origin, interBrokerProtocolVersion, requestLocal, verificationGuard)
requestLocal: RequestLocal, verificationGuard: VerificationGuard): LogAppendInfo = {
val appendInfo = super.appendAsLeader(records, leaderEpoch, origin, requestLocal, verificationGuard)
appendSemaphore.acquire()
appendInfo
}

View File

@ -431,7 +431,6 @@ class PartitionTest extends AbstractPartitionTest {
partition = new Partition(
topicPartition,
replicaLagTimeMaxMs = ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_DEFAULT,
interBrokerProtocolVersion = MetadataVersion.latestTesting,
localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId),
time,
@ -1270,7 +1269,6 @@ class PartitionTest extends AbstractPartitionTest {
configRepository.setTopicConfig(topicPartition.topic, TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "2")
partition = new Partition(topicPartition,
replicaLagTimeMaxMs = ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_DEFAULT,
interBrokerProtocolVersion = interBrokerProtocolVersion,
localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId),
time,
@ -1361,7 +1359,6 @@ class PartitionTest extends AbstractPartitionTest {
val mockMetadataCache: KRaftMetadataCache = mock(classOf[KRaftMetadataCache])
val partition = spy(new Partition(topicPartition,
replicaLagTimeMaxMs = ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_DEFAULT,
interBrokerProtocolVersion = interBrokerProtocolVersion,
localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId),
time,
@ -1593,7 +1590,6 @@ class PartitionTest extends AbstractPartitionTest {
val partition = new Partition(
topicPartition,
replicaLagTimeMaxMs = ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_DEFAULT,
interBrokerProtocolVersion = MetadataVersion.latestTesting,
localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId),
time,
@ -1701,7 +1697,6 @@ class PartitionTest extends AbstractPartitionTest {
val partition = new Partition(
topicPartition,
replicaLagTimeMaxMs = ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_DEFAULT,
interBrokerProtocolVersion = MetadataVersion.latestTesting,
localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId),
time,
@ -1808,7 +1803,6 @@ class PartitionTest extends AbstractPartitionTest {
val partition = new Partition(
topicPartition,
replicaLagTimeMaxMs = ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_DEFAULT,
interBrokerProtocolVersion = MetadataVersion.latestTesting,
localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId),
time,
@ -1900,7 +1894,6 @@ class PartitionTest extends AbstractPartitionTest {
val partition = new Partition(
topicPartition,
replicaLagTimeMaxMs = ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_DEFAULT,
interBrokerProtocolVersion = MetadataVersion.latestTesting,
localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId),
time,
@ -1966,7 +1959,6 @@ class PartitionTest extends AbstractPartitionTest {
val partition = new Partition(
topicPartition,
replicaLagTimeMaxMs = ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_DEFAULT,
interBrokerProtocolVersion = MetadataVersion.latestTesting,
localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId),
time,
@ -2122,7 +2114,6 @@ class PartitionTest extends AbstractPartitionTest {
val partition = new Partition(
topicPartition,
replicaLagTimeMaxMs = ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_DEFAULT,
interBrokerProtocolVersion = MetadataVersion.latestTesting,
localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId),
time,
@ -2205,7 +2196,6 @@ class PartitionTest extends AbstractPartitionTest {
val partition = new Partition(
topicPartition,
replicaLagTimeMaxMs = ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_DEFAULT,
interBrokerProtocolVersion = MetadataVersion.IBP_3_7_IV2,
localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId),
time,
@ -2578,12 +2568,11 @@ class PartitionTest extends AbstractPartitionTest {
time = time,
brokerId = brokerId,
brokerEpochSupplier = () => 0,
metadataVersionSupplier = () => MetadataVersion.IBP_3_0_IV0
metadataVersionSupplier = () => MetadataVersion.IBP_3_0_IV1
)
partition = new Partition(topicPartition,
replicaLagTimeMaxMs = ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_DEFAULT,
interBrokerProtocolVersion = interBrokerProtocolVersion,
localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId),
time,
@ -2730,7 +2719,6 @@ class PartitionTest extends AbstractPartitionTest {
// Create new Partition object for same topicPartition
val partition2 = new Partition(topicPartition,
replicaLagTimeMaxMs = ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_DEFAULT,
interBrokerProtocolVersion = MetadataVersion.latestTesting,
localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId),
time,
@ -2775,7 +2763,6 @@ class PartitionTest extends AbstractPartitionTest {
// Create new Partition object for same topicPartition
val partition2 = new Partition(topicPartition,
replicaLagTimeMaxMs = ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_DEFAULT,
interBrokerProtocolVersion = MetadataVersion.latestTesting,
localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId),
time,
@ -2858,7 +2845,7 @@ class PartitionTest extends AbstractPartitionTest {
def testUpdateAssignmentAndIsr(): Unit = {
val topicPartition = new TopicPartition("test", 1)
val partition = new Partition(
topicPartition, 1000, MetadataVersion.latestTesting, 0, () => defaultBrokerEpoch(0),
topicPartition, 1000, 0, () => defaultBrokerEpoch(0),
Time.SYSTEM, mock(classOf[AlterPartitionListener]), mock(classOf[DelayedOperations]),
mock(classOf[KRaftMetadataCache]), mock(classOf[LogManager]), mock(classOf[AlterPartitionManager]))
@ -2933,7 +2920,6 @@ class PartitionTest extends AbstractPartitionTest {
val spyLogManager = spy(logManager)
val partition = new Partition(topicPartition,
replicaLagTimeMaxMs = ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_DEFAULT,
interBrokerProtocolVersion = MetadataVersion.latestTesting,
localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId),
time,
@ -2972,7 +2958,6 @@ class PartitionTest extends AbstractPartitionTest {
val partition = new Partition(topicPartition,
replicaLagTimeMaxMs = ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_DEFAULT,
interBrokerProtocolVersion = MetadataVersion.latestTesting,
localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId),
time,
@ -3014,7 +2999,6 @@ class PartitionTest extends AbstractPartitionTest {
val partition = new Partition(topicPartition,
replicaLagTimeMaxMs = ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_DEFAULT,
interBrokerProtocolVersion = MetadataVersion.latestTesting,
localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId),
time,
@ -3766,7 +3750,6 @@ class PartitionTest extends AbstractPartitionTest {
val spyLogManager = spy(logManager)
val partition = new Partition(topicPartition,
replicaLagTimeMaxMs = ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_DEFAULT,
interBrokerProtocolVersion = MetadataVersion.latestTesting,
localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId),
time,
@ -3812,7 +3795,6 @@ class PartitionTest extends AbstractPartitionTest {
val spyLogManager = spy(logManager)
val partition = new Partition(topicPartition,
replicaLagTimeMaxMs = ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_DEFAULT,
interBrokerProtocolVersion = MetadataVersion.latestTesting,
localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId),
time,
@ -3858,7 +3840,6 @@ class PartitionTest extends AbstractPartitionTest {
val spyLogManager = spy(logManager)
val partition = new Partition(topicPartition,
replicaLagTimeMaxMs = ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_DEFAULT,
interBrokerProtocolVersion = MetadataVersion.latestTesting,
localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId),
time,
@ -3904,7 +3885,6 @@ class PartitionTest extends AbstractPartitionTest {
val spyLogManager = spy(logManager)
val partition = new Partition(topicPartition,
replicaLagTimeMaxMs = ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_DEFAULT,
interBrokerProtocolVersion = MetadataVersion.latestTesting,
localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId),
time,
@ -3950,7 +3930,6 @@ class PartitionTest extends AbstractPartitionTest {
val spyLogManager = spy(logManager)
val partition = new Partition(topicPartition,
replicaLagTimeMaxMs = ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_DEFAULT,
interBrokerProtocolVersion = MetadataVersion.latestTesting,
localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId),
time,
@ -3997,7 +3976,6 @@ class PartitionTest extends AbstractPartitionTest {
val spyLogManager = spy(logManager)
val partition = new Partition(topicPartition,
replicaLagTimeMaxMs = ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_DEFAULT,
interBrokerProtocolVersion = MetadataVersion.latestTesting,
localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId),
time,
@ -4052,7 +4030,6 @@ class PartitionTest extends AbstractPartitionTest {
val spyLogManager = spy(logManager)
val partition = new Partition(topicPartition,
replicaLagTimeMaxMs = ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_DEFAULT,
interBrokerProtocolVersion = MetadataVersion.latestTesting,
localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId),
time,

View File

@ -45,7 +45,6 @@ import org.apache.kafka.common.utils.Utils
import org.apache.kafka.coordinator.group.{GroupCoordinatorConfig, OffsetAndMetadata, OffsetConfig}
import org.apache.kafka.coordinator.group.generated.{CoordinatorRecordType, GroupMetadataValue, OffsetCommitKey, OffsetCommitValue, GroupMetadataKey => GroupMetadataKeyData}
import org.apache.kafka.server.common.{MetadataVersion, RequestLocal}
import org.apache.kafka.server.common.MetadataVersion._
import org.apache.kafka.server.metrics.KafkaYammerMetrics
import org.apache.kafka.server.storage.log.FetchIsolation
import org.apache.kafka.server.util.{KafkaScheduler, MockTime}
@ -1059,22 +1058,20 @@ class GroupMetadataManagerTest {
}
@Test
def testCurrentStateTimestampForAllGroupMetadataVersions(): Unit = {
def testCurrentStateTimestampForAllVersions(): Unit = {
val generation = 1
val protocol = "range"
val memberId = "memberId"
for (metadataVersion <- MetadataVersion.VERSIONS) {
val groupMetadataRecord = buildStableGroupRecordWithMember(generation, protocolType, protocol, memberId, metadataVersion = metadataVersion)
for (version <- 0 to 3) {
val groupMetadataRecord = buildStableGroupRecordWithMember(generation, protocolType, protocol, memberId,
groupMetadataValueVersion = version.toShort)
val deserializedGroupMetadata = GroupMetadataManager.readGroupMessageValue(groupId, groupMetadataRecord.value(), time)
// GROUP_METADATA_VALUE_SCHEMA_V2 or higher should correctly set the currentStateTimestamp
if (metadataVersion.isAtLeast(IBP_2_1_IV0))
assertEquals(Some(time.milliseconds()), deserializedGroupMetadata.currentStateTimestamp,
s"the metadataVersion $metadataVersion doesn't set the currentStateTimestamp correctly.")
if (version >= 2)
assertEquals(Some(time.milliseconds()), deserializedGroupMetadata.currentStateTimestamp)
else
assertTrue(deserializedGroupMetadata.currentStateTimestamp.isEmpty,
s"the metadataVersion $metadataVersion should not set the currentStateTimestamp.")
assertTrue(deserializedGroupMetadata.currentStateTimestamp.isEmpty)
}
}
@ -1083,10 +1080,10 @@ class GroupMetadataManagerTest {
val generation = 1
val protocol = "range"
val memberId = "memberId"
val oldMetadataVersions = Array(IBP_0_9_0, IBP_0_10_1_IV0, IBP_2_1_IV0)
for (metadataVersion <- oldMetadataVersions) {
val groupMetadataRecord = buildStableGroupRecordWithMember(generation, protocolType, protocol, memberId, metadataVersion = metadataVersion)
for (version <- 0 to 2) {
val groupMetadataRecord = buildStableGroupRecordWithMember(generation, protocolType, protocol, memberId,
groupMetadataValueVersion = version.toShort)
val deserializedGroupMetadata = GroupMetadataManager.readGroupMessageValue(groupId, groupMetadataRecord.value(), time)
assertEquals(groupId, deserializedGroupMetadata.groupId)
@ -2477,10 +2474,11 @@ class GroupMetadataManagerTest {
new TopicPartition("bar", 0) -> 8992L
)
val metadataVersion = IBP_1_1_IV0
val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets, metadataVersion = metadataVersion, retentionTimeOpt = Some(100))
val offsetCommitValueVersion = 1.toShort
val groupMetadataValueVersion = 1.toShort
val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets, offsetCommitValueVersion = offsetCommitValueVersion, retentionTimeOpt = Some(100))
val memberId = "98098230493"
val groupMetadataRecord = buildStableGroupRecordWithMember(generation, protocolType, protocol, memberId, metadataVersion = metadataVersion)
val groupMetadataRecord = buildStableGroupRecordWithMember(generation, protocolType, protocol, memberId, groupMetadataValueVersion = groupMetadataValueVersion)
val records = MemoryRecords.withRecords(startOffset, Compression.NONE,
(offsetCommitRecords ++ Seq(groupMetadataRecord)).toArray: _*)
@ -2551,34 +2549,17 @@ class GroupMetadataManagerTest {
time.milliseconds(),
noExpiration)
def verifySerde(metadataVersion: MetadataVersion, expectedOffsetCommitValueVersion: Int): Unit = {
val bytes = GroupMetadataManager.offsetCommitValue(offsetAndMetadata, metadataVersion)
val buffer = ByteBuffer.wrap(bytes)
assertEquals(expectedOffsetCommitValueVersion, buffer.getShort(0).toInt)
val deserializedOffsetAndMetadata = GroupMetadataManager.readOffsetMessageValue(buffer)
assertEquals(offsetAndMetadata.committedOffset, deserializedOffsetAndMetadata.committedOffset)
assertEquals(offsetAndMetadata.metadata, deserializedOffsetAndMetadata.metadata)
assertEquals(offsetAndMetadata.commitTimestampMs, deserializedOffsetAndMetadata.commitTimestampMs)
// Serialization drops the leader epoch silently if an older inter-broker protocol is in use
val expectedLeaderEpoch = if (expectedOffsetCommitValueVersion >= 3)
offsetAndMetadata.leaderEpoch
else
noLeader
assertEquals(expectedLeaderEpoch, deserializedOffsetAndMetadata.leaderEpoch)
}
for (version <- MetadataVersion.VERSIONS) {
val expectedSchemaVersion = version match {
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)
}
val bytes = GroupMetadataManager.offsetCommitValue(offsetAndMetadata)
val buffer = ByteBuffer.wrap(bytes)
val expectedOffsetCommitValueVersion = 3
assertEquals(expectedOffsetCommitValueVersion, buffer.getShort(0).toInt)
val deserializedOffsetAndMetadata = GroupMetadataManager.readOffsetMessageValue(buffer)
assertEquals(offsetAndMetadata.committedOffset, deserializedOffsetAndMetadata.committedOffset)
assertEquals(offsetAndMetadata.metadata, deserializedOffsetAndMetadata.metadata)
assertEquals(offsetAndMetadata.commitTimestampMs, deserializedOffsetAndMetadata.commitTimestampMs)
val expectedLeaderEpoch = offsetAndMetadata.leaderEpoch
assertEquals(expectedLeaderEpoch, deserializedOffsetAndMetadata.leaderEpoch)
assertEquals(offsetAndMetadata, deserializedOffsetAndMetadata)
}
@Test
@ -2593,45 +2574,12 @@ class GroupMetadataManagerTest {
time.milliseconds(),
OptionalLong.of(time.milliseconds() + 1000))
def verifySerde(metadataVersion: MetadataVersion): Unit = {
val bytes = GroupMetadataManager.offsetCommitValue(offsetAndMetadata, metadataVersion)
val buffer = ByteBuffer.wrap(bytes)
assertEquals(1, buffer.getShort(0).toInt)
val bytes = GroupMetadataManager.offsetCommitValue(offsetAndMetadata)
val buffer = ByteBuffer.wrap(bytes)
assertEquals(1, buffer.getShort(0).toInt)
val deserializedOffsetAndMetadata = GroupMetadataManager.readOffsetMessageValue(buffer)
assertEquals(offsetAndMetadata, deserializedOffsetAndMetadata)
}
for (version <- MetadataVersion.VERSIONS)
verifySerde(version)
}
@Test
def testSerdeOffsetCommitValueWithNoneExpireTimestamp(): Unit = {
val offsetAndMetadata = new OffsetAndMetadata(
537L,
noLeader,
"metadata",
time.milliseconds(),
noExpiration)
def verifySerde(metadataVersion: MetadataVersion): Unit = {
val bytes = GroupMetadataManager.offsetCommitValue(offsetAndMetadata, metadataVersion)
val buffer = ByteBuffer.wrap(bytes)
val version = buffer.getShort(0).toInt
if (metadataVersion.isLessThan(IBP_2_1_IV0))
assertEquals(1, version)
else if (metadataVersion.isLessThan(IBP_2_1_IV1))
assertEquals(2, version)
else
assertEquals(3, version)
val deserializedOffsetAndMetadata = GroupMetadataManager.readOffsetMessageValue(buffer)
assertEquals(offsetAndMetadata, deserializedOffsetAndMetadata)
}
for (version <- MetadataVersion.VERSIONS)
verifySerde(version)
val deserializedOffsetAndMetadata = GroupMetadataManager.readOffsetMessageValue(buffer)
assertEquals(offsetAndMetadata, deserializedOffsetAndMetadata)
}
@Test
@ -2942,20 +2890,20 @@ class GroupMetadataManagerTest {
protocol: String,
memberId: String,
assignmentBytes: Array[Byte] = Array.emptyByteArray,
metadataVersion: MetadataVersion = MetadataVersion.latestTesting): SimpleRecord = {
groupMetadataValueVersion: Short = 3): 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 (metadataVersion.isAtLeast(IBP_2_1_IV0)) Some(time.milliseconds()) else None, Seq(member), time)
if (groupMetadataValueVersion >= 2.toShort) Some(time.milliseconds()) else None, Seq(member), time)
val groupMetadataKey = GroupMetadataManager.groupMetadataKey(groupId)
val groupMetadataValue = GroupMetadataManager.groupMetadataValue(group, Map(memberId -> assignmentBytes), metadataVersion)
val groupMetadataValue = GroupMetadataManager.groupMetadataValue(group, Map(memberId -> assignmentBytes), groupMetadataValueVersion)
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, MetadataVersion.latestTesting)
val groupMetadataValue = GroupMetadataManager.groupMetadataValue(group, Map.empty)
new SimpleRecord(groupMetadataKey, groupMetadataValue)
}
@ -2999,7 +2947,7 @@ class GroupMetadataManagerTest {
private def createCommittedOffsetRecords(committedOffsets: Map[TopicPartition, Long],
groupId: String = groupId,
metadataVersion: MetadataVersion = MetadataVersion.latestTesting,
offsetCommitValueVersion: Short = 3,
retentionTimeOpt: Option[Long] = None): Seq[SimpleRecord] = {
committedOffsets.map { case (topicPartition, offset) =>
val commitTimestamp = time.milliseconds()
@ -3011,7 +2959,7 @@ class GroupMetadataManagerTest {
new OffsetAndMetadata(offset, noLeader, "", commitTimestamp, noExpiration)
}
val offsetCommitKey = GroupMetadataManager.offsetCommitKey(groupId, topicPartition)
val offsetCommitValue = GroupMetadataManager.offsetCommitValue(offsetAndMetadata, metadataVersion)
val offsetCommitValue = GroupMetadataManager.offsetCommitValue(offsetAndMetadata, offsetCommitValueVersion)
new SimpleRecord(offsetCommitKey, offsetCommitValue)
}.toSeq
}

View File

@ -298,10 +298,10 @@ class TransactionMarkerChannelManagerTest {
assertEquals(1, channelManager.queueForBroker(broker2.id).get.totalNumMarkers(txnTopicPartition1))
assertEquals(0, channelManager.queueForBroker(broker2.id).get.totalNumMarkers(txnTopicPartition2))
val expectedBroker1Request = new WriteTxnMarkersRequest.Builder(ApiKeys.WRITE_TXN_MARKERS.latestVersion(),
val expectedBroker1Request = new WriteTxnMarkersRequest.Builder(
asList(new WriteTxnMarkersRequest.TxnMarkerEntry(producerId1, producerEpoch, coordinatorEpoch, txnResult, asList(partition1)),
new WriteTxnMarkersRequest.TxnMarkerEntry(producerId2, producerEpoch, coordinatorEpoch, txnResult, asList(partition1)))).build()
val expectedBroker2Request = new WriteTxnMarkersRequest.Builder(ApiKeys.WRITE_TXN_MARKERS.latestVersion(),
val expectedBroker2Request = new WriteTxnMarkersRequest.Builder(
asList(new WriteTxnMarkersRequest.TxnMarkerEntry(producerId1, producerEpoch, coordinatorEpoch, txnResult, asList(partition2)))).build()
val requests: Map[Node, WriteTxnMarkersRequest] = channelManager.generateRequests().asScala.map { handler =>
@ -368,10 +368,10 @@ class TransactionMarkerChannelManagerTest {
assertEquals(1, channelManager.queueForUnknownBroker.totalNumMarkers(txnTopicPartition1))
assertEquals(1, channelManager.queueForUnknownBroker.totalNumMarkers(txnTopicPartition2))
val expectedBroker1Request = new WriteTxnMarkersRequest.Builder(ApiKeys.WRITE_TXN_MARKERS.latestVersion(),
val expectedBroker1Request = new WriteTxnMarkersRequest.Builder(
asList(new WriteTxnMarkersRequest.TxnMarkerEntry(producerId1, producerEpoch, coordinatorEpoch, txnResult, asList(partition1)),
new WriteTxnMarkersRequest.TxnMarkerEntry(producerId2, producerEpoch, coordinatorEpoch, txnResult, asList(partition1)))).build()
val expectedBroker2Request = new WriteTxnMarkersRequest.Builder(ApiKeys.WRITE_TXN_MARKERS.latestVersion(),
val expectedBroker2Request = new WriteTxnMarkersRequest.Builder(
asList(new WriteTxnMarkersRequest.TxnMarkerEntry(producerId1, producerEpoch, coordinatorEpoch, txnResult, asList(partition2)))).build()
val firstDrainedRequests: Map[Node, WriteTxnMarkersRequest] = channelManager.generateRequests().asScala.map { handler =>

View File

@ -24,10 +24,9 @@ import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.errors.KafkaStorageException
import org.apache.kafka.common.record.{ControlRecordType, DefaultRecordBatch, MemoryRecords, RecordBatch, RecordVersion, SimpleRecord, TimestampType}
import org.apache.kafka.common.record.{ControlRecordType, DefaultRecordBatch, MemoryRecords, RecordBatch, SimpleRecord, TimestampType}
import org.apache.kafka.common.utils.{Time, Utils}
import org.apache.kafka.coordinator.transaction.TransactionLogConfig
import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.server.util.{MockTime, Scheduler}
import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache
import org.apache.kafka.storage.internals.log.{AbortedTxn, CleanerConfig, EpochEntry, LocalLog, LogConfig, LogDirFailureChannel, LogFileUtils, LogLoader, LogOffsetMetadata, LogSegment, LogSegments, LogStartOffsetIncrementReason, OffsetIndex, ProducerStateManager, ProducerStateManagerConfig, SnapshotFile}
@ -128,7 +127,7 @@ class LogLoaderTest {
brokerTopicStats = new BrokerTopicStats(),
logDirFailureChannel = logDirFailureChannel,
time = time,
keepPartitionMetadataFile = config.usesTopicId,
keepPartitionMetadataFile = true,
remoteStorageSystemEnable = config.remoteLogManagerConfig.isRemoteStorageSystemEnabled(),
initialTaskDelayMs = config.logInitialTaskDelayMs) {
@ -246,70 +245,7 @@ class LogLoaderTest {
}
@Test
def testProducerSnapshotsRecoveryAfterUncleanShutdownV1(): Unit = {
testProducerSnapshotsRecoveryAfterUncleanShutdown(MetadataVersion.minSupportedFor(RecordVersion.V1).version)
}
@Test
def testProducerSnapshotsRecoveryAfterUncleanShutdownCurrentMessageFormat(): Unit = {
testProducerSnapshotsRecoveryAfterUncleanShutdown(MetadataVersion.latestTesting.version)
}
private def createLog(dir: File,
config: LogConfig,
brokerTopicStats: BrokerTopicStats = brokerTopicStats,
logStartOffset: Long = 0L,
recoveryPoint: Long = 0L,
scheduler: Scheduler = mockTime.scheduler,
time: Time = mockTime,
maxTransactionTimeoutMs: Int = maxTransactionTimeoutMs,
maxProducerIdExpirationMs: Int = producerStateManagerConfig.producerIdExpirationMs,
producerIdExpirationCheckIntervalMs: Int = producerIdExpirationCheckIntervalMs,
lastShutdownClean: Boolean = true): UnifiedLog = {
val log = LogTestUtils.createLog(dir, config, brokerTopicStats, scheduler, time, logStartOffset, recoveryPoint,
maxTransactionTimeoutMs, new ProducerStateManagerConfig(maxProducerIdExpirationMs, false), producerIdExpirationCheckIntervalMs, lastShutdownClean)
logsToClose = logsToClose :+ log
log
}
private def createLogWithOffsetOverflow(logConfig: LogConfig): (UnifiedLog, LogSegment) = {
LogTestUtils.initializeLogDirWithOverflowedSegment(logDir)
val log = createLog(logDir, logConfig, recoveryPoint = Long.MaxValue)
val segmentWithOverflow = LogTestUtils.firstOverflowSegment(log).getOrElse {
throw new AssertionError("Failed to create log with a segment which has overflowed offsets")
}
(log, segmentWithOverflow)
}
private def recoverAndCheck(config: LogConfig, expectedKeys: Iterable[Long]): UnifiedLog = {
// method is called only in case of recovery from hard reset
val recoveredLog = LogTestUtils.recoverAndCheck(logDir, config, expectedKeys, brokerTopicStats, mockTime, mockTime.scheduler)
logsToClose = logsToClose :+ recoveredLog
recoveredLog
}
/**
* Wrap a single record log buffer with leader epoch.
*/
private def singletonRecordsWithLeaderEpoch(value: Array[Byte],
key: Array[Byte] = null,
leaderEpoch: Int,
offset: Long,
codec: Compression = Compression.NONE,
timestamp: Long = RecordBatch.NO_TIMESTAMP,
magicValue: Byte = RecordBatch.CURRENT_MAGIC_VALUE): MemoryRecords = {
val records = Seq(new SimpleRecord(timestamp, key, value))
val buf = ByteBuffer.allocate(DefaultRecordBatch.sizeInBytes(records.asJava))
val builder = MemoryRecords.builder(buf, magicValue, codec, TimestampType.CREATE_TIME, offset,
mockTime.milliseconds, leaderEpoch)
records.foreach(builder.append)
builder.build()
}
private def testProducerSnapshotsRecoveryAfterUncleanShutdown(messageFormatVersion: String): Unit = {
def testProducerSnapshotsRecoveryAfterUncleanShutdown(): Unit = {
val logProps = new Properties()
logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, "640")
val logConfig = new LogConfig(logProps)
@ -413,6 +349,60 @@ class LogLoaderTest {
log.close()
}
private def createLog(dir: File,
config: LogConfig,
brokerTopicStats: BrokerTopicStats = brokerTopicStats,
logStartOffset: Long = 0L,
recoveryPoint: Long = 0L,
scheduler: Scheduler = mockTime.scheduler,
time: Time = mockTime,
maxTransactionTimeoutMs: Int = maxTransactionTimeoutMs,
maxProducerIdExpirationMs: Int = producerStateManagerConfig.producerIdExpirationMs,
producerIdExpirationCheckIntervalMs: Int = producerIdExpirationCheckIntervalMs,
lastShutdownClean: Boolean = true): UnifiedLog = {
val log = LogTestUtils.createLog(dir, config, brokerTopicStats, scheduler, time, logStartOffset, recoveryPoint,
maxTransactionTimeoutMs, new ProducerStateManagerConfig(maxProducerIdExpirationMs, false), producerIdExpirationCheckIntervalMs, lastShutdownClean)
logsToClose = logsToClose :+ log
log
}
private def createLogWithOffsetOverflow(logConfig: LogConfig): (UnifiedLog, LogSegment) = {
LogTestUtils.initializeLogDirWithOverflowedSegment(logDir)
val log = createLog(logDir, logConfig, recoveryPoint = Long.MaxValue)
val segmentWithOverflow = LogTestUtils.firstOverflowSegment(log).getOrElse {
throw new AssertionError("Failed to create log with a segment which has overflowed offsets")
}
(log, segmentWithOverflow)
}
private def recoverAndCheck(config: LogConfig, expectedKeys: Iterable[Long]): UnifiedLog = {
// method is called only in case of recovery from hard reset
val recoveredLog = LogTestUtils.recoverAndCheck(logDir, config, expectedKeys, brokerTopicStats, mockTime, mockTime.scheduler)
logsToClose = logsToClose :+ recoveredLog
recoveredLog
}
/**
* Wrap a single record log buffer with leader epoch.
*/
private def singletonRecordsWithLeaderEpoch(value: Array[Byte],
key: Array[Byte] = null,
leaderEpoch: Int,
offset: Long,
codec: Compression = Compression.NONE,
timestamp: Long = RecordBatch.NO_TIMESTAMP,
magicValue: Byte = RecordBatch.CURRENT_MAGIC_VALUE): MemoryRecords = {
val records = Seq(new SimpleRecord(timestamp, key, value))
val buf = ByteBuffer.allocate(DefaultRecordBatch.sizeInBytes(records.asJava))
val builder = MemoryRecords.builder(buf, magicValue, codec, TimestampType.CREATE_TIME, offset,
mockTime.milliseconds, leaderEpoch)
records.foreach(builder.append)
builder.build()
}
@Test
def testSkipLoadingIfEmptyProducerStateBeforeTruncation(): Unit = {
val maxTransactionTimeoutMs = 60000

View File

@ -23,7 +23,6 @@ import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersion
import org.apache.kafka.common.message.ApiMessageType
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol.ApiKeys
import org.apache.kafka.common.record.RecordVersion
import org.apache.kafka.common.requests.{ApiVersionsRequest, ApiVersionsResponse, RequestUtils}
import org.apache.kafka.common.utils.Utils
import org.apache.kafka.server.common.{EligibleLeaderReplicasVersion, GroupVersion, MetadataVersion, TransactionVersion}
@ -95,7 +94,6 @@ abstract class AbstractApiVersionsRequestTest(cluster: ClusterInstance) {
} else {
ApiVersionsResponse.intersectForwardableApis(
ApiMessageType.ListenerType.BROKER,
RecordVersion.current,
NodeApiVersions.create(ApiKeys.controllerApis().asScala.map(ApiVersionsResponse.toApiVersion).asJava).allSupportedApiVersions(),
enableUnstableLastVersion,
clientTelemetryEnabled

View File

@ -343,8 +343,6 @@ class AbstractFetcherManagerTest {
override protected def logEndOffset(topicPartition: TopicPartition): Long = 1
override protected def endOffsetForEpoch(topicPartition: TopicPartition, epoch: Int): Option[OffsetAndEpoch] = Some(new OffsetAndEpoch(1, 0))
override protected val isOffsetForLeaderEpochSupported: Boolean = false
}
}

View File

@ -390,43 +390,6 @@ class AbstractFetcherThreadTest {
assertEquals(leaderState.highWatermark, replicaState.highWatermark)
}
@Test
def testTruncateToHighWatermarkIfLeaderEpochRequestNotSupported(): Unit = {
val highWatermark = 2L
val partition = new TopicPartition("topic", 0)
val mockLeaderEndPoint = new MockLeaderEndPoint(truncateOnFetch = truncateOnFetch, version = version) {
override def fetchEpochEndOffsets(partitions: Map[TopicPartition, EpochData]): Map[TopicPartition, EpochEndOffset] =
throw new UnsupportedOperationException
override val isTruncationOnFetchSupported: Boolean = false
}
val mockTierStateMachine = new MockTierStateMachine(mockLeaderEndPoint)
val fetcher = new MockFetcherThread(mockLeaderEndPoint, mockTierStateMachine) {
override def truncate(topicPartition: TopicPartition, truncationState: OffsetTruncationState): Unit = {
assertEquals(highWatermark, truncationState.offset)
assertTrue(truncationState.truncationCompleted)
super.truncate(topicPartition, truncationState)
}
override protected val isOffsetForLeaderEpochSupported: Boolean = false
}
val replicaLog = Seq(
mkBatch(baseOffset = 0, leaderEpoch = 0, new SimpleRecord("a".getBytes)),
mkBatch(baseOffset = 1, leaderEpoch = 2, new SimpleRecord("b".getBytes)),
mkBatch(baseOffset = 2, leaderEpoch = 4, new SimpleRecord("c".getBytes)))
val replicaState = PartitionState(replicaLog, leaderEpoch = 5, highWatermark)
fetcher.setReplicaState(partition, replicaState)
fetcher.addPartitions(Map(partition -> initialFetchState(topicIds.get(partition.topic), highWatermark, leaderEpoch = 5)))
fetcher.mockLeader.setReplicaPartitionStateCallback(fetcher.replicaPartitionState)
fetcher.doWork()
assertEquals(highWatermark, replicaState.logEndOffset)
assertEquals(highWatermark, fetcher.fetchState(partition).get.fetchOffset)
assertTrue(fetcher.fetchState(partition).get.isReadyForFetch)
}
@Test
def testTruncateToHighWatermarkIfLeaderEpochInfoNotAvailable(): Unit = {
val highWatermark = 2L

View File

@ -32,7 +32,7 @@ import org.apache.kafka.common.requests.RequestHeader
import org.apache.kafka.common.requests.{AbstractRequest, AlterPartitionRequest, AlterPartitionResponse}
import org.apache.kafka.metadata.{LeaderAndIsr, LeaderRecoveryState}
import org.apache.kafka.server.common.{ControllerRequestCompletionHandler, MetadataVersion, NodeToControllerChannelManager}
import org.apache.kafka.server.common.MetadataVersion.{IBP_2_7_IV2, IBP_3_2_IV0, IBP_3_5_IV1}
import org.apache.kafka.server.common.MetadataVersion.{IBP_3_0_IV1, IBP_3_2_IV0, IBP_3_5_IV1}
import org.apache.kafka.server.util.{MockScheduler, MockTime}
import org.apache.kafka.test.TestUtils.assertFutureThrows
import org.junit.jupiter.api.Assertions._
@ -99,23 +99,15 @@ class AlterPartitionManagerTest {
.setTopicName(topic)
.setTopicId(topicId)
if (metadataVersion.isTopicIdsSupported) {
val newIsrWithBrokerEpoch = new ListBuffer[BrokerState]()
newIsrWithBrokerEpoch.append(new BrokerState().setBrokerId(1).setBrokerEpoch(101))
newIsrWithBrokerEpoch.append(new BrokerState().setBrokerId(2).setBrokerEpoch(102))
newIsrWithBrokerEpoch.append(new BrokerState().setBrokerId(3).setBrokerEpoch(103))
topicData.partitions.add(new AlterPartitionRequestData.PartitionData()
.setPartitionIndex(0)
.setLeaderEpoch(1)
.setPartitionEpoch(10)
.setNewIsrWithEpochs(newIsrWithBrokerEpoch.toList.asJava))
} else {
topicData.partitions.add(new AlterPartitionRequestData.PartitionData()
.setPartitionIndex(0)
.setLeaderEpoch(1)
.setPartitionEpoch(10)
.setNewIsr(List(1, 2, 3).map(Integer.valueOf).asJava))
}
val newIsrWithBrokerEpoch = new ListBuffer[BrokerState]()
newIsrWithBrokerEpoch.append(new BrokerState().setBrokerId(1).setBrokerEpoch(101))
newIsrWithBrokerEpoch.append(new BrokerState().setBrokerId(2).setBrokerEpoch(102))
newIsrWithBrokerEpoch.append(new BrokerState().setBrokerId(3).setBrokerEpoch(103))
topicData.partitions.add(new AlterPartitionRequestData.PartitionData()
.setPartitionIndex(0)
.setLeaderEpoch(1)
.setPartitionEpoch(10)
.setNewIsrWithEpochs(newIsrWithBrokerEpoch.toList.asJava))
expectedAlterPartitionData.topics.add(topicData)
@ -148,7 +140,6 @@ class AlterPartitionManagerTest {
@ParameterizedTest
@MethodSource(Array("provideMetadataVersions"))
def testOverwriteWithinBatch(metadataVersion: MetadataVersion): Unit = {
val canUseTopicIds = metadataVersion.isAtLeast(MetadataVersion.IBP_2_8_IV0)
val capture: ArgumentCaptor[AbstractRequest.Builder[AlterPartitionRequest]] = ArgumentCaptor.forClass(classOf[AbstractRequest.Builder[AlterPartitionRequest]])
val callbackCapture: ArgumentCaptor[ControllerRequestCompletionHandler] = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler])
@ -168,7 +159,7 @@ class AlterPartitionManagerTest {
val alterPartitionResp = partitionResponse()
val resp = makeClientResponse(
response = alterPartitionResp,
version = if (canUseTopicIds) ApiKeys.ALTER_PARTITION.latestVersion else 1
version = ApiKeys.ALTER_PARTITION.latestVersion
)
verify(brokerToController).sendRequest(capture.capture(), callbackCapture.capture())
callbackCapture.getValue.onComplete(resp)
@ -422,11 +413,7 @@ class AlterPartitionManagerTest {
@ParameterizedTest
@MethodSource(Array("provideMetadataVersions"))
def testPartitionMissingInResponse(metadataVersion: MetadataVersion): Unit = {
val expectedVersion = if (metadataVersion.isTopicIdsSupported) {
ApiKeys.ALTER_PARTITION.latestVersion
} else {
1.toShort
}
val expectedVersion = ApiKeys.ALTER_PARTITION.latestVersion
val leaderAndIsr = new LeaderAndIsr(1, 1, List(1, 2, 3).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, 10)
val controlledEpoch = 0
val brokerEpoch = 2
@ -486,74 +473,6 @@ class AlterPartitionManagerTest {
assertFutureThrows(future2, classOf[UnknownServerException])
}
@ParameterizedTest
@MethodSource(Array("provideMetadataVersions"))
def testPartialTopicIds(metadataVersion: MetadataVersion): Unit = {
val canUseTopicIds = metadataVersion.isAtLeast(MetadataVersion.IBP_2_8_IV0)
val foo = new TopicIdPartition(Uuid.ZERO_UUID, 0, "foo")
val bar = new TopicIdPartition(Uuid.randomUuid(), 0, "bar")
val zar = new TopicIdPartition(Uuid.randomUuid(), 0, "zar")
val leaderAndIsr = new LeaderAndIsr(1, 1, List(1, 2, 3).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, 10)
val controlledEpoch = 0
val brokerEpoch = 2
val scheduler = new MockScheduler(time)
val brokerToController = Mockito.mock(classOf[NodeToControllerChannelManager])
val alterPartitionManager = new DefaultAlterPartitionManager(
brokerToController,
scheduler,
time,
brokerId,
() => brokerEpoch,
() => metadataVersion
)
alterPartitionManager.start()
// Submits an alter isr update with zar, which has a topic id.
val future1 = alterPartitionManager.submit(zar, leaderAndIsr, controlledEpoch)
// The latest version is expected if all the submitted partitions
// have topic ids and IBP >= 2.8; version 1 should be used otherwise.
val callback1 = verifySendRequest(brokerToController, alterPartitionRequestMatcher(
expectedTopicPartitions = Set(zar),
expectedVersion = if (canUseTopicIds) ApiKeys.ALTER_PARTITION.latestVersion else 1
))
// Submits two additional alter isr changes with foo and bar while the previous one
// is still inflight. foo has no topic id, bar has one.
val future2 = alterPartitionManager.submit(foo, leaderAndIsr, controlledEpoch)
val future3 = alterPartitionManager.submit(bar, leaderAndIsr, controlledEpoch)
// Completes the first request. That triggers the next one.
callback1.onComplete(makeClientResponse(
response = makeAlterPartition(Seq(makeAlterPartitionTopicData(zar, Errors.NONE))),
version = if (canUseTopicIds) ApiKeys.ALTER_PARTITION.latestVersion else 1
))
assertTrue(future1.isDone)
assertFalse(future2.isDone)
assertFalse(future3.isDone)
// Version 1 is expected because foo does not have a topic id.
val callback2 = verifySendRequest(brokerToController, alterPartitionRequestMatcher(
expectedTopicPartitions = Set(foo, bar),
expectedVersion = 1
))
// Completes the second request.
callback2.onComplete(makeClientResponse(
response = makeAlterPartition(Seq(
makeAlterPartitionTopicData(foo, Errors.NONE),
makeAlterPartitionTopicData(bar, Errors.NONE),
)),
version = 1
))
assertTrue(future1.isDone)
assertTrue(future2.isDone)
assertTrue(future3.isDone)
}
private def verifySendRequest(
brokerToController: NodeToControllerChannelManager,
expectedRequest: ArgumentMatcher[AbstractRequest.Builder[_ <: AbstractRequest]]
@ -609,25 +528,6 @@ class AlterPartitionManagerTest {
)
}
private def makeAlterPartition(
topics: Seq[AlterPartitionResponseData.TopicData]
): AlterPartitionResponse = {
new AlterPartitionResponse(new AlterPartitionResponseData().setTopics(topics.asJava))
}
private def makeAlterPartitionTopicData(
topicIdPartition: TopicIdPartition,
error: Errors
): AlterPartitionResponseData.TopicData = {
new AlterPartitionResponseData.TopicData()
.setTopicName(topicIdPartition.topic)
.setTopicId(topicIdPartition.topicId)
.setPartitions(Collections.singletonList(
new AlterPartitionResponseData.PartitionData()
.setPartitionIndex(topicIdPartition.partition)
.setErrorCode(error.code)))
}
private def partitionResponse(
tp: TopicIdPartition = tp0,
error: Errors = Errors.NONE,
@ -660,7 +560,7 @@ object AlterPartitionManagerTest {
// Supports KIP-704: unclean leader recovery
IBP_3_2_IV0,
// Supports KIP-497: alter partition
IBP_2_7_IV2
IBP_3_0_IV1
)
}

View File

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

View File

@ -30,7 +30,7 @@ class FinalizedFeatureCacheTest {
@Test
def testEmpty(): Unit = {
assertTrue(new ZkMetadataCache(1, MetadataVersion.IBP_2_8_IV1, BrokerFeatures.createDefault(true)).getFeatureOption.isEmpty)
assertTrue(new ZkMetadataCache(1, MetadataVersion.IBP_3_0_IV1, BrokerFeatures.createDefault(true)).getFeatureOption.isEmpty)
}
def asJava(input: Map[String, Short]): java.util.Map[String, java.lang.Short] = {
@ -45,7 +45,7 @@ class FinalizedFeatureCacheTest {
val finalizedFeatures = Map[String, Short]("feature_1" -> 4)
val cache = new ZkMetadataCache(1, MetadataVersion.IBP_2_8_IV1, brokerFeatures)
val cache = new ZkMetadataCache(1, MetadataVersion.IBP_3_0_IV1, brokerFeatures)
cache.updateFeaturesOrThrow(finalizedFeatures, 10)
assertTrue(cache.getFeatureOption.isDefined)
assertEquals(asJava(finalizedFeatures), cache.getFeatureOption.get.finalizedFeatures())
@ -67,7 +67,7 @@ class FinalizedFeatureCacheTest {
val finalizedFeatures = Map[String, Short]("feature_1" -> 2)
val cache = new ZkMetadataCache(1, MetadataVersion.IBP_2_8_IV1, brokerFeatures)
val cache = new ZkMetadataCache(1, MetadataVersion.IBP_3_0_IV1, brokerFeatures)
assertThrows(classOf[FeatureCacheUpdateException], () => cache.updateFeaturesOrThrow(finalizedFeatures, 12))
// Check that the failed updateOrThrow call did not make any mutations.
@ -82,7 +82,7 @@ class FinalizedFeatureCacheTest {
val finalizedFeatures = Map[String, Short]("feature_1" -> 3)
val cache = new ZkMetadataCache(1, MetadataVersion.IBP_2_8_IV1, brokerFeatures)
val cache = new ZkMetadataCache(1, MetadataVersion.IBP_3_0_IV1, brokerFeatures)
cache.updateFeaturesOrThrow(finalizedFeatures, 12)
assertTrue(cache.getFeatureOption.isDefined)
assertEquals(asJava(finalizedFeatures), cache.getFeatureOption.get.finalizedFeatures())
@ -97,7 +97,7 @@ class FinalizedFeatureCacheTest {
val finalizedFeatures = Map[String, Short]("feature_1" -> 3)
val cache = new ZkMetadataCache(1, MetadataVersion.IBP_2_8_IV1, brokerFeatures)
val cache = new ZkMetadataCache(1, MetadataVersion.IBP_3_0_IV1, brokerFeatures)
cache.updateFeaturesOrThrow(finalizedFeatures, 12)
assertTrue(cache.getFeatureOption.isDefined)
assertEquals(asJava(finalizedFeatures), cache.getFeatureOption.get.finalizedFeatures())

View File

@ -2839,7 +2839,7 @@ class KafkaApisTest extends Logging {
// This test verifies the response will not be sent prematurely because of calling replicaManager append
// with no records.
val topicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0)
val writeTxnMarkersRequest = new WriteTxnMarkersRequest.Builder(ApiKeys.WRITE_TXN_MARKERS.latestVersion(),
val writeTxnMarkersRequest = new WriteTxnMarkersRequest.Builder(
asList(
new TxnMarkerEntry(1, 1.toShort, 0, TransactionResult.COMMIT, asList(topicPartition)),
new TxnMarkerEntry(2, 1.toShort, 0, TransactionResult.COMMIT, asList(topicPartition)),
@ -2973,7 +2973,6 @@ class KafkaApisTest extends Logging {
)
val writeTxnMarkersRequest = new WriteTxnMarkersRequest.Builder(
ApiKeys.WRITE_TXN_MARKERS.latestVersion(),
List(
new TxnMarkerEntry(
1L,
@ -3099,7 +3098,6 @@ class KafkaApisTest extends Logging {
)
val writeTxnMarkersRequest = new WriteTxnMarkersRequest.Builder(
ApiKeys.WRITE_TXN_MARKERS.latestVersion(),
List(
new TxnMarkerEntry(
1L,
@ -3225,7 +3223,6 @@ class KafkaApisTest extends Logging {
val offset0 = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0)
val writeTxnMarkersRequest = new WriteTxnMarkersRequest.Builder(
ApiKeys.WRITE_TXN_MARKERS.latestVersion(),
List(
new TxnMarkerEntry(
1L,
@ -9705,7 +9702,7 @@ class KafkaApisTest extends Logging {
}
private def createWriteTxnMarkersRequest(partitions: util.List[TopicPartition]) = {
val writeTxnMarkersRequest = new WriteTxnMarkersRequest.Builder(ApiKeys.WRITE_TXN_MARKERS.latestVersion(),
val writeTxnMarkersRequest = new WriteTxnMarkersRequest.Builder(
asList(new TxnMarkerEntry(1, 1.toShort, 0, TransactionResult.COMMIT, partitions))).build()
(writeTxnMarkersRequest, buildRequest(writeTxnMarkersRequest))
}

View File

@ -38,7 +38,6 @@ import org.apache.kafka.coordinator.transaction.{TransactionLogConfig, Transacti
import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.raft.QuorumConfig
import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.server.common.MetadataVersion.IBP_0_8_2
import org.apache.kafka.server.config.{DelegationTokenManagerConfigs, KRaftConfigs, QuotaConfig, ReplicationConfigs, ServerConfigs, ServerLogConfigs, ServerTopicConfigSynonyms, ZkConfigs}
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
import org.apache.kafka.server.metrics.MetricConfigs
@ -613,16 +612,16 @@ class KafkaConfigTest {
val conf = KafkaConfig.fromProps(props)
assertEquals(MetadataVersion.latestProduction, conf.interBrokerProtocolVersion)
props.setProperty(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, "0.8.2.0")
props.setProperty(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, "3.0.0-IV1")
val conf2 = KafkaConfig.fromProps(props)
assertEquals(IBP_0_8_2, conf2.interBrokerProtocolVersion)
assertEquals(MetadataVersion.IBP_3_0_IV1, conf2.interBrokerProtocolVersion)
// check that 0.8.2.0 is the same as 0.8.2.1
props.setProperty(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, "0.8.2.1")
// check that patch version doesn't affect equality
props.setProperty(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, "3.0.1-IV1")
val conf3 = KafkaConfig.fromProps(props)
assertEquals(IBP_0_8_2, conf3.interBrokerProtocolVersion)
assertEquals(MetadataVersion.IBP_3_0_IV1, conf3.interBrokerProtocolVersion)
//check that latest is newer than 0.8.2
//check that latest is newer than 3.0.1-IV0
assertTrue(MetadataVersion.latestTesting.isAtLeast(conf3.interBrokerProtocolVersion))
}
@ -1640,15 +1639,6 @@ class KafkaConfigTest {
}
}
@Test
def testInvalidInterBrokerProtocolVersionKRaft(): Unit = {
val props = new Properties()
props.putAll(kraftProps())
props.setProperty(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, "2.8")
assertEquals("A non-KRaft version 2.8 given for inter.broker.protocol.version. The minimum version is 3.0-IV1",
assertThrows(classOf[ConfigException], () => new KafkaConfig(props)).getMessage)
}
@Test
def testDefaultInterBrokerProtocolVersionKRaft(): Unit = {
val props = new Properties()

View File

@ -163,6 +163,4 @@ class MockFetcherThread(val mockLeader: MockLeaderEndPoint,
assertEquals(expectedEpoch, fetchState(partition).flatMap(_.lastFetchedEpoch))
}
}
override protected val isOffsetForLeaderEpochSupported: Boolean = true
}

View File

@ -22,7 +22,7 @@ import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition
import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderTopic
import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderTopicCollection
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.record.RecordBatch
import org.apache.kafka.common.requests.{OffsetsForLeaderEpochRequest, OffsetsForLeaderEpochResponse}
import org.junit.jupiter.api.Assertions._
@ -40,8 +40,7 @@ class OffsetsForLeaderEpochRequestTest extends BaseRequestTest {
val partition = new TopicPartition(topic, 0)
val epochs = offsetForLeaderTopicCollectionFor(partition, 0, RecordBatch.NO_PARTITION_LEADER_EPOCH)
val request = OffsetsForLeaderEpochRequest.Builder.forFollower(
ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion, epochs, 1).build()
val request = OffsetsForLeaderEpochRequest.Builder.forFollower(epochs, 1).build()
// Unknown topic
val randomBrokerId = brokers.head.config.brokerId
@ -69,8 +68,7 @@ class OffsetsForLeaderEpochRequestTest extends BaseRequestTest {
def assertResponseErrorForEpoch(error: Errors, brokerId: Int, currentLeaderEpoch: Optional[Integer]): Unit = {
val epochs = offsetForLeaderTopicCollectionFor(topicPartition, 0,
currentLeaderEpoch.orElse(RecordBatch.NO_PARTITION_LEADER_EPOCH))
val request = OffsetsForLeaderEpochRequest.Builder.forFollower(
ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion, epochs, 1).build()
val request = OffsetsForLeaderEpochRequest.Builder.forFollower(epochs, 1).build()
assertResponseError(error, brokerId, request)
}

View File

@ -29,7 +29,6 @@ import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid}
import org.apache.kafka.common.message.{FetchResponseData, UpdateMetadataRequestData}
import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition
import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset
import org.apache.kafka.common.protocol.Errors._
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch, RecordValidationStats, SimpleRecord}
import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.{UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET}
@ -38,17 +37,16 @@ import org.apache.kafka.common.utils.{LogContext, Time}
import org.apache.kafka.server.BrokerFeatures
import org.apache.kafka.server.config.ReplicationConfigs
import org.apache.kafka.server.common.{MetadataVersion, OffsetAndEpoch}
import org.apache.kafka.server.common.MetadataVersion.IBP_2_6_IV0
import org.apache.kafka.server.network.BrokerEndPoint
import org.apache.kafka.storage.internals.log.LogAppendInfo
import org.apache.kafka.storage.log.metrics.BrokerTopicStats
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, Disabled, Test}
import org.junit.jupiter.api.{AfterEach, Test}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
import org.mockito.ArgumentCaptor
import org.mockito.ArgumentMatchers.{any, anyBoolean, anyLong}
import org.mockito.Mockito.{mock, never, times, verify, when}
import org.mockito.Mockito.{mock, times, verify, when}
import java.nio.charset.StandardCharsets
import java.util
@ -130,96 +128,12 @@ class ReplicaFetcherThreadTest {
ApiKeys.FETCH.latestVersion(true),
testingVersion.fetchRequestVersion
)
assertEquals(
ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion(true),
testingVersion.offsetForLeaderEpochRequestVersion
)
assertEquals(
ApiKeys.LIST_OFFSETS.latestVersion(true),
testingVersion.listOffsetRequestVersion
)
}
@Disabled("KAFKA-18370")
@Test
def testFetchLeaderEpochRequestIfLastEpochDefinedForSomePartitions(): Unit = {
val config = kafkaConfigNoTruncateOnFetch
//Setup all dependencies
val quota: ReplicationQuotaManager = mock(classOf[ReplicationQuotaManager])
val logManager: LogManager = mock(classOf[LogManager])
val replicaAlterLogDirsManager: ReplicaAlterLogDirsManager = mock(classOf[ReplicaAlterLogDirsManager])
val log: UnifiedLog = mock(classOf[UnifiedLog])
val partition: Partition = mock(classOf[Partition])
val replicaManager: ReplicaManager = mock(classOf[ReplicaManager])
val leaderEpoch = 5
//Stubs
when(partition.localLogOrException).thenReturn(log)
when(log.logEndOffset).thenReturn(0)
when(log.highWatermark).thenReturn(0)
when(log.latestEpoch)
.thenReturn(Some(leaderEpoch))
.thenReturn(Some(leaderEpoch))
.thenReturn(None) // t2p1 doesn't support epochs
when(log.endOffsetForEpoch(leaderEpoch)).thenReturn(
Some(new OffsetAndEpoch(0, leaderEpoch)))
when(replicaManager.metadataCache).thenReturn(metadataCache)
when(replicaManager.logManager).thenReturn(logManager)
when(replicaManager.replicaAlterLogDirsManager).thenReturn(replicaAlterLogDirsManager)
when(replicaManager.brokerTopicStats).thenReturn(mock(classOf[BrokerTopicStats]))
stub(partition, replicaManager, log)
//Define the offsets for the OffsetsForLeaderEpochResponse
val offsets = Map(
t1p0 -> newOffsetForLeaderPartitionResult(t1p0, leaderEpoch, 1),
t1p1 -> newOffsetForLeaderPartitionResult(t1p1, leaderEpoch, 1)).asJava
//Create the fetcher thread
val mockNetwork = new MockBlockingSender(offsets, brokerEndPoint, Time.SYSTEM)
val thread = createReplicaFetcherThread(
"bob",
0,
config,
failedPartitions,
replicaManager,
quota,
mockNetwork)
// topic 1 supports epoch, t2 doesn't.
thread.addPartitions(Map(
t1p0 -> initialFetchState(Some(topicId1), 0L),
t1p1 -> initialFetchState(Some(topicId2), 0L),
t2p1 -> initialFetchState(Some(topicId2), 0L)))
assertPartitionStates(thread, shouldBeReadyForFetch = false, shouldBeTruncatingLog = true, shouldBeDelayed = false)
//Loop 1
thread.doWork()
assertEquals(1, mockNetwork.epochFetchCount)
assertEquals(1, mockNetwork.fetchCount)
assertPartitionStates(thread, shouldBeReadyForFetch = true, shouldBeTruncatingLog = false, shouldBeDelayed = false)
//Loop 2 we should not fetch epochs
thread.doWork()
assertEquals(1, mockNetwork.epochFetchCount)
assertEquals(2, mockNetwork.fetchCount)
assertPartitionStates(thread, shouldBeReadyForFetch = true, shouldBeTruncatingLog = false, shouldBeDelayed = false)
//Loop 3 we should not fetch epochs
thread.doWork()
assertEquals(1, mockNetwork.epochFetchCount)
assertEquals(3, mockNetwork.fetchCount)
assertPartitionStates(thread, shouldBeReadyForFetch = true, shouldBeTruncatingLog = false, shouldBeDelayed = false)
//Assert that truncate to is called exactly once (despite two loops)
verify(partition, times(3)).truncateTo(anyLong(), anyBoolean())
}
/**
* Assert that all partitions' states are as expected
*
@ -281,18 +195,12 @@ class ReplicaFetcherThreadTest {
verify(mockBlockingSend).sendRequest(any())
}
@Disabled("KAFKA-18370")
@Test
def shouldFetchLeaderEpochOnFirstFetchOnlyIfLeaderEpochKnownToBothIbp26(): Unit = {
verifyFetchLeaderEpochOnFirstFetch(IBP_2_6_IV0)
}
@Test
def shouldNotFetchLeaderEpochOnFirstFetchWithTruncateOnFetch(): Unit = {
verifyFetchLeaderEpochOnFirstFetch(MetadataVersion.latestTesting, epochFetchCount = 0)
}
private def verifyFetchLeaderEpochOnFirstFetch(ibp: MetadataVersion, epochFetchCount: Int = 1): Unit = {
private def verifyFetchLeaderEpochOnFirstFetch(ibp: MetadataVersion, epochFetchCount: Int): Unit = {
val props = TestUtils.createBrokerConfig(1)
props.setProperty(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, ibp.version)
val config = KafkaConfig.fromProps(props)
@ -355,213 +263,6 @@ class ReplicaFetcherThreadTest {
assertEquals(3, mockNetwork.fetchCount)
}
@Disabled("KAFKA-18370")
@Test
def shouldTruncateToOffsetSpecifiedInEpochOffsetResponse(): Unit = {
//Create a capture to track what partitions/offsets are truncated
val truncateToCapture: ArgumentCaptor[Long] = ArgumentCaptor.forClass(classOf[Long])
// Setup all the dependencies
val config = kafkaConfigNoTruncateOnFetch
val quota: ReplicationQuotaManager = mock(classOf[ReplicationQuotaManager])
val logManager: LogManager = mock(classOf[LogManager])
val replicaAlterLogDirsManager: ReplicaAlterLogDirsManager = mock(classOf[ReplicaAlterLogDirsManager])
val log: UnifiedLog = mock(classOf[UnifiedLog])
val partition: Partition = mock(classOf[Partition])
val replicaManager: ReplicaManager = mock(classOf[ReplicaManager])
val leaderEpoch = 5
val initialLEO = 200
//Stubs
when(partition.localLogOrException).thenReturn(log)
when(log.highWatermark).thenReturn(initialLEO - 1)
when(log.latestEpoch).thenReturn(Some(leaderEpoch))
when(log.endOffsetForEpoch(leaderEpoch)).thenReturn(
Some(new OffsetAndEpoch(initialLEO, leaderEpoch)))
when(log.logEndOffset).thenReturn(initialLEO)
when(replicaManager.metadataCache).thenReturn(metadataCache)
when(replicaManager.localLogOrException(any[TopicPartition])).thenReturn(log)
when(replicaManager.logManager).thenReturn(logManager)
when(replicaManager.replicaAlterLogDirsManager).thenReturn(replicaAlterLogDirsManager)
when(replicaManager.brokerTopicStats).thenReturn(mock(classOf[BrokerTopicStats]))
stub(partition, replicaManager, log)
//Define the offsets for the OffsetsForLeaderEpochResponse, these are used for truncation
val offsetsReply = Map(
t1p0 -> newOffsetForLeaderPartitionResult(t1p0, leaderEpoch, 156),
t2p1 -> newOffsetForLeaderPartitionResult(t2p1, leaderEpoch, 172)).asJava
//Create the thread
val mockNetwork = new MockBlockingSender(offsetsReply, brokerEndPoint, Time.SYSTEM)
val thread = createReplicaFetcherThread(
"bob",
0,
config,
failedPartitions,
replicaManager,
quota,
mockNetwork
)
thread.addPartitions(Map(t1p0 -> initialFetchState(Some(topicId1), 0L), t2p1 -> initialFetchState(Some(topicId2), 0L)))
//Run it
thread.doWork()
//We should have truncated to the offsets in the response
verify(partition, times(2)).truncateTo(truncateToCapture.capture(), anyBoolean())
assertTrue(truncateToCapture.getAllValues.asScala.contains(156),
"Expected " + t1p0 + " to truncate to offset 156 (truncation offsets: " + truncateToCapture.getAllValues + ")")
assertTrue(truncateToCapture.getAllValues.asScala.contains(172),
"Expected " + t2p1 + " to truncate to offset 172 (truncation offsets: " + truncateToCapture.getAllValues + ")")
}
@Disabled("KAFKA-18370")
@Test
def shouldTruncateToOffsetSpecifiedInEpochOffsetResponseIfFollowerHasNoMoreEpochs(): Unit = {
// Create a capture to track what partitions/offsets are truncated
val truncateToCapture: ArgumentCaptor[Long] = ArgumentCaptor.forClass(classOf[Long])
// Setup all the dependencies
val config = kafkaConfigNoTruncateOnFetch
val quota: ReplicationQuotaManager = mock(classOf[ReplicationQuotaManager])
val logManager: LogManager = mock(classOf[LogManager])
val replicaAlterLogDirsManager: ReplicaAlterLogDirsManager = mock(classOf[ReplicaAlterLogDirsManager])
val log: UnifiedLog = mock(classOf[UnifiedLog])
val partition: Partition = mock(classOf[Partition])
val replicaManager: ReplicaManager = mock(classOf[ReplicaManager])
val leaderEpochAtFollower = 5
val leaderEpochAtLeader = 4
val initialLEO = 200
//Stubs
when(partition.localLogOrException).thenReturn(log)
when(log.highWatermark).thenReturn(initialLEO - 3)
when(log.latestEpoch).thenReturn(Some(leaderEpochAtFollower))
when(log.endOffsetForEpoch(leaderEpochAtLeader)).thenReturn(None)
when(log.logEndOffset).thenReturn(initialLEO)
when(replicaManager.metadataCache).thenReturn(metadataCache)
when(replicaManager.localLogOrException(any[TopicPartition])).thenReturn(log)
when(replicaManager.logManager).thenReturn(logManager)
when(replicaManager.replicaAlterLogDirsManager).thenReturn(replicaAlterLogDirsManager)
when(replicaManager.brokerTopicStats).thenReturn(mock(classOf[BrokerTopicStats]))
stub(partition, replicaManager, log)
//Define the offsets for the OffsetsForLeaderEpochResponse, these are used for truncation
val offsetsReply = Map(
t1p0 -> newOffsetForLeaderPartitionResult(t1p0, leaderEpochAtLeader, 156),
t2p1 -> newOffsetForLeaderPartitionResult(t2p1, leaderEpochAtLeader, 202)).asJava
//Create the thread
val mockNetwork = new MockBlockingSender(offsetsReply, brokerEndPoint, Time.SYSTEM)
val thread = createReplicaFetcherThread(
"bob",
0,
config,
failedPartitions,
replicaManager,
quota,
mockNetwork
)
thread.addPartitions(Map(t1p0 -> initialFetchState(Some(topicId1), 0L), t2p1 -> initialFetchState(Some(topicId2), 0L)))
//Run it
thread.doWork()
//We should have truncated to the offsets in the response
verify(partition, times(2)).truncateTo(truncateToCapture.capture(), anyBoolean())
assertTrue(truncateToCapture.getAllValues.asScala.contains(156),
"Expected " + t1p0 + " to truncate to offset 156 (truncation offsets: " + truncateToCapture.getAllValues + ")")
assertTrue(truncateToCapture.getAllValues.asScala.contains(initialLEO),
"Expected " + t2p1 + " to truncate to offset " + initialLEO +
" (truncation offsets: " + truncateToCapture.getAllValues + ")")
}
@Disabled("KAFKA-18370")
@Test
def shouldFetchLeaderEpochSecondTimeIfLeaderRepliesWithEpochNotKnownToFollower(): Unit = {
// Create a capture to track what partitions/offsets are truncated
val truncateToCapture: ArgumentCaptor[Long] = ArgumentCaptor.forClass(classOf[Long])
val config = kafkaConfigNoTruncateOnFetch
// Setup all dependencies
val quota: ReplicationQuotaManager = mock(classOf[ReplicationQuotaManager])
val logManager: LogManager = mock(classOf[LogManager])
val replicaAlterLogDirsManager: ReplicaAlterLogDirsManager = mock(classOf[ReplicaAlterLogDirsManager])
val log: UnifiedLog = mock(classOf[UnifiedLog])
val partition: Partition = mock(classOf[Partition])
val replicaManager: ReplicaManager = mock(classOf[ReplicaManager])
val initialLEO = 200
// Stubs
when(partition.localLogOrException).thenReturn(log)
when(log.highWatermark).thenReturn(initialLEO - 2)
when(log.latestEpoch).thenReturn(Some(5))
when(log.endOffsetForEpoch(4)).thenReturn(
Some(new OffsetAndEpoch(120, 3)))
when(log.endOffsetForEpoch(3)).thenReturn(
Some(new OffsetAndEpoch(120, 3)))
when(log.logEndOffset).thenReturn(initialLEO)
when(replicaManager.metadataCache).thenReturn(metadataCache)
when(replicaManager.localLogOrException(any[TopicPartition])).thenReturn(log)
when(replicaManager.logManager).thenReturn(logManager)
when(replicaManager.replicaAlterLogDirsManager).thenReturn(replicaAlterLogDirsManager)
when(replicaManager.brokerTopicStats).thenReturn(mock(classOf[BrokerTopicStats]))
stub(partition, replicaManager, log)
// Define the offsets for the OffsetsForLeaderEpochResponse
val offsets = Map(
t1p0 -> newOffsetForLeaderPartitionResult(t1p0, 4, 155),
t1p1 -> newOffsetForLeaderPartitionResult(t1p1, 4, 143)).asJava
// Create the fetcher thread
val mockNetwork = new MockBlockingSender(offsets, brokerEndPoint, Time.SYSTEM)
val thread = createReplicaFetcherThread(
"bob",
0,
config,
failedPartitions,
replicaManager,
quota,
mockNetwork
)
thread.addPartitions(Map(t1p0 -> initialFetchState(Some(topicId1), 0L), t1p1 -> initialFetchState(Some(topicId1), 0L)))
// Loop 1 -- both topic partitions will need to fetch another leader epoch
thread.doWork()
assertEquals(1, mockNetwork.epochFetchCount)
assertEquals(0, mockNetwork.fetchCount)
// Loop 2 should do the second fetch for both topic partitions because the leader replied with
// epoch 4 while follower knows only about epoch 3
val nextOffsets = Map(
t1p0 -> newOffsetForLeaderPartitionResult(t1p0, 3, 101),
t1p1 -> newOffsetForLeaderPartitionResult(t1p1, 3, 102)).asJava
mockNetwork.setOffsetsForNextResponse(nextOffsets)
thread.doWork()
assertEquals(2, mockNetwork.epochFetchCount)
assertEquals(1, mockNetwork.fetchCount)
assertTrue(mockNetwork.lastUsedOffsetForLeaderEpochVersion >= 3,
"OffsetsForLeaderEpochRequest version.")
//Loop 3 we should not fetch epochs
thread.doWork()
assertEquals(2, mockNetwork.epochFetchCount)
assertEquals(2, mockNetwork.fetchCount)
verify(partition, times(4)).truncateTo(truncateToCapture.capture(), anyBoolean())
//We should have truncated to the offsets in the second response
assertTrue(truncateToCapture.getAllValues.asScala.contains(102),
"Expected " + t1p1 + " to truncate to offset 102 (truncation offsets: " + truncateToCapture.getAllValues + ")")
assertTrue(truncateToCapture.getAllValues.asScala.contains(101),
"Expected " + t1p0 + " to truncate to offset 101 (truncation offsets: " + truncateToCapture.getAllValues + ")")
}
@Test
def shouldTruncateIfLeaderRepliesWithDivergingEpochNotKnownToFollower(): Unit = {
@ -853,329 +554,6 @@ class ReplicaFetcherThreadTest {
assertEquals(Some(lastFetchedEpoch), thread.fetchState(t1p0).flatMap(_.lastFetchedEpoch))
}
@Disabled("KAFKA-18370")
@Test
def shouldUseLeaderEndOffsetIfInterBrokerVersionBelow20(): Unit = {
// Create a capture to track what partitions/offsets are truncated
val truncateToCapture: ArgumentCaptor[Long] = ArgumentCaptor.forClass(classOf[Long])
val props = TestUtils.createBrokerConfig(1)
props.put(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, "0.11.0")
val config = KafkaConfig.fromProps(props)
// Setup all dependencies
val quota: ReplicationQuotaManager = mock(classOf[ReplicationQuotaManager])
val logManager: LogManager = mock(classOf[LogManager])
val replicaAlterLogDirsManager: ReplicaAlterLogDirsManager = mock(classOf[ReplicaAlterLogDirsManager])
val log: UnifiedLog = mock(classOf[UnifiedLog])
val partition: Partition = mock(classOf[Partition])
val replicaManager: ReplicaManager = mock(classOf[ReplicaManager])
val initialLEO = 200
// Stubs
when(partition.localLogOrException).thenReturn(log)
when(log.highWatermark).thenReturn(initialLEO - 2)
when(log.latestEpoch).thenReturn(Some(5))
when(log.endOffsetForEpoch(4)).thenReturn(
Some(new OffsetAndEpoch(120, 3)))
when(log.endOffsetForEpoch(3)).thenReturn(
Some(new OffsetAndEpoch(120, 3)))
when(log.logEndOffset).thenReturn(initialLEO)
when(replicaManager.metadataCache).thenReturn(metadataCache)
when(replicaManager.localLogOrException(any[TopicPartition])).thenReturn(log)
when(replicaManager.logManager).thenReturn(logManager)
when(replicaManager.replicaAlterLogDirsManager).thenReturn(replicaAlterLogDirsManager)
when(replicaManager.brokerTopicStats).thenReturn(mock(classOf[BrokerTopicStats]))
stub(partition, replicaManager, log)
// Define the offsets for the OffsetsForLeaderEpochResponse with undefined epoch to simulate
// older protocol version
val offsets = Map(
t1p0 -> newOffsetForLeaderPartitionResult(t1p0, UNDEFINED_EPOCH, 155),
t1p1 -> newOffsetForLeaderPartitionResult(t1p1, UNDEFINED_EPOCH, 143)).asJava
// Create the fetcher thread
val mockNetwork = new MockBlockingSender(offsets, brokerEndPoint, Time.SYSTEM)
val thread = createReplicaFetcherThread(
"bob",
0,
config,
failedPartitions,
replicaManager,
quota,
mockNetwork
)
thread.addPartitions(Map(t1p0 -> initialFetchState(Some(topicId1), 0L), t1p1 -> initialFetchState(Some(topicId1), 0L)))
// Loop 1 -- both topic partitions will truncate to leader offset even though they don't know
// about leader epoch
thread.doWork()
assertEquals(1, mockNetwork.epochFetchCount)
assertEquals(1, mockNetwork.fetchCount)
assertEquals(0, mockNetwork.lastUsedOffsetForLeaderEpochVersion, "OffsetsForLeaderEpochRequest version.")
//Loop 2 we should not fetch epochs
thread.doWork()
assertEquals(1, mockNetwork.epochFetchCount)
assertEquals(2, mockNetwork.fetchCount)
//We should have truncated to the offsets in the first response
verify(partition, times(2)).truncateTo(truncateToCapture.capture(), anyBoolean())
assertTrue(truncateToCapture.getAllValues.asScala.contains(155),
"Expected " + t1p0 + " to truncate to offset 155 (truncation offsets: " + truncateToCapture.getAllValues + ")")
assertTrue(truncateToCapture.getAllValues.asScala.contains(143),
"Expected " + t1p1 + " to truncate to offset 143 (truncation offsets: " + truncateToCapture.getAllValues + ")")
}
@Disabled("KAFKA-18370")
@Test
def shouldTruncateToInitialFetchOffsetIfLeaderReturnsUndefinedOffset(): Unit = {
//Create a capture to track what partitions/offsets are truncated
val truncated: ArgumentCaptor[Long] = ArgumentCaptor.forClass(classOf[Long])
// Setup all the dependencies
val config = kafkaConfigNoTruncateOnFetch
val quota: ReplicationQuotaManager = mock(classOf[ReplicationQuotaManager])
val logManager: LogManager = mock(classOf[LogManager])
val replicaAlterLogDirsManager: ReplicaAlterLogDirsManager = mock(classOf[ReplicaAlterLogDirsManager])
val log: UnifiedLog = mock(classOf[UnifiedLog])
val partition: Partition = mock(classOf[Partition])
val replicaManager: ReplicaManager = mock(classOf[ReplicaManager])
val initialFetchOffset = 100
//Stubs
when(partition.localLogOrException).thenReturn(log)
when(log.highWatermark).thenReturn(initialFetchOffset)
when(log.latestEpoch).thenReturn(Some(5))
when(replicaManager.metadataCache).thenReturn(metadataCache)
when(replicaManager.logManager).thenReturn(logManager)
when(replicaManager.replicaAlterLogDirsManager).thenReturn(replicaAlterLogDirsManager)
when(replicaManager.brokerTopicStats).thenReturn(mock(classOf[BrokerTopicStats]))
stub(partition, replicaManager, log)
//Define the offsets for the OffsetsForLeaderEpochResponse, these are used for truncation
val offsetsReply = Map(
t1p0 -> newOffsetForLeaderPartitionResult(t1p0, UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET)).asJava
//Create the thread
val mockNetwork = new MockBlockingSender(offsetsReply, brokerEndPoint, Time.SYSTEM)
val thread = createReplicaFetcherThread(
"bob",
0,
config,
failedPartitions,
replicaManager,
quota,
mockNetwork
)
thread.addPartitions(Map(t1p0 -> initialFetchState(Some(topicId1), initialFetchOffset)))
//Run it
thread.doWork()
//We should have truncated to initial fetch offset
verify(partition).truncateTo(truncated.capture(), anyBoolean())
assertEquals(initialFetchOffset, truncated.getValue)
}
@Disabled("KAFKA-18370")
@Test
def shouldPollIndefinitelyIfLeaderReturnsAnyException(): Unit = {
//Create a capture to track what partitions/offsets are truncated
val truncated: ArgumentCaptor[Long] = ArgumentCaptor.forClass(classOf[Long])
// Setup all the dependencies
val config = kafkaConfigNoTruncateOnFetch
val quota: ReplicationQuotaManager = mock(classOf[ReplicationQuotaManager])
val logManager: LogManager = mock(classOf[LogManager])
val replicaAlterLogDirsManager: ReplicaAlterLogDirsManager = mock(classOf[ReplicaAlterLogDirsManager])
val log: UnifiedLog = mock(classOf[UnifiedLog])
val partition: Partition = mock(classOf[Partition])
val replicaManager: ReplicaManager = mock(classOf[ReplicaManager])
val leaderEpoch = 5
val highWaterMark = 100
val initialLeo = 300
//Stubs
when(log.highWatermark).thenReturn(highWaterMark)
when(partition.localLogOrException).thenReturn(log)
when(log.latestEpoch).thenReturn(Some(leaderEpoch))
// this is for the last reply with EpochEndOffset(5, 156)
when(log.endOffsetForEpoch(leaderEpoch)).thenReturn(
Some(new OffsetAndEpoch(initialLeo, leaderEpoch)))
when(log.logEndOffset).thenReturn(initialLeo)
when(replicaManager.metadataCache).thenReturn(metadataCache)
when(replicaManager.localLogOrException(any[TopicPartition])).thenReturn(log)
when(replicaManager.logManager).thenReturn(logManager)
when(replicaManager.replicaAlterLogDirsManager).thenReturn(replicaAlterLogDirsManager)
when(replicaManager.brokerTopicStats).thenReturn(mock(classOf[BrokerTopicStats]))
stub(partition, replicaManager, log)
//Define the offsets for the OffsetsForLeaderEpochResponse, these are used for truncation
val offsetsReply = mutable.Map(
t1p0 -> newOffsetForLeaderPartitionResult(t1p0, NOT_LEADER_OR_FOLLOWER, UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET),
t1p1 -> newOffsetForLeaderPartitionResult(t1p1, UNKNOWN_SERVER_ERROR, UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET)
).asJava
//Create the thread
val mockNetwork = new MockBlockingSender(offsetsReply, brokerEndPoint, Time.SYSTEM)
val thread = createReplicaFetcherThread(
"bob",
0,
config,
failedPartitions,
replicaManager,
quota,
mockNetwork
)
thread.addPartitions(Map(t1p0 -> initialFetchState(Some(topicId1), 0L), t1p1 -> initialFetchState(Some(topicId1), 0L)))
//Run thread 3 times
(0 to 3).foreach { _ =>
thread.doWork()
}
//Then should loop continuously while there is no leader
verify(partition, never()).truncateTo(anyLong(), anyBoolean())
//New leader elected and replies
offsetsReply.put(t1p0, newOffsetForLeaderPartitionResult(t1p0, leaderEpoch, 156))
thread.doWork()
//Now the final call should have actually done a truncation (to offset 156)
verify(partition).truncateTo(truncated.capture(), anyBoolean())
assertEquals(156, truncated.getValue)
}
@Disabled("KAFKA-18370")
@Test
def shouldMovePartitionsOutOfTruncatingLogState(): Unit = {
val config = kafkaConfigNoTruncateOnFetch
//Setup all stubs
val quota: ReplicationQuotaManager = mock(classOf[ReplicationQuotaManager])
val logManager: LogManager = mock(classOf[LogManager])
val replicaAlterLogDirsManager: ReplicaAlterLogDirsManager = mock(classOf[ReplicaAlterLogDirsManager])
val log: UnifiedLog = mock(classOf[UnifiedLog])
val partition: Partition = mock(classOf[Partition])
val replicaManager: ReplicaManager = mock(classOf[ReplicaManager])
val leaderEpoch = 4
//Stub return values
when(partition.localLogOrException).thenReturn(log)
when(log.highWatermark).thenReturn(0)
when(log.latestEpoch).thenReturn(Some(leaderEpoch))
when(log.endOffsetForEpoch(leaderEpoch)).thenReturn(
Some(new OffsetAndEpoch(0, leaderEpoch)))
when(replicaManager.metadataCache).thenReturn(metadataCache)
when(replicaManager.logManager).thenReturn(logManager)
when(replicaManager.replicaAlterLogDirsManager).thenReturn(replicaAlterLogDirsManager)
stub(partition, replicaManager, log)
//Define the offsets for the OffsetsForLeaderEpochResponse
val offsetsReply = Map(
t1p0 -> newOffsetForLeaderPartitionResult(t1p0, leaderEpoch, 1),
t1p1 -> newOffsetForLeaderPartitionResult(t1p1, leaderEpoch, 1)
).asJava
//Create the fetcher thread
val mockNetwork = new MockBlockingSender(offsetsReply, brokerEndPoint, Time.SYSTEM)
val thread = createReplicaFetcherThread(
"bob",
0,
config,
failedPartitions,
replicaManager,
quota,
mockNetwork
)
//When
thread.addPartitions(Map(t1p0 -> initialFetchState(Some(topicId1), 0L), t1p1 -> initialFetchState(Some(topicId1), 0L)))
//Then all partitions should start in an TruncatingLog state
assertEquals(Option(Truncating), thread.fetchState(t1p0).map(_.state))
assertEquals(Option(Truncating), thread.fetchState(t1p1).map(_.state))
//When
thread.doWork()
//Then none should be TruncatingLog anymore
assertEquals(Option(Fetching), thread.fetchState(t1p0).map(_.state))
assertEquals(Option(Fetching), thread.fetchState(t1p1).map(_.state))
verify(partition, times(2)).truncateTo(0L, false)
}
@Disabled("KAFKA-18370")
@Test
def shouldFilterPartitionsMadeLeaderDuringLeaderEpochRequest(): Unit ={
val config = kafkaConfigNoTruncateOnFetch
val truncateToCapture: ArgumentCaptor[Long] = ArgumentCaptor.forClass(classOf[Long])
val initialLEO = 100
//Setup all stubs
val quota: ReplicationQuotaManager = mock(classOf[ReplicationQuotaManager])
val logManager: LogManager = mock(classOf[LogManager])
val replicaAlterLogDirsManager: ReplicaAlterLogDirsManager = mock(classOf[ReplicaAlterLogDirsManager])
val log: UnifiedLog = mock(classOf[UnifiedLog])
val partition: Partition = mock(classOf[Partition])
val replicaManager: ReplicaManager = mock(classOf[ReplicaManager])
//Stub return values
when(partition.localLogOrException).thenReturn(log)
when(log.highWatermark).thenReturn(initialLEO - 2)
when(log.latestEpoch).thenReturn(Some(5))
when(log.endOffsetForEpoch(5)).thenReturn(Some(new OffsetAndEpoch(initialLEO, 5)))
when(log.logEndOffset).thenReturn(initialLEO)
when(replicaManager.metadataCache).thenReturn(metadataCache)
when(replicaManager.localLogOrException(any[TopicPartition])).thenReturn(log)
when(replicaManager.logManager).thenReturn(logManager)
when(replicaManager.replicaAlterLogDirsManager).thenReturn(replicaAlterLogDirsManager)
stub(partition, replicaManager, log)
//Define the offsets for the OffsetsForLeaderEpochResponse
val offsetsReply = Map(
t1p0 -> newOffsetForLeaderPartitionResult(t1p0, 5, 52),
t1p1 -> newOffsetForLeaderPartitionResult(t1p1, 5, 49)
).asJava
//Create the fetcher thread
val mockNetwork = new MockBlockingSender(offsetsReply, brokerEndPoint, Time.SYSTEM)
val thread = createReplicaFetcherThread(
"bob",
0,
config,
failedPartitions,
replicaManager,
quota,
mockNetwork
)
//When
thread.addPartitions(Map(t1p0 -> initialFetchState(Some(topicId1), 0L), t1p1 -> initialFetchState(Some(topicId1), 0L)))
//When the epoch request is outstanding, remove one of the partitions to simulate a leader change. We do this via a callback passed to the mock thread
val partitionThatBecameLeader = t1p0
mockNetwork.setEpochRequestCallback(() => {
thread.removePartitions(Set(partitionThatBecameLeader))
})
//When
thread.doWork()
//Then we should not have truncated the partition that became leader. Exactly one partition should be truncated.
verify(partition).truncateTo(truncateToCapture.capture(), anyBoolean())
assertEquals(49, truncateToCapture.getValue)
}
@Test
def shouldCatchExceptionFromBlockingSendWhenShuttingDownReplicaFetcherThread(): Unit = {
val props = TestUtils.createBrokerConfig(1)
@ -1448,10 +826,4 @@ class ReplicaFetcherThreadTest {
when(replicaManager.localLogOrException(t2p1)).thenReturn(log)
when(replicaManager.getPartitionOrException(t2p1)).thenReturn(partition)
}
private def kafkaConfigNoTruncateOnFetch: KafkaConfig = {
val props = TestUtils.createBrokerConfig(1)
props.setProperty(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, IBP_2_6_IV0.version)
KafkaConfig.fromProps(props)
}
}

View File

@ -479,7 +479,7 @@ class RequestQuotaTest extends BaseRequestTest {
)
case ApiKeys.WRITE_TXN_MARKERS =>
new WriteTxnMarkersRequest.Builder(ApiKeys.WRITE_TXN_MARKERS.latestVersion(), List.empty.asJava)
new WriteTxnMarkersRequest.Builder(java.util.List.of[WriteTxnMarkersRequest.TxnMarkerEntry])
case ApiKeys.TXN_OFFSET_COMMIT =>
new TxnOffsetCommitRequest.Builder(
@ -638,7 +638,7 @@ class RequestQuotaTest extends BaseRequestTest {
Topic.CLUSTER_METADATA_TOPIC_PARTITION))
case ApiKeys.ALTER_PARTITION =>
new AlterPartitionRequest.Builder(new AlterPartitionRequestData(), true)
new AlterPartitionRequest.Builder(new AlterPartitionRequestData())
case ApiKeys.UPDATE_FEATURES =>
new UpdateFeaturesRequest.Builder(new UpdateFeaturesRequestData())

View File

@ -26,7 +26,6 @@ import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.{OffsetFo
import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol.ApiKeys
import org.apache.kafka.common.protocol.Errors._
import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET
import org.apache.kafka.common.requests.{OffsetsForLeaderEpochRequest, OffsetsForLeaderEpochResponse}
@ -321,8 +320,7 @@ class LeaderEpochIntegrationTest extends QuorumTestHarness with Logging {
.setLeaderEpoch(leaderEpoch))
}
val request = OffsetsForLeaderEpochRequest.Builder.forFollower(
ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion, topics, 1)
val request = OffsetsForLeaderEpochRequest.Builder.forFollower(topics, 1)
val response = sender.sendRequest(request)
response.responseBody.asInstanceOf[OffsetsForLeaderEpochResponse].data.topics.asScala.flatMap { topic =>
topic.partitions.asScala.map { partition =>

View File

@ -57,7 +57,6 @@ import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroupMember;
import org.apache.kafka.coordinator.group.modern.consumer.ResolvedRegularExpression;
import org.apache.kafka.coordinator.group.modern.share.ShareGroupMember;
import org.apache.kafka.server.common.ApiMessageAndVersion;
import org.apache.kafka.server.common.MetadataVersion;
import java.util.ArrayList;
import java.util.Collections;
@ -70,6 +69,9 @@ import java.util.Set;
* the __consumer_offsets topic.
*/
public class GroupCoordinatorRecordHelpers {
private static final short GROUP_METADATA_VALUE_VERSION = 3;
private GroupCoordinatorRecordHelpers() {}
/**
@ -443,13 +445,11 @@ public class GroupCoordinatorRecordHelpers {
*
* @param group The classic group.
* @param assignment The classic group assignment.
* @param metadataVersion The metadata version.
* @return The record.
*/
public static CoordinatorRecord newGroupMetadataRecord(
ClassicGroup group,
Map<String, byte[]> assignment,
MetadataVersion metadataVersion
Map<String, byte[]> assignment
) {
List<GroupMetadataValue.MemberMetadata> members = new ArrayList<>(group.allMembers().size());
group.allMembers().forEach(member -> {
@ -491,7 +491,7 @@ public class GroupCoordinatorRecordHelpers {
.setLeader(group.leaderOrNull())
.setCurrentStateTimestamp(group.currentStateTimestampOrDefault())
.setMembers(members),
metadataVersion.groupMetadataValueVersion()
GROUP_METADATA_VALUE_VERSION
)
);
}
@ -519,12 +519,10 @@ public class GroupCoordinatorRecordHelpers {
* Creates an empty GroupMetadata record.
*
* @param group The classic group.
* @param metadataVersion The metadata version.
* @return The record.
*/
public static CoordinatorRecord newEmptyGroupMetadataRecord(
ClassicGroup group,
MetadataVersion metadataVersion
ClassicGroup group
) {
return new CoordinatorRecord(
new ApiMessageAndVersion(
@ -540,7 +538,7 @@ public class GroupCoordinatorRecordHelpers {
.setLeader(null)
.setCurrentStateTimestamp(group.currentStateTimestampOrDefault())
.setMembers(Collections.emptyList()),
metadataVersion.groupMetadataValueVersion()
GROUP_METADATA_VALUE_VERSION
)
);
}
@ -552,17 +550,15 @@ public class GroupCoordinatorRecordHelpers {
* @param topic The topic name.
* @param partitionId The partition id.
* @param offsetAndMetadata The offset and metadata.
* @param metadataVersion The metadata version.
* @return The record.
*/
public static CoordinatorRecord newOffsetCommitRecord(
String groupId,
String topic,
int partitionId,
OffsetAndMetadata offsetAndMetadata,
MetadataVersion metadataVersion
OffsetAndMetadata offsetAndMetadata
) {
short version = metadataVersion.offsetCommitValueVersion(offsetAndMetadata.expireTimestampMs.isPresent());
short version = offsetCommitValueVersion(offsetAndMetadata.expireTimestampMs.isPresent());
return new CoordinatorRecord(
new ApiMessageAndVersion(
@ -585,6 +581,16 @@ public class GroupCoordinatorRecordHelpers {
);
}
static short offsetCommitValueVersion(boolean expireTimestampMs) {
if (expireTimestampMs) {
return 1;
} else {
// Serialize with the highest supported non-flexible version
// until a tagged field is introduced or the version is bumped.
return 3;
}
}
/**
* Creates an OffsetCommit tombstone record.
*

View File

@ -731,7 +731,6 @@ public class GroupCoordinatorShard implements CoordinatorShard<CoordinatorRecord
public void onLoaded(MetadataImage newImage) {
MetadataDelta emptyDelta = new MetadataDelta(newImage);
groupMetadataManager.onNewMetadataImage(newImage, emptyDelta);
offsetMetadataManager.onNewMetadataImage(newImage, emptyDelta);
coordinatorMetrics.activateMetricsShard(metricsShard);
groupMetadataManager.onLoaded();
@ -756,7 +755,6 @@ public class GroupCoordinatorShard implements CoordinatorShard<CoordinatorRecord
@Override
public void onNewMetadataImage(MetadataImage newImage, MetadataDelta delta) {
groupMetadataManager.onNewMetadataImage(newImage, delta);
offsetMetadataManager.onNewMetadataImage(newImage, delta);
}
private static OffsetCommitKey convertLegacyOffsetCommitKey(

View File

@ -1017,7 +1017,7 @@ public class GroupMetadataManager {
throw new GroupIdNotFoundException(String.format("Cannot downgrade the classic group %s: %s.",
consumerGroup.groupId(), e.getMessage()));
}
classicGroup.createClassicGroupRecords(metadataImage.features().metadataVersion(), records);
classicGroup.createClassicGroupRecords(records);
// Directly update the states instead of replaying the records because
// the classicGroup reference is needed for triggering the rebalance.
@ -4411,7 +4411,7 @@ public class GroupMetadataManager {
});
records.add(
GroupCoordinatorRecordHelpers.newEmptyGroupMetadataRecord(group, metadataImage.features().metadataVersion())
GroupCoordinatorRecordHelpers.newEmptyGroupMetadataRecord(group)
);
return new CoordinatorResult<>(records, appendFuture, false);
@ -4841,7 +4841,7 @@ public class GroupMetadataManager {
});
List<CoordinatorRecord> records = Collections.singletonList(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(
group, Collections.emptyMap(), metadataImage.features().metadataVersion()));
group, Collections.emptyMap()));
return new CoordinatorResult<>(records, appendFuture, false);
@ -5503,7 +5503,7 @@ public class GroupMetadataManager {
});
List<CoordinatorRecord> records = Collections.singletonList(
GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, groupAssignment, metadataImage.features().metadataVersion())
GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, groupAssignment)
);
return new CoordinatorResult<>(records, appendFuture, false);
@ -5650,7 +5650,7 @@ public class GroupMetadataManager {
});
List<CoordinatorRecord> records = Collections.singletonList(
GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, assignment, metadataImage.features().metadataVersion())
GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, assignment)
);
return new CoordinatorResult<>(records, appendFuture, false);
}

View File

@ -47,7 +47,6 @@ import org.apache.kafka.coordinator.group.generated.OffsetCommitKey;
import org.apache.kafka.coordinator.group.generated.OffsetCommitValue;
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics;
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
import org.apache.kafka.image.MetadataDelta;
import org.apache.kafka.image.MetadataImage;
import org.apache.kafka.timeline.SnapshotRegistry;
import org.apache.kafka.timeline.TimelineHashMap;
@ -168,11 +167,6 @@ public class OffsetMetadataManager {
*/
private final Time time;
/**
* The metadata image.
*/
private MetadataImage metadataImage;
/**
* The group metadata manager.
*/
@ -284,7 +278,6 @@ public class OffsetMetadataManager {
this.snapshotRegistry = snapshotRegistry;
this.log = logContext.logger(OffsetMetadataManager.class);
this.time = time;
this.metadataImage = metadataImage;
this.groupMetadataManager = groupMetadataManager;
this.config = config;
this.metrics = metrics;
@ -498,8 +491,7 @@ public class OffsetMetadataManager {
request.groupId(),
topic.name(),
partition.partitionIndex(),
offsetAndMetadata,
metadataImage.features().metadataVersion()
offsetAndMetadata
));
}
});
@ -558,8 +550,7 @@ public class OffsetMetadataManager {
request.groupId(),
topic.name(),
partition.partitionIndex(),
offsetAndMetadata,
metadataImage.features().metadataVersion()
offsetAndMetadata
));
}
});
@ -1111,16 +1102,6 @@ public class OffsetMetadataManager {
}
}
/**
* A new metadata image is available.
*
* @param newImage The new metadata image.
* @param delta The delta image.
*/
public void onNewMetadataImage(MetadataImage newImage, MetadataDelta delta) {
metadataImage = newImage;
}
/**
* @return The offset for the provided groupId and topic partition or null
* if it does not exist.

View File

@ -41,7 +41,6 @@ import org.apache.kafka.coordinator.group.OffsetExpirationConditionImpl;
import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroup;
import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroupMember;
import org.apache.kafka.image.MetadataImage;
import org.apache.kafka.server.common.MetadataVersion;
import org.slf4j.Logger;
@ -1433,11 +1432,9 @@ public class ClassicGroup implements Group {
/**
* Populate the record list with the records needed to create the given classic group.
*
* @param metadataVersion The MetadataVersion.
* @param records The list to which the new records are added.
*/
public void createClassicGroupRecords(
MetadataVersion metadataVersion,
List<CoordinatorRecord> records
) {
Map<String, byte[]> assignments = new HashMap<>();
@ -1445,7 +1442,7 @@ public class ClassicGroup implements Group {
assignments.put(classicGroupMember.memberId(), classicGroupMember.assignment())
);
records.add(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(this, assignments, metadataVersion));
records.add(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(this, assignments));
}
/**

View File

@ -50,13 +50,8 @@ import org.apache.kafka.coordinator.group.modern.TopicMetadata;
import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroupMember;
import org.apache.kafka.coordinator.group.modern.consumer.ResolvedRegularExpression;
import org.apache.kafka.server.common.ApiMessageAndVersion;
import org.apache.kafka.server.common.MetadataVersion;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.Arguments;
import org.junit.jupiter.params.provider.EnumSource;
import org.junit.jupiter.params.provider.MethodSource;
import java.util.ArrayList;
import java.util.Arrays;
@ -70,7 +65,6 @@ import java.util.Optional;
import java.util.OptionalInt;
import java.util.OptionalLong;
import java.util.Set;
import java.util.stream.Stream;
import static org.apache.kafka.coordinator.group.Assertions.assertRecordEquals;
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkOrderedAssignment;
@ -454,21 +448,8 @@ public class GroupCoordinatorRecordHelpersTest {
));
}
private static Stream<Arguments> metadataToExpectedGroupMetadataValue() {
return Stream.of(
Arguments.arguments(MetadataVersion.IBP_0_10_0_IV0, (short) 0),
Arguments.arguments(MetadataVersion.IBP_1_1_IV0, (short) 1),
Arguments.arguments(MetadataVersion.IBP_2_2_IV0, (short) 2),
Arguments.arguments(MetadataVersion.IBP_3_5_IV0, (short) 3)
);
}
@ParameterizedTest
@MethodSource("metadataToExpectedGroupMetadataValue")
public void testNewGroupMetadataRecord(
MetadataVersion metadataVersion,
short expectedGroupMetadataValueVersion
) {
@Test
public void testNewGroupMetadataRecord() {
Time time = new MockTime();
List<GroupMetadataValue.MemberMetadata> expectedMembers = new ArrayList<>();
@ -509,7 +490,7 @@ public class GroupCoordinatorRecordHelpersTest {
.setGeneration(1)
.setCurrentStateTimestamp(time.milliseconds())
.setMembers(expectedMembers),
expectedGroupMetadataValueVersion));
(short) 3));
ClassicGroup group = new ClassicGroup(
new LogContext(),
@ -544,8 +525,7 @@ public class GroupCoordinatorRecordHelpersTest {
group.initNextGeneration();
CoordinatorRecord groupMetadataRecord = GroupCoordinatorRecordHelpers.newGroupMetadataRecord(
group,
assignment,
metadataVersion
assignment
);
assertEquals(expectedRecord, groupMetadataRecord);
@ -610,8 +590,7 @@ public class GroupCoordinatorRecordHelpersTest {
assertThrows(IllegalStateException.class, () ->
GroupCoordinatorRecordHelpers.newGroupMetadataRecord(
group,
Collections.emptyMap(),
MetadataVersion.IBP_3_5_IV2
Collections.emptyMap()
));
}
@ -661,17 +640,12 @@ public class GroupCoordinatorRecordHelpersTest {
assertThrows(IllegalStateException.class, () ->
GroupCoordinatorRecordHelpers.newGroupMetadataRecord(
group,
Collections.emptyMap(),
MetadataVersion.IBP_3_5_IV2
Collections.emptyMap()
));
}
@ParameterizedTest
@MethodSource("metadataToExpectedGroupMetadataValue")
public void testEmptyGroupMetadataRecord(
MetadataVersion metadataVersion,
short expectedGroupMetadataValueVersion
) {
@Test
public void testEmptyGroupMetadataRecord() {
Time time = new MockTime();
List<GroupMetadataValue.MemberMetadata> expectedMembers = Collections.emptyList();
@ -689,7 +663,7 @@ public class GroupCoordinatorRecordHelpersTest {
.setGeneration(0)
.setCurrentStateTimestamp(time.milliseconds())
.setMembers(expectedMembers),
expectedGroupMetadataValueVersion));
(short) 3));
ClassicGroup group = new ClassicGroup(
new LogContext(),
@ -700,16 +674,20 @@ public class GroupCoordinatorRecordHelpersTest {
group.initNextGeneration();
CoordinatorRecord groupMetadataRecord = GroupCoordinatorRecordHelpers.newEmptyGroupMetadataRecord(
group,
metadataVersion
group
);
assertEquals(expectedRecord, groupMetadataRecord);
}
@ParameterizedTest
@EnumSource(value = MetadataVersion.class)
public void testNewOffsetCommitRecord(MetadataVersion metadataVersion) {
@Test
public void testOffsetCommitValueVersion() {
assertEquals((short) 1, GroupCoordinatorRecordHelpers.offsetCommitValueVersion(true));
assertEquals((short) 3, GroupCoordinatorRecordHelpers.offsetCommitValueVersion(false));
}
@Test
public void testNewOffsetCommitRecord() {
OffsetCommitKey key = new OffsetCommitKey()
.setGroup("group-id")
.setTopic("foo")
@ -727,8 +705,7 @@ public class GroupCoordinatorRecordHelpersTest {
(short) 1),
new ApiMessageAndVersion(
value,
metadataVersion.offsetCommitValueVersion(false)
)
GroupCoordinatorRecordHelpers.offsetCommitValueVersion(false))
);
assertEquals(expectedRecord, GroupCoordinatorRecordHelpers.newOffsetCommitRecord(
@ -740,8 +717,7 @@ public class GroupCoordinatorRecordHelpersTest {
OptionalInt.of(10),
"metadata",
1234L,
OptionalLong.empty()),
metadataVersion
OptionalLong.empty())
));
value.setLeaderEpoch(-1);
@ -755,14 +731,12 @@ public class GroupCoordinatorRecordHelpersTest {
OptionalInt.empty(),
"metadata",
1234L,
OptionalLong.empty()),
metadataVersion
OptionalLong.empty())
));
}
@ParameterizedTest
@EnumSource(value = MetadataVersion.class)
public void testNewOffsetCommitRecordWithExpireTimestamp(MetadataVersion metadataVersion) {
@Test
public void testNewOffsetCommitRecordWithExpireTimestamp() {
CoordinatorRecord expectedRecord = new CoordinatorRecord(
new ApiMessageAndVersion(
new OffsetCommitKey()
@ -790,8 +764,7 @@ public class GroupCoordinatorRecordHelpersTest {
OptionalInt.of(10),
"metadata",
1234L,
OptionalLong.of(5678L)),
metadataVersion
OptionalLong.of(5678L))
));
}

View File

@ -94,7 +94,6 @@ import org.apache.kafka.coordinator.group.modern.share.ShareGroupMember;
import org.apache.kafka.image.MetadataDelta;
import org.apache.kafka.image.MetadataImage;
import org.apache.kafka.image.MetadataProvenance;
import org.apache.kafka.server.common.MetadataVersion;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
@ -3646,7 +3645,7 @@ public class GroupMetadataManagerTest {
ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
assertEquals(
List.of(GroupCoordinatorRecordHelpers.newEmptyGroupMetadataRecord(group, MetadataVersion.latestTesting())),
List.of(GroupCoordinatorRecordHelpers.newEmptyGroupMetadataRecord(group)),
joinResult.records
);
}
@ -3719,8 +3718,7 @@ public class GroupMetadataManagerTest {
.setLeader("member-0")
.setProtocolType("consumer")
.setProtocol("range")
.setCurrentStateTimestamp(context.time.milliseconds()),
MetadataVersion.latestTesting());
.setCurrentStateTimestamp(context.time.milliseconds()));
context.replay(groupMetadataRecord);
ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
@ -3777,8 +3775,7 @@ public class GroupMetadataManagerTest {
.setLeader("member-0")
.setProtocolType("consumer")
.setProtocol("range")
.setCurrentStateTimestamp(context.time.milliseconds()),
MetadataVersion.latestTesting());
.setCurrentStateTimestamp(context.time.milliseconds()));
context.replay(groupMetadataRecord);
context.groupMetadataManager.onLoaded();
@ -3817,8 +3814,7 @@ public class GroupMetadataManagerTest {
.setLeader("member-0")
.setProtocolType("consumer")
.setProtocol("range")
.setCurrentStateTimestamp(context.time.milliseconds()),
MetadataVersion.latestTesting());
.setCurrentStateTimestamp(context.time.milliseconds()));
context.replay(groupMetadataRecord);
context.groupMetadataManager.onLoaded();
@ -4716,7 +4712,7 @@ public class GroupMetadataManagerTest {
timeouts.forEach(timeout -> {
assertEquals(classicGroupHeartbeatKey("group-id", memberId), timeout.key);
assertEquals(
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, group.groupAssignment(), MetadataVersion.latestTesting())),
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, group.groupAssignment())),
timeout.result.records()
);
});
@ -5196,7 +5192,7 @@ public class GroupMetadataManagerTest {
);
assertEquals(
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, group.groupAssignment(), MetadataVersion.latestTesting())),
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, group.groupAssignment())),
joinResult.records
);
assertFalse(joinResult.joinFuture.isDone());
@ -5323,7 +5319,7 @@ public class GroupMetadataManagerTest {
);
assertEquals(
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, group.groupAssignment(), MetadataVersion.latestTesting())),
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, group.groupAssignment())),
joinResult.records
);
assertFalse(joinResult.joinFuture.isDone());
@ -5399,7 +5395,7 @@ public class GroupMetadataManagerTest {
supportSkippingAssignment);
assertEquals(
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, group.groupAssignment(), MetadataVersion.latestTesting())),
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, group.groupAssignment())),
joinResult.records
);
assertFalse(joinResult.joinFuture.isDone());
@ -5536,8 +5532,7 @@ public class GroupMetadataManagerTest {
.setLeader(null)
.setProtocolType("consumer")
.setProtocol(null)
.setCurrentStateTimestamp(context.time.milliseconds()),
MetadataVersion.latestTesting())
.setCurrentStateTimestamp(context.time.milliseconds()))
);
assertEquals(1, timeouts.size());
@ -5967,7 +5962,7 @@ public class GroupMetadataManagerTest {
);
assertEquals(
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, group.groupAssignment(), MetadataVersion.latestTesting())),
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, group.groupAssignment())),
joinResult.records
);
// Simulate a successful write to the log.
@ -6280,7 +6275,7 @@ public class GroupMetadataManagerTest {
);
assertEquals(
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, group.groupAssignment(), MetadataVersion.latestTesting())),
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, group.groupAssignment())),
followerJoinResult.records
);
// Simulate a failed write to the log.
@ -6337,7 +6332,7 @@ public class GroupMetadataManagerTest {
leaderSyncResult.appendFuture.complete(null);
assertEquals(
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, group.groupAssignment(), MetadataVersion.latestTesting())),
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, group.groupAssignment())),
leaderSyncResult.records
);
@ -6387,7 +6382,7 @@ public class GroupMetadataManagerTest {
);
assertEquals(
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, group.groupAssignment(), MetadataVersion.latestTesting())),
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, group.groupAssignment())),
followerJoinResult.records
);
@ -6599,7 +6594,7 @@ public class GroupMetadataManagerTest {
);
assertEquals(
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, group.groupAssignment(), MetadataVersion.latestTesting())),
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, group.groupAssignment())),
followerJoinResult.records
);
// Simulate a successful write to log.
@ -6806,7 +6801,7 @@ public class GroupMetadataManagerTest {
);
assertEquals(
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, group.groupAssignment(), MetadataVersion.latestTesting())),
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, group.groupAssignment())),
leaderJoinResult.records
);
// Simulate a successful write to log.
@ -7587,7 +7582,7 @@ public class GroupMetadataManagerTest {
assertEquals(
List.of(
GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, updatedAssignment, MetadataVersion.latestTesting())),
GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, updatedAssignment)),
syncResult.records
);
@ -8246,7 +8241,7 @@ public class GroupMetadataManagerTest {
ExpiredTimeout<Void, CoordinatorRecord> timeout = timeouts.get(0);
assertEquals(classicGroupSyncKey("group-id"), timeout.key);
assertEquals(
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, group.groupAssignment(), MetadataVersion.latestTesting())),
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, group.groupAssignment())),
timeout.result.records()
);
@ -8402,7 +8397,7 @@ public class GroupMetadataManagerTest {
if (response.memberId().equals(leaderId)) {
assertEquals(
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, group.groupAssignment(), MetadataVersion.latestTesting())),
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, group.groupAssignment())),
syncResult.records
);
@ -8497,8 +8492,7 @@ public class GroupMetadataManagerTest {
.setLeader(null)
.setProtocolType("classic")
.setProtocol("range")
.setCurrentStateTimestamp(context.time.milliseconds()),
MetadataVersion.latestTesting()));
.setCurrentStateTimestamp(context.time.milliseconds())));
// Create one share group record.
context.replay(GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(shareGroupId, 6));
context.commit();
@ -8792,8 +8786,7 @@ public class GroupMetadataManagerTest {
context.replay(GroupMetadataManagerTestContext.newGroupMetadataRecord(
"group-id",
groupMetadataValue,
MetadataVersion.latestTesting()
groupMetadataValue
));
context.verifyDescribeGroupsReturnsDeadGroup("group-id");
context.commit();
@ -8841,8 +8834,7 @@ public class GroupMetadataManagerTest {
context.replay(GroupMetadataManagerTestContext.newGroupMetadataRecord(
"group-id",
groupMetadataValue,
MetadataVersion.latestTesting()
groupMetadataValue
));
ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
context.groupMetadataManager.prepareRebalance(group, "trigger rebalance");
@ -9101,7 +9093,7 @@ public class GroupMetadataManagerTest {
))
);
assertEquals(
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, group.groupAssignment(), MetadataVersion.latestTesting())),
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, group.groupAssignment())),
leaveResult.records()
);
// Simulate a successful write to the log.
@ -9571,7 +9563,7 @@ public class GroupMetadataManagerTest {
// Even if there are more group metadata records loaded than tombstone records, the last replayed record
// (tombstone in this test) is the latest state of the group. Hence, the overall metric count should be 0.
IntStream.range(0, 5).forEach(__ ->
context.replay(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, Collections.emptyMap(), MetadataVersion.LATEST_PRODUCTION))
context.replay(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, Collections.emptyMap()))
);
IntStream.range(0, 4).forEach(__ ->
context.replay(GroupCoordinatorRecordHelpers.newGroupMetadataTombstoneRecord("group-id"))
@ -9638,7 +9630,7 @@ public class GroupMetadataManagerTest {
EMPTY,
context.time
);
context.replay(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(classicGroup, classicGroup.groupAssignment(), MetadataVersion.latestTesting()));
context.replay(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(classicGroup, classicGroup.groupAssignment()));
context.groupMetadataManager.getOrMaybeCreateClassicGroup(classicGroupId, false).transitionTo(PREPARING_REBALANCE);
assertThrows(GroupIdNotFoundException.class, () ->
@ -9666,7 +9658,7 @@ public class GroupMetadataManagerTest {
EMPTY,
context.time
);
context.replay(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(classicGroup, classicGroup.groupAssignment(), MetadataVersion.latestTesting()));
context.replay(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(classicGroup, classicGroup.groupAssignment()));
CoordinatorResult<ConsumerGroupHeartbeatResponseData, CoordinatorRecord> result = context.consumerGroupHeartbeat(
new ConsumerGroupHeartbeatRequestData()
@ -9810,7 +9802,7 @@ public class GroupMetadataManagerTest {
group.transitionTo(COMPLETING_REBALANCE);
group.transitionTo(STABLE);
context.replay(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, assignments, metadataImage.features().metadataVersion()));
context.replay(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, assignments));
context.commit();
group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(groupId, false);
@ -9996,7 +9988,7 @@ public class GroupMetadataManagerTest {
group.transitionTo(COMPLETING_REBALANCE);
group.transitionTo(STABLE);
context.replay(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, assignments, metadataImage.features().metadataVersion()));
context.replay(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, assignments));
context.commit();
group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(groupId, false);
@ -10201,7 +10193,7 @@ public class GroupMetadataManagerTest {
group.transitionTo(COMPLETING_REBALANCE);
group.transitionTo(STABLE);
context.replay(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, assignments, metadataImage.features().metadataVersion()));
context.replay(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, assignments));
context.commit();
group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(groupId, false);
@ -10282,7 +10274,7 @@ public class GroupMetadataManagerTest {
group.transitionTo(COMPLETING_REBALANCE);
group.transitionTo(STABLE);
context.replay(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, assignments, metadataImage.features().metadataVersion()));
context.replay(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, assignments));
context.commit();
// The static member rejoins with new protocol after a restart, triggering the upgrade.
@ -10765,7 +10757,7 @@ public class GroupMetadataManagerTest {
group.transitionTo(COMPLETING_REBALANCE);
group.transitionTo(STABLE);
context.replay(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, assignments, metadataImage.features().metadataVersion()));
context.replay(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, assignments));
context.commit();
group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(groupId, false);
@ -11156,7 +11148,7 @@ public class GroupMetadataManagerTest {
),
List.of(GroupCoordinatorRecordHelpers.newConsumerGroupSubscriptionMetadataTombstoneRecord(groupId)),
List.of(GroupCoordinatorRecordHelpers.newConsumerGroupEpochTombstoneRecord(groupId)),
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(expectedClassicGroup, assignments, MetadataVersion.latestTesting()))
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(expectedClassicGroup, assignments))
),
result.records()
);
@ -11343,7 +11335,7 @@ public class GroupMetadataManagerTest {
),
List.of(GroupCoordinatorRecordHelpers.newConsumerGroupSubscriptionMetadataTombstoneRecord(groupId)),
List.of(GroupCoordinatorRecordHelpers.newConsumerGroupEpochTombstoneRecord(groupId)),
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(expectedClassicGroup, assignments, MetadataVersion.latestTesting()))
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(expectedClassicGroup, assignments))
),
timeout.result.records()
);
@ -11548,7 +11540,7 @@ public class GroupMetadataManagerTest {
),
List.of(GroupCoordinatorRecordHelpers.newConsumerGroupSubscriptionMetadataTombstoneRecord(groupId)),
List.of(GroupCoordinatorRecordHelpers.newConsumerGroupEpochTombstoneRecord(groupId)),
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(expectedClassicGroup, assignments, MetadataVersion.latestTesting()))
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(expectedClassicGroup, assignments))
),
timeout.result.records()
);
@ -11783,7 +11775,7 @@ public class GroupMetadataManagerTest {
List.of(GroupCoordinatorRecordHelpers.newConsumerGroupEpochTombstoneRecord(groupId)),
// Create the classic group.
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(expectedClassicGroup, assignments, MetadataVersion.latestTesting()))
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(expectedClassicGroup, assignments))
),
result.records
);

View File

@ -95,7 +95,6 @@ import org.apache.kafka.coordinator.group.modern.share.ShareGroup;
import org.apache.kafka.coordinator.group.modern.share.ShareGroupBuilder;
import org.apache.kafka.image.MetadataImage;
import org.apache.kafka.server.common.ApiMessageAndVersion;
import org.apache.kafka.server.common.MetadataVersion;
import org.apache.kafka.timeline.SnapshotRegistry;
import java.net.InetAddress;
@ -227,8 +226,7 @@ public class GroupMetadataManagerTestContext {
public static CoordinatorRecord newGroupMetadataRecord(
String groupId,
GroupMetadataValue value,
MetadataVersion metadataVersion
GroupMetadataValue value
) {
return new CoordinatorRecord(
new ApiMessageAndVersion(
@ -238,7 +236,7 @@ public class GroupMetadataManagerTestContext {
),
new ApiMessageAndVersion(
value,
metadataVersion.groupMetadataValueVersion()
(short) 3
)
);
}
@ -850,7 +848,7 @@ public class GroupMetadataManagerTestContext {
.build());
assertEquals(
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, group.groupAssignment(), MetadataVersion.latestTesting())),
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, group.groupAssignment())),
syncResult.records
);
// Simulate a successful write to the log.
@ -1058,7 +1056,7 @@ public class GroupMetadataManagerTestContext {
));
assertEquals(
List.of(
GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, groupAssignment, MetadataVersion.latestTesting())),
GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, groupAssignment)),
leaderSyncResult.records
);
@ -1118,7 +1116,7 @@ public class GroupMetadataManagerTestContext {
// Now the group is stable, with the one member that joined above
assertEquals(
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, group.groupAssignment(), MetadataVersion.latestTesting())),
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, group.groupAssignment())),
syncResult.records
);
// Simulate a successful write to log.
@ -1156,7 +1154,7 @@ public class GroupMetadataManagerTestContext {
syncResult = sendClassicGroupSync(syncRequest.setGenerationId(nextGenerationId));
assertEquals(
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, group.groupAssignment(), MetadataVersion.latestTesting())),
List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, group.groupAssignment())),
syncResult.records
);
// Simulate a successful write to log.
@ -1219,8 +1217,7 @@ public class GroupMetadataManagerTestContext {
.setLeader(null)
.setProtocolType("consumer")
.setProtocol(null)
.setCurrentStateTimestamp(time.milliseconds()),
MetadataVersion.latestTesting()
.setCurrentStateTimestamp(time.milliseconds())
));

View File

@ -63,7 +63,6 @@ import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroup;
import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroupMember;
import org.apache.kafka.image.MetadataImage;
import org.apache.kafka.server.common.ApiMessageAndVersion;
import org.apache.kafka.server.common.MetadataVersion;
import org.apache.kafka.timeline.SnapshotRegistry;
import org.junit.jupiter.api.Test;
@ -450,8 +449,7 @@ public class OffsetMetadataManagerTest {
"metadata",
commitTimestamp,
OptionalLong.empty()
),
MetadataVersion.latestTesting()
)
));
}
@ -908,8 +906,7 @@ public class OffsetMetadataManagerTest {
"",
context.time.milliseconds(),
OptionalLong.of(context.time.milliseconds() + 1234L)
),
MetadataImage.EMPTY.features().metadataVersion()
)
)),
result.records()
);
@ -1016,8 +1013,7 @@ public class OffsetMetadataManagerTest {
"",
context.time.milliseconds(),
OptionalLong.empty()
),
MetadataImage.EMPTY.features().metadataVersion()
)
)),
result.records()
);
@ -1076,8 +1072,7 @@ public class OffsetMetadataManagerTest {
"",
context.time.milliseconds(),
OptionalLong.empty()
),
MetadataImage.EMPTY.features().metadataVersion()
)
)),
result.records()
);
@ -1240,8 +1235,7 @@ public class OffsetMetadataManagerTest {
"",
context.time.milliseconds(),
OptionalLong.empty()
),
MetadataImage.EMPTY.features().metadataVersion()
)
)),
result.records()
);
@ -1307,8 +1301,7 @@ public class OffsetMetadataManagerTest {
"metadata",
context.time.milliseconds(),
OptionalLong.empty()
),
MetadataImage.EMPTY.features().metadataVersion()
)
)),
result.records()
);
@ -1384,8 +1377,7 @@ public class OffsetMetadataManagerTest {
"small",
context.time.milliseconds(),
OptionalLong.empty()
),
MetadataImage.EMPTY.features().metadataVersion()
)
)),
result.records()
);
@ -1451,8 +1443,7 @@ public class OffsetMetadataManagerTest {
"metadata",
context.time.milliseconds(),
OptionalLong.empty()
),
MetadataImage.EMPTY.features().metadataVersion()
)
)),
result.records()
);
@ -1609,8 +1600,7 @@ public class OffsetMetadataManagerTest {
"metadata",
context.time.milliseconds(),
OptionalLong.empty()
),
MetadataImage.EMPTY.features().metadataVersion()
)
)),
result.records()
);
@ -3137,8 +3127,7 @@ public class OffsetMetadataManagerTest {
groupId,
topic,
partition,
offsetAndMetadata,
MetadataImage.EMPTY.features().metadataVersion()
offsetAndMetadata
));
assertEquals(offsetAndMetadata, context.offsetMetadataManager.offset(
@ -3160,8 +3149,7 @@ public class OffsetMetadataManagerTest {
groupId,
topic,
partition,
offsetAndMetadata,
MetadataImage.EMPTY.features().metadataVersion()
offsetAndMetadata
));
assertEquals(offsetAndMetadata, context.offsetMetadataManager.pendingTransactionalOffset(

View File

@ -29,7 +29,6 @@ import org.apache.kafka.common.record.RecordBatch;
import org.apache.kafka.common.record.Records;
import org.apache.kafka.common.utils.Exit;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.server.common.MetadataVersion;
import org.apache.kafka.server.common.RequestLocal;
import org.apache.kafka.server.storage.log.FetchIsolation;
import org.apache.kafka.server.util.MockTime;
@ -166,7 +165,6 @@ public class StressTestLog {
LogAppendInfo logAppendInfo = log.appendAsLeader(records,
0,
AppendOrigin.CLIENT,
MetadataVersion.LATEST_PRODUCTION,
RequestLocal.noCaching(),
VerificationGuard.SENTINEL);

View File

@ -32,7 +32,6 @@ import org.apache.kafka.common.utils.Exit;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.coordinator.transaction.TransactionLogConfig;
import org.apache.kafka.server.common.MetadataVersion;
import org.apache.kafka.server.common.RequestLocal;
import org.apache.kafka.server.util.CommandLineUtils;
import org.apache.kafka.server.util.KafkaScheduler;
@ -327,7 +326,6 @@ public class TestLinearWriteSpeed {
messages,
0,
AppendOrigin.CLIENT,
MetadataVersion.latestProduction(),
RequestLocal.noCaching(),
VerificationGuard.SENTINEL
);

View File

@ -124,8 +124,7 @@ public class PartitionMakeFollowerBenchmark {
Mockito.when(offsetCheckpoints.fetch(logDir.getAbsolutePath(), tp)).thenReturn(Optional.of(0L));
AlterPartitionListener alterPartitionListener = Mockito.mock(AlterPartitionListener.class);
AlterPartitionManager alterPartitionManager = Mockito.mock(AlterPartitionManager.class);
partition = new Partition(tp, 100,
MetadataVersion.latestTesting(), 0, () -> -1, Time.SYSTEM,
partition = new Partition(tp, 100, 0, () -> -1, Time.SYSTEM,
alterPartitionListener, delayedOperations,
Mockito.mock(MetadataCache.class), logManager, alterPartitionManager, topicId);
partition.createLogIfNotExists(true, false, offsetCheckpoints, topicId, Option.empty());

View File

@ -128,7 +128,7 @@ public class UpdateFollowerFetchStateBenchmark {
AlterPartitionListener alterPartitionListener = Mockito.mock(AlterPartitionListener.class);
AlterPartitionManager alterPartitionManager = Mockito.mock(AlterPartitionManager.class);
partition = new Partition(topicPartition, 100,
MetadataVersion.latestTesting(), 0, () -> -1, Time.SYSTEM,
0, () -> -1, Time.SYSTEM,
alterPartitionListener, delayedOperations,
Mockito.mock(MetadataCache.class), logManager, alterPartitionManager, topicId);
partition.makeLeader(partitionState, offsetCheckpoints, topicId, Option.empty());

View File

@ -23,7 +23,6 @@ import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.record.TimestampType;
import org.apache.kafka.common.utils.PrimitiveRef;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.server.common.MetadataVersion;
import org.apache.kafka.storage.internals.log.AppendOrigin;
import org.apache.kafka.storage.internals.log.LogValidator;
@ -55,8 +54,7 @@ public class CompressedRecordBatchValidationBenchmark extends BaseRecordBatchBen
MemoryRecords records = MemoryRecords.readableRecords(singleBatchBuffer.duplicate());
new LogValidator(records, new TopicPartition("a", 0),
Time.SYSTEM, compressionType, compression(), false, messageVersion,
TimestampType.CREATE_TIME, Long.MAX_VALUE, Long.MAX_VALUE, 0, AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
TimestampType.CREATE_TIME, Long.MAX_VALUE, Long.MAX_VALUE, 0, AppendOrigin.CLIENT
).validateMessagesAndAssignOffsetsCompressed(PrimitiveRef.ofLong(startingOffset),
validatorMetricsRecorder, requestLocal.bufferSupplier());
}

View File

@ -23,7 +23,6 @@ import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.record.TimestampType;
import org.apache.kafka.common.utils.PrimitiveRef;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.server.common.MetadataVersion;
import org.apache.kafka.storage.internals.log.AppendOrigin;
import org.apache.kafka.storage.internals.log.LogValidator;
@ -51,8 +50,7 @@ public class UncompressedRecordBatchValidationBenchmark extends BaseRecordBatchB
MemoryRecords records = MemoryRecords.readableRecords(singleBatchBuffer.duplicate());
new LogValidator(records, new TopicPartition("a", 0),
Time.SYSTEM, CompressionType.NONE, Compression.NONE, false,
messageVersion, TimestampType.CREATE_TIME, Long.MAX_VALUE, Long.MAX_VALUE, 0, AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
messageVersion, TimestampType.CREATE_TIME, Long.MAX_VALUE, Long.MAX_VALUE, 0, AppendOrigin.CLIENT
).assignOffsetsNonCompressed(PrimitiveRef.ofLong(startingOffset), validatorMetricsRecorder);
}
}

View File

@ -349,9 +349,9 @@ public class FeatureControlManagerTest {
public void testCanUseSafeDowngradeIfMetadataDidNotChange() {
FeatureControlManager manager = new FeatureControlManager.Builder().
setQuorumFeatures(features(MetadataVersion.FEATURE_NAME,
MetadataVersion.IBP_3_0_IV0.featureLevel(), MetadataVersion.IBP_3_3_IV1.featureLevel())).
MetadataVersion.IBP_3_0_IV1.featureLevel(), MetadataVersion.IBP_3_3_IV1.featureLevel())).
setMetadataVersion(MetadataVersion.IBP_3_1_IV0).
setMinimumBootstrapVersion(MetadataVersion.IBP_3_0_IV0).
setMinimumBootstrapVersion(MetadataVersion.IBP_3_0_IV1).
build();
assertEquals(ControllerResult.of(Collections.emptyList(), ApiError.NONE),
manager.updateFeatures(
@ -364,7 +364,7 @@ public class FeatureControlManagerTest {
public void testCannotDowngradeBefore3_3_IV0() {
FeatureControlManager manager = new FeatureControlManager.Builder().
setQuorumFeatures(features(MetadataVersion.FEATURE_NAME,
MetadataVersion.IBP_3_0_IV0.featureLevel(), MetadataVersion.IBP_3_3_IV3.featureLevel())).
MetadataVersion.IBP_3_0_IV1.featureLevel(), MetadataVersion.IBP_3_3_IV3.featureLevel())).
setMetadataVersion(MetadataVersion.IBP_3_3_IV0).
build();
assertEquals(ControllerResult.of(Collections.emptyList(), new ApiError(Errors.INVALID_UPDATE_VERSION,

View File

@ -718,7 +718,7 @@ public class QuorumControllerTest {
alterPartitionRequest.topics().add(topicData);
active.alterPartition(ANONYMOUS_CONTEXT, new AlterPartitionRequest
.Builder(alterPartitionRequest, false).build((short) 0).data()).get();
.Builder(alterPartitionRequest).build((short) 0).data()).get();
AtomicLong lastHeartbeatMs = new AtomicLong(getMonotonicMs(active.time()));
sendBrokerHeartbeatToUnfenceBrokers(active, allBrokers, brokerEpochs);

View File

@ -1886,7 +1886,7 @@ public class ReplicationControlManagerTest {
setNewIsrWithEpochs(isrWithDefaultEpoch(3, 0, 2, 1))))));
ControllerResult<AlterPartitionResponseData> alterPartitionResult = replication.alterPartition(
requestContext,
new AlterPartitionRequest.Builder(alterPartitionRequestData, version > 1).build(version).data());
new AlterPartitionRequest.Builder(alterPartitionRequestData).build(version).data());
Errors expectedError = version > 1 ? NEW_LEADER_ELECTED : FENCED_LEADER_EPOCH;
assertEquals(new AlterPartitionResponseData().setTopics(singletonList(
new AlterPartitionResponseData.TopicData().
@ -1950,7 +1950,7 @@ public class ReplicationControlManagerTest {
anonymousContextFor(ApiKeys.ALTER_PARTITION, version);
ControllerResult<AlterPartitionResponseData> alterPartitionResult =
replication.alterPartition(requestContext, new AlterPartitionRequest.Builder(alterIsrRequest, version > 1).build(version).data());
replication.alterPartition(requestContext, new AlterPartitionRequest.Builder(alterIsrRequest).build(version).data());
Errors expectedError = version <= 1 ? OPERATION_NOT_ATTEMPTED : INELIGIBLE_REPLICA;
assertEquals(
@ -2035,7 +2035,7 @@ public class ReplicationControlManagerTest {
anonymousContextFor(ApiKeys.ALTER_PARTITION, version);
ControllerResult<AlterPartitionResponseData> alterPartitionResult =
replication.alterPartition(requestContext, new AlterPartitionRequest.Builder(alterIsrRequest, version > 1).build(version).data());
replication.alterPartition(requestContext, new AlterPartitionRequest.Builder(alterIsrRequest).build(version).data());
// The late arrived AlterPartition request should be rejected when version >= 3.
if (version >= 3) {
@ -2100,7 +2100,7 @@ public class ReplicationControlManagerTest {
anonymousContextFor(ApiKeys.ALTER_PARTITION, version);
ControllerResult<AlterPartitionResponseData> alterPartitionResult =
replication.alterPartition(requestContext, new AlterPartitionRequest.Builder(alterIsrRequest, version > 1).build(version).data());
replication.alterPartition(requestContext, new AlterPartitionRequest.Builder(alterIsrRequest).build(version).data());
Errors expectedError = version <= 1 ? OPERATION_NOT_ATTEMPTED : INELIGIBLE_REPLICA;
assertEquals(
@ -2955,7 +2955,7 @@ public class ReplicationControlManagerTest {
setNewIsrWithEpochs(isrWithDefaultEpoch(0, 1, 2))))));
ControllerResult<AlterPartitionResponseData> alterPartitionResult = replication.alterPartition(
anonymousContextFor(ApiKeys.ALTER_PARTITION),
new AlterPartitionRequest.Builder(alterPartitionRequestData, true).build().data());
new AlterPartitionRequest.Builder(alterPartitionRequestData).build().data());
assertEquals(new AlterPartitionResponseData().setTopics(singletonList(
new AlterPartitionResponseData.TopicData().
setTopicId(topicId).
@ -3030,7 +3030,7 @@ public class ReplicationControlManagerTest {
setNewIsrWithEpochs(isrWithDefaultEpoch(0, 1, 2, 3, 4, 5))))));
ControllerResult<AlterPartitionResponseData> alterPartitionResultTwo = replication.alterPartition(
anonymousContextFor(ApiKeys.ALTER_PARTITION),
new AlterPartitionRequest.Builder(alterPartitionRequestDataTwo, true).build().data());
new AlterPartitionRequest.Builder(alterPartitionRequestDataTwo).build().data());
assertEquals(new AlterPartitionResponseData().setTopics(singletonList(
new AlterPartitionResponseData.TopicData().
setTopicId(topicId).

View File

@ -83,7 +83,7 @@ public class BootstrapDirectoryTest {
try (BootstrapTestDirectory testDirectory = new BootstrapTestDirectory().createDirectory()) {
assertEquals(BootstrapMetadata.fromVersion(MetadataVersion.MINIMUM_BOOTSTRAP_VERSION,
"the minimum version bootstrap with metadata.version 3.3-IV0"),
new BootstrapDirectory(testDirectory.path(), Optional.of("2.7")).read());
new BootstrapDirectory(testDirectory.path(), Optional.of("3.0")).read());
}
}

View File

@ -16,7 +16,6 @@
*/
package org.apache.kafka.server.common;
import org.apache.kafka.common.record.RecordVersion;
import java.util.Arrays;
import java.util.HashMap;
@ -45,103 +44,6 @@ import java.util.regex.Pattern;
*/
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
// LeaderAndIsrRequest 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(1, "3.0", "IV1", true),
@ -290,34 +192,6 @@ public enum MetadataVersion {
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 isAlterPartitionSupported() {
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 boolean isLeaderRecoverySupported() {
return this.isAtLeast(IBP_3_2_IV0);
}
@ -358,16 +232,6 @@ public enum MetadataVersion {
return this.featureLevel > 0;
}
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;
}
}
public boolean isBrokerRegistrationChangeRecordSupported() {
return this.isAtLeast(IBP_3_3_IV2);
}
@ -438,43 +302,8 @@ public enum MetadataVersion {
return 14;
} else if (this.isAtLeast(IBP_3_1_IV0)) {
return 13;
} else if (this.isAtLeast(IBP_2_7_IV1)) {
return 12;
} else if (this.isAtLeast(IBP_2_3_IV1)) {
return 11;
} else if (this.isAtLeast(IBP_2_1_IV2)) {
return 10;
} else if (this.isAtLeast(IBP_2_0_IV1)) {
return 8;
} else if (this.isAtLeast(IBP_1_1_IV0)) {
return 7;
} else if (this.isAtLeast(IBP_0_11_0_IV1)) {
return 5;
} else if (this.isAtLeast(IBP_0_11_0_IV0)) {
return 4;
} else if (this.isAtLeast(IBP_0_10_1_IV1)) {
return 3;
} else if (this.isAtLeast(IBP_0_10_0_IV0)) {
return 2;
} else if (this.isAtLeast(IBP_0_9_0)) {
return 1;
} else {
return 0;
}
}
public short offsetForLeaderEpochRequestVersion() {
if (this.isAtLeast(IBP_2_8_IV0)) {
return 4;
} else if (this.isAtLeast(IBP_2_3_IV1)) {
return 3;
} else if (this.isAtLeast(IBP_2_1_IV1)) {
return 2;
} else if (this.isAtLeast(IBP_2_0_IV0)) {
return 1;
} else {
return 0;
}
return 12;
}
public short listOffsetRequestVersion() {
@ -484,48 +313,8 @@ public enum MetadataVersion {
return 9;
} else if (this.isAtLeast(IBP_3_5_IV0)) {
return 8;
} else if (this.isAtLeast(IBP_3_0_IV1)) {
} else {
return 7;
} else if (this.isAtLeast(IBP_2_8_IV0)) {
return 6;
} else if (this.isAtLeast(IBP_2_2_IV1)) {
return 5;
} else if (this.isAtLeast(IBP_2_1_IV1)) {
return 4;
} else if (this.isAtLeast(IBP_2_0_IV1)) {
return 3;
} else if (this.isAtLeast(IBP_0_11_0_IV0)) {
return 2;
} else if (this.isAtLeast(IBP_0_10_1_IV2)) {
return 1;
} else {
return 0;
}
}
public short groupMetadataValueVersion() {
if (this.isLessThan(IBP_0_10_1_IV0)) {
return 0;
} else if (this.isLessThan(IBP_2_1_IV0)) {
return 1;
} else if (this.isLessThan(IBP_2_3_IV0)) {
return 2;
} else {
// Serialize with the highest supported non-flexible version
// until a tagged field is introduced or the version is bumped.
return 3;
}
}
public short offsetCommitValueVersion(boolean expireTimestampMs) {
if (isLessThan(MetadataVersion.IBP_2_1_IV0) || expireTimestampMs) {
return 1;
} else if (isLessThan(MetadataVersion.IBP_2_1_IV1)) {
return 2;
} else {
// Serialize with the highest supported non-flexible version
// until a tagged field is introduced or the version is bumped.
return 3;
}
}
@ -600,22 +389,6 @@ public enum MetadataVersion {
throw new IllegalArgumentException("No MetadataVersion with feature level " + 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);
}
}
// Testing only
public static MetadataVersion latestTesting() {
return VERSIONS[VERSIONS.length - 1];
@ -654,14 +427,6 @@ public enum MetadataVersion {
return version != lowVersion;
}
public short writeTxnMarkersRequestVersion() {
if (isAtLeast(IBP_2_8_IV0)) {
return 1;
} else {
return 0;
}
}
public boolean isAtLeast(MetadataVersion otherVersion) {
return this.compareTo(otherVersion) >= 0;
}

View File

@ -111,7 +111,7 @@ public class FeatureTest {
assertThrows(IllegalArgumentException.class,
() -> Feature.validateVersion(
TestFeatureVersion.TEST_1,
Collections.singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_2_8_IV0.featureLevel())
Collections.singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_0_IV1.featureLevel())
)
);

View File

@ -18,7 +18,6 @@
package org.apache.kafka.server.common;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.record.RecordVersion;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
@ -27,8 +26,6 @@ import org.junit.jupiter.params.provider.EnumSource;
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;
class MetadataVersionTest {
@ -50,102 +47,7 @@ class MetadataVersionTest {
@Test
@SuppressWarnings("checkstyle:JavaNCSS")
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"));
// 2.7-IV2 is the latest production version in the 2.7 line
assertEquals(IBP_2_7_IV2, MetadataVersion.fromVersionString("2.7"));
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"));
@ -199,44 +101,8 @@ class MetadataVersionTest {
assertEquals(IBP_4_0_IV3, MetadataVersion.fromVersionString("4.0-IV3"));
}
@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());
@ -266,31 +132,6 @@ class MetadataVersionTest {
@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());
@ -332,13 +173,12 @@ class MetadataVersionTest {
assertFalse(MetadataVersion.checkIfMetadataChanged(IBP_3_2_IV0, IBP_3_2_IV0));
assertTrue(MetadataVersion.checkIfMetadataChanged(IBP_3_2_IV0, IBP_3_1_IV0));
assertTrue(MetadataVersion.checkIfMetadataChanged(IBP_3_2_IV0, IBP_3_0_IV1));
assertTrue(MetadataVersion.checkIfMetadataChanged(IBP_3_2_IV0, IBP_3_0_IV0));
assertTrue(MetadataVersion.checkIfMetadataChanged(IBP_3_2_IV0, IBP_2_8_IV1));
assertTrue(MetadataVersion.checkIfMetadataChanged(IBP_3_2_IV0, IBP_3_0_IV1));
assertTrue(MetadataVersion.checkIfMetadataChanged(IBP_3_3_IV1, IBP_3_3_IV0));
// Check that argument order doesn't matter
assertTrue(MetadataVersion.checkIfMetadataChanged(IBP_3_0_IV0, IBP_3_2_IV0));
assertTrue(MetadataVersion.checkIfMetadataChanged(IBP_2_8_IV1, IBP_3_2_IV0));
assertTrue(MetadataVersion.checkIfMetadataChanged(IBP_3_1_IV0, IBP_3_2_IV0));
assertTrue(MetadataVersion.checkIfMetadataChanged(IBP_3_0_IV1, IBP_3_2_IV0));
}
@Test
@ -430,42 +270,6 @@ class MetadataVersionTest {
assertEquals(expectedVersion, metadataVersion.registerBrokerRecordVersion());
}
@ParameterizedTest
@EnumSource(value = MetadataVersion.class)
public void testGroupMetadataValueVersion(MetadataVersion metadataVersion) {
final short expectedVersion;
if (metadataVersion.isAtLeast(MetadataVersion.IBP_2_3_IV0)) {
expectedVersion = 3;
} else if (metadataVersion.isAtLeast(IBP_2_1_IV0)) {
expectedVersion = 2;
} else if (metadataVersion.isAtLeast(IBP_0_10_1_IV0)) {
expectedVersion = 1;
} else {
expectedVersion = 0;
}
assertEquals(expectedVersion, metadataVersion.groupMetadataValueVersion());
}
@ParameterizedTest
@EnumSource(value = MetadataVersion.class)
public void testOffsetCommitValueVersion(MetadataVersion metadataVersion) {
final short expectedVersion;
if (metadataVersion.isAtLeast(MetadataVersion.IBP_2_1_IV1)) {
expectedVersion = 3;
} else if (metadataVersion.isAtLeast(IBP_2_1_IV0)) {
expectedVersion = 2;
} else {
expectedVersion = 1;
}
assertEquals(expectedVersion, metadataVersion.offsetCommitValueVersion(false));
}
@ParameterizedTest
@EnumSource(value = MetadataVersion.class)
public void testOffsetCommitValueVersionWithExpiredTimestamp(MetadataVersion metadataVersion) {
assertEquals((short) 1, metadataVersion.offsetCommitValueVersion(true));
}
@Test
public void assertLatestProductionIsLessThanLatest() {
assertTrue(LATEST_PRODUCTION.ordinal() < MetadataVersion.latestTesting().ordinal(),
@ -483,8 +287,6 @@ class MetadataVersionTest {
MetadataVersion mv = MetadataVersion.latestProduction();
assertTrue(mv.listOffsetRequestVersion() <= ApiKeys.LIST_OFFSETS.latestVersion(false));
assertTrue(mv.fetchRequestVersion() <= ApiKeys.FETCH.latestVersion(false));
assertTrue(mv.offsetForLeaderEpochRequestVersion() <= ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion(false));
assertTrue(mv.writeTxnMarkersRequestVersion() <= ApiKeys.WRITE_TXN_MARKERS.latestVersion(false));
}
@Test

View File

@ -21,7 +21,6 @@ import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.errors.CorruptRecordException;
import org.apache.kafka.common.errors.InvalidTimestampException;
import org.apache.kafka.common.errors.UnsupportedCompressionTypeException;
import org.apache.kafka.common.errors.UnsupportedForMessageFormatException;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.record.AbstractRecords;
@ -40,7 +39,6 @@ import org.apache.kafka.common.utils.CloseableIterator;
import org.apache.kafka.common.utils.PrimitiveRef;
import org.apache.kafka.common.utils.PrimitiveRef.LongRef;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.server.common.MetadataVersion;
import java.nio.ByteBuffer;
import java.util.ArrayList;
@ -49,8 +47,6 @@ import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
import static org.apache.kafka.server.common.MetadataVersion.IBP_2_1_IV0;
public class LogValidator {
public interface MetricsRecorder {
@ -110,7 +106,6 @@ public class LogValidator {
private final long timestampAfterMaxMs;
private final int partitionLeaderEpoch;
private final AppendOrigin origin;
private final MetadataVersion interBrokerProtocolVersion;
public LogValidator(MemoryRecords records,
TopicPartition topicPartition,
@ -123,8 +118,7 @@ public class LogValidator {
long timestampBeforeMaxMs,
long timestampAfterMaxMs,
int partitionLeaderEpoch,
AppendOrigin origin,
MetadataVersion interBrokerProtocolVersion) {
AppendOrigin origin) {
this.records = records;
this.topicPartition = topicPartition;
this.time = time;
@ -137,7 +131,6 @@ public class LogValidator {
this.timestampAfterMaxMs = timestampAfterMaxMs;
this.partitionLeaderEpoch = partitionLeaderEpoch;
this.origin = origin;
this.interBrokerProtocolVersion = interBrokerProtocolVersion;
}
/**
@ -332,10 +325,6 @@ public class LogValidator {
public ValidationResult validateMessagesAndAssignOffsetsCompressed(LongRef offsetCounter,
MetricsRecorder metricsRecorder,
BufferSupplier bufferSupplier) {
if (targetCompression.type() == CompressionType.ZSTD && 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");
// No in place assignment situation 1
boolean inPlaceAssignment = sourceCompressionType == targetCompression.type();
long now = time.milliseconds();

View File

@ -21,7 +21,6 @@ import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.errors.CorruptRecordException;
import org.apache.kafka.common.errors.InvalidTimestampException;
import org.apache.kafka.common.errors.UnsupportedCompressionTypeException;
import org.apache.kafka.common.errors.UnsupportedForMessageFormatException;
import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.common.record.ControlRecordType;
@ -38,7 +37,6 @@ import org.apache.kafka.common.record.SimpleRecord;
import org.apache.kafka.common.record.TimestampType;
import org.apache.kafka.common.utils.PrimitiveRef;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.server.common.MetadataVersion;
import org.apache.kafka.server.common.RequestLocal;
import org.apache.kafka.server.util.MockTime;
import org.apache.kafka.storage.internals.log.LogValidator.ValidationResult;
@ -186,8 +184,7 @@ public class LogValidatorTest {
5000L,
5000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching().bufferSupplier()
));
@ -221,8 +218,7 @@ public class LogValidatorTest {
1000L,
1000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
);
LogValidator.ValidationResult validatedResults = validator.validateMessagesAndAssignOffsets(
@ -271,8 +267,7 @@ public class LogValidatorTest {
1000L,
1000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
);
LogValidator.ValidationResult validatedResults = logValidator.validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(0),
@ -355,8 +350,7 @@ public class LogValidatorTest {
1000L,
1000L,
partitionLeaderEpoch,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(0L),
metricsRecorder,
@ -461,8 +455,7 @@ public class LogValidatorTest {
1000L,
1000L,
RecordBatch.NO_PRODUCER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.IBP_2_3_IV1
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(0L), metricsRecorder, RequestLocal.withThreadConfinedCaching().bufferSupplier());
}
@ -547,8 +540,7 @@ public class LogValidatorTest {
1000L,
1000L,
partitionLeaderEpoch,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
);
LogValidator.ValidationResult validatedResults = validator.validateMessagesAndAssignOffsets(
@ -636,8 +628,7 @@ public class LogValidatorTest {
1000L,
1000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(0),
metricsRecorder,
@ -668,8 +659,7 @@ public class LogValidatorTest {
1000L,
1000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(0),
metricsRecorder,
@ -700,8 +690,7 @@ public class LogValidatorTest {
1000L,
1000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(0),
metricsRecorder,
@ -743,8 +732,7 @@ public class LogValidatorTest {
1000L,
1000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
);
@ -797,8 +785,7 @@ public class LogValidatorTest {
1000L,
1000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
);
@ -835,8 +822,7 @@ public class LogValidatorTest {
1000L,
1000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(0),
metricsRecorder,
@ -869,8 +855,7 @@ public class LogValidatorTest {
1000L,
1000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(0),
metricsRecorder,
@ -899,8 +884,7 @@ public class LogValidatorTest {
1000L,
1000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(offset),
metricsRecorder,
@ -931,8 +915,7 @@ public class LogValidatorTest {
1000L,
1000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(offset),
metricsRecorder,
@ -962,8 +945,7 @@ public class LogValidatorTest {
5000L,
5000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(offset),
metricsRecorder,
@ -993,8 +975,7 @@ public class LogValidatorTest {
5000L,
5000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(offset),
metricsRecorder,
@ -1025,8 +1006,7 @@ public class LogValidatorTest {
5000L,
5000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(offset),
metricsRecorder,
@ -1057,8 +1037,7 @@ public class LogValidatorTest {
5000L,
5000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(offset),
metricsRecorder,
@ -1087,8 +1066,7 @@ public class LogValidatorTest {
1000L,
1000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(offset),
metricsRecorder,
@ -1123,8 +1101,7 @@ public class LogValidatorTest {
1000L,
1000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(offset),
metricsRecorder,
@ -1160,8 +1137,7 @@ public class LogValidatorTest {
1000L,
1000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(offset),
metricsRecorder,
@ -1197,8 +1173,7 @@ public class LogValidatorTest {
1000L,
1000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(offset),
metricsRecorder,
@ -1231,8 +1206,7 @@ public class LogValidatorTest {
5000L,
5000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(offset),
metricsRecorder,
@ -1257,8 +1231,7 @@ public class LogValidatorTest {
5000L,
5000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.COORDINATOR,
MetadataVersion.latestTesting()
AppendOrigin.COORDINATOR
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(offset),
metricsRecorder,
@ -1287,8 +1260,7 @@ public class LogValidatorTest {
5000L,
5000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(offset),
metricsRecorder,
@ -1315,8 +1287,7 @@ public class LogValidatorTest {
5000L,
5000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching().bufferSupplier()
).validatedRecords, offset);
@ -1339,8 +1310,7 @@ public class LogValidatorTest {
1000L,
1000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching().bufferSupplier()
).validatedRecords, offset);
@ -1364,8 +1334,7 @@ public class LogValidatorTest {
1000L,
1000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching().bufferSupplier()
).validatedRecords, offset);
@ -1389,8 +1358,7 @@ public class LogValidatorTest {
5000L,
5000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching().bufferSupplier()
).validatedRecords, offset);
@ -1415,8 +1383,7 @@ public class LogValidatorTest {
5000L,
5000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching().bufferSupplier()
).validatedRecords, offset);
@ -1442,8 +1409,7 @@ public class LogValidatorTest {
5000L,
5000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching().bufferSupplier()
));
@ -1470,8 +1436,7 @@ public class LogValidatorTest {
5000L,
5000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching().bufferSupplier()
));
@ -1495,8 +1460,7 @@ public class LogValidatorTest {
5000L,
5000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching().bufferSupplier()
).validatedRecords, offset);
@ -1522,8 +1486,7 @@ public class LogValidatorTest {
5000L,
5000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching().bufferSupplier()
).validatedRecords, offset);
@ -1551,8 +1514,7 @@ public class LogValidatorTest {
5000L,
5000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(0L), metricsRecorder, RequestLocal.withThreadConfinedCaching().bufferSupplier()
));
@ -1560,31 +1522,6 @@ public class LogValidatorTest {
assertEquals(metricsRecorder.recordInvalidOffsetCount, 1);
}
@Test
public void testZStdCompressedWithUnavailableIBPVersion() {
// The timestamps should be overwritten
MemoryRecords records = createRecords(RecordBatch.MAGIC_VALUE_V2, 1234L, Compression.NONE);
assertThrows(UnsupportedCompressionTypeException.class, () ->
new LogValidator(
records,
topicPartition,
time,
CompressionType.NONE,
Compression.zstd().build(),
false,
RecordBatch.MAGIC_VALUE_V2,
TimestampType.LOG_APPEND_TIME,
1000L,
1000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.IBP_2_0_IV1
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(0L), metricsRecorder, RequestLocal.withThreadConfinedCaching().bufferSupplier()
)
);
}
@Test
public void testInvalidTimestampExceptionHasBatchIndex() {
long now = System.currentTimeMillis();
@ -1604,8 +1541,7 @@ public class LogValidatorTest {
1000L,
1000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(0L), metricsRecorder, RequestLocal.withThreadConfinedCaching().bufferSupplier()
)
@ -1691,8 +1627,7 @@ public class LogValidatorTest {
timestampBeforeMaxConfig,
timestampAfterMaxConfig,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(0L), metricsRecorder, RequestLocal.withThreadConfinedCaching().bufferSupplier()
)
@ -1724,8 +1659,7 @@ public class LogValidatorTest {
timestampBeforeMaxConfig,
timestampAfterMaxConfig,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(0L), metricsRecorder, RequestLocal.withThreadConfinedCaching().bufferSupplier()
)
@ -1766,8 +1700,7 @@ public class LogValidatorTest {
5000L,
5000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
);
LogValidator.ValidationResult result = validator.validateMessagesAndAssignOffsets(
@ -1805,8 +1738,7 @@ public class LogValidatorTest {
5000L,
5000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
);
LogValidator.ValidationResult result = validator.validateMessagesAndAssignOffsets(
@ -1865,8 +1797,7 @@ public class LogValidatorTest {
1000L,
1000L,
partitionLeaderEpoch,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
offsetCounter,
metricsRecorder,
@ -1934,8 +1865,7 @@ public class LogValidatorTest {
1000L,
1000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(0L),
metricsRecorder,
@ -1961,8 +1891,7 @@ public class LogValidatorTest {
1000L,
1000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(0),
metricsRecorder,
@ -2006,8 +1935,7 @@ public class LogValidatorTest {
1000L,
1000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(0),
metricsRecorder,
@ -2049,8 +1977,7 @@ public class LogValidatorTest {
1000L,
1000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting()
AppendOrigin.CLIENT
).validateMessagesAndAssignOffsets(
offsetCounter,
metricsRecorder,

View File

@ -65,7 +65,7 @@ public class ClusterConfigTest {
.setControllerSecurityProtocol(SecurityProtocol.SASL_PLAINTEXT)
.setControllerListenerName(ListenerName.normalised("CONTROLLER"))
.setTrustStoreFile(trustStoreFile)
.setMetadataVersion(MetadataVersion.IBP_0_8_0)
.setMetadataVersion(MetadataVersion.IBP_3_0_IV1)
.setServerProperties(Collections.singletonMap("broker", "broker_value"))
.setConsumerProperties(Collections.singletonMap("consumer", "consumer_value"))
.setProducerProperties(Collections.singletonMap("producer", "producer_value"))

View File

@ -196,7 +196,7 @@ public class FeatureCommand {
);
versionMappingParser.addArgument("--release-version")
.help("The release version to use for the corresponding feature mapping. The minimum is " +
MetadataVersion.IBP_3_0_IV0 + "; the default is " + MetadataVersion.LATEST_PRODUCTION)
MetadataVersion.IBP_3_0_IV1 + "; the default is " + MetadataVersion.LATEST_PRODUCTION)
.action(store());
}