From f34e7c2e4df48316c18a7a956cb4616b22236a22 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Fri, 9 Oct 2020 13:53:43 +0800 Subject: [PATCH 01/15] KAFKA-9628 Replace Produce request with automated protocol --- .../apache/kafka/common/protocol/ApiKeys.java | 4 +- .../common/requests/AbstractRequest.java | 3 +- .../kafka/common/requests/ProduceRequest.java | 248 +++++------------- .../common/message/ProduceRequest.json | 8 +- .../common/requests/RequestResponseTest.java | 18 ++ 5 files changed, 96 insertions(+), 185 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java index 7bed902837272..214788a610665 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java @@ -111,6 +111,7 @@ import org.apache.kafka.common.message.OffsetDeleteResponseData; import org.apache.kafka.common.message.OffsetFetchRequestData; import org.apache.kafka.common.message.OffsetFetchResponseData; +import org.apache.kafka.common.message.ProduceRequestData; import org.apache.kafka.common.message.RenewDelegationTokenRequestData; import org.apache.kafka.common.message.RenewDelegationTokenResponseData; import org.apache.kafka.common.message.SaslAuthenticateRequestData; @@ -138,7 +139,6 @@ import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.requests.OffsetsForLeaderEpochRequest; import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse; -import org.apache.kafka.common.requests.ProduceRequest; import org.apache.kafka.common.requests.ProduceResponse; import java.nio.ByteBuffer; @@ -157,7 +157,7 @@ * Identifiers for all the Kafka APIs */ public enum ApiKeys { - PRODUCE(0, "Produce", ProduceRequest.schemaVersions(), ProduceResponse.schemaVersions()), + PRODUCE(0, "Produce", ProduceRequestData.SCHEMAS, ProduceResponse.schemaVersions()), FETCH(1, "Fetch", FetchRequestData.SCHEMAS, FetchResponseData.SCHEMAS), LIST_OFFSETS(2, "ListOffsets", ListOffsetRequestData.SCHEMAS, ListOffsetResponseData.SCHEMAS), METADATA(3, "Metadata", MetadataRequestData.SCHEMAS, MetadataResponseData.SCHEMAS), diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java index f2c6ed487258e..a0121666dfcf2 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.message.FetchRequestData; import org.apache.kafka.common.message.AlterIsrRequestData; +import org.apache.kafka.common.message.ProduceRequestData; import org.apache.kafka.common.network.NetworkSend; import org.apache.kafka.common.network.Send; import org.apache.kafka.common.protocol.ApiKeys; @@ -146,7 +147,7 @@ public Map errorCounts(Throwable e) { public static AbstractRequest parseRequest(ApiKeys apiKey, short apiVersion, Struct struct) { switch (apiKey) { case PRODUCE: - return new ProduceRequest(struct, apiVersion); + return new ProduceRequest(new ProduceRequestData(struct, apiVersion), apiVersion); case FETCH: return new FetchRequest(new FetchRequestData(struct, apiVersion), apiVersion); case LIST_OFFSETS: diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java index 210b71da22158..83d02abac18d6 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -19,117 +19,28 @@ import org.apache.kafka.common.InvalidRecordException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.UnsupportedCompressionTypeException; +import org.apache.kafka.common.message.ProduceRequestData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.CommonFields; +import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.types.ArrayOf; -import org.apache.kafka.common.protocol.types.Field; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MutableRecordBatch; import org.apache.kafka.common.record.RecordBatch; -import org.apache.kafka.common.utils.CollectionUtils; import org.apache.kafka.common.utils.Utils; import java.nio.ByteBuffer; -import java.util.ArrayList; +import java.util.AbstractMap; import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; - -import static org.apache.kafka.common.protocol.CommonFields.NULLABLE_TRANSACTIONAL_ID; -import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID; -import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME; -import static org.apache.kafka.common.protocol.types.Type.INT16; -import static org.apache.kafka.common.protocol.types.Type.INT32; -import static org.apache.kafka.common.protocol.types.Type.RECORDS; +import java.util.stream.Collectors; public class ProduceRequest extends AbstractRequest { - private static final String ACKS_KEY_NAME = "acks"; - private static final String TIMEOUT_KEY_NAME = "timeout"; - private static final String TOPIC_DATA_KEY_NAME = "topic_data"; - - // topic level field names - private static final String PARTITION_DATA_KEY_NAME = "data"; - - // partition level field names - private static final String RECORD_SET_KEY_NAME = "record_set"; - - - private static final Schema TOPIC_PRODUCE_DATA_V0 = new Schema( - TOPIC_NAME, - new Field(PARTITION_DATA_KEY_NAME, new ArrayOf(new Schema( - PARTITION_ID, - new Field(RECORD_SET_KEY_NAME, RECORDS))))); - - private static final Schema PRODUCE_REQUEST_V0 = new Schema( - new Field(ACKS_KEY_NAME, INT16, "The number of acknowledgments the producer requires the leader to have " + - "received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for " + - "only the leader and -1 for the full ISR."), - new Field(TIMEOUT_KEY_NAME, INT32, "The time to await a response in ms."), - new Field(TOPIC_DATA_KEY_NAME, new ArrayOf(TOPIC_PRODUCE_DATA_V0))); - - /** - * The body of PRODUCE_REQUEST_V1 is the same as PRODUCE_REQUEST_V0. - * The version number is bumped up to indicate that the client supports quota throttle time field in the response. - */ - private static final Schema PRODUCE_REQUEST_V1 = PRODUCE_REQUEST_V0; - /** - * The body of PRODUCE_REQUEST_V2 is the same as PRODUCE_REQUEST_V1. - * The version number is bumped up to indicate that message format V1 is used which has relative offset and - * timestamp. - */ - private static final Schema PRODUCE_REQUEST_V2 = PRODUCE_REQUEST_V1; - - // Produce request V3 adds the transactional id which is used for authorization when attempting to write - // transactional data. This version also adds support for message format V2. - private static final Schema PRODUCE_REQUEST_V3 = new Schema( - CommonFields.NULLABLE_TRANSACTIONAL_ID, - new Field(ACKS_KEY_NAME, INT16, "The number of acknowledgments the producer requires the leader to have " + - "received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 " + - "for only the leader and -1 for the full ISR."), - new Field(TIMEOUT_KEY_NAME, INT32, "The time to await a response in ms."), - new Field(TOPIC_DATA_KEY_NAME, new ArrayOf(TOPIC_PRODUCE_DATA_V0))); - - /** - * The body of PRODUCE_REQUEST_V4 is the same as PRODUCE_REQUEST_V3. - * The version number is bumped up to indicate that the client supports KafkaStorageException. - * The KafkaStorageException will be translated to NotLeaderOrFollowerException in the response if version <= 3 - */ - private static final Schema PRODUCE_REQUEST_V4 = PRODUCE_REQUEST_V3; - - /** - * The body of the PRODUCE_REQUEST_V5 is the same as PRODUCE_REQUEST_V4. - * The version number is bumped since the PRODUCE_RESPONSE_V5 includes an additional partition level - * field: the log_start_offset. - */ - private static final Schema PRODUCE_REQUEST_V5 = PRODUCE_REQUEST_V4; - - /** - * The version number is bumped to indicate that on quota violation brokers send out responses before throttling. - */ - private static final Schema PRODUCE_REQUEST_V6 = PRODUCE_REQUEST_V5; - - /** - * V7 bumped up to indicate ZStandard capability. (see KIP-110) - */ - private static final Schema PRODUCE_REQUEST_V7 = PRODUCE_REQUEST_V6; - - /** - * V8 bumped up to add two new fields record_errors offset list and error_message to {@link org.apache.kafka.common.requests.ProduceResponse.PartitionResponse} - * (See KIP-467) - */ - private static final Schema PRODUCE_REQUEST_V8 = PRODUCE_REQUEST_V7; - - public static Schema[] schemaVersions() { - return new Schema[] {PRODUCE_REQUEST_V0, PRODUCE_REQUEST_V1, PRODUCE_REQUEST_V2, PRODUCE_REQUEST_V3, - PRODUCE_REQUEST_V4, PRODUCE_REQUEST_V5, PRODUCE_REQUEST_V6, PRODUCE_REQUEST_V7, PRODUCE_REQUEST_V8}; - } public static class Builder extends AbstractRequest.Builder { private final short acks; @@ -194,7 +105,27 @@ private ProduceRequest build(short version, boolean validate) { ProduceRequest.validateRecords(version, records); } } - return new ProduceRequest(version, acks, timeout, partitionRecords, transactionalId); + + List tpd = partitionRecords + .entrySet() + .stream() + .collect(Collectors.groupingBy(e -> e.getKey().topic())) + .entrySet() + .stream() + .map(e -> new ProduceRequestData.TopicProduceData() + .setName(e.getKey()) + .setPartitions(e.getValue().stream() + .map(tpAndRecord -> new ProduceRequestData.PartitionProduceData() + .setPartitionIndex(tpAndRecord.getKey().partition()) + .setRecords(tpAndRecord.getValue().buffer())) + .collect(Collectors.toList()))) + .collect(Collectors.toList()); + + return new ProduceRequest(new ProduceRequestData() + .setAcks(acks) + .setTimeout(timeout) + .setTransactionalId(transactionalId) + .setTopicData(tpd), version); } @Override @@ -210,65 +141,42 @@ public String toString() { } } + /** + * We have to copy acks, timeout, transactionalId and partitionSizes from data since data maybe reset to eliminate + * the reference to ByteBuffer but those metadata are still useful. + */ private final short acks; private final int timeout; private final String transactionalId; - - private final Map partitionSizes; - + // visible for testing + final Map partitionSizes; + private boolean hasTransactionalRecords = false; + private boolean hasIdempotentRecords = false; // This is set to null by `clearPartitionRecords` to prevent unnecessary memory retention when a produce request is // put in the purgatory (due to client throttling, it can take a while before the response is sent). // Care should be taken in methods that use this field. - private volatile Map partitionRecords; - private boolean hasTransactionalRecords = false; - private boolean hasIdempotentRecords = false; + private volatile ProduceRequestData data; - private ProduceRequest(short version, short acks, int timeout, Map partitionRecords, String transactionalId) { + public ProduceRequest(ProduceRequestData produceRequestData, short version) { super(ApiKeys.PRODUCE, version); - this.acks = acks; - this.timeout = timeout; - - this.transactionalId = transactionalId; - this.partitionRecords = partitionRecords; - this.partitionSizes = createPartitionSizes(partitionRecords); - - for (MemoryRecords records : partitionRecords.values()) { - setFlags(records); - } - } - - private static Map createPartitionSizes(Map partitionRecords) { - Map result = new HashMap<>(partitionRecords.size()); - for (Map.Entry entry : partitionRecords.entrySet()) - result.put(entry.getKey(), entry.getValue().sizeInBytes()); - return result; - } - - public ProduceRequest(Struct struct, short version) { - super(ApiKeys.PRODUCE, version); - partitionRecords = new HashMap<>(); - for (Object topicDataObj : struct.getArray(TOPIC_DATA_KEY_NAME)) { - Struct topicData = (Struct) topicDataObj; - String topic = topicData.get(TOPIC_NAME); - for (Object partitionResponseObj : topicData.getArray(PARTITION_DATA_KEY_NAME)) { - Struct partitionResponse = (Struct) partitionResponseObj; - int partition = partitionResponse.get(PARTITION_ID); - MemoryRecords records = (MemoryRecords) partitionResponse.getRecords(RECORD_SET_KEY_NAME); - setFlags(records); - partitionRecords.put(new TopicPartition(topic, partition), records); - } - } - partitionSizes = createPartitionSizes(partitionRecords); - acks = struct.getShort(ACKS_KEY_NAME); - timeout = struct.getInt(TIMEOUT_KEY_NAME); - transactionalId = struct.getOrElse(NULLABLE_TRANSACTIONAL_ID, null); - } - - private void setFlags(MemoryRecords records) { - Iterator iterator = records.batches().iterator(); - MutableRecordBatch entry = iterator.next(); - hasIdempotentRecords = hasIdempotentRecords || entry.hasProducerId(); - hasTransactionalRecords = hasTransactionalRecords || entry.isTransactional(); + this.data = produceRequestData; + this.data.topicData().forEach(topicProduceData -> topicProduceData.partitions() + .forEach(partitionProduceData -> { + MemoryRecords records = MemoryRecords.readableRecords(partitionProduceData.records()); + Iterator iterator = records.batches().iterator(); + MutableRecordBatch entry = iterator.next(); + hasIdempotentRecords = hasIdempotentRecords || entry.hasProducerId(); + hasTransactionalRecords = hasTransactionalRecords || entry.isTransactional(); + })); + this.acks = data.acks(); + this.timeout = data.timeout(); + this.transactionalId = data.transactionalId(); + this.partitionSizes = data.topicData() + .stream() + .flatMap(e -> e.partitions() + .stream() + .map(p -> new AbstractMap.SimpleEntry<>(new TopicPartition(e.name(), p.partitionIndex()), p.records().limit()))) + .collect(Collectors.groupingBy(AbstractMap.SimpleEntry::getKey, Collectors.summingInt(AbstractMap.SimpleEntry::getValue))); } /** @@ -277,31 +185,10 @@ private void setFlags(MemoryRecords records) { @Override public Struct toStruct() { // Store it in a local variable to protect against concurrent updates - Map partitionRecords = partitionRecordsOrFail(); - short version = version(); - Struct struct = new Struct(ApiKeys.PRODUCE.requestSchema(version)); - Map> recordsByTopic = CollectionUtils.groupPartitionDataByTopic(partitionRecords); - struct.set(ACKS_KEY_NAME, acks); - struct.set(TIMEOUT_KEY_NAME, timeout); - struct.setIfExists(NULLABLE_TRANSACTIONAL_ID, transactionalId); - - List topicDatas = new ArrayList<>(recordsByTopic.size()); - for (Map.Entry> topicEntry : recordsByTopic.entrySet()) { - Struct topicData = struct.instance(TOPIC_DATA_KEY_NAME); - topicData.set(TOPIC_NAME, topicEntry.getKey()); - List partitionArray = new ArrayList<>(); - for (Map.Entry partitionEntry : topicEntry.getValue().entrySet()) { - MemoryRecords records = partitionEntry.getValue(); - Struct part = topicData.instance(PARTITION_DATA_KEY_NAME) - .set(PARTITION_ID, partitionEntry.getKey()) - .set(RECORD_SET_KEY_NAME, records); - partitionArray.add(part); - } - topicData.set(PARTITION_DATA_KEY_NAME, partitionArray.toArray()); - topicDatas.add(topicData); - } - struct.set(TOPIC_DATA_KEY_NAME, topicDatas.toArray()); - return struct; + ProduceRequestData tmp = data; + if (tmp == null) + throw new IllegalStateException("The partition records are no longer available because clearPartitionRecords() has been invoked."); + return tmp.toStruct(version()); } @Override @@ -367,19 +254,24 @@ public boolean hasIdempotentRecords() { } /** - * Returns the partition records or throws IllegalStateException if clearPartitionRecords() has been invoked. + * convert the generated data to Map. + * Noted that the cost of conversion can be expensive so caller should keep and reuse the returned collection. */ public Map partitionRecordsOrFail() { // Store it in a local variable to protect against concurrent updates - Map partitionRecords = this.partitionRecords; - if (partitionRecords == null) - throw new IllegalStateException("The partition records are no longer available because " + - "clearPartitionRecords() has been invoked."); - return partitionRecords; + ProduceRequestData tmp = data; + if (tmp == null) + throw new IllegalStateException("The partition records are no longer available because clearPartitionRecords() has been invoked."); + Map partitionRecords = new HashMap<>(); + tmp.topicData().forEach(tpData -> tpData.partitions().forEach(p -> { + TopicPartition tp = new TopicPartition(tpData.name(), p.partitionIndex()); + partitionRecords.put(tp, MemoryRecords.readableRecords(p.records())); + })); + return Collections.unmodifiableMap(partitionRecords); } public void clearPartitionRecords() { - partitionRecords = null; + data = null; } public static void validateRecords(short version, MemoryRecords records) { @@ -409,7 +301,7 @@ public static void validateRecords(short version, MemoryRecords records) { } public static ProduceRequest parse(ByteBuffer buffer, short version) { - return new ProduceRequest(ApiKeys.PRODUCE.parseRequest(version, buffer), version); + return new ProduceRequest(new ProduceRequestData(new ByteBufferAccessor(buffer), version), version); } public static byte requiredMagicForVersion(short produceRequestVersion) { diff --git a/clients/src/main/resources/common/message/ProduceRequest.json b/clients/src/main/resources/common/message/ProduceRequest.json index a872da75c7e9b..cede9677db47b 100644 --- a/clients/src/main/resources/common/message/ProduceRequest.json +++ b/clients/src/main/resources/common/message/ProduceRequest.json @@ -33,13 +33,13 @@ "validVersions": "0-8", "flexibleVersions": "none", "fields": [ - { "name": "TransactionalId", "type": "string", "versions": "3+", "nullableVersions": "0+", "entityType": "transactionalId", + { "name": "TransactionalId", "type": "string", "versions": "3+", "nullableVersions": "3+", "ignorable": true, "entityType": "transactionalId", "about": "The transactional ID, or null if the producer is not transactional." }, { "name": "Acks", "type": "int16", "versions": "0+", "about": "The number of acknowledgments the producer requires the leader to have received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for only the leader and -1 for the full ISR." }, - { "name": "TimeoutMs", "type": "int32", "versions": "0+", + { "name": "Timeout", "type": "int32", "versions": "0+", "about": "The timeout to await a response in miliseconds." }, - { "name": "Topics", "type": "[]TopicProduceData", "versions": "0+", + { "name": "TopicData", "type": "[]TopicProduceData", "versions": "0+", "about": "Each topic to produce to.", "fields": [ { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", "about": "The topic name." }, @@ -47,7 +47,7 @@ "about": "Each partition to produce to.", "fields": [ { "name": "PartitionIndex", "type": "int32", "versions": "0+", "about": "The partition index." }, - { "name": "Records", "type": "bytes", "versions": "0+", "nullableVersions": "0+", + { "name": "Records", "type": "bytes", "zeroCopy": true, "versions": "0+", "nullableVersions": "0+", "about": "The record data to be produced." } ]} ]} diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 71048d8752dde..e2bdb9542710e 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -625,6 +625,24 @@ public void cannotUseFindCoordinatorV0ToFindTransactionCoordinator() { builder.build((short) 0); } + @Test + public void testPartitionSize() { + TopicPartition tp0 = new TopicPartition("test", 0); + TopicPartition tp1 = new TopicPartition("test", 1); + MemoryRecords records0 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, + CompressionType.NONE, new SimpleRecord("woot".getBytes())); + MemoryRecords records1 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, + CompressionType.NONE, new SimpleRecord("woot".getBytes()), new SimpleRecord("woot".getBytes())); + Map produceData = new HashMap<>(); + produceData.put(tp0, records0); + produceData.put(tp1, records1); + ProduceRequest request = ProduceRequest.Builder.forMagic(RecordBatch.MAGIC_VALUE_V2, (short) 1, 5000, produceData, "transactionalId") + .build((short) 3); + assertEquals(2, request.partitionSizes.size()); + assertEquals(records0.sizeInBytes(), (int) request.partitionSizes.get(tp0)); + assertEquals(records1.sizeInBytes(), (int) request.partitionSizes.get(tp1)); + } + @Test public void produceRequestToStringTest() { ProduceRequest request = createProduceRequest(ApiKeys.PRODUCE.latestVersion()); From f68102df76c76d1b3a5ec055def10a34e83ac128 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Sat, 10 Oct 2020 00:23:50 +0800 Subject: [PATCH 02/15] Replace Produce response with automated protocol --- .../apache/kafka/common/protocol/ApiKeys.java | 4 +- .../common/requests/AbstractResponse.java | 3 +- .../kafka/common/requests/ProduceRequest.java | 27 +- .../common/requests/ProduceResponse.java | 302 ++++-------------- .../common/message/ProduceResponse.json | 2 +- .../kafka/clients/NetworkClientTest.java | 1 + .../common/requests/ProduceResponseTest.java | 6 +- 7 files changed, 84 insertions(+), 261 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java index 214788a610665..03ee450511c83 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java @@ -112,6 +112,7 @@ import org.apache.kafka.common.message.OffsetFetchRequestData; import org.apache.kafka.common.message.OffsetFetchResponseData; import org.apache.kafka.common.message.ProduceRequestData; +import org.apache.kafka.common.message.ProduceResponseData; import org.apache.kafka.common.message.RenewDelegationTokenRequestData; import org.apache.kafka.common.message.RenewDelegationTokenResponseData; import org.apache.kafka.common.message.SaslAuthenticateRequestData; @@ -139,7 +140,6 @@ import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.requests.OffsetsForLeaderEpochRequest; import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse; -import org.apache.kafka.common.requests.ProduceResponse; import java.nio.ByteBuffer; import java.util.Arrays; @@ -157,7 +157,7 @@ * Identifiers for all the Kafka APIs */ public enum ApiKeys { - PRODUCE(0, "Produce", ProduceRequestData.SCHEMAS, ProduceResponse.schemaVersions()), + PRODUCE(0, "Produce", ProduceRequestData.SCHEMAS, ProduceResponseData.SCHEMAS), FETCH(1, "Fetch", FetchRequestData.SCHEMAS, FetchResponseData.SCHEMAS), LIST_OFFSETS(2, "ListOffsets", ListOffsetRequestData.SCHEMAS, ListOffsetResponseData.SCHEMAS), METADATA(3, "Metadata", MetadataRequestData.SCHEMAS, MetadataResponseData.SCHEMAS), diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java index 2ff143c0eb4ab..104148d75c0da 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.message.EnvelopeResponseData; import org.apache.kafka.common.message.FetchResponseData; import org.apache.kafka.common.message.AlterIsrResponseData; +import org.apache.kafka.common.message.ProduceResponseData; import org.apache.kafka.common.network.NetworkSend; import org.apache.kafka.common.network.Send; import org.apache.kafka.common.protocol.ApiKeys; @@ -96,7 +97,7 @@ public static AbstractResponse parseResponse(ByteBuffer byteBuffer, RequestHeade public static AbstractResponse parseResponse(ApiKeys apiKey, Struct struct, short version) { switch (apiKey) { case PRODUCE: - return new ProduceResponse(struct); + return new ProduceResponse(new ProduceResponseData(struct, version)); case FETCH: return new FetchResponse<>(new FetchResponseData(struct, version)); case LIST_OFFSETS: diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java index 83d02abac18d6..b1b4000f8e1b5 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.UnsupportedCompressionTypeException; import org.apache.kafka.common.message.ProduceRequestData; +import org.apache.kafka.common.message.ProduceResponseData; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; @@ -214,13 +215,25 @@ public ProduceResponse getErrorResponse(int throttleTimeMs, Throwable e) { return null; Errors error = Errors.forException(e); - Map responseMap = new HashMap<>(); - ProduceResponse.PartitionResponse partitionResponse = new ProduceResponse.PartitionResponse(error); - - for (TopicPartition tp : partitions()) - responseMap.put(tp, partitionResponse); - - return new ProduceResponse(responseMap, throttleTimeMs); + return new ProduceResponse(new ProduceResponseData() + .setResponses(partitionSizes.keySet() + .stream() + .collect(Collectors.groupingBy(TopicPartition::topic)) + .entrySet() + .stream() + .map(tp -> new ProduceResponseData.TopicProduceResponse() + .setName(tp.getKey()) + .setPartitions(tp.getValue().stream().map(p -> new ProduceResponseData.PartitionProduceResponse() + .setPartitionIndex(p.partition()) + .setBaseOffset(ProduceResponse.INVALID_OFFSET) + .setErrorCode(error.code()) + .setErrorMessage(error.message()) + .setLogAppendTimeMs(RecordBatch.NO_TIMESTAMP) + .setLogStartOffset(ProduceResponse.INVALID_OFFSET) + .setRecordErrors(Collections.emptyList())) + .collect(Collectors.toList()))) + .collect(Collectors.toList())) + .setThrottleTimeMs(throttleTimeMs)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java index 205d9aba5a683..8cac3ed464ce7 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -17,179 +17,48 @@ package org.apache.kafka.common.requests; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.message.ProduceResponseData; +import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.types.ArrayOf; -import org.apache.kafka.common.protocol.types.Field; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.record.RecordBatch; -import org.apache.kafka.common.utils.CollectionUtils; import java.nio.ByteBuffer; -import java.util.ArrayList; +import java.util.AbstractMap; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; - -import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE; -import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID; -import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS; -import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME; -import static org.apache.kafka.common.protocol.types.Type.INT64; +import java.util.stream.Collectors; /** - * This wrapper supports both v0 and v1 of ProduceResponse. + * This wrapper supports both v0 and v8 of ProduceResponse. */ public class ProduceResponse extends AbstractResponse { - - private static final String RESPONSES_KEY_NAME = "responses"; - - // topic level field names - private static final String PARTITION_RESPONSES_KEY_NAME = "partition_responses"; - public static final long INVALID_OFFSET = -1L; + private final ProduceResponseData data; + private final Map responses; - /** - * Possible error code: - * - * {@link Errors#CORRUPT_MESSAGE} - * {@link Errors#UNKNOWN_TOPIC_OR_PARTITION} - * {@link Errors#NOT_LEADER_OR_FOLLOWER} - * {@link Errors#MESSAGE_TOO_LARGE} - * {@link Errors#INVALID_TOPIC_EXCEPTION} - * {@link Errors#RECORD_LIST_TOO_LARGE} - * {@link Errors#NOT_ENOUGH_REPLICAS} - * {@link Errors#NOT_ENOUGH_REPLICAS_AFTER_APPEND} - * {@link Errors#INVALID_REQUIRED_ACKS} - * {@link Errors#TOPIC_AUTHORIZATION_FAILED} - * {@link Errors#UNSUPPORTED_FOR_MESSAGE_FORMAT} - * {@link Errors#INVALID_PRODUCER_EPOCH} - * {@link Errors#CLUSTER_AUTHORIZATION_FAILED} - * {@link Errors#TRANSACTIONAL_ID_AUTHORIZATION_FAILED} - * {@link Errors#INVALID_RECORD} - */ - - private static final String BASE_OFFSET_KEY_NAME = "base_offset"; - private static final String LOG_APPEND_TIME_KEY_NAME = "log_append_time"; - private static final String LOG_START_OFFSET_KEY_NAME = "log_start_offset"; - private static final String RECORD_ERRORS_KEY_NAME = "record_errors"; - private static final String BATCH_INDEX_KEY_NAME = "batch_index"; - private static final String BATCH_INDEX_ERROR_MESSAGE_KEY_NAME = "batch_index_error_message"; - private static final String ERROR_MESSAGE_KEY_NAME = "error_message"; - - private static final Field.Int64 LOG_START_OFFSET_FIELD = new Field.Int64(LOG_START_OFFSET_KEY_NAME, - "The start offset of the log at the time this produce response was created", INVALID_OFFSET); - private static final Field.NullableStr BATCH_INDEX_ERROR_MESSAGE_FIELD = new Field.NullableStr(BATCH_INDEX_ERROR_MESSAGE_KEY_NAME, - "The error message of the record that caused the batch to be dropped"); - private static final Field.NullableStr ERROR_MESSAGE_FIELD = new Field.NullableStr(ERROR_MESSAGE_KEY_NAME, - "The global error message summarizing the common root cause of the records that caused the batch to be dropped"); - - private static final Schema PRODUCE_RESPONSE_V0 = new Schema( - new Field(RESPONSES_KEY_NAME, new ArrayOf(new Schema( - TOPIC_NAME, - new Field(PARTITION_RESPONSES_KEY_NAME, new ArrayOf(new Schema( - PARTITION_ID, - ERROR_CODE, - new Field(BASE_OFFSET_KEY_NAME, INT64)))))))); - - private static final Schema PRODUCE_RESPONSE_V1 = new Schema( - new Field(RESPONSES_KEY_NAME, new ArrayOf(new Schema( - TOPIC_NAME, - new Field(PARTITION_RESPONSES_KEY_NAME, new ArrayOf(new Schema( - PARTITION_ID, - ERROR_CODE, - new Field(BASE_OFFSET_KEY_NAME, INT64))))))), - THROTTLE_TIME_MS); - - /** - * PRODUCE_RESPONSE_V2 added a timestamp field in the per partition response status. - * The timestamp is log append time if the topic is configured to use log append time. Or it is NoTimestamp when create - * time is used for the topic. - */ - private static final Schema PRODUCE_RESPONSE_V2 = new Schema( - new Field(RESPONSES_KEY_NAME, new ArrayOf(new Schema( - TOPIC_NAME, - new Field(PARTITION_RESPONSES_KEY_NAME, new ArrayOf(new Schema( - PARTITION_ID, - ERROR_CODE, - new Field(BASE_OFFSET_KEY_NAME, INT64), - new Field(LOG_APPEND_TIME_KEY_NAME, INT64, "The timestamp returned by broker after appending " + - "the messages. If CreateTime is used for the topic, the timestamp will be -1. " + - "If LogAppendTime is used for the topic, the timestamp will be " + - "the broker local time when the messages are appended."))))))), - THROTTLE_TIME_MS); - - private static final Schema PRODUCE_RESPONSE_V3 = PRODUCE_RESPONSE_V2; - - /** - * The body of PRODUCE_RESPONSE_V4 is the same as PRODUCE_RESPONSE_V3. - * The version number is bumped up to indicate that the client supports KafkaStorageException. - * The KafkaStorageException will be translated to NotLeaderOrFollowerException in the response if version <= 3 - */ - private static final Schema PRODUCE_RESPONSE_V4 = PRODUCE_RESPONSE_V3; - - - /** - * Add in the log_start_offset field to the partition response to filter out spurious OutOfOrderSequencExceptions - * on the client. - */ - public static final Schema PRODUCE_RESPONSE_V5 = new Schema( - new Field(RESPONSES_KEY_NAME, new ArrayOf(new Schema( - TOPIC_NAME, - new Field(PARTITION_RESPONSES_KEY_NAME, new ArrayOf(new Schema( - PARTITION_ID, - ERROR_CODE, - new Field(BASE_OFFSET_KEY_NAME, INT64), - new Field(LOG_APPEND_TIME_KEY_NAME, INT64, "The timestamp returned by broker after appending " + - "the messages. If CreateTime is used for the topic, the timestamp will be -1. " + - "If LogAppendTime is used for the topic, the timestamp will be the broker local " + - "time when the messages are appended."), - LOG_START_OFFSET_FIELD)))))), - THROTTLE_TIME_MS); - - /** - * The version number is bumped to indicate that on quota violation brokers send out responses before throttling. - */ - private static final Schema PRODUCE_RESPONSE_V6 = PRODUCE_RESPONSE_V5; - - /** - * V7 bumped up to indicate ZStandard capability. (see KIP-110) - */ - private static final Schema PRODUCE_RESPONSE_V7 = PRODUCE_RESPONSE_V6; - - /** - * V8 adds record_errors and error_message. (see KIP-467) - */ - public static final Schema PRODUCE_RESPONSE_V8 = new Schema( - new Field(RESPONSES_KEY_NAME, new ArrayOf(new Schema( - TOPIC_NAME, - new Field(PARTITION_RESPONSES_KEY_NAME, new ArrayOf(new Schema( - PARTITION_ID, - ERROR_CODE, - new Field(BASE_OFFSET_KEY_NAME, INT64), - new Field(LOG_APPEND_TIME_KEY_NAME, INT64, "The timestamp returned by broker after appending " + - "the messages. If CreateTime is used for the topic, the timestamp will be -1. " + - "If LogAppendTime is used for the topic, the timestamp will be the broker local " + - "time when the messages are appended."), - LOG_START_OFFSET_FIELD, - new Field(RECORD_ERRORS_KEY_NAME, new ArrayOf(new Schema( - new Field.Int32(BATCH_INDEX_KEY_NAME, "The batch index of the record " + - "that caused the batch to be dropped"), - BATCH_INDEX_ERROR_MESSAGE_FIELD - )), "The batch indices of records that caused the batch to be dropped"), - ERROR_MESSAGE_FIELD)))))), - THROTTLE_TIME_MS); - - public static Schema[] schemaVersions() { - return new Schema[]{PRODUCE_RESPONSE_V0, PRODUCE_RESPONSE_V1, PRODUCE_RESPONSE_V2, PRODUCE_RESPONSE_V3, - PRODUCE_RESPONSE_V4, PRODUCE_RESPONSE_V5, PRODUCE_RESPONSE_V6, PRODUCE_RESPONSE_V7, PRODUCE_RESPONSE_V8}; + public ProduceResponse(ProduceResponseData produceResponseData) { + this.data = produceResponseData; + this.responses = data.responses() + .stream() + .flatMap(t -> t.partitions() + .stream() + .map(p -> new AbstractMap.SimpleEntry<>(new TopicPartition(t.name(), p.partitionIndex()), + new PartitionResponse( + Errors.forCode(p.errorCode()), + p.baseOffset(), + p.logAppendTimeMs(), + p.logStartOffset(), + p.recordErrors() + .stream() + .map(e -> new RecordError(e.batchIndex(), e.batchIndexErrorMessage())) + .collect(Collectors.toList()), + p.errorMessage())))) + .collect(Collectors.toMap(AbstractMap.SimpleEntry::getKey, AbstractMap.SimpleEntry::getValue)); } - private final Map responses; - private final int throttleTimeMs; - /** * Constructor for Version 0 * @param responses Produced data grouped by topic-partition @@ -205,100 +74,37 @@ public ProduceResponse(Map responses) { */ public ProduceResponse(Map responses, int throttleTimeMs) { this.responses = responses; - this.throttleTimeMs = throttleTimeMs; - } - - /** - * Constructor from a {@link Struct}. - */ - public ProduceResponse(Struct struct) { - responses = new HashMap<>(); - for (Object topicResponse : struct.getArray(RESPONSES_KEY_NAME)) { - Struct topicRespStruct = (Struct) topicResponse; - String topic = topicRespStruct.get(TOPIC_NAME); - - for (Object partResponse : topicRespStruct.getArray(PARTITION_RESPONSES_KEY_NAME)) { - Struct partRespStruct = (Struct) partResponse; - int partition = partRespStruct.get(PARTITION_ID); - Errors error = Errors.forCode(partRespStruct.get(ERROR_CODE)); - long offset = partRespStruct.getLong(BASE_OFFSET_KEY_NAME); - long logAppendTime = partRespStruct.hasField(LOG_APPEND_TIME_KEY_NAME) ? - partRespStruct.getLong(LOG_APPEND_TIME_KEY_NAME) : RecordBatch.NO_TIMESTAMP; - long logStartOffset = partRespStruct.getOrElse(LOG_START_OFFSET_FIELD, INVALID_OFFSET); - - List recordErrors = Collections.emptyList(); - if (partRespStruct.hasField(RECORD_ERRORS_KEY_NAME)) { - Object[] recordErrorsArray = partRespStruct.getArray(RECORD_ERRORS_KEY_NAME); - if (recordErrorsArray.length > 0) { - recordErrors = new ArrayList<>(recordErrorsArray.length); - for (Object indexAndMessage : recordErrorsArray) { - Struct indexAndMessageStruct = (Struct) indexAndMessage; - recordErrors.add(new RecordError( - indexAndMessageStruct.getInt(BATCH_INDEX_KEY_NAME), - indexAndMessageStruct.get(BATCH_INDEX_ERROR_MESSAGE_FIELD) - )); - } - } - } - - String errorMessage = partRespStruct.getOrElse(ERROR_MESSAGE_FIELD, null); - TopicPartition tp = new TopicPartition(topic, partition); - responses.put(tp, new PartitionResponse(error, offset, logAppendTime, logStartOffset, recordErrors, errorMessage)); - } - } - this.throttleTimeMs = struct.getOrElse(THROTTLE_TIME_MS, DEFAULT_THROTTLE_TIME); + this.data = new ProduceResponseData() + .setResponses(responses.entrySet() + .stream() + .collect(Collectors.groupingBy(e -> e.getKey().topic())) + .entrySet() + .stream() + .map(topicData -> new ProduceResponseData.TopicProduceResponse() + .setName(topicData.getKey()) + .setPartitions(topicData.getValue() + .stream() + .map(p -> new ProduceResponseData.PartitionProduceResponse() + .setPartitionIndex(p.getKey().partition()) + .setBaseOffset(p.getValue().baseOffset) + .setLogStartOffset(p.getValue().logStartOffset) + .setLogAppendTimeMs(p.getValue().logAppendTime) + .setErrorMessage(p.getValue().errorMessage) + .setErrorCode(p.getValue().error.code()) + .setRecordErrors(p.getValue().recordErrors + .stream() + .map(e -> new ProduceResponseData.BatchIndexAndErrorMessage() + .setBatchIndex(e.batchIndex) + .setBatchIndexErrorMessage(e.message)) + .collect(Collectors.toList()))) + .collect(Collectors.toList()))) + .collect(Collectors.toList())) + .setThrottleTimeMs(throttleTimeMs); } @Override protected Struct toStruct(short version) { - Struct struct = new Struct(ApiKeys.PRODUCE.responseSchema(version)); - - Map> responseByTopic = CollectionUtils.groupPartitionDataByTopic(responses); - List topicDatas = new ArrayList<>(responseByTopic.size()); - for (Map.Entry> entry : responseByTopic.entrySet()) { - Struct topicData = struct.instance(RESPONSES_KEY_NAME); - topicData.set(TOPIC_NAME, entry.getKey()); - List partitionArray = new ArrayList<>(); - for (Map.Entry partitionEntry : entry.getValue().entrySet()) { - PartitionResponse part = partitionEntry.getValue(); - short errorCode = part.error.code(); - // If producer sends ProduceRequest V3 or earlier, the client library is not guaranteed to recognize the error code - // for KafkaStorageException. In this case the client library will translate KafkaStorageException to - // UnknownServerException which is not retriable. We can ensure that producer will update metadata and retry - // by converting the KafkaStorageException to NotLeaderOrFollowerException in the response if ProduceRequest version <= 3 - if (errorCode == Errors.KAFKA_STORAGE_ERROR.code() && version <= 3) - errorCode = Errors.NOT_LEADER_OR_FOLLOWER.code(); - Struct partStruct = topicData.instance(PARTITION_RESPONSES_KEY_NAME) - .set(PARTITION_ID, partitionEntry.getKey()) - .set(ERROR_CODE, errorCode) - .set(BASE_OFFSET_KEY_NAME, part.baseOffset); - partStruct.setIfExists(LOG_APPEND_TIME_KEY_NAME, part.logAppendTime); - partStruct.setIfExists(LOG_START_OFFSET_FIELD, part.logStartOffset); - - if (partStruct.hasField(RECORD_ERRORS_KEY_NAME)) { - List recordErrors = Collections.emptyList(); - if (!part.recordErrors.isEmpty()) { - recordErrors = new ArrayList<>(); - for (RecordError indexAndMessage : part.recordErrors) { - Struct indexAndMessageStruct = partStruct.instance(RECORD_ERRORS_KEY_NAME) - .set(BATCH_INDEX_KEY_NAME, indexAndMessage.batchIndex) - .set(BATCH_INDEX_ERROR_MESSAGE_FIELD, indexAndMessage.message); - recordErrors.add(indexAndMessageStruct); - } - } - partStruct.set(RECORD_ERRORS_KEY_NAME, recordErrors.toArray()); - } - - partStruct.setIfExists(ERROR_MESSAGE_FIELD, part.errorMessage); - partitionArray.add(partStruct); - } - topicData.set(PARTITION_RESPONSES_KEY_NAME, partitionArray.toArray()); - topicDatas.add(topicData); - } - struct.set(RESPONSES_KEY_NAME, topicDatas.toArray()); - struct.setIfExists(THROTTLE_TIME_MS, throttleTimeMs); - - return struct; + return data.toStruct(version); } public Map responses() { @@ -307,7 +113,7 @@ public Map responses() { @Override public int throttleTimeMs() { - return this.throttleTimeMs; + return this.data.throttleTimeMs(); } @Override @@ -390,7 +196,7 @@ public RecordError(int batchIndex) { } public static ProduceResponse parse(ByteBuffer buffer, short version) { - return new ProduceResponse(ApiKeys.PRODUCE.responseSchema(version).read(buffer)); + return new ProduceResponse(new ProduceResponseData(new ByteBufferAccessor(buffer), version)); } @Override diff --git a/clients/src/main/resources/common/message/ProduceResponse.json b/clients/src/main/resources/common/message/ProduceResponse.json index 77ab0655be57f..770313aff5061 100644 --- a/clients/src/main/resources/common/message/ProduceResponse.json +++ b/clients/src/main/resources/common/message/ProduceResponse.json @@ -60,7 +60,7 @@ "about": "The global error message summarizing the common root cause of the records that caused the batch to be dropped"} ]} ]}, - { "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true, + { "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true, "default": "0", "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." } ] } diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java index 23edfcc0137f5..67408941c7b98 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -200,6 +200,7 @@ private void checkSimpleRequestResponse(NetworkClient networkClient) { request.apiKey().responseHeaderVersion(PRODUCE.latestVersion())); Struct resp = new Struct(PRODUCE.responseSchema(PRODUCE.latestVersion())); resp.set("responses", new Object[0]); + resp.set(CommonFields.THROTTLE_TIME_MS, 100); Struct responseHeaderStruct = respHeader.toStruct(); int size = responseHeaderStruct.sizeOf() + resp.sizeOf(); ByteBuffer buffer = ByteBuffer.allocate(size); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ProduceResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ProduceResponseTest.java index ea6e99882d200..459f97ecc6489 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/ProduceResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/ProduceResponseTest.java @@ -18,6 +18,7 @@ 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.protocol.types.Struct; @@ -31,6 +32,7 @@ import static org.apache.kafka.common.protocol.ApiKeys.PRODUCE; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; public class ProduceResponseTest { @@ -100,7 +102,7 @@ public void produceResponseRecordErrorsTest() { ProduceResponse response = new ProduceResponse(responseData); Struct struct = response.toStruct(ver); assertEquals("Should use schema version " + ver, ApiKeys.PRODUCE.responseSchema(ver), struct.schema()); - ProduceResponse.PartitionResponse deserialized = new ProduceResponse(struct).responses().get(tp); + ProduceResponse.PartitionResponse deserialized = new ProduceResponse(new ProduceResponseData(struct, ver)).responses().get(tp); if (ver >= 8) { assertEquals(1, deserialized.recordErrors.size()); assertEquals(3, deserialized.recordErrors.get(0).batchIndex); @@ -108,7 +110,7 @@ public void produceResponseRecordErrorsTest() { assertEquals("Produce failed", deserialized.errorMessage); } else { assertEquals(0, deserialized.recordErrors.size()); - assertEquals(null, deserialized.errorMessage); + assertNull(deserialized.errorMessage); } } } From 96e354aacdb37243a6d438fa70844171e49d8185 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Sun, 1 Nov 2020 23:39:05 +0800 Subject: [PATCH 03/15] add compatibility test; benchmark; a bit refactor on server to use protocol data --- .../clients/producer/internals/Sender.java | 19 +++- .../kafka/common/protocol/CommonFields.java | 10 -- .../kafka/common/requests/ProduceRequest.java | 101 +++++------------ .../common/requests/ProduceResponse.java | 106 +++++++++++++----- .../kafka/common/requests/RequestUtils.java | 30 ++++- .../common/message/ProduceRequest.json | 10 +- .../kafka/clients/NetworkClientTest.java | 20 ++-- .../producer/internals/SenderTest.java | 39 ++++--- .../internals/TransactionManagerTest.java | 11 +- .../common/requests/ProduceRequestTest.java | 64 +++++++++-- .../common/requests/ProduceResponseTest.java | 84 ++++++++++++++ .../common/requests/RequestResponseTest.java | 10 +- .../common/requests/SchemaTestUtils.java | 55 +++++++++ .../main/scala/kafka/server/KafkaApis.scala | 18 +-- .../producer/ProducerRequestBenchmark.java | 95 ++++++++++++++++ .../producer/ProducerResponseBenchmark.java | 82 ++++++++++++++ 16 files changed, 587 insertions(+), 167 deletions(-) create mode 100644 clients/src/test/java/org/apache/kafka/common/requests/SchemaTestUtils.java create mode 100644 jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerRequestBenchmark.java create mode 100644 jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerResponseBenchmark.java diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index eab77bdb88a07..f5a7908de3f49 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -59,6 +59,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.stream.Collectors; import static org.apache.kafka.common.record.RecordBatch.NO_TIMESTAMP; @@ -560,13 +561,23 @@ private void handleProduceResponse(ClientResponse response, Map entry : produceResponse.responses().entrySet()) { - TopicPartition tp = entry.getKey(); - ProduceResponse.PartitionResponse partResp = entry.getValue(); + produceResponse.data().responses().forEach(r -> r.partitions().forEach(p -> { + TopicPartition tp = new TopicPartition(r.name(), p.partitionIndex()); + ProduceResponse.PartitionResponse partResp = new ProduceResponse.PartitionResponse( + Errors.forCode(p.errorCode()), + p.baseOffset(), + p.logAppendTimeMs(), + p.logStartOffset(), + p.recordErrors() + .stream() + .map(e -> new ProduceResponse.RecordError(e.batchIndex(), e.batchIndexErrorMessage())) + .collect(Collectors.toList()), + p.errorMessage()); ProducerBatch batch = batches.get(tp); completeBatch(batch, partResp, correlationId, now); - } + })); this.sensors.recordLatency(response.destination(), response.requestLatencyMs()); } else { // this is the acks = 0 case, just complete all requests diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/CommonFields.java b/clients/src/main/java/org/apache/kafka/common/protocol/CommonFields.java index fe756a8597e7a..122d4da602324 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/CommonFields.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/CommonFields.java @@ -32,14 +32,4 @@ public class CommonFields { "If the epoch provided by the client is larger than the current epoch known to the broker, then " + "the UNKNOWN_LEADER_EPOCH error code will be returned. If the provided epoch is smaller, then " + "the FENCED_LEADER_EPOCH error code will be returned."); - - // Group APIs - public static final Field.Str GROUP_ID = new Field.Str("group_id", "The unique group identifier"); - - // Transactional APIs - public static final Field.Str TRANSACTIONAL_ID = new Field.Str("transactional_id", "The transactional id corresponding to the transaction."); - public static final Field.NullableStr NULLABLE_TRANSACTIONAL_ID = new Field.NullableStr("transactional_id", - "The transactional id or null if the producer is not transactional"); - public static final Field.Int64 PRODUCER_ID = new Field.Int64("producer_id", "Current producer id in use by the transactional id."); - public static final Field.Int16 PRODUCER_EPOCH = new Field.Int16("producer_epoch", "Current epoch associated with the producer id."); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java index b1b4000f8e1b5..5288eb6d05782 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -20,25 +20,24 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.UnsupportedCompressionTypeException; import org.apache.kafka.common.message.ProduceRequestData; -import org.apache.kafka.common.message.ProduceResponseData; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.MemoryRecords; -import org.apache.kafka.common.record.MutableRecordBatch; import org.apache.kafka.common.record.RecordBatch; +import org.apache.kafka.common.record.Records; import org.apache.kafka.common.utils.Utils; import java.nio.ByteBuffer; import java.util.AbstractMap; import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.function.Function; import java.util.stream.Collectors; public class ProduceRequest extends AbstractRequest { @@ -114,11 +113,11 @@ private ProduceRequest build(short version, boolean validate) { .entrySet() .stream() .map(e -> new ProduceRequestData.TopicProduceData() - .setName(e.getKey()) - .setPartitions(e.getValue().stream() + .setTopic(e.getKey()) + .setData(e.getValue().stream() .map(tpAndRecord -> new ProduceRequestData.PartitionProduceData() - .setPartitionIndex(tpAndRecord.getKey().partition()) - .setRecords(tpAndRecord.getValue().buffer())) + .setPartition(tpAndRecord.getKey().partition()) + .setRecordSet(tpAndRecord.getValue())) .collect(Collectors.toList()))) .collect(Collectors.toList()); @@ -151,8 +150,6 @@ public String toString() { private final String transactionalId; // visible for testing final Map partitionSizes; - private boolean hasTransactionalRecords = false; - private boolean hasIdempotentRecords = false; // This is set to null by `clearPartitionRecords` to prevent unnecessary memory retention when a produce request is // put in the purgatory (due to client throttling, it can take a while before the response is sent). // Care should be taken in methods that use this field. @@ -161,35 +158,34 @@ public String toString() { public ProduceRequest(ProduceRequestData produceRequestData, short version) { super(ApiKeys.PRODUCE, version); this.data = produceRequestData; - this.data.topicData().forEach(topicProduceData -> topicProduceData.partitions() - .forEach(partitionProduceData -> { - MemoryRecords records = MemoryRecords.readableRecords(partitionProduceData.records()); - Iterator iterator = records.batches().iterator(); - MutableRecordBatch entry = iterator.next(); - hasIdempotentRecords = hasIdempotentRecords || entry.hasProducerId(); - hasTransactionalRecords = hasTransactionalRecords || entry.isTransactional(); - })); this.acks = data.acks(); this.timeout = data.timeout(); this.transactionalId = data.transactionalId(); this.partitionSizes = data.topicData() .stream() - .flatMap(e -> e.partitions() + .flatMap(e -> e.data() .stream() - .map(p -> new AbstractMap.SimpleEntry<>(new TopicPartition(e.name(), p.partitionIndex()), p.records().limit()))) + .map(p -> new AbstractMap.SimpleEntry<>(new TopicPartition(e.topic(), p.partition()), p.recordSet().sizeInBytes()))) .collect(Collectors.groupingBy(AbstractMap.SimpleEntry::getKey, Collectors.summingInt(AbstractMap.SimpleEntry::getValue))); } /** - * Visible for testing. + * @return data or IllegalStateException if the data is removed (to prevent unnecessary memory retention). */ - @Override - public Struct toStruct() { + public ProduceRequestData dataOrException() { // Store it in a local variable to protect against concurrent updates ProduceRequestData tmp = data; if (tmp == null) throw new IllegalStateException("The partition records are no longer available because clearPartitionRecords() has been invoked."); - return tmp.toStruct(version()); + return tmp; + } + + /** + * Visible for testing. + */ + @Override + public Struct toStruct() { + return dataOrException().toStruct(version()); } @Override @@ -211,29 +207,11 @@ public String toString(boolean verbose) { @Override public ProduceResponse getErrorResponse(int throttleTimeMs, Throwable e) { /* In case the producer doesn't actually want any response */ - if (acks == 0) - return null; - - Errors error = Errors.forException(e); - return new ProduceResponse(new ProduceResponseData() - .setResponses(partitionSizes.keySet() - .stream() - .collect(Collectors.groupingBy(TopicPartition::topic)) - .entrySet() - .stream() - .map(tp -> new ProduceResponseData.TopicProduceResponse() - .setName(tp.getKey()) - .setPartitions(tp.getValue().stream().map(p -> new ProduceResponseData.PartitionProduceResponse() - .setPartitionIndex(p.partition()) - .setBaseOffset(ProduceResponse.INVALID_OFFSET) - .setErrorCode(error.code()) - .setErrorMessage(error.message()) - .setLogAppendTimeMs(RecordBatch.NO_TIMESTAMP) - .setLogStartOffset(ProduceResponse.INVALID_OFFSET) - .setRecordErrors(Collections.emptyList())) - .collect(Collectors.toList()))) - .collect(Collectors.toList())) - .setThrottleTimeMs(throttleTimeMs)); + if (acks == 0) return null; + ProduceResponse.PartitionResponse partitionResponse = new ProduceResponse.PartitionResponse(Errors.forException(e)); + return new ProduceResponse(partitions() + .stream() + .collect(Collectors.toMap(Function.identity(), ignored -> partitionResponse)), throttleTimeMs); } @Override @@ -258,43 +236,18 @@ public String transactionalId() { return transactionalId; } - public boolean hasTransactionalRecords() { - return hasTransactionalRecords; - } - - public boolean hasIdempotentRecords() { - return hasIdempotentRecords; - } - - /** - * convert the generated data to Map. - * Noted that the cost of conversion can be expensive so caller should keep and reuse the returned collection. - */ - public Map partitionRecordsOrFail() { - // Store it in a local variable to protect against concurrent updates - ProduceRequestData tmp = data; - if (tmp == null) - throw new IllegalStateException("The partition records are no longer available because clearPartitionRecords() has been invoked."); - Map partitionRecords = new HashMap<>(); - tmp.topicData().forEach(tpData -> tpData.partitions().forEach(p -> { - TopicPartition tp = new TopicPartition(tpData.name(), p.partitionIndex()); - partitionRecords.put(tp, MemoryRecords.readableRecords(p.records())); - })); - return Collections.unmodifiableMap(partitionRecords); - } - public void clearPartitionRecords() { data = null; } - public static void validateRecords(short version, MemoryRecords records) { + public static void validateRecords(short version, Records records) { if (version >= 3) { - Iterator iterator = records.batches().iterator(); + Iterator iterator = records.batches().iterator(); if (!iterator.hasNext()) throw new InvalidRecordException("Produce requests with version " + version + " must have at least " + "one record batch"); - MutableRecordBatch entry = iterator.next(); + RecordBatch entry = iterator.next(); if (entry.magic() != RecordBatch.MAGIC_VALUE_V2) throw new InvalidRecordException("Produce requests with version " + version + " are only allowed to " + "contain record batches with magic version 2"); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java index 8cac3ed464ce7..9edd4842f16db 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -29,34 +29,36 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.stream.Collectors; /** * This wrapper supports both v0 and v8 of ProduceResponse. + * + * Possible error code: + * + * {@link Errors#CORRUPT_MESSAGE} + * {@link Errors#UNKNOWN_TOPIC_OR_PARTITION} + * {@link Errors#NOT_LEADER_OR_FOLLOWER} + * {@link Errors#MESSAGE_TOO_LARGE} + * {@link Errors#INVALID_TOPIC_EXCEPTION} + * {@link Errors#RECORD_LIST_TOO_LARGE} + * {@link Errors#NOT_ENOUGH_REPLICAS} + * {@link Errors#NOT_ENOUGH_REPLICAS_AFTER_APPEND} + * {@link Errors#INVALID_REQUIRED_ACKS} + * {@link Errors#TOPIC_AUTHORIZATION_FAILED} + * {@link Errors#UNSUPPORTED_FOR_MESSAGE_FORMAT} + * {@link Errors#INVALID_PRODUCER_EPOCH} + * {@link Errors#CLUSTER_AUTHORIZATION_FAILED} + * {@link Errors#TRANSACTIONAL_ID_AUTHORIZATION_FAILED} + * {@link Errors#INVALID_RECORD} */ public class ProduceResponse extends AbstractResponse { public static final long INVALID_OFFSET = -1L; private final ProduceResponseData data; - private final Map responses; public ProduceResponse(ProduceResponseData produceResponseData) { this.data = produceResponseData; - this.responses = data.responses() - .stream() - .flatMap(t -> t.partitions() - .stream() - .map(p -> new AbstractMap.SimpleEntry<>(new TopicPartition(t.name(), p.partitionIndex()), - new PartitionResponse( - Errors.forCode(p.errorCode()), - p.baseOffset(), - p.logAppendTimeMs(), - p.logStartOffset(), - p.recordErrors() - .stream() - .map(e -> new RecordError(e.batchIndex(), e.batchIndexErrorMessage())) - .collect(Collectors.toList()), - p.errorMessage())))) - .collect(Collectors.toMap(AbstractMap.SimpleEntry::getKey, AbstractMap.SimpleEntry::getValue)); } /** @@ -73,8 +75,7 @@ public ProduceResponse(Map responses) { * @param throttleTimeMs Time in milliseconds the response was throttled */ public ProduceResponse(Map responses, int throttleTimeMs) { - this.responses = responses; - this.data = new ProduceResponseData() + this(new ProduceResponseData() .setResponses(responses.entrySet() .stream() .collect(Collectors.groupingBy(e -> e.getKey().topic())) @@ -99,16 +100,42 @@ public ProduceResponse(Map responses, int thr .collect(Collectors.toList()))) .collect(Collectors.toList()))) .collect(Collectors.toList())) - .setThrottleTimeMs(throttleTimeMs); + .setThrottleTimeMs(throttleTimeMs)); } + /** + * Visible for testing. + */ @Override - protected Struct toStruct(short version) { + public Struct toStruct(short version) { return data.toStruct(version); } + public ProduceResponseData data() { + return this.data; + } + + /** + * this method is used by testing only. + * TODO: refactor the tests which are using this method and then remove this method from production code. + */ public Map responses() { - return this.responses; + return data.responses() + .stream() + .flatMap(t -> t.partitions() + .stream() + .map(p -> new AbstractMap.SimpleEntry<>(new TopicPartition(t.name(), p.partitionIndex()), + new PartitionResponse( + Errors.forCode(p.errorCode()), + p.baseOffset(), + p.logAppendTimeMs(), + p.logStartOffset(), + p.recordErrors() + .stream() + .map(e -> new RecordError(e.batchIndex(), e.batchIndexErrorMessage())) + .collect(Collectors.toList()), + p.errorMessage())))) + .collect(Collectors.toMap(AbstractMap.SimpleEntry::getKey, AbstractMap.SimpleEntry::getValue)); } @Override @@ -119,9 +146,7 @@ public int throttleTimeMs() { @Override public Map errorCounts() { Map errorCounts = new HashMap<>(); - responses.values().forEach(response -> - updateErrorCounts(errorCounts, response.error) - ); + data.responses().forEach(t -> t.partitions().forEach(p -> updateErrorCounts(errorCounts, Errors.forCode(p.errorCode())))); return errorCounts; } @@ -154,6 +179,24 @@ public PartitionResponse(Errors error, long baseOffset, long logAppendTime, long this.errorMessage = errorMessage; } + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + PartitionResponse that = (PartitionResponse) o; + return baseOffset == that.baseOffset && + logAppendTime == that.logAppendTime && + logStartOffset == that.logStartOffset && + error == that.error && + Objects.equals(recordErrors, that.recordErrors) && + Objects.equals(errorMessage, that.errorMessage); + } + + @Override + public int hashCode() { + return Objects.hash(error, baseOffset, logAppendTime, logStartOffset, recordErrors, errorMessage); + } + @Override public String toString() { StringBuilder b = new StringBuilder(); @@ -193,6 +236,19 @@ public RecordError(int batchIndex) { this.message = null; } + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + RecordError that = (RecordError) o; + return batchIndex == that.batchIndex && + Objects.equals(message, that.message); + } + + @Override + public int hashCode() { + return Objects.hash(batchIndex, message); + } } public static ProduceResponse parse(ByteBuffer buffer, short version) { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/RequestUtils.java b/clients/src/main/java/org/apache/kafka/common/requests/RequestUtils.java index be72edbd8ef9c..ae1d1df14fe1d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/RequestUtils.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/RequestUtils.java @@ -18,10 +18,14 @@ import org.apache.kafka.common.protocol.types.Field; import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.record.BaseRecords; import org.apache.kafka.common.record.RecordBatch; +import org.apache.kafka.common.record.Records; import java.nio.ByteBuffer; +import java.util.Iterator; import java.util.Optional; +import java.util.function.Predicate; public final class RequestUtils { @@ -48,4 +52,28 @@ public static ByteBuffer serialize(Struct headerStruct, Struct bodyStruct) { buffer.rewind(); return buffer; } -} + + public static boolean hasIdempotentRecords(ProduceRequest request) { + return anyMatch(request, RecordBatch::hasProducerId); + } + + public static boolean hasTransactionalRecords(ProduceRequest request) { + return anyMatch(request, RecordBatch::isTransactional); + } + + private static boolean anyMatch(ProduceRequest request, Predicate predicate) { + return request.dataOrException().topicData() + .stream() + .anyMatch(topicProduceData -> topicProduceData.data() + .stream() + .anyMatch(partitionProduceData -> { + BaseRecords records = partitionProduceData.recordSet(); + if (records instanceof Records) { + Iterator iterator = ((Records) records).batches().iterator(); + return iterator.hasNext() && predicate.test(iterator.next()); + } else return false; + })); + } + + +} \ No newline at end of file diff --git a/clients/src/main/resources/common/message/ProduceRequest.json b/clients/src/main/resources/common/message/ProduceRequest.json index cede9677db47b..7fbde3f30030c 100644 --- a/clients/src/main/resources/common/message/ProduceRequest.json +++ b/clients/src/main/resources/common/message/ProduceRequest.json @@ -33,7 +33,7 @@ "validVersions": "0-8", "flexibleVersions": "none", "fields": [ - { "name": "TransactionalId", "type": "string", "versions": "3+", "nullableVersions": "3+", "ignorable": true, "entityType": "transactionalId", + { "name": "TransactionalId", "type": "string", "versions": "3+", "nullableVersions": "3+", "default": "null", "entityType": "transactionalId", "about": "The transactional ID, or null if the producer is not transactional." }, { "name": "Acks", "type": "int16", "versions": "0+", "about": "The number of acknowledgments the producer requires the leader to have received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for only the leader and -1 for the full ISR." }, @@ -41,13 +41,13 @@ "about": "The timeout to await a response in miliseconds." }, { "name": "TopicData", "type": "[]TopicProduceData", "versions": "0+", "about": "Each topic to produce to.", "fields": [ - { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", + { "name": "Topic", "type": "string", "versions": "0+", "entityType": "topicName", "about": "The topic name." }, - { "name": "Partitions", "type": "[]PartitionProduceData", "versions": "0+", + { "name": "Data", "type": "[]PartitionProduceData", "versions": "0+", "about": "Each partition to produce to.", "fields": [ - { "name": "PartitionIndex", "type": "int32", "versions": "0+", + { "name": "Partition", "type": "int32", "versions": "0+", "about": "The partition index." }, - { "name": "Records", "type": "bytes", "zeroCopy": true, "versions": "0+", "nullableVersions": "0+", + { "name": "RecordSet", "type": "records", "versions": "0+", "nullableVersions": "0+", "about": "The record data to be produced." } ]} ]} diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java index 67408941c7b98..00597ca2d21ac 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -25,9 +25,9 @@ import org.apache.kafka.common.message.ApiVersionsResponseData; import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersionsResponseKey; import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersionsResponseKeyCollection; +import org.apache.kafka.common.message.ProduceResponseData; import org.apache.kafka.common.network.NetworkReceive; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.CommonFields; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.requests.ApiVersionsResponse; @@ -198,9 +198,9 @@ private void checkSimpleRequestResponse(NetworkClient networkClient) { ResponseHeader respHeader = new ResponseHeader(request.correlationId(), request.apiKey().responseHeaderVersion(PRODUCE.latestVersion())); - Struct resp = new Struct(PRODUCE.responseSchema(PRODUCE.latestVersion())); - resp.set("responses", new Object[0]); - resp.set(CommonFields.THROTTLE_TIME_MS, 100); + Struct resp = new ProduceResponseData() + .setThrottleTimeMs(100) + .toStruct(ProduceResponseData.HIGHEST_SUPPORTED_VERSION); Struct responseHeaderStruct = respHeader.toStruct(); int size = responseHeaderStruct.sizeOf() + resp.sizeOf(); ByteBuffer buffer = ByteBuffer.allocate(size); @@ -511,9 +511,9 @@ public void testConnectionThrottling() { ResponseHeader respHeader = new ResponseHeader(request.correlationId(), request.apiKey().responseHeaderVersion(PRODUCE.latestVersion())); - Struct resp = new Struct(PRODUCE.responseSchema(PRODUCE.latestVersion())); - resp.set("responses", new Object[0]); - resp.set(CommonFields.THROTTLE_TIME_MS, 100); + Struct resp = new ProduceResponseData() + .setThrottleTimeMs(100) + .toStruct(ProduceResponseData.HIGHEST_SUPPORTED_VERSION); Struct responseHeaderStruct = respHeader.toStruct(); int size = responseHeaderStruct.sizeOf() + resp.sizeOf(); ByteBuffer buffer = ByteBuffer.allocate(size); @@ -607,9 +607,9 @@ private void sendResponse(ResponseHeader respHeader, Struct response) { } private void sendThrottledProduceResponse(int correlationId, int throttleMs) { - Struct resp = new Struct(PRODUCE.responseSchema(PRODUCE.latestVersion())); - resp.set("responses", new Object[0]); - resp.set(CommonFields.THROTTLE_TIME_MS, throttleMs); + Struct resp = new ProduceResponseData() + .setThrottleTimeMs(throttleMs) + .toStruct(ProduceResponseData.HIGHEST_SUPPORTED_VERSION); sendResponse(new ResponseHeader(correlationId, PRODUCE.responseHeaderVersion(PRODUCE.latestVersion())), resp); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java index d5302595cee29..2c9172050c39f 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java @@ -28,6 +28,7 @@ import org.apache.kafka.common.errors.TransactionAbortedException; import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType; import org.apache.kafka.common.requests.MetadataRequest; +import org.apache.kafka.common.requests.RequestUtils; import org.apache.kafka.common.utils.ProducerIdAndEpoch; import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.Cluster; @@ -156,6 +157,15 @@ public void tearDown() { this.metrics.close(); } + private static Map partitionRecords(ProduceRequest request) { + Map partitionRecords = new HashMap<>(); + request.dataOrException().topicData().forEach(tpData -> tpData.data().forEach(p -> { + TopicPartition tp = new TopicPartition(tpData.topic(), p.partition()); + partitionRecords.put(tp, (MemoryRecords) p.recordSet()); + })); + return Collections.unmodifiableMap(partitionRecords); + } + @Test public void testSimple() throws Exception { long offset = 0; @@ -195,7 +205,7 @@ public void testMessageFormatDownConversion() throws Exception { if (request.version() != 2) return false; - MemoryRecords records = request.partitionRecordsOrFail().get(tp0); + MemoryRecords records = partitionRecords(request).get(tp0); return records != null && records.sizeInBytes() > 0 && records.hasMatchingMagic(RecordBatch.MAGIC_VALUE_V1); @@ -241,7 +251,7 @@ public void testDownConversionForMismatchedMagicValues() throws Exception { if (request.version() != 2) return false; - Map recordsMap = request.partitionRecordsOrFail(); + Map recordsMap = partitionRecords(request); if (recordsMap.size() != 2) return false; @@ -648,7 +658,7 @@ public void testCanRetryWithoutIdempotence() throws Exception { client.respond(body -> { ProduceRequest request = (ProduceRequest) body; - assertFalse(request.hasIdempotentRecords()); + assertFalse(RequestUtils.hasIdempotentRecords(request)); return true; }, produceResponse(tp0, -1L, Errors.TOPIC_AUTHORIZATION_FAILED, 0)); sender.runOnce(); @@ -1806,9 +1816,9 @@ void sendIdempotentProducerResponse(int expectedSequence, TopicPartition tp, Err void sendIdempotentProducerResponse(final int expectedSequence, TopicPartition tp, Errors responseError, long responseOffset, long logStartOffset) { client.respond(body -> { ProduceRequest produceRequest = (ProduceRequest) body; - assertTrue(produceRequest.hasIdempotentRecords()); + assertTrue(RequestUtils.hasIdempotentRecords(produceRequest)); - MemoryRecords records = produceRequest.partitionRecordsOrFail().get(tp0); + MemoryRecords records = partitionRecords(produceRequest).get(tp0); Iterator batchIterator = records.batches().iterator(); RecordBatch firstBatch = batchIterator.next(); assertFalse(batchIterator.hasNext()); @@ -1829,8 +1839,7 @@ public void testClusterAuthorizationExceptionInProduceRequest() throws Exception // cluster authorization is a fatal error for the producer Future future = appendToAccumulator(tp0); client.prepareResponse( - body -> body instanceof ProduceRequest && - ((ProduceRequest) body).hasIdempotentRecords(), + body -> body instanceof ProduceRequest && RequestUtils.hasIdempotentRecords((ProduceRequest) body), produceResponse(tp0, -1, Errors.CLUSTER_AUTHORIZATION_FAILED, 0)); sender.runOnce(); @@ -1858,8 +1867,7 @@ public void testCancelInFlightRequestAfterFatalError() throws Exception { sender.runOnce(); client.respond( - body -> body instanceof ProduceRequest && - ((ProduceRequest) body).hasIdempotentRecords(), + body -> body instanceof ProduceRequest && RequestUtils.hasIdempotentRecords((ProduceRequest) body), produceResponse(tp0, -1, Errors.CLUSTER_AUTHORIZATION_FAILED, 0)); sender.runOnce(); @@ -1871,8 +1879,7 @@ public void testCancelInFlightRequestAfterFatalError() throws Exception { // Should be fine if the second response eventually returns client.respond( - body -> body instanceof ProduceRequest && - ((ProduceRequest) body).hasIdempotentRecords(), + body -> body instanceof ProduceRequest && RequestUtils.hasIdempotentRecords((ProduceRequest) body), produceResponse(tp1, 0, Errors.NONE, 0)); sender.runOnce(); } @@ -1888,8 +1895,7 @@ public void testUnsupportedForMessageFormatInProduceRequest() throws Exception { Future future = appendToAccumulator(tp0); client.prepareResponse( - body -> body instanceof ProduceRequest && - ((ProduceRequest) body).hasIdempotentRecords(), + body -> body instanceof ProduceRequest && RequestUtils.hasIdempotentRecords((ProduceRequest) body), produceResponse(tp0, -1, Errors.UNSUPPORTED_FOR_MESSAGE_FORMAT, 0)); sender.runOnce(); @@ -1910,8 +1916,7 @@ public void testUnsupportedVersionInProduceRequest() throws Exception { Future future = appendToAccumulator(tp0); client.prepareUnsupportedVersionResponse( - body -> body instanceof ProduceRequest && - ((ProduceRequest) body).hasIdempotentRecords()); + body -> body instanceof ProduceRequest && RequestUtils.hasIdempotentRecords((ProduceRequest) body)); sender.runOnce(); assertFutureFailure(future, UnsupportedVersionException.class); @@ -1940,7 +1945,7 @@ public void testSequenceNumberIncrement() throws InterruptedException { client.prepareResponse(body -> { if (body instanceof ProduceRequest) { ProduceRequest request = (ProduceRequest) body; - MemoryRecords records = request.partitionRecordsOrFail().get(tp0); + MemoryRecords records = partitionRecords(request).get(tp0); Iterator batchIterator = records.batches().iterator(); assertTrue(batchIterator.hasNext()); RecordBatch batch = batchIterator.next(); @@ -2601,7 +2606,7 @@ private MockClient.RequestMatcher produceRequestMatcher(final TopicPartition tp, return false; ProduceRequest request = (ProduceRequest) body; - Map recordsMap = request.partitionRecordsOrFail(); + Map recordsMap = partitionRecords(request); MemoryRecords records = recordsMap.get(tp); if (records == null) return false; diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java index ee3f8925e8636..8fb3850683b4a 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java @@ -3356,7 +3356,16 @@ private void prepareProduceResponse(Errors error, final long producerId, final s private MockClient.RequestMatcher produceRequestMatcher(final long producerId, final short epoch, TopicPartition tp) { return body -> { ProduceRequest produceRequest = (ProduceRequest) body; - MemoryRecords records = produceRequest.partitionRecordsOrFail().get(tp); + MemoryRecords records = produceRequest.dataOrException().topicData() + .stream() + .filter(t -> t.topic().equals(tp.topic())) + .findAny() + .get() + .data() + .stream() + .filter(p -> p.partition() == tp.partition()) + .map(p -> (MemoryRecords) p.recordSet()) + .findAny().get(); assertNotNull(records); Iterator batchIterator = records.batches().iterator(); assertTrue(batchIterator.hasNext()); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java index 95d719adb3bb7..7700bb317bbf8 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java @@ -19,7 +19,11 @@ import org.apache.kafka.common.InvalidRecordException; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.message.ProduceRequestData; import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.types.ArrayOf; +import org.apache.kafka.common.protocol.types.Field; +import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecordsBuilder; @@ -35,6 +39,11 @@ import java.util.LinkedHashMap; import java.util.Map; +import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID; +import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME; +import static org.apache.kafka.common.protocol.types.Type.INT16; +import static org.apache.kafka.common.protocol.types.Type.INT32; +import static org.apache.kafka.common.protocol.types.Type.RECORDS; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -52,19 +61,19 @@ public void shouldBeFlaggedAsTransactionalWhenTransactionalRecords() throws Exce (short) 1, 1, 1, simpleRecord); final ProduceRequest request = ProduceRequest.Builder.forCurrentMagic((short) -1, 10, Collections.singletonMap(new TopicPartition("topic", 1), memoryRecords)).build(); - assertTrue(request.hasTransactionalRecords()); + assertTrue(RequestUtils.hasTransactionalRecords(request)); } @Test public void shouldNotBeFlaggedAsTransactionalWhenNoRecords() throws Exception { final ProduceRequest request = createNonIdempotentNonTransactionalRecords(); - assertFalse(request.hasTransactionalRecords()); + assertFalse(RequestUtils.hasTransactionalRecords(request)); } @Test public void shouldNotBeFlaggedAsIdempotentWhenRecordsNotIdempotent() throws Exception { final ProduceRequest request = createNonIdempotentNonTransactionalRecords(); - assertFalse(request.hasTransactionalRecords()); + assertFalse(RequestUtils.hasTransactionalRecords(request)); } @Test @@ -73,7 +82,7 @@ public void shouldBeFlaggedAsIdempotentWhenIdempotentRecords() throws Exception (short) 1, 1, 1, simpleRecord); final ProduceRequest request = ProduceRequest.Builder.forCurrentMagic((short) -1, 10, Collections.singletonMap(new TopicPartition("topic", 1), memoryRecords)).build(); - assertTrue(request.hasIdempotentRecords()); + assertTrue(RequestUtils.hasIdempotentRecords(request)); } @Test @@ -200,8 +209,8 @@ public void testMixedTransactionalData() { recordsByPartition, transactionalId); final ProduceRequest request = builder.build(); - assertTrue(request.hasTransactionalRecords()); - assertTrue(request.hasIdempotentRecords()); + assertTrue(RequestUtils.hasTransactionalRecords(request)); + assertTrue(RequestUtils.hasIdempotentRecords(request)); } @Test @@ -223,8 +232,8 @@ public void testMixedIdempotentData() { recordsByPartition, null); final ProduceRequest request = builder.build(); - assertFalse(request.hasTransactionalRecords()); - assertTrue(request.hasIdempotentRecords()); + assertFalse(RequestUtils.hasTransactionalRecords(request)); + assertTrue(RequestUtils.hasIdempotentRecords(request)); } private void assertThrowsInvalidRecordExceptionForAllVersions(ProduceRequest.Builder builder) { @@ -248,4 +257,43 @@ private ProduceRequest createNonIdempotentNonTransactionalRecords() { return ProduceRequest.Builder.forCurrentMagic((short) -1, 10, Collections.singletonMap(new TopicPartition("topic", 1), memoryRecords)).build(); } + + /** + * the schema in this test is from previous code and the automatic protocol should be compatible to previous schema. + */ + @Test + public void testCompatibility() { + String acksKeyName = "acks"; + String timeoutKeyName = "timeout"; + String topicDataKeyName = "topic_data"; + String partitionDataKeyName = "data"; + String recordSetKeyName = "record_set"; + Schema topicProduceData0 = new Schema(TOPIC_NAME, + new Field(partitionDataKeyName, new ArrayOf(new Schema(PARTITION_ID, new Field(recordSetKeyName, RECORDS))))); + Schema produceRequestV0 = new Schema( + new Field(acksKeyName, INT16), + new Field(timeoutKeyName, INT32), + new Field(topicDataKeyName, new ArrayOf(topicProduceData0))); + Schema produceRequestV1 = produceRequestV0; + Schema produceRequestV2 = produceRequestV1; + Schema produceRequestV3 = new Schema( + new Field.NullableStr("transactional_id", "The transactional id or null if the producer is not transactional"), + new Field(acksKeyName, INT16), + new Field(timeoutKeyName, INT32), + new Field(topicDataKeyName, new ArrayOf(topicProduceData0))); + Schema produceRequestV4 = produceRequestV3; + Schema produceRequestV5 = produceRequestV4; + Schema produceRequestV6 = produceRequestV5; + Schema produceRequestV7 = produceRequestV6; + Schema produceRequestV8 = produceRequestV7; + Schema[] schemaVersions = new Schema[] { + produceRequestV0, produceRequestV1, produceRequestV2, + produceRequestV3, produceRequestV4, produceRequestV5, + produceRequestV6, produceRequestV7, produceRequestV8 + }; + int schemaVersion = 0; + for (Schema previousSchema : schemaVersions) { + SchemaTestUtils.assertEquals(previousSchema, ProduceRequestData.SCHEMAS[schemaVersion++]); + } + } } diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ProduceResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ProduceResponseTest.java index 459f97ecc6489..186ab106102a6 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/ProduceResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/ProduceResponseTest.java @@ -21,6 +21,9 @@ import org.apache.kafka.common.message.ProduceResponseData; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.types.ArrayOf; +import org.apache.kafka.common.protocol.types.Field; +import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.record.RecordBatch; import org.junit.Test; @@ -31,6 +34,11 @@ import java.util.Map; import static org.apache.kafka.common.protocol.ApiKeys.PRODUCE; +import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE; +import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID; +import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS; +import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME; +import static org.apache.kafka.common.protocol.types.Type.INT64; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; @@ -114,4 +122,80 @@ public void produceResponseRecordErrorsTest() { } } } + + /** + * the schema in this test is from previous code and the automatic protocol should be compatible to previous schema. + */ + @Test + public void testCompatibility() { + String responseKeyName = "responses"; + String partitionResponsesKeyName = "partition_responses"; + long invalidOffset = -1L; + String baseOffsetKeyName = "base_offset"; + String logAppendTimeKeyName = "log_append_time"; + String logStartOffsetKeyName = "log_start_offset"; + String recordErrorsKeyName = "record_errors"; + String batchIndexKeyName = "batch_index"; + String batchIndexErrorMessageKeyName = "batch_index_error_message"; + String errorMessageKeyName = "error_message"; + + Field.Int64 logStartOffsetField = new Field.Int64(logStartOffsetKeyName, + "The start offset of the log at the time this produce response was created", invalidOffset); + Field.NullableStr batchIndexErrorMessageField = new Field.NullableStr(batchIndexErrorMessageKeyName, + "The error message of the record that caused the batch to be dropped"); + Field.NullableStr errorMessageField = new Field.NullableStr(errorMessageKeyName, + "The global error message summarizing the common root cause of the records that caused the batch to be dropped"); + + Schema produceResponseV0 = new Schema( + new Field(responseKeyName, new ArrayOf(new Schema(TOPIC_NAME, + new Field(partitionResponsesKeyName, new ArrayOf(new Schema(PARTITION_ID, ERROR_CODE, + new Field(baseOffsetKeyName, INT64)))))))); + + Schema produceResponseV1 = new Schema( + new Field(responseKeyName, new ArrayOf(new Schema(TOPIC_NAME, + new Field(partitionResponsesKeyName, new ArrayOf(new Schema(PARTITION_ID, ERROR_CODE, + new Field(baseOffsetKeyName, INT64))))))), + THROTTLE_TIME_MS); + + Schema produceResponseV2 = new Schema( + new Field(responseKeyName, new ArrayOf(new Schema(TOPIC_NAME, + new Field(partitionResponsesKeyName, new ArrayOf(new Schema(PARTITION_ID, ERROR_CODE, + new Field(baseOffsetKeyName, INT64), + new Field(logAppendTimeKeyName, INT64))))))), + THROTTLE_TIME_MS); + Schema produceResponseV3 = produceResponseV2; + Schema produceResponseV4 = produceResponseV3; + Schema produceResponseV5 = new Schema( + new Field(responseKeyName, new ArrayOf(new Schema(TOPIC_NAME, + new Field(partitionResponsesKeyName, new ArrayOf(new Schema(PARTITION_ID, ERROR_CODE, + new Field(baseOffsetKeyName, INT64), + new Field(logAppendTimeKeyName, INT64), + logStartOffsetField)))))), + THROTTLE_TIME_MS); + Schema produceResponseV6 = produceResponseV5; + Schema produceResponseV7 = produceResponseV6; + Schema produceResponseV8 = new Schema( + new Field(responseKeyName, new ArrayOf(new Schema(TOPIC_NAME, + new Field(partitionResponsesKeyName, new ArrayOf(new Schema(PARTITION_ID, ERROR_CODE, + new Field(baseOffsetKeyName, INT64), + new Field(logAppendTimeKeyName, INT64), + logStartOffsetField, + new Field(recordErrorsKeyName, new ArrayOf(new Schema( + new Field.Int32(batchIndexKeyName, "The batch index of the record " + + "that caused the batch to be dropped"), + batchIndexErrorMessageField + ))), + errorMessageField)))))), + THROTTLE_TIME_MS); + + Schema[] schemaVersions = new Schema[]{ + produceResponseV0, produceResponseV1, produceResponseV2, + produceResponseV3, produceResponseV4, produceResponseV5, + produceResponseV6, produceResponseV7, produceResponseV8}; + + int schemaVersion = 0; + for (Schema previousSchema : schemaVersions) { + SchemaTestUtils.assertEquals(previousSchema, ProduceResponseData.SCHEMAS[schemaVersion++]); + } + } } diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index e2bdb9542710e..880dc037e1f26 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -646,7 +646,7 @@ public void testPartitionSize() { @Test public void produceRequestToStringTest() { ProduceRequest request = createProduceRequest(ApiKeys.PRODUCE.latestVersion()); - assertEquals(1, request.partitionRecordsOrFail().size()); + assertEquals(1, request.dataOrException().topicData().size()); assertFalse(request.toString(false).contains("partitionSizes")); assertTrue(request.toString(false).contains("numPartitions=1")); assertTrue(request.toString(true).contains("partitionSizes")); @@ -654,8 +654,8 @@ public void produceRequestToStringTest() { request.clearPartitionRecords(); try { - request.partitionRecordsOrFail(); - fail("partitionRecordsOrFail should fail after clearPartitionRecords()"); + request.dataOrException(); + fail("dataOrException should fail after clearPartitionRecords()"); } catch (IllegalStateException e) { // OK } @@ -670,7 +670,7 @@ public void produceRequestToStringTest() { @Test public void produceRequestGetErrorResponseTest() { ProduceRequest request = createProduceRequest(ApiKeys.PRODUCE.latestVersion()); - Set partitions = new HashSet<>(request.partitionRecordsOrFail().keySet()); + Set partitions = new HashSet<>(request.partitionSizes.keySet()); ProduceResponse errorResponse = (ProduceResponse) request.getErrorResponse(new NotEnoughReplicasException()); assertEquals(partitions, errorResponse.responses().keySet()); @@ -1414,7 +1414,7 @@ private ProduceRequest createProduceRequest(int version) { byte magic = version == 2 ? RecordBatch.MAGIC_VALUE_V1 : RecordBatch.MAGIC_VALUE_V2; MemoryRecords records = MemoryRecords.withRecords(magic, CompressionType.NONE, new SimpleRecord("woot".getBytes())); Map produceData = Collections.singletonMap(new TopicPartition("test", 0), records); - return ProduceRequest.Builder.forMagic(magic, (short) 1, 5000, produceData, "transactionalId") + return ProduceRequest.Builder.forMagic(magic, (short) 1, 5000, produceData, version >= 3 ? "transactionalId" : null) .build((short) version); } diff --git a/clients/src/test/java/org/apache/kafka/common/requests/SchemaTestUtils.java b/clients/src/test/java/org/apache/kafka/common/requests/SchemaTestUtils.java new file mode 100644 index 0000000000000..0a1cc1c562a50 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/requests/SchemaTestUtils.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.common.requests; + +import org.apache.kafka.common.protocol.types.ArrayOf; +import org.apache.kafka.common.protocol.types.BoundField; +import org.apache.kafka.common.protocol.types.Field; +import org.apache.kafka.common.protocol.types.Schema; +import org.apache.kafka.common.protocol.types.Type; +import org.junit.Assert; + +import static org.junit.Assert.assertTrue; + +public final class SchemaTestUtils { + + private SchemaTestUtils() { + } + + static void assertEquals(Type lhs, Type rhs) { + if (lhs instanceof Schema) { + assertTrue(rhs instanceof Schema); + Schema lhsSchema = (Schema) lhs; + Schema rhsSchema = (Schema) rhs; + Assert.assertEquals(lhsSchema.numFields(), rhsSchema.numFields()); + int fieldIndex = 0; + for (BoundField f : lhsSchema.fields()) { + Field previousField = f.def; + Field currentField = lhsSchema.fields()[fieldIndex++].def; + Assert.assertEquals(previousField.name, currentField.name); + assertEquals(previousField.type, currentField.type); + Assert.assertEquals(previousField.hasDefaultValue, currentField.hasDefaultValue); + Assert.assertEquals(previousField.defaultValue, currentField.defaultValue); + } + } else if (lhs instanceof ArrayOf) { + assertTrue(rhs instanceof ArrayOf); + lhs.arrayElementType().ifPresent(lhsType -> + rhs.arrayElementType().ifPresent(rhsType -> assertEquals(lhsType, rhsType))); + } else Assert.assertEquals(lhs, rhs); + } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index dca78b4de5de9..685979e93def5 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -566,7 +566,7 @@ class KafkaApis(val requestChannel: RequestChannel, val produceRequest = request.body[ProduceRequest] val numBytesAppended = request.header.toStruct.sizeOf + request.sizeOfBodyInBytes - if (produceRequest.hasTransactionalRecords) { + if (RequestUtils.hasTransactionalRecords(produceRequest)) { val isAuthorizedTransactional = produceRequest.transactionalId != null && authorize(request.context, WRITE, TRANSACTIONAL_ID, produceRequest.transactionalId) if (!isAuthorizedTransactional) { @@ -575,19 +575,23 @@ class KafkaApis(val requestChannel: RequestChannel, } // Note that authorization to a transactionalId implies ProducerId authorization - } else if (produceRequest.hasIdempotentRecords && !authorize(request.context, IDEMPOTENT_WRITE, CLUSTER, CLUSTER_NAME)) { + } else if (RequestUtils.hasIdempotentRecords(produceRequest) && !authorize(request.context, IDEMPOTENT_WRITE, CLUSTER, CLUSTER_NAME)) { sendErrorResponseMaybeThrottle(request, Errors.CLUSTER_AUTHORIZATION_FAILED.exception) return } - val produceRecords = produceRequest.partitionRecordsOrFail.asScala val unauthorizedTopicResponses = mutable.Map[TopicPartition, PartitionResponse]() val nonExistingTopicResponses = mutable.Map[TopicPartition, PartitionResponse]() val invalidRequestResponses = mutable.Map[TopicPartition, PartitionResponse]() val authorizedRequestInfo = mutable.Map[TopicPartition, MemoryRecords]() - val authorizedTopics = filterByAuthorized(request.context, WRITE, TOPIC, produceRecords)(_._1.topic) - - for ((topicPartition, memoryRecords) <- produceRecords) { + val authorizedTopics = filterByAuthorized(request.context, WRITE, TOPIC, produceRequest.dataOrException().topicData().asScala)(_.topic()) + + produceRequest.dataOrException().topicData().forEach(topic => topic.data().forEach { partition => + val topicPartition = new TopicPartition(topic.topic(), partition.partition()) + // This caller assumes the type is MemoryRecords and that is true on current serialization + // We cast the type to avoid causing big change to code base. + // TODO: maybe we need to refactor code to avoid this casting + val memoryRecords = partition.recordSet().asInstanceOf[MemoryRecords] if (!authorizedTopics.contains(topicPartition.topic)) unauthorizedTopicResponses += topicPartition -> new PartitionResponse(Errors.TOPIC_AUTHORIZATION_FAILED) else if (!metadataCache.contains(topicPartition)) @@ -600,7 +604,7 @@ class KafkaApis(val requestChannel: RequestChannel, case e: ApiException => invalidRequestResponses += topicPartition -> new PartitionResponse(Errors.forException(e)) } - } + }) // the callback for sending a produce response def sendResponseCallback(responseStatus: Map[TopicPartition, PartitionResponse]): Unit = { diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerRequestBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerRequestBenchmark.java new file mode 100644 index 0000000000000..c14c1d58b744a --- /dev/null +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerRequestBenchmark.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.jmh.producer; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.message.ProduceRequestData; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.record.CompressionType; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.SimpleRecord; +import org.apache.kafka.common.requests.ProduceRequest; +import org.apache.kafka.common.requests.ProduceResponse; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Warmup; + +import java.nio.charset.StandardCharsets; +import java.util.AbstractMap; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +@State(Scope.Benchmark) +@Fork(value = 1) +@Warmup(iterations = 5) +@Measurement(iterations = 15) +@BenchmarkMode(Mode.AverageTime) +@OutputTimeUnit(TimeUnit.NANOSECONDS) +public class ProducerRequestBenchmark { + private static final int NUMBER_OF_PARTITIONS = 3; + private static final int NUMBER_OF_RECORDS = 3; + private static final Map PARTITION_MEMORY_RECORDS_MAP = IntStream.range(0, NUMBER_OF_PARTITIONS) + .mapToObj(partitionIndex -> new AbstractMap.SimpleEntry<>( + new TopicPartition("tp", partitionIndex), + MemoryRecords.withRecords(CompressionType.NONE, IntStream.range(0, NUMBER_OF_RECORDS) + .mapToObj(recordIndex -> new SimpleRecord(100, "hello0".getBytes(StandardCharsets.UTF_8))) + .collect(Collectors.toList()) + .toArray(new SimpleRecord[0])) + )) + .collect(Collectors.toMap(AbstractMap.SimpleEntry::getKey, AbstractMap.SimpleEntry::getValue)); + + private static ProduceRequest request() { + return new ProduceRequest.Builder( + ProduceRequestData.LOWEST_SUPPORTED_VERSION, + ProduceRequestData.HIGHEST_SUPPORTED_VERSION, + (short) 1, + 100, + PARTITION_MEMORY_RECORDS_MAP, + null) + .build(); + } + + private static final ProduceRequest REQUEST = request(); + + @Benchmark + @OutputTimeUnit(TimeUnit.NANOSECONDS) + public ProduceRequest constructorProduceRequest() { + return request(); + } + + @Benchmark + @OutputTimeUnit(TimeUnit.NANOSECONDS) + public ProduceResponse constructorErrorResponse() { + return REQUEST.getErrorResponse(0, Errors.INVALID_REQUEST.exception()); + } + + @Benchmark + @OutputTimeUnit(TimeUnit.NANOSECONDS) + public Struct constructorStruct() { + return REQUEST.toStruct(); + } +} diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerResponseBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerResponseBenchmark.java new file mode 100644 index 0000000000000..cacd1eb95e305 --- /dev/null +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerResponseBenchmark.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.jmh.producer; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.message.ProduceResponseData; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.requests.AbstractResponse; +import org.apache.kafka.common.requests.ProduceResponse; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Warmup; + +import java.util.AbstractMap; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +@State(Scope.Benchmark) +@Fork(value = 1) +@Warmup(iterations = 5) +@Measurement(iterations = 15) +@BenchmarkMode(Mode.AverageTime) +@OutputTimeUnit(TimeUnit.NANOSECONDS) +public class ProducerResponseBenchmark { + private static final int NUMBER_OF_PARTITIONS = 3; + private static final int NUMBER_OF_RECORDS = 3; + private static final Map PARTITION_RESPONSE_MAP = IntStream.range(0, NUMBER_OF_PARTITIONS) + .mapToObj(partitionIndex -> new AbstractMap.SimpleEntry<>( + new TopicPartition("tp", partitionIndex), + new ProduceResponse.PartitionResponse( + Errors.NONE, + 0, + 0, + 0, + IntStream.range(0, NUMBER_OF_RECORDS) + .mapToObj(ProduceResponse.RecordError::new) + .collect(Collectors.toList())) + )) + .collect(Collectors.toMap(AbstractMap.SimpleEntry::getKey, AbstractMap.SimpleEntry::getValue)); + + private static ProduceResponse response() { + return new ProduceResponse(PARTITION_RESPONSE_MAP); + } + + private static final ProduceResponse RESPONSE = response(); + + @Benchmark + @OutputTimeUnit(TimeUnit.NANOSECONDS) + public AbstractResponse constructorProduceResponse() { + return response(); + } + + @Benchmark + @OutputTimeUnit(TimeUnit.NANOSECONDS) + public Struct constructorStruct() { + return RESPONSE.toStruct(ProduceResponseData.HIGHEST_SUPPORTED_VERSION); + } +} From 0c8e4fc6781a0c14f63e8b668e550e9c1df1881b Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Tue, 3 Nov 2020 18:47:18 +0800 Subject: [PATCH 04/15] optimize getErrorResponse; fix SchemaTestUtils --- .../clients/producer/internals/Sender.java | 6 ++--- .../kafka/common/requests/ProduceRequest.java | 23 +++++++++++++++---- .../common/requests/ProduceResponse.java | 16 ++++++------- .../common/message/ProduceResponse.json | 8 +++---- .../kafka/common/message/MessageTest.java | 22 +++++++++--------- .../common/requests/SchemaTestUtils.java | 5 ++-- 6 files changed, 46 insertions(+), 34 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index f5a7908de3f49..357ecbd5b89d2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -563,12 +563,12 @@ private void handleProduceResponse(ClientResponse response, Map r.partitions().forEach(p -> { - TopicPartition tp = new TopicPartition(r.name(), p.partitionIndex()); + produceResponse.data().responses().forEach(r -> r.partitionResponses().forEach(p -> { + TopicPartition tp = new TopicPartition(r.topic(), p.partition()); ProduceResponse.PartitionResponse partResp = new ProduceResponse.PartitionResponse( Errors.forCode(p.errorCode()), p.baseOffset(), - p.logAppendTimeMs(), + p.logAppendTime(), p.logStartOffset(), p.recordErrors() .stream() diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java index 5288eb6d05782..3599e34cf5045 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.UnsupportedCompressionTypeException; import org.apache.kafka.common.message.ProduceRequestData; +import org.apache.kafka.common.message.ProduceResponseData; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; @@ -37,9 +38,10 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.function.Function; import java.util.stream.Collectors; +import static org.apache.kafka.common.requests.ProduceResponse.INVALID_OFFSET; + public class ProduceRequest extends AbstractRequest { public static class Builder extends AbstractRequest.Builder { @@ -208,10 +210,21 @@ public String toString(boolean verbose) { public ProduceResponse getErrorResponse(int throttleTimeMs, Throwable e) { /* In case the producer doesn't actually want any response */ if (acks == 0) return null; - ProduceResponse.PartitionResponse partitionResponse = new ProduceResponse.PartitionResponse(Errors.forException(e)); - return new ProduceResponse(partitions() - .stream() - .collect(Collectors.toMap(Function.identity(), ignored -> partitionResponse)), throttleTimeMs); + Errors error = Errors.forException(e); + return new ProduceResponse(new ProduceResponseData() + .setResponses(data.topicData().stream().map(t -> new ProduceResponseData.TopicProduceResponse() + .setPartitionResponses(t.data().stream().map(p -> new ProduceResponseData.PartitionProduceResponse() + .setPartition(p.partition()) + .setRecordErrors(Collections.emptyList()) + .setBaseOffset(INVALID_OFFSET) + .setLogAppendTime(RecordBatch.NO_TIMESTAMP) + .setLogStartOffset(INVALID_OFFSET) + .setErrorMessage(e.getMessage()) + .setErrorCode(error.code())) + .collect(Collectors.toList())) + .setTopic(t.topic())) + .collect(Collectors.toList())) + .setThrottleTimeMs(throttleTimeMs)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java index 9edd4842f16db..83f8c465052e8 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -82,14 +82,14 @@ public ProduceResponse(Map responses, int thr .entrySet() .stream() .map(topicData -> new ProduceResponseData.TopicProduceResponse() - .setName(topicData.getKey()) - .setPartitions(topicData.getValue() + .setTopic(topicData.getKey()) + .setPartitionResponses(topicData.getValue() .stream() .map(p -> new ProduceResponseData.PartitionProduceResponse() - .setPartitionIndex(p.getKey().partition()) + .setPartition(p.getKey().partition()) .setBaseOffset(p.getValue().baseOffset) .setLogStartOffset(p.getValue().logStartOffset) - .setLogAppendTimeMs(p.getValue().logAppendTime) + .setLogAppendTime(p.getValue().logAppendTime) .setErrorMessage(p.getValue().errorMessage) .setErrorCode(p.getValue().error.code()) .setRecordErrors(p.getValue().recordErrors @@ -122,13 +122,13 @@ public ProduceResponseData data() { public Map responses() { return data.responses() .stream() - .flatMap(t -> t.partitions() + .flatMap(t -> t.partitionResponses() .stream() - .map(p -> new AbstractMap.SimpleEntry<>(new TopicPartition(t.name(), p.partitionIndex()), + .map(p -> new AbstractMap.SimpleEntry<>(new TopicPartition(t.topic(), p.partition()), new PartitionResponse( Errors.forCode(p.errorCode()), p.baseOffset(), - p.logAppendTimeMs(), + p.logAppendTime(), p.logStartOffset(), p.recordErrors() .stream() @@ -146,7 +146,7 @@ public int throttleTimeMs() { @Override public Map errorCounts() { Map errorCounts = new HashMap<>(); - data.responses().forEach(t -> t.partitions().forEach(p -> updateErrorCounts(errorCounts, Errors.forCode(p.errorCode())))); + data.responses().forEach(t -> t.partitionResponses().forEach(p -> updateErrorCounts(errorCounts, Errors.forCode(p.errorCode())))); return errorCounts; } diff --git a/clients/src/main/resources/common/message/ProduceResponse.json b/clients/src/main/resources/common/message/ProduceResponse.json index 770313aff5061..0c3b5aa300cd6 100644 --- a/clients/src/main/resources/common/message/ProduceResponse.json +++ b/clients/src/main/resources/common/message/ProduceResponse.json @@ -35,17 +35,17 @@ "fields": [ { "name": "Responses", "type": "[]TopicProduceResponse", "versions": "0+", "about": "Each produce response", "fields": [ - { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", + { "name": "Topic", "type": "string", "versions": "0+", "entityType": "topicName", "about": "The topic name" }, - { "name": "Partitions", "type": "[]PartitionProduceResponse", "versions": "0+", + { "name": "PartitionResponses", "type": "[]PartitionProduceResponse", "versions": "0+", "about": "Each partition that we produced to within the topic.", "fields": [ - { "name": "PartitionIndex", "type": "int32", "versions": "0+", + { "name": "Partition", "type": "int32", "versions": "0+", "about": "The partition index." }, { "name": "ErrorCode", "type": "int16", "versions": "0+", "about": "The error code, or 0 if there was no error." }, { "name": "BaseOffset", "type": "int64", "versions": "0+", "about": "The base offset." }, - { "name": "LogAppendTimeMs", "type": "int64", "versions": "2+", "default": "-1", "ignorable": true, + { "name": "LogAppendTime", "type": "int64", "versions": "2+", "default": "-1", "ignorable": true, "about": "The timestamp returned by broker after appending the messages. If CreateTime is used for the topic, the timestamp will be -1. If LogAppendTime is used for the topic, the timestamp will be the broker local time when the messages are appended." }, { "name": "LogStartOffset", "type": "int64", "versions": "5+", "default": "-1", "ignorable": true, "about": "The log start offset." }, diff --git a/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java b/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java index f02db2374ea87..1f7aa8840d417 100644 --- a/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java +++ b/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java @@ -680,10 +680,10 @@ public void testProduceResponseVersions() throws Exception { testAllMessageRoundTrips(new ProduceResponseData() .setResponses(singletonList( new ProduceResponseData.TopicProduceResponse() - .setName(topicName) - .setPartitions(singletonList( + .setTopic(topicName) + .setPartitionResponses(singletonList( new ProduceResponseData.PartitionProduceResponse() - .setPartitionIndex(partitionIndex) + .setPartition(partitionIndex) .setErrorCode(errorCode) .setBaseOffset(baseOffset)))))); @@ -691,13 +691,13 @@ public void testProduceResponseVersions() throws Exception { () -> new ProduceResponseData() .setResponses(singletonList( new ProduceResponseData.TopicProduceResponse() - .setName(topicName) - .setPartitions(singletonList( + .setTopic(topicName) + .setPartitionResponses(singletonList( new ProduceResponseData.PartitionProduceResponse() - .setPartitionIndex(partitionIndex) + .setPartition(partitionIndex) .setErrorCode(errorCode) .setBaseOffset(baseOffset) - .setLogAppendTimeMs(logAppendTimeMs) + .setLogAppendTime(logAppendTimeMs) .setLogStartOffset(logStartOffset) .setRecordErrors(singletonList( new ProduceResponseData.BatchIndexAndErrorMessage() @@ -710,16 +710,16 @@ public void testProduceResponseVersions() throws Exception { ProduceResponseData responseData = response.get(); if (version < 8) { - responseData.responses().get(0).partitions().get(0).setRecordErrors(Collections.emptyList()); - responseData.responses().get(0).partitions().get(0).setErrorMessage(null); + responseData.responses().get(0).partitionResponses().get(0).setRecordErrors(Collections.emptyList()); + responseData.responses().get(0).partitionResponses().get(0).setErrorMessage(null); } if (version < 5) { - responseData.responses().get(0).partitions().get(0).setLogStartOffset(-1); + responseData.responses().get(0).partitionResponses().get(0).setLogStartOffset(-1); } if (version < 2) { - responseData.responses().get(0).partitions().get(0).setLogAppendTimeMs(-1); + responseData.responses().get(0).partitionResponses().get(0).setLogAppendTime(-1); } if (version < 1) { diff --git a/clients/src/test/java/org/apache/kafka/common/requests/SchemaTestUtils.java b/clients/src/test/java/org/apache/kafka/common/requests/SchemaTestUtils.java index 0a1cc1c562a50..f5ded1c84d6a7 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/SchemaTestUtils.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/SchemaTestUtils.java @@ -40,11 +40,10 @@ static void assertEquals(Type lhs, Type rhs) { int fieldIndex = 0; for (BoundField f : lhsSchema.fields()) { Field previousField = f.def; - Field currentField = lhsSchema.fields()[fieldIndex++].def; + Field currentField = rhsSchema.fields()[fieldIndex++].def; Assert.assertEquals(previousField.name, currentField.name); assertEquals(previousField.type, currentField.type); - Assert.assertEquals(previousField.hasDefaultValue, currentField.hasDefaultValue); - Assert.assertEquals(previousField.defaultValue, currentField.defaultValue); + // hasDefaultValue and defaultValue are not used by automatic protocol so we don't need to check them. } } else if (lhs instanceof ArrayOf) { assertTrue(rhs instanceof ArrayOf); From 15a33f7d28eddf5442e56802d41c99821db257da Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Wed, 4 Nov 2020 11:01:14 +0800 Subject: [PATCH 05/15] fix produceRequestGetErrorResponseTest --- .../kafka/common/requests/ProduceRequest.java | 24 ++++++++++--------- 1 file changed, 13 insertions(+), 11 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java index 3599e34cf5045..fe577e68a3ab9 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -212,17 +212,19 @@ public ProduceResponse getErrorResponse(int throttleTimeMs, Throwable e) { if (acks == 0) return null; Errors error = Errors.forException(e); return new ProduceResponse(new ProduceResponseData() - .setResponses(data.topicData().stream().map(t -> new ProduceResponseData.TopicProduceResponse() - .setPartitionResponses(t.data().stream().map(p -> new ProduceResponseData.PartitionProduceResponse() - .setPartition(p.partition()) - .setRecordErrors(Collections.emptyList()) - .setBaseOffset(INVALID_OFFSET) - .setLogAppendTime(RecordBatch.NO_TIMESTAMP) - .setLogStartOffset(INVALID_OFFSET) - .setErrorMessage(e.getMessage()) - .setErrorCode(error.code())) - .collect(Collectors.toList())) - .setTopic(t.topic())) + .setResponses(partitions().stream().collect(Collectors.groupingBy(TopicPartition::topic)).entrySet() + .stream() + .map(entry -> new ProduceResponseData.TopicProduceResponse() + .setPartitionResponses(entry.getValue().stream().map(p -> new ProduceResponseData.PartitionProduceResponse() + .setPartition(p.partition()) + .setRecordErrors(Collections.emptyList()) + .setBaseOffset(INVALID_OFFSET) + .setLogAppendTime(RecordBatch.NO_TIMESTAMP) + .setLogStartOffset(INVALID_OFFSET) + .setErrorMessage(e.getMessage()) + .setErrorCode(error.code())) + .collect(Collectors.toList())) + .setTopic(entry.getKey())) .collect(Collectors.toList())) .setThrottleTimeMs(throttleTimeMs)); } From c7c4877642c9371e42f55470f5ded60ce385efb0 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Sun, 8 Nov 2020 12:17:18 +0800 Subject: [PATCH 06/15] address review comments --- .../clients/producer/internals/Sender.java | 4 +- .../kafka/common/requests/ProduceRequest.java | 60 +++++++++++-------- .../common/requests/ProduceResponse.java | 10 ++-- .../kafka/common/requests/RequestUtils.java | 4 +- .../common/message/ProduceRequest.json | 10 ++-- .../common/message/ProduceResponse.json | 6 +- .../producer/internals/SenderTest.java | 6 +- .../internals/TransactionManagerTest.java | 8 +-- .../kafka/common/message/MessageTest.java | 12 ++-- .../common/requests/RequestResponseTest.java | 8 +-- .../common/requests/SchemaTestUtils.java | 13 +--- .../main/scala/kafka/server/KafkaApis.scala | 10 ++-- 12 files changed, 77 insertions(+), 74 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index 357ecbd5b89d2..07591fa7fc43d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -564,11 +564,11 @@ private void handleProduceResponse(ClientResponse response, Map r.partitionResponses().forEach(p -> { - TopicPartition tp = new TopicPartition(r.topic(), p.partition()); + TopicPartition tp = new TopicPartition(r.name(), p.index()); ProduceResponse.PartitionResponse partResp = new ProduceResponse.PartitionResponse( Errors.forCode(p.errorCode()), p.baseOffset(), - p.logAppendTime(), + p.logAppendTimeMs(), p.logStartOffset(), p.recordErrors() .stream() diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java index fe577e68a3ab9..7bb49a0c997d2 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -33,7 +33,6 @@ import java.nio.ByteBuffer; import java.util.AbstractMap; -import java.util.Collection; import java.util.Collections; import java.util.Iterator; import java.util.List; @@ -115,17 +114,17 @@ private ProduceRequest build(short version, boolean validate) { .entrySet() .stream() .map(e -> new ProduceRequestData.TopicProduceData() - .setTopic(e.getKey()) - .setData(e.getValue().stream() + .setName(e.getKey()) + .setPartitionData(e.getValue().stream() .map(tpAndRecord -> new ProduceRequestData.PartitionProduceData() - .setPartition(tpAndRecord.getKey().partition()) - .setRecordSet(tpAndRecord.getValue())) + .setIndex(tpAndRecord.getKey().partition()) + .setRecords(tpAndRecord.getValue())) .collect(Collectors.toList()))) .collect(Collectors.toList()); return new ProduceRequest(new ProduceRequestData() .setAcks(acks) - .setTimeout(timeout) + .setTimeoutMs(timeout) .setTransactionalId(transactionalId) .setTopicData(tpd), version); } @@ -150,25 +149,36 @@ public String toString() { private final short acks; private final int timeout; private final String transactionalId; - // visible for testing - final Map partitionSizes; // This is set to null by `clearPartitionRecords` to prevent unnecessary memory retention when a produce request is // put in the purgatory (due to client throttling, it can take a while before the response is sent). // Care should be taken in methods that use this field. private volatile ProduceRequestData data; + // the partitionSizes is lazily initialized since it is used by server-side in production. + private volatile Map partitionSizes; public ProduceRequest(ProduceRequestData produceRequestData, short version) { super(ApiKeys.PRODUCE, version); this.data = produceRequestData; this.acks = data.acks(); - this.timeout = data.timeout(); + this.timeout = data.timeoutMs(); this.transactionalId = data.transactionalId(); - this.partitionSizes = data.topicData() - .stream() - .flatMap(e -> e.data() - .stream() - .map(p -> new AbstractMap.SimpleEntry<>(new TopicPartition(e.topic(), p.partition()), p.recordSet().sizeInBytes()))) - .collect(Collectors.groupingBy(AbstractMap.SimpleEntry::getKey, Collectors.summingInt(AbstractMap.SimpleEntry::getValue))); + } + + public Map partitionSizes() { + if (partitionSizes != null) return partitionSizes; + else { + // this method may be called by different thread (see the comment on data) + synchronized (this) { + if (partitionSizes == null) + partitionSizes = data.topicData() + .stream() + .flatMap(e -> e.partitionData() + .stream() + .map(p -> new AbstractMap.SimpleEntry<>(new TopicPartition(e.name(), p.index()), p.records().sizeInBytes()))) + .collect(Collectors.groupingBy(AbstractMap.SimpleEntry::getKey, Collectors.summingInt(AbstractMap.SimpleEntry::getValue))); + } + } + return partitionSizes; } /** @@ -198,9 +208,9 @@ public String toString(boolean verbose) { .append(",timeout=").append(timeout); if (verbose) - bld.append(",partitionSizes=").append(Utils.mkString(partitionSizes, "[", "]", "=", ",")); + bld.append(",partitionSizes=").append(Utils.mkString(partitionSizes(), "[", "]", "=", ",")); else - bld.append(",numPartitions=").append(partitionSizes.size()); + bld.append(",numPartitions=").append(partitionSizes().size()); bld.append("}"); return bld.toString(); @@ -212,19 +222,19 @@ public ProduceResponse getErrorResponse(int throttleTimeMs, Throwable e) { if (acks == 0) return null; Errors error = Errors.forException(e); return new ProduceResponse(new ProduceResponseData() - .setResponses(partitions().stream().collect(Collectors.groupingBy(TopicPartition::topic)).entrySet() + .setResponses(partitionSizes().keySet().stream().collect(Collectors.groupingBy(TopicPartition::topic)).entrySet() .stream() .map(entry -> new ProduceResponseData.TopicProduceResponse() .setPartitionResponses(entry.getValue().stream().map(p -> new ProduceResponseData.PartitionProduceResponse() - .setPartition(p.partition()) + .setIndex(p.partition()) .setRecordErrors(Collections.emptyList()) .setBaseOffset(INVALID_OFFSET) - .setLogAppendTime(RecordBatch.NO_TIMESTAMP) + .setLogAppendTimeMs(RecordBatch.NO_TIMESTAMP) .setLogStartOffset(INVALID_OFFSET) .setErrorMessage(e.getMessage()) .setErrorCode(error.code())) .collect(Collectors.toList())) - .setTopic(entry.getKey())) + .setName(entry.getKey())) .collect(Collectors.toList())) .setThrottleTimeMs(throttleTimeMs)); } @@ -232,11 +242,7 @@ public ProduceResponse getErrorResponse(int throttleTimeMs, Throwable e) { @Override public Map errorCounts(Throwable e) { Errors error = Errors.forException(e); - return Collections.singletonMap(error, partitions().size()); - } - - private Collection partitions() { - return partitionSizes.keySet(); + return Collections.singletonMap(error, partitionSizes().size()); } public short acks() { @@ -253,6 +259,8 @@ public String transactionalId() { public void clearPartitionRecords() { data = null; + // lazily initialize partitionSizes. + partitionSizes(); } public static void validateRecords(short version, Records records) { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java index 83f8c465052e8..411c2789c3e3f 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -82,14 +82,14 @@ public ProduceResponse(Map responses, int thr .entrySet() .stream() .map(topicData -> new ProduceResponseData.TopicProduceResponse() - .setTopic(topicData.getKey()) + .setName(topicData.getKey()) .setPartitionResponses(topicData.getValue() .stream() .map(p -> new ProduceResponseData.PartitionProduceResponse() - .setPartition(p.getKey().partition()) + .setIndex(p.getKey().partition()) .setBaseOffset(p.getValue().baseOffset) .setLogStartOffset(p.getValue().logStartOffset) - .setLogAppendTime(p.getValue().logAppendTime) + .setLogAppendTimeMs(p.getValue().logAppendTime) .setErrorMessage(p.getValue().errorMessage) .setErrorCode(p.getValue().error.code()) .setRecordErrors(p.getValue().recordErrors @@ -124,11 +124,11 @@ public Map responses() { .stream() .flatMap(t -> t.partitionResponses() .stream() - .map(p -> new AbstractMap.SimpleEntry<>(new TopicPartition(t.topic(), p.partition()), + .map(p -> new AbstractMap.SimpleEntry<>(new TopicPartition(t.name(), p.index()), new PartitionResponse( Errors.forCode(p.errorCode()), p.baseOffset(), - p.logAppendTime(), + p.logAppendTimeMs(), p.logStartOffset(), p.recordErrors() .stream() diff --git a/clients/src/main/java/org/apache/kafka/common/requests/RequestUtils.java b/clients/src/main/java/org/apache/kafka/common/requests/RequestUtils.java index ae1d1df14fe1d..13a95275273c3 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/RequestUtils.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/RequestUtils.java @@ -64,10 +64,10 @@ public static boolean hasTransactionalRecords(ProduceRequest request) { private static boolean anyMatch(ProduceRequest request, Predicate predicate) { return request.dataOrException().topicData() .stream() - .anyMatch(topicProduceData -> topicProduceData.data() + .anyMatch(topicProduceData -> topicProduceData.partitionData() .stream() .anyMatch(partitionProduceData -> { - BaseRecords records = partitionProduceData.recordSet(); + BaseRecords records = partitionProduceData.records(); if (records instanceof Records) { Iterator iterator = ((Records) records).batches().iterator(); return iterator.hasNext() && predicate.test(iterator.next()); diff --git a/clients/src/main/resources/common/message/ProduceRequest.json b/clients/src/main/resources/common/message/ProduceRequest.json index 7fbde3f30030c..b626e41e832f2 100644 --- a/clients/src/main/resources/common/message/ProduceRequest.json +++ b/clients/src/main/resources/common/message/ProduceRequest.json @@ -37,17 +37,17 @@ "about": "The transactional ID, or null if the producer is not transactional." }, { "name": "Acks", "type": "int16", "versions": "0+", "about": "The number of acknowledgments the producer requires the leader to have received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for only the leader and -1 for the full ISR." }, - { "name": "Timeout", "type": "int32", "versions": "0+", + { "name": "TimeoutMs", "type": "int32", "versions": "0+", "about": "The timeout to await a response in miliseconds." }, { "name": "TopicData", "type": "[]TopicProduceData", "versions": "0+", "about": "Each topic to produce to.", "fields": [ - { "name": "Topic", "type": "string", "versions": "0+", "entityType": "topicName", + { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", "about": "The topic name." }, - { "name": "Data", "type": "[]PartitionProduceData", "versions": "0+", + { "name": "PartitionData", "type": "[]PartitionProduceData", "versions": "0+", "about": "Each partition to produce to.", "fields": [ - { "name": "Partition", "type": "int32", "versions": "0+", + { "name": "Index", "type": "int32", "versions": "0+", "about": "The partition index." }, - { "name": "RecordSet", "type": "records", "versions": "0+", "nullableVersions": "0+", + { "name": "Records", "type": "records", "versions": "0+", "nullableVersions": "0+", "about": "The record data to be produced." } ]} ]} diff --git a/clients/src/main/resources/common/message/ProduceResponse.json b/clients/src/main/resources/common/message/ProduceResponse.json index 0c3b5aa300cd6..a68bd6fd98719 100644 --- a/clients/src/main/resources/common/message/ProduceResponse.json +++ b/clients/src/main/resources/common/message/ProduceResponse.json @@ -35,17 +35,17 @@ "fields": [ { "name": "Responses", "type": "[]TopicProduceResponse", "versions": "0+", "about": "Each produce response", "fields": [ - { "name": "Topic", "type": "string", "versions": "0+", "entityType": "topicName", + { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", "about": "The topic name" }, { "name": "PartitionResponses", "type": "[]PartitionProduceResponse", "versions": "0+", "about": "Each partition that we produced to within the topic.", "fields": [ - { "name": "Partition", "type": "int32", "versions": "0+", + { "name": "Index", "type": "int32", "versions": "0+", "about": "The partition index." }, { "name": "ErrorCode", "type": "int16", "versions": "0+", "about": "The error code, or 0 if there was no error." }, { "name": "BaseOffset", "type": "int64", "versions": "0+", "about": "The base offset." }, - { "name": "LogAppendTime", "type": "int64", "versions": "2+", "default": "-1", "ignorable": true, + { "name": "LogAppendTimeMs", "type": "int64", "versions": "2+", "default": "-1", "ignorable": true, "about": "The timestamp returned by broker after appending the messages. If CreateTime is used for the topic, the timestamp will be -1. If LogAppendTime is used for the topic, the timestamp will be the broker local time when the messages are appended." }, { "name": "LogStartOffset", "type": "int64", "versions": "5+", "default": "-1", "ignorable": true, "about": "The log start offset." }, diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java index 2c9172050c39f..7fc9c0f0b12b7 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java @@ -159,9 +159,9 @@ public void tearDown() { private static Map partitionRecords(ProduceRequest request) { Map partitionRecords = new HashMap<>(); - request.dataOrException().topicData().forEach(tpData -> tpData.data().forEach(p -> { - TopicPartition tp = new TopicPartition(tpData.topic(), p.partition()); - partitionRecords.put(tp, (MemoryRecords) p.recordSet()); + request.dataOrException().topicData().forEach(tpData -> tpData.partitionData().forEach(p -> { + TopicPartition tp = new TopicPartition(tpData.name(), p.index()); + partitionRecords.put(tp, (MemoryRecords) p.records()); })); return Collections.unmodifiableMap(partitionRecords); } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java index 8fb3850683b4a..d7d7b8b4594f7 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java @@ -3358,13 +3358,13 @@ private MockClient.RequestMatcher produceRequestMatcher(final long producerId, f ProduceRequest produceRequest = (ProduceRequest) body; MemoryRecords records = produceRequest.dataOrException().topicData() .stream() - .filter(t -> t.topic().equals(tp.topic())) + .filter(t -> t.name().equals(tp.topic())) .findAny() .get() - .data() + .partitionData() .stream() - .filter(p -> p.partition() == tp.partition()) - .map(p -> (MemoryRecords) p.recordSet()) + .filter(p -> p.index() == tp.partition()) + .map(p -> (MemoryRecords) p.records()) .findAny().get(); assertNotNull(records); Iterator batchIterator = records.batches().iterator(); diff --git a/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java b/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java index 1f7aa8840d417..25faa0d714640 100644 --- a/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java +++ b/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java @@ -680,10 +680,10 @@ public void testProduceResponseVersions() throws Exception { testAllMessageRoundTrips(new ProduceResponseData() .setResponses(singletonList( new ProduceResponseData.TopicProduceResponse() - .setTopic(topicName) + .setName(topicName) .setPartitionResponses(singletonList( new ProduceResponseData.PartitionProduceResponse() - .setPartition(partitionIndex) + .setIndex(partitionIndex) .setErrorCode(errorCode) .setBaseOffset(baseOffset)))))); @@ -691,13 +691,13 @@ public void testProduceResponseVersions() throws Exception { () -> new ProduceResponseData() .setResponses(singletonList( new ProduceResponseData.TopicProduceResponse() - .setTopic(topicName) + .setName(topicName) .setPartitionResponses(singletonList( new ProduceResponseData.PartitionProduceResponse() - .setPartition(partitionIndex) + .setIndex(partitionIndex) .setErrorCode(errorCode) .setBaseOffset(baseOffset) - .setLogAppendTime(logAppendTimeMs) + .setLogAppendTimeMs(logAppendTimeMs) .setLogStartOffset(logStartOffset) .setRecordErrors(singletonList( new ProduceResponseData.BatchIndexAndErrorMessage() @@ -719,7 +719,7 @@ public void testProduceResponseVersions() throws Exception { } if (version < 2) { - responseData.responses().get(0).partitionResponses().get(0).setLogAppendTime(-1); + responseData.responses().get(0).partitionResponses().get(0).setLogAppendTimeMs(-1); } if (version < 1) { diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 880dc037e1f26..264e2b38644ed 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -638,9 +638,9 @@ public void testPartitionSize() { produceData.put(tp1, records1); ProduceRequest request = ProduceRequest.Builder.forMagic(RecordBatch.MAGIC_VALUE_V2, (short) 1, 5000, produceData, "transactionalId") .build((short) 3); - assertEquals(2, request.partitionSizes.size()); - assertEquals(records0.sizeInBytes(), (int) request.partitionSizes.get(tp0)); - assertEquals(records1.sizeInBytes(), (int) request.partitionSizes.get(tp1)); + assertEquals(2, request.partitionSizes().size()); + assertEquals(records0.sizeInBytes(), (int) request.partitionSizes().get(tp0)); + assertEquals(records1.sizeInBytes(), (int) request.partitionSizes().get(tp1)); } @Test @@ -670,7 +670,7 @@ public void produceRequestToStringTest() { @Test public void produceRequestGetErrorResponseTest() { ProduceRequest request = createProduceRequest(ApiKeys.PRODUCE.latestVersion()); - Set partitions = new HashSet<>(request.partitionSizes.keySet()); + Set partitions = new HashSet<>(request.partitionSizes().keySet()); ProduceResponse errorResponse = (ProduceResponse) request.getErrorResponse(new NotEnoughReplicasException()); assertEquals(partitions, errorResponse.responses().keySet()); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/SchemaTestUtils.java b/clients/src/test/java/org/apache/kafka/common/requests/SchemaTestUtils.java index f5ded1c84d6a7..c39d8e7708439 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/SchemaTestUtils.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/SchemaTestUtils.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.protocol.types.ArrayOf; import org.apache.kafka.common.protocol.types.BoundField; -import org.apache.kafka.common.protocol.types.Field; import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Type; import org.junit.Assert; @@ -33,20 +32,14 @@ private SchemaTestUtils() { static void assertEquals(Type lhs, Type rhs) { if (lhs instanceof Schema) { - assertTrue(rhs instanceof Schema); + assertTrue("excepted: Schema, actual: " + rhs, rhs instanceof Schema); Schema lhsSchema = (Schema) lhs; Schema rhsSchema = (Schema) rhs; Assert.assertEquals(lhsSchema.numFields(), rhsSchema.numFields()); int fieldIndex = 0; - for (BoundField f : lhsSchema.fields()) { - Field previousField = f.def; - Field currentField = rhsSchema.fields()[fieldIndex++].def; - Assert.assertEquals(previousField.name, currentField.name); - assertEquals(previousField.type, currentField.type); - // hasDefaultValue and defaultValue are not used by automatic protocol so we don't need to check them. - } + for (BoundField f : lhsSchema.fields()) assertEquals(f.def.type, rhsSchema.fields()[fieldIndex++].def.type); } else if (lhs instanceof ArrayOf) { - assertTrue(rhs instanceof ArrayOf); + assertTrue("excepted: ArrayOf, actual: " + rhs, rhs instanceof ArrayOf); lhs.arrayElementType().ifPresent(lhsType -> rhs.arrayElementType().ifPresent(rhsType -> assertEquals(lhsType, rhsType))); } else Assert.assertEquals(lhs, rhs); diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 685979e93def5..4c9732e25759f 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -584,14 +584,16 @@ class KafkaApis(val requestChannel: RequestChannel, val nonExistingTopicResponses = mutable.Map[TopicPartition, PartitionResponse]() val invalidRequestResponses = mutable.Map[TopicPartition, PartitionResponse]() val authorizedRequestInfo = mutable.Map[TopicPartition, MemoryRecords]() - val authorizedTopics = filterByAuthorized(request.context, WRITE, TOPIC, produceRequest.dataOrException().topicData().asScala)(_.topic()) + // cache the result to avoid redundant authorization calls + val authorizedTopics = filterByAuthorized(request.context, WRITE, TOPIC, + produceRequest.dataOrException().topicData().asScala)(_.name()) - produceRequest.dataOrException().topicData().forEach(topic => topic.data().forEach { partition => - val topicPartition = new TopicPartition(topic.topic(), partition.partition()) + produceRequest.dataOrException.topicData.forEach(topic => topic.partitionData.forEach { partition => + val topicPartition = new TopicPartition(topic.name, partition.index) // This caller assumes the type is MemoryRecords and that is true on current serialization // We cast the type to avoid causing big change to code base. // TODO: maybe we need to refactor code to avoid this casting - val memoryRecords = partition.recordSet().asInstanceOf[MemoryRecords] + val memoryRecords = partition.records.asInstanceOf[MemoryRecords] if (!authorizedTopics.contains(topicPartition.topic)) unauthorizedTopicResponses += topicPartition -> new PartitionResponse(Errors.TOPIC_AUTHORIZATION_FAILED) else if (!metadataCache.contains(topicPartition)) From a9f32e50ecb64f3ef98618d8b54eee4d6a55cf3b Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Mon, 9 Nov 2020 15:49:12 +0800 Subject: [PATCH 07/15] add jira link and improve partitionSizes --- .../clients/producer/internals/Sender.java | 1 + .../kafka/common/requests/ProduceRequest.java | 27 ++++++++++--------- .../common/requests/ProduceResponse.java | 1 + .../main/scala/kafka/server/KafkaApis.scala | 1 + 4 files changed, 18 insertions(+), 12 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index 07591fa7fc43d..33219f6825fdc 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -562,6 +562,7 @@ private void handleProduceResponse(ClientResponse response, Map r.partitionResponses().forEach(p -> { TopicPartition tp = new TopicPartition(r.name(), p.index()); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java index 7bb49a0c997d2..733d31b46a71c 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -32,8 +32,8 @@ import org.apache.kafka.common.utils.Utils; import java.nio.ByteBuffer; -import java.util.AbstractMap; import java.util.Collections; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -164,18 +164,21 @@ public ProduceRequest(ProduceRequestData produceRequestData, short version) { this.transactionalId = data.transactionalId(); } - public Map partitionSizes() { - if (partitionSizes != null) return partitionSizes; - else { + // visible for testing + Map partitionSizes() { + if (partitionSizes == null) { // this method may be called by different thread (see the comment on data) synchronized (this) { - if (partitionSizes == null) - partitionSizes = data.topicData() - .stream() - .flatMap(e -> e.partitionData() - .stream() - .map(p -> new AbstractMap.SimpleEntry<>(new TopicPartition(e.name(), p.index()), p.records().sizeInBytes()))) - .collect(Collectors.groupingBy(AbstractMap.SimpleEntry::getKey, Collectors.summingInt(AbstractMap.SimpleEntry::getValue))); + if (partitionSizes == null) { + partitionSizes = new HashMap<>(); + data.topicData().forEach(topicData -> + topicData.partitionData().forEach(partitionData -> + partitionSizes.compute(new TopicPartition(topicData.name(), partitionData.index()), + (ignored, previousValue) -> + partitionData.records().sizeInBytes() + (previousValue == null ? 0 : previousValue)) + ) + ); + } } } return partitionSizes; @@ -258,9 +261,9 @@ public String transactionalId() { } public void clearPartitionRecords() { - data = null; // lazily initialize partitionSizes. partitionSizes(); + data = null; } public static void validateRecords(short version, Records records) { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java index 411c2789c3e3f..07b9b5dff6b6a 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -118,6 +118,7 @@ public ProduceResponseData data() { /** * this method is used by testing only. * TODO: refactor the tests which are using this method and then remove this method from production code. + * https://issues.apache.org/jira/browse/KAFKA-10697 */ public Map responses() { return data.responses() diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 4c9732e25759f..455aa46e8e7c0 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -593,6 +593,7 @@ class KafkaApis(val requestChannel: RequestChannel, // This caller assumes the type is MemoryRecords and that is true on current serialization // We cast the type to avoid causing big change to code base. // TODO: maybe we need to refactor code to avoid this casting + // https://issues.apache.org/jira/browse/KAFKA-10698 val memoryRecords = partition.records.asInstanceOf[MemoryRecords] if (!authorizedTopics.contains(topicPartition.topic)) unauthorizedTopicResponses += topicPartition -> new PartitionResponse(Errors.TOPIC_AUTHORIZATION_FAILED) From c4bf6bcd948db2845e4528fb4b6e160bf4b95dea Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Thu, 12 Nov 2020 18:46:00 +0800 Subject: [PATCH 08/15] git rid of stream APIs --- .../kafka/common/requests/ProduceRequest.java | 33 +++++----- .../common/requests/ProduceResponse.java | 51 ++++++++-------- .../common/message/ProduceResponse.json | 2 +- .../kafka/common/message/MessageTest.java | 61 +++++++++---------- 4 files changed, 74 insertions(+), 73 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java index 733d31b46a71c..9d2cfc484e38a 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -224,22 +224,23 @@ public ProduceResponse getErrorResponse(int throttleTimeMs, Throwable e) { /* In case the producer doesn't actually want any response */ if (acks == 0) return null; Errors error = Errors.forException(e); - return new ProduceResponse(new ProduceResponseData() - .setResponses(partitionSizes().keySet().stream().collect(Collectors.groupingBy(TopicPartition::topic)).entrySet() - .stream() - .map(entry -> new ProduceResponseData.TopicProduceResponse() - .setPartitionResponses(entry.getValue().stream().map(p -> new ProduceResponseData.PartitionProduceResponse() - .setIndex(p.partition()) - .setRecordErrors(Collections.emptyList()) - .setBaseOffset(INVALID_OFFSET) - .setLogAppendTimeMs(RecordBatch.NO_TIMESTAMP) - .setLogStartOffset(INVALID_OFFSET) - .setErrorMessage(e.getMessage()) - .setErrorCode(error.code())) - .collect(Collectors.toList())) - .setName(entry.getKey())) - .collect(Collectors.toList())) - .setThrottleTimeMs(throttleTimeMs)); + ProduceResponseData data = new ProduceResponseData().setThrottleTimeMs(throttleTimeMs); + partitionSizes().forEach((tp, ignored) -> { + ProduceResponseData.TopicProduceResponse tpr = data.responses().find(tp.topic()); + if (tpr == null) { + tpr = new ProduceResponseData.TopicProduceResponse().setName(tp.topic()); + data.responses().add(tpr); + } + tpr.partitionResponses().add(new ProduceResponseData.PartitionProduceResponse() + .setIndex(tp.partition()) + .setRecordErrors(Collections.emptyList()) + .setBaseOffset(INVALID_OFFSET) + .setLogAppendTimeMs(RecordBatch.NO_TIMESTAMP) + .setLogStartOffset(INVALID_OFFSET) + .setErrorMessage(e.getMessage()) + .setErrorCode(error.code())); + }); + return new ProduceResponse(data); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java index 07b9b5dff6b6a..fc2670290c431 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -75,32 +75,33 @@ public ProduceResponse(Map responses) { * @param throttleTimeMs Time in milliseconds the response was throttled */ public ProduceResponse(Map responses, int throttleTimeMs) { - this(new ProduceResponseData() - .setResponses(responses.entrySet() - .stream() - .collect(Collectors.groupingBy(e -> e.getKey().topic())) - .entrySet() - .stream() - .map(topicData -> new ProduceResponseData.TopicProduceResponse() - .setName(topicData.getKey()) - .setPartitionResponses(topicData.getValue() + this(toData(responses, throttleTimeMs)); + } + + private static ProduceResponseData toData(Map responses, int throttleTimeMs) { + ProduceResponseData data = new ProduceResponseData().setThrottleTimeMs(throttleTimeMs); + responses.forEach((tp, response) -> { + ProduceResponseData.TopicProduceResponse tpr = data.responses().find(tp.topic()); + if (tpr == null) { + tpr = new ProduceResponseData.TopicProduceResponse().setName(tp.topic()); + data.responses().add(tpr); + } + tpr.partitionResponses() + .add(new ProduceResponseData.PartitionProduceResponse() + .setIndex(tp.partition()) + .setBaseOffset(response.baseOffset) + .setLogStartOffset(response.logStartOffset) + .setLogAppendTimeMs(response.logAppendTime) + .setErrorMessage(response.errorMessage) + .setErrorCode(response.error.code()) + .setRecordErrors(response.recordErrors .stream() - .map(p -> new ProduceResponseData.PartitionProduceResponse() - .setIndex(p.getKey().partition()) - .setBaseOffset(p.getValue().baseOffset) - .setLogStartOffset(p.getValue().logStartOffset) - .setLogAppendTimeMs(p.getValue().logAppendTime) - .setErrorMessage(p.getValue().errorMessage) - .setErrorCode(p.getValue().error.code()) - .setRecordErrors(p.getValue().recordErrors - .stream() - .map(e -> new ProduceResponseData.BatchIndexAndErrorMessage() - .setBatchIndex(e.batchIndex) - .setBatchIndexErrorMessage(e.message)) - .collect(Collectors.toList()))) - .collect(Collectors.toList()))) - .collect(Collectors.toList())) - .setThrottleTimeMs(throttleTimeMs)); + .map(e -> new ProduceResponseData.BatchIndexAndErrorMessage() + .setBatchIndex(e.batchIndex) + .setBatchIndexErrorMessage(e.message)) + .collect(Collectors.toList()))); + }); + return data; } /** diff --git a/clients/src/main/resources/common/message/ProduceResponse.json b/clients/src/main/resources/common/message/ProduceResponse.json index a68bd6fd98719..e27acaa47658d 100644 --- a/clients/src/main/resources/common/message/ProduceResponse.json +++ b/clients/src/main/resources/common/message/ProduceResponse.json @@ -35,7 +35,7 @@ "fields": [ { "name": "Responses", "type": "[]TopicProduceResponse", "versions": "0+", "about": "Each produce response", "fields": [ - { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", + { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", "mapKey": true, "about": "The topic name" }, { "name": "PartitionResponses", "type": "[]PartitionProduceResponse", "versions": "0+", "about": "Each partition that we produced to within the topic.", "fields": [ diff --git a/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java b/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java index 25faa0d714640..36dde722c8772 100644 --- a/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java +++ b/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java @@ -678,48 +678,47 @@ public void testProduceResponseVersions() throws Exception { String errorMessage = "global error message"; testAllMessageRoundTrips(new ProduceResponseData() - .setResponses(singletonList( - new ProduceResponseData.TopicProduceResponse() - .setName(topicName) - .setPartitionResponses(singletonList( - new ProduceResponseData.PartitionProduceResponse() - .setIndex(partitionIndex) - .setErrorCode(errorCode) - .setBaseOffset(baseOffset)))))); - - Supplier response = - () -> new ProduceResponseData() - .setResponses(singletonList( - new ProduceResponseData.TopicProduceResponse() - .setName(topicName) - .setPartitionResponses(singletonList( - new ProduceResponseData.PartitionProduceResponse() - .setIndex(partitionIndex) - .setErrorCode(errorCode) - .setBaseOffset(baseOffset) - .setLogAppendTimeMs(logAppendTimeMs) - .setLogStartOffset(logStartOffset) - .setRecordErrors(singletonList( - new ProduceResponseData.BatchIndexAndErrorMessage() - .setBatchIndex(batchIndex) - .setBatchIndexErrorMessage(batchIndexErrorMessage))) - .setErrorMessage(errorMessage))))) - .setThrottleTimeMs(throttleTimeMs); + .setResponses(new ProduceResponseData.TopicProduceResponseCollection(singletonList( + new ProduceResponseData.TopicProduceResponse() + .setName(topicName) + .setPartitionResponses(singletonList( + new ProduceResponseData.PartitionProduceResponse() + .setIndex(partitionIndex) + .setErrorCode(errorCode) + .setBaseOffset(baseOffset)))).iterator()))); + + Supplier response = () -> new ProduceResponseData() + .setResponses(new ProduceResponseData.TopicProduceResponseCollection(singletonList( + new ProduceResponseData.TopicProduceResponse() + .setName(topicName) + .setPartitionResponses(singletonList( + new ProduceResponseData.PartitionProduceResponse() + .setIndex(partitionIndex) + .setErrorCode(errorCode) + .setBaseOffset(baseOffset) + .setLogAppendTimeMs(logAppendTimeMs) + .setLogStartOffset(logStartOffset) + .setRecordErrors(singletonList( + new ProduceResponseData.BatchIndexAndErrorMessage() + .setBatchIndex(batchIndex) + .setBatchIndexErrorMessage(batchIndexErrorMessage))) + .setErrorMessage(errorMessage)))).iterator())) + .setThrottleTimeMs(throttleTimeMs); for (short version = 0; version <= ApiKeys.PRODUCE.latestVersion(); version++) { ProduceResponseData responseData = response.get(); if (version < 8) { - responseData.responses().get(0).partitionResponses().get(0).setRecordErrors(Collections.emptyList()); - responseData.responses().get(0).partitionResponses().get(0).setErrorMessage(null); + responseData.responses().iterator().next().partitionResponses().get(0).setRecordErrors(Collections.emptyList()); + responseData.responses().iterator().next().partitionResponses().get(0).setErrorMessage(null); } if (version < 5) { - responseData.responses().get(0).partitionResponses().get(0).setLogStartOffset(-1); + responseData.responses().iterator().next().partitionResponses().get(0).setLogStartOffset(-1); } if (version < 2) { - responseData.responses().get(0).partitionResponses().get(0).setLogAppendTimeMs(-1); + responseData.responses().iterator().next().partitionResponses().get(0).setLogAppendTimeMs(-1); } if (version < 1) { From b0b3259d0e292ea05a689c64450a617759ec592c Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Mon, 16 Nov 2020 16:33:20 +0800 Subject: [PATCH 09/15] use writable to replace struct --- .../org/apache/kafka/common/requests/ProduceRequest.java | 7 +++++++ .../org/apache/kafka/common/requests/ProduceResponse.java | 7 +++++++ 2 files changed, 14 insertions(+) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java index 9d2cfc484e38a..0eeb94e218093 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -21,9 +21,11 @@ import org.apache.kafka.common.errors.UnsupportedCompressionTypeException; import org.apache.kafka.common.message.ProduceRequestData; import org.apache.kafka.common.message.ProduceResponseData; +import org.apache.kafka.common.network.Send; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.SendBuilder; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.MemoryRecords; @@ -164,6 +166,11 @@ public ProduceRequest(ProduceRequestData produceRequestData, short version) { this.transactionalId = data.transactionalId(); } + @Override + public Send toSend(String destination, RequestHeader header) { + return SendBuilder.buildRequestSend(destination, header, this.data); + } + // visible for testing Map partitionSizes() { if (partitionSizes == null) { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java index fc2670290c431..e5382fea05a03 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -18,8 +18,10 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.message.ProduceResponseData; +import org.apache.kafka.common.network.Send; import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.SendBuilder; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.record.RecordBatch; @@ -78,6 +80,11 @@ public ProduceResponse(Map responses, int thr this(toData(responses, throttleTimeMs)); } + @Override + protected Send toSend(String destination, ResponseHeader header, short apiVersion) { + return SendBuilder.buildResponseSend(destination, header, this.data, apiVersion); + } + private static ProduceResponseData toData(Map responses, int throttleTimeMs) { ProduceResponseData data = new ProduceResponseData().setThrottleTimeMs(throttleTimeMs); responses.forEach((tp, response) -> { From 46449be74ba8cb58970073760f907a2a40d49e86 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Tue, 17 Nov 2020 16:24:48 +0800 Subject: [PATCH 10/15] imporve conversion; add deprecation --- .../clients/producer/internals/Sender.java | 22 ++- .../kafka/common/requests/ProduceRequest.java | 164 ++++++++++-------- .../common/requests/ProduceResponse.java | 6 + .../kafka/common/requests/RequestUtils.java | 48 +++-- .../common/message/ProduceRequest.json | 2 +- .../kafka/clients/NetworkClientTest.java | 6 + .../producer/internals/SenderTest.java | 7 + .../internals/TransactionManagerTest.java | 1 + .../common/requests/ProduceRequestTest.java | 12 ++ .../common/requests/ProduceResponseTest.java | 3 + .../common/requests/RequestResponseTest.java | 5 + .../main/scala/kafka/server/KafkaApis.scala | 14 +- .../kafka/api/AuthorizerIntegrationTest.scala | 2 + .../network/DynamicConnectionQuotaTest.scala | 3 + .../unit/kafka/network/SocketServerTest.scala | 4 + .../kafka/server/EdgeCaseRequestTest.scala | 2 + .../kafka/server/ProduceRequestTest.scala | 6 + .../unit/kafka/server/RequestQuotaTest.scala | 2 + .../producer/ProducerRequestBenchmark.java | 34 ++-- .../producer/ProducerResponseBenchmark.java | 5 + 20 files changed, 235 insertions(+), 113 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index 33219f6825fdc..1b33717d2d837 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -35,6 +35,7 @@ import org.apache.kafka.common.errors.TopicAuthorizationException; import org.apache.kafka.common.errors.TransactionAbortedException; import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; +import org.apache.kafka.common.message.ProduceRequestData; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.metrics.stats.Avg; import org.apache.kafka.common.metrics.stats.Max; @@ -733,7 +734,6 @@ private void sendProduceRequest(long now, int destination, short acks, int timeo if (batches.isEmpty()) return; - Map produceRecordsByPartition = new HashMap<>(batches.size()); final Map recordsByPartition = new HashMap<>(batches.size()); // find the minimum magic version used when creating the record sets @@ -742,7 +742,7 @@ private void sendProduceRequest(long now, int destination, short acks, int timeo 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(); @@ -756,7 +756,14 @@ private void sendProduceRequest(long now, int destination, short acks, int timeo // 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(); - produceRecordsByPartition.put(tp, records); + ProduceRequestData.TopicProduceData tpData = tpd.find(tp.topic()); + if (tpData == null) { + tpData = new ProduceRequestData.TopicProduceData().setName(tp.topic()); + tpd.add(tpData); + } + tpData.partitionData().add(new ProduceRequestData.PartitionProduceData() + .setIndex(tp.partition()) + .setRecords(records)); recordsByPartition.put(tp, batch); } @@ -764,8 +771,13 @@ private void sendProduceRequest(long now, int destination, short acks, int timeo if (transactionManager != null && transactionManager.isTransactional()) { transactionalId = transactionManager.transactionalId(); } - ProduceRequest.Builder requestBuilder = ProduceRequest.Builder.forMagic(minUsedMagic, acks, timeout, - produceRecordsByPartition, transactionalId); + + ProduceRequest.Builder requestBuilder = ProduceRequest.forMagic(minUsedMagic, + new ProduceRequestData() + .setAcks(acks) + .setTimeoutMs(timeout) + .setTransactionalId(transactionalId) + .setTopicData(tpd)); RequestCompletionHandler callback = response -> handleProduceResponse(response, recordsByPartition, time.milliseconds()); String nodeId = Integer.toString(destination); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java index 0eeb94e218093..46aad9e3d0a7b 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -27,6 +27,7 @@ import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.SendBuilder; import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.record.BaseRecords; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.RecordBatch; @@ -37,7 +38,6 @@ import java.util.Collections; import java.util.HashMap; import java.util.Iterator; -import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -45,50 +45,91 @@ public class ProduceRequest extends AbstractRequest { + public static Builder forMagic(byte magic, ProduceRequestData data) { + // 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; + maxVersion = ApiKeys.PRODUCE.latestVersion(); + } + return new Builder(minVersion, maxVersion, data); + } + + private static ProduceRequestData data(short acks, + int timeout, + Map partitionRecords, + String transactionalId) { + return new ProduceRequestData() + .setAcks(acks) + .setTimeoutMs(timeout) + .setTransactionalId(transactionalId) + .setTopicData(new ProduceRequestData.TopicProduceDataCollection(partitionRecords + .entrySet() + .stream() + .collect(Collectors.groupingBy(e -> e.getKey().topic())) + .entrySet() + .stream() + .map(e -> new ProduceRequestData.TopicProduceData() + .setName(e.getKey()) + .setPartitionData(e.getValue().stream() + .map(tpAndRecord -> new ProduceRequestData.PartitionProduceData() + .setIndex(tpAndRecord.getKey().partition()) + .setRecords(tpAndRecord.getValue())) + .collect(Collectors.toList()))) + .iterator())); + } + + public static class Builder extends AbstractRequest.Builder { - private final short acks; - private final int timeout; - private final Map partitionRecords; - private final String transactionalId; + private final ProduceRequestData data; + /** + * @deprecated Since 2.8.0. there is no replacement. + */ + @Deprecated public static Builder forCurrentMagic(short acks, int timeout, Map partitionRecords) { return forMagic(RecordBatch.CURRENT_MAGIC_VALUE, acks, timeout, partitionRecords, null); } + /** + * @deprecated Since 2.8.0. use {@link Builder#forMagic(byte, ProduceRequestData)} instead + */ + @Deprecated public static Builder forMagic(byte magic, short acks, int timeout, Map partitionRecords, String transactionalId) { - // 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; - maxVersion = ApiKeys.PRODUCE.latestVersion(); - } - return new Builder(minVersion, maxVersion, acks, timeout, partitionRecords, transactionalId); + return ProduceRequest.forMagic(magic, data(acks, timeout, partitionRecords, transactionalId)); } + private Builder(short minVersion, + short maxVersion, + ProduceRequestData data) { + super(ApiKeys.PRODUCE, minVersion, maxVersion); + this.data = data; + } + + /** + * @deprecated Since 2.8.0. there is no replacement. + */ + @Deprecated public Builder(short minVersion, short maxVersion, short acks, int timeout, Map partitionRecords, String transactionalId) { - super(ApiKeys.PRODUCE, minVersion, maxVersion); - this.acks = acks; - this.timeout = timeout; - this.partitionRecords = partitionRecords; - this.transactionalId = transactionalId; + this(minVersion, maxVersion, data(acks, timeout, partitionRecords, transactionalId)); } @Override @@ -104,41 +145,21 @@ public ProduceRequest buildUnsafe(short version) { private ProduceRequest build(short version, boolean validate) { if (validate) { // Validate the given records first - for (MemoryRecords records : partitionRecords.values()) { - ProduceRequest.validateRecords(version, records); - } + data.topicData().forEach(tpd -> + tpd.partitionData().forEach(partitionProduceData -> + ProduceRequest.validateRecords(version, partitionProduceData.records()))); } - - List tpd = partitionRecords - .entrySet() - .stream() - .collect(Collectors.groupingBy(e -> e.getKey().topic())) - .entrySet() - .stream() - .map(e -> new ProduceRequestData.TopicProduceData() - .setName(e.getKey()) - .setPartitionData(e.getValue().stream() - .map(tpAndRecord -> new ProduceRequestData.PartitionProduceData() - .setIndex(tpAndRecord.getKey().partition()) - .setRecords(tpAndRecord.getValue())) - .collect(Collectors.toList()))) - .collect(Collectors.toList()); - - return new ProduceRequest(new ProduceRequestData() - .setAcks(acks) - .setTimeoutMs(timeout) - .setTransactionalId(transactionalId) - .setTopicData(tpd), version); + return new ProduceRequest(data, version); } @Override public String toString() { StringBuilder bld = new StringBuilder(); bld.append("(type=ProduceRequest") - .append(", acks=").append(acks) - .append(", timeout=").append(timeout) - .append(", partitionRecords=(").append(partitionRecords) - .append("), transactionalId='").append(transactionalId != null ? transactionalId : "") + .append(", acks=").append(data.acks()) + .append(", timeout=").append(data.timeoutMs()) + .append(", partitionRecords=(").append(data.topicData().stream().flatMap(d -> d.partitionData().stream()).collect(Collectors.toList())) + .append("), transactionalId='").append(data.transactionalId() != null ? data.transactionalId() : "") .append("'"); return bld.toString(); } @@ -168,7 +189,7 @@ public ProduceRequest(ProduceRequestData produceRequestData, short version) { @Override public Send toSend(String destination, RequestHeader header) { - return SendBuilder.buildRequestSend(destination, header, this.data); + return SendBuilder.buildRequestSend(destination, header, dataOrException()); } // visible for testing @@ -274,25 +295,28 @@ public void clearPartitionRecords() { data = null; } - public static void validateRecords(short version, Records records) { + public static void validateRecords(short version, BaseRecords baseRecords) { if (version >= 3) { - Iterator iterator = records.batches().iterator(); - if (!iterator.hasNext()) - throw new InvalidRecordException("Produce requests with version " + version + " must have at least " + - "one record batch"); - - RecordBatch entry = iterator.next(); - if (entry.magic() != RecordBatch.MAGIC_VALUE_V2) - throw new InvalidRecordException("Produce requests with version " + version + " are only allowed to " + - "contain record batches with magic version 2"); - if (version < 7 && entry.compressionType() == CompressionType.ZSTD) { - throw new UnsupportedCompressionTypeException("Produce requests with version " + version + " are not allowed to " + - "use ZStandard compression"); - } + if (baseRecords instanceof Records) { + Records records = (Records) baseRecords; + Iterator iterator = records.batches().iterator(); + if (!iterator.hasNext()) + throw new InvalidRecordException("Produce requests with version " + version + " must have at least " + + "one record batch"); + + RecordBatch entry = iterator.next(); + if (entry.magic() != RecordBatch.MAGIC_VALUE_V2) + throw new InvalidRecordException("Produce requests with version " + version + " are only allowed to " + + "contain record batches with magic version 2"); + if (version < 7 && entry.compressionType() == CompressionType.ZSTD) { + throw new UnsupportedCompressionTypeException("Produce requests with version " + version + " are not allowed to " + + "use ZStandard compression"); + } - if (iterator.hasNext()) - throw new InvalidRecordException("Produce requests with version " + version + " are only allowed to " + - "contain exactly one record batch"); + if (iterator.hasNext()) + throw new InvalidRecordException("Produce requests with version " + version + " are only allowed to " + + "contain exactly one record batch"); + } } // Note that we do not do similar validation for older versions to ensure compatibility with diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java index e5382fea05a03..3b20087a50020 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -66,7 +66,9 @@ public ProduceResponse(ProduceResponseData produceResponseData) { /** * Constructor for Version 0 * @param responses Produced data grouped by topic-partition + * @deprecated Since 2.8.0. This will be removed in a future major release. */ + @Deprecated public ProduceResponse(Map responses) { this(responses, DEFAULT_THROTTLE_TIME); } @@ -75,7 +77,9 @@ public ProduceResponse(Map responses) { * Constructor for the latest version * @param responses Produced data grouped by topic-partition * @param throttleTimeMs Time in milliseconds the response was throttled + * @deprecated Since 2.8.0. This will be removed in a future major release. */ + @Deprecated public ProduceResponse(Map responses, int throttleTimeMs) { this(toData(responses, throttleTimeMs)); } @@ -127,7 +131,9 @@ public ProduceResponseData data() { * this method is used by testing only. * TODO: refactor the tests which are using this method and then remove this method from production code. * https://issues.apache.org/jira/browse/KAFKA-10697 + * @deprecated Since 2.8.0. This will be removed in a future major release. */ + @Deprecated public Map responses() { return data.responses() .stream() diff --git a/clients/src/main/java/org/apache/kafka/common/requests/RequestUtils.java b/clients/src/main/java/org/apache/kafka/common/requests/RequestUtils.java index 13a95275273c3..52145b8322853 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/RequestUtils.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/RequestUtils.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.common.requests; +import org.apache.kafka.common.message.ProduceRequestData; import org.apache.kafka.common.protocol.types.Field; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.record.BaseRecords; @@ -23,9 +24,9 @@ import org.apache.kafka.common.record.Records; import java.nio.ByteBuffer; +import java.util.AbstractMap; import java.util.Iterator; import java.util.Optional; -import java.util.function.Predicate; public final class RequestUtils { @@ -53,27 +54,40 @@ public static ByteBuffer serialize(Struct headerStruct, Struct bodyStruct) { return buffer; } + // visible for testing public static boolean hasIdempotentRecords(ProduceRequest request) { - return anyMatch(request, RecordBatch::hasProducerId); + return flags(request).getKey(); } + // visible for testing public static boolean hasTransactionalRecords(ProduceRequest request) { - return anyMatch(request, RecordBatch::isTransactional); + return flags(request).getValue(); } - private static boolean anyMatch(ProduceRequest request, Predicate predicate) { - return request.dataOrException().topicData() - .stream() - .anyMatch(topicProduceData -> topicProduceData.partitionData() - .stream() - .anyMatch(partitionProduceData -> { - BaseRecords records = partitionProduceData.records(); - if (records instanceof Records) { - Iterator iterator = ((Records) records).batches().iterator(); - return iterator.hasNext() && predicate.test(iterator.next()); - } else return false; - })); + /** + * Get both hasIdempotentRecords flag and hasTransactionalRecords flag from produce request. + * Noted that we find all flags at once to avoid duplicate loop and record batch construction. + * @return first flag is "hasIdempotentRecords" and another is "hasTransactionalRecords" + */ + public static AbstractMap.SimpleEntry flags(ProduceRequest request) { + boolean hasIdempotentRecords = false; + boolean hasTransactionalRecords = false; + for (ProduceRequestData.TopicProduceData tpd : request.dataOrException().topicData()) { + for (ProduceRequestData.PartitionProduceData ppd : tpd.partitionData()) { + BaseRecords records = ppd.records(); + if (records instanceof Records) { + Iterator iterator = ((Records) records).batches().iterator(); + if (iterator.hasNext()) { + RecordBatch batch = iterator.next(); + hasIdempotentRecords = hasIdempotentRecords || batch.hasProducerId(); + hasTransactionalRecords = hasTransactionalRecords || batch.isTransactional(); + } + } + // return early + if (hasIdempotentRecords && hasTransactionalRecords) + return new AbstractMap.SimpleEntry<>(true, true); + } + } + return new AbstractMap.SimpleEntry<>(hasIdempotentRecords, hasTransactionalRecords); } - - } \ No newline at end of file diff --git a/clients/src/main/resources/common/message/ProduceRequest.json b/clients/src/main/resources/common/message/ProduceRequest.json index b626e41e832f2..f5e3c2bb81341 100644 --- a/clients/src/main/resources/common/message/ProduceRequest.json +++ b/clients/src/main/resources/common/message/ProduceRequest.json @@ -41,7 +41,7 @@ "about": "The timeout to await a response in miliseconds." }, { "name": "TopicData", "type": "[]TopicProduceData", "versions": "0+", "about": "Each topic to produce to.", "fields": [ - { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", + { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", "mapKey": true, "about": "The topic name." }, { "name": "PartitionData", "type": "[]PartitionProduceData", "versions": "0+", "about": "Each partition to produce to.", "fields": [ diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java index 00597ca2d21ac..ea32baf5b0ff8 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -147,6 +147,7 @@ public void testDnsLookupFailure() { assertFalse(client.ready(new Node(1234, "badhost", 1234), time.milliseconds())); } + @SuppressWarnings("deprecation") @Test public void testClose() { client.ready(node, time.milliseconds()); @@ -180,6 +181,7 @@ public void testUnsupportedVersionDuringInternalMetadataRequest() { assertEquals(UnsupportedVersionException.class, metadataUpdater.getAndClearFailure().getClass()); } + @SuppressWarnings("deprecation") private void checkSimpleRequestResponse(NetworkClient networkClient) { awaitReady(networkClient, 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 = new ProduceRequest.Builder( @@ -429,6 +431,7 @@ public void testUnsupportedApiVersionsRequestWithoutVersionProvidedByTheBroker() assertTrue(client.isReady(node, time.milliseconds())); } + @SuppressWarnings("deprecation") @Test public void testRequestTimeout() { awaitReady(client, node); // has to be before creating any request, as it may send ApiVersionsRequest and its response is mocked with correlation id 0 @@ -442,6 +445,7 @@ public void testRequestTimeout() { testRequestTimeout(request); } + @SuppressWarnings("deprecation") @Test public void testDefaultRequestTimeout() { awaitReady(client, node); // has to be before creating any request, as it may send ApiVersionsRequest and its response is mocked with correlation id 0 @@ -492,6 +496,7 @@ public void testConnectionSetupTimeout() { ); } + @SuppressWarnings("deprecation") @Test public void testConnectionThrottling() { // Instrument the test to return a response with a 100ms throttle delay. @@ -586,6 +591,7 @@ private int sendEmptyProduceRequest() { return sendEmptyProduceRequest(node.idString()); } + @SuppressWarnings("deprecation") private int sendEmptyProduceRequest(String nodeId) { ProduceRequest.Builder builder = ProduceRequest.Builder.forCurrentMagic((short) 1, 1000, Collections.emptyMap()); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java index 7fc9c0f0b12b7..48aed43692254 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java @@ -218,6 +218,7 @@ public void testMessageFormatDownConversion() throws Exception { 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 @@ -272,6 +273,7 @@ public void testDownConversionForMismatchedMagicValues() throws Exception { /* * Send multiple requests. Verify that the client side quota metrics have the right values */ + @SuppressWarnings("deprecation") @Test public void testQuotaMetrics() { MockSelector selector = new MockSelector(time); @@ -2062,6 +2064,7 @@ public void testTransactionalSplitBatchAndSend() throws Exception { testSplitBatchAndSend(txnManager, producerIdAndEpoch, tp); } + @SuppressWarnings("deprecation") private void testSplitBatchAndSend(TransactionManager txnManager, ProducerIdAndEpoch producerIdAndEpoch, TopicPartition tp) throws Exception { @@ -2175,6 +2178,7 @@ public void testNoDoubleDeallocation() throws Exception { assertEquals(0, sender.inFlightBatches(tp0).size()); } + @SuppressWarnings("deprecation") @Test public void testInflightBatchesExpireOnDeliveryTimeout() throws InterruptedException { long deliveryTimeoutMs = 1500L; @@ -2309,6 +2313,7 @@ public void testResetNextBatchExpiry() throws Exception { } + @SuppressWarnings("deprecation") @Test public void testExpiredBatchesInMultiplePartitions() throws Exception { long deliveryTimeoutMs = 1500L; @@ -2642,6 +2647,7 @@ private FutureRecordMetadata appendToAccumulator(TopicPartition tp, long timesta null, MAX_BLOCK_TIMEOUT, false, time.milliseconds()).future; } + @SuppressWarnings("deprecation") private ProduceResponse produceResponse(TopicPartition tp, long offset, Errors error, int throttleTimeMs, long logStartOffset, String errorMessage) { ProduceResponse.PartitionResponse resp = new ProduceResponse.PartitionResponse(error, offset, RecordBatch.NO_TIMESTAMP, logStartOffset, Collections.emptyList(), errorMessage); @@ -2649,6 +2655,7 @@ private ProduceResponse produceResponse(TopicPartition tp, long offset, Errors e return new ProduceResponse(partResp, throttleTimeMs); } + @SuppressWarnings("deprecation") private ProduceResponse produceResponse(Map responses) { Map partResponses = new LinkedHashMap<>(); for (Map.Entry entry : responses.entrySet()) { diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java index d7d7b8b4594f7..cd71aabb3dfd0 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java @@ -3490,6 +3490,7 @@ private ProduceResponse produceResponse(TopicPartition tp, long offset, Errors e return produceResponse(tp, offset, error, throttleTimeMs, 10); } + @SuppressWarnings("deprecation") private ProduceResponse produceResponse(TopicPartition tp, long offset, Errors error, int throttleTimeMs, int logStartOffset) { ProduceResponse.PartitionResponse resp = new ProduceResponse.PartitionResponse(error, offset, RecordBatch.NO_TIMESTAMP, logStartOffset); Map partResp = singletonMap(tp, resp); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java index 7700bb317bbf8..4ce55fcb884f3 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java @@ -55,6 +55,7 @@ public class ProduceRequestTest { "key".getBytes(), "value".getBytes()); + @SuppressWarnings("deprecation") @Test public void shouldBeFlaggedAsTransactionalWhenTransactionalRecords() throws Exception { final MemoryRecords memoryRecords = MemoryRecords.withTransactionalRecords(0, CompressionType.NONE, 1L, @@ -76,6 +77,7 @@ public void shouldNotBeFlaggedAsIdempotentWhenRecordsNotIdempotent() throws Exce assertFalse(RequestUtils.hasTransactionalRecords(request)); } + @SuppressWarnings("deprecation") @Test public void shouldBeFlaggedAsIdempotentWhenIdempotentRecords() throws Exception { final MemoryRecords memoryRecords = MemoryRecords.withIdempotentRecords(1, CompressionType.NONE, 1L, @@ -85,6 +87,7 @@ public void shouldBeFlaggedAsIdempotentWhenIdempotentRecords() throws Exception assertTrue(RequestUtils.hasIdempotentRecords(request)); } + @SuppressWarnings("deprecation") @Test public void testBuildWithOldMessageFormat() { ByteBuffer buffer = ByteBuffer.allocate(256); @@ -100,6 +103,7 @@ public void testBuildWithOldMessageFormat() { assertEquals(2, requestBuilder.latestAllowedVersion()); } + @SuppressWarnings("deprecation") @Test public void testBuildWithCurrentMessageFormat() { ByteBuffer buffer = ByteBuffer.allocate(256); @@ -115,6 +119,7 @@ public void testBuildWithCurrentMessageFormat() { assertEquals(ApiKeys.PRODUCE.latestVersion(), requestBuilder.latestAllowedVersion()); } + @SuppressWarnings("deprecation") @Test public void testV3AndAboveShouldContainOnlyOneRecordBatch() { ByteBuffer buffer = ByteBuffer.allocate(256); @@ -135,6 +140,7 @@ public void testV3AndAboveShouldContainOnlyOneRecordBatch() { assertThrowsInvalidRecordExceptionForAllVersions(requestBuilder); } + @SuppressWarnings("deprecation") @Test public void testV3AndAboveCannotHaveNoRecordBatches() { Map produceData = new HashMap<>(); @@ -143,6 +149,7 @@ public void testV3AndAboveCannotHaveNoRecordBatches() { assertThrowsInvalidRecordExceptionForAllVersions(requestBuilder); } + @SuppressWarnings("deprecation") @Test public void testV3AndAboveCannotUseMagicV0() { ByteBuffer buffer = ByteBuffer.allocate(256); @@ -156,6 +163,7 @@ public void testV3AndAboveCannotUseMagicV0() { assertThrowsInvalidRecordExceptionForAllVersions(requestBuilder); } + @SuppressWarnings("deprecation") @Test public void testV3AndAboveCannotUseMagicV1() { ByteBuffer buffer = ByteBuffer.allocate(256); @@ -169,6 +177,7 @@ public void testV3AndAboveCannotUseMagicV1() { assertThrowsInvalidRecordExceptionForAllVersions(requestBuilder); } + @SuppressWarnings("deprecation") @Test public void testV6AndBelowCannotUseZStdCompression() { ByteBuffer buffer = ByteBuffer.allocate(256); @@ -189,6 +198,7 @@ public void testV6AndBelowCannotUseZStdCompression() { ProduceRequest.Builder.forCurrentMagic((short) 1, 5000, produceData); } + @SuppressWarnings("deprecation") @Test public void testMixedTransactionalData() { final long producerId = 15L; @@ -213,6 +223,7 @@ public void testMixedTransactionalData() { assertTrue(RequestUtils.hasIdempotentRecords(request)); } + @SuppressWarnings("deprecation") @Test public void testMixedIdempotentData() { final long producerId = 15L; @@ -252,6 +263,7 @@ private void assertThrowsInvalidRecordException(ProduceRequest.Builder builder, } } + @SuppressWarnings("deprecation") private ProduceRequest createNonIdempotentNonTransactionalRecords() { final MemoryRecords memoryRecords = MemoryRecords.withRecords(CompressionType.NONE, simpleRecord); return ProduceRequest.Builder.forCurrentMagic((short) -1, 10, diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ProduceResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ProduceResponseTest.java index 186ab106102a6..1e4eb58d9740e 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/ProduceResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/ProduceResponseTest.java @@ -45,6 +45,7 @@ public class ProduceResponseTest { + @SuppressWarnings("deprecation") @Test public void produceResponseV5Test() { Map responseData = new HashMap<>(); @@ -74,6 +75,7 @@ public void produceResponseV5Test() { assertEquals(responseData, v5Response.responses()); } + @SuppressWarnings("deprecation") @Test public void produceResponseVersionTest() { Map responseData = new HashMap<>(); @@ -96,6 +98,7 @@ public void produceResponseVersionTest() { assertEquals("Response data does not match", responseData, v2Response.responses()); } + @SuppressWarnings("deprecation") @Test public void produceResponseRecordErrorsTest() { Map responseData = new HashMap<>(); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 264e2b38644ed..ef34cf5aad35a 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -625,6 +625,7 @@ public void cannotUseFindCoordinatorV0ToFindTransactionCoordinator() { builder.build((short) 0); } + @SuppressWarnings("deprecation") @Test public void testPartitionSize() { TopicPartition tp0 = new TopicPartition("test", 0); @@ -667,6 +668,7 @@ public void produceRequestToStringTest() { assertFalse(request.toString(true).contains("numPartitions")); } + @SuppressWarnings("deprecation") @Test public void produceRequestGetErrorResponseTest() { ProduceRequest request = createProduceRequest(ApiKeys.PRODUCE.latestVersion()); @@ -1407,6 +1409,7 @@ private OffsetFetchResponse createOffsetFetchResponse() { return new OffsetFetchResponse(Errors.NONE, responseData); } + @SuppressWarnings("deprecation") private ProduceRequest createProduceRequest(int version) { if (version < 2) throw new IllegalArgumentException("Produce request version 2 is not supported"); @@ -1418,6 +1421,7 @@ private ProduceRequest createProduceRequest(int version) { .build((short) version); } + @SuppressWarnings("deprecation") private ProduceResponse createProduceResponse() { Map responseData = new HashMap<>(); responseData.put(new TopicPartition("test", 0), new ProduceResponse.PartitionResponse(Errors.NONE, @@ -1425,6 +1429,7 @@ private ProduceResponse createProduceResponse() { return new ProduceResponse(responseData, 0); } + @SuppressWarnings("deprecation") private ProduceResponse createProduceResponseWithErrorMessage() { Map responseData = new HashMap<>(); responseData.put(new TopicPartition("test", 0), new ProduceResponse.PartitionResponse(Errors.NONE, diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 455aa46e8e7c0..fd93c5fc96e7a 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -90,6 +90,8 @@ import scala.collection.{Map, Seq, Set, immutable, mutable} import scala.util.{Failure, Success, Try} import kafka.coordinator.group.GroupOverview +import scala.annotation.nowarn + /** * Logic to handle the various Kafka requests */ @@ -566,7 +568,11 @@ class KafkaApis(val requestChannel: RequestChannel, val produceRequest = request.body[ProduceRequest] val numBytesAppended = request.header.toStruct.sizeOf + request.sizeOfBodyInBytes - if (RequestUtils.hasTransactionalRecords(produceRequest)) { + val (hasIdempotentRecords, hasTransactionalRecords) = { + val flags = RequestUtils.flags(produceRequest) + (flags.getKey, flags.getValue) + } + if (hasTransactionalRecords) { val isAuthorizedTransactional = produceRequest.transactionalId != null && authorize(request.context, WRITE, TRANSACTIONAL_ID, produceRequest.transactionalId) if (!isAuthorizedTransactional) { @@ -575,7 +581,7 @@ class KafkaApis(val requestChannel: RequestChannel, } // Note that authorization to a transactionalId implies ProducerId authorization - } else if (RequestUtils.hasIdempotentRecords(produceRequest) && !authorize(request.context, IDEMPOTENT_WRITE, CLUSTER, CLUSTER_NAME)) { + } else if (hasIdempotentRecords && !authorize(request.context, IDEMPOTENT_WRITE, CLUSTER, CLUSTER_NAME)) { sendErrorResponseMaybeThrottle(request, Errors.CLUSTER_AUTHORIZATION_FAILED.exception) return } @@ -610,6 +616,10 @@ class KafkaApis(val requestChannel: RequestChannel, }) // the callback for sending a produce response + // TODO: the construction of ProduceResponse is able to accept auto-generated protocol data so + // KafkaApis#handleProduceRequest should apply auto-generated protocol to avoid extra conversion. + // https://issues.apache.org/jira/browse/KAFKA-10730 + @nowarn("cat=deprecation") def sendResponseCallback(responseStatus: Map[TopicPartition, PartitionResponse]): Unit = { val mergedResponseStatus = responseStatus ++ unauthorizedTopicResponses ++ nonExistingTopicResponses ++ invalidRequestResponses var errorInResponse = false diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index c50c5cb776c3a..cbd7647f14dc5 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -152,6 +152,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { classOf[PrincipalBuilder].getName) } + @nowarn("cat=deprecation") val requestKeyToError = Map[ApiKeys, Nothing => Errors]( ApiKeys.METADATA -> ((resp: requests.MetadataResponse) => resp.errors.asScala.find(_._1 == topic).getOrElse(("test", Errors.NONE))._2), ApiKeys.PRODUCE -> ((resp: requests.ProduceResponse) => resp.responses.asScala.find(_._1 == tp).get._2.error), @@ -290,6 +291,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { new requests.MetadataRequest.Builder(List(topic).asJava, allowAutoTopicCreation).build() } + @nowarn("cat=deprecation") private def createProduceRequest = { requests.ProduceRequest.Builder.forCurrentMagic(1, 5000, collection.mutable.Map(tp -> MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes))).asJava). diff --git a/core/src/test/scala/integration/kafka/network/DynamicConnectionQuotaTest.scala b/core/src/test/scala/integration/kafka/network/DynamicConnectionQuotaTest.scala index 78eeeba47f671..98a4e2417f137 100644 --- a/core/src/test/scala/integration/kafka/network/DynamicConnectionQuotaTest.scala +++ b/core/src/test/scala/integration/kafka/network/DynamicConnectionQuotaTest.scala @@ -36,6 +36,7 @@ import org.junit.Assert._ import org.junit.{After, Before, Test} import org.scalatest.Assertions.intercept +import scala.annotation.nowarn import scala.jdk.CollectionConverters._ class DynamicConnectionQuotaTest extends BaseRequestTest { @@ -265,6 +266,7 @@ class DynamicConnectionQuotaTest extends BaseRequestTest { } } + @nowarn("cat=deprecation") private def produceRequest: ProduceRequest = { val topicPartition = new TopicPartition(topic, 0) val memoryRecords = MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord(System.currentTimeMillis(), "key".getBytes, "value".getBytes)) @@ -288,6 +290,7 @@ class DynamicConnectionQuotaTest extends BaseRequestTest { } } + @nowarn("cat=deprecation") private def verifyConnection(socket: Socket): Unit = { val produceResponse = sendAndReceive[ProduceResponse](produceRequest, socket) assertEquals(1, produceResponse.responses.size) diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index 34f5c916110e8..9c9e9c426b141 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -53,6 +53,7 @@ import org.junit.Assert._ import org.junit._ import org.scalatest.Assertions.fail +import scala.annotation.nowarn import scala.jdk.CollectionConverters._ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -205,6 +206,7 @@ class SocketServerTest { server.metrics.close() } + @nowarn("cat=deprecation") private def producerRequestBytes(ack: Short = 0): Array[Byte] = { val correlationId = -1 val clientId = "" @@ -880,6 +882,7 @@ class SocketServerTest { } } + @nowarn("cat=deprecation") @Test def testSslSocketServer(): Unit = { val serverMetrics = new Metrics @@ -925,6 +928,7 @@ class SocketServerTest { checkSaslReauthenticationFailure(false) } + @nowarn("cat=deprecation") def checkSaslReauthenticationFailure(leverageKip152SaslAuthenticateRequest : Boolean): Unit = { shutdownServerAndMetrics(server) // we will use our own instance because we require custom configs val username = "admin" diff --git a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala index 73527a67b210e..e07c13e7c1da7 100755 --- a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala @@ -34,6 +34,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol import org.junit.Assert._ import org.junit.Test +import scala.annotation.nowarn import scala.jdk.CollectionConverters._ class EdgeCaseRequestTest extends KafkaServerTestHarness { @@ -108,6 +109,7 @@ class EdgeCaseRequestTest extends KafkaServerTestHarness { } } + @nowarn("cat=deprecation") @Test def testProduceRequestWithNullClientId(): Unit = { val topic = "topic" diff --git a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala index 4368442a4ca74..bf3c73ffc481b 100644 --- a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala @@ -32,6 +32,7 @@ import org.junit.Assert._ import org.junit.Test import org.scalatest.Assertions.fail +import scala.annotation.nowarn import scala.jdk.CollectionConverters._ /** @@ -42,6 +43,7 @@ class ProduceRequestTest extends BaseRequestTest { val metricsKeySet = KafkaYammerMetrics.defaultRegistry.allMetrics.keySet.asScala + @nowarn("cat=deprecation") @Test def testSimpleProduceRequest(): Unit = { val (partition, leader) = createTopicAndFindPartitionWithLeader("topic") @@ -69,6 +71,7 @@ class ProduceRequestTest extends BaseRequestTest { new SimpleRecord(System.currentTimeMillis(), "key2".getBytes, "value2".getBytes)), 1) } + @nowarn("cat=deprecation") @Test def testProduceWithInvalidTimestamp(): Unit = { val topic = "topic" @@ -105,6 +108,7 @@ class ProduceRequestTest extends BaseRequestTest { assertEquals("One or more records have been rejected due to invalid timestamp", partitionResponse.errorMessage) } + @nowarn("cat=deprecation") @Test def testProduceToNonReplica(): Unit = { val topic = "topic" @@ -136,6 +140,7 @@ class ProduceRequestTest extends BaseRequestTest { }.getOrElse(fail(s"No leader elected for topic $topic")) } + @nowarn("cat=deprecation") @Test def testCorruptLz4ProduceRequest(): Unit = { val (partition, leader) = createTopicAndFindPartitionWithLeader("topic") @@ -159,6 +164,7 @@ class ProduceRequestTest extends BaseRequestTest { assertTrue(TestUtils.meterCount(s"${BrokerTopicStats.InvalidMessageCrcRecordsPerSec}") > 0) } + @nowarn("cat=deprecation") @Test def testZSTDProduceRequest(): Unit = { val topic = "topic" diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index ed886363b174a..d0b94324d3880 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -50,6 +50,7 @@ import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, A import org.junit.Assert._ import org.junit.{After, Before, Test} +import scala.annotation.nowarn import scala.jdk.CollectionConverters._ import scala.collection.mutable.ListBuffer @@ -208,6 +209,7 @@ class RequestQuotaTest extends BaseRequestTest { } } + @nowarn("cat=deprecation") private def requestBuilder(apiKey: ApiKeys): AbstractRequest.Builder[_ <: AbstractRequest] = { apiKey match { case ApiKeys.PRODUCE => diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerRequestBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerRequestBenchmark.java index c14c1d58b744a..58b35f354f92e 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerRequestBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerRequestBenchmark.java @@ -17,12 +17,12 @@ package org.apache.kafka.jmh.producer; -import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.message.ProduceRequestData; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.record.SimpleRecord; import org.apache.kafka.common.requests.ProduceRequest; import org.apache.kafka.common.requests.ProduceResponse; @@ -37,8 +37,8 @@ import org.openjdk.jmh.annotations.Warmup; import java.nio.charset.StandardCharsets; -import java.util.AbstractMap; -import java.util.Map; +import java.util.Collections; +import java.util.List; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -52,25 +52,23 @@ public class ProducerRequestBenchmark { private static final int NUMBER_OF_PARTITIONS = 3; private static final int NUMBER_OF_RECORDS = 3; - private static final Map PARTITION_MEMORY_RECORDS_MAP = IntStream.range(0, NUMBER_OF_PARTITIONS) - .mapToObj(partitionIndex -> new AbstractMap.SimpleEntry<>( - new TopicPartition("tp", partitionIndex), - MemoryRecords.withRecords(CompressionType.NONE, IntStream.range(0, NUMBER_OF_RECORDS) + private static final List TOPIC_PRODUCE_DATA = Collections.singletonList(new ProduceRequestData.TopicProduceData() + .setName("tp") + .setPartitionData(IntStream.range(0, NUMBER_OF_PARTITIONS).mapToObj(partitionIndex -> new ProduceRequestData.PartitionProduceData() + .setIndex(partitionIndex) + .setRecords(MemoryRecords.withRecords(CompressionType.NONE, IntStream.range(0, NUMBER_OF_RECORDS) .mapToObj(recordIndex -> new SimpleRecord(100, "hello0".getBytes(StandardCharsets.UTF_8))) .collect(Collectors.toList()) - .toArray(new SimpleRecord[0])) - )) - .collect(Collectors.toMap(AbstractMap.SimpleEntry::getKey, AbstractMap.SimpleEntry::getValue)); + .toArray(new SimpleRecord[0])))) + .collect(Collectors.toList())) + ); + private static final ProduceRequestData PRODUCE_REQUEST_DATA = new ProduceRequestData() + .setTimeoutMs(100) + .setAcks((short) 1) + .setTopicData(new ProduceRequestData.TopicProduceDataCollection(TOPIC_PRODUCE_DATA.iterator())); private static ProduceRequest request() { - return new ProduceRequest.Builder( - ProduceRequestData.LOWEST_SUPPORTED_VERSION, - ProduceRequestData.HIGHEST_SUPPORTED_VERSION, - (short) 1, - 100, - PARTITION_MEMORY_RECORDS_MAP, - null) - .build(); + return ProduceRequest.forMagic(RecordBatch.CURRENT_MAGIC_VALUE, PRODUCE_REQUEST_DATA).build(); } private static final ProduceRequest REQUEST = request(); diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerResponseBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerResponseBenchmark.java index cacd1eb95e305..c52a7460e6e3a 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerResponseBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerResponseBenchmark.java @@ -62,6 +62,11 @@ public class ProducerResponseBenchmark { )) .collect(Collectors.toMap(AbstractMap.SimpleEntry::getKey, AbstractMap.SimpleEntry::getValue)); + /** + * this method is still used by production so we benchmark it. + * see https://issues.apache.org/jira/browse/KAFKA-10730 + */ + @SuppressWarnings("deprecation") private static ProduceResponse response() { return new ProduceResponse(PARTITION_RESPONSE_MAP); } From f55116a4e06d2cc06bb39dc9645a61a6346b1829 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Wed, 18 Nov 2020 00:42:08 +0800 Subject: [PATCH 11/15] remove deprecation reference of ProduceRequest --- .../kafka/common/requests/ProduceRequest.java | 63 +----- .../common/requests/ProduceResponse.java | 3 - .../kafka/clients/NetworkClientTest.java | 51 +++-- .../producer/internals/SenderTest.java | 8 +- .../common/requests/ProduceRequestTest.java | 195 ++++++++++++------ .../common/requests/RequestResponseTest.java | 31 ++- .../kafka/api/AuthorizerIntegrationTest.scala | 22 +- .../network/DynamicConnectionQuotaTest.scala | 26 ++- .../unit/kafka/network/SocketServerTest.scala | 45 ++-- .../kafka/server/EdgeCaseRequestTest.scala | 21 +- .../kafka/server/ProduceRequestTest.scala | 68 ++++-- .../unit/kafka/server/RequestQuotaTest.scala | 24 ++- 12 files changed, 336 insertions(+), 221 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java index 46aad9e3d0a7b..6860a6f2327ed 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -29,7 +29,6 @@ import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.record.BaseRecords; import org.apache.kafka.common.record.CompressionType; -import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.record.Records; import org.apache.kafka.common.utils.Utils; @@ -62,76 +61,20 @@ public static Builder forMagic(byte magic, ProduceRequestData data) { return new Builder(minVersion, maxVersion, data); } - private static ProduceRequestData data(short acks, - int timeout, - Map partitionRecords, - String transactionalId) { - return new ProduceRequestData() - .setAcks(acks) - .setTimeoutMs(timeout) - .setTransactionalId(transactionalId) - .setTopicData(new ProduceRequestData.TopicProduceDataCollection(partitionRecords - .entrySet() - .stream() - .collect(Collectors.groupingBy(e -> e.getKey().topic())) - .entrySet() - .stream() - .map(e -> new ProduceRequestData.TopicProduceData() - .setName(e.getKey()) - .setPartitionData(e.getValue().stream() - .map(tpAndRecord -> new ProduceRequestData.PartitionProduceData() - .setIndex(tpAndRecord.getKey().partition()) - .setRecords(tpAndRecord.getValue())) - .collect(Collectors.toList()))) - .iterator())); + public static Builder forCurrentMagic(ProduceRequestData data) { + return forMagic(RecordBatch.CURRENT_MAGIC_VALUE, data); } - public static class Builder extends AbstractRequest.Builder { private final ProduceRequestData data; - /** - * @deprecated Since 2.8.0. there is no replacement. - */ - @Deprecated - public static Builder forCurrentMagic(short acks, - int timeout, - Map partitionRecords) { - return forMagic(RecordBatch.CURRENT_MAGIC_VALUE, acks, timeout, partitionRecords, null); - } - - /** - * @deprecated Since 2.8.0. use {@link Builder#forMagic(byte, ProduceRequestData)} instead - */ - @Deprecated - public static Builder forMagic(byte magic, - short acks, - int timeout, - Map partitionRecords, - String transactionalId) { - return ProduceRequest.forMagic(magic, data(acks, timeout, partitionRecords, transactionalId)); - } - - private Builder(short minVersion, + public Builder(short minVersion, short maxVersion, ProduceRequestData data) { super(ApiKeys.PRODUCE, minVersion, maxVersion); this.data = data; } - /** - * @deprecated Since 2.8.0. there is no replacement. - */ - @Deprecated - public Builder(short minVersion, - short maxVersion, - short acks, - int timeout, - Map partitionRecords, - String transactionalId) { - this(minVersion, maxVersion, data(acks, timeout, partitionRecords, transactionalId)); - } - @Override public ProduceRequest build(short version) { return build(version, true); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java index 3b20087a50020..fb17a50b038ab 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -66,7 +66,6 @@ public ProduceResponse(ProduceResponseData produceResponseData) { /** * Constructor for Version 0 * @param responses Produced data grouped by topic-partition - * @deprecated Since 2.8.0. This will be removed in a future major release. */ @Deprecated public ProduceResponse(Map responses) { @@ -77,7 +76,6 @@ public ProduceResponse(Map responses) { * Constructor for the latest version * @param responses Produced data grouped by topic-partition * @param throttleTimeMs Time in milliseconds the response was throttled - * @deprecated Since 2.8.0. This will be removed in a future major release. */ @Deprecated public ProduceResponse(Map responses, int throttleTimeMs) { @@ -131,7 +129,6 @@ public ProduceResponseData data() { * this method is used by testing only. * TODO: refactor the tests which are using this method and then remove this method from production code. * https://issues.apache.org/jira/browse/KAFKA-10697 - * @deprecated Since 2.8.0. This will be removed in a future major release. */ @Deprecated public Map responses() { diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java index ea32baf5b0ff8..93f9ad52a819c 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -25,6 +25,7 @@ import org.apache.kafka.common.message.ApiVersionsResponseData; import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersionsResponseKey; import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersionsResponseKeyCollection; +import org.apache.kafka.common.message.ProduceRequestData; import org.apache.kafka.common.message.ProduceResponseData; import org.apache.kafka.common.network.NetworkReceive; import org.apache.kafka.common.protocol.ApiKeys; @@ -147,7 +148,6 @@ public void testDnsLookupFailure() { assertFalse(client.ready(new Node(1234, "badhost", 1234), time.milliseconds())); } - @SuppressWarnings("deprecation") @Test public void testClose() { client.ready(node, time.milliseconds()); @@ -155,8 +155,11 @@ public void testClose() { client.poll(1, time.milliseconds()); assertTrue("The client should be ready", client.isReady(node, time.milliseconds())); - ProduceRequest.Builder builder = ProduceRequest.Builder.forCurrentMagic((short) 1, 1000, - Collections.emptyMap()); + ProduceRequest.Builder builder = ProduceRequest.forCurrentMagic(new ProduceRequestData() + .setTopicData(new ProduceRequestData.TopicProduceDataCollection()) + .setAcks((short) 1) + .setTimeoutMs(1000) + .setTransactionalId(null)); ClientRequest request = client.newClientRequest(node.idString(), builder, time.milliseconds(), true); client.send(request, time.milliseconds()); assertEquals("There should be 1 in-flight request after send", 1, @@ -181,16 +184,15 @@ public void testUnsupportedVersionDuringInternalMetadataRequest() { assertEquals(UnsupportedVersionException.class, metadataUpdater.getAndClearFailure().getClass()); } - @SuppressWarnings("deprecation") private void checkSimpleRequestResponse(NetworkClient networkClient) { awaitReady(networkClient, 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 = new ProduceRequest.Builder( PRODUCE.latestVersion(), PRODUCE.latestVersion(), - (short) 1, - 1000, - Collections.emptyMap(), - null); + new ProduceRequestData() + .setAcks((short) 1) + .setTimeoutMs(1000) + .setTransactionalId(null)); TestCallbackHandler handler = new TestCallbackHandler(); ClientRequest request = networkClient.newClientRequest(node.idString(), builder, time.milliseconds(), true, defaultRequestTimeoutMs, handler); @@ -431,12 +433,14 @@ public void testUnsupportedApiVersionsRequestWithoutVersionProvidedByTheBroker() assertTrue(client.isReady(node, time.milliseconds())); } - @SuppressWarnings("deprecation") @Test public void testRequestTimeout() { 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.Builder.forCurrentMagic((short) 1, - 1000, Collections.emptyMap()); + ProduceRequest.Builder builder = ProduceRequest.forCurrentMagic(new ProduceRequestData() + .setTopicData(new ProduceRequestData.TopicProduceDataCollection()) + .setAcks((short) 1) + .setTimeoutMs(1000) + .setTransactionalId(null)); TestCallbackHandler handler = new TestCallbackHandler(); int requestTimeoutMs = defaultRequestTimeoutMs + 5000; ClientRequest request = client.newClientRequest(node.idString(), builder, time.milliseconds(), true, @@ -445,12 +449,14 @@ public void testRequestTimeout() { testRequestTimeout(request); } - @SuppressWarnings("deprecation") @Test public void testDefaultRequestTimeout() { 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.Builder.forCurrentMagic((short) 1, - 1000, Collections.emptyMap()); + ProduceRequest.Builder builder = ProduceRequest.forCurrentMagic(new ProduceRequestData() + .setTopicData(new ProduceRequestData.TopicProduceDataCollection()) + .setAcks((short) 1) + .setTimeoutMs(1000) + .setTransactionalId(null)); ClientRequest request = client.newClientRequest(node.idString(), builder, time.milliseconds(), true); assertEquals(defaultRequestTimeoutMs, request.requestTimeoutMs()); testRequestTimeout(request); @@ -496,7 +502,6 @@ public void testConnectionSetupTimeout() { ); } - @SuppressWarnings("deprecation") @Test public void testConnectionThrottling() { // Instrument the test to return a response with a 100ms throttle delay. @@ -504,10 +509,10 @@ public void testConnectionThrottling() { ProduceRequest.Builder builder = new ProduceRequest.Builder( PRODUCE.latestVersion(), PRODUCE.latestVersion(), - (short) 1, - 1000, - Collections.emptyMap(), - null); + new ProduceRequestData() + .setAcks((short) 1) + .setTimeoutMs(1000) + .setTransactionalId(null)); TestCallbackHandler handler = new TestCallbackHandler(); ClientRequest request = client.newClientRequest(node.idString(), builder, time.milliseconds(), true, defaultRequestTimeoutMs, handler); @@ -591,10 +596,12 @@ private int sendEmptyProduceRequest() { return sendEmptyProduceRequest(node.idString()); } - @SuppressWarnings("deprecation") private int sendEmptyProduceRequest(String nodeId) { - ProduceRequest.Builder builder = ProduceRequest.Builder.forCurrentMagic((short) 1, 1000, - Collections.emptyMap()); + ProduceRequest.Builder builder = ProduceRequest.forCurrentMagic(new ProduceRequestData() + .setTopicData(new ProduceRequestData.TopicProduceDataCollection()) + .setAcks((short) 1) + .setTimeoutMs(1000) + .setTransactionalId(null)); TestCallbackHandler handler = new TestCallbackHandler(); ClientRequest request = client.newClientRequest(nodeId, builder, time.milliseconds(), true, defaultRequestTimeoutMs, handler); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java index 48aed43692254..a9f48f5ce92e3 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java @@ -26,6 +26,7 @@ import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.common.errors.InvalidRequestException; import org.apache.kafka.common.errors.TransactionAbortedException; +import org.apache.kafka.common.message.ProduceRequestData; import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType; import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.requests.RequestUtils; @@ -296,8 +297,11 @@ public void testQuotaMetrics() { for (int i = 1; i <= 3; i++) { int throttleTimeMs = 100 * i; - ProduceRequest.Builder builder = ProduceRequest.Builder.forCurrentMagic((short) 1, 1000, - Collections.emptyMap()); + ProduceRequest.Builder builder = ProduceRequest.forCurrentMagic(new ProduceRequestData() + .setTopicData(new ProduceRequestData.TopicProduceDataCollection()) + .setAcks((short) 1) + .setTimeoutMs(1000) + .setTransactionalId(null)); ClientRequest request = client.newClientRequest(node.idString(), builder, time.milliseconds(), true); client.send(request, time.milliseconds()); client.poll(1, time.milliseconds()); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java index 4ce55fcb884f3..ca4079041f095 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java @@ -18,7 +18,6 @@ package org.apache.kafka.common.requests; import org.apache.kafka.common.InvalidRecordException; -import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.message.ProduceRequestData; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.types.ArrayOf; @@ -34,10 +33,8 @@ import org.junit.Test; import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; -import java.util.LinkedHashMap; -import java.util.Map; import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID; import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME; @@ -55,13 +52,22 @@ public class ProduceRequestTest { "key".getBytes(), "value".getBytes()); - @SuppressWarnings("deprecation") @Test public void shouldBeFlaggedAsTransactionalWhenTransactionalRecords() throws Exception { final MemoryRecords memoryRecords = MemoryRecords.withTransactionalRecords(0, CompressionType.NONE, 1L, (short) 1, 1, 1, simpleRecord); - final ProduceRequest request = ProduceRequest.Builder.forCurrentMagic((short) -1, - 10, Collections.singletonMap(new TopicPartition("topic", 1), memoryRecords)).build(); + + final ProduceRequest request = ProduceRequest.forCurrentMagic(new ProduceRequestData() + .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( + new ProduceRequestData.TopicProduceData() + .setName("topic") + .setPartitionData(Collections.singletonList( + new ProduceRequestData.PartitionProduceData() + .setIndex(1) + .setRecords(memoryRecords)))).iterator())) + .setAcks((short) -1) + .setTimeoutMs(10) + .setTransactionalId(null)).build(); assertTrue(RequestUtils.hasTransactionalRecords(request)); } @@ -77,49 +83,62 @@ public void shouldNotBeFlaggedAsIdempotentWhenRecordsNotIdempotent() throws Exce assertFalse(RequestUtils.hasTransactionalRecords(request)); } - @SuppressWarnings("deprecation") @Test public void shouldBeFlaggedAsIdempotentWhenIdempotentRecords() throws Exception { final MemoryRecords memoryRecords = MemoryRecords.withIdempotentRecords(1, CompressionType.NONE, 1L, (short) 1, 1, 1, simpleRecord); - final ProduceRequest request = ProduceRequest.Builder.forCurrentMagic((short) -1, 10, - Collections.singletonMap(new TopicPartition("topic", 1), memoryRecords)).build(); + final ProduceRequest request = ProduceRequest.forCurrentMagic(new ProduceRequestData() + .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( + new ProduceRequestData.TopicProduceData() + .setName("topic") + .setPartitionData(Collections.singletonList( + new ProduceRequestData.PartitionProduceData() + .setIndex(1) + .setRecords(memoryRecords)))).iterator())) + .setAcks((short) -1) + .setTimeoutMs(10) + .setTransactionalId(null)).build(); assertTrue(RequestUtils.hasIdempotentRecords(request)); } - @SuppressWarnings("deprecation") @Test public void testBuildWithOldMessageFormat() { ByteBuffer buffer = ByteBuffer.allocate(256); MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V1, CompressionType.NONE, TimestampType.CREATE_TIME, 0L); builder.append(10L, null, "a".getBytes()); - Map produceData = new HashMap<>(); - produceData.put(new TopicPartition("test", 0), builder.build()); - - ProduceRequest.Builder requestBuilder = ProduceRequest.Builder.forMagic(RecordBatch.MAGIC_VALUE_V1, (short) 1, - 5000, produceData, null); + 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) + .setTransactionalId(null)); assertEquals(2, requestBuilder.oldestAllowedVersion()); assertEquals(2, requestBuilder.latestAllowedVersion()); } - @SuppressWarnings("deprecation") @Test public void testBuildWithCurrentMessageFormat() { ByteBuffer buffer = ByteBuffer.allocate(256); MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, TimestampType.CREATE_TIME, 0L); builder.append(10L, null, "a".getBytes()); - Map produceData = new HashMap<>(); - produceData.put(new TopicPartition("test", 0), builder.build()); - - ProduceRequest.Builder requestBuilder = ProduceRequest.Builder.forMagic(RecordBatch.CURRENT_MAGIC_VALUE, - (short) 1, 5000, produceData, null); + ProduceRequest.Builder requestBuilder = ProduceRequest.forMagic(RecordBatch.CURRENT_MAGIC_VALUE, + 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) + .setTransactionalId(null)); assertEquals(3, requestBuilder.oldestAllowedVersion()); assertEquals(ApiKeys.PRODUCE.latestVersion(), requestBuilder.latestAllowedVersion()); } - @SuppressWarnings("deprecation") @Test public void testV3AndAboveShouldContainOnlyOneRecordBatch() { ByteBuffer buffer = ByteBuffer.allocate(256); @@ -134,22 +153,36 @@ public void testV3AndAboveShouldContainOnlyOneRecordBatch() { buffer.flip(); - Map produceData = new HashMap<>(); - produceData.put(new TopicPartition("test", 0), MemoryRecords.readableRecords(buffer)); - ProduceRequest.Builder requestBuilder = ProduceRequest.Builder.forCurrentMagic((short) 1, 5000, produceData); + ProduceRequest.Builder requestBuilder = ProduceRequest.forCurrentMagic(new ProduceRequestData() + .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( + new ProduceRequestData.TopicProduceData() + .setName("test") + .setPartitionData(Collections.singletonList( + new ProduceRequestData.PartitionProduceData() + .setIndex(0) + .setRecords(MemoryRecords.readableRecords(buffer))))).iterator())) + .setAcks((short) 1) + .setTimeoutMs(5000) + .setTransactionalId(null)); assertThrowsInvalidRecordExceptionForAllVersions(requestBuilder); } - @SuppressWarnings("deprecation") @Test public void testV3AndAboveCannotHaveNoRecordBatches() { - Map produceData = new HashMap<>(); - produceData.put(new TopicPartition("test", 0), MemoryRecords.EMPTY); - ProduceRequest.Builder requestBuilder = ProduceRequest.Builder.forCurrentMagic((short) 1, 5000, produceData); + ProduceRequest.Builder requestBuilder = ProduceRequest.forCurrentMagic(new ProduceRequestData() + .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( + new ProduceRequestData.TopicProduceData() + .setName("test") + .setPartitionData(Collections.singletonList( + new ProduceRequestData.PartitionProduceData() + .setIndex(0) + .setRecords(MemoryRecords.EMPTY)))).iterator())) + .setAcks((short) 1) + .setTimeoutMs(5000) + .setTransactionalId(null)); assertThrowsInvalidRecordExceptionForAllVersions(requestBuilder); } - @SuppressWarnings("deprecation") @Test public void testV3AndAboveCannotUseMagicV0() { ByteBuffer buffer = ByteBuffer.allocate(256); @@ -157,13 +190,20 @@ public void testV3AndAboveCannotUseMagicV0() { TimestampType.NO_TIMESTAMP_TYPE, 0L); builder.append(10L, null, "a".getBytes()); - Map produceData = new HashMap<>(); - produceData.put(new TopicPartition("test", 0), builder.build()); - ProduceRequest.Builder requestBuilder = ProduceRequest.Builder.forCurrentMagic((short) 1, 5000, produceData); + ProduceRequest.Builder requestBuilder = ProduceRequest.forCurrentMagic(new ProduceRequestData() + .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( + new ProduceRequestData.TopicProduceData() + .setName("test") + .setPartitionData(Collections.singletonList( + new ProduceRequestData.PartitionProduceData() + .setIndex(0) + .setRecords(builder.build())))).iterator())) + .setAcks((short) 1) + .setTimeoutMs(5000) + .setTransactionalId(null)); assertThrowsInvalidRecordExceptionForAllVersions(requestBuilder); } - @SuppressWarnings("deprecation") @Test public void testV3AndAboveCannotUseMagicV1() { ByteBuffer buffer = ByteBuffer.allocate(256); @@ -171,13 +211,20 @@ public void testV3AndAboveCannotUseMagicV1() { TimestampType.CREATE_TIME, 0L); builder.append(10L, null, "a".getBytes()); - Map produceData = new HashMap<>(); - produceData.put(new TopicPartition("test", 0), builder.build()); - ProduceRequest.Builder requestBuilder = ProduceRequest.Builder.forCurrentMagic((short) 1, 5000, produceData); + ProduceRequest.Builder requestBuilder = ProduceRequest.forCurrentMagic(new ProduceRequestData() + .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( + new ProduceRequestData.TopicProduceData() + .setName("test") + .setPartitionData(Collections.singletonList(new ProduceRequestData.PartitionProduceData() + .setIndex(0) + .setRecords(builder.build())))) + .iterator())) + .setAcks((short) 1) + .setTimeoutMs(5000) + .setTransactionalId(null)); assertThrowsInvalidRecordExceptionForAllVersions(requestBuilder); } - @SuppressWarnings("deprecation") @Test public void testV6AndBelowCannotUseZStdCompression() { ByteBuffer buffer = ByteBuffer.allocate(256); @@ -185,20 +232,28 @@ public void testV6AndBelowCannotUseZStdCompression() { TimestampType.CREATE_TIME, 0L); builder.append(10L, null, "a".getBytes()); - Map produceData = new HashMap<>(); - produceData.put(new TopicPartition("test", 0), builder.build()); - + ProduceRequestData produceData = new ProduceRequestData() + .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( + new ProduceRequestData.TopicProduceData() + .setName("test") + .setPartitionData(Collections.singletonList(new ProduceRequestData.PartitionProduceData() + .setIndex(0) + .setRecords(builder.build())))) + .iterator())) + .setAcks((short) 1) + .setTimeoutMs(1000) + .setTransactionalId(null); // Can't create ProduceRequest instance with version within [3, 7) for (short version = 3; version < 7; version++) { - ProduceRequest.Builder requestBuilder = new ProduceRequest.Builder(version, version, (short) 1, 5000, produceData, null); + + ProduceRequest.Builder requestBuilder = new ProduceRequest.Builder(version, version, produceData); assertThrowsInvalidRecordExceptionForAllVersions(requestBuilder); } // Works fine with current version (>= 7) - ProduceRequest.Builder.forCurrentMagic((short) 1, 5000, produceData); + ProduceRequest.forCurrentMagic(produceData); } - @SuppressWarnings("deprecation") @Test public void testMixedTransactionalData() { final long producerId = 15L; @@ -211,19 +266,23 @@ public void testMixedTransactionalData() { final MemoryRecords txnRecords = MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, producerEpoch, sequence, new SimpleRecord("bar".getBytes())); - final Map recordsByPartition = new LinkedHashMap<>(); - recordsByPartition.put(new TopicPartition("foo", 0), txnRecords); - recordsByPartition.put(new TopicPartition("foo", 1), nonTxnRecords); - - final ProduceRequest.Builder builder = ProduceRequest.Builder.forMagic(RecordVersion.current().value, (short) -1, 5000, - recordsByPartition, transactionalId); + ProduceRequest.Builder builder = ProduceRequest.forMagic(RecordBatch.CURRENT_MAGIC_VALUE, + new ProduceRequestData() + .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Arrays.asList( + new ProduceRequestData.TopicProduceData().setName("foo").setPartitionData(Collections.singletonList( + new ProduceRequestData.PartitionProduceData().setIndex(0).setRecords(txnRecords))), + new ProduceRequestData.TopicProduceData().setName("foo").setPartitionData(Collections.singletonList( + new ProduceRequestData.PartitionProduceData().setIndex(1).setRecords(nonTxnRecords)))) + .iterator())) + .setAcks((short) 1) + .setTimeoutMs(5000) + .setTransactionalId(null)); final ProduceRequest request = builder.build(); assertTrue(RequestUtils.hasTransactionalRecords(request)); assertTrue(RequestUtils.hasIdempotentRecords(request)); } - @SuppressWarnings("deprecation") @Test public void testMixedIdempotentData() { final long producerId = 15L; @@ -235,12 +294,17 @@ public void testMixedIdempotentData() { final MemoryRecords txnRecords = MemoryRecords.withIdempotentRecords(CompressionType.NONE, producerId, producerEpoch, sequence, new SimpleRecord("bar".getBytes())); - final Map recordsByPartition = new LinkedHashMap<>(); - recordsByPartition.put(new TopicPartition("foo", 0), txnRecords); - recordsByPartition.put(new TopicPartition("foo", 1), nonTxnRecords); - - final ProduceRequest.Builder builder = ProduceRequest.Builder.forMagic(RecordVersion.current().value, (short) -1, 5000, - recordsByPartition, null); + ProduceRequest.Builder builder = ProduceRequest.forMagic(RecordVersion.current().value, + new ProduceRequestData() + .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Arrays.asList( + new ProduceRequestData.TopicProduceData().setName("foo").setPartitionData(Collections.singletonList( + new ProduceRequestData.PartitionProduceData().setIndex(0).setRecords(txnRecords))), + new ProduceRequestData.TopicProduceData().setName("foo").setPartitionData(Collections.singletonList( + new ProduceRequestData.PartitionProduceData().setIndex(1).setRecords(nonTxnRecords)))) + .iterator())) + .setAcks((short) -1) + .setTimeoutMs(5000) + .setTransactionalId(null)); final ProduceRequest request = builder.build(); assertFalse(RequestUtils.hasTransactionalRecords(request)); @@ -263,11 +327,18 @@ private void assertThrowsInvalidRecordException(ProduceRequest.Builder builder, } } - @SuppressWarnings("deprecation") private ProduceRequest createNonIdempotentNonTransactionalRecords() { - final MemoryRecords memoryRecords = MemoryRecords.withRecords(CompressionType.NONE, simpleRecord); - return ProduceRequest.Builder.forCurrentMagic((short) -1, 10, - Collections.singletonMap(new TopicPartition("topic", 1), memoryRecords)).build(); + return ProduceRequest.forCurrentMagic(new ProduceRequestData() + .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( + new ProduceRequestData.TopicProduceData() + .setName("topic") + .setPartitionData(Collections.singletonList(new ProduceRequestData.PartitionProduceData() + .setIndex(1) + .setRecords(MemoryRecords.withRecords(CompressionType.NONE, simpleRecord))))) + .iterator())) + .setAcks((short) -1) + .setTimeoutMs(10) + .setTransactionalId(null)).build(); } /** diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index ef34cf5aad35a..7e51fa2716990 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -131,6 +131,7 @@ import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponsePartitionCollection; import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponseTopic; import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponseTopicCollection; +import org.apache.kafka.common.message.ProduceRequestData; import org.apache.kafka.common.message.RenewDelegationTokenRequestData; import org.apache.kafka.common.message.RenewDelegationTokenResponseData; import org.apache.kafka.common.message.SaslAuthenticateRequestData; @@ -625,7 +626,6 @@ public void cannotUseFindCoordinatorV0ToFindTransactionCoordinator() { builder.build((short) 0); } - @SuppressWarnings("deprecation") @Test public void testPartitionSize() { TopicPartition tp0 = new TopicPartition("test", 0); @@ -634,10 +634,17 @@ public void testPartitionSize() { CompressionType.NONE, new SimpleRecord("woot".getBytes())); MemoryRecords records1 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE, new SimpleRecord("woot".getBytes()), new SimpleRecord("woot".getBytes())); - Map produceData = new HashMap<>(); - produceData.put(tp0, records0); - produceData.put(tp1, records1); - ProduceRequest request = ProduceRequest.Builder.forMagic(RecordBatch.MAGIC_VALUE_V2, (short) 1, 5000, produceData, "transactionalId") + ProduceRequest request = ProduceRequest.forMagic(RecordBatch.MAGIC_VALUE_V2, + new ProduceRequestData() + .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Arrays.asList( + new ProduceRequestData.TopicProduceData().setName(tp0.topic()).setPartitionData( + Collections.singletonList(new ProduceRequestData.PartitionProduceData().setIndex(tp0.partition()).setRecords(records0))), + new ProduceRequestData.TopicProduceData().setName(tp1.topic()).setPartitionData( + Collections.singletonList(new ProduceRequestData.PartitionProduceData().setIndex(tp1.partition()).setRecords(records1)))) + .iterator())) + .setAcks((short) 1) + .setTimeoutMs(5000) + .setTransactionalId("transactionalId")) .build((short) 3); assertEquals(2, request.partitionSizes().size()); assertEquals(records0.sizeInBytes(), (int) request.partitionSizes().get(tp0)); @@ -1413,11 +1420,19 @@ private OffsetFetchResponse createOffsetFetchResponse() { private ProduceRequest createProduceRequest(int version) { if (version < 2) throw new IllegalArgumentException("Produce request version 2 is not supported"); - byte magic = version == 2 ? RecordBatch.MAGIC_VALUE_V1 : RecordBatch.MAGIC_VALUE_V2; MemoryRecords records = MemoryRecords.withRecords(magic, CompressionType.NONE, new SimpleRecord("woot".getBytes())); - Map produceData = Collections.singletonMap(new TopicPartition("test", 0), records); - return ProduceRequest.Builder.forMagic(magic, (short) 1, 5000, produceData, version >= 3 ? "transactionalId" : null) + return ProduceRequest.forMagic(magic, + new ProduceRequestData() + .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( + new ProduceRequestData.TopicProduceData() + .setName("test") + .setPartitionData(Collections.singletonList(new ProduceRequestData.PartitionProduceData() + .setIndex(0) + .setRecords(records)))).iterator())) + .setAcks((short) 1) + .setTimeoutMs(5000) + .setTransactionalId(version >= 3 ? "transactionalId" : null)) .build((short) version); } diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index cbd7647f14dc5..7397f1f634e2a 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -35,7 +35,7 @@ import org.apache.kafka.common.config.internals.BrokerSecurityConfigs import org.apache.kafka.common.config.{ConfigResource, LogLevelConfig} import org.apache.kafka.common.errors._ import org.apache.kafka.common.internals.Topic.GROUP_METADATA_TOPIC_NAME -import org.apache.kafka.common.message.AddOffsetsToTxnRequestData +import org.apache.kafka.common.message.{AddOffsetsToTxnRequestData, AlterPartitionReassignmentsRequestData, AlterReplicaLogDirsRequestData, ControlledShutdownRequestData, CreateAclsRequestData, CreatePartitionsRequestData, CreateTopicsRequestData, DeleteAclsRequestData, DeleteGroupsRequestData, DeleteRecordsRequestData, DeleteTopicsRequestData, DescribeConfigsRequestData, DescribeGroupsRequestData, DescribeLogDirsRequestData, FindCoordinatorRequestData, HeartbeatRequestData, IncrementalAlterConfigsRequestData, JoinGroupRequestData, ListPartitionReassignmentsRequestData, OffsetCommitRequestData, ProduceRequestData, SyncGroupRequestData} import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic import org.apache.kafka.common.message.CreateTopicsRequestData.{CreatableTopic, CreatableTopicCollection} import org.apache.kafka.common.message.IncrementalAlterConfigsRequestData.{AlterConfigsResource, AlterableConfig, AlterableConfigCollection} @@ -45,7 +45,6 @@ import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity import org.apache.kafka.common.message.ListOffsetRequestData.{ListOffsetPartition, ListOffsetTopic} import org.apache.kafka.common.message.StopReplicaRequestData.{StopReplicaPartitionState, StopReplicaTopicState} import org.apache.kafka.common.message.UpdateMetadataRequestData.{UpdateMetadataBroker, UpdateMetadataEndpoint, UpdateMetadataPartitionState} -import org.apache.kafka.common.message.{AlterPartitionReassignmentsRequestData, AlterReplicaLogDirsRequestData, ControlledShutdownRequestData, CreateAclsRequestData, CreatePartitionsRequestData, CreateTopicsRequestData, DeleteAclsRequestData, DeleteGroupsRequestData, DeleteRecordsRequestData, DeleteTopicsRequestData, DescribeConfigsRequestData, DescribeGroupsRequestData, DescribeLogDirsRequestData, FindCoordinatorRequestData, HeartbeatRequestData, IncrementalAlterConfigsRequestData, JoinGroupRequestData, ListPartitionReassignmentsRequestData, OffsetCommitRequestData, SyncGroupRequestData} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch, Records, SimpleRecord} @@ -291,12 +290,19 @@ class AuthorizerIntegrationTest extends BaseRequestTest { new requests.MetadataRequest.Builder(List(topic).asJava, allowAutoTopicCreation).build() } - @nowarn("cat=deprecation") - private def createProduceRequest = { - requests.ProduceRequest.Builder.forCurrentMagic(1, 5000, - collection.mutable.Map(tp -> MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes))).asJava). - build() - } + private def createProduceRequest = + requests.ProduceRequest.forCurrentMagic(new ProduceRequestData() + .setTopicData(new ProduceRequestData.TopicProduceDataCollection( + Collections.singletonList(new ProduceRequestData.TopicProduceData() + .setName(tp.topic()).setPartitionData(Collections.singletonList( + new ProduceRequestData.PartitionProduceData() + .setIndex(tp.partition()) + .setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes)))))) + .iterator)) + .setAcks(1.toShort) + .setTimeoutMs(5000) + .setTransactionalId(null)) + .build() private def createFetchRequest = { val partitionMap = new util.LinkedHashMap[TopicPartition, requests.FetchRequest.PartitionData] diff --git a/core/src/test/scala/integration/kafka/network/DynamicConnectionQuotaTest.scala b/core/src/test/scala/integration/kafka/network/DynamicConnectionQuotaTest.scala index 98a4e2417f137..c67852cef7e4d 100644 --- a/core/src/test/scala/integration/kafka/network/DynamicConnectionQuotaTest.scala +++ b/core/src/test/scala/integration/kafka/network/DynamicConnectionQuotaTest.scala @@ -20,18 +20,19 @@ package kafka.network import java.io.IOException import java.net.{InetAddress, Socket} -import java.util.Properties import java.util.concurrent._ +import java.util.{Collections, Properties} import kafka.server.{BaseRequestTest, KafkaConfig} import kafka.utils.TestUtils import org.apache.kafka.clients.admin.{Admin, AdminClientConfig} +import org.apache.kafka.common.message.ProduceRequestData import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord} import org.apache.kafka.common.requests.{ProduceRequest, ProduceResponse} import org.apache.kafka.common.security.auth.SecurityProtocol -import org.apache.kafka.common.{KafkaException, TopicPartition} +import org.apache.kafka.common.{KafkaException, requests} import org.junit.Assert._ import org.junit.{After, Before, Test} import org.scalatest.Assertions.intercept @@ -266,13 +267,20 @@ class DynamicConnectionQuotaTest extends BaseRequestTest { } } - @nowarn("cat=deprecation") - private def produceRequest: ProduceRequest = { - val topicPartition = new TopicPartition(topic, 0) - val memoryRecords = MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord(System.currentTimeMillis(), "key".getBytes, "value".getBytes)) - val partitionRecords = Map(topicPartition -> memoryRecords) - ProduceRequest.Builder.forCurrentMagic(-1, 3000, partitionRecords.asJava).build() - } + private def produceRequest: ProduceRequest = + requests.ProduceRequest.forCurrentMagic(new ProduceRequestData() + .setTopicData(new ProduceRequestData.TopicProduceDataCollection( + Collections.singletonList(new ProduceRequestData.TopicProduceData() + .setName(topic) + .setPartitionData(Collections.singletonList(new ProduceRequestData.PartitionProduceData() + .setIndex(0) + .setRecords(MemoryRecords.withRecords(CompressionType.NONE, + new SimpleRecord(System.currentTimeMillis(), "key".getBytes, "value".getBytes)))))) + .iterator)) + .setAcks((-1).toShort) + .setTimeoutMs(3000) + .setTransactionalId(null)) + .build() def connectionCount: Int = servers.head.socketServer.connectionCount(localAddress) diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index 9c9e9c426b141..730446726912a 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -24,7 +24,7 @@ import java.nio.channels.{SelectionKey, SocketChannel} import java.nio.charset.StandardCharsets import java.util import java.util.concurrent.{CompletableFuture, ConcurrentLinkedQueue, Executors, TimeUnit} -import java.util.{HashMap, Properties, Random} +import java.util.{Properties, Random} import com.yammer.metrics.core.{Gauge, Meter} import javax.net.ssl._ @@ -33,30 +33,26 @@ import kafka.security.CredentialProvider import kafka.server.{KafkaConfig, ThrottledChannel} import kafka.utils.Implicits._ import kafka.utils.TestUtils -import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.memory.MemoryPool -import org.apache.kafka.common.message.{SaslAuthenticateRequestData, SaslHandshakeRequestData, VoteRequestData} +import org.apache.kafka.common.message.{ProduceRequestData, SaslAuthenticateRequestData, SaslHandshakeRequestData, VoteRequestData} import org.apache.kafka.common.metrics.Metrics -import org.apache.kafka.common.network.ClientInformation import org.apache.kafka.common.network.KafkaChannel.ChannelMuteState -import org.apache.kafka.common.network._ +import org.apache.kafka.common.network.{ClientInformation, _} import org.apache.kafka.common.protocol.{ApiKeys, Errors} -import org.apache.kafka.common.record.MemoryRecords -import org.apache.kafka.common.requests.{AbstractRequest, ApiVersionsRequest, ProduceRequest, RequestHeader, SaslAuthenticateRequest, SaslHandshakeRequest, VoteRequest} +import org.apache.kafka.common.requests +import org.apache.kafka.common.requests._ import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.common.security.scram.internals.ScramMechanism -import org.apache.kafka.common.utils.AppInfoParser -import org.apache.kafka.common.utils.{LogContext, MockTime, Time} +import org.apache.kafka.common.utils.{AppInfoParser, LogContext, MockTime, Time} import org.apache.kafka.test.{TestSslUtils, TestUtils => JTestUtils} import org.apache.log4j.Level import org.junit.Assert._ import org.junit._ import org.scalatest.Assertions.fail -import scala.annotation.nowarn -import scala.jdk.CollectionConverters._ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import scala.jdk.CollectionConverters._ import scala.util.control.ControlThrowable class SocketServerTest { @@ -206,14 +202,17 @@ class SocketServerTest { server.metrics.close() } - @nowarn("cat=deprecation") private def producerRequestBytes(ack: Short = 0): Array[Byte] = { val correlationId = -1 val clientId = "" val ackTimeoutMs = 10000 - val emptyRequest = ProduceRequest.Builder.forCurrentMagic(ack, ackTimeoutMs, - new HashMap[TopicPartition, MemoryRecords]()).build() + val emptyRequest = requests.ProduceRequest.forCurrentMagic(new ProduceRequestData() + .setTopicData(new ProduceRequestData.TopicProduceDataCollection()) + .setAcks(ack) + .setTimeoutMs(ackTimeoutMs) + .setTransactionalId(null)) + .build() val emptyHeader = new RequestHeader(ApiKeys.PRODUCE, emptyRequest.version, clientId, correlationId) val byteBuffer = emptyRequest.serialize(emptyHeader) byteBuffer.rewind() @@ -882,7 +881,6 @@ class SocketServerTest { } } - @nowarn("cat=deprecation") @Test def testSslSocketServer(): Unit = { val serverMetrics = new Metrics @@ -900,8 +898,12 @@ class SocketServerTest { val clientId = "" val ackTimeoutMs = 10000 val ack = 0: Short - val emptyRequest = ProduceRequest.Builder.forCurrentMagic(ack, ackTimeoutMs, - new HashMap[TopicPartition, MemoryRecords]()).build() + val emptyRequest = requests.ProduceRequest.forCurrentMagic(new ProduceRequestData() + .setTopicData(new ProduceRequestData.TopicProduceDataCollection()) + .setAcks(ack) + .setTimeoutMs(ackTimeoutMs) + .setTransactionalId(null)) + .build() val emptyHeader = new RequestHeader(ApiKeys.PRODUCE, emptyRequest.version, clientId, correlationId) val byteBuffer = emptyRequest.serialize(emptyHeader) @@ -928,7 +930,6 @@ class SocketServerTest { checkSaslReauthenticationFailure(false) } - @nowarn("cat=deprecation") def checkSaslReauthenticationFailure(leverageKip152SaslAuthenticateRequest : Boolean): Unit = { shutdownServerAndMetrics(server) // we will use our own instance because we require custom configs val username = "admin" @@ -984,8 +985,12 @@ class SocketServerTest { // ...and now send something to trigger the disconnection val ackTimeoutMs = 10000 val ack = 0: Short - val emptyRequest = ProduceRequest.Builder.forCurrentMagic(ack, ackTimeoutMs, - new HashMap[TopicPartition, MemoryRecords]()).build() + val emptyRequest = requests.ProduceRequest.forCurrentMagic(new ProduceRequestData() + .setTopicData(new ProduceRequestData.TopicProduceDataCollection()) + .setAcks(ack) + .setTimeoutMs(ackTimeoutMs) + .setTransactionalId(null)) + .build() val emptyHeader = new RequestHeader(ApiKeys.PRODUCE, emptyRequest.version, clientId, correlationId) sendApiRequest(socket, emptyRequest, emptyHeader) // wait a little bit for the server-side disconnection to occur since it happens asynchronously diff --git a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala index e07c13e7c1da7..a877037be96bd 100755 --- a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala @@ -20,22 +20,23 @@ package kafka.server import java.io.{DataInputStream, DataOutputStream} import java.net.Socket import java.nio.ByteBuffer +import java.util.Collections import kafka.integration.KafkaServerTestHarness import kafka.network.SocketServer import kafka.utils._ -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.message.ProduceRequestData import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.types.Type import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord} -import org.apache.kafka.common.requests.{ProduceRequest, ProduceResponse, ResponseHeader} +import org.apache.kafka.common.requests.{ProduceResponse, ResponseHeader} import org.apache.kafka.common.security.auth.SecurityProtocol +import org.apache.kafka.common.{TopicPartition, requests} import org.junit.Assert._ import org.junit.Test import scala.annotation.nowarn -import scala.jdk.CollectionConverters._ class EdgeCaseRequestTest extends KafkaServerTestHarness { @@ -121,8 +122,18 @@ class EdgeCaseRequestTest extends KafkaServerTestHarness { val (serializedBytes, responseHeaderVersion) = { val headerBytes = requestHeaderBytes(ApiKeys.PRODUCE.id, version, null, correlationId) - val records = MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("message".getBytes)) - val request = ProduceRequest.Builder.forCurrentMagic(1, 10000, Map(topicPartition -> records).asJava).build() + val request = requests.ProduceRequest.forCurrentMagic(new ProduceRequestData() + .setTopicData(new ProduceRequestData.TopicProduceDataCollection( + Collections.singletonList(new ProduceRequestData.TopicProduceData() + .setName(topicPartition.topic()).setPartitionData(Collections.singletonList( + new ProduceRequestData.PartitionProduceData() + .setIndex(topicPartition.partition()) + .setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("message".getBytes)))))) + .iterator)) + .setAcks(1.toShort) + .setTimeoutMs(10000) + .setTransactionalId(null)) + .build() val byteBuffer = ByteBuffer.allocate(headerBytes.length + request.toStruct.sizeOf) byteBuffer.put(headerBytes) request.toStruct.writeTo(byteBuffer) diff --git a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala index bf3c73ffc481b..ca43d3b8dba05 100644 --- a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala @@ -18,13 +18,14 @@ package kafka.server import java.nio.ByteBuffer -import java.util.Properties +import java.util.{Collections, Properties} import kafka.log.LogConfig import kafka.message.ZStdCompressionCodec import kafka.metrics.KafkaYammerMetrics import kafka.utils.TestUtils import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.message.ProduceRequestData import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record._ import org.apache.kafka.common.requests.{ProduceRequest, ProduceResponse} @@ -50,9 +51,17 @@ class ProduceRequestTest extends BaseRequestTest { def sendAndCheck(memoryRecords: MemoryRecords, expectedOffset: Long): ProduceResponse.PartitionResponse = { val topicPartition = new TopicPartition("topic", partition) - val partitionRecords = Map(topicPartition -> memoryRecords) val produceResponse = sendProduceRequest(leader, - ProduceRequest.Builder.forCurrentMagic(-1, 3000, partitionRecords.asJava).build()) + ProduceRequest.forCurrentMagic(new ProduceRequestData() + .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( + new ProduceRequestData.TopicProduceData() + .setName(topicPartition.topic()) + .setPartitionData(Collections.singletonList(new ProduceRequestData.PartitionProduceData() + .setIndex(topicPartition.partition()) + .setRecords(memoryRecords)))).iterator)) + .setAcks((-1).toShort) + .setTimeoutMs(3000) + .setTransactionalId(null)).build()) assertEquals(1, produceResponse.responses.size) val (tp, partitionResponse) = produceResponse.responses.asScala.head assertEquals(topicPartition, tp) @@ -92,8 +101,16 @@ class ProduceRequestTest extends BaseRequestTest { val records = createRecords(RecordBatch.MAGIC_VALUE_V2, System.currentTimeMillis() - 1001L, CompressionType.GZIP) val topicPartition = new TopicPartition("topic", partition) - val partitionRecords = Map(topicPartition -> records) - val produceResponse = sendProduceRequest(leader, ProduceRequest.Builder.forCurrentMagic(-1, 3000, partitionRecords.asJava).build()) + val produceResponse = sendProduceRequest(leader, ProduceRequest.forCurrentMagic(new ProduceRequestData() + .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( + new ProduceRequestData.TopicProduceData() + .setName(topicPartition.topic()) + .setPartitionData(Collections.singletonList(new ProduceRequestData.PartitionProduceData() + .setIndex(topicPartition.partition()) + .setRecords(records)))).iterator)) + .setAcks((-1).toShort) + .setTimeoutMs(3000) + .setTransactionalId(null)).build()) val (tp, partitionResponse) = produceResponse.responses.asScala.head assertEquals(topicPartition, tp) assertEquals(Errors.INVALID_TIMESTAMP, partitionResponse.error) @@ -124,8 +141,16 @@ class ProduceRequestTest extends BaseRequestTest { // Send the produce request to the non-replica val records = MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("key".getBytes, "value".getBytes)) val topicPartition = new TopicPartition("topic", partition) - val partitionRecords = Map(topicPartition -> records) - val produceRequest = ProduceRequest.Builder.forCurrentMagic(-1, 3000, partitionRecords.asJava).build() + val produceRequest = ProduceRequest.forCurrentMagic(new ProduceRequestData() + .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( + new ProduceRequestData.TopicProduceData() + .setName(topicPartition.topic()) + .setPartitionData(Collections.singletonList(new ProduceRequestData.PartitionProduceData() + .setIndex(topicPartition.partition()) + .setRecords(records)))).iterator)) + .setAcks((-1).toShort) + .setTimeoutMs(3000) + .setTransactionalId(null)).build() val produceResponse = sendProduceRequest(nonReplicaId, produceRequest) assertEquals(1, produceResponse.responses.size) @@ -151,9 +176,16 @@ class ProduceRequestTest extends BaseRequestTest { val lz4ChecksumOffset = 6 memoryRecords.buffer.array.update(DefaultRecordBatch.RECORD_BATCH_OVERHEAD + lz4ChecksumOffset, 0) val topicPartition = new TopicPartition("topic", partition) - val partitionRecords = Map(topicPartition -> memoryRecords) - val produceResponse = sendProduceRequest(leader, - ProduceRequest.Builder.forCurrentMagic(-1, 3000, partitionRecords.asJava).build()) + val produceResponse = sendProduceRequest(leader, ProduceRequest.forCurrentMagic(new ProduceRequestData() + .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( + new ProduceRequestData.TopicProduceData() + .setName(topicPartition.topic()) + .setPartitionData(Collections.singletonList(new ProduceRequestData.PartitionProduceData() + .setIndex(topicPartition.partition()) + .setRecords(memoryRecords)))).iterator)) + .setAcks((-1).toShort) + .setTimeoutMs(3000) + .setTransactionalId(null)).build()) assertEquals(1, produceResponse.responses.size) val (tp, partitionResponse) = produceResponse.responses.asScala.head assertEquals(topicPartition, tp) @@ -178,11 +210,21 @@ class ProduceRequestTest extends BaseRequestTest { val memoryRecords = MemoryRecords.withRecords(CompressionType.ZSTD, new SimpleRecord(System.currentTimeMillis(), "key".getBytes, "value".getBytes)) val topicPartition = new TopicPartition("topic", partition) - val partitionRecords = Map(topicPartition -> memoryRecords) + val partitionRecords = new ProduceRequestData() + .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( + new ProduceRequestData.TopicProduceData() + .setName("topic").setPartitionData(Collections.singletonList( + new ProduceRequestData.PartitionProduceData() + .setIndex(partition) + .setRecords(memoryRecords)))) + .iterator)) + .setAcks((-1).toShort) + .setTimeoutMs(3000) + .setTransactionalId(null) // produce request with v7: works fine! val res1 = sendProduceRequest(leader, - new ProduceRequest.Builder(7, 7, -1, 3000, partitionRecords.asJava, null).build()) + new ProduceRequest.Builder(7, 7, partitionRecords).build()) val (tp1, partitionResponse1) = res1.responses.asScala.head assertEquals(topicPartition, tp1) assertEquals(Errors.NONE, partitionResponse1.error) @@ -191,7 +233,7 @@ class ProduceRequestTest extends BaseRequestTest { // produce request with v3: returns Errors.UNSUPPORTED_COMPRESSION_TYPE. val res2 = sendProduceRequest(leader, - new ProduceRequest.Builder(3, 3, -1, 3000, partitionRecords.asJava, null) + new ProduceRequest.Builder(3, 3, partitionRecords) .buildUnsafe(3)) val (tp2, partitionResponse2) = res2.responses.asScala.head assertEquals(topicPartition, tp2) diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index d0b94324d3880..bd1fb2f093691 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -26,7 +26,6 @@ import kafka.security.authorizer.AclAuthorizer import kafka.utils.TestUtils import org.apache.kafka.common.acl._ import org.apache.kafka.common.config.ConfigResource -import org.apache.kafka.common.message.AddOffsetsToTxnRequestData import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic import org.apache.kafka.common.message.CreateTopicsRequestData.{CreatableTopic, CreatableTopicCollection} import org.apache.kafka.common.message.JoinGroupRequestData.JoinGroupRequestProtocolCollection @@ -35,7 +34,7 @@ import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity import org.apache.kafka.common.message.ListOffsetRequestData.{ListOffsetPartition, ListOffsetTopic} import org.apache.kafka.common.message.StopReplicaRequestData.{StopReplicaPartitionState, StopReplicaTopicState} import org.apache.kafka.common.message.UpdateMetadataRequestData.{UpdateMetadataBroker, UpdateMetadataEndpoint, UpdateMetadataPartitionState} -import org.apache.kafka.common.message._ +import org.apache.kafka.common.message.{AddOffsetsToTxnRequestData, _} import org.apache.kafka.common.metrics.{KafkaMetric, Quota, Sensor} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.ApiKeys @@ -43,16 +42,15 @@ import org.apache.kafka.common.quota.ClientQuotaFilter import org.apache.kafka.common.record._ import org.apache.kafka.common.requests._ import org.apache.kafka.common.resource.{PatternType, ResourceType => AdminResourceType} -import org.apache.kafka.common.security.auth.{AuthenticationContext, KafkaPrincipal, KafkaPrincipalBuilder, KafkaPrincipalSerde, SecurityProtocol} +import org.apache.kafka.common.security.auth._ import org.apache.kafka.common.utils.{Sanitizer, SecurityUtils} -import org.apache.kafka.common.{ElectionType, IsolationLevel, Node, TopicPartition} +import org.apache.kafka.common._ import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult} import org.junit.Assert._ import org.junit.{After, Before, Test} -import scala.annotation.nowarn -import scala.jdk.CollectionConverters._ import scala.collection.mutable.ListBuffer +import scala.jdk.CollectionConverters._ class RequestQuotaTest extends BaseRequestTest { @@ -209,12 +207,20 @@ class RequestQuotaTest extends BaseRequestTest { } } - @nowarn("cat=deprecation") private def requestBuilder(apiKey: ApiKeys): AbstractRequest.Builder[_ <: AbstractRequest] = { apiKey match { case ApiKeys.PRODUCE => - ProduceRequest.Builder.forCurrentMagic(1, 5000, - collection.mutable.Map(tp -> MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes))).asJava) + requests.ProduceRequest.forCurrentMagic(new ProduceRequestData() + .setTopicData(new ProduceRequestData.TopicProduceDataCollection( + Collections.singletonList(new ProduceRequestData.TopicProduceData() + .setName(tp.topic()).setPartitionData(Collections.singletonList( + new ProduceRequestData.PartitionProduceData() + .setIndex(tp.partition()) + .setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes)))))) + .iterator)) + .setAcks(1.toShort) + .setTimeoutMs(5000) + .setTransactionalId(null)) case ApiKeys.FETCH => val partitionMap = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData] From bc11f9012963ba11dd60189dbdbdffde31ec2478 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Wed, 18 Nov 2020 11:00:30 +0800 Subject: [PATCH 12/15] remove compatibility test and redundant TODO --- .../clients/producer/internals/Sender.java | 2 +- .../common/requests/ProduceResponse.java | 2 +- .../common/requests/ProduceRequestTest.java | 47 ----------- .../common/requests/ProduceResponseTest.java | 84 ------------------- .../common/requests/SchemaTestUtils.java | 47 ----------- .../main/scala/kafka/server/KafkaApis.scala | 3 +- 6 files changed, 3 insertions(+), 182 deletions(-) delete mode 100644 clients/src/test/java/org/apache/kafka/common/requests/SchemaTestUtils.java diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index 1b33717d2d837..1c9e118c0fdda 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -562,7 +562,7 @@ private void handleProduceResponse(ClientResponse response, Map r.partitionResponses().forEach(p -> { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java index fb17a50b038ab..3b089cdda9935 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -127,7 +127,7 @@ public ProduceResponseData data() { /** * this method is used by testing only. - * TODO: refactor the tests which are using this method and then remove this method from production code. + * refactor the tests which are using this method and then remove this method from production code. * https://issues.apache.org/jira/browse/KAFKA-10697 */ @Deprecated diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java index ca4079041f095..359e788e55027 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java @@ -20,9 +20,6 @@ import org.apache.kafka.common.InvalidRecordException; import org.apache.kafka.common.message.ProduceRequestData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.types.ArrayOf; -import org.apache.kafka.common.protocol.types.Field; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecordsBuilder; @@ -36,11 +33,6 @@ import java.util.Arrays; import java.util.Collections; -import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID; -import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME; -import static org.apache.kafka.common.protocol.types.Type.INT16; -import static org.apache.kafka.common.protocol.types.Type.INT32; -import static org.apache.kafka.common.protocol.types.Type.RECORDS; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -340,43 +332,4 @@ private ProduceRequest createNonIdempotentNonTransactionalRecords() { .setTimeoutMs(10) .setTransactionalId(null)).build(); } - - /** - * the schema in this test is from previous code and the automatic protocol should be compatible to previous schema. - */ - @Test - public void testCompatibility() { - String acksKeyName = "acks"; - String timeoutKeyName = "timeout"; - String topicDataKeyName = "topic_data"; - String partitionDataKeyName = "data"; - String recordSetKeyName = "record_set"; - Schema topicProduceData0 = new Schema(TOPIC_NAME, - new Field(partitionDataKeyName, new ArrayOf(new Schema(PARTITION_ID, new Field(recordSetKeyName, RECORDS))))); - Schema produceRequestV0 = new Schema( - new Field(acksKeyName, INT16), - new Field(timeoutKeyName, INT32), - new Field(topicDataKeyName, new ArrayOf(topicProduceData0))); - Schema produceRequestV1 = produceRequestV0; - Schema produceRequestV2 = produceRequestV1; - Schema produceRequestV3 = new Schema( - new Field.NullableStr("transactional_id", "The transactional id or null if the producer is not transactional"), - new Field(acksKeyName, INT16), - new Field(timeoutKeyName, INT32), - new Field(topicDataKeyName, new ArrayOf(topicProduceData0))); - Schema produceRequestV4 = produceRequestV3; - Schema produceRequestV5 = produceRequestV4; - Schema produceRequestV6 = produceRequestV5; - Schema produceRequestV7 = produceRequestV6; - Schema produceRequestV8 = produceRequestV7; - Schema[] schemaVersions = new Schema[] { - produceRequestV0, produceRequestV1, produceRequestV2, - produceRequestV3, produceRequestV4, produceRequestV5, - produceRequestV6, produceRequestV7, produceRequestV8 - }; - int schemaVersion = 0; - for (Schema previousSchema : schemaVersions) { - SchemaTestUtils.assertEquals(previousSchema, ProduceRequestData.SCHEMAS[schemaVersion++]); - } - } } diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ProduceResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ProduceResponseTest.java index 1e4eb58d9740e..37cac84db0c15 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/ProduceResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/ProduceResponseTest.java @@ -21,9 +21,6 @@ import org.apache.kafka.common.message.ProduceResponseData; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.types.ArrayOf; -import org.apache.kafka.common.protocol.types.Field; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.record.RecordBatch; import org.junit.Test; @@ -34,11 +31,6 @@ import java.util.Map; import static org.apache.kafka.common.protocol.ApiKeys.PRODUCE; -import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE; -import static org.apache.kafka.common.protocol.CommonFields.PARTITION_ID; -import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS; -import static org.apache.kafka.common.protocol.CommonFields.TOPIC_NAME; -import static org.apache.kafka.common.protocol.types.Type.INT64; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; @@ -125,80 +117,4 @@ public void produceResponseRecordErrorsTest() { } } } - - /** - * the schema in this test is from previous code and the automatic protocol should be compatible to previous schema. - */ - @Test - public void testCompatibility() { - String responseKeyName = "responses"; - String partitionResponsesKeyName = "partition_responses"; - long invalidOffset = -1L; - String baseOffsetKeyName = "base_offset"; - String logAppendTimeKeyName = "log_append_time"; - String logStartOffsetKeyName = "log_start_offset"; - String recordErrorsKeyName = "record_errors"; - String batchIndexKeyName = "batch_index"; - String batchIndexErrorMessageKeyName = "batch_index_error_message"; - String errorMessageKeyName = "error_message"; - - Field.Int64 logStartOffsetField = new Field.Int64(logStartOffsetKeyName, - "The start offset of the log at the time this produce response was created", invalidOffset); - Field.NullableStr batchIndexErrorMessageField = new Field.NullableStr(batchIndexErrorMessageKeyName, - "The error message of the record that caused the batch to be dropped"); - Field.NullableStr errorMessageField = new Field.NullableStr(errorMessageKeyName, - "The global error message summarizing the common root cause of the records that caused the batch to be dropped"); - - Schema produceResponseV0 = new Schema( - new Field(responseKeyName, new ArrayOf(new Schema(TOPIC_NAME, - new Field(partitionResponsesKeyName, new ArrayOf(new Schema(PARTITION_ID, ERROR_CODE, - new Field(baseOffsetKeyName, INT64)))))))); - - Schema produceResponseV1 = new Schema( - new Field(responseKeyName, new ArrayOf(new Schema(TOPIC_NAME, - new Field(partitionResponsesKeyName, new ArrayOf(new Schema(PARTITION_ID, ERROR_CODE, - new Field(baseOffsetKeyName, INT64))))))), - THROTTLE_TIME_MS); - - Schema produceResponseV2 = new Schema( - new Field(responseKeyName, new ArrayOf(new Schema(TOPIC_NAME, - new Field(partitionResponsesKeyName, new ArrayOf(new Schema(PARTITION_ID, ERROR_CODE, - new Field(baseOffsetKeyName, INT64), - new Field(logAppendTimeKeyName, INT64))))))), - THROTTLE_TIME_MS); - Schema produceResponseV3 = produceResponseV2; - Schema produceResponseV4 = produceResponseV3; - Schema produceResponseV5 = new Schema( - new Field(responseKeyName, new ArrayOf(new Schema(TOPIC_NAME, - new Field(partitionResponsesKeyName, new ArrayOf(new Schema(PARTITION_ID, ERROR_CODE, - new Field(baseOffsetKeyName, INT64), - new Field(logAppendTimeKeyName, INT64), - logStartOffsetField)))))), - THROTTLE_TIME_MS); - Schema produceResponseV6 = produceResponseV5; - Schema produceResponseV7 = produceResponseV6; - Schema produceResponseV8 = new Schema( - new Field(responseKeyName, new ArrayOf(new Schema(TOPIC_NAME, - new Field(partitionResponsesKeyName, new ArrayOf(new Schema(PARTITION_ID, ERROR_CODE, - new Field(baseOffsetKeyName, INT64), - new Field(logAppendTimeKeyName, INT64), - logStartOffsetField, - new Field(recordErrorsKeyName, new ArrayOf(new Schema( - new Field.Int32(batchIndexKeyName, "The batch index of the record " + - "that caused the batch to be dropped"), - batchIndexErrorMessageField - ))), - errorMessageField)))))), - THROTTLE_TIME_MS); - - Schema[] schemaVersions = new Schema[]{ - produceResponseV0, produceResponseV1, produceResponseV2, - produceResponseV3, produceResponseV4, produceResponseV5, - produceResponseV6, produceResponseV7, produceResponseV8}; - - int schemaVersion = 0; - for (Schema previousSchema : schemaVersions) { - SchemaTestUtils.assertEquals(previousSchema, ProduceResponseData.SCHEMAS[schemaVersion++]); - } - } } diff --git a/clients/src/test/java/org/apache/kafka/common/requests/SchemaTestUtils.java b/clients/src/test/java/org/apache/kafka/common/requests/SchemaTestUtils.java deleted file mode 100644 index c39d8e7708439..0000000000000 --- a/clients/src/test/java/org/apache/kafka/common/requests/SchemaTestUtils.java +++ /dev/null @@ -1,47 +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 org.apache.kafka.common.requests; - -import org.apache.kafka.common.protocol.types.ArrayOf; -import org.apache.kafka.common.protocol.types.BoundField; -import org.apache.kafka.common.protocol.types.Schema; -import org.apache.kafka.common.protocol.types.Type; -import org.junit.Assert; - -import static org.junit.Assert.assertTrue; - -public final class SchemaTestUtils { - - private SchemaTestUtils() { - } - - static void assertEquals(Type lhs, Type rhs) { - if (lhs instanceof Schema) { - assertTrue("excepted: Schema, actual: " + rhs, rhs instanceof Schema); - Schema lhsSchema = (Schema) lhs; - Schema rhsSchema = (Schema) rhs; - Assert.assertEquals(lhsSchema.numFields(), rhsSchema.numFields()); - int fieldIndex = 0; - for (BoundField f : lhsSchema.fields()) assertEquals(f.def.type, rhsSchema.fields()[fieldIndex++].def.type); - } else if (lhs instanceof ArrayOf) { - assertTrue("excepted: ArrayOf, actual: " + rhs, rhs instanceof ArrayOf); - lhs.arrayElementType().ifPresent(lhsType -> - rhs.arrayElementType().ifPresent(rhsType -> assertEquals(lhsType, rhsType))); - } else Assert.assertEquals(lhs, rhs); - } -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index fd93c5fc96e7a..4392406de5c55 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -598,7 +598,6 @@ class KafkaApis(val requestChannel: RequestChannel, val topicPartition = new TopicPartition(topic.name, partition.index) // This caller assumes the type is MemoryRecords and that is true on current serialization // We cast the type to avoid causing big change to code base. - // TODO: maybe we need to refactor code to avoid this casting // https://issues.apache.org/jira/browse/KAFKA-10698 val memoryRecords = partition.records.asInstanceOf[MemoryRecords] if (!authorizedTopics.contains(topicPartition.topic)) @@ -616,7 +615,7 @@ class KafkaApis(val requestChannel: RequestChannel, }) // the callback for sending a produce response - // TODO: the construction of ProduceResponse is able to accept auto-generated protocol data so + // The construction of ProduceResponse is able to accept auto-generated protocol data so // KafkaApis#handleProduceRequest should apply auto-generated protocol to avoid extra conversion. // https://issues.apache.org/jira/browse/KAFKA-10730 @nowarn("cat=deprecation") From a2ca27d3012eb7a5c0868bce295dd895176475de Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Wed, 18 Nov 2020 20:30:44 +0800 Subject: [PATCH 13/15] address review comment --- .../internals/TransactionManager.java | 1 - .../kafka/common/requests/RequestUtils.java | 2 +- .../common/message/ProduceRequest.json | 2 +- .../kafka/clients/NetworkClientTest.java | 24 +++++------- .../producer/internals/SenderTest.java | 3 +- .../common/requests/ProduceRequestTest.java | 39 +++++++------------ .../common/requests/RequestResponseTest.java | 1 - .../kafka/api/AuthorizerIntegrationTest.scala | 3 +- .../unit/kafka/server/KafkaApisTest.scala | 17 ++++++-- .../unit/kafka/server/RequestQuotaTest.scala | 3 +- .../producer/ProducerResponseBenchmark.java | 5 ++- 11 files changed, 43 insertions(+), 57 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java index d77ad41938eef..8fbaeb463ef96 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java @@ -580,7 +580,6 @@ synchronized void bumpIdempotentEpochAndResetIdIfNeeded() { if (currentState != State.INITIALIZING && !hasProducerId()) { transitionTo(State.INITIALIZING); InitProducerIdRequestData requestData = new InitProducerIdRequestData() - .setTransactionalId(null) .setTransactionTimeoutMs(Integer.MAX_VALUE); InitProducerIdHandler handler = new InitProducerIdHandler(new InitProducerIdRequest.Builder(requestData), false); enqueueRequest(handler); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/RequestUtils.java b/clients/src/main/java/org/apache/kafka/common/requests/RequestUtils.java index 52145b8322853..ff42914ebff34 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/RequestUtils.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/RequestUtils.java @@ -90,4 +90,4 @@ public static AbstractMap.SimpleEntry flags(ProduceRequest req } return new AbstractMap.SimpleEntry<>(hasIdempotentRecords, hasTransactionalRecords); } -} \ No newline at end of file +} diff --git a/clients/src/main/resources/common/message/ProduceRequest.json b/clients/src/main/resources/common/message/ProduceRequest.json index f5e3c2bb81341..b8425ab6a05d4 100644 --- a/clients/src/main/resources/common/message/ProduceRequest.json +++ b/clients/src/main/resources/common/message/ProduceRequest.json @@ -33,7 +33,7 @@ "validVersions": "0-8", "flexibleVersions": "none", "fields": [ - { "name": "TransactionalId", "type": "string", "versions": "3+", "nullableVersions": "3+", "default": "null", "entityType": "transactionalId", + { "name": "TransactionalId", "type": "string", "versions": "3+", "nullableVersions": "3+", "default": "null", "ignorable": true, "entityType": "transactionalId", "about": "The transactional ID, or null if the producer is not transactional." }, { "name": "Acks", "type": "int16", "versions": "0+", "about": "The number of acknowledgments the producer requires the leader to have received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for only the leader and -1 for the full ISR." }, diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java index 93f9ad52a819c..034a0d473e70b 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -158,8 +158,7 @@ public void testClose() { ProduceRequest.Builder builder = ProduceRequest.forCurrentMagic(new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection()) .setAcks((short) 1) - .setTimeoutMs(1000) - .setTransactionalId(null)); + .setTimeoutMs(1000)); ClientRequest request = client.newClientRequest(node.idString(), builder, time.milliseconds(), true); client.send(request, time.milliseconds()); assertEquals("There should be 1 in-flight request after send", 1, @@ -191,8 +190,7 @@ private void checkSimpleRequestResponse(NetworkClient networkClient) { PRODUCE.latestVersion(), new ProduceRequestData() .setAcks((short) 1) - .setTimeoutMs(1000) - .setTransactionalId(null)); + .setTimeoutMs(1000)); TestCallbackHandler handler = new TestCallbackHandler(); ClientRequest request = networkClient.newClientRequest(node.idString(), builder, time.milliseconds(), true, defaultRequestTimeoutMs, handler); @@ -204,7 +202,7 @@ private void checkSimpleRequestResponse(NetworkClient networkClient) { request.apiKey().responseHeaderVersion(PRODUCE.latestVersion())); Struct resp = new ProduceResponseData() .setThrottleTimeMs(100) - .toStruct(ProduceResponseData.HIGHEST_SUPPORTED_VERSION); + .toStruct(PRODUCE.latestVersion()); Struct responseHeaderStruct = respHeader.toStruct(); int size = responseHeaderStruct.sizeOf() + resp.sizeOf(); ByteBuffer buffer = ByteBuffer.allocate(size); @@ -439,8 +437,7 @@ public void testRequestTimeout() { ProduceRequest.Builder builder = ProduceRequest.forCurrentMagic(new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection()) .setAcks((short) 1) - .setTimeoutMs(1000) - .setTransactionalId(null)); + .setTimeoutMs(1000)); TestCallbackHandler handler = new TestCallbackHandler(); int requestTimeoutMs = defaultRequestTimeoutMs + 5000; ClientRequest request = client.newClientRequest(node.idString(), builder, time.milliseconds(), true, @@ -455,8 +452,7 @@ public void testDefaultRequestTimeout() { ProduceRequest.Builder builder = ProduceRequest.forCurrentMagic(new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection()) .setAcks((short) 1) - .setTimeoutMs(1000) - .setTransactionalId(null)); + .setTimeoutMs(1000)); ClientRequest request = client.newClientRequest(node.idString(), builder, time.milliseconds(), true); assertEquals(defaultRequestTimeoutMs, request.requestTimeoutMs()); testRequestTimeout(request); @@ -511,8 +507,7 @@ public void testConnectionThrottling() { PRODUCE.latestVersion(), new ProduceRequestData() .setAcks((short) 1) - .setTimeoutMs(1000) - .setTransactionalId(null)); + .setTimeoutMs(1000)); TestCallbackHandler handler = new TestCallbackHandler(); ClientRequest request = client.newClientRequest(node.idString(), builder, time.milliseconds(), true, defaultRequestTimeoutMs, handler); @@ -523,7 +518,7 @@ public void testConnectionThrottling() { request.apiKey().responseHeaderVersion(PRODUCE.latestVersion())); Struct resp = new ProduceResponseData() .setThrottleTimeMs(100) - .toStruct(ProduceResponseData.HIGHEST_SUPPORTED_VERSION); + .toStruct(PRODUCE.latestVersion()); Struct responseHeaderStruct = respHeader.toStruct(); int size = responseHeaderStruct.sizeOf() + resp.sizeOf(); ByteBuffer buffer = ByteBuffer.allocate(size); @@ -600,8 +595,7 @@ private int sendEmptyProduceRequest(String nodeId) { ProduceRequest.Builder builder = ProduceRequest.forCurrentMagic(new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection()) .setAcks((short) 1) - .setTimeoutMs(1000) - .setTransactionalId(null)); + .setTimeoutMs(1000)); TestCallbackHandler handler = new TestCallbackHandler(); ClientRequest request = client.newClientRequest(nodeId, builder, time.milliseconds(), true, defaultRequestTimeoutMs, handler); @@ -622,7 +616,7 @@ private void sendResponse(ResponseHeader respHeader, Struct response) { private void sendThrottledProduceResponse(int correlationId, int throttleMs) { Struct resp = new ProduceResponseData() .setThrottleTimeMs(throttleMs) - .toStruct(ProduceResponseData.HIGHEST_SUPPORTED_VERSION); + .toStruct(PRODUCE.latestVersion()); sendResponse(new ResponseHeader(correlationId, PRODUCE.responseHeaderVersion(PRODUCE.latestVersion())), resp); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java index a9f48f5ce92e3..a0c8d5639aafc 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java @@ -300,8 +300,7 @@ public void testQuotaMetrics() { ProduceRequest.Builder builder = ProduceRequest.forCurrentMagic(new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection()) .setAcks((short) 1) - .setTimeoutMs(1000) - .setTransactionalId(null)); + .setTimeoutMs(1000)); ClientRequest request = client.newClientRequest(node.idString(), builder, time.milliseconds(), true); client.send(request, time.milliseconds()); client.poll(1, time.milliseconds()); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java index 359e788e55027..156af86b53756 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/ProduceRequestTest.java @@ -58,8 +58,7 @@ public void shouldBeFlaggedAsTransactionalWhenTransactionalRecords() throws Exce .setIndex(1) .setRecords(memoryRecords)))).iterator())) .setAcks((short) -1) - .setTimeoutMs(10) - .setTransactionalId(null)).build(); + .setTimeoutMs(10)).build(); assertTrue(RequestUtils.hasTransactionalRecords(request)); } @@ -88,8 +87,7 @@ public void shouldBeFlaggedAsIdempotentWhenIdempotentRecords() throws Exception .setIndex(1) .setRecords(memoryRecords)))).iterator())) .setAcks((short) -1) - .setTimeoutMs(10) - .setTransactionalId(null)).build(); + .setTimeoutMs(10)).build(); assertTrue(RequestUtils.hasIdempotentRecords(request)); } @@ -106,8 +104,7 @@ public void testBuildWithOldMessageFormat() { new ProduceRequestData.PartitionProduceData().setIndex(9).setRecords(builder.build())))) .iterator())) .setAcks((short) 1) - .setTimeoutMs(5000) - .setTransactionalId(null)); + .setTimeoutMs(5000)); assertEquals(2, requestBuilder.oldestAllowedVersion()); assertEquals(2, requestBuilder.latestAllowedVersion()); } @@ -125,8 +122,7 @@ public void testBuildWithCurrentMessageFormat() { new ProduceRequestData.PartitionProduceData().setIndex(9).setRecords(builder.build())))) .iterator())) .setAcks((short) 1) - .setTimeoutMs(5000) - .setTransactionalId(null)); + .setTimeoutMs(5000)); assertEquals(3, requestBuilder.oldestAllowedVersion()); assertEquals(ApiKeys.PRODUCE.latestVersion(), requestBuilder.latestAllowedVersion()); } @@ -154,8 +150,7 @@ public void testV3AndAboveShouldContainOnlyOneRecordBatch() { .setIndex(0) .setRecords(MemoryRecords.readableRecords(buffer))))).iterator())) .setAcks((short) 1) - .setTimeoutMs(5000) - .setTransactionalId(null)); + .setTimeoutMs(5000)); assertThrowsInvalidRecordExceptionForAllVersions(requestBuilder); } @@ -170,8 +165,7 @@ public void testV3AndAboveCannotHaveNoRecordBatches() { .setIndex(0) .setRecords(MemoryRecords.EMPTY)))).iterator())) .setAcks((short) 1) - .setTimeoutMs(5000) - .setTransactionalId(null)); + .setTimeoutMs(5000)); assertThrowsInvalidRecordExceptionForAllVersions(requestBuilder); } @@ -191,8 +185,7 @@ public void testV3AndAboveCannotUseMagicV0() { .setIndex(0) .setRecords(builder.build())))).iterator())) .setAcks((short) 1) - .setTimeoutMs(5000) - .setTransactionalId(null)); + .setTimeoutMs(5000)); assertThrowsInvalidRecordExceptionForAllVersions(requestBuilder); } @@ -212,8 +205,7 @@ public void testV3AndAboveCannotUseMagicV1() { .setRecords(builder.build())))) .iterator())) .setAcks((short) 1) - .setTimeoutMs(5000) - .setTransactionalId(null)); + .setTimeoutMs(5000)); assertThrowsInvalidRecordExceptionForAllVersions(requestBuilder); } @@ -233,8 +225,7 @@ public void testV6AndBelowCannotUseZStdCompression() { .setRecords(builder.build())))) .iterator())) .setAcks((short) 1) - .setTimeoutMs(1000) - .setTransactionalId(null); + .setTimeoutMs(1000); // Can't create ProduceRequest instance with version within [3, 7) for (short version = 3; version < 7; version++) { @@ -266,10 +257,8 @@ public void testMixedTransactionalData() { new ProduceRequestData.TopicProduceData().setName("foo").setPartitionData(Collections.singletonList( new ProduceRequestData.PartitionProduceData().setIndex(1).setRecords(nonTxnRecords)))) .iterator())) - .setAcks((short) 1) - .setTimeoutMs(5000) - .setTransactionalId(null)); - + .setAcks((short) -1) + .setTimeoutMs(5000)); final ProduceRequest request = builder.build(); assertTrue(RequestUtils.hasTransactionalRecords(request)); assertTrue(RequestUtils.hasIdempotentRecords(request)); @@ -295,8 +284,7 @@ public void testMixedIdempotentData() { new ProduceRequestData.PartitionProduceData().setIndex(1).setRecords(nonTxnRecords)))) .iterator())) .setAcks((short) -1) - .setTimeoutMs(5000) - .setTransactionalId(null)); + .setTimeoutMs(5000)); final ProduceRequest request = builder.build(); assertFalse(RequestUtils.hasTransactionalRecords(request)); @@ -329,7 +317,6 @@ private ProduceRequest createNonIdempotentNonTransactionalRecords() { .setRecords(MemoryRecords.withRecords(CompressionType.NONE, simpleRecord))))) .iterator())) .setAcks((short) -1) - .setTimeoutMs(10) - .setTransactionalId(null)).build(); + .setTimeoutMs(10)).build(); } } diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 7e51fa2716990..e483358121f2e 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -1768,7 +1768,6 @@ private DeleteTopicsResponse createDeleteTopicsResponse() { private InitProducerIdRequest createInitPidRequest() { InitProducerIdRequestData requestData = new InitProducerIdRequestData() - .setTransactionalId(null) .setTransactionTimeoutMs(100); return new InitProducerIdRequest.Builder(requestData).build(); } diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index 7397f1f634e2a..a6f67ca060530 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -300,8 +300,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { .setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes)))))) .iterator)) .setAcks(1.toShort) - .setTimeoutMs(5000) - .setTransactionalId(null)) + .setTimeoutMs(5000)) .build() private def createFetchRequest = { diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 1a534120bd5ea..3a2f99d63a6f7 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -71,6 +71,7 @@ import org.easymock.{Capture, EasyMock, IAnswer, IArgumentMatcher} import org.junit.Assert.{assertArrayEquals, assertEquals, assertNull, assertTrue} import org.junit.{After, Test} +import scala.annotation.nowarn import scala.collection.{Map, Seq, mutable} import scala.compat.java8.OptionConverters._ import scala.jdk.CollectionConverters._ @@ -1232,6 +1233,7 @@ class KafkaApisTest { } } + @nowarn("cat=deprecation") @Test def shouldReplaceProducerFencedWithInvalidProducerEpochInProduceResponse(): Unit = { val topic = "topic" @@ -1245,10 +1247,17 @@ class KafkaApisTest { val tp = new TopicPartition("topic", 0) - val produceRequest = ProduceRequest.Builder.forCurrentMagic(1, 5000, - collection.mutable.Map(tp -> MemoryRecords.withRecords(CompressionType.NONE, - new SimpleRecord("test".getBytes))).asJava) - .build(version.toShort) + val produceRequest = ProduceRequest.forCurrentMagic(new ProduceRequestData() + .setTopicData(new ProduceRequestData.TopicProduceDataCollection( + Collections.singletonList(new ProduceRequestData.TopicProduceData() + .setName(tp.topic()).setPartitionData(Collections.singletonList( + new ProduceRequestData.PartitionProduceData() + .setIndex(tp.partition()) + .setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes)))))) + .iterator)) + .setAcks(1.toShort) + .setTimeoutMs(5000)) + .build(version.toShort) val request = buildRequest(produceRequest) EasyMock.expect(replicaManager.appendRecords(EasyMock.anyLong(), diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index bd1fb2f093691..7ea0dba9c51e5 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -219,8 +219,7 @@ class RequestQuotaTest extends BaseRequestTest { .setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes)))))) .iterator)) .setAcks(1.toShort) - .setTimeoutMs(5000) - .setTransactionalId(null)) + .setTimeoutMs(5000)) case ApiKeys.FETCH => val partitionMap = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData] diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerResponseBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerResponseBenchmark.java index c52a7460e6e3a..307e194431818 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerResponseBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerResponseBenchmark.java @@ -18,7 +18,6 @@ package org.apache.kafka.jmh.producer; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.message.ProduceResponseData; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.requests.AbstractResponse; @@ -39,6 +38,8 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; +import static org.apache.kafka.common.protocol.ApiKeys.PRODUCE; + @State(Scope.Benchmark) @Fork(value = 1) @Warmup(iterations = 5) @@ -82,6 +83,6 @@ public AbstractResponse constructorProduceResponse() { @Benchmark @OutputTimeUnit(TimeUnit.NANOSECONDS) public Struct constructorStruct() { - return RESPONSE.toStruct(ProduceResponseData.HIGHEST_SUPPORTED_VERSION); + return RESPONSE.toStruct(PRODUCE.latestVersion()); } } From 34b5b179e2baa4eb7a1fc7aeec05eb549cf5e7f2 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Wed, 18 Nov 2020 23:41:28 +0800 Subject: [PATCH 14/15] fix failed tests --- .../kafka/clients/producer/internals/TransactionManager.java | 1 + .../org/apache/kafka/common/requests/RequestResponseTest.java | 1 + 2 files changed, 2 insertions(+) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java index 8fbaeb463ef96..d77ad41938eef 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java @@ -580,6 +580,7 @@ synchronized void bumpIdempotentEpochAndResetIdIfNeeded() { if (currentState != State.INITIALIZING && !hasProducerId()) { transitionTo(State.INITIALIZING); InitProducerIdRequestData requestData = new InitProducerIdRequestData() + .setTransactionalId(null) .setTransactionTimeoutMs(Integer.MAX_VALUE); InitProducerIdHandler handler = new InitProducerIdHandler(new InitProducerIdRequest.Builder(requestData), false); enqueueRequest(handler); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index e483358121f2e..7e51fa2716990 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -1768,6 +1768,7 @@ private DeleteTopicsResponse createDeleteTopicsResponse() { private InitProducerIdRequest createInitPidRequest() { InitProducerIdRequestData requestData = new InitProducerIdRequestData() + .setTransactionalId(null) .setTransactionTimeoutMs(100); return new InitProducerIdRequest.Builder(requestData).build(); } From ffbe9a3ee73bc65253c0b319cf08b923dcce6138 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Thu, 19 Nov 2020 02:12:36 +0800 Subject: [PATCH 15/15] ignorable=false --- clients/src/main/resources/common/message/ProduceRequest.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/resources/common/message/ProduceRequest.json b/clients/src/main/resources/common/message/ProduceRequest.json index b8425ab6a05d4..f5e3c2bb81341 100644 --- a/clients/src/main/resources/common/message/ProduceRequest.json +++ b/clients/src/main/resources/common/message/ProduceRequest.json @@ -33,7 +33,7 @@ "validVersions": "0-8", "flexibleVersions": "none", "fields": [ - { "name": "TransactionalId", "type": "string", "versions": "3+", "nullableVersions": "3+", "default": "null", "ignorable": true, "entityType": "transactionalId", + { "name": "TransactionalId", "type": "string", "versions": "3+", "nullableVersions": "3+", "default": "null", "entityType": "transactionalId", "about": "The transactional ID, or null if the producer is not transactional." }, { "name": "Acks", "type": "int16", "versions": "0+", "about": "The number of acknowledgments the producer requires the leader to have received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for only the leader and -1 for the full ISR." },