mirror of https://github.com/apache/kafka.git
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:
parent
a001a96218
commit
a0f9e319c7
|
@ -16,13 +16,8 @@
|
|||
*/
|
||||
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.Map;
|
||||
import java.util.Optional;
|
||||
|
||||
/**
|
||||
* 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 {
|
||||
|
||||
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.
|
||||
private long maxFinalizedFeaturesEpoch = -1;
|
||||
|
@ -50,7 +44,6 @@ public class ApiVersions {
|
|||
|
||||
public synchronized void update(String nodeId, NodeApiVersions nodeApiVersions) {
|
||||
this.nodeApiVersions.put(nodeId, nodeApiVersions);
|
||||
this.maxUsableProduceMagic = computeMaxUsableProduceMagic();
|
||||
if (maxFinalizedFeaturesEpoch < nodeApiVersions.finalizedFeaturesEpoch()) {
|
||||
this.maxFinalizedFeaturesEpoch = nodeApiVersions.finalizedFeaturesEpoch();
|
||||
this.finalizedFeatures = nodeApiVersions.finalizedFeatures();
|
||||
|
@ -59,7 +52,6 @@ public class ApiVersions {
|
|||
|
||||
public synchronized void remove(String nodeId) {
|
||||
this.nodeApiVersions.remove(nodeId);
|
||||
this.maxUsableProduceMagic = computeMaxUsableProduceMagic();
|
||||
}
|
||||
|
||||
public synchronized NodeApiVersions get(String nodeId) {
|
||||
|
@ -74,19 +66,4 @@ public class ApiVersions {
|
|||
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;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -111,23 +111,6 @@ class OffsetFetcherUtils {
|
|||
Errors error = Errors.forCode(partition.errorCode());
|
||||
switch (error) {
|
||||
case NONE:
|
||||
if (!partition.oldStyleOffsets().isEmpty()) {
|
||||
// Handle v0 response with offsets
|
||||
long offset;
|
||||
if (partition.oldStyleOffsets().size() > 1) {
|
||||
throw new IllegalStateException("Unexpected partitionData response of length " +
|
||||
partition.oldStyleOffsets().size());
|
||||
} else {
|
||||
offset = partition.oldStyleOffsets().get(0);
|
||||
}
|
||||
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) {
|
||||
|
@ -138,7 +121,6 @@ class OffsetFetcherUtils {
|
|||
leaderEpoch);
|
||||
fetchedOffsets.put(topicPartition, offsetData);
|
||||
}
|
||||
}
|
||||
break;
|
||||
case UNSUPPORTED_FOR_MESSAGE_FORMAT:
|
||||
// The message format on the broker side is before 0.10.0, which means it does not
|
||||
|
|
|
@ -1004,7 +1004,7 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
|
|||
setReadOnly(record.headers());
|
||||
Header[] headers = record.headers().toArray();
|
||||
|
||||
int serializedSize = AbstractRecords.estimateSizeInBytesUpperBound(apiVersions.maxUsableProduceMagic(),
|
||||
int serializedSize = AbstractRecords.estimateSizeInBytesUpperBound(RecordBatch.CURRENT_MAGIC_VALUE,
|
||||
compression.type(), serializedKey, serializedValue, headers);
|
||||
ensureValidRecordSize(serializedSize);
|
||||
long timestamp = record.timestamp() == null ? nowMs : record.timestamp();
|
||||
|
|
|
@ -27,7 +27,6 @@ import org.apache.kafka.common.Node;
|
|||
import org.apache.kafka.common.PartitionInfo;
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
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.metrics.Metrics;
|
||||
import org.apache.kafka.common.record.AbstractRecords;
|
||||
|
@ -344,8 +343,8 @@ public class RecordAccumulator {
|
|||
}
|
||||
|
||||
if (buffer == null) {
|
||||
byte maxUsableMagic = apiVersions.maxUsableProduceMagic();
|
||||
int size = Math.max(this.batchSize, AbstractRecords.estimateSizeInBytesUpperBound(maxUsableMagic, compression.type(), key, value, headers));
|
||||
int size = Math.max(this.batchSize, AbstractRecords.estimateSizeInBytesUpperBound(
|
||||
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);
|
||||
// This call may block if we exhausted buffer space.
|
||||
buffer = free.allocate(size, maxTimeToBlock);
|
||||
|
@ -408,7 +407,7 @@ public class RecordAccumulator {
|
|||
return appendResult;
|
||||
}
|
||||
|
||||
MemoryRecordsBuilder recordsBuilder = recordsBuilder(buffer, apiVersions.maxUsableProduceMagic());
|
||||
MemoryRecordsBuilder recordsBuilder = recordsBuilder(buffer);
|
||||
ProducerBatch batch = new ProducerBatch(new TopicPartition(topic, partition), recordsBuilder, nowMs);
|
||||
FutureRecordMetadata future = Objects.requireNonNull(batch.tryAppend(timestamp, key, value, headers,
|
||||
callbacks, nowMs));
|
||||
|
@ -419,12 +418,8 @@ public class RecordAccumulator {
|
|||
return new RecordAppendResult(future, dq.size() > 1 || batch.isFull(), true, false, batch.estimatedSizeInBytes());
|
||||
}
|
||||
|
||||
private MemoryRecordsBuilder recordsBuilder(ByteBuffer buffer, byte maxUsableMagic) {
|
||||
if (transactionManager != null && maxUsableMagic < RecordBatch.MAGIC_VALUE_V2) {
|
||||
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);
|
||||
private MemoryRecordsBuilder recordsBuilder(ByteBuffer buffer) {
|
||||
return MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compression, TimestampType.CREATE_TIME, 0L);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -871,27 +871,10 @@ public class Sender implements Runnable {
|
|||
return;
|
||||
|
||||
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();
|
||||
for (ProducerBatch batch : batches) {
|
||||
TopicPartition tp = batch.topicPartition;
|
||||
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());
|
||||
if (tpData == null) {
|
||||
tpData = new ProduceRequestData.TopicProduceData().setName(tp.topic());
|
||||
|
@ -904,18 +887,13 @@ public class Sender implements Runnable {
|
|||
}
|
||||
|
||||
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;
|
||||
if (transactionManager != null && transactionManager.isTransactional()) {
|
||||
transactionalId = transactionManager.transactionalId();
|
||||
if (!transactionManager.isTransactionV2Enabled()) {
|
||||
useTransactionV1Version = true;
|
||||
}
|
||||
useTransactionV1Version = !transactionManager.isTransactionV2Enabled();
|
||||
}
|
||||
|
||||
ProduceRequest.Builder requestBuilder = ProduceRequest.forMagic(minUsedMagic,
|
||||
ProduceRequest.Builder requestBuilder = ProduceRequest.builder(
|
||||
new ProduceRequestData()
|
||||
.setAcks(acks)
|
||||
.setTimeoutMs(timeout)
|
||||
|
|
|
@ -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 " +
|
||||
"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.";
|
||||
|
||||
/**
|
||||
* @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_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 " +
|
||||
"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 see {@link #MESSAGE_DOWNCONVERSION_ENABLE_CONFIG}.
|
||||
*/
|
||||
@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.";
|
||||
}
|
||||
|
|
|
@ -194,7 +194,7 @@ public enum ApiKeys {
|
|||
private static boolean shouldRetainsBufferReference(Schema[] requestSchemas) {
|
||||
boolean requestRetainsBufferReference = false;
|
||||
for (Schema requestVersionSchema : requestSchemas) {
|
||||
if (retainsBufferReference(requestVersionSchema)) {
|
||||
if (requestVersionSchema != null && retainsBufferReference(requestVersionSchema)) {
|
||||
requestRetainsBufferReference = true;
|
||||
break;
|
||||
}
|
||||
|
|
|
@ -224,29 +224,6 @@ public class DescribeConfigsResponse extends AbstractResponse {
|
|||
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
|
||||
public DescribeConfigsResponseData data() {
|
||||
return data;
|
||||
|
@ -272,7 +249,7 @@ public class DescribeConfigsResponse extends AbstractResponse {
|
|||
}
|
||||
|
||||
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
|
||||
|
|
|
@ -30,7 +30,6 @@ import org.apache.kafka.common.protocol.Errors;
|
|||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
|
@ -137,7 +136,6 @@ public class ListOffsetsRequest extends AbstractRequest {
|
|||
|
||||
@Override
|
||||
public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) {
|
||||
short versionId = version();
|
||||
short errorCode = Errors.forException(e).code();
|
||||
|
||||
List<ListOffsetsTopicResponse> responses = new ArrayList<>();
|
||||
|
@ -148,12 +146,8 @@ public class ListOffsetsRequest extends AbstractRequest {
|
|||
ListOffsetsPartitionResponse partitionResponse = new ListOffsetsPartitionResponse()
|
||||
.setErrorCode(errorCode)
|
||||
.setPartitionIndex(partition.partitionIndex());
|
||||
if (versionId == 0) {
|
||||
partitionResponse.setOldStyleOffsets(Collections.emptyList());
|
||||
} else {
|
||||
partitionResponse.setOffset(ListOffsetsResponse.UNKNOWN_OFFSET)
|
||||
.setTimestamp(ListOffsetsResponse.UNKNOWN_TIMESTAMP);
|
||||
}
|
||||
partitions.add(partitionResponse);
|
||||
}
|
||||
topicResponse.setPartitions(partitions);
|
||||
|
|
|
@ -42,27 +42,16 @@ import static org.apache.kafka.common.requests.ProduceResponse.INVALID_OFFSET;
|
|||
public class ProduceRequest extends AbstractRequest {
|
||||
public static final short LAST_STABLE_VERSION_BEFORE_TRANSACTION_V2 = 11;
|
||||
|
||||
public static Builder forMagic(byte magic, ProduceRequestData data, boolean useTransactionV1Version) {
|
||||
// Message format upgrades correspond with a bump in the produce request version. Older
|
||||
// message format versions are generally not supported by the produce request versions
|
||||
// following the bump.
|
||||
|
||||
final short minVersion;
|
||||
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 builder(ProduceRequestData data, boolean useTransactionV1Version) {
|
||||
// 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.
|
||||
short maxVersion = useTransactionV1Version ?
|
||||
LAST_STABLE_VERSION_BEFORE_TRANSACTION_V2 : ApiKeys.PRODUCE.latestVersion();
|
||||
return new Builder(ApiKeys.PRODUCE.oldestVersion(), maxVersion, data);
|
||||
}
|
||||
|
||||
public static Builder forCurrentMagic(ProduceRequestData data) {
|
||||
return forMagic(RecordBatch.CURRENT_MAGIC_VALUE, data, false);
|
||||
public static Builder builder(ProduceRequestData data) {
|
||||
return builder(data, false);
|
||||
}
|
||||
|
||||
public static class Builder extends AbstractRequest.Builder<ProduceRequest> {
|
||||
|
@ -226,7 +215,6 @@ public class ProduceRequest extends AbstractRequest {
|
|||
}
|
||||
|
||||
public static void validateRecords(short version, BaseRecords baseRecords) {
|
||||
if (version >= 3) {
|
||||
if (baseRecords instanceof Records) {
|
||||
Records records = (Records) baseRecords;
|
||||
Iterator<? extends RecordBatch> iterator = records.batches().iterator();
|
||||
|
@ -249,11 +237,6 @@ public class ProduceRequest extends AbstractRequest {
|
|||
}
|
||||
}
|
||||
|
||||
// Note that we do not do similar validation for older versions to ensure compatibility with
|
||||
// clients which send the wrong magic version in the wrong version of the produce request. The broker
|
||||
// did not do this validation before, so we maintain that behavior here.
|
||||
}
|
||||
|
||||
public static ProduceRequest parse(ByteBuffer buffer, short version) {
|
||||
return new ProduceRequest(new ProduceRequestData(new ByteBufferAccessor(buffer), version), version);
|
||||
}
|
||||
|
@ -261,22 +244,4 @@ public class ProduceRequest extends AbstractRequest {
|
|||
public static boolean isTransactionV2Requested(short version) {
|
||||
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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,10 +18,11 @@
|
|||
"type": "request",
|
||||
"listeners": ["zkBroker", "broker"],
|
||||
"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 2 enables flexible versions.
|
||||
"validVersions": "0-2",
|
||||
"deprecatedVersions": "0",
|
||||
"validVersions": "1-2",
|
||||
"flexibleVersions": "2+",
|
||||
"fields": [
|
||||
{ "name": "Dirs", "type": "[]AlterReplicaLogDir", "versions": "0+",
|
||||
|
|
|
@ -17,9 +17,10 @@
|
|||
"apiKey": 34,
|
||||
"type": "response",
|
||||
"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.
|
||||
// Version 2 enables flexible versions.
|
||||
"validVersions": "0-2",
|
||||
"validVersions": "1-2",
|
||||
"flexibleVersions": "2+",
|
||||
"fields": [
|
||||
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
|
||||
|
|
|
@ -18,11 +18,11 @@
|
|||
"type": "request",
|
||||
"listeners": ["zkBroker", "broker", "controller"],
|
||||
"name": "CreateAclsRequest",
|
||||
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
|
||||
// Version 1 adds resource pattern type.
|
||||
// Version 2 enables flexible versions.
|
||||
// Version 3 adds user resource type.
|
||||
"validVersions": "0-3",
|
||||
"deprecatedVersions": "0",
|
||||
"validVersions": "1-3",
|
||||
"flexibleVersions": "2+",
|
||||
"fields": [
|
||||
{ "name": "Creations", "type": "[]AclCreation", "versions": "0+",
|
||||
|
|
|
@ -17,10 +17,11 @@
|
|||
"apiKey": 30,
|
||||
"type": "response",
|
||||
"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.
|
||||
// Version 2 enables flexible versions.
|
||||
// Version 3 adds user resource type.
|
||||
"validVersions": "0-3",
|
||||
"validVersions": "1-3",
|
||||
"flexibleVersions": "2+",
|
||||
"fields": [
|
||||
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
|
||||
|
|
|
@ -18,13 +18,14 @@
|
|||
"type": "request",
|
||||
"listeners": ["zkBroker", "broker", "controller"],
|
||||
"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 2 is the first flexible version.
|
||||
//
|
||||
// Version 3 adds owner principal
|
||||
"validVersions": "0-3",
|
||||
"deprecatedVersions": "0",
|
||||
"validVersions": "1-3",
|
||||
"flexibleVersions": "2+",
|
||||
"fields": [
|
||||
{ "name": "OwnerPrincipalType", "type": "string", "versions": "3+", "nullableVersions": "3+",
|
||||
|
|
|
@ -17,12 +17,14 @@
|
|||
"apiKey": 38,
|
||||
"type": "response",
|
||||
"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.
|
||||
//
|
||||
// Version 2 is the first flexible version.
|
||||
//
|
||||
// Version 3 adds token requester details
|
||||
"validVersions": "0-3",
|
||||
"validVersions": "1-3",
|
||||
"flexibleVersions": "2+",
|
||||
"fields": [
|
||||
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
|
||||
|
|
|
@ -18,6 +18,8 @@
|
|||
"type": "request",
|
||||
"listeners": ["zkBroker", "broker", "controller"],
|
||||
"name": "CreateTopicsRequest",
|
||||
// Versions 0-1 were removed in Apache Kafka 4.0, Version 2 is the new baseline.
|
||||
//
|
||||
// Version 1 adds validateOnly.
|
||||
//
|
||||
// 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).
|
||||
//
|
||||
// Version 7 is the same as version 6.
|
||||
"validVersions": "0-7",
|
||||
"deprecatedVersions": "0-1",
|
||||
"validVersions": "2-7",
|
||||
"flexibleVersions": "5+",
|
||||
"fields": [
|
||||
{ "name": "Topics", "type": "[]CreatableTopic", "versions": "0+",
|
||||
|
|
|
@ -17,6 +17,8 @@
|
|||
"apiKey": 19,
|
||||
"type": "response",
|
||||
"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 2 adds the throttle time.
|
||||
|
@ -32,7 +34,7 @@
|
|||
// 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.
|
||||
"validVersions": "0-7",
|
||||
"validVersions": "2-7",
|
||||
"flexibleVersions": "5+",
|
||||
"fields": [
|
||||
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "2+", "ignorable": true,
|
||||
|
|
|
@ -18,11 +18,11 @@
|
|||
"type": "request",
|
||||
"listeners": ["zkBroker", "broker", "controller"],
|
||||
"name": "DeleteAclsRequest",
|
||||
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
|
||||
// Version 1 adds the pattern type.
|
||||
// Version 2 enables flexible versions.
|
||||
// Version 3 adds the user resource type.
|
||||
"validVersions": "0-3",
|
||||
"deprecatedVersions": "0",
|
||||
"validVersions": "1-3",
|
||||
"flexibleVersions": "2+",
|
||||
"fields": [
|
||||
{ "name": "Filters", "type": "[]DeleteAclsFilter", "versions": "0+",
|
||||
|
|
|
@ -17,11 +17,12 @@
|
|||
"apiKey": 31,
|
||||
"type": "response",
|
||||
"name": "DeleteAclsResponse",
|
||||
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
|
||||
// Version 1 adds the resource pattern type.
|
||||
// Starting in version 1, on quota violation, brokers send out responses before throttling.
|
||||
// Version 2 enables flexible versions.
|
||||
// Version 3 adds the user resource type.
|
||||
"validVersions": "0-3",
|
||||
"validVersions": "1-3",
|
||||
"flexibleVersions": "2+",
|
||||
"fields": [
|
||||
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
|
||||
|
|
|
@ -18,11 +18,12 @@
|
|||
"type": "request",
|
||||
"listeners": ["zkBroker", "broker"],
|
||||
"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 2 is the first flexible version.
|
||||
"validVersions": "0-2",
|
||||
"deprecatedVersions": "0",
|
||||
"validVersions": "1-2",
|
||||
"flexibleVersions": "2+",
|
||||
"fields": [
|
||||
{ "name": "GroupsNames", "type": "[]string", "versions": "0+", "entityType": "groupId",
|
||||
|
|
|
@ -17,10 +17,12 @@
|
|||
"apiKey": 42,
|
||||
"type": "response",
|
||||
"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.
|
||||
//
|
||||
// Version 2 is the first flexible version.
|
||||
"validVersions": "0-2",
|
||||
"validVersions": "1-2",
|
||||
"flexibleVersions": "2+",
|
||||
"fields": [
|
||||
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
|
||||
|
|
|
@ -18,6 +18,7 @@
|
|||
"type": "request",
|
||||
"listeners": ["zkBroker", "broker", "controller"],
|
||||
"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.
|
||||
//
|
||||
// Version 4 is the first flexible version.
|
||||
|
@ -26,8 +27,7 @@
|
|||
// 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.
|
||||
"validVersions": "0-6",
|
||||
"deprecatedVersions": "0",
|
||||
"validVersions": "1-6",
|
||||
"flexibleVersions": "4+",
|
||||
"fields": [
|
||||
{ "name": "Topics", "type": "[]DeleteTopicState", "versions": "6+", "about": "The name or topic ID of the topic.",
|
||||
|
|
|
@ -17,6 +17,8 @@
|
|||
"apiKey": 20,
|
||||
"type": "response",
|
||||
"name": "DeleteTopicsResponse",
|
||||
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
|
||||
//
|
||||
// Version 1 adds the throttle time.
|
||||
//
|
||||
// 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
|
||||
// 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.
|
||||
"validVersions": "0-6",
|
||||
"validVersions": "1-6",
|
||||
"flexibleVersions": "4+",
|
||||
"fields": [
|
||||
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true,
|
||||
|
|
|
@ -18,11 +18,11 @@
|
|||
"type": "request",
|
||||
"listeners": ["zkBroker", "broker", "controller"],
|
||||
"name": "DescribeAclsRequest",
|
||||
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
|
||||
// Version 1 adds resource pattern type.
|
||||
// Version 2 enables flexible versions.
|
||||
// Version 3 adds user resource type.
|
||||
"validVersions": "0-3",
|
||||
"deprecatedVersions": "0",
|
||||
"validVersions": "1-3",
|
||||
"flexibleVersions": "2+",
|
||||
"fields": [
|
||||
{ "name": "ResourceTypeFilter", "type": "int8", "versions": "0+",
|
||||
|
|
|
@ -17,11 +17,12 @@
|
|||
"apiKey": 29,
|
||||
"type": "response",
|
||||
"name": "DescribeAclsResponse",
|
||||
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
|
||||
// Version 1 adds PatternType.
|
||||
// Starting in version 1, on quota violation, brokers send out responses before throttling.
|
||||
// Version 2 enables flexible versions.
|
||||
// Version 3 adds user resource type.
|
||||
"validVersions": "0-3",
|
||||
"validVersions": "1-3",
|
||||
"flexibleVersions": "2+",
|
||||
"fields": [
|
||||
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
|
||||
|
|
|
@ -18,11 +18,11 @@
|
|||
"type": "request",
|
||||
"listeners": ["zkBroker", "broker", "controller"],
|
||||
"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 4 enables flexible versions.
|
||||
"validVersions": "0-4",
|
||||
"deprecatedVersions": "0",
|
||||
"validVersions": "1-4",
|
||||
"flexibleVersions": "4+",
|
||||
"fields": [
|
||||
{ "name": "Resources", "type": "[]DescribeConfigsResource", "versions": "0+",
|
||||
|
|
|
@ -17,10 +17,11 @@
|
|||
"apiKey": 32,
|
||||
"type": "response",
|
||||
"name": "DescribeConfigsResponse",
|
||||
// Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline.
|
||||
// Version 1 adds ConfigSource and the synonyms.
|
||||
// Starting in version 2, on quota violation, brokers send out responses before throttling.
|
||||
// Version 4 enables flexible versions.
|
||||
"validVersions": "0-4",
|
||||
"validVersions": "1-4",
|
||||
"flexibleVersions": "4+",
|
||||
"fields": [
|
||||
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
|
||||
|
@ -43,11 +44,6 @@
|
|||
"about": "The configuration value." },
|
||||
{ "name": "ReadOnly", "type": "bool", "versions": "0+",
|
||||
"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,
|
||||
"about": "The configuration source." },
|
||||
{ "name": "IsSensitive", "type": "bool", "versions": "0+",
|
||||
|
|
|
@ -18,11 +18,11 @@
|
|||
"type": "request",
|
||||
"listeners": ["zkBroker", "broker", "controller"],
|
||||
"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 2 adds flexible version support
|
||||
// Version 3 adds token requester into the response
|
||||
"validVersions": "0-3",
|
||||
"deprecatedVersions": "0",
|
||||
"validVersions": "1-3",
|
||||
"flexibleVersions": "2+",
|
||||
"fields": [
|
||||
{ "name": "Owners", "type": "[]DescribeDelegationTokenOwner", "versions": "0+", "nullableVersions": "0+",
|
||||
|
|
|
@ -17,10 +17,11 @@
|
|||
"apiKey": 41,
|
||||
"type": "response",
|
||||
"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.
|
||||
// Version 2 adds flexible version support
|
||||
// Version 3 adds token requester details
|
||||
"validVersions": "0-3",
|
||||
"validVersions": "1-3",
|
||||
"flexibleVersions": "2+",
|
||||
"fields": [
|
||||
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
|
||||
|
|
|
@ -18,12 +18,12 @@
|
|||
"type": "request",
|
||||
"listeners": ["zkBroker", "broker"],
|
||||
"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 2 is the first flexible version.
|
||||
// Version 3 is the same as version 2 (new field in response).
|
||||
// Version 4 is the same as version 2 (new fields in response).
|
||||
"validVersions": "0-4",
|
||||
"deprecatedVersions": "0",
|
||||
"validVersions": "1-4",
|
||||
"flexibleVersions": "2+",
|
||||
"fields": [
|
||||
{ "name": "Topics", "type": "[]DescribableLogDirTopic", "versions": "0+", "nullableVersions": "0+",
|
||||
|
|
|
@ -17,11 +17,12 @@
|
|||
"apiKey": 35,
|
||||
"type": "response",
|
||||
"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.
|
||||
"validVersions": "0-4",
|
||||
// Version 2 is the first flexible version.
|
||||
// Version 3 adds the top-level ErrorCode field
|
||||
// Version 4 adds the TotalBytes and UsableBytes fields
|
||||
"validVersions": "1-4",
|
||||
"flexibleVersions": "2+",
|
||||
"fields": [
|
||||
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
|
||||
|
|
|
@ -18,10 +18,10 @@
|
|||
"type": "request",
|
||||
"listeners": ["zkBroker", "broker", "controller"],
|
||||
"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 2 adds flexible version support
|
||||
"validVersions": "0-2",
|
||||
"deprecatedVersions": "0",
|
||||
"validVersions": "1-2",
|
||||
"flexibleVersions": "2+",
|
||||
"fields": [
|
||||
{ "name": "Hmac", "type": "bytes", "versions": "0+",
|
||||
|
|
|
@ -17,9 +17,10 @@
|
|||
"apiKey": 40,
|
||||
"type": "response",
|
||||
"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.
|
||||
// Version 2 adds flexible version support
|
||||
"validVersions": "0-2",
|
||||
"validVersions": "1-2",
|
||||
"flexibleVersions": "2+",
|
||||
"fields": [
|
||||
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
|
||||
|
|
|
@ -18,12 +18,11 @@
|
|||
"type": "request",
|
||||
"listeners": ["zkBroker", "broker", "controller"],
|
||||
"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.
|
||||
//
|
||||
// Starting in Version 2, the requester must be able to handle Kafka Log
|
||||
// Message format version 1.
|
||||
//
|
||||
// Version 3 adds MaxBytes. Starting in version 3, the partition ordering in
|
||||
// the request is now relevant. Partitions will be processed in the order
|
||||
// they appear in the request.
|
||||
|
@ -57,8 +56,7 @@
|
|||
// Version 16 is the same as version 15 (KIP-951).
|
||||
//
|
||||
// Version 17 adds directory id support from KIP-853
|
||||
"validVersions": "0-17",
|
||||
"deprecatedVersions": "0-3",
|
||||
"validVersions": "4-17",
|
||||
"flexibleVersions": "12+",
|
||||
"fields": [
|
||||
{ "name": "ClusterId", "type": "string", "versions": "12+", "nullableVersions": "12+", "default": "null",
|
||||
|
|
|
@ -17,10 +17,9 @@
|
|||
"apiKey": 1,
|
||||
"type": "response",
|
||||
"name": "FetchResponse",
|
||||
// Versions 0-3 were removed in Apache Kafka 4.0, Version 4 is the new baseline.
|
||||
//
|
||||
// Version 1 adds throttle time.
|
||||
//
|
||||
// Version 2 and 3 are the same as version 1.
|
||||
// Version 1 adds throttle time. Version 2 and 3 are the same as version 1.
|
||||
//
|
||||
// Version 4 adds features for transactional consumption.
|
||||
//
|
||||
|
@ -49,7 +48,7 @@
|
|||
// Version 16 adds the 'NodeEndpoints' field (KIP-951).
|
||||
//
|
||||
// Version 17 no changes to the response (KIP-853).
|
||||
"validVersions": "0-17",
|
||||
"validVersions": "4-17",
|
||||
"flexibleVersions": "12+",
|
||||
"fields": [
|
||||
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true,
|
||||
|
|
|
@ -18,9 +18,9 @@
|
|||
"type": "request",
|
||||
"listeners": ["zkBroker", "broker"],
|
||||
"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
|
||||
//
|
||||
|
@ -34,8 +34,7 @@
|
|||
// Version 8 adds the Reason field (KIP-800).
|
||||
//
|
||||
// Version 9 is the same as version 8.
|
||||
"validVersions": "0-9",
|
||||
"deprecatedVersions": "0-1",
|
||||
"validVersions": "2-9",
|
||||
"flexibleVersions": "6+",
|
||||
"fields": [
|
||||
{ "name": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId",
|
||||
|
|
|
@ -17,6 +17,8 @@
|
|||
"apiKey": 11,
|
||||
"type": "response",
|
||||
"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 2 adds throttle time.
|
||||
|
@ -35,7 +37,7 @@
|
|||
// Version 8 is the same as version 7.
|
||||
//
|
||||
// Version 9 adds the SkipAssignment field.
|
||||
"validVersions": "0-9",
|
||||
"validVersions": "2-9",
|
||||
"flexibleVersions": "6+",
|
||||
"fields": [
|
||||
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "2+", "ignorable": true,
|
||||
|
|
|
@ -18,6 +18,8 @@
|
|||
"type": "request",
|
||||
"listeners": ["zkBroker", "broker"],
|
||||
"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
|
||||
// offset can be returned.
|
||||
//
|
||||
|
@ -38,8 +40,7 @@
|
|||
// Version 9 enables listing offsets by last tiered offset (KIP-1005).
|
||||
//
|
||||
// Version 10 enables async remote list offsets support (KIP-1075)
|
||||
"validVersions": "0-10",
|
||||
"deprecatedVersions": "0",
|
||||
"validVersions": "1-10",
|
||||
"flexibleVersions": "6+",
|
||||
"latestVersionUnstable": false,
|
||||
"fields": [
|
||||
|
@ -58,9 +59,7 @@
|
|||
{ "name": "CurrentLeaderEpoch", "type": "int32", "versions": "4+", "default": "-1", "ignorable": true,
|
||||
"about": "The current leader epoch." },
|
||||
{ "name": "Timestamp", "type": "int64", "versions": "0+",
|
||||
"about": "The current timestamp." },
|
||||
{ "name": "MaxNumOffsets", "type": "int32", "versions": "0", "default": "1",
|
||||
"about": "The maximum number of offsets to report." }
|
||||
"about": "The current timestamp." }
|
||||
]}
|
||||
]},
|
||||
{ "name": "TimeoutMs", "type": "int32", "versions": "10+", "ignorable": true,
|
||||
|
|
|
@ -17,6 +17,8 @@
|
|||
"apiKey": 2,
|
||||
"type": "response",
|
||||
"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 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 10 enables async remote list offsets support (KIP-1075)
|
||||
"validVersions": "0-10",
|
||||
"validVersions": "1-10",
|
||||
"flexibleVersions": "6+",
|
||||
"fields": [
|
||||
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "2+", "ignorable": true,
|
||||
|
@ -53,8 +55,6 @@
|
|||
"about": "The partition index." },
|
||||
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
|
||||
"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,
|
||||
"about": "The timestamp associated with the returned offset." },
|
||||
{ "name": "Offset", "type": "int64", "versions": "1+", "default": "-1", "ignorable": false,
|
||||
|
|
|
@ -18,14 +18,14 @@
|
|||
"type": "request",
|
||||
"listeners": ["zkBroker", "broker"],
|
||||
"name": "MetadataRequest",
|
||||
"validVersions": "0-13",
|
||||
"deprecatedVersions": "0-3",
|
||||
"validVersions": "4-13",
|
||||
"flexibleVersions": "9+",
|
||||
"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
|
||||
// higher, an empty array indicates "request metadata for no topics," and a null array is used to
|
||||
// indicate "request metadata for all topics."
|
||||
//
|
||||
// Version 2 and 3 are the same as version 1.
|
||||
//
|
||||
// Version 4 adds AllowAutoTopicCreation.
|
||||
|
|
|
@ -17,11 +17,11 @@
|
|||
"apiKey": 3,
|
||||
"type": "response",
|
||||
"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
|
||||
// whether or not the topic is internal.
|
||||
//
|
||||
// Version 2 adds the cluster ID field.
|
||||
//
|
||||
// Version 3 adds the throttle time.
|
||||
//
|
||||
// Version 4 is the same as version 3.
|
||||
|
@ -43,7 +43,7 @@
|
|||
// by the DescribeCluster API (KIP-700).
|
||||
// Version 12 supports topicId.
|
||||
// Version 13 supports top-level error code in the response.
|
||||
"validVersions": "0-13",
|
||||
"validVersions": "4-13",
|
||||
"flexibleVersions": "9+",
|
||||
"fields": [
|
||||
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "3+", "ignorable": true,
|
||||
|
|
|
@ -18,6 +18,8 @@
|
|||
"type": "request",
|
||||
"listeners": ["zkBroker", "broker"],
|
||||
"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 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
|
||||
// request is the same as version 8.
|
||||
"validVersions": "0-9",
|
||||
"deprecatedVersions": "0-1",
|
||||
"validVersions": "2-9",
|
||||
"flexibleVersions": "8+",
|
||||
"fields": [
|
||||
{ "name": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId",
|
||||
|
@ -61,9 +62,6 @@
|
|||
"about": "The message offset to be committed." },
|
||||
{ "name": "CommittedLeaderEpoch", "type": "int32", "versions": "6+", "default": "-1", "ignorable": true,
|
||||
"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+",
|
||||
"about": "Any associated metadata the client wants to keep." }
|
||||
]}
|
||||
|
|
|
@ -17,6 +17,8 @@
|
|||
"apiKey": 8,
|
||||
"type": "response",
|
||||
"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.
|
||||
//
|
||||
// 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
|
||||
// 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.
|
||||
"validVersions": "0-9",
|
||||
"validVersions": "2-9",
|
||||
"flexibleVersions": "8+",
|
||||
// Supported errors:
|
||||
// - GROUP_AUTHORIZATION_FAILED (version 0+)
|
||||
|
|
|
@ -18,6 +18,8 @@
|
|||
"type": "request",
|
||||
"listeners": ["zkBroker", "broker"],
|
||||
"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.
|
||||
//
|
||||
// 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
|
||||
// the MemberId and MemberEpoch fields. Those are filled in and validated when the new consumer protocol is used.
|
||||
"validVersions": "0-9",
|
||||
"deprecatedVersions": "0",
|
||||
"validVersions": "1-9",
|
||||
"flexibleVersions": "6+",
|
||||
"fields": [
|
||||
{ "name": "GroupId", "type": "string", "versions": "0-7", "entityType": "groupId",
|
||||
|
|
|
@ -17,6 +17,8 @@
|
|||
"apiKey": 9,
|
||||
"type": "response",
|
||||
"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 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
|
||||
// the same as version 8 but can return STALE_MEMBER_EPOCH and UNKNOWN_MEMBER_ID errors when the new consumer group
|
||||
// protocol is used.
|
||||
"validVersions": "0-9",
|
||||
"validVersions": "1-9",
|
||||
"flexibleVersions": "6+",
|
||||
// Supported errors:
|
||||
// - GROUP_AUTHORIZATION_FAILED (version 0+)
|
||||
|
|
|
@ -18,6 +18,8 @@
|
|||
"type": "request",
|
||||
"listeners": ["zkBroker", "broker"],
|
||||
"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 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.
|
||||
//
|
||||
// Version 4 enables flexible versions.
|
||||
"validVersions": "0-4",
|
||||
"deprecatedVersions": "0-1",
|
||||
"validVersions": "2-4",
|
||||
"flexibleVersions": "4+",
|
||||
"fields": [
|
||||
{ "name": "ReplicaId", "type": "int32", "versions": "3+", "default": -2, "ignorable": true, "entityType": "brokerId",
|
||||
|
|
|
@ -17,6 +17,8 @@
|
|||
"apiKey": 23,
|
||||
"type": "response",
|
||||
"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 2 added the throttle time.
|
||||
|
@ -24,7 +26,7 @@
|
|||
// Version 3 is the same as version 2.
|
||||
//
|
||||
// Version 4 enables flexible versions.
|
||||
"validVersions": "0-4",
|
||||
"validVersions": "2-4",
|
||||
"flexibleVersions": "4+",
|
||||
"fields": [
|
||||
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "2+", "ignorable": true,
|
||||
|
|
|
@ -18,6 +18,8 @@
|
|||
"type": "request",
|
||||
"listeners": ["zkBroker", "broker"],
|
||||
"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 3 adds the transactional ID, which is used for authorization when attempting to write
|
||||
|
@ -42,8 +44,7 @@
|
|||
// 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
|
||||
// a transaction.
|
||||
"validVersions": "0-12",
|
||||
"deprecatedVersions": "0-2",
|
||||
"validVersions": "3-12",
|
||||
"flexibleVersions": "9+",
|
||||
"fields": [
|
||||
{ "name": "TransactionalId", "type": "string", "versions": "3+", "nullableVersions": "3+", "default": "null", "entityType": "transactionalId",
|
||||
|
|
|
@ -17,16 +17,16 @@
|
|||
"apiKey": 0,
|
||||
"type": "response",
|
||||
"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 3 is the same as version 2.
|
||||
//
|
||||
// Version 4 added KAFKA_STORAGE_ERROR as a possible error code.
|
||||
//
|
||||
// Version 5 added LogStartOffset to filter out spurious
|
||||
// OutOfOrderSequenceExceptions on the client.
|
||||
// Version 5 added LogStartOffset to filter out spurious OutOfOrderSequenceExceptions on the client.
|
||||
//
|
||||
// Version 8 added RecordErrors and ErrorMessage to include information about
|
||||
// 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 12 is the same as version 10 (KIP-890).
|
||||
"validVersions": "0-12",
|
||||
"validVersions": "3-12",
|
||||
"flexibleVersions": "9+",
|
||||
"fields": [
|
||||
{ "name": "Responses", "type": "[]TopicProduceResponse", "versions": "0+",
|
||||
|
|
|
@ -18,10 +18,10 @@
|
|||
"type": "request",
|
||||
"listeners": ["zkBroker", "broker", "controller"],
|
||||
"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 2 adds flexible version support
|
||||
"validVersions": "0-2",
|
||||
"deprecatedVersions": "0",
|
||||
"validVersions": "1-2",
|
||||
"flexibleVersions": "2+",
|
||||
"fields": [
|
||||
{ "name": "Hmac", "type": "bytes", "versions": "0+",
|
||||
|
|
|
@ -17,9 +17,10 @@
|
|||
"apiKey": 39,
|
||||
"type": "response",
|
||||
"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.
|
||||
// Version 2 adds flexible version support
|
||||
"validVersions": "0-2",
|
||||
"validVersions": "1-2",
|
||||
"flexibleVersions": "2+",
|
||||
"fields": [
|
||||
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
|
||||
|
|
|
@ -17,47 +17,15 @@
|
|||
package org.apache.kafka.clients;
|
||||
|
||||
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 java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
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
|
||||
public void testFinalizedFeaturesUpdate() {
|
||||
ApiVersions apiVersions = new ApiVersions();
|
||||
|
@ -96,4 +64,5 @@ public class ApiVersionsTest {
|
|||
assertEquals(1, info.finalizedFeaturesEpoch);
|
||||
assertEquals((short) 2, info.finalizedFeatures.get("transaction.version"));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -215,7 +215,7 @@ public class NetworkClientTest {
|
|||
client.poll(1, time.milliseconds());
|
||||
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())
|
||||
.setAcks((short) 1)
|
||||
.setTimeoutMs(1000));
|
||||
|
@ -632,7 +632,7 @@ public class NetworkClientTest {
|
|||
|
||||
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
|
||||
ProduceRequest.Builder builder = ProduceRequest.forCurrentMagic(new ProduceRequestData()
|
||||
ProduceRequest.Builder builder = ProduceRequest.builder(new ProduceRequestData()
|
||||
.setTopicData(new ProduceRequestData.TopicProduceDataCollection())
|
||||
.setAcks((short) 1)
|
||||
.setTimeoutMs(1000));
|
||||
|
@ -766,33 +766,12 @@ public class NetworkClientTest {
|
|||
.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() {
|
||||
return sendEmptyProduceRequest(client, node.idString());
|
||||
}
|
||||
|
||||
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())
|
||||
.setAcks((short) 1)
|
||||
.setTimeoutMs(1000));
|
||||
|
|
|
@ -102,16 +102,16 @@ public class NodeApiVersionsTest {
|
|||
|
||||
@Test
|
||||
public void testLatestUsableVersion() {
|
||||
NodeApiVersions apiVersions = NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 1, (short) 3);
|
||||
assertEquals(3, apiVersions.latestUsableVersion(ApiKeys.PRODUCE));
|
||||
assertEquals(1, apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 0, (short) 1));
|
||||
assertEquals(1, apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 1, (short) 1));
|
||||
assertEquals(2, apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 1, (short) 2));
|
||||
assertEquals(3, apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 1, (short) 3));
|
||||
assertEquals(2, apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 2, (short) 2));
|
||||
assertEquals(3, apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 2, (short) 3));
|
||||
assertEquals(3, apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 3, (short) 3));
|
||||
assertEquals(3, apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 3, (short) 4));
|
||||
NodeApiVersions apiVersions = NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 8, (short) 10);
|
||||
assertEquals(10, apiVersions.latestUsableVersion(ApiKeys.PRODUCE));
|
||||
assertEquals(8, apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 7, (short) 8));
|
||||
assertEquals(8, apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 8, (short) 8));
|
||||
assertEquals(9, apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 8, (short) 9));
|
||||
assertEquals(10, apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 8, (short) 10));
|
||||
assertEquals(9, apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 9, (short) 9));
|
||||
assertEquals(10, apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 9, (short) 10));
|
||||
assertEquals(10, apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 10, (short) 10));
|
||||
assertEquals(10, apiVersions.latestUsableVersion(ApiKeys.PRODUCE, (short) 10, (short) 11));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -677,7 +677,6 @@ public abstract class ConsumerCoordinatorTest {
|
|||
.setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH)
|
||||
.setCommittedMetadata("")
|
||||
.setCommittedOffset(13L)
|
||||
.setCommitTimestamp(0)
|
||||
))
|
||||
)
|
||||
);
|
||||
|
|
|
@ -24,7 +24,6 @@ import org.apache.kafka.clients.Metadata;
|
|||
import org.apache.kafka.clients.MetadataRecoveryStrategy;
|
||||
import org.apache.kafka.clients.MockClient;
|
||||
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.ConsumerRecord;
|
||||
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.errors.AuthenticationException;
|
||||
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.TopicAuthorizationException;
|
||||
import org.apache.kafka.common.header.Header;
|
||||
|
@ -1263,25 +1261,6 @@ public class FetchRequestManagerTest {
|
|||
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
|
||||
* partial request. For v3 and later FetchRequests, the implementation of KIP-74 changed the behavior
|
||||
|
|
|
@ -37,7 +37,6 @@ import org.apache.kafka.common.TopicIdPartition;
|
|||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.common.Uuid;
|
||||
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.TopicAuthorizationException;
|
||||
import org.apache.kafka.common.header.Header;
|
||||
|
@ -1261,25 +1260,6 @@ public class FetcherTest {
|
|||
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
|
||||
* partial request. For v3 and later FetchRequests, the implementation of KIP-74 changed the behavior
|
||||
|
|
|
@ -1113,43 +1113,6 @@ public class OffsetFetcherTest {
|
|||
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
|
||||
public void testOffsetValidationRequestGrouping() {
|
||||
buildFetcher();
|
||||
|
|
|
@ -28,7 +28,6 @@ import org.apache.kafka.common.Node;
|
|||
import org.apache.kafka.common.PartitionInfo;
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
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.protocol.ApiKeys;
|
||||
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.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import 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.");
|
||||
}
|
||||
|
||||
@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
|
||||
public void testRecordsDrainedWhenTransactionCompleting() throws Exception {
|
||||
int batchSize = 1025;
|
||||
|
|
|
@ -206,91 +206,6 @@ public class SenderTest {
|
|||
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
|
||||
*/
|
||||
|
@ -319,7 +234,7 @@ public class SenderTest {
|
|||
|
||||
for (int i = 1; i <= 3; i++) {
|
||||
int throttleTimeMs = 100 * i;
|
||||
ProduceRequest.Builder builder = ProduceRequest.forCurrentMagic(new ProduceRequestData()
|
||||
ProduceRequest.Builder builder = ProduceRequest.builder(new ProduceRequestData()
|
||||
.setTopicData(new ProduceRequestData.TopicProduceDataCollection())
|
||||
.setAcks((short) 1)
|
||||
.setTimeoutMs(1000));
|
||||
|
|
|
@ -29,6 +29,7 @@ import java.util.Set;
|
|||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.junit.jupiter.api.Assertions.fail;
|
||||
|
||||
|
@ -97,21 +98,27 @@ public class ApiMessageTypeTest {
|
|||
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
|
||||
public void testAllVersionsHaveSchemas() {
|
||||
for (ApiMessageType type : ApiMessageType.values()) {
|
||||
assertEquals(0, type.lowestSupportedVersion());
|
||||
assertTrue(type.lowestSupportedVersion() >= 0);
|
||||
|
||||
assertEquals(type.requestSchemas().length, type.responseSchemas().length,
|
||||
"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) {
|
||||
Schema schema = type.requestSchemas()[i];
|
||||
if (i >= type.lowestSupportedVersion())
|
||||
assertNotNull(schema);
|
||||
for (Schema schema : type.responseSchemas())
|
||||
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);
|
||||
}
|
||||
|
|
|
@ -82,7 +82,7 @@ public final class MessageTest {
|
|||
|
||||
private final String memberId = "memberId";
|
||||
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
|
||||
public void testAddOffsetsToTxnVersions() throws Exception {
|
||||
|
@ -199,21 +199,17 @@ public final class MessageTest {
|
|||
public void testListOffsetsResponseVersions() throws Exception {
|
||||
ListOffsetsPartitionResponse partition = new ListOffsetsPartitionResponse()
|
||||
.setErrorCode(Errors.NONE.code())
|
||||
.setPartitionIndex(0)
|
||||
.setOldStyleOffsets(Collections.singletonList(321L));
|
||||
.setPartitionIndex(0);
|
||||
List<ListOffsetsTopicResponse> topics = Collections.singletonList(new ListOffsetsTopicResponse()
|
||||
.setName("topic")
|
||||
.setPartitions(Collections.singletonList(partition)));
|
||||
Supplier<ListOffsetsResponseData> response = () -> new ListOffsetsResponseData()
|
||||
.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();
|
||||
if (version > 0) {
|
||||
responseData.topics().get(0).partitions().get(0)
|
||||
.setOldStyleOffsets(Collections.emptyList())
|
||||
.setOffset(456L)
|
||||
.setTimestamp(123L);
|
||||
}
|
||||
if (version > 1) {
|
||||
responseData.setThrottleTimeMs(1000);
|
||||
}
|
||||
|
@ -399,7 +395,8 @@ public final class MessageTest {
|
|||
.setPartitions(singletonList(partitionDataNoCurrentEpoch)));
|
||||
|
||||
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);
|
||||
|
||||
// Version 3 adds the optional replica Id field
|
||||
|
@ -475,22 +472,13 @@ public final class MessageTest {
|
|||
.setCommittedLeaderEpoch(10)
|
||||
.setCommittedMetadata(metadata)
|
||||
.setCommittedOffset(offset)
|
||||
.setCommitTimestamp(20)
|
||||
))))
|
||||
.setRetentionTimeMs(20);
|
||||
|
||||
for (short version : ApiKeys.OFFSET_COMMIT.allVersions()) {
|
||||
OffsetCommitRequestData requestData = request.get();
|
||||
if (version < 1) {
|
||||
requestData.setMemberId("");
|
||||
requestData.setGenerationIdOrMemberEpoch(-1);
|
||||
}
|
||||
|
||||
if (version != 1) {
|
||||
requestData.topics().get(0).partitions().get(0).setCommitTimestamp(-1);
|
||||
}
|
||||
|
||||
if (version < 2 || version > 4) {
|
||||
if (version > 4) {
|
||||
requestData.setRetentionTimeMs(-1);
|
||||
}
|
||||
|
||||
|
@ -502,9 +490,7 @@ public final class MessageTest {
|
|||
requestData.setGroupInstanceId(null);
|
||||
}
|
||||
|
||||
if (version == 1) {
|
||||
testEquivalentMessageRoundTrip(version, requestData);
|
||||
} else if (version >= 2 && version <= 4) {
|
||||
if (version >= 2 && version <= 4) {
|
||||
testAllMessageRoundTripsBetweenVersions(version, (short) 5, requestData, requestData);
|
||||
} else {
|
||||
testAllMessageRoundTripsFromVersion(version, requestData);
|
||||
|
@ -627,7 +613,7 @@ public final class MessageTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testOffsetFetchV0ToV7() throws Exception {
|
||||
public void testOffsetFetchV1ToV7() throws Exception {
|
||||
String groupId = "groupId";
|
||||
String topicName = "topic";
|
||||
|
||||
|
@ -655,15 +641,15 @@ public final class MessageTest {
|
|||
for (int version : listOfVersionsNonBatchOffsetFetch) {
|
||||
final short finalVersion = (short) version;
|
||||
if (version < 2) {
|
||||
assertThrows(NullPointerException.class, () -> testAllMessageRoundTripsOffsetFetchFromVersionV0ToV7(finalVersion, allPartitionData));
|
||||
assertThrows(NullPointerException.class, () -> testAllMessageRoundTripsOffsetFetchFromVersionToV7(finalVersion, allPartitionData));
|
||||
} else {
|
||||
testAllMessageRoundTripsOffsetFetchFromVersionV0ToV7((short) version, allPartitionData);
|
||||
testAllMessageRoundTripsOffsetFetchFromVersionToV7((short) version, allPartitionData);
|
||||
}
|
||||
|
||||
if (version < 7) {
|
||||
assertThrows(UnsupportedVersionException.class, () -> testAllMessageRoundTripsOffsetFetchFromVersionV0ToV7(finalVersion, requireStableData));
|
||||
assertThrows(UnsupportedVersionException.class, () -> testAllMessageRoundTripsOffsetFetchFromVersionToV7(finalVersion, requireStableData));
|
||||
} else {
|
||||
testAllMessageRoundTripsOffsetFetchFromVersionV0ToV7(finalVersion, requireStableData);
|
||||
testAllMessageRoundTripsOffsetFetchFromVersionToV7(finalVersion, requireStableData);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -695,16 +681,16 @@ public final class MessageTest {
|
|||
responseData.topics().get(0).partitions().get(0).setCommittedLeaderEpoch(-1);
|
||||
}
|
||||
|
||||
testAllMessageRoundTripsOffsetFetchFromVersionV0ToV7((short) version, responseData);
|
||||
testAllMessageRoundTripsOffsetFetchFromVersionToV7((short) version, responseData);
|
||||
}
|
||||
}
|
||||
|
||||
private void testAllMessageRoundTripsOffsetFetchV0ToV7(Message message) throws Exception {
|
||||
testDuplication(message);
|
||||
testAllMessageRoundTripsOffsetFetchFromVersionV0ToV7(message.lowestSupportedVersion(), message);
|
||||
testAllMessageRoundTripsOffsetFetchFromVersionToV7(message.lowestSupportedVersion(), message);
|
||||
}
|
||||
|
||||
private void testAllMessageRoundTripsOffsetFetchFromVersionV0ToV7(short fromVersion,
|
||||
private void testAllMessageRoundTripsOffsetFetchFromVersionToV7(short fromVersion,
|
||||
Message message) throws Exception {
|
||||
for (short version = fromVersion; version <= 7; version++) {
|
||||
testEquivalentMessageRoundTrip(version, message);
|
||||
|
@ -1131,15 +1117,16 @@ public final class MessageTest {
|
|||
|
||||
@Test
|
||||
public void testDefaultValues() {
|
||||
verifyWriteRaisesUve((short) 0, "validateOnly",
|
||||
new CreateTopicsRequestData().setValidateOnly(true));
|
||||
verifyWriteSucceeds((short) 0,
|
||||
new CreateTopicsRequestData().setValidateOnly(false));
|
||||
verifyWriteSucceeds((short) 0,
|
||||
verifyWriteSucceeds((short) 2,
|
||||
new OffsetCommitRequestData().setRetentionTimeMs(123));
|
||||
|
||||
verifyWriteRaisesUve((short) 5, "forgotten",
|
||||
new FetchRequestData().setForgottenTopicsData(singletonList(
|
||||
new FetchRequestData.ForgottenTopic().setTopic("foo"))));
|
||||
verifyWriteSucceeds((short) 5, new FetchRequestData());
|
||||
verifyWriteSucceeds((short) 7,
|
||||
new FetchRequestData().setForgottenTopicsData(singletonList(
|
||||
new FetchRequestData.ForgottenTopic().setTopic("foo"))));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -1167,8 +1154,6 @@ public final class MessageTest {
|
|||
for (short version : ApiKeys.CREATE_TOPICS.allVersions()) {
|
||||
verifyWriteRaisesNpe(version, createTopics);
|
||||
}
|
||||
MetadataRequestData metadata = new MetadataRequestData().setTopics(null);
|
||||
verifyWriteRaisesNpe((short) 0, metadata);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -88,8 +88,8 @@ public class ApiVersionsResponseTest {
|
|||
public void shouldHaveCommonlyAgreedApiVersionResponseWithControllerOnForwardableAPIs() {
|
||||
final ApiKeys forwardableAPIKey = ApiKeys.CREATE_ACLS;
|
||||
final ApiKeys nonForwardableAPIKey = ApiKeys.JOIN_GROUP;
|
||||
final short minVersion = 0;
|
||||
final short maxVersion = 1;
|
||||
final short minVersion = 2;
|
||||
final short maxVersion = 3;
|
||||
Map<ApiKeys, ApiVersion> activeControllerApiVersions = Utils.mkMap(
|
||||
Utils.mkEntry(forwardableAPIKey, new ApiVersion()
|
||||
.setApiKey(forwardableAPIKey.id)
|
||||
|
|
|
@ -60,17 +60,7 @@ public class CreateAclsRequestTest {
|
|||
|
||||
@Test
|
||||
public void shouldThrowOnIfUnknown() {
|
||||
assertThrows(IllegalArgumentException.class, () -> new CreateAclsRequest(data(UNKNOWN_ACL1), V0));
|
||||
}
|
||||
|
||||
@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);
|
||||
assertThrows(IllegalArgumentException.class, () -> new CreateAclsRequest(data(UNKNOWN_ACL1), V1));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -62,32 +62,6 @@ public class DeleteAclsRequestTest {
|
|||
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
|
||||
public void shouldRoundTripV1() {
|
||||
final DeleteAclsRequest original = new DeleteAclsRequest.Builder(
|
||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.kafka.common.requests;
|
|||
|
||||
import org.apache.kafka.common.acl.AclOperation;
|
||||
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.DeleteAclsFilterResult;
|
||||
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;
|
||||
|
||||
public class DeleteAclsResponseTest {
|
||||
private static final short V0 = 0;
|
||||
private static final short V1 = 1;
|
||||
|
||||
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(
|
||||
UNKNOWN_ACL));
|
||||
|
||||
@Test
|
||||
public void shouldThrowOnV0IfNotLiteral() {
|
||||
assertThrows(UnsupportedVersionException.class, () -> new DeleteAclsResponse(
|
||||
new DeleteAclsResponseData()
|
||||
.setThrottleTimeMs(10)
|
||||
.setFilterResults(singletonList(PREFIXED_RESPONSE)),
|
||||
V0));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldThrowOnIfUnknown() {
|
||||
assertThrows(IllegalArgumentException.class, () -> new DeleteAclsResponse(
|
||||
|
@ -102,19 +91,6 @@ public class DeleteAclsResponseTest {
|
|||
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
|
||||
public void shouldRoundTripV1() {
|
||||
final DeleteAclsResponse original = new DeleteAclsResponse(
|
||||
|
|
|
@ -21,7 +21,6 @@ import org.apache.kafka.common.acl.AccessControlEntryFilter;
|
|||
import org.apache.kafka.common.acl.AclBindingFilter;
|
||||
import org.apache.kafka.common.acl.AclOperation;
|
||||
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.ResourcePatternFilter;
|
||||
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;
|
||||
|
||||
public class DescribeAclsRequestTest {
|
||||
private static final short V0 = 0;
|
||||
private static final short V1 = 1;
|
||||
|
||||
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),
|
||||
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
|
||||
public void shouldThrowIfUnknown() {
|
||||
assertThrows(IllegalArgumentException.class, () -> new DescribeAclsRequest.Builder(UNKNOWN_FILTER).build(V0));
|
||||
}
|
||||
|
||||
@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);
|
||||
assertThrows(IllegalArgumentException.class, () -> new DescribeAclsRequest.Builder(UNKNOWN_FILTER).build(V1));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -21,7 +21,6 @@ import org.apache.kafka.common.acl.AccessControlEntry;
|
|||
import org.apache.kafka.common.acl.AclBinding;
|
||||
import org.apache.kafka.common.acl.AclOperation;
|
||||
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.AclDescription;
|
||||
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;
|
||||
|
||||
public class DescribeAclsResponseTest {
|
||||
private static final short V0 = 0;
|
||||
private static final short V1 = 1;
|
||||
|
||||
private static final AclDescription ALLOW_CREATE_ACL = buildAclDescription(
|
||||
|
@ -82,30 +80,10 @@ public class DescribeAclsResponseTest {
|
|||
PatternType.LITERAL,
|
||||
Collections.singletonList(DENY_READ_ACL));
|
||||
|
||||
@Test
|
||||
public void shouldThrowOnV0IfNotLiteral() {
|
||||
assertThrows(UnsupportedVersionException.class,
|
||||
() -> buildResponse(10, Errors.NONE, Collections.singletonList(PREFIXED_ACL1)).serialize(V0));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldThrowIfUnknown() {
|
||||
assertThrows(IllegalArgumentException.class,
|
||||
() -> buildResponse(10, Errors.NONE, Collections.singletonList(UNKNOWN_ACL)).serialize(V0));
|
||||
}
|
||||
|
||||
@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);
|
||||
() -> buildResponse(10, Errors.NONE, Collections.singletonList(UNKNOWN_ACL)).serialize(V1));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -19,15 +19,12 @@ package org.apache.kafka.common.requests;
|
|||
import org.apache.kafka.common.errors.InvalidConfigurationException;
|
||||
import org.apache.kafka.common.errors.UnsupportedVersionException;
|
||||
import org.apache.kafka.common.message.JoinGroupRequestData;
|
||||
import org.apache.kafka.common.protocol.MessageUtil;
|
||||
import org.apache.kafka.test.TestUtils;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
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.fail;
|
||||
|
||||
|
@ -68,20 +65,4 @@ public class JoinGroupRequestTest {
|
|||
.setProtocolType("consumer")
|
||||
).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());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -54,7 +54,7 @@ public class ListOffsetsRequestTest {
|
|||
ListOffsetsRequestData data = new ListOffsetsRequestData()
|
||||
.setTopics(topics)
|
||||
.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(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
|
||||
public void testToListOffsetsTopics() {
|
||||
ListOffsetsPartition lop0 = new ListOffsetsPartition()
|
||||
.setPartitionIndex(0)
|
||||
.setCurrentLeaderEpoch(1)
|
||||
.setMaxNumOffsets(2)
|
||||
.setTimestamp(123L);
|
||||
ListOffsetsPartition lop1 = new ListOffsetsPartition()
|
||||
.setPartitionIndex(1)
|
||||
.setCurrentLeaderEpoch(3)
|
||||
.setMaxNumOffsets(4)
|
||||
.setTimestamp(567L);
|
||||
Map<TopicPartition, ListOffsetsPartition> timestampsToSearch = new HashMap<>();
|
||||
timestampsToSearch.put(new TopicPartition("topic", 0), lop0);
|
||||
|
|
|
@ -30,23 +30,13 @@ import java.util.List;
|
|||
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
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.assertTrue;
|
||||
|
||||
public class MetadataRequestTest {
|
||||
|
||||
@Test
|
||||
public void testEmptyMeansAllTopicsV0() {
|
||||
MetadataRequestData data = new MetadataRequestData();
|
||||
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++) {
|
||||
public void testEmptyMeansEmptyForAllVersions() {
|
||||
for (int i = ApiKeys.METADATA.oldestVersion(); i < MetadataRequestData.SCHEMAS.length; i++) {
|
||||
MetadataRequestData data = new MetadataRequestData();
|
||||
data.setAllowAutoTopicCreation(true);
|
||||
MetadataRequest parsedRequest = new MetadataRequest(data, (short) i);
|
||||
|
|
|
@ -92,7 +92,7 @@ public class OffsetCommitRequestTest {
|
|||
|
||||
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);
|
||||
assertEquals(expectedOffsets, request.offsets());
|
||||
|
||||
|
|
|
@ -25,7 +25,6 @@ import org.apache.kafka.common.protocol.ApiKeys;
|
|||
import org.apache.kafka.common.record.MemoryRecords;
|
||||
import org.apache.kafka.common.record.MemoryRecordsBuilder;
|
||||
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.TimestampType;
|
||||
|
||||
|
@ -52,7 +51,7 @@ public class ProduceRequestTest {
|
|||
final MemoryRecords memoryRecords = MemoryRecords.withTransactionalRecords(0, Compression.NONE, 1L,
|
||||
(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(
|
||||
new ProduceRequestData.TopicProduceData()
|
||||
.setName("topic")
|
||||
|
@ -81,7 +80,7 @@ public class ProduceRequestTest {
|
|||
public void shouldBeFlaggedAsIdempotentWhenIdempotentRecords() {
|
||||
final MemoryRecords memoryRecords = MemoryRecords.withIdempotentRecords(1, Compression.NONE, 1L,
|
||||
(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(
|
||||
new ProduceRequestData.TopicProduceData()
|
||||
.setName("topic")
|
||||
|
@ -94,32 +93,13 @@ public class ProduceRequestTest {
|
|||
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
|
||||
public void testBuildWithCurrentMessageFormat() {
|
||||
ByteBuffer buffer = ByteBuffer.allocate(256);
|
||||
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE,
|
||||
Compression.NONE, TimestampType.CREATE_TIME, 0L);
|
||||
builder.append(10L, null, "a".getBytes());
|
||||
ProduceRequest.Builder requestBuilder = ProduceRequest.forMagic(RecordBatch.CURRENT_MAGIC_VALUE,
|
||||
ProduceRequest.Builder requestBuilder = ProduceRequest.builder(
|
||||
new ProduceRequestData()
|
||||
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
|
||||
new ProduceRequestData.TopicProduceData().setName("test").setPartitionData(Collections.singletonList(
|
||||
|
@ -146,7 +126,7 @@ public class ProduceRequestTest {
|
|||
|
||||
buffer.flip();
|
||||
|
||||
ProduceRequest.Builder requestBuilder = ProduceRequest.forCurrentMagic(new ProduceRequestData()
|
||||
ProduceRequest.Builder requestBuilder = ProduceRequest.builder(new ProduceRequestData()
|
||||
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
|
||||
new ProduceRequestData.TopicProduceData()
|
||||
.setName("test")
|
||||
|
@ -161,7 +141,7 @@ public class ProduceRequestTest {
|
|||
|
||||
@Test
|
||||
public void testV3AndAboveCannotHaveNoRecordBatches() {
|
||||
ProduceRequest.Builder requestBuilder = ProduceRequest.forCurrentMagic(new ProduceRequestData()
|
||||
ProduceRequest.Builder requestBuilder = ProduceRequest.builder(new ProduceRequestData()
|
||||
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
|
||||
new ProduceRequestData.TopicProduceData()
|
||||
.setName("test")
|
||||
|
@ -181,7 +161,7 @@ public class ProduceRequestTest {
|
|||
TimestampType.NO_TIMESTAMP_TYPE, 0L);
|
||||
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(
|
||||
new ProduceRequestData.TopicProduceData()
|
||||
.setName("test")
|
||||
|
@ -201,7 +181,7 @@ public class ProduceRequestTest {
|
|||
TimestampType.CREATE_TIME, 0L);
|
||||
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(
|
||||
new ProduceRequestData.TopicProduceData()
|
||||
.setName("test")
|
||||
|
@ -239,7 +219,7 @@ public class ProduceRequestTest {
|
|||
}
|
||||
|
||||
// Works fine with current version (>= 7)
|
||||
ProduceRequest.forCurrentMagic(produceData);
|
||||
ProduceRequest.builder(produceData);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -253,7 +233,7 @@ public class ProduceRequestTest {
|
|||
final MemoryRecords txnRecords = MemoryRecords.withTransactionalRecords(Compression.NONE, producerId,
|
||||
producerEpoch, sequence, new SimpleRecord("bar".getBytes()));
|
||||
|
||||
ProduceRequest.Builder builder = ProduceRequest.forMagic(RecordBatch.CURRENT_MAGIC_VALUE,
|
||||
ProduceRequest.Builder builder = ProduceRequest.builder(
|
||||
new ProduceRequestData()
|
||||
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Arrays.asList(
|
||||
new ProduceRequestData.TopicProduceData().setName("foo").setPartitionData(Collections.singletonList(
|
||||
|
@ -280,7 +260,7 @@ public class ProduceRequestTest {
|
|||
final MemoryRecords idempotentRecords = MemoryRecords.withIdempotentRecords(Compression.NONE, producerId,
|
||||
producerEpoch, sequence, new SimpleRecord("bar".getBytes()));
|
||||
|
||||
ProduceRequest.Builder builder = ProduceRequest.forMagic(RecordVersion.current().value,
|
||||
ProduceRequest.Builder builder = ProduceRequest.builder(
|
||||
new ProduceRequestData()
|
||||
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Arrays.asList(
|
||||
new ProduceRequestData.TopicProduceData().setName("foo").setPartitionData(Collections.singletonList(
|
||||
|
@ -304,7 +284,7 @@ public class ProduceRequestTest {
|
|||
}
|
||||
|
||||
private ProduceRequest createNonIdempotentNonTransactionalRecords() {
|
||||
return ProduceRequest.forCurrentMagic(new ProduceRequestData()
|
||||
return ProduceRequest.builder(new ProduceRequestData()
|
||||
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
|
||||
new ProduceRequestData.TopicProduceData()
|
||||
.setName("topic")
|
||||
|
|
|
@ -19,13 +19,11 @@ package org.apache.kafka.common.requests;
|
|||
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
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.record.RecordBatch;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
|
@ -39,36 +37,6 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
|
|||
|
||||
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")
|
||||
@Test
|
||||
public void produceResponseVersionTest() {
|
||||
|
|
|
@ -117,7 +117,7 @@ public class RequestContextTest {
|
|||
|
||||
@Test
|
||||
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);
|
||||
// corrupt the length of the topics array
|
||||
corruptBuffer.putInt(8, (Integer.MAX_VALUE - 1) / 2);
|
||||
|
@ -134,7 +134,7 @@ public class RequestContextTest {
|
|||
|
||||
@Test
|
||||
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);
|
||||
// corrupt the length of the partitions array
|
||||
corruptBuffer.putInt(17, Integer.MAX_VALUE);
|
||||
|
|
|
@ -151,7 +151,6 @@ import org.apache.kafka.common.message.EndTxnResponseData;
|
|||
import org.apache.kafka.common.message.EnvelopeResponseData;
|
||||
import org.apache.kafka.common.message.ExpireDelegationTokenRequestData;
|
||||
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.FetchSnapshotRequestData;
|
||||
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);
|
||||
checkOlderFetchVersions();
|
||||
// Metadata
|
||||
checkRequest(MetadataRequest.Builder.allTopics().build((short) 2));
|
||||
checkRequest(MetadataRequest.Builder.allTopics().build((short) 4));
|
||||
// OffsetFetch
|
||||
checkRequest(createOffsetFetchRequestWithMultipleGroups((short) 8, true));
|
||||
checkRequest(createOffsetFetchRequestWithMultipleGroups((short) 8, false));
|
||||
|
@ -413,12 +412,11 @@ public class RequestResponseTest {
|
|||
checkRequest(createTxnOffsetCommitRequestWithAutoDowngrade());
|
||||
checkErrorResponse(createTxnOffsetCommitRequestWithAutoDowngrade(), unknownServerException);
|
||||
// DescribeAcls
|
||||
checkErrorResponse(createDescribeAclsRequest((short) 0), new SecurityDisabledException("Security is not enabled."));
|
||||
checkErrorResponse(createCreateAclsRequest((short) 0), new SecurityDisabledException("Security is not enabled."));
|
||||
checkErrorResponse(createDescribeAclsRequest((short) 1), new SecurityDisabledException("Security is not enabled."));
|
||||
checkErrorResponse(createCreateAclsRequest((short) 1), new SecurityDisabledException("Security is not enabled."));
|
||||
// DeleteAcls
|
||||
checkErrorResponse(createDeleteAclsRequest((short) 0), new SecurityDisabledException("Security is not enabled."));
|
||||
checkErrorResponse(createDeleteAclsRequest((short) 1), new SecurityDisabledException("Security is not enabled."));
|
||||
// DescribeConfigs
|
||||
checkRequest(createDescribeConfigsRequestWithConfigEntries((short) 0));
|
||||
checkRequest(createDescribeConfigsRequestWithConfigEntries((short) 1));
|
||||
checkRequest(createDescribeConfigsRequestWithDocumentation((short) 1));
|
||||
checkRequest(createDescribeConfigsRequestWithDocumentation((short) 2));
|
||||
|
@ -478,7 +476,7 @@ public class RequestResponseTest {
|
|||
Compression.NONE, new SimpleRecord("woot".getBytes()));
|
||||
MemoryRecords records1 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2,
|
||||
Compression.NONE, new SimpleRecord("woot".getBytes()), new SimpleRecord("woot".getBytes()));
|
||||
ProduceRequest request = ProduceRequest.forMagic(RecordBatch.MAGIC_VALUE_V2,
|
||||
ProduceRequest request = ProduceRequest.builder(
|
||||
new ProduceRequestData()
|
||||
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(asList(
|
||||
new ProduceRequestData.TopicProduceData().setName(tp0.topic()).setPartitionData(
|
||||
|
@ -490,7 +488,7 @@ public class RequestResponseTest {
|
|||
.setTimeoutMs(5000)
|
||||
.setTransactionalId("transactionalId"),
|
||||
true)
|
||||
.build((short) 3);
|
||||
.build((short) 7);
|
||||
assertEquals(2, request.partitionSizes().size());
|
||||
assertEquals(records0.sizeInBytes(), (int) request.partitionSizes().get(tp0));
|
||||
assertEquals(records1.sizeInBytes(), (int) request.partitionSizes().get(tp1));
|
||||
|
@ -546,33 +544,13 @@ public class RequestResponseTest {
|
|||
|
||||
@Test
|
||||
public void fetchResponseVersionTest() {
|
||||
LinkedHashMap<TopicIdPartition, FetchResponseData.PartitionData> responseData = new LinkedHashMap<>();
|
||||
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));
|
||||
FetchResponseData.PartitionData partitionData = new FetchResponseData.PartitionData()
|
||||
.setPartitionIndex(0)
|
||||
.setHighWatermark(1000000)
|
||||
.setLogStartOffset(-1)
|
||||
.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<>();
|
||||
idResponseData.put(new TopicIdPartition(id, new TopicPartition("test", 0)),
|
||||
new FetchResponseData.PartitionData()
|
||||
|
@ -690,14 +668,6 @@ public class RequestResponseTest {
|
|||
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
|
||||
public void testFetchRequestIsolationLevel() {
|
||||
FetchRequest request = createFetchRequest((short) 4, IsolationLevel.READ_COMMITTED);
|
||||
|
@ -724,24 +694,6 @@ public class RequestResponseTest {
|
|||
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
|
||||
public void testSerializeWithHeader() {
|
||||
CreatableTopicCollection topicsToCreate = new CreatableTopicCollection(1);
|
||||
|
@ -789,14 +741,6 @@ public class RequestResponseTest {
|
|||
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
|
||||
public void testOffsetFetchRequestBuilderToStringV0ToV7() {
|
||||
List<Boolean> stableFlags = asList(true, false);
|
||||
|
@ -2352,19 +2296,7 @@ public class RequestResponseTest {
|
|||
}
|
||||
|
||||
private ListOffsetsRequest createListOffsetRequest(short version) {
|
||||
if (version == 0) {
|
||||
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) {
|
||||
if (version == 1) {
|
||||
ListOffsetsTopic topic = new ListOffsetsTopic()
|
||||
.setName("test")
|
||||
.setPartitions(singletonList(new ListOffsetsPartition()
|
||||
|
@ -2394,16 +2326,7 @@ public class RequestResponseTest {
|
|||
}
|
||||
|
||||
private ListOffsetsResponse createListOffsetResponse(short version) {
|
||||
if (version == 0) {
|
||||
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()) {
|
||||
if (version >= 1 && version <= LIST_OFFSETS.latestVersion()) {
|
||||
ListOffsetsPartitionResponse partition = new ListOffsetsPartitionResponse()
|
||||
.setPartitionIndex(0)
|
||||
.setErrorCode(Errors.NONE.code())
|
||||
|
@ -2564,22 +2487,9 @@ public class RequestResponseTest {
|
|||
}
|
||||
|
||||
private ProduceRequest createProduceRequest(short version) {
|
||||
if (version < 2) {
|
||||
MemoryRecords records = MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("blah".getBytes()));
|
||||
ProduceRequestData data = new ProduceRequestData()
|
||||
.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,
|
||||
MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, Compression.NONE,
|
||||
new SimpleRecord("woot".getBytes()));
|
||||
return ProduceRequest.builder(
|
||||
new ProduceRequestData()
|
||||
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(singletonList(
|
||||
new ProduceRequestData.TopicProduceData()
|
||||
|
|
|
@ -414,7 +414,6 @@ private[group] class GroupCoordinatorAdapter(
|
|||
partition.committedOffset,
|
||||
partition.committedLeaderEpoch,
|
||||
partition.committedMetadata,
|
||||
partition.commitTimestamp,
|
||||
expireTimeMs
|
||||
)
|
||||
}
|
||||
|
@ -473,7 +472,6 @@ private[group] class GroupCoordinatorAdapter(
|
|||
partition.committedOffset,
|
||||
partition.committedLeaderEpoch,
|
||||
partition.committedMetadata,
|
||||
OffsetCommitRequest.DEFAULT_TIMESTAMP, // means that currentTimeMs is used.
|
||||
None
|
||||
)
|
||||
}
|
||||
|
@ -501,7 +499,6 @@ private[group] class GroupCoordinatorAdapter(
|
|||
offset: Long,
|
||||
leaderEpoch: Int,
|
||||
metadata: String,
|
||||
commitTimestamp: Long,
|
||||
expireTimestamp: Option[Long]
|
||||
): OffsetAndMetadata = {
|
||||
new OffsetAndMetadata(
|
||||
|
@ -514,10 +511,7 @@ private[group] class GroupCoordinatorAdapter(
|
|||
case null => OffsetAndMetadata.NO_METADATA
|
||||
case metadata => metadata
|
||||
},
|
||||
commitTimestamp match {
|
||||
case OffsetCommitRequest.DEFAULT_TIMESTAMP => currentTimeMs
|
||||
case customTimestamp => customTimestamp
|
||||
},
|
||||
currentTimeMs,
|
||||
expireTimestamp match {
|
||||
case Some(timestamp) => OptionalLong.of(timestamp)
|
||||
case None => OptionalLong.empty()
|
||||
|
|
|
@ -19,7 +19,7 @@ package kafka.server
|
|||
|
||||
import com.typesafe.scalalogging.Logger
|
||||
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.protocol.Errors
|
||||
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.
|
||||
* 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 =
|
||||
FetchSession.partitionsToLogString(partitions, isTraceEnabled)
|
||||
|
@ -334,8 +334,8 @@ trait FetchContext extends Logging {
|
|||
/**
|
||||
* Return an empty throttled response due to quota violation.
|
||||
*/
|
||||
def getThrottledResponse(throttleTimeMs: Int): FetchResponse =
|
||||
FetchResponse.of(Errors.NONE, throttleTimeMs, INVALID_SESSION_ID, new FetchSession.RESP_MAP)
|
||||
def getThrottledResponse(throttleTimeMs: Int, nodeEndpoints: util.List[Node]): FetchResponse =
|
||||
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.
|
||||
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")
|
||||
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)
|
||||
}
|
||||
|
||||
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)}")
|
||||
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)
|
||||
}
|
||||
|
||||
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 = {
|
||||
val cachedPartitions = new FetchSession.CACHE_MAP(updates.size)
|
||||
updates.forEach { (part, respData) =>
|
||||
|
@ -444,7 +444,7 @@ class FullFetchContext(private val time: Time,
|
|||
updates.size, usesTopicIds, () => createNewSession)
|
||||
debug(s"Full fetch context with session id $responseSessionId returning " +
|
||||
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 {
|
||||
// Check to make sure that the session epoch didn't change in between
|
||||
// creating this fetch context and generating this response.
|
||||
|
@ -541,7 +541,7 @@ class IncrementalFetchContext(private val time: Time,
|
|||
if (session.epoch != expectedEpoch) {
|
||||
info(s"Incremental fetch session ${session.id} expected epoch $expectedEpoch, but " +
|
||||
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 {
|
||||
// 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)
|
||||
|
@ -550,12 +550,12 @@ class IncrementalFetchContext(private val time: Time,
|
|||
}
|
||||
debug(s"Incremental fetch context with session id ${session.id} returning " +
|
||||
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 {
|
||||
// Check to make sure that the session epoch didn't change in between
|
||||
// creating this fetch context and generating this response.
|
||||
|
@ -563,9 +563,9 @@ class IncrementalFetchContext(private val time: Time,
|
|||
if (session.epoch != expectedEpoch) {
|
||||
info(s"Incremental fetch session ${session.id} expected epoch $expectedEpoch, but " +
|
||||
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 {
|
||||
FetchResponse.of(Errors.NONE, throttleTimeMs, session.id, new FetchSession.RESP_MAP)
|
||||
FetchResponse.of(Errors.NONE, throttleTimeMs, session.id, new FetchSession.RESP_MAP, nodeEndpoints)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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._
|
||||
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.record._
|
||||
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.MetadataVersion.{IBP_0_11_0_IV0, IBP_2_3_IV0}
|
||||
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.{ErroneousAndValidPartitionData, SharePartitionKey}
|
||||
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.log.metrics.BrokerTopicStats
|
||||
|
||||
import java.lang.{Long => JLong}
|
||||
import java.nio.ByteBuffer
|
||||
import java.time.Duration
|
||||
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
|
||||
def processResponseCallback(responsePartitionData: Seq[(TopicIdPartition, FetchPartitionData)]): Unit = {
|
||||
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) }
|
||||
|
||||
def createResponse(throttleTimeMs: Int, unconvertedFetchResponse: FetchResponse): FetchResponse = {
|
||||
// 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)
|
||||
def recordBytesOutMetric(fetchResponse: FetchResponse): Unit = {
|
||||
// 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 =>
|
||||
// If the topic name was not known, we will have no bytes out.
|
||||
if (topicResponse.topic != null) {
|
||||
|
@ -974,23 +881,19 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
}
|
||||
}
|
||||
}
|
||||
response
|
||||
}
|
||||
|
||||
if (fetchRequest.isFromFollower) {
|
||||
// We've already evaluated against the quota and are good to go. Just need to record it now.
|
||||
val unconvertedFetchResponse = fetchContext.updateAndGenerateResponseData(partitions)
|
||||
val responseSize = KafkaApis.sizeOfThrottledPartitions(versionId, unconvertedFetchResponse, quotas.leader)
|
||||
val fetchResponse = fetchContext.updateAndGenerateResponseData(partitions, Seq.empty.asJava)
|
||||
val responseSize = KafkaApis.sizeOfThrottledPartitions(versionId, fetchResponse, quotas.leader)
|
||||
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, " +
|
||||
s"metadata=${unconvertedFetchResponse.sessionId}")
|
||||
requestHelper.sendResponseExemptThrottle(request, createResponse(0, unconvertedFetchResponse), onFetchComplete(request))
|
||||
s"metadata=${fetchResponse.sessionId}")
|
||||
recordBytesOutMetric(fetchResponse)
|
||||
requestHelper.sendResponseExemptThrottle(request, fetchResponse)
|
||||
} 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
|
||||
// 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.
|
||||
|
@ -1000,7 +903,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
val bandwidthThrottleTimeMs = quotas.fetch.maybeRecordAndGetThrottleTimeMs(request, responseSize, timeMs)
|
||||
|
||||
val maxThrottleTimeMs = math.max(bandwidthThrottleTimeMs, requestThrottleTimeMs)
|
||||
val unconvertedFetchResponse = if (maxThrottleTimeMs > 0) {
|
||||
val fetchResponse = if (maxThrottleTimeMs > 0) {
|
||||
request.apiThrottleTimeMs = maxThrottleTimeMs
|
||||
// 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.
|
||||
|
@ -1011,18 +914,19 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
requestHelper.throttle(quotas.request, request, requestThrottleTimeMs)
|
||||
}
|
||||
// If throttling is required, return an empty response.
|
||||
fetchContext.getThrottledResponse(maxThrottleTimeMs)
|
||||
fetchContext.getThrottledResponse(maxThrottleTimeMs, nodeEndpoints.values.toSeq.asJava)
|
||||
} else {
|
||||
// Get the actual response. This will update the fetch context.
|
||||
val unconvertedFetchResponse = fetchContext.updateAndGenerateResponseData(partitions)
|
||||
val responsePartitionsSize = unconvertedFetchResponse.data().responses().stream().mapToInt(_.partitions().size()).sum()
|
||||
val fetchResponse = fetchContext.updateAndGenerateResponseData(partitions, nodeEndpoints.values.toSeq.asJava)
|
||||
val responsePartitionsSize = fetchResponse.data().responses().stream().mapToInt(_.partitions().size()).sum()
|
||||
trace(s"Sending Fetch response with partitions.size=$responsePartitionsSize, " +
|
||||
s"metadata=${unconvertedFetchResponse.sessionId}")
|
||||
unconvertedFetchResponse
|
||||
s"metadata=${fetchResponse.sessionId}")
|
||||
fetchResponse
|
||||
}
|
||||
|
||||
recordBytesOutMetric(fetchResponse)
|
||||
// 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
|
||||
|
||||
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 clientId = request.header.clientId
|
||||
val offsetRequest = request.body[ListOffsetsRequest]
|
||||
|
@ -1180,17 +1004,20 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
buildErrorResponse(Errors.TOPIC_AUTHORIZATION_FAILED, partition)).asJava)
|
||||
)
|
||||
|
||||
def sendV1ResponseCallback(response: List[ListOffsetsTopicResponse]): Unit = {
|
||||
def sendResponseCallback(response: Seq[ListOffsetsTopicResponse]): Unit = {
|
||||
val mergedResponses = response ++ unauthorizedResponseStatus
|
||||
responseCallback(mergedResponses)
|
||||
requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
|
||||
new ListOffsetsResponse(new ListOffsetsResponseData()
|
||||
.setThrottleTimeMs(requestThrottleMs)
|
||||
.setTopics(mergedResponses.asJava)))
|
||||
}
|
||||
|
||||
if (authorizedRequestInfo.isEmpty) {
|
||||
sendV1ResponseCallback(List.empty)
|
||||
sendResponseCallback(Seq.empty)
|
||||
} else {
|
||||
replicaManager.fetchOffset(authorizedRequestInfo, offsetRequest.duplicatePartitions().asScala,
|
||||
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) {
|
||||
requestHelper.sendMaybeThrottle(request, shareFetchRequest.getErrorResponse(AbstractResponse.DEFAULT_THROTTLE_TIME, exception))
|
||||
} else {
|
||||
requestChannel.sendResponse(request, result, onFetchComplete(request))
|
||||
requestChannel.sendResponse(request, result, None)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -4744,22 +4571,6 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
.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 = {
|
||||
groupCoordinator.isNewGroupCoordinator && config.shareGroupConfig.isShareGroupEnabled
|
||||
}
|
||||
|
|
|
@ -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 logMessageDownConversionEnable: Boolean = getBoolean(ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_CONFIG)
|
||||
|
||||
def logDirFailureTimeoutMs: Long = getLong(ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG)
|
||||
|
||||
/** ********* 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_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_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_BYTES_CONFIG, remoteLogManagerConfig.logLocalRetentionBytes: java.lang.Long)
|
||||
logProps
|
||||
|
|
|
@ -29,7 +29,6 @@ import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEnd
|
|||
import org.apache.kafka.common.protocol.Errors
|
||||
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.IBP_0_10_1_IV2
|
||||
import org.apache.kafka.server.network.BrokerEndPoint
|
||||
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
@ -123,11 +122,7 @@ class RemoteLeaderEndPoint(logPrefix: String,
|
|||
.partitions.asScala.find(_.partitionIndex == topicPartition.partition).get
|
||||
|
||||
Errors.forCode(responsePartition.errorCode) match {
|
||||
case Errors.NONE =>
|
||||
if (metadataVersion.isAtLeast(IBP_0_10_1_IV2))
|
||||
new OffsetAndEpoch(responsePartition.offset, responsePartition.leaderEpoch)
|
||||
else
|
||||
new OffsetAndEpoch(responsePartition.oldStyleOffsets.get(0), responsePartition.leaderEpoch)
|
||||
case Errors.NONE => new OffsetAndEpoch(responsePartition.offset, responsePartition.leaderEpoch)
|
||||
case error => throw error.exception
|
||||
}
|
||||
}
|
||||
|
|
|
@ -264,7 +264,7 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
|
|||
}
|
||||
|
||||
private def createProduceRequest =
|
||||
requests.ProduceRequest.forCurrentMagic(new ProduceRequestData()
|
||||
requests.ProduceRequest.builder(new ProduceRequestData()
|
||||
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(
|
||||
Collections.singletonList(new ProduceRequestData.TopicProduceData()
|
||||
.setName(tp.topic).setPartitionData(Collections.singletonList(
|
||||
|
@ -389,7 +389,6 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
|
|||
.setPartitionIndex(part)
|
||||
.setCommittedOffset(0)
|
||||
.setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH)
|
||||
.setCommitTimestamp(OffsetCommitRequest.DEFAULT_TIMESTAMP)
|
||||
.setCommittedMetadata("metadata")
|
||||
)))
|
||||
)
|
||||
|
@ -2305,8 +2304,7 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
|
|||
def testMetadataClusterAuthorizedOperationsWithoutDescribeCluster(quorum: String): Unit = {
|
||||
removeAllClientAcls()
|
||||
|
||||
// MetadataRequest versions older than 1 are not supported.
|
||||
for (version <- 1 to ApiKeys.METADATA.latestVersion) {
|
||||
for (version <- ApiKeys.METADATA.oldestVersion to ApiKeys.METADATA.latestVersion) {
|
||||
testMetadataClusterClusterAuthorizedOperations(version.toShort, 0)
|
||||
}
|
||||
}
|
||||
|
@ -2326,8 +2324,7 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
|
|||
val expectedClusterAuthorizedOperations = Utils.to32BitField(
|
||||
acls.map(_.operation.code.asInstanceOf[JByte]).asJava)
|
||||
|
||||
// MetadataRequest versions older than 1 are not supported.
|
||||
for (version <- 1 to ApiKeys.METADATA.latestVersion) {
|
||||
for (version <- ApiKeys.METADATA.oldestVersion to ApiKeys.METADATA.latestVersion) {
|
||||
testMetadataClusterClusterAuthorizedOperations(version.toShort, expectedClusterAuthorizedOperations)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -305,7 +305,7 @@ class DynamicConnectionQuotaTest extends BaseRequestTest {
|
|||
}
|
||||
|
||||
private def produceRequest: ProduceRequest =
|
||||
requests.ProduceRequest.forCurrentMagic(new ProduceRequestData()
|
||||
requests.ProduceRequest.builder(new ProduceRequestData()
|
||||
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(
|
||||
Collections.singletonList(new ProduceRequestData.TopicProduceData()
|
||||
.setName(topic)
|
||||
|
|
|
@ -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_BEFORE_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"))
|
||||
|
||||
// Verify that all broker defaults have been updated
|
||||
|
|
|
@ -678,7 +678,6 @@ class GroupCoordinatorAdapterTest {
|
|||
new OffsetCommitRequestData.OffsetCommitRequestPartition()
|
||||
.setPartitionIndex(0)
|
||||
.setCommittedOffset(100)
|
||||
.setCommitTimestamp(now)
|
||||
.setCommittedLeaderEpoch(1)
|
||||
).asJava)
|
||||
).asJava)
|
||||
|
|
|
@ -32,7 +32,6 @@ import org.junit.jupiter.api.Assertions.assertEquals
|
|||
import org.junit.jupiter.api.Test
|
||||
import org.mockito.Mockito.mock
|
||||
|
||||
import java.util.Collections
|
||||
import scala.jdk.OptionConverters.RichOption
|
||||
|
||||
class RequestConvertToJsonTest {
|
||||
|
@ -51,21 +50,6 @@ class RequestConvertToJsonTest {
|
|||
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
|
||||
def testRequestDesc(): Unit = {
|
||||
val alterIsrRequest = new AlterPartitionRequest(new AlterPartitionRequestData(), 0)
|
||||
|
|
|
@ -37,7 +37,6 @@ import org.apache.kafka.common.security.scram.internals.ScramMechanism
|
|||
import org.apache.kafka.common.utils._
|
||||
import org.apache.kafka.network.RequestConvertToJson
|
||||
import org.apache.kafka.network.SocketServerConfigs
|
||||
import org.apache.kafka.network.metrics.RequestMetrics
|
||||
import org.apache.kafka.security.CredentialProvider
|
||||
import org.apache.kafka.server.common.{FinalizedFeatures, MetadataVersion}
|
||||
import org.apache.kafka.server.config.QuotaConfig
|
||||
|
@ -121,8 +120,9 @@ class SocketServerTest {
|
|||
val outgoing = new DataOutputStream(socket.getOutputStream)
|
||||
id match {
|
||||
case Some(id) =>
|
||||
outgoing.writeInt(request.length + 2)
|
||||
outgoing.writeInt(request.length + 4)
|
||||
outgoing.writeShort(id)
|
||||
outgoing.writeShort(ApiKeys.PRODUCE.oldestVersion)
|
||||
case None =>
|
||||
outgoing.writeInt(request.length)
|
||||
}
|
||||
|
@ -236,7 +236,7 @@ class SocketServerTest {
|
|||
val clientId = ""
|
||||
val ackTimeoutMs = 10000
|
||||
|
||||
val emptyRequest = requests.ProduceRequest.forCurrentMagic(new ProduceRequestData()
|
||||
val emptyRequest = requests.ProduceRequest.builder(new ProduceRequestData()
|
||||
.setTopicData(new ProduceRequestData.TopicProduceDataCollection())
|
||||
.setAcks(ack)
|
||||
.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
|
||||
def testStagedListenerStartup(): Unit = {
|
||||
shutdownServerAndMetrics(server)
|
||||
|
@ -1063,7 +1017,7 @@ class SocketServerTest {
|
|||
val clientId = ""
|
||||
val ackTimeoutMs = 10000
|
||||
val ack = 0: Short
|
||||
val emptyRequest = requests.ProduceRequest.forCurrentMagic(new ProduceRequestData()
|
||||
val emptyRequest = requests.ProduceRequest.builder(new ProduceRequestData()
|
||||
.setTopicData(new ProduceRequestData.TopicProduceDataCollection())
|
||||
.setAcks(ack)
|
||||
.setTimeoutMs(ackTimeoutMs)
|
||||
|
@ -1145,7 +1099,7 @@ class SocketServerTest {
|
|||
// ...and now send something to trigger the disconnection
|
||||
val ackTimeoutMs = 10000
|
||||
val ack = 0: Short
|
||||
val emptyRequest = requests.ProduceRequest.forCurrentMagic(new ProduceRequestData()
|
||||
val emptyRequest = requests.ProduceRequest.builder(new ProduceRequestData()
|
||||
.setTopicData(new ProduceRequestData.TopicProduceDataCollection())
|
||||
.setAcks(ack)
|
||||
.setTimeoutMs(ackTimeoutMs)
|
||||
|
|
|
@ -73,7 +73,7 @@ class ApiVersionManagerTest {
|
|||
|
||||
@Test
|
||||
def testControllerApiIntersection(): Unit = {
|
||||
val controllerMinVersion: Short = 1
|
||||
val controllerMinVersion: Short = 3
|
||||
val controllerMaxVersion: Short = 5
|
||||
|
||||
val forwardingManager = Mockito.mock(classOf[ForwardingManager])
|
||||
|
|
|
@ -339,8 +339,8 @@ class AutoTopicCreationManagerTest {
|
|||
topicsCollection.add(getNewTopic(topicName))
|
||||
val createTopicApiVersion = new ApiVersionsResponseData.ApiVersion()
|
||||
.setApiKey(ApiKeys.CREATE_TOPICS.id)
|
||||
.setMinVersion(0)
|
||||
.setMaxVersion(0)
|
||||
.setMinVersion(2)
|
||||
.setMaxVersion(2)
|
||||
Mockito.when(brokerToController.controllerApiVersions())
|
||||
.thenReturn(Optional.of(NodeApiVersions.create(Collections.singleton(createTopicApiVersion))))
|
||||
|
||||
|
|
|
@ -897,7 +897,7 @@ class ConsumerProtocolMigrationTest(cluster: ClusterInstance) extends GroupCoord
|
|||
)
|
||||
|
||||
// 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) {
|
||||
commitOffset(
|
||||
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.
|
||||
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) {
|
||||
commitOffset(
|
||||
groupId = groupId,
|
||||
|
|
|
@ -22,7 +22,7 @@ import kafka.network.SocketServer
|
|||
import kafka.utils.{Logging, TestUtils}
|
||||
import org.apache.kafka.common.message.DeleteTopicsRequestData
|
||||
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.DeleteTopicsResponse
|
||||
import org.apache.kafka.common.requests.MetadataRequest
|
||||
|
@ -141,12 +141,8 @@ class DeleteTopicsRequestTest extends BaseRequestTest with Logging {
|
|||
@ParameterizedTest
|
||||
@ValueSource(strings = Array("kraft"))
|
||||
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
|
||||
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")
|
||||
|
||||
val topicName = s"topic-$version"
|
||||
|
|
|
@ -129,7 +129,7 @@ class EdgeCaseRequestTest extends KafkaServerTestHarness {
|
|||
val version = ApiKeys.PRODUCE.latestVersion: Short
|
||||
val (serializedBytes, responseHeaderVersion) = {
|
||||
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(
|
||||
Collections.singletonList(new ProduceRequestData.TopicProduceData()
|
||||
.setName(topicPartition.topic()).setPartitionData(Collections.singletonList(
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
}
|
|
@ -116,11 +116,12 @@ class FetchRequestMaxBytesTest extends BaseRequestTest {
|
|||
|
||||
private def expectNextRecords(expected: IndexedSeq[Array[Byte]],
|
||||
fetchOffset: Long): Unit = {
|
||||
val requestVersion = 4: Short
|
||||
val response = sendFetchRequest(0,
|
||||
FetchRequest.Builder.forConsumer(3, Int.MaxValue, 0,
|
||||
FetchRequest.Builder.forConsumer(requestVersion, Int.MaxValue, 0,
|
||||
Map(testTopicPartition ->
|
||||
new PartitionData(Uuid.ZERO_UUID, fetchOffset, 0, Integer.MAX_VALUE, Optional.empty())).asJava).build(3))
|
||||
val records = FetchResponse.recordsOrFail(response.responseData(getTopicNames().asJava, 3).get(testTopicPartition)).records()
|
||||
new PartitionData(Uuid.ZERO_UUID, fetchOffset, 0, Integer.MAX_VALUE, Optional.empty())).asJava).build(requestVersion))
|
||||
val records = FetchResponse.recordsOrFail(response.responseData(getTopicNames().asJava, requestVersion).get(testTopicPartition)).records()
|
||||
assertNotNull(records)
|
||||
val recordsList = records.asScala.toList
|
||||
assertEquals(expected.size, recordsList.size)
|
||||
|
|
|
@ -20,16 +20,15 @@ import kafka.utils.TestUtils
|
|||
import org.apache.kafka.clients.producer.ProducerRecord
|
||||
import org.apache.kafka.common.config.TopicConfig
|
||||
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.serialization.{ByteArraySerializer, StringSerializer}
|
||||
import org.apache.kafka.common.serialization.StringSerializer
|
||||
import org.apache.kafka.common.{IsolationLevel, TopicIdPartition, TopicPartition, Uuid}
|
||||
import org.apache.kafka.server.record.BrokerCompressionType
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
import org.junit.jupiter.params.provider.ValueSource
|
||||
|
||||
import java.io.DataInputStream
|
||||
import java.util
|
||||
import java.util.Optional
|
||||
import scala.collection.Seq
|
||||
|
@ -363,156 +362,6 @@ class FetchRequestTest extends BaseFetchRequestTest {
|
|||
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,
|
||||
* those partitions are returned in all incremental fetch requests.
|
||||
|
@ -654,7 +503,7 @@ class FetchRequestTest extends BaseFetchRequestTest {
|
|||
|
||||
val res0 = sendFetchRequest(leaderId, req0)
|
||||
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!
|
||||
val req1= new FetchRequest.Builder(0, 10, -1, -1, Int.MaxValue, 0,
|
||||
|
@ -702,63 +551,34 @@ class FetchRequestTest extends BaseFetchRequestTest {
|
|||
"key3", "value3")).get
|
||||
producer2.close()
|
||||
|
||||
// fetch request with fetch version v1 (magic 0):
|
||||
// gzip compressed record is returned with down-conversion.
|
||||
// zstd compressed record raises UNSUPPORTED_COMPRESSION_TYPE error.
|
||||
val req0 = new FetchRequest.Builder(0, 1, -1, -1, Int.MaxValue, 0,
|
||||
// fetch request with version 4: even though zstd is officially only supported from v10, this actually succeeds
|
||||
// since the server validation is only active when zstd is configured via a topic config, the server doesn't
|
||||
// check the record batches and hence has no mechanism to detect the case where the producer sent record batches
|
||||
// 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))
|
||||
.setMaxBytes(800)
|
||||
.build()
|
||||
|
||||
.setMaxBytes(800).build()
|
||||
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(1, 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)
|
||||
assertEquals(3, records(data0).size)
|
||||
|
||||
// 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))
|
||||
.setMaxBytes(800).build()
|
||||
val res4 = sendFetchRequest(leaderId, req4)
|
||||
val data4 = res4.responseData(topicNames, 10).get(topicPartition)
|
||||
assertEquals(Errors.NONE.code, data4.errorCode)
|
||||
assertEquals(3, records(data4).size)
|
||||
val res1 = sendFetchRequest(leaderId, req1)
|
||||
val data1 = res1.responseData(topicNames, 10).get(topicPartition)
|
||||
assertEquals(Errors.NONE.code, data1.errorCode)
|
||||
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))
|
||||
.setMaxBytes(800).build()
|
||||
val res5 = sendFetchRequest(leaderId, req5)
|
||||
val data5 = res5.responseData(topicNames, ApiKeys.FETCH.latestVersion()).get(topicPartition)
|
||||
assertEquals(Errors.NONE.code, data5.errorCode)
|
||||
assertEquals(3, records(data5).size)
|
||||
val res2 = sendFetchRequest(leaderId, req2)
|
||||
val data2 = res2.responseData(topicNames, ApiKeys.FETCH.latestVersion()).get(topicPartition)
|
||||
assertEquals(Errors.NONE.code, data2.errorCode)
|
||||
assertEquals(3, records(data2).size)
|
||||
}
|
||||
|
||||
private def checkFetchResponse(expectedPartitions: Seq[TopicPartition], fetchResponse: FetchResponse,
|
||||
|
|
|
@ -210,7 +210,7 @@ class FetchSessionTest {
|
|||
.setLastStableOffset(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
|
||||
val requestData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
|
||||
|
@ -227,7 +227,7 @@ class FetchSessionTest {
|
|||
assertEquals(Optional.empty(), epochs1(tp0))
|
||||
assertEquals(Optional.of(1), epochs2(tp1))
|
||||
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
|
||||
val requestData3 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
|
||||
|
@ -310,7 +310,7 @@ class FetchSessionTest {
|
|||
.setLastStableOffset(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
|
||||
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.empty, tp2 -> Optional.of(1)),
|
||||
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
|
||||
val requestData3 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
|
||||
|
@ -411,7 +411,7 @@ class FetchSessionTest {
|
|||
.setHighWatermark(10)
|
||||
.setLastStableOffset(10)
|
||||
.setLogStartOffset(10))
|
||||
val resp2 = context2.updateAndGenerateResponseData(respData2)
|
||||
val resp2 = context2.updateAndGenerateResponseData(respData2, Seq.empty.asJava)
|
||||
assertEquals(Errors.NONE, resp2.error())
|
||||
assertTrue(resp2.sessionId() != INVALID_SESSION_ID)
|
||||
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(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
|
||||
val request4 = createRequest(new JFetchMetadata(resp2.sessionId() + 1, 1), reqData2, EMPTY_PART_LIST, isFromFollower = false)
|
||||
|
@ -441,7 +441,7 @@ class FetchSessionTest {
|
|||
topicNames
|
||||
)
|
||||
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.
|
||||
val reqData5 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
|
||||
|
@ -463,7 +463,7 @@ class FetchSessionTest {
|
|||
assertEquals(entry.getValue, data)
|
||||
})
|
||||
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(resp2.sessionId(), resp5.sessionId())
|
||||
assertEquals(0, resp5.responseData(topicNames, request5.version).size())
|
||||
|
@ -480,7 +480,7 @@ class FetchSessionTest {
|
|||
)
|
||||
assertEquals(classOf[SessionErrorContext], context6.getClass)
|
||||
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
|
||||
val reqData7 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
|
||||
|
@ -493,7 +493,7 @@ class FetchSessionTest {
|
|||
request7.forgottenTopics(topicNames),
|
||||
topicNames
|
||||
)
|
||||
val resp7 = context7.getThrottledResponse(100)
|
||||
val resp7 = context7.getThrottledResponse(100, Seq.empty.asJava)
|
||||
assertEquals(Errors.NONE, resp7.error())
|
||||
assertEquals(resp2.sessionId(), resp7.sessionId())
|
||||
assertEquals(100, resp7.throttleTimeMs())
|
||||
|
@ -531,7 +531,7 @@ class FetchSessionTest {
|
|||
.setHighWatermark(100)
|
||||
.setLastStableOffset(100)
|
||||
.setLogStartOffset(100))
|
||||
val resp8 = context8.updateAndGenerateResponseData(respData8)
|
||||
val resp8 = context8.updateAndGenerateResponseData(respData8, Seq.empty.asJava)
|
||||
assertEquals(Errors.NONE, resp8.error)
|
||||
nextSessionId = resp8.sessionId
|
||||
} while (nextSessionId == prevSessionId)
|
||||
|
@ -579,7 +579,7 @@ class FetchSessionTest {
|
|||
.setHighWatermark(10)
|
||||
.setLastStableOffset(10)
|
||||
.setLogStartOffset(10))
|
||||
val resp1 = context1.updateAndGenerateResponseData(respData1)
|
||||
val resp1 = context1.updateAndGenerateResponseData(respData1, Seq.empty.asJava)
|
||||
assertEquals(Errors.NONE, resp1.error())
|
||||
assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
|
||||
assertEquals(2, resp1.responseData(topicNames, request1.version).size())
|
||||
|
@ -620,7 +620,7 @@ class FetchSessionTest {
|
|||
.setHighWatermark(10)
|
||||
.setLastStableOffset(10)
|
||||
.setLogStartOffset(10))
|
||||
val resp2 = context2.updateAndGenerateResponseData(respData2)
|
||||
val resp2 = context2.updateAndGenerateResponseData(respData2, Seq.empty.asJava)
|
||||
assertEquals(Errors.NONE, resp2.error)
|
||||
assertEquals(1, resp2.responseData(topicNames, request2.version).size)
|
||||
assertTrue(resp2.sessionId > 0)
|
||||
|
@ -667,7 +667,7 @@ class FetchSessionTest {
|
|||
.setHighWatermark(10)
|
||||
.setLastStableOffset(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.
|
||||
assertEquals(Errors.NONE, resp1.error())
|
||||
assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
|
||||
|
@ -723,7 +723,7 @@ class FetchSessionTest {
|
|||
respData1.put(emptyZar0, new FetchResponseData.PartitionData()
|
||||
.setPartitionIndex(1)
|
||||
.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.
|
||||
assertEquals(Errors.NONE, resp1.error())
|
||||
assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
|
||||
|
@ -767,7 +767,7 @@ class FetchSessionTest {
|
|||
respData2.put(emptyZar0, new FetchResponseData.PartitionData()
|
||||
.setPartitionIndex(1)
|
||||
.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.
|
||||
assertEquals(Errors.NONE, resp2.error())
|
||||
assertTrue(resp2.sessionId() != INVALID_SESSION_ID)
|
||||
|
@ -813,7 +813,7 @@ class FetchSessionTest {
|
|||
.setHighWatermark(100)
|
||||
.setLastStableOffset(100)
|
||||
.setLogStartOffset(100))
|
||||
val resp1 = context1.updateAndGenerateResponseData(respData1)
|
||||
val resp1 = context1.updateAndGenerateResponseData(respData1, Seq.empty.asJava)
|
||||
assertEquals(Errors.NONE, resp1.error())
|
||||
assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
|
||||
|
||||
|
@ -835,7 +835,7 @@ class FetchSessionTest {
|
|||
assertEquals(classOf[SessionErrorContext], context2.getClass)
|
||||
val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
|
||||
assertEquals(Errors.FETCH_SESSION_TOPIC_ID_ERROR,
|
||||
context2.updateAndGenerateResponseData(respData2).error())
|
||||
context2.updateAndGenerateResponseData(respData2, Seq.empty.asJava).error())
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -869,7 +869,7 @@ class FetchSessionTest {
|
|||
.setHighWatermark(100)
|
||||
.setLastStableOffset(100)
|
||||
.setLogStartOffset(100))
|
||||
val resp1 = context1.updateAndGenerateResponseData(respData1)
|
||||
val resp1 = context1.updateAndGenerateResponseData(respData1, Seq.empty.asJava)
|
||||
assertEquals(Errors.NONE, resp1.error())
|
||||
assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
|
||||
|
||||
|
@ -890,7 +890,7 @@ class FetchSessionTest {
|
|||
assertEquals(classOf[SessionErrorContext], context2.getClass)
|
||||
val respData2 = new util.LinkedHashMap[TopicIdPartition, FetchResponseData.PartitionData]
|
||||
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
|
||||
|
@ -934,7 +934,7 @@ class FetchSessionTest {
|
|||
.setLastStableOffset(-1)
|
||||
.setLogStartOffset(-1)
|
||||
.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())
|
||||
assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
|
||||
assertEquals(2, resp1.responseData(topicNames, request1.version).size)
|
||||
|
@ -961,7 +961,7 @@ class FetchSessionTest {
|
|||
.setLastStableOffset(-1)
|
||||
.setLogStartOffset(-1)
|
||||
.setErrorCode(Errors.INCONSISTENT_TOPIC_ID.code))
|
||||
val resp2 = context2.updateAndGenerateResponseData(respData2)
|
||||
val resp2 = context2.updateAndGenerateResponseData(respData2, Seq.empty.asJava)
|
||||
|
||||
assertEquals(Errors.NONE, resp2.error)
|
||||
assertTrue(resp2.sessionId > 0)
|
||||
|
@ -1031,7 +1031,7 @@ class FetchSessionTest {
|
|||
noErrorResponse
|
||||
)
|
||||
}
|
||||
context.updateAndGenerateResponseData(data).sessionId
|
||||
context.updateAndGenerateResponseData(data, Seq.empty.asJava).sessionId
|
||||
}
|
||||
|
||||
val foo = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0))
|
||||
|
@ -1154,7 +1154,7 @@ class FetchSessionTest {
|
|||
noErrorResponse
|
||||
)
|
||||
}
|
||||
context.updateAndGenerateResponseData(data).sessionId
|
||||
context.updateAndGenerateResponseData(data, Seq.empty.asJava).sessionId
|
||||
}
|
||||
|
||||
val foo = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0))
|
||||
|
@ -1254,7 +1254,7 @@ class FetchSessionTest {
|
|||
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))
|
||||
|
@ -1357,7 +1357,7 @@ class FetchSessionTest {
|
|||
.setHighWatermark(10)
|
||||
.setLastStableOffset(10)
|
||||
.setLogStartOffset(10))
|
||||
val session1resp = session1context1.updateAndGenerateResponseData(respData1)
|
||||
val session1resp = session1context1.updateAndGenerateResponseData(respData1, Seq.empty.asJava)
|
||||
assertEquals(Errors.NONE, session1resp.error())
|
||||
assertTrue(session1resp.sessionId() != INVALID_SESSION_ID)
|
||||
assertEquals(2, session1resp.responseData(topicNames, session1request1.version).size)
|
||||
|
@ -1394,7 +1394,7 @@ class FetchSessionTest {
|
|||
.setHighWatermark(10)
|
||||
.setLastStableOffset(10)
|
||||
.setLogStartOffset(10))
|
||||
val session2resp = session2context.updateAndGenerateResponseData(respData1)
|
||||
val session2resp = session2context.updateAndGenerateResponseData(respData1, Seq.empty.asJava)
|
||||
assertEquals(Errors.NONE, session2resp.error())
|
||||
assertTrue(session2resp.sessionId() != INVALID_SESSION_ID)
|
||||
assertEquals(2, session2resp.responseData(topicNames, session2request1.version()).size())
|
||||
|
@ -1452,7 +1452,7 @@ class FetchSessionTest {
|
|||
.setHighWatermark(10)
|
||||
.setLastStableOffset(10)
|
||||
.setLogStartOffset(10))
|
||||
val session3resp = session3context.updateAndGenerateResponseData(respData3)
|
||||
val session3resp = session3context.updateAndGenerateResponseData(respData3, Seq.empty.asJava)
|
||||
assertEquals(Errors.NONE, session3resp.error())
|
||||
assertTrue(session3resp.sessionId() != INVALID_SESSION_ID)
|
||||
assertEquals(2, session3resp.responseData(topicNames, session3request1.version).size)
|
||||
|
@ -1500,7 +1500,7 @@ class FetchSessionTest {
|
|||
.setHighWatermark(10)
|
||||
.setLastStableOffset(10)
|
||||
.setLogStartOffset(10))
|
||||
val session1resp = session1context.updateAndGenerateResponseData(respData1)
|
||||
val session1resp = session1context.updateAndGenerateResponseData(respData1, Seq.empty.asJava)
|
||||
assertEquals(Errors.NONE, session1resp.error())
|
||||
assertTrue(session1resp.sessionId() != INVALID_SESSION_ID)
|
||||
assertEquals(2, session1resp.responseData(topicNames, session1request.version).size)
|
||||
|
@ -1538,7 +1538,7 @@ class FetchSessionTest {
|
|||
.setHighWatermark(10)
|
||||
.setLastStableOffset(10)
|
||||
.setLogStartOffset(10))
|
||||
val session2resp = session2context.updateAndGenerateResponseData(session2RespData)
|
||||
val session2resp = session2context.updateAndGenerateResponseData(session2RespData, Seq.empty.asJava)
|
||||
assertEquals(Errors.NONE, session2resp.error())
|
||||
assertTrue(session2resp.sessionId() != INVALID_SESSION_ID)
|
||||
assertEquals(2, session2resp.responseData(topicNames, session2request.version).size)
|
||||
|
@ -1578,7 +1578,7 @@ class FetchSessionTest {
|
|||
.setHighWatermark(10)
|
||||
.setLastStableOffset(10)
|
||||
.setLogStartOffset(10))
|
||||
val session3resp = session3context.updateAndGenerateResponseData(respData3)
|
||||
val session3resp = session3context.updateAndGenerateResponseData(respData3, Seq.empty.asJava)
|
||||
assertEquals(Errors.NONE, session3resp.error())
|
||||
assertTrue(session3resp.sessionId() != INVALID_SESSION_ID)
|
||||
assertEquals(2, session3resp.responseData(topicNames, session3request.version).size)
|
||||
|
@ -1621,7 +1621,7 @@ class FetchSessionTest {
|
|||
.setHighWatermark(10)
|
||||
.setLastStableOffset(10)
|
||||
.setLogStartOffset(10))
|
||||
val session4resp = session3context.updateAndGenerateResponseData(respData4)
|
||||
val session4resp = session3context.updateAndGenerateResponseData(respData4, Seq.empty.asJava)
|
||||
assertEquals(Errors.NONE, session4resp.error())
|
||||
assertTrue(session4resp.sessionId() != INVALID_SESSION_ID)
|
||||
assertEquals(2, session4resp.responseData(topicNames, session4request.version).size)
|
||||
|
@ -1669,7 +1669,7 @@ class FetchSessionTest {
|
|||
.setHighWatermark(10)
|
||||
.setLastStableOffset(10)
|
||||
.setLogStartOffset(10))
|
||||
val resp1 = context1.updateAndGenerateResponseData(respData1)
|
||||
val resp1 = context1.updateAndGenerateResponseData(respData1, Seq.empty.asJava)
|
||||
assertEquals(Errors.NONE, resp1.error)
|
||||
assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
|
||||
assertEquals(2, resp1.responseData(topicNames, request1.version).size)
|
||||
|
@ -1691,7 +1691,7 @@ class FetchSessionTest {
|
|||
)
|
||||
assertEquals(classOf[SessionlessFetchContext], context2.getClass)
|
||||
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)
|
||||
assertTrue(resp2.responseData(topicNames, request2.version).isEmpty)
|
||||
assertEquals(0, cacheShard.size)
|
||||
|
@ -1735,7 +1735,7 @@ class FetchSessionTest {
|
|||
.setLastStableOffset(105)
|
||||
.setLogStartOffset(0)
|
||||
.setDivergingEpoch(divergingEpoch))
|
||||
val resp1 = context1.updateAndGenerateResponseData(respData)
|
||||
val resp1 = context1.updateAndGenerateResponseData(respData, Seq.empty.asJava)
|
||||
assertEquals(Errors.NONE, resp1.error)
|
||||
assertNotEquals(INVALID_SESSION_ID, resp1.sessionId)
|
||||
assertEquals(util.Set.of(tp1.topicPartition, tp2.topicPartition), resp1.responseData(topicNames, request1.version).keySet)
|
||||
|
@ -1752,7 +1752,7 @@ class FetchSessionTest {
|
|||
topicNames
|
||||
)
|
||||
assertEquals(classOf[IncrementalFetchContext], context2.getClass)
|
||||
val resp2 = context2.updateAndGenerateResponseData(respData)
|
||||
val resp2 = context2.updateAndGenerateResponseData(respData, Seq.empty.asJava)
|
||||
assertEquals(Errors.NONE, resp2.error)
|
||||
assertEquals(resp1.sessionId, resp2.sessionId)
|
||||
assertEquals(Collections.singleton(tp2.topicPartition), resp2.responseData(topicNames, request2.version).keySet)
|
||||
|
@ -1764,7 +1764,7 @@ class FetchSessionTest {
|
|||
.setLastStableOffset(105)
|
||||
.setLogStartOffset(0)
|
||||
.setDivergingEpoch(divergingEpoch))
|
||||
val resp3 = context2.updateAndGenerateResponseData(respData)
|
||||
val resp3 = context2.updateAndGenerateResponseData(respData, Seq.empty.asJava)
|
||||
assertEquals(Errors.NONE, resp3.error)
|
||||
assertEquals(resp1.sessionId, resp3.sessionId)
|
||||
assertEquals(util.Set.of(tp1.topicPartition, tp2.topicPartition), resp3.responseData(topicNames, request2.version).keySet)
|
||||
|
@ -1776,7 +1776,7 @@ class FetchSessionTest {
|
|||
.setHighWatermark(110)
|
||||
.setLastStableOffset(110)
|
||||
.setLogStartOffset(0))
|
||||
val resp4 = context2.updateAndGenerateResponseData(respData)
|
||||
val resp4 = context2.updateAndGenerateResponseData(respData, Seq.empty.asJava)
|
||||
assertEquals(Errors.NONE, resp4.error)
|
||||
assertEquals(resp1.sessionId, resp4.sessionId)
|
||||
assertEquals(util.Set.of(tp1.topicPartition, tp2.topicPartition), resp4.responseData(topicNames, request2.version).keySet)
|
||||
|
@ -1820,7 +1820,7 @@ class FetchSessionTest {
|
|||
.setLastStableOffset(50)
|
||||
.setLogStartOffset(0))
|
||||
|
||||
val resp1 = context1.updateAndGenerateResponseData(respData1)
|
||||
val resp1 = context1.updateAndGenerateResponseData(respData1, Seq.empty.asJava)
|
||||
assertEquals(Errors.NONE, resp1.error)
|
||||
assertNotEquals(INVALID_SESSION_ID, resp1.sessionId)
|
||||
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
|
||||
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(resp1.sessionId, resp2.sessionId)
|
||||
assertEquals(Collections.emptySet(), resp2.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet)
|
||||
|
@ -1860,7 +1860,7 @@ class FetchSessionTest {
|
|||
.setHighWatermark(50)
|
||||
.setLastStableOffset(50)
|
||||
.setLogStartOffset(0))
|
||||
val resp3 = context2.updateAndGenerateResponseData(respData3)
|
||||
val resp3 = context2.updateAndGenerateResponseData(respData3, Seq.empty.asJava)
|
||||
assertEquals(Errors.NONE, resp3.error)
|
||||
assertEquals(resp1.sessionId, resp3.sessionId)
|
||||
assertEquals(util.Set.of(tp1.topicPartition, tp2.topicPartition), resp3.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet)
|
||||
|
|
|
@ -2651,7 +2651,7 @@ class KafkaApisTest extends Logging {
|
|||
|
||||
val tp = new TopicPartition("topic", 0)
|
||||
|
||||
val produceRequest = ProduceRequest.forCurrentMagic(new ProduceRequestData()
|
||||
val produceRequest = ProduceRequest.builder(new ProduceRequestData()
|
||||
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(
|
||||
Collections.singletonList(new ProduceRequestData.TopicProduceData()
|
||||
.setName(tp.topic).setPartitionData(Collections.singletonList(
|
||||
|
@ -2713,7 +2713,7 @@ class KafkaApisTest extends Logging {
|
|||
val newLeaderId = 2
|
||||
val newLeaderEpoch = 5
|
||||
|
||||
val produceRequest = ProduceRequest.forCurrentMagic(new ProduceRequestData()
|
||||
val produceRequest = ProduceRequest.builder(new ProduceRequestData()
|
||||
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(
|
||||
Collections.singletonList(new ProduceRequestData.TopicProduceData()
|
||||
.setName(tp.topic).setPartitionData(Collections.singletonList(
|
||||
|
@ -2778,7 +2778,7 @@ class KafkaApisTest extends Logging {
|
|||
|
||||
val tp = new TopicPartition(topic, 0)
|
||||
|
||||
val produceRequest = ProduceRequest.forCurrentMagic(new ProduceRequestData()
|
||||
val produceRequest = ProduceRequest.builder(new ProduceRequestData()
|
||||
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(
|
||||
Collections.singletonList(new ProduceRequestData.TopicProduceData()
|
||||
.setName(tp.topic).setPartitionData(Collections.singletonList(
|
||||
|
@ -2842,7 +2842,7 @@ class KafkaApisTest extends Logging {
|
|||
|
||||
val tp = new TopicPartition(topic, 0)
|
||||
|
||||
val produceRequest = ProduceRequest.forCurrentMagic(new ProduceRequestData()
|
||||
val produceRequest = ProduceRequest.builder(new ProduceRequestData()
|
||||
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(
|
||||
Collections.singletonList(new ProduceRequestData.TopicProduceData()
|
||||
.setName(tp.topic).setPartitionData(Collections.singletonList(
|
||||
|
@ -2899,13 +2899,13 @@ class KafkaApisTest extends Logging {
|
|||
|
||||
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)
|
||||
|
||||
val tp = new TopicPartition("topic", 0)
|
||||
|
||||
val produceRequest = ProduceRequest.forCurrentMagic(new ProduceRequestData()
|
||||
val produceRequest = ProduceRequest.builder(new ProduceRequestData()
|
||||
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(
|
||||
Collections.singletonList(new ProduceRequestData.TopicProduceData()
|
||||
.setName(tp.topic).setPartitionData(Collections.singletonList(
|
||||
|
@ -4376,11 +4376,6 @@ class KafkaApisTest extends Logging {
|
|||
testConsumerListOffsetWithUnsupportedVersion(ListOffsetsRequest.LATEST_TIERED_TIMESTAMP, 8)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testListOffsetNegativeTimestampWithZeroVersion(): Unit = {
|
||||
testConsumerListOffsetWithUnsupportedVersion(-3, 0)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testListOffsetNegativeTimestampWithOneOrAboveVersion(): Unit = {
|
||||
testConsumerListOffsetWithUnsupportedVersion(-6, 1)
|
||||
|
@ -4758,9 +4753,9 @@ class KafkaApisTest extends Logging {
|
|||
assertEquals(Errors.NOT_LEADER_OR_FOLLOWER.code, partitionData.errorCode)
|
||||
assertEquals(newLeaderId, partitionData.currentLeader.leaderId())
|
||||
assertEquals(newLeaderEpoch, partitionData.currentLeader.leaderEpoch())
|
||||
val node = response.data.nodeEndpoints.asScala.head
|
||||
assertEquals(2, node.nodeId)
|
||||
assertEquals("broker2", node.host)
|
||||
val node = response.data.nodeEndpoints.asScala
|
||||
assertEquals(Seq(2), node.map(_.nodeId))
|
||||
assertEquals(Seq("broker2"), node.map(_.host))
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -1205,8 +1205,6 @@ class KafkaConfigTest {
|
|||
assertDynamic(kafkaConfigProp, 10007, () => config.logIndexIntervalBytes)
|
||||
case TopicConfig.MAX_MESSAGE_BYTES_CONFIG =>
|
||||
assertDynamic(kafkaConfigProp, 10008, () => config.messageMaxBytes)
|
||||
case TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_CONFIG =>
|
||||
assertDynamic(kafkaConfigProp, false, () => config.logMessageDownConversionEnable)
|
||||
case TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG =>
|
||||
assertDynamic(kafkaConfigProp, 10015L, () => config.logMessageTimestampBeforeMaxMs)
|
||||
case TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG =>
|
||||
|
|
|
@ -165,13 +165,6 @@ class ListOffsetsRequestTest extends BaseRequestTest {
|
|||
|
||||
private[this] def fetchOffsetAndEpochWithError(serverId: Int, timestamp: Long, version: Short): (Long, Int, Short) = {
|
||||
val partitionData = sendRequest(serverId, timestamp, version)
|
||||
|
||||
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())
|
||||
}
|
||||
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue