KAFKA-18269: Remove deprecated protocol APIs support (KIP-896, KIP-724) (#18291)

Included in this change:
1. Remove deprecated protocol api versions from json files.
3. Remove fields that are no longer used from json files (affects ListOffsets, OffsetCommit, DescribeConfigs).
4. Remove record down-conversion support from KafkaApis.
5. No longer return `Errors.UNSUPPORTED_COMPRESSION_TYPE` on the fetch path[1].
6. Deprecate `TopicConfig. MESSAGE_DOWNCONVERSION_ENABLE_CONFIG` and made the relevant
configs (`message.downconversion.enable` and `log.message.downcoversion.enable`) no-ops since
down-conversion is no longer supported. It was an oversight not to deprecate this via KIP-724.
7. Fix `shouldRetainsBufferReference` to handle null request schemas for a given version.
8. Simplify producer logic since it only supports the v2 record format now.
9. Fix tests so they don't exercise protocol api versions that have been removed.
10. Add upgrade note.

Testing:
1. System tests have a lot of failures, but those tests fail for trunk too and I didn't see any issues specific to this change - it's hard to be sure given the number of failing tests, but let's not block on that given the other testing that has been done (see below).
3. Java producers and consumers with version 0.9-0.10.1 don't have api versions support and hence they fail in an ungraceful manner: the broker disconnects and the clients reconnect until the relevant timeout is triggered.
4. Same thing seems to happen for the console producer 0.10.2 although it's unclear why since api versions should be supported. I will look into this separately, it's unlikely to be related to this PR.
5. Console consumer 0.10.2 fails with the expected error and a reasonable message[2].
6. Console producer and consumer 0.11.0 works fine, newer versions should naturally also work fine.
7. kcat 1.5.0 (based on librdkafka 1.1.0) produce and consume fail with a reasonable message[3][4].
8. kcat 1.6.0-1.7.0 (based on librdkafka 1.5.0 and 1.7.0 respectively) consume fails with a reasonable message[5].
9. kcat 1.6.0-1.7.0 produce works fine.
10. kcat 1.7.1  (based on librdkafka 1.8.2) works fine for consumer and produce.
11. confluent-go-client (librdkafka based) 1.8.2 works fine for consumer and produce.
12. I will test more clients, but I don't think we need to block the PR on that.

Note that this also completes part of KIP-724: produce v2 and lower as well as fetch v3 and lower are no longer supported.

Future PRs will remove conditional code that is no longer needed (some of that has been done in KafkaApis,
but only what was required due to the schema changes). We can probably do that in master only as it does
not change behavior.

Note that I did not touch `ignorable` fields even though some of them could have been
changed. The reasoning is that this could result in incompatible changes for clients
that use new protocol versions without setting such fields _if_ we don't manually
validate their presence. I will file a JIRA ticket to look into this carefully for each
case (i.e. if we do validate their presence for the appropriate versions, we can
set them to ignorable=false in the json file).

[1] We would return this error if a fetch < v10 was used and the compression topic config was set
to zstd, but we would not do the same for the case where zstd was compressed at the producer
level (the most common case). Since there is no efficient way to do the check for the common
case, I made it consistent for both by having no checks.
[2] ```org.apache.kafka.common.errors.UnsupportedVersionException: The broker is too new to support JOIN_GROUP version 1```
[3]```METADATA|rdkafka#producer-1| [thrd:main]: localhost:9092/bootstrap: Metadata request failed: connected: Local: Required feature not supported by broker (0ms): Permanent```
[4]```METADATA|rdkafka#consumer-1| [thrd:main]: localhost:9092/bootstrap: Metadata request failed: connected: Local: Required feature not supported by broker (0ms): Permanent```
[5] `ERROR: Topic test-topic [0] error: Failed to query logical offset END: Local: Required feature not supported by broker`

Cherry-pick was clean, minor changes required to LogOffsetTest due to a commit that is in master, but not 4.0.

Reviewers: David Arthur <mumrah@gmail.com>
This commit is contained in:
Ismael Juma 2024-12-21 00:08:33 -08:00 committed by GitHub
parent a001a96218
commit a0f9e319c7
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
115 changed files with 474 additions and 1994 deletions

View File

@ -16,13 +16,8 @@
*/ */
package org.apache.kafka.clients; package org.apache.kafka.clients;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.record.RecordBatch;
import org.apache.kafka.common.requests.ProduceRequest;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import java.util.Optional;
/** /**
* Maintains node api versions for access outside of NetworkClient (which is where the information is derived). * Maintains node api versions for access outside of NetworkClient (which is where the information is derived).
@ -33,7 +28,6 @@ import java.util.Optional;
public class ApiVersions { public class ApiVersions {
private final Map<String, NodeApiVersions> nodeApiVersions = new HashMap<>(); private final Map<String, NodeApiVersions> nodeApiVersions = new HashMap<>();
private byte maxUsableProduceMagic = RecordBatch.CURRENT_MAGIC_VALUE;
// The maximum finalized feature epoch of all the node api versions. // The maximum finalized feature epoch of all the node api versions.
private long maxFinalizedFeaturesEpoch = -1; private long maxFinalizedFeaturesEpoch = -1;
@ -50,7 +44,6 @@ public class ApiVersions {
public synchronized void update(String nodeId, NodeApiVersions nodeApiVersions) { public synchronized void update(String nodeId, NodeApiVersions nodeApiVersions) {
this.nodeApiVersions.put(nodeId, nodeApiVersions); this.nodeApiVersions.put(nodeId, nodeApiVersions);
this.maxUsableProduceMagic = computeMaxUsableProduceMagic();
if (maxFinalizedFeaturesEpoch < nodeApiVersions.finalizedFeaturesEpoch()) { if (maxFinalizedFeaturesEpoch < nodeApiVersions.finalizedFeaturesEpoch()) {
this.maxFinalizedFeaturesEpoch = nodeApiVersions.finalizedFeaturesEpoch(); this.maxFinalizedFeaturesEpoch = nodeApiVersions.finalizedFeaturesEpoch();
this.finalizedFeatures = nodeApiVersions.finalizedFeatures(); this.finalizedFeatures = nodeApiVersions.finalizedFeatures();
@ -59,7 +52,6 @@ public class ApiVersions {
public synchronized void remove(String nodeId) { public synchronized void remove(String nodeId) {
this.nodeApiVersions.remove(nodeId); this.nodeApiVersions.remove(nodeId);
this.maxUsableProduceMagic = computeMaxUsableProduceMagic();
} }
public synchronized NodeApiVersions get(String nodeId) { public synchronized NodeApiVersions get(String nodeId) {
@ -74,19 +66,4 @@ public class ApiVersions {
return new FinalizedFeaturesInfo(maxFinalizedFeaturesEpoch, finalizedFeatures); return new FinalizedFeaturesInfo(maxFinalizedFeaturesEpoch, finalizedFeatures);
} }
private byte computeMaxUsableProduceMagic() {
// use a magic version which is supported by all brokers to reduce the chance that
// we will need to convert the messages when they are ready to be sent.
Optional<Byte> knownBrokerNodesMinRequiredMagicForProduce = this.nodeApiVersions.values().stream()
.filter(versions -> versions.apiVersion(ApiKeys.PRODUCE) != null) // filter out Raft controller nodes
.map(versions -> ProduceRequest.requiredMagicForVersion(versions.latestUsableVersion(ApiKeys.PRODUCE)))
.min(Byte::compare);
return (byte) Math.min(RecordBatch.CURRENT_MAGIC_VALUE,
knownBrokerNodesMinRequiredMagicForProduce.orElse(RecordBatch.CURRENT_MAGIC_VALUE));
}
public synchronized byte maxUsableProduceMagic() {
return maxUsableProduceMagic;
}
} }

View File

@ -111,33 +111,15 @@ class OffsetFetcherUtils {
Errors error = Errors.forCode(partition.errorCode()); Errors error = Errors.forCode(partition.errorCode());
switch (error) { switch (error) {
case NONE: case NONE:
if (!partition.oldStyleOffsets().isEmpty()) { log.debug("Handling ListOffsetResponse response for {}. Fetched offset {}, timestamp {}",
// Handle v0 response with offsets topicPartition, partition.offset(), partition.timestamp());
long offset; if (partition.offset() != ListOffsetsResponse.UNKNOWN_OFFSET) {
if (partition.oldStyleOffsets().size() > 1) { Optional<Integer> leaderEpoch = (partition.leaderEpoch() == ListOffsetsResponse.UNKNOWN_EPOCH)
throw new IllegalStateException("Unexpected partitionData response of length " + ? Optional.empty()
partition.oldStyleOffsets().size()); : Optional.of(partition.leaderEpoch());
} else { OffsetFetcherUtils.ListOffsetData offsetData = new OffsetFetcherUtils.ListOffsetData(partition.offset(), partition.timestamp(),
offset = partition.oldStyleOffsets().get(0); leaderEpoch);
} fetchedOffsets.put(topicPartition, offsetData);
log.debug("Handling v0 ListOffsetResponse response for {}. Fetched offset {}",
topicPartition, offset);
if (offset != ListOffsetsResponse.UNKNOWN_OFFSET) {
OffsetFetcherUtils.ListOffsetData offsetData = new OffsetFetcherUtils.ListOffsetData(offset, null, Optional.empty());
fetchedOffsets.put(topicPartition, offsetData);
}
} else {
// Handle v1 and later response or v0 without offsets
log.debug("Handling ListOffsetResponse response for {}. Fetched offset {}, timestamp {}",
topicPartition, partition.offset(), partition.timestamp());
if (partition.offset() != ListOffsetsResponse.UNKNOWN_OFFSET) {
Optional<Integer> leaderEpoch = (partition.leaderEpoch() == ListOffsetsResponse.UNKNOWN_EPOCH)
? Optional.empty()
: Optional.of(partition.leaderEpoch());
OffsetFetcherUtils.ListOffsetData offsetData = new OffsetFetcherUtils.ListOffsetData(partition.offset(), partition.timestamp(),
leaderEpoch);
fetchedOffsets.put(topicPartition, offsetData);
}
} }
break; break;
case UNSUPPORTED_FOR_MESSAGE_FORMAT: case UNSUPPORTED_FOR_MESSAGE_FORMAT:
@ -458,4 +440,4 @@ class OffsetFetcherUtils {
this.leaderEpoch = leaderEpoch; this.leaderEpoch = leaderEpoch;
} }
} }
} }

View File

@ -1004,7 +1004,7 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
setReadOnly(record.headers()); setReadOnly(record.headers());
Header[] headers = record.headers().toArray(); Header[] headers = record.headers().toArray();
int serializedSize = AbstractRecords.estimateSizeInBytesUpperBound(apiVersions.maxUsableProduceMagic(), int serializedSize = AbstractRecords.estimateSizeInBytesUpperBound(RecordBatch.CURRENT_MAGIC_VALUE,
compression.type(), serializedKey, serializedValue, headers); compression.type(), serializedKey, serializedValue, headers);
ensureValidRecordSize(serializedSize); ensureValidRecordSize(serializedSize);
long timestamp = record.timestamp() == null ? nowMs : record.timestamp(); long timestamp = record.timestamp() == null ? nowMs : record.timestamp();

View File

@ -27,7 +27,6 @@ import org.apache.kafka.common.Node;
import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.compress.Compression; import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.header.Header; import org.apache.kafka.common.header.Header;
import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.record.AbstractRecords; import org.apache.kafka.common.record.AbstractRecords;
@ -344,8 +343,8 @@ public class RecordAccumulator {
} }
if (buffer == null) { if (buffer == null) {
byte maxUsableMagic = apiVersions.maxUsableProduceMagic(); int size = Math.max(this.batchSize, AbstractRecords.estimateSizeInBytesUpperBound(
int size = Math.max(this.batchSize, AbstractRecords.estimateSizeInBytesUpperBound(maxUsableMagic, compression.type(), key, value, headers)); RecordBatch.CURRENT_MAGIC_VALUE, compression.type(), key, value, headers));
log.trace("Allocating a new {} byte message buffer for topic {} partition {} with remaining timeout {}ms", size, topic, effectivePartition, maxTimeToBlock); log.trace("Allocating a new {} byte message buffer for topic {} partition {} with remaining timeout {}ms", size, topic, effectivePartition, maxTimeToBlock);
// This call may block if we exhausted buffer space. // This call may block if we exhausted buffer space.
buffer = free.allocate(size, maxTimeToBlock); buffer = free.allocate(size, maxTimeToBlock);
@ -408,7 +407,7 @@ public class RecordAccumulator {
return appendResult; return appendResult;
} }
MemoryRecordsBuilder recordsBuilder = recordsBuilder(buffer, apiVersions.maxUsableProduceMagic()); MemoryRecordsBuilder recordsBuilder = recordsBuilder(buffer);
ProducerBatch batch = new ProducerBatch(new TopicPartition(topic, partition), recordsBuilder, nowMs); ProducerBatch batch = new ProducerBatch(new TopicPartition(topic, partition), recordsBuilder, nowMs);
FutureRecordMetadata future = Objects.requireNonNull(batch.tryAppend(timestamp, key, value, headers, FutureRecordMetadata future = Objects.requireNonNull(batch.tryAppend(timestamp, key, value, headers,
callbacks, nowMs)); callbacks, nowMs));
@ -419,12 +418,8 @@ public class RecordAccumulator {
return new RecordAppendResult(future, dq.size() > 1 || batch.isFull(), true, false, batch.estimatedSizeInBytes()); return new RecordAppendResult(future, dq.size() > 1 || batch.isFull(), true, false, batch.estimatedSizeInBytes());
} }
private MemoryRecordsBuilder recordsBuilder(ByteBuffer buffer, byte maxUsableMagic) { private MemoryRecordsBuilder recordsBuilder(ByteBuffer buffer) {
if (transactionManager != null && maxUsableMagic < RecordBatch.MAGIC_VALUE_V2) { return MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compression, TimestampType.CREATE_TIME, 0L);
throw new UnsupportedVersionException("Attempting to use idempotence with a broker which does not " +
"support the required message format (v2). The broker must be version 0.11 or later.");
}
return MemoryRecords.builder(buffer, maxUsableMagic, compression, TimestampType.CREATE_TIME, 0L);
} }
/** /**

View File

@ -871,27 +871,10 @@ public class Sender implements Runnable {
return; return;
final Map<TopicPartition, ProducerBatch> recordsByPartition = new HashMap<>(batches.size()); final Map<TopicPartition, ProducerBatch> recordsByPartition = new HashMap<>(batches.size());
// find the minimum magic version used when creating the record sets
byte minUsedMagic = apiVersions.maxUsableProduceMagic();
for (ProducerBatch batch : batches) {
if (batch.magic() < minUsedMagic)
minUsedMagic = batch.magic();
}
ProduceRequestData.TopicProduceDataCollection tpd = new ProduceRequestData.TopicProduceDataCollection(); ProduceRequestData.TopicProduceDataCollection tpd = new ProduceRequestData.TopicProduceDataCollection();
for (ProducerBatch batch : batches) { for (ProducerBatch batch : batches) {
TopicPartition tp = batch.topicPartition; TopicPartition tp = batch.topicPartition;
MemoryRecords records = batch.records(); MemoryRecords records = batch.records();
// down convert if necessary to the minimum magic used. In general, there can be a delay between the time
// that the producer starts building the batch and the time that we send the request, and we may have
// chosen the message format based on out-dated metadata. In the worst case, we optimistically chose to use
// the new message format, but found that the broker didn't support it, so we need to down-convert on the
// client before sending. This is intended to handle edge cases around cluster upgrades where brokers may
// not all support the same message format version. For example, if a partition migrates from a broker
// which is supporting the new magic version to one which doesn't, then we will need to convert.
if (!records.hasMatchingMagic(minUsedMagic))
records = batch.records().downConvert(minUsedMagic, 0, time).records();
ProduceRequestData.TopicProduceData tpData = tpd.find(tp.topic()); ProduceRequestData.TopicProduceData tpData = tpd.find(tp.topic());
if (tpData == null) { if (tpData == null) {
tpData = new ProduceRequestData.TopicProduceData().setName(tp.topic()); tpData = new ProduceRequestData.TopicProduceData().setName(tp.topic());
@ -904,18 +887,13 @@ public class Sender implements Runnable {
} }
String transactionalId = null; String transactionalId = null;
// When we use transaction V1 protocol in transaction we set the request version upper limit to
// LAST_STABLE_VERSION_BEFORE_TRANSACTION_V2 so that the broker knows that we're using transaction protocol V1.
boolean useTransactionV1Version = false; boolean useTransactionV1Version = false;
if (transactionManager != null && transactionManager.isTransactional()) { if (transactionManager != null && transactionManager.isTransactional()) {
transactionalId = transactionManager.transactionalId(); transactionalId = transactionManager.transactionalId();
if (!transactionManager.isTransactionV2Enabled()) { useTransactionV1Version = !transactionManager.isTransactionV2Enabled();
useTransactionV1Version = true;
}
} }
ProduceRequest.Builder requestBuilder = ProduceRequest.forMagic(minUsedMagic, ProduceRequest.Builder requestBuilder = ProduceRequest.builder(
new ProduceRequestData() new ProduceRequestData()
.setAcks(acks) .setAcks(acks)
.setTimeoutMs(timeout) .setTimeoutMs(timeout)

View File

@ -234,10 +234,18 @@ public class TopicConfig {
"or equal to the broker's timestamp, with the maximum allowable difference determined by the value set in this " + "or equal to the broker's timestamp, with the maximum allowable difference determined by the value set in this " +
"configuration. If message.timestamp.type=CreateTime, the message will be rejected if the difference in " + "configuration. If message.timestamp.type=CreateTime, the message will be rejected if the difference in " +
"timestamps exceeds this specified threshold. This configuration is ignored if message.timestamp.type=LogAppendTime."; "timestamps exceeds this specified threshold. This configuration is ignored if message.timestamp.type=LogAppendTime.";
/**
* @deprecated down-conversion is not possible in Apache Kafka 4.0 and newer, hence this configuration is a no-op,
* and it is deprecated for removal in Apache Kafka 5.0.
*/
@Deprecated
public static final String MESSAGE_DOWNCONVERSION_ENABLE_CONFIG = "message.downconversion.enable"; public static final String MESSAGE_DOWNCONVERSION_ENABLE_CONFIG = "message.downconversion.enable";
public static final String MESSAGE_DOWNCONVERSION_ENABLE_DOC = "This configuration controls whether " +
"down-conversion of message formats is enabled to satisfy consume requests. When set to <code>false</code>, " + /**
"broker will not perform down-conversion for consumers expecting an older message format. The broker responds " + * @deprecated see {@link #MESSAGE_DOWNCONVERSION_ENABLE_CONFIG}.
"with <code>UNSUPPORTED_VERSION</code> error for consume requests from such older clients. This configuration" + */
"does not apply to any message format conversion that might be required for replication to followers."; @Deprecated
public static final String MESSAGE_DOWNCONVERSION_ENABLE_DOC = "Down-conversion is not possible in Apache Kafka 4.0 and newer, " +
"hence this configuration is no-op and it is deprecated for removal in Apache Kafka 5.0.";
} }

View File

