From d8a26b6fbaead74096fde0ea41ebe2d337c01bc7 Mon Sep 17 00:00:00 2001 From: Anastasia Vela Date: Wed, 9 Sep 2020 16:45:25 -0700 Subject: [PATCH 01/22] KAFKA-10525: Emit JSONs with new auto-generated schema MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Kafka’s request and response traces currently output in a format that is JSON-like and are not easily parsable. There is a new auto-generated schema for each request type that supports outputting JSON payloads for request and response payloads. These can be adapted to provide structured request tracing. --- .../common/requests/RequestResponseTest.java | 2 +- .../scala/kafka/network/RequestChannel.scala | 48 +-- .../kafka/network/RequestConvertToJson.scala | 343 ++++++++++++++++++ .../main/scala/kafka/server/KafkaApis.scala | 6 +- .../kafka/tools/TestRaftRequestHandler.scala | 6 +- .../kafka/network/RequestChannelTest.scala | 29 +- .../network/RequestConvertToJsonTest.scala | 188 ++++++++++ .../unit/kafka/network/SocketServerTest.scala | 17 +- .../jmh/common/FetchRequestBenchmark.java | 6 + .../common/ListOffsetRequestBenchmark.java | 80 ++++ .../jmh/common/ProduceRequestBenchmark.java | 92 +++++ .../metadata/MetadataRequestBenchmark.java | 6 + 12 files changed, 779 insertions(+), 44 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/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 733843f65361a..c6595b5328c8c 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.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; diff --git a/core/src/main/scala/kafka/network/RequestChannel.scala b/core/src/main/scala/kafka/network/RequestChannel.scala index 39ca69f1f1893..161da79eaa497 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 @@ -123,9 +125,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 } @@ -200,7 +202,7 @@ object RequestChannel extends Logging { } } - trace(s"Processor $processor received request: ${requestDesc(true)}") + trace(s"Processor $processor received request: ${RequestConvertToJson.requestDesc(header, loggableRequest, true).toString}") def requestThreadTimeNanos: Long = { if (apiLocalCompleteTimeNanos == -1L) apiLocalCompleteTimeNanos = Time.SYSTEM.nanoseconds @@ -262,29 +264,13 @@ object RequestChannel extends Logging { 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, response, loggableRequest, + context, session, detailsEnabled, + totalTimeMs, requestQueueTimeMs, apiLocalTimeMs, + apiRemoteTimeMs, apiThrottleTimeMs, responseQueueTimeMs, + responseSendTimeMs, temporaryMemoryBytes, + messageConversionsTimeMs) + requestLogger.debug("Completed request:" + desc.toString) } } @@ -314,24 +300,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 0000000000000..1e9f26de4889f --- /dev/null +++ b/core/src/main/scala/kafka/network/RequestConvertToJson.scala @@ -0,0 +1,343 @@ +/* + * 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.{ArrayNode, DoubleNode, IntNode, JsonNodeFactory, LongNode, NullNode, ObjectNode, ShortNode, TextNode} +import kafka.network.RequestChannel.{Response, Session} +import org.apache.kafka.common.message._ +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.record.RecordBatch +import org.apache.kafka.common.requests._ +import org.apache.kafka.common.utils.CollectionUtils + +import scala.jdk.CollectionConverters._ + +object RequestConvertToJson { + def request(request: AbstractRequest, verbose: Boolean): 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: 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 => offsetsForLeaderEpochRequestNode(req, request.version()) + case req: ProduceRequest => produceRequestNode(req, request.version(), verbose) + 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()) + } + } + + 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: ExpireDelegationTokenResponse => ExpireDelegationTokenResponseDataJsonConverter.write(res.data(), version) + case res: FetchResponse[_] => FetchResponseDataJsonConverter.write(res.data(), version) + 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 => offsetsForLeaderEpochResponseNode(res, version) + case res: ProduceResponse => produceResponseNode(res, 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) + } + } + + def requestHeaderNode(header: RequestHeader): JsonNode = { + val node = RequestHeaderDataJsonConverter.write(header.data(), header.headerVersion()).asInstanceOf[ObjectNode] + node.set("requestApiKey", new TextNode(header.apiKey.toString)) + node + } + + def requestDescMetrics(header: RequestHeader, res: Response, req: AbstractRequest, + context: RequestContext, session: Session, verbose: Boolean, + totalTimeMs: Double, requestQueueTimeMs: Double, apiLocalTimeMs: Double, + apiRemoteTimeMs: Double, apiThrottleTimeMs: Long, responseQueueTimeMs: Double, + responseSendTimeMs: Double, temporaryMemoryBytes: Long, + messageConversionsTimeMs: Double): JsonNode = { + val node = new ObjectNode(JsonNodeFactory.instance) + node.set("requestHeader", requestHeaderNode(header)) + node.set("request", request(req, verbose)) + node.set("response", res.responseLog.getOrElse(new TextNode(""))) + node.set("connection", new TextNode(context.connectionId)) + node.set("totalTime", new DoubleNode(totalTimeMs)) + node.set("requestQueueTime", new DoubleNode(requestQueueTimeMs)) + node.set("localTime", new DoubleNode(apiLocalTimeMs)) + node.set("remoteTime", new DoubleNode(apiRemoteTimeMs)) + node.set("throttleTime", new LongNode(apiThrottleTimeMs)) + node.set("responseQueueTime", new DoubleNode(responseQueueTimeMs)) + node.set("sendTime", 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", new TextNode(context.clientInformation.toString)) + if (temporaryMemoryBytes > 0) + node.set("temporaryMemoryBytes", new LongNode(temporaryMemoryBytes)) + if (messageConversionsTimeMs > 0) + node.set("messageConversionsTime", new DoubleNode(messageConversionsTimeMs)) + node + } + + def requestDesc(header: RequestHeader, req: AbstractRequest, verbose: Boolean): JsonNode = { + val node = new ObjectNode(JsonNodeFactory.instance) + node.set("requestHeader", requestHeaderNode(header)) + node.set("request", request(req, verbose)) + node + } + + /** + * Temporary until switch to use the generated schemas. + */ + def offsetsForLeaderEpochRequestNode(request: OffsetsForLeaderEpochRequest, version: Short): JsonNode = { + val node = new ObjectNode(JsonNodeFactory.instance) + if (version >= 3) { + node.set("replicaId", new IntNode(request.replicaId)) + } + val topicsToPartitionEpochs = CollectionUtils.groupPartitionDataByTopic(request.epochsByTopicPartition) + val topicsArray = new ArrayNode(JsonNodeFactory.instance) + for (topicToEpochs <- topicsToPartitionEpochs.entrySet.asScala) { + val topicsData = new ObjectNode(JsonNodeFactory.instance) + topicsData.set("name", new TextNode(topicToEpochs.getKey)) + val partitionsArray = new ArrayNode(JsonNodeFactory.instance) + for (partitionEpoch <- topicToEpochs.getValue.entrySet.asScala) { + val partitionData = partitionEpoch.getValue + val partitionNode = new ObjectNode(JsonNodeFactory.instance) + partitionNode.set("partitionIndex", new IntNode(partitionEpoch.getKey)) + partitionNode.set("leaderEpoch", new IntNode(partitionData.leaderEpoch)) + if (version >= 2) { + val leaderEpoch = partitionData.currentLeaderEpoch + partitionNode.set("currentLeaderEpoch", new IntNode(leaderEpoch.orElse(RecordBatch.NO_PARTITION_LEADER_EPOCH))) + } + partitionsArray.add(partitionNode) + } + topicsData.set("partitions", partitionsArray) + topicsArray.add(topicsData) + } + node.set("topics", topicsArray) + node + } + + /** + * Temporary until switch to use the generated schemas. + */ + def produceRequestNode(request: ProduceRequest, version: Short, verbose: Boolean): JsonNode = { + val node = new ObjectNode(JsonNodeFactory.instance) + if (version >= 3) { + if (request.transactionalId == null) { + node.set("transactionalId", NullNode.instance) + } else { + node.set("transactionalId", new TextNode(request.transactionalId)) + } + } + node.set("acks", new ShortNode(request.acks)) + node.set("timeoutMs", new IntNode(request.timeout)) + if (verbose) { + val partSizes = new ArrayNode(JsonNodeFactory.instance) + for (partSize <- request.partitionSizes().entrySet.asScala) { + val part = new ObjectNode(JsonNodeFactory.instance) + val topic = partSize.getKey + part.set("partition", new TextNode(topic.toString)) + part.set("size", new IntNode(partSize.getValue)) + partSizes.add(part) + } + node.set("partitionSizes", partSizes) + } else { + node.set("partitionSizes", new IntNode(request.partitionSizes.size)) + } + node + } + + /** + * Temporary until switch to use the generated schemas. + */ + def offsetsForLeaderEpochResponseNode(response: OffsetsForLeaderEpochResponse, version: Short): JsonNode = { + val node = new ObjectNode(JsonNodeFactory.instance) + if (version >= 2) { + node.set("throttleTimeMs", new IntNode(response.throttleTimeMs)) + } + val endOffsetsByTopic = CollectionUtils.groupPartitionDataByTopic(response.responses) + val topicsArray = new ArrayNode(JsonNodeFactory.instance) + for (topicToPartitionEpochs <- endOffsetsByTopic.entrySet.asScala) { + val topicData = new ObjectNode(JsonNodeFactory.instance) + topicData.set("name", new TextNode(topicToPartitionEpochs.getKey)) + val partitionsArray = new ArrayNode(JsonNodeFactory.instance) + for (partitionEndOffset <- topicToPartitionEpochs.getValue.entrySet.asScala) { + val epochEndOffset = partitionEndOffset.getValue + val partitionData = new ObjectNode(JsonNodeFactory.instance) + partitionData.set("errorCode", new ShortNode(epochEndOffset.error.code)) + partitionData.set("partitionIndex", new IntNode(partitionEndOffset.getKey)) + if (version >= 1) partitionData.set("leaderEpoch", new IntNode(epochEndOffset.leaderEpoch)) + partitionData.set("endOffset", new LongNode(epochEndOffset.endOffset)) + partitionsArray.add(partitionData) + } + topicData.set("partitions", partitionsArray) + topicsArray.add(topicData) + } + node.set("topics", topicsArray) + node + } + + /** + * Temporary until switch to use the generated schemas. + */ + def produceResponseNode(response: ProduceResponse, version: Short): JsonNode = { + val node = new ObjectNode(JsonNodeFactory.instance) + val responseByTopic = CollectionUtils.groupPartitionDataByTopic(response.responses) + val responsesArray = new ArrayNode(JsonNodeFactory.instance) + for (entry <- responseByTopic.entrySet.asScala) { + val topicData = new ObjectNode(JsonNodeFactory.instance) + topicData.set("name", new TextNode(entry.getKey)) + val partitionsArray = new ArrayNode(JsonNodeFactory.instance) + for (partitionEntry <- entry.getValue.entrySet.asScala) { + val part = partitionEntry.getValue + val partitionData = new ObjectNode(JsonNodeFactory.instance) + partitionData.set("partitionIndex", new IntNode(partitionEntry.getKey)) + var errorCode = part.error.code + if (errorCode == Errors.KAFKA_STORAGE_ERROR.code && version <= 3) { + errorCode = Errors.NOT_LEADER_OR_FOLLOWER.code + } + partitionData.set("errorCode", new ShortNode(errorCode)) + partitionData.set("baseOffset", new LongNode(part.baseOffset)) + if (version >= 2) { + partitionData.set("logAppendTimeMs", new LongNode(part.logAppendTime)) + } + if (version >= 5) { + partitionData.set("logStartOffset", new LongNode(part.logStartOffset)) + } + if (version >= 8) { + val recordErrorsArray = new ArrayNode(JsonNodeFactory.instance) + for (indexAndMessage <- part.recordErrors.asScala) { + val indexAndMessageData = new ObjectNode(JsonNodeFactory.instance) + indexAndMessageData.set("batchIndex", new IntNode(indexAndMessage.batchIndex)) + if (indexAndMessage.message == null) indexAndMessageData.set("batchIndexErrorMessage", NullNode.instance) + else indexAndMessageData.set("batchIndexErrorMessage", new TextNode(indexAndMessage.message)) + recordErrorsArray.add(indexAndMessageData) + } + partitionData.set("recordErrors", recordErrorsArray) + if (part.errorMessage == null) partitionData.set("errorMessage", NullNode.instance) + else partitionData.set("errorMessage", new TextNode(part.errorMessage)) + } + partitionsArray.add(partitionData) + } + topicData.set("partitions", partitionsArray) + responsesArray.add(topicData) + } + node.set("responses", responsesArray) + if (version >= 1) { + node.set("throttleTimeMs", new IntNode(response.throttleTimeMs)) + } + node + } +} diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 4b25af6dba7de..047e0d3241854 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -34,7 +34,7 @@ import kafka.coordinator.group.{GroupCoordinator, JoinGroupResult, LeaveGroupRes import kafka.coordinator.transaction.{InitProducerIdResult, TransactionCoordinator} import kafka.log.AppendOrigin import kafka.message.ZStdCompressionCodec -import kafka.network.RequestChannel +import kafka.network.{RequestChannel, RequestConvertToJson} import kafka.security.authorizer.{AclEntry, AuthorizerUtils} import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota} import kafka.utils.{CoreUtils, Logging} @@ -183,7 +183,7 @@ class KafkaApis(val requestChannel: RequestChannel, */ override def handle(request: RequestChannel.Request): Unit = { try { - trace(s"Handling request:${request.requestDesc(true)} from connection ${request.context.connectionId};" + + trace(s"Handling request:${RequestConvertToJson.requestDesc(request.header, request.loggableRequest, true).toString} from connection ${request.context.connectionId};" + s"securityProtocol:${request.context.securityProtocol},principal:${request.context.principal}") request.envelope.foreach { envelope => @@ -3513,7 +3513,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 fefd274927df2..19df2fcaf6689 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 @@ -39,7 +40,7 @@ class TestRaftRequestHandler( override def handle(request: RequestChannel.Request): Unit = { try { - trace(s"Handling request:${request.requestDesc(true)} from connection ${request.context.connectionId};" + + trace(s"Handling request:${RequestConvertToJson.requestDesc(request.header, request.loggableRequest, true)} from connection ${request.context.connectionId};" + s"securityProtocol:${request.context.securityProtocol},principal:${request.context.principal}") request.header.apiKey match { case ApiKeys.VOTE @@ -94,10 +95,9 @@ class TestRaftRequestHandler( case Some(response) => val responseSend = request.context.buildResponseSend(response) val responseString = - if (RequestChannel.isRequestLoggingEnabled) Some(response.toString) + if (RequestChannel.isRequestLoggingEnabled) Some(RequestConvertToJson.requestHeaderNode(request.header)) else None new RequestChannel.SendResponse(request, responseSend, responseString, None) - case None => new RequestChannel.NoOpResponse(request) } sendResponse(response) diff --git a/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala b/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala index 8039e8333ebfa..834a2df6bc718 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.loggableRequest, true).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.loggableRequest, true).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, true).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 0000000000000..b73900736b42d --- /dev/null +++ b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala @@ -0,0 +1,188 @@ +/** + * 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.util.HashMap + +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.message._ +import org.junit.Test +import org.apache.kafka.common.protocol.ApiKeys +import org.apache.kafka.common.record.MemoryRecords +import org.apache.kafka.common.requests._ + +import scala.collection.mutable.ArrayBuffer + +class RequestConvertToJsonTest { + + def createRequestsFromApiKey(apiKey: ApiKeys, version: Short): AbstractRequest = apiKey match { + case ApiKeys.PRODUCE => ProduceRequest.Builder.forCurrentMagic(0.toShort, 10000, new HashMap[TopicPartition, MemoryRecords]()).build() + case ApiKeys.FETCH => new FetchRequest(new FetchRequestData(), version) + case ApiKeys.LIST_OFFSETS => new ListOffsetRequest(new ListOffsetRequestData().toStruct(version), version) + case ApiKeys.METADATA => new MetadataRequest(new MetadataRequestData(), version) + case ApiKeys.OFFSET_COMMIT => new OffsetCommitRequest(new OffsetCommitRequestData(), version) + case ApiKeys.OFFSET_FETCH => new OffsetFetchRequest(new OffsetFetchRequestData().toStruct(version), version) + case ApiKeys.FIND_COORDINATOR => new FindCoordinatorRequest(new FindCoordinatorRequestData().toStruct(version), version) + case ApiKeys.JOIN_GROUP => new JoinGroupRequest(new JoinGroupRequestData(), version) + case ApiKeys.HEARTBEAT => new HeartbeatRequest(new HeartbeatRequestData().toStruct(version), version) + case ApiKeys.LEAVE_GROUP => new LeaveGroupRequest(new LeaveGroupRequestData().toStruct(version), version) + case ApiKeys.SYNC_GROUP => new SyncGroupRequest(new SyncGroupRequestData(), version) + case ApiKeys.STOP_REPLICA => new StopReplicaRequest(new StopReplicaRequestData().toStruct(version), version) + case ApiKeys.CONTROLLED_SHUTDOWN => new ControlledShutdownRequest(new ControlledShutdownRequestData().toStruct(version), version) + case ApiKeys.UPDATE_METADATA => new UpdateMetadataRequest(new UpdateMetadataRequestData().toStruct(version), version) + case ApiKeys.LEADER_AND_ISR => new LeaderAndIsrRequest(new LeaderAndIsrRequestData().toStruct(version), version) + case ApiKeys.DESCRIBE_GROUPS => new DescribeGroupsRequest(new DescribeGroupsRequestData().toStruct(version), version) + case ApiKeys.LIST_GROUPS => new ListGroupsRequest(new ListGroupsRequestData(), version) + case ApiKeys.SASL_HANDSHAKE => new SaslHandshakeRequest(new SaslHandshakeRequestData()) + case ApiKeys.API_VERSIONS => new ApiVersionsRequest(new ApiVersionsRequestData(), version) + case ApiKeys.CREATE_TOPICS => new CreateTopicsRequest(new CreateTopicsRequestData().toStruct(version), version) + case ApiKeys.DELETE_TOPICS => new DeleteTopicsRequest(new DeleteTopicsRequestData().toStruct(version), version) + case ApiKeys.DELETE_RECORDS => new DeleteRecordsRequest(new DeleteRecordsRequestData().toStruct(version), version) + case ApiKeys.INIT_PRODUCER_ID => new InitProducerIdRequest(new InitProducerIdRequestData().toStruct(version), version) + case ApiKeys.OFFSET_FOR_LEADER_EPOCH => new OffsetsForLeaderEpochRequest(new OffsetForLeaderEpochResponseData().toStruct(version), version) + case ApiKeys.ADD_PARTITIONS_TO_TXN => new AddPartitionsToTxnRequest(new AddPartitionsToTxnRequestData(), version) + case ApiKeys.ADD_OFFSETS_TO_TXN => new AddOffsetsToTxnRequest(new AddOffsetsToTxnRequestData(), version) + case ApiKeys.END_TXN => new EndTxnRequest(new EndTxnRequestData().toStruct(version), version) + case ApiKeys.WRITE_TXN_MARKERS => new WriteTxnMarkersRequest(new WriteTxnMarkersRequestData().toStruct(version), version) + case ApiKeys.TXN_OFFSET_COMMIT => new TxnOffsetCommitRequest(new TxnOffsetCommitRequestData(), version) + case ApiKeys.DESCRIBE_ACLS => + val data = new DescribeAclsRequestData().setResourceTypeFilter(1).setOperation(2).setPermissionType(2) + new DescribeAclsRequest(data.toStruct(version), version) + case ApiKeys.CREATE_ACLS => new CreateAclsRequest(new CreateAclsRequestData().toStruct(version), version) + case ApiKeys.DELETE_ACLS => new DeleteAclsRequest(new DeleteAclsRequestData().toStruct(version), version) + case ApiKeys.DESCRIBE_CONFIGS => new DescribeConfigsRequest(new DescribeConfigsRequestData(), version) + case ApiKeys.ALTER_CONFIGS => new AlterConfigsRequest(new AlterConfigsRequestData(), version) + case ApiKeys.ALTER_REPLICA_LOG_DIRS => new AlterReplicaLogDirsRequest(new AlterReplicaLogDirsRequestData(), version) + case ApiKeys.DESCRIBE_LOG_DIRS => new DescribeLogDirsRequest(new DescribeLogDirsRequestData(), version) + case ApiKeys.SASL_AUTHENTICATE => new SaslAuthenticateRequest(new SaslAuthenticateRequestData(), version) + case ApiKeys.CREATE_PARTITIONS => new CreatePartitionsRequest(new CreatePartitionsRequestData().toStruct(version), version) + case ApiKeys.CREATE_DELEGATION_TOKEN => new CreateDelegationTokenRequest(new CreateDelegationTokenRequestData().toStruct(version), version) + case ApiKeys.RENEW_DELEGATION_TOKEN => new RenewDelegationTokenRequest(new RenewDelegationTokenRequestData(), version) + case ApiKeys.EXPIRE_DELEGATION_TOKEN => new ExpireDelegationTokenRequest(new ExpireDelegationTokenRequestData().toStruct(version), version) + case ApiKeys.DESCRIBE_DELEGATION_TOKEN => new DescribeDelegationTokenRequest(new DescribeDelegationTokenRequestData(), version) + case ApiKeys.DELETE_GROUPS => new DeleteGroupsRequest(new DeleteGroupsRequestData(), version) + case ApiKeys.ELECT_LEADERS => new ElectLeadersRequest(new ElectLeadersRequestData().toStruct(version), version) + case ApiKeys.INCREMENTAL_ALTER_CONFIGS => new IncrementalAlterConfigsRequest.Builder(new IncrementalAlterConfigsRequestData()).build(version) + case ApiKeys.ALTER_PARTITION_REASSIGNMENTS => new AlterPartitionReassignmentsRequest.Builder(new AlterPartitionReassignmentsRequestData()).build(version) + case ApiKeys.LIST_PARTITION_REASSIGNMENTS => new ListPartitionReassignmentsRequest.Builder(new ListPartitionReassignmentsRequestData()).build(version) + case ApiKeys.OFFSET_DELETE => new OffsetDeleteRequest(new OffsetDeleteRequestData(), version) + case ApiKeys.DESCRIBE_CLIENT_QUOTAS => new DescribeClientQuotasRequest(new DescribeClientQuotasRequestData(), version) + case ApiKeys.ALTER_CLIENT_QUOTAS => new AlterClientQuotasRequest(new AlterClientQuotasRequestData(), version) + case ApiKeys.DESCRIBE_USER_SCRAM_CREDENTIALS => new DescribeUserScramCredentialsRequest.Builder(new DescribeUserScramCredentialsRequestData()).build(version) + case ApiKeys.ALTER_USER_SCRAM_CREDENTIALS => new AlterUserScramCredentialsRequest.Builder(new AlterUserScramCredentialsRequestData()).build(version) + case ApiKeys.VOTE => new VoteRequest.Builder(new VoteRequestData()).build(version) + case ApiKeys.BEGIN_QUORUM_EPOCH => new BeginQuorumEpochRequest.Builder(new BeginQuorumEpochRequestData()).build(version) + case ApiKeys.END_QUORUM_EPOCH => new EndQuorumEpochRequest.Builder(new EndQuorumEpochRequestData()).build(version) + case ApiKeys.DESCRIBE_QUORUM => new DescribeQuorumRequest.Builder(new DescribeQuorumRequestData()).build(version) + case ApiKeys.ALTER_ISR => new AlterIsrRequest.Builder(new AlterIsrRequestData()).build(version) + case ApiKeys.UPDATE_FEATURES => new UpdateFeaturesRequest.Builder(new UpdateFeaturesRequestData()).build(version) + case _ => throw new AssertionError(String.format("Request type %s is not tested in `RequestConvertToJsonTest`", apiKey)) + } + + def createResponseFromApiKey(apiKey: ApiKeys, version: Short): AbstractResponse = apiKey match { + case ApiKeys.PRODUCE => new ProduceResponse(new ProduceResponseData().toStruct(version)) + case ApiKeys.FETCH => new FetchResponse(new FetchResponseData()) + case ApiKeys.LIST_OFFSETS => new ListOffsetResponse(new ListOffsetResponseData()) + case ApiKeys.METADATA => new MetadataResponse(new MetadataResponseData()) + case ApiKeys.OFFSET_COMMIT => new OffsetCommitResponse(new OffsetCommitResponseData()) + case ApiKeys.OFFSET_FETCH => new OffsetFetchResponse(new OffsetFetchResponseData().toStruct(version), version) + case ApiKeys.FIND_COORDINATOR => new FindCoordinatorResponse(new FindCoordinatorResponseData()) + case ApiKeys.JOIN_GROUP => new JoinGroupResponse(new JoinGroupResponseData()) + case ApiKeys.HEARTBEAT => new HeartbeatResponse(new HeartbeatResponseData()) + case ApiKeys.LEAVE_GROUP => new LeaveGroupResponse(new LeaveGroupResponseData()) + case ApiKeys.SYNC_GROUP => new SyncGroupResponse(new SyncGroupResponseData()) + case ApiKeys.STOP_REPLICA => new StopReplicaResponse(new StopReplicaResponseData()) + case ApiKeys.CONTROLLED_SHUTDOWN => new ControlledShutdownResponse(new ControlledShutdownResponseData()) + case ApiKeys.UPDATE_METADATA => new UpdateMetadataResponse(new UpdateMetadataResponseData()) + case ApiKeys.LEADER_AND_ISR => new LeaderAndIsrResponse(new LeaderAndIsrResponseData()) + case ApiKeys.DESCRIBE_GROUPS => new DescribeGroupsResponse(new DescribeGroupsResponseData()) + case ApiKeys.LIST_GROUPS => new ListGroupsResponse(new ListGroupsResponseData()) + case ApiKeys.SASL_HANDSHAKE => new SaslHandshakeResponse(new SaslHandshakeResponseData()) + case ApiKeys.API_VERSIONS => new ApiVersionsResponse(new ApiVersionsResponseData()) + case ApiKeys.CREATE_TOPICS => new CreateTopicsResponse(new CreateTopicsResponseData()) + case ApiKeys.DELETE_TOPICS => new DeleteTopicsResponse(new DeleteTopicsResponseData()) + case ApiKeys.DELETE_RECORDS => new DeleteRecordsResponse(new DeleteRecordsResponseData()) + case ApiKeys.INIT_PRODUCER_ID => new InitProducerIdResponse(new InitProducerIdResponseData()) + case ApiKeys.OFFSET_FOR_LEADER_EPOCH => new OffsetsForLeaderEpochResponse(new OffsetForLeaderEpochResponseData().toStruct(version)) + case ApiKeys.ADD_PARTITIONS_TO_TXN => new AddPartitionsToTxnResponse(new AddPartitionsToTxnResponseData().toStruct(version), version) + case ApiKeys.ADD_OFFSETS_TO_TXN => new AddOffsetsToTxnResponse(new AddOffsetsToTxnResponseData()) + case ApiKeys.END_TXN => new EndTxnResponse(new EndTxnResponseData()) + case ApiKeys.WRITE_TXN_MARKERS => new WriteTxnMarkersResponse(new WriteTxnMarkersResponseData().toStruct(version), version) + case ApiKeys.TXN_OFFSET_COMMIT => new TxnOffsetCommitResponse(new TxnOffsetCommitResponseData()) + case ApiKeys.DESCRIBE_ACLS => new DescribeAclsResponse(new DescribeAclsResponseData()) + case ApiKeys.CREATE_ACLS => new CreateAclsResponse(new CreateAclsResponseData()) + case ApiKeys.DELETE_ACLS => new DeleteAclsResponse(new DeleteAclsResponseData()) + case ApiKeys.DESCRIBE_CONFIGS => new DescribeConfigsResponse(new DescribeConfigsResponseData()) + case ApiKeys.ALTER_CONFIGS => new AlterConfigsResponse(new AlterConfigsResponseData()) + case ApiKeys.ALTER_REPLICA_LOG_DIRS => new AlterReplicaLogDirsResponse(new AlterReplicaLogDirsResponseData()) + case ApiKeys.DESCRIBE_LOG_DIRS => new DescribeLogDirsResponse(new DescribeLogDirsResponseData()) + case ApiKeys.SASL_AUTHENTICATE => new SaslAuthenticateResponse(new SaslAuthenticateResponseData()) + case ApiKeys.CREATE_PARTITIONS => new CreatePartitionsResponse(new CreatePartitionsResponseData()) + case ApiKeys.CREATE_DELEGATION_TOKEN => new CreateDelegationTokenResponse(new CreateDelegationTokenResponseData()) + case ApiKeys.RENEW_DELEGATION_TOKEN => new RenewDelegationTokenResponse(new RenewDelegationTokenResponseData()) + case ApiKeys.EXPIRE_DELEGATION_TOKEN => new ExpireDelegationTokenResponse(new ExpireDelegationTokenResponseData()) + case ApiKeys.DESCRIBE_DELEGATION_TOKEN => new DescribeDelegationTokenResponse(new DescribeDelegationTokenResponseData().toStruct(version), version) + case ApiKeys.DELETE_GROUPS => new DeleteGroupsResponse(new DeleteGroupsResponseData()) + case ApiKeys.ELECT_LEADERS => new ElectLeadersResponse(new ElectLeadersResponseData().toStruct(version), version) + case ApiKeys.INCREMENTAL_ALTER_CONFIGS => new IncrementalAlterConfigsResponse(new IncrementalAlterConfigsResponseData()) + case ApiKeys.ALTER_PARTITION_REASSIGNMENTS => new AlterPartitionReassignmentsResponse(new AlterPartitionReassignmentsResponseData()) + case ApiKeys.LIST_PARTITION_REASSIGNMENTS => new ListPartitionReassignmentsResponse(new ListPartitionReassignmentsResponseData()) + case ApiKeys.OFFSET_DELETE => new OffsetDeleteResponse(new OffsetDeleteResponseData()) + case ApiKeys.DESCRIBE_CLIENT_QUOTAS => new DescribeClientQuotasResponse(new DescribeClientQuotasResponseData().toStruct(version), version) + case ApiKeys.ALTER_CLIENT_QUOTAS => new AlterClientQuotasResponse(new AlterClientQuotasResponseData().toStruct(version), version) + case ApiKeys.DESCRIBE_USER_SCRAM_CREDENTIALS => new DescribeUserScramCredentialsResponse(new DescribeUserScramCredentialsResponseData()) + case ApiKeys.ALTER_USER_SCRAM_CREDENTIALS => new AlterUserScramCredentialsResponse(new AlterUserScramCredentialsResponseData()) + case ApiKeys.VOTE => new VoteResponse(new VoteResponseData()) + case ApiKeys.BEGIN_QUORUM_EPOCH => new BeginQuorumEpochResponse(new BeginQuorumEpochResponseData()) + case ApiKeys.END_QUORUM_EPOCH => new EndQuorumEpochResponse(new EndQuorumEpochResponseData()) + case ApiKeys.DESCRIBE_QUORUM => new DescribeQuorumResponse(new DescribeQuorumResponseData()) + case ApiKeys.ALTER_ISR => new AlterIsrResponse(new AlterIsrResponseData()) + case ApiKeys.UPDATE_FEATURES => new UpdateFeaturesResponse(new UpdateFeaturesResponseData()) + case _ => throw new AssertionError(String.format("Response type %s not tested in `RequestConvertToJsonTest`", apiKey)) + } + + @Test + def testAllRequestTypesHandled(): Unit = { + val unhandledKeys = ArrayBuffer[String]() + ApiKeys.values().foreach(key => { + val version: Short = 0 + val req = createRequestsFromApiKey(key, version) + try { + RequestConvertToJson.request(req, false) + } catch { + case _ : Throwable => unhandledKeys += key.toString + } + }) + assert(unhandledKeys.isEmpty, String.format("%s request keys not handled in RequestConvertToJson", unhandledKeys)) + } + + @Test + def testAllResponseTypesHandled(): Unit = { + val unhandledKeys = ArrayBuffer[String]() + ApiKeys.values().foreach(key => { + val version: Short = 0 + val res = createResponseFromApiKey(key, version) + try { + RequestConvertToJson.response(res, version) + } catch { + case _ : Throwable => unhandledKeys += key.toString + } + }) + assert(unhandledKeys.isEmpty, String.format("%s response keys not handled in RequestConvertToJson", unhandledKeys)) + } +} diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index c3181e5bf2833..bc6b0afed38c0 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -25,6 +25,11 @@ import java.nio.charset.StandardCharsets import java.util import java.util.concurrent.{CompletableFuture, ConcurrentLinkedQueue, Executors, TimeUnit} import java.util.{Properties, Random} +<<<<<<< HEAD +======= + +import com.fasterxml.jackson.databind.node.{JsonNodeFactory, ObjectNode, TextNode} +>>>>>>> 1d0de0692c... KAFKA-10525: Emit JSONs with new auto-generated schema import com.yammer.metrics.core.{Gauge, Meter} import javax.net.ssl._ @@ -140,7 +145,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 +682,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) @@ -1092,8 +1099,10 @@ 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 send = new NetworkSend(request.context.connectionId, ByteBuffer.allocate(responseBufferSize)) + 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/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 b24ebf50df7bb..b5b6109cd6184 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 int testSerializeFetchRequestForReplica() throws IOException { send.writeTo(channel); return channel.buffer().limit(); } + + @Benchmark + public String testRequestToJson() { + return RequestConvertToJson.request(consumerRequest, false).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 0000000000000..58eaaf68cd219 --- /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, false).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 0000000000000..c41a73d4e4064 --- /dev/null +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/ProduceRequestBenchmark.java @@ -0,0 +1,92 @@ +/* + * 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.TopicPartition; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.record.CompressionType; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.MemoryRecordsBuilder; +import org.apache.kafka.common.record.RecordBatch; +import org.apache.kafka.common.record.TimestampType; +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.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +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 ProduceRequestBenchmark { + @Param({"10", "500", "1000"}) + private int topicCount; + + @Param({"3", "10", "20"}) + private int partitionCount; + + Map produceData; + + ProduceRequest produceRequest; + + @Setup(Level.Trial) + public void setup() { + this.produceData = new HashMap<>(); + for (int topicIdx = 0; topicIdx < topicCount; topicIdx++) { + String topic = UUID.randomUUID().toString(); + for (int partitionId = 0; partitionId < partitionCount; partitionId++) { + ByteBuffer buffer = ByteBuffer.allocate(256); + MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, + CompressionType.NONE, TimestampType.CREATE_TIME, 0L); + try { + builder.append(10L, null, "a".getBytes(StandardCharsets.US_ASCII)); + } catch (Exception e) { + continue; + } + produceData.put(new TopicPartition(topic, partitionId), builder.build()); + } + } + + this.produceRequest = ProduceRequest.Builder.forCurrentMagic((short) -1, 3000, produceData) + .build(ApiKeys.PRODUCE.latestVersion()); + } + + @Benchmark + public String testRequestToJson() { + return RequestConvertToJson.request(produceRequest, false).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 63df9761a17c2..a081530454824 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.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 @@ private RequestChannel.Request buildAllTopicMetadataRequest() { public void testMetadataRequestForAllTopics() { kafkaApis.handleTopicMetadataRequest(allTopicMetadataRequest); } + + @Benchmark + public String testRequestToJson() { + return RequestConvertToJson.requestDesc(allTopicMetadataRequest.header(), allTopicMetadataRequest.loggableRequest(), false).toString(); + } } From c3e1e1f1aa7412b602917b7a8ed00dbb9f8f0cea Mon Sep 17 00:00:00 2001 From: anatasiavela Date: Wed, 28 Oct 2020 20:27:27 -0700 Subject: [PATCH 02/22] add extra lines --- .../org/apache/kafka/common/requests/InitProducerIdRequest.java | 1 + 1 file changed, 1 insertion(+) 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 08e75cb536c63..49a37e9f72798 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 @@ -76,4 +76,5 @@ public static InitProducerIdRequest parse(ByteBuffer buffer, short version) { protected InitProducerIdRequestData data() { return data; } + } From 0c7203b6296c0c18b5127726114657c6a3d967ac Mon Sep 17 00:00:00 2001 From: anatasiavela Date: Wed, 28 Oct 2020 20:28:45 -0700 Subject: [PATCH 03/22] remove extra spaces --- .../org/apache/kafka/common/requests/InitProducerIdRequest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 49a37e9f72798..db96e9e10ea99 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 @@ -76,5 +76,5 @@ public static InitProducerIdRequest parse(ByteBuffer buffer, short version) { protected InitProducerIdRequestData data() { return data; } - + } From 38747a5782951f6740a1e1cebf007c73a6884b7d Mon Sep 17 00:00:00 2001 From: anatasiavela Date: Fri, 30 Oct 2020 01:34:01 -0700 Subject: [PATCH 04/22] add data accessor and creat tests for manually created data --- .../apache/kafka/clients/NetworkClient.java | 12 +- .../kafka/clients/admin/KafkaAdminClient.java | 11 +- .../internals/AbstractCoordinator.java | 8 +- .../internals/TransactionManager.java | 2 +- .../requests/AddOffsetsToTxnRequest.java | 2 +- .../requests/AddOffsetsToTxnResponse.java | 2 +- .../requests/AddPartitionsToTxnRequest.java | 2 +- .../requests/AddPartitionsToTxnResponse.java | 6 +- .../common/requests/ApiVersionsResponse.java | 2 +- .../requests/BeginQuorumEpochRequest.java | 2 +- .../requests/BeginQuorumEpochResponse.java | 2 +- .../common/requests/DeleteGroupsRequest.java | 2 +- .../common/requests/DeleteGroupsResponse.java | 2 +- .../common/requests/DeleteTopicsResponse.java | 2 +- .../requests/DescribeGroupsResponse.java | 2 +- .../requests/DescribeQuorumRequest.java | 6 +- .../requests/DescribeQuorumResponse.java | 2 +- .../DescribeUserScramCredentialsRequest.java | 2 +- .../requests/EndQuorumEpochRequest.java | 2 +- .../requests/EndQuorumEpochResponse.java | 2 +- .../kafka/common/requests/EndTxnRequest.java | 2 +- .../kafka/common/requests/EndTxnResponse.java | 2 +- .../common/requests/HeartbeatRequest.java | 2 +- .../requests/InitProducerIdRequest.java | 2 +- .../common/requests/LeaveGroupResponse.java | 2 +- .../common/requests/OffsetDeleteRequest.java | 2 +- .../common/requests/OffsetDeleteResponse.java | 2 +- .../common/requests/OffsetFetchRequest.java | 2 +- .../common/requests/OffsetFetchResponse.java | 6 +- .../common/requests/SyncGroupRequest.java | 2 +- .../common/requests/SyncGroupResponse.java | 2 +- .../requests/TxnOffsetCommitRequest.java | 2 +- .../requests/TxnOffsetCommitResponse.java | 6 +- .../kafka/common/requests/VoteRequest.java | 2 +- .../kafka/common/requests/VoteResponse.java | 2 +- .../requests/WriteTxnMarkersRequest.java | 2 +- .../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 | 76 +++- .../requests/OffsetFetchResponseTest.java | 6 +- .../common/requests/RequestResponseTest.java | 12 +- .../requests/TxnOffsetCommitRequestTest.java | 6 +- .../authenticator/SaslAuthenticatorTest.java | 12 +- .../kafka/network/RequestConvertToJson.scala | 351 +++++++++--------- .../network/RequestConvertToJsonTest.scala | 220 +++++------ 50 files changed, 494 insertions(+), 446 deletions(-) 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 849c8873b5eb6..840185da464b6 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 @@ else if (req.isInternalRequest && response instanceof ApiVersionsResponse) 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 @@ private void handleApiVersionsResponse(List responses, // 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 @@ private void handleApiVersionsResponse(List responses, } 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 cf0a6f6949f90..f7ac1084cd899 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 @@ void handleResponse(AbstractResponse abstractResponse) { } // 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,13 @@ ApiVersionsRequest.Builder createRequest(int timeoutMs) { @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 if (options.sendRequestToController() && + apiVersionsResponse.data().errorCode() == Errors.NOT_CONTROLLER.code()) { + handleNotControllerError(Errors.NOT_CONTROLLER); } 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 4ca71bb626a06..29842a90c3362 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 void handle(SyncGroupResponse syncResponse, 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 void handle(SyncGroupResponse syncResponse, 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 void handle(SyncGroupResponse syncResponse, 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 d77ad41938eef..077316b81b492 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 @@ -1623,7 +1623,7 @@ Priority priority() { @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/AddOffsetsToTxnRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnRequest.java index 4ad56aa94268f..390b61ade2ffe 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 @@ public class AddOffsetsToTxnRequest extends AbstractRequest { - public AddOffsetsToTxnRequestData data; + private final AddOffsetsToTxnRequestData data; public static class Builder extends AbstractRequest.Builder { public AddOffsetsToTxnRequestData 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 6908e51960609..a1291f221970d 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 @@ */ public class AddOffsetsToTxnResponse extends AbstractResponse { - public AddOffsetsToTxnResponseData data; + private final AddOffsetsToTxnResponseData data; public AddOffsetsToTxnResponse(AddOffsetsToTxnResponseData data) { super(ApiKeys.ADD_OFFSETS_TO_TXN); 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 57645789ca000..7bc37a4a2dfa4 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 @@ public class AddPartitionsToTxnRequest extends AbstractRequest { - public final AddPartitionsToTxnRequestData data; + private final AddPartitionsToTxnRequestData data; private List cachedPartitions = null; 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 c6da3af0a9319..d301e16fb2de7 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 @@ */ public class AddPartitionsToTxnResponse extends AbstractResponse { - public final AddPartitionsToTxnResponseData data; + private final AddPartitionsToTxnResponseData data; private Map cachedErrorsMap = null; @@ -89,6 +89,10 @@ topicName, new AddPartitionsToTxnPartitionResultCollection() .setResults(topicCollection); } + public AddPartitionsToTxnResponseData data() { + return data; + } + @Override public int throttleTimeMs() { return data.throttleTimeMs(); 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 eaf8113fcd0b7..4d7397ad96df1 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 c83e29dd3d52a..a7c61f97caaca 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 String toString() { } } - public final BeginQuorumEpochRequestData data; + private final BeginQuorumEpochRequestData data; private BeginQuorumEpochRequest(BeginQuorumEpochRequestData data, short version) { super(ApiKeys.BEGIN_QUORUM_EPOCH, version); 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 c3e80eccaa82c..f25ac81c85e07 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 @@ * - {@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); 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 d09a4d4c02d77..72e68424bc661 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 String toString() { } } - public final DeleteGroupsRequestData data; + private final DeleteGroupsRequestData data; public DeleteGroupsRequest(DeleteGroupsRequestData data, short version) { super(ApiKeys.DELETE_GROUPS, version); 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 a8e8d482de1d1..627c5900c42c5 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 @@ */ public class DeleteGroupsResponse extends AbstractResponse { - public final DeleteGroupsResponseData data; + private final DeleteGroupsResponseData data; public DeleteGroupsResponse(DeleteGroupsResponseData data) { super(ApiKeys.DELETE_GROUPS); 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 3584ddfbfc497..dece4ec31685a 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); 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 67c43d87ea092..f0edbd75e1ae5 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); 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 c318c214b6adf..4bc7dfbf07f97 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 String toString() { } } - public final DescribeQuorumRequestData data; + private final DescribeQuorumRequestData data; private DescribeQuorumRequest(DescribeQuorumRequestData data, short version) { super(ApiKeys.DESCRIBE_QUORUM, version); @@ -71,6 +71,10 @@ public static DescribeQuorumRequestData singletonRequest(TopicPartition topicPar ))); } + public DescribeQuorumRequestData data() { + return data; + } + @Override protected 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 cb1a5ec277c08..cb45fb19085e3 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 @@ * - {@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); 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 8b28d6b18628b..a2a90712bd3a5 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 String toString() { } } - private DescribeUserScramCredentialsRequestData data; + private final DescribeUserScramCredentialsRequestData data; private final short version; private DescribeUserScramCredentialsRequest(DescribeUserScramCredentialsRequestData data, short version) { 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 b9f87d73fbd24..ca9abe5168924 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 String toString() { } } - public final EndQuorumEpochRequestData data; + private final EndQuorumEpochRequestData data; private EndQuorumEpochRequest(EndQuorumEpochRequestData data, short version) { super(ApiKeys.END_QUORUM_EPOCH, version); 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 9b446663ea540..1b853ef23ca59 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 @@ * - {@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); 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 fdfafd005ca66..0eb3c5b5d3e79 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 @@ public class EndTxnRequest extends AbstractRequest { - public final EndTxnRequestData data; + private final EndTxnRequestData data; public static class Builder extends AbstractRequest.Builder { public final EndTxnRequestData 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 b782cefd50717..4eb9b7b5fe1ea 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 @@ */ public class EndTxnResponse extends AbstractResponse { - public final EndTxnResponseData data; + private final EndTxnResponseData data; public EndTxnResponse(EndTxnResponseData data) { super(ApiKeys.END_TXN); 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 754566db7a425..4cdd2ab0e0954 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 String toString() { } } - public final HeartbeatRequestData data; + private final HeartbeatRequestData data; private HeartbeatRequest(HeartbeatRequestData data, short version) { super(ApiKeys.HEARTBEAT, version); 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 db96e9e10ea99..bd8d996086868 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 String toString() { } } - public final InitProducerIdRequestData data; + private final InitProducerIdRequestData data; private InitProducerIdRequest(InitProducerIdRequestData data, short version) { super(ApiKeys.INIT_PRODUCER_ID, version); 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 3bc70674f8a75..56f28a70514ec 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 @@ */ public class LeaveGroupResponse extends AbstractResponse { - public final LeaveGroupResponseData data; + private final LeaveGroupResponseData data; public LeaveGroupResponse(LeaveGroupResponseData data) { super(ApiKeys.LEAVE_GROUP); 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 2f9d0203a9d7a..8a3e48e349919 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 String toString() { } } - public final OffsetDeleteRequestData data; + private final OffsetDeleteRequestData data; public OffsetDeleteRequest(OffsetDeleteRequestData data, short version) { super(ApiKeys.OFFSET_DELETE, version); 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 ceb932fb21d1e..2e3b4e5455b2a 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 @@ */ public class OffsetDeleteResponse extends AbstractResponse { - public final OffsetDeleteResponseData data; + private final OffsetDeleteResponseData data; public OffsetDeleteResponse(OffsetDeleteResponseData data) { super(ApiKeys.OFFSET_DELETE); 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 fd08c725399cb..b05b4ac41901c 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 { 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 3e30968715f8d..4f673bc769010 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 { @@ -176,6 +176,10 @@ private static Errors topLevelError(OffsetFetchResponseData data) { return Errors.NONE; } + public OffsetFetchResponseData data() { + return data; + } + @Override public int throttleTimeMs() { return data.throttleTimeMs(); 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 18e5eb8b80964..6651e965cd164 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 String toString() { } } - public final SyncGroupRequestData data; + private final SyncGroupRequestData data; public SyncGroupRequest(SyncGroupRequestData data, short version) { super(ApiKeys.SYNC_GROUP, version); 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 a26b6c900bbdf..cfe18d6e1b3cf 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 @@ public class SyncGroupResponse extends AbstractResponse { - public final SyncGroupResponseData data; + private final SyncGroupResponseData data; public SyncGroupResponse(SyncGroupResponseData data) { super(ApiKeys.SYNC_GROUP); 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 055793cf54b15..15af67d8787f6 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 { 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 1ead0ec6d6073..b1dde629feb42 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 @@ */ public class TxnOffsetCommitResponse extends AbstractResponse { - public final TxnOffsetCommitResponseData data; + private final TxnOffsetCommitResponseData data; public TxnOffsetCommitResponse(TxnOffsetCommitResponseData data) { super(ApiKeys.TXN_OFFSET_COMMIT); @@ -77,6 +77,10 @@ public TxnOffsetCommitResponse(int requestThrottleMs, Map 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 void testUsableVersionLatestVersions() { @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 e8499a84691dc..c24fa8577cc1a 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 void testNoGenerationWillNotTriggerProtocolNameCheck() { 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 void testNoGenerationWillNotTriggerProtocolNameCheck() { } 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 @@ private boolean joinGroupWithProtocolTypeAndName(String joinGroupResponseProtoco 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 843689294dafd..1c7cfb47bdbe3 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 void testNormalJoinGroupLeader() { 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 void testOutdatedCoordinatorAssignment() { 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 void testOutdatedCoordinatorAssignment() { 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 void testPatternJoinGroupLeader() { 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 void testForceMetadataRefreshForPatternSubscriptionDuringRebalance() { 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 void testNormalJoinGroupFollower() { 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 void testPatternJoinGroupFollower() { 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 void testUpdateMetadataDuringRebalance() { 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 void testCommitOffsetRebalanceInProgress() { 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 @@ private void prepareJoinAndSyncResponse(String consumerId, int generation, List< 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 2cef6f688f869..57c8474996dce 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 void testSendTxnOffsetsWithGroupId() { 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 @@ private void sendOffsetsWithGroupMetadata(final short maxVersion) { 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 995dfc77d38e4..a054f6f1811d7 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 @@ private void prepareAndReceiveInitProducerId(long producerId, short producerEpoc 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 468099fc13be3..c2977ea9f66d1 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 void testUnsupportedInitTransactions() { 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 void testFencedInstanceIdInTxnOffsetCommitByGroupMetadata() { 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 void testUnknownMemberIdInTxnOffsetCommitByGroupMetadata() { 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 void testIllegalGenerationInTxnOffsetCommitByGroupMetadata() { 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 @@ private void prepareInitPidResponse(Errors error, boolean shouldDisconnect, long .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 @@ private MockClient.RequestMatcher addPartitionsRequestMatcher(final TopicPartiti 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 @@ private void sendEndTxnResponse(Errors error, final TransactionResult result, fi 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 @@ private void prepareAddOffsetsToTxnResponse(final Errors error, 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 @@ private void prepareTxnOffsetCommitResponse(final String consumerGroupId, 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 @@ private void prepareTxnOffsetCommitResponse(final String consumerGroupId, 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 @@ private void initializeIdempotentProducerId(long producerId, short epoch) { .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 86e6bd7595feb..1758a978f0b2b 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 void testConstructor() { 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 98b0c149ce11b..0178b1e5eac15 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 @@ -32,17 +32,43 @@ public class ApiVersionsResponseTest { + @Test + public void shouldCreateApiResponseOnlyWithKeysSupportedByMagicValue() { + final ApiVersionsResponse response = ApiVersionsResponse.apiVersionsResponse( + 10, + RecordBatch.MAGIC_VALUE_V1, + Features.emptySupportedFeatures()); + verifyApiKeysForMagic(response, RecordBatch.MAGIC_VALUE_V1); + assertEquals(10, response.throttleTimeMs()); + assertTrue(response.data().supportedFeatures().isEmpty()); + assertTrue(response.data().finalizedFeatures().isEmpty()); + assertEquals(ApiVersionsResponse.UNKNOWN_FINALIZED_FEATURES_EPOCH, response.data().finalizedFeaturesEpoch()); + } + @Test public void 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 shouldReturnAllKeysWhenMagicIsCurrentValueAndThrottleMsIsDefaultThrottle() { + ApiVersionsResponse response = ApiVersionsResponse.apiVersionsResponse( + AbstractResponse.DEFAULT_THROTTLE_TIME, + RecordBatch.CURRENT_MAGIC_VALUE, + Features.emptySupportedFeatures()); + assertEquals(new HashSet<>(ApiKeys.enabledApis()), apiKeysInResponse(response)); + assertEquals(AbstractResponse.DEFAULT_THROTTLE_TIME, response.throttleTimeMs()); + assertTrue(response.data().supportedFeatures().isEmpty()); + assertTrue(response.data().finalizedFeatures().isEmpty()); + assertEquals(ApiVersionsResponse.UNKNOWN_FINALIZED_FEATURES_EPOCH, 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 +90,49 @@ public void shouldHaveCorrectDefaultApiVersionsResponse() { } } - 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()); } +<<<<<<< HEAD +======= + @Test + public void shouldReturnFeatureKeysWhenMagicIsCurrentValueAndThrottleMsIsDefaultThrottle() { + ApiVersionsResponse response = ApiVersionsResponse.apiVersionsResponse( + 10, + RecordBatch.MAGIC_VALUE_V1, + Features.supportedFeatures(Utils.mkMap(Utils.mkEntry("feature", new SupportedVersionRange((short) 1, (short) 4)))), + Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry("feature", new FinalizedVersionRange((short) 2, (short) 3)))), + 10); + verifyApiKeysForMagic(response, RecordBatch.MAGIC_VALUE_V1); + assertEquals(10, response.throttleTimeMs()); + + assertEquals(1, response.data().supportedFeatures().size()); + SupportedFeatureKey sKey = response.data().supportedFeatures().find("feature"); + assertNotNull(sKey); + assertEquals(1, sKey.minVersion()); + assertEquals(4, sKey.maxVersion()); + + assertEquals(1, response.data().finalizedFeatures().size()); + FinalizedFeatureKey fKey = response.data().finalizedFeatures().find("feature"); + assertNotNull(fKey); + assertEquals(2, fKey.minVersionLevel()); + assertEquals(3, fKey.maxVersionLevel()); + + assertEquals(10, response.data().finalizedFeaturesEpoch()); + } + + private void verifyApiKeysForMagic(final ApiVersionsResponse response, final byte maxMagic) { + for (final ApiVersionsResponseKey version : response.data().apiKeys()) { + assertTrue(ApiKeys.forId(version.apiKey()).minRequiredInterBrokerMagic <= maxMagic); + } + } +>>>>>>> fd978a9d20... add data accessor and creat tests for manually created data 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 b022a52e7ffc9..06e611f49a6da 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 void testStructBuild() { 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 void testNullableMetadata() { .setMetadata(null)) )) ); - assertEquals(expectedData, response.data); + assertEquals(expectedData, response.data()); } @Test @@ -226,6 +226,6 @@ public void testUseDefaultLeaderEpoch() { .setMetadata(metadata)) )) ); - assertEquals(expectedData, response.data); + assertEquals(expectedData, response.data()); } } 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 c6595b5328c8c..2ebc3c4b9cbe3 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 @@ -937,9 +937,9 @@ public void testApiVersionResponseWithUnsupportedError() { 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 void testApiVersionResponseWithNotUnsupportedError() { 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 void testApiVersionResponseParsingFallback() { 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 void testApiVersionResponseParsing() { 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 204753e435156..98da6bcd9f44f 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 void testEnableGroupMetadataAutoDowngrade() { 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 2375a9b9996a2..34ee542a5c6cd 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 void testApiVersionsRequestWithServerUnsupportedVersion() throws Exceptio 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 void testInvalidApiVersionsRequest() throws Exception { 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 void testValidApiVersionsRequest() throws Exception { 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 @@ protected SaslServerAuthenticator buildServerAuthenticator(Map config 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 @@ private SaslHandshakeResponse sendHandshakeRequestReceiveResponse(String node, s 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/core/src/main/scala/kafka/network/RequestConvertToJson.scala b/core/src/main/scala/kafka/network/RequestConvertToJson.scala index 1e9f26de4889f..65633cc8d7c4f 100644 --- a/core/src/main/scala/kafka/network/RequestConvertToJson.scala +++ b/core/src/main/scala/kafka/network/RequestConvertToJson.scala @@ -17,10 +17,13 @@ package kafka.network +import java.util + import com.fasterxml.jackson.databind.JsonNode -import com.fasterxml.jackson.databind.node.{ArrayNode, DoubleNode, IntNode, JsonNodeFactory, LongNode, NullNode, ObjectNode, ShortNode, TextNode} +import com.fasterxml.jackson.databind.node.{ArrayNode, BinaryNode, DoubleNode, IntNode, JsonNodeFactory, LongNode, NullNode, ObjectNode, ShortNode, TextNode} import kafka.network.RequestChannel.{Response, Session} import org.apache.kafka.common.message._ +import org.apache.kafka.common.network.ClientInformation import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record.RecordBatch import org.apache.kafka.common.requests._ @@ -31,64 +34,65 @@ import scala.jdk.CollectionConverters._ object RequestConvertToJson { def request(request: AbstractRequest, verbose: Boolean): 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: 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 => offsetsForLeaderEpochRequestNode(req, request.version()) - case req: ProduceRequest => produceRequestNode(req, request.version(), verbose) - 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 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: 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 => offsetsForLeaderEpochRequestNode(req, request.version) + case req: ProduceRequest => produceRequestNode(req, request.version, verbose) + 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 Exception("Unexpected request type encountered: " + request) } } @@ -96,93 +100,101 @@ object RequestConvertToJson { 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: 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: 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: 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: 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: ExpireDelegationTokenResponse => ExpireDelegationTokenResponseDataJsonConverter.write(res.data(), version) - case res: FetchResponse[_] => FetchResponseDataJsonConverter.write(res.data(), version) - 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: ExpireDelegationTokenResponse => ExpireDelegationTokenResponseDataJsonConverter.write(res.data, version) + case res: FetchResponse[_] => FetchResponseDataJsonConverter.write(res.data, version) + 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: 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: 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 => offsetsForLeaderEpochResponseNode(res, version) case res: ProduceResponse => produceResponseNode(res, 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: 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: 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 Exception("Unexpected response type encountered: " + response) } } def requestHeaderNode(header: RequestHeader): JsonNode = { val node = RequestHeaderDataJsonConverter.write(header.data(), header.headerVersion()).asInstanceOf[ObjectNode] - node.set("requestApiKey", new TextNode(header.apiKey.toString)) + node.set("requestApiKeyName", new TextNode(header.apiKey.toString)) + node + } + + def clientInfoNode(clientInfo: ClientInformation): JsonNode = { + val node = new ObjectNode(JsonNodeFactory.instance) + node.set("softwareName", new TextNode(clientInfo.softwareName())) + node.set("softwareName", new TextNode(clientInfo.softwareVersion())) node } def requestDescMetrics(header: RequestHeader, res: Response, req: AbstractRequest, - context: RequestContext, session: Session, verbose: Boolean, - totalTimeMs: Double, requestQueueTimeMs: Double, apiLocalTimeMs: Double, - apiRemoteTimeMs: Double, apiThrottleTimeMs: Long, responseQueueTimeMs: Double, - responseSendTimeMs: Double, temporaryMemoryBytes: Long, - messageConversionsTimeMs: Double): JsonNode = { + context: RequestContext, session: Session, verbose: Boolean, + totalTimeMs: Double, requestQueueTimeMs: Double, apiLocalTimeMs: Double, + apiRemoteTimeMs: Double, apiThrottleTimeMs: Long, responseQueueTimeMs: Double, + responseSendTimeMs: Double, temporaryMemoryBytes: Long, + messageConversionsTimeMs: Double): JsonNode = { val node = new ObjectNode(JsonNodeFactory.instance) node.set("requestHeader", requestHeaderNode(header)) node.set("request", request(req, verbose)) node.set("response", res.responseLog.getOrElse(new TextNode(""))) node.set("connection", new TextNode(context.connectionId)) - node.set("totalTime", new DoubleNode(totalTimeMs)) - node.set("requestQueueTime", new DoubleNode(requestQueueTimeMs)) - node.set("localTime", new DoubleNode(apiLocalTimeMs)) - node.set("remoteTime", new DoubleNode(apiRemoteTimeMs)) - node.set("throttleTime", new LongNode(apiThrottleTimeMs)) - node.set("responseQueueTime", new DoubleNode(responseQueueTimeMs)) - node.set("sendTime", new DoubleNode(responseSendTimeMs)) + 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", new TextNode(context.clientInformation.toString)) + node.set("clientInformation", clientInfoNode(context.clientInformation)) if (temporaryMemoryBytes > 0) node.set("temporaryMemoryBytes", new LongNode(temporaryMemoryBytes)) if (messageConversionsTimeMs > 0) @@ -205,25 +217,24 @@ object RequestConvertToJson { if (version >= 3) { node.set("replicaId", new IntNode(request.replicaId)) } - val topicsToPartitionEpochs = CollectionUtils.groupPartitionDataByTopic(request.epochsByTopicPartition) + val topics = CollectionUtils.groupPartitionDataByTopic(request.epochsByTopicPartition) val topicsArray = new ArrayNode(JsonNodeFactory.instance) - for (topicToEpochs <- topicsToPartitionEpochs.entrySet.asScala) { - val topicsData = new ObjectNode(JsonNodeFactory.instance) - topicsData.set("name", new TextNode(topicToEpochs.getKey)) + topics.forEach { (topicName, partitions) => + val topicNode = new ObjectNode(JsonNodeFactory.instance) + topicNode.set("name", new TextNode(topicName)) val partitionsArray = new ArrayNode(JsonNodeFactory.instance) - for (partitionEpoch <- topicToEpochs.getValue.entrySet.asScala) { - val partitionData = partitionEpoch.getValue + partitions.forEach { (partitionIndex, partitionData) => val partitionNode = new ObjectNode(JsonNodeFactory.instance) - partitionNode.set("partitionIndex", new IntNode(partitionEpoch.getKey)) - partitionNode.set("leaderEpoch", new IntNode(partitionData.leaderEpoch)) + partitionNode.set("partitionIndex", new IntNode(partitionIndex)) if (version >= 2) { val leaderEpoch = partitionData.currentLeaderEpoch partitionNode.set("currentLeaderEpoch", new IntNode(leaderEpoch.orElse(RecordBatch.NO_PARTITION_LEADER_EPOCH))) } + partitionNode.set("leaderEpoch", new IntNode(partitionData.leaderEpoch)) partitionsArray.add(partitionNode) } - topicsData.set("partitions", partitionsArray) - topicsArray.add(topicsData) + topicNode.set("partitions", partitionsArray) + topicsArray.add(topicNode) } node.set("topics", topicsArray) node @@ -243,19 +254,25 @@ object RequestConvertToJson { } node.set("acks", new ShortNode(request.acks)) node.set("timeoutMs", new IntNode(request.timeout)) - if (verbose) { - val partSizes = new ArrayNode(JsonNodeFactory.instance) - for (partSize <- request.partitionSizes().entrySet.asScala) { - val part = new ObjectNode(JsonNodeFactory.instance) - val topic = partSize.getKey - part.set("partition", new TextNode(topic.toString)) - part.set("size", new IntNode(partSize.getValue)) - partSizes.add(part) + val topics = CollectionUtils.groupPartitionDataByTopic(request.partitionRecordsOrFail()) + val topicsArray = new ArrayNode(JsonNodeFactory.instance) + topics.forEach { (topicName, partitions) => + val topicNode = new ObjectNode(JsonNodeFactory.instance) + topicNode.set("name", new TextNode(topicName)) + val partitionsArray = new ArrayNode(JsonNodeFactory.instance) + partitions.forEach { (partitionIndex, partitionData)=> + val partitionNode = new ObjectNode(JsonNodeFactory.instance) + partitionNode.set("partitionIndex", new IntNode(partitionIndex)) + if (partitionData == null) + partitionNode.set("records", NullNode.instance) + else + partitionNode.set("records", new BinaryNode(util.Arrays.copyOf(partitionData.buffer().array(), partitionData.validBytes()))) + partitionsArray.add(partitionNode) } - node.set("partitionSizes", partSizes) - } else { - node.set("partitionSizes", new IntNode(request.partitionSizes.size)) + topicNode.set("partitions", partitionsArray) + topicsArray.add(topicNode) } + node.set("topics", topicsArray) node } @@ -267,23 +284,22 @@ object RequestConvertToJson { if (version >= 2) { node.set("throttleTimeMs", new IntNode(response.throttleTimeMs)) } - val endOffsetsByTopic = CollectionUtils.groupPartitionDataByTopic(response.responses) + val topics = CollectionUtils.groupPartitionDataByTopic(response.responses) val topicsArray = new ArrayNode(JsonNodeFactory.instance) - for (topicToPartitionEpochs <- endOffsetsByTopic.entrySet.asScala) { - val topicData = new ObjectNode(JsonNodeFactory.instance) - topicData.set("name", new TextNode(topicToPartitionEpochs.getKey)) + topics.forEach { (topicName, partitions) => + val topicNode = new ObjectNode(JsonNodeFactory.instance) + topicNode.set("name", new TextNode(topicName)) val partitionsArray = new ArrayNode(JsonNodeFactory.instance) - for (partitionEndOffset <- topicToPartitionEpochs.getValue.entrySet.asScala) { - val epochEndOffset = partitionEndOffset.getValue - val partitionData = new ObjectNode(JsonNodeFactory.instance) - partitionData.set("errorCode", new ShortNode(epochEndOffset.error.code)) - partitionData.set("partitionIndex", new IntNode(partitionEndOffset.getKey)) - if (version >= 1) partitionData.set("leaderEpoch", new IntNode(epochEndOffset.leaderEpoch)) - partitionData.set("endOffset", new LongNode(epochEndOffset.endOffset)) - partitionsArray.add(partitionData) + partitions.forEach { (partitionIndex, partitionData) => + val partitionNode = new ObjectNode(JsonNodeFactory.instance) + partitionNode.set("errorCode", new ShortNode(partitionData.error.code)) + partitionNode.set("partitionIndex", new IntNode(partitionIndex)) + if (version >= 1) partitionNode.set("leaderEpoch", new IntNode(partitionData.leaderEpoch)) + partitionNode.set("endOffset", new LongNode(partitionData.endOffset)) + partitionsArray.add(partitionNode) } - topicData.set("partitions", partitionsArray) - topicsArray.add(topicData) + topicNode.set("partitions", partitionsArray) + topicsArray.add(topicNode) } node.set("topics", topicsArray) node @@ -294,45 +310,44 @@ object RequestConvertToJson { */ def produceResponseNode(response: ProduceResponse, version: Short): JsonNode = { val node = new ObjectNode(JsonNodeFactory.instance) - val responseByTopic = CollectionUtils.groupPartitionDataByTopic(response.responses) + val topics = CollectionUtils.groupPartitionDataByTopic(response.responses) val responsesArray = new ArrayNode(JsonNodeFactory.instance) - for (entry <- responseByTopic.entrySet.asScala) { - val topicData = new ObjectNode(JsonNodeFactory.instance) - topicData.set("name", new TextNode(entry.getKey)) + topics.forEach { (topicName, partitions) => + val topicNode = new ObjectNode(JsonNodeFactory.instance) + topicNode.set("name", new TextNode(topicName)) val partitionsArray = new ArrayNode(JsonNodeFactory.instance) - for (partitionEntry <- entry.getValue.entrySet.asScala) { - val part = partitionEntry.getValue - val partitionData = new ObjectNode(JsonNodeFactory.instance) - partitionData.set("partitionIndex", new IntNode(partitionEntry.getKey)) - var errorCode = part.error.code + partitions.forEach { (partitionIndex, partitionData) => + val partitionNode = new ObjectNode(JsonNodeFactory.instance) + partitionNode.set("partitionIndex", new IntNode(partitionIndex)) + var errorCode = partitionData.error.code if (errorCode == Errors.KAFKA_STORAGE_ERROR.code && version <= 3) { errorCode = Errors.NOT_LEADER_OR_FOLLOWER.code } - partitionData.set("errorCode", new ShortNode(errorCode)) - partitionData.set("baseOffset", new LongNode(part.baseOffset)) + partitionNode.set("errorCode", new ShortNode(errorCode)) + partitionNode.set("baseOffset", new LongNode(partitionData.baseOffset)) if (version >= 2) { - partitionData.set("logAppendTimeMs", new LongNode(part.logAppendTime)) + partitionNode.set("logAppendTimeMs", new LongNode(partitionData.logAppendTime)) } if (version >= 5) { - partitionData.set("logStartOffset", new LongNode(part.logStartOffset)) + partitionNode.set("logStartOffset", new LongNode(partitionData.logStartOffset)) } if (version >= 8) { val recordErrorsArray = new ArrayNode(JsonNodeFactory.instance) - for (indexAndMessage <- part.recordErrors.asScala) { + for (indexAndMessage <- partitionData.recordErrors.asScala) { val indexAndMessageData = new ObjectNode(JsonNodeFactory.instance) indexAndMessageData.set("batchIndex", new IntNode(indexAndMessage.batchIndex)) if (indexAndMessage.message == null) indexAndMessageData.set("batchIndexErrorMessage", NullNode.instance) else indexAndMessageData.set("batchIndexErrorMessage", new TextNode(indexAndMessage.message)) recordErrorsArray.add(indexAndMessageData) } - partitionData.set("recordErrors", recordErrorsArray) - if (part.errorMessage == null) partitionData.set("errorMessage", NullNode.instance) - else partitionData.set("errorMessage", new TextNode(part.errorMessage)) + partitionNode.set("recordErrors", recordErrorsArray) + if (partitionData.errorMessage == null) partitionNode.set("errorMessage", NullNode.instance) + else partitionNode.set("errorMessage", new TextNode(partitionData.errorMessage)) } - partitionsArray.add(partitionData) + partitionsArray.add(partitionNode) } - topicData.set("partitions", partitionsArray) - responsesArray.add(topicData) + topicNode.set("partitions", partitionsArray) + responsesArray.add(topicNode) } node.set("responses", responsesArray) if (version >= 1) { diff --git a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala index b73900736b42d..dce310787d139 100644 --- a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala +++ b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala @@ -17,155 +17,44 @@ package kafka.network -import java.util.HashMap +import java.nio.ByteBuffer +import java.util +import java.util.{Collections, Optional} import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.message._ import org.junit.Test -import org.apache.kafka.common.protocol.ApiKeys -import org.apache.kafka.common.record.MemoryRecords +import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch, TimestampType} +import org.apache.kafka.common.requests.OffsetsForLeaderEpochRequest.PartitionData import org.apache.kafka.common.requests._ import scala.collection.mutable.ArrayBuffer class RequestConvertToJsonTest { - def createRequestsFromApiKey(apiKey: ApiKeys, version: Short): AbstractRequest = apiKey match { - case ApiKeys.PRODUCE => ProduceRequest.Builder.forCurrentMagic(0.toShort, 10000, new HashMap[TopicPartition, MemoryRecords]()).build() - case ApiKeys.FETCH => new FetchRequest(new FetchRequestData(), version) - case ApiKeys.LIST_OFFSETS => new ListOffsetRequest(new ListOffsetRequestData().toStruct(version), version) - case ApiKeys.METADATA => new MetadataRequest(new MetadataRequestData(), version) - case ApiKeys.OFFSET_COMMIT => new OffsetCommitRequest(new OffsetCommitRequestData(), version) - case ApiKeys.OFFSET_FETCH => new OffsetFetchRequest(new OffsetFetchRequestData().toStruct(version), version) - case ApiKeys.FIND_COORDINATOR => new FindCoordinatorRequest(new FindCoordinatorRequestData().toStruct(version), version) - case ApiKeys.JOIN_GROUP => new JoinGroupRequest(new JoinGroupRequestData(), version) - case ApiKeys.HEARTBEAT => new HeartbeatRequest(new HeartbeatRequestData().toStruct(version), version) - case ApiKeys.LEAVE_GROUP => new LeaveGroupRequest(new LeaveGroupRequestData().toStruct(version), version) - case ApiKeys.SYNC_GROUP => new SyncGroupRequest(new SyncGroupRequestData(), version) - case ApiKeys.STOP_REPLICA => new StopReplicaRequest(new StopReplicaRequestData().toStruct(version), version) - case ApiKeys.CONTROLLED_SHUTDOWN => new ControlledShutdownRequest(new ControlledShutdownRequestData().toStruct(version), version) - case ApiKeys.UPDATE_METADATA => new UpdateMetadataRequest(new UpdateMetadataRequestData().toStruct(version), version) - case ApiKeys.LEADER_AND_ISR => new LeaderAndIsrRequest(new LeaderAndIsrRequestData().toStruct(version), version) - case ApiKeys.DESCRIBE_GROUPS => new DescribeGroupsRequest(new DescribeGroupsRequestData().toStruct(version), version) - case ApiKeys.LIST_GROUPS => new ListGroupsRequest(new ListGroupsRequestData(), version) - case ApiKeys.SASL_HANDSHAKE => new SaslHandshakeRequest(new SaslHandshakeRequestData()) - case ApiKeys.API_VERSIONS => new ApiVersionsRequest(new ApiVersionsRequestData(), version) - case ApiKeys.CREATE_TOPICS => new CreateTopicsRequest(new CreateTopicsRequestData().toStruct(version), version) - case ApiKeys.DELETE_TOPICS => new DeleteTopicsRequest(new DeleteTopicsRequestData().toStruct(version), version) - case ApiKeys.DELETE_RECORDS => new DeleteRecordsRequest(new DeleteRecordsRequestData().toStruct(version), version) - case ApiKeys.INIT_PRODUCER_ID => new InitProducerIdRequest(new InitProducerIdRequestData().toStruct(version), version) - case ApiKeys.OFFSET_FOR_LEADER_EPOCH => new OffsetsForLeaderEpochRequest(new OffsetForLeaderEpochResponseData().toStruct(version), version) - case ApiKeys.ADD_PARTITIONS_TO_TXN => new AddPartitionsToTxnRequest(new AddPartitionsToTxnRequestData(), version) - case ApiKeys.ADD_OFFSETS_TO_TXN => new AddOffsetsToTxnRequest(new AddOffsetsToTxnRequestData(), version) - case ApiKeys.END_TXN => new EndTxnRequest(new EndTxnRequestData().toStruct(version), version) - case ApiKeys.WRITE_TXN_MARKERS => new WriteTxnMarkersRequest(new WriteTxnMarkersRequestData().toStruct(version), version) - case ApiKeys.TXN_OFFSET_COMMIT => new TxnOffsetCommitRequest(new TxnOffsetCommitRequestData(), version) - case ApiKeys.DESCRIBE_ACLS => - val data = new DescribeAclsRequestData().setResourceTypeFilter(1).setOperation(2).setPermissionType(2) - new DescribeAclsRequest(data.toStruct(version), version) - case ApiKeys.CREATE_ACLS => new CreateAclsRequest(new CreateAclsRequestData().toStruct(version), version) - case ApiKeys.DELETE_ACLS => new DeleteAclsRequest(new DeleteAclsRequestData().toStruct(version), version) - case ApiKeys.DESCRIBE_CONFIGS => new DescribeConfigsRequest(new DescribeConfigsRequestData(), version) - case ApiKeys.ALTER_CONFIGS => new AlterConfigsRequest(new AlterConfigsRequestData(), version) - case ApiKeys.ALTER_REPLICA_LOG_DIRS => new AlterReplicaLogDirsRequest(new AlterReplicaLogDirsRequestData(), version) - case ApiKeys.DESCRIBE_LOG_DIRS => new DescribeLogDirsRequest(new DescribeLogDirsRequestData(), version) - case ApiKeys.SASL_AUTHENTICATE => new SaslAuthenticateRequest(new SaslAuthenticateRequestData(), version) - case ApiKeys.CREATE_PARTITIONS => new CreatePartitionsRequest(new CreatePartitionsRequestData().toStruct(version), version) - case ApiKeys.CREATE_DELEGATION_TOKEN => new CreateDelegationTokenRequest(new CreateDelegationTokenRequestData().toStruct(version), version) - case ApiKeys.RENEW_DELEGATION_TOKEN => new RenewDelegationTokenRequest(new RenewDelegationTokenRequestData(), version) - case ApiKeys.EXPIRE_DELEGATION_TOKEN => new ExpireDelegationTokenRequest(new ExpireDelegationTokenRequestData().toStruct(version), version) - case ApiKeys.DESCRIBE_DELEGATION_TOKEN => new DescribeDelegationTokenRequest(new DescribeDelegationTokenRequestData(), version) - case ApiKeys.DELETE_GROUPS => new DeleteGroupsRequest(new DeleteGroupsRequestData(), version) - case ApiKeys.ELECT_LEADERS => new ElectLeadersRequest(new ElectLeadersRequestData().toStruct(version), version) - case ApiKeys.INCREMENTAL_ALTER_CONFIGS => new IncrementalAlterConfigsRequest.Builder(new IncrementalAlterConfigsRequestData()).build(version) - case ApiKeys.ALTER_PARTITION_REASSIGNMENTS => new AlterPartitionReassignmentsRequest.Builder(new AlterPartitionReassignmentsRequestData()).build(version) - case ApiKeys.LIST_PARTITION_REASSIGNMENTS => new ListPartitionReassignmentsRequest.Builder(new ListPartitionReassignmentsRequestData()).build(version) - case ApiKeys.OFFSET_DELETE => new OffsetDeleteRequest(new OffsetDeleteRequestData(), version) - case ApiKeys.DESCRIBE_CLIENT_QUOTAS => new DescribeClientQuotasRequest(new DescribeClientQuotasRequestData(), version) - case ApiKeys.ALTER_CLIENT_QUOTAS => new AlterClientQuotasRequest(new AlterClientQuotasRequestData(), version) - case ApiKeys.DESCRIBE_USER_SCRAM_CREDENTIALS => new DescribeUserScramCredentialsRequest.Builder(new DescribeUserScramCredentialsRequestData()).build(version) - case ApiKeys.ALTER_USER_SCRAM_CREDENTIALS => new AlterUserScramCredentialsRequest.Builder(new AlterUserScramCredentialsRequestData()).build(version) - case ApiKeys.VOTE => new VoteRequest.Builder(new VoteRequestData()).build(version) - case ApiKeys.BEGIN_QUORUM_EPOCH => new BeginQuorumEpochRequest.Builder(new BeginQuorumEpochRequestData()).build(version) - case ApiKeys.END_QUORUM_EPOCH => new EndQuorumEpochRequest.Builder(new EndQuorumEpochRequestData()).build(version) - case ApiKeys.DESCRIBE_QUORUM => new DescribeQuorumRequest.Builder(new DescribeQuorumRequestData()).build(version) - case ApiKeys.ALTER_ISR => new AlterIsrRequest.Builder(new AlterIsrRequestData()).build(version) - case ApiKeys.UPDATE_FEATURES => new UpdateFeaturesRequest.Builder(new UpdateFeaturesRequestData()).build(version) - case _ => throw new AssertionError(String.format("Request type %s is not tested in `RequestConvertToJsonTest`", apiKey)) - } - - def createResponseFromApiKey(apiKey: ApiKeys, version: Short): AbstractResponse = apiKey match { - case ApiKeys.PRODUCE => new ProduceResponse(new ProduceResponseData().toStruct(version)) - case ApiKeys.FETCH => new FetchResponse(new FetchResponseData()) - case ApiKeys.LIST_OFFSETS => new ListOffsetResponse(new ListOffsetResponseData()) - case ApiKeys.METADATA => new MetadataResponse(new MetadataResponseData()) - case ApiKeys.OFFSET_COMMIT => new OffsetCommitResponse(new OffsetCommitResponseData()) - case ApiKeys.OFFSET_FETCH => new OffsetFetchResponse(new OffsetFetchResponseData().toStruct(version), version) - case ApiKeys.FIND_COORDINATOR => new FindCoordinatorResponse(new FindCoordinatorResponseData()) - case ApiKeys.JOIN_GROUP => new JoinGroupResponse(new JoinGroupResponseData()) - case ApiKeys.HEARTBEAT => new HeartbeatResponse(new HeartbeatResponseData()) - case ApiKeys.LEAVE_GROUP => new LeaveGroupResponse(new LeaveGroupResponseData()) - case ApiKeys.SYNC_GROUP => new SyncGroupResponse(new SyncGroupResponseData()) - case ApiKeys.STOP_REPLICA => new StopReplicaResponse(new StopReplicaResponseData()) - case ApiKeys.CONTROLLED_SHUTDOWN => new ControlledShutdownResponse(new ControlledShutdownResponseData()) - case ApiKeys.UPDATE_METADATA => new UpdateMetadataResponse(new UpdateMetadataResponseData()) - case ApiKeys.LEADER_AND_ISR => new LeaderAndIsrResponse(new LeaderAndIsrResponseData()) - case ApiKeys.DESCRIBE_GROUPS => new DescribeGroupsResponse(new DescribeGroupsResponseData()) - case ApiKeys.LIST_GROUPS => new ListGroupsResponse(new ListGroupsResponseData()) - case ApiKeys.SASL_HANDSHAKE => new SaslHandshakeResponse(new SaslHandshakeResponseData()) - case ApiKeys.API_VERSIONS => new ApiVersionsResponse(new ApiVersionsResponseData()) - case ApiKeys.CREATE_TOPICS => new CreateTopicsResponse(new CreateTopicsResponseData()) - case ApiKeys.DELETE_TOPICS => new DeleteTopicsResponse(new DeleteTopicsResponseData()) - case ApiKeys.DELETE_RECORDS => new DeleteRecordsResponse(new DeleteRecordsResponseData()) - case ApiKeys.INIT_PRODUCER_ID => new InitProducerIdResponse(new InitProducerIdResponseData()) - case ApiKeys.OFFSET_FOR_LEADER_EPOCH => new OffsetsForLeaderEpochResponse(new OffsetForLeaderEpochResponseData().toStruct(version)) - case ApiKeys.ADD_PARTITIONS_TO_TXN => new AddPartitionsToTxnResponse(new AddPartitionsToTxnResponseData().toStruct(version), version) - case ApiKeys.ADD_OFFSETS_TO_TXN => new AddOffsetsToTxnResponse(new AddOffsetsToTxnResponseData()) - case ApiKeys.END_TXN => new EndTxnResponse(new EndTxnResponseData()) - case ApiKeys.WRITE_TXN_MARKERS => new WriteTxnMarkersResponse(new WriteTxnMarkersResponseData().toStruct(version), version) - case ApiKeys.TXN_OFFSET_COMMIT => new TxnOffsetCommitResponse(new TxnOffsetCommitResponseData()) - case ApiKeys.DESCRIBE_ACLS => new DescribeAclsResponse(new DescribeAclsResponseData()) - case ApiKeys.CREATE_ACLS => new CreateAclsResponse(new CreateAclsResponseData()) - case ApiKeys.DELETE_ACLS => new DeleteAclsResponse(new DeleteAclsResponseData()) - case ApiKeys.DESCRIBE_CONFIGS => new DescribeConfigsResponse(new DescribeConfigsResponseData()) - case ApiKeys.ALTER_CONFIGS => new AlterConfigsResponse(new AlterConfigsResponseData()) - case ApiKeys.ALTER_REPLICA_LOG_DIRS => new AlterReplicaLogDirsResponse(new AlterReplicaLogDirsResponseData()) - case ApiKeys.DESCRIBE_LOG_DIRS => new DescribeLogDirsResponse(new DescribeLogDirsResponseData()) - case ApiKeys.SASL_AUTHENTICATE => new SaslAuthenticateResponse(new SaslAuthenticateResponseData()) - case ApiKeys.CREATE_PARTITIONS => new CreatePartitionsResponse(new CreatePartitionsResponseData()) - case ApiKeys.CREATE_DELEGATION_TOKEN => new CreateDelegationTokenResponse(new CreateDelegationTokenResponseData()) - case ApiKeys.RENEW_DELEGATION_TOKEN => new RenewDelegationTokenResponse(new RenewDelegationTokenResponseData()) - case ApiKeys.EXPIRE_DELEGATION_TOKEN => new ExpireDelegationTokenResponse(new ExpireDelegationTokenResponseData()) - case ApiKeys.DESCRIBE_DELEGATION_TOKEN => new DescribeDelegationTokenResponse(new DescribeDelegationTokenResponseData().toStruct(version), version) - case ApiKeys.DELETE_GROUPS => new DeleteGroupsResponse(new DeleteGroupsResponseData()) - case ApiKeys.ELECT_LEADERS => new ElectLeadersResponse(new ElectLeadersResponseData().toStruct(version), version) - case ApiKeys.INCREMENTAL_ALTER_CONFIGS => new IncrementalAlterConfigsResponse(new IncrementalAlterConfigsResponseData()) - case ApiKeys.ALTER_PARTITION_REASSIGNMENTS => new AlterPartitionReassignmentsResponse(new AlterPartitionReassignmentsResponseData()) - case ApiKeys.LIST_PARTITION_REASSIGNMENTS => new ListPartitionReassignmentsResponse(new ListPartitionReassignmentsResponseData()) - case ApiKeys.OFFSET_DELETE => new OffsetDeleteResponse(new OffsetDeleteResponseData()) - case ApiKeys.DESCRIBE_CLIENT_QUOTAS => new DescribeClientQuotasResponse(new DescribeClientQuotasResponseData().toStruct(version), version) - case ApiKeys.ALTER_CLIENT_QUOTAS => new AlterClientQuotasResponse(new AlterClientQuotasResponseData().toStruct(version), version) - case ApiKeys.DESCRIBE_USER_SCRAM_CREDENTIALS => new DescribeUserScramCredentialsResponse(new DescribeUserScramCredentialsResponseData()) - case ApiKeys.ALTER_USER_SCRAM_CREDENTIALS => new AlterUserScramCredentialsResponse(new AlterUserScramCredentialsResponseData()) - case ApiKeys.VOTE => new VoteResponse(new VoteResponseData()) - case ApiKeys.BEGIN_QUORUM_EPOCH => new BeginQuorumEpochResponse(new BeginQuorumEpochResponseData()) - case ApiKeys.END_QUORUM_EPOCH => new EndQuorumEpochResponse(new EndQuorumEpochResponseData()) - case ApiKeys.DESCRIBE_QUORUM => new DescribeQuorumResponse(new DescribeQuorumResponseData()) - case ApiKeys.ALTER_ISR => new AlterIsrResponse(new AlterIsrResponseData()) - case ApiKeys.UPDATE_FEATURES => new UpdateFeaturesResponse(new UpdateFeaturesResponseData()) - case _ => throw new AssertionError(String.format("Response type %s not tested in `RequestConvertToJsonTest`", apiKey)) - } - @Test def testAllRequestTypesHandled(): Unit = { val unhandledKeys = ArrayBuffer[String]() ApiKeys.values().foreach(key => { val version: Short = 0 - val req = createRequestsFromApiKey(key, version) + var req: AbstractRequest = null + if (key == ApiKeys.PRODUCE) { + // There's inconsistency with the toStruct schema in ProduceRequest + // and ProduceRequestDataJsonConverters where the field names don't + // match so the struct does not have the correct field names. This is + // a temporary workaround until ProduceRequest starts using ProduceRequestData + req = ProduceRequest.Builder.forCurrentMagic(0.toShort, 10000, new util.HashMap[TopicPartition, MemoryRecords]()).build() + } else if (key == ApiKeys.DESCRIBE_QUORUM) { + req = new DescribeQuorumRequest(new DescribeQuorumRequestData().toStruct(version), version) + } else { + val struct = ApiMessageType.fromApiKey(key.id).newRequest().toStruct(version) + req = AbstractRequest.parseRequest(key, version, struct) + } try { RequestConvertToJson.request(req, false) } catch { - case _ : Throwable => unhandledKeys += key.toString + case _ : AssertionError => unhandledKeys += key.toString } }) assert(unhandledKeys.isEmpty, String.format("%s request keys not handled in RequestConvertToJson", unhandledKeys)) @@ -176,13 +65,78 @@ class RequestConvertToJsonTest { val unhandledKeys = ArrayBuffer[String]() ApiKeys.values().foreach(key => { val version: Short = 0 - val res = createResponseFromApiKey(key, version) + val struct = ApiMessageType.fromApiKey(key.id).newResponse().toStruct(version) + val res = AbstractResponse.parseResponse(key, struct, version) try { RequestConvertToJson.response(res, version) } catch { - case _ : Throwable => unhandledKeys += key.toString + case _ : AssertionError => unhandledKeys += key.toString } }) assert(unhandledKeys.isEmpty, String.format("%s response keys not handled in RequestConvertToJson", unhandledKeys)) } + + @Test + def testFormatOfOffsetsForLeaderEpochRequestNode(): Unit = { + val partitionDataMap = new util.HashMap[TopicPartition, PartitionData] + partitionDataMap.put(new TopicPartition("topic1", 0), new PartitionData(Optional.of(0), 1)) + + val version: Short = 3 + val request = OffsetsForLeaderEpochRequest.Builder.forConsumer(partitionDataMap).build(version) + val manualGenNode = RequestConvertToJson.request(request, false) + + val requestData = OffsetForLeaderEpochRequestDataJsonConverter.read(manualGenNode, version) + val autoGenNode = OffsetForLeaderEpochRequestDataJsonConverter.write(requestData, version) + + assert(manualGenNode.equals(autoGenNode)) + } + + @Test + def testFormatOfProduceRequestNode(): Unit = { + val buffer = ByteBuffer.allocate(256) + val builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, TimestampType.CREATE_TIME, 0L) + builder.append(10L, null, "a".getBytes) + val produceData = new util.HashMap[TopicPartition, MemoryRecords] + produceData.put(new TopicPartition("test", 0), builder.build) + + val version: Short = 3 + val request = ProduceRequest.Builder.forCurrentMagic(1.toShort, 5000, produceData).build(version) + val manualGenNode = RequestConvertToJson.request(request, false) + + val requestData = ProduceRequestDataJsonConverter.read(manualGenNode, version) + val autoGenNode = ProduceRequestDataJsonConverter.write(requestData, version) + + assert(manualGenNode.equals(autoGenNode)) + } + + @Test + def testFormatOfOffsetsForLeaderEpochResponseNode(): Unit = { + val endOffsetMap = new util.HashMap[TopicPartition, EpochEndOffset] + endOffsetMap.put(new TopicPartition("topic1", 0), new EpochEndOffset(1, 10L)) + + val version: Short = 3 + val response = new OffsetsForLeaderEpochResponse(endOffsetMap) + val manualGenNode = RequestConvertToJson.response(response, version) + + val requestData = OffsetForLeaderEpochResponseDataJsonConverter.read(manualGenNode, version) + val autoGenNode = OffsetForLeaderEpochResponseDataJsonConverter.write(requestData, version) + + assert(manualGenNode.equals(autoGenNode)) + } + + @Test + def testFormatOfProduceResponseNode(): Unit = { + val responseData = new util.HashMap[TopicPartition, ProduceResponse.PartitionResponse] + val partResponse = new ProduceResponse.PartitionResponse(Errors.NONE, 10000, RecordBatch.NO_TIMESTAMP, 100, Collections.singletonList(new ProduceResponse.RecordError(3, "Record error")), "Produce failed") + responseData.put(new TopicPartition("topic1", 0), partResponse) + + val version: Short = 3 + val response = new ProduceResponse(responseData) + val manualGenNode = RequestConvertToJson.response(response, version) + + val requestData = ProduceResponseDataJsonConverter.read(manualGenNode, version) + val autoGenNode = ProduceResponseDataJsonConverter.write(requestData, version) + + assert(manualGenNode.equals(autoGenNode)) + } } From 29392c482c0c0b26b96fa023fec0cd507dcc0bb5 Mon Sep 17 00:00:00 2001 From: anatasiavela Date: Fri, 30 Oct 2020 09:25:20 -0700 Subject: [PATCH 05/22] access data from accessor --- .../distributed/WorkerCoordinatorTest.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) 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 8c36751320ea3..2ac05c09d377b 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 void testNormalJoinGroupLeader() { @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 void testNormalJoinGroupFollower() { @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 void testJoinLeaderCannotAssign() { @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, From fb0633cabbd0737ef685726560f05cb7a8711ef7 Mon Sep 17 00:00:00 2001 From: anatasiavela Date: Fri, 30 Oct 2020 16:13:19 -0700 Subject: [PATCH 06/22] remove DESCRIBE_QUORUM case and add better error msg --- core/src/main/scala/kafka/network/RequestChannel.scala | 2 +- .../scala/kafka/network/RequestConvertToJson.scala | 10 ++++++---- .../unit/kafka/network/RequestConvertToJsonTest.scala | 2 -- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/kafka/network/RequestChannel.scala b/core/src/main/scala/kafka/network/RequestChannel.scala index 161da79eaa497..d332d337d78c5 100644 --- a/core/src/main/scala/kafka/network/RequestChannel.scala +++ b/core/src/main/scala/kafka/network/RequestChannel.scala @@ -202,7 +202,7 @@ object RequestChannel extends Logging { } } - trace(s"Processor $processor received request: ${RequestConvertToJson.requestDesc(header, loggableRequest, true).toString}") + trace(s"Processor $processor received request: ${RequestConvertToJson.requestDesc(header, loggableRequest, true)}") def requestThreadTimeNanos: Long = { if (apiLocalCompleteTimeNanos == -1L) apiLocalCompleteTimeNanos = Time.SYSTEM.nanoseconds diff --git a/core/src/main/scala/kafka/network/RequestConvertToJson.scala b/core/src/main/scala/kafka/network/RequestConvertToJson.scala index 65633cc8d7c4f..d164caf4db045 100644 --- a/core/src/main/scala/kafka/network/RequestConvertToJson.scala +++ b/core/src/main/scala/kafka/network/RequestConvertToJson.scala @@ -81,7 +81,7 @@ object RequestConvertToJson { case req: OffsetDeleteRequest => OffsetDeleteRequestDataJsonConverter.write(req.data, request.version) case req: OffsetFetchRequest => OffsetFetchRequestDataJsonConverter.write(req.data, request.version) case req: OffsetsForLeaderEpochRequest => offsetsForLeaderEpochRequestNode(req, request.version) - case req: ProduceRequest => produceRequestNode(req, request.version, verbose) + case req: ProduceRequest => produceRequestNode(req, request.version) 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) @@ -92,7 +92,8 @@ object RequestConvertToJson { 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 Exception("Unexpected request type encountered: " + request) + case _ => throw new AssertionError(String.format("ApiKey %s is not currently handled in `request`, the " + + "code should be updated to do so.", request)); } } @@ -156,7 +157,8 @@ object RequestConvertToJson { 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 Exception("Unexpected response type encountered: " + response) + case _ => throw new AssertionError(String.format("ApiKey %s is not currently handled in `response`, the " + + "code should be updated to do so.", response)); } } @@ -243,7 +245,7 @@ object RequestConvertToJson { /** * Temporary until switch to use the generated schemas. */ - def produceRequestNode(request: ProduceRequest, version: Short, verbose: Boolean): JsonNode = { + def produceRequestNode(request: ProduceRequest, version: Short): JsonNode = { val node = new ObjectNode(JsonNodeFactory.instance) if (version >= 3) { if (request.transactionalId == null) { diff --git a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala index dce310787d139..056ea42883dc9 100644 --- a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala +++ b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala @@ -45,8 +45,6 @@ class RequestConvertToJsonTest { // match so the struct does not have the correct field names. This is // a temporary workaround until ProduceRequest starts using ProduceRequestData req = ProduceRequest.Builder.forCurrentMagic(0.toShort, 10000, new util.HashMap[TopicPartition, MemoryRecords]()).build() - } else if (key == ApiKeys.DESCRIBE_QUORUM) { - req = new DescribeQuorumRequest(new DescribeQuorumRequestData().toStruct(version), version) } else { val struct = ApiMessageType.fromApiKey(key.id).newRequest().toStruct(version) req = AbstractRequest.parseRequest(key, version, struct) From 32f1e9db525d50728f3e36d8c59aaec7d718f90c Mon Sep 17 00:00:00 2001 From: anatasiavela Date: Sun, 1 Nov 2020 22:42:10 -0800 Subject: [PATCH 07/22] change error messages and asserts --- .../scala/kafka/network/RequestConvertToJson.scala | 8 ++++---- .../kafka/network/RequestConvertToJsonTest.scala | 13 +++++++------ 2 files changed, 11 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/kafka/network/RequestConvertToJson.scala b/core/src/main/scala/kafka/network/RequestConvertToJson.scala index d164caf4db045..c3d0dbba404c8 100644 --- a/core/src/main/scala/kafka/network/RequestConvertToJson.scala +++ b/core/src/main/scala/kafka/network/RequestConvertToJson.scala @@ -92,8 +92,8 @@ object RequestConvertToJson { 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 AssertionError(String.format("ApiKey %s is not currently handled in `request`, the " + - "code should be updated to do so.", request)); + case _ => throw new AssertionError(s"Request type $request is not currently handled in `request`, the " + + "code should be updated to do so."); } } @@ -157,8 +157,8 @@ object RequestConvertToJson { 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 AssertionError(String.format("ApiKey %s is not currently handled in `response`, the " + - "code should be updated to do so.", response)); + case _ => throw new AssertionError(s"Response type $response is not currently handled in `response`, the " + + "code should be updated to do so."); } } diff --git a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala index 056ea42883dc9..0a72f55c4d156 100644 --- a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala +++ b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala @@ -28,6 +28,7 @@ import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch, TimestampType} import org.apache.kafka.common.requests.OffsetsForLeaderEpochRequest.PartitionData import org.apache.kafka.common.requests._ +import org.junit.Assert.assertEquals import scala.collection.mutable.ArrayBuffer @@ -55,7 +56,7 @@ class RequestConvertToJsonTest { case _ : AssertionError => unhandledKeys += key.toString } }) - assert(unhandledKeys.isEmpty, String.format("%s request keys not handled in RequestConvertToJson", unhandledKeys)) + assertEquals("Unhandled request keys", ArrayBuffer.empty, unhandledKeys) } @Test @@ -71,7 +72,7 @@ class RequestConvertToJsonTest { case _ : AssertionError => unhandledKeys += key.toString } }) - assert(unhandledKeys.isEmpty, String.format("%s response keys not handled in RequestConvertToJson", unhandledKeys)) + assertEquals("Unhandled response keys", ArrayBuffer.empty, unhandledKeys) } @Test @@ -86,7 +87,7 @@ class RequestConvertToJsonTest { val requestData = OffsetForLeaderEpochRequestDataJsonConverter.read(manualGenNode, version) val autoGenNode = OffsetForLeaderEpochRequestDataJsonConverter.write(requestData, version) - assert(manualGenNode.equals(autoGenNode)) + assertEquals(manualGenNode, autoGenNode) } @Test @@ -104,7 +105,7 @@ class RequestConvertToJsonTest { val requestData = ProduceRequestDataJsonConverter.read(manualGenNode, version) val autoGenNode = ProduceRequestDataJsonConverter.write(requestData, version) - assert(manualGenNode.equals(autoGenNode)) + assertEquals(manualGenNode, autoGenNode) } @Test @@ -119,7 +120,7 @@ class RequestConvertToJsonTest { val requestData = OffsetForLeaderEpochResponseDataJsonConverter.read(manualGenNode, version) val autoGenNode = OffsetForLeaderEpochResponseDataJsonConverter.write(requestData, version) - assert(manualGenNode.equals(autoGenNode)) + assertEquals(manualGenNode, autoGenNode) } @Test @@ -135,6 +136,6 @@ class RequestConvertToJsonTest { val requestData = ProduceResponseDataJsonConverter.read(manualGenNode, version) val autoGenNode = ProduceResponseDataJsonConverter.write(requestData, version) - assert(manualGenNode.equals(autoGenNode)) + assertEquals(manualGenNode, autoGenNode) } } From ab50c8fe258959cd8366ca6dbf56994046ab6f76 Mon Sep 17 00:00:00 2001 From: anatasiavela Date: Mon, 2 Nov 2020 11:05:49 -0800 Subject: [PATCH 08/22] swap expected and actual, change error msg, add accessor to private data field --- .../requests/InitProducerIdResponse.java | 2 +- .../kafka/network/RequestConvertToJson.scala | 4 +-- .../network/RequestConvertToJsonTest.scala | 32 +++++++++---------- 3 files changed, 19 insertions(+), 19 deletions(-) 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 6066c13ef5ab6..580107f2e0982 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 @@ * - {@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); diff --git a/core/src/main/scala/kafka/network/RequestConvertToJson.scala b/core/src/main/scala/kafka/network/RequestConvertToJson.scala index c3d0dbba404c8..c350789228ce2 100644 --- a/core/src/main/scala/kafka/network/RequestConvertToJson.scala +++ b/core/src/main/scala/kafka/network/RequestConvertToJson.scala @@ -92,7 +92,7 @@ object RequestConvertToJson { 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 AssertionError(s"Request type $request is not currently handled in `request`, the " + + case _ => throw new AssertionError(s"ApiKey ${request.api} is not currently handled in `request`, the " + "code should be updated to do so."); } } @@ -157,7 +157,7 @@ object RequestConvertToJson { 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 AssertionError(s"Response type $response is not currently handled in `response`, the " + + case _ => throw new AssertionError(s"ApiKey $response is not currently handled in `response`, the " + "code should be updated to do so."); } } diff --git a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala index 0a72f55c4d156..46c6848562f5b 100644 --- a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala +++ b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala @@ -82,12 +82,12 @@ class RequestConvertToJsonTest { val version: Short = 3 val request = OffsetsForLeaderEpochRequest.Builder.forConsumer(partitionDataMap).build(version) - val manualGenNode = RequestConvertToJson.request(request, false) + val actualNode = RequestConvertToJson.request(request, false) - val requestData = OffsetForLeaderEpochRequestDataJsonConverter.read(manualGenNode, version) - val autoGenNode = OffsetForLeaderEpochRequestDataJsonConverter.write(requestData, version) + val requestData = OffsetForLeaderEpochRequestDataJsonConverter.read(actualNode, version) + val expectedNode = OffsetForLeaderEpochRequestDataJsonConverter.write(requestData, version) - assertEquals(manualGenNode, autoGenNode) + assertEquals(expectedNode, actualNode) } @Test @@ -100,12 +100,12 @@ class RequestConvertToJsonTest { val version: Short = 3 val request = ProduceRequest.Builder.forCurrentMagic(1.toShort, 5000, produceData).build(version) - val manualGenNode = RequestConvertToJson.request(request, false) + val actualNode = RequestConvertToJson.request(request, false) - val requestData = ProduceRequestDataJsonConverter.read(manualGenNode, version) - val autoGenNode = ProduceRequestDataJsonConverter.write(requestData, version) + val requestData = ProduceRequestDataJsonConverter.read(actualNode, version) + val expectedNode = ProduceRequestDataJsonConverter.write(requestData, version) - assertEquals(manualGenNode, autoGenNode) + assertEquals(expectedNode, actualNode) } @Test @@ -115,12 +115,12 @@ class RequestConvertToJsonTest { val version: Short = 3 val response = new OffsetsForLeaderEpochResponse(endOffsetMap) - val manualGenNode = RequestConvertToJson.response(response, version) + val actualNode = RequestConvertToJson.response(response, version) - val requestData = OffsetForLeaderEpochResponseDataJsonConverter.read(manualGenNode, version) - val autoGenNode = OffsetForLeaderEpochResponseDataJsonConverter.write(requestData, version) + val requestData = OffsetForLeaderEpochResponseDataJsonConverter.read(actualNode, version) + val expectedNode = OffsetForLeaderEpochResponseDataJsonConverter.write(requestData, version) - assertEquals(manualGenNode, autoGenNode) + assertEquals(expectedNode, actualNode) } @Test @@ -131,11 +131,11 @@ class RequestConvertToJsonTest { val version: Short = 3 val response = new ProduceResponse(responseData) - val manualGenNode = RequestConvertToJson.response(response, version) + val actualNode = RequestConvertToJson.response(response, version) - val requestData = ProduceResponseDataJsonConverter.read(manualGenNode, version) - val autoGenNode = ProduceResponseDataJsonConverter.write(requestData, version) + val requestData = ProduceResponseDataJsonConverter.read(actualNode, version) + val expectedNode = ProduceResponseDataJsonConverter.write(requestData, version) - assertEquals(manualGenNode, autoGenNode) + assertEquals(expectedNode, actualNode) } } From 97237afbd72335b7617a500afebe5b894aa5bdbc Mon Sep 17 00:00:00 2001 From: anatasiavela Date: Mon, 2 Nov 2020 11:12:43 -0800 Subject: [PATCH 09/22] change to data accessor method --- .../kafka/clients/producer/internals/TransactionManager.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java index 077316b81b492..e58469b003a7e 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 void handleResponse(AbstractResponse response) { 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; From 32578e449bb3808f9d19ac8dd7a38e1d91e2fdba Mon Sep 17 00:00:00 2001 From: anatasiavela Date: Mon, 2 Nov 2020 11:18:11 -0800 Subject: [PATCH 10/22] change recordSet field to print sizeInBytes --- .../org/apache/kafka/message/JsonConverterGenerator.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) 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 94a79cb403d31..66391bc0f0368 100644 --- a/generator/src/main/java/org/apache/kafka/message/JsonConverterGenerator.java +++ b/generator/src/main/java/org/apache/kafka/message/JsonConverterGenerator.java @@ -380,8 +380,9 @@ private void generateVariableLengthTargetToJson(Target target, Versions versions target.sourceVariable(), target.sourceVariable()))); } } else if (target.field().type().isRecords()) { - headerGenerator.addImport(MessageGenerator.BINARY_NODE_CLASS); - buffer.printf("%s;%n", target.assignmentStatement("new BinaryNode(new byte[]{})")); + headerGenerator.addImport(MessageGenerator.INT_NODE_CLASS); + buffer.printf("%s;%n", target.assignmentStatement( + String.format("new IntNode(%s.sizeInBytes())", target.sourceVariable()))); } else if (target.field().type().isArray()) { headerGenerator.addImport(MessageGenerator.ARRAY_NODE_CLASS); headerGenerator.addImport(MessageGenerator.JSON_NODE_FACTORY_CLASS); From ab73c10d4246341a67af90f7bcdd6ab5500da5d5 Mon Sep 17 00:00:00 2001 From: anatasiavela Date: Mon, 2 Nov 2020 20:35:29 -0800 Subject: [PATCH 11/22] trigger build From d3b548799abe15655ee1fdafdab5712bc1ea6444 Mon Sep 17 00:00:00 2001 From: anatasiavela Date: Thu, 5 Nov 2020 00:59:17 -0800 Subject: [PATCH 12/22] add verbose flag to records in JsonConverter --- .../message/SimpleExampleMessageTest.java | 4 +- .../kafka/network/RequestConvertToJson.scala | 258 +++++++++--------- .../kafka/tools/TestRaftRequestHandler.scala | 3 +- .../network/RequestConvertToJsonTest.scala | 26 +- .../unit/kafka/network/SocketServerTest.scala | 7 +- .../kafka/message/JsonConverterGenerator.java | 21 +- .../kafka/raft/FileBasedStateStore.java | 4 +- 7 files changed, 166 insertions(+), 157 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/common/message/SimpleExampleMessageTest.java b/clients/src/test/java/org/apache/kafka/common/message/SimpleExampleMessageTest.java index b3e45e5a64e09..0b3d92d11dbbe 100644 --- a/clients/src/test/java/org/apache/kafka/common/message/SimpleExampleMessageTest.java +++ b/clients/src/test/java/org/apache/kafka/common/message/SimpleExampleMessageTest.java @@ -394,8 +394,8 @@ private void testRoundTrip(SimpleExampleMessageData message, assertEquals(message.hashCode(), messageFromStruct.hashCode()); // Check JSON serialization - JsonNode serializedJson = SimpleExampleMessageDataJsonConverter.write(message, version); - SimpleExampleMessageData messageFromJson = SimpleExampleMessageDataJsonConverter.read(serializedJson, version); + JsonNode serializedJson = SimpleExampleMessageDataJsonConverter.write(message, version, true); + SimpleExampleMessageData messageFromJson = SimpleExampleMessageDataJsonConverter.read(serializedJson, version, true); validator.accept(messageFromJson); assertEquals(message, messageFromJson); assertEquals(message.hashCode(), messageFromJson.hashCode()); diff --git a/core/src/main/scala/kafka/network/RequestConvertToJson.scala b/core/src/main/scala/kafka/network/RequestConvertToJson.scala index c350789228ce2..ecc7cafa36153 100644 --- a/core/src/main/scala/kafka/network/RequestConvertToJson.scala +++ b/core/src/main/scala/kafka/network/RequestConvertToJson.scala @@ -34,136 +34,136 @@ import scala.jdk.CollectionConverters._ object RequestConvertToJson { def request(request: AbstractRequest, verbose: Boolean): 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: 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 => offsetsForLeaderEpochRequestNode(req, request.version) - case req: ProduceRequest => produceRequestNode(req, request.version) - 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 AssertionError(s"ApiKey ${request.api} is not currently handled in `request`, the " + + case req: AddOffsetsToTxnRequest => AddOffsetsToTxnRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: AddPartitionsToTxnRequest => AddPartitionsToTxnRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: AlterClientQuotasRequest => AlterClientQuotasRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: AlterConfigsRequest => AlterConfigsRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: AlterIsrRequest => AlterIsrRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: AlterPartitionReassignmentsRequest => AlterPartitionReassignmentsRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: AlterReplicaLogDirsRequest => AlterReplicaLogDirsRequestDataJsonConverter.write(req.data, request.version, verbose) + case res: AlterUserScramCredentialsRequest => AlterUserScramCredentialsRequestDataJsonConverter.write(res.data, request.version, verbose) + case req: ApiVersionsRequest => ApiVersionsRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: BeginQuorumEpochRequest => BeginQuorumEpochRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: ControlledShutdownRequest => ControlledShutdownRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: CreateAclsRequest => CreateAclsRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: CreateDelegationTokenRequest => CreateDelegationTokenRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: CreatePartitionsRequest => CreatePartitionsRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: CreateTopicsRequest => CreateTopicsRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: DeleteAclsRequest => DeleteAclsRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: DeleteGroupsRequest => DeleteGroupsRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: DeleteRecordsRequest => DeleteRecordsRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: DeleteTopicsRequest => DeleteTopicsRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: DescribeAclsRequest => DescribeAclsRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: DescribeClientQuotasRequest => DescribeClientQuotasRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: DescribeConfigsRequest => DescribeConfigsRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: DescribeDelegationTokenRequest => DescribeDelegationTokenRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: DescribeGroupsRequest => DescribeGroupsRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: DescribeLogDirsRequest => DescribeLogDirsRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: DescribeQuorumRequest => DescribeQuorumRequestDataJsonConverter.write(req.data, request.version, verbose) + case res: DescribeUserScramCredentialsRequest => DescribeUserScramCredentialsRequestDataJsonConverter.write(res.data, request.version, verbose) + case req: ElectLeadersRequest => ElectLeadersRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: EndTxnRequest => EndTxnRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: EndQuorumEpochRequest => EndQuorumEpochRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: ExpireDelegationTokenRequest => ExpireDelegationTokenRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: FetchRequest => FetchRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: FindCoordinatorRequest => FindCoordinatorRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: HeartbeatRequest => HeartbeatRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: IncrementalAlterConfigsRequest => IncrementalAlterConfigsRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: InitProducerIdRequest => InitProducerIdRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: JoinGroupRequest => JoinGroupRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: LeaderAndIsrRequest => LeaderAndIsrRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: LeaveGroupRequest => LeaveGroupRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: ListGroupsRequest => ListGroupsRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: ListOffsetRequest => ListOffsetRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: ListPartitionReassignmentsRequest => ListPartitionReassignmentsRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: MetadataRequest => MetadataRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: OffsetCommitRequest => OffsetCommitRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: OffsetDeleteRequest => OffsetDeleteRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: OffsetFetchRequest => OffsetFetchRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: OffsetsForLeaderEpochRequest => offsetsForLeaderEpochRequestNode(req, request.version, verbose) + case req: ProduceRequest => produceRequestNode(req, request.version, verbose) + case req: RenewDelegationTokenRequest => RenewDelegationTokenRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: SaslAuthenticateRequest => SaslAuthenticateRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: SaslHandshakeRequest => SaslHandshakeRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: StopReplicaRequest => StopReplicaRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: SyncGroupRequest => SyncGroupRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: TxnOffsetCommitRequest => TxnOffsetCommitRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: UpdateFeaturesRequest => UpdateFeaturesRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: UpdateMetadataRequest => UpdateMetadataRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: VoteRequest => VoteRequestDataJsonConverter.write(req.data, request.version, verbose) + case req: WriteTxnMarkersRequest => WriteTxnMarkersRequestDataJsonConverter.write(req.data, request.version, verbose) + case _ => throw new IllegalStateException(s"ApiKey ${request.api} is not currently handled in `request`, the " + "code should be updated to do so."); } } - def response(response: AbstractResponse, version: Short): JsonNode = { + def response(response: AbstractResponse, version: Short, verbose: Boolean): 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: ExpireDelegationTokenResponse => ExpireDelegationTokenResponseDataJsonConverter.write(res.data, version) - case res: FetchResponse[_] => FetchResponseDataJsonConverter.write(res.data, version) - 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 => offsetsForLeaderEpochResponseNode(res, version) - case res: ProduceResponse => produceResponseNode(res, 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 AssertionError(s"ApiKey $response is not currently handled in `response`, the " + + case res: AddOffsetsToTxnResponse => AddOffsetsToTxnResponseDataJsonConverter.write(res.data, version, verbose) + case res: AddPartitionsToTxnResponse => AddPartitionsToTxnResponseDataJsonConverter.write(res.data, version, verbose) + case res: AlterClientQuotasResponse => AlterClientQuotasResponseDataJsonConverter.write(res.data, version, verbose) + case res: AlterConfigsResponse => AlterConfigsResponseDataJsonConverter.write(res.data, version, verbose) + case res: AlterIsrResponse => AlterIsrResponseDataJsonConverter.write(res.data, version, verbose) + case res: AlterPartitionReassignmentsResponse => AlterPartitionReassignmentsResponseDataJsonConverter.write(res.data, version, verbose) + case res: AlterReplicaLogDirsResponse => AlterReplicaLogDirsResponseDataJsonConverter.write(res.data, version, verbose) + case res: AlterUserScramCredentialsResponse => AlterUserScramCredentialsResponseDataJsonConverter.write(res.data, version, verbose) + case res: ApiVersionsResponse => ApiVersionsResponseDataJsonConverter.write(res.data, version, verbose) + case res: BeginQuorumEpochResponse => BeginQuorumEpochResponseDataJsonConverter.write(res.data, version, verbose) + case res: ControlledShutdownResponse => ControlledShutdownResponseDataJsonConverter.write(res.data, version, verbose) + case res: CreateAclsResponse => CreateAclsResponseDataJsonConverter.write(res.data, version, verbose) + case res: CreateDelegationTokenResponse => CreateDelegationTokenResponseDataJsonConverter.write(res.data, version, verbose) + case res: CreatePartitionsResponse => CreatePartitionsResponseDataJsonConverter.write(res.data, version, verbose) + case res: CreateTopicsResponse => CreateTopicsResponseDataJsonConverter.write(res.data, version, verbose) + case res: DeleteAclsResponse => DeleteAclsResponseDataJsonConverter.write(res.data, version, verbose) + case res: DeleteGroupsResponse => DeleteGroupsResponseDataJsonConverter.write(res.data, version, verbose) + case res: DeleteRecordsResponse => DeleteRecordsResponseDataJsonConverter.write(res.data, version, verbose) + case res: DeleteTopicsResponse => DeleteTopicsResponseDataJsonConverter.write(res.data, version, verbose) + case res: DescribeAclsResponse => DescribeAclsResponseDataJsonConverter.write(res.data, version, verbose) + case res: DescribeClientQuotasResponse => DescribeClientQuotasResponseDataJsonConverter.write(res.data, version, verbose) + case res: DescribeConfigsResponse => DescribeConfigsResponseDataJsonConverter.write(res.data, version, verbose) + case res: DescribeDelegationTokenResponse => DescribeDelegationTokenResponseDataJsonConverter.write(res.data, version, verbose) + case res: DescribeGroupsResponse => DescribeGroupsResponseDataJsonConverter.write(res.data, version, verbose) + case res: DescribeLogDirsResponse => DescribeLogDirsResponseDataJsonConverter.write(res.data, version, verbose) + case res: DescribeQuorumResponse => DescribeQuorumResponseDataJsonConverter.write(res.data, version, verbose) + case res: DescribeUserScramCredentialsResponse => DescribeUserScramCredentialsResponseDataJsonConverter.write(res.data, version, verbose) + case res: ElectLeadersResponse => ElectLeadersResponseDataJsonConverter.write(res.data, version, verbose) + case res: EndTxnResponse => EndTxnResponseDataJsonConverter.write(res.data, version, verbose) + case res: EndQuorumEpochResponse => EndQuorumEpochResponseDataJsonConverter.write(res.data, version, verbose) + case res: ExpireDelegationTokenResponse => ExpireDelegationTokenResponseDataJsonConverter.write(res.data, version, verbose) + case res: FetchResponse[_] => FetchResponseDataJsonConverter.write(res.data, version, verbose) + case res: FindCoordinatorResponse => FindCoordinatorResponseDataJsonConverter.write(res.data, version, verbose) + case res: HeartbeatResponse => HeartbeatResponseDataJsonConverter.write(res.data, version, verbose) + case res: IncrementalAlterConfigsResponse => IncrementalAlterConfigsResponseDataJsonConverter.write(res.data, version, verbose) + case res: InitProducerIdResponse => InitProducerIdResponseDataJsonConverter.write(res.data, version, verbose) + case res: JoinGroupResponse => JoinGroupResponseDataJsonConverter.write(res.data, version, verbose) + case res: LeaderAndIsrResponse => LeaderAndIsrResponseDataJsonConverter.write(res.data, version, verbose) + case res: LeaveGroupResponse => LeaveGroupResponseDataJsonConverter.write(res.data, version, verbose) + case res: ListGroupsResponse => ListGroupsResponseDataJsonConverter.write(res.data, version, verbose) + case res: ListOffsetResponse => ListOffsetResponseDataJsonConverter.write(res.data, version, verbose) + case res: ListPartitionReassignmentsResponse => ListPartitionReassignmentsResponseDataJsonConverter.write(res.data, version, verbose) + case res: MetadataResponse => MetadataResponseDataJsonConverter.write(res.data, version, verbose) + case res: OffsetCommitResponse => OffsetCommitResponseDataJsonConverter.write(res.data, version, verbose) + case res: OffsetDeleteResponse => OffsetDeleteResponseDataJsonConverter.write(res.data, version, verbose) + case res: OffsetFetchResponse => OffsetFetchResponseDataJsonConverter.write(res.data, version, verbose) + case res: OffsetsForLeaderEpochResponse => offsetsForLeaderEpochResponseNode(res, version, verbose) + case res: ProduceResponse => produceResponseNode(res, version, verbose) + case res: RenewDelegationTokenResponse => RenewDelegationTokenResponseDataJsonConverter.write(res.data, version, verbose) + case res: SaslAuthenticateResponse => SaslAuthenticateResponseDataJsonConverter.write(res.data, version, verbose) + case res: SaslHandshakeResponse => SaslHandshakeResponseDataJsonConverter.write(res.data, version, verbose) + case res: StopReplicaResponse => StopReplicaResponseDataJsonConverter.write(res.data, version, verbose) + case res: SyncGroupResponse => SyncGroupResponseDataJsonConverter.write(res.data, version, verbose) + case res: TxnOffsetCommitResponse => TxnOffsetCommitResponseDataJsonConverter.write(res.data, version, verbose) + case res: UpdateFeaturesResponse => UpdateFeaturesResponseDataJsonConverter.write(res.data, version, verbose) + case res: UpdateMetadataResponse => UpdateMetadataResponseDataJsonConverter.write(res.data, version, verbose) + case res: WriteTxnMarkersResponse => WriteTxnMarkersResponseDataJsonConverter.write(res.data, version, verbose) + case res: VoteResponse => VoteResponseDataJsonConverter.write(res.data, version, verbose) + case _ => throw new IllegalStateException(s"ApiKey $response 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()).asInstanceOf[ObjectNode] + def requestHeaderNode(header: RequestHeader, verbose: Boolean): JsonNode = { + val node = RequestHeaderDataJsonConverter.write(header.data(), header.headerVersion(), verbose).asInstanceOf[ObjectNode] node.set("requestApiKeyName", new TextNode(header.apiKey.toString)) node } @@ -171,7 +171,7 @@ object RequestConvertToJson { def clientInfoNode(clientInfo: ClientInformation): JsonNode = { val node = new ObjectNode(JsonNodeFactory.instance) node.set("softwareName", new TextNode(clientInfo.softwareName())) - node.set("softwareName", new TextNode(clientInfo.softwareVersion())) + node.set("softwareVersion", new TextNode(clientInfo.softwareVersion())) node } @@ -182,7 +182,7 @@ object RequestConvertToJson { responseSendTimeMs: Double, temporaryMemoryBytes: Long, messageConversionsTimeMs: Double): JsonNode = { val node = new ObjectNode(JsonNodeFactory.instance) - node.set("requestHeader", requestHeaderNode(header)) + node.set("requestHeader", requestHeaderNode(header, verbose)) node.set("request", request(req, verbose)) node.set("response", res.responseLog.getOrElse(new TextNode(""))) node.set("connection", new TextNode(context.connectionId)) @@ -206,7 +206,7 @@ object RequestConvertToJson { def requestDesc(header: RequestHeader, req: AbstractRequest, verbose: Boolean): JsonNode = { val node = new ObjectNode(JsonNodeFactory.instance) - node.set("requestHeader", requestHeaderNode(header)) + node.set("requestHeader", requestHeaderNode(header, verbose)) node.set("request", request(req, verbose)) node } @@ -214,7 +214,7 @@ object RequestConvertToJson { /** * Temporary until switch to use the generated schemas. */ - def offsetsForLeaderEpochRequestNode(request: OffsetsForLeaderEpochRequest, version: Short): JsonNode = { + def offsetsForLeaderEpochRequestNode(request: OffsetsForLeaderEpochRequest, version: Short, verbose: Boolean): JsonNode = { val node = new ObjectNode(JsonNodeFactory.instance) if (version >= 3) { node.set("replicaId", new IntNode(request.replicaId)) @@ -245,7 +245,7 @@ object RequestConvertToJson { /** * Temporary until switch to use the generated schemas. */ - def produceRequestNode(request: ProduceRequest, version: Short): JsonNode = { + def produceRequestNode(request: ProduceRequest, version: Short, verbose: Boolean): JsonNode = { val node = new ObjectNode(JsonNodeFactory.instance) if (version >= 3) { if (request.transactionalId == null) { @@ -262,7 +262,7 @@ object RequestConvertToJson { val topicNode = new ObjectNode(JsonNodeFactory.instance) topicNode.set("name", new TextNode(topicName)) val partitionsArray = new ArrayNode(JsonNodeFactory.instance) - partitions.forEach { (partitionIndex, partitionData)=> + partitions.forEach { (partitionIndex, partitionData) => val partitionNode = new ObjectNode(JsonNodeFactory.instance) partitionNode.set("partitionIndex", new IntNode(partitionIndex)) if (partitionData == null) @@ -281,7 +281,7 @@ object RequestConvertToJson { /** * Temporary until switch to use the generated schemas. */ - def offsetsForLeaderEpochResponseNode(response: OffsetsForLeaderEpochResponse, version: Short): JsonNode = { + def offsetsForLeaderEpochResponseNode(response: OffsetsForLeaderEpochResponse, version: Short, verbose: Boolean): JsonNode = { val node = new ObjectNode(JsonNodeFactory.instance) if (version >= 2) { node.set("throttleTimeMs", new IntNode(response.throttleTimeMs)) @@ -310,7 +310,7 @@ object RequestConvertToJson { /** * Temporary until switch to use the generated schemas. */ - def produceResponseNode(response: ProduceResponse, version: Short): JsonNode = { + def produceResponseNode(response: ProduceResponse, version: Short, verbose: Boolean): JsonNode = { val node = new ObjectNode(JsonNodeFactory.instance) val topics = CollectionUtils.groupPartitionDataByTopic(response.responses) val responsesArray = new ArrayNode(JsonNodeFactory.instance) diff --git a/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala b/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala index 19df2fcaf6689..dc5b7f6c13a88 100644 --- a/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala +++ b/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala @@ -95,9 +95,10 @@ class TestRaftRequestHandler( case Some(response) => val responseSend = request.context.buildResponseSend(response) val responseString = - if (RequestChannel.isRequestLoggingEnabled) Some(RequestConvertToJson.requestHeaderNode(request.header)) + if (RequestChannel.isRequestLoggingEnabled) Some(RequestConvertToJson.response(response, request.context.apiVersion, true)) else None new RequestChannel.SendResponse(request, responseSend, responseString, None) + case None => new RequestChannel.NoOpResponse(request) } sendResponse(response) diff --git a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala index 46c6848562f5b..dfaf29afc1fff 100644 --- a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala +++ b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala @@ -67,7 +67,7 @@ class RequestConvertToJsonTest { val struct = ApiMessageType.fromApiKey(key.id).newResponse().toStruct(version) val res = AbstractResponse.parseResponse(key, struct, version) try { - RequestConvertToJson.response(res, version) + RequestConvertToJson.response(res, version, false) } catch { case _ : AssertionError => unhandledKeys += key.toString } @@ -82,10 +82,10 @@ class RequestConvertToJsonTest { val version: Short = 3 val request = OffsetsForLeaderEpochRequest.Builder.forConsumer(partitionDataMap).build(version) - val actualNode = RequestConvertToJson.request(request, false) + val actualNode = RequestConvertToJson.request(request, true) - val requestData = OffsetForLeaderEpochRequestDataJsonConverter.read(actualNode, version) - val expectedNode = OffsetForLeaderEpochRequestDataJsonConverter.write(requestData, version) + val requestData = OffsetForLeaderEpochRequestDataJsonConverter.read(actualNode, version, true) + val expectedNode = OffsetForLeaderEpochRequestDataJsonConverter.write(requestData, version, true) assertEquals(expectedNode, actualNode) } @@ -100,10 +100,10 @@ class RequestConvertToJsonTest { val version: Short = 3 val request = ProduceRequest.Builder.forCurrentMagic(1.toShort, 5000, produceData).build(version) - val actualNode = RequestConvertToJson.request(request, false) + val actualNode = RequestConvertToJson.request(request, true) - val requestData = ProduceRequestDataJsonConverter.read(actualNode, version) - val expectedNode = ProduceRequestDataJsonConverter.write(requestData, version) + val requestData = ProduceRequestDataJsonConverter.read(actualNode, version, true) + val expectedNode = ProduceRequestDataJsonConverter.write(requestData, version, true) assertEquals(expectedNode, actualNode) } @@ -115,10 +115,10 @@ class RequestConvertToJsonTest { val version: Short = 3 val response = new OffsetsForLeaderEpochResponse(endOffsetMap) - val actualNode = RequestConvertToJson.response(response, version) + val actualNode = RequestConvertToJson.response(response, version, true) - val requestData = OffsetForLeaderEpochResponseDataJsonConverter.read(actualNode, version) - val expectedNode = OffsetForLeaderEpochResponseDataJsonConverter.write(requestData, version) + val requestData = OffsetForLeaderEpochResponseDataJsonConverter.read(actualNode, version, true) + val expectedNode = OffsetForLeaderEpochResponseDataJsonConverter.write(requestData, version, true) assertEquals(expectedNode, actualNode) } @@ -131,10 +131,10 @@ class RequestConvertToJsonTest { val version: Short = 3 val response = new ProduceResponse(responseData) - val actualNode = RequestConvertToJson.response(response, version) + val actualNode = RequestConvertToJson.response(response, version, true) - val requestData = ProduceResponseDataJsonConverter.read(actualNode, version) - val expectedNode = ProduceResponseDataJsonConverter.write(requestData, version) + val requestData = ProduceResponseDataJsonConverter.read(actualNode, version, true) + val expectedNode = ProduceResponseDataJsonConverter.write(requestData, version, true) assertEquals(expectedNode, actualNode) } diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index bc6b0afed38c0..43c966018eb08 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -25,11 +25,6 @@ import java.nio.charset.StandardCharsets import java.util import java.util.concurrent.{CompletableFuture, ConcurrentLinkedQueue, Executors, TimeUnit} import java.util.{Properties, Random} -<<<<<<< HEAD -======= - -import com.fasterxml.jackson.databind.node.{JsonNodeFactory, ObjectNode, TextNode} ->>>>>>> 1d0de0692c... KAFKA-10525: Emit JSONs with new auto-generated schema import com.yammer.metrics.core.{Gauge, Meter} import javax.net.ssl._ @@ -682,7 +677,7 @@ class SocketServerTest { server.dataPlaneRequestChannel.sendResponse(response) } val throttledChannel = new ThrottledChannel(request, new MockTime(), 100, channelThrottlingCallback) - val headerLog = RequestConvertToJson.requestHeaderNode(request.header) + val headerLog = RequestConvertToJson.requestHeaderNode(request.header, true) val response = if (!noOpResponse) new RequestChannel.SendResponse(request, send, Some(headerLog), None) 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 66391bc0f0368..302284ca8a8c8 100644 --- a/generator/src/main/java/org/apache/kafka/message/JsonConverterGenerator.java +++ b/generator/src/main/java/org/apache/kafka/message/JsonConverterGenerator.java @@ -83,7 +83,7 @@ private void generateRead(String className, StructSpec struct, Versions parentVersions) { headerGenerator.addImport(MessageGenerator.JSON_NODE_CLASS); - buffer.printf("public static %s read(JsonNode _node, short _version) {%n", + buffer.printf("public static %s read(JsonNode _node, short _version, boolean _verbose) {%n", className); buffer.incrementIndent(); buffer.printf("%s _object = new %s();%n", className, className); @@ -227,9 +227,13 @@ private void generateVariableLengthTargetFromJson(Target target, Versions curVer headerGenerator.addImport(MessageGenerator.MESSAGE_UTIL_CLASS); headerGenerator.addImport(MessageGenerator.BYTE_BUFFER_CLASS); headerGenerator.addImport(MessageGenerator.MEMORY_RECORDS_CLASS); + buffer.printf("if (_verbose) {%n"); + buffer.incrementIndent(); buffer.printf("%s;%n", target.assignmentStatement( String.format("MemoryRecords.readableRecords(ByteBuffer.wrap(MessageUtil.jsonNodeToBinary(%s, \"%s\")))", target.sourceVariable(), target.humanReadableName()))); + buffer.decrementIndent(); + buffer.printf("}%n"); } else if (target.field().type().isArray()) { buffer.printf("if (!%s.isArray()) {%n", target.sourceVariable()); buffer.incrementIndent(); @@ -250,7 +254,7 @@ private void generateVariableLengthTargetFromJson(Target target, Versions curVer buffer.printf("}%n"); } else if (target.field().type().isStruct()) { buffer.printf("%s;%n", target.assignmentStatement( - String.format("%s%s.read(%s, _version)", + String.format("%s%s.read(%s, _version, _verbose)", target.field().type().toString(), SUFFIX, target.sourceVariable()))); } else { throw new RuntimeException("Unexpected type " + target.field().type()); @@ -261,7 +265,7 @@ 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 _verbose) {%n", className); buffer.incrementIndent(); VersionConditional.forVersions(struct.versions(), parentVersions). @@ -381,8 +385,17 @@ private void generateVariableLengthTargetToJson(Target target, Versions versions } } else if (target.field().type().isRecords()) { headerGenerator.addImport(MessageGenerator.INT_NODE_CLASS); + headerGenerator.addImport(MessageGenerator.BINARY_NODE_CLASS); + buffer.printf("if (_verbose) {%n"); + buffer.incrementIndent(); + buffer.printf("%s;%n", target.assignmentStatement("new BinaryNode(new byte[]{})")); + buffer.decrementIndent(); + buffer.printf("} else {%n"); + buffer.incrementIndent(); buffer.printf("%s;%n", target.assignmentStatement( String.format("new IntNode(%s.sizeInBytes())", 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); @@ -403,7 +416,7 @@ private void generateVariableLengthTargetToJson(Target target, Versions versions 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, _verbose)", target.field().type().toString(), target.sourceVariable()))); } else { throw new RuntimeException("unknown type " + target.field().type()); diff --git a/raft/src/main/java/org/apache/kafka/raft/FileBasedStateStore.java b/raft/src/main/java/org/apache/kafka/raft/FileBasedStateStore.java index ae96279e6d41a..ea9f7f1c67631 100644 --- a/raft/src/main/java/org/apache/kafka/raft/FileBasedStateStore.java +++ b/raft/src/main/java/org/apache/kafka/raft/FileBasedStateStore.java @@ -90,7 +90,7 @@ private QuorumStateData readStateFromFile(File file) throws IOException { } final short dataVersion = dataVersionNode.shortValue(); - return QuorumStateDataJsonConverter.read(dataObject, dataVersion); + return QuorumStateDataJsonConverter.read(dataObject, dataVersion, true); } } @@ -140,7 +140,7 @@ private void writeElectionStateToFile(final File stateFile, QuorumStateData stat new OutputStreamWriter(fileOutputStream, StandardCharsets.UTF_8))) { short version = state.highestSupportedVersion(); - ObjectNode jsonState = (ObjectNode) QuorumStateDataJsonConverter.write(state, version); + ObjectNode jsonState = (ObjectNode) QuorumStateDataJsonConverter.write(state, version, true); jsonState.set(DATA_VERSION, new ShortNode(version)); writer.write(jsonState.toString()); writer.flush(); From bd9a82bd642090a7efbf64ac429a43fa9a040264 Mon Sep 17 00:00:00 2001 From: anatasiavela Date: Thu, 5 Nov 2020 17:59:50 -0800 Subject: [PATCH 13/22] add overload method for verbose flag --- .../message/SimpleExampleMessageTest.java | 2 +- .../scala/kafka/network/RequestChannel.scala | 5 +- .../kafka/network/RequestConvertToJson.scala | 36 +++--- .../main/scala/kafka/server/KafkaApis.scala | 2 +- .../kafka/tools/TestRaftRequestHandler.scala | 2 +- .../kafka/network/RequestChannelTest.scala | 4 +- .../network/RequestConvertToJsonTest.scala | 104 ++++++++++++++---- .../unit/kafka/network/SocketServerTest.scala | 5 +- .../kafka/message/JsonConverterGenerator.java | 41 +++++-- .../metadata/MetadataRequestBenchmark.java | 2 +- .../kafka/raft/FileBasedStateStore.java | 4 +- 11 files changed, 143 insertions(+), 64 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/common/message/SimpleExampleMessageTest.java b/clients/src/test/java/org/apache/kafka/common/message/SimpleExampleMessageTest.java index 0b3d92d11dbbe..a11960b5a4a67 100644 --- a/clients/src/test/java/org/apache/kafka/common/message/SimpleExampleMessageTest.java +++ b/clients/src/test/java/org/apache/kafka/common/message/SimpleExampleMessageTest.java @@ -395,7 +395,7 @@ private void testRoundTrip(SimpleExampleMessageData message, // Check JSON serialization JsonNode serializedJson = SimpleExampleMessageDataJsonConverter.write(message, version, true); - SimpleExampleMessageData messageFromJson = SimpleExampleMessageDataJsonConverter.read(serializedJson, version, true); + SimpleExampleMessageData messageFromJson = SimpleExampleMessageDataJsonConverter.read(serializedJson, version); validator.accept(messageFromJson); assertEquals(message, messageFromJson); assertEquals(message.hashCode(), messageFromJson.hashCode()); diff --git a/core/src/main/scala/kafka/network/RequestChannel.scala b/core/src/main/scala/kafka/network/RequestChannel.scala index d332d337d78c5..b984e8ce25213 100644 --- a/core/src/main/scala/kafka/network/RequestChannel.scala +++ b/core/src/main/scala/kafka/network/RequestChannel.scala @@ -202,7 +202,7 @@ object RequestChannel extends Logging { } } - trace(s"Processor $processor received request: ${RequestConvertToJson.requestDesc(header, loggableRequest, true)}") + trace(s"Processor $processor received request: ${RequestConvertToJson.requestDesc(header, loggableRequest)}") def requestThreadTimeNanos: Long = { if (apiLocalCompleteTimeNanos == -1L) apiLocalCompleteTimeNanos = Time.SYSTEM.nanoseconds @@ -263,9 +263,8 @@ object RequestChannel extends Logging { recordNetworkThreadTimeCallback.foreach(record => record(networkThreadTimeNanos)) if (isRequestLoggingEnabled) { - val detailsEnabled = requestLogger.underlying.isTraceEnabled val desc = RequestConvertToJson.requestDescMetrics(header, response, loggableRequest, - context, session, detailsEnabled, + context, session, totalTimeMs, requestQueueTimeMs, apiLocalTimeMs, apiRemoteTimeMs, apiThrottleTimeMs, responseQueueTimeMs, responseSendTimeMs, temporaryMemoryBytes, diff --git a/core/src/main/scala/kafka/network/RequestConvertToJson.scala b/core/src/main/scala/kafka/network/RequestConvertToJson.scala index ecc7cafa36153..b964caf308380 100644 --- a/core/src/main/scala/kafka/network/RequestConvertToJson.scala +++ b/core/src/main/scala/kafka/network/RequestConvertToJson.scala @@ -162,12 +162,19 @@ object RequestConvertToJson { } } - def requestHeaderNode(header: RequestHeader, verbose: Boolean): JsonNode = { - val node = RequestHeaderDataJsonConverter.write(header.data(), header.headerVersion(), verbose).asInstanceOf[ObjectNode] + 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, req: AbstractRequest): JsonNode = { + val node = new ObjectNode(JsonNodeFactory.instance) + node.set("requestHeader", requestHeaderNode(header)) + node.set("request", request(req, false)) + node + } + def clientInfoNode(clientInfo: ClientInformation): JsonNode = { val node = new ObjectNode(JsonNodeFactory.instance) node.set("softwareName", new TextNode(clientInfo.softwareName())) @@ -176,14 +183,12 @@ object RequestConvertToJson { } def requestDescMetrics(header: RequestHeader, res: Response, req: AbstractRequest, - context: RequestContext, session: Session, verbose: Boolean, + context: RequestContext, session: Session, totalTimeMs: Double, requestQueueTimeMs: Double, apiLocalTimeMs: Double, apiRemoteTimeMs: Double, apiThrottleTimeMs: Long, responseQueueTimeMs: Double, responseSendTimeMs: Double, temporaryMemoryBytes: Long, messageConversionsTimeMs: Double): JsonNode = { - val node = new ObjectNode(JsonNodeFactory.instance) - node.set("requestHeader", requestHeaderNode(header, verbose)) - node.set("request", request(req, verbose)) + val node = requestDesc(header, req).asInstanceOf[ObjectNode] node.set("response", res.responseLog.getOrElse(new TextNode(""))) node.set("connection", new TextNode(context.connectionId)) node.set("totalTimeMs", new DoubleNode(totalTimeMs)) @@ -204,13 +209,6 @@ object RequestConvertToJson { node } - def requestDesc(header: RequestHeader, req: AbstractRequest, verbose: Boolean): JsonNode = { - val node = new ObjectNode(JsonNodeFactory.instance) - node.set("requestHeader", requestHeaderNode(header, verbose)) - node.set("request", request(req, verbose)) - node - } - /** * Temporary until switch to use the generated schemas. */ @@ -245,7 +243,7 @@ object RequestConvertToJson { /** * Temporary until switch to use the generated schemas. */ - def produceRequestNode(request: ProduceRequest, version: Short, verbose: Boolean): JsonNode = { + def produceRequestNode(request: ProduceRequest, version: Short, serializeRecords: Boolean): JsonNode = { val node = new ObjectNode(JsonNodeFactory.instance) if (version >= 3) { if (request.transactionalId == null) { @@ -265,10 +263,14 @@ object RequestConvertToJson { partitions.forEach { (partitionIndex, partitionData) => val partitionNode = new ObjectNode(JsonNodeFactory.instance) partitionNode.set("partitionIndex", new IntNode(partitionIndex)) - if (partitionData == null) + if (partitionData == null) { partitionNode.set("records", NullNode.instance) - else - partitionNode.set("records", new BinaryNode(util.Arrays.copyOf(partitionData.buffer().array(), partitionData.validBytes()))) + } else { + if (serializeRecords) + partitionNode.set("records", new BinaryNode(util.Arrays.copyOf(partitionData.buffer().array(), partitionData.validBytes()))) + else + partitionNode.set("records", new IntNode(partitionData.validBytes())) + } partitionsArray.add(partitionNode) } topicNode.set("partitions", partitionsArray) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 047e0d3241854..77c52b6104e27 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -183,7 +183,7 @@ class KafkaApis(val requestChannel: RequestChannel, */ override def handle(request: RequestChannel.Request): Unit = { try { - trace(s"Handling request:${RequestConvertToJson.requestDesc(request.header, request.loggableRequest, true).toString} from connection ${request.context.connectionId};" + + trace(s"Handling request:${RequestConvertToJson.requestDesc(request.header, request.loggableRequest).toString} from connection ${request.context.connectionId};" + s"securityProtocol:${request.context.securityProtocol},principal:${request.context.principal}") request.envelope.foreach { envelope => diff --git a/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala b/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala index dc5b7f6c13a88..7f9a6602c069e 100644 --- a/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala +++ b/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala @@ -40,7 +40,7 @@ class TestRaftRequestHandler( override def handle(request: RequestChannel.Request): Unit = { try { - trace(s"Handling request:${RequestConvertToJson.requestDesc(request.header, request.loggableRequest, true)} from connection ${request.context.connectionId};" + + trace(s"Handling request:${RequestConvertToJson.requestDesc(request.header, request.loggableRequest)} from connection ${request.context.connectionId};" + s"securityProtocol:${request.context.securityProtocol},principal:${request.context.principal}") request.header.apiKey match { case ApiKeys.VOTE diff --git a/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala b/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala index 834a2df6bc718..bbb2fc2ada1db 100644 --- a/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala +++ b/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala @@ -54,7 +54,7 @@ class RequestChannelTest { val loggableAlterConfigs = alterConfigs.loggableRequest.asInstanceOf[AlterConfigsRequest] val loggedConfig = loggableAlterConfigs.configs.get(resource) assertEquals(expectedValues, toMap(loggedConfig)) - val alterConfigsDesc = RequestConvertToJson.requestDesc(alterConfigs.header, alterConfigs.loggableRequest, true).toString + val alterConfigsDesc = RequestConvertToJson.requestDesc(alterConfigs.header, alterConfigs.loggableRequest).toString assertFalse(s"Sensitive config logged $alterConfigsDesc", alterConfigsDesc.contains(sensitiveValue)) } @@ -118,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 = RequestConvertToJson.requestDesc(alterConfigs.header, alterConfigs.loggableRequest, true).toString + val alterConfigsDesc = RequestConvertToJson.requestDesc(alterConfigs.header, alterConfigs.loggableRequest).toString assertFalse(s"Sensitive config logged $alterConfigsDesc", alterConfigsDesc.contains(sensitiveValue)) } diff --git a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala index dfaf29afc1fff..778b52b915bc1 100644 --- a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala +++ b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala @@ -17,17 +17,25 @@ package kafka.network +import java.net.InetAddress import java.nio.ByteBuffer import java.util import java.util.{Collections, Optional} +import com.fasterxml.jackson.databind.JsonNode +import com.fasterxml.jackson.databind.node.ObjectNode +import kafka.network import org.apache.kafka.common.TopicPartition +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, Errors} -import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch, TimestampType} +import org.apache.kafka.common.record.{MemoryRecords, RecordBatch} import org.apache.kafka.common.requests.OffsetsForLeaderEpochRequest.PartitionData 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 @@ -38,14 +46,14 @@ class RequestConvertToJsonTest { def testAllRequestTypesHandled(): Unit = { val unhandledKeys = ArrayBuffer[String]() ApiKeys.values().foreach(key => { - val version: Short = 0 + val version: Short = key.latestVersion() var req: AbstractRequest = null if (key == ApiKeys.PRODUCE) { // There's inconsistency with the toStruct schema in ProduceRequest // and ProduceRequestDataJsonConverters where the field names don't // match so the struct does not have the correct field names. This is // a temporary workaround until ProduceRequest starts using ProduceRequestData - req = ProduceRequest.Builder.forCurrentMagic(0.toShort, 10000, new util.HashMap[TopicPartition, MemoryRecords]()).build() + req = ProduceRequest.Builder.forCurrentMagic(0.toShort, 10000, new util.HashMap[TopicPartition, MemoryRecords].build() } else { val struct = ApiMessageType.fromApiKey(key.id).newRequest().toStruct(version) req = AbstractRequest.parseRequest(key, version, struct) @@ -53,7 +61,7 @@ class RequestConvertToJsonTest { try { RequestConvertToJson.request(req, false) } catch { - case _ : AssertionError => unhandledKeys += key.toString + case _ : IllegalStateException => unhandledKeys += key.toString } }) assertEquals("Unhandled request keys", ArrayBuffer.empty, unhandledKeys) @@ -63,13 +71,13 @@ class RequestConvertToJsonTest { def testAllResponseTypesHandled(): Unit = { val unhandledKeys = ArrayBuffer[String]() ApiKeys.values().foreach(key => { - val version: Short = 0 + val version: Short = key.latestVersion() val struct = ApiMessageType.fromApiKey(key.id).newResponse().toStruct(version) val res = AbstractResponse.parseResponse(key, struct, version) try { RequestConvertToJson.response(res, version, false) } catch { - case _ : AssertionError => unhandledKeys += key.toString + case _ : IllegalStateException => unhandledKeys += key.toString } }) assertEquals("Unhandled response keys", ArrayBuffer.empty, unhandledKeys) @@ -80,11 +88,11 @@ class RequestConvertToJsonTest { val partitionDataMap = new util.HashMap[TopicPartition, PartitionData] partitionDataMap.put(new TopicPartition("topic1", 0), new PartitionData(Optional.of(0), 1)) - val version: Short = 3 + val version: Short = ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion val request = OffsetsForLeaderEpochRequest.Builder.forConsumer(partitionDataMap).build(version) val actualNode = RequestConvertToJson.request(request, true) - val requestData = OffsetForLeaderEpochRequestDataJsonConverter.read(actualNode, version, true) + val requestData = OffsetForLeaderEpochRequestDataJsonConverter.read(actualNode, version) val expectedNode = OffsetForLeaderEpochRequestDataJsonConverter.write(requestData, version, true) assertEquals(expectedNode, actualNode) @@ -92,18 +100,15 @@ class RequestConvertToJsonTest { @Test def testFormatOfProduceRequestNode(): Unit = { - val buffer = ByteBuffer.allocate(256) - val builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, TimestampType.CREATE_TIME, 0L) - builder.append(10L, null, "a".getBytes) - val produceData = new util.HashMap[TopicPartition, MemoryRecords] - produceData.put(new TopicPartition("test", 0), builder.build) - - val version: Short = 3 - val request = ProduceRequest.Builder.forCurrentMagic(1.toShort, 5000, produceData).build(version) - val actualNode = RequestConvertToJson.request(request, true) + val produceDataMap = new util.HashMap[TopicPartition, MemoryRecords] + + val version: Short = ApiKeys.PRODUCE.latestVersion + val serializeRecords: Boolean = false; + val request = ProduceRequest.Builder.forMagic(2, 0.toShort, 0, produceDataMap, "").build() + val actualNode = RequestConvertToJson.request(request, serializeRecords) - val requestData = ProduceRequestDataJsonConverter.read(actualNode, version, true) - val expectedNode = ProduceRequestDataJsonConverter.write(requestData, version, true) + val requestData = new ProduceRequestData() + val expectedNode = ProduceRequestDataJsonConverter.write(requestData, version, serializeRecords) assertEquals(expectedNode, actualNode) } @@ -113,11 +118,11 @@ class RequestConvertToJsonTest { val endOffsetMap = new util.HashMap[TopicPartition, EpochEndOffset] endOffsetMap.put(new TopicPartition("topic1", 0), new EpochEndOffset(1, 10L)) - val version: Short = 3 + val version: Short = ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion val response = new OffsetsForLeaderEpochResponse(endOffsetMap) val actualNode = RequestConvertToJson.response(response, version, true) - val requestData = OffsetForLeaderEpochResponseDataJsonConverter.read(actualNode, version, true) + val requestData = OffsetForLeaderEpochResponseDataJsonConverter.read(actualNode, version) val expectedNode = OffsetForLeaderEpochResponseDataJsonConverter.write(requestData, version, true) assertEquals(expectedNode, actualNode) @@ -129,13 +134,66 @@ class RequestConvertToJsonTest { val partResponse = new ProduceResponse.PartitionResponse(Errors.NONE, 10000, RecordBatch.NO_TIMESTAMP, 100, Collections.singletonList(new ProduceResponse.RecordError(3, "Record error")), "Produce failed") responseData.put(new TopicPartition("topic1", 0), partResponse) - val version: Short = 3 + val version: Short = ApiKeys.PRODUCE.latestVersion val response = new ProduceResponse(responseData) val actualNode = RequestConvertToJson.response(response, version, true) - val requestData = ProduceResponseDataJsonConverter.read(actualNode, version, true) + val requestData = ProduceResponseDataJsonConverter.read(actualNode, version) val expectedNode = ProduceResponseDataJsonConverter.write(requestData, version, true) assertEquals(expectedNode, actualNode) } + + @Test def testFieldsRequestDescMetrics(): Unit = { + val expectedFields = Set("requestHeader", "request", "response", "connection", + "totalTimeMs", "requestQueueTimeMs", "localTimeMs", "remoteTimeMs", "throttleTimeMs", + "responseQueueTimeMs", "sendTimeMs", "securityProtocol", "principal", "listener", + "clientInformation", "softwareName", "softwareVersion", "temporaryMemoryBytes", "messageConversionsTime") + + val req = request(new AlterIsrRequest(new AlterIsrRequestData(), 0)) + val byteBuffer = req.body[AbstractRequest].serialize(req.header) + val send = new NetworkSend(req.context.connectionId, byteBuffer) + val headerLog = RequestConvertToJson.requestHeaderNode(req.header) + val res = new RequestChannel.SendResponse(req, send, Some(headerLog), None) + + val node = RequestConvertToJson.requestDescMetrics(req.header, res, req.loggableRequest, req.context, req.session, + 1, 1, 1, 1, 1, 1, 1, 1, 1).asInstanceOf[ObjectNode] + val foundFields = getFieldNames(node) + + assertEquals(expectedFields, foundFields) + } + + def request(req: AbstractRequest): RequestChannel.Request = { + val buffer = req.serialize(new RequestHeader(req.api, req.version, "client-id", 1)) + 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")) + } + + def getFieldNames(node: JsonNode): Set[String] = { + var fieldNames = Set[String]() + node.fields.forEachRemaining { entry => + fieldNames += entry.getKey + if (entry.getValue.isObject && entry.getKey != "request" && entry.getKey != "requestHeader" && entry.getKey != "response") { + fieldNames ++= getFieldNames(entry.getValue) + } + } + fieldNames + } } diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index 43c966018eb08..927a932fa3639 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 @@ -677,7 +678,7 @@ class SocketServerTest { server.dataPlaneRequestChannel.sendResponse(response) } val throttledChannel = new ThrottledChannel(request, new MockTime(), 100, channelThrottlingCallback) - val headerLog = RequestConvertToJson.requestHeaderNode(request.header, true) + val headerLog = RequestConvertToJson.requestHeaderNode(request.header) val response = if (!noOpResponse) new RequestChannel.SendResponse(request, send, Some(headerLog), None) 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 302284ca8a8c8..f9afdb5d5c51a 100644 --- a/generator/src/main/java/org/apache/kafka/message/JsonConverterGenerator.java +++ b/generator/src/main/java/org/apache/kafka/message/JsonConverterGenerator.java @@ -77,13 +77,14 @@ private void generateConverters(String name, Versions parentVersions) { generateRead(name, spec, parentVersions); generateWrite(name, spec, parentVersions); + generateOverloadWrite(name); } private void generateRead(String className, StructSpec struct, Versions parentVersions) { headerGenerator.addImport(MessageGenerator.JSON_NODE_CLASS); - buffer.printf("public static %s read(JsonNode _node, short _version, boolean _verbose) {%n", + buffer.printf("public static %s read(JsonNode _node, short _version) {%n", className); buffer.incrementIndent(); buffer.printf("%s _object = new %s();%n", className, className); @@ -227,13 +228,9 @@ private void generateVariableLengthTargetFromJson(Target target, Versions curVer headerGenerator.addImport(MessageGenerator.MESSAGE_UTIL_CLASS); headerGenerator.addImport(MessageGenerator.BYTE_BUFFER_CLASS); headerGenerator.addImport(MessageGenerator.MEMORY_RECORDS_CLASS); - buffer.printf("if (_verbose) {%n"); - buffer.incrementIndent(); buffer.printf("%s;%n", target.assignmentStatement( String.format("MemoryRecords.readableRecords(ByteBuffer.wrap(MessageUtil.jsonNodeToBinary(%s, \"%s\")))", target.sourceVariable(), target.humanReadableName()))); - buffer.decrementIndent(); - buffer.printf("}%n"); } else if (target.field().type().isArray()) { buffer.printf("if (!%s.isArray()) {%n", target.sourceVariable()); buffer.incrementIndent(); @@ -254,18 +251,27 @@ private void generateVariableLengthTargetFromJson(Target target, Versions curVer buffer.printf("}%n"); } else if (target.field().type().isStruct()) { buffer.printf("%s;%n", target.assignmentStatement( - String.format("%s%s.read(%s, _version, _verbose)", + String.format("%s%s.read(%s, _version)", target.field().type().toString(), SUFFIX, target.sourceVariable()))); } else { throw new RuntimeException("Unexpected type " + target.field().type()); } } + 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, boolean _verbose) {%n", + buffer.printf("public static JsonNode write(%s _object, short _version, boolean _serializeRecords) {%n", className); buffer.incrementIndent(); VersionConditional.forVersions(struct.versions(), parentVersions). @@ -379,14 +385,27 @@ private void generateVariableLengthTargetToJson(Target target, Versions versions target.sourceVariable()))); } else { headerGenerator.addImport(MessageGenerator.ARRAYS_CLASS); + headerGenerator.addImport(MessageGenerator.INT_NODE_CLASS); + buffer.printf("if (_serializeRecords) {%n"); + buffer.incrementIndent(); + buffer.printf("%s;%n", target.assignmentStatement( + String.format("new BinaryNode(Arrays.copyOf(%s, %s.length))", + target.sourceVariable(), target.sourceVariable()))); + buffer.decrementIndent(); + buffer.printf("} else {%n"); + buffer.incrementIndent(); buffer.printf("%s;%n", target.assignmentStatement( - String.format("new BinaryNode(Arrays.copyOf(%s, %s.length))", - target.sourceVariable(), target.sourceVariable()))); + String.format("new IntNode(%s.length)", target.sourceVariable()))); + buffer.decrementIndent(); + buffer.printf("}%n"); } } else if (target.field().type().isRecords()) { headerGenerator.addImport(MessageGenerator.INT_NODE_CLASS); headerGenerator.addImport(MessageGenerator.BINARY_NODE_CLASS); - buffer.printf("if (_verbose) {%n"); + // 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(); @@ -416,7 +435,7 @@ private void generateVariableLengthTargetToJson(Target target, Versions versions 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, _verbose)", + 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/metadata/MetadataRequestBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/MetadataRequestBenchmark.java index a081530454824..fbc4ccd1df7ca 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 @@ -217,6 +217,6 @@ public void testMetadataRequestForAllTopics() { @Benchmark public String testRequestToJson() { - return RequestConvertToJson.requestDesc(allTopicMetadataRequest.header(), allTopicMetadataRequest.loggableRequest(), false).toString(); + return RequestConvertToJson.requestDesc(allTopicMetadataRequest.header(), allTopicMetadataRequest.loggableRequest()).toString(); } } diff --git a/raft/src/main/java/org/apache/kafka/raft/FileBasedStateStore.java b/raft/src/main/java/org/apache/kafka/raft/FileBasedStateStore.java index ea9f7f1c67631..ae96279e6d41a 100644 --- a/raft/src/main/java/org/apache/kafka/raft/FileBasedStateStore.java +++ b/raft/src/main/java/org/apache/kafka/raft/FileBasedStateStore.java @@ -90,7 +90,7 @@ private QuorumStateData readStateFromFile(File file) throws IOException { } final short dataVersion = dataVersionNode.shortValue(); - return QuorumStateDataJsonConverter.read(dataObject, dataVersion, true); + return QuorumStateDataJsonConverter.read(dataObject, dataVersion); } } @@ -140,7 +140,7 @@ private void writeElectionStateToFile(final File stateFile, QuorumStateData stat new OutputStreamWriter(fileOutputStream, StandardCharsets.UTF_8))) { short version = state.highestSupportedVersion(); - ObjectNode jsonState = (ObjectNode) QuorumStateDataJsonConverter.write(state, version, true); + ObjectNode jsonState = (ObjectNode) QuorumStateDataJsonConverter.write(state, version); jsonState.set(DATA_VERSION, new ShortNode(version)); writer.write(jsonState.toString()); writer.flush(); From 0fa879aed005435b8f56534322123f639598296a Mon Sep 17 00:00:00 2001 From: anatasiavela Date: Thu, 5 Nov 2020 18:12:33 -0800 Subject: [PATCH 14/22] add parentheses --- .../scala/unit/kafka/network/RequestConvertToJsonTest.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala index 778b52b915bc1..43a1ed55ee0ba 100644 --- a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala +++ b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala @@ -53,7 +53,7 @@ class RequestConvertToJsonTest { // and ProduceRequestDataJsonConverters where the field names don't // match so the struct does not have the correct field names. This is // a temporary workaround until ProduceRequest starts using ProduceRequestData - req = ProduceRequest.Builder.forCurrentMagic(0.toShort, 10000, new util.HashMap[TopicPartition, MemoryRecords].build() + req = ProduceRequest.Builder.forCurrentMagic(0.toShort, 10000, new util.HashMap[TopicPartition, MemoryRecords]).build() } else { val struct = ApiMessageType.fromApiKey(key.id).newRequest().toStruct(version) req = AbstractRequest.parseRequest(key, version, struct) From 4dd1524301a9dda7dec710d59c9eae204df36a8c Mon Sep 17 00:00:00 2001 From: anatasiavela Date: Thu, 12 Nov 2020 15:02:15 -0800 Subject: [PATCH 15/22] remove verbose tag in unnecessary places --- .../message/SimpleExampleMessageTest.java | 2 +- .../kafka/network/RequestConvertToJson.scala | 244 +++++++++--------- .../kafka/tools/TestRaftRequestHandler.scala | 2 +- .../kafka/network/RequestChannelTest.scala | 2 +- .../network/RequestConvertToJsonTest.scala | 89 ++++--- .../kafka/message/JsonConverterGenerator.java | 10 - .../jmh/common/FetchRequestBenchmark.java | 2 +- .../common/ListOffsetRequestBenchmark.java | 2 +- .../jmh/common/ProduceRequestBenchmark.java | 2 +- 9 files changed, 177 insertions(+), 178 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/common/message/SimpleExampleMessageTest.java b/clients/src/test/java/org/apache/kafka/common/message/SimpleExampleMessageTest.java index a11960b5a4a67..b3e45e5a64e09 100644 --- a/clients/src/test/java/org/apache/kafka/common/message/SimpleExampleMessageTest.java +++ b/clients/src/test/java/org/apache/kafka/common/message/SimpleExampleMessageTest.java @@ -394,7 +394,7 @@ private void testRoundTrip(SimpleExampleMessageData message, assertEquals(message.hashCode(), messageFromStruct.hashCode()); // Check JSON serialization - JsonNode serializedJson = SimpleExampleMessageDataJsonConverter.write(message, version, true); + JsonNode serializedJson = SimpleExampleMessageDataJsonConverter.write(message, version); SimpleExampleMessageData messageFromJson = SimpleExampleMessageDataJsonConverter.read(serializedJson, version); validator.accept(messageFromJson); assertEquals(message, messageFromJson); diff --git a/core/src/main/scala/kafka/network/RequestConvertToJson.scala b/core/src/main/scala/kafka/network/RequestConvertToJson.scala index b964caf308380..3ace86f4e4741 100644 --- a/core/src/main/scala/kafka/network/RequestConvertToJson.scala +++ b/core/src/main/scala/kafka/network/RequestConvertToJson.scala @@ -32,131 +32,131 @@ import org.apache.kafka.common.utils.CollectionUtils import scala.jdk.CollectionConverters._ object RequestConvertToJson { - def request(request: AbstractRequest, verbose: Boolean): JsonNode = { + def request(request: AbstractRequest): JsonNode = { request match { - case req: AddOffsetsToTxnRequest => AddOffsetsToTxnRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: AddPartitionsToTxnRequest => AddPartitionsToTxnRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: AlterClientQuotasRequest => AlterClientQuotasRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: AlterConfigsRequest => AlterConfigsRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: AlterIsrRequest => AlterIsrRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: AlterPartitionReassignmentsRequest => AlterPartitionReassignmentsRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: AlterReplicaLogDirsRequest => AlterReplicaLogDirsRequestDataJsonConverter.write(req.data, request.version, verbose) - case res: AlterUserScramCredentialsRequest => AlterUserScramCredentialsRequestDataJsonConverter.write(res.data, request.version, verbose) - case req: ApiVersionsRequest => ApiVersionsRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: BeginQuorumEpochRequest => BeginQuorumEpochRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: ControlledShutdownRequest => ControlledShutdownRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: CreateAclsRequest => CreateAclsRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: CreateDelegationTokenRequest => CreateDelegationTokenRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: CreatePartitionsRequest => CreatePartitionsRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: CreateTopicsRequest => CreateTopicsRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: DeleteAclsRequest => DeleteAclsRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: DeleteGroupsRequest => DeleteGroupsRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: DeleteRecordsRequest => DeleteRecordsRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: DeleteTopicsRequest => DeleteTopicsRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: DescribeAclsRequest => DescribeAclsRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: DescribeClientQuotasRequest => DescribeClientQuotasRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: DescribeConfigsRequest => DescribeConfigsRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: DescribeDelegationTokenRequest => DescribeDelegationTokenRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: DescribeGroupsRequest => DescribeGroupsRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: DescribeLogDirsRequest => DescribeLogDirsRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: DescribeQuorumRequest => DescribeQuorumRequestDataJsonConverter.write(req.data, request.version, verbose) - case res: DescribeUserScramCredentialsRequest => DescribeUserScramCredentialsRequestDataJsonConverter.write(res.data, request.version, verbose) - case req: ElectLeadersRequest => ElectLeadersRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: EndTxnRequest => EndTxnRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: EndQuorumEpochRequest => EndQuorumEpochRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: ExpireDelegationTokenRequest => ExpireDelegationTokenRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: FetchRequest => FetchRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: FindCoordinatorRequest => FindCoordinatorRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: HeartbeatRequest => HeartbeatRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: IncrementalAlterConfigsRequest => IncrementalAlterConfigsRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: InitProducerIdRequest => InitProducerIdRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: JoinGroupRequest => JoinGroupRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: LeaderAndIsrRequest => LeaderAndIsrRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: LeaveGroupRequest => LeaveGroupRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: ListGroupsRequest => ListGroupsRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: ListOffsetRequest => ListOffsetRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: ListPartitionReassignmentsRequest => ListPartitionReassignmentsRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: MetadataRequest => MetadataRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: OffsetCommitRequest => OffsetCommitRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: OffsetDeleteRequest => OffsetDeleteRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: OffsetFetchRequest => OffsetFetchRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: OffsetsForLeaderEpochRequest => offsetsForLeaderEpochRequestNode(req, request.version, verbose) - case req: ProduceRequest => produceRequestNode(req, request.version, verbose) - case req: RenewDelegationTokenRequest => RenewDelegationTokenRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: SaslAuthenticateRequest => SaslAuthenticateRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: SaslHandshakeRequest => SaslHandshakeRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: StopReplicaRequest => StopReplicaRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: SyncGroupRequest => SyncGroupRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: TxnOffsetCommitRequest => TxnOffsetCommitRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: UpdateFeaturesRequest => UpdateFeaturesRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: UpdateMetadataRequest => UpdateMetadataRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: VoteRequest => VoteRequestDataJsonConverter.write(req.data, request.version, verbose) - case req: WriteTxnMarkersRequest => WriteTxnMarkersRequestDataJsonConverter.write(req.data, request.version, verbose) + 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: 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 => offsetsForLeaderEpochRequestNode(req, request.version) + case req: ProduceRequest => produceRequestNode(req, 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.api} is not currently handled in `request`, the " + "code should be updated to do so."); } } - def response(response: AbstractResponse, version: Short, verbose: Boolean): JsonNode = { + def response(response: AbstractResponse, version: Short): JsonNode = { response match { - case res: AddOffsetsToTxnResponse => AddOffsetsToTxnResponseDataJsonConverter.write(res.data, version, verbose) - case res: AddPartitionsToTxnResponse => AddPartitionsToTxnResponseDataJsonConverter.write(res.data, version, verbose) - case res: AlterClientQuotasResponse => AlterClientQuotasResponseDataJsonConverter.write(res.data, version, verbose) - case res: AlterConfigsResponse => AlterConfigsResponseDataJsonConverter.write(res.data, version, verbose) - case res: AlterIsrResponse => AlterIsrResponseDataJsonConverter.write(res.data, version, verbose) - case res: AlterPartitionReassignmentsResponse => AlterPartitionReassignmentsResponseDataJsonConverter.write(res.data, version, verbose) - case res: AlterReplicaLogDirsResponse => AlterReplicaLogDirsResponseDataJsonConverter.write(res.data, version, verbose) - case res: AlterUserScramCredentialsResponse => AlterUserScramCredentialsResponseDataJsonConverter.write(res.data, version, verbose) - case res: ApiVersionsResponse => ApiVersionsResponseDataJsonConverter.write(res.data, version, verbose) - case res: BeginQuorumEpochResponse => BeginQuorumEpochResponseDataJsonConverter.write(res.data, version, verbose) - case res: ControlledShutdownResponse => ControlledShutdownResponseDataJsonConverter.write(res.data, version, verbose) - case res: CreateAclsResponse => CreateAclsResponseDataJsonConverter.write(res.data, version, verbose) - case res: CreateDelegationTokenResponse => CreateDelegationTokenResponseDataJsonConverter.write(res.data, version, verbose) - case res: CreatePartitionsResponse => CreatePartitionsResponseDataJsonConverter.write(res.data, version, verbose) - case res: CreateTopicsResponse => CreateTopicsResponseDataJsonConverter.write(res.data, version, verbose) - case res: DeleteAclsResponse => DeleteAclsResponseDataJsonConverter.write(res.data, version, verbose) - case res: DeleteGroupsResponse => DeleteGroupsResponseDataJsonConverter.write(res.data, version, verbose) - case res: DeleteRecordsResponse => DeleteRecordsResponseDataJsonConverter.write(res.data, version, verbose) - case res: DeleteTopicsResponse => DeleteTopicsResponseDataJsonConverter.write(res.data, version, verbose) - case res: DescribeAclsResponse => DescribeAclsResponseDataJsonConverter.write(res.data, version, verbose) - case res: DescribeClientQuotasResponse => DescribeClientQuotasResponseDataJsonConverter.write(res.data, version, verbose) - case res: DescribeConfigsResponse => DescribeConfigsResponseDataJsonConverter.write(res.data, version, verbose) - case res: DescribeDelegationTokenResponse => DescribeDelegationTokenResponseDataJsonConverter.write(res.data, version, verbose) - case res: DescribeGroupsResponse => DescribeGroupsResponseDataJsonConverter.write(res.data, version, verbose) - case res: DescribeLogDirsResponse => DescribeLogDirsResponseDataJsonConverter.write(res.data, version, verbose) - case res: DescribeQuorumResponse => DescribeQuorumResponseDataJsonConverter.write(res.data, version, verbose) - case res: DescribeUserScramCredentialsResponse => DescribeUserScramCredentialsResponseDataJsonConverter.write(res.data, version, verbose) - case res: ElectLeadersResponse => ElectLeadersResponseDataJsonConverter.write(res.data, version, verbose) - case res: EndTxnResponse => EndTxnResponseDataJsonConverter.write(res.data, version, verbose) - case res: EndQuorumEpochResponse => EndQuorumEpochResponseDataJsonConverter.write(res.data, version, verbose) - case res: ExpireDelegationTokenResponse => ExpireDelegationTokenResponseDataJsonConverter.write(res.data, version, verbose) - case res: FetchResponse[_] => FetchResponseDataJsonConverter.write(res.data, version, verbose) - case res: FindCoordinatorResponse => FindCoordinatorResponseDataJsonConverter.write(res.data, version, verbose) - case res: HeartbeatResponse => HeartbeatResponseDataJsonConverter.write(res.data, version, verbose) - case res: IncrementalAlterConfigsResponse => IncrementalAlterConfigsResponseDataJsonConverter.write(res.data, version, verbose) - case res: InitProducerIdResponse => InitProducerIdResponseDataJsonConverter.write(res.data, version, verbose) - case res: JoinGroupResponse => JoinGroupResponseDataJsonConverter.write(res.data, version, verbose) - case res: LeaderAndIsrResponse => LeaderAndIsrResponseDataJsonConverter.write(res.data, version, verbose) - case res: LeaveGroupResponse => LeaveGroupResponseDataJsonConverter.write(res.data, version, verbose) - case res: ListGroupsResponse => ListGroupsResponseDataJsonConverter.write(res.data, version, verbose) - case res: ListOffsetResponse => ListOffsetResponseDataJsonConverter.write(res.data, version, verbose) - case res: ListPartitionReassignmentsResponse => ListPartitionReassignmentsResponseDataJsonConverter.write(res.data, version, verbose) - case res: MetadataResponse => MetadataResponseDataJsonConverter.write(res.data, version, verbose) - case res: OffsetCommitResponse => OffsetCommitResponseDataJsonConverter.write(res.data, version, verbose) - case res: OffsetDeleteResponse => OffsetDeleteResponseDataJsonConverter.write(res.data, version, verbose) - case res: OffsetFetchResponse => OffsetFetchResponseDataJsonConverter.write(res.data, version, verbose) - case res: OffsetsForLeaderEpochResponse => offsetsForLeaderEpochResponseNode(res, version, verbose) - case res: ProduceResponse => produceResponseNode(res, version, verbose) - case res: RenewDelegationTokenResponse => RenewDelegationTokenResponseDataJsonConverter.write(res.data, version, verbose) - case res: SaslAuthenticateResponse => SaslAuthenticateResponseDataJsonConverter.write(res.data, version, verbose) - case res: SaslHandshakeResponse => SaslHandshakeResponseDataJsonConverter.write(res.data, version, verbose) - case res: StopReplicaResponse => StopReplicaResponseDataJsonConverter.write(res.data, version, verbose) - case res: SyncGroupResponse => SyncGroupResponseDataJsonConverter.write(res.data, version, verbose) - case res: TxnOffsetCommitResponse => TxnOffsetCommitResponseDataJsonConverter.write(res.data, version, verbose) - case res: UpdateFeaturesResponse => UpdateFeaturesResponseDataJsonConverter.write(res.data, version, verbose) - case res: UpdateMetadataResponse => UpdateMetadataResponseDataJsonConverter.write(res.data, version, verbose) - case res: WriteTxnMarkersResponse => WriteTxnMarkersResponseDataJsonConverter.write(res.data, version, verbose) - case res: VoteResponse => VoteResponseDataJsonConverter.write(res.data, version, verbose) + 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: 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 => offsetsForLeaderEpochResponseNode(res, version) + case res: ProduceResponse => produceResponseNode(res, 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 is not currently handled in `response`, the " + "code should be updated to do so."); } @@ -171,7 +171,7 @@ object RequestConvertToJson { def requestDesc(header: RequestHeader, req: AbstractRequest): JsonNode = { val node = new ObjectNode(JsonNodeFactory.instance) node.set("requestHeader", requestHeaderNode(header)) - node.set("request", request(req, false)) + node.set("request", request(req)) node } @@ -212,7 +212,7 @@ object RequestConvertToJson { /** * Temporary until switch to use the generated schemas. */ - def offsetsForLeaderEpochRequestNode(request: OffsetsForLeaderEpochRequest, version: Short, verbose: Boolean): JsonNode = { + def offsetsForLeaderEpochRequestNode(request: OffsetsForLeaderEpochRequest, version: Short): JsonNode = { val node = new ObjectNode(JsonNodeFactory.instance) if (version >= 3) { node.set("replicaId", new IntNode(request.replicaId)) @@ -283,7 +283,7 @@ object RequestConvertToJson { /** * Temporary until switch to use the generated schemas. */ - def offsetsForLeaderEpochResponseNode(response: OffsetsForLeaderEpochResponse, version: Short, verbose: Boolean): JsonNode = { + def offsetsForLeaderEpochResponseNode(response: OffsetsForLeaderEpochResponse, version: Short): JsonNode = { val node = new ObjectNode(JsonNodeFactory.instance) if (version >= 2) { node.set("throttleTimeMs", new IntNode(response.throttleTimeMs)) @@ -312,7 +312,7 @@ object RequestConvertToJson { /** * Temporary until switch to use the generated schemas. */ - def produceResponseNode(response: ProduceResponse, version: Short, verbose: Boolean): JsonNode = { + def produceResponseNode(response: ProduceResponse, version: Short): JsonNode = { val node = new ObjectNode(JsonNodeFactory.instance) val topics = CollectionUtils.groupPartitionDataByTopic(response.responses) val responsesArray = new ArrayNode(JsonNodeFactory.instance) diff --git a/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala b/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala index 7f9a6602c069e..456d14c4164c3 100644 --- a/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala +++ b/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala @@ -95,7 +95,7 @@ class TestRaftRequestHandler( case Some(response) => val responseSend = request.context.buildResponseSend(response) val responseString = - if (RequestChannel.isRequestLoggingEnabled) Some(RequestConvertToJson.response(response, request.context.apiVersion, true)) + 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 bbb2fc2ada1db..6d14e2485968e 100644 --- a/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala +++ b/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala @@ -176,7 +176,7 @@ class RequestChannelTest { val alterConfigs = request(new AlterConfigsRequest.Builder(Collections.singletonMap(resource, new Config(entries.asJavaCollection)), true).build()) - assertTrue(isValidJson(RequestConvertToJson.request(alterConfigs.loggableRequest, true).toString)) + assertTrue(isValidJson(RequestConvertToJson.request(alterConfigs.loggableRequest).toString)) } private def isValidJson(str: String): Boolean = { diff --git a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala index 43a1ed55ee0ba..92cd0d4da6f62 100644 --- a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala +++ b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala @@ -22,8 +22,7 @@ import java.nio.ByteBuffer import java.util import java.util.{Collections, Optional} -import com.fasterxml.jackson.databind.JsonNode -import com.fasterxml.jackson.databind.node.ObjectNode +import com.fasterxml.jackson.databind.node.{DoubleNode, LongNode, ObjectNode, TextNode} import kafka.network import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.memory.MemoryPool @@ -45,41 +44,40 @@ class RequestConvertToJsonTest { @Test def testAllRequestTypesHandled(): Unit = { val unhandledKeys = ArrayBuffer[String]() - ApiKeys.values().foreach(key => { + ApiKeys.values().foreach { key => { val version: Short = key.latestVersion() - var req: AbstractRequest = null - if (key == ApiKeys.PRODUCE) { + val req = if (key == ApiKeys.PRODUCE) { // There's inconsistency with the toStruct schema in ProduceRequest // and ProduceRequestDataJsonConverters where the field names don't // match so the struct does not have the correct field names. This is // a temporary workaround until ProduceRequest starts using ProduceRequestData - req = ProduceRequest.Builder.forCurrentMagic(0.toShort, 10000, new util.HashMap[TopicPartition, MemoryRecords]).build() + ProduceRequest.Builder.forCurrentMagic(0.toShort, 10000, new util.HashMap[TopicPartition, MemoryRecords]).build() } else { val struct = ApiMessageType.fromApiKey(key.id).newRequest().toStruct(version) - req = AbstractRequest.parseRequest(key, version, struct) + AbstractRequest.parseRequest(key, version, struct) } try { - RequestConvertToJson.request(req, false) + 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 => { + ApiKeys.values().foreach { key => { val version: Short = key.latestVersion() val struct = ApiMessageType.fromApiKey(key.id).newResponse().toStruct(version) val res = AbstractResponse.parseResponse(key, struct, version) try { - RequestConvertToJson.response(res, version, false) + RequestConvertToJson.response(res, version) } catch { case _ : IllegalStateException => unhandledKeys += key.toString } - }) + }} assertEquals("Unhandled response keys", ArrayBuffer.empty, unhandledKeys) } @@ -90,7 +88,7 @@ class RequestConvertToJsonTest { val version: Short = ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion val request = OffsetsForLeaderEpochRequest.Builder.forConsumer(partitionDataMap).build(version) - val actualNode = RequestConvertToJson.request(request, true) + val actualNode = RequestConvertToJson.request(request) val requestData = OffsetForLeaderEpochRequestDataJsonConverter.read(actualNode, version) val expectedNode = OffsetForLeaderEpochRequestDataJsonConverter.write(requestData, version, true) @@ -103,12 +101,11 @@ class RequestConvertToJsonTest { val produceDataMap = new util.HashMap[TopicPartition, MemoryRecords] val version: Short = ApiKeys.PRODUCE.latestVersion - val serializeRecords: Boolean = false; val request = ProduceRequest.Builder.forMagic(2, 0.toShort, 0, produceDataMap, "").build() - val actualNode = RequestConvertToJson.request(request, serializeRecords) + val actualNode = RequestConvertToJson.request(request) val requestData = new ProduceRequestData() - val expectedNode = ProduceRequestDataJsonConverter.write(requestData, version, serializeRecords) + val expectedNode = ProduceRequestDataJsonConverter.write(requestData, version) assertEquals(expectedNode, actualNode) } @@ -120,7 +117,7 @@ class RequestConvertToJsonTest { val version: Short = ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion val response = new OffsetsForLeaderEpochResponse(endOffsetMap) - val actualNode = RequestConvertToJson.response(response, version, true) + val actualNode = RequestConvertToJson.response(response, version) val requestData = OffsetForLeaderEpochResponseDataJsonConverter.read(actualNode, version) val expectedNode = OffsetForLeaderEpochResponseDataJsonConverter.write(requestData, version, true) @@ -136,7 +133,7 @@ class RequestConvertToJsonTest { val version: Short = ApiKeys.PRODUCE.latestVersion val response = new ProduceResponse(responseData) - val actualNode = RequestConvertToJson.response(response, version, true) + val actualNode = RequestConvertToJson.response(response, version) val requestData = ProduceResponseDataJsonConverter.read(actualNode, version) val expectedNode = ProduceResponseDataJsonConverter.write(requestData, version, true) @@ -144,26 +141,49 @@ class RequestConvertToJsonTest { assertEquals(expectedNode, actualNode) } - @Test def testFieldsRequestDescMetrics(): Unit = { - val expectedFields = Set("requestHeader", "request", "response", "connection", - "totalTimeMs", "requestQueueTimeMs", "localTimeMs", "remoteTimeMs", "throttleTimeMs", - "responseQueueTimeMs", "sendTimeMs", "securityProtocol", "principal", "listener", - "clientInformation", "softwareName", "softwareVersion", "temporaryMemoryBytes", "messageConversionsTime") - + @Test + def testRequestDescMetrics(): Unit = { val req = request(new AlterIsrRequest(new AlterIsrRequestData(), 0)) val byteBuffer = req.body[AbstractRequest].serialize(req.header) val send = new NetworkSend(req.context.connectionId, byteBuffer) val headerLog = RequestConvertToJson.requestHeaderNode(req.header) val res = new RequestChannel.SendResponse(req, send, Some(headerLog), None) - val node = RequestConvertToJson.requestDescMetrics(req.header, res, req.loggableRequest, req.context, req.session, - 1, 1, 1, 1, 1, 1, 1, 1, 1).asInstanceOf[ObjectNode] - val foundFields = getFieldNames(node) + 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.loggableRequest).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, res, req.loggableRequest, req.context, req.session, + totalTimeMs, requestQueueTimeMs, apiLocalTimeMs, apiRemoteTimeMs, apiThrottleTimeMs, responseQueueTimeMs, + responseSendTimeMs, temporaryMemoryBytes, messageConversionsTimeMs).asInstanceOf[ObjectNode] - assertEquals(expectedFields, foundFields) + assertEquals(expectedNode, actualNode) } - def request(req: AbstractRequest): RequestChannel.Request = { + private def request(req: AbstractRequest): RequestChannel.Request = { val buffer = req.serialize(new RequestHeader(req.api, req.version, "client-id", 1)) val requestContext = newRequestContext(buffer) new network.RequestChannel.Request(processor = 1, @@ -185,15 +205,4 @@ class RequestConvertToJsonTest { SecurityProtocol.PLAINTEXT, new ClientInformation("name", "version")) } - - def getFieldNames(node: JsonNode): Set[String] = { - var fieldNames = Set[String]() - node.fields.forEachRemaining { entry => - fieldNames += entry.getKey - if (entry.getValue.isObject && entry.getKey != "request" && entry.getKey != "requestHeader" && entry.getKey != "response") { - fieldNames ++= getFieldNames(entry.getValue) - } - } - fieldNames - } } 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 f9afdb5d5c51a..9670874a8c9e3 100644 --- a/generator/src/main/java/org/apache/kafka/message/JsonConverterGenerator.java +++ b/generator/src/main/java/org/apache/kafka/message/JsonConverterGenerator.java @@ -385,19 +385,9 @@ private void generateVariableLengthTargetToJson(Target target, Versions versions target.sourceVariable()))); } else { headerGenerator.addImport(MessageGenerator.ARRAYS_CLASS); - headerGenerator.addImport(MessageGenerator.INT_NODE_CLASS); - buffer.printf("if (_serializeRecords) {%n"); - buffer.incrementIndent(); buffer.printf("%s;%n", target.assignmentStatement( String.format("new BinaryNode(Arrays.copyOf(%s, %s.length))", target.sourceVariable(), target.sourceVariable()))); - buffer.decrementIndent(); - buffer.printf("} else {%n"); - buffer.incrementIndent(); - buffer.printf("%s;%n", target.assignmentStatement( - String.format("new IntNode(%s.length)", target.sourceVariable()))); - buffer.decrementIndent(); - buffer.printf("}%n"); } } else if (target.field().type().isRecords()) { headerGenerator.addImport(MessageGenerator.INT_NODE_CLASS); 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 b5b6109cd6184..b9b2448b94ce5 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 @@ -129,6 +129,6 @@ public int testSerializeFetchRequestForReplica() throws IOException { @Benchmark public String testRequestToJson() { - return RequestConvertToJson.request(consumerRequest, false).toString(); + 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 index 58eaaf68cd219..83510046e3cc2 100644 --- 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 @@ -75,6 +75,6 @@ public void setup() { @Benchmark public String testRequestToJson() { - return RequestConvertToJson.request(offsetRequest, false).toString(); + 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 index c41a73d4e4064..ef07ca4ca6809 100644 --- 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 @@ -87,6 +87,6 @@ public void setup() { @Benchmark public String testRequestToJson() { - return RequestConvertToJson.request(produceRequest, false).toString(); + return RequestConvertToJson.request(produceRequest).toString(); } } From b102b67fd5c44cd2b01e2e11000176d87b786d8d Mon Sep 17 00:00:00 2001 From: anatasiavela Date: Thu, 19 Nov 2020 21:45:12 -0800 Subject: [PATCH 16/22] integrate with rebase --- .../kafka/clients/admin/KafkaAdminClient.java | 3 - .../kafka/common/requests/ProduceRequest.java | 4 + .../requests/ApiVersionsResponseTest.java | 61 ------- .../kafka/network/RequestConvertToJson.scala | 170 +----------------- .../network/RequestConvertToJsonTest.scala | 83 +-------- .../jmh/common/ProduceRequestBenchmark.java | 32 +--- 6 files changed, 18 insertions(+), 335 deletions(-) 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 f7ac1084cd899..4558e86d8b07f 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 @@ -4361,9 +4361,6 @@ void handleResponse(AbstractResponse response) { final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response; if (apiVersionsResponse.data().errorCode() == Errors.NONE.code()) { future.complete(createFeatureMetadata(apiVersionsResponse)); - } else if (options.sendRequestToController() && - apiVersionsResponse.data().errorCode() == Errors.NOT_CONTROLLER.code()) { - handleNotControllerError(Errors.NOT_CONTROLLER); } else { future.completeExceptionally(Errors.forCode(apiVersionsResponse.data().errorCode()).exception()); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java index 758631a1d87aa..9af463b9aa083 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -217,6 +217,10 @@ public String transactionalId() { return transactionalId; } + public ProduceRequestData data() { + return data; + } + public void clearPartitionRecords() { // lazily initialize partitionSizes. partitionSizes(); 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 0178b1e5eac15..b578e2c39db69 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 @@ -32,19 +32,6 @@ public class ApiVersionsResponseTest { - @Test - public void shouldCreateApiResponseOnlyWithKeysSupportedByMagicValue() { - final ApiVersionsResponse response = ApiVersionsResponse.apiVersionsResponse( - 10, - RecordBatch.MAGIC_VALUE_V1, - Features.emptySupportedFeatures()); - verifyApiKeysForMagic(response, RecordBatch.MAGIC_VALUE_V1); - assertEquals(10, response.throttleTimeMs()); - assertTrue(response.data().supportedFeatures().isEmpty()); - assertTrue(response.data().finalizedFeatures().isEmpty()); - assertEquals(ApiVersionsResponse.UNKNOWN_FINALIZED_FEATURES_EPOCH, response.data().finalizedFeaturesEpoch()); - } - @Test public void shouldCreateApiResponseThatHasAllApiKeysSupportedByBroker() { assertEquals(apiKeysInResponse(ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE), new HashSet<>(ApiKeys.enabledApis())); @@ -53,19 +40,6 @@ public void shouldCreateApiResponseThatHasAllApiKeysSupportedByBroker() { assertEquals(ApiVersionsResponse.UNKNOWN_FINALIZED_FEATURES_EPOCH, ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.data().finalizedFeaturesEpoch()); } - @Test - public void shouldReturnAllKeysWhenMagicIsCurrentValueAndThrottleMsIsDefaultThrottle() { - ApiVersionsResponse response = ApiVersionsResponse.apiVersionsResponse( - AbstractResponse.DEFAULT_THROTTLE_TIME, - RecordBatch.CURRENT_MAGIC_VALUE, - Features.emptySupportedFeatures()); - assertEquals(new HashSet<>(ApiKeys.enabledApis()), apiKeysInResponse(response)); - assertEquals(AbstractResponse.DEFAULT_THROTTLE_TIME, response.throttleTimeMs()); - assertTrue(response.data().supportedFeatures().isEmpty()); - assertTrue(response.data().finalizedFeatures().isEmpty()); - assertEquals(ApiVersionsResponse.UNKNOWN_FINALIZED_FEATURES_EPOCH, response.data().finalizedFeaturesEpoch()); - } - @Test public void shouldHaveCorrectDefaultApiVersionsResponse() { Collection apiVersions = ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.data().apiKeys(); @@ -95,41 +69,6 @@ public void shouldHaveCorrectDefaultApiVersionsResponse() { assertEquals(ApiVersionsResponse.UNKNOWN_FINALIZED_FEATURES_EPOCH, ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.data().finalizedFeaturesEpoch()); } -<<<<<<< HEAD -======= - @Test - public void shouldReturnFeatureKeysWhenMagicIsCurrentValueAndThrottleMsIsDefaultThrottle() { - ApiVersionsResponse response = ApiVersionsResponse.apiVersionsResponse( - 10, - RecordBatch.MAGIC_VALUE_V1, - Features.supportedFeatures(Utils.mkMap(Utils.mkEntry("feature", new SupportedVersionRange((short) 1, (short) 4)))), - Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry("feature", new FinalizedVersionRange((short) 2, (short) 3)))), - 10); - verifyApiKeysForMagic(response, RecordBatch.MAGIC_VALUE_V1); - assertEquals(10, response.throttleTimeMs()); - - assertEquals(1, response.data().supportedFeatures().size()); - SupportedFeatureKey sKey = response.data().supportedFeatures().find("feature"); - assertNotNull(sKey); - assertEquals(1, sKey.minVersion()); - assertEquals(4, sKey.maxVersion()); - - assertEquals(1, response.data().finalizedFeatures().size()); - FinalizedFeatureKey fKey = response.data().finalizedFeatures().find("feature"); - assertNotNull(fKey); - assertEquals(2, fKey.minVersionLevel()); - assertEquals(3, fKey.maxVersionLevel()); - - assertEquals(10, response.data().finalizedFeaturesEpoch()); - } - - private void verifyApiKeysForMagic(final ApiVersionsResponse response, final byte maxMagic) { - for (final ApiVersionsResponseKey version : response.data().apiKeys()) { - assertTrue(ApiKeys.forId(version.apiKey()).minRequiredInterBrokerMagic <= maxMagic); - } - } ->>>>>>> fd978a9d20... add data accessor and creat tests for manually created data - private Set apiKeysInResponse(final ApiVersionsResponse apiVersions) { final Set apiKeys = new HashSet<>(); for (final ApiVersionsResponseKey version : apiVersions.data().apiKeys()) { diff --git a/core/src/main/scala/kafka/network/RequestConvertToJson.scala b/core/src/main/scala/kafka/network/RequestConvertToJson.scala index 3ace86f4e4741..907fc484ede1f 100644 --- a/core/src/main/scala/kafka/network/RequestConvertToJson.scala +++ b/core/src/main/scala/kafka/network/RequestConvertToJson.scala @@ -17,19 +17,12 @@ package kafka.network -import java.util - import com.fasterxml.jackson.databind.JsonNode -import com.fasterxml.jackson.databind.node.{ArrayNode, BinaryNode, DoubleNode, IntNode, JsonNodeFactory, LongNode, NullNode, ObjectNode, ShortNode, TextNode} +import com.fasterxml.jackson.databind.node.{DoubleNode, JsonNodeFactory, LongNode, ObjectNode, TextNode} import kafka.network.RequestChannel.{Response, Session} import org.apache.kafka.common.message._ import org.apache.kafka.common.network.ClientInformation -import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.record.RecordBatch import org.apache.kafka.common.requests._ -import org.apache.kafka.common.utils.CollectionUtils - -import scala.jdk.CollectionConverters._ object RequestConvertToJson { def request(request: AbstractRequest): JsonNode = { @@ -64,6 +57,7 @@ object RequestConvertToJson { 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) @@ -80,8 +74,8 @@ object RequestConvertToJson { 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 => offsetsForLeaderEpochRequestNode(req, request.version) - case req: ProduceRequest => produceRequestNode(req, request.version, false) + case req: OffsetsForLeaderEpochRequest => OffsetForLeaderEpochRequestDataJsonConverter.write(req.data, request.version) + case req: ProduceRequest => if (req.data == null) null else ProduceRequestDataJsonConverter.write(req.data, request.version) 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) @@ -129,6 +123,7 @@ object RequestConvertToJson { 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) @@ -145,8 +140,8 @@ object RequestConvertToJson { 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 => offsetsForLeaderEpochResponseNode(res, version) - case res: ProduceResponse => produceResponseNode(res, 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) @@ -208,155 +203,4 @@ object RequestConvertToJson { node.set("messageConversionsTime", new DoubleNode(messageConversionsTimeMs)) node } - - /** - * Temporary until switch to use the generated schemas. - */ - def offsetsForLeaderEpochRequestNode(request: OffsetsForLeaderEpochRequest, version: Short): JsonNode = { - val node = new ObjectNode(JsonNodeFactory.instance) - if (version >= 3) { - node.set("replicaId", new IntNode(request.replicaId)) - } - val topics = CollectionUtils.groupPartitionDataByTopic(request.epochsByTopicPartition) - val topicsArray = new ArrayNode(JsonNodeFactory.instance) - topics.forEach { (topicName, partitions) => - val topicNode = new ObjectNode(JsonNodeFactory.instance) - topicNode.set("name", new TextNode(topicName)) - val partitionsArray = new ArrayNode(JsonNodeFactory.instance) - partitions.forEach { (partitionIndex, partitionData) => - val partitionNode = new ObjectNode(JsonNodeFactory.instance) - partitionNode.set("partitionIndex", new IntNode(partitionIndex)) - if (version >= 2) { - val leaderEpoch = partitionData.currentLeaderEpoch - partitionNode.set("currentLeaderEpoch", new IntNode(leaderEpoch.orElse(RecordBatch.NO_PARTITION_LEADER_EPOCH))) - } - partitionNode.set("leaderEpoch", new IntNode(partitionData.leaderEpoch)) - partitionsArray.add(partitionNode) - } - topicNode.set("partitions", partitionsArray) - topicsArray.add(topicNode) - } - node.set("topics", topicsArray) - node - } - - /** - * Temporary until switch to use the generated schemas. - */ - def produceRequestNode(request: ProduceRequest, version: Short, serializeRecords: Boolean): JsonNode = { - val node = new ObjectNode(JsonNodeFactory.instance) - if (version >= 3) { - if (request.transactionalId == null) { - node.set("transactionalId", NullNode.instance) - } else { - node.set("transactionalId", new TextNode(request.transactionalId)) - } - } - node.set("acks", new ShortNode(request.acks)) - node.set("timeoutMs", new IntNode(request.timeout)) - val topics = CollectionUtils.groupPartitionDataByTopic(request.partitionRecordsOrFail()) - val topicsArray = new ArrayNode(JsonNodeFactory.instance) - topics.forEach { (topicName, partitions) => - val topicNode = new ObjectNode(JsonNodeFactory.instance) - topicNode.set("name", new TextNode(topicName)) - val partitionsArray = new ArrayNode(JsonNodeFactory.instance) - partitions.forEach { (partitionIndex, partitionData) => - val partitionNode = new ObjectNode(JsonNodeFactory.instance) - partitionNode.set("partitionIndex", new IntNode(partitionIndex)) - if (partitionData == null) { - partitionNode.set("records", NullNode.instance) - } else { - if (serializeRecords) - partitionNode.set("records", new BinaryNode(util.Arrays.copyOf(partitionData.buffer().array(), partitionData.validBytes()))) - else - partitionNode.set("records", new IntNode(partitionData.validBytes())) - } - partitionsArray.add(partitionNode) - } - topicNode.set("partitions", partitionsArray) - topicsArray.add(topicNode) - } - node.set("topics", topicsArray) - node - } - - /** - * Temporary until switch to use the generated schemas. - */ - def offsetsForLeaderEpochResponseNode(response: OffsetsForLeaderEpochResponse, version: Short): JsonNode = { - val node = new ObjectNode(JsonNodeFactory.instance) - if (version >= 2) { - node.set("throttleTimeMs", new IntNode(response.throttleTimeMs)) - } - val topics = CollectionUtils.groupPartitionDataByTopic(response.responses) - val topicsArray = new ArrayNode(JsonNodeFactory.instance) - topics.forEach { (topicName, partitions) => - val topicNode = new ObjectNode(JsonNodeFactory.instance) - topicNode.set("name", new TextNode(topicName)) - val partitionsArray = new ArrayNode(JsonNodeFactory.instance) - partitions.forEach { (partitionIndex, partitionData) => - val partitionNode = new ObjectNode(JsonNodeFactory.instance) - partitionNode.set("errorCode", new ShortNode(partitionData.error.code)) - partitionNode.set("partitionIndex", new IntNode(partitionIndex)) - if (version >= 1) partitionNode.set("leaderEpoch", new IntNode(partitionData.leaderEpoch)) - partitionNode.set("endOffset", new LongNode(partitionData.endOffset)) - partitionsArray.add(partitionNode) - } - topicNode.set("partitions", partitionsArray) - topicsArray.add(topicNode) - } - node.set("topics", topicsArray) - node - } - - /** - * Temporary until switch to use the generated schemas. - */ - def produceResponseNode(response: ProduceResponse, version: Short): JsonNode = { - val node = new ObjectNode(JsonNodeFactory.instance) - val topics = CollectionUtils.groupPartitionDataByTopic(response.responses) - val responsesArray = new ArrayNode(JsonNodeFactory.instance) - topics.forEach { (topicName, partitions) => - val topicNode = new ObjectNode(JsonNodeFactory.instance) - topicNode.set("name", new TextNode(topicName)) - val partitionsArray = new ArrayNode(JsonNodeFactory.instance) - partitions.forEach { (partitionIndex, partitionData) => - val partitionNode = new ObjectNode(JsonNodeFactory.instance) - partitionNode.set("partitionIndex", new IntNode(partitionIndex)) - var errorCode = partitionData.error.code - if (errorCode == Errors.KAFKA_STORAGE_ERROR.code && version <= 3) { - errorCode = Errors.NOT_LEADER_OR_FOLLOWER.code - } - partitionNode.set("errorCode", new ShortNode(errorCode)) - partitionNode.set("baseOffset", new LongNode(partitionData.baseOffset)) - if (version >= 2) { - partitionNode.set("logAppendTimeMs", new LongNode(partitionData.logAppendTime)) - } - if (version >= 5) { - partitionNode.set("logStartOffset", new LongNode(partitionData.logStartOffset)) - } - if (version >= 8) { - val recordErrorsArray = new ArrayNode(JsonNodeFactory.instance) - for (indexAndMessage <- partitionData.recordErrors.asScala) { - val indexAndMessageData = new ObjectNode(JsonNodeFactory.instance) - indexAndMessageData.set("batchIndex", new IntNode(indexAndMessage.batchIndex)) - if (indexAndMessage.message == null) indexAndMessageData.set("batchIndexErrorMessage", NullNode.instance) - else indexAndMessageData.set("batchIndexErrorMessage", new TextNode(indexAndMessage.message)) - recordErrorsArray.add(indexAndMessageData) - } - partitionNode.set("recordErrors", recordErrorsArray) - if (partitionData.errorMessage == null) partitionNode.set("errorMessage", NullNode.instance) - else partitionNode.set("errorMessage", new TextNode(partitionData.errorMessage)) - } - partitionsArray.add(partitionNode) - } - topicNode.set("partitions", partitionsArray) - responsesArray.add(topicNode) - } - node.set("responses", responsesArray) - if (version >= 1) { - node.set("throttleTimeMs", new IntNode(response.throttleTimeMs)) - } - node - } } diff --git a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala index 92cd0d4da6f62..b975316800d98 100644 --- a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala +++ b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala @@ -19,19 +19,13 @@ package kafka.network import java.net.InetAddress import java.nio.ByteBuffer -import java.util -import java.util.{Collections, Optional} - import com.fasterxml.jackson.databind.node.{DoubleNode, LongNode, ObjectNode, TextNode} import kafka.network -import org.apache.kafka.common.TopicPartition 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, Errors} -import org.apache.kafka.common.record.{MemoryRecords, RecordBatch} -import org.apache.kafka.common.requests.OffsetsForLeaderEpochRequest.PartitionData +import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.requests._ import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.easymock.EasyMock.createNiceMock @@ -46,16 +40,8 @@ class RequestConvertToJsonTest { val unhandledKeys = ArrayBuffer[String]() ApiKeys.values().foreach { key => { val version: Short = key.latestVersion() - val req = if (key == ApiKeys.PRODUCE) { - // There's inconsistency with the toStruct schema in ProduceRequest - // and ProduceRequestDataJsonConverters where the field names don't - // match so the struct does not have the correct field names. This is - // a temporary workaround until ProduceRequest starts using ProduceRequestData - ProduceRequest.Builder.forCurrentMagic(0.toShort, 10000, new util.HashMap[TopicPartition, MemoryRecords]).build() - } else { - val struct = ApiMessageType.fromApiKey(key.id).newRequest().toStruct(version) - AbstractRequest.parseRequest(key, version, struct) - } + val struct = ApiMessageType.fromApiKey(key.id).newRequest().toStruct(version) + val req = AbstractRequest.parseRequest(key, version, struct) try { RequestConvertToJson.request(req) } catch { @@ -81,66 +67,6 @@ class RequestConvertToJsonTest { assertEquals("Unhandled response keys", ArrayBuffer.empty, unhandledKeys) } - @Test - def testFormatOfOffsetsForLeaderEpochRequestNode(): Unit = { - val partitionDataMap = new util.HashMap[TopicPartition, PartitionData] - partitionDataMap.put(new TopicPartition("topic1", 0), new PartitionData(Optional.of(0), 1)) - - val version: Short = ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion - val request = OffsetsForLeaderEpochRequest.Builder.forConsumer(partitionDataMap).build(version) - val actualNode = RequestConvertToJson.request(request) - - val requestData = OffsetForLeaderEpochRequestDataJsonConverter.read(actualNode, version) - val expectedNode = OffsetForLeaderEpochRequestDataJsonConverter.write(requestData, version, true) - - assertEquals(expectedNode, actualNode) - } - - @Test - def testFormatOfProduceRequestNode(): Unit = { - val produceDataMap = new util.HashMap[TopicPartition, MemoryRecords] - - val version: Short = ApiKeys.PRODUCE.latestVersion - val request = ProduceRequest.Builder.forMagic(2, 0.toShort, 0, produceDataMap, "").build() - val actualNode = RequestConvertToJson.request(request) - - val requestData = new ProduceRequestData() - val expectedNode = ProduceRequestDataJsonConverter.write(requestData, version) - - assertEquals(expectedNode, actualNode) - } - - @Test - def testFormatOfOffsetsForLeaderEpochResponseNode(): Unit = { - val endOffsetMap = new util.HashMap[TopicPartition, EpochEndOffset] - endOffsetMap.put(new TopicPartition("topic1", 0), new EpochEndOffset(1, 10L)) - - val version: Short = ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion - val response = new OffsetsForLeaderEpochResponse(endOffsetMap) - val actualNode = RequestConvertToJson.response(response, version) - - val requestData = OffsetForLeaderEpochResponseDataJsonConverter.read(actualNode, version) - val expectedNode = OffsetForLeaderEpochResponseDataJsonConverter.write(requestData, version, true) - - assertEquals(expectedNode, actualNode) - } - - @Test - def testFormatOfProduceResponseNode(): Unit = { - val responseData = new util.HashMap[TopicPartition, ProduceResponse.PartitionResponse] - val partResponse = new ProduceResponse.PartitionResponse(Errors.NONE, 10000, RecordBatch.NO_TIMESTAMP, 100, Collections.singletonList(new ProduceResponse.RecordError(3, "Record error")), "Produce failed") - responseData.put(new TopicPartition("topic1", 0), partResponse) - - val version: Short = ApiKeys.PRODUCE.latestVersion - val response = new ProduceResponse(responseData) - val actualNode = RequestConvertToJson.response(response, version) - - val requestData = ProduceResponseDataJsonConverter.read(actualNode, version) - val expectedNode = ProduceResponseDataJsonConverter.write(requestData, version, true) - - assertEquals(expectedNode, actualNode) - } - @Test def testRequestDescMetrics(): Unit = { val req = request(new AlterIsrRequest(new AlterIsrRequestData(), 0)) @@ -203,6 +129,7 @@ class RequestConvertToJsonTest { new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "user"), ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT), SecurityProtocol.PLAINTEXT, - new ClientInformation("name", "version")) + new ClientInformation("name", "version"), + false) } } 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 index ef07ca4ca6809..405458fef0274 100644 --- 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 @@ -18,13 +18,8 @@ package org.apache.kafka.jmh.common; import kafka.network.RequestConvertToJson; -import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.message.ProduceRequestData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.record.CompressionType; -import org.apache.kafka.common.record.MemoryRecords; -import org.apache.kafka.common.record.MemoryRecordsBuilder; -import org.apache.kafka.common.record.RecordBatch; -import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.requests.ProduceRequest; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; @@ -39,11 +34,6 @@ import org.openjdk.jmh.annotations.State; import org.openjdk.jmh.annotations.Warmup; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.util.HashMap; -import java.util.Map; -import java.util.UUID; import java.util.concurrent.TimeUnit; @State(Scope.Benchmark) @@ -59,29 +49,11 @@ public class ProduceRequestBenchmark { @Param({"3", "10", "20"}) private int partitionCount; - Map produceData; - ProduceRequest produceRequest; @Setup(Level.Trial) public void setup() { - this.produceData = new HashMap<>(); - for (int topicIdx = 0; topicIdx < topicCount; topicIdx++) { - String topic = UUID.randomUUID().toString(); - for (int partitionId = 0; partitionId < partitionCount; partitionId++) { - ByteBuffer buffer = ByteBuffer.allocate(256); - MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, - CompressionType.NONE, TimestampType.CREATE_TIME, 0L); - try { - builder.append(10L, null, "a".getBytes(StandardCharsets.US_ASCII)); - } catch (Exception e) { - continue; - } - produceData.put(new TopicPartition(topic, partitionId), builder.build()); - } - } - - this.produceRequest = ProduceRequest.Builder.forCurrentMagic((short) -1, 3000, produceData) + this.produceRequest = ProduceRequest.forCurrentMagic(new ProduceRequestData()) .build(ApiKeys.PRODUCE.latestVersion()); } From 3e439d5d81c9cbede152775385c046b6208a1c04 Mon Sep 17 00:00:00 2001 From: anatasiavela Date: Thu, 19 Nov 2020 22:02:23 -0800 Subject: [PATCH 17/22] handle null ProduceRequest --- .../main/scala/kafka/network/RequestConvertToJson.scala | 4 ++-- core/src/main/scala/kafka/server/KafkaApis.scala | 2 +- .../org/apache/kafka/message/JsonConverterGenerator.java | 8 ++++---- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/kafka/network/RequestConvertToJson.scala b/core/src/main/scala/kafka/network/RequestConvertToJson.scala index 907fc484ede1f..19fb830ea1764 100644 --- a/core/src/main/scala/kafka/network/RequestConvertToJson.scala +++ b/core/src/main/scala/kafka/network/RequestConvertToJson.scala @@ -18,7 +18,7 @@ package kafka.network import com.fasterxml.jackson.databind.JsonNode -import com.fasterxml.jackson.databind.node.{DoubleNode, JsonNodeFactory, LongNode, ObjectNode, TextNode} +import com.fasterxml.jackson.databind.node.{DoubleNode, JsonNodeFactory, LongNode, NullNode, ObjectNode, TextNode} import kafka.network.RequestChannel.{Response, Session} import org.apache.kafka.common.message._ import org.apache.kafka.common.network.ClientInformation @@ -75,7 +75,7 @@ object RequestConvertToJson { 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 => if (req.data == null) null else ProduceRequestDataJsonConverter.write(req.data, request.version) + case req: ProduceRequest => if (req.data == null) NullNode.instance else ProduceRequestDataJsonConverter.write(req.data, request.version) 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) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 77c52b6104e27..cdac2309ff362 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -183,7 +183,7 @@ class KafkaApis(val requestChannel: RequestChannel, */ override def handle(request: RequestChannel.Request): Unit = { try { - trace(s"Handling request:${RequestConvertToJson.requestDesc(request.header, request.loggableRequest).toString} from connection ${request.context.connectionId};" + + trace(s"Handling request:${RequestConvertToJson.requestDesc(request.header, request.loggableRequest)} from connection ${request.context.connectionId};" + s"securityProtocol:${request.context.securityProtocol},principal:${request.context.principal}") request.envelope.foreach { envelope => 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 9670874a8c9e3..2d6c77879248e 100644 --- a/generator/src/main/java/org/apache/kafka/message/JsonConverterGenerator.java +++ b/generator/src/main/java/org/apache/kafka/message/JsonConverterGenerator.java @@ -386,12 +386,12 @@ private void generateVariableLengthTargetToJson(Target target, Versions versions } else { headerGenerator.addImport(MessageGenerator.ARRAYS_CLASS); buffer.printf("%s;%n", target.assignmentStatement( - String.format("new BinaryNode(Arrays.copyOf(%s, %s.length))", - target.sourceVariable(), target.sourceVariable()))); + String.format("new BinaryNode(Arrays.copyOf(%s, %s.length))", + target.sourceVariable(), target.sourceVariable()))); } } else if (target.field().type().isRecords()) { - headerGenerator.addImport(MessageGenerator.INT_NODE_CLASS); 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. @@ -402,7 +402,7 @@ private void generateVariableLengthTargetToJson(Target target, Versions versions buffer.printf("} else {%n"); buffer.incrementIndent(); buffer.printf("%s;%n", target.assignmentStatement( - String.format("new IntNode(%s.sizeInBytes())", target.sourceVariable()))); + String.format("new IntNode(%s.sizeInBytes())", target.sourceVariable()))); buffer.decrementIndent(); buffer.printf("}%n"); } else if (target.field().type().isArray()) { From 4bca18e9b36a8597078f45b4b8eda15e645fecfd Mon Sep 17 00:00:00 2001 From: anatasiavela Date: Wed, 2 Dec 2020 19:39:26 -0800 Subject: [PATCH 18/22] add specific case for ProduceRequest --- .../kafka/common/requests/ProduceRequest.java | 16 +++--- .../common/requests/RequestResponseTest.java | 8 +-- .../kafka/network/RequestConvertToJson.scala | 50 +++++++++++++++++-- 3 files changed, 59 insertions(+), 15 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java index 9af463b9aa083..44e2868602529 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -128,7 +128,7 @@ public ProduceRequest(ProduceRequestData produceRequestData, short version) { } // visible for testing - Map partitionSizes() { + Map createOrGetPartitionSizes() { if (partitionSizes == null) { // this method may be called by different thread (see the comment on data) synchronized (this) { @@ -167,9 +167,9 @@ public String toString(boolean verbose) { .append(",timeout=").append(timeout); if (verbose) - bld.append(",partitionSizes=").append(Utils.mkString(partitionSizes(), "[", "]", "=", ",")); + bld.append(",partitionSizes=").append(Utils.mkString(createOrGetPartitionSizes(), "[", "]", "=", ",")); else - bld.append(",numPartitions=").append(partitionSizes().size()); + bld.append(",numPartitions=").append(createOrGetPartitionSizes().size()); bld.append("}"); return bld.toString(); @@ -181,7 +181,7 @@ public ProduceResponse getErrorResponse(int throttleTimeMs, Throwable e) { if (acks == 0) return null; ApiError apiError = ApiError.fromThrowable(e); ProduceResponseData data = new ProduceResponseData().setThrottleTimeMs(throttleTimeMs); - partitionSizes().forEach((tp, ignored) -> { + createOrGetPartitionSizes().forEach((tp, ignored) -> { ProduceResponseData.TopicProduceResponse tpr = data.responses().find(tp.topic()); if (tpr == null) { tpr = new ProduceResponseData.TopicProduceResponse().setName(tp.topic()); @@ -202,7 +202,7 @@ public ProduceResponse getErrorResponse(int throttleTimeMs, Throwable e) { @Override public Map errorCounts(Throwable e) { Errors error = Errors.forException(e); - return Collections.singletonMap(error, partitionSizes().size()); + return Collections.singletonMap(error, createOrGetPartitionSizes().size()); } public short acks() { @@ -221,9 +221,13 @@ public ProduceRequestData data() { return data; } + public Map partitionSizes() { + return partitionSizes; + } + public void clearPartitionRecords() { // lazily initialize partitionSizes. - partitionSizes(); + createOrGetPartitionSizes(); data = null; } 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 2ebc3c4b9cbe3..b4b66d6275b37 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -646,9 +646,9 @@ public void testPartitionSize() { .setTimeoutMs(5000) .setTransactionalId("transactionalId")) .build((short) 3); - assertEquals(2, request.partitionSizes().size()); - assertEquals(records0.sizeInBytes(), (int) request.partitionSizes().get(tp0)); - assertEquals(records1.sizeInBytes(), (int) request.partitionSizes().get(tp1)); + assertEquals(2, request.createOrGetPartitionSizes().size()); + assertEquals(records0.sizeInBytes(), (int) request.createOrGetPartitionSizes().get(tp0)); + assertEquals(records1.sizeInBytes(), (int) request.createOrGetPartitionSizes().get(tp1)); } @Test @@ -679,7 +679,7 @@ public void produceRequestToStringTest() { @Test public void produceRequestGetErrorResponseTest() { ProduceRequest request = createProduceRequest(ApiKeys.PRODUCE.latestVersion()); - Set partitions = new HashSet<>(request.partitionSizes().keySet()); + Set partitions = new HashSet<>(request.createOrGetPartitionSizes().keySet()); ProduceResponse errorResponse = (ProduceResponse) request.getErrorResponse(new NotEnoughReplicasException()); assertEquals(partitions, errorResponse.responses().keySet()); diff --git a/core/src/main/scala/kafka/network/RequestConvertToJson.scala b/core/src/main/scala/kafka/network/RequestConvertToJson.scala index 19fb830ea1764..056b5b4365713 100644 --- a/core/src/main/scala/kafka/network/RequestConvertToJson.scala +++ b/core/src/main/scala/kafka/network/RequestConvertToJson.scala @@ -18,8 +18,9 @@ package kafka.network import com.fasterxml.jackson.databind.JsonNode -import com.fasterxml.jackson.databind.node.{DoubleNode, JsonNodeFactory, LongNode, NullNode, ObjectNode, TextNode} +import com.fasterxml.jackson.databind.node.{ArrayNode, DoubleNode, IntNode, JsonNodeFactory, LongNode, NullNode, ObjectNode, ShortNode, TextNode} import kafka.network.RequestChannel.{Response, Session} +import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.message._ import org.apache.kafka.common.network.ClientInformation import org.apache.kafka.common.requests._ @@ -75,7 +76,7 @@ object RequestConvertToJson { 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 => if (req.data == null) NullNode.instance else ProduceRequestDataJsonConverter.write(req.data, request.version) + case req: ProduceRequest => if (req.data == null) produceRequestJson(req, request.version, false) else ProduceRequestDataJsonConverter.write(req.data, request.version) 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) @@ -158,7 +159,7 @@ object RequestConvertToJson { } def requestHeaderNode(header: RequestHeader): JsonNode = { - val node = RequestHeaderDataJsonConverter.write(header.data(), header.headerVersion(), false).asInstanceOf[ObjectNode] + val node = RequestHeaderDataJsonConverter.write(header.data, header.headerVersion, false).asInstanceOf[ObjectNode] node.set("requestApiKeyName", new TextNode(header.apiKey.toString)) node } @@ -172,8 +173,15 @@ object RequestConvertToJson { 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.set("softwareName", new TextNode(clientInfo.softwareName)) + node.set("softwareVersion", new TextNode(clientInfo.softwareVersion)) + node + } + + def topicPartitionNode(topicPartition: TopicPartition): JsonNode = { + val node = new ObjectNode(JsonNodeFactory.instance) + node.set("topic", new TextNode(topicPartition.topic)) + node.set("partition", new IntNode(topicPartition.partition)) node } @@ -203,4 +211,36 @@ object RequestConvertToJson { node.set("messageConversionsTime", new DoubleNode(messageConversionsTimeMs)) node } + + /** + * This specific handling of ProduceRequest is for when it goes into purgatory and its data becomes null. + */ + def produceRequestJson(req: ProduceRequest, version: Short, serializeRecords: Boolean): JsonNode = { + val node = new ObjectNode(JsonNodeFactory.instance) + if (version >= 3) { + if (req.transactionalId == null) + node.set("transactionalId", NullNode.instance) + else + node.set("transactionalId", new TextNode(req.transactionalId)) + } + node.set("acks", new ShortNode(req.acks)) + node.set("timeoutMs", new IntNode(req.timeout)) + if (serializeRecords) { + if (req.partitionSizes() != null) { + val topicDataNode = new ArrayNode(JsonNodeFactory.instance) + req.partitionSizes().forEach { (topicPartition, size) => { + val partitionNode = new ObjectNode(JsonNodeFactory.instance) + partitionNode.set("topicPartition", topicPartitionNode(topicPartition)) + partitionNode.set("size", new IntNode(size)) + topicDataNode.add(partitionNode) + }} + node.set("partitionSizes", topicDataNode) + } else { + node.set("partitionSizes", NullNode.instance) + } + } else { + node.set("numPartitions", new IntNode(req.partitionSizes().size())) + } + node + } } From 9cc67279ff1b8ea4d0fe014dbaf4787a662b5155 Mon Sep 17 00:00:00 2001 From: anatasiavela Date: Fri, 4 Dec 2020 09:20:56 -0800 Subject: [PATCH 19/22] make produce request consistent --- core/src/main/scala/kafka/network/RequestConvertToJson.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/network/RequestConvertToJson.scala b/core/src/main/scala/kafka/network/RequestConvertToJson.scala index 056b5b4365713..c1d63bda8e620 100644 --- a/core/src/main/scala/kafka/network/RequestConvertToJson.scala +++ b/core/src/main/scala/kafka/network/RequestConvertToJson.scala @@ -76,7 +76,7 @@ object RequestConvertToJson { 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 => if (req.data == null) produceRequestJson(req, request.version, false) else ProduceRequestDataJsonConverter.write(req.data, request.version) + case req: ProduceRequest => produceRequestJson(req, 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) @@ -213,7 +213,7 @@ object RequestConvertToJson { } /** - * This specific handling of ProduceRequest is for when it goes into purgatory and its data becomes null. + * ProduceRequest has a specific handling because it can go into purgatory where its data becomes null. */ def produceRequestJson(req: ProduceRequest, version: Short, serializeRecords: Boolean): JsonNode = { val node = new ObjectNode(JsonNodeFactory.instance) From 5f91474f006841d679bbe63d64fcdf3cc93282bc Mon Sep 17 00:00:00 2001 From: anatasiavela Date: Thu, 10 Dec 2020 18:28:21 -0800 Subject: [PATCH 20/22] remove special handling for ProduceRequest --- .../kafka/common/requests/ProduceRequest.java | 16 ++--- .../common/requests/RequestResponseTest.java | 8 +-- .../scala/kafka/network/RequestChannel.scala | 18 +++--- .../kafka/network/RequestConvertToJson.scala | 59 ++++++------------- .../main/scala/kafka/server/KafkaApis.scala | 4 +- .../kafka/tools/TestRaftRequestHandler.scala | 2 +- .../kafka/network/RequestChannelTest.scala | 4 +- .../network/RequestConvertToJsonTest.scala | 4 +- .../metadata/MetadataRequestBenchmark.java | 3 +- 9 files changed, 48 insertions(+), 70 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java index 44e2868602529..9af463b9aa083 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -128,7 +128,7 @@ public ProduceRequest(ProduceRequestData produceRequestData, short version) { } // visible for testing - Map createOrGetPartitionSizes() { + Map partitionSizes() { if (partitionSizes == null) { // this method may be called by different thread (see the comment on data) synchronized (this) { @@ -167,9 +167,9 @@ public String toString(boolean verbose) { .append(",timeout=").append(timeout); if (verbose) - bld.append(",partitionSizes=").append(Utils.mkString(createOrGetPartitionSizes(), "[", "]", "=", ",")); + bld.append(",partitionSizes=").append(Utils.mkString(partitionSizes(), "[", "]", "=", ",")); else - bld.append(",numPartitions=").append(createOrGetPartitionSizes().size()); + bld.append(",numPartitions=").append(partitionSizes().size()); bld.append("}"); return bld.toString(); @@ -181,7 +181,7 @@ public ProduceResponse getErrorResponse(int throttleTimeMs, Throwable e) { if (acks == 0) return null; ApiError apiError = ApiError.fromThrowable(e); ProduceResponseData data = new ProduceResponseData().setThrottleTimeMs(throttleTimeMs); - createOrGetPartitionSizes().forEach((tp, ignored) -> { + partitionSizes().forEach((tp, ignored) -> { ProduceResponseData.TopicProduceResponse tpr = data.responses().find(tp.topic()); if (tpr == null) { tpr = new ProduceResponseData.TopicProduceResponse().setName(tp.topic()); @@ -202,7 +202,7 @@ public ProduceResponse getErrorResponse(int throttleTimeMs, Throwable e) { @Override public Map errorCounts(Throwable e) { Errors error = Errors.forException(e); - return Collections.singletonMap(error, createOrGetPartitionSizes().size()); + return Collections.singletonMap(error, partitionSizes().size()); } public short acks() { @@ -221,13 +221,9 @@ public ProduceRequestData data() { return data; } - public Map partitionSizes() { - return partitionSizes; - } - public void clearPartitionRecords() { // lazily initialize partitionSizes. - createOrGetPartitionSizes(); + partitionSizes(); data = null; } 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 b4b66d6275b37..2ebc3c4b9cbe3 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -646,9 +646,9 @@ public void testPartitionSize() { .setTimeoutMs(5000) .setTransactionalId("transactionalId")) .build((short) 3); - assertEquals(2, request.createOrGetPartitionSizes().size()); - assertEquals(records0.sizeInBytes(), (int) request.createOrGetPartitionSizes().get(tp0)); - assertEquals(records1.sizeInBytes(), (int) request.createOrGetPartitionSizes().get(tp1)); + assertEquals(2, request.partitionSizes().size()); + assertEquals(records0.sizeInBytes(), (int) request.partitionSizes().get(tp0)); + assertEquals(records1.sizeInBytes(), (int) request.partitionSizes().get(tp1)); } @Test @@ -679,7 +679,7 @@ public void produceRequestToStringTest() { @Test public void produceRequestGetErrorResponseTest() { ProduceRequest request = createProduceRequest(ApiKeys.PRODUCE.latestVersion()); - Set partitions = new HashSet<>(request.createOrGetPartitionSizes().keySet()); + Set partitions = new HashSet<>(request.partitionSizes().keySet()); ProduceResponse errorResponse = (ProduceResponse) request.getErrorResponse(new NotEnoughReplicasException()); assertEquals(partitions, errorResponse.responses().keySet()); diff --git a/core/src/main/scala/kafka/network/RequestChannel.scala b/core/src/main/scala/kafka/network/RequestChannel.scala index b984e8ce25213..e7dea665a2846 100644 --- a/core/src/main/scala/kafka/network/RequestChannel.scala +++ b/core/src/main/scala/kafka/network/RequestChannel.scala @@ -125,6 +125,13 @@ object RequestChannel extends Logging { } } + def requestLog: Option[JsonNode] = { + if (RequestChannel.isRequestLoggingEnabled) + Some(RequestConvertToJson.request(loggableRequest)) + else + None + } + def responseNode(response: AbstractResponse): Option[JsonNode] = { if (RequestChannel.isRequestLoggingEnabled) Some(RequestConvertToJson.response(response, context.apiVersion())) @@ -141,11 +148,8 @@ object RequestChannel extends Logging { } } - def requestDesc(details: Boolean): String = { - val forwardDescription = envelope.map { request => - s"Forwarded request: ${request.context} " - }.getOrElse("") - s"$forwardDescription$header -- ${loggableRequest.toString(details)}" + def requestDesc: JsonNode = { + RequestConvertToJson.requestDesc(header, requestLog, envelope) } def body[T <: AbstractRequest](implicit classTag: ClassTag[T], @nowarn("cat=unused") nn: NotNothing[T]): T = { @@ -202,7 +206,7 @@ object RequestChannel extends Logging { } } - trace(s"Processor $processor received request: ${RequestConvertToJson.requestDesc(header, loggableRequest)}") + trace(s"Processor $processor received request: $requestDesc") def requestThreadTimeNanos: Long = { if (apiLocalCompleteTimeNanos == -1L) apiLocalCompleteTimeNanos = Time.SYSTEM.nanoseconds @@ -263,7 +267,7 @@ object RequestChannel extends Logging { recordNetworkThreadTimeCallback.foreach(record => record(networkThreadTimeNanos)) if (isRequestLoggingEnabled) { - val desc = RequestConvertToJson.requestDescMetrics(header, response, loggableRequest, + val desc = RequestConvertToJson.requestDescMetrics(header, requestLog, response.responseLog, context, session, totalTimeMs, requestQueueTimeMs, apiLocalTimeMs, apiRemoteTimeMs, apiThrottleTimeMs, responseQueueTimeMs, diff --git a/core/src/main/scala/kafka/network/RequestConvertToJson.scala b/core/src/main/scala/kafka/network/RequestConvertToJson.scala index c1d63bda8e620..71c3683fae1bb 100644 --- a/core/src/main/scala/kafka/network/RequestConvertToJson.scala +++ b/core/src/main/scala/kafka/network/RequestConvertToJson.scala @@ -18,8 +18,8 @@ package kafka.network import com.fasterxml.jackson.databind.JsonNode -import com.fasterxml.jackson.databind.node.{ArrayNode, DoubleNode, IntNode, JsonNodeFactory, LongNode, NullNode, ObjectNode, ShortNode, TextNode} -import kafka.network.RequestChannel.{Response, Session} +import com.fasterxml.jackson.databind.node.{BooleanNode, DoubleNode, IntNode, JsonNodeFactory, LongNode, ObjectNode, TextNode} +import kafka.network.RequestChannel.Session import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.message._ import org.apache.kafka.common.network.ClientInformation @@ -76,7 +76,7 @@ object RequestConvertToJson { 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 => produceRequestJson(req, request.version, false) + case req: ProduceRequest => ProduceRequestDataJsonConverter.write(req.data, request.version) 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) @@ -164,10 +164,21 @@ object RequestConvertToJson { node } - def requestDesc(header: RequestHeader, req: AbstractRequest): JsonNode = { + def requestContextNode(context: RequestContext): JsonNode = { val node = new ObjectNode(JsonNodeFactory.instance) + node.set("connection", new TextNode(context.connectionId)) + node.set("securityProtocol", new TextNode(context.securityProtocol.toString)) + node.set("listener", new TextNode(context.listenerName.value)) + node.set("clientInformation", clientInfoNode(context.clientInformation)) + node + } + + def requestDesc(header: RequestHeader, requestNode: Option[JsonNode], envelope: Option[RequestChannel.Request] = None): JsonNode = { + val node = new ObjectNode(JsonNodeFactory.instance) + node.set("isForwarded", if (envelope.isDefined) BooleanNode.TRUE else BooleanNode.FALSE) + node.set("forwardedRequest", envelope.map(request => requestContextNode(request.context)).getOrElse(new TextNode(""))) node.set("requestHeader", requestHeaderNode(header)) - node.set("request", request(req)) + node.set("request", requestNode.getOrElse(new TextNode(""))) node } @@ -185,14 +196,14 @@ object RequestConvertToJson { node } - def requestDescMetrics(header: RequestHeader, res: Response, req: AbstractRequest, + def requestDescMetrics(header: RequestHeader, requestNode: Option[JsonNode], responseNode: Option[JsonNode], context: RequestContext, session: Session, totalTimeMs: Double, requestQueueTimeMs: Double, apiLocalTimeMs: Double, apiRemoteTimeMs: Double, apiThrottleTimeMs: Long, responseQueueTimeMs: Double, responseSendTimeMs: Double, temporaryMemoryBytes: Long, messageConversionsTimeMs: Double): JsonNode = { - val node = requestDesc(header, req).asInstanceOf[ObjectNode] - node.set("response", res.responseLog.getOrElse(new TextNode(""))) + val node = requestDesc(header, requestNode).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)) @@ -211,36 +222,4 @@ object RequestConvertToJson { node.set("messageConversionsTime", new DoubleNode(messageConversionsTimeMs)) node } - - /** - * ProduceRequest has a specific handling because it can go into purgatory where its data becomes null. - */ - def produceRequestJson(req: ProduceRequest, version: Short, serializeRecords: Boolean): JsonNode = { - val node = new ObjectNode(JsonNodeFactory.instance) - if (version >= 3) { - if (req.transactionalId == null) - node.set("transactionalId", NullNode.instance) - else - node.set("transactionalId", new TextNode(req.transactionalId)) - } - node.set("acks", new ShortNode(req.acks)) - node.set("timeoutMs", new IntNode(req.timeout)) - if (serializeRecords) { - if (req.partitionSizes() != null) { - val topicDataNode = new ArrayNode(JsonNodeFactory.instance) - req.partitionSizes().forEach { (topicPartition, size) => { - val partitionNode = new ObjectNode(JsonNodeFactory.instance) - partitionNode.set("topicPartition", topicPartitionNode(topicPartition)) - partitionNode.set("size", new IntNode(size)) - topicDataNode.add(partitionNode) - }} - node.set("partitionSizes", topicDataNode) - } else { - node.set("partitionSizes", NullNode.instance) - } - } else { - node.set("numPartitions", new IntNode(req.partitionSizes().size())) - } - node - } } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index cdac2309ff362..f8f35586e91a0 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -34,7 +34,7 @@ import kafka.coordinator.group.{GroupCoordinator, JoinGroupResult, LeaveGroupRes import kafka.coordinator.transaction.{InitProducerIdResult, TransactionCoordinator} import kafka.log.AppendOrigin import kafka.message.ZStdCompressionCodec -import kafka.network.{RequestChannel, RequestConvertToJson} +import kafka.network.RequestChannel import kafka.security.authorizer.{AclEntry, AuthorizerUtils} import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota} import kafka.utils.{CoreUtils, Logging} @@ -183,7 +183,7 @@ class KafkaApis(val requestChannel: RequestChannel, */ override def handle(request: RequestChannel.Request): Unit = { try { - trace(s"Handling request:${RequestConvertToJson.requestDesc(request.header, request.loggableRequest)} from connection ${request.context.connectionId};" + + trace(s"Handling request:${request.requestLog} from connection ${request.context.connectionId};" + s"securityProtocol:${request.context.securityProtocol},principal:${request.context.principal}") request.envelope.foreach { envelope => diff --git a/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala b/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala index 456d14c4164c3..83f0c81f4dce4 100644 --- a/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala +++ b/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala @@ -40,7 +40,7 @@ class TestRaftRequestHandler( override def handle(request: RequestChannel.Request): Unit = { try { - trace(s"Handling request:${RequestConvertToJson.requestDesc(request.header, request.loggableRequest)} from connection ${request.context.connectionId};" + + trace(s"Handling request:${request.requestDesc} from connection ${request.context.connectionId};" + s"securityProtocol:${request.context.securityProtocol},principal:${request.context.principal}") request.header.apiKey match { case ApiKeys.VOTE diff --git a/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala b/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala index 6d14e2485968e..c43064d42e96c 100644 --- a/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala +++ b/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala @@ -54,7 +54,7 @@ class RequestChannelTest { val loggableAlterConfigs = alterConfigs.loggableRequest.asInstanceOf[AlterConfigsRequest] val loggedConfig = loggableAlterConfigs.configs.get(resource) assertEquals(expectedValues, toMap(loggedConfig)) - val alterConfigsDesc = RequestConvertToJson.requestDesc(alterConfigs.header, alterConfigs.loggableRequest).toString + val alterConfigsDesc = alterConfigs.requestDesc.toString assertFalse(s"Sensitive config logged $alterConfigsDesc", alterConfigsDesc.contains(sensitiveValue)) } @@ -118,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 = RequestConvertToJson.requestDesc(alterConfigs.header, alterConfigs.loggableRequest).toString + val alterConfigsDesc = alterConfigs.requestDesc.toString assertFalse(s"Sensitive config logged $alterConfigsDesc", alterConfigsDesc.contains(sensitiveValue)) } diff --git a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala index b975316800d98..945cdac605c3f 100644 --- a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala +++ b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala @@ -85,7 +85,7 @@ class RequestConvertToJsonTest { val temporaryMemoryBytes = 8 val messageConversionsTimeMs = 9 - val expectedNode = RequestConvertToJson.requestDesc(req.header, req.loggableRequest).asInstanceOf[ObjectNode] + val expectedNode = req.requestDesc.asInstanceOf[ObjectNode] expectedNode.set("response", res.responseLog.getOrElse(new TextNode(""))) expectedNode.set("connection", new TextNode(req.context.connectionId)) expectedNode.set("totalTimeMs", new DoubleNode(totalTimeMs)) @@ -102,7 +102,7 @@ class RequestConvertToJsonTest { expectedNode.set("temporaryMemoryBytes", new LongNode(temporaryMemoryBytes)) expectedNode.set("messageConversionsTime", new DoubleNode(messageConversionsTimeMs)) - val actualNode = RequestConvertToJson.requestDescMetrics(req.header, res, req.loggableRequest, req.context, req.session, + val actualNode = RequestConvertToJson.requestDescMetrics(req.header, req.requestLog, res.responseLog, req.context, req.session, totalTimeMs, requestQueueTimeMs, apiLocalTimeMs, apiRemoteTimeMs, apiThrottleTimeMs, responseQueueTimeMs, responseSendTimeMs, temporaryMemoryBytes, messageConversionsTimeMs).asInstanceOf[ObjectNode] 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 fbc4ccd1df7ca..a2d38de1cbbfc 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,7 +21,6 @@ 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; @@ -217,6 +216,6 @@ public void testMetadataRequestForAllTopics() { @Benchmark public String testRequestToJson() { - return RequestConvertToJson.requestDesc(allTopicMetadataRequest.header(), allTopicMetadataRequest.loggableRequest()).toString(); + return allTopicMetadataRequest.requestDesc().toString(); } } From 97072bbaeba0b5dde308ac5dfb414ef7c2d1ddea Mon Sep 17 00:00:00 2001 From: anatasiavela Date: Fri, 11 Dec 2020 12:04:10 -0800 Subject: [PATCH 21/22] rebase and address comments --- .../common/requests/AbstractRequest.java | 2 +- .../requests/AddOffsetsToTxnRequest.java | 2 +- .../requests/AddOffsetsToTxnResponse.java | 2 +- .../requests/AddPartitionsToTxnRequest.java | 2 +- .../requests/AddPartitionsToTxnResponse.java | 6 +-- .../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 + .../requests/BeginQuorumEpochRequest.java | 2 +- .../requests/BeginQuorumEpochResponse.java | 2 +- .../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 | 2 +- .../common/requests/DeleteGroupsResponse.java | 2 +- .../requests/DeleteRecordsResponse.java | 1 + .../common/requests/DeleteTopicsRequest.java | 1 + .../common/requests/DeleteTopicsResponse.java | 1 + .../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 | 1 + .../requests/DescribeLogDirsResponse.java | 1 + .../requests/DescribeQuorumRequest.java | 6 +-- .../requests/DescribeQuorumResponse.java | 2 +- .../DescribeUserScramCredentialsRequest.java | 1 + .../DescribeUserScramCredentialsResponse.java | 1 + .../common/requests/ElectLeadersRequest.java | 1 + .../common/requests/ElectLeadersResponse.java | 1 + .../requests/EndQuorumEpochRequest.java | 2 +- .../requests/EndQuorumEpochResponse.java | 2 +- .../kafka/common/requests/EndTxnRequest.java | 2 +- .../kafka/common/requests/EndTxnResponse.java | 2 +- .../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 | 2 +- .../common/requests/HeartbeatResponse.java | 2 +- .../IncrementalAlterConfigsRequest.java | 1 + .../IncrementalAlterConfigsResponse.java | 1 + .../requests/InitProducerIdRequest.java | 2 +- .../requests/InitProducerIdResponse.java | 2 +- .../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 | 2 +- .../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 | 2 +- .../common/requests/OffsetDeleteResponse.java | 2 +- .../common/requests/OffsetFetchRequest.java | 2 +- .../common/requests/OffsetFetchResponse.java | 6 +-- .../kafka/common/requests/ProduceRequest.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 | 2 +- .../common/requests/SyncGroupResponse.java | 2 +- .../requests/TxnOffsetCommitRequest.java | 2 +- .../requests/TxnOffsetCommitResponse.java | 6 +-- .../requests/UpdateFeaturesRequest.java | 1 + .../requests/UpdateFeaturesResponse.java | 1 + .../requests/UpdateMetadataRequest.java | 2 +- .../requests/UpdateMetadataResponse.java | 2 +- .../kafka/common/requests/VoteRequest.java | 2 +- .../kafka/common/requests/VoteResponse.java | 2 +- .../requests/WriteTxnMarkersRequest.java | 2 +- .../common/requests/RequestContextTest.java | 4 +- .../scala/kafka/network/RequestChannel.scala | 24 ++++++----- .../kafka/network/RequestConvertToJson.scala | 29 +++++-------- .../main/scala/kafka/server/KafkaApis.scala | 13 +++--- .../kafka/tools/TestRaftRequestHandler.scala | 3 +- .../kafka/network/RequestChannelTest.scala | 4 +- .../network/RequestConvertToJsonTest.scala | 43 +++++++++++++------ .../unit/kafka/network/SocketServerTest.scala | 2 +- .../metadata/MetadataRequestBenchmark.java | 3 +- 112 files changed, 174 insertions(+), 129 deletions(-) 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 d09c2d6b612c6..28988895c074d 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 final Send toSend(RequestHeader header) { 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 390b61ade2ffe..1e5f9862178bf 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 @@ -53,7 +53,7 @@ public AddOffsetsToTxnRequest(AddOffsetsToTxnRequestData data, short version) { } @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 a1291f221970d..ce9a6cf7d6063 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 @@ -57,7 +57,7 @@ public int throttleTimeMs() { } @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 7bc37a4a2dfa4..1034c0f7adc55 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 @@ -112,7 +112,7 @@ public List partitions() { } @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 d301e16fb2de7..57b2a5a5d7c08 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 @@ -89,10 +89,6 @@ topicName, new AddPartitionsToTxnPartitionResultCollection() .setResults(topicCollection); } - public AddPartitionsToTxnResponseData data() { - return data; - } - @Override public int throttleTimeMs() { return data.throttleTimeMs(); @@ -121,7 +117,7 @@ public Map errorCounts() { } @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 7178475123633..d03c2671a290f 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 boolean validateOnly() { } @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 bce548822d895..fcacc5d95ef07 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 Map errorCounts() { } @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 f30e8b9c1ffe9..b4d35d52ae35f 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 boolean validateOnly() { } @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 7ce86aedf432e..516c2ce76aa85 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 AlterIsrRequest(AlterIsrRequestData data, short apiVersion) { 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 433ba66bdadd3..c3106ed94cbde 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 AlterIsrResponse(AlterIsrResponseData data) { 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 6aea8b1edcc87..ab166b812718c 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 static AlterPartitionReassignmentsResponse parse(ByteBuffer buffer, short 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 5eba039fa556b..68a87e6bf407e 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 AlterReplicaLogDirsRequest(AlterReplicaLogDirsRequestData data, short ver } @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 c319ec344525a..1ca7ea77aa422 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 static AlterUserScramCredentialsRequest parse(ByteBuffer buffer, short ve 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 2fa4937242742..97c0b7d17b204 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 AlterUserScramCredentialsResponse(AlterUserScramCredentialsResponseData r this.data = responseData; } + @Override public AlterUserScramCredentialsResponseData data() { return data; } 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 a7c61f97caaca..0794fb460959a 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 @@ -54,7 +54,7 @@ private BeginQuorumEpochRequest(BeginQuorumEpochRequestData data, short version) } @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 f25ac81c85e07..c8c0328c93a15 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 @@ -86,7 +86,7 @@ public Map errorCounts() { } @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 f3c063a7a060d..088c351302fdd 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 static ControlledShutdownRequest parse(ByteBuffer buffer, short version) 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 1add8000364e0..73b6a50268379 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 static ControlledShutdownResponse parse(ByteBuffer buffer, short version) 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 2ce651565b745..29df8326bc5a6 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 List aclCreations() { } @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 d0149b9a2a0d7..8bc6643f9de01 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 CreateAclsResponse(CreateAclsResponseData data) { } @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 e1d4cfaab80bd..1fee1b71eb3a4 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 static CreateDelegationTokenRequest parse(ByteBuffer buffer, short versio 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 9d39b6b641345..b679a30c8dd5c 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 static CreateDelegationTokenResponse prepareResponse(int throttleTimeMs, 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 8b0b9f81cc92c..d371bbb216995 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 String toString() { 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 e0af04b07ae6a..e59ac981f112a 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 CreatePartitionsResponse(CreatePartitionsResponseData data) { 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 7ec2c9adde3c4..9a1032b8c0955 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 CreateTopicsRequest(CreateTopicsRequestData data, short version) { 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 c15da20dfc79a..dd0627742587c 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 CreateTopicsResponse(CreateTopicsResponseData data) { 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 6face08f5675d..98fd6589b2280 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 List filters() { } @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 3ff8a9834f888..7482953a00d64 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 DeleteAclsResponse(DeleteAclsResponseData data, short version) { } @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 72e68424bc661..87d6deedc12c4 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 @@ -75,7 +75,7 @@ public static DeleteGroupsRequest parse(ByteBuffer buffer, short version) { } @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 627c5900c42c5..4cbffda422138 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 @@ -47,7 +47,7 @@ public DeleteGroupsResponse(DeleteGroupsResponseData data) { } @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 ef34102f8a8fd..b090543faddfb 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 DeleteRecordsResponse(DeleteRecordsResponseData data) { 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 5ab64184aeab0..dfd2e72df03d5 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 @@ private DeleteTopicsRequest(DeleteTopicsRequestData data, short version) { 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 dece4ec31685a..2090c4fd2e2ee 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 @@ -50,6 +50,7 @@ public int throttleTimeMs() { 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 e6f9c38fd598b..1ddf5bf99fc89 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 @@ else if (patternType != PatternType.LITERAL) } } + @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 4308c9ea45bf7..c4190e65640ea 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 DescribeAclsResponse(DescribeAclsResponseData data, short version) { } @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 1f167b0ee3c2b..68f4ecf9c0702 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 ClientQuotaFilter filter() { } @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 94fca64221d0f..3a81e21dac65e 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 int throttleTimeMs() { } @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 3e67140d375d0..9bf59e844a6c3 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 DescribeDelegationTokenRequest(DescribeDelegationTokenRequestData data, s 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 f6579a06b72da..4a2162f53aaef 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 Map errorCounts() { } @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 db0490dc1b498..eff5bb9fff1ec 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 @@ private DescribeGroupsRequest(DescribeGroupsRequestData data, short version) { 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 f0edbd75e1ae5..360caf01e468e 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 @@ -105,6 +105,7 @@ public static DescribedGroup groupMetadata( 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 61149253d5982..cd1326be9d1f9 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 DescribeLogDirsResponse(DescribeLogDirsResponseData data) { 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 4bc7dfbf07f97..acdb11c664420 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 @@ -71,12 +71,8 @@ public static DescribeQuorumRequestData singletonRequest(TopicPartition topicPar ))); } - public DescribeQuorumRequestData data() { - return data; - } - @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 cb45fb19085e3..cbf945b70409a 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 @@ -63,7 +63,7 @@ public Map errorCounts() { } @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 a2a90712bd3a5..0142e5ab6f592 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 @@ -58,6 +58,7 @@ public static DescribeUserScramCredentialsRequest parse(ByteBuffer buffer, short 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 a736c2c6876e5..001cefae41a6b 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 DescribeUserScramCredentialsResponse(DescribeUserScramCredentialsResponse 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 89600a9072465..92f6b45eed59d 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 @@ private ElectLeadersRequest(ElectLeadersRequestData data, short version) { 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 65eb09751b12d..88d4d19fc021e 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 ElectLeadersResponse( 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 ca9abe5168924..136bc54d50b2e 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 @@ -55,7 +55,7 @@ private EndQuorumEpochRequest(EndQuorumEpochRequestData data, short version) { } @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 1b853ef23ca59..ac2c0c5c9d5c3 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 @@ -64,7 +64,7 @@ public Map errorCounts() { } @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 0eb3c5b5d3e79..c9ea98005fd09 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 @@ -60,7 +60,7 @@ public TransactionResult result() { } @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 4eb9b7b5fe1ea..029e7d0ce5909 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 @@ -61,7 +61,7 @@ public Map errorCounts() { } @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 4be5259458b6d..5e8d3faea40ee 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 static EnvelopeRequest parse(ByteBuffer buffer, short version) { 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 c7c8d24e4a338..529f616bb26fc 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 Errors error() { 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 ac03944ab029e..85b02382baf89 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 static ExpireDelegationTokenRequest parse(ByteBuffer buffer, short versio } @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 451c736ffb7e6..163ee78d0adcd 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 Map errorCounts() { } @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 d65a55e6c2b29..30b462bd21933 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 b6fdbfc1e63a5..dc512a554e246 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 static FindCoordinatorRequest parse(ByteBuffer buffer, short version) { 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 917b087fbab35..11f3d48887900 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 FindCoordinatorResponse(FindCoordinatorResponseData data) { 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 4cdd2ab0e0954..482e61a255a8e 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 @@ -72,7 +72,7 @@ public static HeartbeatRequest parse(ByteBuffer buffer, short version) { } @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 c8fdae2507c6b..eb402fcbab9f7 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 Map errorCounts() { } @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 210e18b37accb..2bc591410f9a9 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 static IncrementalAlterConfigsRequest parse(ByteBuffer buffer, short vers 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 99427c7ad9f3e..b5887de9b4b75 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 IncrementalAlterConfigsResponse(IncrementalAlterConfigsResponseData data) 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 bd8d996086868..5c24b41b351df 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 @@ -73,7 +73,7 @@ public static InitProducerIdRequest parse(ByteBuffer buffer, short version) { } @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 580107f2e0982..f8451d7863b3f 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 @@ -54,7 +54,7 @@ public Map errorCounts() { } @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 7be16b6410c3b..35155a085b6f7 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 @@ private void maybeOverrideRebalanceTimeout(short version) { } } + @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 8472175edab4a..336c82462a21d 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 JoinGroupResponse(JoinGroupResponseData data) { 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 9927dda1de6e6..833e0255336d2 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 List liveLeaders() { 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 1e1069edea3a1..974dde84ec4c7 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 static LeaderAndIsrResponse parse(ByteBuffer buffer, short version) { } @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 ad7547c648ee7..8ce95350fc9bd 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 @@ private LeaveGroupRequest(LeaveGroupRequestData data, short version) { 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 56f28a70514ec..9a59139f4e77c 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 @@ -122,7 +122,7 @@ public Map errorCounts() { } @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 336368395b76f..270c43c0568ad 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 ListGroupsResponse(ListGroupsResponseData data) { 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 86cfc6632b7a0..03affd11b6706 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 static ListPartitionReassignmentsRequest parse(ByteBuffer buffer, short v 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 3449f1cabe9ba..4a890e8b50cd1 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 static ListPartitionReassignmentsResponse parse(ByteBuffer buffer, short 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 5aceb2291ff45..816f600061568 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 MetadataRequest(MetadataRequestData data, short version) { 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 ba06439bad1ec..097c26e687dea 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 MetadataResponse(MetadataResponseData data, short version) { } @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 6542ed425746a..9869da5d254b9 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 OffsetCommitRequest(OffsetCommitRequestData data, short version) { 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 b2851a68deb84..2ed0e312983ce 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 OffsetCommitResponse(Map responseData) { 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 8a3e48e349919..28b763d520f00 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 @@ -71,7 +71,7 @@ public static OffsetDeleteRequest parse(ByteBuffer buffer, short version) { } @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 2e3b4e5455b2a..79f6f4e6d3495 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 @@ -52,7 +52,7 @@ public OffsetDeleteResponse(OffsetDeleteResponseData data) { } @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 b05b4ac41901c..c35d479d64dba 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 @@ -175,7 +175,7 @@ public boolean isAllPartitions() { } @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 4f673bc769010..594eb0e5bcf93 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 @@ -176,10 +176,6 @@ private static Errors topLevelError(OffsetFetchResponseData data) { return Errors.NONE; } - public OffsetFetchResponseData data() { - return data; - } - @Override public int throttleTimeMs() { return data.throttleTimeMs(); @@ -223,7 +219,7 @@ public static OffsetFetchResponse parse(ByteBuffer buffer, short version) { } @Override - protected OffsetFetchResponseData data() { + public OffsetFetchResponseData data() { return data; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java index 9af463b9aa083..758631a1d87aa 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -217,10 +217,6 @@ public String transactionalId() { return transactionalId; } - public ProduceRequestData data() { - return data; - } - public void clearPartitionRecords() { // lazily initialize partitionSizes. partitionSizes(); 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 2860576d53972..b94e48b54a1c6 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 @@ private static ProduceResponseData toData(Map 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 2c364aef3dff7..91a9f968d9d48 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 static RenewDelegationTokenRequest parse(ByteBuffer buffer, short version 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 2ef840378acb3..30708ff038c25 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 Map errorCounts() { } @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 35da4caf9c5c1..e2080ce841bfc 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 SaslAuthenticateRequest(SaslAuthenticateRequestData data, short version) 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 2e1a2189a39be..bd12d3d4ae7bb 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 int throttleTimeMs() { } @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 e64dd6b2cb792..09d3a87c8e727 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 SaslHandshakeRequest(SaslHandshakeRequestData data, short version) { 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 b0ac2efcdec5f..63c047a06196b 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 int throttleTimeMs() { } @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 2b0aef766935b..4326aaffd8731 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 static StopReplicaRequest parse(ByteBuffer buffer, short version) { } @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 27a2502e75103..10ab153f440b9 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 int throttleTimeMs() { } @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 6651e965cd164..8242b71a03fbd 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 @@ -92,7 +92,7 @@ public static SyncGroupRequest parse(ByteBuffer buffer, short version) { } @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 cfe18d6e1b3cf..822a3e78b9949 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 @@ -48,7 +48,7 @@ public Map errorCounts() { } @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 15af67d8787f6..e96f81ae7e2f9 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 @@ -168,7 +168,7 @@ static List getTopics(Map liveBrokers() { } @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 4749067a9a786..cc7749a47242c 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 static UpdateMetadataResponse parse(ByteBuffer buffer, short version) { } @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 e2dcab11b1698..8fba2f085d566 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 @@ -55,7 +55,7 @@ private VoteRequest(VoteRequestData data, short version) { } @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 8000457b292a6..51991adcf0cbb 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 @@ -83,7 +83,7 @@ public Map errorCounts() { } @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 52dc600524d13..64a3df49c7b39 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 @@ -142,7 +142,7 @@ private WriteTxnMarkersRequest(WriteTxnMarkersRequestData data, short version) { } @Override - protected WriteTxnMarkersRequestData data() { + public WriteTxnMarkersRequestData data() { return 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 50f41556b0bad..5bef2e42a530c 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 void testSerdeUnsupportedApiVersionRequest() throws Exception { 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/core/src/main/scala/kafka/network/RequestChannel.scala b/core/src/main/scala/kafka/network/RequestChannel.scala index e7dea665a2846..b59033d078883 100644 --- a/core/src/main/scala/kafka/network/RequestChannel.scala +++ b/core/src/main/scala/kafka/network/RequestChannel.scala @@ -99,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 @@ -125,13 +131,6 @@ object RequestChannel extends Logging { } } - def requestLog: Option[JsonNode] = { - if (RequestChannel.isRequestLoggingEnabled) - Some(RequestConvertToJson.request(loggableRequest)) - else - None - } - def responseNode(response: AbstractResponse): Option[JsonNode] = { if (RequestChannel.isRequestLoggingEnabled) Some(RequestConvertToJson.response(response, context.apiVersion())) @@ -148,8 +147,11 @@ object RequestChannel extends Logging { } } - def requestDesc: JsonNode = { - RequestConvertToJson.requestDesc(header, requestLog, envelope) + def requestDesc(details: Boolean): String = { + val forwardDescription = envelope.map { request => + s"Forwarded request: ${request.context} " + }.getOrElse("") + s"$forwardDescription$header -- ${loggableRequest.toString(details)}" } def body[T <: AbstractRequest](implicit classTag: ClassTag[T], @nowarn("cat=unused") nn: NotNothing[T]): T = { @@ -206,7 +208,7 @@ object RequestChannel extends Logging { } } - trace(s"Processor $processor received request: $requestDesc") + trace(s"Processor $processor received request: ${RequestConvertToJson.requestDesc(header, requestLog, isForwarded)}") def requestThreadTimeNanos: Long = { if (apiLocalCompleteTimeNanos == -1L) apiLocalCompleteTimeNanos = Time.SYSTEM.nanoseconds @@ -268,7 +270,7 @@ object RequestChannel extends Logging { if (isRequestLoggingEnabled) { val desc = RequestConvertToJson.requestDescMetrics(header, requestLog, response.responseLog, - context, session, + context, session, isForwarded, totalTimeMs, requestQueueTimeMs, apiLocalTimeMs, apiRemoteTimeMs, apiThrottleTimeMs, responseQueueTimeMs, responseSendTimeMs, temporaryMemoryBytes, diff --git a/core/src/main/scala/kafka/network/RequestConvertToJson.scala b/core/src/main/scala/kafka/network/RequestConvertToJson.scala index 71c3683fae1bb..09919aeefbbdd 100644 --- a/core/src/main/scala/kafka/network/RequestConvertToJson.scala +++ b/core/src/main/scala/kafka/network/RequestConvertToJson.scala @@ -18,9 +18,8 @@ package kafka.network import com.fasterxml.jackson.databind.JsonNode -import com.fasterxml.jackson.databind.node.{BooleanNode, DoubleNode, IntNode, JsonNodeFactory, LongNode, ObjectNode, TextNode} +import com.fasterxml.jackson.databind.node.{BooleanNode, DoubleNode, JsonNodeFactory, LongNode, ObjectNode, TextNode} import kafka.network.RequestChannel.Session -import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.message._ import org.apache.kafka.common.network.ClientInformation import org.apache.kafka.common.requests._ @@ -76,7 +75,7 @@ object RequestConvertToJson { 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) + 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) @@ -87,7 +86,7 @@ object RequestConvertToJson { 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.api} is not currently handled in `request`, the " + + case _ => throw new IllegalStateException(s"ApiKey ${request.apiKey} is not currently handled in `request`, the " + "code should be updated to do so."); } } @@ -141,8 +140,8 @@ object RequestConvertToJson { 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: 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) @@ -153,7 +152,7 @@ object RequestConvertToJson { 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 is not currently handled in `response`, the " + + case _ => throw new IllegalStateException(s"ApiKey ${response.apiKey} is not currently handled in `response`, the " + "code should be updated to do so."); } } @@ -173,10 +172,9 @@ object RequestConvertToJson { node } - def requestDesc(header: RequestHeader, requestNode: Option[JsonNode], envelope: Option[RequestChannel.Request] = None): JsonNode = { + def requestDesc(header: RequestHeader, requestNode: Option[JsonNode], isForwarded: Boolean): JsonNode = { val node = new ObjectNode(JsonNodeFactory.instance) - node.set("isForwarded", if (envelope.isDefined) BooleanNode.TRUE else BooleanNode.FALSE) - node.set("forwardedRequest", envelope.map(request => requestContextNode(request.context)).getOrElse(new TextNode(""))) + node.set("isForwarded", if (isForwarded) BooleanNode.TRUE else BooleanNode.FALSE) node.set("requestHeader", requestHeaderNode(header)) node.set("request", requestNode.getOrElse(new TextNode(""))) node @@ -189,20 +187,13 @@ object RequestConvertToJson { node } - def topicPartitionNode(topicPartition: TopicPartition): JsonNode = { - val node = new ObjectNode(JsonNodeFactory.instance) - node.set("topic", new TextNode(topicPartition.topic)) - node.set("partition", new IntNode(topicPartition.partition)) - node - } - def requestDescMetrics(header: RequestHeader, requestNode: Option[JsonNode], responseNode: Option[JsonNode], - context: RequestContext, session: Session, + 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).asInstanceOf[ObjectNode] + 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)) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index f8f35586e91a0..dd8985810790e 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -24,6 +24,7 @@ import java.util import java.util.{Collections, Optional} import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicInteger + import kafka.admin.{AdminUtils, RackAwareMode} import kafka.api.ElectLeadersRequestOps import kafka.api.{ApiVersion, KAFKA_0_11_0_IV0, KAFKA_2_3_IV0} @@ -34,7 +35,7 @@ import kafka.coordinator.group.{GroupCoordinator, JoinGroupResult, LeaveGroupRes import kafka.coordinator.transaction.{InitProducerIdResult, TransactionCoordinator} import kafka.log.AppendOrigin import kafka.message.ZStdCompressionCodec -import kafka.network.RequestChannel +import kafka.network.{RequestChannel, RequestConvertToJson} import kafka.security.authorizer.{AclEntry, AuthorizerUtils} import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota} import kafka.utils.{CoreUtils, Logging} @@ -62,9 +63,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} @@ -97,7 +98,8 @@ import scala.annotation.nowarn /** * Logic to handle the various Kafka requests */ -class KafkaApis(val requestChannel: RequestChannel, +class +KafkaApis(val requestChannel: RequestChannel, val replicaManager: ReplicaManager, val adminManager: AdminManager, val groupCoordinator: GroupCoordinator, @@ -183,7 +185,8 @@ class KafkaApis(val requestChannel: RequestChannel, */ override def handle(request: RequestChannel.Request): Unit = { try { - trace(s"Handling request:${request.requestLog} from connection ${request.context.connectionId};" + + trace(s"Handling request:${RequestConvertToJson.requestDesc(request.header, request.requestLog, request.isForwarded)} " + + s"from connection ${request.context.connectionId};" + s"securityProtocol:${request.context.securityProtocol},principal:${request.context.principal}") request.envelope.foreach { envelope => diff --git a/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala b/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala index 83f0c81f4dce4..b2007221d6f5d 100644 --- a/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala +++ b/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala @@ -40,7 +40,8 @@ class TestRaftRequestHandler( override def handle(request: RequestChannel.Request): Unit = { try { - trace(s"Handling request:${request.requestDesc} from connection ${request.context.connectionId};" + + trace(s"Handling request:${RequestConvertToJson.requestDesc(request.header, request.requestLog, request.isForwarded)} " + + s"from connection ${request.context.connectionId};" + s"securityProtocol:${request.context.securityProtocol},principal:${request.context.principal}") request.header.apiKey match { case ApiKeys.VOTE diff --git a/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala b/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala index c43064d42e96c..b7b950794fef6 100644 --- a/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala +++ b/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala @@ -54,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.toString + val alterConfigsDesc = RequestConvertToJson.requestDesc(alterConfigs.header, alterConfigs.requestLog, alterConfigs.isForwarded).toString assertFalse(s"Sensitive config logged $alterConfigsDesc", alterConfigsDesc.contains(sensitiveValue)) } @@ -118,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.toString + val alterConfigsDesc = RequestConvertToJson.requestDesc(alterConfigs.header, alterConfigs.requestLog, alterConfigs.isForwarded).toString assertFalse(s"Sensitive config logged $alterConfigsDesc", alterConfigsDesc.contains(sensitiveValue)) } diff --git a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala index 945cdac605c3f..21efb1581a00b 100644 --- a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala +++ b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala @@ -19,13 +19,14 @@ package kafka.network import java.net.InetAddress import java.nio.ByteBuffer + import com.fasterxml.jackson.databind.node.{DoubleNode, LongNode, ObjectNode, TextNode} import kafka.network 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 +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 @@ -40,8 +41,20 @@ class RequestConvertToJsonTest { val unhandledKeys = ArrayBuffer[String]() ApiKeys.values().foreach { key => { val version: Short = key.latestVersion() - val struct = ApiMessageType.fromApiKey(key.id).newRequest().toStruct(version) - val req = AbstractRequest.parseRequest(key, version, struct) + 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 { @@ -56,8 +69,13 @@ class RequestConvertToJsonTest { val unhandledKeys = ArrayBuffer[String]() ApiKeys.values().foreach { key => { val version: Short = key.latestVersion() - val struct = ApiMessageType.fromApiKey(key.id).newResponse().toStruct(version) - val res = AbstractResponse.parseResponse(key, struct, version) + 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 { @@ -69,9 +87,9 @@ class RequestConvertToJsonTest { @Test def testRequestDescMetrics(): Unit = { - val req = request(new AlterIsrRequest(new AlterIsrRequestData(), 0)) - val byteBuffer = req.body[AbstractRequest].serialize(req.header) - val send = new NetworkSend(req.context.connectionId, byteBuffer) + val rwq = new AlterIsrRequest(new AlterIsrRequestData(), 0) + val req = request(rwq) + val send = new NetworkSend(req.context.connectionId, rwq.toSend(req.header)) val headerLog = RequestConvertToJson.requestHeaderNode(req.header) val res = new RequestChannel.SendResponse(req, send, Some(headerLog), None) @@ -85,7 +103,7 @@ class RequestConvertToJsonTest { val temporaryMemoryBytes = 8 val messageConversionsTimeMs = 9 - val expectedNode = req.requestDesc.asInstanceOf[ObjectNode] + 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)) @@ -102,15 +120,16 @@ class RequestConvertToJsonTest { 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, + 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) } - private def request(req: AbstractRequest): RequestChannel.Request = { - val buffer = req.serialize(new RequestHeader(req.api, req.version, "client-id", 1)) + 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, diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index 927a932fa3639..e3c683ce17723 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -1095,7 +1095,7 @@ class SocketServerTest { val requestMetrics = channel.metrics(request.header.apiKey.name) def totalTimeHistCount(): Long = requestMetrics.totalTimeHist.count - val send = new NetworkSend(request.context.connectionId, ByteBuffer.allocate(responseBufferSize)) + val send = new NetworkSend(request.context.connectionId, ByteBufferSend.sizePrefixed(ByteBuffer.allocate(responseBufferSize))) val headerLog = new ObjectNode(JsonNodeFactory.instance) headerLog.set("response", new TextNode("someResponse")) channel.sendResponse(new RequestChannel.SendResponse(request, send, Some(headerLog), None)) 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 a2d38de1cbbfc..d5a9dc4317f3a 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.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; @@ -216,6 +217,6 @@ public void testMetadataRequestForAllTopics() { @Benchmark public String testRequestToJson() { - return allTopicMetadataRequest.requestDesc().toString(); + return RequestConvertToJson.requestDesc(allTopicMetadataRequest.header(), allTopicMetadataRequest.requestLog(), allTopicMetadataRequest.isForwarded()).toString(); } } From 1fc3a11ca54b9958c1ef20fbca9abb4606ecbdd0 Mon Sep 17 00:00:00 2001 From: anatasiavela Date: Mon, 14 Dec 2020 09:59:50 -0800 Subject: [PATCH 22/22] address minor comments --- .../scala/kafka/network/RequestChannel.scala | 4 +- .../kafka/network/RequestConvertToJson.scala | 9 ---- .../main/scala/kafka/server/KafkaApis.scala | 9 ++-- .../kafka/tools/TestRaftRequestHandler.scala | 3 +- .../network/RequestConvertToJsonTest.scala | 51 +++++++++++++++++-- .../kafka/message/JsonConverterGenerator.java | 5 +- 6 files changed, 56 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/kafka/network/RequestChannel.scala b/core/src/main/scala/kafka/network/RequestChannel.scala index b59033d078883..0c03d70188165 100644 --- a/core/src/main/scala/kafka/network/RequestChannel.scala +++ b/core/src/main/scala/kafka/network/RequestChannel.scala @@ -133,7 +133,7 @@ object RequestChannel extends Logging { def responseNode(response: AbstractResponse): Option[JsonNode] = { if (RequestChannel.isRequestLoggingEnabled) - Some(RequestConvertToJson.response(response, context.apiVersion())) + Some(RequestConvertToJson.response(response, context.apiVersion)) else None } @@ -208,7 +208,7 @@ object RequestChannel extends Logging { } } - trace(s"Processor $processor received request: ${RequestConvertToJson.requestDesc(header, requestLog, isForwarded)}") + trace(s"Processor $processor received request: ${requestDesc(true)}") def requestThreadTimeNanos: Long = { if (apiLocalCompleteTimeNanos == -1L) apiLocalCompleteTimeNanos = Time.SYSTEM.nanoseconds diff --git a/core/src/main/scala/kafka/network/RequestConvertToJson.scala b/core/src/main/scala/kafka/network/RequestConvertToJson.scala index 09919aeefbbdd..243c2a4ca50ed 100644 --- a/core/src/main/scala/kafka/network/RequestConvertToJson.scala +++ b/core/src/main/scala/kafka/network/RequestConvertToJson.scala @@ -163,15 +163,6 @@ object RequestConvertToJson { node } - def requestContextNode(context: RequestContext): JsonNode = { - val node = new ObjectNode(JsonNodeFactory.instance) - node.set("connection", new TextNode(context.connectionId)) - node.set("securityProtocol", new TextNode(context.securityProtocol.toString)) - node.set("listener", new TextNode(context.listenerName.value)) - node.set("clientInformation", clientInfoNode(context.clientInformation)) - 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) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index dd8985810790e..2d221708a194c 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -24,7 +24,6 @@ import java.util import java.util.{Collections, Optional} import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicInteger - import kafka.admin.{AdminUtils, RackAwareMode} import kafka.api.ElectLeadersRequestOps import kafka.api.{ApiVersion, KAFKA_0_11_0_IV0, KAFKA_2_3_IV0} @@ -35,7 +34,7 @@ import kafka.coordinator.group.{GroupCoordinator, JoinGroupResult, LeaveGroupRes import kafka.coordinator.transaction.{InitProducerIdResult, TransactionCoordinator} import kafka.log.AppendOrigin import kafka.message.ZStdCompressionCodec -import kafka.network.{RequestChannel, RequestConvertToJson} +import kafka.network.RequestChannel import kafka.security.authorizer.{AclEntry, AuthorizerUtils} import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota} import kafka.utils.{CoreUtils, Logging} @@ -98,8 +97,7 @@ import scala.annotation.nowarn /** * Logic to handle the various Kafka requests */ -class -KafkaApis(val requestChannel: RequestChannel, +class KafkaApis(val requestChannel: RequestChannel, val replicaManager: ReplicaManager, val adminManager: AdminManager, val groupCoordinator: GroupCoordinator, @@ -185,8 +183,7 @@ KafkaApis(val requestChannel: RequestChannel, */ override def handle(request: RequestChannel.Request): Unit = { try { - trace(s"Handling request:${RequestConvertToJson.requestDesc(request.header, request.requestLog, request.isForwarded)} " + - s"from connection ${request.context.connectionId};" + + trace(s"Handling request:${request.requestDesc(true)} from connection ${request.context.connectionId};" + s"securityProtocol:${request.context.securityProtocol},principal:${request.context.principal}") request.envelope.foreach { envelope => diff --git a/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala b/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala index b2007221d6f5d..4fad0db4f1e83 100644 --- a/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala +++ b/core/src/main/scala/kafka/tools/TestRaftRequestHandler.scala @@ -40,8 +40,7 @@ class TestRaftRequestHandler( override def handle(request: RequestChannel.Request): Unit = { try { - trace(s"Handling request:${RequestConvertToJson.requestDesc(request.header, request.requestLog, request.isForwarded)} " + - s"from connection ${request.context.connectionId};" + + trace(s"Handling request:${request.requestDesc(true)} from connection ${request.context.connectionId};" + s"securityProtocol:${request.context.securityProtocol},principal:${request.context.principal}") request.header.apiKey match { case ApiKeys.VOTE diff --git a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala index 21efb1581a00b..5dd7e31496bc7 100644 --- a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala +++ b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala @@ -20,8 +20,9 @@ package kafka.network import java.net.InetAddress import java.nio.ByteBuffer -import com.fasterxml.jackson.databind.node.{DoubleNode, LongNode, ObjectNode, TextNode} +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} @@ -85,11 +86,53 @@ class RequestConvertToJsonTest { 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 rwq = new AlterIsrRequest(new AlterIsrRequestData(), 0) - val req = request(rwq) - val send = new NetworkSend(req.context.connectionId, rwq.toSend(req.header)) + 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) 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 2d6c77879248e..56809f44661be 100644 --- a/generator/src/main/java/org/apache/kafka/message/JsonConverterGenerator.java +++ b/generator/src/main/java/org/apache/kafka/message/JsonConverterGenerator.java @@ -401,8 +401,9 @@ private void generateVariableLengthTargetToJson(Target target, Versions versions buffer.decrementIndent(); buffer.printf("} else {%n"); buffer.incrementIndent(); - buffer.printf("%s;%n", target.assignmentStatement( - String.format("new IntNode(%s.sizeInBytes())", target.sourceVariable()))); + 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()) {