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 4558e86d8b07f..c449eeb72df3d 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 @@ -126,10 +126,10 @@ import org.apache.kafka.common.message.LeaveGroupResponseData.MemberResponse; import org.apache.kafka.common.message.ListGroupsRequestData; import org.apache.kafka.common.message.ListGroupsResponseData; -import org.apache.kafka.common.message.ListOffsetRequestData.ListOffsetPartition; -import org.apache.kafka.common.message.ListOffsetRequestData.ListOffsetTopic; -import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetPartitionResponse; -import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetTopicResponse; +import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition; +import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsTopic; +import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse; +import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse; import org.apache.kafka.common.message.ListPartitionReassignmentsRequestData; import org.apache.kafka.common.message.MetadataRequestData; import org.apache.kafka.common.message.OffsetCommitRequestData; @@ -215,8 +215,8 @@ import org.apache.kafka.common.requests.LeaveGroupResponse; import org.apache.kafka.common.requests.ListGroupsRequest; import org.apache.kafka.common.requests.ListGroupsResponse; -import org.apache.kafka.common.requests.ListOffsetRequest; -import org.apache.kafka.common.requests.ListOffsetResponse; +import org.apache.kafka.common.requests.ListOffsetsRequest; +import org.apache.kafka.common.requests.ListOffsetsResponse; import org.apache.kafka.common.requests.ListPartitionReassignmentsRequest; import org.apache.kafka.common.requests.ListPartitionReassignmentsResponse; import org.apache.kafka.common.requests.MetadataRequest; @@ -3975,7 +3975,7 @@ private List getListOffsetsCalls(MetadataOperationContext new IllegalStateException("No Metadata response")); List calls = new ArrayList<>(); // grouping topic partitions per leader - Map> leaders = new HashMap<>(); + Map> leaders = new HashMap<>(); for (Map.Entry entry: topicPartitionOffsets.entrySet()) { @@ -3985,15 +3985,15 @@ private List getListOffsetsCalls(MetadataOperationContext leadersOnNode = leaders.computeIfAbsent(node, k -> new HashMap()); - ListOffsetTopic topic = leadersOnNode.computeIfAbsent(tp.topic(), k -> new ListOffsetTopic().setName(tp.topic())); - topic.partitions().add(new ListOffsetPartition().setPartitionIndex(tp.partition()).setTimestamp(offsetQuery)); + Map leadersOnNode = leaders.computeIfAbsent(node, k -> new HashMap<>()); + ListOffsetsTopic topic = leadersOnNode.computeIfAbsent(tp.topic(), k -> new ListOffsetsTopic().setName(tp.topic())); + topic.partitions().add(new ListOffsetsPartition().setPartitionIndex(tp.partition()).setTimestamp(offsetQuery)); } else { future.completeExceptionally(Errors.LEADER_NOT_AVAILABLE.exception()); } @@ -4002,27 +4002,27 @@ private List getListOffsetsCalls(MetadataOperationContext> entry : leaders.entrySet()) { + for (final Map.Entry> entry : leaders.entrySet()) { final int brokerId = entry.getKey().id(); calls.add(new Call("listOffsets on broker " + brokerId, context.deadline(), new ConstantNodeIdProvider(brokerId)) { - final List partitionsToQuery = new ArrayList<>(entry.getValue().values()); + final List partitionsToQuery = new ArrayList<>(entry.getValue().values()); @Override - ListOffsetRequest.Builder createRequest(int timeoutMs) { - return ListOffsetRequest.Builder + ListOffsetsRequest.Builder createRequest(int timeoutMs) { + return ListOffsetsRequest.Builder .forConsumer(true, context.options().isolationLevel()) .setTargetTimes(partitionsToQuery); } @Override void handleResponse(AbstractResponse abstractResponse) { - ListOffsetResponse response = (ListOffsetResponse) abstractResponse; + ListOffsetsResponse response = (ListOffsetsResponse) abstractResponse; Map retryTopicPartitionOffsets = new HashMap<>(); - for (ListOffsetTopicResponse topic : response.topics()) { - for (ListOffsetPartitionResponse partition : topic.partitions()) { + for (ListOffsetsTopicResponse topic : response.topics()) { + for (ListOffsetsPartitionResponse partition : topic.partitions()) { TopicPartition tp = new TopicPartition(topic.name(), partition.partitionIndex()); KafkaFutureImpl future = futures.get(tp); Errors error = Errors.forCode(partition.errorCode()); @@ -4032,7 +4032,7 @@ void handleResponse(AbstractResponse abstractResponse) { } else if (MetadataOperationContext.shouldRefreshMetadata(error)) { retryTopicPartitionOffsets.put(tp, offsetRequestSpec); } else if (error == Errors.NONE) { - Optional leaderEpoch = (partition.leaderEpoch() == ListOffsetResponse.UNKNOWN_EPOCH) + Optional leaderEpoch = (partition.leaderEpoch() == ListOffsetsResponse.UNKNOWN_EPOCH) ? Optional.empty() : Optional.of(partition.leaderEpoch()); future.complete(new ListOffsetsResultInfo(partition.offset(), partition.timestamp(), leaderEpoch)); @@ -4044,8 +4044,8 @@ void handleResponse(AbstractResponse abstractResponse) { if (retryTopicPartitionOffsets.isEmpty()) { // The server should send back a response for every topic partition. But do a sanity check anyway. - for (ListOffsetTopic topic : partitionsToQuery) { - for (ListOffsetPartition partition : topic.partitions()) { + for (ListOffsetsTopic topic : partitionsToQuery) { + for (ListOffsetsPartition partition : topic.partitions()) { TopicPartition tp = new TopicPartition(topic.name(), partition.partitionIndex()); ApiException error = new ApiException("The response from broker " + brokerId + " did not contain a result for topic partition " + tp); @@ -4063,8 +4063,8 @@ void handleResponse(AbstractResponse abstractResponse) { @Override void handleFailure(Throwable throwable) { - for (ListOffsetTopic topic : entry.getValue().values()) { - for (ListOffsetPartition partition : topic.partitions()) { + for (ListOffsetsTopic topic : entry.getValue().values()) { + for (ListOffsetsPartition partition : topic.partitions()) { TopicPartition tp = new TopicPartition(topic.name(), partition.partitionIndex()); KafkaFutureImpl future = futures.get(tp); future.completeExceptionally(throwable); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index 7b1101a0a8a19..d5dbf22c02bb5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -48,9 +48,9 @@ import org.apache.kafka.common.errors.TopicAuthorizationException; import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.header.internals.RecordHeaders; -import org.apache.kafka.common.message.ListOffsetRequestData.ListOffsetPartition; -import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetPartitionResponse; -import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetTopicResponse; +import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition; +import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse; +import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse; import org.apache.kafka.common.metrics.Gauge; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Sensor; @@ -70,8 +70,8 @@ import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.requests.FetchResponse; -import org.apache.kafka.common.requests.ListOffsetRequest; -import org.apache.kafka.common.requests.ListOffsetResponse; +import org.apache.kafka.common.requests.ListOffsetsRequest; +import org.apache.kafka.common.requests.ListOffsetsResponse; import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.requests.MetadataResponse; import org.apache.kafka.common.requests.OffsetsForLeaderEpochRequest; @@ -440,17 +440,17 @@ private RequestFuture sendMetadataRequest(MetadataRequest.Builde private Long offsetResetStrategyTimestamp(final TopicPartition partition) { OffsetResetStrategy strategy = subscriptions.resetStrategy(partition); if (strategy == OffsetResetStrategy.EARLIEST) - return ListOffsetRequest.EARLIEST_TIMESTAMP; + return ListOffsetsRequest.EARLIEST_TIMESTAMP; else if (strategy == OffsetResetStrategy.LATEST) - return ListOffsetRequest.LATEST_TIMESTAMP; + return ListOffsetsRequest.LATEST_TIMESTAMP; else return null; } private OffsetResetStrategy timestampToOffsetResetStrategy(long timestamp) { - if (timestamp == ListOffsetRequest.EARLIEST_TIMESTAMP) + if (timestamp == ListOffsetsRequest.EARLIEST_TIMESTAMP) return OffsetResetStrategy.EARLIEST; - else if (timestamp == ListOffsetRequest.LATEST_TIMESTAMP) + else if (timestamp == ListOffsetsRequest.LATEST_TIMESTAMP) return OffsetResetStrategy.LATEST; else return null; @@ -563,11 +563,11 @@ private ListOffsetResult fetchOffsetsByTimes(Map timestamp } public Map beginningOffsets(Collection partitions, Timer timer) { - return beginningOrEndOffset(partitions, ListOffsetRequest.EARLIEST_TIMESTAMP, timer); + return beginningOrEndOffset(partitions, ListOffsetsRequest.EARLIEST_TIMESTAMP, timer); } public Map endOffsets(Collection partitions, Timer timer) { - return beginningOrEndOffset(partitions, ListOffsetRequest.LATEST_TIMESTAMP, timer); + return beginningOrEndOffset(partitions, ListOffsetsRequest.LATEST_TIMESTAMP, timer); } private Map beginningOrEndOffset(Collection partitions, @@ -733,11 +733,11 @@ void resetOffsetIfNeeded(TopicPartition partition, OffsetResetStrategy requested } private void resetOffsetsAsync(Map partitionResetTimestamps) { - Map> timestampsToSearchByNode = + Map> timestampsToSearchByNode = groupListOffsetRequests(partitionResetTimestamps, new HashSet<>()); - for (Map.Entry> entry : timestampsToSearchByNode.entrySet()) { + for (Map.Entry> entry : timestampsToSearchByNode.entrySet()) { Node node = entry.getKey(); - final Map resetTimestamps = entry.getValue(); + final Map resetTimestamps = entry.getValue(); subscriptions.setNextAllowedRetry(resetTimestamps.keySet(), time.milliseconds() + requestTimeoutMs); RequestFuture future = sendListOffsetRequest(node, resetTimestamps, false); @@ -752,7 +752,7 @@ public void onSuccess(ListOffsetResult result) { for (Map.Entry fetchedOffset : result.fetchedOffsets.entrySet()) { TopicPartition partition = fetchedOffset.getKey(); ListOffsetData offsetData = fetchedOffset.getValue(); - ListOffsetPartition requestedReset = resetTimestamps.get(partition); + ListOffsetsPartition requestedReset = resetTimestamps.get(partition); resetOffsetIfNeeded(partition, timestampToOffsetResetStrategy(requestedReset.timestamp()), offsetData); } } @@ -884,7 +884,7 @@ private void maybeSetOffsetForLeaderException(RuntimeException e) { private RequestFuture sendListOffsetsRequests(final Map timestampsToSearch, final boolean requireTimestamps) { final Set partitionsToRetry = new HashSet<>(); - Map> timestampsToSearchByNode = + Map> timestampsToSearchByNode = groupListOffsetRequests(timestampsToSearch, partitionsToRetry); if (timestampsToSearchByNode.isEmpty()) return RequestFuture.failure(new StaleMetadataException()); @@ -893,7 +893,7 @@ private RequestFuture sendListOffsetsRequests(final Map fetchedTimestampOffsets = new HashMap<>(); final AtomicInteger remainingResponses = new AtomicInteger(timestampsToSearchByNode.size()); - for (Map.Entry> entry : timestampsToSearchByNode.entrySet()) { + for (Map.Entry> entry : timestampsToSearchByNode.entrySet()) { RequestFuture future = sendListOffsetRequest(entry.getKey(), entry.getValue(), requireTimestamps); future.addListener(new RequestFutureListener() { @@ -930,10 +930,10 @@ public void onFailure(RuntimeException e) { * @param partitionsToRetry A set of topic partitions that will be extended with partitions * that need metadata update or re-connect to the leader. */ - private Map> groupListOffsetRequests( + private Map> groupListOffsetRequests( Map timestampsToSearch, Set partitionsToRetry) { - final Map partitionDataMap = new HashMap<>(); + final Map partitionDataMap = new HashMap<>(); for (Map.Entry entry: timestampsToSearch.entrySet()) { TopicPartition tp = entry.getKey(); Long offset = entry.getValue(); @@ -955,8 +955,8 @@ private Map> groupListOffsetReque leader, tp); partitionsToRetry.add(tp); } else { - int currentLeaderEpoch = leaderAndEpoch.epoch.orElse(ListOffsetResponse.UNKNOWN_EPOCH); - partitionDataMap.put(tp, new ListOffsetPartition() + int currentLeaderEpoch = leaderAndEpoch.epoch.orElse(ListOffsetsResponse.UNKNOWN_EPOCH); + partitionDataMap.put(tp, new ListOffsetsPartition() .setPartitionIndex(tp.partition()) .setTimestamp(offset) .setCurrentLeaderEpoch(currentLeaderEpoch)); @@ -975,18 +975,18 @@ private Map> groupListOffsetReque * @return A response which can be polled to obtain the corresponding timestamps and offsets. */ private RequestFuture sendListOffsetRequest(final Node node, - final Map timestampsToSearch, + final Map timestampsToSearch, boolean requireTimestamp) { - ListOffsetRequest.Builder builder = ListOffsetRequest.Builder + ListOffsetsRequest.Builder builder = ListOffsetsRequest.Builder .forConsumer(requireTimestamp, isolationLevel) - .setTargetTimes(ListOffsetRequest.toListOffsetTopics(timestampsToSearch)); + .setTargetTimes(ListOffsetsRequest.toListOffsetsTopics(timestampsToSearch)); log.debug("Sending ListOffsetRequest {} to broker {}", builder, node); return client.send(node, builder) .compose(new RequestFutureAdapter() { @Override public void onSuccess(ClientResponse response, RequestFuture future) { - ListOffsetResponse lor = (ListOffsetResponse) response.responseBody(); + ListOffsetsResponse lor = (ListOffsetsResponse) response.responseBody(); log.trace("Received ListOffsetResponse {} from broker {}", lor, node); handleListOffsetResponse(lor, future); } @@ -995,7 +995,7 @@ public void onSuccess(ClientResponse response, RequestFuture f /** * Callback for the response of the list offset call above. - * @param listOffsetResponse The response from the server. + * @param listOffsetsResponse The response from the server. * @param future The future to be completed when the response returns. Note that any partition-level errors will * generally fail the entire future result. The one exception is UNSUPPORTED_FOR_MESSAGE_FORMAT, * which indicates that the broker does not support the v1 message format. Partitions with this @@ -1003,14 +1003,14 @@ public void onSuccess(ClientResponse response, RequestFuture f * value of each partition may be null only for v0. In v1 and later the ListOffset API would not * return a null timestamp (-1 is returned instead when necessary). */ - private void handleListOffsetResponse(ListOffsetResponse listOffsetResponse, + private void handleListOffsetResponse(ListOffsetsResponse listOffsetsResponse, RequestFuture future) { Map fetchedOffsets = new HashMap<>(); Set partitionsToRetry = new HashSet<>(); Set unauthorizedTopics = new HashSet<>(); - for (ListOffsetTopicResponse topic : listOffsetResponse.topics()) { - for (ListOffsetPartitionResponse partition : topic.partitions()) { + for (ListOffsetsTopicResponse topic : listOffsetsResponse.topics()) { + for (ListOffsetsPartitionResponse partition : topic.partitions()) { TopicPartition topicPartition = new TopicPartition(topic.name(), partition.partitionIndex()); Errors error = Errors.forCode(partition.errorCode()); switch (error) { @@ -1027,7 +1027,7 @@ private void handleListOffsetResponse(ListOffsetResponse listOffsetResponse, } log.debug("Handling v0 ListOffsetResponse response for {}. Fetched offset {}", topicPartition, offset); - if (offset != ListOffsetResponse.UNKNOWN_OFFSET) { + if (offset != ListOffsetsResponse.UNKNOWN_OFFSET) { ListOffsetData offsetData = new ListOffsetData(offset, null, Optional.empty()); fetchedOffsets.put(topicPartition, offsetData); } @@ -1035,8 +1035,8 @@ private void handleListOffsetResponse(ListOffsetResponse listOffsetResponse, // Handle v1 and later response or v0 without offsets log.debug("Handling ListOffsetResponse response for {}. Fetched offset {}, timestamp {}", topicPartition, partition.offset(), partition.timestamp()); - if (partition.offset() != ListOffsetResponse.UNKNOWN_OFFSET) { - Optional leaderEpoch = (partition.leaderEpoch() == ListOffsetResponse.UNKNOWN_EPOCH) + if (partition.offset() != ListOffsetsResponse.UNKNOWN_OFFSET) { + Optional leaderEpoch = (partition.leaderEpoch() == ListOffsetsResponse.UNKNOWN_EPOCH) ? Optional.empty() : Optional.of(partition.leaderEpoch()); ListOffsetData offsetData = new ListOffsetData(partition.offset(), partition.timestamp(), diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java index 21e0b0ffe7d46..3fe016e5f0335 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java @@ -16,135 +16,16 @@ */ package org.apache.kafka.common.protocol; -import org.apache.kafka.common.message.AddOffsetsToTxnRequestData; -import org.apache.kafka.common.message.AddOffsetsToTxnResponseData; -import org.apache.kafka.common.message.AddPartitionsToTxnRequestData; -import org.apache.kafka.common.message.AddPartitionsToTxnResponseData; -import org.apache.kafka.common.message.AlterClientQuotasRequestData; -import org.apache.kafka.common.message.AlterClientQuotasResponseData; -import org.apache.kafka.common.message.AlterConfigsRequestData; -import org.apache.kafka.common.message.AlterConfigsResponseData; -import org.apache.kafka.common.message.AlterIsrRequestData; -import org.apache.kafka.common.message.AlterIsrResponseData; -import org.apache.kafka.common.message.AlterPartitionReassignmentsRequestData; -import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData; -import org.apache.kafka.common.message.AlterReplicaLogDirsRequestData; -import org.apache.kafka.common.message.AlterReplicaLogDirsResponseData; -import org.apache.kafka.common.message.AlterUserScramCredentialsRequestData; -import org.apache.kafka.common.message.AlterUserScramCredentialsResponseData; import org.apache.kafka.common.message.ApiMessageType; -import org.apache.kafka.common.message.ApiVersionsRequestData; -import org.apache.kafka.common.message.ApiVersionsResponseData; -import org.apache.kafka.common.message.BeginQuorumEpochRequestData; -import org.apache.kafka.common.message.BeginQuorumEpochResponseData; -import org.apache.kafka.common.message.ControlledShutdownRequestData; -import org.apache.kafka.common.message.ControlledShutdownResponseData; -import org.apache.kafka.common.message.CreateAclsRequestData; -import org.apache.kafka.common.message.CreateAclsResponseData; -import org.apache.kafka.common.message.CreateDelegationTokenRequestData; -import org.apache.kafka.common.message.CreateDelegationTokenResponseData; -import org.apache.kafka.common.message.CreatePartitionsRequestData; -import org.apache.kafka.common.message.CreatePartitionsResponseData; -import org.apache.kafka.common.message.CreateTopicsRequestData; -import org.apache.kafka.common.message.CreateTopicsResponseData; -import org.apache.kafka.common.message.DeleteAclsRequestData; -import org.apache.kafka.common.message.DeleteAclsResponseData; -import org.apache.kafka.common.message.DeleteGroupsRequestData; -import org.apache.kafka.common.message.DeleteGroupsResponseData; -import org.apache.kafka.common.message.DeleteRecordsRequestData; -import org.apache.kafka.common.message.DeleteRecordsResponseData; -import org.apache.kafka.common.message.DeleteTopicsRequestData; -import org.apache.kafka.common.message.DeleteTopicsResponseData; -import org.apache.kafka.common.message.DescribeAclsRequestData; -import org.apache.kafka.common.message.DescribeAclsResponseData; -import org.apache.kafka.common.message.DescribeClientQuotasRequestData; -import org.apache.kafka.common.message.DescribeClientQuotasResponseData; -import org.apache.kafka.common.message.DescribeConfigsRequestData; -import org.apache.kafka.common.message.DescribeConfigsResponseData; -import org.apache.kafka.common.message.DescribeDelegationTokenRequestData; -import org.apache.kafka.common.message.DescribeDelegationTokenResponseData; -import org.apache.kafka.common.message.DescribeGroupsRequestData; -import org.apache.kafka.common.message.DescribeGroupsResponseData; -import org.apache.kafka.common.message.DescribeLogDirsRequestData; -import org.apache.kafka.common.message.DescribeLogDirsResponseData; -import org.apache.kafka.common.message.DescribeQuorumRequestData; -import org.apache.kafka.common.message.DescribeQuorumResponseData; -import org.apache.kafka.common.message.DescribeUserScramCredentialsRequestData; -import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData; -import org.apache.kafka.common.message.ElectLeadersRequestData; -import org.apache.kafka.common.message.ElectLeadersResponseData; -import org.apache.kafka.common.message.EndQuorumEpochRequestData; -import org.apache.kafka.common.message.EndQuorumEpochResponseData; -import org.apache.kafka.common.message.EndTxnRequestData; -import org.apache.kafka.common.message.EndTxnResponseData; -import org.apache.kafka.common.message.EnvelopeRequestData; -import org.apache.kafka.common.message.EnvelopeResponseData; -import org.apache.kafka.common.message.ExpireDelegationTokenRequestData; -import org.apache.kafka.common.message.ExpireDelegationTokenResponseData; -import org.apache.kafka.common.message.FetchRequestData; -import org.apache.kafka.common.message.FetchResponseData; -import org.apache.kafka.common.message.FindCoordinatorRequestData; -import org.apache.kafka.common.message.FindCoordinatorResponseData; -import org.apache.kafka.common.message.HeartbeatRequestData; -import org.apache.kafka.common.message.HeartbeatResponseData; -import org.apache.kafka.common.message.IncrementalAlterConfigsRequestData; -import org.apache.kafka.common.message.IncrementalAlterConfigsResponseData; -import org.apache.kafka.common.message.InitProducerIdRequestData; -import org.apache.kafka.common.message.InitProducerIdResponseData; -import org.apache.kafka.common.message.JoinGroupRequestData; -import org.apache.kafka.common.message.JoinGroupResponseData; -import org.apache.kafka.common.message.LeaderAndIsrRequestData; -import org.apache.kafka.common.message.LeaderAndIsrResponseData; -import org.apache.kafka.common.message.LeaveGroupRequestData; -import org.apache.kafka.common.message.LeaveGroupResponseData; -import org.apache.kafka.common.message.ListGroupsRequestData; -import org.apache.kafka.common.message.ListGroupsResponseData; -import org.apache.kafka.common.message.ListOffsetRequestData; -import org.apache.kafka.common.message.ListOffsetResponseData; -import org.apache.kafka.common.message.ListPartitionReassignmentsRequestData; -import org.apache.kafka.common.message.ListPartitionReassignmentsResponseData; -import org.apache.kafka.common.message.MetadataRequestData; -import org.apache.kafka.common.message.MetadataResponseData; -import org.apache.kafka.common.message.OffsetCommitRequestData; -import org.apache.kafka.common.message.OffsetCommitResponseData; -import org.apache.kafka.common.message.OffsetDeleteRequestData; -import org.apache.kafka.common.message.OffsetDeleteResponseData; -import org.apache.kafka.common.message.OffsetFetchRequestData; -import org.apache.kafka.common.message.OffsetFetchResponseData; -import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData; -import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData; -import org.apache.kafka.common.message.ProduceRequestData; -import org.apache.kafka.common.message.ProduceResponseData; -import org.apache.kafka.common.message.RenewDelegationTokenRequestData; -import org.apache.kafka.common.message.RenewDelegationTokenResponseData; -import org.apache.kafka.common.message.SaslAuthenticateRequestData; -import org.apache.kafka.common.message.SaslAuthenticateResponseData; -import org.apache.kafka.common.message.SaslHandshakeRequestData; -import org.apache.kafka.common.message.SaslHandshakeResponseData; -import org.apache.kafka.common.message.StopReplicaRequestData; -import org.apache.kafka.common.message.StopReplicaResponseData; -import org.apache.kafka.common.message.SyncGroupRequestData; -import org.apache.kafka.common.message.SyncGroupResponseData; -import org.apache.kafka.common.message.TxnOffsetCommitRequestData; -import org.apache.kafka.common.message.TxnOffsetCommitResponseData; -import org.apache.kafka.common.message.UpdateFeaturesRequestData; -import org.apache.kafka.common.message.UpdateFeaturesResponseData; -import org.apache.kafka.common.message.UpdateMetadataRequestData; -import org.apache.kafka.common.message.UpdateMetadataResponseData; -import org.apache.kafka.common.message.VoteRequestData; -import org.apache.kafka.common.message.VoteResponseData; -import org.apache.kafka.common.message.WriteTxnMarkersRequestData; -import org.apache.kafka.common.message.WriteTxnMarkersResponseData; import org.apache.kafka.common.protocol.types.Schema; -import org.apache.kafka.common.protocol.types.SchemaException; -import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.protocol.types.Type; import org.apache.kafka.common.record.RecordBatch; -import java.nio.ByteBuffer; import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Function; import java.util.stream.Collectors; import static org.apache.kafka.common.protocol.types.Type.BYTES; @@ -157,124 +38,74 @@ * Identifiers for all the Kafka APIs */ public enum ApiKeys { - PRODUCE(0, "Produce", ProduceRequestData.SCHEMAS, ProduceResponseData.SCHEMAS), - FETCH(1, "Fetch", FetchRequestData.SCHEMAS, FetchResponseData.SCHEMAS), - LIST_OFFSETS(2, "ListOffsets", ListOffsetRequestData.SCHEMAS, ListOffsetResponseData.SCHEMAS), - METADATA(3, "Metadata", MetadataRequestData.SCHEMAS, MetadataResponseData.SCHEMAS), - LEADER_AND_ISR(4, "LeaderAndIsr", true, LeaderAndIsrRequestData.SCHEMAS, LeaderAndIsrResponseData.SCHEMAS), - STOP_REPLICA(5, "StopReplica", true, StopReplicaRequestData.SCHEMAS, StopReplicaResponseData.SCHEMAS), - UPDATE_METADATA(6, "UpdateMetadata", true, UpdateMetadataRequestData.SCHEMAS, UpdateMetadataResponseData.SCHEMAS), - CONTROLLED_SHUTDOWN(7, "ControlledShutdown", true, ControlledShutdownRequestData.SCHEMAS, - ControlledShutdownResponseData.SCHEMAS), - OFFSET_COMMIT(8, "OffsetCommit", OffsetCommitRequestData.SCHEMAS, OffsetCommitResponseData.SCHEMAS), - OFFSET_FETCH(9, "OffsetFetch", OffsetFetchRequestData.SCHEMAS, OffsetFetchResponseData.SCHEMAS), - FIND_COORDINATOR(10, "FindCoordinator", FindCoordinatorRequestData.SCHEMAS, - FindCoordinatorResponseData.SCHEMAS), - JOIN_GROUP(11, "JoinGroup", JoinGroupRequestData.SCHEMAS, JoinGroupResponseData.SCHEMAS), - HEARTBEAT(12, "Heartbeat", HeartbeatRequestData.SCHEMAS, HeartbeatResponseData.SCHEMAS), - LEAVE_GROUP(13, "LeaveGroup", LeaveGroupRequestData.SCHEMAS, LeaveGroupResponseData.SCHEMAS), - SYNC_GROUP(14, "SyncGroup", SyncGroupRequestData.SCHEMAS, SyncGroupResponseData.SCHEMAS), - DESCRIBE_GROUPS(15, "DescribeGroups", DescribeGroupsRequestData.SCHEMAS, - DescribeGroupsResponseData.SCHEMAS), - LIST_GROUPS(16, "ListGroups", ListGroupsRequestData.SCHEMAS, ListGroupsResponseData.SCHEMAS), - SASL_HANDSHAKE(17, "SaslHandshake", SaslHandshakeRequestData.SCHEMAS, SaslHandshakeResponseData.SCHEMAS), - API_VERSIONS(18, "ApiVersions", ApiVersionsRequestData.SCHEMAS, ApiVersionsResponseData.SCHEMAS) { - @Override - public Struct parseResponse(short version, ByteBuffer buffer) { - // Fallback to version 0 for ApiVersions response. If a client sends an ApiVersionsRequest - // using a version higher than that supported by the broker, a version 0 response is sent - // to the client indicating UNSUPPORTED_VERSION. - return parseResponse(version, buffer, (short) 0); - } - }, - CREATE_TOPICS(19, "CreateTopics", CreateTopicsRequestData.SCHEMAS, CreateTopicsResponseData.SCHEMAS, true), - DELETE_TOPICS(20, "DeleteTopics", DeleteTopicsRequestData.SCHEMAS, DeleteTopicsResponseData.SCHEMAS, true), - DELETE_RECORDS(21, "DeleteRecords", DeleteRecordsRequestData.SCHEMAS, DeleteRecordsResponseData.SCHEMAS), - INIT_PRODUCER_ID(22, "InitProducerId", InitProducerIdRequestData.SCHEMAS, InitProducerIdResponseData.SCHEMAS), - OFFSET_FOR_LEADER_EPOCH(23, "OffsetForLeaderEpoch", false, OffsetForLeaderEpochRequestData.SCHEMAS, - OffsetForLeaderEpochResponseData.SCHEMAS), - ADD_PARTITIONS_TO_TXN(24, "AddPartitionsToTxn", false, RecordBatch.MAGIC_VALUE_V2, - AddPartitionsToTxnRequestData.SCHEMAS, AddPartitionsToTxnResponseData.SCHEMAS), - ADD_OFFSETS_TO_TXN(25, "AddOffsetsToTxn", false, RecordBatch.MAGIC_VALUE_V2, AddOffsetsToTxnRequestData.SCHEMAS, - AddOffsetsToTxnResponseData.SCHEMAS), - END_TXN(26, "EndTxn", false, RecordBatch.MAGIC_VALUE_V2, EndTxnRequestData.SCHEMAS, EndTxnResponseData.SCHEMAS), - WRITE_TXN_MARKERS(27, "WriteTxnMarkers", true, RecordBatch.MAGIC_VALUE_V2, WriteTxnMarkersRequestData.SCHEMAS, - WriteTxnMarkersResponseData.SCHEMAS), - TXN_OFFSET_COMMIT(28, "TxnOffsetCommit", false, RecordBatch.MAGIC_VALUE_V2, TxnOffsetCommitRequestData.SCHEMAS, - TxnOffsetCommitResponseData.SCHEMAS), - DESCRIBE_ACLS(29, "DescribeAcls", DescribeAclsRequestData.SCHEMAS, DescribeAclsResponseData.SCHEMAS), - CREATE_ACLS(30, "CreateAcls", CreateAclsRequestData.SCHEMAS, CreateAclsResponseData.SCHEMAS, true), - DELETE_ACLS(31, "DeleteAcls", DeleteAclsRequestData.SCHEMAS, DeleteAclsResponseData.SCHEMAS, true), - DESCRIBE_CONFIGS(32, "DescribeConfigs", DescribeConfigsRequestData.SCHEMAS, - DescribeConfigsResponseData.SCHEMAS), - ALTER_CONFIGS(33, "AlterConfigs", AlterConfigsRequestData.SCHEMAS, - AlterConfigsResponseData.SCHEMAS, true), - ALTER_REPLICA_LOG_DIRS(34, "AlterReplicaLogDirs", AlterReplicaLogDirsRequestData.SCHEMAS, - AlterReplicaLogDirsResponseData.SCHEMAS), - DESCRIBE_LOG_DIRS(35, "DescribeLogDirs", DescribeLogDirsRequestData.SCHEMAS, - DescribeLogDirsResponseData.SCHEMAS), - SASL_AUTHENTICATE(36, "SaslAuthenticate", SaslAuthenticateRequestData.SCHEMAS, - SaslAuthenticateResponseData.SCHEMAS), - CREATE_PARTITIONS(37, "CreatePartitions", CreatePartitionsRequestData.SCHEMAS, - CreatePartitionsResponseData.SCHEMAS, true), - CREATE_DELEGATION_TOKEN(38, "CreateDelegationToken", CreateDelegationTokenRequestData.SCHEMAS, - CreateDelegationTokenResponseData.SCHEMAS, true), - RENEW_DELEGATION_TOKEN(39, "RenewDelegationToken", RenewDelegationTokenRequestData.SCHEMAS, - RenewDelegationTokenResponseData.SCHEMAS, true), - EXPIRE_DELEGATION_TOKEN(40, "ExpireDelegationToken", ExpireDelegationTokenRequestData.SCHEMAS, - ExpireDelegationTokenResponseData.SCHEMAS, true), - DESCRIBE_DELEGATION_TOKEN(41, "DescribeDelegationToken", DescribeDelegationTokenRequestData.SCHEMAS, - DescribeDelegationTokenResponseData.SCHEMAS), - DELETE_GROUPS(42, "DeleteGroups", DeleteGroupsRequestData.SCHEMAS, DeleteGroupsResponseData.SCHEMAS), - ELECT_LEADERS(43, "ElectLeaders", ElectLeadersRequestData.SCHEMAS, - ElectLeadersResponseData.SCHEMAS), - INCREMENTAL_ALTER_CONFIGS(44, "IncrementalAlterConfigs", IncrementalAlterConfigsRequestData.SCHEMAS, - IncrementalAlterConfigsResponseData.SCHEMAS, true), - ALTER_PARTITION_REASSIGNMENTS(45, "AlterPartitionReassignments", AlterPartitionReassignmentsRequestData.SCHEMAS, - AlterPartitionReassignmentsResponseData.SCHEMAS, true), - LIST_PARTITION_REASSIGNMENTS(46, "ListPartitionReassignments", ListPartitionReassignmentsRequestData.SCHEMAS, - ListPartitionReassignmentsResponseData.SCHEMAS), - OFFSET_DELETE(47, "OffsetDelete", OffsetDeleteRequestData.SCHEMAS, OffsetDeleteResponseData.SCHEMAS), - DESCRIBE_CLIENT_QUOTAS(48, "DescribeClientQuotas", DescribeClientQuotasRequestData.SCHEMAS, - DescribeClientQuotasResponseData.SCHEMAS), - ALTER_CLIENT_QUOTAS(49, "AlterClientQuotas", AlterClientQuotasRequestData.SCHEMAS, - AlterClientQuotasResponseData.SCHEMAS, true), - DESCRIBE_USER_SCRAM_CREDENTIALS(50, "DescribeUserScramCredentials", DescribeUserScramCredentialsRequestData.SCHEMAS, - DescribeUserScramCredentialsResponseData.SCHEMAS), - ALTER_USER_SCRAM_CREDENTIALS(51, "AlterUserScramCredentials", AlterUserScramCredentialsRequestData.SCHEMAS, - AlterUserScramCredentialsResponseData.SCHEMAS, true), - VOTE(52, "Vote", true, false, - VoteRequestData.SCHEMAS, VoteResponseData.SCHEMAS), - BEGIN_QUORUM_EPOCH(53, "BeginQuorumEpoch", true, false, - BeginQuorumEpochRequestData.SCHEMAS, BeginQuorumEpochResponseData.SCHEMAS), - END_QUORUM_EPOCH(54, "EndQuorumEpoch", true, false, - EndQuorumEpochRequestData.SCHEMAS, EndQuorumEpochResponseData.SCHEMAS), - DESCRIBE_QUORUM(55, "DescribeQuorum", true, false, - DescribeQuorumRequestData.SCHEMAS, DescribeQuorumResponseData.SCHEMAS), - ALTER_ISR(56, "AlterIsr", true, AlterIsrRequestData.SCHEMAS, AlterIsrResponseData.SCHEMAS), - UPDATE_FEATURES(57, "UpdateFeatures", - UpdateFeaturesRequestData.SCHEMAS, UpdateFeaturesResponseData.SCHEMAS, true), - ENVELOPE(58, "Envelope", true, false, EnvelopeRequestData.SCHEMAS, EnvelopeResponseData.SCHEMAS); - - private static final ApiKeys[] ID_TO_TYPE; - private static final int MIN_API_KEY = 0; - public static final int MAX_API_KEY; - - static { - int maxKey = -1; - for (ApiKeys key : ApiKeys.values()) - maxKey = Math.max(maxKey, key.id); - ApiKeys[] idToType = new ApiKeys[maxKey + 1]; - for (ApiKeys key : ApiKeys.values()) - idToType[key.id] = key; - ID_TO_TYPE = idToType; - MAX_API_KEY = maxKey; - } - - /** the permanent and immutable id of an API--this can't change ever */ + PRODUCE(ApiMessageType.PRODUCE), + FETCH(ApiMessageType.FETCH), + LIST_OFFSETS(ApiMessageType.LIST_OFFSETS), + METADATA(ApiMessageType.METADATA), + LEADER_AND_ISR(ApiMessageType.LEADER_AND_ISR, true), + STOP_REPLICA(ApiMessageType.STOP_REPLICA, true), + UPDATE_METADATA(ApiMessageType.UPDATE_METADATA, true), + CONTROLLED_SHUTDOWN(ApiMessageType.CONTROLLED_SHUTDOWN, true), + OFFSET_COMMIT(ApiMessageType.OFFSET_COMMIT), + OFFSET_FETCH(ApiMessageType.OFFSET_FETCH), + FIND_COORDINATOR(ApiMessageType.FIND_COORDINATOR), + JOIN_GROUP(ApiMessageType.JOIN_GROUP), + HEARTBEAT(ApiMessageType.HEARTBEAT), + LEAVE_GROUP(ApiMessageType.LEAVE_GROUP), + SYNC_GROUP(ApiMessageType.SYNC_GROUP), + DESCRIBE_GROUPS(ApiMessageType.DESCRIBE_GROUPS), + LIST_GROUPS(ApiMessageType.LIST_GROUPS), + SASL_HANDSHAKE(ApiMessageType.SASL_HANDSHAKE), + API_VERSIONS(ApiMessageType.API_VERSIONS), + CREATE_TOPICS(ApiMessageType.CREATE_TOPICS, false, true), + DELETE_TOPICS(ApiMessageType.DELETE_TOPICS, false, true), + DELETE_RECORDS(ApiMessageType.DELETE_RECORDS), + INIT_PRODUCER_ID(ApiMessageType.INIT_PRODUCER_ID), + OFFSET_FOR_LEADER_EPOCH(ApiMessageType.OFFSET_FOR_LEADER_EPOCH), + ADD_PARTITIONS_TO_TXN(ApiMessageType.ADD_PARTITIONS_TO_TXN, false, RecordBatch.MAGIC_VALUE_V2, false), + ADD_OFFSETS_TO_TXN(ApiMessageType.ADD_OFFSETS_TO_TXN, false, RecordBatch.MAGIC_VALUE_V2, false), + END_TXN(ApiMessageType.END_TXN, false, RecordBatch.MAGIC_VALUE_V2, false), + WRITE_TXN_MARKERS(ApiMessageType.WRITE_TXN_MARKERS, true, RecordBatch.MAGIC_VALUE_V2, false), + TXN_OFFSET_COMMIT(ApiMessageType.TXN_OFFSET_COMMIT, false, RecordBatch.MAGIC_VALUE_V2, false), + DESCRIBE_ACLS(ApiMessageType.DESCRIBE_ACLS), + CREATE_ACLS(ApiMessageType.CREATE_ACLS, false, true), + DELETE_ACLS(ApiMessageType.DELETE_ACLS, false, true), + DESCRIBE_CONFIGS(ApiMessageType.DESCRIBE_CONFIGS), + ALTER_CONFIGS(ApiMessageType.ALTER_CONFIGS, false, true), + ALTER_REPLICA_LOG_DIRS(ApiMessageType.ALTER_REPLICA_LOG_DIRS), + DESCRIBE_LOG_DIRS(ApiMessageType.DESCRIBE_LOG_DIRS), + SASL_AUTHENTICATE(ApiMessageType.SASL_AUTHENTICATE), + CREATE_PARTITIONS(ApiMessageType.CREATE_PARTITIONS, false, true), + CREATE_DELEGATION_TOKEN(ApiMessageType.CREATE_DELEGATION_TOKEN, false, true), + RENEW_DELEGATION_TOKEN(ApiMessageType.RENEW_DELEGATION_TOKEN, false, true), + EXPIRE_DELEGATION_TOKEN(ApiMessageType.EXPIRE_DELEGATION_TOKEN, false, true), + DESCRIBE_DELEGATION_TOKEN(ApiMessageType.DESCRIBE_DELEGATION_TOKEN), + DELETE_GROUPS(ApiMessageType.DELETE_GROUPS), + ELECT_LEADERS(ApiMessageType.ELECT_LEADERS), + INCREMENTAL_ALTER_CONFIGS(ApiMessageType.INCREMENTAL_ALTER_CONFIGS, false, true), + ALTER_PARTITION_REASSIGNMENTS(ApiMessageType.ALTER_PARTITION_REASSIGNMENTS, false, true), + LIST_PARTITION_REASSIGNMENTS(ApiMessageType.LIST_PARTITION_REASSIGNMENTS), + OFFSET_DELETE(ApiMessageType.OFFSET_DELETE), + DESCRIBE_CLIENT_QUOTAS(ApiMessageType.DESCRIBE_CLIENT_QUOTAS), + ALTER_CLIENT_QUOTAS(ApiMessageType.ALTER_CLIENT_QUOTAS, false, true), + DESCRIBE_USER_SCRAM_CREDENTIALS(ApiMessageType.DESCRIBE_USER_SCRAM_CREDENTIALS), + ALTER_USER_SCRAM_CREDENTIALS(ApiMessageType.ALTER_USER_SCRAM_CREDENTIALS, false, true), + VOTE(ApiMessageType.VOTE, true, RecordBatch.MAGIC_VALUE_V0, false, false), + BEGIN_QUORUM_EPOCH(ApiMessageType.BEGIN_QUORUM_EPOCH, true, RecordBatch.MAGIC_VALUE_V0, false, false), + END_QUORUM_EPOCH(ApiMessageType.END_QUORUM_EPOCH, true, RecordBatch.MAGIC_VALUE_V0, false, false), + DESCRIBE_QUORUM(ApiMessageType.DESCRIBE_QUORUM, true, RecordBatch.MAGIC_VALUE_V0, false, false), + ALTER_ISR(ApiMessageType.ALTER_ISR, true), + UPDATE_FEATURES(ApiMessageType.UPDATE_FEATURES, false, true), + ENVELOPE(ApiMessageType.ENVELOPE, true, RecordBatch.MAGIC_VALUE_V0, false, false); + + // The generator ensures every `ApiMessageType` has a unique id + private static final Map ID_TO_TYPE = Arrays.stream(ApiKeys.values()) + .collect(Collectors.toMap(key -> (int) key.id, Function.identity())); + + /** the permanent and immutable id of an API - this can't change ever */ public final short id; - /** an english description of the api--this is for debugging and can change */ + /** An english description of the api - used for debugging and metric names, it can potentially be changed via a KIP */ public final String name; /** indicates if this is a ClusterAction request used only by brokers */ @@ -289,63 +120,41 @@ public Struct parseResponse(short version, ByteBuffer buffer) { /** indicates whether the API is enabled for forwarding **/ public final boolean forwardable; - public final Schema[] requestSchemas; - public final Schema[] responseSchemas; public final boolean requiresDelayedAllocation; - ApiKeys(int id, String name, Schema[] requestSchemas, Schema[] responseSchemas) { - this(id, name, false, requestSchemas, responseSchemas); - } + public final ApiMessageType messageType; - ApiKeys(int id, String name, boolean clusterAction, Schema[] requestSchemas, Schema[] responseSchemas) { - this(id, name, clusterAction, RecordBatch.MAGIC_VALUE_V0, requestSchemas, responseSchemas); + ApiKeys(ApiMessageType messageType) { + this(messageType, false); } - ApiKeys(int id, String name, Schema[] requestSchemas, Schema[] responseSchemas, boolean forwardable) { - this(id, name, false, RecordBatch.MAGIC_VALUE_V0, true, requestSchemas, responseSchemas, forwardable); + ApiKeys(ApiMessageType messageType, boolean clusterAction) { + this(messageType, clusterAction, RecordBatch.MAGIC_VALUE_V0, false); } - ApiKeys(int id, String name, boolean clusterAction, boolean isEnabled, Schema[] requestSchemas, Schema[] responseSchemas) { - this(id, name, clusterAction, RecordBatch.MAGIC_VALUE_V0, isEnabled, requestSchemas, responseSchemas, false); + ApiKeys(ApiMessageType messageType, boolean clusterAction, boolean forwardable) { + this(messageType, clusterAction, RecordBatch.MAGIC_VALUE_V0, forwardable); } - ApiKeys(int id, String name, boolean clusterAction, byte minRequiredInterBrokerMagic, - Schema[] requestSchemas, Schema[] responseSchemas) { - this(id, name, clusterAction, minRequiredInterBrokerMagic, true, requestSchemas, responseSchemas, false); + ApiKeys(ApiMessageType messageType, boolean clusterAction, byte minRequiredInterBrokerMagic, boolean forwardable) { + this(messageType, clusterAction, minRequiredInterBrokerMagic, forwardable, true); } ApiKeys( - int id, - String name, + ApiMessageType messageType, boolean clusterAction, byte minRequiredInterBrokerMagic, - boolean isEnabled, - Schema[] requestSchemas, - Schema[] responseSchemas, - boolean forwardable + boolean forwardable, + boolean isEnabled ) { - if (id < 0) - throw new IllegalArgumentException("id must not be negative, id: " + id); - this.id = (short) id; - this.name = name; + this.messageType = messageType; + this.id = messageType.apiKey(); + this.name = messageType.name; this.clusterAction = clusterAction; this.minRequiredInterBrokerMagic = minRequiredInterBrokerMagic; this.isEnabled = isEnabled; - if (requestSchemas.length != responseSchemas.length) - throw new IllegalStateException(requestSchemas.length + " request versions for api " + name - + " but " + responseSchemas.length + " response versions."); - - for (int i = 0; i < requestSchemas.length; ++i) { - if (requestSchemas[i] == null) - throw new IllegalStateException("Request schema for api " + name + " for version " + i + " is null"); - if (responseSchemas[i] == null) - throw new IllegalStateException("Response schema for api " + name + " for version " + i + " is null"); - } - - this.requiresDelayedAllocation = forwardable || shouldRetainsBufferReference(requestSchemas); - this.requestSchemas = requestSchemas; - this.responseSchemas = responseSchemas; + this.requiresDelayedAllocation = forwardable || shouldRetainsBufferReference(messageType.requestSchemas()); this.forwardable = forwardable; } @@ -361,57 +170,23 @@ private static boolean shouldRetainsBufferReference(Schema[] requestSchemas) { } public static ApiKeys forId(int id) { - if (!hasId(id)) - throw new IllegalArgumentException(String.format("Unexpected ApiKeys id `%s`, it should be between `%s` " + - "and `%s` (inclusive)", id, MIN_API_KEY, MAX_API_KEY)); - return ID_TO_TYPE[id]; + ApiKeys apiKey = ID_TO_TYPE.get(id); + if (apiKey == null) { + throw new IllegalArgumentException("Unexpected api key: " + id); + } + return apiKey; } public static boolean hasId(int id) { - return id >= MIN_API_KEY && id <= MAX_API_KEY; + return ID_TO_TYPE.containsKey(id); } public short latestVersion() { - return (short) (requestSchemas.length - 1); + return messageType.highestSupportedVersion(); } public short oldestVersion() { - return 0; - } - - public Schema requestSchema(short version) { - return schemaFor(requestSchemas, version); - } - - public Schema responseSchema(short version) { - return schemaFor(responseSchemas, version); - } - - public Struct parseRequest(short version, ByteBuffer buffer) { - return requestSchema(version).read(buffer); - } - - public Struct parseResponse(short version, ByteBuffer buffer) { - return responseSchema(version).read(buffer); - } - - protected Struct parseResponse(short version, ByteBuffer buffer, short fallbackVersion) { - int bufferPosition = buffer.position(); - try { - return responseSchema(version).read(buffer); - } catch (SchemaException e) { - if (version != fallbackVersion) { - buffer.position(bufferPosition); - return responseSchema(fallbackVersion).read(buffer); - } else - throw e; - } - } - - private Schema schemaFor(Schema[] versions, short version) { - if (!isVersionSupported(version)) - throw new IllegalArgumentException("Invalid version for API key " + this + ": " + version); - return versions[version]; + return messageType.lowestSupportedVersion(); } public boolean isVersionSupported(short apiVersion) { @@ -419,11 +194,11 @@ public boolean isVersionSupported(short apiVersion) { } public short requestHeaderVersion(short apiVersion) { - return ApiMessageType.fromApiKey(id).requestHeaderVersion(apiVersion); + return messageType.requestHeaderVersion(apiVersion); } public short responseHeaderVersion(short apiVersion) { - return ApiMessageType.fromApiKey(id).responseHeaderVersion(apiVersion); + return messageType.responseHeaderVersion(apiVersion); } private static String toHtml() { diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/CommonFields.java b/clients/src/main/java/org/apache/kafka/common/protocol/CommonFields.java deleted file mode 100644 index 122d4da602324..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/protocol/CommonFields.java +++ /dev/null @@ -1,35 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.common.protocol; - -import org.apache.kafka.common.protocol.types.Field; - -public class CommonFields { - public static final Field.Int32 THROTTLE_TIME_MS = new Field.Int32("throttle_time_ms", - "Duration in milliseconds for which the request was throttled due to quota violation (Zero if the " + - "request did not violate any quota)", 0); - public static final Field.Str TOPIC_NAME = new Field.Str("topic", "Name of topic"); - public static final Field.Int32 PARTITION_ID = new Field.Int32("partition", "Topic partition id"); - public static final Field.Int16 ERROR_CODE = new Field.Int16("error_code", "Response error code"); - public static final Field.NullableStr ERROR_MESSAGE = new Field.NullableStr("error_message", "Response error message"); - public static final Field.Int32 LEADER_EPOCH = new Field.Int32("leader_epoch", "The leader epoch"); - public static final Field.Int32 CURRENT_LEADER_EPOCH = new Field.Int32("current_leader_epoch", - "The current leader epoch, if provided, is used to fence consumers/replicas with old metadata. " + - "If the epoch provided by the client is larger than the current epoch known to the broker, then " + - "the UNKNOWN_LEADER_EPOCH error code will be returned. If the provided epoch is smaller, then " + - "the FENCED_LEADER_EPOCH error code will be returned."); -} diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java index 5332369f0f9be..a0d566929aebf 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java @@ -133,6 +133,8 @@ * the client if the request version suggests that the client may not recognize the new error code. * * Do not add exceptions that occur only on the client or only on the server here. + * + * @see org.apache.kafka.common.network.SslTransportLayer */ public enum Errors { UNKNOWN_SERVER_ERROR(-1, "The server experienced an unexpected error when processing the request.", diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java index 6e7d49380f9af..0d573dbde4763 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java @@ -143,7 +143,7 @@ public static String toHtml() { b.append("):\n\n"); // Requests b.append("Requests:
\n"); - Schema[] requests = key.requestSchemas; + Schema[] requests = key.messageType.requestSchemas(); for (int i = 0; i < requests.length; i++) { Schema schema = requests[i]; // Schema @@ -164,7 +164,7 @@ public static String toHtml() { // Responses b.append("Responses:
\n"); - Schema[] responses = key.responseSchemas; + Schema[] responses = key.messageType.responseSchemas(); for (int i = 0; i < responses.length; i++) { Schema schema = responses[i]; // Schema 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 28988895c074d..6df68cfd23d93 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 @@ -162,7 +162,7 @@ private static AbstractRequest doParseRequest(ApiKeys apiKey, short apiVersion, case FETCH: return FetchRequest.parse(buffer, apiVersion); case LIST_OFFSETS: - return ListOffsetRequest.parse(buffer, apiVersion); + return ListOffsetsRequest.parse(buffer, apiVersion); case METADATA: return MetadataRequest.parse(buffer, apiVersion); case OFFSET_COMMIT: diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java index 617ddbb1c132d..466ed9198b0ae 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java @@ -119,7 +119,7 @@ public static AbstractResponse parseResponse(ApiKeys apiKey, ByteBuffer response case FETCH: return FetchResponse.parse(responseBuffer, version); case LIST_OFFSETS: - return ListOffsetResponse.parse(responseBuffer, version); + return ListOffsetsResponse.parse(responseBuffer, version); case METADATA: return MetadataResponse.parse(responseBuffer, version); case OFFSET_COMMIT: diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsRequest.java similarity index 64% rename from clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java rename to clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsRequest.java index 9ebb06bdd6511..658188da7e5dc 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsRequest.java @@ -27,28 +27,28 @@ import org.apache.kafka.common.IsolationLevel; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.message.ListOffsetRequestData; -import org.apache.kafka.common.message.ListOffsetRequestData.ListOffsetPartition; -import org.apache.kafka.common.message.ListOffsetRequestData.ListOffsetTopic; -import org.apache.kafka.common.message.ListOffsetResponseData; -import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetPartitionResponse; -import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetTopicResponse; +import org.apache.kafka.common.message.ListOffsetsRequestData; +import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition; +import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsTopic; +import org.apache.kafka.common.message.ListOffsetsResponseData; +import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse; +import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; -public class ListOffsetRequest extends AbstractRequest { +public class ListOffsetsRequest extends AbstractRequest { public static final long EARLIEST_TIMESTAMP = -2L; public static final long LATEST_TIMESTAMP = -1L; public static final int CONSUMER_REPLICA_ID = -1; public static final int DEBUGGING_REPLICA_ID = -2; - private final ListOffsetRequestData data; + private final ListOffsetsRequestData data; private final Set duplicatePartitions; - public static class Builder extends AbstractRequest.Builder { - private final ListOffsetRequestData data; + public static class Builder extends AbstractRequest.Builder { + private final ListOffsetsRequestData data; public static Builder forReplica(short allowedVersion, int replicaId) { return new Builder((short) 0, allowedVersion, replicaId, IsolationLevel.READ_UNCOMMITTED); @@ -68,19 +68,19 @@ private Builder(short oldestAllowedVersion, int replicaId, IsolationLevel isolationLevel) { super(ApiKeys.LIST_OFFSETS, oldestAllowedVersion, latestAllowedVersion); - data = new ListOffsetRequestData() + data = new ListOffsetsRequestData() .setIsolationLevel(isolationLevel.id()) .setReplicaId(replicaId); } - public Builder setTargetTimes(List topics) { + public Builder setTargetTimes(List topics) { data.setTopics(topics); return this; } @Override - public ListOffsetRequest build(short version) { - return new ListOffsetRequest(data, version); + public ListOffsetsRequest build(short version) { + return new ListOffsetsRequest(data, version); } @Override @@ -92,13 +92,13 @@ public String toString() { /** * Private constructor with a specified version. */ - private ListOffsetRequest(ListOffsetRequestData data, short version) { + private ListOffsetsRequest(ListOffsetsRequestData data, short version) { super(ApiKeys.LIST_OFFSETS, version); this.data = data; duplicatePartitions = new HashSet<>(); Set partitions = new HashSet<>(); - for (ListOffsetTopic topic : data.topics()) { - for (ListOffsetPartition partition : topic.partitions()) { + for (ListOffsetsTopic topic : data.topics()) { + for (ListOffsetsPartition partition : topic.partitions()) { TopicPartition tp = new TopicPartition(topic.name(), partition.partitionIndex()); if (!partitions.add(tp)) { duplicatePartitions.add(tp); @@ -112,33 +112,33 @@ public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { short versionId = version(); short errorCode = Errors.forException(e).code(); - List responses = new ArrayList<>(); - for (ListOffsetTopic topic : data.topics()) { - ListOffsetTopicResponse topicResponse = new ListOffsetTopicResponse().setName(topic.name()); - List partitions = new ArrayList<>(); - for (ListOffsetPartition partition : topic.partitions()) { - ListOffsetPartitionResponse partitionResponse = new ListOffsetPartitionResponse() + List responses = new ArrayList<>(); + for (ListOffsetsTopic topic : data.topics()) { + ListOffsetsTopicResponse topicResponse = new ListOffsetsTopicResponse().setName(topic.name()); + List partitions = new ArrayList<>(); + for (ListOffsetsPartition partition : topic.partitions()) { + ListOffsetsPartitionResponse partitionResponse = new ListOffsetsPartitionResponse() .setErrorCode(errorCode) .setPartitionIndex(partition.partitionIndex()); if (versionId == 0) { partitionResponse.setOldStyleOffsets(Collections.emptyList()); } else { - partitionResponse.setOffset(ListOffsetResponse.UNKNOWN_OFFSET) - .setTimestamp(ListOffsetResponse.UNKNOWN_TIMESTAMP); + partitionResponse.setOffset(ListOffsetsResponse.UNKNOWN_OFFSET) + .setTimestamp(ListOffsetsResponse.UNKNOWN_TIMESTAMP); } partitions.add(partitionResponse); } topicResponse.setPartitions(partitions); responses.add(topicResponse); } - ListOffsetResponseData responseData = new ListOffsetResponseData() + ListOffsetsResponseData responseData = new ListOffsetsResponseData() .setThrottleTimeMs(throttleTimeMs) .setTopics(responses); - return new ListOffsetResponse(responseData); + return new ListOffsetsResponse(responseData); } @Override - public ListOffsetRequestData data() { + public ListOffsetsRequestData data() { return data; } @@ -150,7 +150,7 @@ public IsolationLevel isolationLevel() { return IsolationLevel.forId(data.isolationLevel()); } - public List topics() { + public List topics() { return data.topics(); } @@ -158,24 +158,24 @@ public Set duplicatePartitions() { return duplicatePartitions; } - public static ListOffsetRequest parse(ByteBuffer buffer, short version) { - return new ListOffsetRequest(new ListOffsetRequestData(new ByteBufferAccessor(buffer), version), version); + public static ListOffsetsRequest parse(ByteBuffer buffer, short version) { + return new ListOffsetsRequest(new ListOffsetsRequestData(new ByteBufferAccessor(buffer), version), version); } - public static List toListOffsetTopics(Map timestampsToSearch) { - Map topics = new HashMap<>(); - for (Map.Entry entry : timestampsToSearch.entrySet()) { + public static List toListOffsetsTopics(Map timestampsToSearch) { + Map topics = new HashMap<>(); + for (Map.Entry entry : timestampsToSearch.entrySet()) { TopicPartition tp = entry.getKey(); - ListOffsetTopic topic = topics.computeIfAbsent(tp.topic(), k -> new ListOffsetTopic().setName(tp.topic())); + ListOffsetsTopic topic = topics.computeIfAbsent(tp.topic(), k -> new ListOffsetsTopic().setName(tp.topic())); topic.partitions().add(entry.getValue()); } return new ArrayList<>(topics.values()); } - public static ListOffsetTopic singletonRequestData(String topic, int partitionIndex, long timestamp, int maxNumOffsets) { - return new ListOffsetTopic() + public static ListOffsetsTopic singletonRequestData(String topic, int partitionIndex, long timestamp, int maxNumOffsets) { + return new ListOffsetsTopic() .setName(topic) - .setPartitions(Collections.singletonList(new ListOffsetPartition() + .setPartitions(Collections.singletonList(new ListOffsetsPartition() .setPartitionIndex(partitionIndex) .setTimestamp(timestamp) .setMaxNumOffsets(maxNumOffsets))); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsResponse.java similarity index 80% rename from clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java rename to clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsResponse.java index ca57382d9c0ca..8c4a51b542b47 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsResponse.java @@ -23,9 +23,9 @@ import java.util.Map; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.message.ListOffsetResponseData; -import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetPartitionResponse; -import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetTopicResponse; +import org.apache.kafka.common.message.ListOffsetsResponseData; +import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse; +import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; @@ -47,14 +47,14 @@ * - {@link Errors#LEADER_NOT_AVAILABLE} The leader's HW has not caught up after recent election (v4 protocol) * - {@link Errors#OFFSET_NOT_AVAILABLE} The leader's HW has not caught up after recent election (v5+ protocol) */ -public class ListOffsetResponse extends AbstractResponse { +public class ListOffsetsResponse extends AbstractResponse { public static final long UNKNOWN_TIMESTAMP = -1L; public static final long UNKNOWN_OFFSET = -1L; public static final int UNKNOWN_EPOCH = RecordBatch.NO_PARTITION_LEADER_EPOCH; - private final ListOffsetResponseData data; + private final ListOffsetsResponseData data; - public ListOffsetResponse(ListOffsetResponseData data) { + public ListOffsetsResponse(ListOffsetsResponseData data) { super(ApiKeys.LIST_OFFSETS); this.data = data; } @@ -65,11 +65,11 @@ public int throttleTimeMs() { } @Override - public ListOffsetResponseData data() { + public ListOffsetsResponseData data() { return data; } - public List topics() { + public List topics() { return data.topics(); } @@ -84,8 +84,8 @@ public Map errorCounts() { return errorCounts; } - public static ListOffsetResponse parse(ByteBuffer buffer, short version) { - return new ListOffsetResponse(new ListOffsetResponseData(new ByteBufferAccessor(buffer), version)); + public static ListOffsetsResponse parse(ByteBuffer buffer, short version) { + return new ListOffsetsResponse(new ListOffsetsResponseData(new ByteBufferAccessor(buffer), version)); } @Override @@ -98,10 +98,10 @@ public boolean shouldClientThrottle(short version) { return version >= 3; } - public static ListOffsetTopicResponse singletonListOffsetTopicResponse(TopicPartition tp, Errors error, long timestamp, long offset, int epoch) { - return new ListOffsetTopicResponse() + public static ListOffsetsTopicResponse singletonListOffsetsTopicResponse(TopicPartition tp, Errors error, long timestamp, long offset, int epoch) { + return new ListOffsetsTopicResponse() .setName(tp.topic()) - .setPartitions(Collections.singletonList(new ListOffsetPartitionResponse() + .setPartitions(Collections.singletonList(new ListOffsetsPartitionResponse() .setPartitionIndex(tp.partition()) .setErrorCode(error.code()) .setTimestamp(timestamp) diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index 34aed9d98bf0d..ce2e7af4f2bfc 100755 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -548,15 +548,15 @@ public static String join(T[] strs, String separator) { } /** - * Create a string representation of a list joined by the given separator - * @param list The list of items + * Create a string representation of a collection joined by the given separator + * @param collection The list of items * @param separator The separator * @return The string representation. */ - public static String join(Collection list, String separator) { - Objects.requireNonNull(list); + public static String join(Collection collection, String separator) { + Objects.requireNonNull(collection); StringBuilder sb = new StringBuilder(); - Iterator iter = list.iterator(); + Iterator iter = collection.iterator(); while (iter.hasNext()) { sb.append(iter.next()); if (iter.hasNext()) diff --git a/clients/src/main/resources/common/message/ListOffsetRequest.json b/clients/src/main/resources/common/message/ListOffsetsRequest.json similarity index 93% rename from clients/src/main/resources/common/message/ListOffsetRequest.json rename to clients/src/main/resources/common/message/ListOffsetsRequest.json index 8d1d6cadf8db2..9855a4bf9058d 100644 --- a/clients/src/main/resources/common/message/ListOffsetRequest.json +++ b/clients/src/main/resources/common/message/ListOffsetsRequest.json @@ -16,7 +16,7 @@ { "apiKey": 2, "type": "request", - "name": "ListOffsetRequest", + "name": "ListOffsetsRequest", // Version 1 removes MaxNumOffsets. From this version forward, only a single // offset can be returned. // @@ -36,11 +36,11 @@ "about": "The broker ID of the requestor, or -1 if this request is being made by a normal consumer." }, { "name": "IsolationLevel", "type": "int8", "versions": "2+", "about": "This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records" }, - { "name": "Topics", "type": "[]ListOffsetTopic", "versions": "0+", + { "name": "Topics", "type": "[]ListOffsetsTopic", "versions": "0+", "about": "Each topic in the request.", "fields": [ { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", "about": "The topic name." }, - { "name": "Partitions", "type": "[]ListOffsetPartition", "versions": "0+", + { "name": "Partitions", "type": "[]ListOffsetsPartition", "versions": "0+", "about": "Each partition in the request.", "fields": [ { "name": "PartitionIndex", "type": "int32", "versions": "0+", "about": "The partition index." }, diff --git a/clients/src/main/resources/common/message/ListOffsetResponse.json b/clients/src/main/resources/common/message/ListOffsetsResponse.json similarity index 93% rename from clients/src/main/resources/common/message/ListOffsetResponse.json rename to clients/src/main/resources/common/message/ListOffsetsResponse.json index ffa15621c7881..727bb8f274a99 100644 --- a/clients/src/main/resources/common/message/ListOffsetResponse.json +++ b/clients/src/main/resources/common/message/ListOffsetsResponse.json @@ -16,7 +16,7 @@ { "apiKey": 2, "type": "response", - "name": "ListOffsetResponse", + "name": "ListOffsetsResponse", // Version 1 removes the offsets array in favor of returning a single offset. // Version 1 also adds the timestamp associated with the returned offset. // @@ -34,11 +34,11 @@ "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "2+", "ignorable": true, "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, - { "name": "Topics", "type": "[]ListOffsetTopicResponse", "versions": "0+", + { "name": "Topics", "type": "[]ListOffsetsTopicResponse", "versions": "0+", "about": "Each topic in the response.", "fields": [ { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", "about": "The topic name" }, - { "name": "Partitions", "type": "[]ListOffsetPartitionResponse", "versions": "0+", + { "name": "Partitions", "type": "[]ListOffsetsPartitionResponse", "versions": "0+", "about": "Each partition in the response.", "fields": [ { "name": "PartitionIndex", "type": "int32", "versions": "0+", "about": "The partition index." }, diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java index 8980289824e3f..7fe501f1f45ab 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java @@ -104,8 +104,8 @@ import org.apache.kafka.common.message.LeaveGroupResponseData; import org.apache.kafka.common.message.LeaveGroupResponseData.MemberResponse; import org.apache.kafka.common.message.ListGroupsResponseData; -import org.apache.kafka.common.message.ListOffsetResponseData; -import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetTopicResponse; +import org.apache.kafka.common.message.ListOffsetsResponseData; +import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse; import org.apache.kafka.common.message.ListPartitionReassignmentsResponseData; import org.apache.kafka.common.message.MetadataResponseData.MetadataResponseTopic; import org.apache.kafka.common.message.MetadataResponseData.MetadataResponsePartition; @@ -152,7 +152,7 @@ import org.apache.kafka.common.requests.LeaveGroupResponse; import org.apache.kafka.common.requests.ListGroupsRequest; import org.apache.kafka.common.requests.ListGroupsResponse; -import org.apache.kafka.common.requests.ListOffsetResponse; +import org.apache.kafka.common.requests.ListOffsetsResponse; import org.apache.kafka.common.requests.ListPartitionReassignmentsResponse; import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.requests.MetadataResponse; @@ -3816,13 +3816,13 @@ public void testListOffsets() throws Exception { env.kafkaClient().prepareResponse(prepareMetadataResponse(cluster, Errors.NONE)); - ListOffsetTopicResponse t0 = ListOffsetResponse.singletonListOffsetTopicResponse(tp0, Errors.NONE, -1L, 123L, 321); - ListOffsetTopicResponse t1 = ListOffsetResponse.singletonListOffsetTopicResponse(tp1, Errors.NONE, -1L, 234L, 432); - ListOffsetTopicResponse t2 = ListOffsetResponse.singletonListOffsetTopicResponse(tp2, Errors.NONE, 123456789L, 345L, 543); - ListOffsetResponseData responseData = new ListOffsetResponseData() + ListOffsetsTopicResponse t0 = ListOffsetsResponse.singletonListOffsetsTopicResponse(tp0, Errors.NONE, -1L, 123L, 321); + ListOffsetsTopicResponse t1 = ListOffsetsResponse.singletonListOffsetsTopicResponse(tp1, Errors.NONE, -1L, 234L, 432); + ListOffsetsTopicResponse t2 = ListOffsetsResponse.singletonListOffsetsTopicResponse(tp2, Errors.NONE, 123456789L, 345L, 543); + ListOffsetsResponseData responseData = new ListOffsetsResponseData() .setThrottleTimeMs(0) .setTopics(Arrays.asList(t0, t1, t2)); - env.kafkaClient().prepareResponse(new ListOffsetResponse(responseData)); + env.kafkaClient().prepareResponse(new ListOffsetsResponse(responseData)); Map partitions = new HashMap<>(); partitions.put(tp0, OffsetSpec.latest()); @@ -3878,27 +3878,27 @@ public void testListOffsetsRetriableErrors() throws Exception { env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); env.kafkaClient().prepareResponse(prepareMetadataResponse(cluster, Errors.NONE)); // listoffsets response from broker 0 - ListOffsetTopicResponse t0 = ListOffsetResponse.singletonListOffsetTopicResponse(tp0, Errors.LEADER_NOT_AVAILABLE, -1L, 123L, 321); - ListOffsetTopicResponse t1 = ListOffsetResponse.singletonListOffsetTopicResponse(tp1, Errors.NONE, -1L, 987L, 789); - ListOffsetResponseData responseData = new ListOffsetResponseData() + ListOffsetsTopicResponse t0 = ListOffsetsResponse.singletonListOffsetsTopicResponse(tp0, Errors.LEADER_NOT_AVAILABLE, -1L, 123L, 321); + ListOffsetsTopicResponse t1 = ListOffsetsResponse.singletonListOffsetsTopicResponse(tp1, Errors.NONE, -1L, 987L, 789); + ListOffsetsResponseData responseData = new ListOffsetsResponseData() .setThrottleTimeMs(0) .setTopics(Arrays.asList(t0, t1)); - env.kafkaClient().prepareResponseFrom(new ListOffsetResponse(responseData), node0); + env.kafkaClient().prepareResponseFrom(new ListOffsetsResponse(responseData), node0); // listoffsets response from broker 1 - ListOffsetTopicResponse t2 = ListOffsetResponse.singletonListOffsetTopicResponse(tp2, Errors.NONE, -1L, 456L, 654); - responseData = new ListOffsetResponseData() + ListOffsetsTopicResponse t2 = ListOffsetsResponse.singletonListOffsetsTopicResponse(tp2, Errors.NONE, -1L, 456L, 654); + responseData = new ListOffsetsResponseData() .setThrottleTimeMs(0) .setTopics(Arrays.asList(t2)); - env.kafkaClient().prepareResponseFrom(new ListOffsetResponse(responseData), node1); + env.kafkaClient().prepareResponseFrom(new ListOffsetsResponse(responseData), node1); // metadata refresh because of LEADER_NOT_AVAILABLE env.kafkaClient().prepareResponse(prepareMetadataResponse(cluster, Errors.NONE)); // listoffsets response from broker 0 - t0 = ListOffsetResponse.singletonListOffsetTopicResponse(tp0, Errors.NONE, -1L, 345L, 543); - responseData = new ListOffsetResponseData() + t0 = ListOffsetsResponse.singletonListOffsetsTopicResponse(tp0, Errors.NONE, -1L, 345L, 543); + responseData = new ListOffsetsResponseData() .setThrottleTimeMs(0) .setTopics(Arrays.asList(t0)); - env.kafkaClient().prepareResponseFrom(new ListOffsetResponse(responseData), node0); + env.kafkaClient().prepareResponseFrom(new ListOffsetsResponse(responseData), node0); Map partitions = new HashMap<>(); partitions.put(tp0, OffsetSpec.latest()); @@ -3944,11 +3944,11 @@ public void testListOffsetsNonRetriableErrors() throws Exception { env.kafkaClient().prepareResponse(prepareMetadataResponse(cluster, Errors.NONE)); - ListOffsetTopicResponse t0 = ListOffsetResponse.singletonListOffsetTopicResponse(tp0, Errors.TOPIC_AUTHORIZATION_FAILED, -1L, -1L, -1); - ListOffsetResponseData responseData = new ListOffsetResponseData() + ListOffsetsTopicResponse t0 = ListOffsetsResponse.singletonListOffsetsTopicResponse(tp0, Errors.TOPIC_AUTHORIZATION_FAILED, -1L, -1L, -1); + ListOffsetsResponseData responseData = new ListOffsetsResponseData() .setThrottleTimeMs(0) .setTopics(Arrays.asList(t0)); - env.kafkaClient().prepareResponse(new ListOffsetResponse(responseData)); + env.kafkaClient().prepareResponse(new ListOffsetsResponse(responseData)); Map partitions = new HashMap<>(); partitions.put(tp0, OffsetSpec.latest()); @@ -4152,17 +4152,17 @@ public void testListOffsetsMetadataRetriableErrors() throws Exception { env.kafkaClient().prepareResponse(prepareMetadataResponse(cluster, Errors.NONE)); // listoffsets response from broker 0 - ListOffsetTopicResponse t0 = ListOffsetResponse.singletonListOffsetTopicResponse(tp0, Errors.NONE, -1L, 345L, 543); - ListOffsetResponseData responseData = new ListOffsetResponseData() + ListOffsetsTopicResponse t0 = ListOffsetsResponse.singletonListOffsetsTopicResponse(tp0, Errors.NONE, -1L, 345L, 543); + ListOffsetsResponseData responseData = new ListOffsetsResponseData() .setThrottleTimeMs(0) .setTopics(Arrays.asList(t0)); - env.kafkaClient().prepareResponseFrom(new ListOffsetResponse(responseData), node0); + env.kafkaClient().prepareResponseFrom(new ListOffsetsResponse(responseData), node0); // listoffsets response from broker 1 - ListOffsetTopicResponse t1 = ListOffsetResponse.singletonListOffsetTopicResponse(tp1, Errors.NONE, -1L, 789L, 987); - responseData = new ListOffsetResponseData() + ListOffsetsTopicResponse t1 = ListOffsetsResponse.singletonListOffsetsTopicResponse(tp1, Errors.NONE, -1L, 789L, 987); + responseData = new ListOffsetsResponseData() .setThrottleTimeMs(0) .setTopics(Arrays.asList(t1)); - env.kafkaClient().prepareResponseFrom(new ListOffsetResponse(responseData), node1); + env.kafkaClient().prepareResponseFrom(new ListOffsetsResponse(responseData), node1); Map partitions = new HashMap<>(); partitions.put(tp0, OffsetSpec.latest()); @@ -4203,12 +4203,12 @@ public void testListOffsetsWithMultiplePartitionsLeaderChange() throws Exception env.kafkaClient().prepareResponse(prepareMetadataResponse(oldCluster, Errors.NONE)); - ListOffsetTopicResponse t0 = ListOffsetResponse.singletonListOffsetTopicResponse(tp0, Errors.NOT_LEADER_OR_FOLLOWER, -1L, 345L, 543); - ListOffsetTopicResponse t1 = ListOffsetResponse.singletonListOffsetTopicResponse(tp1, Errors.LEADER_NOT_AVAILABLE, -2L, 123L, 456); - ListOffsetResponseData responseData = new ListOffsetResponseData() + ListOffsetsTopicResponse t0 = ListOffsetsResponse.singletonListOffsetsTopicResponse(tp0, Errors.NOT_LEADER_OR_FOLLOWER, -1L, 345L, 543); + ListOffsetsTopicResponse t1 = ListOffsetsResponse.singletonListOffsetsTopicResponse(tp1, Errors.LEADER_NOT_AVAILABLE, -2L, 123L, 456); + ListOffsetsResponseData responseData = new ListOffsetsResponseData() .setThrottleTimeMs(0) .setTopics(Arrays.asList(t0, t1)); - env.kafkaClient().prepareResponseFrom(new ListOffsetResponse(responseData), node0); + env.kafkaClient().prepareResponseFrom(new ListOffsetsResponse(responseData), node0); final PartitionInfo newPInfo1 = new PartitionInfo("foo", 0, node1, new Node[]{node0, node1, node2}, new Node[]{node0, node1, node2}); @@ -4221,17 +4221,17 @@ public void testListOffsetsWithMultiplePartitionsLeaderChange() throws Exception env.kafkaClient().prepareResponse(prepareMetadataResponse(newCluster, Errors.NONE)); - t0 = ListOffsetResponse.singletonListOffsetTopicResponse(tp0, Errors.NONE, -1L, 345L, 543); - responseData = new ListOffsetResponseData() + t0 = ListOffsetsResponse.singletonListOffsetsTopicResponse(tp0, Errors.NONE, -1L, 345L, 543); + responseData = new ListOffsetsResponseData() .setThrottleTimeMs(0) .setTopics(Arrays.asList(t0)); - env.kafkaClient().prepareResponseFrom(new ListOffsetResponse(responseData), node1); + env.kafkaClient().prepareResponseFrom(new ListOffsetsResponse(responseData), node1); - t1 = ListOffsetResponse.singletonListOffsetTopicResponse(tp1, Errors.NONE, -2L, 123L, 456); - responseData = new ListOffsetResponseData() + t1 = ListOffsetsResponse.singletonListOffsetsTopicResponse(tp1, Errors.NONE, -2L, 123L, 456); + responseData = new ListOffsetsResponseData() .setThrottleTimeMs(0) .setTopics(Arrays.asList(t1)); - env.kafkaClient().prepareResponseFrom(new ListOffsetResponse(responseData), node2); + env.kafkaClient().prepareResponseFrom(new ListOffsetsResponse(responseData), node2); Map partitions = new HashMap<>(); partitions.put(tp0, OffsetSpec.latest()); @@ -4267,11 +4267,11 @@ public void testListOffsetsWithLeaderChange() throws Exception { env.kafkaClient().prepareResponse(prepareMetadataResponse(oldCluster, Errors.NONE)); - ListOffsetTopicResponse t0 = ListOffsetResponse.singletonListOffsetTopicResponse(tp0, Errors.NOT_LEADER_OR_FOLLOWER, -1L, 345L, 543); - ListOffsetResponseData responseData = new ListOffsetResponseData() + ListOffsetsTopicResponse t0 = ListOffsetsResponse.singletonListOffsetsTopicResponse(tp0, Errors.NOT_LEADER_OR_FOLLOWER, -1L, 345L, 543); + ListOffsetsResponseData responseData = new ListOffsetsResponseData() .setThrottleTimeMs(0) .setTopics(Arrays.asList(t0)); - env.kafkaClient().prepareResponseFrom(new ListOffsetResponse(responseData), node0); + env.kafkaClient().prepareResponseFrom(new ListOffsetsResponse(responseData), node0); // updating leader from node0 to node1 and metadata refresh because of NOT_LEADER_OR_FOLLOWER final PartitionInfo newPartitionInfo = new PartitionInfo("foo", 0, node1, @@ -4281,11 +4281,11 @@ public void testListOffsetsWithLeaderChange() throws Exception { env.kafkaClient().prepareResponse(prepareMetadataResponse(newCluster, Errors.NONE)); - t0 = ListOffsetResponse.singletonListOffsetTopicResponse(tp0, Errors.NONE, -2L, 123L, 456); - responseData = new ListOffsetResponseData() + t0 = ListOffsetsResponse.singletonListOffsetsTopicResponse(tp0, Errors.NONE, -2L, 123L, 456); + responseData = new ListOffsetsResponseData() .setThrottleTimeMs(0) .setTopics(Arrays.asList(t0)); - env.kafkaClient().prepareResponseFrom(new ListOffsetResponse(responseData), node1); + env.kafkaClient().prepareResponseFrom(new ListOffsetsResponse(responseData), node1); Map partitions = new HashMap<>(); partitions.put(tp0, OffsetSpec.latest()); @@ -4356,11 +4356,11 @@ public void testListOffsetsPartialResponse() throws Exception { env.kafkaClient().prepareResponse(prepareMetadataResponse(cluster, Errors.NONE)); - ListOffsetTopicResponse t0 = ListOffsetResponse.singletonListOffsetTopicResponse(tp0, Errors.NONE, -2L, 123L, 456); - ListOffsetResponseData data = new ListOffsetResponseData() + ListOffsetsTopicResponse t0 = ListOffsetsResponse.singletonListOffsetsTopicResponse(tp0, Errors.NONE, -2L, 123L, 456); + ListOffsetsResponseData data = new ListOffsetsResponseData() .setThrottleTimeMs(0) .setTopics(Arrays.asList(t0)); - env.kafkaClient().prepareResponseFrom(new ListOffsetResponse(data), node0); + env.kafkaClient().prepareResponseFrom(new ListOffsetsResponse(data), node0); Map partitions = new HashMap<>(); partitions.put(tp0, OffsetSpec.latest()); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 0040452667431..7abcc2a16f52c 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -49,12 +49,12 @@ import org.apache.kafka.common.message.JoinGroupRequestData; import org.apache.kafka.common.message.JoinGroupResponseData; import org.apache.kafka.common.message.LeaveGroupResponseData; -import org.apache.kafka.common.message.ListOffsetResponseData; -import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetTopicResponse; -import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetPartitionResponse; +import org.apache.kafka.common.message.ListOffsetsResponseData; +import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse; +import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse; import org.apache.kafka.common.internals.ClusterResourceListeners; import org.apache.kafka.common.message.SyncGroupResponseData; -import org.apache.kafka.common.message.ListOffsetRequestData.ListOffsetPartition; +import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.network.Selectable; @@ -73,8 +73,8 @@ import org.apache.kafka.common.requests.JoinGroupRequest; import org.apache.kafka.common.requests.JoinGroupResponse; import org.apache.kafka.common.requests.LeaveGroupResponse; -import org.apache.kafka.common.requests.ListOffsetRequest; -import org.apache.kafka.common.requests.ListOffsetResponse; +import org.apache.kafka.common.requests.ListOffsetsRequest; +import org.apache.kafka.common.requests.ListOffsetsResponse; import org.apache.kafka.common.requests.MetadataResponse; import org.apache.kafka.common.requests.OffsetCommitRequest; import org.apache.kafka.common.requests.OffsetCommitResponse; @@ -601,19 +601,19 @@ public void testFetchProgressWithMissingPartitionPosition() { consumer.seekToBeginning(singleton(tp1)); client.prepareResponse(body -> { - ListOffsetRequest request = (ListOffsetRequest) body; - List partitions = request.topics().stream().flatMap(t -> { + ListOffsetsRequest request = (ListOffsetsRequest) body; + List partitions = request.topics().stream().flatMap(t -> { if (t.name().equals(topic)) return Stream.of(t.partitions()); else return Stream.empty(); }).flatMap(List::stream).collect(Collectors.toList()); - ListOffsetPartition expectedTp0 = new ListOffsetPartition() + ListOffsetsPartition expectedTp0 = new ListOffsetsPartition() .setPartitionIndex(tp0.partition()) - .setTimestamp(ListOffsetRequest.LATEST_TIMESTAMP); - ListOffsetPartition expectedTp1 = new ListOffsetPartition() + .setTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP); + ListOffsetsPartition expectedTp1 = new ListOffsetsPartition() .setPartitionIndex(tp1.partition()) - .setTimestamp(ListOffsetRequest.EARLIEST_TIMESTAMP); + .setTimestamp(ListOffsetsRequest.EARLIEST_TIMESTAMP); return partitions.contains(expectedTp0) && partitions.contains(expectedTp1); }, listOffsetsResponse(Collections.singletonMap(tp0, 50L), Collections.singletonMap(tp1, Errors.NOT_LEADER_OR_FOLLOWER))); client.prepareResponse( @@ -2200,35 +2200,35 @@ private OffsetFetchResponse offsetResponse(Map offsets, Er return new OffsetFetchResponse(Errors.NONE, partitionData); } - private ListOffsetResponse listOffsetsResponse(Map offsets) { + private ListOffsetsResponse listOffsetsResponse(Map offsets) { return listOffsetsResponse(offsets, Collections.emptyMap()); } - private ListOffsetResponse listOffsetsResponse(Map partitionOffsets, - Map partitionErrors) { - Map responses = new HashMap<>(); + private ListOffsetsResponse listOffsetsResponse(Map partitionOffsets, + Map partitionErrors) { + Map responses = new HashMap<>(); for (Map.Entry partitionOffset : partitionOffsets.entrySet()) { TopicPartition tp = partitionOffset.getKey(); - ListOffsetTopicResponse topic = responses.computeIfAbsent(tp.topic(), k -> new ListOffsetTopicResponse().setName(tp.topic())); - topic.partitions().add(new ListOffsetPartitionResponse() + ListOffsetsTopicResponse topic = responses.computeIfAbsent(tp.topic(), k -> new ListOffsetsTopicResponse().setName(tp.topic())); + topic.partitions().add(new ListOffsetsPartitionResponse() .setPartitionIndex(tp.partition()) .setErrorCode(Errors.NONE.code()) - .setTimestamp(ListOffsetResponse.UNKNOWN_TIMESTAMP) + .setTimestamp(ListOffsetsResponse.UNKNOWN_TIMESTAMP) .setOffset(partitionOffset.getValue())); } for (Map.Entry partitionError : partitionErrors.entrySet()) { TopicPartition tp = partitionError.getKey(); - ListOffsetTopicResponse topic = responses.computeIfAbsent(tp.topic(), k -> new ListOffsetTopicResponse().setName(tp.topic())); - topic.partitions().add(new ListOffsetPartitionResponse() + ListOffsetsTopicResponse topic = responses.computeIfAbsent(tp.topic(), k -> new ListOffsetsTopicResponse().setName(tp.topic())); + topic.partitions().add(new ListOffsetsPartitionResponse() .setPartitionIndex(tp.partition()) .setErrorCode(partitionError.getValue().code()) - .setTimestamp(ListOffsetResponse.UNKNOWN_TIMESTAMP) - .setOffset(ListOffsetResponse.UNKNOWN_OFFSET)); + .setTimestamp(ListOffsetsResponse.UNKNOWN_TIMESTAMP) + .setOffset(ListOffsetsResponse.UNKNOWN_OFFSET)); } - ListOffsetResponseData data = new ListOffsetResponseData() + ListOffsetsResponseData data = new ListOffsetsResponseData() .setTopics(new ArrayList<>(responses.values())); - return new ListOffsetResponse(data); + return new ListOffsetsResponse(data); } private FetchResponse fetchResponse(Map fetches) { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index 5fd65791927f6..b8f9e4cf87edb 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -48,11 +48,11 @@ import org.apache.kafka.common.header.Header; import org.apache.kafka.common.header.internals.RecordHeader; import org.apache.kafka.common.internals.ClusterResourceListeners; -import org.apache.kafka.common.message.ListOffsetRequestData.ListOffsetTopic; -import org.apache.kafka.common.message.ListOffsetRequestData.ListOffsetPartition; -import org.apache.kafka.common.message.ListOffsetResponseData; -import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetTopicResponse; -import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetPartitionResponse; +import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsTopic; +import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition; +import org.apache.kafka.common.message.ListOffsetsResponseData; +import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse; +import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse; import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData; import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition; import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData; @@ -81,8 +81,8 @@ import org.apache.kafka.common.requests.ApiVersionsResponse; import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.requests.FetchResponse; -import org.apache.kafka.common.requests.ListOffsetRequest; -import org.apache.kafka.common.requests.ListOffsetResponse; +import org.apache.kafka.common.requests.ListOffsetsRequest; +import org.apache.kafka.common.requests.ListOffsetsResponse; import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.requests.MetadataResponse; import org.apache.kafka.common.requests.OffsetsForLeaderEpochRequest; @@ -1446,7 +1446,7 @@ public void testUpdateFetchPositionResetToDefaultOffset() { assignFromUser(singleton(tp0)); subscriptions.requestOffsetReset(tp0); - client.prepareResponse(listOffsetRequestMatcher(ListOffsetRequest.EARLIEST_TIMESTAMP, + client.prepareResponse(listOffsetRequestMatcher(ListOffsetsRequest.EARLIEST_TIMESTAMP, validLeaderEpoch), listOffsetResponse(Errors.NONE, 1L, 5L)); fetcher.resetOffsetsIfNeeded(); consumerClient.pollNoWakeup(); @@ -1463,7 +1463,7 @@ public void testUpdateFetchPositionResetToLatestOffset() { client.updateMetadata(initialUpdateResponse); - client.prepareResponse(listOffsetRequestMatcher(ListOffsetRequest.LATEST_TIMESTAMP), + client.prepareResponse(listOffsetRequestMatcher(ListOffsetsRequest.LATEST_TIMESTAMP), listOffsetResponse(Errors.NONE, 1L, 5L)); fetcher.resetOffsetsIfNeeded(); consumerClient.pollNoWakeup(); @@ -1482,7 +1482,7 @@ public void testFetchOffsetErrors() { subscriptions.requestOffsetReset(tp0, OffsetResetStrategy.LATEST); // Fail with OFFSET_NOT_AVAILABLE - client.prepareResponse(listOffsetRequestMatcher(ListOffsetRequest.LATEST_TIMESTAMP, + client.prepareResponse(listOffsetRequestMatcher(ListOffsetsRequest.LATEST_TIMESTAMP, validLeaderEpoch), listOffsetResponse(Errors.OFFSET_NOT_AVAILABLE, 1L, 5L), false); fetcher.resetOffsetsIfNeeded(); consumerClient.pollNoWakeup(); @@ -1492,7 +1492,7 @@ public void testFetchOffsetErrors() { // Fail with LEADER_NOT_AVAILABLE time.sleep(retryBackoffMs); - client.prepareResponse(listOffsetRequestMatcher(ListOffsetRequest.LATEST_TIMESTAMP, + client.prepareResponse(listOffsetRequestMatcher(ListOffsetsRequest.LATEST_TIMESTAMP, validLeaderEpoch), listOffsetResponse(Errors.LEADER_NOT_AVAILABLE, 1L, 5L), false); fetcher.resetOffsetsIfNeeded(); consumerClient.pollNoWakeup(); @@ -1502,7 +1502,7 @@ public void testFetchOffsetErrors() { // Back to normal time.sleep(retryBackoffMs); - client.prepareResponse(listOffsetRequestMatcher(ListOffsetRequest.LATEST_TIMESTAMP), + client.prepareResponse(listOffsetRequestMatcher(ListOffsetsRequest.LATEST_TIMESTAMP), listOffsetResponse(Errors.NONE, 1L, 5L), false); fetcher.resetOffsetsIfNeeded(); consumerClient.pollNoWakeup(); @@ -1530,7 +1530,7 @@ private void testListOffsetsSendsIsolationLevel(IsolationLevel isolationLevel) { subscriptions.requestOffsetReset(tp0, OffsetResetStrategy.LATEST); client.prepareResponse(body -> { - ListOffsetRequest request = (ListOffsetRequest) body; + ListOffsetsRequest request = (ListOffsetsRequest) body; return request.isolationLevel() == isolationLevel; }, listOffsetResponse(Errors.NONE, 1L, 5L)); fetcher.resetOffsetsIfNeeded(); @@ -1558,7 +1558,7 @@ public void testResetOffsetsSkipsBlackedOutConnections() { assertEquals(OffsetResetStrategy.EARLIEST, subscriptions.resetStrategy(tp0)); time.sleep(500); - client.prepareResponse(listOffsetRequestMatcher(ListOffsetRequest.EARLIEST_TIMESTAMP), + client.prepareResponse(listOffsetRequestMatcher(ListOffsetsRequest.EARLIEST_TIMESTAMP), listOffsetResponse(Errors.NONE, 1L, 5L)); fetcher.resetOffsetsIfNeeded(); consumerClient.pollNoWakeup(); @@ -1574,7 +1574,7 @@ public void testUpdateFetchPositionResetToEarliestOffset() { assignFromUser(singleton(tp0)); subscriptions.requestOffsetReset(tp0, OffsetResetStrategy.EARLIEST); - client.prepareResponse(listOffsetRequestMatcher(ListOffsetRequest.EARLIEST_TIMESTAMP, + client.prepareResponse(listOffsetRequestMatcher(ListOffsetsRequest.EARLIEST_TIMESTAMP, validLeaderEpoch), listOffsetResponse(Errors.NONE, 1L, 5L)); fetcher.resetOffsetsIfNeeded(); consumerClient.pollNoWakeup(); @@ -1591,7 +1591,7 @@ public void testResetOffsetsMetadataRefresh() { subscriptions.requestOffsetReset(tp0, OffsetResetStrategy.LATEST); // First fetch fails with stale metadata - client.prepareResponse(listOffsetRequestMatcher(ListOffsetRequest.LATEST_TIMESTAMP, + client.prepareResponse(listOffsetRequestMatcher(ListOffsetsRequest.LATEST_TIMESTAMP, validLeaderEpoch), listOffsetResponse(Errors.NOT_LEADER_OR_FOLLOWER, 1L, 5L), false); fetcher.resetOffsetsIfNeeded(); consumerClient.pollNoWakeup(); @@ -1604,7 +1604,7 @@ public void testResetOffsetsMetadataRefresh() { // Next fetch succeeds time.sleep(retryBackoffMs); - client.prepareResponse(listOffsetRequestMatcher(ListOffsetRequest.LATEST_TIMESTAMP), + client.prepareResponse(listOffsetRequestMatcher(ListOffsetsRequest.LATEST_TIMESTAMP), listOffsetResponse(Errors.NONE, 1L, 5L)); fetcher.resetOffsetsIfNeeded(); consumerClient.pollNoWakeup(); @@ -1626,7 +1626,7 @@ public void testListOffsetNoUpdateMissingEpoch() { // Return a ListOffsets response with leaderEpoch=1, we should ignore it subscriptions.requestOffsetReset(tp0, OffsetResetStrategy.LATEST); - client.prepareResponse(listOffsetRequestMatcher(ListOffsetRequest.LATEST_TIMESTAMP), + client.prepareResponse(listOffsetRequestMatcher(ListOffsetsRequest.LATEST_TIMESTAMP), listOffsetResponse(tp0, Errors.NONE, 1L, 5L, 1)); fetcher.resetOffsetsIfNeeded(); consumerClient.pollNoWakeup(); @@ -1651,7 +1651,7 @@ public void testListOffsetUpdateEpoch() { subscriptions.requestOffsetReset(tp0, OffsetResetStrategy.LATEST); // Now we see a ListOffsets with leaderEpoch=2 epoch, we trigger a metadata update - client.prepareResponse(listOffsetRequestMatcher(ListOffsetRequest.LATEST_TIMESTAMP, 1), + client.prepareResponse(listOffsetRequestMatcher(ListOffsetsRequest.LATEST_TIMESTAMP, 1), listOffsetResponse(tp0, Errors.NONE, 1L, 5L, 2)); fetcher.resetOffsetsIfNeeded(); consumerClient.pollNoWakeup(); @@ -1668,7 +1668,7 @@ public void testUpdateFetchPositionDisconnect() { subscriptions.requestOffsetReset(tp0, OffsetResetStrategy.LATEST); // First request gets a disconnect - client.prepareResponse(listOffsetRequestMatcher(ListOffsetRequest.LATEST_TIMESTAMP, + client.prepareResponse(listOffsetRequestMatcher(ListOffsetsRequest.LATEST_TIMESTAMP, validLeaderEpoch), listOffsetResponse(Errors.NONE, 1L, 5L), true); fetcher.resetOffsetsIfNeeded(); consumerClient.pollNoWakeup(); @@ -1687,7 +1687,7 @@ public void testUpdateFetchPositionDisconnect() { // Next one succeeds time.sleep(retryBackoffMs); - client.prepareResponse(listOffsetRequestMatcher(ListOffsetRequest.LATEST_TIMESTAMP), + client.prepareResponse(listOffsetRequestMatcher(ListOffsetsRequest.LATEST_TIMESTAMP), listOffsetResponse(Errors.NONE, 1L, 5L)); fetcher.resetOffsetsIfNeeded(); consumerClient.pollNoWakeup(); @@ -1846,7 +1846,7 @@ public void testRestOffsetsAuthorizationFailure() { subscriptions.requestOffsetReset(tp0, OffsetResetStrategy.LATEST); // First request gets a disconnect - client.prepareResponse(listOffsetRequestMatcher(ListOffsetRequest.LATEST_TIMESTAMP, + client.prepareResponse(listOffsetRequestMatcher(ListOffsetsRequest.LATEST_TIMESTAMP, validLeaderEpoch), listOffsetResponse(Errors.TOPIC_AUTHORIZATION_FAILED, -1, -1), false); fetcher.resetOffsetsIfNeeded(); consumerClient.pollNoWakeup(); @@ -1867,7 +1867,7 @@ public void testRestOffsetsAuthorizationFailure() { // Next one succeeds time.sleep(retryBackoffMs); - client.prepareResponse(listOffsetRequestMatcher(ListOffsetRequest.LATEST_TIMESTAMP), + client.prepareResponse(listOffsetRequestMatcher(ListOffsetsRequest.LATEST_TIMESTAMP), listOffsetResponse(Errors.NONE, 1L, 5L)); fetcher.resetOffsetsIfNeeded(); consumerClient.pollNoWakeup(); @@ -1884,7 +1884,7 @@ public void testUpdateFetchPositionOfPausedPartitionsRequiringOffsetReset() { subscriptions.pause(tp0); // paused partition does not have a valid position subscriptions.requestOffsetReset(tp0, OffsetResetStrategy.LATEST); - client.prepareResponse(listOffsetRequestMatcher(ListOffsetRequest.LATEST_TIMESTAMP, + client.prepareResponse(listOffsetRequestMatcher(ListOffsetsRequest.LATEST_TIMESTAMP, validLeaderEpoch), listOffsetResponse(Errors.NONE, 1L, 10L)); fetcher.resetOffsetsIfNeeded(); consumerClient.pollNoWakeup(); @@ -2504,74 +2504,74 @@ public void testGetOffsetByTimeWithPartitionsRetryCouldTriggerMetadataUpdate() { client.updateMetadata(initialUpdateResponse); final long fetchTimestamp = 10L; - ListOffsetPartitionResponse tp0NoError = new ListOffsetPartitionResponse() + ListOffsetsPartitionResponse tp0NoError = new ListOffsetsPartitionResponse() .setPartitionIndex(tp0.partition()) .setErrorCode(Errors.NONE.code()) .setTimestamp(fetchTimestamp) .setOffset(4L); - List topics = Collections.singletonList( - new ListOffsetTopicResponse() + List topics = Collections.singletonList( + new ListOffsetsTopicResponse() .setName(tp0.topic()) .setPartitions(Arrays.asList( tp0NoError, - new ListOffsetPartitionResponse() + new ListOffsetsPartitionResponse() .setPartitionIndex(tp1.partition()) .setErrorCode(retriableError.code()) - .setTimestamp(ListOffsetRequest.LATEST_TIMESTAMP) + .setTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP) .setOffset(-1L)))); - ListOffsetResponseData data = new ListOffsetResponseData() + ListOffsetsResponseData data = new ListOffsetsResponseData() .setThrottleTimeMs(0) .setTopics(topics); client.prepareResponseFrom(body -> { - boolean isListOffsetRequest = body instanceof ListOffsetRequest; + boolean isListOffsetRequest = body instanceof ListOffsetsRequest; if (isListOffsetRequest) { - ListOffsetRequest request = (ListOffsetRequest) body; - List expectedTopics = Collections.singletonList( - new ListOffsetTopic() + ListOffsetsRequest request = (ListOffsetsRequest) body; + List expectedTopics = Collections.singletonList( + new ListOffsetsTopic() .setName(tp0.topic()) .setPartitions(Arrays.asList( - new ListOffsetPartition() + new ListOffsetsPartition() .setPartitionIndex(tp1.partition()) .setTimestamp(fetchTimestamp) - .setCurrentLeaderEpoch(ListOffsetResponse.UNKNOWN_EPOCH), - new ListOffsetPartition() + .setCurrentLeaderEpoch(ListOffsetsResponse.UNKNOWN_EPOCH), + new ListOffsetsPartition() .setPartitionIndex(tp0.partition()) .setTimestamp(fetchTimestamp) - .setCurrentLeaderEpoch(ListOffsetResponse.UNKNOWN_EPOCH)))); + .setCurrentLeaderEpoch(ListOffsetsResponse.UNKNOWN_EPOCH)))); return request.topics().equals(expectedTopics); } else { return false; } - }, new ListOffsetResponse(data), originalLeader); + }, new ListOffsetsResponse(data), originalLeader); client.prepareMetadataUpdate(updatedMetadata); // If the metadata wasn't updated before retrying, the fetcher would consult the original leader and hit a NOT_LEADER exception. // We will count the answered future response in the end to verify if this is the case. - List topicsWithFatalError = Collections.singletonList( - new ListOffsetTopicResponse() + List topicsWithFatalError = Collections.singletonList( + new ListOffsetsTopicResponse() .setName(tp0.topic()) .setPartitions(Arrays.asList( tp0NoError, - new ListOffsetPartitionResponse() + new ListOffsetsPartitionResponse() .setPartitionIndex(tp1.partition()) .setErrorCode(Errors.NOT_LEADER_OR_FOLLOWER.code()) - .setTimestamp(ListOffsetRequest.LATEST_TIMESTAMP) + .setTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP) .setOffset(-1L)))); - ListOffsetResponseData dataWithFatalError = new ListOffsetResponseData() + ListOffsetsResponseData dataWithFatalError = new ListOffsetsResponseData() .setThrottleTimeMs(0) .setTopics(topicsWithFatalError); - client.prepareResponseFrom(new ListOffsetResponse(dataWithFatalError), originalLeader); + client.prepareResponseFrom(new ListOffsetsResponse(dataWithFatalError), originalLeader); // The request to new leader must only contain one partition tp1 with error. client.prepareResponseFrom(body -> { - boolean isListOffsetRequest = body instanceof ListOffsetRequest; + boolean isListOffsetRequest = body instanceof ListOffsetsRequest; if (isListOffsetRequest) { - ListOffsetRequest request = (ListOffsetRequest) body; + ListOffsetsRequest request = (ListOffsetsRequest) body; - ListOffsetTopic requestTopic = request.topics().get(0); - ListOffsetPartition expectedPartition = new ListOffsetPartition() + ListOffsetsTopic requestTopic = request.topics().get(0); + ListOffsetsPartition expectedPartition = new ListOffsetsPartition() .setPartitionIndex(tp1.partition()) .setTimestamp(fetchTimestamp) .setCurrentLeaderEpoch(newLeaderEpoch); @@ -2632,10 +2632,10 @@ public void testGetOffsetsIncludesLeaderEpoch() { // Check for epoch in outgoing request MockClient.RequestMatcher matcher = body -> { - if (body instanceof ListOffsetRequest) { - ListOffsetRequest offsetRequest = (ListOffsetRequest) body; + if (body instanceof ListOffsetsRequest) { + ListOffsetsRequest offsetRequest = (ListOffsetsRequest) body; int epoch = offsetRequest.topics().get(0).partitions().get(0).currentLeaderEpoch(); - assertTrue("Expected Fetcher to set leader epoch in request", epoch != ListOffsetResponse.UNKNOWN_EPOCH); + assertTrue("Expected Fetcher to set leader epoch in request", epoch != ListOffsetsResponse.UNKNOWN_EPOCH); assertEquals("Expected leader epoch to match epoch from metadata update", epoch, 99); return true; } else { @@ -2679,9 +2679,9 @@ public void testGetOffsetsForTimesWhenSomeTopicPartitionLeadersNotKnownInitially metadata.fetch().leaderFor(t2p0)); Map timestampToSearch = new HashMap<>(); - timestampToSearch.put(tp0, ListOffsetRequest.LATEST_TIMESTAMP); - timestampToSearch.put(tp1, ListOffsetRequest.LATEST_TIMESTAMP); - timestampToSearch.put(t2p0, ListOffsetRequest.LATEST_TIMESTAMP); + timestampToSearch.put(tp0, ListOffsetsRequest.LATEST_TIMESTAMP); + timestampToSearch.put(tp1, ListOffsetsRequest.LATEST_TIMESTAMP); + timestampToSearch.put(t2p0, ListOffsetsRequest.LATEST_TIMESTAMP); Map offsetAndTimestampMap = fetcher.offsetsForTimes(timestampToSearch, time.timer(Long.MAX_VALUE)); @@ -2714,12 +2714,12 @@ public void testGetOffsetsForTimesWhenSomeTopicPartitionLeadersDisconnectExcepti MetadataResponse updatedMetadata = RequestTestUtils.metadataUpdateWith(1, partitionNumByTopic); client.prepareMetadataUpdate(updatedMetadata); - client.prepareResponse(listOffsetRequestMatcher(ListOffsetRequest.LATEST_TIMESTAMP), + client.prepareResponse(listOffsetRequestMatcher(ListOffsetsRequest.LATEST_TIMESTAMP), listOffsetResponse(tp0, Errors.NONE, 1000L, 11L), true); client.prepareResponseFrom(listOffsetResponse(tp0, Errors.NONE, 1000L, 11L), metadata.fetch().leaderFor(tp0)); Map timestampToSearch = new HashMap<>(); - timestampToSearch.put(tp0, ListOffsetRequest.LATEST_TIMESTAMP); + timestampToSearch.put(tp0, ListOffsetsRequest.LATEST_TIMESTAMP); Map offsetAndTimestampMap = fetcher.offsetsForTimes(timestampToSearch, time.timer(Long.MAX_VALUE)); assertNotNull("Expect Fetcher.offsetsForTimes() to return non-null result for " + tp0, @@ -2732,26 +2732,26 @@ public void testGetOffsetsForTimesWhenSomeTopicPartitionLeadersDisconnectExcepti public void testBatchedListOffsetsMetadataErrors() { buildFetcher(); - ListOffsetResponseData data = new ListOffsetResponseData() + ListOffsetsResponseData data = new ListOffsetsResponseData() .setThrottleTimeMs(0) - .setTopics(Collections.singletonList(new ListOffsetTopicResponse() + .setTopics(Collections.singletonList(new ListOffsetsTopicResponse() .setName(tp0.topic()) .setPartitions(Arrays.asList( - new ListOffsetPartitionResponse() + new ListOffsetsPartitionResponse() .setPartitionIndex(tp0.partition()) .setErrorCode(Errors.NOT_LEADER_OR_FOLLOWER.code()) - .setTimestamp(ListOffsetResponse.UNKNOWN_TIMESTAMP) - .setOffset(ListOffsetResponse.UNKNOWN_OFFSET), - new ListOffsetPartitionResponse() + .setTimestamp(ListOffsetsResponse.UNKNOWN_TIMESTAMP) + .setOffset(ListOffsetsResponse.UNKNOWN_OFFSET), + new ListOffsetsPartitionResponse() .setPartitionIndex(tp1.partition()) .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) - .setTimestamp(ListOffsetResponse.UNKNOWN_TIMESTAMP) - .setOffset(ListOffsetResponse.UNKNOWN_OFFSET))))); - client.prepareResponse(new ListOffsetResponse(data)); + .setTimestamp(ListOffsetsResponse.UNKNOWN_TIMESTAMP) + .setOffset(ListOffsetsResponse.UNKNOWN_OFFSET))))); + client.prepareResponse(new ListOffsetsResponse(data)); Map offsetsToSearch = new HashMap<>(); - offsetsToSearch.put(tp0, ListOffsetRequest.EARLIEST_TIMESTAMP); - offsetsToSearch.put(tp1, ListOffsetRequest.EARLIEST_TIMESTAMP); + offsetsToSearch.put(tp0, ListOffsetsRequest.EARLIEST_TIMESTAMP); + offsetsToSearch.put(tp1, ListOffsetsRequest.EARLIEST_TIMESTAMP); fetcher.offsetsForTimes(offsetsToSearch, time.timer(0)); } @@ -3596,17 +3596,17 @@ private void testGetOffsetsForTimesWithUnknownOffset() { MetadataResponse initialMetadataUpdate = RequestTestUtils.metadataUpdateWith(1, singletonMap(topicName, 1)); client.updateMetadata(initialMetadataUpdate); - ListOffsetResponseData data = new ListOffsetResponseData() + ListOffsetsResponseData data = new ListOffsetsResponseData() .setThrottleTimeMs(0) - .setTopics(Collections.singletonList(new ListOffsetTopicResponse() + .setTopics(Collections.singletonList(new ListOffsetsTopicResponse() .setName(tp0.topic()) - .setPartitions(Collections.singletonList(new ListOffsetPartitionResponse() + .setPartitions(Collections.singletonList(new ListOffsetsPartitionResponse() .setPartitionIndex(tp0.partition()) .setErrorCode(Errors.NONE.code()) - .setTimestamp(ListOffsetResponse.UNKNOWN_TIMESTAMP) - .setOffset(ListOffsetResponse.UNKNOWN_OFFSET))))); + .setTimestamp(ListOffsetsResponse.UNKNOWN_TIMESTAMP) + .setOffset(ListOffsetsResponse.UNKNOWN_OFFSET))))); - client.prepareResponseFrom(new ListOffsetResponse(data), + client.prepareResponseFrom(new ListOffsetsResponse(data), metadata.fetch().leaderFor(tp0)); Map timestampToSearch = new HashMap<>(); @@ -3633,17 +3633,17 @@ public void testGetOffsetsForTimesWithUnknownOffsetV0() { apiVersions.update(node.idString(), NodeApiVersions.create( ApiKeys.LIST_OFFSETS.id, (short) 0, (short) 0)); - ListOffsetResponseData data = new ListOffsetResponseData() + ListOffsetsResponseData data = new ListOffsetsResponseData() .setThrottleTimeMs(0) - .setTopics(Collections.singletonList(new ListOffsetTopicResponse() + .setTopics(Collections.singletonList(new ListOffsetsTopicResponse() .setName(tp0.topic()) - .setPartitions(Collections.singletonList(new ListOffsetPartitionResponse() + .setPartitions(Collections.singletonList(new ListOffsetsPartitionResponse() .setPartitionIndex(tp0.partition()) .setErrorCode(Errors.NONE.code()) - .setTimestamp(ListOffsetResponse.UNKNOWN_TIMESTAMP) + .setTimestamp(ListOffsetsResponse.UNKNOWN_TIMESTAMP) .setOldStyleOffsets(Collections.emptyList()))))); - client.prepareResponseFrom(new ListOffsetResponse(data), + client.prepareResponseFrom(new ListOffsetsResponse(data), metadata.fetch().leaderFor(tp0)); Map timestampToSearch = new HashMap<>(); @@ -4296,7 +4296,7 @@ public void testCorruptMessageError() { public void testBeginningOffsets() { buildFetcher(); assignFromUser(singleton(tp0)); - client.prepareResponse(listOffsetResponse(tp0, Errors.NONE, ListOffsetRequest.EARLIEST_TIMESTAMP, 2L)); + client.prepareResponse(listOffsetResponse(tp0, Errors.NONE, ListOffsetsRequest.EARLIEST_TIMESTAMP, 2L)); assertEquals(singletonMap(tp0, 2L), fetcher.beginningOffsets(singleton(tp0), time.timer(5000L))); } @@ -4304,7 +4304,7 @@ public void testBeginningOffsets() { public void testBeginningOffsetsDuplicateTopicPartition() { buildFetcher(); assignFromUser(singleton(tp0)); - client.prepareResponse(listOffsetResponse(tp0, Errors.NONE, ListOffsetRequest.EARLIEST_TIMESTAMP, 2L)); + client.prepareResponse(listOffsetResponse(tp0, Errors.NONE, ListOffsetsRequest.EARLIEST_TIMESTAMP, 2L)); assertEquals(singletonMap(tp0, 2L), fetcher.beginningOffsets(asList(tp0, tp0), time.timer(5000L))); } @@ -4316,7 +4316,7 @@ public void testBeginningOffsetsMultipleTopicPartitions() { expectedOffsets.put(tp1, 4L); expectedOffsets.put(tp2, 6L); assignFromUser(expectedOffsets.keySet()); - client.prepareResponse(listOffsetResponse(expectedOffsets, Errors.NONE, ListOffsetRequest.EARLIEST_TIMESTAMP, ListOffsetResponse.UNKNOWN_EPOCH)); + client.prepareResponse(listOffsetResponse(expectedOffsets, Errors.NONE, ListOffsetsRequest.EARLIEST_TIMESTAMP, ListOffsetsResponse.UNKNOWN_EPOCH)); assertEquals(expectedOffsets, fetcher.beginningOffsets(asList(tp0, tp1, tp2), time.timer(5000L))); } @@ -4330,7 +4330,7 @@ public void testBeginningOffsetsEmpty() { public void testEndOffsets() { buildFetcher(); assignFromUser(singleton(tp0)); - client.prepareResponse(listOffsetResponse(tp0, Errors.NONE, ListOffsetRequest.LATEST_TIMESTAMP, 5L)); + client.prepareResponse(listOffsetResponse(tp0, Errors.NONE, ListOffsetsRequest.LATEST_TIMESTAMP, 5L)); assertEquals(singletonMap(tp0, 5L), fetcher.endOffsets(singleton(tp0), time.timer(5000L))); } @@ -4338,7 +4338,7 @@ public void testEndOffsets() { public void testEndOffsetsDuplicateTopicPartition() { buildFetcher(); assignFromUser(singleton(tp0)); - client.prepareResponse(listOffsetResponse(tp0, Errors.NONE, ListOffsetRequest.LATEST_TIMESTAMP, 5L)); + client.prepareResponse(listOffsetResponse(tp0, Errors.NONE, ListOffsetsRequest.LATEST_TIMESTAMP, 5L)); assertEquals(singletonMap(tp0, 5L), fetcher.endOffsets(asList(tp0, tp0), time.timer(5000L))); } @@ -4350,7 +4350,7 @@ public void testEndOffsetsMultipleTopicPartitions() { expectedOffsets.put(tp1, 7L); expectedOffsets.put(tp2, 9L); assignFromUser(expectedOffsets.keySet()); - client.prepareResponse(listOffsetResponse(expectedOffsets, Errors.NONE, ListOffsetRequest.LATEST_TIMESTAMP, ListOffsetResponse.UNKNOWN_EPOCH)); + client.prepareResponse(listOffsetResponse(expectedOffsets, Errors.NONE, ListOffsetsRequest.LATEST_TIMESTAMP, ListOffsetsResponse.UNKNOWN_EPOCH)); assertEquals(expectedOffsets, fetcher.endOffsets(asList(tp0, tp1, tp2), time.timer(5000L))); } @@ -4403,15 +4403,15 @@ private Map offsetForLeaderPartitionMa } private MockClient.RequestMatcher listOffsetRequestMatcher(final long timestamp) { - return listOffsetRequestMatcher(timestamp, ListOffsetResponse.UNKNOWN_EPOCH); + return listOffsetRequestMatcher(timestamp, ListOffsetsResponse.UNKNOWN_EPOCH); } private MockClient.RequestMatcher listOffsetRequestMatcher(final long timestamp, final int leaderEpoch) { // matches any list offset request with the provided timestamp return body -> { - ListOffsetRequest req = (ListOffsetRequest) body; - ListOffsetTopic topic = req.topics().get(0); - ListOffsetPartition partition = topic.partitions().get(0); + ListOffsetsRequest req = (ListOffsetsRequest) body; + ListOffsetsTopic topic = req.topics().get(0); + ListOffsetsPartition partition = topic.partitions().get(0); return tp0.topic().equals(topic.name()) && tp0.partition() == partition.partitionIndex() && timestamp == partition.timestamp() @@ -4419,40 +4419,40 @@ private MockClient.RequestMatcher listOffsetRequestMatcher(final long timestamp, }; } - private ListOffsetResponse listOffsetResponse(Errors error, long timestamp, long offset) { + private ListOffsetsResponse listOffsetResponse(Errors error, long timestamp, long offset) { return listOffsetResponse(tp0, error, timestamp, offset); } - private ListOffsetResponse listOffsetResponse(TopicPartition tp, Errors error, long timestamp, long offset) { - return listOffsetResponse(tp, error, timestamp, offset, ListOffsetResponse.UNKNOWN_EPOCH); + private ListOffsetsResponse listOffsetResponse(TopicPartition tp, Errors error, long timestamp, long offset) { + return listOffsetResponse(tp, error, timestamp, offset, ListOffsetsResponse.UNKNOWN_EPOCH); } - private ListOffsetResponse listOffsetResponse(TopicPartition tp, Errors error, long timestamp, long offset, int leaderEpoch) { + private ListOffsetsResponse listOffsetResponse(TopicPartition tp, Errors error, long timestamp, long offset, int leaderEpoch) { Map offsets = new HashMap<>(); offsets.put(tp, offset); return listOffsetResponse(offsets, error, timestamp, leaderEpoch); } - private ListOffsetResponse listOffsetResponse(Map offsets, Errors error, long timestamp, int leaderEpoch) { - Map> responses = new HashMap<>(); + private ListOffsetsResponse listOffsetResponse(Map offsets, Errors error, long timestamp, int leaderEpoch) { + Map> responses = new HashMap<>(); for (Map.Entry entry : offsets.entrySet()) { TopicPartition tp = entry.getKey(); responses.putIfAbsent(tp.topic(), new ArrayList<>()); - responses.get(tp.topic()).add(new ListOffsetPartitionResponse() + responses.get(tp.topic()).add(new ListOffsetsPartitionResponse() .setPartitionIndex(tp.partition()) .setErrorCode(error.code()) .setOffset(entry.getValue()) .setTimestamp(timestamp) .setLeaderEpoch(leaderEpoch)); } - List topics = new ArrayList<>(); - for (Map.Entry> response : responses.entrySet()) { - topics.add(new ListOffsetTopicResponse() + List topics = new ArrayList<>(); + for (Map.Entry> response : responses.entrySet()) { + topics.add(new ListOffsetsTopicResponse() .setName(response.getKey()) .setPartitions(response.getValue())); } - ListOffsetResponseData data = new ListOffsetResponseData().setTopics(topics); - return new ListOffsetResponse(data); + ListOffsetsResponseData data = new ListOffsetsResponseData().setTopics(topics); + return new ListOffsetsResponse(data); } private FetchResponse fullFetchResponseWithAbortedTransactions(MemoryRecords records, diff --git a/clients/src/test/java/org/apache/kafka/common/message/ApiMessageTypeTest.java b/clients/src/test/java/org/apache/kafka/common/message/ApiMessageTypeTest.java index 8fdb6a090ad96..2cb68fb7e70dd 100644 --- a/clients/src/test/java/org/apache/kafka/common/message/ApiMessageTypeTest.java +++ b/clients/src/test/java/org/apache/kafka/common/message/ApiMessageTypeTest.java @@ -18,8 +18,10 @@ package org.apache.kafka.common.message; import org.apache.kafka.common.errors.UnsupportedVersionException; +import org.apache.kafka.common.protocol.types.Schema; import org.junit.Rule; import org.junit.Test; +import org.junit.jupiter.api.Assertions; import org.junit.rules.Timeout; import java.util.HashSet; @@ -28,6 +30,8 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; public class ApiMessageTypeTest { @Rule @@ -94,4 +98,29 @@ public void testHeaderVersion() { assertEquals((short) 2, ApiMessageType.CREATE_TOPICS.requestHeaderVersion((short) 5)); assertEquals((short) 1, ApiMessageType.CREATE_TOPICS.responseHeaderVersion((short) 5)); } + + /** + * Kafka currently supports direct upgrades from 0.8 to the latest version. As such, it has to support all apis + * starting from version 0 and we must have schemas from the oldest version to the latest. + */ + @Test + public void testAllVersionsHaveSchemas() { + for (ApiMessageType type : ApiMessageType.values()) { + Assertions.assertEquals(0, type.lowestSupportedVersion()); + + assertEquals(type.requestSchemas().length, type.responseSchemas().length); + for (Schema schema : type.requestSchemas()) + assertNotNull(schema); + for (Schema schema : type.responseSchemas()) + assertNotNull(schema); + + assertEquals(type.highestSupportedVersion() + 1, type.requestSchemas().length); + } + } + + @Test + public void testApiIdsArePositive() { + for (ApiMessageType type : ApiMessageType.values()) + assertTrue(type.apiKey() >= 0); + } } diff --git a/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java b/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java index 52cf41584f70d..47ffb466bf869 100644 --- a/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java +++ b/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java @@ -28,10 +28,10 @@ import org.apache.kafka.common.message.DescribeGroupsResponseData.DescribedGroupMember; import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember; import org.apache.kafka.common.message.LeaveGroupResponseData.MemberResponse; -import org.apache.kafka.common.message.ListOffsetRequestData.ListOffsetPartition; -import org.apache.kafka.common.message.ListOffsetRequestData.ListOffsetTopic; -import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetPartitionResponse; -import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetTopicResponse; +import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition; +import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsTopic; +import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse; +import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse; import org.apache.kafka.common.message.OffsetCommitRequestData.OffsetCommitRequestPartition; import org.apache.kafka.common.message.OffsetCommitRequestData.OffsetCommitRequestTopic; import org.apache.kafka.common.message.OffsetCommitResponseData.OffsetCommitResponsePartition; @@ -48,14 +48,10 @@ import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Message; import org.apache.kafka.common.protocol.ObjectSerializationCache; -import org.apache.kafka.common.protocol.types.BoundField; import org.apache.kafka.common.protocol.types.RawTaggedField; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.SchemaException; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.protocol.types.Type; -import org.apache.kafka.common.utils.Utils; -import org.junit.Assert; import org.junit.Rule; import org.junit.Test; import org.junit.rules.Timeout; @@ -170,12 +166,12 @@ public void testJoinGroupRequestVersions() throws Exception { @Test public void testListOffsetsRequestVersions() throws Exception { - List v = Collections.singletonList(new ListOffsetTopic() + List v = Collections.singletonList(new ListOffsetsTopic() .setName("topic") - .setPartitions(Collections.singletonList(new ListOffsetPartition() + .setPartitions(Collections.singletonList(new ListOffsetsPartition() .setPartitionIndex(0) .setTimestamp(123L)))); - Supplier newRequest = () -> new ListOffsetRequestData() + Supplier newRequest = () -> new ListOffsetsRequestData() .setTopics(v) .setReplicaId(0); testAllMessageRoundTrips(newRequest.get()); @@ -184,17 +180,17 @@ public void testListOffsetsRequestVersions() throws Exception { @Test public void testListOffsetsResponseVersions() throws Exception { - ListOffsetPartitionResponse partition = new ListOffsetPartitionResponse() + ListOffsetsPartitionResponse partition = new ListOffsetsPartitionResponse() .setErrorCode(Errors.NONE.code()) .setPartitionIndex(0) .setOldStyleOffsets(Collections.singletonList(321L)); - List topics = Collections.singletonList(new ListOffsetTopicResponse() + List topics = Collections.singletonList(new ListOffsetsTopicResponse() .setName("topic") .setPartitions(Collections.singletonList(partition))); - Supplier response = () -> new ListOffsetResponseData() + Supplier response = () -> new ListOffsetsResponseData() .setTopics(topics); for (short version = 0; version <= ApiKeys.LIST_OFFSETS.latestVersion(); version++) { - ListOffsetResponseData responseData = response.get(); + ListOffsetsResponseData responseData = response.get(); if (version > 0) { responseData.topics().get(0).partitions().get(0) .setOldStyleOffsets(Collections.emptyList()) @@ -879,52 +875,6 @@ public void testMessageVersions() { } } - /** - * Test that the JSON request files match the schemas accessible through the ApiKey class. - */ - @Test - public void testRequestSchemas() { - for (ApiKeys apiKey : ApiKeys.values()) { - Schema[] manualSchemas = apiKey.requestSchemas; - Schema[] generatedSchemas = ApiMessageType.fromApiKey(apiKey.id).requestSchemas(); - Assert.assertEquals("Mismatching request SCHEMAS lengths " + - "for api key " + apiKey, manualSchemas.length, generatedSchemas.length); - for (int v = 0; v < manualSchemas.length; v++) { - try { - if (generatedSchemas[v] != null) { - compareTypes(manualSchemas[v], generatedSchemas[v]); - } - } catch (Exception e) { - throw new RuntimeException("Failed to compare request schemas " + - "for version " + v + " of " + apiKey, e); - } - } - } - } - - /** - * Test that the JSON response files match the schemas accessible through the ApiKey class. - */ - @Test - public void testResponseSchemas() { - for (ApiKeys apiKey : ApiKeys.values()) { - Schema[] manualSchemas = apiKey.responseSchemas; - Schema[] generatedSchemas = ApiMessageType.fromApiKey(apiKey.id).responseSchemas(); - Assert.assertEquals("Mismatching response SCHEMAS lengths " + - "for api key " + apiKey, manualSchemas.length, generatedSchemas.length); - for (int v = 0; v < manualSchemas.length; v++) { - try { - if (generatedSchemas[v] != null) { - compareTypes(manualSchemas[v], generatedSchemas[v]); - } - } catch (Exception e) { - throw new RuntimeException("Failed to compare response schemas " + - "for version " + v + " of " + apiKey, e); - } - } - } - } - private static class NamedType { final String name; final Type type; @@ -952,59 +902,6 @@ public String toString() { } } - private static void compareTypes(Schema schemaA, Schema schemaB) { - compareTypes(new NamedType("schemaA", schemaA), - new NamedType("schemaB", schemaB)); - } - - private static void compareTypes(NamedType typeA, NamedType typeB) { - List listA = flatten(typeA); - List listB = flatten(typeB); - if (listA.size() != listB.size()) { - throw new RuntimeException("Can't match up structures: typeA has " + - Utils.join(listA, ", ") + ", but typeB has " + - Utils.join(listB, ", ")); - } - for (int i = 0; i < listA.size(); i++) { - NamedType entryA = listA.get(i); - NamedType entryB = listB.get(i); - if (!entryA.hasSimilarType(entryB)) { - throw new RuntimeException("Type " + entryA + " in schema A " + - "does not match type " + entryB + " in schema B."); - } - if (entryA.type.isNullable() != entryB.type.isNullable()) { - throw new RuntimeException(String.format( - "Type %s in Schema A is %s, but type %s in " + - "Schema B is %s", - entryA, entryA.type.isNullable() ? "nullable" : "non-nullable", - entryB, entryB.type.isNullable() ? "nullable" : "non-nullable")); - } - if (entryA.type.isArray()) { - compareTypes(new NamedType(entryA.name, entryA.type.arrayElementType().get()), - new NamedType(entryB.name, entryB.type.arrayElementType().get())); - } - } - } - - /** - * We want to remove Schema nodes from the hierarchy before doing - * our comparison. The reason is because Schema nodes don't change what - * is written to the wire. Schema(STRING, Schema(INT, SHORT)) is equivalent to - * Schema(STRING, INT, SHORT). This function translates schema nodes into their - * component types. - */ - private static List flatten(NamedType type) { - if (!(type.type instanceof Schema)) { - return singletonList(type); - } - Schema schema = (Schema) type.type; - ArrayList results = new ArrayList<>(); - for (BoundField field : schema.fields()) { - results.addAll(flatten(new NamedType(field.def.name, field.def.type))); - } - return results; - } - @Test public void testDefaultValues() { verifyWriteRaisesUve((short) 0, "validateOnly", diff --git a/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java b/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java index 5634fd202f732..fd0c097fe2dcc 100644 --- a/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java +++ b/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java @@ -39,11 +39,6 @@ public void testForIdWithInvalidIdHigh() { ApiKeys.forId(10000); } - @Test(expected = IllegalArgumentException.class) - public void schemaVersionOutOfRange() { - ApiKeys.PRODUCE.requestSchema((short) ApiKeys.PRODUCE.requestSchemas.length); - } - @Test public void testAlterIsrIsClusterAction() { assertTrue(ApiKeys.ALTER_ISR.clusterAction); @@ -65,8 +60,8 @@ public void testResponseThrottleTime() { // Newer protocol apis include throttle time ms even for cluster actions Set clusterActionsWithThrottleTimeMs = EnumSet.of(ApiKeys.ALTER_ISR); for (ApiKeys apiKey: ApiKeys.values()) { - Schema responseSchema = apiKey.responseSchema(apiKey.latestVersion()); - BoundField throttleTimeField = responseSchema.get(CommonFields.THROTTLE_TIME_MS.name); + Schema responseSchema = apiKey.messageType.responseSchemas()[apiKey.latestVersion()]; + BoundField throttleTimeField = responseSchema.get("throttle_time_ms"); if ((apiKey.clusterAction && !clusterActionsWithThrottleTimeMs.contains(apiKey)) || authenticationKeys.contains(apiKey)) assertNull("Unexpected throttle time field: " + apiKey, throttleTimeField); 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 b578e2c39db69..957580e2f7d2b 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 @@ -53,14 +53,18 @@ public void shouldHaveCorrectDefaultApiVersionsResponse() { // Check if versions less than min version are indeed set as null, i.e., deprecated. for (int i = 0; i < version.minVersion(); ++i) { - assertNull("Request version " + i + " for API " + version.apiKey() + " must be null", key.requestSchemas[i]); - assertNull("Response version " + i + " for API " + version.apiKey() + " must be null", key.responseSchemas[i]); + assertNull("Request version " + i + " for API " + version.apiKey() + " must be null", + key.messageType.requestSchemas()[i]); + assertNull("Response version " + i + " for API " + version.apiKey() + " must be null", + key.messageType.responseSchemas()[i]); } // Check if versions between min and max versions are non null, i.e., valid. for (int i = version.minVersion(); i <= version.maxVersion(); ++i) { - assertNotNull("Request version " + i + " for API " + version.apiKey() + " must not be null", key.requestSchemas[i]); - assertNotNull("Response version " + i + " for API " + version.apiKey() + " must not be null", key.responseSchemas[i]); + assertNotNull("Request version " + i + " for API " + version.apiKey() + " must not be null", + key.messageType.requestSchemas()[i]); + assertNotNull("Response version " + i + " for API " + version.apiKey() + " must not be null", + key.messageType.responseSchemas()[i]); } } diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ListOffsetRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ListOffsetsRequestTest.java similarity index 59% rename from clients/src/test/java/org/apache/kafka/common/requests/ListOffsetRequestTest.java rename to clients/src/test/java/org/apache/kafka/common/requests/ListOffsetsRequestTest.java index 80af40e653332..da6fec3b44db9 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/ListOffsetRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/ListOffsetsRequestTest.java @@ -27,65 +27,65 @@ import org.apache.kafka.common.IsolationLevel; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.message.ListOffsetRequestData; -import org.apache.kafka.common.message.ListOffsetRequestData.ListOffsetPartition; -import org.apache.kafka.common.message.ListOffsetRequestData.ListOffsetTopic; -import org.apache.kafka.common.message.ListOffsetResponseData; -import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetPartitionResponse; -import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetTopicResponse; +import org.apache.kafka.common.message.ListOffsetsRequestData; +import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition; +import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsTopic; +import org.apache.kafka.common.message.ListOffsetsResponseData; +import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse; +import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.MessageUtil; import org.junit.Test; -public class ListOffsetRequestTest { +public class ListOffsetsRequestTest { @Test public void testDuplicatePartitions() { - List topics = Collections.singletonList( - new ListOffsetTopic() + List topics = Collections.singletonList( + new ListOffsetsTopic() .setName("topic") .setPartitions(Arrays.asList( - new ListOffsetPartition() + new ListOffsetsPartition() .setPartitionIndex(0), - new ListOffsetPartition() + new ListOffsetsPartition() .setPartitionIndex(0)))); - ListOffsetRequestData data = new ListOffsetRequestData() + ListOffsetsRequestData data = new ListOffsetsRequestData() .setTopics(topics) .setReplicaId(-1); - ListOffsetRequest request = ListOffsetRequest.parse(MessageUtil.toByteBuffer(data, (short) 0), (short) 0); + ListOffsetsRequest request = ListOffsetsRequest.parse(MessageUtil.toByteBuffer(data, (short) 0), (short) 0); assertEquals(Collections.singleton(new TopicPartition("topic", 0)), request.duplicatePartitions()); } @Test public void testGetErrorResponse() { for (short version = 1; version <= ApiKeys.LIST_OFFSETS.latestVersion(); version++) { - List topics = Arrays.asList( - new ListOffsetTopic() + List topics = Arrays.asList( + new ListOffsetsTopic() .setName("topic") .setPartitions(Collections.singletonList( - new ListOffsetPartition() + new ListOffsetsPartition() .setPartitionIndex(0)))); - ListOffsetRequest request = ListOffsetRequest.Builder + ListOffsetsRequest request = ListOffsetsRequest.Builder .forConsumer(true, IsolationLevel.READ_COMMITTED) .setTargetTimes(topics) .build(version); - ListOffsetResponse response = (ListOffsetResponse) request.getErrorResponse(0, Errors.NOT_LEADER_OR_FOLLOWER.exception()); + ListOffsetsResponse response = (ListOffsetsResponse) request.getErrorResponse(0, Errors.NOT_LEADER_OR_FOLLOWER.exception()); - List v = Collections.singletonList( - new ListOffsetTopicResponse() + List v = Collections.singletonList( + new ListOffsetsTopicResponse() .setName("topic") .setPartitions(Collections.singletonList( - new ListOffsetPartitionResponse() + new ListOffsetsPartitionResponse() .setErrorCode(Errors.NOT_LEADER_OR_FOLLOWER.code()) - .setLeaderEpoch(ListOffsetResponse.UNKNOWN_EPOCH) - .setOffset(ListOffsetResponse.UNKNOWN_OFFSET) + .setLeaderEpoch(ListOffsetsResponse.UNKNOWN_EPOCH) + .setOffset(ListOffsetsResponse.UNKNOWN_OFFSET) .setPartitionIndex(0) - .setTimestamp(ListOffsetResponse.UNKNOWN_TIMESTAMP)))); - ListOffsetResponseData data = new ListOffsetResponseData() + .setTimestamp(ListOffsetsResponse.UNKNOWN_TIMESTAMP)))); + ListOffsetsResponseData data = new ListOffsetsResponseData() .setThrottleTimeMs(0) .setTopics(v); - ListOffsetResponse expectedResponse = new ListOffsetResponse(data); + ListOffsetsResponse expectedResponse = new ListOffsetsResponse(data); assertEquals(expectedResponse.data().topics(), response.data().topics()); assertEquals(expectedResponse.throttleTimeMs(), response.throttleTimeMs()); } @@ -93,52 +93,52 @@ public void testGetErrorResponse() { @Test public void testGetErrorResponseV0() { - List topics = Arrays.asList( - new ListOffsetTopic() + List topics = Arrays.asList( + new ListOffsetsTopic() .setName("topic") .setPartitions(Collections.singletonList( - new ListOffsetPartition() + new ListOffsetsPartition() .setPartitionIndex(0)))); - ListOffsetRequest request = ListOffsetRequest.Builder + ListOffsetsRequest request = ListOffsetsRequest.Builder .forConsumer(true, IsolationLevel.READ_UNCOMMITTED) .setTargetTimes(topics) .build((short) 0); - ListOffsetResponse response = (ListOffsetResponse) request.getErrorResponse(0, Errors.NOT_LEADER_OR_FOLLOWER.exception()); + ListOffsetsResponse response = (ListOffsetsResponse) request.getErrorResponse(0, Errors.NOT_LEADER_OR_FOLLOWER.exception()); - List v = Collections.singletonList( - new ListOffsetTopicResponse() + List v = Collections.singletonList( + new ListOffsetsTopicResponse() .setName("topic") .setPartitions(Collections.singletonList( - new ListOffsetPartitionResponse() + new ListOffsetsPartitionResponse() .setErrorCode(Errors.NOT_LEADER_OR_FOLLOWER.code()) .setOldStyleOffsets(Collections.emptyList()) .setPartitionIndex(0)))); - ListOffsetResponseData data = new ListOffsetResponseData() + ListOffsetsResponseData data = new ListOffsetsResponseData() .setThrottleTimeMs(0) .setTopics(v); - ListOffsetResponse expectedResponse = new ListOffsetResponse(data); + ListOffsetsResponse expectedResponse = new ListOffsetsResponse(data); assertEquals(expectedResponse.data().topics(), response.data().topics()); assertEquals(expectedResponse.throttleTimeMs(), response.throttleTimeMs()); } @Test - public void testToListOffsetTopics() { - ListOffsetPartition lop0 = new ListOffsetPartition() + public void testToListOffsetsTopics() { + ListOffsetsPartition lop0 = new ListOffsetsPartition() .setPartitionIndex(0) .setCurrentLeaderEpoch(1) .setMaxNumOffsets(2) .setTimestamp(123L); - ListOffsetPartition lop1 = new ListOffsetPartition() + ListOffsetsPartition lop1 = new ListOffsetsPartition() .setPartitionIndex(1) .setCurrentLeaderEpoch(3) .setMaxNumOffsets(4) .setTimestamp(567L); - Map timestampsToSearch = new HashMap<>(); + Map timestampsToSearch = new HashMap<>(); timestampsToSearch.put(new TopicPartition("topic", 0), lop0); timestampsToSearch.put(new TopicPartition("topic", 1), lop1); - List listOffsetTopics = ListOffsetRequest.toListOffsetTopics(timestampsToSearch); + List listOffsetTopics = ListOffsetsRequest.toListOffsetsTopics(timestampsToSearch); assertEquals(1, listOffsetTopics.size()); - ListOffsetTopic topic = listOffsetTopics.get(0); + ListOffsetsTopic topic = listOffsetTopics.get(0); assertEquals("topic", topic.name()); assertEquals(2, topic.partitions().size()); assertTrue(topic.partitions().contains(lop0)); 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 06e611f49a6da..e4ea98a4589f6 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 @@ -34,7 +34,6 @@ import java.util.Map; import java.util.Optional; -import static org.apache.kafka.common.protocol.CommonFields.ERROR_CODE; import static org.apache.kafka.common.requests.AbstractResponse.DEFAULT_THROTTLE_TIME; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -42,6 +41,8 @@ public class OffsetFetchResponseTest { + private static final String ERROR_CODE = "error_code"; + private final int throttleTimeMs = 10; private final int offset = 100; private final String metadata = "metadata"; 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..886cd2a3c7434 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 @@ -116,11 +116,11 @@ import org.apache.kafka.common.message.LeaveGroupResponseData; import org.apache.kafka.common.message.ListGroupsRequestData; import org.apache.kafka.common.message.ListGroupsResponseData; -import org.apache.kafka.common.message.ListOffsetRequestData.ListOffsetPartition; -import org.apache.kafka.common.message.ListOffsetRequestData.ListOffsetTopic; -import org.apache.kafka.common.message.ListOffsetResponseData; -import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetPartitionResponse; -import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetTopicResponse; +import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition; +import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsTopic; +import org.apache.kafka.common.message.ListOffsetsResponseData; +import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse; +import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse; import org.apache.kafka.common.message.ListPartitionReassignmentsRequestData; import org.apache.kafka.common.message.ListPartitionReassignmentsResponseData; import org.apache.kafka.common.message.OffsetCommitRequestData; @@ -1255,40 +1255,40 @@ private DeleteGroupsResponse createDeleteGroupsResponse() { ); } - private ListOffsetRequest createListOffsetRequest(int version) { + private ListOffsetsRequest createListOffsetRequest(int version) { if (version == 0) { - ListOffsetTopic topic = new ListOffsetTopic() + ListOffsetsTopic topic = new ListOffsetsTopic() .setName("test") - .setPartitions(Arrays.asList(new ListOffsetPartition() + .setPartitions(Arrays.asList(new ListOffsetsPartition() .setPartitionIndex(0) .setTimestamp(1000000L) .setMaxNumOffsets(10) .setCurrentLeaderEpoch(5))); - return ListOffsetRequest.Builder + return ListOffsetsRequest.Builder .forConsumer(false, IsolationLevel.READ_UNCOMMITTED) .setTargetTimes(Collections.singletonList(topic)) .build((short) version); } else if (version == 1) { - ListOffsetTopic topic = new ListOffsetTopic() + ListOffsetsTopic topic = new ListOffsetsTopic() .setName("test") - .setPartitions(Arrays.asList(new ListOffsetPartition() + .setPartitions(Arrays.asList(new ListOffsetsPartition() .setPartitionIndex(0) .setTimestamp(1000000L) .setCurrentLeaderEpoch(5))); - return ListOffsetRequest.Builder + return ListOffsetsRequest.Builder .forConsumer(true, IsolationLevel.READ_UNCOMMITTED) .setTargetTimes(Collections.singletonList(topic)) .build((short) version); } else if (version >= 2 && version <= LIST_OFFSETS.latestVersion()) { - ListOffsetPartition partition = new ListOffsetPartition() + ListOffsetsPartition partition = new ListOffsetsPartition() .setPartitionIndex(0) .setTimestamp(1000000L) .setCurrentLeaderEpoch(5); - ListOffsetTopic topic = new ListOffsetTopic() + ListOffsetsTopic topic = new ListOffsetsTopic() .setName("test") .setPartitions(Arrays.asList(partition)); - return ListOffsetRequest.Builder + return ListOffsetsRequest.Builder .forConsumer(true, IsolationLevel.READ_COMMITTED) .setTargetTimes(Collections.singletonList(topic)) .build((short) version); @@ -1297,18 +1297,18 @@ private ListOffsetRequest createListOffsetRequest(int version) { } } - private ListOffsetResponse createListOffsetResponse(int version) { + private ListOffsetsResponse createListOffsetResponse(int version) { if (version == 0) { - ListOffsetResponseData data = new ListOffsetResponseData() - .setTopics(Collections.singletonList(new ListOffsetTopicResponse() + ListOffsetsResponseData data = new ListOffsetsResponseData() + .setTopics(Collections.singletonList(new ListOffsetsTopicResponse() .setName("test") - .setPartitions(Collections.singletonList(new ListOffsetPartitionResponse() + .setPartitions(Collections.singletonList(new ListOffsetsPartitionResponse() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code()) .setOldStyleOffsets(asList(100L)))))); - return new ListOffsetResponse(data); + return new ListOffsetsResponse(data); } else if (version >= 1 && version <= LIST_OFFSETS.latestVersion()) { - ListOffsetPartitionResponse partition = new ListOffsetPartitionResponse() + ListOffsetsPartitionResponse partition = new ListOffsetsPartitionResponse() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code()) .setTimestamp(10000L) @@ -1316,11 +1316,11 @@ private ListOffsetResponse createListOffsetResponse(int version) { if (version >= 4) { partition.setLeaderEpoch(27); } - ListOffsetResponseData data = new ListOffsetResponseData() - .setTopics(Collections.singletonList(new ListOffsetTopicResponse() + ListOffsetsResponseData data = new ListOffsetsResponseData() + .setTopics(Collections.singletonList(new ListOffsetsTopicResponse() .setName("test") .setPartitions(Collections.singletonList(partition)))); - return new ListOffsetResponse(data); + return new ListOffsetsResponse(data); } else { throw new IllegalArgumentException("Illegal ListOffsetResponse version " + version); } 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 34ee542a5c6cd..195d9599ce040 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 @@ -60,9 +60,9 @@ import org.apache.kafka.common.message.ApiVersionsResponseData; import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersionsResponseKey; import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersionsResponseKeyCollection; -import org.apache.kafka.common.message.ListOffsetResponseData; -import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetTopicResponse; -import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetPartitionResponse; +import org.apache.kafka.common.message.ListOffsetsResponseData; +import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse; +import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse; import org.apache.kafka.common.message.RequestHeaderData; import org.apache.kafka.common.message.SaslAuthenticateRequestData; import org.apache.kafka.common.message.SaslHandshakeRequestData; @@ -83,7 +83,7 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.types.SchemaException; -import org.apache.kafka.common.requests.ListOffsetResponse; +import org.apache.kafka.common.requests.ListOffsetsResponse; import org.apache.kafka.common.requests.RequestTestUtils; import org.apache.kafka.common.security.auth.Login; import org.apache.kafka.common.security.auth.SecurityProtocol; @@ -1593,17 +1593,17 @@ SaslClient createSaslClient() { @Test public void testConvertListOffsetResponseToSaslHandshakeResponse() { - ListOffsetResponseData data = new ListOffsetResponseData() + ListOffsetsResponseData data = new ListOffsetsResponseData() .setThrottleTimeMs(0) - .setTopics(Collections.singletonList(new ListOffsetTopicResponse() + .setTopics(Collections.singletonList(new ListOffsetsTopicResponse() .setName("topic") - .setPartitions(Collections.singletonList(new ListOffsetPartitionResponse() + .setPartitions(Collections.singletonList(new ListOffsetsPartitionResponse() .setErrorCode(Errors.NONE.code()) - .setLeaderEpoch(ListOffsetResponse.UNKNOWN_EPOCH) + .setLeaderEpoch(ListOffsetsResponse.UNKNOWN_EPOCH) .setPartitionIndex(0) .setOffset(0) .setTimestamp(0))))); - ListOffsetResponse response = new ListOffsetResponse(data); + ListOffsetsResponse response = new ListOffsetsResponse(data); ByteBuffer buffer = RequestTestUtils.serializeResponseWithHeader(response, LIST_OFFSETS.latestVersion(), 0); final RequestHeader header0 = new RequestHeader(LIST_OFFSETS, LIST_OFFSETS.latestVersion(), "id", SaslClientAuthenticator.MIN_RESERVED_CORRELATION_ID); Assert.assertThrows(SchemaException.class, () -> NetworkClient.parseResponse(buffer.duplicate(), header0)); diff --git a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala index d8a01abe1cbf2..2fc55bd7b6611 100755 --- a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala +++ b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala @@ -19,7 +19,6 @@ package kafka.admin import java.time.{Duration, Instant} import java.util.Properties - import com.fasterxml.jackson.dataformat.csv.CsvMapper import com.fasterxml.jackson.module.scala.DefaultScalaModule import com.fasterxml.jackson.module.scala.experimental.ScalaObjectMapper @@ -40,9 +39,9 @@ import org.apache.kafka.common.protocol.Errors import scala.collection.immutable.TreeMap import scala.reflect.ClassTag -import org.apache.kafka.common.requests.ListOffsetResponse import org.apache.kafka.common.ConsumerGroupState import joptsimple.OptionException +import org.apache.kafka.common.requests.ListOffsetsResponse object ConsumerGroupCommand extends Logging { @@ -679,7 +678,7 @@ object ConsumerGroupCommand extends Logging { withTimeoutMs(new ListOffsetsOptions) ).all.get val (successfulOffsetsForTimes, unsuccessfulOffsetsForTimes) = - offsets.asScala.partition(_._2.offset != ListOffsetResponse.UNKNOWN_OFFSET) + offsets.asScala.partition(_._2.offset != ListOffsetsResponse.UNKNOWN_OFFSET) val successfulLogTimestampOffsets = successfulOffsetsForTimes.map { case (topicPartition, listOffsetsResultInfo) => topicPartition -> LogOffsetResult.LogOffset(listOffsetsResultInfo.offset) diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index f47eac69b9cb9..c0a7d76e508a0 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -1188,10 +1188,10 @@ class Partition(val topicPartition: TopicPartition, // If we're in the lagging HW state after a leader election, throw OffsetNotAvailable for "latest" offset // or for a timestamp lookup that is beyond the last fetchable offset. timestamp match { - case ListOffsetRequest.LATEST_TIMESTAMP => + case ListOffsetsRequest.LATEST_TIMESTAMP => maybeOffsetsError.map(e => throw e) .orElse(Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, lastFetchableOffset, Optional.of(leaderEpoch)))) - case ListOffsetRequest.EARLIEST_TIMESTAMP => + case ListOffsetsRequest.EARLIEST_TIMESTAMP => getOffsetByTimestamp case _ => getOffsetByTimestamp.filter(timestampAndOffset => timestampAndOffset.offset < lastFetchableOffset) diff --git a/core/src/main/scala/kafka/log/IndexEntry.scala b/core/src/main/scala/kafka/log/IndexEntry.scala index c8dd200030457..705366e32f57d 100644 --- a/core/src/main/scala/kafka/log/IndexEntry.scala +++ b/core/src/main/scala/kafka/log/IndexEntry.scala @@ -17,7 +17,7 @@ package kafka.log -import org.apache.kafka.common.requests.ListOffsetResponse +import org.apache.kafka.common.requests.ListOffsetsResponse sealed trait IndexEntry { // We always use Long for both key and value to avoid boxing. @@ -48,5 +48,5 @@ case class TimestampOffset(timestamp: Long, offset: Long) extends IndexEntry { } object TimestampOffset { - val Unknown = TimestampOffset(ListOffsetResponse.UNKNOWN_TIMESTAMP, ListOffsetResponse.UNKNOWN_OFFSET) + val Unknown = TimestampOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, ListOffsetsResponse.UNKNOWN_OFFSET) } diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index d28f32c030aab..438f2342be762 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -26,7 +26,6 @@ import java.util.Optional import java.util.concurrent.atomic._ import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap, TimeUnit} import java.util.regex.Pattern - import kafka.api.{ApiVersion, KAFKA_0_10_0_IV0} import kafka.common.{LogSegmentOffsetOverflowException, LongRef, OffsetsOutOfOrderException, UnexpectedAppendOffsetException} import kafka.message.{BrokerCompressionCodec, CompressionCodec, NoCompressionCodec} @@ -40,9 +39,9 @@ import org.apache.kafka.common.message.FetchResponseData import org.apache.kafka.common.record.FileRecords.TimestampAndOffset import org.apache.kafka.common.record._ import org.apache.kafka.common.requests.FetchResponse.AbortedTransaction +import org.apache.kafka.common.requests.ListOffsetsRequest import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET import org.apache.kafka.common.requests.ProduceResponse.RecordError -import org.apache.kafka.common.requests.ListOffsetRequest import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{InvalidRecordException, KafkaException, TopicPartition} @@ -1610,8 +1609,8 @@ class Log(@volatile private var _dir: File, debug(s"Searching offset for timestamp $targetTimestamp") if (config.messageFormatVersion < KAFKA_0_10_0_IV0 && - targetTimestamp != ListOffsetRequest.EARLIEST_TIMESTAMP && - targetTimestamp != ListOffsetRequest.LATEST_TIMESTAMP) + targetTimestamp != ListOffsetsRequest.EARLIEST_TIMESTAMP && + targetTimestamp != ListOffsetsRequest.LATEST_TIMESTAMP) throw new UnsupportedForMessageFormatException(s"Cannot search offsets based on timestamp because message format version " + s"for partition $topicPartition is ${config.messageFormatVersion} which is earlier than the minimum " + s"required version $KAFKA_0_10_0_IV0") @@ -1620,7 +1619,7 @@ class Log(@volatile private var _dir: File, // constant time access while being safe to use with concurrent collections unlike `toArray`. val segmentsCopy = logSegments.toBuffer // For the earliest and latest, we do not need to return the timestamp. - if (targetTimestamp == ListOffsetRequest.EARLIEST_TIMESTAMP) { + if (targetTimestamp == ListOffsetsRequest.EARLIEST_TIMESTAMP) { // The first cached epoch usually corresponds to the log start offset, but we have to verify this since // it may not be true following a message format version bump as the epoch will not be available for // log entries written in the older format. @@ -1630,7 +1629,7 @@ class Log(@volatile private var _dir: File, case _ => Optional.empty[Integer]() } Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, logStartOffset, epochOpt)) - } else if (targetTimestamp == ListOffsetRequest.LATEST_TIMESTAMP) { + } else if (targetTimestamp == ListOffsetsRequest.LATEST_TIMESTAMP) { val latestEpochOpt = leaderEpochCache.flatMap(_.latestEpoch).map(_.asInstanceOf[Integer]) val epochOptional = Optional.ofNullable(latestEpochOpt.orNull) Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, logEndOffset, epochOptional)) @@ -1661,9 +1660,9 @@ class Log(@volatile private var _dir: File, var startIndex = -1 timestamp match { - case ListOffsetRequest.LATEST_TIMESTAMP => + case ListOffsetsRequest.LATEST_TIMESTAMP => startIndex = offsetTimeArray.length - 1 - case ListOffsetRequest.EARLIEST_TIMESTAMP => + case ListOffsetsRequest.EARLIEST_TIMESTAMP => startIndex = 0 case _ => var isFound = false diff --git a/core/src/main/scala/kafka/network/RequestConvertToJson.scala b/core/src/main/scala/kafka/network/RequestConvertToJson.scala index 243c2a4ca50ed..f610df9846b51 100644 --- a/core/src/main/scala/kafka/network/RequestConvertToJson.scala +++ b/core/src/main/scala/kafka/network/RequestConvertToJson.scala @@ -68,7 +68,7 @@ object RequestConvertToJson { 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: ListOffsetsRequest => ListOffsetsRequestDataJsonConverter.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) @@ -134,7 +134,7 @@ object RequestConvertToJson { 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: ListOffsetsResponse => ListOffsetsResponseDataJsonConverter.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) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 3e01fb5f5581d..d6fb2fb78789e 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -50,7 +50,7 @@ import org.apache.kafka.common.internals.{FatalExitError, Topic} import org.apache.kafka.common.internals.Topic.{GROUP_METADATA_TOPIC_NAME, TRANSACTION_STATE_TOPIC_NAME, isInternal} import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic import org.apache.kafka.common.message.CreatePartitionsResponseData.CreatePartitionsTopicResult -import org.apache.kafka.common.message.{AddOffsetsToTxnResponseData, AlterClientQuotasResponseData, AlterConfigsResponseData, AlterPartitionReassignmentsResponseData, AlterReplicaLogDirsResponseData, CreateAclsResponseData, CreatePartitionsResponseData, CreateTopicsResponseData, DeleteAclsResponseData, DeleteGroupsResponseData, DeleteRecordsResponseData, DeleteTopicsResponseData, DescribeAclsResponseData, DescribeClientQuotasResponseData, DescribeConfigsResponseData, DescribeGroupsResponseData, DescribeLogDirsResponseData, EndTxnResponseData, ExpireDelegationTokenResponseData, FindCoordinatorResponseData, HeartbeatResponseData, InitProducerIdResponseData, JoinGroupResponseData, LeaveGroupResponseData, ListGroupsResponseData, ListOffsetResponseData, ListPartitionReassignmentsResponseData, MetadataResponseData, OffsetCommitRequestData, OffsetCommitResponseData, OffsetDeleteResponseData, RenewDelegationTokenResponseData, SaslAuthenticateResponseData, SaslHandshakeResponseData, StopReplicaResponseData, SyncGroupResponseData, UpdateMetadataResponseData} +import org.apache.kafka.common.message.{AddOffsetsToTxnResponseData, AlterClientQuotasResponseData, AlterConfigsResponseData, AlterPartitionReassignmentsResponseData, AlterReplicaLogDirsResponseData, CreateAclsResponseData, CreatePartitionsResponseData, CreateTopicsResponseData, DeleteAclsResponseData, DeleteGroupsResponseData, DeleteRecordsResponseData, DeleteTopicsResponseData, DescribeAclsResponseData, DescribeClientQuotasResponseData, DescribeConfigsResponseData, DescribeGroupsResponseData, DescribeLogDirsResponseData, EndTxnResponseData, ExpireDelegationTokenResponseData, FindCoordinatorResponseData, HeartbeatResponseData, InitProducerIdResponseData, JoinGroupResponseData, LeaveGroupResponseData, ListGroupsResponseData, ListOffsetsResponseData, ListPartitionReassignmentsResponseData, MetadataResponseData, OffsetCommitRequestData, OffsetCommitResponseData, OffsetDeleteResponseData, RenewDelegationTokenResponseData, SaslAuthenticateResponseData, SaslHandshakeResponseData, StopReplicaResponseData, SyncGroupResponseData, UpdateMetadataResponseData} import org.apache.kafka.common.message.CreateTopicsResponseData.{CreatableTopicResult, CreatableTopicResultCollection} import org.apache.kafka.common.message.DeleteGroupsResponseData.{DeletableGroupResult, DeletableGroupResultCollection} import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData.{ReassignablePartitionResponse, ReassignableTopicResponse} @@ -60,8 +60,8 @@ import org.apache.kafka.common.message.DeleteRecordsResponseData.{DeleteRecordsP import org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult import org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult 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.ListOffsetsRequestData.ListOffsetsPartition +import org.apache.kafka.common.message.ListOffsetsResponseData.{ListOffsetsPartitionResponse, ListOffsetsTopicResponse} import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderTopic import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.{EpochEndOffset, OffsetForLeaderTopicResult, OffsetForLeaderTopicResultCollection} @@ -1019,24 +1019,24 @@ class KafkaApis(val requestChannel: RequestChannel, else handleListOffsetRequestV1AndAbove(request) - sendResponseMaybeThrottle(request, requestThrottleMs => new ListOffsetResponse(new ListOffsetResponseData() + sendResponseMaybeThrottle(request, requestThrottleMs => new ListOffsetsResponse(new ListOffsetsResponseData() .setThrottleTimeMs(requestThrottleMs) .setTopics(topics.asJava))) } - private def handleListOffsetRequestV0(request : RequestChannel.Request) : List[ListOffsetTopicResponse] = { + private def handleListOffsetRequestV0(request : RequestChannel.Request) : List[ListOffsetsTopicResponse] = { val correlationId = request.header.correlationId val clientId = request.header.clientId - val offsetRequest = request.body[ListOffsetRequest] + val offsetRequest = request.body[ListOffsetsRequest] val (authorizedRequestInfo, unauthorizedRequestInfo) = partitionSeqByAuthorized(request.context, DESCRIBE, TOPIC, offsetRequest.topics.asScala.toSeq)(_.name) val unauthorizedResponseStatus = unauthorizedRequestInfo.map(topic => - new ListOffsetTopicResponse() + new ListOffsetsTopicResponse() .setName(topic.name) .setPartitions(topic.partitions.asScala.map(partition => - new ListOffsetPartitionResponse() + new ListOffsetsPartitionResponse() .setPartitionIndex(partition.partitionIndex) .setErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code)).asJava) ) @@ -1050,9 +1050,9 @@ class KafkaApis(val requestChannel: RequestChannel, topicPartition = topicPartition, timestamp = partition.timestamp, maxNumOffsets = partition.maxNumOffsets, - isFromConsumer = offsetRequest.replicaId == ListOffsetRequest.CONSUMER_REPLICA_ID, - fetchOnlyFromLeader = offsetRequest.replicaId != ListOffsetRequest.DEBUGGING_REPLICA_ID) - new ListOffsetPartitionResponse() + isFromConsumer = offsetRequest.replicaId == ListOffsetsRequest.CONSUMER_REPLICA_ID, + fetchOnlyFromLeader = offsetRequest.replicaId != ListOffsetsRequest.DEBUGGING_REPLICA_ID) + new ListOffsetsPartitionResponse() .setPartitionIndex(partition.partitionIndex) .setErrorCode(Errors.NONE.code) .setOldStyleOffsets(offsets.map(JLong.valueOf).asJava) @@ -1064,40 +1064,40 @@ class KafkaApis(val requestChannel: RequestChannel, _ : KafkaStorageException) => debug("Offset request with correlation id %d from client %s on partition %s failed due to %s".format( correlationId, clientId, topicPartition, e.getMessage)) - new ListOffsetPartitionResponse() + new ListOffsetsPartitionResponse() .setPartitionIndex(partition.partitionIndex) .setErrorCode(Errors.forException(e).code) case e: Throwable => error("Error while responding to offset request", e) - new ListOffsetPartitionResponse() + new ListOffsetsPartitionResponse() .setPartitionIndex(partition.partitionIndex) .setErrorCode(Errors.forException(e).code) } } - new ListOffsetTopicResponse().setName(topic.name).setPartitions(responsePartitions.asJava) + new ListOffsetsTopicResponse().setName(topic.name).setPartitions(responsePartitions.asJava) } (responseTopics ++ unauthorizedResponseStatus).toList } - private def handleListOffsetRequestV1AndAbove(request : RequestChannel.Request): List[ListOffsetTopicResponse] = { + private def handleListOffsetRequestV1AndAbove(request : RequestChannel.Request): List[ListOffsetsTopicResponse] = { val correlationId = request.header.correlationId val clientId = request.header.clientId - val offsetRequest = request.body[ListOffsetRequest] + val offsetRequest = request.body[ListOffsetsRequest] val version = request.header.apiVersion - def buildErrorResponse(e: Errors, partition: ListOffsetPartition): ListOffsetPartitionResponse = { - new ListOffsetPartitionResponse() + def buildErrorResponse(e: Errors, partition: ListOffsetsPartition): ListOffsetsPartitionResponse = { + new ListOffsetsPartitionResponse() .setPartitionIndex(partition.partitionIndex) .setErrorCode(e.code) - .setTimestamp(ListOffsetResponse.UNKNOWN_TIMESTAMP) - .setOffset(ListOffsetResponse.UNKNOWN_OFFSET) + .setTimestamp(ListOffsetsResponse.UNKNOWN_TIMESTAMP) + .setOffset(ListOffsetsResponse.UNKNOWN_OFFSET) } val (authorizedRequestInfo, unauthorizedRequestInfo) = partitionSeqByAuthorized(request.context, DESCRIBE, TOPIC, offsetRequest.topics.asScala.toSeq)(_.name) val unauthorizedResponseStatus = unauthorizedRequestInfo.map(topic => - new ListOffsetTopicResponse() + new ListOffsetsTopicResponse() .setName(topic.name) .setPartitions(topic.partitions.asScala.map(partition => buildErrorResponse(Errors.TOPIC_AUTHORIZATION_FAILED, partition)).asJava) @@ -1112,8 +1112,8 @@ class KafkaApis(val requestChannel: RequestChannel, buildErrorResponse(Errors.INVALID_REQUEST, partition) } else { try { - val fetchOnlyFromLeader = offsetRequest.replicaId != ListOffsetRequest.DEBUGGING_REPLICA_ID - val isClientRequest = offsetRequest.replicaId == ListOffsetRequest.CONSUMER_REPLICA_ID + val fetchOnlyFromLeader = offsetRequest.replicaId != ListOffsetsRequest.DEBUGGING_REPLICA_ID + val isClientRequest = offsetRequest.replicaId == ListOffsetsRequest.CONSUMER_REPLICA_ID val isolationLevelOpt = if (isClientRequest) Some(offsetRequest.isolationLevel) else @@ -1122,12 +1122,12 @@ class KafkaApis(val requestChannel: RequestChannel, val foundOpt = replicaManager.fetchOffsetForTimestamp(topicPartition, partition.timestamp, isolationLevelOpt, - if (partition.currentLeaderEpoch == ListOffsetResponse.UNKNOWN_EPOCH) Optional.empty() else Optional.of(partition.currentLeaderEpoch), + if (partition.currentLeaderEpoch == ListOffsetsResponse.UNKNOWN_EPOCH) Optional.empty() else Optional.of(partition.currentLeaderEpoch), fetchOnlyFromLeader) val response = foundOpt match { case Some(found) => - val partitionResponse = new ListOffsetPartitionResponse() + val partitionResponse = new ListOffsetsPartitionResponse() .setPartitionIndex(partition.partitionIndex) .setErrorCode(Errors.NONE.code) .setTimestamp(found.timestamp) @@ -1167,7 +1167,7 @@ class KafkaApis(val requestChannel: RequestChannel, } } } - new ListOffsetTopicResponse().setName(topic.name).setPartitions(responsePartitions.asJava) + new ListOffsetsTopicResponse().setName(topic.name).setPartitions(responsePartitions.asJava) } (responseTopics ++ unauthorizedResponseStatus).toList } diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index 14c080339a7d3..1d3a38a0c2b57 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -29,7 +29,7 @@ import kafka.utils.Implicits._ import org.apache.kafka.clients.FetchSessionHandler import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.KafkaStorageException -import org.apache.kafka.common.message.ListOffsetRequestData.{ListOffsetPartition, ListOffsetTopic} +import org.apache.kafka.common.message.ListOffsetsRequestData.{ListOffsetsPartition, ListOffsetsTopic} import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.Errors @@ -227,26 +227,26 @@ class ReplicaFetcherThread(name: String, } override protected def fetchEarliestOffsetFromLeader(topicPartition: TopicPartition, currentLeaderEpoch: Int): Long = { - fetchOffsetFromLeader(topicPartition, currentLeaderEpoch, ListOffsetRequest.EARLIEST_TIMESTAMP) + fetchOffsetFromLeader(topicPartition, currentLeaderEpoch, ListOffsetsRequest.EARLIEST_TIMESTAMP) } override protected def fetchLatestOffsetFromLeader(topicPartition: TopicPartition, currentLeaderEpoch: Int): Long = { - fetchOffsetFromLeader(topicPartition, currentLeaderEpoch, ListOffsetRequest.LATEST_TIMESTAMP) + fetchOffsetFromLeader(topicPartition, currentLeaderEpoch, ListOffsetsRequest.LATEST_TIMESTAMP) } private def fetchOffsetFromLeader(topicPartition: TopicPartition, currentLeaderEpoch: Int, earliestOrLatest: Long): Long = { - val topic = new ListOffsetTopic() + val topic = new ListOffsetsTopic() .setName(topicPartition.topic) .setPartitions(Collections.singletonList( - new ListOffsetPartition() + new ListOffsetsPartition() .setPartitionIndex(topicPartition.partition) .setCurrentLeaderEpoch(currentLeaderEpoch) .setTimestamp(earliestOrLatest))) - val requestBuilder = ListOffsetRequest.Builder.forReplica(listOffsetRequestVersion, replicaId) + val requestBuilder = ListOffsetsRequest.Builder.forReplica(listOffsetRequestVersion, replicaId) .setTargetTimes(Collections.singletonList(topic)) val clientResponse = leaderEndpoint.sendRequest(requestBuilder) - val response = clientResponse.responseBody.asInstanceOf[ListOffsetResponse] + val response = clientResponse.responseBody.asInstanceOf[ListOffsetsResponse] val responsePartition = response.topics.asScala.find(_.name == topicPartition.topic).get .partitions.asScala.find(_.partitionIndex == topicPartition.partition).get diff --git a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala index 4de15b9f373f5..ce58214b79da6 100755 --- a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala @@ -23,7 +23,6 @@ import java.time.Duration import java.util.concurrent.CountDownLatch import java.util.regex.Pattern import java.util.{Collections, Locale, Map, Properties, Random} - import com.typesafe.scalalogging.LazyLogging import joptsimple._ import kafka.utils.Implicits._ @@ -32,7 +31,7 @@ import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, ConsumerReco import org.apache.kafka.common.{MessageFormatter, TopicPartition} import org.apache.kafka.common.errors.{AuthenticationException, TimeoutException, WakeupException} import org.apache.kafka.common.record.TimestampType -import org.apache.kafka.common.requests.ListOffsetRequest +import org.apache.kafka.common.requests.ListOffsetsRequest import org.apache.kafka.common.serialization.{ByteArrayDeserializer, Deserializer} import org.apache.kafka.common.utils.Utils @@ -338,8 +337,8 @@ object ConsoleConsumer extends Logging { val offsetArg = if (options.has(offsetOpt)) { options.valueOf(offsetOpt).toLowerCase(Locale.ROOT) match { - case "earliest" => ListOffsetRequest.EARLIEST_TIMESTAMP - case "latest" => ListOffsetRequest.LATEST_TIMESTAMP + case "earliest" => ListOffsetsRequest.EARLIEST_TIMESTAMP + case "latest" => ListOffsetsRequest.LATEST_TIMESTAMP case offsetString => try { val offset = offsetString.toLong @@ -351,8 +350,8 @@ object ConsoleConsumer extends Logging { } } } - else if (fromBeginning) ListOffsetRequest.EARLIEST_TIMESTAMP - else ListOffsetRequest.LATEST_TIMESTAMP + else if (fromBeginning) ListOffsetsRequest.EARLIEST_TIMESTAMP + else ListOffsetsRequest.LATEST_TIMESTAMP CommandLineUtils.checkRequiredArgs(parser, options, bootstrapServerOpt) @@ -405,7 +404,7 @@ object ConsoleConsumer extends Logging { seek(topic, partitionId, offset) case (Some(topic), Some(partitionId), None, None) => // default to latest if no offset is provided - seek(topic, partitionId, ListOffsetRequest.LATEST_TIMESTAMP) + seek(topic, partitionId, ListOffsetsRequest.LATEST_TIMESTAMP) case (Some(topic), None, None, None) => consumer.subscribe(Collections.singletonList(topic)) case (None, None, None, Some(whitelist)) => @@ -422,8 +421,8 @@ object ConsoleConsumer extends Logging { val topicPartition = new TopicPartition(topic, partitionId) consumer.assign(Collections.singletonList(topicPartition)) offset match { - case ListOffsetRequest.EARLIEST_TIMESTAMP => consumer.seekToBeginning(Collections.singletonList(topicPartition)) - case ListOffsetRequest.LATEST_TIMESTAMP => consumer.seekToEnd(Collections.singletonList(topicPartition)) + case ListOffsetsRequest.EARLIEST_TIMESTAMP => consumer.seekToBeginning(Collections.singletonList(topicPartition)) + case ListOffsetsRequest.LATEST_TIMESTAMP => consumer.seekToEnd(Collections.singletonList(topicPartition)) case _ => consumer.seek(topicPartition, offset) } } diff --git a/core/src/main/scala/kafka/tools/GetOffsetShell.scala b/core/src/main/scala/kafka/tools/GetOffsetShell.scala index 76b16f57ec8be..7606247ebd6c1 100644 --- a/core/src/main/scala/kafka/tools/GetOffsetShell.scala +++ b/core/src/main/scala/kafka/tools/GetOffsetShell.scala @@ -19,12 +19,11 @@ package kafka.tools import java.util.Properties - import joptsimple._ import kafka.utils.{CommandLineUtils, Exit, ToolsUtils} import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer} +import org.apache.kafka.common.requests.ListOffsetsRequest import org.apache.kafka.common.{PartitionInfo, TopicPartition} -import org.apache.kafka.common.requests.ListOffsetRequest import org.apache.kafka.common.serialization.ByteArrayDeserializer import scala.jdk.CollectionConverters._ @@ -124,8 +123,8 @@ object GetOffsetShell { /* Note that the value of the map can be null */ val partitionOffsets: collection.Map[TopicPartition, java.lang.Long] = listOffsetsTimestamp match { - case ListOffsetRequest.EARLIEST_TIMESTAMP => consumer.beginningOffsets(topicPartitions.asJava).asScala - case ListOffsetRequest.LATEST_TIMESTAMP => consumer.endOffsets(topicPartitions.asJava).asScala + case ListOffsetsRequest.EARLIEST_TIMESTAMP => consumer.beginningOffsets(topicPartitions.asJava).asScala + case ListOffsetsRequest.LATEST_TIMESTAMP => consumer.endOffsets(topicPartitions.asJava).asScala case _ => val timestampsToSearch = topicPartitions.map(tp => tp -> (listOffsetsTimestamp: java.lang.Long)).toMap.asJava consumer.offsetsForTimes(timestampsToSearch).asScala.map { case (k, x) => diff --git a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala index 0060f3c69f909..a00753eb273f7 100644 --- a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala +++ b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala @@ -37,7 +37,7 @@ import org.apache.kafka.common.network.{NetworkReceive, Selectable, Selector} import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.record.MemoryRecords import org.apache.kafka.common.requests.AbstractRequest.Builder -import org.apache.kafka.common.requests.{AbstractRequest, FetchResponse, ListOffsetRequest, FetchRequest => JFetchRequest} +import org.apache.kafka.common.requests.{AbstractRequest, FetchResponse, ListOffsetsRequest, FetchRequest => JFetchRequest} import org.apache.kafka.common.serialization.StringDeserializer import org.apache.kafka.common.utils.{LogContext, Time} import org.apache.kafka.common.{Node, TopicPartition} @@ -228,9 +228,9 @@ object ReplicaVerificationTool extends Logging { initialOffsetTime: Long): collection.Map[TopicPartition, Long] = { val consumer = createConsumer(consumerConfig) try { - if (ListOffsetRequest.LATEST_TIMESTAMP == initialOffsetTime) + if (ListOffsetsRequest.LATEST_TIMESTAMP == initialOffsetTime) consumer.endOffsets(topicPartitions.asJava).asScala.map { case (k, v) => k -> v.longValue } - else if (ListOffsetRequest.EARLIEST_TIMESTAMP == initialOffsetTime) + else if (ListOffsetsRequest.EARLIEST_TIMESTAMP == initialOffsetTime) consumer.beginningOffsets(topicPartitions.asJava).asScala.map { case (k, v) => k -> v.longValue } else { val timestampsToSearch = topicPartitions.map(tp => tp -> (initialOffsetTime: java.lang.Long)).toMap diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index e8cce7abe8118..e1ee87ace03df 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -42,7 +42,7 @@ import org.apache.kafka.common.message.IncrementalAlterConfigsRequestData.{Alter import org.apache.kafka.common.message.JoinGroupRequestData.JoinGroupRequestProtocolCollection import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity -import org.apache.kafka.common.message.ListOffsetRequestData.{ListOffsetPartition, ListOffsetTopic} +import org.apache.kafka.common.message.ListOffsetsRequestData.{ListOffsetsPartition, ListOffsetsTopic} import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderTopic import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderTopicCollection @@ -159,7 +159,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { ApiKeys.METADATA -> ((resp: requests.MetadataResponse) => resp.errors.asScala.find(_._1 == topic).getOrElse(("test", Errors.NONE))._2), ApiKeys.PRODUCE -> ((resp: requests.ProduceResponse) => resp.responses.asScala.find(_._1 == tp).get._2.error), ApiKeys.FETCH -> ((resp: requests.FetchResponse[Records]) => resp.responseData.asScala.find(_._1 == tp).get._2.error), - ApiKeys.LIST_OFFSETS -> ((resp: requests.ListOffsetResponse) => { + ApiKeys.LIST_OFFSETS -> ((resp: ListOffsetsResponse) => { Errors.forCode( resp.data .topics.asScala.find(_.name == topic).get @@ -321,10 +321,10 @@ class AuthorizerIntegrationTest extends BaseRequestTest { } private def createListOffsetsRequest = { - requests.ListOffsetRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED).setTargetTimes( - List(new ListOffsetTopic() + requests.ListOffsetsRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED).setTargetTimes( + List(new ListOffsetsTopic() .setName(tp.topic) - .setPartitions(List(new ListOffsetPartition() + .setPartitions(List(new ListOffsetsPartition() .setPartitionIndex(tp.partition) .setTimestamp(0L) .setCurrentLeaderEpoch(27)).asJava)).asJava diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala index 93aed43892b94..ccf5c9a99ea67 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala @@ -19,7 +19,6 @@ package kafka.cluster import java.nio.ByteBuffer import java.util.Optional import java.util.concurrent.{CountDownLatch, Semaphore} - import com.yammer.metrics.core.Metric import kafka.api.{ApiVersion, LeaderAndIsr} import kafka.common.UnexpectedAppendOffsetException @@ -34,8 +33,8 @@ import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrParti import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record.FileRecords.TimestampAndOffset import org.apache.kafka.common.record._ +import org.apache.kafka.common.requests.ListOffsetsRequest import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.{UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET} -import org.apache.kafka.common.requests.ListOffsetRequest import org.apache.kafka.common.utils.SystemTime import org.apache.kafka.common.{IsolationLevel, TopicPartition} import org.junit.Assert._ @@ -529,7 +528,7 @@ class PartitionTest extends AbstractPartitionTest { val leaderEpoch = 5 val partition = setupPartitionWithMocks(leaderEpoch, isLeader = true) - val timestampAndOffsetOpt = partition.fetchOffsetForTimestamp(ListOffsetRequest.LATEST_TIMESTAMP, + val timestampAndOffsetOpt = partition.fetchOffsetForTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP, isolationLevel = None, currentLeaderEpoch = Optional.empty(), fetchOnlyFromLeader = true) @@ -621,14 +620,14 @@ class PartitionTest extends AbstractPartitionTest { assertEquals(2, partition.localLogOrException.highWatermark) // Get the LEO - fetchOffsetsForTimestamp(ListOffsetRequest.LATEST_TIMESTAMP, None) match { + fetchOffsetsForTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP, None) match { case Right(Some(offsetAndTimestamp)) => assertEquals(5, offsetAndTimestamp.offset) case Right(None) => fail("Should have seen some offsets") case Left(e) => fail("Should not have seen an error") } // Get the HW - fetchOffsetsForTimestamp(ListOffsetRequest.LATEST_TIMESTAMP, Some(IsolationLevel.READ_UNCOMMITTED)) match { + fetchOffsetsForTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP, Some(IsolationLevel.READ_UNCOMMITTED)) match { case Right(Some(offsetAndTimestamp)) => assertEquals(2, offsetAndTimestamp.offset) case Right(None) => fail("Should have seen some offsets") case Left(e) => fail("Should not have seen an error") @@ -662,7 +661,7 @@ class PartitionTest extends AbstractPartitionTest { assertTrue(partition.makeLeader(newLeaderState, offsetCheckpoints)) // Try to get offsets as a client - fetchOffsetsForTimestamp(ListOffsetRequest.LATEST_TIMESTAMP, Some(IsolationLevel.READ_UNCOMMITTED)) match { + fetchOffsetsForTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP, Some(IsolationLevel.READ_UNCOMMITTED)) match { case Right(Some(offsetAndTimestamp)) => fail("Should have failed with OffsetNotAvailable") case Right(None) => fail("Should have seen an error") case Left(e: OffsetNotAvailableException) => // ok @@ -670,14 +669,14 @@ class PartitionTest extends AbstractPartitionTest { } // If request is not from a client, we skip the check - fetchOffsetsForTimestamp(ListOffsetRequest.LATEST_TIMESTAMP, None) match { + fetchOffsetsForTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP, None) match { case Right(Some(offsetAndTimestamp)) => assertEquals(5, offsetAndTimestamp.offset) case Right(None) => fail("Should have seen some offsets") case Left(e: ApiException) => fail(s"Got ApiException $e") } // If we request the earliest timestamp, we skip the check - fetchOffsetsForTimestamp(ListOffsetRequest.EARLIEST_TIMESTAMP, Some(IsolationLevel.READ_UNCOMMITTED)) match { + fetchOffsetsForTimestamp(ListOffsetsRequest.EARLIEST_TIMESTAMP, Some(IsolationLevel.READ_UNCOMMITTED)) match { case Right(Some(offsetAndTimestamp)) => assertEquals(0, offsetAndTimestamp.offset) case Right(None) => fail("Should have seen some offsets") case Left(e: ApiException) => fail(s"Got ApiException $e") @@ -709,7 +708,7 @@ class PartitionTest extends AbstractPartitionTest { updateFollowerFetchState(follower2, LogOffsetMetadata(5)) // Error goes away - fetchOffsetsForTimestamp(ListOffsetRequest.LATEST_TIMESTAMP, Some(IsolationLevel.READ_UNCOMMITTED)) match { + fetchOffsetsForTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP, Some(IsolationLevel.READ_UNCOMMITTED)) match { case Right(Some(offsetAndTimestamp)) => assertEquals(5, offsetAndTimestamp.offset) case Right(None) => fail("Should have seen some offsets") case Left(e: ApiException) => fail(s"Got ApiException $e") @@ -834,7 +833,7 @@ class PartitionTest extends AbstractPartitionTest { partition.appendRecordsToLeader(records, origin = AppendOrigin.Client, requiredAcks = 0) def fetchLatestOffset(isolationLevel: Option[IsolationLevel]): TimestampAndOffset = { - val res = partition.fetchOffsetForTimestamp(ListOffsetRequest.LATEST_TIMESTAMP, + val res = partition.fetchOffsetForTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP, isolationLevel = isolationLevel, currentLeaderEpoch = Optional.empty(), fetchOnlyFromLeader = true) @@ -843,7 +842,7 @@ class PartitionTest extends AbstractPartitionTest { } def fetchEarliestOffset(isolationLevel: Option[IsolationLevel]): TimestampAndOffset = { - val res = partition.fetchOffsetForTimestamp(ListOffsetRequest.EARLIEST_TIMESTAMP, + val res = partition.fetchOffsetForTimestamp(ListOffsetsRequest.EARLIEST_TIMESTAMP, isolationLevel = isolationLevel, currentLeaderEpoch = Optional.empty(), fetchOnlyFromLeader = true) diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index a3aea7a6d0201..ce52c6b539128 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -39,7 +39,7 @@ import org.apache.kafka.common.record.MemoryRecords.RecordFilter import org.apache.kafka.common.record.MemoryRecords.RecordFilter.BatchRetention import org.apache.kafka.common.record._ import org.apache.kafka.common.requests.FetchResponse.AbortedTransaction -import org.apache.kafka.common.requests.{ListOffsetRequest, ListOffsetResponse} +import org.apache.kafka.common.requests.{ListOffsetsRequest, ListOffsetsResponse} import org.apache.kafka.common.utils.{Time, Utils} import org.easymock.EasyMock import org.junit.Assert._ @@ -2451,17 +2451,17 @@ class LogTest { assertEquals(Some(new TimestampAndOffset(secondTimestamp, 1L, Optional.of(secondLeaderEpoch))), log.fetchOffsetByTimestamp(secondTimestamp)) - assertEquals(Some(new TimestampAndOffset(ListOffsetResponse.UNKNOWN_TIMESTAMP, 0L, Optional.of(firstLeaderEpoch))), - log.fetchOffsetByTimestamp(ListOffsetRequest.EARLIEST_TIMESTAMP)) - assertEquals(Some(new TimestampAndOffset(ListOffsetResponse.UNKNOWN_TIMESTAMP, 2L, Optional.of(secondLeaderEpoch))), - log.fetchOffsetByTimestamp(ListOffsetRequest.LATEST_TIMESTAMP)) + assertEquals(Some(new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 0L, Optional.of(firstLeaderEpoch))), + log.fetchOffsetByTimestamp(ListOffsetsRequest.EARLIEST_TIMESTAMP)) + assertEquals(Some(new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 2L, Optional.of(secondLeaderEpoch))), + log.fetchOffsetByTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP)) // The cache can be updated directly after a leader change. // The new latest offset should reflect the updated epoch. log.maybeAssignEpochStartOffset(2, 2L) - assertEquals(Some(new TimestampAndOffset(ListOffsetResponse.UNKNOWN_TIMESTAMP, 2L, Optional.of(2))), - log.fetchOffsetByTimestamp(ListOffsetRequest.LATEST_TIMESTAMP)) + assertEquals(Some(new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 2L, Optional.of(2))), + log.fetchOffsetByTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP)) } /** diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 04e42aa616870..e49e9dee6e264 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -46,7 +46,7 @@ import org.apache.kafka.common.memory.MemoryPool import org.apache.kafka.common.message.CreateTopicsRequestData.{CreatableTopic, CreatableTopicCollection} import org.apache.kafka.common.message.JoinGroupRequestData.JoinGroupRequestProtocol import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity -import org.apache.kafka.common.message.ListOffsetRequestData.{ListOffsetPartition, ListOffsetTopic} +import org.apache.kafka.common.message.ListOffsetsRequestData.{ListOffsetsPartition, ListOffsetsTopic} import org.apache.kafka.common.message.MetadataResponseData.MetadataResponseTopic import org.apache.kafka.common.message.OffsetDeleteRequestData.{OffsetDeleteRequestPartition, OffsetDeleteRequestTopic, OffsetDeleteRequestTopicCollection} import org.apache.kafka.common.message.StopReplicaRequestData.{StopReplicaPartitionState, StopReplicaTopicState} @@ -1774,7 +1774,7 @@ class KafkaApisTest { EasyMock.expect(replicaManager.fetchOffsetForTimestamp( EasyMock.eq(tp), - EasyMock.eq(ListOffsetRequest.EARLIEST_TIMESTAMP), + EasyMock.eq(ListOffsetsRequest.EARLIEST_TIMESTAMP), EasyMock.eq(Some(isolationLevel)), EasyMock.eq(currentLeaderEpoch), fetchOnlyFromLeader = EasyMock.eq(true)) @@ -1783,27 +1783,27 @@ class KafkaApisTest { val capturedResponse = expectNoThrottling() EasyMock.replay(replicaManager, clientRequestQuotaManager, requestChannel) - val targetTimes = List(new ListOffsetTopic() + val targetTimes = List(new ListOffsetsTopic() .setName(tp.topic) - .setPartitions(List(new ListOffsetPartition() + .setPartitions(List(new ListOffsetsPartition() .setPartitionIndex(tp.partition) - .setTimestamp(ListOffsetRequest.EARLIEST_TIMESTAMP) + .setTimestamp(ListOffsetsRequest.EARLIEST_TIMESTAMP) .setCurrentLeaderEpoch(currentLeaderEpoch.get)).asJava)).asJava - val listOffsetRequest = ListOffsetRequest.Builder.forConsumer(true, isolationLevel) + val listOffsetRequest = ListOffsetsRequest.Builder.forConsumer(true, isolationLevel) .setTargetTimes(targetTimes).build() val request = buildRequest(listOffsetRequest) createKafkaApis().handleListOffsetRequest(request) val response = readResponse(listOffsetRequest, capturedResponse) - .asInstanceOf[ListOffsetResponse] + .asInstanceOf[ListOffsetsResponse] val partitionDataOptional = response.topics.asScala.find(_.name == tp.topic).get .partitions.asScala.find(_.partitionIndex == tp.partition) assertTrue(partitionDataOptional.isDefined) val partitionData = partitionDataOptional.get assertEquals(error.code, partitionData.errorCode) - assertEquals(ListOffsetResponse.UNKNOWN_OFFSET, partitionData.offset) - assertEquals(ListOffsetResponse.UNKNOWN_TIMESTAMP, partitionData.timestamp) + assertEquals(ListOffsetsResponse.UNKNOWN_OFFSET, partitionData.offset) + assertEquals(ListOffsetsResponse.UNKNOWN_TIMESTAMP, partitionData.timestamp) } @Test @@ -2849,26 +2849,26 @@ class KafkaApisTest { EasyMock.expect(replicaManager.fetchOffsetForTimestamp( EasyMock.eq(tp), - EasyMock.eq(ListOffsetRequest.LATEST_TIMESTAMP), + EasyMock.eq(ListOffsetsRequest.LATEST_TIMESTAMP), EasyMock.eq(Some(isolationLevel)), EasyMock.eq(currentLeaderEpoch), fetchOnlyFromLeader = EasyMock.eq(true)) - ).andReturn(Some(new TimestampAndOffset(ListOffsetResponse.UNKNOWN_TIMESTAMP, latestOffset, currentLeaderEpoch))) + ).andReturn(Some(new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, latestOffset, currentLeaderEpoch))) val capturedResponse = expectNoThrottling() EasyMock.replay(replicaManager, clientRequestQuotaManager, requestChannel) - val targetTimes = List(new ListOffsetTopic() + val targetTimes = List(new ListOffsetsTopic() .setName(tp.topic) - .setPartitions(List(new ListOffsetPartition() + .setPartitions(List(new ListOffsetsPartition() .setPartitionIndex(tp.partition) - .setTimestamp(ListOffsetRequest.LATEST_TIMESTAMP)).asJava)).asJava - val listOffsetRequest = ListOffsetRequest.Builder.forConsumer(true, isolationLevel) + .setTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP)).asJava)).asJava + val listOffsetRequest = ListOffsetsRequest.Builder.forConsumer(true, isolationLevel) .setTargetTimes(targetTimes).build() val request = buildRequest(listOffsetRequest) createKafkaApis().handleListOffsetRequest(request) - val response = readResponse(listOffsetRequest, capturedResponse).asInstanceOf[ListOffsetResponse] + val response = readResponse(listOffsetRequest, capturedResponse).asInstanceOf[ListOffsetsResponse] val partitionDataOptional = response.topics.asScala.find(_.name == tp.topic).get .partitions.asScala.find(_.partitionIndex == tp.partition) assertTrue(partitionDataOptional.isDefined) @@ -2876,7 +2876,7 @@ class KafkaApisTest { val partitionData = partitionDataOptional.get assertEquals(Errors.NONE.code, partitionData.errorCode) assertEquals(latestOffset, partitionData.offset) - assertEquals(ListOffsetResponse.UNKNOWN_TIMESTAMP, partitionData.timestamp) + assertEquals(ListOffsetsResponse.UNKNOWN_TIMESTAMP, partitionData.timestamp) } private def createWriteTxnMarkersRequest(partitions: util.List[TopicPartition]) = { diff --git a/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala b/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala index ce324c77cd9a2..f96f527f7e3f7 100644 --- a/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala @@ -19,9 +19,9 @@ package kafka.server import java.util.Optional import kafka.utils.TestUtils -import org.apache.kafka.common.message.ListOffsetRequestData.{ListOffsetPartition, ListOffsetTopic} +import org.apache.kafka.common.message.ListOffsetsRequestData.{ListOffsetsPartition, ListOffsetsTopic} import org.apache.kafka.common.protocol.{ApiKeys, Errors} -import org.apache.kafka.common.requests.{ListOffsetRequest, ListOffsetResponse} +import org.apache.kafka.common.requests.{ListOffsetsRequest, ListOffsetsResponse} import org.apache.kafka.common.{IsolationLevel, TopicPartition} import org.junit.Assert._ import org.junit.Test @@ -35,25 +35,25 @@ class ListOffsetsRequestTest extends BaseRequestTest { @Test def testListOffsetsErrorCodes(): Unit = { - val targetTimes = List(new ListOffsetTopic() + val targetTimes = List(new ListOffsetsTopic() .setName(topic) - .setPartitions(List(new ListOffsetPartition() + .setPartitions(List(new ListOffsetsPartition() .setPartitionIndex(partition.partition) - .setTimestamp(ListOffsetRequest.EARLIEST_TIMESTAMP) + .setTimestamp(ListOffsetsRequest.EARLIEST_TIMESTAMP) .setCurrentLeaderEpoch(0)).asJava)).asJava - val consumerRequest = ListOffsetRequest.Builder + val consumerRequest = ListOffsetsRequest.Builder .forConsumer(false, IsolationLevel.READ_UNCOMMITTED) .setTargetTimes(targetTimes) .build() - val replicaRequest = ListOffsetRequest.Builder + val replicaRequest = ListOffsetsRequest.Builder .forReplica(ApiKeys.LIST_OFFSETS.latestVersion, servers.head.config.brokerId) .setTargetTimes(targetTimes) .build() - val debugReplicaRequest = ListOffsetRequest.Builder - .forReplica(ApiKeys.LIST_OFFSETS.latestVersion, ListOffsetRequest.DEBUGGING_REPLICA_ID) + val debugReplicaRequest = ListOffsetsRequest.Builder + .forReplica(ApiKeys.LIST_OFFSETS.latestVersion, ListOffsetsRequest.DEBUGGING_REPLICA_ID) .setTargetTimes(targetTimes) .build() @@ -81,15 +81,15 @@ class ListOffsetsRequestTest extends BaseRequestTest { } def assertResponseErrorForEpoch(error: Errors, brokerId: Int, currentLeaderEpoch: Optional[Integer]): Unit = { - val listOffsetPartition = new ListOffsetPartition() + val listOffsetPartition = new ListOffsetsPartition() .setPartitionIndex(partition.partition) - .setTimestamp(ListOffsetRequest.EARLIEST_TIMESTAMP) + .setTimestamp(ListOffsetsRequest.EARLIEST_TIMESTAMP) if (currentLeaderEpoch.isPresent) listOffsetPartition.setCurrentLeaderEpoch(currentLeaderEpoch.get) - val targetTimes = List(new ListOffsetTopic() + val targetTimes = List(new ListOffsetsTopic() .setName(topic) .setPartitions(List(listOffsetPartition).asJava)).asJava - val request = ListOffsetRequest.Builder + val request = ListOffsetsRequest.Builder .forConsumer(false, IsolationLevel.READ_UNCOMMITTED) .setTargetTimes(targetTimes) .build() @@ -127,13 +127,13 @@ class ListOffsetsRequestTest extends BaseRequestTest { def fetchOffsetAndEpoch(serverId: Int, timestamp: Long, version: Short): (Long, Int) = { - val targetTimes = List(new ListOffsetTopic() + val targetTimes = List(new ListOffsetsTopic() .setName(topic) - .setPartitions(List(new ListOffsetPartition() + .setPartitions(List(new ListOffsetsPartition() .setPartitionIndex(partition.partition) .setTimestamp(timestamp)).asJava)).asJava - val builder = ListOffsetRequest.Builder + val builder = ListOffsetsRequest.Builder .forConsumer(false, IsolationLevel.READ_UNCOMMITTED) .setTargetTimes(targetTimes) @@ -160,8 +160,8 @@ class ListOffsetsRequestTest extends BaseRequestTest { TestUtils.generateAndProduceMessages(servers, topic, 10) assertEquals((0L, 0), fetchOffsetAndEpoch(firstLeaderId, 0L, -1)) - assertEquals((0L, 0), fetchOffsetAndEpoch(firstLeaderId, ListOffsetRequest.EARLIEST_TIMESTAMP, -1)) - assertEquals((10L, 0), fetchOffsetAndEpoch(firstLeaderId, ListOffsetRequest.LATEST_TIMESTAMP, -1)) + assertEquals((0L, 0), fetchOffsetAndEpoch(firstLeaderId, ListOffsetsRequest.EARLIEST_TIMESTAMP, -1)) + assertEquals((10L, 0), fetchOffsetAndEpoch(firstLeaderId, ListOffsetsRequest.LATEST_TIMESTAMP, -1)) // Kill the first leader so that we can verify the epoch change when fetching the latest offset killBroker(firstLeaderId) @@ -170,13 +170,13 @@ class ListOffsetsRequestTest extends BaseRequestTest { // No changes to written data assertEquals((0L, 0), fetchOffsetAndEpoch(secondLeaderId, 0L, -1)) - assertEquals((0L, 0), fetchOffsetAndEpoch(secondLeaderId, ListOffsetRequest.EARLIEST_TIMESTAMP, -1)) + assertEquals((0L, 0), fetchOffsetAndEpoch(secondLeaderId, ListOffsetsRequest.EARLIEST_TIMESTAMP, -1)) assertEquals((0L, 0), fetchOffsetAndEpoch(secondLeaderId, 0L, -1)) - assertEquals((0L, 0), fetchOffsetAndEpoch(secondLeaderId, ListOffsetRequest.EARLIEST_TIMESTAMP, -1)) + assertEquals((0L, 0), fetchOffsetAndEpoch(secondLeaderId, ListOffsetsRequest.EARLIEST_TIMESTAMP, -1)) // The latest offset reflects the updated epoch - assertEquals((10L, secondLeaderEpoch), fetchOffsetAndEpoch(secondLeaderId, ListOffsetRequest.LATEST_TIMESTAMP, -1)) + assertEquals((10L, secondLeaderEpoch), fetchOffsetAndEpoch(secondLeaderId, ListOffsetsRequest.LATEST_TIMESTAMP, -1)) } @Test @@ -189,21 +189,21 @@ class ListOffsetsRequestTest extends BaseRequestTest { for (version <- ApiKeys.LIST_OFFSETS.oldestVersion to ApiKeys.LIST_OFFSETS.latestVersion) { if (version == 0) { assertEquals((-1L, -1), fetchOffsetAndEpoch(firstLeaderId, 0L, version.toShort)) - assertEquals((0L, -1), fetchOffsetAndEpoch(firstLeaderId, ListOffsetRequest.EARLIEST_TIMESTAMP, version.toShort)) - assertEquals((10L, -1), fetchOffsetAndEpoch(firstLeaderId, ListOffsetRequest.LATEST_TIMESTAMP, version.toShort)) + assertEquals((0L, -1), fetchOffsetAndEpoch(firstLeaderId, ListOffsetsRequest.EARLIEST_TIMESTAMP, version.toShort)) + assertEquals((10L, -1), fetchOffsetAndEpoch(firstLeaderId, ListOffsetsRequest.LATEST_TIMESTAMP, version.toShort)) } else if (version >= 1 && version <= 3) { assertEquals((0L, -1), fetchOffsetAndEpoch(firstLeaderId, 0L, version.toShort)) - assertEquals((0L, -1), fetchOffsetAndEpoch(firstLeaderId, ListOffsetRequest.EARLIEST_TIMESTAMP, version.toShort)) - assertEquals((10L, -1), fetchOffsetAndEpoch(firstLeaderId, ListOffsetRequest.LATEST_TIMESTAMP, version.toShort)) + assertEquals((0L, -1), fetchOffsetAndEpoch(firstLeaderId, ListOffsetsRequest.EARLIEST_TIMESTAMP, version.toShort)) + assertEquals((10L, -1), fetchOffsetAndEpoch(firstLeaderId, ListOffsetsRequest.LATEST_TIMESTAMP, version.toShort)) } else if (version >= 4) { assertEquals((0L, 0), fetchOffsetAndEpoch(firstLeaderId, 0L, version.toShort)) - assertEquals((0L, 0), fetchOffsetAndEpoch(firstLeaderId, ListOffsetRequest.EARLIEST_TIMESTAMP, version.toShort)) - assertEquals((10L, 0), fetchOffsetAndEpoch(firstLeaderId, ListOffsetRequest.LATEST_TIMESTAMP, version.toShort)) + assertEquals((0L, 0), fetchOffsetAndEpoch(firstLeaderId, ListOffsetsRequest.EARLIEST_TIMESTAMP, version.toShort)) + assertEquals((10L, 0), fetchOffsetAndEpoch(firstLeaderId, ListOffsetsRequest.LATEST_TIMESTAMP, version.toShort)) } } } - private def assertResponseError(error: Errors, brokerId: Int, request: ListOffsetRequest): Unit = { + private def assertResponseError(error: Errors, brokerId: Int, request: ListOffsetsRequest): Unit = { val response = sendRequest(brokerId, request) assertEquals(request.topics.size, response.topics.size) response.topics.asScala.foreach { topic => @@ -213,7 +213,7 @@ class ListOffsetsRequestTest extends BaseRequestTest { } } - private def sendRequest(leaderId: Int, request: ListOffsetRequest): ListOffsetResponse = { - connectAndReceive[ListOffsetResponse](request, destination = brokerSocketServer(leaderId)) + private def sendRequest(leaderId: Int, request: ListOffsetsRequest): ListOffsetsResponse = { + connectAndReceive[ListOffsetsResponse](request, destination = brokerSocketServer(leaderId)) } } diff --git a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala index e89a69854ca75..0b02ea6f7b92a 100755 --- a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala @@ -23,13 +23,13 @@ import java.util.{Optional, Properties, Random} import kafka.log.{ClientRecordDeletion, Log, LogSegment} import kafka.utils.{MockTime, TestUtils} -import org.apache.kafka.common.message.ListOffsetRequestData.ListOffsetTopic -import org.apache.kafka.common.message.ListOffsetRequestData.ListOffsetPartition -import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetPartitionResponse -import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetTopicResponse +import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsTopic +import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition +import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse +import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record.MemoryRecords -import org.apache.kafka.common.requests.{FetchRequest, FetchResponse, ListOffsetRequest, ListOffsetResponse} +import org.apache.kafka.common.requests.{FetchRequest, FetchResponse, ListOffsetsRequest, ListOffsetsResponse} import org.apache.kafka.common.{IsolationLevel, TopicPartition} import org.easymock.{EasyMock, IAnswer} import org.junit.Assert._ @@ -58,8 +58,8 @@ class LogOffsetTest extends BaseRequestTest { @Test def testGetOffsetsForUnknownTopic(): Unit = { val topicPartition = new TopicPartition("foo", 0) - val request = ListOffsetRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED) - .setTargetTimes(buildTargetTimes(topicPartition, ListOffsetRequest.LATEST_TIMESTAMP, 10).asJava).build(0) + val request = ListOffsetsRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED) + .setTargetTimes(buildTargetTimes(topicPartition, ListOffsetsRequest.LATEST_TIMESTAMP, 10).asJava).build(0) val response = sendListOffsetsRequest(request) assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION.code, findPartition(response.topics.asScala, topicPartition).errorCode) } @@ -85,13 +85,13 @@ class LogOffsetTest extends BaseRequestTest { log.maybeIncrementLogStartOffset(3, ClientRecordDeletion) log.deleteOldSegments() - val offsets = log.legacyFetchOffsetsBefore(ListOffsetRequest.LATEST_TIMESTAMP, 15) + val offsets = log.legacyFetchOffsetsBefore(ListOffsetsRequest.LATEST_TIMESTAMP, 15) assertEquals(Seq(20L, 18L, 16L, 14L, 12L, 10L, 8L, 6L, 4L, 3L), offsets) TestUtils.waitUntilTrue(() => TestUtils.isLeaderLocalOnBroker(topic, topicPartition.partition, server), "Leader should be elected") - val request = ListOffsetRequest.Builder.forReplica(0, 0) - .setTargetTimes(buildTargetTimes(topicPartition, ListOffsetRequest.LATEST_TIMESTAMP, 15).asJava).build() + val request = ListOffsetsRequest.Builder.forReplica(0, 0) + .setTargetTimes(buildTargetTimes(topicPartition, ListOffsetsRequest.LATEST_TIMESTAMP, 15).asJava).build() val consumerOffsets = findPartition(sendListOffsetsRequest(request).topics.asScala, topicPartition).oldStyleOffsets.asScala assertEquals(Seq(20L, 18L, 16L, 14L, 12L, 10L, 8L, 6L, 4L, 3L), consumerOffsets) } @@ -112,13 +112,13 @@ class LogOffsetTest extends BaseRequestTest { log.appendAsLeader(TestUtils.singletonRecords(value = Integer.toString(42).getBytes()), leaderEpoch = 0) log.flush() - val offsets = log.legacyFetchOffsetsBefore(ListOffsetRequest.LATEST_TIMESTAMP, 15) + val offsets = log.legacyFetchOffsetsBefore(ListOffsetsRequest.LATEST_TIMESTAMP, 15) assertEquals(Seq(20L, 18L, 16L, 14L, 12L, 10L, 8L, 6L, 4L, 2L, 0L), offsets) TestUtils.waitUntilTrue(() => TestUtils.isLeaderLocalOnBroker(topic, topicPartition.partition, server), "Leader should be elected") - val request = ListOffsetRequest.Builder.forReplica(0, 0) - .setTargetTimes(buildTargetTimes(topicPartition, ListOffsetRequest.LATEST_TIMESTAMP, 15).asJava).build() + val request = ListOffsetsRequest.Builder.forReplica(0, 0) + .setTargetTimes(buildTargetTimes(topicPartition, ListOffsetsRequest.LATEST_TIMESTAMP, 15).asJava).build() val consumerOffsets = findPartition(sendListOffsetsRequest(request).topics.asScala, topicPartition).oldStyleOffsets.asScala assertEquals(Seq(20L, 18L, 16L, 14L, 12L, 10L, 8L, 6L, 4L, 2L, 0L), consumerOffsets) @@ -144,8 +144,8 @@ class LogOffsetTest extends BaseRequestTest { var offsetChanged = false for (_ <- 1 to 14) { val topicPartition = new TopicPartition(topic, 0) - val request = ListOffsetRequest.Builder.forReplica(0, 0) - .setTargetTimes(buildTargetTimes(topicPartition, ListOffsetRequest.EARLIEST_TIMESTAMP, 1).asJava).build() + val request = ListOffsetsRequest.Builder.forReplica(0, 0) + .setTargetTimes(buildTargetTimes(topicPartition, ListOffsetsRequest.EARLIEST_TIMESTAMP, 1).asJava).build() val consumerOffsets = findPartition(sendListOffsetsRequest(request).topics.asScala, topicPartition).oldStyleOffsets.asScala if (consumerOffsets.head == 1) offsetChanged = true @@ -176,7 +176,7 @@ class LogOffsetTest extends BaseRequestTest { TestUtils.waitUntilTrue(() => TestUtils.isLeaderLocalOnBroker(topic, topicPartition.partition, server), "Leader should be elected") - val request = ListOffsetRequest.Builder.forReplica(0, 0) + val request = ListOffsetsRequest.Builder.forReplica(0, 0) .setTargetTimes(buildTargetTimes(topicPartition, now, 15).asJava).build() val consumerOffsets = findPartition(sendListOffsetsRequest(request).topics.asScala, topicPartition).oldStyleOffsets.asScala assertEquals(Seq(20L, 18L, 16L, 14L, 12L, 10L, 8L, 6L, 4L, 2L, 0L), consumerOffsets) @@ -197,14 +197,14 @@ class LogOffsetTest extends BaseRequestTest { log.appendAsLeader(TestUtils.singletonRecords(value = Integer.toString(42).getBytes()), leaderEpoch = 0) log.flush() - val offsets = log.legacyFetchOffsetsBefore(ListOffsetRequest.EARLIEST_TIMESTAMP, 10) + val offsets = log.legacyFetchOffsetsBefore(ListOffsetsRequest.EARLIEST_TIMESTAMP, 10) assertEquals(Seq(0L), offsets) TestUtils.waitUntilTrue(() => TestUtils.isLeaderLocalOnBroker(topic, topicPartition.partition, server), "Leader should be elected") - val request = ListOffsetRequest.Builder.forReplica(0, 0) - .setTargetTimes(buildTargetTimes(topicPartition, ListOffsetRequest.EARLIEST_TIMESTAMP, 10).asJava).build() + val request = ListOffsetsRequest.Builder.forReplica(0, 0) + .setTargetTimes(buildTargetTimes(topicPartition, ListOffsetsRequest.EARLIEST_TIMESTAMP, 10).asJava).build() val consumerOffsets = findPartition(sendListOffsetsRequest(request).topics.asScala, topicPartition).oldStyleOffsets.asScala assertEquals(Seq(0L), consumerOffsets) } @@ -247,25 +247,25 @@ class LogOffsetTest extends BaseRequestTest { private def server: KafkaServer = servers.head - private def sendListOffsetsRequest(request: ListOffsetRequest): ListOffsetResponse = { - connectAndReceive[ListOffsetResponse](request) + private def sendListOffsetsRequest(request: ListOffsetsRequest): ListOffsetsResponse = { + connectAndReceive[ListOffsetsResponse](request) } private def sendFetchRequest(request: FetchRequest): FetchResponse[MemoryRecords] = { connectAndReceive[FetchResponse[MemoryRecords]](request) } - private def buildTargetTimes(tp: TopicPartition, timestamp: Long, maxNumOffsets: Int): List[ListOffsetTopic] = { - List(new ListOffsetTopic() + private def buildTargetTimes(tp: TopicPartition, timestamp: Long, maxNumOffsets: Int): List[ListOffsetsTopic] = { + List(new ListOffsetsTopic() .setName(tp.topic) - .setPartitions(List(new ListOffsetPartition() + .setPartitions(List(new ListOffsetsPartition() .setPartitionIndex(tp.partition) .setTimestamp(timestamp) .setMaxNumOffsets(maxNumOffsets)).asJava) ) } - private def findPartition(topics: Buffer[ListOffsetTopicResponse], tp: TopicPartition): ListOffsetPartitionResponse = { + private def findPartition(topics: Buffer[ListOffsetsTopicResponse], tp: TopicPartition): ListOffsetsPartitionResponse = { topics.find(_.name == tp.topic).get .partitions.asScala.find(_.partitionIndex == tp.partition).get } diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index bf22a40d7c5cb..660ae7dfebab4 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -31,7 +31,7 @@ import org.apache.kafka.common.message.CreateTopicsRequestData.{CreatableTopic, import org.apache.kafka.common.message.JoinGroupRequestData.JoinGroupRequestProtocolCollection import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity -import org.apache.kafka.common.message.ListOffsetRequestData.{ListOffsetPartition, ListOffsetTopic} +import org.apache.kafka.common.message.ListOffsetsRequestData.{ListOffsetsPartition, ListOffsetsTopic} import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderTopic import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderTopicCollection @@ -233,13 +233,13 @@ class RequestQuotaTest extends BaseRequestTest { new MetadataRequest.Builder(List(topic).asJava, true) case ApiKeys.LIST_OFFSETS => - val topic = new ListOffsetTopic() + val topic = new ListOffsetsTopic() .setName(tp.topic) - .setPartitions(List(new ListOffsetPartition() + .setPartitions(List(new ListOffsetsPartition() .setPartitionIndex(tp.partition) .setTimestamp(0L) .setCurrentLeaderEpoch(15)).asJava) - ListOffsetRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED) + ListOffsetsRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED) .setTargetTimes(List(topic).asJava) case ApiKeys.LEADER_AND_ISR => diff --git a/generator/src/main/java/org/apache/kafka/message/ApiMessageTypeGenerator.java b/generator/src/main/java/org/apache/kafka/message/ApiMessageTypeGenerator.java index 610b0f8129212..075ee485550b0 100644 --- a/generator/src/main/java/org/apache/kafka/message/ApiMessageTypeGenerator.java +++ b/generator/src/main/java/org/apache/kafka/message/ApiMessageTypeGenerator.java @@ -136,6 +136,10 @@ private void generate() { buffer.printf("%n"); generateNewApiMessageMethod("response"); buffer.printf("%n"); + generateAccessor("lowestSupportedVersion", "short"); + buffer.printf("%n"); + generateAccessor("highestSupportedVersion", "short"); + buffer.printf("%n"); generateAccessor("apiKey", "short"); buffer.printf("%n"); generateAccessor("requestSchemas", "Schema[]"); @@ -158,32 +162,39 @@ private void generateEnumValues() { ApiData apiData = entry.getValue(); String name = apiData.name(); numProcessed++; - buffer.printf("%s(\"%s\", (short) %d, %s, %s)%s%n", + buffer.printf("%s(\"%s\", (short) %d, %s, %s, (short) %d, (short) %d)%s%n", MessageGenerator.toSnakeCase(name).toUpperCase(Locale.ROOT), MessageGenerator.capitalizeFirst(name), entry.getKey(), apiData.requestSchema(), apiData.responseSchema(), + apiData.requestSpec.struct().versions().lowest(), + apiData.requestSpec.struct().versions().highest(), (numProcessed == apis.size()) ? ";" : ","); } } private void generateInstanceVariables() { - buffer.printf("private final String name;%n"); + buffer.printf("public final String name;%n"); buffer.printf("private final short apiKey;%n"); buffer.printf("private final Schema[] requestSchemas;%n"); buffer.printf("private final Schema[] responseSchemas;%n"); + buffer.printf("private final short lowestSupportedVersion;%n"); + buffer.printf("private final short highestSupportedVersion;%n"); headerGenerator.addImport(MessageGenerator.SCHEMA_CLASS); } private void generateEnumConstructor() { buffer.printf("ApiMessageType(String name, short apiKey, " + - "Schema[] requestSchemas, Schema[] responseSchemas) {%n"); + "Schema[] requestSchemas, Schema[] responseSchemas, " + + "short lowestSupportedVersion, short highestSupportedVersion) {%n"); buffer.incrementIndent(); buffer.printf("this.name = name;%n"); buffer.printf("this.apiKey = apiKey;%n"); buffer.printf("this.requestSchemas = requestSchemas;%n"); buffer.printf("this.responseSchemas = responseSchemas;%n"); + buffer.printf("this.lowestSupportedVersion = lowestSupportedVersion;%n"); + buffer.printf("this.highestSupportedVersion = highestSupportedVersion;%n"); buffer.decrementIndent(); buffer.printf("}%n"); } 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 83510046e3cc2..326916e5f14ac 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 @@ -20,9 +20,9 @@ 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.message.ListOffsetsRequestData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.requests.ListOffsetRequest; +import org.apache.kafka.common.requests.ListOffsetsRequest; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; @@ -54,9 +54,9 @@ public class ListOffsetRequestBenchmark { @Param({"3", "10", "20"}) private int partitionCount; - Map offsetData; + Map offsetData; - ListOffsetRequest offsetRequest; + ListOffsetsRequest offsetRequest; @Setup(Level.Trial) public void setup() { @@ -64,12 +64,12 @@ public void setup() { 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(); + ListOffsetsRequestData.ListOffsetsPartition data = new ListOffsetsRequestData.ListOffsetsPartition(); this.offsetData.put(new TopicPartition(topic, partitionId), data); } } - this.offsetRequest = ListOffsetRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED) + this.offsetRequest = ListOffsetsRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED) .build(ApiKeys.LIST_OFFSETS.latestVersion()); }