@ -194,7 +194,7 @@ public enum ApiKeys {
private static boolean shouldRetainsBufferReference(Schema[] requestSchemas) { private static boolean shouldRetainsBufferReference(Schema[] requestSchemas) {
boolean requestRetainsBufferReference = false; boolean requestRetainsBufferReference = false;
for (Schema requestVersionSchema : requestSchemas) { for (Schema requestVersionSchema : requestSchemas) {
if (retainsBufferReference(requestVersionSchema)) { if (requestVersionSchema != null && retainsBufferReference(requestVersionSchema)) {
requestRetainsBufferReference = true; requestRetainsBufferReference = true;
break; break;
} }

View File

@ -224,29 +224,6 @@ public class DescribeConfigsResponse extends AbstractResponse {
this.data = data; this.data = data;
} }
// This constructor should only be used after deserialization, it has special handling for version 0
private DescribeConfigsResponse(DescribeConfigsResponseData data, short version) {
super(ApiKeys.DESCRIBE_CONFIGS);
this.data = data;
if (version == 0) {
for (DescribeConfigsResponseData.DescribeConfigsResult result : data.results()) {
for (DescribeConfigsResponseData.DescribeConfigsResourceResult config : result.configs()) {
if (config.isDefault()) {
config.setConfigSource(ConfigSource.DEFAULT_CONFIG.id);
} else {
if (result.resourceType() == ConfigResource.Type.BROKER.id()) {
config.setConfigSource(ConfigSource.STATIC_BROKER_CONFIG.id);
} else if (result.resourceType() == ConfigResource.Type.TOPIC.id()) {
config.setConfigSource(ConfigSource.TOPIC_CONFIG.id);
} else {
config.setConfigSource(ConfigSource.UNKNOWN.id);
}
}
}
}
}
}
@Override @Override
public DescribeConfigsResponseData data() { public DescribeConfigsResponseData data() {
return data; return data;
@ -272,7 +249,7 @@ public class DescribeConfigsResponse extends AbstractResponse {
} }
public static DescribeConfigsResponse parse(ByteBuffer buffer, short version) { public static DescribeConfigsResponse parse(ByteBuffer buffer, short version) {
return new DescribeConfigsResponse(new DescribeConfigsResponseData(new ByteBufferAccessor(buffer), version), version); return new DescribeConfigsResponse(new DescribeConfigsResponseData(new ByteBufferAccessor(buffer), version));
} }
@Override @Override

View File

@ -30,7 +30,6 @@ import org.apache.kafka.common.protocol.Errors;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.List; import java.util.List;
@ -137,7 +136,6 @@ public class ListOffsetsRequest extends AbstractRequest {
@Override @Override
public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) {
short versionId = version();
short errorCode = Errors.forException(e).code(); short errorCode = Errors.forException(e).code();
List<ListOffsetsTopicResponse> responses = new ArrayList<>(); List<ListOffsetsTopicResponse> responses = new ArrayList<>();
@ -148,12 +146,8 @@ public class ListOffsetsRequest extends AbstractRequest {
ListOffsetsPartitionResponse partitionResponse = new ListOffsetsPartitionResponse() ListOffsetsPartitionResponse partitionResponse = new ListOffsetsPartitionResponse()
.setErrorCode(errorCode) .setErrorCode(errorCode)
.setPartitionIndex(partition.partitionIndex()); .setPartitionIndex(partition.partitionIndex());
if (versionId == 0) { partitionResponse.setOffset(ListOffsetsResponse.UNKNOWN_OFFSET)
partitionResponse.setOldStyleOffsets(Collections.emptyList()); .setTimestamp(ListOffsetsResponse.UNKNOWN_TIMESTAMP);
} else {
partitionResponse.setOffset(ListOffsetsResponse.UNKNOWN_OFFSET)
.setTimestamp(ListOffsetsResponse.UNKNOWN_TIMESTAMP);
}
partitions.add(partitionResponse); partitions.add(partitionResponse);
} }
topicResponse.setPartitions(partitions); topicResponse.setPartitions(partitions);

View File

@ -42,27 +42,16 @@ import static org.apache.kafka.common.requests.ProduceResponse.INVALID_OFFSET;
public class ProduceRequest extends AbstractRequest { public class ProduceRequest extends AbstractRequest {
public static final short LAST_STABLE_VERSION_BEFORE_TRANSACTION_V2 = 11; public static final short LAST_STABLE_VERSION_BEFORE_TRANSACTION_V2 = 11;
public static Builder forMagic(byte magic, ProduceRequestData data, boolean useTransactionV1Version) { public static Builder builder(ProduceRequestData data, boolean useTransactionV1Version) {
// Message format upgrades correspond with a bump in the produce request version. Older // When we use transaction V1 protocol in transaction we set the request version upper limit to
// message format versions are generally not supported by the produce request versions // LAST_STABLE_VERSION_BEFORE_TRANSACTION_V2 so that the broker knows that we're using transaction protocol V1.
// following the bump. short maxVersion = useTransactionV1Version ?
LAST_STABLE_VERSION_BEFORE_TRANSACTION_V2 : ApiKeys.PRODUCE.latestVersion();
final short minVersion; return new Builder(ApiKeys.PRODUCE.oldestVersion(), maxVersion, data);
final short maxVersion;
if (magic < RecordBatch.MAGIC_VALUE_V2) {
minVersion = 2;
maxVersion = 2;
} else {
minVersion = 3;
short latestVersion = ApiKeys.PRODUCE.latestVersion();
maxVersion = useTransactionV1Version ?
(short) Math.min(latestVersion, LAST_STABLE_VERSION_BEFORE_TRANSACTION_V2) : latestVersion;
}
return new Builder(minVersion, maxVersion, data);
} }
public static Builder forCurrentMagic(ProduceRequestData data) { public static Builder builder(ProduceRequestData data) {
return forMagic(RecordBatch.CURRENT_MAGIC_VALUE, data, false); return builder(data, false);
} }
public static class Builder extends AbstractRequest.Builder<ProduceRequest> { public static class Builder extends AbstractRequest.Builder<ProduceRequest> {
@ -226,32 +215,26 @@ public class ProduceRequest extends AbstractRequest {
} }
public static void validateRecords(short version, BaseRecords baseRecords) { public static void validateRecords(short version, BaseRecords baseRecords) {
if (version >= 3) { if (baseRecords instanceof Records) {
if (baseRecords instanceof Records) { Records records = (Records) baseRecords;
Records records = (Records) baseRecords; Iterator<? extends RecordBatch> iterator = records.batches().iterator();
Iterator<? extends RecordBatch> iterator = records.batches().iterator(); if (!iterator.hasNext())
if (!iterator.hasNext()) throw new InvalidRecordException("Produce requests with version " + version + " must have at least " +
throw new InvalidRecordException("Produce requests with version " + version + " must have at least " + "one record batch per partition");
"one record batch per partition");
RecordBatch entry = iterator.next(); RecordBatch entry = iterator.next();
if (entry.magic() != RecordBatch.MAGIC_VALUE_V2) if (entry.magic() != RecordBatch.MAGIC_VALUE_V2)
throw new InvalidRecordException("Produce requests with version " + version + " are only allowed to " + throw new InvalidRecordException("Produce requests with version " + version + " are only allowed to " +
"contain record batches with magic version 2"); "contain record batches with magic version 2");
if (version < 7 && entry.compressionType() == CompressionType.ZSTD) { if (version < 7 && entry.compressionType() == CompressionType.ZSTD) {
throw new UnsupportedCompressionTypeException("Produce requests with version " + version + " are not allowed to " + throw new UnsupportedCompressionTypeException("Produce requests with version " + version + " are not allowed to " +
"use ZStandard compression"); "use ZStandard compression");
}
if (iterator.hasNext())
throw new InvalidRecordException("Produce requests with version " + version + " are only allowed to " +
"contain exactly one record batch per partition");
} }
}
// Note that we do not do similar validation for older versions to ensure compatibility with if (iterator.hasNext())
// clients which send the wrong magic version in the wrong version of the produce request. The broker throw new InvalidRecordException("Produce requests with version " + version + " are only allowed to " +
// did not do this validation before, so we maintain that behavior here. "contain exactly one record batch per partition");
}
} }
public static ProduceRequest parse(ByteBuffer buffer, short version) { public static ProduceRequest parse(ByteBuffer buffer, short version) {
@ -261,22 +244,4 @@ public class ProduceRequest extends AbstractRequest {
public static boolean isTransactionV2Requested(short version) { public static boolean isTransactionV2Requested(short version) {
return version > LAST_STABLE_VERSION_BEFORE_TRANSACTION_V2; return version > LAST_STABLE_VERSION_BEFORE_TRANSACTION_V2;
} }
public static byte requiredMagicForVersion(short produceRequestVersion) {
if (produceRequestVersion < ApiKeys.PRODUCE.oldestVersion() || produceRequestVersion > ApiKeys.PRODUCE.latestVersion())
throw new IllegalArgumentException("Magic value to use for produce request version " +
produceRequestVersion + " is not known");
switch (produceRequestVersion) {
case 0:
case 1:
return RecordBatch.MAGIC_VALUE_V0;
case 2:
return RecordBatch.MAGIC_VALUE_V1;
default:
return RecordBatch.MAGIC_VALUE_V2;
}
}
} }

View File

@ -18,10 +18,11 @@
"type": "request", "type": "request",
"listeners": ["zkBroker", "broker"], "listeners": ["zkBroker", "broker"],
"name": "AlterReplicaLogDirsRequest", "name": "AlterReplicaLogDirsRequest",
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
//
// Version 1 is the same as version 0. // Version 1 is the same as version 0.
// Version 2 enables flexible versions. // Version 2 enables flexible versions.
"validVersions": "0-2", "validVersions": "1-2",
"deprecatedVersions": "0",
"flexibleVersions": "2+", "flexibleVersions": "2+",
"fields": [ "fields": [
{ "name": "Dirs", "type": "[]AlterReplicaLogDir", "versions": "0+", { "name": "Dirs", "type": "[]AlterReplicaLogDir", "versions": "0+",

View File

@ -17,9 +17,10 @@
"apiKey": 34, "apiKey": 34,
"type": "response", "type": "response",
"name": "AlterReplicaLogDirsResponse", "name": "AlterReplicaLogDirsResponse",
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
// Starting in version 1, on quota violation brokers send out responses before throttling. // Starting in version 1, on quota violation brokers send out responses before throttling.
// Version 2 enables flexible versions. // Version 2 enables flexible versions.
"validVersions": "0-2", "validVersions": "1-2",
"flexibleVersions": "2+", "flexibleVersions": "2+",
"fields": [ "fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",

View File

@ -18,11 +18,11 @@
"type": "request", "type": "request",
"listeners": ["zkBroker", "broker", "controller"], "listeners": ["zkBroker", "broker", "controller"],
"name": "CreateAclsRequest", "name": "CreateAclsRequest",
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
// Version 1 adds resource pattern type. // Version 1 adds resource pattern type.
// Version 2 enables flexible versions. // Version 2 enables flexible versions.
// Version 3 adds user resource type. // Version 3 adds user resource type.
"validVersions": "0-3", "validVersions": "1-3",
"deprecatedVersions": "0",
"flexibleVersions": "2+", "flexibleVersions": "2+",
"fields": [ "fields": [
{ "name": "Creations", "type": "[]AclCreation", "versions": "0+", { "name": "Creations", "type": "[]AclCreation", "versions": "0+",

View File

@ -17,10 +17,11 @@
"apiKey": 30, "apiKey": 30,
"type": "response", "type": "response",
"name": "CreateAclsResponse", "name": "CreateAclsResponse",
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
// Starting in version 1, on quota violation, brokers send out responses before throttling. // Starting in version 1, on quota violation, brokers send out responses before throttling.
// Version 2 enables flexible versions. // Version 2 enables flexible versions.
// Version 3 adds user resource type. // Version 3 adds user resource type.
"validVersions": "0-3", "validVersions": "1-3",
"flexibleVersions": "2+", "flexibleVersions": "2+",
"fields": [ "fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",

View File

@ -18,13 +18,14 @@
"type": "request", "type": "request",
"listeners": ["zkBroker", "broker", "controller"], "listeners": ["zkBroker", "broker", "controller"],
"name": "CreateDelegationTokenRequest", "name": "CreateDelegationTokenRequest",
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
//
// Version 1 is the same as version 0. // Version 1 is the same as version 0.
// //
// Version 2 is the first flexible version. // Version 2 is the first flexible version.
// //
// Version 3 adds owner principal // Version 3 adds owner principal
"validVersions": "0-3", "validVersions": "1-3",
"deprecatedVersions": "0",
"flexibleVersions": "2+", "flexibleVersions": "2+",
"fields": [ "fields": [
{ "name": "OwnerPrincipalType", "type": "string", "versions": "3+", "nullableVersions": "3+", { "name": "OwnerPrincipalType", "type": "string", "versions": "3+", "nullableVersions": "3+",

View File

@ -17,12 +17,14 @@
"apiKey": 38, "apiKey": 38,
"type": "response", "type": "response",
"name": "CreateDelegationTokenResponse", "name": "CreateDelegationTokenResponse",
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
//
// Starting in version 1, on quota violation, brokers send out responses before throttling. // Starting in version 1, on quota violation, brokers send out responses before throttling.
// //
// Version 2 is the first flexible version. // Version 2 is the first flexible version.
// //
// Version 3 adds token requester details // Version 3 adds token requester details
"validVersions": "0-3", "validVersions": "1-3",
"flexibleVersions": "2+", "flexibleVersions": "2+",
"fields": [ "fields": [
{ "name": "ErrorCode", "type": "int16", "versions": "0+", { "name": "ErrorCode", "type": "int16", "versions": "0+",

View File

@ -18,6 +18,8 @@
"type": "request", "type": "request",
"listeners": ["zkBroker", "broker", "controller"], "listeners": ["zkBroker", "broker", "controller"],
"name": "CreateTopicsRequest", "name": "CreateTopicsRequest",
// Versions 0-1 were removed in Apache Kafka 4.0, Version 2 is the new baseline.
//
// Version 1 adds validateOnly. // Version 1 adds validateOnly.
// //
// Version 4 makes partitions/replicationFactor optional even when assignments are not present (KIP-464) // Version 4 makes partitions/replicationFactor optional even when assignments are not present (KIP-464)
@ -29,8 +31,7 @@
// in the response if the topics creation is throttled (KIP-599). // in the response if the topics creation is throttled (KIP-599).
// //
// Version 7 is the same as version 6. // Version 7 is the same as version 6.
"validVersions": "0-7", "validVersions": "2-7",
"deprecatedVersions": "0-1",
"flexibleVersions": "5+", "flexibleVersions": "5+",
"fields": [ "fields": [
{ "name": "Topics", "type": "[]CreatableTopic", "versions": "0+", { "name": "Topics", "type": "[]CreatableTopic", "versions": "0+",

View File

@ -17,6 +17,8 @@
"apiKey": 19, "apiKey": 19,
"type": "response", "type": "response",
"name": "CreateTopicsResponse", "name": "CreateTopicsResponse",
// Versions 0-1 were removed in Apache Kafka 4.0, Version 2 is the new baseline.
//
// Version 1 adds a per-topic error message string. // Version 1 adds a per-topic error message string.
// //
// Version 2 adds the throttle time. // Version 2 adds the throttle time.
@ -32,7 +34,7 @@
// in the response if the topics creation is throttled (KIP-599). // in the response if the topics creation is throttled (KIP-599).
// //
// Version 7 returns the topic ID of the newly created topic if creation is successful. // Version 7 returns the topic ID of the newly created topic if creation is successful.
"validVersions": "0-7", "validVersions": "2-7",
"flexibleVersions": "5+", "flexibleVersions": "5+",
"fields": [ "fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "2+", "ignorable": true, { "name": "ThrottleTimeMs", "type": "int32", "versions": "2+", "ignorable": true,

View File

@ -18,11 +18,11 @@
"type": "request", "type": "request",
"listeners": ["zkBroker", "broker", "controller"], "listeners": ["zkBroker", "broker", "controller"],
"name": "DeleteAclsRequest", "name": "DeleteAclsRequest",
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
// Version 1 adds the pattern type. // Version 1 adds the pattern type.
// Version 2 enables flexible versions. // Version 2 enables flexible versions.
// Version 3 adds the user resource type. // Version 3 adds the user resource type.
"validVersions": "0-3", "validVersions": "1-3",
"deprecatedVersions": "0",
"flexibleVersions": "2+", "flexibleVersions": "2+",
"fields": [ "fields": [
{ "name": "Filters", "type": "[]DeleteAclsFilter", "versions": "0+", { "name": "Filters", "type": "[]DeleteAclsFilter", "versions": "0+",

View File

@ -17,11 +17,12 @@
"apiKey": 31, "apiKey": 31,
"type": "response", "type": "response",
"name": "DeleteAclsResponse", "name": "DeleteAclsResponse",
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
// Version 1 adds the resource pattern type. // Version 1 adds the resource pattern type.
// Starting in version 1, on quota violation, brokers send out responses before throttling. // Starting in version 1, on quota violation, brokers send out responses before throttling.
// Version 2 enables flexible versions. // Version 2 enables flexible versions.
// Version 3 adds the user resource type. // Version 3 adds the user resource type.
"validVersions": "0-3", "validVersions": "1-3",
"flexibleVersions": "2+", "flexibleVersions": "2+",
"fields": [ "fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",

View File

@ -18,11 +18,12 @@
"type": "request", "type": "request",
"listeners": ["zkBroker", "broker"], "listeners": ["zkBroker", "broker"],
"name": "DeleteGroupsRequest", "name": "DeleteGroupsRequest",
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
//
// Version 1 is the same as version 0. // Version 1 is the same as version 0.
// //
// Version 2 is the first flexible version. // Version 2 is the first flexible version.
"validVersions": "0-2", "validVersions": "1-2",
"deprecatedVersions": "0",
"flexibleVersions": "2+", "flexibleVersions": "2+",
"fields": [ "fields": [
{ "name": "GroupsNames", "type": "[]string", "versions": "0+", "entityType": "groupId", { "name": "GroupsNames", "type": "[]string", "versions": "0+", "entityType": "groupId",

View File

@ -17,10 +17,12 @@
"apiKey": 42, "apiKey": 42,
"type": "response", "type": "response",
"name": "DeleteGroupsResponse", "name": "DeleteGroupsResponse",
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
//
// Starting in version 1, on quota violation, brokers send out responses before throttling. // Starting in version 1, on quota violation, brokers send out responses before throttling.
// //
// Version 2 is the first flexible version. // Version 2 is the first flexible version.
"validVersions": "0-2", "validVersions": "1-2",
"flexibleVersions": "2+", "flexibleVersions": "2+",
"fields": [ "fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",

View File

@ -18,6 +18,7 @@
"type": "request", "type": "request",
"listeners": ["zkBroker", "broker", "controller"], "listeners": ["zkBroker", "broker", "controller"],
"name": "DeleteTopicsRequest", "name": "DeleteTopicsRequest",
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
// Versions 0, 1, 2, and 3 are the same. // Versions 0, 1, 2, and 3 are the same.
// //
// Version 4 is the first flexible version. // Version 4 is the first flexible version.
@ -26,8 +27,7 @@
// in the response if the topics deletion is throttled (KIP-599). // in the response if the topics deletion is throttled (KIP-599).
// //
// Version 6 reorganizes topics, adds topic IDs and allows topic names to be null. // Version 6 reorganizes topics, adds topic IDs and allows topic names to be null.
"validVersions": "0-6", "validVersions": "1-6",
"deprecatedVersions": "0",
"flexibleVersions": "4+", "flexibleVersions": "4+",
"fields": [ "fields": [
{ "name": "Topics", "type": "[]DeleteTopicState", "versions": "6+", "about": "The name or topic ID of the topic.", { "name": "Topics", "type": "[]DeleteTopicState", "versions": "6+", "about": "The name or topic ID of the topic.",

View File

@ -17,6 +17,8 @@
"apiKey": 20, "apiKey": 20,
"type": "response", "type": "response",
"name": "DeleteTopicsResponse", "name": "DeleteTopicsResponse",
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
//
// Version 1 adds the throttle time. // Version 1 adds the throttle time.
// //
// Starting in version 2, on quota violation, brokers send out responses before throttling. // Starting in version 2, on quota violation, brokers send out responses before throttling.
@ -31,7 +33,7 @@
// Version 6 adds topic ID to responses. An UNSUPPORTED_VERSION error code will be returned when attempting to // Version 6 adds topic ID to responses. An UNSUPPORTED_VERSION error code will be returned when attempting to
// delete using topic IDs when IBP < 2.8. UNKNOWN_TOPIC_ID error code will be returned when IBP is at least 2.8, but // delete using topic IDs when IBP < 2.8. UNKNOWN_TOPIC_ID error code will be returned when IBP is at least 2.8, but
// the topic ID was not found. // the topic ID was not found.
"validVersions": "0-6", "validVersions": "1-6",
"flexibleVersions": "4+", "flexibleVersions": "4+",
"fields": [ "fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true, { "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true,

View File

@ -18,11 +18,11 @@
"type": "request", "type": "request",
"listeners": ["zkBroker", "broker", "controller"], "listeners": ["zkBroker", "broker", "controller"],
"name": "DescribeAclsRequest", "name": "DescribeAclsRequest",
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
// Version 1 adds resource pattern type. // Version 1 adds resource pattern type.
// Version 2 enables flexible versions. // Version 2 enables flexible versions.
// Version 3 adds user resource type. // Version 3 adds user resource type.
"validVersions": "0-3", "validVersions": "1-3",
"deprecatedVersions": "0",
"flexibleVersions": "2+", "flexibleVersions": "2+",
"fields": [ "fields": [
{ "name": "ResourceTypeFilter", "type": "int8", "versions": "0+", { "name": "ResourceTypeFilter", "type": "int8", "versions": "0+",

View File

@ -17,11 +17,12 @@
"apiKey": 29, "apiKey": 29,
"type": "response", "type": "response",
"name": "DescribeAclsResponse", "name": "DescribeAclsResponse",
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
// Version 1 adds PatternType. // Version 1 adds PatternType.
// Starting in version 1, on quota violation, brokers send out responses before throttling. // Starting in version 1, on quota violation, brokers send out responses before throttling.
// Version 2 enables flexible versions. // Version 2 enables flexible versions.
// Version 3 adds user resource type. // Version 3 adds user resource type.
"validVersions": "0-3", "validVersions": "1-3",
"flexibleVersions": "2+", "flexibleVersions": "2+",
"fields": [ "fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",

View File

@ -18,11 +18,11 @@
"type": "request", "type": "request",
"listeners": ["zkBroker", "broker", "controller"], "listeners": ["zkBroker", "broker", "controller"],
"name": "DescribeConfigsRequest", "name": "DescribeConfigsRequest",
// Version 1 adds IncludeSynonyms. // Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
// Version 1 adds IncludeSynonyms and removes IsDefault.
// Version 2 is the same as version 1. // Version 2 is the same as version 1.
// Version 4 enables flexible versions. // Version 4 enables flexible versions.
"validVersions": "0-4", "validVersions": "1-4",
"deprecatedVersions": "0",
"flexibleVersions": "4+", "flexibleVersions": "4+",
"fields": [ "fields": [
{ "name": "Resources", "type": "[]DescribeConfigsResource", "versions": "0+", { "name": "Resources", "type": "[]DescribeConfigsResource", "versions": "0+",

View File

@ -17,10 +17,11 @@
"apiKey": 32, "apiKey": 32,
"type": "response", "type": "response",
"name": "DescribeConfigsResponse", "name": "DescribeConfigsResponse",
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
// Version 1 adds ConfigSource and the synonyms. // Version 1 adds ConfigSource and the synonyms.
// Starting in version 2, on quota violation, brokers send out responses before throttling. // Starting in version 2, on quota violation, brokers send out responses before throttling.
// Version 4 enables flexible versions. // Version 4 enables flexible versions.
"validVersions": "0-4", "validVersions": "1-4",
"flexibleVersions": "4+", "flexibleVersions": "4+",
"fields": [ "fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
@ -43,11 +44,6 @@
"about": "The configuration value." }, "about": "The configuration value." },
{ "name": "ReadOnly", "type": "bool", "versions": "0+", { "name": "ReadOnly", "type": "bool", "versions": "0+",
"about": "True if the configuration is read-only." }, "about": "True if the configuration is read-only." },
{ "name": "IsDefault", "type": "bool", "versions": "0",
"about": "True if the configuration is not set." },
// Note: the v0 default for this field that should be exposed to callers is
// context-dependent. For example, if the resource is a broker, this should default to 4.
// -1 is just a placeholder value.
{ "name": "ConfigSource", "type": "int8", "versions": "1+", "default": "-1", "ignorable": true, { "name": "ConfigSource", "type": "int8", "versions": "1+", "default": "-1", "ignorable": true,
"about": "The configuration source." }, "about": "The configuration source." },
{ "name": "IsSensitive", "type": "bool", "versions": "0+", { "name": "IsSensitive", "type": "bool", "versions": "0+",

View File

@ -18,11 +18,11 @@
"type": "request", "type": "request",
"listeners": ["zkBroker", "broker", "controller"], "listeners": ["zkBroker", "broker", "controller"],
"name": "DescribeDelegationTokenRequest", "name": "DescribeDelegationTokenRequest",
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
// Version 1 is the same as version 0. // Version 1 is the same as version 0.
// Version 2 adds flexible version support // Version 2 adds flexible version support
// Version 3 adds token requester into the response // Version 3 adds token requester into the response
"validVersions": "0-3", "validVersions": "1-3",
"deprecatedVersions": "0",
"flexibleVersions": "2+", "flexibleVersions": "2+",
"fields": [ "fields": [
{ "name": "Owners", "type": "[]DescribeDelegationTokenOwner", "versions": "0+", "nullableVersions": "0+", { "name": "Owners", "type": "[]DescribeDelegationTokenOwner", "versions": "0+", "nullableVersions": "0+",

View File

@ -17,10 +17,11 @@
"apiKey": 41, "apiKey": 41,
"type": "response", "type": "response",
"name": "DescribeDelegationTokenResponse", "name": "DescribeDelegationTokenResponse",
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
// Starting in version 1, on quota violation, brokers send out responses before throttling. // Starting in version 1, on quota violation, brokers send out responses before throttling.
// Version 2 adds flexible version support // Version 2 adds flexible version support
// Version 3 adds token requester details // Version 3 adds token requester details
"validVersions": "0-3", "validVersions": "1-3",
"flexibleVersions": "2+", "flexibleVersions": "2+",
"fields": [ "fields": [
{ "name": "ErrorCode", "type": "int16", "versions": "0+", { "name": "ErrorCode", "type": "int16", "versions": "0+",

View File

@ -18,12 +18,12 @@
"type": "request", "type": "request",
"listeners": ["zkBroker", "broker"], "listeners": ["zkBroker", "broker"],
"name": "DescribeLogDirsRequest", "name": "DescribeLogDirsRequest",
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
// Version 1 is the same as version 0. // Version 1 is the same as version 0.
// Version 2 is the first flexible version. // Version 2 is the first flexible version.
// Version 3 is the same as version 2 (new field in response). // Version 3 is the same as version 2 (new field in response).
// Version 4 is the same as version 2 (new fields in response). // Version 4 is the same as version 2 (new fields in response).
"validVersions": "0-4", "validVersions": "1-4",
"deprecatedVersions": "0",
"flexibleVersions": "2+", "flexibleVersions": "2+",
"fields": [ "fields": [
{ "name": "Topics", "type": "[]DescribableLogDirTopic", "versions": "0+", "nullableVersions": "0+", { "name": "Topics", "type": "[]DescribableLogDirTopic", "versions": "0+", "nullableVersions": "0+",

View File

@ -17,11 +17,12 @@
"apiKey": 35, "apiKey": 35,
"type": "response", "type": "response",
"name": "DescribeLogDirsResponse", "name": "DescribeLogDirsResponse",
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
// Starting in version 1, on quota violation, brokers send out responses before throttling. // Starting in version 1, on quota violation, brokers send out responses before throttling.
"validVersions": "0-4",
// Version 2 is the first flexible version. // Version 2 is the first flexible version.
// Version 3 adds the top-level ErrorCode field // Version 3 adds the top-level ErrorCode field
// Version 4 adds the TotalBytes and UsableBytes fields // Version 4 adds the TotalBytes and UsableBytes fields
"validVersions": "1-4",
"flexibleVersions": "2+", "flexibleVersions": "2+",
"fields": [ "fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",

View File

@ -18,10 +18,10 @@
"type": "request", "type": "request",
"listeners": ["zkBroker", "broker", "controller"], "listeners": ["zkBroker", "broker", "controller"],
"name": "ExpireDelegationTokenRequest", "name": "ExpireDelegationTokenRequest",
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
// Version 1 is the same as version 0. // Version 1 is the same as version 0.
// Version 2 adds flexible version support // Version 2 adds flexible version support
"validVersions": "0-2", "validVersions": "1-2",
"deprecatedVersions": "0",
"flexibleVersions": "2+", "flexibleVersions": "2+",
"fields": [ "fields": [
{ "name": "Hmac", "type": "bytes", "versions": "0+", { "name": "Hmac", "type": "bytes", "versions": "0+",

View File

@ -17,9 +17,10 @@
"apiKey": 40, "apiKey": 40,
"type": "response", "type": "response",
"name": "ExpireDelegationTokenResponse", "name": "ExpireDelegationTokenResponse",
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
// Starting in version 1, on quota violation, brokers send out responses before throttling. // Starting in version 1, on quota violation, brokers send out responses before throttling.
// Version 2 adds flexible version support // Version 2 adds flexible version support
"validVersions": "0-2", "validVersions": "1-2",
"flexibleVersions": "2+", "flexibleVersions": "2+",
"fields": [ "fields": [
{ "name": "ErrorCode", "type": "int16", "versions": "0+", { "name": "ErrorCode", "type": "int16", "versions": "0+",

View File

@ -18,12 +18,11 @@
"type": "request", "type": "request",
"listeners": ["zkBroker", "broker", "controller"], "listeners": ["zkBroker", "broker", "controller"],
"name": "FetchRequest", "name": "FetchRequest",
// Versions 0-3 were removed in Apache Kafka 4.0, Version 4 is the new baseline.
// //
// Version 1 is the same as version 0. // Version 1 is the same as version 0.
//
// Starting in Version 2, the requester must be able to handle Kafka Log // Starting in Version 2, the requester must be able to handle Kafka Log
// Message format version 1. // Message format version 1.
//
// Version 3 adds MaxBytes. Starting in version 3, the partition ordering in // Version 3 adds MaxBytes. Starting in version 3, the partition ordering in
// the request is now relevant. Partitions will be processed in the order // the request is now relevant. Partitions will be processed in the order
// they appear in the request. // they appear in the request.
@ -57,8 +56,7 @@
// Version 16 is the same as version 15 (KIP-951). // Version 16 is the same as version 15 (KIP-951).
// //
// Version 17 adds directory id support from KIP-853 // Version 17 adds directory id support from KIP-853
"validVersions": "0-17", "validVersions": "4-17",
"deprecatedVersions": "0-3",
"flexibleVersions": "12+", "flexibleVersions": "12+",
"fields": [ "fields": [
{ "name": "ClusterId", "type": "string", "versions": "12+", "nullableVersions": "12+", "default": "null", { "name": "ClusterId", "type": "string", "versions": "12+", "nullableVersions": "12+", "default": "null",

View File

@ -17,10 +17,9 @@
"apiKey": 1, "apiKey": 1,
"type": "response", "type": "response",
"name": "FetchResponse", "name": "FetchResponse",
// Versions 0-3 were removed in Apache Kafka 4.0, Version 4 is the new baseline.
// //
// Version 1 adds throttle time. // Version 1 adds throttle time. Version 2 and 3 are the same as version 1.
//
// Version 2 and 3 are the same as version 1.
// //
// Version 4 adds features for transactional consumption. // Version 4 adds features for transactional consumption.
// //
@ -49,7 +48,7 @@
// Version 16 adds the 'NodeEndpoints' field (KIP-951). // Version 16 adds the 'NodeEndpoints' field (KIP-951).
// //
// Version 17 no changes to the response (KIP-853). // Version 17 no changes to the response (KIP-853).
"validVersions": "0-17", "validVersions": "4-17",
"flexibleVersions": "12+", "flexibleVersions": "12+",
"fields": [ "fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true, { "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true,

View File

@ -18,9 +18,9 @@
"type": "request", "type": "request",
"listeners": ["zkBroker", "broker"], "listeners": ["zkBroker", "broker"],
"name": "JoinGroupRequest", "name": "JoinGroupRequest",
// Version 1 adds RebalanceTimeoutMs. // Versions 0-1 were removed in Apache Kafka 4.0, Version 2 is the new baseline.
// //
// Version 2 and 3 are the same as version 1. // Version 1 adds RebalanceTimeoutMs. Version 2 and 3 are the same as version 1.
// //
// Starting from version 4, the client needs to issue a second request to join group // Starting from version 4, the client needs to issue a second request to join group
// //
@ -34,8 +34,7 @@
// Version 8 adds the Reason field (KIP-800). // Version 8 adds the Reason field (KIP-800).
// //
// Version 9 is the same as version 8. // Version 9 is the same as version 8.
"validVersions": "0-9", "validVersions": "2-9",
"deprecatedVersions": "0-1",
"flexibleVersions": "6+", "flexibleVersions": "6+",
"fields": [ "fields": [
{ "name": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId", { "name": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId",

View File

@ -17,6 +17,8 @@
"apiKey": 11, "apiKey": 11,
"type": "response", "type": "response",
"name": "JoinGroupResponse", "name": "JoinGroupResponse",
// Versions 0-1 were removed in Apache Kafka 4.0, Version 2 is the new baseline.
//
// Version 1 is the same as version 0. // Version 1 is the same as version 0.
// //
// Version 2 adds throttle time. // Version 2 adds throttle time.
@ -35,7 +37,7 @@
// Version 8 is the same as version 7. // Version 8 is the same as version 7.
// //
// Version 9 adds the SkipAssignment field. // Version 9 adds the SkipAssignment field.
"validVersions": "0-9", "validVersions": "2-9",
"flexibleVersions": "6+", "flexibleVersions": "6+",
"fields": [ "fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "2+", "ignorable": true, { "name": "ThrottleTimeMs", "type": "int32", "versions": "2+", "ignorable": true,

View File

@ -18,6 +18,8 @@
"type": "request", "type": "request",
"listeners": ["zkBroker", "broker"], "listeners": ["zkBroker", "broker"],
"name": "ListOffsetsRequest", "name": "ListOffsetsRequest",
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
//
// Version 1 removes MaxNumOffsets. From this version forward, only a single // Version 1 removes MaxNumOffsets. From this version forward, only a single
// offset can be returned. // offset can be returned.
// //
@ -38,8 +40,7 @@
// Version 9 enables listing offsets by last tiered offset (KIP-1005). // Version 9 enables listing offsets by last tiered offset (KIP-1005).
// //
// Version 10 enables async remote list offsets support (KIP-1075) // Version 10 enables async remote list offsets support (KIP-1075)
"validVersions": "0-10", "validVersions": "1-10",
"deprecatedVersions": "0",
"flexibleVersions": "6+", "flexibleVersions": "6+",
"latestVersionUnstable": false, "latestVersionUnstable": false,
"fields": [ "fields": [
@ -58,9 +59,7 @@
{ "name": "CurrentLeaderEpoch", "type": "int32", "versions": "4+", "default": "-1", "ignorable": true, { "name": "CurrentLeaderEpoch", "type": "int32", "versions": "4+", "default": "-1", "ignorable": true,
"about": "The current leader epoch." }, "about": "The current leader epoch." },
{ "name": "Timestamp", "type": "int64", "versions": "0+", { "name": "Timestamp", "type": "int64", "versions": "0+",
"about": "The current timestamp." }, "about": "The current timestamp." }
{ "name": "MaxNumOffsets", "type": "int32", "versions": "0", "default": "1",
"about": "The maximum number of offsets to report." }
]} ]}
]}, ]},
{ "name": "TimeoutMs", "type": "int32", "versions": "10+", "ignorable": true, { "name": "TimeoutMs", "type": "int32", "versions": "10+", "ignorable": true,

View File

@ -17,6 +17,8 @@
"apiKey": 2, "apiKey": 2,
"type": "response", "type": "response",
"name": "ListOffsetsResponse", "name": "ListOffsetsResponse",
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
//
// Version 1 removes the offsets array in favor of returning a single offset. // Version 1 removes the offsets array in favor of returning a single offset.
// Version 1 also adds the timestamp associated with the returned offset. // Version 1 also adds the timestamp associated with the returned offset.
// //
@ -38,7 +40,7 @@
// Version 9 enables listing offsets by last tiered offset (KIP-1005). // Version 9 enables listing offsets by last tiered offset (KIP-1005).
// //
// Version 10 enables async remote list offsets support (KIP-1075) // Version 10 enables async remote list offsets support (KIP-1075)
"validVersions": "0-10", "validVersions": "1-10",
"flexibleVersions": "6+", "flexibleVersions": "6+",
"fields": [ "fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "2+", "ignorable": true, { "name": "ThrottleTimeMs", "type": "int32", "versions": "2+", "ignorable": true,
@ -53,8 +55,6 @@
"about": "The partition index." }, "about": "The partition index." },
{ "name": "ErrorCode", "type": "int16", "versions": "0+", { "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The partition error code, or 0 if there was no error." }, "about": "The partition error code, or 0 if there was no error." },
{ "name": "OldStyleOffsets", "type": "[]int64", "versions": "0", "ignorable": false,
"about": "The result offsets." },
{ "name": "Timestamp", "type": "int64", "versions": "1+", "default": "-1", "ignorable": false, { "name": "Timestamp", "type": "int64", "versions": "1+", "default": "-1", "ignorable": false,
"about": "The timestamp associated with the returned offset." }, "about": "The timestamp associated with the returned offset." },
{ "name": "Offset", "type": "int64", "versions": "1+", "default": "-1", "ignorable": false, { "name": "Offset", "type": "int64", "versions": "1+", "default": "-1", "ignorable": false,

View File

@ -18,14 +18,14 @@
"type": "request", "type": "request",
"listeners": ["zkBroker", "broker"], "listeners": ["zkBroker", "broker"],
"name": "MetadataRequest", "name": "MetadataRequest",
"validVersions": "0-13", "validVersions": "4-13",
"deprecatedVersions": "0-3",
"flexibleVersions": "9+", "flexibleVersions": "9+",
"fields": [ "fields": [
// Versions 0-3 were removed in Apache Kafka 4.0, Version 4 is the new baseline.
//
// In version 0, an empty array indicates "request metadata for all topics." In version 1 and // In version 0, an empty array indicates "request metadata for all topics." In version 1 and
// higher, an empty array indicates "request metadata for no topics," and a null array is used to // higher, an empty array indicates "request metadata for no topics," and a null array is used to
// indicate "request metadata for all topics." // indicate "request metadata for all topics."
//
// Version 2 and 3 are the same as version 1. // Version 2 and 3 are the same as version 1.
// //
// Version 4 adds AllowAutoTopicCreation. // Version 4 adds AllowAutoTopicCreation.

View File

@ -17,11 +17,11 @@
"apiKey": 3, "apiKey": 3,
"type": "response", "type": "response",
"name": "MetadataResponse", "name": "MetadataResponse",
// Versions 0-3 were removed in Apache Kafka 4.0, Version 4 is the new baseline.
//
// Version 1 adds fields for the rack of each broker, the controller id, and // Version 1 adds fields for the rack of each broker, the controller id, and
// whether or not the topic is internal. // whether or not the topic is internal.
//
// Version 2 adds the cluster ID field. // Version 2 adds the cluster ID field.
//
// Version 3 adds the throttle time. // Version 3 adds the throttle time.
// //
// Version 4 is the same as version 3. // Version 4 is the same as version 3.
@ -43,7 +43,7 @@
// by the DescribeCluster API (KIP-700). // by the DescribeCluster API (KIP-700).
// Version 12 supports topicId. // Version 12 supports topicId.
// Version 13 supports top-level error code in the response. // Version 13 supports top-level error code in the response.
"validVersions": "0-13", "validVersions": "4-13",
"flexibleVersions": "9+", "flexibleVersions": "9+",
"fields": [ "fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "3+", "ignorable": true, { "name": "ThrottleTimeMs", "type": "int32", "versions": "3+", "ignorable": true,

View File

@ -18,6 +18,8 @@
"type": "request", "type": "request",
"listeners": ["zkBroker", "broker"], "listeners": ["zkBroker", "broker"],
"name": "OffsetCommitRequest", "name": "OffsetCommitRequest",
// Versions 0-1 were removed in Apache Kafka 4.0, Version 2 is the new baseline.
//
// Version 1 adds timestamp and group membership information, as well as the commit timestamp. // Version 1 adds timestamp and group membership information, as well as the commit timestamp.
// //
// Version 2 adds retention time. It removes the commit timestamp added in version 1. // Version 2 adds retention time. It removes the commit timestamp added in version 1.
@ -34,8 +36,7 @@
// //
// Version 9 is the first version that can be used with the new consumer group protocol (KIP-848). The // Version 9 is the first version that can be used with the new consumer group protocol (KIP-848). The
// request is the same as version 8. // request is the same as version 8.
"validVersions": "0-9", "validVersions": "2-9",
"deprecatedVersions": "0-1",
"flexibleVersions": "8+", "flexibleVersions": "8+",
"fields": [ "fields": [
{ "name": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId", { "name": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId",
@ -61,9 +62,6 @@
"about": "The message offset to be committed." }, "about": "The message offset to be committed." },
{ "name": "CommittedLeaderEpoch", "type": "int32", "versions": "6+", "default": "-1", "ignorable": true, { "name": "CommittedLeaderEpoch", "type": "int32", "versions": "6+", "default": "-1", "ignorable": true,
"about": "The leader epoch of this partition." }, "about": "The leader epoch of this partition." },
// CommitTimestamp has been removed from v2 and later.
{ "name": "CommitTimestamp", "type": "int64", "versions": "1", "default": "-1",
"about": "The timestamp of the commit." },
{ "name": "CommittedMetadata", "type": "string", "versions": "0+", "nullableVersions": "0+", { "name": "CommittedMetadata", "type": "string", "versions": "0+", "nullableVersions": "0+",
"about": "Any associated metadata the client wants to keep." } "about": "Any associated metadata the client wants to keep." }
]} ]}

View File

@ -17,6 +17,8 @@
"apiKey": 8, "apiKey": 8,
"type": "response", "type": "response",
"name": "OffsetCommitResponse", "name": "OffsetCommitResponse",
// Versions 0-1 were removed in Apache Kafka 4.0, Version 2 is the new baseline.
//
// Versions 1 and 2 are the same as version 0. // Versions 1 and 2 are the same as version 0.
// //
// Version 3 adds the throttle time to the response. // Version 3 adds the throttle time to the response.
@ -32,7 +34,7 @@
// Version 9 is the first version that can be used with the new consumer group protocol (KIP-848). The response is // Version 9 is the first version that can be used with the new consumer group protocol (KIP-848). The response is
// the same as version 8 but can return STALE_MEMBER_EPOCH when the new consumer group protocol is used and // the same as version 8 but can return STALE_MEMBER_EPOCH when the new consumer group protocol is used and
// GROUP_ID_NOT_FOUND when the group does not exist for both protocols. // GROUP_ID_NOT_FOUND when the group does not exist for both protocols.
"validVersions": "0-9", "validVersions": "2-9",
"flexibleVersions": "8+", "flexibleVersions": "8+",
// Supported errors: // Supported errors:
// - GROUP_AUTHORIZATION_FAILED (version 0+) // - GROUP_AUTHORIZATION_FAILED (version 0+)

View File

@ -18,6 +18,8 @@
"type": "request", "type": "request",
"listeners": ["zkBroker", "broker"], "listeners": ["zkBroker", "broker"],
"name": "OffsetFetchRequest", "name": "OffsetFetchRequest",
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
//
// In version 0, the request read offsets from ZK. // In version 0, the request read offsets from ZK.
// //
// Starting in version 1, the broker supports fetching offsets from the internal __consumer_offsets topic. // Starting in version 1, the broker supports fetching offsets from the internal __consumer_offsets topic.
@ -36,8 +38,7 @@
// //
// Version 9 is the first version that can be used with the new consumer group protocol (KIP-848). It adds // Version 9 is the first version that can be used with the new consumer group protocol (KIP-848). It adds
// the MemberId and MemberEpoch fields. Those are filled in and validated when the new consumer protocol is used. // the MemberId and MemberEpoch fields. Those are filled in and validated when the new consumer protocol is used.
"validVersions": "0-9", "validVersions": "1-9",
"deprecatedVersions": "0",
"flexibleVersions": "6+", "flexibleVersions": "6+",
"fields": [ "fields": [
{ "name": "GroupId", "type": "string", "versions": "0-7", "entityType": "groupId", { "name": "GroupId", "type": "string", "versions": "0-7", "entityType": "groupId",

View File

@ -17,6 +17,8 @@
"apiKey": 9, "apiKey": 9,
"type": "response", "type": "response",
"name": "OffsetFetchResponse", "name": "OffsetFetchResponse",
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
//
// Version 1 is the same as version 0. // Version 1 is the same as version 0.
// //
// Version 2 adds a top-level error code. // Version 2 adds a top-level error code.
@ -36,7 +38,7 @@
// Version 9 is the first version that can be used with the new consumer group protocol (KIP-848). The response is // Version 9 is the first version that can be used with the new consumer group protocol (KIP-848). The response is
// the same as version 8 but can return STALE_MEMBER_EPOCH and UNKNOWN_MEMBER_ID errors when the new consumer group // the same as version 8 but can return STALE_MEMBER_EPOCH and UNKNOWN_MEMBER_ID errors when the new consumer group
// protocol is used. // protocol is used.
"validVersions": "0-9", "validVersions": "1-9",
"flexibleVersions": "6+", "flexibleVersions": "6+",
// Supported errors: // Supported errors:
// - GROUP_AUTHORIZATION_FAILED (version 0+) // - GROUP_AUTHORIZATION_FAILED (version 0+)

View File

@ -18,6 +18,8 @@
"type": "request", "type": "request",
"listeners": ["zkBroker", "broker"], "listeners": ["zkBroker", "broker"],
"name": "OffsetForLeaderEpochRequest", "name": "OffsetForLeaderEpochRequest",
// Versions 0-1 were removed in Apache Kafka 4.0, Version 2 is the new baseline.
//
// Version 1 is the same as version 0. // Version 1 is the same as version 0.
// //
// Version 2 adds the current leader epoch to support fencing. // Version 2 adds the current leader epoch to support fencing.
@ -27,8 +29,7 @@
// Followers will use this replicaId when using an older version of the protocol. // Followers will use this replicaId when using an older version of the protocol.
// //
// Version 4 enables flexible versions. // Version 4 enables flexible versions.
"validVersions": "0-4", "validVersions": "2-4",
"deprecatedVersions": "0-1",
"flexibleVersions": "4+", "flexibleVersions": "4+",
"fields": [ "fields": [
{ "name": "ReplicaId", "type": "int32", "versions": "3+", "default": -2, "ignorable": true, "entityType": "brokerId", { "name": "ReplicaId", "type": "int32", "versions": "3+", "default": -2, "ignorable": true, "entityType": "brokerId",

View File

@ -17,6 +17,8 @@
"apiKey": 23, "apiKey": 23,
"type": "response", "type": "response",
"name": "OffsetForLeaderEpochResponse", "name": "OffsetForLeaderEpochResponse",
// Versions 0-1 were removed in Apache Kafka 4.0, Version 2 is the new baseline.
//
// Version 1 added the leader epoch to the response. // Version 1 added the leader epoch to the response.
// //
// Version 2 added the throttle time. // Version 2 added the throttle time.
@ -24,7 +26,7 @@
// Version 3 is the same as version 2. // Version 3 is the same as version 2.
// //
// Version 4 enables flexible versions. // Version 4 enables flexible versions.
"validVersions": "0-4", "validVersions": "2-4",
"flexibleVersions": "4+", "flexibleVersions": "4+",
"fields": [ "fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "2+", "ignorable": true, { "name": "ThrottleTimeMs", "type": "int32", "versions": "2+", "ignorable": true,

View File

@ -18,13 +18,15 @@
"type": "request", "type": "request",
"listeners": ["zkBroker", "broker"], "listeners": ["zkBroker", "broker"],
"name": "ProduceRequest", "name": "ProduceRequest",
// Versions 0-2 were removed in Apache Kafka 4.0, Version 3 is the new baseline.
//
// Version 1 and 2 are the same as version 0. // Version 1 and 2 are the same as version 0.
// //
// Version 3 adds the transactional ID, which is used for authorization when attempting to write // Version 3 adds the transactional ID, which is used for authorization when attempting to write
// transactional data. Version 3 also adds support for Kafka Message Format v2. // transactional data. Version 3 also adds support for Kafka Message Format v2.
// //
// Version 4 is the same as version 3, but the requester must be prepared to handle a // Version 4 is the same as version 3, but the requester must be prepared to handle a
// KAFKA_STORAGE_ERROR. // KAFKA_STORAGE_ERROR.
// //
// Version 5 and 6 are the same as version 3. // Version 5 and 6 are the same as version 3.
// //
@ -42,8 +44,7 @@
// transaction V2 (KIP_890 part 2) is enabled, the produce request will also include the function for a // transaction V2 (KIP_890 part 2) is enabled, the produce request will also include the function for a
// AddPartitionsToTxn call. If V2 is disabled, the client can't use produce request version higher than 11 within // AddPartitionsToTxn call. If V2 is disabled, the client can't use produce request version higher than 11 within
// a transaction. // a transaction.
"validVersions": "0-12", "validVersions": "3-12",
"deprecatedVersions": "0-2",
"flexibleVersions": "9+", "flexibleVersions": "9+",
"fields": [ "fields": [
{ "name": "TransactionalId", "type": "string", "versions": "3+", "nullableVersions": "3+", "default": "null", "entityType": "transactionalId", { "name": "TransactionalId", "type": "string", "versions": "3+", "nullableVersions": "3+", "default": "null", "entityType": "transactionalId",

View File

@ -17,16 +17,16 @@
"apiKey": 0, "apiKey": 0,
"type": "response", "type": "response",
"name": "ProduceResponse", "name": "ProduceResponse",
// Version 1 added the throttle time. // Versions 0-2 were removed in Apache Kafka 4.0, Version 3 is the new baseline.
// //
// Version 1 added the throttle time.
// Version 2 added the log append time. // Version 2 added the log append time.
// //
// Version 3 is the same as version 2. // Version 3 is the same as version 2.
// //
// Version 4 added KAFKA_STORAGE_ERROR as a possible error code. // Version 4 added KAFKA_STORAGE_ERROR as a possible error code.
// //
// Version 5 added LogStartOffset to filter out spurious // Version 5 added LogStartOffset to filter out spurious OutOfOrderSequenceExceptions on the client.
// OutOfOrderSequenceExceptions on the client.
// //
// Version 8 added RecordErrors and ErrorMessage to include information about // Version 8 added RecordErrors and ErrorMessage to include information about
// records that cause the whole batch to be dropped. See KIP-467 for details. // records that cause the whole batch to be dropped. See KIP-467 for details.
@ -38,7 +38,7 @@
// Version 11 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). // Version 11 adds support for new error code TRANSACTION_ABORTABLE (KIP-890).
// //
// Version 12 is the same as version 10 (KIP-890). // Version 12 is the same as version 10 (KIP-890).
"validVersions": "0-12", "validVersions": "3-12",
"flexibleVersions": "9+", "flexibleVersions": "9+",
"fields": [ "fields": [
{ "name": "Responses", "type": "[]TopicProduceResponse", "versions": "0+", { "name": "Responses", "type": "[]TopicProduceResponse", "versions": "0+",

View File

@ -18,10 +18,10 @@
"type": "request", "type": "request",
"listeners": ["zkBroker", "broker", "controller"], "listeners": ["zkBroker", "broker", "controller"],
"name": "RenewDelegationTokenRequest", "name": "RenewDelegationTokenRequest",
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
// Version 1 is the same as version 0. // Version 1 is the same as version 0.
// Version 2 adds flexible version support // Version 2 adds flexible version support
"validVersions": "0-2", "validVersions": "1-2",
"deprecatedVersions": "0",
"flexibleVersions": "2+", "flexibleVersions": "2+",
"fields": [ "fields": [
{ "name": "Hmac", "type": "bytes", "versions": "0+", { "name": "Hmac", "type": "bytes", "versions": "0+",

View File

@ -17,9 +17,10 @@
"apiKey": 39, "apiKey": 39,
"type": "response", "type": "response",
"name": "RenewDelegationTokenResponse", "name": "RenewDelegationTokenResponse",
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
// Starting in version 1, on quota violation, brokers send out responses before throttling. // Starting in version 1, on quota violation, brokers send out responses before throttling.
// Version 2 adds flexible version support // Version 2 adds flexible version support
"validVersions": "0-2", "validVersions": "1-2",
"flexibleVersions": "2+", "flexibleVersions": "2+",
"fields": [ "fields": [
{ "name": "ErrorCode", "type": "int16", "versions": "0+", { "name": "ErrorCode", "type": "int16", "versions": "0+",

View File

@ -17,47 +17,15 @@
package org.apache.kafka.clients; package org.apache.kafka.clients;
import org.apache.kafka.common.message.ApiVersionsResponseData; import org.apache.kafka.common.message.ApiVersionsResponseData;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.record.RecordBatch;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertEquals;
public class ApiVersionsTest { public class ApiVersionsTest {
@Test
public void testMaxUsableProduceMagic() {
ApiVersions apiVersions = new ApiVersions();
assertEquals(RecordBatch.CURRENT_MAGIC_VALUE, apiVersions.maxUsableProduceMagic());
apiVersions.update("0", NodeApiVersions.create());
assertEquals(RecordBatch.CURRENT_MAGIC_VALUE, apiVersions.maxUsableProduceMagic());
apiVersions.update("1", NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 2));
assertEquals(RecordBatch.MAGIC_VALUE_V1, apiVersions.maxUsableProduceMagic());
apiVersions.remove("1");
assertEquals(RecordBatch.CURRENT_MAGIC_VALUE, apiVersions.maxUsableProduceMagic());
}
@Test
public void testMaxUsableProduceMagicWithRaftController() {
ApiVersions apiVersions = new ApiVersions();
assertEquals(RecordBatch.CURRENT_MAGIC_VALUE, apiVersions.maxUsableProduceMagic());
// something that doesn't support PRODUCE, which is the case with Raft-based controllers
apiVersions.update("2", NodeApiVersions.create(Collections.singleton(
new ApiVersionsResponseData.ApiVersion()
.setApiKey(ApiKeys.FETCH.id)
.setMinVersion((short) 0)
.setMaxVersion((short) 2))));
assertEquals(RecordBatch.CURRENT_MAGIC_VALUE, apiVersions.maxUsableProduceMagic());
}
@Test @Test
public void testFinalizedFeaturesUpdate() { public void testFinalizedFeaturesUpdate() {
ApiVersions apiVersions = new ApiVersions(); ApiVersions apiVersions = new ApiVersions();
@ -96,4 +64,5 @@ public class ApiVersionsTest {
assertEquals(1, info.finalizedFeaturesEpoch); assertEquals(1, info.finalizedFeaturesEpoch);
assertEquals((short) 2, info.finalizedFeatures.get("transaction.version")); assertEquals((short) 2, info.finalizedFeatures.get("transaction.version"));
} }
} }

View File

@ -215,7 +215,7 @@ public class NetworkClientTest {
client.poll(1, time.milliseconds()); client.poll(1, time.milliseconds());
assertTrue(client.isReady(node, time.milliseconds()), "The client should be ready"); assertTrue(client.isReady(node, time.milliseconds()), "The client should be ready");
ProduceRequest.Builder builder = ProduceRequest.forCurrentMagic(new ProduceRequestData() ProduceRequest.Builder builder = ProduceRequest.builder(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection()) .setTopicData(new ProduceRequestData.TopicProduceDataCollection())
.setAcks((short) 1) .setAcks((short) 1)
.setTimeoutMs(1000)); .setTimeoutMs(1000));
@ -632,7 +632,7 @@ public class NetworkClientTest {
private ClientResponse produce(NetworkClient client, int requestTimeoutMs, boolean shouldEmulateTimeout) { private ClientResponse produce(NetworkClient client, int requestTimeoutMs, boolean shouldEmulateTimeout) {
awaitReady(client, node); // has to be before creating any request, as it may send ApiVersionsRequest and its response is mocked with correlation id 0 awaitReady(client, node); // has to be before creating any request, as it may send ApiVersionsRequest and its response is mocked with correlation id 0
ProduceRequest.Builder builder = ProduceRequest.forCurrentMagic(new ProduceRequestData() ProduceRequest.Builder builder = ProduceRequest.builder(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection()) .setTopicData(new ProduceRequestData.TopicProduceDataCollection())
.setAcks((short) 1) .setAcks((short) 1)
.setTimeoutMs(1000)); .setTimeoutMs(1000));
@ -766,33 +766,12 @@ public class NetworkClientTest {
.setApiKeys(versionList)); .setApiKeys(versionList));
} }
@Test
public void testThrottlingNotEnabledForConnectionToOlderBroker() {
// Instrument the test so that the max protocol version for PRODUCE returned from the node is 5 and thus
// client-side throttling is not enabled. Also, return a response with a 100ms throttle delay.
setExpectedApiVersionsResponse(createExpectedApiVersionsResponse(PRODUCE, (short) 5));
while (!client.ready(node, time.milliseconds()))
client.poll(1, time.milliseconds());
selector.clear();
int correlationId = sendEmptyProduceRequest();
client.poll(1, time.milliseconds());
sendThrottledProduceResponse(correlationId, 100, (short) 5);
client.poll(1, time.milliseconds());
// Since client-side throttling is disabled, the connection is ready even though the response indicated a
// throttle delay.
assertTrue(client.ready(node, time.milliseconds()));
assertEquals(0, client.throttleDelayMs(node, time.milliseconds()));
}
private int sendEmptyProduceRequest() { private int sendEmptyProduceRequest() {
return sendEmptyProduceRequest(client, node.idString()); return sendEmptyProduceRequest(client, node.idString());
} }
private int sendEmptyProduceRequest(NetworkClient client, String nodeId) { private int sendEmptyProduceRequest(NetworkClient client, String nodeId) {
ProduceRequest.Builder builder = ProduceRequest.forCurrentMagic(new ProduceRequestData() ProduceRequest.Builder builder = ProduceRequest.builder(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection()) .setTopicData(new ProduceRequestData.TopicProduceDataCollection())
.setAcks((short) 1) .setAcks((short) 1)
.setTimeoutMs(1000)); .setTimeoutMs(1000));

View File

@ -102,16 +102,16 @@ public class NodeApiVersionsTest {
@Test @Test
public void testLatestUsableVersion() { public void testLatestUsableVersion() {
NodeApiVersions apiVersions = NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 1, (short) 3); NodeApiVersions apiVersions = NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 8, (short) 10);
assertEquals(3, apiVersions.latestUsableVersion(ApiKeys.PRODUCE)); assertEquals(10, apiVersions.latestUsableVersion(ApiKeys.PRODUCE));
assertEquals(1, apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 0, (short) 1)); assertEquals(8, apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 7, (short) 8));
assertEquals(1, apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 1, (short) 1)); assertEquals(8, apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 8, (short) 8));
assertEquals(2, apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 1, (short) 2)); assertEquals(9, apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 8, (short) 9));
assertEquals(3, apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 1, (short) 3)); assertEquals(10, apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 8, (short) 10));
assertEquals(2, apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 2, (short) 2)); assertEquals(9, apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 9, (short) 9));
assertEquals(3, apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 2, (short) 3)); assertEquals(10, apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 9, (short) 10));
assertEquals(3, apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 3, (short) 3)); assertEquals(10, apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 10, (short) 10));
assertEquals(3, apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 3, (short) 4)); assertEquals(10, apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 10, (short) 11));
} }
@Test @Test

View File

@ -677,7 +677,6 @@ public abstract class ConsumerCoordinatorTest {
.setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH) .setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH)
.setCommittedMetadata("") .setCommittedMetadata("")
.setCommittedOffset(13L) .setCommittedOffset(13L)
.setCommitTimestamp(0)
)) ))
) )
); );

View File

@ -24,7 +24,6 @@ import org.apache.kafka.clients.Metadata;
import org.apache.kafka.clients.MetadataRecoveryStrategy; import org.apache.kafka.clients.MetadataRecoveryStrategy;
import org.apache.kafka.clients.MockClient; import org.apache.kafka.clients.MockClient;
import org.apache.kafka.clients.NetworkClient; import org.apache.kafka.clients.NetworkClient;
import org.apache.kafka.clients.NodeApiVersions;
import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.OffsetOutOfRangeException; import org.apache.kafka.clients.consumer.OffsetOutOfRangeException;
@ -42,7 +41,6 @@ import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.compress.Compression; import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.errors.AuthenticationException; import org.apache.kafka.common.errors.AuthenticationException;
import org.apache.kafka.common.errors.DisconnectException; import org.apache.kafka.common.errors.DisconnectException;
import org.apache.kafka.common.errors.RecordTooLargeException;
import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.errors.SerializationException;
import org.apache.kafka.common.errors.TopicAuthorizationException; import org.apache.kafka.common.errors.TopicAuthorizationException;
import org.apache.kafka.common.header.Header; import org.apache.kafka.common.header.Header;
@ -1263,25 +1261,6 @@ public class FetchRequestManagerTest {
assertEquals(30L, consumerRecords.get(2).offset()); assertEquals(30L, consumerRecords.get(2).offset());
} }
/**
* Test the case where the client makes a pre-v3 FetchRequest, but the server replies with only a partial
* request. This happens when a single message is larger than the per-partition limit.
*/
@Test
public void testFetchRequestWhenRecordTooLarge() {
try {
buildFetcher();
client.setNodeApiVersions(NodeApiVersions.create(ApiKeys.FETCH.id, (short) 2, (short) 2));
makeFetchRequestWithIncompleteRecord();
assertThrows(RecordTooLargeException.class, this::collectFetch);
// the position should not advance since no data has been returned
assertEquals(0, subscriptions.position(tp0).offset);
} finally {
client.setNodeApiVersions(NodeApiVersions.create());
}
}
/** /**
* Test the case where the client makes a post KIP-74 FetchRequest, but the server replies with only a * Test the case where the client makes a post KIP-74 FetchRequest, but the server replies with only a
* partial request. For v3 and later FetchRequests, the implementation of KIP-74 changed the behavior * partial request. For v3 and later FetchRequests, the implementation of KIP-74 changed the behavior

View File

@ -37,7 +37,6 @@ import org.apache.kafka.common.TopicIdPartition;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.Uuid; import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.compress.Compression; import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.errors.RecordTooLargeException;
import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.errors.SerializationException;
import org.apache.kafka.common.errors.TopicAuthorizationException; import org.apache.kafka.common.errors.TopicAuthorizationException;
import org.apache.kafka.common.header.Header; import org.apache.kafka.common.header.Header;
@ -1261,25 +1260,6 @@ public class FetcherTest {
assertEquals(30L, consumerRecords.get(2).offset()); assertEquals(30L, consumerRecords.get(2).offset());
} }
/**
* Test the case where the client makes a pre-v3 FetchRequest, but the server replies with only a partial
* request. This happens when a single message is larger than the per-partition limit.
*/
@Test
public void testFetchRequestWhenRecordTooLarge() {
try {
buildFetcher();
client.setNodeApiVersions(NodeApiVersions.create(ApiKeys.FETCH.id, (short) 2, (short) 2));
makeFetchRequestWithIncompleteRecord();
assertThrows(RecordTooLargeException.class, this::collectFetch);
// the position should not advance since no data has been returned
assertEquals(0, subscriptions.position(tp0).offset);
} finally {
client.setNodeApiVersions(NodeApiVersions.create());
}
}
/** /**
* Test the case where the client makes a post KIP-74 FetchRequest, but the server replies with only a * Test the case where the client makes a post KIP-74 FetchRequest, but the server replies with only a
* partial request. For v3 and later FetchRequests, the implementation of KIP-74 changed the behavior * partial request. For v3 and later FetchRequests, the implementation of KIP-74 changed the behavior

View File

@ -1113,43 +1113,6 @@ public class OffsetFetcherTest {
assertNull(offsetAndTimestampMap.get(tp0)); assertNull(offsetAndTimestampMap.get(tp0));
} }
@Test
public void testGetOffsetsForTimesWithUnknownOffsetV0() {
buildFetcher();
// Empty map
assertTrue(offsetFetcher.offsetsForTimes(new HashMap<>(), time.timer(100L)).isEmpty());
// Unknown Offset
client.reset();
// Ensure metadata has both partition.
MetadataResponse initialMetadataUpdate = RequestTestUtils.metadataUpdateWithIds(1, singletonMap(topicName, 1), topicIds);
client.updateMetadata(initialMetadataUpdate);
// Force LIST_OFFSETS version 0
Node node = metadata.fetch().nodes().get(0);
apiVersions.update(node.idString(), NodeApiVersions.create(
ApiKeys.LIST_OFFSETS.id, (short) 0, (short) 0));
ListOffsetsResponseData data = new ListOffsetsResponseData()
.setThrottleTimeMs(0)
.setTopics(Collections.singletonList(new ListOffsetsTopicResponse()
.setName(tp0.topic())
.setPartitions(Collections.singletonList(new ListOffsetsPartitionResponse()
.setPartitionIndex(tp0.partition())
.setErrorCode(Errors.NONE.code())
.setTimestamp(ListOffsetsResponse.UNKNOWN_TIMESTAMP)
.setOldStyleOffsets(Collections.emptyList())))));
client.prepareResponseFrom(new ListOffsetsResponse(data),
metadata.fetch().leaderFor(tp0));
Map<TopicPartition, Long> timestampToSearch = new HashMap<>();
timestampToSearch.put(tp0, 0L);
Map<TopicPartition, OffsetAndTimestamp> offsetAndTimestampMap =
offsetFetcher.offsetsForTimes(timestampToSearch, time.timer(Long.MAX_VALUE));
assertTrue(offsetAndTimestampMap.containsKey(tp0));
assertNull(offsetAndTimestampMap.get(tp0));
}
@Test @Test
public void testOffsetValidationRequestGrouping() { public void testOffsetValidationRequestGrouping() {
buildFetcher(); buildFetcher();

View File

@ -28,7 +28,6 @@ import org.apache.kafka.common.Node;
import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.compress.Compression; import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
@ -79,7 +78,6 @@ import static java.util.Arrays.asList;
import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail; import static org.junit.jupiter.api.Assertions.fail;
@ -969,26 +967,6 @@ public class RecordAccumulatorTest {
assertFalse(drained.get(node1.id()).isEmpty(), "The batch should have been drained."); assertFalse(drained.get(node1.id()).isEmpty(), "The batch should have been drained.");
} }
@Test
public void testIdempotenceWithOldMagic() {
// Simulate talking to an older broker, ie. one which supports a lower magic.
ApiVersions apiVersions = new ApiVersions();
int batchSize = 1025;
int deliveryTimeoutMs = 3200;
int lingerMs = 10;
long retryBackoffMs = 100L;
long totalSize = 10 * batchSize;
String metricGrpName = "producer-metrics";
apiVersions.update("foobar", NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 2));
TransactionManager transactionManager = new TransactionManager(new LogContext(), null, 0, retryBackoffMs, apiVersions);
RecordAccumulator accum = new RecordAccumulator(logContext, batchSize + DefaultRecordBatch.RECORD_BATCH_OVERHEAD,
Compression.NONE, lingerMs, retryBackoffMs, retryBackoffMs, deliveryTimeoutMs, metrics, metricGrpName, time, apiVersions, transactionManager,
new BufferPool(totalSize, batchSize, metrics, time, metricGrpName));
assertThrows(UnsupportedVersionException.class,
() -> accum.append(topic, partition1, 0L, key, value, Record.EMPTY_HEADERS, null, 0, false, time.milliseconds(), cluster));
}
@Test @Test
public void testRecordsDrainedWhenTransactionCompleting() throws Exception { public void testRecordsDrainedWhenTransactionCompleting() throws Exception {
int batchSize = 1025; int batchSize = 1025;

View File

@ -206,91 +206,6 @@ public class SenderTest {
assertEquals(offset, future.get().offset()); assertEquals(offset, future.get().offset());
} }
@Test
public void testMessageFormatDownConversion() throws Exception {
// this test case verifies the behavior when the version of the produce request supported by the
// broker changes after the record set is created
long offset = 0;
// start off support produce request v3
apiVersions.update("0", NodeApiVersions.create());
Future<RecordMetadata> future = appendToAccumulator(tp0, 0L, "key", "value");
// now the partition leader supports only v2
apiVersions.update("0", NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 2));
client.prepareResponse(body -> {
ProduceRequest request = (ProduceRequest) body;
if (request.version() != 2)
return false;
MemoryRecords records = partitionRecords(request).get(tp0);
return records != null &&
records.sizeInBytes() > 0 &&
records.hasMatchingMagic(RecordBatch.MAGIC_VALUE_V1);
}, produceResponse(tp0, offset, Errors.NONE, 0));
sender.runOnce(); // connect
sender.runOnce(); // send produce request
assertTrue(future.isDone(), "Request should be completed");
assertEquals(offset, future.get().offset());
}
@SuppressWarnings("deprecation")
@Test
public void testDownConversionForMismatchedMagicValues() throws Exception {
// it can happen that we construct a record set with mismatching magic values (perhaps
// because the partition leader changed after the record set was initially constructed)
// in this case, we down-convert record sets with newer magic values to match the oldest
// created record set
long offset = 0;
// start off support produce request v3
apiVersions.update("0", NodeApiVersions.create());
Future<RecordMetadata> future1 = appendToAccumulator(tp0, 0L, "key", "value");
// now the partition leader supports only v2
apiVersions.update("0", NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 2));
Future<RecordMetadata> future2 = appendToAccumulator(tp1, 0L, "key", "value");
// start off support produce request v3
apiVersions.update("0", NodeApiVersions.create());
ProduceResponse.PartitionResponse resp = new ProduceResponse.PartitionResponse(Errors.NONE, offset, RecordBatch.NO_TIMESTAMP, 100);
Map<TopicPartition, ProduceResponse.PartitionResponse> partResp = new HashMap<>();
partResp.put(tp0, resp);
partResp.put(tp1, resp);
ProduceResponse produceResponse = new ProduceResponse(partResp, 0);
client.prepareResponse(body -> {
ProduceRequest request = (ProduceRequest) body;
if (request.version() != 2)
return false;
Map<TopicPartition, MemoryRecords> recordsMap = partitionRecords(request);
if (recordsMap.size() != 2)
return false;
for (MemoryRecords records : recordsMap.values()) {
if (records == null || records.sizeInBytes() == 0 || !records.hasMatchingMagic(RecordBatch.MAGIC_VALUE_V1))
return false;
}
return true;
}, produceResponse);
sender.runOnce(); // connect
sender.runOnce(); // send produce request
assertTrue(future1.isDone(), "Request should be completed");
assertTrue(future2.isDone(), "Request should be completed");
}
/* /*
* Send multiple requests. Verify that the client side quota metrics have the right values * Send multiple requests. Verify that the client side quota metrics have the right values
*/ */
@ -319,7 +234,7 @@ public class SenderTest {
for (int i = 1; i <= 3; i++) { for (int i = 1; i <= 3; i++) {
int throttleTimeMs = 100 * i; int throttleTimeMs = 100 * i;
ProduceRequest.Builder builder = ProduceRequest.forCurrentMagic(new ProduceRequestData() ProduceRequest.Builder builder = ProduceRequest.builder(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection()) .setTopicData(new ProduceRequestData.TopicProduceDataCollection())
.setAcks((short) 1) .setAcks((short) 1)
.setTimeoutMs(1000)); .setTimeoutMs(1000));

View File

@ -29,6 +29,7 @@ import java.util.Set;
import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail; import static org.junit.jupiter.api.Assertions.fail;
@ -97,21 +98,27 @@ public class ApiMessageTypeTest {
assertEquals((short) 1, ApiMessageType.CREATE_TOPICS.responseHeaderVersion((short) 5)); assertEquals((short) 1, ApiMessageType.CREATE_TOPICS.responseHeaderVersion((short) 5));
} }
/**
* Kafka currently supports direct upgrades from 0.8 to the latest version. As such, it has to support all apis
* starting from version 0 and we must have schemas from the oldest version to the latest.
*/
@Test @Test
public void testAllVersionsHaveSchemas() { public void testAllVersionsHaveSchemas() {
for (ApiMessageType type : ApiMessageType.values()) { for (ApiMessageType type : ApiMessageType.values()) {
assertEquals(0, type.lowestSupportedVersion()); assertTrue(type.lowestSupportedVersion() >= 0);
assertEquals(type.requestSchemas().length, type.responseSchemas().length, assertEquals(type.requestSchemas().length, type.responseSchemas().length,
"request and response schemas must be the same length for " + type.name()); "request and response schemas must be the same length for " + type.name());
for (Schema schema : type.requestSchemas()) for (int i = 0; i < type.requestSchemas().length; ++i) {
assertNotNull(schema); Schema schema = type.requestSchemas()[i];
for (Schema schema : type.responseSchemas()) if (i >= type.lowestSupportedVersion())
assertNotNull(schema); assertNotNull(schema);
else
assertNull(schema);
}
for (int i = 0; i < type.responseSchemas().length; ++i) {
Schema schema = type.responseSchemas()[i];
if (i >= type.lowestSupportedVersion())
assertNotNull(schema);
else
assertNull(schema);
}
assertEquals(type.highestSupportedVersion(true) + 1, type.requestSchemas().length); assertEquals(type.highestSupportedVersion(true) + 1, type.requestSchemas().length);
} }

View File

@ -82,7 +82,7 @@ public final class MessageTest {
private final String memberId = "memberId"; private final String memberId = "memberId";
private final String instanceId = "instanceId"; private final String instanceId = "instanceId";
private final List<Integer> listOfVersionsNonBatchOffsetFetch = Arrays.asList(0, 1, 2, 3, 4, 5, 6, 7); private final List<Integer> listOfVersionsNonBatchOffsetFetch = Arrays.asList(1, 2, 3, 4, 5, 6, 7);
@Test @Test
public void testAddOffsetsToTxnVersions() throws Exception { public void testAddOffsetsToTxnVersions() throws Exception {
@ -199,21 +199,17 @@ public final class MessageTest {
public void testListOffsetsResponseVersions() throws Exception { public void testListOffsetsResponseVersions() throws Exception {
ListOffsetsPartitionResponse partition = new ListOffsetsPartitionResponse() ListOffsetsPartitionResponse partition = new ListOffsetsPartitionResponse()
.setErrorCode(Errors.NONE.code()) .setErrorCode(Errors.NONE.code())
.setPartitionIndex(0) .setPartitionIndex(0);
.setOldStyleOffsets(Collections.singletonList(321L));
List<ListOffsetsTopicResponse> topics = Collections.singletonList(new ListOffsetsTopicResponse() List<ListOffsetsTopicResponse> topics = Collections.singletonList(new ListOffsetsTopicResponse()
.setName("topic") .setName("topic")
.setPartitions(Collections.singletonList(partition))); .setPartitions(Collections.singletonList(partition)));
Supplier<ListOffsetsResponseData> response = () -> new ListOffsetsResponseData() Supplier<ListOffsetsResponseData> response = () -> new ListOffsetsResponseData()
.setTopics(topics); .setTopics(topics);
for (short version : ApiKeys.LIST_OFFSETS.allVersions()) { for (short version = ApiKeys.LIST_OFFSETS.oldestVersion(); version <= ApiKeys.LIST_OFFSETS.latestVersion(); ++version) {
ListOffsetsResponseData responseData = response.get(); ListOffsetsResponseData responseData = response.get();
if (version > 0) { responseData.topics().get(0).partitions().get(0)
responseData.topics().get(0).partitions().get(0) .setOffset(456L)
.setOldStyleOffsets(Collections.emptyList()) .setTimestamp(123L);
.setOffset(456L)
.setTimestamp(123L);
}
if (version > 1) { if (version > 1) {
responseData.setThrottleTimeMs(1000); responseData.setThrottleTimeMs(1000);
} }
@ -399,7 +395,8 @@ public final class MessageTest {
.setPartitions(singletonList(partitionDataNoCurrentEpoch))); .setPartitions(singletonList(partitionDataNoCurrentEpoch)));
testAllMessageRoundTrips(data); testAllMessageRoundTrips(data);
testAllMessageRoundTripsBeforeVersion((short) 2, partitionDataWithCurrentEpoch, partitionDataNoCurrentEpoch); short lowestVersion = ApiKeys.OFFSET_FOR_LEADER_EPOCH.oldestVersion();
testAllMessageRoundTripsBetweenVersions(lowestVersion, (short) 2, partitionDataWithCurrentEpoch, partitionDataNoCurrentEpoch);
testAllMessageRoundTripsFromVersion((short) 2, partitionDataWithCurrentEpoch); testAllMessageRoundTripsFromVersion((short) 2, partitionDataWithCurrentEpoch);
// Version 3 adds the optional replica Id field // Version 3 adds the optional replica Id field
@ -475,22 +472,13 @@ public final class MessageTest {
.setCommittedLeaderEpoch(10) .setCommittedLeaderEpoch(10)
.setCommittedMetadata(metadata) .setCommittedMetadata(metadata)
.setCommittedOffset(offset) .setCommittedOffset(offset)
.setCommitTimestamp(20)
)))) ))))
.setRetentionTimeMs(20); .setRetentionTimeMs(20);
for (short version : ApiKeys.OFFSET_COMMIT.allVersions()) { for (short version : ApiKeys.OFFSET_COMMIT.allVersions()) {
OffsetCommitRequestData requestData = request.get(); OffsetCommitRequestData requestData = request.get();
if (version < 1) {
requestData.setMemberId("");
requestData.setGenerationIdOrMemberEpoch(-1);
}
if (version != 1) { if (version > 4) {
requestData.topics().get(0).partitions().get(0).setCommitTimestamp(-1);
}
if (version < 2 || version > 4) {
requestData.setRetentionTimeMs(-1); requestData.setRetentionTimeMs(-1);
} }
@ -502,9 +490,7 @@ public final class MessageTest {
requestData.setGroupInstanceId(null); requestData.setGroupInstanceId(null);
} }
if (version == 1) { if (version >= 2 && version <= 4) {
testEquivalentMessageRoundTrip(version, requestData);
} else if (version >= 2 && version <= 4) {
testAllMessageRoundTripsBetweenVersions(version, (short) 5, requestData, requestData); testAllMessageRoundTripsBetweenVersions(version, (short) 5, requestData, requestData);
} else { } else {
testAllMessageRoundTripsFromVersion(version, requestData); testAllMessageRoundTripsFromVersion(version, requestData);
@ -627,7 +613,7 @@ public final class MessageTest {
} }
@Test @Test
public void testOffsetFetchV0ToV7() throws Exception { public void testOffsetFetchV1ToV7() throws Exception {
String groupId = "groupId"; String groupId = "groupId";
String topicName = "topic"; String topicName = "topic";
@ -655,15 +641,15 @@ public final class MessageTest {
for (int version : listOfVersionsNonBatchOffsetFetch) { for (int version : listOfVersionsNonBatchOffsetFetch) {
final short finalVersion = (short) version; final short finalVersion = (short) version;
if (version < 2) { if (version < 2) {
assertThrows(NullPointerException.class, () -> testAllMessageRoundTripsOffsetFetchFromVersionV0ToV7(finalVersion, allPartitionData)); assertThrows(NullPointerException.class, () -> testAllMessageRoundTripsOffsetFetchFromVersionToV7(finalVersion, allPartitionData));
} else { } else {
testAllMessageRoundTripsOffsetFetchFromVersionV0ToV7((short) version, allPartitionData); testAllMessageRoundTripsOffsetFetchFromVersionToV7((short) version, allPartitionData);
} }
if (version < 7) { if (version < 7) {
assertThrows(UnsupportedVersionException.class, () -> testAllMessageRoundTripsOffsetFetchFromVersionV0ToV7(finalVersion, requireStableData)); assertThrows(UnsupportedVersionException.class, () -> testAllMessageRoundTripsOffsetFetchFromVersionToV7(finalVersion, requireStableData));
} else { } else {
testAllMessageRoundTripsOffsetFetchFromVersionV0ToV7(finalVersion, requireStableData); testAllMessageRoundTripsOffsetFetchFromVersionToV7(finalVersion, requireStableData);
} }
} }
@ -695,17 +681,17 @@ public final class MessageTest {
responseData.topics().get(0).partitions().get(0).setCommittedLeaderEpoch(-1); responseData.topics().get(0).partitions().get(0).setCommittedLeaderEpoch(-1);
} }
testAllMessageRoundTripsOffsetFetchFromVersionV0ToV7((short) version, responseData); testAllMessageRoundTripsOffsetFetchFromVersionToV7((short) version, responseData);
} }
} }
private void testAllMessageRoundTripsOffsetFetchV0ToV7(Message message) throws Exception { private void testAllMessageRoundTripsOffsetFetchV0ToV7(Message message) throws Exception {
testDuplication(message); testDuplication(message);
testAllMessageRoundTripsOffsetFetchFromVersionV0ToV7(message.lowestSupportedVersion(), message); testAllMessageRoundTripsOffsetFetchFromVersionToV7(message.lowestSupportedVersion(), message);
} }
private void testAllMessageRoundTripsOffsetFetchFromVersionV0ToV7(short fromVersion, private void testAllMessageRoundTripsOffsetFetchFromVersionToV7(short fromVersion,
Message message) throws Exception { Message message) throws Exception {
for (short version = fromVersion; version <= 7; version++) { for (short version = fromVersion; version <= 7; version++) {
testEquivalentMessageRoundTrip(version, message); testEquivalentMessageRoundTrip(version, message);
} }
@ -1131,15 +1117,16 @@ public final class MessageTest {
@Test @Test
public void testDefaultValues() { public void testDefaultValues() {
verifyWriteRaisesUve((short) 0, "validateOnly", verifyWriteSucceeds((short) 2,
new CreateTopicsRequestData().setValidateOnly(true));
verifyWriteSucceeds((short) 0,
new CreateTopicsRequestData().setValidateOnly(false));
verifyWriteSucceeds((short) 0,
new OffsetCommitRequestData().setRetentionTimeMs(123)); new OffsetCommitRequestData().setRetentionTimeMs(123));
verifyWriteRaisesUve((short) 5, "forgotten", verifyWriteRaisesUve((short) 5, "forgotten",
new FetchRequestData().setForgottenTopicsData(singletonList( new FetchRequestData().setForgottenTopicsData(singletonList(
new FetchRequestData.ForgottenTopic().setTopic("foo")))); new FetchRequestData.ForgottenTopic().setTopic("foo"))));
verifyWriteSucceeds((short) 5, new FetchRequestData());
verifyWriteSucceeds((short) 7,
new FetchRequestData().setForgottenTopicsData(singletonList(
new FetchRequestData.ForgottenTopic().setTopic("foo"))));
} }
@Test @Test
@ -1167,8 +1154,6 @@ public final class MessageTest {
for (short version : ApiKeys.CREATE_TOPICS.allVersions()) { for (short version : ApiKeys.CREATE_TOPICS.allVersions()) {
verifyWriteRaisesNpe(version, createTopics); verifyWriteRaisesNpe(version, createTopics);
} }
MetadataRequestData metadata = new MetadataRequestData().setTopics(null);
verifyWriteRaisesNpe((short) 0, metadata);
} }
@Test @Test

View File

@ -88,8 +88,8 @@ public class ApiVersionsResponseTest {
public void shouldHaveCommonlyAgreedApiVersionResponseWithControllerOnForwardableAPIs() { public void shouldHaveCommonlyAgreedApiVersionResponseWithControllerOnForwardableAPIs() {
final ApiKeys forwardableAPIKey = ApiKeys.CREATE_ACLS; final ApiKeys forwardableAPIKey = ApiKeys.CREATE_ACLS;
final ApiKeys nonForwardableAPIKey = ApiKeys.JOIN_GROUP; final ApiKeys nonForwardableAPIKey = ApiKeys.JOIN_GROUP;
final short minVersion = 0; final short minVersion = 2;
final short maxVersion = 1; final short maxVersion = 3;
Map<ApiKeys, ApiVersion> activeControllerApiVersions = Utils.mkMap( Map<ApiKeys, ApiVersion> activeControllerApiVersions = Utils.mkMap(
Utils.mkEntry(forwardableAPIKey, new ApiVersion() Utils.mkEntry(forwardableAPIKey, new ApiVersion()
.setApiKey(forwardableAPIKey.id) .setApiKey(forwardableAPIKey.id)

View File

@ -60,17 +60,7 @@ public class CreateAclsRequestTest {
@Test @Test
public void shouldThrowOnIfUnknown() { public void shouldThrowOnIfUnknown() {
assertThrows(IllegalArgumentException.class, () -> new CreateAclsRequest(data(UNKNOWN_ACL1), V0)); assertThrows(IllegalArgumentException.class, () -> new CreateAclsRequest(data(UNKNOWN_ACL1), V1));
}
@Test
public void shouldRoundTripV0() {
final CreateAclsRequest original = new CreateAclsRequest(data(LITERAL_ACL1, LITERAL_ACL2), V0);
final ByteBuffer buffer = original.serialize();
final CreateAclsRequest result = CreateAclsRequest.parse(buffer, V0);
assertRequestEquals(original, result);
} }
@Test @Test

View File

@ -62,32 +62,6 @@ public class DeleteAclsRequestTest {
assertThrows(IllegalArgumentException.class, () -> new DeleteAclsRequest.Builder(requestData(UNKNOWN_FILTER)).build(V1)); assertThrows(IllegalArgumentException.class, () -> new DeleteAclsRequest.Builder(requestData(UNKNOWN_FILTER)).build(V1));
} }
@Test
public void shouldRoundTripLiteralV0() {
final DeleteAclsRequest original = new DeleteAclsRequest.Builder(requestData(LITERAL_FILTER)).build(V0);
final ByteBuffer buffer = original.serialize();
final DeleteAclsRequest result = DeleteAclsRequest.parse(buffer, V0);
assertRequestEquals(original, result);
}
@Test
public void shouldRoundTripAnyV0AsLiteral() {
final DeleteAclsRequest original = new DeleteAclsRequest.Builder(requestData(ANY_FILTER)).build(V0);
final DeleteAclsRequest expected = new DeleteAclsRequest.Builder(requestData(
new AclBindingFilter(new ResourcePatternFilter(
ANY_FILTER.patternFilter().resourceType(),
ANY_FILTER.patternFilter().name(),
PatternType.LITERAL),
ANY_FILTER.entryFilter()))
).build(V0);
final DeleteAclsRequest result = DeleteAclsRequest.parse(original.serialize(), V0);
assertRequestEquals(expected, result);
}
@Test @Test
public void shouldRoundTripV1() { public void shouldRoundTripV1() {
final DeleteAclsRequest original = new DeleteAclsRequest.Builder( final DeleteAclsRequest original = new DeleteAclsRequest.Builder(

View File

@ -19,7 +19,6 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.acl.AclOperation; import org.apache.kafka.common.acl.AclOperation;
import org.apache.kafka.common.acl.AclPermissionType; import org.apache.kafka.common.acl.AclPermissionType;
import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.message.DeleteAclsResponseData; import org.apache.kafka.common.message.DeleteAclsResponseData;
import org.apache.kafka.common.message.DeleteAclsResponseData.DeleteAclsFilterResult; import org.apache.kafka.common.message.DeleteAclsResponseData.DeleteAclsFilterResult;
import org.apache.kafka.common.message.DeleteAclsResponseData.DeleteAclsMatchingAcl; import org.apache.kafka.common.message.DeleteAclsResponseData.DeleteAclsMatchingAcl;
@ -36,7 +35,6 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertThrows;
public class DeleteAclsResponseTest { public class DeleteAclsResponseTest {
private static final short V0 = 0;
private static final short V1 = 1; private static final short V1 = 1;
private static final DeleteAclsMatchingAcl LITERAL_ACL1 = new DeleteAclsMatchingAcl() private static final DeleteAclsMatchingAcl LITERAL_ACL1 = new DeleteAclsMatchingAcl()
@ -84,15 +82,6 @@ public class DeleteAclsResponseTest {
private static final DeleteAclsFilterResult UNKNOWN_RESPONSE = new DeleteAclsFilterResult().setMatchingAcls(singletonList( private static final DeleteAclsFilterResult UNKNOWN_RESPONSE = new DeleteAclsFilterResult().setMatchingAcls(singletonList(
UNKNOWN_ACL)); UNKNOWN_ACL));
@Test
public void shouldThrowOnV0IfNotLiteral() {
assertThrows(UnsupportedVersionException.class, () -> new DeleteAclsResponse(
new DeleteAclsResponseData()
.setThrottleTimeMs(10)
.setFilterResults(singletonList(PREFIXED_RESPONSE)),
V0));
}
@Test @Test
public void shouldThrowOnIfUnknown() { public void shouldThrowOnIfUnknown() {
assertThrows(IllegalArgumentException.class, () -> new DeleteAclsResponse( assertThrows(IllegalArgumentException.class, () -> new DeleteAclsResponse(
@ -102,19 +91,6 @@ public class DeleteAclsResponseTest {
V1)); V1));
} }
@Test
public void shouldRoundTripV0() {
final DeleteAclsResponse original = new DeleteAclsResponse(
new DeleteAclsResponseData()
.setThrottleTimeMs(10)
.setFilterResults(singletonList(LITERAL_RESPONSE)),
V0);
final ByteBuffer buffer = original.serialize(V0);
final DeleteAclsResponse result = DeleteAclsResponse.parse(buffer, V0);
assertEquals(original.filterResults(), result.filterResults());
}
@Test @Test
public void shouldRoundTripV1() { public void shouldRoundTripV1() {
final DeleteAclsResponse original = new DeleteAclsResponse( final DeleteAclsResponse original = new DeleteAclsResponse(

View File

@ -21,7 +21,6 @@ import org.apache.kafka.common.acl.AccessControlEntryFilter;
import org.apache.kafka.common.acl.AclBindingFilter; import org.apache.kafka.common.acl.AclBindingFilter;
import org.apache.kafka.common.acl.AclOperation; import org.apache.kafka.common.acl.AclOperation;
import org.apache.kafka.common.acl.AclPermissionType; import org.apache.kafka.common.acl.AclPermissionType;
import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.resource.PatternType; import org.apache.kafka.common.resource.PatternType;
import org.apache.kafka.common.resource.ResourcePatternFilter; import org.apache.kafka.common.resource.ResourcePatternFilter;
import org.apache.kafka.common.resource.ResourceType; import org.apache.kafka.common.resource.ResourceType;
@ -32,7 +31,6 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertThrows;
public class DescribeAclsRequestTest { public class DescribeAclsRequestTest {
private static final short V0 = 0;
private static final short V1 = 1; private static final short V1 = 1;
private static final AclBindingFilter LITERAL_FILTER = new AclBindingFilter(new ResourcePatternFilter(ResourceType.TOPIC, "foo", PatternType.LITERAL), private static final AclBindingFilter LITERAL_FILTER = new AclBindingFilter(new ResourcePatternFilter(ResourceType.TOPIC, "foo", PatternType.LITERAL),
@ -47,43 +45,9 @@ public class DescribeAclsRequestTest {
private static final AclBindingFilter UNKNOWN_FILTER = new AclBindingFilter(new ResourcePatternFilter(ResourceType.UNKNOWN, "foo", PatternType.LITERAL), private static final AclBindingFilter UNKNOWN_FILTER = new AclBindingFilter(new ResourcePatternFilter(ResourceType.UNKNOWN, "foo", PatternType.LITERAL),
new AccessControlEntryFilter("User:ANONYMOUS", "127.0.0.1", AclOperation.READ, AclPermissionType.DENY)); new AccessControlEntryFilter("User:ANONYMOUS", "127.0.0.1", AclOperation.READ, AclPermissionType.DENY));
@Test
public void shouldThrowOnV0IfPrefixed() {
assertThrows(UnsupportedVersionException.class, () -> new DescribeAclsRequest.Builder(PREFIXED_FILTER).build(V0));
}
@Test @Test
public void shouldThrowIfUnknown() { public void shouldThrowIfUnknown() {
assertThrows(IllegalArgumentException.class, () -> new DescribeAclsRequest.Builder(UNKNOWN_FILTER).build(V0)); assertThrows(IllegalArgumentException.class, () -> new DescribeAclsRequest.Builder(UNKNOWN_FILTER).build(V1));
}
@Test
public void shouldRoundTripLiteralV0() {
final DescribeAclsRequest original = new DescribeAclsRequest.Builder(LITERAL_FILTER).build(V0);
final DescribeAclsRequest result = DescribeAclsRequest.parse(original.serialize(), V0);
assertRequestEquals(original, result);
}
@Test
public void shouldRoundTripAnyV0AsLiteral() {
final DescribeAclsRequest original = new DescribeAclsRequest.Builder(ANY_FILTER).build(V0);
final DescribeAclsRequest expected = new DescribeAclsRequest.Builder(
new AclBindingFilter(new ResourcePatternFilter(
ANY_FILTER.patternFilter().resourceType(),
ANY_FILTER.patternFilter().name(),
PatternType.LITERAL),
ANY_FILTER.entryFilter())).build(V0);
final DescribeAclsRequest result = DescribeAclsRequest.parse(original.serialize(), V0);
assertRequestEquals(expected, result);
}
@Test
public void shouldRoundTripLiteralV1() {
final DescribeAclsRequest original = new DescribeAclsRequest.Builder(LITERAL_FILTER).build(V1);
final DescribeAclsRequest result = DescribeAclsRequest.parse(original.serialize(), V1);
assertRequestEquals(original, result);
} }
@Test @Test

View File

@ -21,7 +21,6 @@ import org.apache.kafka.common.acl.AccessControlEntry;
import org.apache.kafka.common.acl.AclBinding; import org.apache.kafka.common.acl.AclBinding;
import org.apache.kafka.common.acl.AclOperation; import org.apache.kafka.common.acl.AclOperation;
import org.apache.kafka.common.acl.AclPermissionType; import org.apache.kafka.common.acl.AclPermissionType;
import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.message.DescribeAclsResponseData; import org.apache.kafka.common.message.DescribeAclsResponseData;
import org.apache.kafka.common.message.DescribeAclsResponseData.AclDescription; import org.apache.kafka.common.message.DescribeAclsResponseData.AclDescription;
import org.apache.kafka.common.message.DescribeAclsResponseData.DescribeAclsResource; import org.apache.kafka.common.message.DescribeAclsResponseData.DescribeAclsResource;
@ -43,7 +42,6 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertThrows;
public class DescribeAclsResponseTest { public class DescribeAclsResponseTest {
private static final short V0 = 0;
private static final short V1 = 1; private static final short V1 = 1;
private static final AclDescription ALLOW_CREATE_ACL = buildAclDescription( private static final AclDescription ALLOW_CREATE_ACL = buildAclDescription(
@ -82,30 +80,10 @@ public class DescribeAclsResponseTest {
PatternType.LITERAL, PatternType.LITERAL,
Collections.singletonList(DENY_READ_ACL)); Collections.singletonList(DENY_READ_ACL));
@Test
public void shouldThrowOnV0IfNotLiteral() {
assertThrows(UnsupportedVersionException.class,
() -> buildResponse(10, Errors.NONE, Collections.singletonList(PREFIXED_ACL1)).serialize(V0));
}
@Test @Test
public void shouldThrowIfUnknown() { public void shouldThrowIfUnknown() {
assertThrows(IllegalArgumentException.class, assertThrows(IllegalArgumentException.class,
() -> buildResponse(10, Errors.NONE, Collections.singletonList(UNKNOWN_ACL)).serialize(V0)); () -> buildResponse(10, Errors.NONE, Collections.singletonList(UNKNOWN_ACL)).serialize(V1));
}
@Test
public void shouldRoundTripV0() {
List<DescribeAclsResource> resources = Arrays.asList(LITERAL_ACL1, LITERAL_ACL2);
final DescribeAclsResponse original = buildResponse(10, Errors.NONE, resources);
final ByteBuffer buffer = original.serialize(V0);
final DescribeAclsResponse result = DescribeAclsResponse.parse(buffer, V0);
assertResponseEquals(original, result);
final DescribeAclsResponse result2 = buildResponse(10, Errors.NONE, DescribeAclsResponse.aclsResources(
DescribeAclsResponse.aclBindings(resources)));
assertResponseEquals(original, result2);
} }
@Test @Test

View File

@ -19,15 +19,12 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.errors.InvalidConfigurationException; import org.apache.kafka.common.errors.InvalidConfigurationException;
import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.message.JoinGroupRequestData; import org.apache.kafka.common.message.JoinGroupRequestData;
import org.apache.kafka.common.protocol.MessageUtil;
import org.apache.kafka.test.TestUtils; import org.apache.kafka.test.TestUtils;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import java.nio.ByteBuffer;
import java.util.Arrays; import java.util.Arrays;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.fail; import static org.junit.jupiter.api.Assertions.fail;
@ -68,20 +65,4 @@ public class JoinGroupRequestTest {
.setProtocolType("consumer") .setProtocolType("consumer")
).build((short) 4)); ).build((short) 4));
} }
@Test
public void testRebalanceTimeoutDefaultsToSessionTimeoutV0() {
int sessionTimeoutMs = 30000;
short version = 0;
ByteBuffer buffer = MessageUtil.toByteBuffer(new JoinGroupRequestData()
.setGroupId("groupId")
.setMemberId("consumerId")
.setProtocolType("consumer")
.setSessionTimeoutMs(sessionTimeoutMs), version);
JoinGroupRequest request = JoinGroupRequest.parse(buffer, version);
assertEquals(sessionTimeoutMs, request.data().sessionTimeoutMs());
assertEquals(sessionTimeoutMs, request.data().rebalanceTimeoutMs());
}
} }

View File

@ -54,7 +54,7 @@ public class ListOffsetsRequestTest {
ListOffsetsRequestData data = new ListOffsetsRequestData() ListOffsetsRequestData data = new ListOffsetsRequestData()
.setTopics(topics) .setTopics(topics)
.setReplicaId(-1); .setReplicaId(-1);
ListOffsetsRequest request = ListOffsetsRequest.parse(MessageUtil.toByteBuffer(data, (short) 0), (short) 0); ListOffsetsRequest request = ListOffsetsRequest.parse(MessageUtil.toByteBuffer(data, (short) 1), (short) 1);
assertEquals(Collections.singleton(new TopicPartition("topic", 0)), request.duplicatePartitions()); assertEquals(Collections.singleton(new TopicPartition("topic", 0)), request.duplicatePartitions());
assertEquals(0, data.timeoutMs()); // default value assertEquals(0, data.timeoutMs()); // default value
} }
@ -93,47 +93,15 @@ public class ListOffsetsRequestTest {
} }
} }
@Test
public void testGetErrorResponseV0() {
List<ListOffsetsTopic> topics = Collections.singletonList(
new ListOffsetsTopic()
.setName("topic")
.setPartitions(Collections.singletonList(
new ListOffsetsPartition()
.setPartitionIndex(0))));
ListOffsetsRequest request = ListOffsetsRequest.Builder
.forConsumer(true, IsolationLevel.READ_UNCOMMITTED)
.setTargetTimes(topics)
.build((short) 0);
ListOffsetsResponse response = (ListOffsetsResponse) request.getErrorResponse(0, Errors.NOT_LEADER_OR_FOLLOWER.exception());
List<ListOffsetsTopicResponse> v = Collections.singletonList(
new ListOffsetsTopicResponse()
.setName("topic")
.setPartitions(Collections.singletonList(
new ListOffsetsPartitionResponse()
.setErrorCode(Errors.NOT_LEADER_OR_FOLLOWER.code())
.setOldStyleOffsets(Collections.emptyList())
.setPartitionIndex(0))));
ListOffsetsResponseData data = new ListOffsetsResponseData()
.setThrottleTimeMs(0)
.setTopics(v);
ListOffsetsResponse expectedResponse = new ListOffsetsResponse(data);
assertEquals(expectedResponse.data().topics(), response.data().topics());
assertEquals(expectedResponse.throttleTimeMs(), response.throttleTimeMs());
}
@Test @Test
public void testToListOffsetsTopics() { public void testToListOffsetsTopics() {
ListOffsetsPartition lop0 = new ListOffsetsPartition() ListOffsetsPartition lop0 = new ListOffsetsPartition()
.setPartitionIndex(0) .setPartitionIndex(0)
.setCurrentLeaderEpoch(1) .setCurrentLeaderEpoch(1)
.setMaxNumOffsets(2)
.setTimestamp(123L); .setTimestamp(123L);
ListOffsetsPartition lop1 = new ListOffsetsPartition() ListOffsetsPartition lop1 = new ListOffsetsPartition()
.setPartitionIndex(1) .setPartitionIndex(1)
.setCurrentLeaderEpoch(3) .setCurrentLeaderEpoch(3)
.setMaxNumOffsets(4)
.setTimestamp(567L); .setTimestamp(567L);
Map<TopicPartition, ListOffsetsPartition> timestampsToSearch = new HashMap<>(); Map<TopicPartition, ListOffsetsPartition> timestampsToSearch = new HashMap<>();
timestampsToSearch.put(new TopicPartition("topic", 0), lop0); timestampsToSearch.put(new TopicPartition("topic", 0), lop0);
@ -174,4 +142,4 @@ public class ListOffsetsRequestTest {
assertEquals((short) 8, requireEarliestLocalTimestampRequestBuilder.oldestAllowedVersion()); assertEquals((short) 8, requireEarliestLocalTimestampRequestBuilder.oldestAllowedVersion());
assertEquals((short) 9, requireTieredStorageTimestampRequestBuilder.oldestAllowedVersion()); assertEquals((short) 9, requireTieredStorageTimestampRequestBuilder.oldestAllowedVersion());
} }
} }

View File

@ -30,23 +30,13 @@ import java.util.List;
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
public class MetadataRequestTest { public class MetadataRequestTest {
@Test @Test
public void testEmptyMeansAllTopicsV0() { public void testEmptyMeansEmptyForAllVersions() {
MetadataRequestData data = new MetadataRequestData(); for (int i = ApiKeys.METADATA.oldestVersion(); i < MetadataRequestData.SCHEMAS.length; i++) {
MetadataRequest parsedRequest = new MetadataRequest(data, (short) 0);
assertTrue(parsedRequest.isAllTopics());
assertNull(parsedRequest.topics());
}
@Test
public void testEmptyMeansEmptyForVersionsAboveV0() {
for (int i = 1; i < MetadataRequestData.SCHEMAS.length; i++) {
MetadataRequestData data = new MetadataRequestData(); MetadataRequestData data = new MetadataRequestData();
data.setAllowAutoTopicCreation(true); data.setAllowAutoTopicCreation(true);
MetadataRequest parsedRequest = new MetadataRequest(data, (short) i); MetadataRequest parsedRequest = new MetadataRequest(data, (short) i);

View File

@ -92,7 +92,7 @@ public class OffsetCommitRequestTest {
OffsetCommitRequest.Builder builder = new OffsetCommitRequest.Builder(data); OffsetCommitRequest.Builder builder = new OffsetCommitRequest.Builder(data);
for (short version : ApiKeys.TXN_OFFSET_COMMIT.allVersions()) { for (short version : ApiKeys.OFFSET_COMMIT.allVersions()) {
OffsetCommitRequest request = builder.build(version); OffsetCommitRequest request = builder.build(version);
assertEquals(expectedOffsets, request.offsets()); assertEquals(expectedOffsets, request.offsets());

View File

@ -25,7 +25,6 @@ import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.record.MemoryRecordsBuilder; import org.apache.kafka.common.record.MemoryRecordsBuilder;
import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.record.RecordBatch;
import org.apache.kafka.common.record.RecordVersion;
import org.apache.kafka.common.record.SimpleRecord; import org.apache.kafka.common.record.SimpleRecord;
import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.record.TimestampType;
@ -52,7 +51,7 @@ public class ProduceRequestTest {
final MemoryRecords memoryRecords = MemoryRecords.withTransactionalRecords(0, Compression.NONE, 1L, final MemoryRecords memoryRecords = MemoryRecords.withTransactionalRecords(0, Compression.NONE, 1L,
(short) 1, 1, 1, simpleRecord); (short) 1, 1, 1, simpleRecord);
final ProduceRequest request = ProduceRequest.forCurrentMagic(new ProduceRequestData() final ProduceRequest request = ProduceRequest.builder(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
new ProduceRequestData.TopicProduceData() new ProduceRequestData.TopicProduceData()
.setName("topic") .setName("topic")
@ -81,7 +80,7 @@ public class ProduceRequestTest {
public void shouldBeFlaggedAsIdempotentWhenIdempotentRecords() { public void shouldBeFlaggedAsIdempotentWhenIdempotentRecords() {
final MemoryRecords memoryRecords = MemoryRecords.withIdempotentRecords(1, Compression.NONE, 1L, final MemoryRecords memoryRecords = MemoryRecords.withIdempotentRecords(1, Compression.NONE, 1L,
(short) 1, 1, 1, simpleRecord); (short) 1, 1, 1, simpleRecord);
final ProduceRequest request = ProduceRequest.forCurrentMagic(new ProduceRequestData() final ProduceRequest request = ProduceRequest.builder(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
new ProduceRequestData.TopicProduceData() new ProduceRequestData.TopicProduceData()
.setName("topic") .setName("topic")
@ -94,32 +93,13 @@ public class ProduceRequestTest {
assertTrue(RequestTestUtils.hasIdempotentRecords(request)); assertTrue(RequestTestUtils.hasIdempotentRecords(request));
} }
@Test
public void testBuildWithOldMessageFormat() {
ByteBuffer buffer = ByteBuffer.allocate(256);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V1, Compression.NONE,
TimestampType.CREATE_TIME, 0L);
builder.append(10L, null, "a".getBytes());
ProduceRequest.Builder requestBuilder = ProduceRequest.forMagic(RecordBatch.MAGIC_VALUE_V1,
new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
new ProduceRequestData.TopicProduceData().setName("test").setPartitionData(Collections.singletonList(
new ProduceRequestData.PartitionProduceData().setIndex(9).setRecords(builder.build()))))
.iterator()))
.setAcks((short) 1)
.setTimeoutMs(5000),
true);
assertEquals(2, requestBuilder.oldestAllowedVersion());
assertEquals(2, requestBuilder.latestAllowedVersion());
}
@Test @Test
public void testBuildWithCurrentMessageFormat() { public void testBuildWithCurrentMessageFormat() {
ByteBuffer buffer = ByteBuffer.allocate(256); ByteBuffer buffer = ByteBuffer.allocate(256);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE,
Compression.NONE, TimestampType.CREATE_TIME, 0L); Compression.NONE, TimestampType.CREATE_TIME, 0L);
builder.append(10L, null, "a".getBytes()); builder.append(10L, null, "a".getBytes());
ProduceRequest.Builder requestBuilder = ProduceRequest.forMagic(RecordBatch.CURRENT_MAGIC_VALUE, ProduceRequest.Builder requestBuilder = ProduceRequest.builder(
new ProduceRequestData() new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
new ProduceRequestData.TopicProduceData().setName("test").setPartitionData(Collections.singletonList( new ProduceRequestData.TopicProduceData().setName("test").setPartitionData(Collections.singletonList(
@ -146,7 +126,7 @@ public class ProduceRequestTest {
buffer.flip(); buffer.flip();
ProduceRequest.Builder requestBuilder = ProduceRequest.forCurrentMagic(new ProduceRequestData() ProduceRequest.Builder requestBuilder = ProduceRequest.builder(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
new ProduceRequestData.TopicProduceData() new ProduceRequestData.TopicProduceData()
.setName("test") .setName("test")
@ -161,7 +141,7 @@ public class ProduceRequestTest {
@Test @Test
public void testV3AndAboveCannotHaveNoRecordBatches() { public void testV3AndAboveCannotHaveNoRecordBatches() {
ProduceRequest.Builder requestBuilder = ProduceRequest.forCurrentMagic(new ProduceRequestData() ProduceRequest.Builder requestBuilder = ProduceRequest.builder(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
new ProduceRequestData.TopicProduceData() new ProduceRequestData.TopicProduceData()
.setName("test") .setName("test")
@ -181,7 +161,7 @@ public class ProduceRequestTest {
TimestampType.NO_TIMESTAMP_TYPE, 0L); TimestampType.NO_TIMESTAMP_TYPE, 0L);
builder.append(10L, null, "a".getBytes()); builder.append(10L, null, "a".getBytes());
ProduceRequest.Builder requestBuilder = ProduceRequest.forCurrentMagic(new ProduceRequestData() ProduceRequest.Builder requestBuilder = ProduceRequest.builder(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
new ProduceRequestData.TopicProduceData() new ProduceRequestData.TopicProduceData()
.setName("test") .setName("test")
@ -201,7 +181,7 @@ public class ProduceRequestTest {
TimestampType.CREATE_TIME, 0L); TimestampType.CREATE_TIME, 0L);
builder.append(10L, null, "a".getBytes()); builder.append(10L, null, "a".getBytes());
ProduceRequest.Builder requestBuilder = ProduceRequest.forCurrentMagic(new ProduceRequestData() ProduceRequest.Builder requestBuilder = ProduceRequest.builder(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
new ProduceRequestData.TopicProduceData() new ProduceRequestData.TopicProduceData()
.setName("test") .setName("test")
@ -239,7 +219,7 @@ public class ProduceRequestTest {
} }
// Works fine with current version (>= 7) // Works fine with current version (>= 7)
ProduceRequest.forCurrentMagic(produceData); ProduceRequest.builder(produceData);
} }
@Test @Test
@ -253,7 +233,7 @@ public class ProduceRequestTest {
final MemoryRecords txnRecords = MemoryRecords.withTransactionalRecords(Compression.NONE, producerId, final MemoryRecords txnRecords = MemoryRecords.withTransactionalRecords(Compression.NONE, producerId,
producerEpoch, sequence, new SimpleRecord("bar".getBytes())); producerEpoch, sequence, new SimpleRecord("bar".getBytes()));
ProduceRequest.Builder builder = ProduceRequest.forMagic(RecordBatch.CURRENT_MAGIC_VALUE, ProduceRequest.Builder builder = ProduceRequest.builder(
new ProduceRequestData() new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Arrays.asList( .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Arrays.asList(
new ProduceRequestData.TopicProduceData().setName("foo").setPartitionData(Collections.singletonList( new ProduceRequestData.TopicProduceData().setName("foo").setPartitionData(Collections.singletonList(
@ -280,7 +260,7 @@ public class ProduceRequestTest {
final MemoryRecords idempotentRecords = MemoryRecords.withIdempotentRecords(Compression.NONE, producerId, final MemoryRecords idempotentRecords = MemoryRecords.withIdempotentRecords(Compression.NONE, producerId,
producerEpoch, sequence, new SimpleRecord("bar".getBytes())); producerEpoch, sequence, new SimpleRecord("bar".getBytes()));
ProduceRequest.Builder builder = ProduceRequest.forMagic(RecordVersion.current().value, ProduceRequest.Builder builder = ProduceRequest.builder(
new ProduceRequestData() new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Arrays.asList( .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Arrays.asList(
new ProduceRequestData.TopicProduceData().setName("foo").setPartitionData(Collections.singletonList( new ProduceRequestData.TopicProduceData().setName("foo").setPartitionData(Collections.singletonList(
@ -304,7 +284,7 @@ public class ProduceRequestTest {
} }
private ProduceRequest createNonIdempotentNonTransactionalRecords() { private ProduceRequest createNonIdempotentNonTransactionalRecords() {
return ProduceRequest.forCurrentMagic(new ProduceRequestData() return ProduceRequest.builder(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
new ProduceRequestData.TopicProduceData() new ProduceRequestData.TopicProduceData()
.setName("topic") .setName("topic")

View File

@ -19,13 +19,11 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.message.ProduceResponseData; import org.apache.kafka.common.message.ProduceResponseData;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.record.RecordBatch;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import java.nio.ByteBuffer;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
@ -39,36 +37,6 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
public class ProduceResponseTest { public class ProduceResponseTest {
@SuppressWarnings("deprecation")
@Test
public void produceResponseV5Test() {
Map<TopicPartition, ProduceResponse.PartitionResponse> responseData = new HashMap<>();
TopicPartition tp0 = new TopicPartition("test", 0);
responseData.put(tp0, new ProduceResponse.PartitionResponse(Errors.NONE, 10000, RecordBatch.NO_TIMESTAMP, 100));
ProduceResponse v5Response = new ProduceResponse(responseData, 10);
short version = 5;
ByteBuffer buffer = RequestTestUtils.serializeResponseWithHeader(v5Response, version, 0);
ResponseHeader.parse(buffer, ApiKeys.PRODUCE.responseHeaderVersion(version)); // throw away.
ProduceResponse v5FromBytes = (ProduceResponse) AbstractResponse.parseResponse(ApiKeys.PRODUCE, buffer, version);
assertEquals(1, v5FromBytes.data().responses().size());
ProduceResponseData.TopicProduceResponse topicProduceResponse = v5FromBytes.data().responses().iterator().next();
assertEquals(1, topicProduceResponse.partitionResponses().size());
ProduceResponseData.PartitionProduceResponse partitionProduceResponse = topicProduceResponse.partitionResponses().iterator().next();
TopicPartition tp = new TopicPartition(topicProduceResponse.name(), partitionProduceResponse.index());
assertEquals(tp0, tp);
assertEquals(100, partitionProduceResponse.logStartOffset());
assertEquals(10000, partitionProduceResponse.baseOffset());
assertEquals(RecordBatch.NO_TIMESTAMP, partitionProduceResponse.logAppendTimeMs());
assertEquals(Errors.NONE, Errors.forCode(partitionProduceResponse.errorCode()));
assertNull(partitionProduceResponse.errorMessage());
assertTrue(partitionProduceResponse.recordErrors().isEmpty());
}
@SuppressWarnings("deprecation") @SuppressWarnings("deprecation")
@Test @Test
public void produceResponseVersionTest() { public void produceResponseVersionTest() {

View File

@ -117,7 +117,7 @@ public class RequestContextTest {
@Test @Test
public void testInvalidRequestForImplicitHashCollection() throws UnknownHostException { public void testInvalidRequestForImplicitHashCollection() throws UnknownHostException {
short version = (short) 5; // choose a version with fixed length encoding, for simplicity short version = (short) 7; // choose a version with fixed length encoding, for simplicity
ByteBuffer corruptBuffer = produceRequest(version); ByteBuffer corruptBuffer = produceRequest(version);
// corrupt the length of the topics array // corrupt the length of the topics array
corruptBuffer.putInt(8, (Integer.MAX_VALUE - 1) / 2); corruptBuffer.putInt(8, (Integer.MAX_VALUE - 1) / 2);
@ -134,7 +134,7 @@ public class RequestContextTest {
@Test @Test
public void testInvalidRequestForArrayList() throws UnknownHostException { public void testInvalidRequestForArrayList() throws UnknownHostException {
short version = (short) 5; // choose a version with fixed length encoding, for simplicity short version = (short) 7; // choose a version with fixed length encoding, for simplicity
ByteBuffer corruptBuffer = produceRequest(version); ByteBuffer corruptBuffer = produceRequest(version);
// corrupt the length of the partitions array // corrupt the length of the partitions array
corruptBuffer.putInt(17, Integer.MAX_VALUE); corruptBuffer.putInt(17, Integer.MAX_VALUE);

View File

@ -151,7 +151,6 @@ import org.apache.kafka.common.message.EndTxnResponseData;
import org.apache.kafka.common.message.EnvelopeResponseData; import org.apache.kafka.common.message.EnvelopeResponseData;
import org.apache.kafka.common.message.ExpireDelegationTokenRequestData; import org.apache.kafka.common.message.ExpireDelegationTokenRequestData;
import org.apache.kafka.common.message.ExpireDelegationTokenResponseData; import org.apache.kafka.common.message.ExpireDelegationTokenResponseData;
import org.apache.kafka.common.message.FetchRequestData;
import org.apache.kafka.common.message.FetchResponseData; import org.apache.kafka.common.message.FetchResponseData;
import org.apache.kafka.common.message.FetchSnapshotRequestData; import org.apache.kafka.common.message.FetchSnapshotRequestData;
import org.apache.kafka.common.message.FetchSnapshotResponseData; import org.apache.kafka.common.message.FetchSnapshotResponseData;
@ -379,7 +378,7 @@ public class RequestResponseTest {
checkResponse(createFetchResponse(Errors.FETCH_SESSION_ID_NOT_FOUND, 123), (short) 7); checkResponse(createFetchResponse(Errors.FETCH_SESSION_ID_NOT_FOUND, 123), (short) 7);
checkOlderFetchVersions(); checkOlderFetchVersions();
// Metadata // Metadata
checkRequest(MetadataRequest.Builder.allTopics().build((short) 2)); checkRequest(MetadataRequest.Builder.allTopics().build((short) 4));
// OffsetFetch // OffsetFetch
checkRequest(createOffsetFetchRequestWithMultipleGroups((short) 8, true)); checkRequest(createOffsetFetchRequestWithMultipleGroups((short) 8, true));
checkRequest(createOffsetFetchRequestWithMultipleGroups((short) 8, false)); checkRequest(createOffsetFetchRequestWithMultipleGroups((short) 8, false));
@ -413,12 +412,11 @@ public class RequestResponseTest {
checkRequest(createTxnOffsetCommitRequestWithAutoDowngrade()); checkRequest(createTxnOffsetCommitRequestWithAutoDowngrade());
checkErrorResponse(createTxnOffsetCommitRequestWithAutoDowngrade(), unknownServerException); checkErrorResponse(createTxnOffsetCommitRequestWithAutoDowngrade(), unknownServerException);
// DescribeAcls // DescribeAcls
checkErrorResponse(createDescribeAclsRequest((short) 0), new SecurityDisabledException("Security is not enabled.")); checkErrorResponse(createDescribeAclsRequest((short) 1), new SecurityDisabledException("Security is not enabled."));
checkErrorResponse(createCreateAclsRequest((short) 0), new SecurityDisabledException("Security is not enabled.")); checkErrorResponse(createCreateAclsRequest((short) 1), new SecurityDisabledException("Security is not enabled."));
// DeleteAcls // DeleteAcls
checkErrorResponse(createDeleteAclsRequest((short) 0), new SecurityDisabledException("Security is not enabled.")); checkErrorResponse(createDeleteAclsRequest((short) 1), new SecurityDisabledException("Security is not enabled."));
// DescribeConfigs // DescribeConfigs
checkRequest(createDescribeConfigsRequestWithConfigEntries((short) 0));
checkRequest(createDescribeConfigsRequestWithConfigEntries((short) 1)); checkRequest(createDescribeConfigsRequestWithConfigEntries((short) 1));
checkRequest(createDescribeConfigsRequestWithDocumentation((short) 1)); checkRequest(createDescribeConfigsRequestWithDocumentation((short) 1));
checkRequest(createDescribeConfigsRequestWithDocumentation((short) 2)); checkRequest(createDescribeConfigsRequestWithDocumentation((short) 2));
@ -478,7 +476,7 @@ public class RequestResponseTest {
Compression.NONE, new SimpleRecord("woot".getBytes())); Compression.NONE, new SimpleRecord("woot".getBytes()));
MemoryRecords records1 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, MemoryRecords records1 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2,
Compression.NONE, new SimpleRecord("woot".getBytes()), new SimpleRecord("woot".getBytes())); Compression.NONE, new SimpleRecord("woot".getBytes()), new SimpleRecord("woot".getBytes()));
ProduceRequest request = ProduceRequest.forMagic(RecordBatch.MAGIC_VALUE_V2, ProduceRequest request = ProduceRequest.builder(
new ProduceRequestData() new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(asList( .setTopicData(new ProduceRequestData.TopicProduceDataCollection(asList(
new ProduceRequestData.TopicProduceData().setName(tp0.topic()).setPartitionData( new ProduceRequestData.TopicProduceData().setName(tp0.topic()).setPartitionData(
@ -490,7 +488,7 @@ public class RequestResponseTest {
.setTimeoutMs(5000) .setTimeoutMs(5000)
.setTransactionalId("transactionalId"), .setTransactionalId("transactionalId"),
true) true)
.build((short) 3); .build((short) 7);
assertEquals(2, request.partitionSizes().size()); assertEquals(2, request.partitionSizes().size());
assertEquals(records0.sizeInBytes(), (int) request.partitionSizes().get(tp0)); assertEquals(records0.sizeInBytes(), (int) request.partitionSizes().get(tp0));
assertEquals(records1.sizeInBytes(), (int) request.partitionSizes().get(tp1)); assertEquals(records1.sizeInBytes(), (int) request.partitionSizes().get(tp1));
@ -546,33 +544,13 @@ public class RequestResponseTest {
@Test @Test
public void fetchResponseVersionTest() { public void fetchResponseVersionTest() {
LinkedHashMap<TopicIdPartition, FetchResponseData.PartitionData> responseData = new LinkedHashMap<>();
Uuid id = Uuid.randomUuid(); Uuid id = Uuid.randomUuid();
Map<Uuid, String> topicNames = Collections.singletonMap(id, "test");
TopicPartition tp = new TopicPartition("test", 0);
MemoryRecords records = MemoryRecords.readableRecords(ByteBuffer.allocate(10)); MemoryRecords records = MemoryRecords.readableRecords(ByteBuffer.allocate(10));
FetchResponseData.PartitionData partitionData = new FetchResponseData.PartitionData() FetchResponseData.PartitionData partitionData = new FetchResponseData.PartitionData()
.setPartitionIndex(0) .setPartitionIndex(0)
.setHighWatermark(1000000) .setHighWatermark(1000000)
.setLogStartOffset(-1) .setLogStartOffset(-1)
.setRecords(records); .setRecords(records);
// Use zero UUID since we are comparing with old request versions
responseData.put(new TopicIdPartition(Uuid.ZERO_UUID, tp), partitionData);
LinkedHashMap<TopicPartition, FetchResponseData.PartitionData> tpResponseData = new LinkedHashMap<>();
tpResponseData.put(tp, partitionData);
FetchResponse v0Response = FetchResponse.of(Errors.NONE, 0, INVALID_SESSION_ID, responseData);
FetchResponse v1Response = FetchResponse.of(Errors.NONE, 10, INVALID_SESSION_ID, responseData);
FetchResponse v0Deserialized = FetchResponse.parse(v0Response.serialize((short) 0), (short) 0);
FetchResponse v1Deserialized = FetchResponse.parse(v1Response.serialize((short) 1), (short) 1);
assertEquals(0, v0Deserialized.throttleTimeMs(), "Throttle time must be zero");
assertEquals(10, v1Deserialized.throttleTimeMs(), "Throttle time must be 10");
assertEquals(tpResponseData, v0Deserialized.responseData(topicNames, (short) 0), "Response data does not match");
assertEquals(tpResponseData, v1Deserialized.responseData(topicNames, (short) 1), "Response data does not match");
LinkedHashMap<TopicIdPartition, FetchResponseData.PartitionData> idResponseData = new LinkedHashMap<>(); LinkedHashMap<TopicIdPartition, FetchResponseData.PartitionData> idResponseData = new LinkedHashMap<>();
idResponseData.put(new TopicIdPartition(id, new TopicPartition("test", 0)), idResponseData.put(new TopicIdPartition(id, new TopicPartition("test", 0)),
new FetchResponseData.PartitionData() new FetchResponseData.PartitionData()
@ -690,14 +668,6 @@ public class RequestResponseTest {
assertTrue(exception.getMessage().contains("[foo, bar]")); assertTrue(exception.getMessage().contains("[foo, bar]"));
} }
@Test
public void testFetchRequestMaxBytesOldVersions() {
final short version = 1;
FetchRequest fr = createFetchRequest(version);
FetchRequest fr2 = FetchRequest.parse(fr.serialize(), version);
assertEquals(fr2.maxBytes(), fr.maxBytes());
}
@Test @Test
public void testFetchRequestIsolationLevel() { public void testFetchRequestIsolationLevel() {
FetchRequest request = createFetchRequest((short) 4, IsolationLevel.READ_COMMITTED); FetchRequest request = createFetchRequest((short) 4, IsolationLevel.READ_COMMITTED);
@ -724,24 +694,6 @@ public class RequestResponseTest {
assertEquals(request.isolationLevel(), deserialized.isolationLevel()); assertEquals(request.isolationLevel(), deserialized.isolationLevel());
} }
@Test
public void testFetchRequestCompat() {
Map<TopicPartition, FetchRequest.PartitionData> fetchData = new HashMap<>();
fetchData.put(new TopicPartition("test", 0), new FetchRequest.PartitionData(Uuid.ZERO_UUID, 100, 2, 100, Optional.of(42)));
FetchRequest req = FetchRequest.Builder
.forConsumer((short) 2, 100, 100, fetchData)
.metadata(new FetchMetadata(10, 20))
.isolationLevel(IsolationLevel.READ_COMMITTED)
.build((short) 2);
FetchRequestData data = req.data();
ObjectSerializationCache cache = new ObjectSerializationCache();
int size = data.size(cache, (short) 2);
ByteBufferAccessor writer = new ByteBufferAccessor(ByteBuffer.allocate(size));
data.write(writer, cache, (short) 2);
}
@Test @Test
public void testSerializeWithHeader() { public void testSerializeWithHeader() {
CreatableTopicCollection topicsToCreate = new CreatableTopicCollection(1); CreatableTopicCollection topicsToCreate = new CreatableTopicCollection(1);
@ -789,14 +741,6 @@ public class RequestResponseTest {
assertThrows(IllegalArgumentException.class, () -> createTopicsRequest.serializeWithHeader(requestHeader)); assertThrows(IllegalArgumentException.class, () -> createTopicsRequest.serializeWithHeader(requestHeader));
} }
@Test
public void testJoinGroupRequestV0RebalanceTimeout() {
final short version = 0;
JoinGroupRequest jgr = createJoinGroupRequest(version);
JoinGroupRequest jgr2 = JoinGroupRequest.parse(jgr.serialize(), version);
assertEquals(jgr2.data().rebalanceTimeoutMs(), jgr.data().rebalanceTimeoutMs());
}
@Test @Test
public void testOffsetFetchRequestBuilderToStringV0ToV7() { public void testOffsetFetchRequestBuilderToStringV0ToV7() {
List<Boolean> stableFlags = asList(true, false); List<Boolean> stableFlags = asList(true, false);
@ -2352,19 +2296,7 @@ public class RequestResponseTest {
} }
private ListOffsetsRequest createListOffsetRequest(short version) { private ListOffsetsRequest createListOffsetRequest(short version) {
if (version == 0) { if (version == 1) {
ListOffsetsTopic topic = new ListOffsetsTopic()
.setName("test")
.setPartitions(singletonList(new ListOffsetsPartition()
.setPartitionIndex(0)
.setTimestamp(1000000L)
.setMaxNumOffsets(10)
.setCurrentLeaderEpoch(5)));
return ListOffsetsRequest.Builder
.forConsumer(false, IsolationLevel.READ_UNCOMMITTED)
.setTargetTimes(singletonList(topic))
.build(version);
} else if (version == 1) {
ListOffsetsTopic topic = new ListOffsetsTopic() ListOffsetsTopic topic = new ListOffsetsTopic()
.setName("test") .setName("test")
.setPartitions(singletonList(new ListOffsetsPartition() .setPartitions(singletonList(new ListOffsetsPartition()
@ -2394,16 +2326,7 @@ public class RequestResponseTest {
} }
private ListOffsetsResponse createListOffsetResponse(short version) { private ListOffsetsResponse createListOffsetResponse(short version) {
if (version == 0) { if (version >= 1 && version <= LIST_OFFSETS.latestVersion()) {
ListOffsetsResponseData data = new ListOffsetsResponseData()
.setTopics(singletonList(new ListOffsetsTopicResponse()
.setName("test")
.setPartitions(singletonList(new ListOffsetsPartitionResponse()
.setPartitionIndex(0)
.setErrorCode(Errors.NONE.code())
.setOldStyleOffsets(singletonList(100L))))));
return new ListOffsetsResponse(data);
} else if (version >= 1 && version <= LIST_OFFSETS.latestVersion()) {
ListOffsetsPartitionResponse partition = new ListOffsetsPartitionResponse() ListOffsetsPartitionResponse partition = new ListOffsetsPartitionResponse()
.setPartitionIndex(0) .setPartitionIndex(0)
.setErrorCode(Errors.NONE.code()) .setErrorCode(Errors.NONE.code())
@ -2564,22 +2487,9 @@ public class RequestResponseTest {
} }
private ProduceRequest createProduceRequest(short version) { private ProduceRequest createProduceRequest(short version) {
if (version < 2) { MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, Compression.NONE,
MemoryRecords records = MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("blah".getBytes())); new SimpleRecord("woot".getBytes()));
ProduceRequestData data = new ProduceRequestData() return ProduceRequest.builder(
.setAcks((short) -1)
.setTimeoutMs(123)
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(singletonList(
new ProduceRequestData.TopicProduceData()
.setName("topic1")
.setPartitionData(singletonList(new ProduceRequestData.PartitionProduceData()
.setIndex(1)
.setRecords(records)))).iterator()));
return new ProduceRequest.Builder(version, version, data).build(version);
}
byte magic = version == 2 ? RecordBatch.MAGIC_VALUE_V1 : RecordBatch.MAGIC_VALUE_V2;
MemoryRecords records = MemoryRecords.withRecords(magic, Compression.NONE, new SimpleRecord("woot".getBytes()));
return ProduceRequest.forMagic(magic,
new ProduceRequestData() new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(singletonList( .setTopicData(new ProduceRequestData.TopicProduceDataCollection(singletonList(
new ProduceRequestData.TopicProduceData() new ProduceRequestData.TopicProduceData()

View File

@ -414,7 +414,6 @@ private[group] class GroupCoordinatorAdapter(
partition.committedOffset, partition.committedOffset,
partition.committedLeaderEpoch, partition.committedLeaderEpoch,
partition.committedMetadata, partition.committedMetadata,
partition.commitTimestamp,
expireTimeMs expireTimeMs
) )
} }
@ -473,7 +472,6 @@ private[group] class GroupCoordinatorAdapter(
partition.committedOffset, partition.committedOffset,
partition.committedLeaderEpoch, partition.committedLeaderEpoch,
partition.committedMetadata, partition.committedMetadata,
OffsetCommitRequest.DEFAULT_TIMESTAMP, // means that currentTimeMs is used.
None None
) )
} }
@ -501,7 +499,6 @@ private[group] class GroupCoordinatorAdapter(
offset: Long, offset: Long,
leaderEpoch: Int, leaderEpoch: Int,
metadata: String, metadata: String,
commitTimestamp: Long,
expireTimestamp: Option[Long] expireTimestamp: Option[Long]
): OffsetAndMetadata = { ): OffsetAndMetadata = {
new OffsetAndMetadata( new OffsetAndMetadata(
@ -514,10 +511,7 @@ private[group] class GroupCoordinatorAdapter(
case null => OffsetAndMetadata.NO_METADATA case null => OffsetAndMetadata.NO_METADATA
case metadata => metadata case metadata => metadata
}, },
commitTimestamp match { currentTimeMs,
case OffsetCommitRequest.DEFAULT_TIMESTAMP => currentTimeMs
case customTimestamp => customTimestamp
},
expireTimestamp match { expireTimestamp match {
case Some(timestamp) => OptionalLong.of(timestamp) case Some(timestamp) => OptionalLong.of(timestamp)
case None => OptionalLong.empty() case None => OptionalLong.empty()

View File

@ -19,7 +19,7 @@ package kafka.server
import com.typesafe.scalalogging.Logger import com.typesafe.scalalogging.Logger
import kafka.utils.Logging import kafka.utils.Logging
import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.common.{Node, TopicIdPartition, TopicPartition, Uuid}
import org.apache.kafka.common.message.FetchResponseData import org.apache.kafka.common.message.FetchResponseData
import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.requests.FetchMetadata.{FINAL_EPOCH, INITIAL_EPOCH, INVALID_SESSION_ID} import org.apache.kafka.common.requests.FetchMetadata.{FINAL_EPOCH, INITIAL_EPOCH, INVALID_SESSION_ID}
@ -326,7 +326,7 @@ trait FetchContext extends Logging {
* Updates the fetch context with new partition information. Generates response data. * Updates the fetch context with new partition information. Generates response data.
* The response data may require subsequent down-conversion. * The response data may require subsequent down-conversion.
*/ */
def updateAndGenerateResponseData(updates: FetchSession.RESP_MAP): FetchResponse def updateAndGenerateResponseData(updates: FetchSession.RESP_MAP, nodeEndpoints: util.List[Node]): FetchResponse
def partitionsToLogString(partitions: util.Collection[TopicIdPartition]): String = def partitionsToLogString(partitions: util.Collection[TopicIdPartition]): String =
FetchSession.partitionsToLogString(partitions, isTraceEnabled) FetchSession.partitionsToLogString(partitions, isTraceEnabled)
@ -334,8 +334,8 @@ trait FetchContext extends Logging {
/** /**
* Return an empty throttled response due to quota violation. * Return an empty throttled response due to quota violation.
*/ */
def getThrottledResponse(throttleTimeMs: Int): FetchResponse = def getThrottledResponse(throttleTimeMs: Int, nodeEndpoints: util.List[Node]): FetchResponse =
FetchResponse.of(Errors.NONE, throttleTimeMs, INVALID_SESSION_ID, new FetchSession.RESP_MAP) FetchResponse.of(Errors.NONE, throttleTimeMs, INVALID_SESSION_ID, new FetchSession.RESP_MAP, nodeEndpoints)
} }
/** /**
@ -352,9 +352,9 @@ class SessionErrorContext(val error: Errors,
} }
// Because of the fetch session error, we don't know what partitions were supposed to be in this request. // Because of the fetch session error, we don't know what partitions were supposed to be in this request.
override def updateAndGenerateResponseData(updates: FetchSession.RESP_MAP): FetchResponse = { override def updateAndGenerateResponseData(updates: FetchSession.RESP_MAP, nodeEndpoints: util.List[Node]): FetchResponse = {
debug(s"Session error fetch context returning $error") debug(s"Session error fetch context returning $error")
FetchResponse.of(error, 0, INVALID_SESSION_ID, new FetchSession.RESP_MAP) FetchResponse.of(error, 0, INVALID_SESSION_ID, new FetchSession.RESP_MAP, nodeEndpoints)
} }
} }
@ -382,9 +382,9 @@ class SessionlessFetchContext(val fetchData: util.Map[TopicIdPartition, FetchReq
FetchResponse.sizeOf(versionId, updates.entrySet.iterator) FetchResponse.sizeOf(versionId, updates.entrySet.iterator)
} }
override def updateAndGenerateResponseData(updates: FetchSession.RESP_MAP): FetchResponse = { override def updateAndGenerateResponseData(updates: FetchSession.RESP_MAP, nodeEndpoints: util.List[Node]): FetchResponse = {
debug(s"Sessionless fetch context returning ${partitionsToLogString(updates.keySet)}") debug(s"Sessionless fetch context returning ${partitionsToLogString(updates.keySet)}")
FetchResponse.of(Errors.NONE, 0, INVALID_SESSION_ID, updates) FetchResponse.of(Errors.NONE, 0, INVALID_SESSION_ID, updates, nodeEndpoints)
} }
} }
@ -430,7 +430,7 @@ class FullFetchContext(private val time: Time,
FetchResponse.sizeOf(versionId, updates.entrySet.iterator) FetchResponse.sizeOf(versionId, updates.entrySet.iterator)
} }
override def updateAndGenerateResponseData(updates: FetchSession.RESP_MAP): FetchResponse = { override def updateAndGenerateResponseData(updates: FetchSession.RESP_MAP, nodeEndpoints: util.List[Node]): FetchResponse = {
def createNewSession: FetchSession.CACHE_MAP = { def createNewSession: FetchSession.CACHE_MAP = {
val cachedPartitions = new FetchSession.CACHE_MAP(updates.size) val cachedPartitions = new FetchSession.CACHE_MAP(updates.size)
updates.forEach { (part, respData) => updates.forEach { (part, respData) =>
@ -444,7 +444,7 @@ class FullFetchContext(private val time: Time,
updates.size, usesTopicIds, () => createNewSession) updates.size, usesTopicIds, () => createNewSession)
debug(s"Full fetch context with session id $responseSessionId returning " + debug(s"Full fetch context with session id $responseSessionId returning " +
s"${partitionsToLogString(updates.keySet)}") s"${partitionsToLogString(updates.keySet)}")
FetchResponse.of(Errors.NONE, 0, responseSessionId, updates) FetchResponse.of(Errors.NONE, 0, responseSessionId, updates, nodeEndpoints)
} }
} }
@ -533,7 +533,7 @@ class IncrementalFetchContext(private val time: Time,
} }
} }
override def updateAndGenerateResponseData(updates: FetchSession.RESP_MAP): FetchResponse = { override def updateAndGenerateResponseData(updates: FetchSession.RESP_MAP, nodeEndpoints: util.List[Node]): FetchResponse = {
session.synchronized { session.synchronized {
// Check to make sure that the session epoch didn't change in between // Check to make sure that the session epoch didn't change in between
// creating this fetch context and generating this response. // creating this fetch context and generating this response.
@ -541,7 +541,7 @@ class IncrementalFetchContext(private val time: Time,
if (session.epoch != expectedEpoch) { if (session.epoch != expectedEpoch) {
info(s"Incremental fetch session ${session.id} expected epoch $expectedEpoch, but " + info(s"Incremental fetch session ${session.id} expected epoch $expectedEpoch, but " +
s"got ${session.epoch}. Possible duplicate request.") s"got ${session.epoch}. Possible duplicate request.")
FetchResponse.of(Errors.INVALID_FETCH_SESSION_EPOCH, 0, session.id, new FetchSession.RESP_MAP) FetchResponse.of(Errors.INVALID_FETCH_SESSION_EPOCH, 0, session.id, new FetchSession.RESP_MAP, nodeEndpoints)
} else { } else {
// Iterate over the update list using PartitionIterator. This will prune updates which don't need to be sent // Iterate over the update list using PartitionIterator. This will prune updates which don't need to be sent
val partitionIter = new PartitionIterator(updates.entrySet.iterator, true) val partitionIter = new PartitionIterator(updates.entrySet.iterator, true)
@ -550,12 +550,12 @@ class IncrementalFetchContext(private val time: Time,
} }
debug(s"Incremental fetch context with session id ${session.id} returning " + debug(s"Incremental fetch context with session id ${session.id} returning " +
s"${partitionsToLogString(updates.keySet)}") s"${partitionsToLogString(updates.keySet)}")
FetchResponse.of(Errors.NONE, 0, session.id, updates) FetchResponse.of(Errors.NONE, 0, session.id, updates, nodeEndpoints)
} }
} }
} }
override def getThrottledResponse(throttleTimeMs: Int): FetchResponse = { override def getThrottledResponse(throttleTimeMs: Int, nodeEndpoints: util.List[Node]): FetchResponse = {
session.synchronized { session.synchronized {
// Check to make sure that the session epoch didn't change in between // Check to make sure that the session epoch didn't change in between
// creating this fetch context and generating this response. // creating this fetch context and generating this response.
@ -563,9 +563,9 @@ class IncrementalFetchContext(private val time: Time,
if (session.epoch != expectedEpoch) { if (session.epoch != expectedEpoch) {
info(s"Incremental fetch session ${session.id} expected epoch $expectedEpoch, but " + info(s"Incremental fetch session ${session.id} expected epoch $expectedEpoch, but " +
s"got ${session.epoch}. Possible duplicate request.") s"got ${session.epoch}. Possible duplicate request.")
FetchResponse.of(Errors.INVALID_FETCH_SESSION_EPOCH, throttleTimeMs, session.id, new FetchSession.RESP_MAP) FetchResponse.of(Errors.INVALID_FETCH_SESSION_EPOCH, throttleTimeMs, session.id, new FetchSession.RESP_MAP, nodeEndpoints)
} else { } else {
FetchResponse.of(Errors.NONE, throttleTimeMs, session.id, new FetchSession.RESP_MAP) FetchResponse.of(Errors.NONE, throttleTimeMs, session.id, new FetchSession.RESP_MAP, nodeEndpoints)
} }
} }
} }

View File

@ -52,7 +52,7 @@ import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetFor
import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.{EpochEndOffset, OffsetForLeaderTopicResult, OffsetForLeaderTopicResultCollection} import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.{EpochEndOffset, OffsetForLeaderTopicResult, OffsetForLeaderTopicResultCollection}
import org.apache.kafka.common.message._ import org.apache.kafka.common.message._
import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.network.{ListenerName, NetworkSend, Send} import org.apache.kafka.common.network.{ListenerName}
import org.apache.kafka.common.protocol.{ApiKeys, ApiMessage, Errors} import org.apache.kafka.common.protocol.{ApiKeys, ApiMessage, Errors}
import org.apache.kafka.common.record._ import org.apache.kafka.common.record._
import org.apache.kafka.common.replica.ClientMetadata import org.apache.kafka.common.replica.ClientMetadata
@ -74,7 +74,6 @@ import org.apache.kafka.server.authorizer._
import org.apache.kafka.server.common.{GroupVersion, MetadataVersion, RequestLocal, TransactionVersion} 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.MetadataVersion.{IBP_0_11_0_IV0, IBP_2_3_IV0}
import org.apache.kafka.server.purgatory.TopicPartitionOperationKey import org.apache.kafka.server.purgatory.TopicPartitionOperationKey
import org.apache.kafka.server.record.BrokerCompressionType
import org.apache.kafka.server.share.context.ShareFetchContext import org.apache.kafka.server.share.context.ShareFetchContext
import org.apache.kafka.server.share.{ErroneousAndValidPartitionData, SharePartitionKey} import org.apache.kafka.server.share.{ErroneousAndValidPartitionData, SharePartitionKey}
import org.apache.kafka.server.share.acknowledge.ShareAcknowledgementBatch import org.apache.kafka.server.share.acknowledge.ShareAcknowledgementBatch
@ -82,7 +81,6 @@ import org.apache.kafka.server.storage.log.{FetchIsolation, FetchParams, FetchPa
import org.apache.kafka.storage.internals.log.AppendOrigin import org.apache.kafka.storage.internals.log.AppendOrigin
import org.apache.kafka.storage.log.metrics.BrokerTopicStats import org.apache.kafka.storage.log.metrics.BrokerTopicStats
import java.lang.{Long => JLong}
import java.nio.ByteBuffer import java.nio.ByteBuffer
import java.time.Duration import java.time.Duration
import java.util import java.util
@ -834,81 +832,6 @@ class KafkaApis(val requestChannel: RequestChannel,
} }
} }
def maybeConvertFetchedData(tp: TopicIdPartition,
partitionData: FetchResponseData.PartitionData): FetchResponseData.PartitionData = {
// We will never return a logConfig when the topic is unresolved and the name is null. This is ok since we won't have any records to convert.
val logConfig = replicaManager.getLogConfig(tp.topicPartition)
if (logConfig.exists(_.compressionType == BrokerCompressionType.ZSTD) && versionId < 10) {
trace(s"Fetching messages is disabled for ZStandard compressed partition $tp. Sending unsupported version response to $clientId.")
FetchResponse.partitionResponse(tp, Errors.UNSUPPORTED_COMPRESSION_TYPE)
} else {
// Down-conversion of fetched records is needed when the on-disk magic value is greater than what is
// supported by the fetch request version.
// If the inter-broker protocol version is `3.0` or higher, the log config message format version is
// always `3.0` (i.e. magic value is `v2`). As a result, we always go through the down-conversion
// path if the fetch version is 3 or lower (in rare cases the down-conversion may not be needed, but
// it's not worth optimizing for them).
// If the inter-broker protocol version is lower than `3.0`, we rely on the log config message format
// version as a proxy for the on-disk magic value to maintain the long-standing behavior originally
// introduced in Kafka 0.10.0. An important implication is that it's unsafe to downgrade the message
// format version after a single message has been produced (the broker would return the message(s)
// without down-conversion irrespective of the fetch version).
val unconvertedRecords = FetchResponse.recordsOrFail(partitionData)
val downConvertMagic =
logConfig.map(_.recordVersion.value).flatMap { magic =>
if (magic > RecordBatch.MAGIC_VALUE_V0 && versionId <= 1)
Some(RecordBatch.MAGIC_VALUE_V0)
else if (magic > RecordBatch.MAGIC_VALUE_V1 && versionId <= 3)
Some(RecordBatch.MAGIC_VALUE_V1)
else
None
}
downConvertMagic match {
case Some(magic) =>
// For fetch requests from clients, check if down-conversion is disabled for the particular partition
if (!fetchRequest.isFromFollower && !logConfig.forall(_.messageDownConversionEnable)) {
trace(s"Conversion to message format ${downConvertMagic.get} is disabled for partition $tp. Sending unsupported version response to $clientId.")
FetchResponse.partitionResponse(tp, Errors.UNSUPPORTED_VERSION)
} else {
try {
trace(s"Down converting records from partition $tp to message format version $magic for fetch request from $clientId")
// Because down-conversion is extremely memory intensive, we want to try and delay the down-conversion as much
// as possible. With KIP-283, we have the ability to lazily down-convert in a chunked manner. The lazy, chunked
// down-conversion always guarantees that at least one batch of messages is down-converted and sent out to the
// client.
new FetchResponseData.PartitionData()
.setPartitionIndex(tp.partition)
.setErrorCode(maybeDownConvertStorageError(Errors.forCode(partitionData.errorCode)).code)
.setHighWatermark(partitionData.highWatermark)
.setLastStableOffset(partitionData.lastStableOffset)
.setLogStartOffset(partitionData.logStartOffset)
.setAbortedTransactions(partitionData.abortedTransactions)
.setRecords(new LazyDownConversionRecords(tp.topicPartition, unconvertedRecords, magic, fetchContext.getFetchOffset(tp).get, time))
.setPreferredReadReplica(partitionData.preferredReadReplica())
} catch {
case e: UnsupportedCompressionTypeException =>
trace("Received unsupported compression type error during down-conversion", e)
FetchResponse.partitionResponse(tp, Errors.UNSUPPORTED_COMPRESSION_TYPE)
}
}
case None =>
new FetchResponseData.PartitionData()
.setPartitionIndex(tp.partition)
.setErrorCode(maybeDownConvertStorageError(Errors.forCode(partitionData.errorCode)).code)
.setHighWatermark(partitionData.highWatermark)
.setLastStableOffset(partitionData.lastStableOffset)
.setLogStartOffset(partitionData.logStartOffset)
.setAbortedTransactions(partitionData.abortedTransactions)
.setRecords(unconvertedRecords)
.setPreferredReadReplica(partitionData.preferredReadReplica)
.setDivergingEpoch(partitionData.divergingEpoch)
.setCurrentLeader(partitionData.currentLeader())
}
}
}
// the callback for process a fetch response, invoked before throttling // the callback for process a fetch response, invoked before throttling
def processResponseCallback(responsePartitionData: Seq[(TopicIdPartition, FetchPartitionData)]): Unit = { def processResponseCallback(responsePartitionData: Seq[(TopicIdPartition, FetchPartitionData)]): Unit = {
val partitions = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData] val partitions = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
@ -947,25 +870,9 @@ class KafkaApis(val requestChannel: RequestChannel,
} }
erroneous.foreach { case (tp, data) => partitions.put(tp, data) } erroneous.foreach { case (tp, data) => partitions.put(tp, data) }
def createResponse(throttleTimeMs: Int, unconvertedFetchResponse: FetchResponse): FetchResponse = { def recordBytesOutMetric(fetchResponse: FetchResponse): Unit = {
// Down-convert messages for each partition if required
val convertedData = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
unconvertedFetchResponse.data().responses().forEach { topicResponse =>
topicResponse.partitions().forEach { unconvertedPartitionData =>
val tp = new TopicIdPartition(topicResponse.topicId, new TopicPartition(topicResponse.topic, unconvertedPartitionData.partitionIndex()))
val error = Errors.forCode(unconvertedPartitionData.errorCode)
if (error != Errors.NONE)
debug(s"Fetch request with correlation id ${request.header.correlationId} from client $clientId " +
s"on partition $tp failed due to ${error.exceptionName}")
convertedData.put(tp, maybeConvertFetchedData(tp, unconvertedPartitionData))
}
}
// Prepare fetch response from converted data
val response =
FetchResponse.of(unconvertedFetchResponse.error, throttleTimeMs, unconvertedFetchResponse.sessionId, convertedData, nodeEndpoints.values.toList.asJava)
// record the bytes out metrics only when the response is being sent // record the bytes out metrics only when the response is being sent
response.data.responses.forEach { topicResponse => fetchResponse.data.responses.forEach { topicResponse =>
topicResponse.partitions.forEach { data => topicResponse.partitions.forEach { data =>
// If the topic name was not known, we will have no bytes out. // If the topic name was not known, we will have no bytes out.
if (topicResponse.topic != null) { if (topicResponse.topic != null) {
@ -974,23 +881,19 @@ class KafkaApis(val requestChannel: RequestChannel,
} }
} }
} }
response
} }
if (fetchRequest.isFromFollower) { if (fetchRequest.isFromFollower) {
// We've already evaluated against the quota and are good to go. Just need to record it now. // We've already evaluated against the quota and are good to go. Just need to record it now.
val unconvertedFetchResponse = fetchContext.updateAndGenerateResponseData(partitions) val fetchResponse = fetchContext.updateAndGenerateResponseData(partitions, Seq.empty.asJava)
val responseSize = KafkaApis.sizeOfThrottledPartitions(versionId, unconvertedFetchResponse, quotas.leader) val responseSize = KafkaApis.sizeOfThrottledPartitions(versionId, fetchResponse, quotas.leader)
quotas.leader.record(responseSize) quotas.leader.record(responseSize)
val responsePartitionsSize = unconvertedFetchResponse.data().responses().stream().mapToInt(_.partitions().size()).sum() val responsePartitionsSize = fetchResponse.data().responses().stream().mapToInt(_.partitions().size()).sum()
trace(s"Sending Fetch response with partitions.size=$responsePartitionsSize, " + trace(s"Sending Fetch response with partitions.size=$responsePartitionsSize, " +
s"metadata=${unconvertedFetchResponse.sessionId}") s"metadata=${fetchResponse.sessionId}")
requestHelper.sendResponseExemptThrottle(request, createResponse(0, unconvertedFetchResponse), onFetchComplete(request)) recordBytesOutMetric(fetchResponse)
requestHelper.sendResponseExemptThrottle(request, fetchResponse)
} else { } else {
// Fetch size used to determine throttle time is calculated before any down conversions.
// This may be slightly different from the actual response size. But since down conversions
// result in data being loaded into memory, we should do this only when we are not going to throttle.
//
// Record both bandwidth and request quota-specific values and throttle by muting the channel if any of the // Record both bandwidth and request quota-specific values and throttle by muting the channel if any of the
// quotas have been violated. If both quotas have been violated, use the max throttle time between the two // quotas have been violated. If both quotas have been violated, use the max throttle time between the two
// quotas. When throttled, we unrecord the recorded bandwidth quota value. // quotas. When throttled, we unrecord the recorded bandwidth quota value.
@ -1000,7 +903,7 @@ class KafkaApis(val requestChannel: RequestChannel,
val bandwidthThrottleTimeMs = quotas.fetch.maybeRecordAndGetThrottleTimeMs(request, responseSize, timeMs) val bandwidthThrottleTimeMs = quotas.fetch.maybeRecordAndGetThrottleTimeMs(request, responseSize, timeMs)
val maxThrottleTimeMs = math.max(bandwidthThrottleTimeMs, requestThrottleTimeMs) val maxThrottleTimeMs = math.max(bandwidthThrottleTimeMs, requestThrottleTimeMs)
val unconvertedFetchResponse = if (maxThrottleTimeMs > 0) { val fetchResponse = if (maxThrottleTimeMs > 0) {
request.apiThrottleTimeMs = maxThrottleTimeMs request.apiThrottleTimeMs = maxThrottleTimeMs
// Even if we need to throttle for request quota violation, we should "unrecord" the already recorded value // Even if we need to throttle for request quota violation, we should "unrecord" the already recorded value
// from the fetch quota because we are going to return an empty response. // from the fetch quota because we are going to return an empty response.
@ -1011,18 +914,19 @@ class KafkaApis(val requestChannel: RequestChannel,
requestHelper.throttle(quotas.request, request, requestThrottleTimeMs) requestHelper.throttle(quotas.request, request, requestThrottleTimeMs)
} }
// If throttling is required, return an empty response. // If throttling is required, return an empty response.
fetchContext.getThrottledResponse(maxThrottleTimeMs) fetchContext.getThrottledResponse(maxThrottleTimeMs, nodeEndpoints.values.toSeq.asJava)
} else { } else {
// Get the actual response. This will update the fetch context. // Get the actual response. This will update the fetch context.
val unconvertedFetchResponse = fetchContext.updateAndGenerateResponseData(partitions) val fetchResponse = fetchContext.updateAndGenerateResponseData(partitions, nodeEndpoints.values.toSeq.asJava)
val responsePartitionsSize = unconvertedFetchResponse.data().responses().stream().mapToInt(_.partitions().size()).sum() val responsePartitionsSize = fetchResponse.data().responses().stream().mapToInt(_.partitions().size()).sum()
trace(s"Sending Fetch response with partitions.size=$responsePartitionsSize, " + trace(s"Sending Fetch response with partitions.size=$responsePartitionsSize, " +
s"metadata=${unconvertedFetchResponse.sessionId}") s"metadata=${fetchResponse.sessionId}")
unconvertedFetchResponse fetchResponse
} }
recordBytesOutMetric(fetchResponse)
// Send the response immediately. // Send the response immediately.
requestChannel.sendResponse(request, createResponse(maxThrottleTimeMs, unconvertedFetchResponse), onFetchComplete(request)) requestChannel.sendResponse(request, fetchResponse, None)
} }
} }
@ -1077,86 +981,6 @@ class KafkaApis(val requestChannel: RequestChannel,
if (fetchRequest.isFromFollower) quotas.leader else UNBOUNDED_QUOTA if (fetchRequest.isFromFollower) quotas.leader else UNBOUNDED_QUOTA
def handleListOffsetRequest(request: RequestChannel.Request): Unit = { def handleListOffsetRequest(request: RequestChannel.Request): Unit = {
val version = request.header.apiVersion
def sendResponseCallback(response: List[ListOffsetsTopicResponse]): Unit = {
requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
new ListOffsetsResponse(new ListOffsetsResponseData()
.setThrottleTimeMs(requestThrottleMs)
.setTopics(response.asJava)))
}
if (version == 0)
sendResponseCallback(handleListOffsetRequestV0(request))
else
handleListOffsetRequestV1AndAbove(request, sendResponseCallback)
}
private def handleListOffsetRequestV0(request : RequestChannel.Request) : List[ListOffsetsTopicResponse] = {
val correlationId = request.header.correlationId
val clientId = request.header.clientId
val offsetRequest = request.body[ListOffsetsRequest]
val (authorizedRequestInfo, unauthorizedRequestInfo) = authHelper.partitionSeqByAuthorized(request.context,
DESCRIBE, TOPIC, offsetRequest.topics.asScala.toSeq)(_.name)
val unauthorizedResponseStatus = unauthorizedRequestInfo.map(topic =>
new ListOffsetsTopicResponse()
.setName(topic.name)
.setPartitions(topic.partitions.asScala.map(partition =>
new ListOffsetsPartitionResponse()
.setPartitionIndex(partition.partitionIndex)
.setErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code)).asJava)
)
val responseTopics = authorizedRequestInfo.map { topic =>
val responsePartitions = topic.partitions.asScala.map { partition =>
if (partition.timestamp() < ListOffsetsRequest.EARLIEST_TIMESTAMP) {
// Negative timestamps are reserved for some functions.
// For v0 requests, negative timestamps only support LATEST_TIMESTAMP (-1) and EARLIEST_TIMESTAMP (-2).
new ListOffsetsPartitionResponse()
.setPartitionIndex(partition.partitionIndex)
.setErrorCode(Errors.UNSUPPORTED_VERSION.code)
} else {
val topicPartition = new TopicPartition(topic.name, partition.partitionIndex)
try {
val offsets = replicaManager.legacyFetchOffsetsForTimestamp(
topicPartition = topicPartition,
timestamp = partition.timestamp,
maxNumOffsets = partition.maxNumOffsets,
isFromConsumer = offsetRequest.replicaId == ListOffsetsRequest.CONSUMER_REPLICA_ID,
fetchOnlyFromLeader = offsetRequest.replicaId != ListOffsetsRequest.DEBUGGING_REPLICA_ID)
new ListOffsetsPartitionResponse()
.setPartitionIndex(partition.partitionIndex)
.setErrorCode(Errors.NONE.code)
.setOldStyleOffsets(offsets.map(JLong.valueOf).asJava)
} catch {
// NOTE: UnknownTopicOrPartitionException and NotLeaderOrFollowerException are special cases since these error messages
// are typically transient and there is no value in logging the entire stack trace for the same
case e @ (_ : UnknownTopicOrPartitionException |
_ : NotLeaderOrFollowerException |
_ : KafkaStorageException) =>
debug("Offset request with correlation id %d from client %s on partition %s failed due to %s".format(
correlationId, clientId, topicPartition, e.getMessage))
new ListOffsetsPartitionResponse()
.setPartitionIndex(partition.partitionIndex)
.setErrorCode(Errors.forException(e).code)
case e: Throwable =>
error("Error while responding to offset request", e)
new ListOffsetsPartitionResponse()
.setPartitionIndex(partition.partitionIndex)
.setErrorCode(Errors.forException(e).code)
}
}
}
new ListOffsetsTopicResponse().setName(topic.name).setPartitions(responsePartitions.asJava)
}
(responseTopics ++ unauthorizedResponseStatus).toList
}
private def handleListOffsetRequestV1AndAbove(request : RequestChannel.Request,
responseCallback: List[ListOffsetsTopicResponse] => Unit): Unit = {
val correlationId = request.header.correlationId val correlationId = request.header.correlationId
val clientId = request.header.clientId val clientId = request.header.clientId
val offsetRequest = request.body[ListOffsetsRequest] val offsetRequest = request.body[ListOffsetsRequest]
@ -1180,17 +1004,20 @@ class KafkaApis(val requestChannel: RequestChannel,
buildErrorResponse(Errors.TOPIC_AUTHORIZATION_FAILED, partition)).asJava) buildErrorResponse(Errors.TOPIC_AUTHORIZATION_FAILED, partition)).asJava)
) )
def sendV1ResponseCallback(response: List[ListOffsetsTopicResponse]): Unit = { def sendResponseCallback(response: Seq[ListOffsetsTopicResponse]): Unit = {
val mergedResponses = response ++ unauthorizedResponseStatus val mergedResponses = response ++ unauthorizedResponseStatus
responseCallback(mergedResponses) requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
new ListOffsetsResponse(new ListOffsetsResponseData()
.setThrottleTimeMs(requestThrottleMs)
.setTopics(mergedResponses.asJava)))
} }
if (authorizedRequestInfo.isEmpty) { if (authorizedRequestInfo.isEmpty) {
sendV1ResponseCallback(List.empty) sendResponseCallback(Seq.empty)
} else { } else {
replicaManager.fetchOffset(authorizedRequestInfo, offsetRequest.duplicatePartitions().asScala, replicaManager.fetchOffset(authorizedRequestInfo, offsetRequest.duplicatePartitions().asScala,
offsetRequest.isolationLevel(), offsetRequest.replicaId(), clientId, correlationId, version, offsetRequest.isolationLevel(), offsetRequest.replicaId(), clientId, correlationId, version,
buildErrorResponse, sendV1ResponseCallback, offsetRequest.timeoutMs()) buildErrorResponse, sendResponseCallback, offsetRequest.timeoutMs())
} }
} }
@ -4217,7 +4044,7 @@ class KafkaApis(val requestChannel: RequestChannel,
if (exception != null) { if (exception != null) {
requestHelper.sendMaybeThrottle(request, shareFetchRequest.getErrorResponse(AbstractResponse.DEFAULT_THROTTLE_TIME, exception)) requestHelper.sendMaybeThrottle(request, shareFetchRequest.getErrorResponse(AbstractResponse.DEFAULT_THROTTLE_TIME, exception))
} else { } else {
requestChannel.sendResponse(request, result, onFetchComplete(request)) requestChannel.sendResponse(request, result, None)
} }
} }
} }
@ -4744,22 +4571,6 @@ class KafkaApis(val requestChannel: RequestChannel,
.setCurrentLeader(partitionData.currentLeader) .setCurrentLeader(partitionData.currentLeader)
} }
private def onFetchComplete(request: RequestChannel.Request): Option[Send => Unit] = {
def updateConversionStats(send: Send): Unit = {
send match {
case send: MultiRecordsSend if send.recordConversionStats != null =>
send.recordConversionStats.asScala.toMap.foreach {
case (tp, stats) => updateRecordConversionStats(request, tp, stats)
}
case send: NetworkSend =>
updateConversionStats(send.send())
case _ =>
}
}
Some(updateConversionStats)
}
private def isShareGroupProtocolEnabled: Boolean = { private def isShareGroupProtocolEnabled: Boolean = {
groupCoordinator.isNewGroupCoordinator && config.shareGroupConfig.isShareGroupEnabled groupCoordinator.isNewGroupCoordinator && config.shareGroupConfig.isShareGroupEnabled
} }

View File

@ -493,8 +493,6 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _])
def logMessageTimestampAfterMaxMs: Long = getLong(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG) def logMessageTimestampAfterMaxMs: Long = getLong(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG)
def logMessageDownConversionEnable: Boolean = getBoolean(ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_CONFIG)
def logDirFailureTimeoutMs: Long = getLong(ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG) def logDirFailureTimeoutMs: Long = getLong(ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG)
/** ********* Replication configuration ***********/ /** ********* Replication configuration ***********/
@ -1029,7 +1027,6 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _])
logProps.put(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG, logMessageTimestampType.name) logProps.put(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG, logMessageTimestampType.name)
logProps.put(TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG, logMessageTimestampBeforeMaxMs: java.lang.Long) logProps.put(TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG, logMessageTimestampBeforeMaxMs: java.lang.Long)
logProps.put(TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG, logMessageTimestampAfterMaxMs: java.lang.Long) logProps.put(TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG, logMessageTimestampAfterMaxMs: java.lang.Long)
logProps.put(TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_CONFIG, logMessageDownConversionEnable: java.lang.Boolean)
logProps.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, remoteLogManagerConfig.logLocalRetentionMs: java.lang.Long) logProps.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, remoteLogManagerConfig.logLocalRetentionMs: java.lang.Long)
logProps.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, remoteLogManagerConfig.logLocalRetentionBytes: java.lang.Long) logProps.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, remoteLogManagerConfig.logLocalRetentionBytes: java.lang.Long)
logProps logProps

View File

@ -29,7 +29,6 @@ import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEnd
import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.requests.{FetchRequest, FetchResponse, ListOffsetsRequest, ListOffsetsResponse, OffsetsForLeaderEpochRequest, OffsetsForLeaderEpochResponse} import org.apache.kafka.common.requests.{FetchRequest, FetchResponse, ListOffsetsRequest, ListOffsetsResponse, OffsetsForLeaderEpochRequest, OffsetsForLeaderEpochResponse}
import org.apache.kafka.server.common.{MetadataVersion, OffsetAndEpoch} import org.apache.kafka.server.common.{MetadataVersion, OffsetAndEpoch}
import org.apache.kafka.server.common.MetadataVersion.IBP_0_10_1_IV2
import org.apache.kafka.server.network.BrokerEndPoint import org.apache.kafka.server.network.BrokerEndPoint
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
@ -123,11 +122,7 @@ class RemoteLeaderEndPoint(logPrefix: String,
.partitions.asScala.find(_.partitionIndex == topicPartition.partition).get .partitions.asScala.find(_.partitionIndex == topicPartition.partition).get
Errors.forCode(responsePartition.errorCode) match { Errors.forCode(responsePartition.errorCode) match {
case Errors.NONE => case Errors.NONE => new OffsetAndEpoch(responsePartition.offset, responsePartition.leaderEpoch)
if (metadataVersion.isAtLeast(IBP_0_10_1_IV2))
new OffsetAndEpoch(responsePartition.offset, responsePartition.leaderEpoch)
else
new OffsetAndEpoch(responsePartition.oldStyleOffsets.get(0), responsePartition.leaderEpoch)
case error => throw error.exception case error => throw error.exception
} }
} }

View File

@ -264,7 +264,7 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
} }
private def createProduceRequest = private def createProduceRequest =
requests.ProduceRequest.forCurrentMagic(new ProduceRequestData() requests.ProduceRequest.builder(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection( .setTopicData(new ProduceRequestData.TopicProduceDataCollection(
Collections.singletonList(new ProduceRequestData.TopicProduceData() Collections.singletonList(new ProduceRequestData.TopicProduceData()
.setName(tp.topic).setPartitionData(Collections.singletonList( .setName(tp.topic).setPartitionData(Collections.singletonList(
@ -389,7 +389,6 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
.setPartitionIndex(part) .setPartitionIndex(part)
.setCommittedOffset(0) .setCommittedOffset(0)
.setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH) .setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH)
.setCommitTimestamp(OffsetCommitRequest.DEFAULT_TIMESTAMP)
.setCommittedMetadata("metadata") .setCommittedMetadata("metadata")
))) )))
) )
@ -2305,8 +2304,7 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
def testMetadataClusterAuthorizedOperationsWithoutDescribeCluster(quorum: String): Unit = { def testMetadataClusterAuthorizedOperationsWithoutDescribeCluster(quorum: String): Unit = {
removeAllClientAcls() removeAllClientAcls()
// MetadataRequest versions older than 1 are not supported. for (version <- ApiKeys.METADATA.oldestVersion to ApiKeys.METADATA.latestVersion) {
for (version <- 1 to ApiKeys.METADATA.latestVersion) {
testMetadataClusterClusterAuthorizedOperations(version.toShort, 0) testMetadataClusterClusterAuthorizedOperations(version.toShort, 0)
} }
} }
@ -2326,8 +2324,7 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
val expectedClusterAuthorizedOperations = Utils.to32BitField( val expectedClusterAuthorizedOperations = Utils.to32BitField(
acls.map(_.operation.code.asInstanceOf[JByte]).asJava) acls.map(_.operation.code.asInstanceOf[JByte]).asJava)
// MetadataRequest versions older than 1 are not supported. for (version <- ApiKeys.METADATA.oldestVersion to ApiKeys.METADATA.latestVersion) {
for (version <- 1 to ApiKeys.METADATA.latestVersion) {
testMetadataClusterClusterAuthorizedOperations(version.toShort, expectedClusterAuthorizedOperations) testMetadataClusterClusterAuthorizedOperations(version.toShort, expectedClusterAuthorizedOperations)
} }
} }

View File

@ -305,7 +305,7 @@ class DynamicConnectionQuotaTest extends BaseRequestTest {
} }
private def produceRequest: ProduceRequest = private def produceRequest: ProduceRequest =
requests.ProduceRequest.forCurrentMagic(new ProduceRequestData() requests.ProduceRequest.builder(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection( .setTopicData(new ProduceRequestData.TopicProduceDataCollection(
Collections.singletonList(new ProduceRequestData.TopicProduceData() Collections.singletonList(new ProduceRequestData.TopicProduceData()
.setName(topic) .setName(topic)

View File

@ -609,7 +609,6 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
props.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_CONFIG, TimestampType.LOG_APPEND_TIME.toString) props.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_CONFIG, TimestampType.LOG_APPEND_TIME.toString)
props.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG, "1000") props.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG, "1000")
props.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG, "1000") props.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG, "1000")
props.put(ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_CONFIG, "false")
reconfigureServers(props, perBrokerConfig = false, (ServerLogConfigs.LOG_SEGMENT_BYTES_CONFIG, "4000")) reconfigureServers(props, perBrokerConfig = false, (ServerLogConfigs.LOG_SEGMENT_BYTES_CONFIG, "4000"))
// Verify that all broker defaults have been updated // Verify that all broker defaults have been updated

View File

@ -678,7 +678,6 @@ class GroupCoordinatorAdapterTest {
new OffsetCommitRequestData.OffsetCommitRequestPartition() new OffsetCommitRequestData.OffsetCommitRequestPartition()
.setPartitionIndex(0) .setPartitionIndex(0)
.setCommittedOffset(100) .setCommittedOffset(100)
.setCommitTimestamp(now)
.setCommittedLeaderEpoch(1) .setCommittedLeaderEpoch(1)
).asJava) ).asJava)
).asJava) ).asJava)

View File

@ -32,7 +32,6 @@ import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.Test import org.junit.jupiter.api.Test
import org.mockito.Mockito.mock import org.mockito.Mockito.mock
import java.util.Collections
import scala.jdk.OptionConverters.RichOption import scala.jdk.OptionConverters.RichOption
class RequestConvertToJsonTest { class RequestConvertToJsonTest {
@ -51,21 +50,6 @@ class RequestConvertToJsonTest {
assertEquals(expectedNode, actualNode) assertEquals(expectedNode, actualNode)
} }
@Test
def testRequestHeaderNodeWithDeprecatedApiVersion(): Unit = {
val fetchRequest = FetchRequest.Builder.forConsumer(0, 0, 0, Collections.emptyMap()).build(0)
val req = request(fetchRequest)
val header = req.header
val expectedNode = RequestHeaderDataJsonConverter.write(header.data, header.headerVersion, false).asInstanceOf[ObjectNode]
expectedNode.set("requestApiKeyName", new TextNode(header.apiKey.toString))
expectedNode.set("requestApiVersionDeprecated", BooleanNode.TRUE)
val actualNode = RequestConvertToJson.requestHeaderNode(header)
assertEquals(expectedNode, actualNode)
}
@Test @Test
def testRequestDesc(): Unit = { def testRequestDesc(): Unit = {
val alterIsrRequest = new AlterPartitionRequest(new AlterPartitionRequestData(), 0) val alterIsrRequest = new AlterPartitionRequest(new AlterPartitionRequestData(), 0)

View File

@ -37,7 +37,6 @@ import org.apache.kafka.common.security.scram.internals.ScramMechanism
import org.apache.kafka.common.utils._ import org.apache.kafka.common.utils._
import org.apache.kafka.network.RequestConvertToJson import org.apache.kafka.network.RequestConvertToJson
import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.network.metrics.RequestMetrics
import org.apache.kafka.security.CredentialProvider import org.apache.kafka.security.CredentialProvider
import org.apache.kafka.server.common.{FinalizedFeatures, MetadataVersion} import org.apache.kafka.server.common.{FinalizedFeatures, MetadataVersion}
import org.apache.kafka.server.config.QuotaConfig import org.apache.kafka.server.config.QuotaConfig
@ -121,8 +120,9 @@ class SocketServerTest {
val outgoing = new DataOutputStream(socket.getOutputStream) val outgoing = new DataOutputStream(socket.getOutputStream)
id match { id match {
case Some(id) => case Some(id) =>
outgoing.writeInt(request.length + 2) outgoing.writeInt(request.length + 4)
outgoing.writeShort(id) outgoing.writeShort(id)
outgoing.writeShort(ApiKeys.PRODUCE.oldestVersion)
case None => case None =>
outgoing.writeInt(request.length) outgoing.writeInt(request.length)
} }
@ -236,7 +236,7 @@ class SocketServerTest {
val clientId = "" val clientId = ""
val ackTimeoutMs = 10000 val ackTimeoutMs = 10000
val emptyRequest = requests.ProduceRequest.forCurrentMagic(new ProduceRequestData() val emptyRequest = requests.ProduceRequest.builder(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection()) .setTopicData(new ProduceRequestData.TopicProduceDataCollection())
.setAcks(ack) .setAcks(ack)
.setTimeoutMs(ackTimeoutMs) .setTimeoutMs(ackTimeoutMs)
@ -314,52 +314,6 @@ class SocketServerTest {
) )
} }
@Test
def testRequestPerSecAndDeprecatedRequestsPerSecMetrics(): Unit = {
val clientName = "apache-kafka-java"
val clientVersion = AppInfoParser.getVersion
def deprecatedRequestsPerSec(requestVersion: Short): Option[Long] =
TestUtils.meterCountOpt(s"${RequestMetrics.DEPRECATED_REQUESTS_PER_SEC},request=Produce,version=$requestVersion," +
s"clientSoftwareName=$clientName,clientSoftwareVersion=$clientVersion")
def requestsPerSec(requestVersion: Short): Option[Long] =
TestUtils.meterCountOpt(s"${RequestMetrics.REQUESTS_PER_SEC},request=Produce,version=$requestVersion")
val plainSocket = connect()
val address = plainSocket.getLocalAddress
val clientId = "clientId"
sendRequest(plainSocket, apiVersionRequestBytes(clientId, ApiKeys.API_VERSIONS.latestVersion))
var receivedReq = receiveRequest(server.dataPlaneRequestChannel)
server.dataPlaneRequestChannel.sendNoOpResponse(receivedReq)
var requestVersion = ApiKeys.PRODUCE.latestVersion
sendRequest(plainSocket, producerRequestBytes(requestVersion))
receivedReq = receiveRequest(server.dataPlaneRequestChannel)
assertEquals(clientName, receivedReq.context.clientInformation.softwareName)
assertEquals(clientVersion, receivedReq.context.clientInformation.softwareVersion)
server.dataPlaneRequestChannel.sendNoOpResponse(receivedReq)
TestUtils.waitUntilTrue(() => requestsPerSec(requestVersion).isDefined, "RequestsPerSec metric could not be found")
assertTrue(requestsPerSec(requestVersion).getOrElse(0L) > 0, "RequestsPerSec should be higher than 0")
assertEquals(None, deprecatedRequestsPerSec(requestVersion))
requestVersion = 2
sendRequest(plainSocket, producerRequestBytes(requestVersion))
receivedReq = receiveRequest(server.dataPlaneRequestChannel)
server.dataPlaneRequestChannel.sendNoOpResponse(receivedReq)
TestUtils.waitUntilTrue(() => deprecatedRequestsPerSec(requestVersion).isDefined, "DeprecatedRequestsPerSec metric could not be found")
assertTrue(deprecatedRequestsPerSec(requestVersion).getOrElse(0L) > 0, "DeprecatedRequestsPerSec should be higher than 0")
plainSocket.setSoLinger(true, 0)
plainSocket.close()
TestUtils.waitUntilTrue(() => server.connectionCount(address) == 0, msg = "Connection not closed")
}
@Test @Test
def testStagedListenerStartup(): Unit = { def testStagedListenerStartup(): Unit = {
shutdownServerAndMetrics(server) shutdownServerAndMetrics(server)
@ -1063,7 +1017,7 @@ class SocketServerTest {
val clientId = "" val clientId = ""
val ackTimeoutMs = 10000 val ackTimeoutMs = 10000
val ack = 0: Short val ack = 0: Short
val emptyRequest = requests.ProduceRequest.forCurrentMagic(new ProduceRequestData() val emptyRequest = requests.ProduceRequest.builder(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection()) .setTopicData(new ProduceRequestData.TopicProduceDataCollection())
.setAcks(ack) .setAcks(ack)
.setTimeoutMs(ackTimeoutMs) .setTimeoutMs(ackTimeoutMs)
@ -1145,7 +1099,7 @@ class SocketServerTest {
// ...and now send something to trigger the disconnection // ...and now send something to trigger the disconnection
val ackTimeoutMs = 10000 val ackTimeoutMs = 10000
val ack = 0: Short val ack = 0: Short
val emptyRequest = requests.ProduceRequest.forCurrentMagic(new ProduceRequestData() val emptyRequest = requests.ProduceRequest.builder(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection()) .setTopicData(new ProduceRequestData.TopicProduceDataCollection())
.setAcks(ack) .setAcks(ack)
.setTimeoutMs(ackTimeoutMs) .setTimeoutMs(ackTimeoutMs)

View File

@ -73,7 +73,7 @@ class ApiVersionManagerTest {
@Test @Test
def testControllerApiIntersection(): Unit = { def testControllerApiIntersection(): Unit = {
val controllerMinVersion: Short = 1 val controllerMinVersion: Short = 3
val controllerMaxVersion: Short = 5 val controllerMaxVersion: Short = 5
val forwardingManager = Mockito.mock(classOf[ForwardingManager]) val forwardingManager = Mockito.mock(classOf[ForwardingManager])

View File

@ -339,8 +339,8 @@ class AutoTopicCreationManagerTest {
topicsCollection.add(getNewTopic(topicName)) topicsCollection.add(getNewTopic(topicName))
val createTopicApiVersion = new ApiVersionsResponseData.ApiVersion() val createTopicApiVersion = new ApiVersionsResponseData.ApiVersion()
.setApiKey(ApiKeys.CREATE_TOPICS.id) .setApiKey(ApiKeys.CREATE_TOPICS.id)
.setMinVersion(0) .setMinVersion(2)
.setMaxVersion(0) .setMaxVersion(2)
Mockito.when(brokerToController.controllerApiVersions()) Mockito.when(brokerToController.controllerApiVersions())
.thenReturn(Optional.of(NodeApiVersions.create(Collections.singleton(createTopicApiVersion)))) .thenReturn(Optional.of(NodeApiVersions.create(Collections.singleton(createTopicApiVersion))))

View File

@ -897,7 +897,7 @@ class ConsumerProtocolMigrationTest(cluster: ClusterInstance) extends GroupCoord
) )
// Member 1 commits offset. Start from version 1 because version 0 goes to ZK. // Member 1 commits offset. Start from version 1 because version 0 goes to ZK.
for (version <- 1 to ApiKeys.OFFSET_COMMIT.latestVersion(isUnstableApiEnabled)) { for (version <- ApiKeys.OFFSET_COMMIT.oldestVersion to ApiKeys.OFFSET_COMMIT.latestVersion(isUnstableApiEnabled)) {
for (partitionId <- 0 to 2) { for (partitionId <- 0 to 2) {
commitOffset( commitOffset(
groupId = groupId, groupId = groupId,
@ -1196,7 +1196,7 @@ class ConsumerProtocolMigrationTest(cluster: ClusterInstance) extends GroupCoord
) )
// Member 1 commits offset. Start from version 1 because version 0 goes to ZK. // Member 1 commits offset. Start from version 1 because version 0 goes to ZK.
for (version <- 1 to ApiKeys.OFFSET_COMMIT.latestVersion(isUnstableApiEnabled)) { for (version <- ApiKeys.OFFSET_COMMIT.oldestVersion to ApiKeys.OFFSET_COMMIT.latestVersion(isUnstableApiEnabled)) {
for (partitionId <- 0 to 2) { for (partitionId <- 0 to 2) {
commitOffset( commitOffset(
groupId = groupId, groupId = groupId,

View File

@ -22,7 +22,7 @@ import kafka.network.SocketServer
import kafka.utils.{Logging, TestUtils} import kafka.utils.{Logging, TestUtils}
import org.apache.kafka.common.message.DeleteTopicsRequestData import org.apache.kafka.common.message.DeleteTopicsRequestData
import org.apache.kafka.common.message.DeleteTopicsRequestData.DeleteTopicState import org.apache.kafka.common.message.DeleteTopicsRequestData.DeleteTopicState
import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.requests.DeleteTopicsRequest import org.apache.kafka.common.requests.DeleteTopicsRequest
import org.apache.kafka.common.requests.DeleteTopicsResponse import org.apache.kafka.common.requests.DeleteTopicsResponse
import org.apache.kafka.common.requests.MetadataRequest import org.apache.kafka.common.requests.MetadataRequest
@ -141,26 +141,22 @@ class DeleteTopicsRequestTest extends BaseRequestTest with Logging {
@ParameterizedTest @ParameterizedTest
@ValueSource(strings = Array("kraft")) @ValueSource(strings = Array("kraft"))
def testDeleteTopicsVersions(quorum: String): Unit = { def testDeleteTopicsVersions(quorum: String): Unit = {
// This test assumes that the current valid versions are 0-6 please adjust the test if there are changes.
assertEquals(0, DeleteTopicsRequestData.LOWEST_SUPPORTED_VERSION)
assertEquals(6, DeleteTopicsRequestData.HIGHEST_SUPPORTED_VERSION)
val timeout = 10000 val timeout = 10000
DeleteTopicsRequestData.SCHEMAS.indices.foreach { version => for (version <- ApiKeys.DELETE_TOPICS.oldestVersion to ApiKeys.DELETE_TOPICS.latestVersion) {
info(s"Creating and deleting tests for version $version") info(s"Creating and deleting tests for version $version")
val topicName = s"topic-$version" val topicName = s"topic-$version"
createTopic(topicName) createTopic(topicName)
val data = new DeleteTopicsRequestData().setTimeoutMs(timeout) val data = new DeleteTopicsRequestData().setTimeoutMs(timeout)
if (version < 6) { if (version < 6) {
data.setTopicNames(util.Arrays.asList(topicName)) data.setTopicNames(util.Arrays.asList(topicName))
} else { } else {
data.setTopics(util.Arrays.asList(new DeleteTopicState().setName(topicName))) data.setTopics(util.Arrays.asList(new DeleteTopicState().setName(topicName)))
} }
validateValidDeleteTopicRequests(new DeleteTopicsRequest.Builder(data).build(version.toShort)) validateValidDeleteTopicRequests(new DeleteTopicsRequest.Builder(data).build(version.toShort))
} }
} }
} }

View File

@ -129,7 +129,7 @@ class EdgeCaseRequestTest extends KafkaServerTestHarness {
val version = ApiKeys.PRODUCE.latestVersion: Short val version = ApiKeys.PRODUCE.latestVersion: Short
val (serializedBytes, responseHeaderVersion) = { val (serializedBytes, responseHeaderVersion) = {
val headerBytes = requestHeaderBytes(ApiKeys.PRODUCE.id, version, "", correlationId) val headerBytes = requestHeaderBytes(ApiKeys.PRODUCE.id, version, "", correlationId)
val request = requests.ProduceRequest.forCurrentMagic(new ProduceRequestData() val request = requests.ProduceRequest.builder(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection( .setTopicData(new ProduceRequestData.TopicProduceDataCollection(
Collections.singletonList(new ProduceRequestData.TopicProduceData() Collections.singletonList(new ProduceRequestData.TopicProduceData()
.setName(topicPartition.topic()).setPartitionData(Collections.singletonList( .setName(topicPartition.topic()).setPartitionData(Collections.singletonList(

View File

@ -1,275 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.server
import java.util
import java.util.{Optional, Properties}
import kafka.utils.TestUtils
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.message.FetchResponseData
import org.apache.kafka.common.{TopicPartition, Uuid}
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.requests.{FetchRequest, FetchResponse}
import org.apache.kafka.common.serialization.StringSerializer
import org.apache.kafka.network.metrics.RequestMetrics
import org.apache.kafka.server.config.ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_CONFIG
import org.apache.kafka.storage.log.metrics.BrokerTopicMetrics
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
import scala.jdk.CollectionConverters._
class FetchRequestDownConversionConfigTest extends BaseRequestTest {
private var producer: KafkaProducer[String, String] = _
override def brokerCount: Int = 2
@BeforeEach
override def setUp(testInfo: TestInfo): Unit = {
super.setUp(testInfo)
initProducer()
}
@AfterEach
override def tearDown(): Unit = {
if (producer != null)
producer.close()
super.tearDown()
}
override protected def brokerPropertyOverrides(properties: Properties): Unit = {
super.brokerPropertyOverrides(properties)
properties.put(LOG_MESSAGE_DOWNCONVERSION_ENABLE_CONFIG, "false")
}
private def initProducer(): Unit = {
producer = TestUtils.createProducer(bootstrapServers(),
keySerializer = new StringSerializer, valueSerializer = new StringSerializer)
}
private def createTopics(numTopics: Int, numPartitions: Int,
configs: Map[String, String] = Map.empty, topicSuffixStart: Int = 0): Map[TopicPartition, Int] = {
val topics = (0 until numTopics).map(t => s"topic${t + topicSuffixStart}")
val topicConfig = new Properties
topicConfig.setProperty(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, 1.toString)
configs.foreach { case (k, v) => topicConfig.setProperty(k, v) }
topics.flatMap { topic =>
val partitionToLeader = createTopic(
topic,
numPartitions = numPartitions,
replicationFactor = 2,
topicConfig = topicConfig
)
partitionToLeader.map { case (partition, leader) => new TopicPartition(topic, partition) -> leader }
}.toMap
}
private def createPartitionMap(maxPartitionBytes: Int, topicPartitions: Seq[TopicPartition],
topicIds: Map[String, Uuid],
offsetMap: Map[TopicPartition, Long] = Map.empty): util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData] = {
val partitionMap = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
topicPartitions.foreach { tp =>
partitionMap.put(tp, new FetchRequest.PartitionData(topicIds.getOrElse(tp.topic, Uuid.ZERO_UUID), offsetMap.getOrElse(tp, 0), 0L,
maxPartitionBytes, Optional.empty()))
}
partitionMap
}
private def sendFetchRequest(leaderId: Int, request: FetchRequest): FetchResponse = {
connectAndReceive[FetchResponse](request, destination = brokerSocketServer(leaderId))
}
/**
* Tests that fetch request that require down-conversion returns with an error response when down-conversion is disabled on broker.
*/
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testV1FetchWithDownConversionDisabled(quorum: String): Unit = {
val topicMap = createTopics(numTopics = 5, numPartitions = 1)
val topicPartitions = topicMap.keySet.toSeq
val topicIds = getTopicIds()
val topicNames = topicIds.map(_.swap)
topicPartitions.foreach(tp => producer.send(new ProducerRecord(tp.topic(), "key", "value")).get())
val fetchRequest = FetchRequest.Builder.forConsumer(1, Int.MaxValue, 0, createPartitionMap(1024,
topicPartitions, topicIds)).build(1)
val fetchResponse = sendFetchRequest(topicMap.head._2, fetchRequest)
val fetchResponseData = fetchResponse.responseData(topicNames.asJava, 1)
topicPartitions.foreach(tp => assertEquals(Errors.UNSUPPORTED_VERSION, Errors.forCode(fetchResponseData.get(tp).errorCode)))
}
/**
* Tests that "message.downconversion.enable" has no effect when down-conversion is not required.
*/
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testLatestFetchWithDownConversionDisabled(quorum: String): Unit = {
val topicMap = createTopics(numTopics = 5, numPartitions = 1)
val topicPartitions = topicMap.keySet.toSeq
val topicIds = getTopicIds()
val topicNames = topicIds.map(_.swap)
topicPartitions.foreach(tp => producer.send(new ProducerRecord(tp.topic(), "key", "value")).get())
val fetchRequest = FetchRequest.Builder.forConsumer(ApiKeys.FETCH.latestVersion, Int.MaxValue, 0, createPartitionMap(1024,
topicPartitions, topicIds)).build()
val fetchResponse = sendFetchRequest(topicMap.head._2, fetchRequest)
val fetchResponseData = fetchResponse.responseData(topicNames.asJava, ApiKeys.FETCH.latestVersion)
topicPartitions.foreach(tp => assertEquals(Errors.NONE, Errors.forCode(fetchResponseData.get(tp).errorCode)))
}
/**
* Tests that "message.downconversion.enable" has no effect when down-conversion is not required on last version before topic IDs.
*/
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testV12WithDownConversionDisabled(quorum: String): Unit = {
val topicMap = createTopics(numTopics = 5, numPartitions = 1)
val topicPartitions = topicMap.keySet.toSeq
val topicIds = getTopicIds()
val topicNames = topicIds.map(_.swap)
topicPartitions.foreach(tp => producer.send(new ProducerRecord(tp.topic(), "key", "value")).get())
val fetchRequest = FetchRequest.Builder.forConsumer(ApiKeys.FETCH.latestVersion, Int.MaxValue, 0, createPartitionMap(1024,
topicPartitions, topicIds)).build(12)
val fetchResponse = sendFetchRequest(topicMap.head._2, fetchRequest)
val fetchResponseData = fetchResponse.responseData(topicNames.asJava, 12)
topicPartitions.foreach(tp => assertEquals(Errors.NONE, Errors.forCode(fetchResponseData.get(tp).errorCode)))
}
/**
* Tests that "message.downconversion.enable" can be set at topic level, and its configuration is obeyed for client
* fetch requests.
*/
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testV1FetchFromConsumer(quorum: String): Unit = {
testV1Fetch(isFollowerFetch = false)
}
/**
* Tests that "message.downconversion.enable" has no effect on fetch requests from replicas.
*/
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testV1FetchFromReplica(quorum: String): Unit = {
testV1Fetch(isFollowerFetch = true)
}
def testV1Fetch(isFollowerFetch: Boolean): Unit = {
val fetchRequest = "request=Fetch"
val fetchTemporaryMemoryBytesMetricName = s"${RequestMetrics.TEMPORARY_MEMORY_BYTES},$fetchRequest"
val fetchMessageConversionsTimeMsMetricName = s"${RequestMetrics.MESSAGE_CONVERSIONS_TIME_MS},$fetchRequest"
val initialFetchMessageConversionsPerSec = TestUtils.metersCount(BrokerTopicMetrics.FETCH_MESSAGE_CONVERSIONS_PER_SEC)
val initialFetchMessageConversionsTimeMs = TestUtils.metersCount(fetchMessageConversionsTimeMsMetricName)
val initialFetchTemporaryMemoryBytes = TestUtils.metersCount(fetchTemporaryMemoryBytesMetricName)
val topicWithDownConversionEnabled = "foo"
val topicWithDownConversionDisabled = "bar"
val replicaIds = brokers.map(_.config.brokerId)
val leaderId = replicaIds.head
val followerId = replicaIds.last
val admin = createAdminClient()
val topicWithDownConversionDisabledId = TestUtils.createTopicWithAdminRaw(
admin,
topicWithDownConversionDisabled,
replicaAssignment = Map(0 -> replicaIds)
)
val topicConfig = new Properties
topicConfig.put(TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_CONFIG, "true")
val topicWithDownConversionEnabledId = TestUtils.createTopicWithAdminRaw(
admin,
topicWithDownConversionEnabled,
replicaAssignment = Map(0 -> replicaIds),
topicConfig = topicConfig
)
val partitionWithDownConversionEnabled = new TopicPartition(topicWithDownConversionEnabled, 0)
val partitionWithDownConversionDisabled = new TopicPartition(topicWithDownConversionDisabled, 0)
val allTopicPartitions = Seq(
partitionWithDownConversionEnabled,
partitionWithDownConversionDisabled
)
allTopicPartitions.foreach { tp =>
producer.send(new ProducerRecord(tp.topic, "key", "value")).get()
}
val topicIdMap = Map(
topicWithDownConversionEnabled -> topicWithDownConversionEnabledId,
topicWithDownConversionDisabled -> topicWithDownConversionDisabledId
)
val fetchResponseData = sendFetch(
leaderId,
allTopicPartitions,
topicIdMap,
fetchVersion = 1,
replicaIdOpt = if (isFollowerFetch) Some(followerId) else None
)
def error(tp: TopicPartition): Errors = {
Errors.forCode(fetchResponseData.get(tp).errorCode)
}
def verifyMetrics(): Unit = {
TestUtils.waitUntilTrue(() => TestUtils.metersCount(BrokerTopicMetrics.FETCH_MESSAGE_CONVERSIONS_PER_SEC) > initialFetchMessageConversionsPerSec,
s"The `FetchMessageConversionsPerSec` metric count is not incremented after 5 seconds. " +
s"init: $initialFetchMessageConversionsPerSec final: ${TestUtils.metersCount(BrokerTopicMetrics.FETCH_MESSAGE_CONVERSIONS_PER_SEC)}", 5000)
TestUtils.waitUntilTrue(() => TestUtils.metersCount(fetchMessageConversionsTimeMsMetricName) > initialFetchMessageConversionsTimeMs,
s"The `MessageConversionsTimeMs` in fetch request metric count is not incremented after 5 seconds. " +
s"init: $initialFetchMessageConversionsTimeMs final: ${TestUtils.metersCount(fetchMessageConversionsTimeMsMetricName)}", 5000)
TestUtils.waitUntilTrue(() => TestUtils.metersCount(fetchTemporaryMemoryBytesMetricName) > initialFetchTemporaryMemoryBytes,
s"The `TemporaryMemoryBytes` in fetch request metric count is not incremented after 5 seconds. " +
s"init: $initialFetchTemporaryMemoryBytes final: ${TestUtils.metersCount(fetchTemporaryMemoryBytesMetricName)}", 5000)
}
assertEquals(Errors.NONE, error(partitionWithDownConversionEnabled))
if (isFollowerFetch) {
assertEquals(Errors.NONE, error(partitionWithDownConversionDisabled))
} else {
assertEquals(Errors.UNSUPPORTED_VERSION, error(partitionWithDownConversionDisabled))
}
verifyMetrics()
}
private def sendFetch(
leaderId: Int,
partitions: Seq[TopicPartition],
topicIdMap: Map[String, Uuid],
fetchVersion: Short,
replicaIdOpt: Option[Int]
): util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData] = {
val topicNameMap = topicIdMap.map(_.swap)
val partitionMap = createPartitionMap(1024, partitions, topicIdMap)
val fetchRequest = replicaIdOpt.map { replicaId =>
FetchRequest.Builder.forReplica(fetchVersion, replicaId, -1, Int.MaxValue, 0, partitionMap)
.build(fetchVersion)
}.getOrElse {
FetchRequest.Builder.forConsumer(fetchVersion, Int.MaxValue, 0, partitionMap)
.build(fetchVersion)
}
val fetchResponse = sendFetchRequest(leaderId, fetchRequest)
fetchResponse.responseData(topicNameMap.asJava, fetchVersion)
}
}

View File

@ -116,11 +116,12 @@ class FetchRequestMaxBytesTest extends BaseRequestTest {
private def expectNextRecords(expected: IndexedSeq[Array[Byte]], private def expectNextRecords(expected: IndexedSeq[Array[Byte]],
fetchOffset: Long): Unit = { fetchOffset: Long): Unit = {
val requestVersion = 4: Short
val response = sendFetchRequest(0, val response = sendFetchRequest(0,
FetchRequest.Builder.forConsumer(3, Int.MaxValue, 0, FetchRequest.Builder.forConsumer(requestVersion, Int.MaxValue, 0,
Map(testTopicPartition -> Map(testTopicPartition ->
new PartitionData(Uuid.ZERO_UUID, fetchOffset, 0, Integer.MAX_VALUE, Optional.empty())).asJava).build(3)) new PartitionData(Uuid.ZERO_UUID, fetchOffset, 0, Integer.MAX_VALUE, Optional.empty())).asJava).build(requestVersion))
val records = FetchResponse.recordsOrFail(response.responseData(getTopicNames().asJava, 3).get(testTopicPartition)).records() val records = FetchResponse.recordsOrFail(response.responseData(getTopicNames().asJava, requestVersion).get(testTopicPartition)).records()
assertNotNull(records) assertNotNull(records)
val recordsList = records.asScala.toList val recordsList = records.asScala.toList
assertEquals(expected.size, recordsList.size) assertEquals(expected.size, recordsList.size)

View File

@ -20,16 +20,15 @@ import kafka.utils.TestUtils
import org.apache.kafka.clients.producer.ProducerRecord import org.apache.kafka.clients.producer.ProducerRecord
import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.record.{CompressionType, RecordBatch} import org.apache.kafka.common.record.CompressionType
import org.apache.kafka.common.requests.{FetchRequest, FetchResponse, FetchMetadata => JFetchMetadata} import org.apache.kafka.common.requests.{FetchRequest, FetchResponse, FetchMetadata => JFetchMetadata}
import org.apache.kafka.common.serialization.{ByteArraySerializer, StringSerializer} import org.apache.kafka.common.serialization.StringSerializer
import org.apache.kafka.common.{IsolationLevel, TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.common.{IsolationLevel, TopicIdPartition, TopicPartition, Uuid}
import org.apache.kafka.server.record.BrokerCompressionType import org.apache.kafka.server.record.BrokerCompressionType
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource import org.junit.jupiter.params.provider.ValueSource
import java.io.DataInputStream
import java.util import java.util
import java.util.Optional import java.util.Optional
import scala.collection.Seq import scala.collection.Seq
@ -363,156 +362,6 @@ class FetchRequestTest extends BaseFetchRequestTest {
assertResponseErrorForEpoch(Errors.UNKNOWN_LEADER_EPOCH, 2, Optional.of(leaderEpoch + 1)) assertResponseErrorForEpoch(Errors.UNKNOWN_LEADER_EPOCH, 2, Optional.of(leaderEpoch + 1))
} }
/**
* Tests that down-conversions don't leak memory. Large down conversions are triggered
* in the server. The client closes its connection after reading partial data when the
* channel is muted in the server. If buffers are not released this will result in OOM.
*/
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testDownConversionWithConnectionFailure(quorum: String): Unit = {
val (topicPartition, leaderId) = createTopics(numTopics = 1, numPartitions = 1).head
val topicIds = getTopicIds().asJava
val topicNames = topicIds.asScala.map(_.swap).asJava
val msgValueLen = 100 * 1000
val batchSize = 4 * msgValueLen
val producer = TestUtils.createProducer(
bootstrapServers(),
lingerMs = Int.MaxValue,
deliveryTimeoutMs = Int.MaxValue,
batchSize = batchSize,
keySerializer = new StringSerializer,
valueSerializer = new ByteArraySerializer)
val bytes = new Array[Byte](msgValueLen)
val futures = try {
(0 to 1000).map { _ =>
producer.send(new ProducerRecord(topicPartition.topic, topicPartition.partition, "key", bytes))
}
} finally {
producer.close()
}
// Check futures to ensure sends succeeded, but do this after close since the last
// batch is not complete, but sent when the producer is closed
futures.foreach(_.get)
def fetch(version: Short, maxPartitionBytes: Int, closeAfterPartialResponse: Boolean): Option[FetchResponse] = {
val fetchRequest = FetchRequest.Builder.forConsumer(version, Int.MaxValue, 0, createPartitionMap(maxPartitionBytes,
Seq(topicPartition))).build(version)
val socket = connect(brokerSocketServer(leaderId))
try {
send(fetchRequest, socket)
if (closeAfterPartialResponse) {
// read some data to ensure broker has muted this channel and then close socket
val size = new DataInputStream(socket.getInputStream).readInt()
// Check that we have received almost `maxPartitionBytes` (minus a tolerance) since in
// the case of OOM, the size will be significantly smaller. We can't check for exactly
// maxPartitionBytes since we use approx message sizes that include only the message value.
assertTrue(size > maxPartitionBytes - batchSize,
s"Fetch size too small $size, broker may have run out of memory")
None
} else {
Some(receive[FetchResponse](socket, ApiKeys.FETCH, version))
}
} finally {
socket.close()
}
}
val version = 1.toShort
(0 to 15).foreach(_ => fetch(version, maxPartitionBytes = msgValueLen * 1000, closeAfterPartialResponse = true))
val response = fetch(version, maxPartitionBytes = batchSize, closeAfterPartialResponse = false)
val fetchResponse = response.getOrElse(throw new IllegalStateException("No fetch response"))
val partitionData = fetchResponse.responseData(topicNames, version).get(topicPartition)
assertEquals(Errors.NONE.code, partitionData.errorCode)
val batches = FetchResponse.recordsOrFail(partitionData).batches.asScala.toBuffer
assertEquals(3, batches.size) // size is 3 (not 4) since maxPartitionBytes=msgValueSize*4, excluding key and headers
}
/**
* Ensure that we respect the fetch offset when returning records that were converted from an uncompressed v2
* record batch to multiple v0/v1 record batches with size 1. If the fetch offset points to inside the record batch,
* some records have to be dropped during the conversion.
*/
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testDownConversionFromBatchedToUnbatchedRespectsOffset(quorum: String): Unit = {
// Increase linger so that we have control over the batches created
producer = TestUtils.createProducer(bootstrapServers(),
retries = 5,
keySerializer = new StringSerializer,
valueSerializer = new StringSerializer,
lingerMs = 30 * 1000,
deliveryTimeoutMs = 60 * 1000)
val (topicPartition, leaderId) = createTopics(numTopics = 1, numPartitions = 1).head
val topic = topicPartition.topic
val topicIds = getTopicIds().asJava
val topicNames = topicIds.asScala.map(_.swap).asJava
val firstBatchFutures = (0 until 10).map(i => producer.send(new ProducerRecord(topic, s"key-$i", s"value-$i")))
producer.flush()
val secondBatchFutures = (10 until 25).map(i => producer.send(new ProducerRecord(topic, s"key-$i", s"value-$i")))
producer.flush()
firstBatchFutures.foreach(_.get)
secondBatchFutures.foreach(_.get)
def check(fetchOffset: Long, requestVersion: Short, expectedOffset: Long, expectedNumBatches: Int, expectedMagic: Byte): Unit = {
var batchesReceived = 0
var currentFetchOffset = fetchOffset
var currentExpectedOffset = expectedOffset
// With KIP-283, we might not receive all batches in a single fetch request so loop through till we have consumed
// all batches we are interested in.
while (batchesReceived < expectedNumBatches) {
val fetchRequest = FetchRequest.Builder.forConsumer(requestVersion, Int.MaxValue, 0, createPartitionMap(Int.MaxValue,
Seq(topicPartition), Map(topicPartition -> currentFetchOffset))).build(requestVersion)
val fetchResponse = sendFetchRequest(leaderId, fetchRequest)
// validate response
val partitionData = fetchResponse.responseData(topicNames, requestVersion).get(topicPartition)
assertEquals(Errors.NONE.code, partitionData.errorCode)
assertTrue(partitionData.highWatermark > 0)
val batches = FetchResponse.recordsOrFail(partitionData).batches.asScala.toBuffer
val batch = batches.head
assertEquals(expectedMagic, batch.magic)
assertEquals(currentExpectedOffset, batch.baseOffset)
currentFetchOffset = batches.last.lastOffset + 1
currentExpectedOffset += (batches.last.lastOffset - batches.head.baseOffset + 1)
batchesReceived += batches.size
}
assertEquals(expectedNumBatches, batchesReceived)
}
// down conversion to message format 0, batches of 1 message are returned so we receive the exact offset we requested
check(fetchOffset = 3, expectedOffset = 3, requestVersion = 1, expectedNumBatches = 22,
expectedMagic = RecordBatch.MAGIC_VALUE_V0)
check(fetchOffset = 15, expectedOffset = 15, requestVersion = 1, expectedNumBatches = 10,
expectedMagic = RecordBatch.MAGIC_VALUE_V0)
// down conversion to message format 1, batches of 1 message are returned so we receive the exact offset we requested
check(fetchOffset = 3, expectedOffset = 3, requestVersion = 3, expectedNumBatches = 22,
expectedMagic = RecordBatch.MAGIC_VALUE_V1)
check(fetchOffset = 15, expectedOffset = 15, requestVersion = 3, expectedNumBatches = 10,
expectedMagic = RecordBatch.MAGIC_VALUE_V1)
// no down conversion, we receive a single batch so the received offset won't necessarily be the same
check(fetchOffset = 3, expectedOffset = 0, requestVersion = 4, expectedNumBatches = 2,
expectedMagic = RecordBatch.MAGIC_VALUE_V2)
check(fetchOffset = 15, expectedOffset = 10, requestVersion = 4, expectedNumBatches = 1,
expectedMagic = RecordBatch.MAGIC_VALUE_V2)
// no down conversion, we receive a single batch and the exact offset we requested because it happens to be the
// offset of the first record in the batch
check(fetchOffset = 10, expectedOffset = 10, requestVersion = 4, expectedNumBatches = 1,
expectedMagic = RecordBatch.MAGIC_VALUE_V2)
}
/** /**
* Test that when an incremental fetch session contains partitions with an error, * Test that when an incremental fetch session contains partitions with an error,
* those partitions are returned in all incremental fetch requests. * those partitions are returned in all incremental fetch requests.
@ -654,7 +503,7 @@ class FetchRequestTest extends BaseFetchRequestTest {
val res0 = sendFetchRequest(leaderId, req0) val res0 = sendFetchRequest(leaderId, req0)
val data0 = res0.responseData(topicNames, 9).get(topicPartition) val data0 = res0.responseData(topicNames, 9).get(topicPartition)
assertEquals(Errors.UNSUPPORTED_COMPRESSION_TYPE.code, data0.errorCode) assertEquals(Errors.NONE.code, data0.errorCode)
// fetch request with version 10: works fine! // fetch request with version 10: works fine!
val req1= new FetchRequest.Builder(0, 10, -1, -1, Int.MaxValue, 0, val req1= new FetchRequest.Builder(0, 10, -1, -1, Int.MaxValue, 0,
@ -702,63 +551,34 @@ class FetchRequestTest extends BaseFetchRequestTest {
"key3", "value3")).get "key3", "value3")).get
producer2.close() producer2.close()
// fetch request with fetch version v1 (magic 0): // fetch request with version 4: even though zstd is officially only supported from v10, this actually succeeds
// gzip compressed record is returned with down-conversion. // since the server validation is only active when zstd is configured via a topic config, the server doesn't
// zstd compressed record raises UNSUPPORTED_COMPRESSION_TYPE error. // check the record batches and hence has no mechanism to detect the case where the producer sent record batches
val req0 = new FetchRequest.Builder(0, 1, -1, -1, Int.MaxValue, 0, // compressed with zstd and the topic config for compression is the default
val req0 = new FetchRequest.Builder(0, 4, -1, -1, Int.MaxValue, 0,
createPartitionMap(300, Seq(topicPartition), Map.empty)) createPartitionMap(300, Seq(topicPartition), Map.empty))
.setMaxBytes(800) .setMaxBytes(800).build()
.build()
val res0 = sendFetchRequest(leaderId, req0) val res0 = sendFetchRequest(leaderId, req0)
val data0 = res0.responseData(topicNames, 1).get(topicPartition) val data0 = res0.responseData(topicNames, 10).get(topicPartition)
assertEquals(Errors.NONE.code, data0.errorCode) assertEquals(Errors.NONE.code, data0.errorCode)
assertEquals(1, records(data0).size) assertEquals(3, records(data0).size)
val req1 = new FetchRequest.Builder(0, 1, -1, -1, Int.MaxValue, 0,
createPartitionMap(300, Seq(topicPartition), Map(topicPartition -> 1L)))
.setMaxBytes(800).build()
val res1 = sendFetchRequest(leaderId, req1)
val data1 = res1.responseData(topicNames, 1).get(topicPartition)
assertEquals(Errors.UNSUPPORTED_COMPRESSION_TYPE.code, data1.errorCode)
// fetch request with fetch version v3 (magic 1):
// gzip compressed record is returned with down-conversion.
// zstd compressed record raises UNSUPPORTED_COMPRESSION_TYPE error.
val req2 = new FetchRequest.Builder(2, 3, -1, -1, Int.MaxValue, 0,
createPartitionMap(300, Seq(topicPartition), Map.empty))
.setMaxBytes(800).build()
val res2 = sendFetchRequest(leaderId, req2)
val data2 = res2.responseData(topicNames, 3).get(topicPartition)
assertEquals(Errors.NONE.code, data2.errorCode)
assertEquals(1, records(data2).size)
val req3 = new FetchRequest.Builder(0, 1, -1, -1, Int.MaxValue, 0,
createPartitionMap(300, Seq(topicPartition), Map(topicPartition -> 1L)))
.setMaxBytes(800).build()
val res3 = sendFetchRequest(leaderId, req3)
val data3 = res3.responseData(topicNames, 1).get(topicPartition)
assertEquals(Errors.UNSUPPORTED_COMPRESSION_TYPE.code, data3.errorCode)
// fetch request with version 10: works fine! // fetch request with version 10: works fine!
val req4 = new FetchRequest.Builder(0, 10, -1, -1, Int.MaxValue, 0, val req1 = new FetchRequest.Builder(0, 10, -1, -1, Int.MaxValue, 0,
createPartitionMap(300, Seq(topicPartition), Map.empty)) createPartitionMap(300, Seq(topicPartition), Map.empty))
.setMaxBytes(800).build() .setMaxBytes(800).build()
val res4 = sendFetchRequest(leaderId, req4) val res1 = sendFetchRequest(leaderId, req1)
val data4 = res4.responseData(topicNames, 10).get(topicPartition) val data1 = res1.responseData(topicNames, 10).get(topicPartition)
assertEquals(Errors.NONE.code, data4.errorCode) assertEquals(Errors.NONE.code, data1.errorCode)
assertEquals(3, records(data4).size) assertEquals(3, records(data1).size)
val req5 = new FetchRequest.Builder(0, ApiKeys.FETCH.latestVersion(), -1, -1, Int.MaxValue, 0, val req2 = new FetchRequest.Builder(0, ApiKeys.FETCH.latestVersion(), -1, -1, Int.MaxValue, 0,
createPartitionMap(300, Seq(topicPartition), Map.empty)) createPartitionMap(300, Seq(topicPartition), Map.empty))
.setMaxBytes(800).build() .setMaxBytes(800).build()
val res5 = sendFetchRequest(leaderId, req5) val res2 = sendFetchRequest(leaderId, req2)
val data5 = res5.responseData(topicNames, ApiKeys.FETCH.latestVersion()).get(topicPartition) val data2 = res2.responseData(topicNames, ApiKeys.FETCH.latestVersion()).get(topicPartition)
assertEquals(Errors.NONE.code, data5.errorCode) assertEquals(Errors.NONE.code, data2.errorCode)
assertEquals(3, records(data5).size) assertEquals(3, records(data2).size)
} }
private def checkFetchResponse(expectedPartitions: Seq[TopicPartition], fetchResponse: FetchResponse, private def checkFetchResponse(expectedPartitions: Seq[TopicPartition], fetchResponse: FetchResponse,

View File

@ -210,7 +210,7 @@ class FetchSessionTest {
.setLastStableOffset(5) .setLastStableOffset(5)
.setLogStartOffset(5)) .setLogStartOffset(5))
val sessionId = context1.updateAndGenerateResponseData(response).sessionId() val sessionId = context1.updateAndGenerateResponseData(response, Seq.empty.asJava).sessionId()
// With no changes, the cached epochs should remain the same // With no changes, the cached epochs should remain the same
val requestData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData] val requestData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
@ -227,7 +227,7 @@ class FetchSessionTest {
assertEquals(Optional.empty(), epochs1(tp0)) assertEquals(Optional.empty(), epochs1(tp0))
assertEquals(Optional.of(1), epochs2(tp1)) assertEquals(Optional.of(1), epochs2(tp1))
assertEquals(Optional.of(2), epochs2(tp2)) assertEquals(Optional.of(2), epochs2(tp2))
context2.updateAndGenerateResponseData(response).sessionId() context2.updateAndGenerateResponseData(response, Seq.empty.asJava).sessionId()
// Now verify we can change the leader epoch and the context is updated // Now verify we can change the leader epoch and the context is updated
val requestData3 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData] val requestData3 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
@ -310,7 +310,7 @@ class FetchSessionTest {
.setLastStableOffset(5) .setLastStableOffset(5)
.setLogStartOffset(5)) .setLogStartOffset(5))
val sessionId = context1.updateAndGenerateResponseData(response).sessionId() val sessionId = context1.updateAndGenerateResponseData(response, Seq.empty.asJava).sessionId()
// With no changes, the cached epochs should remain the same // With no changes, the cached epochs should remain the same
val requestData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData] val requestData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
@ -326,7 +326,7 @@ class FetchSessionTest {
assertEquals(Map(tp0 -> Optional.empty, tp1 -> Optional.of(1), tp2 -> Optional.of(2)), cachedLeaderEpochs(context2)) assertEquals(Map(tp0 -> Optional.empty, tp1 -> Optional.of(1), tp2 -> Optional.of(2)), cachedLeaderEpochs(context2))
assertEquals(Map(tp0 -> Optional.empty, tp1 -> Optional.empty, tp2 -> Optional.of(1)), assertEquals(Map(tp0 -> Optional.empty, tp1 -> Optional.empty, tp2 -> Optional.of(1)),
cachedLastFetchedEpochs(context2)) cachedLastFetchedEpochs(context2))
context2.updateAndGenerateResponseData(response).sessionId() context2.updateAndGenerateResponseData(response, Seq.empty.asJava).sessionId()
// Now verify we can change the leader epoch and the context is updated // Now verify we can change the leader epoch and the context is updated
val requestData3 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData] val requestData3 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
@ -411,7 +411,7 @@ class FetchSessionTest {
.setHighWatermark(10) .setHighWatermark(10)
.setLastStableOffset(10) .setLastStableOffset(10)
.setLogStartOffset(10)) .setLogStartOffset(10))
val resp2 = context2.updateAndGenerateResponseData(respData2) val resp2 = context2.updateAndGenerateResponseData(respData2, Seq.empty.asJava)
assertEquals(Errors.NONE, resp2.error()) assertEquals(Errors.NONE, resp2.error())
assertTrue(resp2.sessionId() != INVALID_SESSION_ID) assertTrue(resp2.sessionId() != INVALID_SESSION_ID)
assertEquals(respData2.asScala.map { case (tp, data) => (tp.topicPartition, data)}.toMap.asJava, resp2.responseData(topicNames, request2.version)) assertEquals(respData2.asScala.map { case (tp, data) => (tp.topicPartition, data)}.toMap.asJava, resp2.responseData(topicNames, request2.version))
@ -428,7 +428,7 @@ class FetchSessionTest {
) )
assertEquals(classOf[SessionErrorContext], context3.getClass) assertEquals(classOf[SessionErrorContext], context3.getClass)
assertEquals(Errors.INVALID_FETCH_SESSION_EPOCH, assertEquals(Errors.INVALID_FETCH_SESSION_EPOCH,
context3.updateAndGenerateResponseData(respData2).error()) context3.updateAndGenerateResponseData(respData2, Seq.empty.asJava).error())
// Test trying to create a new session with a non-existent session id // Test trying to create a new session with a non-existent session id
val request4 = createRequest(new JFetchMetadata(resp2.sessionId() + 1, 1), reqData2, EMPTY_PART_LIST, isFromFollower = false) val request4 = createRequest(new JFetchMetadata(resp2.sessionId() + 1, 1), reqData2, EMPTY_PART_LIST, isFromFollower = false)
@ -441,7 +441,7 @@ class FetchSessionTest {
topicNames topicNames
) )
assertEquals(Errors.FETCH_SESSION_ID_NOT_FOUND, assertEquals(Errors.FETCH_SESSION_ID_NOT_FOUND,
context4.updateAndGenerateResponseData(respData2).error()) context4.updateAndGenerateResponseData(respData2, Seq.empty.asJava).error())
// Continue the first fetch session we created. // Continue the first fetch session we created.
val reqData5 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData] val reqData5 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
@ -463,7 +463,7 @@ class FetchSessionTest {
assertEquals(entry.getValue, data) assertEquals(entry.getValue, data)
}) })
assertEquals(10, context5.getFetchOffset(tp1).get) assertEquals(10, context5.getFetchOffset(tp1).get)
val resp5 = context5.updateAndGenerateResponseData(respData2) val resp5 = context5.updateAndGenerateResponseData(respData2, Seq.empty.asJava)
assertEquals(Errors.NONE, resp5.error()) assertEquals(Errors.NONE, resp5.error())
assertEquals(resp2.sessionId(), resp5.sessionId()) assertEquals(resp2.sessionId(), resp5.sessionId())
assertEquals(0, resp5.responseData(topicNames, request5.version).size()) assertEquals(0, resp5.responseData(topicNames, request5.version).size())
@ -480,7 +480,7 @@ class FetchSessionTest {
) )
assertEquals(classOf[SessionErrorContext], context6.getClass) assertEquals(classOf[SessionErrorContext], context6.getClass)
assertEquals(Errors.INVALID_FETCH_SESSION_EPOCH, assertEquals(Errors.INVALID_FETCH_SESSION_EPOCH,
context6.updateAndGenerateResponseData(respData2).error()) context6.updateAndGenerateResponseData(respData2, Seq.empty.asJava).error())
// Test generating a throttled response for the incremental fetch session // Test generating a throttled response for the incremental fetch session
val reqData7 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData] val reqData7 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
@ -493,7 +493,7 @@ class FetchSessionTest {
request7.forgottenTopics(topicNames), request7.forgottenTopics(topicNames),
topicNames topicNames
) )
val resp7 = context7.getThrottledResponse(100) val resp7 = context7.getThrottledResponse(100, Seq.empty.asJava)
assertEquals(Errors.NONE, resp7.error()) assertEquals(Errors.NONE, resp7.error())
assertEquals(resp2.sessionId(), resp7.sessionId()) assertEquals(resp2.sessionId(), resp7.sessionId())
assertEquals(100, resp7.throttleTimeMs()) assertEquals(100, resp7.throttleTimeMs())
@ -531,7 +531,7 @@ class FetchSessionTest {
.setHighWatermark(100) .setHighWatermark(100)
.setLastStableOffset(100) .setLastStableOffset(100)
.setLogStartOffset(100)) .setLogStartOffset(100))
val resp8 = context8.updateAndGenerateResponseData(respData8) val resp8 = context8.updateAndGenerateResponseData(respData8, Seq.empty.asJava)
assertEquals(Errors.NONE, resp8.error) assertEquals(Errors.NONE, resp8.error)
nextSessionId = resp8.sessionId nextSessionId = resp8.sessionId
} while (nextSessionId == prevSessionId) } while (nextSessionId == prevSessionId)
@ -579,7 +579,7 @@ class FetchSessionTest {
.setHighWatermark(10) .setHighWatermark(10)
.setLastStableOffset(10) .setLastStableOffset(10)
.setLogStartOffset(10)) .setLogStartOffset(10))
val resp1 = context1.updateAndGenerateResponseData(respData1) val resp1 = context1.updateAndGenerateResponseData(respData1, Seq.empty.asJava)
assertEquals(Errors.NONE, resp1.error()) assertEquals(Errors.NONE, resp1.error())
assertTrue(resp1.sessionId() != INVALID_SESSION_ID) assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
assertEquals(2, resp1.responseData(topicNames, request1.version).size()) assertEquals(2, resp1.responseData(topicNames, request1.version).size())
@ -620,7 +620,7 @@ class FetchSessionTest {
.setHighWatermark(10) .setHighWatermark(10)
.setLastStableOffset(10) .setLastStableOffset(10)
.setLogStartOffset(10)) .setLogStartOffset(10))
val resp2 = context2.updateAndGenerateResponseData(respData2) val resp2 = context2.updateAndGenerateResponseData(respData2, Seq.empty.asJava)
assertEquals(Errors.NONE, resp2.error) assertEquals(Errors.NONE, resp2.error)
assertEquals(1, resp2.responseData(topicNames, request2.version).size) assertEquals(1, resp2.responseData(topicNames, request2.version).size)
assertTrue(resp2.sessionId > 0) assertTrue(resp2.sessionId > 0)
@ -667,7 +667,7 @@ class FetchSessionTest {
.setHighWatermark(10) .setHighWatermark(10)
.setLastStableOffset(10) .setLastStableOffset(10)
.setLogStartOffset(10)) .setLogStartOffset(10))
val resp1 = context1.updateAndGenerateResponseData(respData1) val resp1 = context1.updateAndGenerateResponseData(respData1, Seq.empty.asJava)
// Since we are ignoring IDs, we should have no errors. // Since we are ignoring IDs, we should have no errors.
assertEquals(Errors.NONE, resp1.error()) assertEquals(Errors.NONE, resp1.error())
assertTrue(resp1.sessionId() != INVALID_SESSION_ID) assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
@ -723,7 +723,7 @@ class FetchSessionTest {
respData1.put(emptyZar0, new FetchResponseData.PartitionData() respData1.put(emptyZar0, new FetchResponseData.PartitionData()
.setPartitionIndex(1) .setPartitionIndex(1)
.setErrorCode(Errors.UNKNOWN_TOPIC_ID.code)) .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code))
val resp1 = context1.updateAndGenerateResponseData(respData1) val resp1 = context1.updateAndGenerateResponseData(respData1, Seq.empty.asJava)
// On the latest request version, we should have unknown topic ID errors. // On the latest request version, we should have unknown topic ID errors.
assertEquals(Errors.NONE, resp1.error()) assertEquals(Errors.NONE, resp1.error())
assertTrue(resp1.sessionId() != INVALID_SESSION_ID) assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
@ -767,7 +767,7 @@ class FetchSessionTest {
respData2.put(emptyZar0, new FetchResponseData.PartitionData() respData2.put(emptyZar0, new FetchResponseData.PartitionData()
.setPartitionIndex(1) .setPartitionIndex(1)
.setErrorCode(Errors.UNKNOWN_TOPIC_ID.code)) .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code))
val resp2 = context2.updateAndGenerateResponseData(respData2) val resp2 = context2.updateAndGenerateResponseData(respData2, Seq.empty.asJava)
// Since we are ignoring IDs, we should have no errors. // Since we are ignoring IDs, we should have no errors.
assertEquals(Errors.NONE, resp2.error()) assertEquals(Errors.NONE, resp2.error())
assertTrue(resp2.sessionId() != INVALID_SESSION_ID) assertTrue(resp2.sessionId() != INVALID_SESSION_ID)
@ -813,7 +813,7 @@ class FetchSessionTest {
.setHighWatermark(100) .setHighWatermark(100)
.setLastStableOffset(100) .setLastStableOffset(100)
.setLogStartOffset(100)) .setLogStartOffset(100))
val resp1 = context1.updateAndGenerateResponseData(respData1) val resp1 = context1.updateAndGenerateResponseData(respData1, Seq.empty.asJava)
assertEquals(Errors.NONE, resp1.error()) assertEquals(Errors.NONE, resp1.error())
assertTrue(resp1.sessionId() != INVALID_SESSION_ID) assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
@ -835,7 +835,7 @@ class FetchSessionTest {
assertEquals(classOf[SessionErrorContext], context2.getClass) assertEquals(classOf[SessionErrorContext], context2.getClass)
val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData] val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
assertEquals(Errors.FETCH_SESSION_TOPIC_ID_ERROR, assertEquals(Errors.FETCH_SESSION_TOPIC_ID_ERROR,
context2.updateAndGenerateResponseData(respData2).error()) context2.updateAndGenerateResponseData(respData2, Seq.empty.asJava).error())
} }
@Test @Test
@ -869,7 +869,7 @@ class FetchSessionTest {
.setHighWatermark(100) .setHighWatermark(100)
.setLastStableOffset(100) .setLastStableOffset(100)
.setLogStartOffset(100)) .setLogStartOffset(100))
val resp1 = context1.updateAndGenerateResponseData(respData1) val resp1 = context1.updateAndGenerateResponseData(respData1, Seq.empty.asJava)
assertEquals(Errors.NONE, resp1.error()) assertEquals(Errors.NONE, resp1.error())
assertTrue(resp1.sessionId() != INVALID_SESSION_ID) assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
@ -890,7 +890,7 @@ class FetchSessionTest {
assertEquals(classOf[SessionErrorContext], context2.getClass) assertEquals(classOf[SessionErrorContext], context2.getClass)
val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData] val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
assertEquals(Errors.FETCH_SESSION_TOPIC_ID_ERROR, assertEquals(Errors.FETCH_SESSION_TOPIC_ID_ERROR,
context2.updateAndGenerateResponseData(respData2).error()) context2.updateAndGenerateResponseData(respData2, Seq.empty.asJava).error())
} }
// This test simulates a session where the topic ID changes broker side (the one handling the request) in both the metadata cache and the log // This test simulates a session where the topic ID changes broker side (the one handling the request) in both the metadata cache and the log
@ -934,7 +934,7 @@ class FetchSessionTest {
.setLastStableOffset(-1) .setLastStableOffset(-1)
.setLogStartOffset(-1) .setLogStartOffset(-1)
.setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code)) .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code))
val resp1 = context1.updateAndGenerateResponseData(respData1) val resp1 = context1.updateAndGenerateResponseData(respData1, Seq.empty.asJava)
assertEquals(Errors.NONE, resp1.error()) assertEquals(Errors.NONE, resp1.error())
assertTrue(resp1.sessionId() != INVALID_SESSION_ID) assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
assertEquals(2, resp1.responseData(topicNames, request1.version).size) assertEquals(2, resp1.responseData(topicNames, request1.version).size)
@ -961,7 +961,7 @@ class FetchSessionTest {
.setLastStableOffset(-1) .setLastStableOffset(-1)
.setLogStartOffset(-1) .setLogStartOffset(-1)
.setErrorCode(Errors.INCONSISTENT_TOPIC_ID.code)) .setErrorCode(Errors.INCONSISTENT_TOPIC_ID.code))
val resp2 = context2.updateAndGenerateResponseData(respData2) val resp2 = context2.updateAndGenerateResponseData(respData2, Seq.empty.asJava)
assertEquals(Errors.NONE, resp2.error) assertEquals(Errors.NONE, resp2.error)
assertTrue(resp2.sessionId > 0) assertTrue(resp2.sessionId > 0)
@ -1031,7 +1031,7 @@ class FetchSessionTest {
noErrorResponse noErrorResponse
) )
} }
context.updateAndGenerateResponseData(data).sessionId context.updateAndGenerateResponseData(data, Seq.empty.asJava).sessionId
} }
val foo = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)) val foo = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0))
@ -1154,7 +1154,7 @@ class FetchSessionTest {
noErrorResponse noErrorResponse
) )
} }
context.updateAndGenerateResponseData(data).sessionId context.updateAndGenerateResponseData(data, Seq.empty.asJava).sessionId
} }
val foo = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)) val foo = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0))
@ -1254,7 +1254,7 @@ class FetchSessionTest {
errorResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION.code) errorResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION.code)
) )
} }
context.updateAndGenerateResponseData(data) context.updateAndGenerateResponseData(data, Seq.empty.asJava)
} }
val foo = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)) val foo = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0))
@ -1357,7 +1357,7 @@ class FetchSessionTest {
.setHighWatermark(10) .setHighWatermark(10)
.setLastStableOffset(10) .setLastStableOffset(10)
.setLogStartOffset(10)) .setLogStartOffset(10))
val session1resp = session1context1.updateAndGenerateResponseData(respData1) val session1resp = session1context1.updateAndGenerateResponseData(respData1, Seq.empty.asJava)
assertEquals(Errors.NONE, session1resp.error()) assertEquals(Errors.NONE, session1resp.error())
assertTrue(session1resp.sessionId() != INVALID_SESSION_ID) assertTrue(session1resp.sessionId() != INVALID_SESSION_ID)
assertEquals(2, session1resp.responseData(topicNames, session1request1.version).size) assertEquals(2, session1resp.responseData(topicNames, session1request1.version).size)
@ -1394,7 +1394,7 @@ class FetchSessionTest {
.setHighWatermark(10) .setHighWatermark(10)
.setLastStableOffset(10) .setLastStableOffset(10)
.setLogStartOffset(10)) .setLogStartOffset(10))
val session2resp = session2context.updateAndGenerateResponseData(respData1) val session2resp = session2context.updateAndGenerateResponseData(respData1, Seq.empty.asJava)
assertEquals(Errors.NONE, session2resp.error()) assertEquals(Errors.NONE, session2resp.error())
assertTrue(session2resp.sessionId() != INVALID_SESSION_ID) assertTrue(session2resp.sessionId() != INVALID_SESSION_ID)
assertEquals(2, session2resp.responseData(topicNames, session2request1.version()).size()) assertEquals(2, session2resp.responseData(topicNames, session2request1.version()).size())
@ -1452,7 +1452,7 @@ class FetchSessionTest {
.setHighWatermark(10) .setHighWatermark(10)
.setLastStableOffset(10) .setLastStableOffset(10)
.setLogStartOffset(10)) .setLogStartOffset(10))
val session3resp = session3context.updateAndGenerateResponseData(respData3) val session3resp = session3context.updateAndGenerateResponseData(respData3, Seq.empty.asJava)
assertEquals(Errors.NONE, session3resp.error()) assertEquals(Errors.NONE, session3resp.error())
assertTrue(session3resp.sessionId() != INVALID_SESSION_ID) assertTrue(session3resp.sessionId() != INVALID_SESSION_ID)
assertEquals(2, session3resp.responseData(topicNames, session3request1.version).size) assertEquals(2, session3resp.responseData(topicNames, session3request1.version).size)
@ -1500,7 +1500,7 @@ class FetchSessionTest {
.setHighWatermark(10) .setHighWatermark(10)
.setLastStableOffset(10) .setLastStableOffset(10)
.setLogStartOffset(10)) .setLogStartOffset(10))
val session1resp = session1context.updateAndGenerateResponseData(respData1) val session1resp = session1context.updateAndGenerateResponseData(respData1, Seq.empty.asJava)
assertEquals(Errors.NONE, session1resp.error()) assertEquals(Errors.NONE, session1resp.error())
assertTrue(session1resp.sessionId() != INVALID_SESSION_ID) assertTrue(session1resp.sessionId() != INVALID_SESSION_ID)
assertEquals(2, session1resp.responseData(topicNames, session1request.version).size) assertEquals(2, session1resp.responseData(topicNames, session1request.version).size)
@ -1538,7 +1538,7 @@ class FetchSessionTest {
.setHighWatermark(10) .setHighWatermark(10)
.setLastStableOffset(10) .setLastStableOffset(10)
.setLogStartOffset(10)) .setLogStartOffset(10))
val session2resp = session2context.updateAndGenerateResponseData(session2RespData) val session2resp = session2context.updateAndGenerateResponseData(session2RespData, Seq.empty.asJava)
assertEquals(Errors.NONE, session2resp.error()) assertEquals(Errors.NONE, session2resp.error())
assertTrue(session2resp.sessionId() != INVALID_SESSION_ID) assertTrue(session2resp.sessionId() != INVALID_SESSION_ID)
assertEquals(2, session2resp.responseData(topicNames, session2request.version).size) assertEquals(2, session2resp.responseData(topicNames, session2request.version).size)
@ -1578,7 +1578,7 @@ class FetchSessionTest {
.setHighWatermark(10) .setHighWatermark(10)
.setLastStableOffset(10) .setLastStableOffset(10)
.setLogStartOffset(10)) .setLogStartOffset(10))
val session3resp = session3context.updateAndGenerateResponseData(respData3) val session3resp = session3context.updateAndGenerateResponseData(respData3, Seq.empty.asJava)
assertEquals(Errors.NONE, session3resp.error()) assertEquals(Errors.NONE, session3resp.error())
assertTrue(session3resp.sessionId() != INVALID_SESSION_ID) assertTrue(session3resp.sessionId() != INVALID_SESSION_ID)
assertEquals(2, session3resp.responseData(topicNames, session3request.version).size) assertEquals(2, session3resp.responseData(topicNames, session3request.version).size)
@ -1621,7 +1621,7 @@ class FetchSessionTest {
.setHighWatermark(10) .setHighWatermark(10)
.setLastStableOffset(10) .setLastStableOffset(10)
.setLogStartOffset(10)) .setLogStartOffset(10))
val session4resp = session3context.updateAndGenerateResponseData(respData4) val session4resp = session3context.updateAndGenerateResponseData(respData4, Seq.empty.asJava)
assertEquals(Errors.NONE, session4resp.error()) assertEquals(Errors.NONE, session4resp.error())
assertTrue(session4resp.sessionId() != INVALID_SESSION_ID) assertTrue(session4resp.sessionId() != INVALID_SESSION_ID)
assertEquals(2, session4resp.responseData(topicNames, session4request.version).size) assertEquals(2, session4resp.responseData(topicNames, session4request.version).size)
@ -1669,7 +1669,7 @@ class FetchSessionTest {
.setHighWatermark(10) .setHighWatermark(10)
.setLastStableOffset(10) .setLastStableOffset(10)
.setLogStartOffset(10)) .setLogStartOffset(10))
val resp1 = context1.updateAndGenerateResponseData(respData1) val resp1 = context1.updateAndGenerateResponseData(respData1, Seq.empty.asJava)
assertEquals(Errors.NONE, resp1.error) assertEquals(Errors.NONE, resp1.error)
assertTrue(resp1.sessionId() != INVALID_SESSION_ID) assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
assertEquals(2, resp1.responseData(topicNames, request1.version).size) assertEquals(2, resp1.responseData(topicNames, request1.version).size)
@ -1691,7 +1691,7 @@ class FetchSessionTest {
) )
assertEquals(classOf[SessionlessFetchContext], context2.getClass) assertEquals(classOf[SessionlessFetchContext], context2.getClass)
val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData] val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
val resp2 = context2.updateAndGenerateResponseData(respData2) val resp2 = context2.updateAndGenerateResponseData(respData2, Seq.empty.asJava)
assertEquals(INVALID_SESSION_ID, resp2.sessionId) assertEquals(INVALID_SESSION_ID, resp2.sessionId)
assertTrue(resp2.responseData(topicNames, request2.version).isEmpty) assertTrue(resp2.responseData(topicNames, request2.version).isEmpty)
assertEquals(0, cacheShard.size) assertEquals(0, cacheShard.size)
@ -1735,7 +1735,7 @@ class FetchSessionTest {
.setLastStableOffset(105) .setLastStableOffset(105)
.setLogStartOffset(0) .setLogStartOffset(0)
.setDivergingEpoch(divergingEpoch)) .setDivergingEpoch(divergingEpoch))
val resp1 = context1.updateAndGenerateResponseData(respData) val resp1 = context1.updateAndGenerateResponseData(respData, Seq.empty.asJava)
assertEquals(Errors.NONE, resp1.error) assertEquals(Errors.NONE, resp1.error)
assertNotEquals(INVALID_SESSION_ID, resp1.sessionId) assertNotEquals(INVALID_SESSION_ID, resp1.sessionId)
assertEquals(util.Set.of(tp1.topicPartition, tp2.topicPartition), resp1.responseData(topicNames, request1.version).keySet) assertEquals(util.Set.of(tp1.topicPartition, tp2.topicPartition), resp1.responseData(topicNames, request1.version).keySet)
@ -1752,7 +1752,7 @@ class FetchSessionTest {
topicNames topicNames
) )
assertEquals(classOf[IncrementalFetchContext], context2.getClass) assertEquals(classOf[IncrementalFetchContext], context2.getClass)
val resp2 = context2.updateAndGenerateResponseData(respData) val resp2 = context2.updateAndGenerateResponseData(respData, Seq.empty.asJava)
assertEquals(Errors.NONE, resp2.error) assertEquals(Errors.NONE, resp2.error)
assertEquals(resp1.sessionId, resp2.sessionId) assertEquals(resp1.sessionId, resp2.sessionId)
assertEquals(Collections.singleton(tp2.topicPartition), resp2.responseData(topicNames, request2.version).keySet) assertEquals(Collections.singleton(tp2.topicPartition), resp2.responseData(topicNames, request2.version).keySet)
@ -1764,7 +1764,7 @@ class FetchSessionTest {
.setLastStableOffset(105) .setLastStableOffset(105)
.setLogStartOffset(0) .setLogStartOffset(0)
.setDivergingEpoch(divergingEpoch)) .setDivergingEpoch(divergingEpoch))
val resp3 = context2.updateAndGenerateResponseData(respData) val resp3 = context2.updateAndGenerateResponseData(respData, Seq.empty.asJava)
assertEquals(Errors.NONE, resp3.error) assertEquals(Errors.NONE, resp3.error)
assertEquals(resp1.sessionId, resp3.sessionId) assertEquals(resp1.sessionId, resp3.sessionId)
assertEquals(util.Set.of(tp1.topicPartition, tp2.topicPartition), resp3.responseData(topicNames, request2.version).keySet) assertEquals(util.Set.of(tp1.topicPartition, tp2.topicPartition), resp3.responseData(topicNames, request2.version).keySet)
@ -1776,7 +1776,7 @@ class FetchSessionTest {
.setHighWatermark(110) .setHighWatermark(110)
.setLastStableOffset(110) .setLastStableOffset(110)
.setLogStartOffset(0)) .setLogStartOffset(0))
val resp4 = context2.updateAndGenerateResponseData(respData) val resp4 = context2.updateAndGenerateResponseData(respData, Seq.empty.asJava)
assertEquals(Errors.NONE, resp4.error) assertEquals(Errors.NONE, resp4.error)
assertEquals(resp1.sessionId, resp4.sessionId) assertEquals(resp1.sessionId, resp4.sessionId)
assertEquals(util.Set.of(tp1.topicPartition, tp2.topicPartition), resp4.responseData(topicNames, request2.version).keySet) assertEquals(util.Set.of(tp1.topicPartition, tp2.topicPartition), resp4.responseData(topicNames, request2.version).keySet)
@ -1820,7 +1820,7 @@ class FetchSessionTest {
.setLastStableOffset(50) .setLastStableOffset(50)
.setLogStartOffset(0)) .setLogStartOffset(0))
val resp1 = context1.updateAndGenerateResponseData(respData1) val resp1 = context1.updateAndGenerateResponseData(respData1, Seq.empty.asJava)
assertEquals(Errors.NONE, resp1.error) assertEquals(Errors.NONE, resp1.error)
assertNotEquals(INVALID_SESSION_ID, resp1.sessionId) assertNotEquals(INVALID_SESSION_ID, resp1.sessionId)
assertEquals(util.Set.of(tp1.topicPartition, tp2.topicPartition, tp3.topicPartition), resp1.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet()) assertEquals(util.Set.of(tp1.topicPartition, tp2.topicPartition, tp3.topicPartition), resp1.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet())
@ -1836,7 +1836,7 @@ class FetchSessionTest {
// Response is empty // Response is empty
val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData] val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
val resp2 = context2.updateAndGenerateResponseData(respData2) val resp2 = context2.updateAndGenerateResponseData(respData2, Seq.empty.asJava)
assertEquals(Errors.NONE, resp2.error) assertEquals(Errors.NONE, resp2.error)
assertEquals(resp1.sessionId, resp2.sessionId) assertEquals(resp1.sessionId, resp2.sessionId)
assertEquals(Collections.emptySet(), resp2.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet) assertEquals(Collections.emptySet(), resp2.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet)
@ -1860,7 +1860,7 @@ class FetchSessionTest {
.setHighWatermark(50) .setHighWatermark(50)
.setLastStableOffset(50) .setLastStableOffset(50)
.setLogStartOffset(0)) .setLogStartOffset(0))
val resp3 = context2.updateAndGenerateResponseData(respData3) val resp3 = context2.updateAndGenerateResponseData(respData3, Seq.empty.asJava)
assertEquals(Errors.NONE, resp3.error) assertEquals(Errors.NONE, resp3.error)
assertEquals(resp1.sessionId, resp3.sessionId) assertEquals(resp1.sessionId, resp3.sessionId)
assertEquals(util.Set.of(tp1.topicPartition, tp2.topicPartition), resp3.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet) assertEquals(util.Set.of(tp1.topicPartition, tp2.topicPartition), resp3.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet)

View File

@ -2651,7 +2651,7 @@ class KafkaApisTest extends Logging {
val tp = new TopicPartition("topic", 0) val tp = new TopicPartition("topic", 0)
val produceRequest = ProduceRequest.forCurrentMagic(new ProduceRequestData() val produceRequest = ProduceRequest.builder(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection( .setTopicData(new ProduceRequestData.TopicProduceDataCollection(
Collections.singletonList(new ProduceRequestData.TopicProduceData() Collections.singletonList(new ProduceRequestData.TopicProduceData()
.setName(tp.topic).setPartitionData(Collections.singletonList( .setName(tp.topic).setPartitionData(Collections.singletonList(
@ -2713,7 +2713,7 @@ class KafkaApisTest extends Logging {
val newLeaderId = 2 val newLeaderId = 2
val newLeaderEpoch = 5 val newLeaderEpoch = 5
val produceRequest = ProduceRequest.forCurrentMagic(new ProduceRequestData() val produceRequest = ProduceRequest.builder(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection( .setTopicData(new ProduceRequestData.TopicProduceDataCollection(
Collections.singletonList(new ProduceRequestData.TopicProduceData() Collections.singletonList(new ProduceRequestData.TopicProduceData()
.setName(tp.topic).setPartitionData(Collections.singletonList( .setName(tp.topic).setPartitionData(Collections.singletonList(
@ -2778,7 +2778,7 @@ class KafkaApisTest extends Logging {
val tp = new TopicPartition(topic, 0) val tp = new TopicPartition(topic, 0)
val produceRequest = ProduceRequest.forCurrentMagic(new ProduceRequestData() val produceRequest = ProduceRequest.builder(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection( .setTopicData(new ProduceRequestData.TopicProduceDataCollection(
Collections.singletonList(new ProduceRequestData.TopicProduceData() Collections.singletonList(new ProduceRequestData.TopicProduceData()
.setName(tp.topic).setPartitionData(Collections.singletonList( .setName(tp.topic).setPartitionData(Collections.singletonList(
@ -2842,7 +2842,7 @@ class KafkaApisTest extends Logging {
val tp = new TopicPartition(topic, 0) val tp = new TopicPartition(topic, 0)
val produceRequest = ProduceRequest.forCurrentMagic(new ProduceRequestData() val produceRequest = ProduceRequest.builder(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection( .setTopicData(new ProduceRequestData.TopicProduceDataCollection(
Collections.singletonList(new ProduceRequestData.TopicProduceData() Collections.singletonList(new ProduceRequestData.TopicProduceData()
.setName(tp.topic).setPartitionData(Collections.singletonList( .setName(tp.topic).setPartitionData(Collections.singletonList(
@ -2899,13 +2899,13 @@ class KafkaApisTest extends Logging {
addTopicToMetadataCache(topic, numPartitions = 2) addTopicToMetadataCache(topic, numPartitions = 2)
for (version <- 3 to ApiKeys.PRODUCE.latestVersion) { for (version <- ApiKeys.PRODUCE.oldestVersion to ApiKeys.PRODUCE.latestVersion) {
reset(replicaManager, clientQuotaManager, clientRequestQuotaManager, requestChannel, txnCoordinator) reset(replicaManager, clientQuotaManager, clientRequestQuotaManager, requestChannel, txnCoordinator)
val tp = new TopicPartition("topic", 0) val tp = new TopicPartition("topic", 0)
val produceRequest = ProduceRequest.forCurrentMagic(new ProduceRequestData() val produceRequest = ProduceRequest.builder(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection( .setTopicData(new ProduceRequestData.TopicProduceDataCollection(
Collections.singletonList(new ProduceRequestData.TopicProduceData() Collections.singletonList(new ProduceRequestData.TopicProduceData()
.setName(tp.topic).setPartitionData(Collections.singletonList( .setName(tp.topic).setPartitionData(Collections.singletonList(
@ -4376,11 +4376,6 @@ class KafkaApisTest extends Logging {
testConsumerListOffsetWithUnsupportedVersion(ListOffsetsRequest.LATEST_TIERED_TIMESTAMP, 8) testConsumerListOffsetWithUnsupportedVersion(ListOffsetsRequest.LATEST_TIERED_TIMESTAMP, 8)
} }
@Test
def testListOffsetNegativeTimestampWithZeroVersion(): Unit = {
testConsumerListOffsetWithUnsupportedVersion(-3, 0)
}
@Test @Test
def testListOffsetNegativeTimestampWithOneOrAboveVersion(): Unit = { def testListOffsetNegativeTimestampWithOneOrAboveVersion(): Unit = {
testConsumerListOffsetWithUnsupportedVersion(-6, 1) testConsumerListOffsetWithUnsupportedVersion(-6, 1)
@ -4758,9 +4753,9 @@ class KafkaApisTest extends Logging {
assertEquals(Errors.NOT_LEADER_OR_FOLLOWER.code, partitionData.errorCode) assertEquals(Errors.NOT_LEADER_OR_FOLLOWER.code, partitionData.errorCode)
assertEquals(newLeaderId, partitionData.currentLeader.leaderId()) assertEquals(newLeaderId, partitionData.currentLeader.leaderId())
assertEquals(newLeaderEpoch, partitionData.currentLeader.leaderEpoch()) assertEquals(newLeaderEpoch, partitionData.currentLeader.leaderEpoch())
val node = response.data.nodeEndpoints.asScala.head val node = response.data.nodeEndpoints.asScala
assertEquals(2, node.nodeId) assertEquals(Seq(2), node.map(_.nodeId))
assertEquals("broker2", node.host) assertEquals(Seq("broker2"), node.map(_.host))
} }
@Test @Test

View File

@ -1205,8 +1205,6 @@ class KafkaConfigTest {
assertDynamic(kafkaConfigProp, 10007, () => config.logIndexIntervalBytes) assertDynamic(kafkaConfigProp, 10007, () => config.logIndexIntervalBytes)
case TopicConfig.MAX_MESSAGE_BYTES_CONFIG => case TopicConfig.MAX_MESSAGE_BYTES_CONFIG =>
assertDynamic(kafkaConfigProp, 10008, () => config.messageMaxBytes) assertDynamic(kafkaConfigProp, 10008, () => config.messageMaxBytes)
case TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_CONFIG =>
assertDynamic(kafkaConfigProp, false, () => config.logMessageDownConversionEnable)
case TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG => case TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG =>
assertDynamic(kafkaConfigProp, 10015L, () => config.logMessageTimestampBeforeMaxMs) assertDynamic(kafkaConfigProp, 10015L, () => config.logMessageTimestampBeforeMaxMs)
case TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG => case TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG =>

View File

@ -165,14 +165,7 @@ class ListOffsetsRequestTest extends BaseRequestTest {
private[this] def fetchOffsetAndEpochWithError(serverId: Int, timestamp: Long, version: Short): (Long, Int, Short) = { private[this] def fetchOffsetAndEpochWithError(serverId: Int, timestamp: Long, version: Short): (Long, Int, Short) = {
val partitionData = sendRequest(serverId, timestamp, version) val partitionData = sendRequest(serverId, timestamp, version)
(partitionData.offset, partitionData.leaderEpoch, partitionData.errorCode())
if (version == 0) {
if (partitionData.oldStyleOffsets().isEmpty)
(-1, partitionData.leaderEpoch, partitionData.errorCode())
else
(partitionData.oldStyleOffsets().asScala.head, partitionData.leaderEpoch, partitionData.errorCode())
} else
(partitionData.offset, partitionData.leaderEpoch, partitionData.errorCode())
} }
@ParameterizedTest @ParameterizedTest

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