From 1a10c3445e157da1d2fd670c043f19c385465eb0 Mon Sep 17 00:00:00 2001 From: Anastasia Vela Date: Tue, 15 Dec 2020 05:33:36 -0800 Subject: [PATCH] KAFKA-10525: Emit JSONs with new auto-generated schema (KIP-673) (#9526) This patch updates the request logger to output request and response payloads in JSON. Payloads are converted to JSON based on their auto-generated schema. Reviewers: Lucas Bradstreet , David Mao , David Jacot --- .../apache/kafka/clients/NetworkClient.java | 12 +- .../kafka/clients/admin/KafkaAdminClient.java | 8 +- .../internals/AbstractCoordinator.java | 8 +- .../internals/TransactionManager.java | 6 +- .../common/requests/AbstractRequest.java | 2 +- .../requests/AddOffsetsToTxnRequest.java | 4 +- .../requests/AddOffsetsToTxnResponse.java | 4 +- .../requests/AddPartitionsToTxnRequest.java | 4 +- .../requests/AddPartitionsToTxnResponse.java | 4 +- .../requests/AlterClientQuotasRequest.java | 2 +- .../requests/AlterClientQuotasResponse.java | 2 +- .../common/requests/AlterConfigsRequest.java | 2 +- .../common/requests/AlterIsrRequest.java | 1 + .../common/requests/AlterIsrResponse.java | 1 + .../AlterPartitionReassignmentsResponse.java | 1 + .../requests/AlterReplicaLogDirsRequest.java | 2 +- .../AlterUserScramCredentialsRequest.java | 1 + .../AlterUserScramCredentialsResponse.java | 1 + .../common/requests/ApiVersionsResponse.java | 2 +- .../requests/BeginQuorumEpochRequest.java | 4 +- .../requests/BeginQuorumEpochResponse.java | 4 +- .../requests/ControlledShutdownRequest.java | 1 + .../requests/ControlledShutdownResponse.java | 1 + .../common/requests/CreateAclsRequest.java | 2 +- .../common/requests/CreateAclsResponse.java | 2 +- .../CreateDelegationTokenRequest.java | 1 + .../CreateDelegationTokenResponse.java | 1 + .../requests/CreatePartitionsRequest.java | 1 + .../requests/CreatePartitionsResponse.java | 1 + .../common/requests/CreateTopicsRequest.java | 1 + .../common/requests/CreateTopicsResponse.java | 1 + .../common/requests/DeleteAclsRequest.java | 2 +- .../common/requests/DeleteAclsResponse.java | 2 +- .../common/requests/DeleteGroupsRequest.java | 4 +- .../common/requests/DeleteGroupsResponse.java | 4 +- .../requests/DeleteRecordsResponse.java | 1 + .../common/requests/DeleteTopicsRequest.java | 1 + .../common/requests/DeleteTopicsResponse.java | 3 +- .../common/requests/DescribeAclsRequest.java | 1 + .../common/requests/DescribeAclsResponse.java | 2 +- .../requests/DescribeClientQuotasRequest.java | 2 +- .../DescribeClientQuotasResponse.java | 2 +- .../DescribeDelegationTokenRequest.java | 1 + .../DescribeDelegationTokenResponse.java | 2 +- .../requests/DescribeGroupsRequest.java | 1 + .../requests/DescribeGroupsResponse.java | 3 +- .../requests/DescribeLogDirsResponse.java | 1 + .../requests/DescribeQuorumRequest.java | 4 +- .../requests/DescribeQuorumResponse.java | 4 +- .../DescribeUserScramCredentialsRequest.java | 3 +- .../DescribeUserScramCredentialsResponse.java | 1 + .../common/requests/ElectLeadersRequest.java | 1 + .../common/requests/ElectLeadersResponse.java | 1 + .../requests/EndQuorumEpochRequest.java | 4 +- .../requests/EndQuorumEpochResponse.java | 4 +- .../kafka/common/requests/EndTxnRequest.java | 4 +- .../kafka/common/requests/EndTxnResponse.java | 4 +- .../common/requests/EnvelopeRequest.java | 1 + .../common/requests/EnvelopeResponse.java | 1 + .../ExpireDelegationTokenRequest.java | 2 +- .../ExpireDelegationTokenResponse.java | 2 +- .../kafka/common/requests/FetchResponse.java | 1 + .../requests/FindCoordinatorRequest.java | 1 + .../requests/FindCoordinatorResponse.java | 1 + .../common/requests/HeartbeatRequest.java | 4 +- .../common/requests/HeartbeatResponse.java | 2 +- .../IncrementalAlterConfigsRequest.java | 1 + .../IncrementalAlterConfigsResponse.java | 1 + .../requests/InitProducerIdRequest.java | 5 +- .../requests/InitProducerIdResponse.java | 4 +- .../common/requests/JoinGroupRequest.java | 1 + .../common/requests/JoinGroupResponse.java | 1 + .../common/requests/LeaderAndIsrRequest.java | 3 +- .../common/requests/LeaderAndIsrResponse.java | 2 +- .../common/requests/LeaveGroupRequest.java | 1 + .../common/requests/LeaveGroupResponse.java | 4 +- .../common/requests/ListGroupsResponse.java | 1 + .../ListPartitionReassignmentsRequest.java | 1 + .../ListPartitionReassignmentsResponse.java | 1 + .../common/requests/MetadataRequest.java | 1 + .../common/requests/MetadataResponse.java | 2 +- .../common/requests/OffsetCommitRequest.java | 1 + .../common/requests/OffsetCommitResponse.java | 1 + .../common/requests/OffsetDeleteRequest.java | 4 +- .../common/requests/OffsetDeleteResponse.java | 4 +- .../common/requests/OffsetFetchRequest.java | 4 +- .../common/requests/OffsetFetchResponse.java | 4 +- .../common/requests/ProduceResponse.java | 1 + .../requests/RenewDelegationTokenRequest.java | 1 + .../RenewDelegationTokenResponse.java | 2 +- .../requests/SaslAuthenticateRequest.java | 1 + .../requests/SaslAuthenticateResponse.java | 2 +- .../common/requests/SaslHandshakeRequest.java | 1 + .../requests/SaslHandshakeResponse.java | 2 +- .../common/requests/StopReplicaRequest.java | 2 +- .../common/requests/StopReplicaResponse.java | 2 +- .../common/requests/SyncGroupRequest.java | 4 +- .../common/requests/SyncGroupResponse.java | 4 +- .../requests/TxnOffsetCommitRequest.java | 4 +- .../requests/TxnOffsetCommitResponse.java | 4 +- .../requests/UpdateFeaturesRequest.java | 1 + .../requests/UpdateFeaturesResponse.java | 1 + .../requests/UpdateMetadataRequest.java | 2 +- .../requests/UpdateMetadataResponse.java | 2 +- .../kafka/common/requests/VoteRequest.java | 4 +- .../kafka/common/requests/VoteResponse.java | 4 +- .../requests/WriteTxnMarkersRequest.java | 4 +- .../kafka/clients/NodeApiVersionsTest.java | 6 +- .../internals/AbstractCoordinatorTest.java | 12 +- .../internals/ConsumerCoordinatorTest.java | 40 ++-- .../clients/producer/KafkaProducerTest.java | 4 +- .../producer/internals/SenderTest.java | 2 +- .../internals/TransactionManagerTest.java | 74 +++---- .../AddPartitionsToTxnRequestTest.java | 6 +- .../requests/ApiVersionsResponseTest.java | 17 +- .../requests/OffsetFetchResponseTest.java | 6 +- .../common/requests/RequestContextTest.java | 4 +- .../common/requests/RequestResponseTest.java | 14 +- .../requests/TxnOffsetCommitRequestTest.java | 6 +- .../authenticator/SaslAuthenticatorTest.java | 12 +- .../distributed/WorkerCoordinatorTest.java | 16 +- .../scala/kafka/network/RequestChannel.scala | 53 ++--- .../kafka/network/RequestConvertToJson.scala | 207 ++++++++++++++++++ .../main/scala/kafka/server/KafkaApis.scala | 6 +- .../kafka/tools/TestRaftRequestHandler.scala | 3 +- .../kafka/network/RequestChannelTest.scala | 29 ++- .../network/RequestConvertToJsonTest.scala | 197 +++++++++++++++++ .../unit/kafka/network/SocketServerTest.scala | 13 +- .../kafka/message/JsonConverterGenerator.java | 30 ++- .../jmh/common/FetchRequestBenchmark.java | 6 + .../common/ListOffsetRequestBenchmark.java | 80 +++++++ .../jmh/common/ProduceRequestBenchmark.java | 64 ++++++ .../metadata/MetadataRequestBenchmark.java | 6 + 133 files changed, 913 insertions(+), 259 deletions(-) create mode 100644 core/src/main/scala/kafka/network/RequestConvertToJson.scala create mode 100644 core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala create mode 100644 jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/ListOffsetRequestBenchmark.java create mode 100644 jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/ProduceRequestBenchmark.java diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index 849c8873b5e..840185da464 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -893,10 +893,10 @@ public class NetworkClient implements KafkaClient { private void handleApiVersionsResponse(List responses, InFlightRequest req, long now, ApiVersionsResponse apiVersionsResponse) { final String node = req.destination; - if (apiVersionsResponse.data.errorCode() != Errors.NONE.code()) { - if (req.request.version() == 0 || apiVersionsResponse.data.errorCode() != Errors.UNSUPPORTED_VERSION.code()) { + if (apiVersionsResponse.data().errorCode() != Errors.NONE.code()) { + if (req.request.version() == 0 || apiVersionsResponse.data().errorCode() != Errors.UNSUPPORTED_VERSION.code()) { log.warn("Received error {} from node {} when making an ApiVersionsRequest with correlation id {}. Disconnecting.", - Errors.forCode(apiVersionsResponse.data.errorCode()), node, req.header.correlationId()); + Errors.forCode(apiVersionsResponse.data().errorCode()), node, req.header.correlationId()); this.selector.close(node); processDisconnection(responses, node, now, ChannelState.LOCAL_CLOSE); } else { @@ -904,8 +904,8 @@ public class NetworkClient implements KafkaClient { // the ApiVersionsRequest when an UNSUPPORTED_VERSION error is returned. // If not provided, the client falls back to version 0. short maxApiVersion = 0; - if (apiVersionsResponse.data.apiKeys().size() > 0) { - ApiVersionsResponseKey apiVersion = apiVersionsResponse.data.apiKeys().find(ApiKeys.API_VERSIONS.id); + if (apiVersionsResponse.data().apiKeys().size() > 0) { + ApiVersionsResponseKey apiVersion = apiVersionsResponse.data().apiKeys().find(ApiKeys.API_VERSIONS.id); if (apiVersion != null) { maxApiVersion = apiVersion.maxVersion(); } @@ -914,7 +914,7 @@ public class NetworkClient implements KafkaClient { } return; } - NodeApiVersions nodeVersionInfo = new NodeApiVersions(apiVersionsResponse.data.apiKeys()); + NodeApiVersions nodeVersionInfo = new NodeApiVersions(apiVersionsResponse.data().apiKeys()); apiVersions.update(node, nodeVersionInfo); this.connectionStates.ready(node); log.debug("Recorded API versions for node {}: {}", node, nodeVersionInfo); diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index cf0a6f6949f..4558e86d8b0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -3451,12 +3451,12 @@ public class KafkaAdminClient extends AdminClient { } // If the error is an error at the group level, the future is failed with it - final Errors groupError = Errors.forCode(response.data.errorCode()); + final Errors groupError = Errors.forCode(response.data().errorCode()); if (handleGroupRequestError(groupError, context.future())) return; final Map partitions = new HashMap<>(); - response.data.topics().forEach(topic -> topic.partitions().forEach(partition -> partitions.put( + response.data().topics().forEach(topic -> topic.partitions().forEach(partition -> partitions.put( new TopicPartition(topic.name(), partition.partitionIndex()), Errors.forCode(partition.errorCode()))) ); @@ -4359,10 +4359,10 @@ public class KafkaAdminClient extends AdminClient { @Override void handleResponse(AbstractResponse response) { final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response; - if (apiVersionsResponse.data.errorCode() == Errors.NONE.code()) { + if (apiVersionsResponse.data().errorCode() == Errors.NONE.code()) { future.complete(createFeatureMetadata(apiVersionsResponse)); } else { - future.completeExceptionally(Errors.forCode(apiVersionsResponse.data.errorCode()).exception()); + future.completeExceptionally(Errors.forCode(apiVersionsResponse.data().errorCode()).exception()); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java index 4ca71bb626a..29842a90c33 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java @@ -732,9 +732,9 @@ public abstract class AbstractCoordinator implements Closeable { RequestFuture future) { Errors error = syncResponse.error(); if (error == Errors.NONE) { - if (isProtocolTypeInconsistent(syncResponse.data.protocolType())) { + if (isProtocolTypeInconsistent(syncResponse.data().protocolType())) { log.error("SyncGroup failed due to inconsistent Protocol Type, received {} but expected {}", - syncResponse.data.protocolType(), protocolType()); + syncResponse.data().protocolType(), protocolType()); future.raise(Errors.INCONSISTENT_GROUP_PROTOCOL); } else { log.debug("Received successful SyncGroup response: {}", syncResponse); @@ -743,7 +743,7 @@ public abstract class AbstractCoordinator implements Closeable { synchronized (AbstractCoordinator.this) { if (!generation.equals(Generation.NO_GENERATION) && state == MemberState.COMPLETING_REBALANCE) { // check protocol name only if the generation is not reset - final String protocolName = syncResponse.data.protocolName(); + final String protocolName = syncResponse.data().protocolName(); final boolean protocolNameInconsistent = protocolName != null && !protocolName.equals(generation.protocolName); @@ -761,7 +761,7 @@ public abstract class AbstractCoordinator implements Closeable { sensors.successfulRebalanceSensor.record(lastRebalanceEndMs - lastRebalanceStartMs); lastRebalanceStartMs = -1L; - future.complete(ByteBuffer.wrap(syncResponse.data.assignment())); + future.complete(ByteBuffer.wrap(syncResponse.data().assignment())); } } else { log.info("Generation data was cleared by heartbeat thread to {} and state is now {} before " + 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 d77ad41938e..e58469b003a 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 @@ -1348,8 +1348,8 @@ public class TransactionManager { Errors error = initProducerIdResponse.error(); if (error == Errors.NONE) { - ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(initProducerIdResponse.data.producerId(), - initProducerIdResponse.data.producerEpoch()); + ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(initProducerIdResponse.data().producerId(), + initProducerIdResponse.data().producerEpoch()); setProducerIdAndEpoch(producerIdAndEpoch); transitionTo(State.READY); lastError = null; @@ -1623,7 +1623,7 @@ public class TransactionManager { @Override public void handleResponse(AbstractResponse response) { AddOffsetsToTxnResponse addOffsetsToTxnResponse = (AddOffsetsToTxnResponse) response; - Errors error = Errors.forCode(addOffsetsToTxnResponse.data.errorCode()); + Errors error = Errors.forCode(addOffsetsToTxnResponse.data().errorCode()); if (error == Errors.NONE) { log.debug("Successfully added partition for consumer group {} to transaction", builder.data.groupId()); 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 d09c2d6b612..28988895c07 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 @@ -102,7 +102,7 @@ public abstract class AbstractRequest implements AbstractRequestResponse { return SendBuilder.buildRequestSend(header, data()); } - protected abstract Message data(); + public abstract Message data(); // Visible for testing public final ByteBuffer serialize() { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnRequest.java index 4ad56aa9426..1e5f9862178 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnRequest.java @@ -26,7 +26,7 @@ import java.nio.ByteBuffer; public class AddOffsetsToTxnRequest extends AbstractRequest { - public AddOffsetsToTxnRequestData data; + private final AddOffsetsToTxnRequestData data; public static class Builder extends AbstractRequest.Builder { public AddOffsetsToTxnRequestData data; @@ -53,7 +53,7 @@ public class AddOffsetsToTxnRequest extends AbstractRequest { } @Override - protected AddOffsetsToTxnRequestData data() { + public AddOffsetsToTxnRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnResponse.java index 6908e519606..ce9a6cf7d60 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnResponse.java @@ -39,7 +39,7 @@ import java.util.Map; */ public class AddOffsetsToTxnResponse extends AbstractResponse { - public AddOffsetsToTxnResponseData data; + private final AddOffsetsToTxnResponseData data; public AddOffsetsToTxnResponse(AddOffsetsToTxnResponseData data) { super(ApiKeys.ADD_OFFSETS_TO_TXN); @@ -57,7 +57,7 @@ public class AddOffsetsToTxnResponse extends AbstractResponse { } @Override - protected AddOffsetsToTxnResponseData data() { + public AddOffsetsToTxnResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java index 57645789ca0..1034c0f7adc 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java @@ -32,7 +32,7 @@ import java.util.Map; public class AddPartitionsToTxnRequest extends AbstractRequest { - public final AddPartitionsToTxnRequestData data; + private final AddPartitionsToTxnRequestData data; private List cachedPartitions = null; @@ -112,7 +112,7 @@ public class AddPartitionsToTxnRequest extends AbstractRequest { } @Override - protected AddPartitionsToTxnRequestData data() { + public AddPartitionsToTxnRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java index c6da3af0a93..57b2a5a5d7c 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java @@ -46,7 +46,7 @@ import java.util.Map; */ public class AddPartitionsToTxnResponse extends AbstractResponse { - public final AddPartitionsToTxnResponseData data; + private final AddPartitionsToTxnResponseData data; private Map cachedErrorsMap = null; @@ -117,7 +117,7 @@ public class AddPartitionsToTxnResponse extends AbstractResponse { } @Override - protected AddPartitionsToTxnResponseData data() { + public AddPartitionsToTxnResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterClientQuotasRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterClientQuotasRequest.java index 71784751236..d03c2671a29 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AlterClientQuotasRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterClientQuotasRequest.java @@ -111,7 +111,7 @@ public class AlterClientQuotasRequest extends AbstractRequest { } @Override - protected AlterClientQuotasRequestData data() { + public AlterClientQuotasRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterClientQuotasResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterClientQuotasResponse.java index bce548822d8..fcacc5d95ef 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AlterClientQuotasResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterClientQuotasResponse.java @@ -77,7 +77,7 @@ public class AlterClientQuotasResponse extends AbstractResponse { } @Override - protected AlterClientQuotasResponseData data() { + public AlterClientQuotasResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsRequest.java index f30e8b9c1ff..b4d35d52ae3 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsRequest.java @@ -112,7 +112,7 @@ public class AlterConfigsRequest extends AbstractRequest { } @Override - protected AlterConfigsRequestData data() { + public AlterConfigsRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterIsrRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterIsrRequest.java index 7ce86aedf43..516c2ce76aa 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AlterIsrRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterIsrRequest.java @@ -34,6 +34,7 @@ public class AlterIsrRequest extends AbstractRequest { this.data = data; } + @Override public AlterIsrRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterIsrResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterIsrResponse.java index 433ba66bdad..c3106ed94cb 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AlterIsrResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterIsrResponse.java @@ -35,6 +35,7 @@ public class AlterIsrResponse extends AbstractResponse { this.data = data; } + @Override public AlterIsrResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionReassignmentsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionReassignmentsResponse.java index 6aea8b1edcc..ab166b81271 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionReassignmentsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionReassignmentsResponse.java @@ -40,6 +40,7 @@ public class AlterPartitionReassignmentsResponse extends AbstractResponse { new AlterPartitionReassignmentsResponseData(new ByteBufferAccessor(buffer), version)); } + @Override public AlterPartitionReassignmentsResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaLogDirsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaLogDirsRequest.java index 5eba039fa55..68a87e6bf40 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaLogDirsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaLogDirsRequest.java @@ -61,7 +61,7 @@ public class AlterReplicaLogDirsRequest extends AbstractRequest { } @Override - protected AlterReplicaLogDirsRequestData data() { + public AlterReplicaLogDirsRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterUserScramCredentialsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterUserScramCredentialsRequest.java index c319ec34452..1ca7ea77aa4 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AlterUserScramCredentialsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterUserScramCredentialsRequest.java @@ -59,6 +59,7 @@ public class AlterUserScramCredentialsRequest extends AbstractRequest { return new AlterUserScramCredentialsRequest(new AlterUserScramCredentialsRequestData(new ByteBufferAccessor(buffer), version), version); } + @Override public AlterUserScramCredentialsRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterUserScramCredentialsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterUserScramCredentialsResponse.java index 2fa49372427..97c0b7d17b2 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AlterUserScramCredentialsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterUserScramCredentialsResponse.java @@ -33,6 +33,7 @@ public class AlterUserScramCredentialsResponse extends AbstractResponse { this.data = responseData; } + @Override public AlterUserScramCredentialsResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java index eaf8113fcd0..4d7397ad96d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java @@ -46,7 +46,7 @@ public class ApiVersionsResponse extends AbstractResponse { public static final ApiVersionsResponse DEFAULT_API_VERSIONS_RESPONSE = createApiVersionsResponse( DEFAULT_THROTTLE_TIME, RecordBatch.CURRENT_MAGIC_VALUE); - public final ApiVersionsResponseData data; + private final ApiVersionsResponseData data; public ApiVersionsResponse(ApiVersionsResponseData data) { super(ApiKeys.API_VERSIONS); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/BeginQuorumEpochRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/BeginQuorumEpochRequest.java index c83e29dd3d5..0794fb46095 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/BeginQuorumEpochRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/BeginQuorumEpochRequest.java @@ -46,7 +46,7 @@ public class BeginQuorumEpochRequest extends AbstractRequest { } } - public final BeginQuorumEpochRequestData data; + private final BeginQuorumEpochRequestData data; private BeginQuorumEpochRequest(BeginQuorumEpochRequestData data, short version) { super(ApiKeys.BEGIN_QUORUM_EPOCH, version); @@ -54,7 +54,7 @@ public class BeginQuorumEpochRequest extends AbstractRequest { } @Override - protected BeginQuorumEpochRequestData data() { + public BeginQuorumEpochRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/BeginQuorumEpochResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/BeginQuorumEpochResponse.java index c3e80eccaa8..c8c0328c93a 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/BeginQuorumEpochResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/BeginQuorumEpochResponse.java @@ -42,7 +42,7 @@ import java.util.Map; * - {@link Errors#UNKNOWN_TOPIC_OR_PARTITION} */ public class BeginQuorumEpochResponse extends AbstractResponse { - public final BeginQuorumEpochResponseData data; + private final BeginQuorumEpochResponseData data; public BeginQuorumEpochResponse(BeginQuorumEpochResponseData data) { super(ApiKeys.BEGIN_QUORUM_EPOCH); @@ -86,7 +86,7 @@ public class BeginQuorumEpochResponse extends AbstractResponse { } @Override - protected BeginQuorumEpochResponseData data() { + public BeginQuorumEpochResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java index f3c063a7a06..088c351302f 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java @@ -65,6 +65,7 @@ public class ControlledShutdownRequest extends AbstractRequest { version); } + @Override public ControlledShutdownRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownResponse.java index 1add8000364..73b6a502683 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownResponse.java @@ -62,6 +62,7 @@ public class ControlledShutdownResponse extends AbstractResponse { return new ControlledShutdownResponse(new ControlledShutdownResponseData(new ByteBufferAccessor(buffer), version)); } + @Override public ControlledShutdownResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsRequest.java index 2ce651565b7..29df8326bc5 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsRequest.java @@ -70,7 +70,7 @@ public class CreateAclsRequest extends AbstractRequest { } @Override - protected CreateAclsRequestData data() { + public CreateAclsRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsResponse.java index d0149b9a2a0..8bc6643f9de 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsResponse.java @@ -34,7 +34,7 @@ public class CreateAclsResponse extends AbstractResponse { } @Override - protected CreateAclsResponseData data() { + public CreateAclsResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreateDelegationTokenRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/CreateDelegationTokenRequest.java index e1d4cfaab80..1fee1b71eb3 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/CreateDelegationTokenRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/CreateDelegationTokenRequest.java @@ -38,6 +38,7 @@ public class CreateDelegationTokenRequest extends AbstractRequest { version); } + @Override public CreateDelegationTokenRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreateDelegationTokenResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/CreateDelegationTokenResponse.java index 9d39b6b6413..b679a30c8dd 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/CreateDelegationTokenResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/CreateDelegationTokenResponse.java @@ -64,6 +64,7 @@ public class CreateDelegationTokenResponse extends AbstractResponse { return prepareResponse(throttleTimeMs, error, owner, -1, -1, -1, "", ByteBuffer.wrap(new byte[] {})); } + @Override public CreateDelegationTokenResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreatePartitionsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/CreatePartitionsRequest.java index 8b0b9f81cc9..d371bbb2169 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/CreatePartitionsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/CreatePartitionsRequest.java @@ -55,6 +55,7 @@ public class CreatePartitionsRequest extends AbstractRequest { this.data = data; } + @Override public CreatePartitionsRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreatePartitionsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/CreatePartitionsResponse.java index e0af04b07ae..e59ac981f11 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/CreatePartitionsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/CreatePartitionsResponse.java @@ -35,6 +35,7 @@ public class CreatePartitionsResponse extends AbstractResponse { this.data = data; } + @Override public CreatePartitionsResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java index 7ec2c9adde3..9a1032b8c09 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java @@ -77,6 +77,7 @@ public class CreateTopicsRequest extends AbstractRequest { this.data = data; } + @Override public CreateTopicsRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsResponse.java index c15da20dfc7..dd062774258 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsResponse.java @@ -50,6 +50,7 @@ public class CreateTopicsResponse extends AbstractResponse { this.data = data; } + @Override public CreateTopicsResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsRequest.java index 6face08f567..98fd6589b22 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsRequest.java @@ -100,7 +100,7 @@ public class DeleteAclsRequest extends AbstractRequest { } @Override - protected DeleteAclsRequestData data() { + public DeleteAclsRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsResponse.java index 3ff8a9834f8..7482953a00d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsResponse.java @@ -51,7 +51,7 @@ public class DeleteAclsResponse extends AbstractResponse { } @Override - protected DeleteAclsResponseData data() { + public DeleteAclsResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteGroupsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteGroupsRequest.java index d09a4d4c02d..87d6deedc12 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteGroupsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteGroupsRequest.java @@ -46,7 +46,7 @@ public class DeleteGroupsRequest extends AbstractRequest { } } - public final DeleteGroupsRequestData data; + private final DeleteGroupsRequestData data; public DeleteGroupsRequest(DeleteGroupsRequestData data, short version) { super(ApiKeys.DELETE_GROUPS, version); @@ -75,7 +75,7 @@ public class DeleteGroupsRequest extends AbstractRequest { } @Override - protected DeleteGroupsRequestData data() { + public DeleteGroupsRequestData data() { return data; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteGroupsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteGroupsResponse.java index a8e8d482de1..4cbffda4221 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteGroupsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteGroupsResponse.java @@ -39,7 +39,7 @@ import java.util.Map; */ public class DeleteGroupsResponse extends AbstractResponse { - public final DeleteGroupsResponseData data; + private final DeleteGroupsResponseData data; public DeleteGroupsResponse(DeleteGroupsResponseData data) { super(ApiKeys.DELETE_GROUPS); @@ -47,7 +47,7 @@ public class DeleteGroupsResponse extends AbstractResponse { } @Override - protected DeleteGroupsResponseData data() { + public DeleteGroupsResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteRecordsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteRecordsResponse.java index ef34102f8a8..b090543fadd 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteRecordsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteRecordsResponse.java @@ -46,6 +46,7 @@ public class DeleteRecordsResponse extends AbstractResponse { this.data = data; } + @Override public DeleteRecordsResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsRequest.java index 5ab64184aea..dfd2e72df03 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsRequest.java @@ -52,6 +52,7 @@ public class DeleteTopicsRequest extends AbstractRequest { this.data = data; } + @Override public DeleteTopicsRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsResponse.java index 3584ddfbfc4..2090c4fd2e2 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsResponse.java @@ -38,7 +38,7 @@ public class DeleteTopicsResponse extends AbstractResponse { * INVALID_REQUEST(42) * TOPIC_DELETION_DISABLED(73) */ - private DeleteTopicsResponseData data; + private final DeleteTopicsResponseData data; public DeleteTopicsResponse(DeleteTopicsResponseData data) { super(ApiKeys.DELETE_TOPICS); @@ -50,6 +50,7 @@ public class DeleteTopicsResponse extends AbstractResponse { return data.throttleTimeMs(); } + @Override public DeleteTopicsResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsRequest.java index e6f9c38fd59..1ddf5bf99fc 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsRequest.java @@ -89,6 +89,7 @@ public class DescribeAclsRequest extends AbstractRequest { } } + @Override public DescribeAclsRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsResponse.java index 4308c9ea45b..c4190e65640 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsResponse.java @@ -62,7 +62,7 @@ public class DescribeAclsResponse extends AbstractResponse { } @Override - protected DescribeAclsResponseData data() { + public DescribeAclsResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeClientQuotasRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeClientQuotasRequest.java index 1f167b0ee3c..68f4ecf9c07 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeClientQuotasRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeClientQuotasRequest.java @@ -106,7 +106,7 @@ public class DescribeClientQuotasRequest extends AbstractRequest { } @Override - protected DescribeClientQuotasRequestData data() { + public DescribeClientQuotasRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeClientQuotasResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeClientQuotasResponse.java index 94fca64221d..3a81e21dac6 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeClientQuotasResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeClientQuotasResponse.java @@ -71,7 +71,7 @@ public class DescribeClientQuotasResponse extends AbstractResponse { } @Override - protected DescribeClientQuotasResponseData data() { + public DescribeClientQuotasResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeDelegationTokenRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeDelegationTokenRequest.java index 3e67140d375..9bf59e844a6 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeDelegationTokenRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeDelegationTokenRequest.java @@ -60,6 +60,7 @@ public class DescribeDelegationTokenRequest extends AbstractRequest { this.data = data; } + @Override public DescribeDelegationTokenRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeDelegationTokenResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeDelegationTokenResponse.java index f6579a06b72..4a2162f53aa 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeDelegationTokenResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeDelegationTokenResponse.java @@ -80,7 +80,7 @@ public class DescribeDelegationTokenResponse extends AbstractResponse { } @Override - protected DescribeDelegationTokenResponseData data() { + public DescribeDelegationTokenResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsRequest.java index db0490dc1b4..eff5bb9fff1 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsRequest.java @@ -52,6 +52,7 @@ public class DescribeGroupsRequest extends AbstractRequest { this.data = data; } + @Override public DescribeGroupsRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsResponse.java index 67c43d87ea0..360caf01e46 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsResponse.java @@ -44,7 +44,7 @@ public class DescribeGroupsResponse extends AbstractResponse { * AUTHORIZATION_FAILED (29) */ - private DescribeGroupsResponseData data; + private final DescribeGroupsResponseData data; public DescribeGroupsResponse(DescribeGroupsResponseData data) { super(ApiKeys.DESCRIBE_GROUPS); @@ -105,6 +105,7 @@ public class DescribeGroupsResponse extends AbstractResponse { return groupMetadata; } + @Override public DescribeGroupsResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsResponse.java index 61149253d59..cd1326be9d1 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsResponse.java @@ -39,6 +39,7 @@ public class DescribeLogDirsResponse extends AbstractResponse { this.data = data; } + @Override public DescribeLogDirsResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeQuorumRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeQuorumRequest.java index c318c214b6a..acdb11c6644 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeQuorumRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeQuorumRequest.java @@ -49,7 +49,7 @@ public class DescribeQuorumRequest extends AbstractRequest { } } - public final DescribeQuorumRequestData data; + private final DescribeQuorumRequestData data; private DescribeQuorumRequest(DescribeQuorumRequestData data, short version) { super(ApiKeys.DESCRIBE_QUORUM, version); @@ -72,7 +72,7 @@ public class DescribeQuorumRequest extends AbstractRequest { } @Override - protected DescribeQuorumRequestData data() { + public DescribeQuorumRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeQuorumResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeQuorumResponse.java index cb1a5ec277c..cbf945b7040 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeQuorumResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeQuorumResponse.java @@ -41,7 +41,7 @@ import java.util.Map; * - {@link Errors#UNKNOWN_TOPIC_OR_PARTITION} */ public class DescribeQuorumResponse extends AbstractResponse { - public final DescribeQuorumResponseData data; + private final DescribeQuorumResponseData data; public DescribeQuorumResponse(DescribeQuorumResponseData data) { super(ApiKeys.DESCRIBE_QUORUM); @@ -63,7 +63,7 @@ public class DescribeQuorumResponse extends AbstractResponse { } @Override - protected DescribeQuorumResponseData data() { + public DescribeQuorumResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeUserScramCredentialsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeUserScramCredentialsRequest.java index 8b28d6b1862..0142e5ab6f5 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeUserScramCredentialsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeUserScramCredentialsRequest.java @@ -44,7 +44,7 @@ public class DescribeUserScramCredentialsRequest extends AbstractRequest { } } - private DescribeUserScramCredentialsRequestData data; + private final DescribeUserScramCredentialsRequestData data; private final short version; private DescribeUserScramCredentialsRequest(DescribeUserScramCredentialsRequestData data, short version) { @@ -58,6 +58,7 @@ public class DescribeUserScramCredentialsRequest extends AbstractRequest { new ByteBufferAccessor(buffer), version), version); } + @Override public DescribeUserScramCredentialsRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeUserScramCredentialsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeUserScramCredentialsResponse.java index a736c2c6876..001cefae41a 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeUserScramCredentialsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeUserScramCredentialsResponse.java @@ -33,6 +33,7 @@ public class DescribeUserScramCredentialsResponse extends AbstractResponse { this.data = responseData; } + @Override public DescribeUserScramCredentialsResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ElectLeadersRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ElectLeadersRequest.java index 89600a90724..92f6b45eed5 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ElectLeadersRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ElectLeadersRequest.java @@ -90,6 +90,7 @@ public class ElectLeadersRequest extends AbstractRequest { this.data = data; } + @Override public ElectLeadersRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ElectLeadersResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ElectLeadersResponse.java index 65eb09751b1..88d4d19fc02 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ElectLeadersResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ElectLeadersResponse.java @@ -52,6 +52,7 @@ public class ElectLeadersResponse extends AbstractResponse { data.setReplicaElectionResults(electionResults); } + @Override public ElectLeadersResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/EndQuorumEpochRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/EndQuorumEpochRequest.java index b9f87d73fbd..136bc54d50b 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/EndQuorumEpochRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/EndQuorumEpochRequest.java @@ -47,7 +47,7 @@ public class EndQuorumEpochRequest extends AbstractRequest { } } - public final EndQuorumEpochRequestData data; + private final EndQuorumEpochRequestData data; private EndQuorumEpochRequest(EndQuorumEpochRequestData data, short version) { super(ApiKeys.END_QUORUM_EPOCH, version); @@ -55,7 +55,7 @@ public class EndQuorumEpochRequest extends AbstractRequest { } @Override - protected EndQuorumEpochRequestData data() { + public EndQuorumEpochRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/EndQuorumEpochResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/EndQuorumEpochResponse.java index 9b446663ea5..ac2c0c5c9d5 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/EndQuorumEpochResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/EndQuorumEpochResponse.java @@ -42,7 +42,7 @@ import java.util.Map; * - {@link Errors#UNKNOWN_TOPIC_OR_PARTITION} */ public class EndQuorumEpochResponse extends AbstractResponse { - public final EndQuorumEpochResponseData data; + private final EndQuorumEpochResponseData data; public EndQuorumEpochResponse(EndQuorumEpochResponseData data) { super(ApiKeys.END_QUORUM_EPOCH); @@ -64,7 +64,7 @@ public class EndQuorumEpochResponse extends AbstractResponse { } @Override - protected EndQuorumEpochResponseData data() { + public EndQuorumEpochResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/EndTxnRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/EndTxnRequest.java index fdfafd005ca..c9ea98005fd 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/EndTxnRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/EndTxnRequest.java @@ -26,7 +26,7 @@ import java.nio.ByteBuffer; public class EndTxnRequest extends AbstractRequest { - public final EndTxnRequestData data; + private final EndTxnRequestData data; public static class Builder extends AbstractRequest.Builder { public final EndTxnRequestData data; @@ -60,7 +60,7 @@ public class EndTxnRequest extends AbstractRequest { } @Override - protected EndTxnRequestData data() { + public EndTxnRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/EndTxnResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/EndTxnResponse.java index b782cefd507..029e7d0ce59 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/EndTxnResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/EndTxnResponse.java @@ -38,7 +38,7 @@ import java.util.Map; */ public class EndTxnResponse extends AbstractResponse { - public final EndTxnResponseData data; + private final EndTxnResponseData data; public EndTxnResponse(EndTxnResponseData data) { super(ApiKeys.END_TXN); @@ -61,7 +61,7 @@ public class EndTxnResponse extends AbstractResponse { } @Override - protected EndTxnResponseData data() { + public EndTxnResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/EnvelopeRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/EnvelopeRequest.java index 4be5259458b..5e8d3faea40 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/EnvelopeRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/EnvelopeRequest.java @@ -80,6 +80,7 @@ public class EnvelopeRequest extends AbstractRequest { return new EnvelopeRequest(new EnvelopeRequestData(new ByteBufferAccessor(buffer), version), version); } + @Override public EnvelopeRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/EnvelopeResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/EnvelopeResponse.java index c7c8d24e4a3..529f616bb26 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/EnvelopeResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/EnvelopeResponse.java @@ -57,6 +57,7 @@ public class EnvelopeResponse extends AbstractResponse { return Errors.forCode(data.errorCode()); } + @Override public EnvelopeResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ExpireDelegationTokenRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ExpireDelegationTokenRequest.java index ac03944ab02..85b02382baf 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ExpireDelegationTokenRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ExpireDelegationTokenRequest.java @@ -39,7 +39,7 @@ public class ExpireDelegationTokenRequest extends AbstractRequest { } @Override - protected ExpireDelegationTokenRequestData data() { + public ExpireDelegationTokenRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ExpireDelegationTokenResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ExpireDelegationTokenResponse.java index 451c736ffb7..163ee78d0ad 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ExpireDelegationTokenResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ExpireDelegationTokenResponse.java @@ -52,7 +52,7 @@ public class ExpireDelegationTokenResponse extends AbstractResponse { } @Override - protected ExpireDelegationTokenResponseData data() { + public ExpireDelegationTokenResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java index d65a55e6c2b..30b462bd219 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java @@ -67,6 +67,7 @@ public class FetchResponse extends AbstractResponse { private final FetchResponseData data; private final LinkedHashMap> responseDataMap; + @Override public FetchResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorRequest.java index b6fdbfc1e63..dc512a554e2 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorRequest.java @@ -77,6 +77,7 @@ public class FindCoordinatorRequest extends AbstractRequest { version); } + @Override public FindCoordinatorRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorResponse.java index 917b087fbab..11f3d488879 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorResponse.java @@ -45,6 +45,7 @@ public class FindCoordinatorResponse extends AbstractResponse { this.data = data; } + @Override public FindCoordinatorResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java index 754566db7a4..482e61a255a 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java @@ -50,7 +50,7 @@ public class HeartbeatRequest extends AbstractRequest { } } - public final HeartbeatRequestData data; + private final HeartbeatRequestData data; private HeartbeatRequest(HeartbeatRequestData data, short version) { super(ApiKeys.HEARTBEAT, version); @@ -72,7 +72,7 @@ public class HeartbeatRequest extends AbstractRequest { } @Override - protected HeartbeatRequestData data() { + public HeartbeatRequestData data() { return data; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java index c8fdae2507c..eb402fcbab9 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java @@ -58,7 +58,7 @@ public class HeartbeatResponse extends AbstractResponse { } @Override - protected HeartbeatResponseData data() { + public HeartbeatResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/IncrementalAlterConfigsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/IncrementalAlterConfigsRequest.java index 210e18b37ac..2bc591410f9 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/IncrementalAlterConfigsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/IncrementalAlterConfigsRequest.java @@ -91,6 +91,7 @@ public class IncrementalAlterConfigsRequest extends AbstractRequest { new ByteBufferAccessor(buffer), version), version); } + @Override public IncrementalAlterConfigsRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/IncrementalAlterConfigsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/IncrementalAlterConfigsResponse.java index 99427c7ad9f..b5887de9b4b 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/IncrementalAlterConfigsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/IncrementalAlterConfigsResponse.java @@ -66,6 +66,7 @@ public class IncrementalAlterConfigsResponse extends AbstractResponse { this.data = data; } + @Override public IncrementalAlterConfigsResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdRequest.java index 08e75cb536c..5c24b41b351 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdRequest.java @@ -51,7 +51,7 @@ public class InitProducerIdRequest extends AbstractRequest { } } - public final InitProducerIdRequestData data; + private final InitProducerIdRequestData data; private InitProducerIdRequest(InitProducerIdRequestData data, short version) { super(ApiKeys.INIT_PRODUCER_ID, version); @@ -73,7 +73,8 @@ public class InitProducerIdRequest extends AbstractRequest { } @Override - protected InitProducerIdRequestData data() { + public InitProducerIdRequestData data() { return data; } + } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdResponse.java index 6066c13ef5a..f8451d7863b 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdResponse.java @@ -36,7 +36,7 @@ import java.util.Map; * - {@link Errors#PRODUCER_FENCED} */ public class InitProducerIdResponse extends AbstractResponse { - public final InitProducerIdResponseData data; + private final InitProducerIdResponseData data; public InitProducerIdResponse(InitProducerIdResponseData data) { super(ApiKeys.INIT_PRODUCER_ID); @@ -54,7 +54,7 @@ public class InitProducerIdResponse extends AbstractResponse { } @Override - protected InitProducerIdResponseData data() { + public InitProducerIdResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java index 7be16b6410c..35155a085b6 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java @@ -107,6 +107,7 @@ public class JoinGroupRequest extends AbstractRequest { } } + @Override public JoinGroupRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java index 8472175edab..336c82462a2 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java @@ -33,6 +33,7 @@ public class JoinGroupResponse extends AbstractResponse { this.data = data; } + @Override public JoinGroupResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java index 9927dda1de6..833e0255336 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java @@ -166,7 +166,8 @@ public class LeaderAndIsrRequest extends AbstractControlRequest { return Collections.unmodifiableList(data.liveLeaders()); } - protected LeaderAndIsrRequestData data() { + @Override + public LeaderAndIsrRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java index 1e1069edea3..974dde84ec4 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java @@ -72,7 +72,7 @@ public class LeaderAndIsrResponse extends AbstractResponse { } @Override - protected LeaderAndIsrResponseData data() { + public LeaderAndIsrResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupRequest.java index ad7547c648e..8ce95350fc9 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupRequest.java @@ -87,6 +87,7 @@ public class LeaveGroupRequest extends AbstractRequest { this.data = data; } + @Override public LeaveGroupRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupResponse.java index 3bc70674f8a..9a59139f4e7 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupResponse.java @@ -48,7 +48,7 @@ import java.util.Objects; */ public class LeaveGroupResponse extends AbstractResponse { - public final LeaveGroupResponseData data; + private final LeaveGroupResponseData data; public LeaveGroupResponse(LeaveGroupResponseData data) { super(ApiKeys.LEAVE_GROUP); @@ -122,7 +122,7 @@ public class LeaveGroupResponse extends AbstractResponse { } @Override - protected LeaveGroupResponseData data() { + public LeaveGroupResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java index 336368395b7..270c43c0568 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java @@ -33,6 +33,7 @@ public class ListGroupsResponse extends AbstractResponse { this.data = data; } + @Override public ListGroupsResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListPartitionReassignmentsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListPartitionReassignmentsRequest.java index 86cfc6632b7..03affd11b67 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListPartitionReassignmentsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListPartitionReassignmentsRequest.java @@ -63,6 +63,7 @@ public class ListPartitionReassignmentsRequest extends AbstractRequest { new ByteBufferAccessor(buffer), version), version); } + @Override public ListPartitionReassignmentsRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListPartitionReassignmentsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListPartitionReassignmentsResponse.java index 3449f1cabe9..4a890e8b50c 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListPartitionReassignmentsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListPartitionReassignmentsResponse.java @@ -38,6 +38,7 @@ public class ListPartitionReassignmentsResponse extends AbstractResponse { new ByteBufferAccessor(buffer), version)); } + @Override public ListPartitionReassignmentsResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java index 5aceb2291ff..816f6000615 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java @@ -108,6 +108,7 @@ public class MetadataRequest extends AbstractRequest { this.data = data; } + @Override public MetadataRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java index ba06439bad1..097c26e687d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java @@ -74,7 +74,7 @@ public class MetadataResponse extends AbstractResponse { } @Override - protected MetadataResponseData data() { + public MetadataResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java index 6542ed42574..9869da5d254 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java @@ -73,6 +73,7 @@ public class OffsetCommitRequest extends AbstractRequest { this.data = data; } + @Override public OffsetCommitRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java index b2851a68deb..2ed0e312983 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java @@ -81,6 +81,7 @@ public class OffsetCommitResponse extends AbstractResponse { this(DEFAULT_THROTTLE_TIME, responseData); } + @Override public OffsetCommitResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetDeleteRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetDeleteRequest.java index 2f9d0203a9d..28b763d520f 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetDeleteRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetDeleteRequest.java @@ -46,7 +46,7 @@ public class OffsetDeleteRequest extends AbstractRequest { } } - public final OffsetDeleteRequestData data; + private final OffsetDeleteRequestData data; public OffsetDeleteRequest(OffsetDeleteRequestData data, short version) { super(ApiKeys.OFFSET_DELETE, version); @@ -71,7 +71,7 @@ public class OffsetDeleteRequest extends AbstractRequest { } @Override - protected OffsetDeleteRequestData data() { + public OffsetDeleteRequestData data() { return data; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetDeleteResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetDeleteResponse.java index ceb932fb21d..79f6f4e6d34 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetDeleteResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetDeleteResponse.java @@ -44,7 +44,7 @@ import java.util.Map; */ public class OffsetDeleteResponse extends AbstractResponse { - public final OffsetDeleteResponseData data; + private final OffsetDeleteResponseData data; public OffsetDeleteResponse(OffsetDeleteResponseData data) { super(ApiKeys.OFFSET_DELETE); @@ -52,7 +52,7 @@ public class OffsetDeleteResponse extends AbstractResponse { } @Override - protected OffsetDeleteResponseData data() { + public OffsetDeleteResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java index fd08c725399..c35d479d64d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java @@ -38,7 +38,7 @@ public class OffsetFetchRequest extends AbstractRequest { private static final Logger log = LoggerFactory.getLogger(OffsetFetchRequest.class); private static final List ALL_TOPIC_PARTITIONS = null; - public final OffsetFetchRequestData data; + private final OffsetFetchRequestData data; public static class Builder extends AbstractRequest.Builder { @@ -175,7 +175,7 @@ public class OffsetFetchRequest extends AbstractRequest { } @Override - protected OffsetFetchRequestData data() { + public OffsetFetchRequestData data() { return data; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java index 3e30968715f..594eb0e5bcf 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java @@ -63,7 +63,7 @@ public class OffsetFetchResponse extends AbstractResponse { private static final List PARTITION_ERRORS = Arrays.asList( Errors.UNKNOWN_TOPIC_OR_PARTITION, Errors.TOPIC_AUTHORIZATION_FAILED); - public final OffsetFetchResponseData data; + private final OffsetFetchResponseData data; private final Errors error; public static final class PartitionData { @@ -219,7 +219,7 @@ public class OffsetFetchResponse extends AbstractResponse { } @Override - protected OffsetFetchResponseData data() { + public OffsetFetchResponseData data() { return data; } 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 2860576d539..b94e48b54a1 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 @@ -107,6 +107,7 @@ public class ProduceResponse extends AbstractResponse { return data; } + @Override public ProduceResponseData data() { return this.data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/RenewDelegationTokenRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/RenewDelegationTokenRequest.java index 2c364aef3df..91a9f968d9d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/RenewDelegationTokenRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/RenewDelegationTokenRequest.java @@ -38,6 +38,7 @@ public class RenewDelegationTokenRequest extends AbstractRequest { new ByteBufferAccessor(buffer), version), version); } + @Override public RenewDelegationTokenRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/RenewDelegationTokenResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/RenewDelegationTokenResponse.java index 2ef840378ac..30708ff038c 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/RenewDelegationTokenResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/RenewDelegationTokenResponse.java @@ -44,7 +44,7 @@ public class RenewDelegationTokenResponse extends AbstractResponse { } @Override - protected RenewDelegationTokenResponseData data() { + public RenewDelegationTokenResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SaslAuthenticateRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/SaslAuthenticateRequest.java index 35da4caf9c5..e2080ce841b 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/SaslAuthenticateRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/SaslAuthenticateRequest.java @@ -60,6 +60,7 @@ public class SaslAuthenticateRequest extends AbstractRequest { this.data = data; } + @Override public SaslAuthenticateRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SaslAuthenticateResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/SaslAuthenticateResponse.java index 2e1a2189a39..bd12d3d4ae7 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/SaslAuthenticateResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/SaslAuthenticateResponse.java @@ -68,7 +68,7 @@ public class SaslAuthenticateResponse extends AbstractResponse { } @Override - protected SaslAuthenticateResponseData data() { + public SaslAuthenticateResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java index e64dd6b2cb7..09d3a87c8e7 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java @@ -61,6 +61,7 @@ public class SaslHandshakeRequest extends AbstractRequest { this.data = data; } + @Override public SaslHandshakeRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java index b0ac2efcdec..63c047a0619 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java @@ -58,7 +58,7 @@ public class SaslHandshakeResponse extends AbstractResponse { } @Override - protected SaslHandshakeResponseData data() { + public SaslHandshakeResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java index 2b0aef76693..4326aaffd87 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java @@ -211,7 +211,7 @@ public class StopReplicaRequest extends AbstractControlRequest { } @Override - protected StopReplicaRequestData data() { + public StopReplicaRequestData data() { return data; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java index 27a2502e751..10ab153f440 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java @@ -71,7 +71,7 @@ public class StopReplicaResponse extends AbstractResponse { } @Override - protected StopReplicaResponseData data() { + public StopReplicaResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupRequest.java index 18e5eb8b809..8242b71a03f 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupRequest.java @@ -53,7 +53,7 @@ public class SyncGroupRequest extends AbstractRequest { } } - public final SyncGroupRequestData data; + private final SyncGroupRequestData data; public SyncGroupRequest(SyncGroupRequestData data, short version) { super(ApiKeys.SYNC_GROUP, version); @@ -92,7 +92,7 @@ public class SyncGroupRequest extends AbstractRequest { } @Override - protected SyncGroupRequestData data() { + public SyncGroupRequestData data() { return data; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java index a26b6c900bb..822a3e78b99 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java @@ -26,7 +26,7 @@ import java.util.Map; public class SyncGroupResponse extends AbstractResponse { - public final SyncGroupResponseData data; + private final SyncGroupResponseData data; public SyncGroupResponse(SyncGroupResponseData data) { super(ApiKeys.SYNC_GROUP); @@ -48,7 +48,7 @@ public class SyncGroupResponse extends AbstractResponse { } @Override - protected SyncGroupResponseData data() { + public SyncGroupResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitRequest.java index 055793cf54b..e96f81ae7e2 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitRequest.java @@ -44,7 +44,7 @@ public class TxnOffsetCommitRequest extends AbstractRequest { private static final Logger log = LoggerFactory.getLogger(TxnOffsetCommitRequest.class); - public final TxnOffsetCommitRequestData data; + private final TxnOffsetCommitRequestData data; public static class Builder extends AbstractRequest.Builder { @@ -168,7 +168,7 @@ public class TxnOffsetCommitRequest extends AbstractRequest { } @Override - protected TxnOffsetCommitRequestData data() { + public TxnOffsetCommitRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java index 1ead0ec6d60..b4de54741e6 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java @@ -47,7 +47,7 @@ import java.util.Map; */ public class TxnOffsetCommitResponse extends AbstractResponse { - public final TxnOffsetCommitResponseData data; + private final TxnOffsetCommitResponseData data; public TxnOffsetCommitResponse(TxnOffsetCommitResponseData data) { super(ApiKeys.TXN_OFFSET_COMMIT); @@ -78,7 +78,7 @@ public class TxnOffsetCommitResponse extends AbstractResponse { } @Override - protected TxnOffsetCommitResponseData data() { + public TxnOffsetCommitResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java index d61056f64be..fc189247500 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java @@ -71,6 +71,7 @@ public class UpdateFeaturesRequest extends AbstractRequest { return new UpdateFeaturesResponse(responseData); } + @Override public UpdateFeaturesRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java index 2c28842cd89..028f8855ce5 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java @@ -67,6 +67,7 @@ public class UpdateFeaturesResponse extends AbstractResponse { return data.toString(); } + @Override public UpdateFeaturesResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java index a0ef3471f12..2d443b9b6a2 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java @@ -201,7 +201,7 @@ public class UpdateMetadataRequest extends AbstractControlRequest { } @Override - protected UpdateMetadataRequestData data() { + public UpdateMetadataRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java index 4749067a9a7..cc7749a4724 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java @@ -52,7 +52,7 @@ public class UpdateMetadataResponse extends AbstractResponse { } @Override - protected UpdateMetadataResponseData data() { + public UpdateMetadataResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/VoteRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/VoteRequest.java index d31d019206f..8fba2f085d5 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/VoteRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/VoteRequest.java @@ -47,7 +47,7 @@ public class VoteRequest extends AbstractRequest { } } - public final VoteRequestData data; + private final VoteRequestData data; private VoteRequest(VoteRequestData data, short version) { super(ApiKeys.VOTE, version); @@ -55,7 +55,7 @@ public class VoteRequest extends AbstractRequest { } @Override - protected VoteRequestData data() { + public VoteRequestData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/VoteResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/VoteResponse.java index 5956fe0077e..51991adcf0c 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/VoteResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/VoteResponse.java @@ -42,7 +42,7 @@ import java.util.Map; * - {@link Errors#UNKNOWN_TOPIC_OR_PARTITION} */ public class VoteResponse extends AbstractResponse { - public final VoteResponseData data; + private final VoteResponseData data; public VoteResponse(VoteResponseData data) { super(ApiKeys.VOTE); @@ -83,7 +83,7 @@ public class VoteResponse extends AbstractResponse { } @Override - protected VoteResponseData data() { + public VoteResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersRequest.java index 5998ca7e07e..64a3df49c7b 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersRequest.java @@ -134,7 +134,7 @@ public class WriteTxnMarkersRequest extends AbstractRequest { } } - public final WriteTxnMarkersRequestData data; + private final WriteTxnMarkersRequestData data; private WriteTxnMarkersRequest(WriteTxnMarkersRequestData data, short version) { super(ApiKeys.WRITE_TXN_MARKERS, version); @@ -142,7 +142,7 @@ public class WriteTxnMarkersRequest extends AbstractRequest { } @Override - protected WriteTxnMarkersRequestData data() { + public WriteTxnMarkersRequestData data() { return data; } diff --git a/clients/src/test/java/org/apache/kafka/clients/NodeApiVersionsTest.java b/clients/src/test/java/org/apache/kafka/clients/NodeApiVersionsTest.java index d78d573db3b..3004d053157 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NodeApiVersionsTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NodeApiVersionsTest.java @@ -130,7 +130,7 @@ public class NodeApiVersionsTest { @Test public void testUsableVersionLatestVersions() { List versionList = new LinkedList<>(); - for (ApiVersionsResponseKey apiVersion: ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.data.apiKeys()) { + for (ApiVersionsResponseKey apiVersion: ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.data().apiKeys()) { versionList.add(new ApiVersion(apiVersion)); } // Add an API key that we don't know about. @@ -148,9 +148,9 @@ public class NodeApiVersionsTest { @Test public void testConstructionFromApiVersionsResponse() { ApiVersionsResponse apiVersionsResponse = ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE; - NodeApiVersions versions = new NodeApiVersions(apiVersionsResponse.data.apiKeys()); + NodeApiVersions versions = new NodeApiVersions(apiVersionsResponse.data().apiKeys()); - for (ApiVersionsResponseKey apiVersionKey : apiVersionsResponse.data.apiKeys()) { + for (ApiVersionsResponseKey apiVersionKey : apiVersionsResponse.data().apiKeys()) { ApiVersion apiVersion = versions.apiVersion(ApiKeys.forId(apiVersionKey.apiKey())); assertEquals(apiVersionKey.apiKey(), apiVersion.apiKey); assertEquals(apiVersionKey.minVersion(), apiVersion.minVersion); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java index e8499a84691..c24fa8577cc 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java @@ -447,8 +447,8 @@ public class AbstractCoordinatorTest { coordinator.resetGenerationOnLeaveGroup(); SyncGroupRequest syncGroupRequest = (SyncGroupRequest) body; - return syncGroupRequest.data.protocolType().equals(PROTOCOL_TYPE) - && syncGroupRequest.data.protocolName().equals(PROTOCOL_NAME); + return syncGroupRequest.data().protocolType().equals(PROTOCOL_TYPE) + && syncGroupRequest.data().protocolName().equals(PROTOCOL_NAME); }, syncGroupResponse(Errors.NONE, PROTOCOL_TYPE, wrongProtocolName)); // let the retry to complete successfully to break out of the while loop @@ -467,8 +467,8 @@ public class AbstractCoordinatorTest { } SyncGroupRequest syncGroupRequest = (SyncGroupRequest) body; - return syncGroupRequest.data.protocolType().equals(PROTOCOL_TYPE) - && syncGroupRequest.data.protocolName().equals(PROTOCOL_NAME); + return syncGroupRequest.data().protocolType().equals(PROTOCOL_TYPE) + && syncGroupRequest.data().protocolName().equals(PROTOCOL_NAME); }, syncGroupResponse(Errors.NONE, PROTOCOL_TYPE, PROTOCOL_NAME)); // No exception shall be thrown as the generation is reset. @@ -497,8 +497,8 @@ public class AbstractCoordinatorTest { return false; } SyncGroupRequest syncGroupRequest = (SyncGroupRequest) body; - return syncGroupRequest.data.protocolType().equals(PROTOCOL_TYPE) - && syncGroupRequest.data.protocolName().equals(PROTOCOL_NAME); + return syncGroupRequest.data().protocolType().equals(PROTOCOL_TYPE) + && syncGroupRequest.data().protocolName().equals(PROTOCOL_NAME); }, syncGroupResponse(Errors.NONE, syncGroupResponseProtocolType, syncGroupResponseProtocolName)); return coordinator.joinGroupIfNeeded(mockTime.timer(5000L)); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java index 843689294da..1c7cfb47bdb 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java @@ -710,8 +710,8 @@ public class ConsumerCoordinatorTest { client.prepareResponse(joinGroupLeaderResponse(1, consumerId, memberSubscriptions, Errors.NONE)); client.prepareResponse(body -> { SyncGroupRequest sync = (SyncGroupRequest) body; - return sync.data.memberId().equals(consumerId) && - sync.data.generationId() == 1 && + return sync.data().memberId().equals(consumerId) && + sync.data().generationId() == 1 && sync.groupAssignments().containsKey(consumerId); }, syncGroupResponse(assigned, Errors.NONE)); coordinator.poll(time.timer(Long.MAX_VALUE)); @@ -748,8 +748,8 @@ public class ConsumerCoordinatorTest { 1, consumerId, singletonMap(consumerId, oldSubscription), Errors.NONE)); client.prepareResponse(body -> { SyncGroupRequest sync = (SyncGroupRequest) body; - return sync.data.memberId().equals(consumerId) && - sync.data.generationId() == 1 && + return sync.data().memberId().equals(consumerId) && + sync.data().generationId() == 1 && sync.groupAssignments().containsKey(consumerId); }, syncGroupResponse(oldAssignment, Errors.NONE)); @@ -759,8 +759,8 @@ public class ConsumerCoordinatorTest { 1, consumerId, singletonMap(consumerId, newSubscription), Errors.NONE)); client.prepareResponse(body -> { SyncGroupRequest sync = (SyncGroupRequest) body; - return sync.data.memberId().equals(consumerId) && - sync.data.generationId() == 1 && + return sync.data().memberId().equals(consumerId) && + sync.data().generationId() == 1 && sync.groupAssignments().containsKey(consumerId); }, syncGroupResponse(newAssignment, Errors.NONE)); @@ -834,8 +834,8 @@ public class ConsumerCoordinatorTest { client.prepareResponse(joinGroupLeaderResponse(1, consumerId, memberSubscriptions, Errors.NONE)); client.prepareResponse(body -> { SyncGroupRequest sync = (SyncGroupRequest) body; - return sync.data.memberId().equals(consumerId) && - sync.data.generationId() == 1 && + return sync.data().memberId().equals(consumerId) && + sync.data().generationId() == 1 && sync.groupAssignments().containsKey(consumerId); }, syncGroupResponse(assigned, Errors.NONE)); // expect client to force updating the metadata, if yes gives it both topics @@ -983,8 +983,8 @@ public class ConsumerCoordinatorTest { client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE)); client.prepareResponse(body -> { SyncGroupRequest sync = (SyncGroupRequest) body; - return sync.data.memberId().equals(consumerId) && - sync.data.generationId() == 1 && + return sync.data().memberId().equals(consumerId) && + sync.data().generationId() == 1 && sync.groupAssignments().isEmpty(); }, syncGroupResponse(singletonList(t1p), Errors.NONE)); @@ -1116,8 +1116,8 @@ public class ConsumerCoordinatorTest { client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE)); client.prepareResponse(body -> { SyncGroupRequest sync = (SyncGroupRequest) body; - return sync.data.memberId().equals(consumerId) && - sync.data.generationId() == 1 && + return sync.data().memberId().equals(consumerId) && + sync.data().generationId() == 1 && sync.groupAssignments().isEmpty(); }, syncGroupResponse(assigned, Errors.NONE)); @@ -1180,8 +1180,8 @@ public class ConsumerCoordinatorTest { client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE)); client.prepareResponse(body -> { SyncGroupRequest sync = (SyncGroupRequest) body; - return sync.data.memberId().equals(consumerId) && - sync.data.generationId() == 1 && + return sync.data().memberId().equals(consumerId) && + sync.data().generationId() == 1 && sync.groupAssignments().isEmpty(); }, syncGroupResponse(assigned, Errors.NONE)); // expect client to force updating the metadata, if yes gives it both topics @@ -1410,8 +1410,8 @@ public class ConsumerCoordinatorTest { client.prepareResponse(joinGroupLeaderResponse(1, consumerId, memberSubscriptions, Errors.NONE)); client.prepareResponse(body -> { SyncGroupRequest sync = (SyncGroupRequest) body; - if (sync.data.memberId().equals(consumerId) && - sync.data.generationId() == 1 && + if (sync.data().memberId().equals(consumerId) && + sync.data().generationId() == 1 && sync.groupAssignments().containsKey(consumerId)) { // trigger the metadata update including both topics after the sync group request has been sent Map topicPartitionCounts = new HashMap<>(); @@ -2299,8 +2299,8 @@ public class ConsumerCoordinatorTest { client.prepareResponse(body -> { SyncGroupRequest sync = (SyncGroupRequest) body; - return sync.data.memberId().equals(consumerId) && - sync.data.generationId() == 1 && + return sync.data().memberId().equals(consumerId) && + sync.data().generationId() == 1 && sync.groupAssignments().containsKey(consumerId); }, syncGroupResponse(singletonList(t1p), Errors.NONE)); coordinator.poll(time.timer(Long.MAX_VALUE)); @@ -3148,8 +3148,8 @@ public class ConsumerCoordinatorTest { generation, consumerId, singletonMap(consumerId, subscription), Errors.NONE)); client.prepareResponse(body -> { SyncGroupRequest sync = (SyncGroupRequest) body; - return sync.data.memberId().equals(consumerId) && - sync.data.generationId() == generation && + return sync.data().memberId().equals(consumerId) && + sync.data().generationId() == generation && sync.groupAssignments().containsKey(consumerId); }, syncGroupResponse(assignment, Errors.NONE)); } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java index 2cef6f688f8..57c8474996d 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java @@ -898,7 +898,7 @@ public class KafkaProducerTest { client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, host1)); String groupId = "group"; client.prepareResponse(request -> - ((TxnOffsetCommitRequest) request).data.groupId().equals(groupId), + ((TxnOffsetCommitRequest) request).data().groupId().equals(groupId), txnOffsetsCommitResponse(Collections.singletonMap( new TopicPartition("topic", 0), Errors.NONE))); client.prepareResponse(endTxnResponse(Errors.NONE)); @@ -949,7 +949,7 @@ public class KafkaProducerTest { int generationId = 5; String groupInstanceId = "instance"; client.prepareResponse(request -> { - TxnOffsetCommitRequestData data = ((TxnOffsetCommitRequest) request).data; + TxnOffsetCommitRequestData data = ((TxnOffsetCommitRequest) request).data(); if (maxVersion < 3) { return data.groupId().equals(groupId) && data.memberId().equals(JoinGroupRequest.UNKNOWN_MEMBER_ID) && 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 995dfc77d38..a054f6f1811 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 @@ -2728,7 +2728,7 @@ public class SenderTest { client.prepareResponse( body -> body instanceof InitProducerIdRequest && - ((InitProducerIdRequest) body).data.transactionalId() == null, + ((InitProducerIdRequest) body).data().transactionalId() == null, initProducerIdResponse(producerId, producerEpoch, error)); sender.runOnce(); } 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 468099fc13b..c2977ea9f66 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 @@ -839,8 +839,8 @@ public class TransactionManagerTest { client.prepareUnsupportedVersionResponse(body -> { InitProducerIdRequest initProducerIdRequest = (InitProducerIdRequest) body; - assertEquals(initProducerIdRequest.data.transactionalId(), transactionalId); - assertEquals(initProducerIdRequest.data.transactionTimeoutMs(), transactionTimeoutMs); + assertEquals(initProducerIdRequest.data().transactionalId(), transactionalId); + assertEquals(initProducerIdRequest.data().transactionTimeoutMs(), transactionTimeoutMs); return true; }); @@ -890,11 +890,11 @@ public class TransactionManagerTest { client.prepareResponse(request -> { TxnOffsetCommitRequest txnOffsetCommitRequest = (TxnOffsetCommitRequest) request; - assertEquals(consumerGroupId, txnOffsetCommitRequest.data.groupId()); - assertEquals(producerId, txnOffsetCommitRequest.data.producerId()); - assertEquals(epoch, txnOffsetCommitRequest.data.producerEpoch()); - return txnOffsetCommitRequest.data.groupInstanceId().equals(groupInstanceId) - && !txnOffsetCommitRequest.data.memberId().equals(memberId); + assertEquals(consumerGroupId, txnOffsetCommitRequest.data().groupId()); + assertEquals(producerId, txnOffsetCommitRequest.data().producerId()); + assertEquals(epoch, txnOffsetCommitRequest.data().producerEpoch()); + return txnOffsetCommitRequest.data().groupInstanceId().equals(groupInstanceId) + && !txnOffsetCommitRequest.data().memberId().equals(memberId); }, new TxnOffsetCommitResponse(0, singletonMap(tp, Errors.FENCED_INSTANCE_ID))); runUntil(transactionManager::hasError); @@ -924,10 +924,10 @@ public class TransactionManagerTest { client.prepareResponse(request -> { TxnOffsetCommitRequest txnOffsetCommitRequest = (TxnOffsetCommitRequest) request; - assertEquals(consumerGroupId, txnOffsetCommitRequest.data.groupId()); - assertEquals(producerId, txnOffsetCommitRequest.data.producerId()); - assertEquals(epoch, txnOffsetCommitRequest.data.producerEpoch()); - return !txnOffsetCommitRequest.data.memberId().equals(memberId); + assertEquals(consumerGroupId, txnOffsetCommitRequest.data().groupId()); + assertEquals(producerId, txnOffsetCommitRequest.data().producerId()); + assertEquals(epoch, txnOffsetCommitRequest.data().producerEpoch()); + return !txnOffsetCommitRequest.data().memberId().equals(memberId); }, new TxnOffsetCommitResponse(0, singletonMap(tp, Errors.UNKNOWN_MEMBER_ID))); runUntil(transactionManager::hasError); @@ -959,10 +959,10 @@ public class TransactionManagerTest { prepareTxnOffsetCommitResponse(consumerGroupId, producerId, epoch, singletonMap(tp, Errors.ILLEGAL_GENERATION)); client.prepareResponse(request -> { TxnOffsetCommitRequest txnOffsetCommitRequest = (TxnOffsetCommitRequest) request; - assertEquals(consumerGroupId, txnOffsetCommitRequest.data.groupId()); - assertEquals(producerId, txnOffsetCommitRequest.data.producerId()); - assertEquals(epoch, txnOffsetCommitRequest.data.producerEpoch()); - return txnOffsetCommitRequest.data.generationId() != generationId; + assertEquals(consumerGroupId, txnOffsetCommitRequest.data().groupId()); + assertEquals(producerId, txnOffsetCommitRequest.data().producerId()); + assertEquals(epoch, txnOffsetCommitRequest.data().producerEpoch()); + return txnOffsetCommitRequest.data().generationId() != generationId; }, new TxnOffsetCommitResponse(0, singletonMap(tp, Errors.ILLEGAL_GENERATION))); runUntil(transactionManager::hasError); @@ -3332,8 +3332,8 @@ public class TransactionManagerTest { .setThrottleTimeMs(0); client.prepareResponse(body -> { InitProducerIdRequest initProducerIdRequest = (InitProducerIdRequest) body; - assertEquals(transactionalId, initProducerIdRequest.data.transactionalId()); - assertEquals(transactionTimeoutMs, initProducerIdRequest.data.transactionTimeoutMs()); + assertEquals(transactionalId, initProducerIdRequest.data().transactionalId()); + assertEquals(transactionTimeoutMs, initProducerIdRequest.data().transactionTimeoutMs()); return true; }, new InitProducerIdResponse(responseData), shouldDisconnect); } @@ -3396,10 +3396,10 @@ public class TransactionManagerTest { final short epoch, final long producerId) { return body -> { AddPartitionsToTxnRequest addPartitionsToTxnRequest = (AddPartitionsToTxnRequest) body; - assertEquals(producerId, addPartitionsToTxnRequest.data.producerId()); - assertEquals(epoch, addPartitionsToTxnRequest.data.producerEpoch()); + assertEquals(producerId, addPartitionsToTxnRequest.data().producerId()); + assertEquals(epoch, addPartitionsToTxnRequest.data().producerEpoch()); assertEquals(singletonList(topicPartition), addPartitionsToTxnRequest.partitions()); - assertEquals(transactionalId, addPartitionsToTxnRequest.data.transactionalId()); + assertEquals(transactionalId, addPartitionsToTxnRequest.data().transactionalId()); return true; }; } @@ -3430,9 +3430,9 @@ public class TransactionManagerTest { private MockClient.RequestMatcher endTxnMatcher(final TransactionResult result, final long producerId, final short epoch) { return body -> { EndTxnRequest endTxnRequest = (EndTxnRequest) body; - assertEquals(transactionalId, endTxnRequest.data.transactionalId()); - assertEquals(producerId, endTxnRequest.data.producerId()); - assertEquals(epoch, endTxnRequest.data.producerEpoch()); + assertEquals(transactionalId, endTxnRequest.data().transactionalId()); + assertEquals(producerId, endTxnRequest.data().producerId()); + assertEquals(epoch, endTxnRequest.data().producerEpoch()); assertEquals(result, endTxnRequest.result()); return true; }; @@ -3444,10 +3444,10 @@ public class TransactionManagerTest { final short producerEpoch) { client.prepareResponse(body -> { AddOffsetsToTxnRequest addOffsetsToTxnRequest = (AddOffsetsToTxnRequest) body; - assertEquals(consumerGroupId, addOffsetsToTxnRequest.data.groupId()); - assertEquals(transactionalId, addOffsetsToTxnRequest.data.transactionalId()); - assertEquals(producerId, addOffsetsToTxnRequest.data.producerId()); - assertEquals(producerEpoch, addOffsetsToTxnRequest.data.producerEpoch()); + assertEquals(consumerGroupId, addOffsetsToTxnRequest.data().groupId()); + assertEquals(transactionalId, addOffsetsToTxnRequest.data().transactionalId()); + assertEquals(producerId, addOffsetsToTxnRequest.data().producerId()); + assertEquals(producerEpoch, addOffsetsToTxnRequest.data().producerEpoch()); return true; }, new AddOffsetsToTxnResponse( new AddOffsetsToTxnResponseData() @@ -3461,9 +3461,9 @@ public class TransactionManagerTest { Map txnOffsetCommitResponse) { client.prepareResponse(request -> { TxnOffsetCommitRequest txnOffsetCommitRequest = (TxnOffsetCommitRequest) request; - assertEquals(consumerGroupId, txnOffsetCommitRequest.data.groupId()); - assertEquals(producerId, txnOffsetCommitRequest.data.producerId()); - assertEquals(producerEpoch, txnOffsetCommitRequest.data.producerEpoch()); + assertEquals(consumerGroupId, txnOffsetCommitRequest.data().groupId()); + assertEquals(producerId, txnOffsetCommitRequest.data().producerId()); + assertEquals(producerEpoch, txnOffsetCommitRequest.data().producerEpoch()); return true; }, new TxnOffsetCommitResponse(0, txnOffsetCommitResponse)); } @@ -3477,12 +3477,12 @@ public class TransactionManagerTest { Map txnOffsetCommitResponse) { client.prepareResponse(request -> { TxnOffsetCommitRequest txnOffsetCommitRequest = (TxnOffsetCommitRequest) request; - assertEquals(consumerGroupId, txnOffsetCommitRequest.data.groupId()); - assertEquals(producerId, txnOffsetCommitRequest.data.producerId()); - assertEquals(producerEpoch, txnOffsetCommitRequest.data.producerEpoch()); - assertEquals(groupInstanceId, txnOffsetCommitRequest.data.groupInstanceId()); - assertEquals(memberId, txnOffsetCommitRequest.data.memberId()); - assertEquals(generationId, txnOffsetCommitRequest.data.generationId()); + assertEquals(consumerGroupId, txnOffsetCommitRequest.data().groupId()); + assertEquals(producerId, txnOffsetCommitRequest.data().producerId()); + assertEquals(producerEpoch, txnOffsetCommitRequest.data().producerEpoch()); + assertEquals(groupInstanceId, txnOffsetCommitRequest.data().groupInstanceId()); + assertEquals(memberId, txnOffsetCommitRequest.data().memberId()); + assertEquals(generationId, txnOffsetCommitRequest.data().generationId()); return true; }, new TxnOffsetCommitResponse(0, txnOffsetCommitResponse)); } @@ -3506,7 +3506,7 @@ public class TransactionManagerTest { .setThrottleTimeMs(0); client.prepareResponse(body -> { InitProducerIdRequest initProducerIdRequest = (InitProducerIdRequest) body; - assertNull(initProducerIdRequest.data.transactionalId()); + assertNull(initProducerIdRequest.data().transactionalId()); return true; }, new InitProducerIdResponse(responseData), false); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequestTest.java index 86e6bd7595f..1758a978f0b 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequestTest.java @@ -46,9 +46,9 @@ public class AddPartitionsToTxnRequestTest { for (short version = 0; version <= ApiKeys.ADD_PARTITIONS_TO_TXN.latestVersion(); version++) { AddPartitionsToTxnRequest request = builder.build(version); - assertEquals(transactionalId, request.data.transactionalId()); - assertEquals(producerId, request.data.producerId()); - assertEquals(producerEpoch, request.data.producerEpoch()); + assertEquals(transactionalId, request.data().transactionalId()); + assertEquals(producerId, request.data().producerId()); + assertEquals(producerEpoch, request.data().producerEpoch()); assertEquals(partitions, request.partitions()); AddPartitionsToTxnResponse response = request.getErrorResponse(throttleTimeMs, Errors.UNKNOWN_TOPIC_OR_PARTITION.exception()); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java index 98b0c149ce1..b578e2c39db 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java @@ -35,14 +35,14 @@ public class ApiVersionsResponseTest { @Test public void shouldCreateApiResponseThatHasAllApiKeysSupportedByBroker() { assertEquals(apiKeysInResponse(ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE), new HashSet<>(ApiKeys.enabledApis())); - assertTrue(ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.data.supportedFeatures().isEmpty()); - assertTrue(ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.data.finalizedFeatures().isEmpty()); - assertEquals(ApiVersionsResponse.UNKNOWN_FINALIZED_FEATURES_EPOCH, ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.data.finalizedFeaturesEpoch()); + assertTrue(ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.data().supportedFeatures().isEmpty()); + assertTrue(ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.data().finalizedFeatures().isEmpty()); + assertEquals(ApiVersionsResponse.UNKNOWN_FINALIZED_FEATURES_EPOCH, ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.data().finalizedFeaturesEpoch()); } @Test public void shouldHaveCorrectDefaultApiVersionsResponse() { - Collection apiVersions = ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.data.apiKeys(); + Collection apiVersions = ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.data().apiKeys(); assertEquals("API versions for all API keys must be maintained.", apiVersions.size(), ApiKeys.enabledApis().size()); for (ApiKeys key : ApiKeys.enabledApis()) { @@ -64,15 +64,14 @@ public class ApiVersionsResponseTest { } } - assertTrue(ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.data.supportedFeatures().isEmpty()); - assertTrue(ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.data.finalizedFeatures().isEmpty()); - assertEquals(ApiVersionsResponse.UNKNOWN_FINALIZED_FEATURES_EPOCH, ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.data.finalizedFeaturesEpoch()); + assertTrue(ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.data().supportedFeatures().isEmpty()); + assertTrue(ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.data().finalizedFeatures().isEmpty()); + assertEquals(ApiVersionsResponse.UNKNOWN_FINALIZED_FEATURES_EPOCH, ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.data().finalizedFeaturesEpoch()); } - private Set apiKeysInResponse(final ApiVersionsResponse apiVersions) { final Set apiKeys = new HashSet<>(); - for (final ApiVersionsResponseKey version : apiVersions.data.apiKeys()) { + for (final ApiVersionsResponseKey version : apiVersions.data().apiKeys()) { apiKeys.add(ApiKeys.forId(version.apiKey())); } return apiKeys; diff --git a/clients/src/test/java/org/apache/kafka/common/requests/OffsetFetchResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/OffsetFetchResponseTest.java index b022a52e7ff..06e611f49a6 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/OffsetFetchResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/OffsetFetchResponseTest.java @@ -106,7 +106,7 @@ public class OffsetFetchResponseTest { OffsetFetchResponse latestResponse = new OffsetFetchResponse(throttleTimeMs, Errors.NONE, partitionDataMap); for (short version = 0; version <= ApiKeys.OFFSET_FETCH.latestVersion(); version++) { - Struct struct = latestResponse.data.toStruct(version); + Struct struct = latestResponse.data().toStruct(version); OffsetFetchResponse oldResponse = OffsetFetchResponse.parse(latestResponse.serialize(version), version); @@ -193,7 +193,7 @@ public class OffsetFetchResponseTest { .setMetadata(null)) )) ); - assertEquals(expectedData, response.data); + assertEquals(expectedData, response.data()); } @Test @@ -226,6 +226,6 @@ public class OffsetFetchResponseTest { .setMetadata(metadata)) )) ); - assertEquals(expectedData, response.data); + assertEquals(expectedData, response.data()); } } diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestContextTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestContextTest.java index 50f41556b0b..5bef2e42a53 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestContextTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestContextTest.java @@ -74,8 +74,8 @@ public class RequestContextTest { ApiVersionsResponse response = (ApiVersionsResponse) AbstractResponse.parseResponse(ApiKeys.API_VERSIONS, responseBuffer, (short) 0); - assertEquals(Errors.UNSUPPORTED_VERSION.code(), response.data.errorCode()); - assertTrue(response.data.apiKeys().isEmpty()); + assertEquals(Errors.UNSUPPORTED_VERSION.code(), response.data().errorCode()); + assertTrue(response.data().apiKeys().isEmpty()); } @Test 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 733843f6536..2ebc3c4b9cb 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 @@ -134,11 +134,11 @@ import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResp 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.OffsetForLeaderEpochResponseData.EpochEndOffset; import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition; import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderTopic; import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderTopicCollection; import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData; -import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset; import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.OffsetForLeaderTopicResult; import org.apache.kafka.common.message.ProduceRequestData; import org.apache.kafka.common.message.RenewDelegationTokenRequestData; @@ -937,9 +937,9 @@ public class RequestResponseTest { ApiVersionsRequest request = new ApiVersionsRequest.Builder().build(); ApiVersionsResponse response = request.getErrorResponse(0, Errors.UNSUPPORTED_VERSION.exception()); - assertEquals(Errors.UNSUPPORTED_VERSION.code(), response.data.errorCode()); + assertEquals(Errors.UNSUPPORTED_VERSION.code(), response.data().errorCode()); - ApiVersionsResponseKey apiVersion = response.data.apiKeys().find(ApiKeys.API_VERSIONS.id); + ApiVersionsResponseKey apiVersion = response.data().apiKeys().find(ApiKeys.API_VERSIONS.id); assertNotNull(apiVersion); assertEquals(ApiKeys.API_VERSIONS.id, apiVersion.apiKey()); assertEquals(ApiKeys.API_VERSIONS.oldestVersion(), apiVersion.minVersion()); @@ -951,8 +951,8 @@ public class RequestResponseTest { ApiVersionsRequest request = new ApiVersionsRequest.Builder().build(); ApiVersionsResponse response = request.getErrorResponse(0, Errors.INVALID_REQUEST.exception()); - assertEquals(response.data.errorCode(), Errors.INVALID_REQUEST.code()); - assertTrue(response.data.apiKeys().isEmpty()); + assertEquals(response.data().errorCode(), Errors.INVALID_REQUEST.code()); + assertTrue(response.data().apiKeys().isEmpty()); } @Test @@ -960,7 +960,7 @@ public class RequestResponseTest { ByteBuffer buffer = ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.serialize((short) 0); ApiVersionsResponse response = ApiVersionsResponse.parse(buffer, ApiKeys.API_VERSIONS.latestVersion()); - assertEquals(Errors.NONE.code(), response.data.errorCode()); + assertEquals(Errors.NONE.code(), response.data().errorCode()); } @Test @@ -974,7 +974,7 @@ public class RequestResponseTest { ByteBuffer buffer = ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.serialize(ApiKeys.API_VERSIONS.latestVersion()); ApiVersionsResponse response = ApiVersionsResponse.parse(buffer, ApiKeys.API_VERSIONS.latestVersion()); - assertEquals(Errors.NONE.code(), response.data.errorCode()); + assertEquals(Errors.NONE.code(), response.data().errorCode()); } @Test diff --git a/clients/src/test/java/org/apache/kafka/common/requests/TxnOffsetCommitRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/TxnOffsetCommitRequestTest.java index 204753e4351..98da6bcd9f4 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/TxnOffsetCommitRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/TxnOffsetCommitRequestTest.java @@ -141,9 +141,9 @@ public class TxnOffsetCommitRequestTest extends OffsetCommitRequestTest { initializeBuilderWithGroupMetadata(true); final TxnOffsetCommitRequest request = builderWithGroupMetadata.build(version); - assertEquals(JoinGroupRequest.UNKNOWN_MEMBER_ID, request.data.memberId()); - assertEquals(JoinGroupRequest.UNKNOWN_GENERATION_ID, request.data.generationId()); - assertNull(request.data.groupInstanceId()); + assertEquals(JoinGroupRequest.UNKNOWN_MEMBER_ID, request.data().memberId()); + assertEquals(JoinGroupRequest.UNKNOWN_GENERATION_ID, request.data().generationId()); + assertNull(request.data().groupInstanceId()); } } diff --git a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java index 2375a9b9996..34ee542a5c6 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java @@ -711,9 +711,9 @@ public class SaslAuthenticatorTest { ByteBuffer responseBuffer = waitForResponse(); ResponseHeader.parse(responseBuffer, ApiKeys.API_VERSIONS.responseHeaderVersion((short) 0)); ApiVersionsResponse response = ApiVersionsResponse.parse(responseBuffer, (short) 0); - assertEquals(Errors.UNSUPPORTED_VERSION.code(), response.data.errorCode()); + assertEquals(Errors.UNSUPPORTED_VERSION.code(), response.data().errorCode()); - ApiVersionsResponseKey apiVersion = response.data.apiKeys().find(ApiKeys.API_VERSIONS.id); + ApiVersionsResponseKey apiVersion = response.data().apiKeys().find(ApiKeys.API_VERSIONS.id); assertNotNull(apiVersion); assertEquals(ApiKeys.API_VERSIONS.id, apiVersion.apiKey()); assertEquals(ApiKeys.API_VERSIONS.oldestVersion(), apiVersion.minVersion()); @@ -770,7 +770,7 @@ public class SaslAuthenticatorTest { ResponseHeader.parse(responseBuffer, ApiKeys.API_VERSIONS.responseHeaderVersion(version)); ApiVersionsResponse response = ApiVersionsResponse.parse(responseBuffer, version); - assertEquals(Errors.INVALID_REQUEST.code(), response.data.errorCode()); + assertEquals(Errors.INVALID_REQUEST.code(), response.data().errorCode()); // Send ApiVersionsRequest with a supported version. This should succeed. sendVersionRequestReceiveResponse(node); @@ -811,7 +811,7 @@ public class SaslAuthenticatorTest { ByteBuffer responseBuffer = waitForResponse(); ResponseHeader.parse(responseBuffer, ApiKeys.API_VERSIONS.responseHeaderVersion(version)); ApiVersionsResponse response = ApiVersionsResponse.parse(responseBuffer, version); - assertEquals(Errors.NONE.code(), response.data.errorCode()); + assertEquals(Errors.NONE.code(), response.data().errorCode()); // Test that client can authenticate successfully sendHandshakeRequestReceiveResponse(node, handshakeVersion); @@ -1901,7 +1901,7 @@ public class SaslAuthenticatorTest { protected ApiVersionsResponse apiVersionsResponse() { ApiVersionsResponse defaultApiVersionResponse = ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE; ApiVersionsResponseKeyCollection apiVersions = new ApiVersionsResponseKeyCollection(); - for (ApiVersionsResponseKey apiVersion : defaultApiVersionResponse.data.apiKeys()) { + for (ApiVersionsResponseKey apiVersion : defaultApiVersionResponse.data().apiKeys()) { if (apiVersion.apiKey() != ApiKeys.SASL_AUTHENTICATE.id) { // ApiVersionsResponseKey can NOT be reused in second ApiVersionsResponseKeyCollection // due to the internal pointers it contains. @@ -2183,7 +2183,7 @@ public class SaslAuthenticatorTest { private ApiVersionsResponse sendVersionRequestReceiveResponse(String node) throws Exception { ApiVersionsRequest handshakeRequest = createApiVersionsRequestV0(); ApiVersionsResponse response = (ApiVersionsResponse) sendKafkaRequestReceiveResponse(node, ApiKeys.API_VERSIONS, handshakeRequest); - assertEquals(Errors.NONE.code(), response.data.errorCode()); + assertEquals(Errors.NONE.code(), response.data().errorCode()); return response; } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java index 8c36751320e..2ac05c09d37 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java @@ -255,8 +255,8 @@ public class WorkerCoordinatorTest { @Override public boolean matches(AbstractRequest body) { SyncGroupRequest sync = (SyncGroupRequest) body; - return sync.data.memberId().equals(consumerId) && - sync.data.generationId() == 1 && + return sync.data().memberId().equals(consumerId) && + sync.data().generationId() == 1 && sync.groupAssignments().containsKey(consumerId); } }, syncGroupResponse(ConnectProtocol.Assignment.NO_ERROR, "leader", 1L, Collections.singletonList(connectorId1), @@ -292,9 +292,9 @@ public class WorkerCoordinatorTest { @Override public boolean matches(AbstractRequest body) { SyncGroupRequest sync = (SyncGroupRequest) body; - return sync.data.memberId().equals(memberId) && - sync.data.generationId() == 1 && - sync.data.assignments().isEmpty(); + return sync.data().memberId().equals(memberId) && + sync.data().generationId() == 1 && + sync.data().assignments().isEmpty(); } }, syncGroupResponse(ConnectProtocol.Assignment.NO_ERROR, "leader", 1L, Collections.emptyList(), Collections.singletonList(taskId1x0), Errors.NONE)); @@ -333,9 +333,9 @@ public class WorkerCoordinatorTest { @Override public boolean matches(AbstractRequest body) { SyncGroupRequest sync = (SyncGroupRequest) body; - return sync.data.memberId().equals(memberId) && - sync.data.generationId() == 1 && - sync.data.assignments().isEmpty(); + return sync.data().memberId().equals(memberId) && + sync.data().generationId() == 1 && + sync.data().assignments().isEmpty(); } }; client.prepareResponse(matcher, syncGroupResponse(ConnectProtocol.Assignment.CONFIG_MISMATCH, "leader", 10L, diff --git a/core/src/main/scala/kafka/network/RequestChannel.scala b/core/src/main/scala/kafka/network/RequestChannel.scala index 39ca69f1f18..0c03d701881 100644 --- a/core/src/main/scala/kafka/network/RequestChannel.scala +++ b/core/src/main/scala/kafka/network/RequestChannel.scala @@ -20,6 +20,8 @@ package kafka.network import java.net.InetAddress import java.nio.ByteBuffer import java.util.concurrent._ + +import com.fasterxml.jackson.databind.JsonNode import com.typesafe.scalalogging.Logger import com.yammer.metrics.core.Meter import kafka.log.LogConfig @@ -97,6 +99,12 @@ object RequestChannel extends Logging { private val bodyAndSize: RequestAndSize = context.parseRequest(buffer) + // This is constructed on creation of a Request so that the JSON representation is computed before the request is + // processed by the api layer. Otherwise, a ProduceRequest can occur without its data (ie. it goes into purgatory). + val requestLog: Option[JsonNode] = + if (RequestChannel.isRequestLoggingEnabled) Some(RequestConvertToJson.request(loggableRequest)) + else None + def header: RequestHeader = context.header def sizeOfBodyInBytes: Int = bodyAndSize.size @@ -123,9 +131,9 @@ object RequestChannel extends Logging { } } - def responseString(response: AbstractResponse): Option[String] = { + def responseNode(response: AbstractResponse): Option[JsonNode] = { if (RequestChannel.isRequestLoggingEnabled) - Some(response.toString) + Some(RequestConvertToJson.response(response, context.apiVersion)) else None } @@ -261,30 +269,13 @@ object RequestChannel extends Logging { recordNetworkThreadTimeCallback.foreach(record => record(networkThreadTimeNanos)) if (isRequestLoggingEnabled) { - val detailsEnabled = requestLogger.underlying.isTraceEnabled - val responseString = response.responseString.getOrElse( - throw new IllegalStateException("responseAsString should always be defined if request logging is enabled")) - val builder = new StringBuilder(256) - builder.append("Completed request:").append(requestDesc(detailsEnabled)) - .append(",response:").append(responseString) - .append(" from connection ").append(context.connectionId) - .append(";totalTime:").append(totalTimeMs) - .append(",requestQueueTime:").append(requestQueueTimeMs) - .append(",localTime:").append(apiLocalTimeMs) - .append(",remoteTime:").append(apiRemoteTimeMs) - .append(",throttleTime:").append(apiThrottleTimeMs) - .append(",responseQueueTime:").append(responseQueueTimeMs) - .append(",sendTime:").append(responseSendTimeMs) - .append(",securityProtocol:").append(context.securityProtocol) - .append(",principal:").append(session.principal) - .append(",listener:").append(context.listenerName.value) - .append(",clientInformation:").append(context.clientInformation) - .append(",forwarded:").append(isForwarded) - if (temporaryMemoryBytes > 0) - builder.append(",temporaryMemoryBytes:").append(temporaryMemoryBytes) - if (messageConversionsTimeMs > 0) - builder.append(",messageConversionsTime:").append(messageConversionsTimeMs) - requestLogger.debug(builder.toString) + val desc = RequestConvertToJson.requestDescMetrics(header, requestLog, response.responseLog, + context, session, isForwarded, + totalTimeMs, requestQueueTimeMs, apiLocalTimeMs, + apiRemoteTimeMs, apiThrottleTimeMs, responseQueueTimeMs, + responseSendTimeMs, temporaryMemoryBytes, + messageConversionsTimeMs) + requestLogger.debug("Completed request:" + desc.toString) } } @@ -314,24 +305,24 @@ object RequestChannel extends Logging { def processor: Int = request.processor - def responseString: Option[String] = Some("") + def responseLog: Option[JsonNode] = None def onComplete: Option[Send => Unit] = None override def toString: String } - /** responseAsString should only be defined if request logging is enabled */ + /** responseLogValue should only be defined if request logging is enabled */ class SendResponse(request: Request, val responseSend: Send, - val responseAsString: Option[String], + val responseLogValue: Option[JsonNode], val onCompleteCallback: Option[Send => Unit]) extends Response(request) { - override def responseString: Option[String] = responseAsString + override def responseLog: Option[JsonNode] = responseLogValue override def onComplete: Option[Send => Unit] = onCompleteCallback override def toString: String = - s"Response(type=Send, request=$request, send=$responseSend, asString=$responseAsString)" + s"Response(type=Send, request=$request, send=$responseSend, asString=$responseLogValue)" } class NoOpResponse(request: Request) extends Response(request) { diff --git a/core/src/main/scala/kafka/network/RequestConvertToJson.scala b/core/src/main/scala/kafka/network/RequestConvertToJson.scala new file mode 100644 index 00000000000..243c2a4ca50 --- /dev/null +++ b/core/src/main/scala/kafka/network/RequestConvertToJson.scala @@ -0,0 +1,207 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.network + +import com.fasterxml.jackson.databind.JsonNode +import com.fasterxml.jackson.databind.node.{BooleanNode, DoubleNode, JsonNodeFactory, LongNode, ObjectNode, TextNode} +import kafka.network.RequestChannel.Session +import org.apache.kafka.common.message._ +import org.apache.kafka.common.network.ClientInformation +import org.apache.kafka.common.requests._ + +object RequestConvertToJson { + def request(request: AbstractRequest): JsonNode = { + request match { + case req: AddOffsetsToTxnRequest => AddOffsetsToTxnRequestDataJsonConverter.write(req.data, request.version) + case req: AddPartitionsToTxnRequest => AddPartitionsToTxnRequestDataJsonConverter.write(req.data, request.version) + case req: AlterClientQuotasRequest => AlterClientQuotasRequestDataJsonConverter.write(req.data, request.version) + case req: AlterConfigsRequest => AlterConfigsRequestDataJsonConverter.write(req.data, request.version) + case req: AlterIsrRequest => AlterIsrRequestDataJsonConverter.write(req.data, request.version) + case req: AlterPartitionReassignmentsRequest => AlterPartitionReassignmentsRequestDataJsonConverter.write(req.data, request.version) + case req: AlterReplicaLogDirsRequest => AlterReplicaLogDirsRequestDataJsonConverter.write(req.data, request.version) + case res: AlterUserScramCredentialsRequest => AlterUserScramCredentialsRequestDataJsonConverter.write(res.data, request.version) + case req: ApiVersionsRequest => ApiVersionsRequestDataJsonConverter.write(req.data, request.version) + case req: BeginQuorumEpochRequest => BeginQuorumEpochRequestDataJsonConverter.write(req.data, request.version) + case req: ControlledShutdownRequest => ControlledShutdownRequestDataJsonConverter.write(req.data, request.version) + case req: CreateAclsRequest => CreateAclsRequestDataJsonConverter.write(req.data, request.version) + case req: CreateDelegationTokenRequest => CreateDelegationTokenRequestDataJsonConverter.write(req.data, request.version) + case req: CreatePartitionsRequest => CreatePartitionsRequestDataJsonConverter.write(req.data, request.version) + case req: CreateTopicsRequest => CreateTopicsRequestDataJsonConverter.write(req.data, request.version) + case req: DeleteAclsRequest => DeleteAclsRequestDataJsonConverter.write(req.data, request.version) + case req: DeleteGroupsRequest => DeleteGroupsRequestDataJsonConverter.write(req.data, request.version) + case req: DeleteRecordsRequest => DeleteRecordsRequestDataJsonConverter.write(req.data, request.version) + case req: DeleteTopicsRequest => DeleteTopicsRequestDataJsonConverter.write(req.data, request.version) + case req: DescribeAclsRequest => DescribeAclsRequestDataJsonConverter.write(req.data, request.version) + case req: DescribeClientQuotasRequest => DescribeClientQuotasRequestDataJsonConverter.write(req.data, request.version) + case req: DescribeConfigsRequest => DescribeConfigsRequestDataJsonConverter.write(req.data, request.version) + case req: DescribeDelegationTokenRequest => DescribeDelegationTokenRequestDataJsonConverter.write(req.data, request.version) + case req: DescribeGroupsRequest => DescribeGroupsRequestDataJsonConverter.write(req.data, request.version) + case req: DescribeLogDirsRequest => DescribeLogDirsRequestDataJsonConverter.write(req.data, request.version) + case req: DescribeQuorumRequest => DescribeQuorumRequestDataJsonConverter.write(req.data, request.version) + case res: DescribeUserScramCredentialsRequest => DescribeUserScramCredentialsRequestDataJsonConverter.write(res.data, request.version) + case req: ElectLeadersRequest => ElectLeadersRequestDataJsonConverter.write(req.data, request.version) + case req: EndTxnRequest => EndTxnRequestDataJsonConverter.write(req.data, request.version) + case req: EndQuorumEpochRequest => EndQuorumEpochRequestDataJsonConverter.write(req.data, request.version) + case req: EnvelopeRequest => EnvelopeRequestDataJsonConverter.write(req.data, request.version) + case req: ExpireDelegationTokenRequest => ExpireDelegationTokenRequestDataJsonConverter.write(req.data, request.version) + case req: FetchRequest => FetchRequestDataJsonConverter.write(req.data, request.version) + case req: FindCoordinatorRequest => FindCoordinatorRequestDataJsonConverter.write(req.data, request.version) + case req: HeartbeatRequest => HeartbeatRequestDataJsonConverter.write(req.data, request.version) + case req: IncrementalAlterConfigsRequest => IncrementalAlterConfigsRequestDataJsonConverter.write(req.data, request.version) + case req: InitProducerIdRequest => InitProducerIdRequestDataJsonConverter.write(req.data, request.version) + case req: JoinGroupRequest => JoinGroupRequestDataJsonConverter.write(req.data, request.version) + case req: LeaderAndIsrRequest => LeaderAndIsrRequestDataJsonConverter.write(req.data, request.version) + case req: LeaveGroupRequest => LeaveGroupRequestDataJsonConverter.write(req.data, request.version) + case req: ListGroupsRequest => ListGroupsRequestDataJsonConverter.write(req.data, request.version) + case req: ListOffsetRequest => ListOffsetRequestDataJsonConverter.write(req.data, request.version) + case req: ListPartitionReassignmentsRequest => ListPartitionReassignmentsRequestDataJsonConverter.write(req.data, request.version) + case req: MetadataRequest => MetadataRequestDataJsonConverter.write(req.data, request.version) + case req: OffsetCommitRequest => OffsetCommitRequestDataJsonConverter.write(req.data, request.version) + case req: OffsetDeleteRequest => OffsetDeleteRequestDataJsonConverter.write(req.data, request.version) + case req: OffsetFetchRequest => OffsetFetchRequestDataJsonConverter.write(req.data, request.version) + case req: OffsetsForLeaderEpochRequest => OffsetForLeaderEpochRequestDataJsonConverter.write(req.data, request.version) + case req: ProduceRequest => ProduceRequestDataJsonConverter.write(req.data, request.version, false) + case req: RenewDelegationTokenRequest => RenewDelegationTokenRequestDataJsonConverter.write(req.data, request.version) + case req: SaslAuthenticateRequest => SaslAuthenticateRequestDataJsonConverter.write(req.data, request.version) + case req: SaslHandshakeRequest => SaslHandshakeRequestDataJsonConverter.write(req.data, request.version) + case req: StopReplicaRequest => StopReplicaRequestDataJsonConverter.write(req.data, request.version) + case req: SyncGroupRequest => SyncGroupRequestDataJsonConverter.write(req.data, request.version) + case req: TxnOffsetCommitRequest => TxnOffsetCommitRequestDataJsonConverter.write(req.data, request.version) + case req: UpdateFeaturesRequest => UpdateFeaturesRequestDataJsonConverter.write(req.data, request.version) + case req: UpdateMetadataRequest => UpdateMetadataRequestDataJsonConverter.write(req.data, request.version) + case req: VoteRequest => VoteRequestDataJsonConverter.write(req.data, request.version) + case req: WriteTxnMarkersRequest => WriteTxnMarkersRequestDataJsonConverter.write(req.data, request.version) + case _ => throw new IllegalStateException(s"ApiKey ${request.apiKey} is not currently handled in `request`, the " + + "code should be updated to do so."); + } + } + + def response(response: AbstractResponse, version: Short): JsonNode = { + response match { + case res: AddOffsetsToTxnResponse => AddOffsetsToTxnResponseDataJsonConverter.write(res.data, version) + case res: AddPartitionsToTxnResponse => AddPartitionsToTxnResponseDataJsonConverter.write(res.data, version) + case res: AlterClientQuotasResponse => AlterClientQuotasResponseDataJsonConverter.write(res.data, version) + case res: AlterConfigsResponse => AlterConfigsResponseDataJsonConverter.write(res.data, version) + case res: AlterIsrResponse => AlterIsrResponseDataJsonConverter.write(res.data, version) + case res: AlterPartitionReassignmentsResponse => AlterPartitionReassignmentsResponseDataJsonConverter.write(res.data, version) + case res: AlterReplicaLogDirsResponse => AlterReplicaLogDirsResponseDataJsonConverter.write(res.data, version) + case res: AlterUserScramCredentialsResponse => AlterUserScramCredentialsResponseDataJsonConverter.write(res.data, version) + case res: ApiVersionsResponse => ApiVersionsResponseDataJsonConverter.write(res.data, version) + case res: BeginQuorumEpochResponse => BeginQuorumEpochResponseDataJsonConverter.write(res.data, version) + case res: ControlledShutdownResponse => ControlledShutdownResponseDataJsonConverter.write(res.data, version) + case res: CreateAclsResponse => CreateAclsResponseDataJsonConverter.write(res.data, version) + case res: CreateDelegationTokenResponse => CreateDelegationTokenResponseDataJsonConverter.write(res.data, version) + case res: CreatePartitionsResponse => CreatePartitionsResponseDataJsonConverter.write(res.data, version) + case res: CreateTopicsResponse => CreateTopicsResponseDataJsonConverter.write(res.data, version) + case res: DeleteAclsResponse => DeleteAclsResponseDataJsonConverter.write(res.data, version) + case res: DeleteGroupsResponse => DeleteGroupsResponseDataJsonConverter.write(res.data, version) + case res: DeleteRecordsResponse => DeleteRecordsResponseDataJsonConverter.write(res.data, version) + case res: DeleteTopicsResponse => DeleteTopicsResponseDataJsonConverter.write(res.data, version) + case res: DescribeAclsResponse => DescribeAclsResponseDataJsonConverter.write(res.data, version) + case res: DescribeClientQuotasResponse => DescribeClientQuotasResponseDataJsonConverter.write(res.data, version) + case res: DescribeConfigsResponse => DescribeConfigsResponseDataJsonConverter.write(res.data, version) + case res: DescribeDelegationTokenResponse => DescribeDelegationTokenResponseDataJsonConverter.write(res.data, version) + case res: DescribeGroupsResponse => DescribeGroupsResponseDataJsonConverter.write(res.data, version) + case res: DescribeLogDirsResponse => DescribeLogDirsResponseDataJsonConverter.write(res.data, version) + case res: DescribeQuorumResponse => DescribeQuorumResponseDataJsonConverter.write(res.data, version) + case res: DescribeUserScramCredentialsResponse => DescribeUserScramCredentialsResponseDataJsonConverter.write(res.data, version) + case res: ElectLeadersResponse => ElectLeadersResponseDataJsonConverter.write(res.data, version) + case res: EndTxnResponse => EndTxnResponseDataJsonConverter.write(res.data, version) + case res: EndQuorumEpochResponse => EndQuorumEpochResponseDataJsonConverter.write(res.data, version) + case res: EnvelopeResponse => EnvelopeResponseDataJsonConverter.write(res.data, version) + case res: ExpireDelegationTokenResponse => ExpireDelegationTokenResponseDataJsonConverter.write(res.data, version) + case res: FetchResponse[_] => FetchResponseDataJsonConverter.write(res.data, version, false) + case res: FindCoordinatorResponse => FindCoordinatorResponseDataJsonConverter.write(res.data, version) + case res: HeartbeatResponse => HeartbeatResponseDataJsonConverter.write(res.data, version) + case res: IncrementalAlterConfigsResponse => IncrementalAlterConfigsResponseDataJsonConverter.write(res.data, version) + case res: InitProducerIdResponse => InitProducerIdResponseDataJsonConverter.write(res.data, version) + case res: JoinGroupResponse => JoinGroupResponseDataJsonConverter.write(res.data, version) + case res: LeaderAndIsrResponse => LeaderAndIsrResponseDataJsonConverter.write(res.data, version) + case res: LeaveGroupResponse => LeaveGroupResponseDataJsonConverter.write(res.data, version) + case res: ListGroupsResponse => ListGroupsResponseDataJsonConverter.write(res.data, version) + case res: ListOffsetResponse => ListOffsetResponseDataJsonConverter.write(res.data, version) + case res: ListPartitionReassignmentsResponse => ListPartitionReassignmentsResponseDataJsonConverter.write(res.data, version) + case res: MetadataResponse => MetadataResponseDataJsonConverter.write(res.data, version) + case res: OffsetCommitResponse => OffsetCommitResponseDataJsonConverter.write(res.data, version) + case res: OffsetDeleteResponse => OffsetDeleteResponseDataJsonConverter.write(res.data, version) + case res: OffsetFetchResponse => OffsetFetchResponseDataJsonConverter.write(res.data, version) + case res: OffsetsForLeaderEpochResponse => OffsetForLeaderEpochResponseDataJsonConverter.write(res.data, version) + case res: ProduceResponse => ProduceResponseDataJsonConverter.write(res.data, version) + case res: RenewDelegationTokenResponse => RenewDelegationTokenResponseDataJsonConverter.write(res.data, version) + case res: SaslAuthenticateResponse => SaslAuthenticateResponseDataJsonConverter.write(res.data, version) + case res: SaslHandshakeResponse => SaslHandshakeResponseDataJsonConverter.write(res.data, version) + case res: StopReplicaResponse => StopReplicaResponseDataJsonConverter.write(res.data, version) + case res: SyncGroupResponse => SyncGroupResponseDataJsonConverter.write(res.data, version) + case res: TxnOffsetCommitResponse => TxnOffsetCommitResponseDataJsonConverter.write(res.data, version) + case res: UpdateFeaturesResponse => UpdateFeaturesResponseDataJsonConverter.write(res.data, version) + case res: UpdateMetadataResponse => UpdateMetadataResponseDataJsonConverter.write(res.data, version) + case res: WriteTxnMarkersResponse => WriteTxnMarkersResponseDataJsonConverter.write(res.data, version) + case res: VoteResponse => VoteResponseDataJsonConverter.write(res.data, version) + case _ => throw new IllegalStateException(s"ApiKey ${response.apiKey} is not currently handled in `response`, the " + + "code should be updated to do so."); + } + } + + def requestHeaderNode(header: RequestHeader): JsonNode = { + val node = RequestHeaderDataJsonConverter.write(header.data, header.headerVersion, false).asInstanceOf[ObjectNode] + node.set("requestApiKeyName", new TextNode(header.apiKey.toString)) + node + } + + def requestDesc(header: RequestHeader, requestNode: Option[JsonNode], isForwarded: Boolean): JsonNode = { + val node = new ObjectNode(JsonNodeFactory.instance) + node.set("isForwarded", if (isForwarded) BooleanNode.TRUE else BooleanNode.FALSE) + node.set("requestHeader", requestHeaderNode(header)) + node.set("request", requestNode.getOrElse(new TextNode(""))) + node + } + + def clientInfoNode(clientInfo: ClientInformation): JsonNode = { + val node = new ObjectNode(JsonNodeFactory.instance) + node.set("softwareName", new TextNode(clientInfo.softwareName)) + node.set("softwareVersion", new TextNode(clientInfo.softwareVersion)) + node + } + + def requestDescMetrics(header: RequestHeader, requestNode: Option[JsonNode], responseNode: Option[JsonNode], + context: RequestContext, session: Session, isForwarded: Boolean, + totalTimeMs: Double, requestQueueTimeMs: Double, apiLocalTimeMs: Double, + apiRemoteTimeMs: Double, apiThrottleTimeMs: Long, responseQueueTimeMs: Double, + responseSendTimeMs: Double, temporaryMemoryBytes: Long, + messageConversionsTimeMs: Double): JsonNode = { + val node = requestDesc(header, requestNode, isForwarded).asInstanceOf[ObjectNode] + node.set("response", responseNode.getOrElse(new TextNode(""))) + node.set("connection", new TextNode(context.connectionId)) + node.set("totalTimeMs", new DoubleNode(totalTimeMs)) + node.set("requestQueueTimeMs", new DoubleNode(requestQueueTimeMs)) + node.set("localTimeMs", new DoubleNode(apiLocalTimeMs)) + node.set("remoteTimeMs", new DoubleNode(apiRemoteTimeMs)) + node.set("throttleTimeMs", new LongNode(apiThrottleTimeMs)) + node.set("responseQueueTimeMs", new DoubleNode(responseQueueTimeMs)) + node.set("sendTimeMs", new DoubleNode(responseSendTimeMs)) + node.set("securityProtocol", new TextNode(context.securityProtocol.toString)) + node.set("principal", new TextNode(session.principal.toString)) + node.set("listener", new TextNode(context.listenerName.value)) + node.set("clientInformation", clientInfoNode(context.clientInformation)) + if (temporaryMemoryBytes > 0) + node.set("temporaryMemoryBytes", new LongNode(temporaryMemoryBytes)) + if (messageConversionsTimeMs > 0) + node.set("messageConversionsTime", new DoubleNode(messageConversionsTimeMs)) + node + } +} diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 2e42368b746..3e01fb5f558 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -62,9 +62,9 @@ import org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionR import org.apache.kafka.common.message.LeaveGroupResponseData.MemberResponse import org.apache.kafka.common.message.ListOffsetRequestData.ListOffsetPartition import org.apache.kafka.common.message.ListOffsetResponseData.{ListOffsetPartitionResponse, ListOffsetTopicResponse} -import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.{OffsetForLeaderTopic} +import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderTopic import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData -import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.{OffsetForLeaderTopicResult, OffsetForLeaderTopicResultCollection, EpochEndOffset} +import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.{EpochEndOffset, OffsetForLeaderTopicResult, OffsetForLeaderTopicResultCollection} import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network.{ListenerName, Send} import org.apache.kafka.common.protocol.{ApiKeys, Errors} @@ -3508,7 +3508,7 @@ class KafkaApis(val requestChannel: RequestChannel, new RequestChannel.SendResponse( request, request.buildResponseSend(response), - request.responseString(response), + request.responseNode(response), onComplete ) case None => diff --git a/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala b/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala index fefd274927d..4fad0db4f1e 100644 --- a/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala +++ b/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala @@ -18,6 +18,7 @@ package kafka.tools import kafka.network.RequestChannel +import kafka.network.RequestConvertToJson import kafka.raft.KafkaNetworkChannel import kafka.server.ApiRequestHandler import kafka.utils.Logging @@ -94,7 +95,7 @@ class TestRaftRequestHandler( case Some(response) => val responseSend = request.context.buildResponseSend(response) val responseString = - if (RequestChannel.isRequestLoggingEnabled) Some(response.toString) + if (RequestChannel.isRequestLoggingEnabled) Some(RequestConvertToJson.response(response, request.context.apiVersion)) else None new RequestChannel.SendResponse(request, responseSend, responseString, None) case None => diff --git a/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala b/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala index 8039e8333eb..b7b950794fe 100644 --- a/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala +++ b/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala @@ -18,10 +18,12 @@ package kafka.network +import java.io.IOException import java.net.InetAddress import java.nio.ByteBuffer import java.util.Collections +import com.fasterxml.jackson.databind.ObjectMapper import kafka.network import org.apache.kafka.clients.admin.AlterConfigOp.OpType import org.apache.kafka.common.config.types.Password @@ -52,7 +54,7 @@ class RequestChannelTest { val loggableAlterConfigs = alterConfigs.loggableRequest.asInstanceOf[AlterConfigsRequest] val loggedConfig = loggableAlterConfigs.configs.get(resource) assertEquals(expectedValues, toMap(loggedConfig)) - val alterConfigsDesc = alterConfigs.requestDesc(details = true) + val alterConfigsDesc = RequestConvertToJson.requestDesc(alterConfigs.header, alterConfigs.requestLog, alterConfigs.isForwarded).toString assertFalse(s"Sensitive config logged $alterConfigsDesc", alterConfigsDesc.contains(sensitiveValue)) } @@ -116,7 +118,7 @@ class RequestChannelTest { val loggableAlterConfigs = alterConfigs.loggableRequest.asInstanceOf[IncrementalAlterConfigsRequest] val loggedConfig = loggableAlterConfigs.data.resources.find(resource.`type`.id, resource.name).configs assertEquals(expectedValues, toMap(loggedConfig)) - val alterConfigsDesc = alterConfigs.requestDesc(details = true) + val alterConfigsDesc = RequestConvertToJson.requestDesc(alterConfigs.header, alterConfigs.requestLog, alterConfigs.isForwarded).toString assertFalse(s"Sensitive config logged $alterConfigsDesc", alterConfigsDesc.contains(sensitiveValue)) } @@ -164,6 +166,29 @@ class RequestChannelTest { assertSame(metadataRequest.body[MetadataRequest], metadataRequest.loggableRequest) } + @Test + def testJsonRequests(): Unit = { + val sensitiveValue = "secret" + val resource = new ConfigResource(ConfigResource.Type.BROKER, "1") + val keystorePassword = new ConfigEntry(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, sensitiveValue) + val entries = Seq(keystorePassword) + + val alterConfigs = request(new AlterConfigsRequest.Builder(Collections.singletonMap(resource, + new Config(entries.asJavaCollection)), true).build()) + + assertTrue(isValidJson(RequestConvertToJson.request(alterConfigs.loggableRequest).toString)) + } + + private def isValidJson(str: String): Boolean = { + try { + val mapper = new ObjectMapper + mapper.readTree(str) + true + } catch { + case _: IOException => false + } + } + def request(req: AbstractRequest): RequestChannel.Request = { val buffer = RequestTestUtils.serializeRequestWithHeader(new RequestHeader(req.apiKey, req.version, "client-id", 1), req) diff --git a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala new file mode 100644 index 00000000000..5dd7e31496b --- /dev/null +++ b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala @@ -0,0 +1,197 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.network + +import java.net.InetAddress +import java.nio.ByteBuffer + +import com.fasterxml.jackson.databind.node.{BooleanNode, DoubleNode, JsonNodeFactory, LongNode, ObjectNode, TextNode} +import kafka.network +import kafka.network.RequestConvertToJson.requestHeaderNode +import org.apache.kafka.common.memory.MemoryPool +import org.apache.kafka.common.message._ +import org.apache.kafka.common.network.{ClientInformation, ListenerName, NetworkSend} +import org.junit.Test +import org.apache.kafka.common.protocol.{ApiKeys, ByteBufferAccessor, ObjectSerializationCache} +import org.apache.kafka.common.requests._ +import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} +import org.easymock.EasyMock.createNiceMock +import org.junit.Assert.assertEquals + +import scala.collection.mutable.ArrayBuffer + +class RequestConvertToJsonTest { + + @Test + def testAllRequestTypesHandled(): Unit = { + val unhandledKeys = ArrayBuffer[String]() + ApiKeys.values().foreach { key => { + val version: Short = key.latestVersion() + val cache = new ObjectSerializationCache + val message = key match { + case ApiKeys.DESCRIBE_ACLS => + ApiMessageType.fromApiKey(key.id).newRequest().asInstanceOf[DescribeAclsRequestData] + .setPatternTypeFilter(1).setResourceTypeFilter(1).setPermissionType(1).setOperation(1) + case _ => + ApiMessageType.fromApiKey(key.id).newRequest() + } + val messageSize = message.size(cache, version) + val bytes = new ByteBufferAccessor(ByteBuffer.allocate(messageSize)) + message.write(bytes, cache, version) + bytes.flip() + + val req = AbstractRequest.parseRequest(key, version, bytes.buffer).request + try { + RequestConvertToJson.request(req) + } catch { + case _ : IllegalStateException => unhandledKeys += key.toString + } + }} + assertEquals("Unhandled request keys", ArrayBuffer.empty, unhandledKeys) + } + + @Test + def testAllResponseTypesHandled(): Unit = { + val unhandledKeys = ArrayBuffer[String]() + ApiKeys.values().foreach { key => { + val version: Short = key.latestVersion() + val cache = new ObjectSerializationCache + val message = ApiMessageType.fromApiKey(key.id).newResponse() + val messageSize = message.size(cache, version) + val bytes = new ByteBufferAccessor(ByteBuffer.allocate(messageSize)) + message.write(bytes, cache, version) + bytes.flip() + val res = AbstractResponse.parseResponse(key, bytes.buffer, version) + try { + RequestConvertToJson.response(res, version) + } catch { + case _ : IllegalStateException => unhandledKeys += key.toString + } + }} + assertEquals("Unhandled response keys", ArrayBuffer.empty, unhandledKeys) + } + + @Test + def testRequestHeaderNode(): Unit = { + val alterIsrRequest = new AlterIsrRequest(new AlterIsrRequestData(), 0) + val req = request(alterIsrRequest) + val header = req.header + + val expectedNode = RequestHeaderDataJsonConverter.write(header.data, header.headerVersion, false).asInstanceOf[ObjectNode] + expectedNode.set("requestApiKeyName", new TextNode(header.apiKey.toString)) + + val actualNode = RequestConvertToJson.requestHeaderNode(header) + + assertEquals(expectedNode, actualNode); + } + + @Test + def testClientInfoNode(): Unit = { + val clientInfo = new ClientInformation("name", "1") + + val expectedNode = new ObjectNode(JsonNodeFactory.instance) + expectedNode.set("softwareName", new TextNode(clientInfo.softwareName)) + expectedNode.set("softwareVersion", new TextNode(clientInfo.softwareVersion)) + + val actualNode = RequestConvertToJson.clientInfoNode(clientInfo) + + assertEquals(expectedNode, actualNode) + } + + @Test + def testRequestDesc(): Unit = { + val alterIsrRequest = new AlterIsrRequest(new AlterIsrRequestData(), 0) + val req = request(alterIsrRequest) + + val expectedNode = new ObjectNode(JsonNodeFactory.instance) + expectedNode.set("isForwarded", if (req.isForwarded) BooleanNode.TRUE else BooleanNode.FALSE) + expectedNode.set("requestHeader", requestHeaderNode(req.header)) + expectedNode.set("request", req.requestLog.getOrElse(new TextNode(""))) + + val actualNode = RequestConvertToJson.requestDesc(req.header, req.requestLog, req.isForwarded) + + assertEquals(expectedNode, actualNode) + } + + @Test + def testRequestDescMetrics(): Unit = { + val alterIsrRequest = new AlterIsrRequest(new AlterIsrRequestData(), 0) + val req = request(alterIsrRequest) + val send = new NetworkSend(req.context.connectionId, alterIsrRequest.toSend(req.header)) + val headerLog = RequestConvertToJson.requestHeaderNode(req.header) + val res = new RequestChannel.SendResponse(req, send, Some(headerLog), None) + + val totalTimeMs = 1 + val requestQueueTimeMs = 2 + val apiLocalTimeMs = 3 + val apiRemoteTimeMs = 4 + val apiThrottleTimeMs = 5 + val responseQueueTimeMs = 6 + val responseSendTimeMs = 7 + val temporaryMemoryBytes = 8 + val messageConversionsTimeMs = 9 + + val expectedNode = RequestConvertToJson.requestDesc(req.header, req.requestLog, req.isForwarded).asInstanceOf[ObjectNode] + expectedNode.set("response", res.responseLog.getOrElse(new TextNode(""))) + expectedNode.set("connection", new TextNode(req.context.connectionId)) + expectedNode.set("totalTimeMs", new DoubleNode(totalTimeMs)) + expectedNode.set("requestQueueTimeMs", new DoubleNode(requestQueueTimeMs)) + expectedNode.set("localTimeMs", new DoubleNode(apiLocalTimeMs)) + expectedNode.set("remoteTimeMs", new DoubleNode(apiRemoteTimeMs)) + expectedNode.set("throttleTimeMs", new LongNode(apiThrottleTimeMs)) + expectedNode.set("responseQueueTimeMs", new DoubleNode(responseQueueTimeMs)) + expectedNode.set("sendTimeMs", new DoubleNode(responseSendTimeMs)) + expectedNode.set("securityProtocol", new TextNode(req.context.securityProtocol.toString)) + expectedNode.set("principal", new TextNode(req.session.principal.toString)) + expectedNode.set("listener", new TextNode(req.context.listenerName.value)) + expectedNode.set("clientInformation", RequestConvertToJson.clientInfoNode(req.context.clientInformation)) + expectedNode.set("temporaryMemoryBytes", new LongNode(temporaryMemoryBytes)) + expectedNode.set("messageConversionsTime", new DoubleNode(messageConversionsTimeMs)) + + val actualNode = RequestConvertToJson.requestDescMetrics(req.header, req.requestLog, res.responseLog, req.context, req.session, req.isForwarded, + totalTimeMs, requestQueueTimeMs, apiLocalTimeMs, apiRemoteTimeMs, apiThrottleTimeMs, responseQueueTimeMs, + responseSendTimeMs, temporaryMemoryBytes, messageConversionsTimeMs).asInstanceOf[ObjectNode] + + assertEquals(expectedNode, actualNode) + } + + def request(req: AbstractRequest): RequestChannel.Request = { + val buffer = RequestTestUtils.serializeRequestWithHeader(new RequestHeader(req.apiKey, req.version, "client-id", 1), + req) + val requestContext = newRequestContext(buffer) + new network.RequestChannel.Request(processor = 1, + requestContext, + startTimeNanos = 0, + createNiceMock(classOf[MemoryPool]), + buffer, + createNiceMock(classOf[RequestChannel.Metrics]) + ) + } + + private def newRequestContext(buffer: ByteBuffer): RequestContext = { + new RequestContext( + RequestHeader.parse(buffer), + "connection-id", + InetAddress.getLoopbackAddress, + new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "user"), + ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT), + SecurityProtocol.PLAINTEXT, + new ClientInformation("name", "version"), + false) + } +} diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index c3181e5bf28..e3c683ce177 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -25,8 +25,9 @@ import java.nio.charset.StandardCharsets import java.util import java.util.concurrent.{CompletableFuture, ConcurrentLinkedQueue, Executors, TimeUnit} import java.util.{Properties, Random} -import com.yammer.metrics.core.{Gauge, Meter} +import com.fasterxml.jackson.databind.node.{JsonNodeFactory, ObjectNode, TextNode} +import com.yammer.metrics.core.{Gauge, Meter} import javax.net.ssl._ import kafka.metrics.KafkaYammerMetrics import kafka.security.CredentialProvider @@ -140,7 +141,8 @@ class SocketServerTest { def processRequest(channel: RequestChannel, request: RequestChannel.Request): Unit = { val byteBuffer = RequestTestUtils.serializeRequestWithHeader(request.header, request.body[AbstractRequest]) val send = new NetworkSend(request.context.connectionId, ByteBufferSend.sizePrefixed(byteBuffer)) - channel.sendResponse(new RequestChannel.SendResponse(request, send, Some(request.header.toString), None)) + val headerLog = RequestConvertToJson.requestHeaderNode(request.header) + channel.sendResponse(new RequestChannel.SendResponse(request, send, Some(headerLog), None)) } def processRequestNoOpResponse(channel: RequestChannel, request: RequestChannel.Request): Unit = { @@ -676,9 +678,10 @@ class SocketServerTest { server.dataPlaneRequestChannel.sendResponse(response) } val throttledChannel = new ThrottledChannel(request, new MockTime(), 100, channelThrottlingCallback) + val headerLog = RequestConvertToJson.requestHeaderNode(request.header) val response = if (!noOpResponse) - new RequestChannel.SendResponse(request, send, Some(request.header.toString), None) + new RequestChannel.SendResponse(request, send, Some(headerLog), None) else new RequestChannel.NoOpResponse(request) server.dataPlaneRequestChannel.sendResponse(response) @@ -1093,7 +1096,9 @@ class SocketServerTest { val requestMetrics = channel.metrics(request.header.apiKey.name) def totalTimeHistCount(): Long = requestMetrics.totalTimeHist.count val send = new NetworkSend(request.context.connectionId, ByteBufferSend.sizePrefixed(ByteBuffer.allocate(responseBufferSize))) - channel.sendResponse(new RequestChannel.SendResponse(request, send, Some("someResponse"), None)) + val headerLog = new ObjectNode(JsonNodeFactory.instance) + headerLog.set("response", new TextNode("someResponse")) + channel.sendResponse(new RequestChannel.SendResponse(request, send, Some(headerLog), None)) val expectedTotalTimeCount = totalTimeHistCount() + 1 TestUtils.waitUntilTrue(() => totalTimeHistCount() == expectedTotalTimeCount, diff --git a/generator/src/main/java/org/apache/kafka/message/JsonConverterGenerator.java b/generator/src/main/java/org/apache/kafka/message/JsonConverterGenerator.java index 94a79cb403d..56809f44661 100644 --- a/generator/src/main/java/org/apache/kafka/message/JsonConverterGenerator.java +++ b/generator/src/main/java/org/apache/kafka/message/JsonConverterGenerator.java @@ -77,6 +77,7 @@ public final class JsonConverterGenerator implements MessageClassGenerator { Versions parentVersions) { generateRead(name, spec, parentVersions); generateWrite(name, spec, parentVersions); + generateOverloadWrite(name); } private void generateRead(String className, @@ -257,11 +258,20 @@ public final class JsonConverterGenerator implements MessageClassGenerator { } } + private void generateOverloadWrite(String className) { + buffer.printf("public static JsonNode write(%s _object, short _version) {%n", + className); + buffer.incrementIndent(); + buffer.printf("return write(_object, _version, true);%n"); + buffer.decrementIndent(); + buffer.printf("}%n"); + } + private void generateWrite(String className, StructSpec struct, Versions parentVersions) { headerGenerator.addImport(MessageGenerator.JSON_NODE_CLASS); - buffer.printf("public static JsonNode write(%s _object, short _version) {%n", + buffer.printf("public static JsonNode write(%s _object, short _version, boolean _serializeRecords) {%n", className); buffer.incrementIndent(); VersionConditional.forVersions(struct.versions(), parentVersions). @@ -377,11 +387,25 @@ public final class JsonConverterGenerator implements MessageClassGenerator { headerGenerator.addImport(MessageGenerator.ARRAYS_CLASS); buffer.printf("%s;%n", target.assignmentStatement( String.format("new BinaryNode(Arrays.copyOf(%s, %s.length))", - target.sourceVariable(), target.sourceVariable()))); + target.sourceVariable(), target.sourceVariable()))); } } else if (target.field().type().isRecords()) { headerGenerator.addImport(MessageGenerator.BINARY_NODE_CLASS); + headerGenerator.addImport(MessageGenerator.INT_NODE_CLASS); + // KIP-673: When logging requests/responses, we do not serialize the record, instead we + // output its sizeInBytes, because outputting the bytes is not very useful and can be + // quite expensive. Otherwise, we will serialize the record. + buffer.printf("if (_serializeRecords) {%n"); + buffer.incrementIndent(); buffer.printf("%s;%n", target.assignmentStatement("new BinaryNode(new byte[]{})")); + buffer.decrementIndent(); + buffer.printf("} else {%n"); + buffer.incrementIndent(); + buffer.printf("_node.set(\"%sSizeInBytes\", new IntNode(%s.sizeInBytes()));%n", + target.field().camelCaseName(), + target.sourceVariable()); + buffer.decrementIndent(); + buffer.printf("}%n"); } else if (target.field().type().isArray()) { headerGenerator.addImport(MessageGenerator.ARRAY_NODE_CLASS); headerGenerator.addImport(MessageGenerator.JSON_NODE_FACTORY_CLASS); @@ -402,7 +426,7 @@ public final class JsonConverterGenerator implements MessageClassGenerator { buffer.printf("%s;%n", target.assignmentStatement(arrayInstanceName)); } else if (target.field().type().isStruct()) { buffer.printf("%s;%n", target.assignmentStatement( - String.format("%sJsonConverter.write(%s, _version)", + String.format("%sJsonConverter.write(%s, _version, _serializeRecords)", target.field().type().toString(), target.sourceVariable()))); } else { throw new RuntimeException("unknown type " + target.field().type()); diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/FetchRequestBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/FetchRequestBenchmark.java index b24ebf50df7..b9b2448b94c 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/FetchRequestBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/FetchRequestBenchmark.java @@ -17,6 +17,7 @@ package org.apache.kafka.jmh.common; +import kafka.network.RequestConvertToJson; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.network.Send; import org.apache.kafka.common.protocol.ApiKeys; @@ -125,4 +126,9 @@ public class FetchRequestBenchmark { send.writeTo(channel); return channel.buffer().limit(); } + + @Benchmark + public String testRequestToJson() { + return RequestConvertToJson.request(consumerRequest).toString(); + } } diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/ListOffsetRequestBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/ListOffsetRequestBenchmark.java new file mode 100644 index 00000000000..83510046e3c --- /dev/null +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/ListOffsetRequestBenchmark.java @@ -0,0 +1,80 @@ +/* + * 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.common; + +import kafka.network.RequestConvertToJson; +import org.apache.kafka.common.IsolationLevel; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.message.ListOffsetRequestData; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.requests.ListOffsetRequest; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Warmup; + +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.TimeUnit; + +@State(Scope.Benchmark) +@Fork(value = 1) +@Warmup(iterations = 5) +@Measurement(iterations = 15) +@BenchmarkMode(Mode.AverageTime) +@OutputTimeUnit(TimeUnit.NANOSECONDS) +public class ListOffsetRequestBenchmark { + @Param({"10", "500", "1000"}) + private int topicCount; + + @Param({"3", "10", "20"}) + private int partitionCount; + + Map offsetData; + + ListOffsetRequest offsetRequest; + + @Setup(Level.Trial) + public void setup() { + this.offsetData = new HashMap<>(); + for (int topicIdx = 0; topicIdx < topicCount; topicIdx++) { + String topic = UUID.randomUUID().toString(); + for (int partitionId = 0; partitionId < partitionCount; partitionId++) { + ListOffsetRequestData.ListOffsetPartition data = new ListOffsetRequestData.ListOffsetPartition(); + this.offsetData.put(new TopicPartition(topic, partitionId), data); + } + } + + this.offsetRequest = ListOffsetRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED) + .build(ApiKeys.LIST_OFFSETS.latestVersion()); + } + + @Benchmark + public String testRequestToJson() { + return RequestConvertToJson.request(offsetRequest).toString(); + } +} diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/ProduceRequestBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/ProduceRequestBenchmark.java new file mode 100644 index 00000000000..405458fef02 --- /dev/null +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/ProduceRequestBenchmark.java @@ -0,0 +1,64 @@ +/* + * 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.common; + +import kafka.network.RequestConvertToJson; +import org.apache.kafka.common.message.ProduceRequestData; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.requests.ProduceRequest; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Warmup; + +import java.util.concurrent.TimeUnit; + +@State(Scope.Benchmark) +@Fork(value = 1) +@Warmup(iterations = 5) +@Measurement(iterations = 15) +@BenchmarkMode(Mode.AverageTime) +@OutputTimeUnit(TimeUnit.NANOSECONDS) +public class ProduceRequestBenchmark { + @Param({"10", "500", "1000"}) + private int topicCount; + + @Param({"3", "10", "20"}) + private int partitionCount; + + ProduceRequest produceRequest; + + @Setup(Level.Trial) + public void setup() { + this.produceRequest = ProduceRequest.forCurrentMagic(new ProduceRequestData()) + .build(ApiKeys.PRODUCE.latestVersion()); + } + + @Benchmark + public String testRequestToJson() { + return RequestConvertToJson.request(produceRequest).toString(); + } +} diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/MetadataRequestBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/MetadataRequestBenchmark.java index 63df9761a17..d5a9dc4317f 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/MetadataRequestBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/MetadataRequestBenchmark.java @@ -21,6 +21,7 @@ import kafka.controller.KafkaController; import kafka.coordinator.group.GroupCoordinator; import kafka.coordinator.transaction.TransactionCoordinator; import kafka.network.RequestChannel; +import kafka.network.RequestConvertToJson; import kafka.server.AdminManager; import kafka.server.BrokerFeatures; import kafka.server.BrokerTopicStats; @@ -213,4 +214,9 @@ public class MetadataRequestBenchmark { public void testMetadataRequestForAllTopics() { kafkaApis.handleTopicMetadataRequest(allTopicMetadataRequest); } + + @Benchmark + public String testRequestToJson() { + return RequestConvertToJson.requestDesc(allTopicMetadataRequest.header(), allTopicMetadataRequest.requestLog(), allTopicMetadataRequest.isForwarded()).toString(); + } }