junrao commented on code in PR #14308: URL: https://github.com/apache/kafka/pull/14308#discussion_r1312321020
########## 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)); Review Comment: Does that actually wait for 100ms for the test to complete? If so, we probably want to use a smaller number. Ditto in some other tests. ########## 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, Review Comment: Why is the test named `UnknownOffset`? Also, how is this test different from `testListOffsetsRequest_Success`? ########## 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: This works, but is a bit unintuitive since Errors.NONE doesn't represent an error. ########## clients/src/test/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumerTest.java: ########## @@ -220,6 +225,95 @@ public void testAssignOnEmptyTopicInPartition() { } @Test + public void testBeginningOffsetsFailsIfNullPartitions() { + consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer()); + assertThrows(NullPointerException.class, () -> consumer.beginningOffsets(null, + Duration.ofMillis(1))); + } + + @Test + public void testBeginningOffsets() { + PrototypeAsyncConsumer<?, ?> consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer()); + Map<TopicPartition, OffsetAndTimestamp> expectedOffsetsAndTimestamp = + mockOffsetAndTimestamp(); + Set<TopicPartition> partitions = expectedOffsetsAndTimestamp.keySet(); + doReturn(expectedOffsetsAndTimestamp).when(eventHandler).addAndGet(any(), any()); + Map<TopicPartition, Long> result = + assertDoesNotThrow(() -> consumer.beginningOffsets(partitions, + Duration.ofMillis(1))); + Map<TopicPartition, Long> expectedOffsets = expectedOffsetsAndTimestamp.entrySet().stream() + .collect(Collectors.toMap(e -> e.getKey(), e -> e.getValue().offset())); + assertEquals(expectedOffsets, result); + verify(eventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + ArgumentMatchers.isA(Timer.class)); + } + + @Test + public void testBeginningOffsetsThrowsKafkaExceptionForUnderlyingExecutionFailure() { + PrototypeAsyncConsumer<?, ?> consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer()); + Set<TopicPartition> partitions = mockTopicPartitionOffset().keySet(); + Throwable eventProcessingFailure = new KafkaException("Unexpected failure " + + "processing List Offsets event"); + doThrow(eventProcessingFailure).when(eventHandler).addAndGet(any(), any()); + Throwable consumerError = assertThrows(KafkaException.class, + () -> consumer.beginningOffsets(partitions, + Duration.ofMillis(1))); + assertEquals(eventProcessingFailure, consumerError); + verify(eventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + } + + @Test + public void testBeginningOffsetsTimeoutOnEventProcessingTimeout() { + PrototypeAsyncConsumer<?, ?> consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer()); + doThrow(new TimeoutException()).when(eventHandler).addAndGet(any(), any()); + assertThrows(TimeoutException.class, + () -> consumer.beginningOffsets( + Collections.singletonList(new TopicPartition("t1", 0)), + Duration.ofMillis(1))); + verify(eventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + ArgumentMatchers.isA(Timer.class)); + } + + @Test + public void testOffsetsForTimesOnNullPartitions() { + PrototypeAsyncConsumer<?, ?> consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer()); + assertThrows(NullPointerException.class, () -> consumer.offsetsForTimes(null, + Duration.ofMillis(1))); + } + + @Test + public void testOffsetsForTimes() { + PrototypeAsyncConsumer<?, ?> consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer()); + Map<TopicPartition, OffsetAndTimestamp> expectedResult = mockOffsetAndTimestamp(); + Map<TopicPartition, Long> timestampToSearch = mockTimestampToSearch(); + + doReturn(expectedResult).when(eventHandler).addAndGet(any(), any()); + Map<TopicPartition, OffsetAndTimestamp> result = + assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, Duration.ofMillis(1))); + assertEquals(expectedResult, result); + verify(eventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + ArgumentMatchers.isA(Timer.class)); + } + + // This test ensures same behaviour as the current consumer when offsetsForTimes is called + // with 0 timeout. It should return map with all requested partitions as keys, with null + // OffsetAndTimestamp as value. + @Test + public void testOffsetsForTimesWithZeroTimeout() { + PrototypeAsyncConsumer<?, ?> consumer = newConsumer(time, new StringDeserializer(), new StringDeserializer()); + TopicPartition tp = new TopicPartition("topic1", 0); + Map<TopicPartition, OffsetAndTimestamp> expectedResult = + Collections.singletonMap(tp, null); + Map<TopicPartition, Long> timestampToSearch = Collections.singletonMap(tp, 5L); + + Map<TopicPartition, OffsetAndTimestamp> result = + assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, + Duration.ofMillis(0))); + assertEquals(expectedResult, result); + verify(eventHandler, never()).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + ArgumentMatchers.isA(Timer.class)); + } Review Comment: This is an existing issue. `testWakeup_commitSync` and `clientId` seem unused? ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/DefaultBackgroundThread.java: ########## @@ -42,6 +42,7 @@ import static java.util.Objects.requireNonNull; import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_MAX_INFLIGHT_REQUESTS_PER_CONNECTION; import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_METRIC_GROUP_PREFIX; +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.configuredIsolationLevel; Review Comment: This can be done in a separate PR. The logic in DefaultBackgroundThread is a bit involved. It would be useful to add a comment to document the life cycle of how an event is handled through ApplicationEventProcessor and NetworkClientDelegate. ########## 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); + } + + private void testResponseWithErrorAndUnknownOffsets(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(); + + // Null offsets should be returned for each partition + Map<TopicPartition, OffsetAndTimestamp> expectedOffsets = Collections.singletonMap(TEST_PARTITION_1, null); + verifyRequestSuccessfullyCompleted(fetchOffsetsFuture, expectedOffsets); + } + + @Test + public void testRequestPartiallyFailsWithRetriableError_RetrySucceeds() 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, OffsetAndTimestamp> expectedOffsets = timestampsToSearch.entrySet().stream() + .collect(Collectors.toMap(e -> e.getKey(), e -> new OffsetAndTimestamp(5L, 1L))); + + // List offsets request to 2 brokers successfully built + Map<TopicPartition, Node> partitionLeaders = new HashMap<>(); + partitionLeaders.put(TEST_PARTITION_1, LEADER_1); + partitionLeaders.put(TEST_PARTITION_2, LEADER_2); + expectSuccessfulRequest(partitionLeaders); + CompletableFuture<Map<TopicPartition, OffsetAndTimestamp>> fetchOffsetsFuture = requestManager.fetchOffsets( + timestampsToSearch, + false); + assertEquals(2, requestManager.requestsToSend()); + assertEquals(0, requestManager.requestsToRetry()); + + // Requests successfully sent to both brokers + NetworkClientDelegate.PollResult res = requestManager.poll(time.milliseconds()); + verifySuccessfulPollAwaitingResponse(res, 2); + assertFalse(fetchOffsetsFuture.isDone()); + + // Mixed response with failures and successes. Offsets successfully fetched from one + // broker but retriable UNKNOWN_LEADER_EPOCH received from second broker. + NetworkClientDelegate.UnsentRequest unsentRequest1 = res.unsentRequests.get(0); + ClientResponse clientResponse1 = buildClientResponse( + unsentRequest1, + Collections.singletonMap(TEST_PARTITION_1, expectedOffsets.get(TEST_PARTITION_1))); + clientResponse1.onComplete(); + NetworkClientDelegate.UnsentRequest unsentRequest2 = res.unsentRequests.get(1); + ClientResponse clientResponse2 = buildClientResponseWithErrors( + unsentRequest2, + Collections.singletonMap(TEST_PARTITION_2, Errors.UNKNOWN_LEADER_EPOCH)); + clientResponse2.onComplete(); + + assertFalse(fetchOffsetsFuture.isDone()); + assertEquals(1, requestManager.requestsToRetry()); + assertEquals(0, requestManager.requestsToSend()); + + // Cluster metadata update. Failed requests should be retried + expectSuccessfulRequest(partitionLeaders); + requestManager.onUpdate(new ClusterResource("")); + assertEquals(1, requestManager.requestsToSend()); + + // Following poll should send the request and get a successful response + NetworkClientDelegate.PollResult retriedPoll = requestManager.poll(time.milliseconds()); + verifySuccessfulPollAwaitingResponse(retriedPoll); + NetworkClientDelegate.UnsentRequest unsentRequest = retriedPoll.unsentRequests.get(0); + ClientResponse clientResponse = buildClientResponse(unsentRequest, + Collections.singletonMap(TEST_PARTITION_2, expectedOffsets.get(TEST_PARTITION_2))); + clientResponse.onComplete(); + + // Verify global result with the offset initially retrieved, and the offset that + // initially failed but succeeded after a metadata update + verifyRequestSuccessfullyCompleted(fetchOffsetsFuture, expectedOffsets); + } + + @Test + public void testRequestFailedResponse_NonRetriableAuthError() { + 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 + NetworkClientDelegate.PollResult res = requestManager.poll(time.milliseconds()); + verifySuccessfulPollAwaitingResponse(res); + + // Response received with non-retriable auth error + NetworkClientDelegate.UnsentRequest unsentRequest = res.unsentRequests.get(0); + ClientResponse clientResponse = buildClientResponseWithErrors( + unsentRequest, Collections.singletonMap(TEST_PARTITION_2, Errors.TOPIC_AUTHORIZATION_FAILED)); + clientResponse.onComplete(); + + verifyRequestCompletedWithErrorResponse(fetchOffsetsFuture, TopicAuthorizationException.class); + assertEquals(0, requestManager.requestsToRetry()); + assertEquals(0, requestManager.requestsToSend()); + } + + @Test + public void testRequestFailedResponse_NonRetriableErrorTimeout() { + 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 + NetworkClientDelegate.PollResult res = requestManager.poll(time.milliseconds()); + verifySuccessfulPollAwaitingResponse(res); + + // Response received + NetworkClientDelegate.UnsentRequest unsentRequest = res.unsentRequests.get(0); + ClientResponse clientResponse = buildClientResponseWithErrors( + unsentRequest, Collections.singletonMap(TEST_PARTITION_2, Errors.BROKER_NOT_AVAILABLE)); + clientResponse.onComplete(); + + assertFalse(fetchOffsetsFuture.isDone()); + assertThrows(TimeoutException.class, () -> fetchOffsetsFuture.get(100L, TimeUnit.MILLISECONDS)); + + // Request completed with error. Nothing pending to be sent or retried + assertEquals(0, requestManager.requestsToRetry()); + assertEquals(0, requestManager.requestsToSend()); + } + + @Test + public void testRequestFails_AuthenticationException() { + 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 + NetworkClientDelegate.PollResult res = requestManager.poll(time.milliseconds()); + verifySuccessfulPollAwaitingResponse(res); + + // Response received with auth error + NetworkClientDelegate.UnsentRequest unsentRequest = res.unsentRequests.get(0); + ClientResponse clientResponse = + buildClientResponseWithAuthenticationException(unsentRequest); + clientResponse.onComplete(); + + // Request completed with error. Nothing pending to be sent or retried + verifyRequestCompletedWithErrorResponse(fetchOffsetsFuture, AuthenticationException.class); + assertEquals(0, requestManager.requestsToRetry()); + assertEquals(0, requestManager.requestsToSend()); + } + + private ListOffsetsResponseData.ListOffsetsTopicResponse mockUnknownOffsetResponse( + TopicPartition tp) { + return new ListOffsetsResponseData.ListOffsetsTopicResponse() + .setName(tp.topic()) + .setPartitions(Collections.singletonList(new ListOffsetsResponseData.ListOffsetsPartitionResponse() + .setPartitionIndex(tp.partition()) + .setErrorCode(Errors.NONE.code()) + .setTimestamp(ListOffsetsResponse.UNKNOWN_TIMESTAMP) + .setOffset(ListOffsetsResponse.UNKNOWN_OFFSET))); + } + + private static Stream<Arguments> retriableErrors() { + return Stream.of( + Arguments.of(Errors.NOT_LEADER_OR_FOLLOWER), + Arguments.of(Errors.REPLICA_NOT_AVAILABLE), + Arguments.of(Errors.KAFKA_STORAGE_ERROR), + Arguments.of(Errors.OFFSET_NOT_AVAILABLE), + Arguments.of(Errors.LEADER_NOT_AVAILABLE), + Arguments.of(Errors.FENCED_LEADER_EPOCH), + Arguments.of(Errors.BROKER_NOT_AVAILABLE), + Arguments.of(Errors.INVALID_REQUEST), + Arguments.of(Errors.UNKNOWN_LEADER_EPOCH), + Arguments.of(Errors.UNKNOWN_TOPIC_OR_PARTITION)); + } + + private void verifySuccessfulPollAndResponseReceived( + CompletableFuture<Map<TopicPartition, OffsetAndTimestamp>> actualResult, + Map<TopicPartition, OffsetAndTimestamp> expectedResult) throws ExecutionException, + InterruptedException { + // Following poll should send the request and get a response + NetworkClientDelegate.PollResult retriedPoll = requestManager.poll(time.milliseconds()); + verifySuccessfulPollAwaitingResponse(retriedPoll); + NetworkClientDelegate.UnsentRequest unsentRequest = retriedPoll.unsentRequests.get(0); + ClientResponse clientResponse = buildClientResponse(unsentRequest, + expectedResult); + clientResponse.onComplete(); + verifyRequestSuccessfullyCompleted(actualResult, expectedResult); + } + + private void expectSuccessfulRequest(Map<TopicPartition, Node> partitionLeaders) { + partitionLeaders.forEach((tp, broker) -> { + when(metadata.currentLeader(tp)).thenReturn(testLeaderEpoch(broker)); + when(subscriptionState.isAssigned(tp)).thenReturn(true); + }); + when(metadata.fetch()).thenReturn(testClusterMetadata(partitionLeaders)); + } + + private void expectFailedRequest_MissingLeader() { + when(metadata.currentLeader(any(TopicPartition.class))).thenReturn( + new Metadata.LeaderAndEpoch(Optional.empty(), Optional.of(1))); Review Comment: If the leader is missing, typically there won't be an epoch, right? -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
