lianetm commented on code in PR #14308: URL: https://github.com/apache/kafka/pull/14308#discussion_r1313134230
########## clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManagerTest.java: ########## @@ -0,0 +1,675 @@ +/* + * 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.clients.consumer.internals; + +import org.apache.kafka.clients.ApiVersions; +import org.apache.kafka.clients.ClientResponse; +import org.apache.kafka.clients.Metadata; +import org.apache.kafka.clients.consumer.OffsetAndTimestamp; +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.ClusterResource; +import org.apache.kafka.common.IsolationLevel; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.AuthenticationException; +import org.apache.kafka.common.errors.TopicAuthorizationException; +import org.apache.kafka.common.message.ListOffsetsResponseData; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.AbstractRequest; +import org.apache.kafka.common.requests.ListOffsetsRequest; +import org.apache.kafka.common.requests.ListOffsetsResponse; +import org.apache.kafka.common.requests.RequestHeader; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.MockTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import org.mockito.ArgumentCaptor; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class OffsetsRequestManagerTest { + + private OffsetsRequestManager requestManager; + private ConsumerMetadata metadata; + private SubscriptionState subscriptionState; + private MockTime time; + private static final String TEST_TOPIC = "t1"; + private static final TopicPartition TEST_PARTITION_1 = new TopicPartition(TEST_TOPIC, 1); + private static final TopicPartition TEST_PARTITION_2 = new TopicPartition(TEST_TOPIC, 2); + private static final Node LEADER_1 = new Node(0, "host1", 9092); + private static final Node LEADER_2 = new Node(0, "host2", 9092); + private static final IsolationLevel DEFAULT_ISOLATION_LEVEL = IsolationLevel.READ_COMMITTED; + private static final int RETRY_BACKOFF_MS = 500; + + @BeforeEach + public void setup() { + metadata = mock(ConsumerMetadata.class); + subscriptionState = mock(SubscriptionState.class); + this.time = new MockTime(0); + ApiVersions apiVersions = mock(ApiVersions.class); + requestManager = new OffsetsRequestManager(subscriptionState, metadata, + DEFAULT_ISOLATION_LEVEL, time, RETRY_BACKOFF_MS, + apiVersions, new LogContext()); + } + + @Test + public void testListOffsetsRequest_Success() throws ExecutionException, InterruptedException { + Map<TopicPartition, Long> timestampsToSearch = Collections.singletonMap(TEST_PARTITION_1, + ListOffsetsRequest.EARLIEST_TIMESTAMP); + + expectSuccessfulRequest(Collections.singletonMap(TEST_PARTITION_1, LEADER_1)); + CompletableFuture<Map<TopicPartition, OffsetAndTimestamp>> result = requestManager.fetchOffsets( + timestampsToSearch, + false); + assertEquals(1, requestManager.requestsToSend()); + assertEquals(0, requestManager.requestsToRetry()); + + Map<TopicPartition, OffsetAndTimestamp> expectedOffsets = Collections.singletonMap(TEST_PARTITION_1, new OffsetAndTimestamp(5L, 1L)); + verifySuccessfulPollAndResponseReceived(result, expectedOffsets); + } + + @Test + public void testListOffsetsRequest_UnknownOffset() throws ExecutionException, + InterruptedException { + Map<TopicPartition, Long> timestampsToSearch = Collections.singletonMap(TEST_PARTITION_1, + ListOffsetsRequest.EARLIEST_TIMESTAMP); + + expectSuccessfulRequest(Collections.singletonMap(TEST_PARTITION_1, LEADER_1)); + CompletableFuture<Map<TopicPartition, OffsetAndTimestamp>> result = requestManager.fetchOffsets( + timestampsToSearch, + false); + assertEquals(1, requestManager.requestsToSend()); + assertEquals(0, requestManager.requestsToRetry()); + + Map<TopicPartition, OffsetAndTimestamp> expectedOffsets = Collections.singletonMap(TEST_PARTITION_1, new OffsetAndTimestamp(5L, 1L)); + verifySuccessfulPollAndResponseReceived(result, expectedOffsets); + } + + @Test + public void testListOffsetsWaitingForMetadataUpdate_Timeout() { + Map<TopicPartition, Long> timestampsToSearch = Collections.singletonMap(TEST_PARTITION_1, + ListOffsetsRequest.EARLIEST_TIMESTAMP); + + // Building list offsets request fails with unknown leader + expectFailedRequest_MissingLeader(); + CompletableFuture<Map<TopicPartition, OffsetAndTimestamp>> fetchOffsetsFuture = + requestManager.fetchOffsets(timestampsToSearch, false); + assertEquals(0, requestManager.requestsToSend()); + assertEquals(1, requestManager.requestsToRetry()); + NetworkClientDelegate.PollResult res = requestManager.poll(time.milliseconds()); + assertEquals(0, res.unsentRequests.size()); + // Metadata update not happening within the time boundaries of the request future, so + // future should time out. + assertThrows(TimeoutException.class, () -> fetchOffsetsFuture.get(100L, TimeUnit.MILLISECONDS)); + } + + @Test + public void testListOffsetsRequestMultiplePartitions() throws ExecutionException, + InterruptedException { + Map<TopicPartition, Long> timestampsToSearch = new HashMap<>(); + timestampsToSearch.put(TEST_PARTITION_1, ListOffsetsRequest.EARLIEST_TIMESTAMP); + timestampsToSearch.put(TEST_PARTITION_2, ListOffsetsRequest.EARLIEST_TIMESTAMP); + + + Map<TopicPartition, Node> partitionLeaders = new HashMap<>(); + partitionLeaders.put(TEST_PARTITION_1, LEADER_1); + partitionLeaders.put(TEST_PARTITION_2, LEADER_1); + expectSuccessfulRequest(partitionLeaders); + CompletableFuture<Map<TopicPartition, OffsetAndTimestamp>> result = requestManager.fetchOffsets( + timestampsToSearch, + false); + assertEquals(1, requestManager.requestsToSend()); + assertEquals(0, requestManager.requestsToRetry()); + + Map<TopicPartition, OffsetAndTimestamp> expectedOffsets = timestampsToSearch.entrySet().stream() + .collect(Collectors.toMap(e -> e.getKey(), e -> new OffsetAndTimestamp(5L, 1L))); + verifySuccessfulPollAndResponseReceived(result, expectedOffsets); + } + + @Test + public void testListOffsetsRequestEmpty() throws ExecutionException, InterruptedException { + CompletableFuture<Map<TopicPartition, OffsetAndTimestamp>> result = requestManager.fetchOffsets( + Collections.emptyMap(), + false); + assertEquals(0, requestManager.requestsToSend()); + assertEquals(0, requestManager.requestsToRetry()); + + NetworkClientDelegate.PollResult pollResult = requestManager.poll(time.milliseconds()); + assertTrue(pollResult.unsentRequests.isEmpty()); + + assertEquals(0, requestManager.requestsToRetry()); + assertEquals(0, requestManager.requestsToSend()); + + assertTrue(result.isDone()); + assertFalse(result.isCompletedExceptionally()); + assertTrue(result.get().isEmpty()); + } + + @Test + public void testListOffsetsRequestUnknownOffset() throws ExecutionException, + InterruptedException { + Map<TopicPartition, Long> timestampsToSearch = Collections.singletonMap(TEST_PARTITION_1, + ListOffsetsRequest.EARLIEST_TIMESTAMP); + + expectSuccessfulRequest(Collections.singletonMap(TEST_PARTITION_1, LEADER_1)); + CompletableFuture<Map<TopicPartition, OffsetAndTimestamp>> result = requestManager.fetchOffsets( + timestampsToSearch, + false); + assertEquals(1, requestManager.requestsToSend()); + assertEquals(0, requestManager.requestsToRetry()); + + List<ListOffsetsResponseData.ListOffsetsTopicResponse> topicResponses = Collections.singletonList( + mockUnknownOffsetResponse(TEST_PARTITION_1)); + + NetworkClientDelegate.PollResult retriedPoll = requestManager.poll(time.milliseconds()); + verifySuccessfulPollAwaitingResponse(retriedPoll); + NetworkClientDelegate.UnsentRequest unsentRequest = retriedPoll.unsentRequests.get(0); + ClientResponse clientResponse = buildClientResponse(unsentRequest, topicResponses); + clientResponse.onComplete(); + + Map<TopicPartition, OffsetAndTimestamp> expectedOffsets = + Collections.singletonMap(TEST_PARTITION_1, null); + verifyRequestSuccessfullyCompleted(result, expectedOffsets); + + } + + @Test + public void testListOffsetsWaitingForMetadataUpdate_RetrySucceeds() throws ExecutionException, + InterruptedException { + Map<TopicPartition, Long> timestampsToSearch = Collections.singletonMap(TEST_PARTITION_1, + ListOffsetsRequest.EARLIEST_TIMESTAMP); + + // Building list offsets request fails with unknown leader + expectFailedRequest_MissingLeader(); + CompletableFuture<Map<TopicPartition, OffsetAndTimestamp>> fetchOffsetsFuture = + requestManager.fetchOffsets(timestampsToSearch, + false); + assertEquals(0, requestManager.requestsToSend()); + assertEquals(1, requestManager.requestsToRetry()); + NetworkClientDelegate.PollResult res = requestManager.poll(time.milliseconds()); + assertEquals(0, res.unsentRequests.size()); + assertFalse(fetchOffsetsFuture.isDone()); + + // Cluster metadata update. Previously failed attempt to build the request should be retried + // and succeed + expectSuccessfulRequest(Collections.singletonMap(TEST_PARTITION_1, LEADER_1)); + requestManager.onUpdate(new ClusterResource("")); + assertEquals(1, requestManager.requestsToSend()); + + Map<TopicPartition, OffsetAndTimestamp> expectedOffsets = Collections.singletonMap( + TEST_PARTITION_1, new OffsetAndTimestamp(5L, 1L)); + verifySuccessfulPollAndResponseReceived(fetchOffsetsFuture, expectedOffsets); + } + + @ParameterizedTest + @MethodSource("retriableErrors") + public void testRequestFailsWithRetriableError_RetrySucceeds(Errors error) throws ExecutionException, InterruptedException { + Map<TopicPartition, Long> timestampsToSearch = Collections.singletonMap(TEST_PARTITION_1, + ListOffsetsRequest.EARLIEST_TIMESTAMP); + + // List offsets request successfully built + expectSuccessfulRequest(Collections.singletonMap(TEST_PARTITION_1, LEADER_1)); + CompletableFuture<Map<TopicPartition, OffsetAndTimestamp>> fetchOffsetsFuture = requestManager.fetchOffsets( + timestampsToSearch, + false); + assertEquals(1, requestManager.requestsToSend()); + assertEquals(0, requestManager.requestsToRetry()); + + // Request successfully sent to single broker + NetworkClientDelegate.PollResult res = requestManager.poll(time.milliseconds()); + verifySuccessfulPollAwaitingResponse(res); + assertFalse(fetchOffsetsFuture.isDone()); + + // Response received with error + NetworkClientDelegate.UnsentRequest unsentRequest = res.unsentRequests.get(0); + ClientResponse clientResponse = buildClientResponseWithErrors( + unsentRequest, + Collections.singletonMap(TEST_PARTITION_1, error)); + clientResponse.onComplete(); + assertFalse(fetchOffsetsFuture.isDone()); + assertEquals(1, requestManager.requestsToRetry()); + assertEquals(0, requestManager.requestsToSend()); + + // Cluster metadata update. Failed requests should be retried and succeed + expectSuccessfulRequest(Collections.singletonMap(TEST_PARTITION_1, LEADER_1)); + requestManager.onUpdate(new ClusterResource("")); + assertEquals(1, requestManager.requestsToSend()); + + Map<TopicPartition, OffsetAndTimestamp> expectedOffsets = Collections.singletonMap(TEST_PARTITION_1, new OffsetAndTimestamp(5L, 1L)); + verifySuccessfulPollAndResponseReceived(fetchOffsetsFuture, expectedOffsets); + } + + @Test + public void testRequestNotSupportedErrorReturnsNullOffset() throws ExecutionException, + InterruptedException { + testResponseWithErrorAndUnknownOffsets(Errors.UNSUPPORTED_FOR_MESSAGE_FORMAT); + } + + @Test + public void testRequestWithUnknownOffsetInResponseReturnsNullOffset() throws ExecutionException, + InterruptedException { + testResponseWithErrorAndUnknownOffsets(Errors.NONE); Review Comment: Agree. Given that the logic is the same I will opt for renaming it to "error code" instead of "error", to better group Errors.NONE and Errors.UNSUPPORTED_FOR_MESSAGE_FORMAT. So including `testResponseWithErrorCodeAndUnknownOffsets` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org