From 8159e51a8c3cee09f9aab16644630834061fd1fd Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Tue, 5 Dec 2023 15:18:23 +0100 Subject: [PATCH 01/21] migrate epoch update to mocks --- .../internals/AsyncKafkaConsumerTest.java | 28 --- .../internals/AsyncKafkaConsumerUnitTest.java | 166 ++++++++++++++++++ 2 files changed, 166 insertions(+), 28 deletions(-) create mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index d39b63e979022..cacd2ebc27034 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -185,13 +185,6 @@ public void testInvalidGroupId() { assertThrows(InvalidGroupIdException.class, () -> consumer.committed(new HashSet<>())); } - @Test - public void testFailOnClosedConsumer() { - consumer.close(); - final IllegalStateException res = assertThrows(IllegalStateException.class, consumer::assignment); - assertEquals("This consumer has already been closed.", res.getMessage()); - } - @Test public void testCommitAsync_NullCallback() throws InterruptedException { CompletableFuture future = new CompletableFuture<>(); @@ -259,27 +252,6 @@ public void testCommitted() { } } - @Test - public void testCommittedLeaderEpochUpdate() { - final TopicPartition t0 = new TopicPartition("t0", 2); - final TopicPartition t1 = new TopicPartition("t0", 3); - final TopicPartition t2 = new TopicPartition("t0", 4); - HashMap topicPartitionOffsets = new HashMap<>(); - topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L, Optional.of(2), "")); - topicPartitionOffsets.put(t1, null); - topicPartitionOffsets.put(t2, new OffsetAndMetadata(20L, Optional.of(3), "")); - - CompletableFuture> committedFuture = new CompletableFuture<>(); - committedFuture.complete(topicPartitionOffsets); - - try (MockedConstruction ignored = offsetFetchEventMocker(committedFuture)) { - assertDoesNotThrow(() -> consumer.committed(topicPartitionOffsets.keySet(), Duration.ofMillis(1000))); - } - verify(testBuilder.metadata).updateLastSeenEpochIfNewer(t0, 2); - verify(testBuilder.metadata).updateLastSeenEpochIfNewer(t2, 3); - verify(applicationEventHandler).add(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class)); - } - @Test public void testCommitted_ExceptionThrown() { Map offsets = mockTopicPartitionOffset(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java new file mode 100644 index 0000000000000..dbfb9b91e2e88 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java @@ -0,0 +1,166 @@ +/* + * 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 static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertEquals; +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.verify; +import static org.mockito.Mockito.when; + +import java.time.Duration; +import java.util.Arrays; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.BlockingQueue; +import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.consumer.OffsetCommitCallback; +import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; +import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.FetchCommittedOffsetsApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.OffsetFetchApplicationEvent; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mockito; + +@SuppressWarnings("unchecked") +public class AsyncKafkaConsumerUnitTest { + + private final LogContext logContext = new LogContext(); + private AsyncKafkaConsumer consumer; + private final Deserializers deserializers = mock(Deserializers.class); + private final FetchBuffer fetchBuffer = mock(FetchBuffer.class); + private final FetchCollector fetchCollector = mock(FetchCollector.class); + private final ConsumerInterceptors interceptors = mock(ConsumerInterceptors.class); + private final Time time = new MockTime(); + private final ApplicationEventHandler applicationEventHandler = mock(ApplicationEventHandler.class); + private final BlockingQueue backgroundEventQueue = mock(BlockingQueue.class); + private final Metrics metrics = new Metrics(); + private final SubscriptionState subscriptions = mock(SubscriptionState.class); + private final ConsumerMetadata metadata = mock(ConsumerMetadata.class); + private final List assignors = new LinkedList<>(); + + @BeforeEach + public void setup() { + backgroundEventQueue.clear(); + assignors.clear(); + String clientId = ""; + long retryBackoffMs = 100; + int defaultApiTimeoutMs = 100; + String groupId = "group-id"; + consumer = new AsyncKafkaConsumer<>( + logContext, + clientId, + deserializers, + fetchBuffer, + fetchCollector, + interceptors, + time, + applicationEventHandler, + backgroundEventQueue, + metrics, + subscriptions, + metadata, + retryBackoffMs, + defaultApiTimeoutMs, + assignors, + groupId + ); + } + + @AfterEach + public void cleanup() { + consumer.close(Duration.ZERO); + } + + @Test + public void testCommitSyncLeaderEpochUpdate() { + final TopicPartition t0 = new TopicPartition("t0", 2); + final TopicPartition t1 = new TopicPartition("t0", 3); + HashMap topicPartitionOffsets = new HashMap<>(); + topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L, Optional.of(2), "")); + topicPartitionOffsets.put(t1, new OffsetAndMetadata(20L, Optional.of(1), "")); + + consumer.assign(Arrays.asList(t0, t1)); + + Mockito.doAnswer(invocation -> { + CompletableApplicationEvent event = invocation.getArgument(0); + assertTrue(event instanceof CommitApplicationEvent); + event.future().complete(null); + return null; + }).when(applicationEventHandler).add(any()); + + assertDoesNotThrow(() -> consumer.commitSync(topicPartitionOffsets)); + + verify(metadata).updateLastSeenEpochIfNewer(t0, 2); + verify(metadata).updateLastSeenEpochIfNewer(t1, 1); + } + + @Test + public void testCommitAsyncLeaderEpochUpdate() { + OffsetCommitCallback callback = mock(OffsetCommitCallback.class); + final TopicPartition t0 = new TopicPartition("t0", 2); + final TopicPartition t1 = new TopicPartition("t0", 3); + HashMap topicPartitionOffsets = new HashMap<>(); + topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L, Optional.of(2), "")); + topicPartitionOffsets.put(t1, new OffsetAndMetadata(20L, Optional.of(1), "")); + consumer.assign(Arrays.asList(t0, t1)); + + consumer.commitAsync(topicPartitionOffsets, callback); + + verify(metadata).updateLastSeenEpochIfNewer(t0, 2); + verify(metadata).updateLastSeenEpochIfNewer(t1, 1); + } + + @Test + public void testCommittedLeaderEpochUpdate() { + final TopicPartition t0 = new TopicPartition("t0", 2); + final TopicPartition t1 = new TopicPartition("t0", 3); + final TopicPartition t2 = new TopicPartition("t0", 4); + HashMap topicPartitionOffsets = new HashMap<>(); + topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L, Optional.of(2), "")); + topicPartitionOffsets.put(t1, null); + topicPartitionOffsets.put(t2, new OffsetAndMetadata(20L, Optional.of(3), "")); + + when(applicationEventHandler.addAndGet(any(), any())).thenAnswer(invocation -> { + CompletableApplicationEvent event = invocation.getArgument(0); + assertTrue(event instanceof FetchCommittedOffsetsApplicationEvent); + return topicPartitionOffsets; + }); + + assertDoesNotThrow(() -> consumer.committed(topicPartitionOffsets.keySet(), Duration.ofMillis(1000))); + + verify(metadata).updateLastSeenEpochIfNewer(t0, 2); + verify(metadata).updateLastSeenEpochIfNewer(t2, 3); + } + +} + From 5b95d0331fe6fa45d92be684fe25ad32bde308fa Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Tue, 5 Dec 2023 15:34:54 +0100 Subject: [PATCH 02/21] migrate fenced exception --- .../internals/AsyncKafkaConsumerTest.java | 16 -- .../internals/AsyncKafkaConsumerUnitTest.java | 159 ++++++++++-------- 2 files changed, 92 insertions(+), 83 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index cacd2ebc27034..b61fba6c7f020 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -45,7 +45,6 @@ import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.GroupAuthorizationException; -import org.apache.kafka.common.errors.InvalidGroupIdException; import org.apache.kafka.common.errors.NetworkException; import org.apache.kafka.common.errors.RetriableException; import org.apache.kafka.common.errors.TimeoutException; @@ -79,7 +78,6 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; @@ -178,12 +176,6 @@ public void testSuccessfulStartupShutdownWithAutoCommit() { prepAutocommitOnClose(); } - @Test - public void testInvalidGroupId() { - // Create consumer without group id - resetWithEmptyGroupId(); - assertThrows(InvalidGroupIdException.class, () -> consumer.committed(new HashSet<>())); - } @Test public void testCommitAsync_NullCallback() throws InterruptedException { @@ -232,14 +224,6 @@ private static Stream commitExceptionSupplier() { new GroupAuthorizationException("Group authorization exception")); } - @Test - public void testFencedInstanceException() { - CompletableFuture future = new CompletableFuture<>(); - doReturn(future).when(consumer).commit(new HashMap<>(), false); - assertDoesNotThrow(() -> consumer.commitAsync()); - future.completeExceptionally(Errors.FENCED_INSTANCE_ID.exception()); - } - @Test public void testCommitted() { Map offsets = mockTopicPartitionOffset(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java index dbfb9b91e2e88..aaa32b511ffab 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java @@ -17,10 +17,10 @@ package org.apache.kafka.clients.consumer.internals; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; -import static org.junit.jupiter.api.Assertions.assertEquals; 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.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -28,11 +28,12 @@ import java.time.Duration; import java.util.Arrays; import java.util.HashMap; +import java.util.HashSet; import java.util.LinkedList; -import java.util.List; import java.util.Optional; import java.util.concurrent.BlockingQueue; -import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor; +import java.util.concurrent.CompletableFuture; +import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetCommitCallback; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; @@ -42,42 +43,42 @@ import org.apache.kafka.clients.consumer.internals.events.FetchCommittedOffsetsApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.OffsetFetchApplicationEvent; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.InvalidGroupIdException; import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.mockito.Mockito; @SuppressWarnings("unchecked") public class AsyncKafkaConsumerUnitTest { - private final LogContext logContext = new LogContext(); - private AsyncKafkaConsumer consumer; + private final Time time = new MockTime(); private final Deserializers deserializers = mock(Deserializers.class); private final FetchBuffer fetchBuffer = mock(FetchBuffer.class); private final FetchCollector fetchCollector = mock(FetchCollector.class); private final ConsumerInterceptors interceptors = mock(ConsumerInterceptors.class); - private final Time time = new MockTime(); private final ApplicationEventHandler applicationEventHandler = mock(ApplicationEventHandler.class); private final BlockingQueue backgroundEventQueue = mock(BlockingQueue.class); - private final Metrics metrics = new Metrics(); private final SubscriptionState subscriptions = mock(SubscriptionState.class); private final ConsumerMetadata metadata = mock(ConsumerMetadata.class); - private final List assignors = new LinkedList<>(); - @BeforeEach - public void setup() { - backgroundEventQueue.clear(); - assignors.clear(); + private AsyncKafkaConsumer setup() { + return setup("group-id"); + } + + private AsyncKafkaConsumer setupWithEmptyGroupId() { + return setup(""); + } + + private AsyncKafkaConsumer setup(String groupId) { String clientId = ""; long retryBackoffMs = 100; int defaultApiTimeoutMs = 100; - String groupId = "group-id"; - consumer = new AsyncKafkaConsumer<>( - logContext, + return new AsyncKafkaConsumer<>( + new LogContext(), clientId, deserializers, fetchBuffer, @@ -86,80 +87,104 @@ public void setup() { time, applicationEventHandler, backgroundEventQueue, - metrics, + new Metrics(), subscriptions, metadata, retryBackoffMs, defaultApiTimeoutMs, - assignors, + new LinkedList<>(), groupId ); } - @AfterEach - public void cleanup() { - consumer.close(Duration.ZERO); + @Test + public void testInvalidGroupId() { + try (AsyncKafkaConsumer consumer = setupWithEmptyGroupId()) { + assertThrows(InvalidGroupIdException.class, () -> consumer.committed(new HashSet<>())); + } + } + + @Test + public void testFencedInstanceException() { + try (AsyncKafkaConsumer consumer = setup()) { + + Mockito.doAnswer(invocation -> { + CompletableApplicationEvent event = invocation.getArgument(0); + assertTrue(event instanceof CommitApplicationEvent); + event.future().completeExceptionally(Errors.FENCED_INSTANCE_ID.exception()); + return null; + }).when(applicationEventHandler).add(any()); + + assertDoesNotThrow(() -> consumer.commitAsync()); + } } @Test public void testCommitSyncLeaderEpochUpdate() { - final TopicPartition t0 = new TopicPartition("t0", 2); - final TopicPartition t1 = new TopicPartition("t0", 3); - HashMap topicPartitionOffsets = new HashMap<>(); - topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L, Optional.of(2), "")); - topicPartitionOffsets.put(t1, new OffsetAndMetadata(20L, Optional.of(1), "")); + try (AsyncKafkaConsumer consumer = setup()) { + + final TopicPartition t0 = new TopicPartition("t0", 2); + final TopicPartition t1 = new TopicPartition("t0", 3); + HashMap topicPartitionOffsets = new HashMap<>(); + topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L, Optional.of(2), "")); + topicPartitionOffsets.put(t1, new OffsetAndMetadata(20L, Optional.of(1), "")); - consumer.assign(Arrays.asList(t0, t1)); + consumer.assign(Arrays.asList(t0, t1)); - Mockito.doAnswer(invocation -> { - CompletableApplicationEvent event = invocation.getArgument(0); - assertTrue(event instanceof CommitApplicationEvent); - event.future().complete(null); - return null; - }).when(applicationEventHandler).add(any()); + Mockito.doAnswer(invocation -> { + CompletableApplicationEvent event = invocation.getArgument(0); + assertTrue(event instanceof CommitApplicationEvent); + event.future().complete(null); + return null; + }).when(applicationEventHandler).add(any()); - assertDoesNotThrow(() -> consumer.commitSync(topicPartitionOffsets)); + assertDoesNotThrow(() -> consumer.commitSync(topicPartitionOffsets)); - verify(metadata).updateLastSeenEpochIfNewer(t0, 2); - verify(metadata).updateLastSeenEpochIfNewer(t1, 1); + verify(metadata).updateLastSeenEpochIfNewer(t0, 2); + verify(metadata).updateLastSeenEpochIfNewer(t1, 1); + } } @Test public void testCommitAsyncLeaderEpochUpdate() { - OffsetCommitCallback callback = mock(OffsetCommitCallback.class); - final TopicPartition t0 = new TopicPartition("t0", 2); - final TopicPartition t1 = new TopicPartition("t0", 3); - HashMap topicPartitionOffsets = new HashMap<>(); - topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L, Optional.of(2), "")); - topicPartitionOffsets.put(t1, new OffsetAndMetadata(20L, Optional.of(1), "")); - consumer.assign(Arrays.asList(t0, t1)); - - consumer.commitAsync(topicPartitionOffsets, callback); - - verify(metadata).updateLastSeenEpochIfNewer(t0, 2); - verify(metadata).updateLastSeenEpochIfNewer(t1, 1); + try (AsyncKafkaConsumer consumer = setup()) { + OffsetCommitCallback callback = mock(OffsetCommitCallback.class); + final TopicPartition t0 = new TopicPartition("t0", 2); + final TopicPartition t1 = new TopicPartition("t0", 3); + HashMap topicPartitionOffsets = new HashMap<>(); + topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L, Optional.of(2), "")); + topicPartitionOffsets.put(t1, new OffsetAndMetadata(20L, Optional.of(1), "")); + consumer.assign(Arrays.asList(t0, t1)); + + consumer.commitAsync(topicPartitionOffsets, callback); + + verify(metadata).updateLastSeenEpochIfNewer(t0, 2); + verify(metadata).updateLastSeenEpochIfNewer(t1, 1); + } } @Test public void testCommittedLeaderEpochUpdate() { - final TopicPartition t0 = new TopicPartition("t0", 2); - final TopicPartition t1 = new TopicPartition("t0", 3); - final TopicPartition t2 = new TopicPartition("t0", 4); - HashMap topicPartitionOffsets = new HashMap<>(); - topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L, Optional.of(2), "")); - topicPartitionOffsets.put(t1, null); - topicPartitionOffsets.put(t2, new OffsetAndMetadata(20L, Optional.of(3), "")); - - when(applicationEventHandler.addAndGet(any(), any())).thenAnswer(invocation -> { - CompletableApplicationEvent event = invocation.getArgument(0); - assertTrue(event instanceof FetchCommittedOffsetsApplicationEvent); - return topicPartitionOffsets; - }); - - assertDoesNotThrow(() -> consumer.committed(topicPartitionOffsets.keySet(), Duration.ofMillis(1000))); - - verify(metadata).updateLastSeenEpochIfNewer(t0, 2); - verify(metadata).updateLastSeenEpochIfNewer(t2, 3); + try (AsyncKafkaConsumer consumer = setup()) { + final TopicPartition t0 = new TopicPartition("t0", 2); + final TopicPartition t1 = new TopicPartition("t0", 3); + final TopicPartition t2 = new TopicPartition("t0", 4); + HashMap topicPartitionOffsets = new HashMap<>(); + topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L, Optional.of(2), "")); + topicPartitionOffsets.put(t1, null); + topicPartitionOffsets.put(t2, new OffsetAndMetadata(20L, Optional.of(3), "")); + + when(applicationEventHandler.addAndGet(any(), any())).thenAnswer(invocation -> { + CompletableApplicationEvent event = invocation.getArgument(0); + assertTrue(event instanceof FetchCommittedOffsetsApplicationEvent); + return topicPartitionOffsets; + }); + + assertDoesNotThrow(() -> consumer.committed(topicPartitionOffsets.keySet(), Duration.ofMillis(1000))); + + verify(metadata).updateLastSeenEpochIfNewer(t0, 2); + verify(metadata).updateLastSeenEpochIfNewer(t2, 3); + } } } From 7dcd8ae953464d746bb06068c51a599bb093e0b3 Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Tue, 5 Dec 2023 15:53:55 +0100 Subject: [PATCH 03/21] migrate committed tests --- .../internals/AsyncKafkaConsumerTest.java | 24 --- .../internals/AsyncKafkaConsumerUnitTest.java | 174 +++++++++++------- 2 files changed, 104 insertions(+), 94 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index b61fba6c7f020..bfe8659545b64 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -224,30 +224,6 @@ private static Stream commitExceptionSupplier() { new GroupAuthorizationException("Group authorization exception")); } - @Test - public void testCommitted() { - Map offsets = mockTopicPartitionOffset(); - CompletableFuture> committedFuture = new CompletableFuture<>(); - committedFuture.complete(offsets); - - try (MockedConstruction ignored = offsetFetchEventMocker(committedFuture)) { - assertDoesNotThrow(() -> consumer.committed(offsets.keySet(), Duration.ofMillis(1000))); - verify(applicationEventHandler).add(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class)); - } - } - - @Test - public void testCommitted_ExceptionThrown() { - Map offsets = mockTopicPartitionOffset(); - CompletableFuture> committedFuture = new CompletableFuture<>(); - committedFuture.completeExceptionally(new KafkaException("Test exception")); - - try (MockedConstruction ignored = offsetFetchEventMocker(committedFuture)) { - assertThrows(KafkaException.class, () -> consumer.committed(offsets.keySet(), Duration.ofMillis(1000))); - verify(applicationEventHandler).add(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class)); - } - } - @Test public void testWakeupBeforeCallingPoll() { final String topicName = "foo"; diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java index aaa32b511ffab..6307c166de528 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java @@ -20,7 +20,6 @@ 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.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -30,10 +29,10 @@ import java.util.HashMap; import java.util.HashSet; import java.util.LinkedList; +import java.util.Map; import java.util.Optional; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; -import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetCommitCallback; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; @@ -41,7 +40,7 @@ import org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.FetchCommittedOffsetsApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.OffsetFetchApplicationEvent; +import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.InvalidGroupIdException; import org.apache.kafka.common.metrics.Metrics; @@ -49,12 +48,15 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; import org.mockito.Mockito; @SuppressWarnings("unchecked") public class AsyncKafkaConsumerUnitTest { + private AsyncKafkaConsumer consumer = null; + private final Time time = new MockTime(); private final Deserializers deserializers = mock(Deserializers.class); private final FetchBuffer fetchBuffer = mock(FetchBuffer.class); @@ -65,6 +67,15 @@ public class AsyncKafkaConsumerUnitTest { private final SubscriptionState subscriptions = mock(SubscriptionState.class); private final ConsumerMetadata metadata = mock(ConsumerMetadata.class); + @AfterEach + private void resetAll() { + if (consumer != null) { + consumer.close(); + } + consumer = null; + Mockito.framework().clearInlineMocks(); + } + private AsyncKafkaConsumer setup() { return setup("group-id"); } @@ -99,93 +110,116 @@ private AsyncKafkaConsumer setup(String groupId) { @Test public void testInvalidGroupId() { - try (AsyncKafkaConsumer consumer = setupWithEmptyGroupId()) { - assertThrows(InvalidGroupIdException.class, () -> consumer.committed(new HashSet<>())); - } + assertThrows(InvalidGroupIdException.class, this::setupWithEmptyGroupId); } @Test public void testFencedInstanceException() { - try (AsyncKafkaConsumer consumer = setup()) { - - Mockito.doAnswer(invocation -> { - CompletableApplicationEvent event = invocation.getArgument(0); - assertTrue(event instanceof CommitApplicationEvent); - event.future().completeExceptionally(Errors.FENCED_INSTANCE_ID.exception()); - return null; - }).when(applicationEventHandler).add(any()); - - assertDoesNotThrow(() -> consumer.commitAsync()); - } + consumer = setup(); + Mockito.doAnswer(invocation -> { + CompletableApplicationEvent event = invocation.getArgument(0); + assertTrue(event instanceof CommitApplicationEvent); + event.future().completeExceptionally(Errors.FENCED_INSTANCE_ID.exception()); + return null; + }).when(applicationEventHandler).add(any()); + + assertDoesNotThrow(() -> consumer.commitAsync()); } @Test public void testCommitSyncLeaderEpochUpdate() { - try (AsyncKafkaConsumer consumer = setup()) { - - final TopicPartition t0 = new TopicPartition("t0", 2); - final TopicPartition t1 = new TopicPartition("t0", 3); - HashMap topicPartitionOffsets = new HashMap<>(); - topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L, Optional.of(2), "")); - topicPartitionOffsets.put(t1, new OffsetAndMetadata(20L, Optional.of(1), "")); + consumer = setup(); + final TopicPartition t0 = new TopicPartition("t0", 2); + final TopicPartition t1 = new TopicPartition("t0", 3); + HashMap topicPartitionOffsets = new HashMap<>(); + topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L, Optional.of(2), "")); + topicPartitionOffsets.put(t1, new OffsetAndMetadata(20L, Optional.of(1), "")); + Mockito.doAnswer(invocation -> { + CompletableApplicationEvent event = invocation.getArgument(0); + assertTrue(event instanceof CommitApplicationEvent); + event.future().complete(null); + return null; + }).when(applicationEventHandler).add(any()); + + assertDoesNotThrow(() -> consumer.commitSync(topicPartitionOffsets)); + + verify(metadata).updateLastSeenEpochIfNewer(t0, 2); + verify(metadata).updateLastSeenEpochIfNewer(t1, 1); + } - consumer.assign(Arrays.asList(t0, t1)); + @Test + public void testCommitAsyncLeaderEpochUpdate() { + consumer = setup(); + OffsetCommitCallback callback = mock(OffsetCommitCallback.class); + final TopicPartition t0 = new TopicPartition("t0", 2); + final TopicPartition t1 = new TopicPartition("t0", 3); + HashMap topicPartitionOffsets = new HashMap<>(); + topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L, Optional.of(2), "")); + topicPartitionOffsets.put(t1, new OffsetAndMetadata(20L, Optional.of(1), "")); + consumer.assign(Arrays.asList(t0, t1)); + + consumer.commitAsync(topicPartitionOffsets, callback); + + verify(metadata).updateLastSeenEpochIfNewer(t0, 2); + verify(metadata).updateLastSeenEpochIfNewer(t1, 1); + } - Mockito.doAnswer(invocation -> { - CompletableApplicationEvent event = invocation.getArgument(0); - assertTrue(event instanceof CommitApplicationEvent); - event.future().complete(null); - return null; - }).when(applicationEventHandler).add(any()); + @Test + public void testCommitted() { + consumer = setup(); + Map topicPartitionOffsets = mockTopicPartitionOffset(); - assertDoesNotThrow(() -> consumer.commitSync(topicPartitionOffsets)); + when(applicationEventHandler.addAndGet(any(), any())).thenAnswer(invocation -> { + CompletableApplicationEvent event = invocation.getArgument(0); + assertTrue(event instanceof FetchCommittedOffsetsApplicationEvent); + return topicPartitionOffsets; + }); - verify(metadata).updateLastSeenEpochIfNewer(t0, 2); - verify(metadata).updateLastSeenEpochIfNewer(t1, 1); - } + assertDoesNotThrow(() -> consumer.committed(topicPartitionOffsets.keySet(), Duration.ofMillis(1000))); } @Test - public void testCommitAsyncLeaderEpochUpdate() { - try (AsyncKafkaConsumer consumer = setup()) { - OffsetCommitCallback callback = mock(OffsetCommitCallback.class); - final TopicPartition t0 = new TopicPartition("t0", 2); - final TopicPartition t1 = new TopicPartition("t0", 3); - HashMap topicPartitionOffsets = new HashMap<>(); - topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L, Optional.of(2), "")); - topicPartitionOffsets.put(t1, new OffsetAndMetadata(20L, Optional.of(1), "")); - consumer.assign(Arrays.asList(t0, t1)); - - consumer.commitAsync(topicPartitionOffsets, callback); - - verify(metadata).updateLastSeenEpochIfNewer(t0, 2); - verify(metadata).updateLastSeenEpochIfNewer(t1, 1); - } + public void testCommittedExceptionThrown() { + consumer = setup(); + Map topicPartitionOffsets = mockTopicPartitionOffset(); + when(applicationEventHandler.addAndGet(any(), any())).thenAnswer(invocation -> { + CompletableApplicationEvent event = invocation.getArgument(0); + assertTrue(event instanceof FetchCommittedOffsetsApplicationEvent); + throw new KafkaException("Test exception"); + }); + + assertThrows(KafkaException.class, () -> consumer.committed(topicPartitionOffsets.keySet(), Duration.ofMillis(1000))); } @Test public void testCommittedLeaderEpochUpdate() { - try (AsyncKafkaConsumer consumer = setup()) { - final TopicPartition t0 = new TopicPartition("t0", 2); - final TopicPartition t1 = new TopicPartition("t0", 3); - final TopicPartition t2 = new TopicPartition("t0", 4); - HashMap topicPartitionOffsets = new HashMap<>(); - topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L, Optional.of(2), "")); - topicPartitionOffsets.put(t1, null); - topicPartitionOffsets.put(t2, new OffsetAndMetadata(20L, Optional.of(3), "")); - - when(applicationEventHandler.addAndGet(any(), any())).thenAnswer(invocation -> { - CompletableApplicationEvent event = invocation.getArgument(0); - assertTrue(event instanceof FetchCommittedOffsetsApplicationEvent); - return topicPartitionOffsets; - }); - - assertDoesNotThrow(() -> consumer.committed(topicPartitionOffsets.keySet(), Duration.ofMillis(1000))); - - verify(metadata).updateLastSeenEpochIfNewer(t0, 2); - verify(metadata).updateLastSeenEpochIfNewer(t2, 3); - } + consumer = setup(); + final TopicPartition t0 = new TopicPartition("t0", 2); + final TopicPartition t1 = new TopicPartition("t0", 3); + final TopicPartition t2 = new TopicPartition("t0", 4); + HashMap topicPartitionOffsets = new HashMap<>(); + topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L, Optional.of(2), "")); + topicPartitionOffsets.put(t1, null); + topicPartitionOffsets.put(t2, new OffsetAndMetadata(20L, Optional.of(3), "")); + when(applicationEventHandler.addAndGet(any(), any())).thenAnswer(invocation -> { + CompletableApplicationEvent event = invocation.getArgument(0); + assertTrue(event instanceof FetchCommittedOffsetsApplicationEvent); + return topicPartitionOffsets; + }); + + assertDoesNotThrow(() -> consumer.committed(topicPartitionOffsets.keySet(), Duration.ofMillis(1000))); + + verify(metadata).updateLastSeenEpochIfNewer(t0, 2); + verify(metadata).updateLastSeenEpochIfNewer(t2, 3); } + private HashMap mockTopicPartitionOffset() { + final TopicPartition t0 = new TopicPartition("t0", 2); + final TopicPartition t1 = new TopicPartition("t0", 3); + HashMap topicPartitionOffsets = new HashMap<>(); + topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L)); + topicPartitionOffsets.put(t1, new OffsetAndMetadata(20L)); + return topicPartitionOffsets; + } } From 966c18fca69672904170525c1c89329f774e19ae Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Tue, 5 Dec 2023 16:04:15 +0100 Subject: [PATCH 04/21] migrate subscribed tests --- .../internals/AsyncKafkaConsumerTest.java | 47 +----------- .../internals/AsyncKafkaConsumerUnitTest.java | 72 ++++++++++++++++++- 2 files changed, 71 insertions(+), 48 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index bfe8659545b64..19523a3abf6ab 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -655,52 +655,7 @@ public void testRefreshCommittedOffsetsNotCalledIfNoGroupId() { } @Test - public void testSubscribeGeneratesEvent() { - String topic = "topic1"; - consumer.subscribe(singletonList(topic)); - assertEquals(singleton(topic), consumer.subscription()); - assertTrue(consumer.assignment().isEmpty()); - verify(applicationEventHandler).add(ArgumentMatchers.isA(SubscriptionChangeApplicationEvent.class)); - } - - @Test - public void testUnsubscribeGeneratesUnsubscribeEvent() { - consumer.unsubscribe(); - - // Verify the unsubscribe event was generated and mock its completion. - final ArgumentCaptor captor = ArgumentCaptor.forClass(UnsubscribeApplicationEvent.class); - verify(applicationEventHandler).add(captor.capture()); - UnsubscribeApplicationEvent unsubscribeApplicationEvent = captor.getValue(); - unsubscribeApplicationEvent.future().complete(null); - - assertTrue(consumer.subscription().isEmpty()); - assertTrue(consumer.assignment().isEmpty()); - } - - @Test - public void testSubscribeToEmptyListActsAsUnsubscribe() { - consumer.subscribe(Collections.emptyList()); - assertTrue(consumer.subscription().isEmpty()); - assertTrue(consumer.assignment().isEmpty()); - verify(applicationEventHandler).add(ArgumentMatchers.isA(UnsubscribeApplicationEvent.class)); - } - - @Test - public void testSubscribeToNullTopicCollection() { - assertThrows(IllegalArgumentException.class, () -> consumer.subscribe((List) null)); - } - - @Test - public void testSubscriptionOnNullTopic() { - assertThrows(IllegalArgumentException.class, () -> consumer.subscribe(singletonList(null))); - } - - @Test - public void testSubscriptionOnEmptyTopic() { - String emptyTopic = " "; - assertThrows(IllegalArgumentException.class, () -> consumer.subscribe(singletonList(emptyTopic))); - } - + @Test public void testGroupMetadataAfterCreationWithGroupIdIsNull() { final Properties props = requiredConsumerProperties(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java index 6307c166de528..d518a8e546f1c 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java @@ -16,7 +16,10 @@ */ package org.apache.kafka.clients.consumer.internals; +import static java.util.Collections.singleton; +import static java.util.Collections.singletonList; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; @@ -26,13 +29,13 @@ import java.time.Duration; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.LinkedList; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.BlockingQueue; -import java.util.concurrent.CompletableFuture; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetCommitCallback; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; @@ -40,6 +43,8 @@ import org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.FetchCommittedOffsetsApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.SubscriptionChangeApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.UnsubscribeApplicationEvent; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.InvalidGroupIdException; @@ -50,6 +55,8 @@ import org.apache.kafka.common.utils.Time; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.ArgumentMatchers; import org.mockito.Mockito; @SuppressWarnings("unchecked") @@ -213,6 +220,67 @@ public void testCommittedLeaderEpochUpdate() { verify(metadata).updateLastSeenEpochIfNewer(t2, 3); } + @Test + public void testSubscribeGeneratesEvent() { + consumer = setup(); + String topic = "topic1"; + consumer.subscribe(singletonList(topic)); + assertEquals(singleton(topic), consumer.subscription()); + assertTrue(consumer.assignment().isEmpty()); + verify(applicationEventHandler).add(ArgumentMatchers.isA(SubscriptionChangeApplicationEvent.class)); + } + + @Test + public void testUnsubscribeGeneratesUnsubscribeEvent() { + consumer = setup(); + consumer.unsubscribe(); + + // Verify the unsubscribe event was generated and mock its completion. + final ArgumentCaptor captor = ArgumentCaptor.forClass(UnsubscribeApplicationEvent.class); + verify(applicationEventHandler).add(captor.capture()); + UnsubscribeApplicationEvent unsubscribeApplicationEvent = captor.getValue(); + unsubscribeApplicationEvent.future().complete(null); + + assertTrue(consumer.subscription().isEmpty()); + assertTrue(consumer.assignment().isEmpty()); + } + + @Test + public void testSubscribeToEmptyListActsAsUnsubscribe() { + consumer = setup(); + + Mockito.doAnswer(invocation -> { + CompletableApplicationEvent event = invocation.getArgument(0); + assertTrue(event instanceof UnsubscribeApplicationEvent); + event.future().complete(null); + return null; + }).when(applicationEventHandler).add(any()); + + + consumer.subscribe(Collections.emptyList()); + assertTrue(consumer.subscription().isEmpty()); + assertTrue(consumer.assignment().isEmpty()); + } + + @Test + public void testSubscribeToNullTopicCollection() { + consumer = setup(); + assertThrows(IllegalArgumentException.class, () -> consumer.subscribe((List) null)); + } + + @Test + public void testSubscriptionOnNullTopic() { + consumer = setup(); + assertThrows(IllegalArgumentException.class, () -> consumer.subscribe(singletonList(null))); + } + + @Test + public void testSubscriptionOnEmptyTopic() { + consumer = setup(); + String emptyTopic = " "; + assertThrows(IllegalArgumentException.class, () -> consumer.subscribe(singletonList(emptyTopic))); + } + private HashMap mockTopicPartitionOffset() { final TopicPartition t0 = new TopicPartition("t0", 2); final TopicPartition t1 = new TopicPartition("t0", 3); From 407a85be5b105b00fe157c4a474db05ee8343593 Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Tue, 5 Dec 2023 16:34:39 +0100 Subject: [PATCH 05/21] migrate offset tests --- .../internals/AsyncKafkaConsumerTest.java | 278 +++--------------- .../internals/AsyncKafkaConsumerUnitTest.java | 176 ++++++++++- 2 files changed, 203 insertions(+), 251 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index 19523a3abf6ab..0e71138a0bc30 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -16,7 +16,47 @@ */ package org.apache.kafka.clients.consumer.internals; +import static java.util.Arrays.asList; +import static java.util.Collections.singleton; +import static java.util.Collections.singletonList; +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED; +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertSame; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.atLeast; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockConstruction; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; import java.util.Locale; +import java.util.Map; +import java.util.Optional; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.stream.Stream; import org.apache.kafka.clients.ClientResponse; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; @@ -24,7 +64,6 @@ import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.GroupProtocol; import org.apache.kafka.clients.consumer.OffsetAndMetadata; -import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.clients.consumer.OffsetCommitCallback; import org.apache.kafka.clients.consumer.RetriableCommitFailedException; import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; @@ -35,26 +74,22 @@ import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.FetchCommittedOffsetsApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.GroupMetadataUpdateEvent; -import org.apache.kafka.clients.consumer.internals.events.ListOffsetsApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.NewTopicsMetadataUpdateRequestEvent; import org.apache.kafka.clients.consumer.internals.events.ResetPositionsApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.SubscriptionChangeApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.UnsubscribeApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ValidatePositionsApplicationEvent; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.GroupAuthorizationException; +import org.apache.kafka.common.errors.InvalidGroupIdException; import org.apache.kafka.common.errors.NetworkException; import org.apache.kafka.common.errors.RetriableException; -import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.message.OffsetCommitResponseData; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.FindCoordinatorResponse; import org.apache.kafka.common.requests.JoinGroupRequest; -import org.apache.kafka.common.requests.ListOffsetsRequest; import org.apache.kafka.common.requests.OffsetCommitResponse; import org.apache.kafka.common.requests.RequestHeader; import org.apache.kafka.common.serialization.StringDeserializer; @@ -66,57 +101,12 @@ import org.junit.jupiter.api.function.Executable; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; -import org.mockito.ArgumentCaptor; import org.mockito.ArgumentMatchers; import org.mockito.MockedConstruction; import org.mockito.Mockito; import org.mockito.stubbing.Answer; import org.opentest4j.AssertionFailedError; -import java.time.Duration; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Properties; -import java.util.Set; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static java.util.Arrays.asList; -import static java.util.Collections.singleton; -import static java.util.Collections.singletonList; -import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED; -import static org.apache.kafka.common.utils.Utils.mkEntry; -import static org.apache.kafka.common.utils.Utils.mkMap; -import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertNull; -import static org.junit.jupiter.api.Assertions.assertSame; -import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.junit.jupiter.api.Assertions.fail; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.atLeast; -import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.mockConstruction; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - public class AsyncKafkaConsumerTest { private AsyncKafkaConsumer consumer; @@ -338,56 +328,6 @@ public void testEnsureCommitSyncExecutedCommitAsyncCallbacks() { Errors.NETWORK_EXCEPTION); } - @Test - @SuppressWarnings("deprecation") - public void testPollLongThrowsException() { - Exception e = assertThrows(UnsupportedOperationException.class, () -> consumer.poll(0L)); - assertEquals("Consumer.poll(long) is not supported when \"group.protocol\" is \"consumer\". " + - "This method is deprecated and will be removed in the next major release.", e.getMessage()); - } - - @Test - public void testCommitSyncLeaderEpochUpdate() { - final TopicPartition t0 = new TopicPartition("t0", 2); - final TopicPartition t1 = new TopicPartition("t0", 3); - HashMap topicPartitionOffsets = new HashMap<>(); - topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L, Optional.of(2), "")); - topicPartitionOffsets.put(t1, new OffsetAndMetadata(20L, Optional.of(1), "")); - - consumer.assign(Arrays.asList(t0, t1)); - - CompletableFuture commitFuture = new CompletableFuture<>(); - commitFuture.complete(null); - - try (MockedConstruction ignored = commitEventMocker(commitFuture)) { - assertDoesNotThrow(() -> consumer.commitSync(topicPartitionOffsets)); - } - verify(testBuilder.metadata).updateLastSeenEpochIfNewer(t0, 2); - verify(testBuilder.metadata).updateLastSeenEpochIfNewer(t1, 1); - verify(applicationEventHandler).add(ArgumentMatchers.isA(CommitApplicationEvent.class)); - } - - @Test - public void testCommitAsyncLeaderEpochUpdate() { - MockCommitCallback callback = new MockCommitCallback(); - final TopicPartition t0 = new TopicPartition("t0", 2); - final TopicPartition t1 = new TopicPartition("t0", 3); - HashMap topicPartitionOffsets = new HashMap<>(); - topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L, Optional.of(2), "")); - topicPartitionOffsets.put(t1, new OffsetAndMetadata(20L, Optional.of(1), "")); - - consumer.assign(Arrays.asList(t0, t1)); - - CompletableFuture commitFuture = new CompletableFuture<>(); - commitFuture.complete(null); - - try (MockedConstruction ignored = commitEventMocker(commitFuture)) { - assertDoesNotThrow(() -> consumer.commitAsync(topicPartitionOffsets, callback)); - } - verify(testBuilder.metadata).updateLastSeenEpochIfNewer(t0, 2); - verify(testBuilder.metadata).updateLastSeenEpochIfNewer(t1, 1); - verify(applicationEventHandler).add(ArgumentMatchers.isA(CommitApplicationEvent.class)); - } @Test public void testEnsurePollExecutedCommitAsyncCallbacks() { @@ -471,21 +411,6 @@ private static MockedConstruction offsetF return mockConstruction(FetchCommittedOffsetsApplicationEvent.class, mockInitializer); } - private static MockedConstruction commitEventMocker(CompletableFuture future) { - Answer getInvocationAnswer = invocation -> { - Timer timer = invocation.getArgument(0); - return ConsumerUtils.getResult(future, timer); - }; - - MockedConstruction.MockInitializer mockInitializer = (mock, ctx) -> { - when(mock.get(any())).thenAnswer(getInvocationAnswer); - when(mock.type()).thenReturn(ApplicationEvent.Type.COMMIT); - when(mock.future()).thenReturn(future); - }; - - return mockConstruction(CommitApplicationEvent.class, mockInitializer); - } - @Test public void testAssign() { final TopicPartition tp = new TopicPartition("foo", 3); @@ -518,107 +443,6 @@ public void testAssignOnEmptyTopicInPartition() { assertThrows(IllegalArgumentException.class, () -> consumer.assign(singleton(new TopicPartition(" ", 0)))); } - @Test - public void testBeginningOffsetsFailsIfNullPartitions() { - assertThrows(NullPointerException.class, () -> consumer.beginningOffsets(null, - Duration.ofMillis(1))); - } - - @Test - public void testBeginningOffsets() { - Map expectedOffsetsAndTimestamp = - mockOffsetAndTimestamp(); - Set partitions = expectedOffsetsAndTimestamp.keySet(); - doReturn(expectedOffsetsAndTimestamp).when(applicationEventHandler).addAndGet(any(), any()); - Map result = - assertDoesNotThrow(() -> consumer.beginningOffsets(partitions, - Duration.ofMillis(1))); - Map expectedOffsets = expectedOffsetsAndTimestamp.entrySet().stream() - .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().offset())); - assertEquals(expectedOffsets, result); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), - ArgumentMatchers.isA(Timer.class)); - } - - @Test - public void testBeginningOffsetsThrowsKafkaExceptionForUnderlyingExecutionFailure() { - Set partitions = mockTopicPartitionOffset().keySet(); - Throwable eventProcessingFailure = new KafkaException("Unexpected failure " + - "processing List Offsets event"); - doThrow(eventProcessingFailure).when(applicationEventHandler).addAndGet(any(), any()); - Throwable consumerError = assertThrows(KafkaException.class, - () -> consumer.beginningOffsets(partitions, - Duration.ofMillis(1))); - assertEquals(eventProcessingFailure, consumerError); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); - } - - @Test - public void testBeginningOffsetsTimeoutOnEventProcessingTimeout() { - doThrow(new TimeoutException()).when(applicationEventHandler).addAndGet(any(), any()); - assertThrows(TimeoutException.class, - () -> consumer.beginningOffsets( - Collections.singletonList(new TopicPartition("t1", 0)), - Duration.ofMillis(1))); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), - ArgumentMatchers.isA(Timer.class)); - } - - @Test - public void testOffsetsForTimesOnNullPartitions() { - assertThrows(NullPointerException.class, () -> consumer.offsetsForTimes(null, - Duration.ofMillis(1))); - } - - @Test - public void testOffsetsForTimesFailsOnNegativeTargetTimes() { - assertThrows(IllegalArgumentException.class, - () -> consumer.offsetsForTimes(Collections.singletonMap(new TopicPartition( - "topic1", 1), ListOffsetsRequest.EARLIEST_TIMESTAMP), - Duration.ofMillis(1))); - - assertThrows(IllegalArgumentException.class, - () -> consumer.offsetsForTimes(Collections.singletonMap(new TopicPartition( - "topic1", 1), ListOffsetsRequest.LATEST_TIMESTAMP), - Duration.ofMillis(1))); - - assertThrows(IllegalArgumentException.class, - () -> consumer.offsetsForTimes(Collections.singletonMap(new TopicPartition( - "topic1", 1), ListOffsetsRequest.MAX_TIMESTAMP), - Duration.ofMillis(1))); - } - - @Test - public void testOffsetsForTimes() { - Map expectedResult = mockOffsetAndTimestamp(); - Map timestampToSearch = mockTimestampToSearch(); - - doReturn(expectedResult).when(applicationEventHandler).addAndGet(any(), any()); - Map result = - assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, Duration.ofMillis(1))); - assertEquals(expectedResult, result); - verify(applicationEventHandler).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() { - TopicPartition tp = new TopicPartition("topic1", 0); - Map expectedResult = - Collections.singletonMap(tp, null); - Map timestampToSearch = Collections.singletonMap(tp, 5L); - - Map result = - assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, - Duration.ofMillis(0))); - assertEquals(expectedResult, result); - verify(applicationEventHandler, never()).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), - ArgumentMatchers.isA(Timer.class)); - } - @Test public void testWakeup_committed() { consumer.wakeup(); @@ -654,8 +478,6 @@ public void testRefreshCommittedOffsetsNotCalledIfNoGroupId() { testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(false); } - @Test - @Test public void testGroupMetadataAfterCreationWithGroupIdIsNull() { final Properties props = requiredConsumerProperties(); @@ -982,24 +804,6 @@ private HashMap mockTopicPartitionOffset() { return topicPartitionOffsets; } - private HashMap mockOffsetAndTimestamp() { - final TopicPartition t0 = new TopicPartition("t0", 2); - final TopicPartition t1 = new TopicPartition("t0", 3); - HashMap offsetAndTimestamp = new HashMap<>(); - offsetAndTimestamp.put(t0, new OffsetAndTimestamp(5L, 1L)); - offsetAndTimestamp.put(t1, new OffsetAndTimestamp(6L, 3L)); - return offsetAndTimestamp; - } - - private HashMap mockTimestampToSearch() { - final TopicPartition t0 = new TopicPartition("t0", 2); - final TopicPartition t1 = new TopicPartition("t0", 3); - HashMap timestampToSearch = new HashMap<>(); - timestampToSearch.put(t0, 1L); - timestampToSearch.put(t1, 2L); - return timestampToSearch; - } - private void prepAutocommitOnClose() { Node node = testBuilder.metadata.fetch().nodes().get(0); testBuilder.client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "group-id", node)); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java index d518a8e546f1c..e1666920c514e 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java @@ -23,7 +23,10 @@ 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.doReturn; +import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -31,31 +34,38 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; -import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.concurrent.BlockingQueue; +import java.util.stream.Collectors; import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.clients.consumer.OffsetCommitCallback; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.clients.consumer.RangeAssignor; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.ListOffsetsApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.FetchCommittedOffsetsApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.SubscriptionChangeApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.UnsubscribeApplicationEvent; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.InvalidGroupIdException; +import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.ListOffsetsRequest; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Timer; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; import org.mockito.ArgumentMatchers; import org.mockito.Mockito; @@ -71,11 +81,10 @@ public class AsyncKafkaConsumerUnitTest { private final ConsumerInterceptors interceptors = mock(ConsumerInterceptors.class); private final ApplicationEventHandler applicationEventHandler = mock(ApplicationEventHandler.class); private final BlockingQueue backgroundEventQueue = mock(BlockingQueue.class); - private final SubscriptionState subscriptions = mock(SubscriptionState.class); private final ConsumerMetadata metadata = mock(ConsumerMetadata.class); @AfterEach - private void resetAll() { + public void resetAll() { if (consumer != null) { consumer.close(); } @@ -95,8 +104,10 @@ private AsyncKafkaConsumer setup(String groupId) { String clientId = ""; long retryBackoffMs = 100; int defaultApiTimeoutMs = 100; + LogContext logContext = new LogContext(); + SubscriptionState subscriptionState = new SubscriptionState(logContext, OffsetResetStrategy.LATEST); return new AsyncKafkaConsumer<>( - new LogContext(), + logContext, clientId, deserializers, fetchBuffer, @@ -106,11 +117,11 @@ private AsyncKafkaConsumer setup(String groupId) { applicationEventHandler, backgroundEventQueue, new Metrics(), - subscriptions, + subscriptionState, metadata, retryBackoffMs, defaultApiTimeoutMs, - new LinkedList<>(), + Collections.singletonList(new RangeAssignor()), groupId ); } @@ -220,6 +231,124 @@ public void testCommittedLeaderEpochUpdate() { verify(metadata).updateLastSeenEpochIfNewer(t2, 3); } + @Test + @SuppressWarnings("deprecation") + public void testPollLongThrowsException() { + consumer = setup(); + Exception e = assertThrows(UnsupportedOperationException.class, () -> consumer.poll(0L)); + assertEquals("Consumer.poll(long) is not supported when \"group.protocol\" is \"consumer\". " + + "This method is deprecated and will be removed in the next major release.", e.getMessage()); + } + + @Test + public void testBeginningOffsetsFailsIfNullPartitions() { + consumer = setup(); + assertThrows(NullPointerException.class, () -> consumer.beginningOffsets(null, + Duration.ofMillis(1))); + } + + @Test + public void testBeginningOffsets() { + consumer = setup(); + Map expectedOffsetsAndTimestamp = + mockOffsetAndTimestamp(); + Set partitions = expectedOffsetsAndTimestamp.keySet(); + doReturn(expectedOffsetsAndTimestamp).when(applicationEventHandler).addAndGet(any(), any()); + Map result = + assertDoesNotThrow(() -> consumer.beginningOffsets(partitions, + Duration.ofMillis(1))); + Map expectedOffsets = expectedOffsetsAndTimestamp.entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().offset())); + assertEquals(expectedOffsets, result); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + ArgumentMatchers.isA(Timer.class)); + } + + @Test + public void testBeginningOffsetsThrowsKafkaExceptionForUnderlyingExecutionFailure() { + consumer = setup(); + Set partitions = mockTopicPartitionOffset().keySet(); + Throwable eventProcessingFailure = new KafkaException("Unexpected failure " + + "processing List Offsets event"); + doThrow(eventProcessingFailure).when(applicationEventHandler).addAndGet(any(), any()); + Throwable consumerError = assertThrows(KafkaException.class, + () -> consumer.beginningOffsets(partitions, + Duration.ofMillis(1))); + assertEquals(eventProcessingFailure, consumerError); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + } + + @Test + public void testBeginningOffsetsTimeoutOnEventProcessingTimeout() { + consumer = setup(); + doThrow(new TimeoutException()).when(applicationEventHandler).addAndGet(any(), any()); + assertThrows(TimeoutException.class, + () -> consumer.beginningOffsets( + Collections.singletonList(new TopicPartition("t1", 0)), + Duration.ofMillis(1))); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + ArgumentMatchers.isA(Timer.class)); + } + + @Test + public void testOffsetsForTimesOnNullPartitions() { + consumer = setup(); + assertThrows(NullPointerException.class, () -> consumer.offsetsForTimes(null, + Duration.ofMillis(1))); + } + + @Test + public void testOffsetsForTimesFailsOnNegativeTargetTimes() { + consumer = setup(); + assertThrows(IllegalArgumentException.class, + () -> consumer.offsetsForTimes(Collections.singletonMap(new TopicPartition( + "topic1", 1), ListOffsetsRequest.EARLIEST_TIMESTAMP), + Duration.ofMillis(1))); + + assertThrows(IllegalArgumentException.class, + () -> consumer.offsetsForTimes(Collections.singletonMap(new TopicPartition( + "topic1", 1), ListOffsetsRequest.LATEST_TIMESTAMP), + Duration.ofMillis(1))); + + assertThrows(IllegalArgumentException.class, + () -> consumer.offsetsForTimes(Collections.singletonMap(new TopicPartition( + "topic1", 1), ListOffsetsRequest.MAX_TIMESTAMP), + Duration.ofMillis(1))); + } + + @Test + public void testOffsetsForTimes() { + consumer = setup(); + Map expectedResult = mockOffsetAndTimestamp(); + Map timestampToSearch = mockTimestampToSearch(); + + doReturn(expectedResult).when(applicationEventHandler).addAndGet(any(), any()); + Map result = + assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, Duration.ofMillis(1))); + assertEquals(expectedResult, result); + verify(applicationEventHandler).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() { + consumer = setup(); + TopicPartition tp = new TopicPartition("topic1", 0); + Map expectedResult = + Collections.singletonMap(tp, null); + Map timestampToSearch = Collections.singletonMap(tp, 5L); + + Map result = + assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, + Duration.ofMillis(0))); + assertEquals(expectedResult, result); + verify(applicationEventHandler, never()).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + ArgumentMatchers.isA(Timer.class)); + } + @Test public void testSubscribeGeneratesEvent() { consumer = setup(); @@ -233,13 +362,14 @@ public void testSubscribeGeneratesEvent() { @Test public void testUnsubscribeGeneratesUnsubscribeEvent() { consumer = setup(); - consumer.unsubscribe(); + Mockito.doAnswer(invocation -> { + CompletableApplicationEvent event = invocation.getArgument(0); + assertTrue(event instanceof UnsubscribeApplicationEvent); + event.future().complete(null); + return null; + }).when(applicationEventHandler).add(any()); - // Verify the unsubscribe event was generated and mock its completion. - final ArgumentCaptor captor = ArgumentCaptor.forClass(UnsubscribeApplicationEvent.class); - verify(applicationEventHandler).add(captor.capture()); - UnsubscribeApplicationEvent unsubscribeApplicationEvent = captor.getValue(); - unsubscribeApplicationEvent.future().complete(null); + consumer.unsubscribe(); assertTrue(consumer.subscription().isEmpty()); assertTrue(consumer.assignment().isEmpty()); @@ -256,7 +386,6 @@ public void testSubscribeToEmptyListActsAsUnsubscribe() { return null; }).when(applicationEventHandler).add(any()); - consumer.subscribe(Collections.emptyList()); assertTrue(consumer.subscription().isEmpty()); assertTrue(consumer.assignment().isEmpty()); @@ -289,5 +418,24 @@ private HashMap mockTopicPartitionOffset() { topicPartitionOffsets.put(t1, new OffsetAndMetadata(20L)); return topicPartitionOffsets; } + + private HashMap mockOffsetAndTimestamp() { + final TopicPartition t0 = new TopicPartition("t0", 2); + final TopicPartition t1 = new TopicPartition("t0", 3); + HashMap offsetAndTimestamp = new HashMap<>(); + offsetAndTimestamp.put(t0, new OffsetAndTimestamp(5L, 1L)); + offsetAndTimestamp.put(t1, new OffsetAndTimestamp(6L, 3L)); + return offsetAndTimestamp; + } + + private HashMap mockTimestampToSearch() { + final TopicPartition t0 = new TopicPartition("t0", 2); + final TopicPartition t1 = new TopicPartition("t0", 3); + HashMap timestampToSearch = new HashMap<>(); + timestampToSearch.put(t0, 1L); + timestampToSearch.put(t1, 2L); + return timestampToSearch; + } + } From b40e5c939bc585faa9b7a77b3d9e0121157ca035 Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Tue, 5 Dec 2023 16:45:16 +0100 Subject: [PATCH 06/21] migrate wakeup tests --- .../internals/AsyncKafkaConsumerTest.java | 76 ---------------- .../internals/AsyncKafkaConsumerUnitTest.java | 91 +++++++++++++++++++ 2 files changed, 91 insertions(+), 76 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index 0e71138a0bc30..e2b1933b687ae 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -214,82 +214,6 @@ private static Stream commitExceptionSupplier() { new GroupAuthorizationException("Group authorization exception")); } - @Test - public void testWakeupBeforeCallingPoll() { - final String topicName = "foo"; - final int partition = 3; - final TopicPartition tp = new TopicPartition(topicName, partition); - doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); - Map offsets = mkMap(mkEntry(tp, new OffsetAndMetadata(1))); - doReturn(offsets).when(applicationEventHandler).addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); - consumer.assign(singleton(tp)); - - consumer.wakeup(); - - assertThrows(WakeupException.class, () -> consumer.poll(Duration.ZERO)); - assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); - } - - @Test - public void testWakeupAfterEmptyFetch() { - final String topicName = "foo"; - final int partition = 3; - final TopicPartition tp = new TopicPartition(topicName, partition); - doAnswer(invocation -> { - consumer.wakeup(); - return Fetch.empty(); - }).when(fetchCollector).collectFetch(any(FetchBuffer.class)); - Map offsets = mkMap(mkEntry(tp, new OffsetAndMetadata(1))); - doReturn(offsets).when(applicationEventHandler).addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); - consumer.assign(singleton(tp)); - - assertThrows(WakeupException.class, () -> consumer.poll(Duration.ofMinutes(1))); - assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); - } - - @Test - public void testWakeupAfterNonEmptyFetch() { - final String topicName = "foo"; - final int partition = 3; - final TopicPartition tp = new TopicPartition(topicName, partition); - final List> records = asList( - new ConsumerRecord<>(topicName, partition, 2, "key1", "value1"), - new ConsumerRecord<>(topicName, partition, 3, "key2", "value2") - ); - doAnswer(invocation -> { - consumer.wakeup(); - return Fetch.forPartition(tp, records, true); - }).when(fetchCollector).collectFetch(Mockito.any(FetchBuffer.class)); - Map offsets = mkMap(mkEntry(tp, new OffsetAndMetadata(1))); - doReturn(offsets).when(applicationEventHandler).addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); - consumer.assign(singleton(tp)); - - // since wakeup() is called when the non-empty fetch is returned the wakeup should be ignored - assertDoesNotThrow(() -> consumer.poll(Duration.ofMinutes(1))); - // the previously ignored wake-up should not be ignored in the next call - assertThrows(WakeupException.class, () -> consumer.poll(Duration.ZERO)); - } - - @Test - public void testClearWakeupTriggerAfterPoll() { - final String topicName = "foo"; - final int partition = 3; - final TopicPartition tp = new TopicPartition(topicName, partition); - final List> records = asList( - new ConsumerRecord<>(topicName, partition, 2, "key1", "value1"), - new ConsumerRecord<>(topicName, partition, 3, "key2", "value2") - ); - doReturn(Fetch.forPartition(tp, records, true)) - .when(fetchCollector).collectFetch(any(FetchBuffer.class)); - Map offsets = mkMap(mkEntry(tp, new OffsetAndMetadata(1))); - doReturn(offsets).when(applicationEventHandler).addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); - consumer.assign(singleton(tp)); - - consumer.poll(Duration.ZERO); - - assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); - } - @Test public void testEnsureCallbackExecutedByApplicationThread() { final String currentThread = Thread.currentThread().getName(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java index e1666920c514e..278593199e23f 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java @@ -16,13 +16,17 @@ */ package org.apache.kafka.clients.consumer.internals; +import static java.util.Arrays.asList; import static java.util.Collections.singleton; import static java.util.Collections.singletonList; +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; 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.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; @@ -40,6 +44,8 @@ import java.util.Set; import java.util.concurrent.BlockingQueue; import java.util.stream.Collectors; +import org.apache.kafka.clients.Metadata.LeaderAndEpoch; +import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.clients.consumer.OffsetCommitCallback; @@ -57,6 +63,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.InvalidGroupIdException; import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ListOffsetsRequest; @@ -410,6 +417,90 @@ public void testSubscriptionOnEmptyTopic() { assertThrows(IllegalArgumentException.class, () -> consumer.subscribe(singletonList(emptyTopic))); } + @Test + public void testWakeupBeforeCallingPoll() { + consumer = setup(); + final String topicName = "foo"; + final int partition = 3; + final TopicPartition tp = new TopicPartition(topicName, partition); + doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); + Map offsets = mkMap(mkEntry(tp, new OffsetAndMetadata(1))); + doReturn(offsets).when(applicationEventHandler).addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); + consumer.assign(singleton(tp)); + + consumer.wakeup(); + + assertThrows(WakeupException.class, () -> consumer.poll(Duration.ZERO)); + assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); + } + + @Test + public void testWakeupAfterEmptyFetch() { + consumer = setup(); + final String topicName = "foo"; + final int partition = 3; + final TopicPartition tp = new TopicPartition(topicName, partition); + doAnswer(invocation -> { + consumer.wakeup(); + return Fetch.empty(); + }).when(fetchCollector).collectFetch(any(FetchBuffer.class)); + Map offsets = mkMap(mkEntry(tp, new OffsetAndMetadata(1))); + doReturn(offsets).when(applicationEventHandler).addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); + consumer.assign(singleton(tp)); + + assertThrows(WakeupException.class, () -> consumer.poll(Duration.ofMinutes(1))); + assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); + } + + @Test + public void testWakeupAfterNonEmptyFetch() { + consumer = setup(); + final String topicName = "foo"; + final int partition = 3; + final TopicPartition tp = new TopicPartition(topicName, partition); + final List> records = asList( + new ConsumerRecord<>(topicName, partition, 2, "key1", "value1"), + new ConsumerRecord<>(topicName, partition, 3, "key2", "value2") + ); + doAnswer(invocation -> { + consumer.wakeup(); + return Fetch.forPartition(tp, records, true); + }).when(fetchCollector).collectFetch(Mockito.any(FetchBuffer.class)); + Map offsets = mkMap(mkEntry(tp, new OffsetAndMetadata(1))); + doReturn(offsets).when(applicationEventHandler).addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); + consumer.assign(singleton(tp)); + + // since wakeup() is called when the non-empty fetch is returned the wakeup should be ignored + assertDoesNotThrow(() -> consumer.poll(Duration.ofMinutes(1))); + // the previously ignored wake-up should not be ignored in the next call + assertThrows(WakeupException.class, () -> consumer.poll(Duration.ZERO)); + } + + @Test + public void testClearWakeupTriggerAfterPoll() { + consumer = setup(); + final String topicName = "foo"; + final int partition = 3; + final TopicPartition tp = new TopicPartition(topicName, partition); + final List> records = asList( + new ConsumerRecord<>(topicName, partition, 2, "key1", "value1"), + new ConsumerRecord<>(topicName, partition, 3, "key2", "value2") + ); + doReturn(Fetch.forPartition(tp, records, true)) + .when(fetchCollector).collectFetch(any(FetchBuffer.class)); + Map offsets = mkMap(mkEntry(tp, new OffsetAndMetadata(1))); + doReturn(offsets).when(applicationEventHandler).addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); + consumer.assign(singleton(tp)); + + consumer.poll(Duration.ZERO); + + assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); + } + private HashMap mockTopicPartitionOffset() { final TopicPartition t0 = new TopicPartition("t0", 2); final TopicPartition t1 = new TopicPartition("t0", 3); From 4ff9b573ea6ce332c67e3adf8249e2c3076266be Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Tue, 5 Dec 2023 16:58:54 +0100 Subject: [PATCH 07/21] migrate assing tests --- .../internals/AsyncKafkaConsumerTest.java | 35 -------------- .../internals/AsyncKafkaConsumerUnitTest.java | 48 +++++++++++++++++++ 2 files changed, 48 insertions(+), 35 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index e2b1933b687ae..e97409658ce4c 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -20,8 +20,6 @@ import static java.util.Collections.singleton; import static java.util.Collections.singletonList; import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED; -import static org.apache.kafka.common.utils.Utils.mkEntry; -import static org.apache.kafka.common.utils.Utils.mkMap; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -103,7 +101,6 @@ import org.junit.jupiter.params.provider.MethodSource; import org.mockito.ArgumentMatchers; import org.mockito.MockedConstruction; -import org.mockito.Mockito; import org.mockito.stubbing.Answer; import org.opentest4j.AssertionFailedError; @@ -335,38 +332,6 @@ private static MockedConstruction offsetF return mockConstruction(FetchCommittedOffsetsApplicationEvent.class, mockInitializer); } - @Test - public void testAssign() { - final TopicPartition tp = new TopicPartition("foo", 3); - consumer.assign(singleton(tp)); - assertTrue(consumer.subscription().isEmpty()); - assertTrue(consumer.assignment().contains(tp)); - verify(applicationEventHandler).add(any(AssignmentChangeApplicationEvent.class)); - verify(applicationEventHandler).add(any(NewTopicsMetadataUpdateRequestEvent.class)); - } - - @Test - public void testAssignOnNullTopicPartition() { - assertThrows(IllegalArgumentException.class, () -> consumer.assign(null)); - } - - @Test - public void testAssignOnEmptyTopicPartition() { - consumer.assign(Collections.emptyList()); - assertTrue(consumer.subscription().isEmpty()); - assertTrue(consumer.assignment().isEmpty()); - } - - @Test - public void testAssignOnNullTopicInPartition() { - assertThrows(IllegalArgumentException.class, () -> consumer.assign(singleton(new TopicPartition(null, 0)))); - } - - @Test - public void testAssignOnEmptyTopicInPartition() { - assertThrows(IllegalArgumentException.class, () -> consumer.assign(singleton(new TopicPartition(" ", 0)))); - } - @Test public void testWakeup_committed() { consumer.wakeup(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java index 278593199e23f..56b3a2468c296 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java @@ -52,10 +52,12 @@ import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.clients.consumer.RangeAssignor; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; +import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ListOffsetsApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.NewTopicsMetadataUpdateRequestEvent; import org.apache.kafka.clients.consumer.internals.events.FetchCommittedOffsetsApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.SubscriptionChangeApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.UnsubscribeApplicationEvent; @@ -138,6 +140,51 @@ public void testInvalidGroupId() { assertThrows(InvalidGroupIdException.class, this::setupWithEmptyGroupId); } + @Test + public void testAssign() { + consumer = setup(); + final TopicPartition tp = new TopicPartition("foo", 3); + consumer.assign(singleton(tp)); + assertTrue(consumer.subscription().isEmpty()); + assertTrue(consumer.assignment().contains(tp)); + verify(applicationEventHandler).add(any(AssignmentChangeApplicationEvent.class)); + verify(applicationEventHandler).add(any(NewTopicsMetadataUpdateRequestEvent.class)); + } + + @Test + public void testAssignOnNullTopicPartition() { + consumer = setup(); + assertThrows(IllegalArgumentException.class, () -> consumer.assign(null)); + } + + @Test + public void testAssignOnEmptyTopicPartition() { + consumer = setup(); + + Mockito.doAnswer(invocation -> { + CompletableApplicationEvent event = invocation.getArgument(0); + assertTrue(event instanceof UnsubscribeApplicationEvent); + event.future().complete(null); + return null; + }).when(applicationEventHandler).add(any()); + + consumer.assign(Collections.emptyList()); + assertTrue(consumer.subscription().isEmpty()); + assertTrue(consumer.assignment().isEmpty()); + } + + @Test + public void testAssignOnNullTopicInPartition() { + consumer = setup(); + assertThrows(IllegalArgumentException.class, () -> consumer.assign(singleton(new TopicPartition(null, 0)))); + } + + @Test + public void testAssignOnEmptyTopicInPartition() { + consumer = setup(); + assertThrows(IllegalArgumentException.class, () -> consumer.assign(singleton(new TopicPartition(" ", 0)))); + } + @Test public void testFencedInstanceException() { consumer = setup(); @@ -417,6 +464,7 @@ public void testSubscriptionOnEmptyTopic() { assertThrows(IllegalArgumentException.class, () -> consumer.subscribe(singletonList(emptyTopic))); } + @Test public void testWakeupBeforeCallingPoll() { consumer = setup(); From 283cf510c1c52639d1167f6fa4eb84e5c4b3dfb5 Mon Sep 17 00:00:00 2001 From: Bruno Cadonna Date: Wed, 6 Dec 2023 18:39:24 +0100 Subject: [PATCH 08/21] Add test testCommitAsync_NullCallback as testCommitAsyncWithNullCallback --- .../internals/AsyncKafkaConsumerUnitTest.java | 20 +++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java index 56b3a2468c296..291893d5539f5 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java @@ -75,6 +75,7 @@ import org.apache.kafka.common.utils.Timer; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; import org.mockito.ArgumentMatchers; import org.mockito.Mockito; @@ -219,6 +220,25 @@ public void testCommitSyncLeaderEpochUpdate() { verify(metadata).updateLastSeenEpochIfNewer(t1, 1); } + @Test + public void testCommitAsyncWithNullCallback() { + consumer = setup(); + final TopicPartition t0 = new TopicPartition("t0", 2); + final TopicPartition t1 = new TopicPartition("t0", 3); + HashMap offsets = new HashMap<>(); + offsets.put(t0, new OffsetAndMetadata(10L)); + offsets.put(t1, new OffsetAndMetadata(20L)); + + consumer.commitAsync(offsets, null); + + final ArgumentCaptor commitEventCaptor = ArgumentCaptor.forClass(CommitApplicationEvent.class); + verify(applicationEventHandler).add(commitEventCaptor.capture()); + final CommitApplicationEvent commitEvent = commitEventCaptor.getValue(); + assertEquals(offsets, commitEvent.offsets()); + assertDoesNotThrow(() -> commitEvent.future().complete(null)); + assertDoesNotThrow(() -> consumer.commitAsync(offsets, null)); + } + @Test public void testCommitAsyncLeaderEpochUpdate() { consumer = setup(); From 53b0fdf7c884093b5ec4d8b949e2771da6a6efa5 Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Thu, 7 Dec 2023 23:16:09 +0100 Subject: [PATCH 09/21] use most of the normal construction code for testing and migrate config tests --- .../internals/AsyncKafkaConsumer.java | 66 ++++- .../internals/AsyncKafkaConsumerTest.java | 238 ------------------ .../internals/AsyncKafkaConsumerUnitTest.java | 112 ++++++--- 3 files changed, 138 insertions(+), 278 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 10d706ac7a77f..a80bbf00ff9d4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -251,13 +251,25 @@ private void process(final GroupMetadataUpdateEvent event) { AsyncKafkaConsumer(final ConsumerConfig config, final Deserializer keyDeserializer, final Deserializer valueDeserializer) { - this(config, keyDeserializer, valueDeserializer, new LinkedBlockingQueue<>()); + this( + config, + keyDeserializer, + valueDeserializer, + Time.SYSTEM, + ApplicationEventHandler::new, + FetchCollector::new, + ConsumerMetadata::new + ); } + // Visible for testing AsyncKafkaConsumer(final ConsumerConfig config, final Deserializer keyDeserializer, final Deserializer valueDeserializer, - final LinkedBlockingQueue backgroundEventQueue) { + final Time time, + final ApplicationEventHandlerSupplier applicationEventHandlerFactory, + final FetchCollectorFactory fetchCollectorFactory, + final ConsumerMetadataFactory metadataFactory) { try { GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig( config, @@ -269,7 +281,7 @@ private void process(final GroupMetadataUpdateEvent event) { log.debug("Initializing the Kafka consumer"); this.defaultApiTimeoutMs = config.getInt(ConsumerConfig.DEFAULT_API_TIMEOUT_MS_CONFIG); - this.time = Time.SYSTEM; + this.time = time; List reporters = CommonClientConfigs.metricsReporters(clientId, config); this.clientTelemetryReporter = CommonClientConfigs.telemetryReporter(clientId, config); this.clientTelemetryReporter.ifPresent(reporters::add); @@ -283,7 +295,7 @@ private void process(final GroupMetadataUpdateEvent event) { ClusterResourceListeners clusterResourceListeners = ClientUtils.configureClusterResourceListeners(metrics.reporters(), interceptorList, Arrays.asList(deserializers.keyDeserializer, deserializers.valueDeserializer)); - this.metadata = new ConsumerMetadata(config, subscriptions, logContext, clusterResourceListeners); + this.metadata = metadataFactory.build(config, subscriptions, logContext, clusterResourceListeners); final List addresses = ClientUtils.parseAndValidateAddresses(config); metadata.bootstrap(addresses); @@ -320,7 +332,8 @@ private void process(final GroupMetadataUpdateEvent event) { metadata, applicationEventQueue, requestManagersSupplier); - this.applicationEventHandler = new ApplicationEventHandler(logContext, + this.applicationEventHandler = applicationEventHandlerFactory.build( + logContext, time, applicationEventQueue, applicationEventProcessorSupplier, @@ -335,7 +348,7 @@ private void process(final GroupMetadataUpdateEvent event) { this.groupMetadata = initializeGroupMetadata(config, groupRebalanceConfig); // The FetchCollector is only used on the application thread. - this.fetchCollector = new FetchCollector<>(logContext, + this.fetchCollector = fetchCollectorFactory.build(logContext, metadata, subscriptions, fetchConfig, @@ -482,6 +495,47 @@ private void process(final GroupMetadataUpdateEvent event) { requestManagersSupplier); } + // auxiliary interface for testing + interface ApplicationEventHandlerSupplier { + + ApplicationEventHandler build( + final LogContext logContext, + final Time time, + final BlockingQueue applicationEventQueue, + final Supplier applicationEventProcessorSupplier, + final Supplier networkClientDelegateSupplier, + final Supplier requestManagersSupplier + ); + + } + + // auxiliary interface for testing + interface FetchCollectorFactory { + + FetchCollector build( + final LogContext logContext, + final ConsumerMetadata metadata, + final SubscriptionState subscriptions, + final FetchConfig fetchConfig, + final Deserializers deserializers, + final FetchMetricsManager metricsManager, + final Time time + ); + + } + + // auxiliary interface for testing + interface ConsumerMetadataFactory { + + ConsumerMetadata build( + final ConsumerConfig config, + final SubscriptionState subscriptions, + final LogContext logContext, + final ClusterResourceListeners clusterResourceListeners + ); + + } + private Optional initializeGroupMetadata(final ConsumerConfig config, final GroupRebalanceConfig groupRebalanceConfig) { final Optional groupMetadata = initializeGroupMetadata( diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index e97409658ce4c..73bd4c8e01f8c 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -19,7 +19,6 @@ import static java.util.Arrays.asList; import static java.util.Collections.singleton; import static java.util.Collections.singletonList; -import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -46,7 +45,6 @@ import java.util.Locale; import java.util.Map; import java.util.Optional; -import java.util.Properties; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; @@ -90,7 +88,6 @@ import org.apache.kafka.common.requests.JoinGroupRequest; import org.apache.kafka.common.requests.OffsetCommitResponse; import org.apache.kafka.common.requests.RequestHeader; -import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.utils.Timer; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.AfterEach; @@ -102,7 +99,6 @@ import org.mockito.ArgumentMatchers; import org.mockito.MockedConstruction; import org.mockito.stubbing.Answer; -import org.opentest4j.AssertionFailedError; public class AsyncKafkaConsumerTest { @@ -367,240 +363,6 @@ public void testRefreshCommittedOffsetsNotCalledIfNoGroupId() { testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(false); } - @Test - public void testGroupMetadataAfterCreationWithGroupIdIsNull() { - final Properties props = requiredConsumerProperties(); - final ConsumerConfig config = new ConsumerConfig(props); - try (final AsyncKafkaConsumer consumer = - new AsyncKafkaConsumer<>(config, new StringDeserializer(), new StringDeserializer())) { - - assertFalse(config.unused().contains(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG)); - assertFalse(config.unused().contains(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED)); - final Throwable exception = assertThrows(InvalidGroupIdException.class, consumer::groupMetadata); - assertEquals( - "To use the group management or offset commit APIs, you must " + - "provide a valid " + ConsumerConfig.GROUP_ID_CONFIG + " in the consumer configuration.", - exception.getMessage() - ); - } - } - - @Test - public void testGroupMetadataAfterCreationWithGroupIdIsNotNull() { - final String groupId = "consumerGroupA"; - final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId(groupId)); - try (final AsyncKafkaConsumer consumer = - new AsyncKafkaConsumer<>(config, new StringDeserializer(), new StringDeserializer())) { - - final ConsumerGroupMetadata groupMetadata = consumer.groupMetadata(); - - assertEquals(groupId, groupMetadata.groupId()); - assertEquals(Optional.empty(), groupMetadata.groupInstanceId()); - assertEquals(JoinGroupRequest.UNKNOWN_GENERATION_ID, groupMetadata.generationId()); - assertEquals(JoinGroupRequest.UNKNOWN_MEMBER_ID, groupMetadata.memberId()); - } - } - - @Test - public void testGroupMetadataAfterCreationWithGroupIdIsNotNullAndGroupInstanceIdSet() { - final String groupId = "consumerGroupA"; - final String groupInstanceId = "groupInstanceId1"; - final Properties props = requiredConsumerPropertiesAndGroupId(groupId); - props.put(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG, groupInstanceId); - final ConsumerConfig config = new ConsumerConfig(props); - try (final AsyncKafkaConsumer consumer = - new AsyncKafkaConsumer<>(config, new StringDeserializer(), new StringDeserializer())) { - - final ConsumerGroupMetadata groupMetadata = consumer.groupMetadata(); - - assertEquals(groupId, groupMetadata.groupId()); - assertEquals(Optional.of(groupInstanceId), groupMetadata.groupInstanceId()); - assertEquals(JoinGroupRequest.UNKNOWN_GENERATION_ID, groupMetadata.generationId()); - assertEquals(JoinGroupRequest.UNKNOWN_MEMBER_ID, groupMetadata.memberId()); - } - } - - @Test - public void testGroupMetadataUpdateSingleCall() { - final String groupId = "consumerGroupA"; - final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId(groupId)); - final LinkedBlockingQueue backgroundEventQueue = new LinkedBlockingQueue<>(); - try (final AsyncKafkaConsumer consumer = - new AsyncKafkaConsumer<>(config, new StringDeserializer(), new StringDeserializer(), backgroundEventQueue)) { - final int generation = 1; - final String memberId = "newMemberId"; - final ConsumerGroupMetadata expectedGroupMetadata = new ConsumerGroupMetadata( - groupId, - generation, - memberId, - Optional.empty() - ); - final GroupMetadataUpdateEvent groupMetadataUpdateEvent = new GroupMetadataUpdateEvent( - generation, - memberId - ); - backgroundEventQueue.add(groupMetadataUpdateEvent); - consumer.assign(singletonList(new TopicPartition("topic", 0))); - consumer.poll(Duration.ZERO); - - final ConsumerGroupMetadata actualGroupMetadata = consumer.groupMetadata(); - - assertEquals(expectedGroupMetadata, actualGroupMetadata); - - final ConsumerGroupMetadata secondActualGroupMetadataWithoutUpdate = consumer.groupMetadata(); - - assertEquals(expectedGroupMetadata, secondActualGroupMetadataWithoutUpdate); - } - } - - @Test - public void testBackgroundError() { - final String groupId = "consumerGroupA"; - final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId(groupId)); - final LinkedBlockingQueue backgroundEventQueue = new LinkedBlockingQueue<>(); - try (final AsyncKafkaConsumer consumer = - new AsyncKafkaConsumer<>(config, new StringDeserializer(), new StringDeserializer(), backgroundEventQueue)) { - final KafkaException expectedException = new KafkaException("Nobody expects the Spanish Inquisition"); - final ErrorBackgroundEvent errorBackgroundEvent = new ErrorBackgroundEvent(expectedException); - backgroundEventQueue.add(errorBackgroundEvent); - consumer.assign(singletonList(new TopicPartition("topic", 0))); - - final KafkaException exception = assertThrows(KafkaException.class, () -> consumer.poll(Duration.ZERO)); - - assertEquals(expectedException.getMessage(), exception.getMessage()); - } - } - - @Test - public void testMultipleBackgroundErrors() { - final String groupId = "consumerGroupA"; - final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId(groupId)); - final LinkedBlockingQueue backgroundEventQueue = new LinkedBlockingQueue<>(); - try (final AsyncKafkaConsumer consumer = - new AsyncKafkaConsumer<>(config, new StringDeserializer(), new StringDeserializer(), backgroundEventQueue)) { - final KafkaException expectedException1 = new KafkaException("Nobody expects the Spanish Inquisition"); - final ErrorBackgroundEvent errorBackgroundEvent1 = new ErrorBackgroundEvent(expectedException1); - backgroundEventQueue.add(errorBackgroundEvent1); - final KafkaException expectedException2 = new KafkaException("Spam, Spam, Spam"); - final ErrorBackgroundEvent errorBackgroundEvent2 = new ErrorBackgroundEvent(expectedException2); - backgroundEventQueue.add(errorBackgroundEvent2); - consumer.assign(singletonList(new TopicPartition("topic", 0))); - - final KafkaException exception = assertThrows(KafkaException.class, () -> consumer.poll(Duration.ZERO)); - - assertEquals(expectedException1.getMessage(), exception.getMessage()); - assertTrue(backgroundEventQueue.isEmpty()); - } - } - - @Test - public void testGroupRemoteAssignorUnusedIfGroupIdUndefined() { - final Properties props = requiredConsumerProperties(); - props.put(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, "someAssignor"); - final ConsumerConfig config = new ConsumerConfig(props); - - try (AsyncKafkaConsumer ignored = - new AsyncKafkaConsumer<>(config, new StringDeserializer(), new StringDeserializer())) { - assertTrue(config.unused().contains(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG)); - } - } - - @Test - public void testGroupRemoteAssignorUnusedInGenericProtocol() { - final Properties props = requiredConsumerProperties(); - props.put(ConsumerConfig.GROUP_ID_CONFIG, "consumerGroupA"); - props.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.GENERIC.name().toLowerCase(Locale.ROOT)); - props.put(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, "someAssignor"); - final ConsumerConfig config = new ConsumerConfig(props); - - try (AsyncKafkaConsumer ignored = - new AsyncKafkaConsumer<>(config, new StringDeserializer(), new StringDeserializer())) { - assertTrue(config.unused().contains(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG)); - } - } - - @Test - public void testGroupRemoteAssignorUsedInConsumerProtocol() { - final Properties props = requiredConsumerProperties(); - props.put(ConsumerConfig.GROUP_ID_CONFIG, "consumerGroupA"); - props.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CONSUMER.name().toLowerCase(Locale.ROOT)); - props.put(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, "someAssignor"); - final ConsumerConfig config = new ConsumerConfig(props); - - try (AsyncKafkaConsumer ignored = - new AsyncKafkaConsumer<>(config, new StringDeserializer(), new StringDeserializer())) { - assertFalse(config.unused().contains(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG)); - } - } - - @Test - public void testGroupIdNull() { - final Properties props = requiredConsumerProperties(); - props.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, 10000); - props.put(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED, true); - final ConsumerConfig config = new ConsumerConfig(props); - - try (final AsyncKafkaConsumer consumer = - new AsyncKafkaConsumer<>(config, new StringDeserializer(), new StringDeserializer())) { - assertFalse(config.unused().contains(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG)); - assertFalse(config.unused().contains(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED)); - } catch (final Exception exception) { - throw new AssertionFailedError("The following exception was not expected:", exception); - } - } - - @Test - public void testGroupIdNotNullAndValid() { - final Properties props = requiredConsumerPropertiesAndGroupId("consumerGroupA"); - props.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, 10000); - props.put(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED, true); - final ConsumerConfig config = new ConsumerConfig(props); - - try (final AsyncKafkaConsumer consumer = - new AsyncKafkaConsumer<>(config, new StringDeserializer(), new StringDeserializer())) { - assertTrue(config.unused().contains(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG)); - assertTrue(config.unused().contains(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED)); - } catch (final Exception exception) { - throw new AssertionFailedError("The following exception was not expected:", exception); - } - } - - @Test - public void testGroupIdEmpty() { - testInvalidGroupId(""); - } - - @Test - public void testGroupIdOnlyWhitespaces() { - testInvalidGroupId(" "); - } - - private void testInvalidGroupId(final String groupId) { - final Properties props = requiredConsumerPropertiesAndGroupId(groupId); - final ConsumerConfig config = new ConsumerConfig(props); - - final Exception exception = assertThrows( - KafkaException.class, - () -> new AsyncKafkaConsumer<>(config, new StringDeserializer(), new StringDeserializer()) - ); - - assertEquals("Failed to construct kafka consumer", exception.getMessage()); - } - - private Properties requiredConsumerPropertiesAndGroupId(final String groupId) { - final Properties props = requiredConsumerProperties(); - props.put(ConsumerConfig.GROUP_ID_CONFIG, groupId); - return props; - } - - private Properties requiredConsumerProperties() { - final Properties props = new Properties(); - props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9091"); - return props; - } - private void testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(boolean committedOffsetsEnabled) { // Uncompleted future that will time out if used CompletableFuture> committedFuture = new CompletableFuture<>(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java index 291893d5539f5..0855542a483a6 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java @@ -19,10 +19,12 @@ import static java.util.Arrays.asList; import static java.util.Collections.singleton; import static java.util.Collections.singletonList; +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED; import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; 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; @@ -41,19 +43,17 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Properties; import java.util.Set; -import java.util.concurrent.BlockingQueue; import java.util.stream.Collectors; import org.apache.kafka.clients.Metadata.LeaderAndEpoch; +import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.clients.consumer.OffsetCommitCallback; -import org.apache.kafka.clients.consumer.OffsetResetStrategy; -import org.apache.kafka.clients.consumer.RangeAssignor; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ListOffsetsApplicationEvent; @@ -66,10 +66,9 @@ import org.apache.kafka.common.errors.InvalidGroupIdException; import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.WakeupException; -import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ListOffsetsRequest; -import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; @@ -85,12 +84,8 @@ public class AsyncKafkaConsumerUnitTest { private AsyncKafkaConsumer consumer = null; private final Time time = new MockTime(); - private final Deserializers deserializers = mock(Deserializers.class); - private final FetchBuffer fetchBuffer = mock(FetchBuffer.class); private final FetchCollector fetchCollector = mock(FetchCollector.class); - private final ConsumerInterceptors interceptors = mock(ConsumerInterceptors.class); private final ApplicationEventHandler applicationEventHandler = mock(ApplicationEventHandler.class); - private final BlockingQueue backgroundEventQueue = mock(BlockingQueue.class); private final ConsumerMetadata metadata = mock(ConsumerMetadata.class); @AfterEach @@ -103,42 +98,91 @@ public void resetAll() { } private AsyncKafkaConsumer setup() { - return setup("group-id"); + Properties props = requiredConsumerProperties(); + props.put(ConsumerConfig.GROUP_ID_CONFIG, "group-id"); + final ConsumerConfig config = new ConsumerConfig(props); + return setup(config); } private AsyncKafkaConsumer setupWithEmptyGroupId() { - return setup(""); + Properties props = requiredConsumerProperties(); + props.put(ConsumerConfig.GROUP_ID_CONFIG, ""); + final ConsumerConfig config = new ConsumerConfig(props); + return setup(config); } - private AsyncKafkaConsumer setup(String groupId) { - String clientId = ""; - long retryBackoffMs = 100; - int defaultApiTimeoutMs = 100; - LogContext logContext = new LogContext(); - SubscriptionState subscriptionState = new SubscriptionState(logContext, OffsetResetStrategy.LATEST); + private AsyncKafkaConsumer setup(ConsumerConfig config) { return new AsyncKafkaConsumer<>( - logContext, - clientId, - deserializers, - fetchBuffer, - fetchCollector, - interceptors, + config, + new StringDeserializer(), + new StringDeserializer(), time, - applicationEventHandler, - backgroundEventQueue, - new Metrics(), - subscriptionState, - metadata, - retryBackoffMs, - defaultApiTimeoutMs, - Collections.singletonList(new RangeAssignor()), - groupId + (a,b,c,d,e,f) -> applicationEventHandler, + (a,b,c,d,e,f,g) -> fetchCollector, + (a,b,c,d) -> metadata ); } + @Test + public void testGroupIdNull() { + final Properties props = requiredConsumerProperties(); + props.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, 10000); + props.put(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED, true); + final ConsumerConfig config = new ConsumerConfig(props); + + consumer = setup(config); + assertFalse(config.unused().contains(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG)); + assertFalse(config.unused().contains(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED)); + } + + @Test + public void testGroupIdNotNullAndValid() { + final Properties props = requiredConsumerProperties(); + props.put(ConsumerConfig.GROUP_ID_CONFIG, "consumerGroupA"); + props.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, 10000); + props.put(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED, true); + final ConsumerConfig config = new ConsumerConfig(props); + + consumer = setup(config); + assertTrue(config.unused().contains(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG)); + assertTrue(config.unused().contains(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED)); + } + + @Test + public void testGroupIdEmpty() { + testInvalidGroupId(""); + } + + @Test + public void testGroupIdOnlyWhitespaces() { + testInvalidGroupId(" "); + } + + private void testInvalidGroupId(final String groupId) { + final Properties props = requiredConsumerProperties(); + props.put(ConsumerConfig.GROUP_ID_CONFIG, groupId); + final ConsumerConfig config = new ConsumerConfig(props); + + final Exception exception = assertThrows( + KafkaException.class, + () -> consumer = setup(config) + ); + + assertEquals("Failed to construct kafka consumer", exception.getMessage()); + } + + private Properties requiredConsumerProperties() { + final Properties props = new Properties(); + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9091"); + return props; + } + @Test public void testInvalidGroupId() { - assertThrows(InvalidGroupIdException.class, this::setupWithEmptyGroupId); + KafkaException e = assertThrows(KafkaException.class, this::setupWithEmptyGroupId); + assertTrue(e.getCause() instanceof InvalidGroupIdException); } @Test From 5630c42c720caeb798bd05a8b4ad219b1f16e082 Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Fri, 8 Dec 2023 16:48:19 +0100 Subject: [PATCH 10/21] fix rebase errors, port remote assignor and group metadata tests --- .../internals/AsyncKafkaConsumer.java | 10 +- .../internals/AsyncKafkaConsumerTest.java | 34 +--- .../internals/AsyncKafkaConsumerUnitTest.java | 180 +++++++++++++++++- 3 files changed, 184 insertions(+), 40 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index a80bbf00ff9d4..898236db8b033 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -258,7 +258,8 @@ private void process(final GroupMetadataUpdateEvent event) { Time.SYSTEM, ApplicationEventHandler::new, FetchCollector::new, - ConsumerMetadata::new + ConsumerMetadata::new, + new LinkedBlockingQueue<>() ); } @@ -269,7 +270,8 @@ private void process(final GroupMetadataUpdateEvent event) { final Time time, final ApplicationEventHandlerSupplier applicationEventHandlerFactory, final FetchCollectorFactory fetchCollectorFactory, - final ConsumerMetadataFactory metadataFactory) { + final ConsumerMetadataFactory metadataFactory, + final LinkedBlockingQueue backgroundEventQueue) { try { GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig( config, @@ -510,9 +512,9 @@ ApplicationEventHandler build( } // auxiliary interface for testing - interface FetchCollectorFactory { + interface FetchCollectorFactory { - FetchCollector build( + FetchCollector build( final LogContext logContext, final ConsumerMetadata metadata, final SubscriptionState subscriptions, diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index 73bd4c8e01f8c..c97b21dbee9de 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -21,7 +21,6 @@ import static java.util.Collections.singletonList; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertSame; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -42,7 +41,6 @@ import java.util.Collections; import java.util.HashMap; import java.util.List; -import java.util.Locale; import java.util.Map; import java.util.Optional; import java.util.Set; @@ -51,33 +49,22 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; -import java.util.concurrent.LinkedBlockingQueue; import java.util.stream.Stream; import org.apache.kafka.clients.ClientResponse; -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.consumer.GroupProtocol; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetCommitCallback; import org.apache.kafka.clients.consumer.RetriableCommitFailedException; import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; -import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; -import org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.FetchCommittedOffsetsApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.GroupMetadataUpdateEvent; -import org.apache.kafka.clients.consumer.internals.events.NewTopicsMetadataUpdateRequestEvent; import org.apache.kafka.clients.consumer.internals.events.ResetPositionsApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ValidatePositionsApplicationEvent; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.GroupAuthorizationException; -import org.apache.kafka.common.errors.InvalidGroupIdException; import org.apache.kafka.common.errors.NetworkException; import org.apache.kafka.common.errors.RetriableException; import org.apache.kafka.common.errors.WakeupException; @@ -85,11 +72,9 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.FindCoordinatorResponse; -import org.apache.kafka.common.requests.JoinGroupRequest; import org.apache.kafka.common.requests.OffsetCommitResponse; import org.apache.kafka.common.requests.RequestHeader; import org.apache.kafka.common.utils.Timer; -import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -159,22 +144,11 @@ public void testSuccessfulStartupShutdownWithAutoCommit() { prepAutocommitOnClose(); } - @Test - public void testCommitAsync_NullCallback() throws InterruptedException { - CompletableFuture future = new CompletableFuture<>(); - Map offsets = new HashMap<>(); - offsets.put(new TopicPartition("my-topic", 0), new OffsetAndMetadata(100L)); - offsets.put(new TopicPartition("my-topic", 1), new OffsetAndMetadata(200L)); - - doReturn(future).when(consumer).commit(offsets, false); - consumer.commitAsync(offsets, null); - future.complete(null); - TestUtils.waitForCondition(future::isDone, - 2000, - "commit future should complete"); - - assertFalse(future.isCompletedExceptionally()); + public void testFailOnClosedConsumer() { + consumer.close(); + final IllegalStateException res = assertThrows(IllegalStateException.class, consumer::assignment); + assertEquals("This consumer has already been closed.", res.getMessage()); } @ParameterizedTest diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java index 0855542a483a6..90c6013d24753 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java @@ -41,21 +41,28 @@ import java.util.Collections; import java.util.HashMap; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Optional; import java.util.Properties; import java.util.Set; +import java.util.concurrent.LinkedBlockingQueue; import java.util.stream.Collectors; import org.apache.kafka.clients.Metadata.LeaderAndEpoch; import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.GroupProtocol; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.clients.consumer.OffsetCommitCallback; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.GroupMetadataUpdateEvent; import org.apache.kafka.clients.consumer.internals.events.ListOffsetsApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.NewTopicsMetadataUpdateRequestEvent; import org.apache.kafka.clients.consumer.internals.events.FetchCommittedOffsetsApplicationEvent; @@ -67,6 +74,7 @@ import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.JoinGroupRequest; import org.apache.kafka.common.requests.ListOffsetsRequest; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.utils.MockTime; @@ -87,9 +95,11 @@ public class AsyncKafkaConsumerUnitTest { private final FetchCollector fetchCollector = mock(FetchCollector.class); private final ApplicationEventHandler applicationEventHandler = mock(ApplicationEventHandler.class); private final ConsumerMetadata metadata = mock(ConsumerMetadata.class); + private final LinkedBlockingQueue backgroundEventQueue = new LinkedBlockingQueue<>(); @AfterEach public void resetAll() { + backgroundEventQueue.clear(); if (consumer != null) { consumer.close(); } @@ -98,15 +108,14 @@ public void resetAll() { } private AsyncKafkaConsumer setup() { - Properties props = requiredConsumerProperties(); + final Properties props = requiredConsumerProperties(); props.put(ConsumerConfig.GROUP_ID_CONFIG, "group-id"); final ConsumerConfig config = new ConsumerConfig(props); return setup(config); } private AsyncKafkaConsumer setupWithEmptyGroupId() { - Properties props = requiredConsumerProperties(); - props.put(ConsumerConfig.GROUP_ID_CONFIG, ""); + final Properties props = requiredConsumerPropertiesAndGroupId(""); final ConsumerConfig config = new ConsumerConfig(props); return setup(config); } @@ -117,9 +126,10 @@ private AsyncKafkaConsumer setup(ConsumerConfig config) { new StringDeserializer(), new StringDeserializer(), time, - (a,b,c,d,e,f) -> applicationEventHandler, - (a,b,c,d,e,f,g) -> fetchCollector, - (a,b,c,d) -> metadata + (a, b, c, d, e, f) -> applicationEventHandler, + (a, b, c, d, e, f, g) -> fetchCollector, + (a, b, c, d) -> metadata, + backgroundEventQueue ); } @@ -171,6 +181,12 @@ private void testInvalidGroupId(final String groupId) { assertEquals("Failed to construct kafka consumer", exception.getMessage()); } + private Properties requiredConsumerPropertiesAndGroupId(final String groupId) { + final Properties props = requiredConsumerProperties(); + props.put(ConsumerConfig.GROUP_ID_CONFIG, groupId); + return props; + } + private Properties requiredConsumerProperties() { final Properties props = new Properties(); props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); @@ -467,6 +483,158 @@ public void testOffsetsForTimesWithZeroTimeout() { ArgumentMatchers.isA(Timer.class)); } + @Test + public void testGroupMetadataAfterCreationWithGroupIdIsNull() { + final Properties props = requiredConsumerProperties(); + final ConsumerConfig config = new ConsumerConfig(props); + consumer = setup(config); + + assertFalse(config.unused().contains(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG)); + assertFalse(config.unused().contains(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED)); + final Throwable exception = assertThrows(InvalidGroupIdException.class, consumer::groupMetadata); + assertEquals( + "To use the group management or offset commit APIs, you must " + + "provide a valid " + ConsumerConfig.GROUP_ID_CONFIG + " in the consumer configuration.", + exception.getMessage() + ); + } + + @Test + public void testGroupMetadataAfterCreationWithGroupIdIsNotNull() { + final String groupId = "consumerGroupA"; + final Properties props = requiredConsumerPropertiesAndGroupId("consumerGroupA"); + final ConsumerConfig config = new ConsumerConfig(props); + + consumer = setup(config); + final ConsumerGroupMetadata groupMetadata = consumer.groupMetadata(); + + assertEquals(groupId, groupMetadata.groupId()); + assertEquals(Optional.empty(), groupMetadata.groupInstanceId()); + assertEquals(JoinGroupRequest.UNKNOWN_GENERATION_ID, groupMetadata.generationId()); + assertEquals(JoinGroupRequest.UNKNOWN_MEMBER_ID, groupMetadata.memberId()); + } + + @Test + public void testGroupMetadataAfterCreationWithGroupIdIsNotNullAndGroupInstanceIdSet() { + final String groupId = "consumerGroupA"; + final String groupInstanceId = "groupInstanceId1"; + final Properties props = requiredConsumerPropertiesAndGroupId(groupId); + props.put(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG, groupInstanceId); + final ConsumerConfig config = new ConsumerConfig(props); + consumer = setup(config); + + final ConsumerGroupMetadata groupMetadata = consumer.groupMetadata(); + + assertEquals(groupId, groupMetadata.groupId()); + assertEquals(Optional.of(groupInstanceId), groupMetadata.groupInstanceId()); + assertEquals(JoinGroupRequest.UNKNOWN_GENERATION_ID, groupMetadata.generationId()); + assertEquals(JoinGroupRequest.UNKNOWN_MEMBER_ID, groupMetadata.memberId()); + } + + @Test + public void testGroupMetadataUpdateSingleCall() { + final String groupId = "consumerGroupA"; + final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId(groupId)); + consumer = setup(config); + + doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); + doReturn(mkMap()).when(applicationEventHandler).addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + + final int generation = 1; + final String memberId = "newMemberId"; + final ConsumerGroupMetadata expectedGroupMetadata = new ConsumerGroupMetadata( + groupId, + generation, + memberId, + Optional.empty() + ); + final GroupMetadataUpdateEvent groupMetadataUpdateEvent = new GroupMetadataUpdateEvent( + generation, + memberId + ); + backgroundEventQueue.add(groupMetadataUpdateEvent); + consumer.assign(singletonList(new TopicPartition("topic", 0))); + consumer.poll(Duration.ZERO); + + final ConsumerGroupMetadata actualGroupMetadata = consumer.groupMetadata(); + + assertEquals(expectedGroupMetadata, actualGroupMetadata); + + final ConsumerGroupMetadata secondActualGroupMetadataWithoutUpdate = consumer.groupMetadata(); + + assertEquals(expectedGroupMetadata, secondActualGroupMetadataWithoutUpdate); + } + + @Test + public void testBackgroundError() { + final String groupId = "consumerGroupA"; + final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId(groupId)); + consumer = setup(config); + + final KafkaException expectedException = new KafkaException("Nobody expects the Spanish Inquisition"); + final ErrorBackgroundEvent errorBackgroundEvent = new ErrorBackgroundEvent(expectedException); + backgroundEventQueue.add(errorBackgroundEvent); + consumer.assign(singletonList(new TopicPartition("topic", 0))); + + final KafkaException exception = assertThrows(KafkaException.class, () -> consumer.poll(Duration.ZERO)); + + assertEquals(expectedException.getMessage(), exception.getMessage()); + } + + @Test + public void testMultipleBackgroundErrors() { + final String groupId = "consumerGroupA"; + final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId(groupId)); + consumer = setup(config); + + final KafkaException expectedException1 = new KafkaException("Nobody expects the Spanish Inquisition"); + final ErrorBackgroundEvent errorBackgroundEvent1 = new ErrorBackgroundEvent(expectedException1); + backgroundEventQueue.add(errorBackgroundEvent1); + final KafkaException expectedException2 = new KafkaException("Spam, Spam, Spam"); + final ErrorBackgroundEvent errorBackgroundEvent2 = new ErrorBackgroundEvent(expectedException2); + backgroundEventQueue.add(errorBackgroundEvent2); + consumer.assign(singletonList(new TopicPartition("topic", 0))); + + final KafkaException exception = assertThrows(KafkaException.class, () -> consumer.poll(Duration.ZERO)); + + assertEquals(expectedException1.getMessage(), exception.getMessage()); + assertTrue(backgroundEventQueue.isEmpty()); + } + + @Test + public void testGroupRemoteAssignorUnusedIfGroupIdUndefined() { + final Properties props = requiredConsumerProperties(); + props.put(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, "someAssignor"); + final ConsumerConfig config = new ConsumerConfig(props); + consumer = setup(config); + + assertTrue(config.unused().contains(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG)); + } + + @Test + public void testGroupRemoteAssignorUnusedInGenericProtocol() { + final Properties props = requiredConsumerProperties(); + props.put(ConsumerConfig.GROUP_ID_CONFIG, "consumerGroupA"); + props.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.GENERIC.name().toLowerCase(Locale.ROOT)); + props.put(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, "someAssignor"); + final ConsumerConfig config = new ConsumerConfig(props); + consumer = setup(config); + + assertTrue(config.unused().contains(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG)); + } + + @Test + public void testGroupRemoteAssignorUsedInConsumerProtocol() { + final Properties props = requiredConsumerProperties(); + props.put(ConsumerConfig.GROUP_ID_CONFIG, "consumerGroupA"); + props.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CONSUMER.name().toLowerCase(Locale.ROOT)); + props.put(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, "someAssignor"); + final ConsumerConfig config = new ConsumerConfig(props); + consumer = setup(config); + + assertFalse(config.unused().contains(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG)); + } + @Test public void testSubscribeGeneratesEvent() { consumer = setup(); From ab35f4e77b345dfc7dbccadea958bc4a2edcd7f7 Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Mon, 11 Dec 2023 09:31:18 +0100 Subject: [PATCH 11/21] port long poll test --- .../internals/AsyncKafkaConsumer.java | 3 ++ .../internals/AsyncKafkaConsumerTest.java | 32 ----------------- .../internals/AsyncKafkaConsumerUnitTest.java | 34 +++++++++++++++++++ 3 files changed, 37 insertions(+), 32 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 898236db8b033..e677f30454579 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -1591,6 +1591,9 @@ int callbacks() { return invoker.callbackQueue.size(); } + // Visible for testing + SubscriptionState subscriptions() { return subscriptions; } + /** * Utility class that helps the application thread to invoke user registered {@link OffsetCommitCallback}. This is * achieved by having the background thread register a {@link OffsetCommitCallbackTask} to the invoker upon the diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index c97b21dbee9de..66b94d57884c2 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -384,38 +384,6 @@ private void testRefreshCommittedOffsetsSuccess(Set partitions, } } - @Test - public void testLongPollWaitIsLimited() { - String topicName = "topic1"; - consumer.subscribe(singletonList(topicName)); - - assertEquals(singleton(topicName), consumer.subscription()); - assertTrue(consumer.assignment().isEmpty()); - - final int partition = 3; - final TopicPartition tp = new TopicPartition(topicName, partition); - final List> records = asList( - new ConsumerRecord<>(topicName, partition, 2, "key1", "value1"), - new ConsumerRecord<>(topicName, partition, 3, "key2", "value2") - ); - - // On the first iteration, return no data; on the second, return two records - doAnswer(invocation -> { - // Mock the subscription being assigned as the first fetch is collected - subscriptions.assignFromSubscribed(Collections.singleton(tp)); - return Fetch.empty(); - }).doAnswer(invocation -> { - return Fetch.forPartition(tp, records, true); - }).when(fetchCollector).collectFetch(any(FetchBuffer.class)); - - // And then poll for up to 10000ms, which should return 2 records without timing out - ConsumerRecords returnedRecords = consumer.poll(Duration.ofMillis(10000)); - assertEquals(2, returnedRecords.count()); - - assertEquals(singleton(topicName), consumer.subscription()); - assertEquals(singleton(tp), consumer.assignment()); - } - private void assertNoPendingWakeup(final WakeupTrigger wakeupTrigger) { assertNull(wakeupTrigger.getPendingTask()); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java index 90c6013d24753..f60a7a0423ff9 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java @@ -52,6 +52,7 @@ import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.GroupProtocol; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetAndTimestamp; @@ -635,6 +636,39 @@ public void testGroupRemoteAssignorUsedInConsumerProtocol() { assertFalse(config.unused().contains(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG)); } + @Test + public void testLongPollWaitIsLimited() { + consumer = setup(); + String topicName = "topic1"; + consumer.subscribe(singletonList(topicName)); + + assertEquals(singleton(topicName), consumer.subscription()); + assertTrue(consumer.assignment().isEmpty()); + + final int partition = 3; + final TopicPartition tp = new TopicPartition(topicName, partition); + final List> records = asList( + new ConsumerRecord<>(topicName, partition, 2, "key1", "value1"), + new ConsumerRecord<>(topicName, partition, 3, "key2", "value2") + ); + + // On the first iteration, return no data; on the second, return two records + doAnswer(invocation -> { + // Mock the subscription being assigned as the first fetch is collected + consumer.subscriptions().assignFromSubscribed(Collections.singleton(tp)); + return Fetch.empty(); + }) + .doAnswer(invocation -> Fetch.forPartition(tp, records, true)) + .when(fetchCollector).collectFetch(any(FetchBuffer.class)); + + // And then poll for up to 10000ms, which should return 2 records without timing out + ConsumerRecords returnedRecords = consumer.poll(Duration.ofMillis(10000)); + assertEquals(2, returnedRecords.count()); + + assertEquals(singleton(topicName), consumer.subscription()); + assertEquals(singleton(tp), consumer.assignment()); + } + @Test public void testSubscribeGeneratesEvent() { consumer = setup(); From 4d3fea909463d0e91884a463d9c805480ca64214 Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Mon, 11 Dec 2023 09:42:41 +0100 Subject: [PATCH 12/21] migrate commit callback tests --- .../internals/AsyncKafkaConsumerTest.java | 143 ------------------ .../internals/AsyncKafkaConsumerUnitTest.java | 81 ++++++++++ 2 files changed, 81 insertions(+), 143 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index 66b94d57884c2..262531f3b9752 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -16,20 +16,13 @@ */ package org.apache.kafka.clients.consumer.internals; -import static java.util.Arrays.asList; import static java.util.Collections.singleton; -import static java.util.Collections.singletonList; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNull; -import static org.junit.jupiter.api.Assertions.assertSame; import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.atLeast; -import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.mockConstruction; import static org.mockito.Mockito.never; @@ -45,28 +38,16 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import java.util.concurrent.Future; -import java.util.stream.Stream; import org.apache.kafka.clients.ClientResponse; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.OffsetAndMetadata; -import org.apache.kafka.clients.consumer.OffsetCommitCallback; -import org.apache.kafka.clients.consumer.RetriableCommitFailedException; import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; import org.apache.kafka.clients.consumer.internals.events.FetchCommittedOffsetsApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ResetPositionsApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ValidatePositionsApplicationEvent; -import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.errors.GroupAuthorizationException; -import org.apache.kafka.common.errors.NetworkException; -import org.apache.kafka.common.errors.RetriableException; import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.message.OffsetCommitResponseData; import org.apache.kafka.common.protocol.ApiKeys; @@ -78,9 +59,6 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.function.Executable; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.MethodSource; import org.mockito.ArgumentMatchers; import org.mockito.MockedConstruction; import org.mockito.stubbing.Answer; @@ -88,10 +66,8 @@ public class AsyncKafkaConsumerTest { private AsyncKafkaConsumer consumer; - private FetchCollector fetchCollector; private ConsumerTestBuilder.AsyncKafkaConsumerTestBuilder testBuilder; private ApplicationEventHandler applicationEventHandler; - private SubscriptionState subscriptions; @BeforeEach public void setup() { @@ -103,8 +79,6 @@ private void setup(Optional groupInfo, boo testBuilder = new ConsumerTestBuilder.AsyncKafkaConsumerTestBuilder(groupInfo, enableAutoCommit, true); applicationEventHandler = testBuilder.applicationEventHandler; consumer = testBuilder.consumer; - fetchCollector = testBuilder.fetchCollector; - subscriptions = testBuilder.subscriptions; } @AfterEach @@ -151,123 +125,6 @@ public void testFailOnClosedConsumer() { assertEquals("This consumer has already been closed.", res.getMessage()); } - @ParameterizedTest - @MethodSource("commitExceptionSupplier") - public void testCommitAsync_UserSuppliedCallback(Exception exception) { - CompletableFuture future = new CompletableFuture<>(); - - Map offsets = new HashMap<>(); - offsets.put(new TopicPartition("my-topic", 1), new OffsetAndMetadata(200L)); - - doReturn(future).when(consumer).commit(offsets, false); - MockCommitCallback callback = new MockCommitCallback(); - assertDoesNotThrow(() -> consumer.commitAsync(offsets, callback)); - - if (exception == null) { - future.complete(null); - consumer.maybeInvokeCommitCallbacks(); - assertNull(callback.exception); - } else { - future.completeExceptionally(exception); - consumer.maybeInvokeCommitCallbacks(); - assertSame(exception.getClass(), callback.exception.getClass()); - } - } - - private static Stream commitExceptionSupplier() { - return Stream.of( - null, // For the successful completion scenario - new KafkaException("Test exception"), - new GroupAuthorizationException("Group authorization exception")); - } - - @Test - public void testEnsureCallbackExecutedByApplicationThread() { - final String currentThread = Thread.currentThread().getName(); - ExecutorService backgroundExecutor = Executors.newSingleThreadExecutor(); - MockCommitCallback callback = new MockCommitCallback(); - CountDownLatch latch = new CountDownLatch(1); // Initialize the latch with a count of 1 - try { - CompletableFuture future = new CompletableFuture<>(); - doReturn(future).when(consumer).commit(new HashMap<>(), false); - assertDoesNotThrow(() -> consumer.commitAsync(new HashMap<>(), callback)); - // Simulating some background work - backgroundExecutor.submit(() -> { - future.complete(null); - latch.countDown(); - }); - latch.await(); - assertEquals(1, consumer.callbacks()); - consumer.maybeInvokeCommitCallbacks(); - assertEquals(currentThread, callback.completionThread); - } catch (Exception e) { - fail("Not expecting an exception"); - } finally { - backgroundExecutor.shutdown(); - } - } - - @Test - public void testEnsureCommitSyncExecutedCommitAsyncCallbacks() { - MockCommitCallback callback = new MockCommitCallback(); - CompletableFuture future = new CompletableFuture<>(); - doReturn(future).when(consumer).commit(new HashMap<>(), false); - assertDoesNotThrow(() -> consumer.commitAsync(new HashMap<>(), callback)); - future.completeExceptionally(new NetworkException("Test exception")); - assertMockCommitCallbackInvoked(() -> consumer.commitSync(), - callback, - Errors.NETWORK_EXCEPTION); - } - - - @Test - public void testEnsurePollExecutedCommitAsyncCallbacks() { - MockCommitCallback callback = new MockCommitCallback(); - CompletableFuture future = new CompletableFuture<>(); - consumer.assign(Collections.singleton(new TopicPartition("foo", 0))); - doReturn(future).when(consumer).commit(new HashMap<>(), false); - assertDoesNotThrow(() -> consumer.commitAsync(new HashMap<>(), callback)); - future.complete(null); - assertMockCommitCallbackInvoked(() -> consumer.poll(Duration.ZERO), - callback, - null); - } - - @Test - public void testEnsureShutdownExecutedCommitAsyncCallbacks() { - MockCommitCallback callback = new MockCommitCallback(); - CompletableFuture future = new CompletableFuture<>(); - doReturn(future).when(consumer).commit(new HashMap<>(), false); - assertDoesNotThrow(() -> consumer.commitAsync(new HashMap<>(), callback)); - future.complete(null); - assertMockCommitCallbackInvoked(() -> consumer.close(), - callback, - null); - } - - private void assertMockCommitCallbackInvoked(final Executable task, - final MockCommitCallback callback, - final Errors errors) { - assertDoesNotThrow(task); - assertEquals(1, callback.invoked); - if (errors == null) - assertNull(callback.exception); - else if (errors.exception() instanceof RetriableException) - assertTrue(callback.exception instanceof RetriableCommitFailedException); - } - - private static class MockCommitCallback implements OffsetCommitCallback { - public int invoked = 0; - public Exception exception = null; - public String completionThread; - - @Override - public void onComplete(Map offsets, Exception exception) { - invoked++; - this.completionThread = Thread.currentThread().getName(); - this.exception = exception; - } - } /** * This is a rather ugly bit of code. Not my choice :( * diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java index f60a7a0423ff9..c04932acf012f 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java @@ -25,8 +25,11 @@ import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertSame; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; @@ -46,8 +49,13 @@ import java.util.Optional; import java.util.Properties; import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.LinkedBlockingQueue; import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.kafka.clients.Metadata.LeaderAndEpoch; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; @@ -71,6 +79,7 @@ import org.apache.kafka.clients.consumer.internals.events.UnsubscribeApplicationEvent; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.GroupAuthorizationException; import org.apache.kafka.common.errors.InvalidGroupIdException; import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.WakeupException; @@ -83,6 +92,8 @@ import org.apache.kafka.common.utils.Timer; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import org.mockito.ArgumentCaptor; import org.mockito.ArgumentMatchers; import org.mockito.Mockito; @@ -317,6 +328,76 @@ public void testCommitAsyncLeaderEpochUpdate() { verify(metadata).updateLastSeenEpochIfNewer(t1, 1); } + @ParameterizedTest + @MethodSource("commitExceptionSupplier") + public void testCommitAsyncUserSuppliedCallback(Exception exception) { + consumer = setup(); + + Map offsets = new HashMap<>(); + offsets.put(new TopicPartition("my-topic", 1), new OffsetAndMetadata(200L)); + + Mockito.doAnswer(invocation -> { + CompletableApplicationEvent event = invocation.getArgument(0); + assertTrue(event instanceof CommitApplicationEvent); + if (exception == null) { + event.future().complete(null); + } else { + event.future().completeExceptionally(exception); + } + event.future().complete(null); + return null; + }).when(applicationEventHandler).add(any()); + + MockCommitCallback callback = new MockCommitCallback(); + assertDoesNotThrow(() -> consumer.commitAsync(offsets, callback)); + consumer.maybeInvokeCommitCallbacks(); + + if (exception == null) { + assertNull(callback.exception); + } else { + assertSame(exception.getClass(), callback.exception.getClass()); + } + } + + private static Stream commitExceptionSupplier() { + return Stream.of( + null, // For the successful completion scenario + new KafkaException("Test exception"), + new GroupAuthorizationException("Group authorization exception")); + } + + @Test + public void testEnsureCallbackExecutedByApplicationThread() { + consumer = setup(); + final String currentThread = Thread.currentThread().getName(); + MockCommitCallback callback = new MockCommitCallback(); + + Mockito.doAnswer(invocation -> { + CompletableApplicationEvent event = invocation.getArgument(0); + assertTrue(event instanceof CommitApplicationEvent); + event.future().complete(null); + return null; + }).when(applicationEventHandler).add(any()); + + assertDoesNotThrow(() -> consumer.commitAsync(new HashMap<>(), callback)); + assertEquals(1, consumer.callbacks()); + consumer.maybeInvokeCommitCallbacks(); + assertEquals(currentThread, callback.completionThread); + } + + private static class MockCommitCallback implements OffsetCommitCallback { + public int invoked = 0; + public Exception exception = null; + public String completionThread; + + @Override + public void onComplete(Map offsets, Exception exception) { + invoked++; + this.completionThread = Thread.currentThread().getName(); + this.exception = exception; + } + } + @Test public void testCommitted() { consumer = setup(); From b00135ecb9ef0d06931e1a10a1cce7af3d4d4a66 Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Mon, 11 Dec 2023 09:58:19 +0100 Subject: [PATCH 13/21] migrate wakeup committed --- .../internals/AsyncKafkaConsumerTest.java | 46 ------------------- .../internals/AsyncKafkaConsumerUnitTest.java | 32 +++++++++++++ 2 files changed, 32 insertions(+), 46 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index 262531f3b9752..c8b9fd077eed6 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -99,32 +99,6 @@ private void resetWithEmptyGroupId() { setup(Optional.empty(), false); } - private void resetWithAutoCommitEnabled() { - cleanup(); - setup(ConsumerTestBuilder.createDefaultGroupInformation(), true); - } - - @Test - public void testSuccessfulStartupShutdown() { - assertDoesNotThrow(() -> consumer.close()); - } - - @Test - public void testSuccessfulStartupShutdownWithAutoCommit() { - resetWithAutoCommitEnabled(); - TopicPartition tp = new TopicPartition("topic", 0); - consumer.assign(singleton(tp)); - consumer.seek(tp, 100); - prepAutocommitOnClose(); - } - - @Test - public void testFailOnClosedConsumer() { - consumer.close(); - final IllegalStateException res = assertThrows(IllegalStateException.class, consumer::assignment); - assertEquals("This consumer has already been closed.", res.getMessage()); - } - /** * This is a rather ugly bit of code. Not my choice :( * @@ -159,13 +133,6 @@ private static MockedConstruction offsetF return mockConstruction(FetchCommittedOffsetsApplicationEvent.class, mockInitializer); } - @Test - public void testWakeup_committed() { - consumer.wakeup(); - assertThrows(WakeupException.class, () -> consumer.committed(mockTopicPartitionOffset().keySet())); - assertNoPendingWakeup(consumer.wakeupTrigger()); - } - @Test public void testRefreshCommittedOffsetsSuccess() { TopicPartition partition = new TopicPartition("t1", 1); @@ -241,19 +208,6 @@ private void testRefreshCommittedOffsetsSuccess(Set partitions, } } - private void assertNoPendingWakeup(final WakeupTrigger wakeupTrigger) { - assertNull(wakeupTrigger.getPendingTask()); - } - - private HashMap mockTopicPartitionOffset() { - final TopicPartition t0 = new TopicPartition("t0", 2); - final TopicPartition t1 = new TopicPartition("t0", 3); - HashMap topicPartitionOffsets = new HashMap<>(); - topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L)); - topicPartitionOffsets.put(t1, new OffsetAndMetadata(20L)); - return topicPartitionOffsets; - } - private void prepAutocommitOnClose() { Node node = testBuilder.metadata.fetch().nodes().get(0); testBuilder.client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "group-id", node)); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java index c04932acf012f..dd4e4503edc35 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java @@ -213,6 +213,19 @@ public void testInvalidGroupId() { assertTrue(e.getCause() instanceof InvalidGroupIdException); } + @Test + public void testSuccessfulStartupShutdown() { + consumer = setup(); + assertDoesNotThrow(() -> consumer.close()); + } + + @Test + public void testFailOnClosedConsumer() { + consumer.close(); + final IllegalStateException res = assertThrows(IllegalStateException.class, consumer::assignment); + assertEquals("This consumer has already been closed.", res.getMessage()); + } + @Test public void testAssign() { consumer = setup(); @@ -896,6 +909,25 @@ public void testClearWakeupTriggerAfterPoll() { assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); } + @Test + public void testWakeupCommitted() { + consumer = setup(); + final HashMap offsets = mockTopicPartitionOffset(); + doAnswer(invocation -> { + CompletableApplicationEvent event = invocation.getArgument(0); + Timer timer = invocation.getArgument(1); + assertTrue(event instanceof FetchCommittedOffsetsApplicationEvent); + assertTrue(event.future().isCompletedExceptionally()); + return ConsumerUtils.getResult(event.future(), timer); + }) + .when(applicationEventHandler) + .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + + consumer.wakeup(); + assertThrows(WakeupException.class, () -> consumer.committed(offsets.keySet())); + assertNull(consumer.wakeupTrigger().getPendingTask()); + } + private HashMap mockTopicPartitionOffset() { final TopicPartition t0 = new TopicPartition("t0", 2); final TopicPartition t1 = new TopicPartition("t0", 3); From 90a1f5d0308f2c84300ab7df4c2830f7434cc80a Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Mon, 11 Dec 2023 11:26:58 +0100 Subject: [PATCH 14/21] reorder --- .../internals/AsyncKafkaConsumerTest.java | 1066 ++++++++++++++--- .../internals/AsyncKafkaConsumerUnitTest.java | 959 --------------- 2 files changed, 932 insertions(+), 1093 deletions(-) delete mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index c8b9fd077eed6..955fd126f447f 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -16,125 +16,621 @@ */ package org.apache.kafka.clients.consumer.internals; +import static java.util.Arrays.asList; import static java.util.Collections.singleton; +import static java.util.Collections.singletonList; +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED; +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertSame; 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.atLeast; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.mockConstruction; import static org.mockito.Mockito.never; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import java.time.Duration; -import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Optional; +import java.util.Properties; import java.util.Set; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.Future; -import org.apache.kafka.clients.ClientResponse; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.kafka.clients.Metadata.LeaderAndEpoch; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.GroupProtocol; import org.apache.kafka.clients.consumer.OffsetAndMetadata; -import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; +import org.apache.kafka.clients.consumer.OffsetAndTimestamp; +import org.apache.kafka.clients.consumer.OffsetCommitCallback; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; +import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.FetchCommittedOffsetsApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.GroupMetadataUpdateEvent; +import org.apache.kafka.clients.consumer.internals.events.ListOffsetsApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.NewTopicsMetadataUpdateRequestEvent; import org.apache.kafka.clients.consumer.internals.events.ResetPositionsApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.SubscriptionChangeApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.UnsubscribeApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ValidatePositionsApplicationEvent; -import org.apache.kafka.common.Node; +import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.GroupAuthorizationException; +import org.apache.kafka.common.errors.InvalidGroupIdException; +import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.WakeupException; -import org.apache.kafka.common.message.OffsetCommitResponseData; -import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.requests.FindCoordinatorResponse; -import org.apache.kafka.common.requests.OffsetCommitResponse; -import org.apache.kafka.common.requests.RequestHeader; +import org.apache.kafka.common.requests.JoinGroupRequest; +import org.apache.kafka.common.requests.ListOffsetsRequest; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; +import org.mockito.ArgumentCaptor; import org.mockito.ArgumentMatchers; -import org.mockito.MockedConstruction; -import org.mockito.stubbing.Answer; +import org.mockito.Mockito; +@SuppressWarnings("unchecked") public class AsyncKafkaConsumerTest { - private AsyncKafkaConsumer consumer; - private ConsumerTestBuilder.AsyncKafkaConsumerTestBuilder testBuilder; - private ApplicationEventHandler applicationEventHandler; + private AsyncKafkaConsumer consumer = null; - @BeforeEach - public void setup() { - // By default, the consumer is part of a group and autoCommit is enabled. - setup(ConsumerTestBuilder.createDefaultGroupInformation(), true); + private final Time time = new MockTime(1); + private final FetchCollector fetchCollector = mock(FetchCollector.class); + private final ApplicationEventHandler applicationEventHandler = mock(ApplicationEventHandler.class); + private final ConsumerMetadata metadata = mock(ConsumerMetadata.class); + private final LinkedBlockingQueue backgroundEventQueue = new LinkedBlockingQueue<>(); + + @AfterEach + public void resetAll() { + backgroundEventQueue.clear(); + if (consumer != null) { + consumer.close(); + } + consumer = null; + Mockito.framework().clearInlineMocks(); } - private void setup(Optional groupInfo, boolean enableAutoCommit) { - testBuilder = new ConsumerTestBuilder.AsyncKafkaConsumerTestBuilder(groupInfo, enableAutoCommit, true); - applicationEventHandler = testBuilder.applicationEventHandler; - consumer = testBuilder.consumer; + private AsyncKafkaConsumer setup() { + final Properties props = requiredConsumerProperties(); + props.put(ConsumerConfig.GROUP_ID_CONFIG, "group-id"); + final ConsumerConfig config = new ConsumerConfig(props); + return setup(config); } - @AfterEach - public void cleanup() { - if (testBuilder != null) { - shutDown(); + private AsyncKafkaConsumer setupWithoutGroupId() { + final Properties props = requiredConsumerProperties(); + final ConsumerConfig config = new ConsumerConfig(props); + return setup(config); + } + + @SuppressWarnings("UnusedReturnValue") + private AsyncKafkaConsumer setupWithEmptyGroupId() { + final Properties props = requiredConsumerPropertiesAndGroupId(""); + final ConsumerConfig config = new ConsumerConfig(props); + return setup(config); + } + + private AsyncKafkaConsumer setup(ConsumerConfig config) { + return new AsyncKafkaConsumer<>( + config, + new StringDeserializer(), + new StringDeserializer(), + time, + (a, b, c, d, e, f) -> applicationEventHandler, + (a, b, c, d, e, f, g) -> fetchCollector, + (a, b, c, d) -> metadata, + backgroundEventQueue + ); + } + + @Test + public void testSuccessfulStartupShutdown() { + consumer = setup(); + assertDoesNotThrow(() -> consumer.close()); + } + + @Test + public void testInvalidGroupId() { + KafkaException e = assertThrows(KafkaException.class, this::setupWithEmptyGroupId); + assertTrue(e.getCause() instanceof InvalidGroupIdException); + } + + @Test + public void testCommitAsyncWithNullCallback() { + consumer = setup(); + final TopicPartition t0 = new TopicPartition("t0", 2); + final TopicPartition t1 = new TopicPartition("t0", 3); + HashMap offsets = new HashMap<>(); + offsets.put(t0, new OffsetAndMetadata(10L)); + offsets.put(t1, new OffsetAndMetadata(20L)); + + consumer.commitAsync(offsets, null); + + final ArgumentCaptor commitEventCaptor = ArgumentCaptor.forClass(CommitApplicationEvent.class); + verify(applicationEventHandler).add(commitEventCaptor.capture()); + final CommitApplicationEvent commitEvent = commitEventCaptor.getValue(); + assertEquals(offsets, commitEvent.offsets()); + assertDoesNotThrow(() -> commitEvent.future().complete(null)); + assertDoesNotThrow(() -> consumer.commitAsync(offsets, null)); + } + + private static Stream commitExceptionSupplier() { + return Stream.of( + null, // For the successful completion scenario + new KafkaException("Test exception"), + new GroupAuthorizationException("Group authorization exception")); + } + + @ParameterizedTest + @MethodSource("commitExceptionSupplier") + public void testCommitAsyncUserSuppliedCallback(Exception exception) { + consumer = setup(); + + Map offsets = new HashMap<>(); + offsets.put(new TopicPartition("my-topic", 1), new OffsetAndMetadata(200L)); + + Mockito.doAnswer(invocation -> { + CompletableApplicationEvent event = invocation.getArgument(0); + assertTrue(event instanceof CommitApplicationEvent); + if (exception == null) { + event.future().complete(null); + } else { + event.future().completeExceptionally(exception); + } + event.future().complete(null); + return null; + }).when(applicationEventHandler).add(any()); + + MockCommitCallback callback = new MockCommitCallback(); + assertDoesNotThrow(() -> consumer.commitAsync(offsets, callback)); + consumer.maybeInvokeCommitCallbacks(); + + if (exception == null) { + assertNull(callback.exception); + } else { + assertSame(exception.getClass(), callback.exception.getClass()); + } + } + + @Test + public void testFencedInstanceException() { + consumer = setup(); + Mockito.doAnswer(invocation -> { + CompletableApplicationEvent event = invocation.getArgument(0); + assertTrue(event instanceof CommitApplicationEvent); + event.future().completeExceptionally(Errors.FENCED_INSTANCE_ID.exception()); + return null; + }).when(applicationEventHandler).add(any()); + + assertDoesNotThrow(() -> consumer.commitAsync()); + } + + @Test + public void testCommitted() { + consumer = setup(); + Map topicPartitionOffsets = mockTopicPartitionOffset(); + + when(applicationEventHandler.addAndGet(any(), any())).thenAnswer(invocation -> { + CompletableApplicationEvent event = invocation.getArgument(0); + assertTrue(event instanceof FetchCommittedOffsetsApplicationEvent); + return topicPartitionOffsets; + }); + + assertDoesNotThrow(() -> consumer.committed(topicPartitionOffsets.keySet(), Duration.ofMillis(1000))); + } + + @Test + public void testCommittedExceptionThrown() { + consumer = setup(); + Map topicPartitionOffsets = mockTopicPartitionOffset(); + when(applicationEventHandler.addAndGet(any(), any())).thenAnswer(invocation -> { + CompletableApplicationEvent event = invocation.getArgument(0); + assertTrue(event instanceof FetchCommittedOffsetsApplicationEvent); + throw new KafkaException("Test exception"); + }); + + assertThrows(KafkaException.class, () -> consumer.committed(topicPartitionOffsets.keySet(), Duration.ofMillis(1000))); + } + + @Test + public void testCommittedLeaderEpochUpdate() { + consumer = setup(); + final TopicPartition t0 = new TopicPartition("t0", 2); + final TopicPartition t1 = new TopicPartition("t0", 3); + final TopicPartition t2 = new TopicPartition("t0", 4); + HashMap topicPartitionOffsets = new HashMap<>(); + topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L, Optional.of(2), "")); + topicPartitionOffsets.put(t1, null); + topicPartitionOffsets.put(t2, new OffsetAndMetadata(20L, Optional.of(3), "")); + when(applicationEventHandler.addAndGet(any(), any())).thenAnswer(invocation -> { + CompletableApplicationEvent event = invocation.getArgument(0); + assertTrue(event instanceof FetchCommittedOffsetsApplicationEvent); + return topicPartitionOffsets; + }); + + assertDoesNotThrow(() -> consumer.committed(topicPartitionOffsets.keySet(), Duration.ofMillis(1000))); + + verify(metadata).updateLastSeenEpochIfNewer(t0, 2); + verify(metadata).updateLastSeenEpochIfNewer(t2, 3); + } + + + @Test + public void testWakeupBeforeCallingPoll() { + consumer = setup(); + final String topicName = "foo"; + final int partition = 3; + final TopicPartition tp = new TopicPartition(topicName, partition); + doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); + Map offsets = mkMap(mkEntry(tp, new OffsetAndMetadata(1))); + doReturn(offsets).when(applicationEventHandler).addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); + consumer.assign(singleton(tp)); + + consumer.wakeup(); + + assertThrows(WakeupException.class, () -> consumer.poll(Duration.ZERO)); + assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); + } + + @Test + public void testWakeupAfterEmptyFetch() { + consumer = setup(); + final String topicName = "foo"; + final int partition = 3; + final TopicPartition tp = new TopicPartition(topicName, partition); + doAnswer(invocation -> { + consumer.wakeup(); + return Fetch.empty(); + }).when(fetchCollector).collectFetch(any(FetchBuffer.class)); + Map offsets = mkMap(mkEntry(tp, new OffsetAndMetadata(1))); + doReturn(offsets).when(applicationEventHandler).addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); + consumer.assign(singleton(tp)); + + assertThrows(WakeupException.class, () -> consumer.poll(Duration.ofMinutes(1))); + assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); + } + + @Test + public void testWakeupAfterNonEmptyFetch() { + consumer = setup(); + final String topicName = "foo"; + final int partition = 3; + final TopicPartition tp = new TopicPartition(topicName, partition); + final List> records = asList( + new ConsumerRecord<>(topicName, partition, 2, "key1", "value1"), + new ConsumerRecord<>(topicName, partition, 3, "key2", "value2") + ); + doAnswer(invocation -> { + consumer.wakeup(); + return Fetch.forPartition(tp, records, true); + }).when(fetchCollector).collectFetch(Mockito.any(FetchBuffer.class)); + Map offsets = mkMap(mkEntry(tp, new OffsetAndMetadata(1))); + doReturn(offsets).when(applicationEventHandler).addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); + consumer.assign(singleton(tp)); + + // since wakeup() is called when the non-empty fetch is returned the wakeup should be ignored + assertDoesNotThrow(() -> consumer.poll(Duration.ofMinutes(1))); + // the previously ignored wake-up should not be ignored in the next call + assertThrows(WakeupException.class, () -> consumer.poll(Duration.ZERO)); + } + + @Test + public void testClearWakeupTriggerAfterPoll() { + consumer = setup(); + final String topicName = "foo"; + final int partition = 3; + final TopicPartition tp = new TopicPartition(topicName, partition); + final List> records = asList( + new ConsumerRecord<>(topicName, partition, 2, "key1", "value1"), + new ConsumerRecord<>(topicName, partition, 3, "key2", "value2") + ); + doReturn(Fetch.forPartition(tp, records, true)) + .when(fetchCollector).collectFetch(any(FetchBuffer.class)); + Map offsets = mkMap(mkEntry(tp, new OffsetAndMetadata(1))); + doReturn(offsets).when(applicationEventHandler).addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); + consumer.assign(singleton(tp)); + + consumer.poll(Duration.ZERO); + + assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); + } + + @Test + public void testEnsureCallbackExecutedByApplicationThread() { + consumer = setup(); + final String currentThread = Thread.currentThread().getName(); + MockCommitCallback callback = new MockCommitCallback(); + + Mockito.doAnswer(invocation -> { + CompletableApplicationEvent event = invocation.getArgument(0); + assertTrue(event instanceof CommitApplicationEvent); + event.future().complete(null); + return null; + }).when(applicationEventHandler).add(any()); + + assertDoesNotThrow(() -> consumer.commitAsync(new HashMap<>(), callback)); + assertEquals(1, consumer.callbacks()); + consumer.maybeInvokeCommitCallbacks(); + assertEquals(currentThread, callback.completionThread); + } + + + @Test + @SuppressWarnings("deprecation") + public void testPollLongThrowsException() { + consumer = setup(); + Exception e = assertThrows(UnsupportedOperationException.class, () -> consumer.poll(0L)); + assertEquals("Consumer.poll(long) is not supported when \"group.protocol\" is \"consumer\". " + + "This method is feprecated and will be removed in the next major release.", e.getMessage()); + } + + @Test + public void testFailOnClosedConsumer() { + consumer = setup(); + consumer.close(); + final IllegalStateException res = assertThrows(IllegalStateException.class, consumer::assignment); + assertEquals("This consumer has already been closed.", res.getMessage()); + } + + @Test + public void testCommitSyncLeaderEpochUpdate() { + consumer = setup(); + final TopicPartition t0 = new TopicPartition("t0", 2); + final TopicPartition t1 = new TopicPartition("t0", 3); + HashMap topicPartitionOffsets = new HashMap<>(); + topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L, Optional.of(2), "")); + topicPartitionOffsets.put(t1, new OffsetAndMetadata(20L, Optional.of(1), "")); + Mockito.doAnswer(invocation -> { + CompletableApplicationEvent event = invocation.getArgument(0); + assertTrue(event instanceof CommitApplicationEvent); + event.future().complete(null); + return null; + }).when(applicationEventHandler).add(any()); + + assertDoesNotThrow(() -> consumer.commitSync(topicPartitionOffsets)); + + verify(metadata).updateLastSeenEpochIfNewer(t0, 2); + verify(metadata).updateLastSeenEpochIfNewer(t1, 1); + } + + @Test + public void testCommitAsyncLeaderEpochUpdate() { + consumer = setup(); + OffsetCommitCallback callback = mock(OffsetCommitCallback.class); + final TopicPartition t0 = new TopicPartition("t0", 2); + final TopicPartition t1 = new TopicPartition("t0", 3); + HashMap topicPartitionOffsets = new HashMap<>(); + topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L, Optional.of(2), "")); + topicPartitionOffsets.put(t1, new OffsetAndMetadata(20L, Optional.of(1), "")); + consumer.assign(Arrays.asList(t0, t1)); + + consumer.commitAsync(topicPartitionOffsets, callback); + + verify(metadata).updateLastSeenEpochIfNewer(t0, 2); + verify(metadata).updateLastSeenEpochIfNewer(t1, 1); + } + + private static class MockCommitCallback implements OffsetCommitCallback { + public int invoked = 0; + public Exception exception = null; + public String completionThread; + + @Override + public void onComplete(Map offsets, Exception exception) { + invoked++; + this.completionThread = Thread.currentThread().getName(); + this.exception = exception; } } - private void shutDown() { - prepAutocommitOnClose(); - testBuilder.close(); + @Test + public void testAssign() { + consumer = setup(); + final TopicPartition tp = new TopicPartition("foo", 3); + consumer.assign(singleton(tp)); + assertTrue(consumer.subscription().isEmpty()); + assertTrue(consumer.assignment().contains(tp)); + verify(applicationEventHandler).add(any(AssignmentChangeApplicationEvent.class)); + verify(applicationEventHandler).add(any(NewTopicsMetadataUpdateRequestEvent.class)); + } + + @Test + public void testAssignOnNullTopicPartition() { + consumer = setup(); + assertThrows(IllegalArgumentException.class, () -> consumer.assign(null)); } - private void resetWithEmptyGroupId() { - // Create a consumer that is not configured as part of a group. - cleanup(); - setup(Optional.empty(), false); + @Test + public void testAssignOnEmptyTopicPartition() { + consumer = setup(); + + Mockito.doAnswer(invocation -> { + CompletableApplicationEvent event = invocation.getArgument(0); + assertTrue(event instanceof UnsubscribeApplicationEvent); + event.future().complete(null); + return null; + }).when(applicationEventHandler).add(any()); + + consumer.assign(Collections.emptyList()); + assertTrue(consumer.subscription().isEmpty()); + assertTrue(consumer.assignment().isEmpty()); } - /** - * This is a rather ugly bit of code. Not my choice :( - * - *

- * - * Inside the {@link org.apache.kafka.clients.consumer.Consumer#committed(Set, Duration)} call we create an - * instance of {@link FetchCommittedOffsetsApplicationEvent} that holds the partitions and internally holds a - * {@link CompletableFuture}. We want to test different behaviours of the {@link Future#get()}, such as - * returning normally, timing out, throwing an error, etc. By mocking the construction of the event object that - * is created, we can affect that behavior. - */ - private static MockedConstruction offsetFetchEventMocker(CompletableFuture> future) { - // This "answer" is where we pass the future to be invoked by the ConsumerUtils.getResult() method - Answer> getInvocationAnswer = invocation -> { - // This argument captures the actual argument value that was passed to the event's get() method, so we - // just "forward" that value to our mocked call - Timer timer = invocation.getArgument(0); - return ConsumerUtils.getResult(future, timer); - }; + @Test + public void testAssignOnNullTopicInPartition() { + consumer = setup(); + assertThrows(IllegalArgumentException.class, () -> consumer.assign(singleton(new TopicPartition(null, 0)))); + } - MockedConstruction.MockInitializer mockInitializer = (mock, ctx) -> { - // When the event's get() method is invoked, we call the "answer" method just above - when(mock.get(any())).thenAnswer(getInvocationAnswer); + @Test + public void testAssignOnEmptyTopicInPartition() { + consumer = setup(); + assertThrows(IllegalArgumentException.class, () -> consumer.assign(singleton(new TopicPartition(" ", 0)))); + } - // When the event's type() method is invoked, we have to return the type as it will be null in the mock - when(mock.type()).thenReturn(ApplicationEvent.Type.FETCH_COMMITTED_OFFSETS); + @Test + public void testBeginningOffsetsFailsIfNullPartitions() { + consumer = setup(); + assertThrows(NullPointerException.class, () -> consumer.beginningOffsets(null, + Duration.ofMillis(1))); + } - // This is needed for the WakeupTrigger code that keeps track of the active task - when(mock.future()).thenReturn(future); - }; + @Test + public void testBeginningOffsets() { + consumer = setup(); + Map expectedOffsetsAndTimestamp = + mockOffsetAndTimestamp(); + Set partitions = expectedOffsetsAndTimestamp.keySet(); + doReturn(expectedOffsetsAndTimestamp).when(applicationEventHandler).addAndGet(any(), any()); + Map result = + assertDoesNotThrow(() -> consumer.beginningOffsets(partitions, + Duration.ofMillis(1))); + Map expectedOffsets = expectedOffsetsAndTimestamp.entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().offset())); + assertEquals(expectedOffsets, result); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + ArgumentMatchers.isA(Timer.class)); + } - return mockConstruction(FetchCommittedOffsetsApplicationEvent.class, mockInitializer); + @Test + public void testBeginningOffsetsThrowsKafkaExceptionForUnderlyingExecutionFailure() { + consumer = setup(); + Set partitions = mockTopicPartitionOffset().keySet(); + Throwable eventProcessingFailure = new KafkaException("Unexpected failure " + + "processing List Offsets event"); + doThrow(eventProcessingFailure).when(applicationEventHandler).addAndGet(any(), any()); + Throwable consumerError = assertThrows(KafkaException.class, + () -> consumer.beginningOffsets(partitions, + Duration.ofMillis(1))); + assertEquals(eventProcessingFailure, consumerError); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); } + @Test + public void testBeginningOffsetsTimeoutOnEventProcessingTimeout() { + consumer = setup(); + doThrow(new TimeoutException()).when(applicationEventHandler).addAndGet(any(), any()); + assertThrows(TimeoutException.class, + () -> consumer.beginningOffsets( + Collections.singletonList(new TopicPartition("t1", 0)), + Duration.ofMillis(1))); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + ArgumentMatchers.isA(Timer.class)); + } + + @Test + public void testOffsetsForTimesOnNullPartitions() { + consumer = setup(); + assertThrows(NullPointerException.class, () -> consumer.offsetsForTimes(null, + Duration.ofMillis(1))); + } + + @Test + public void testOffsetsForTimesFailsOnNegativeTargetTimes() { + consumer = setup(); + assertThrows(IllegalArgumentException.class, + () -> consumer.offsetsForTimes(Collections.singletonMap(new TopicPartition( + "topic1", 1), ListOffsetsRequest.EARLIEST_TIMESTAMP), + Duration.ofMillis(1))); + + assertThrows(IllegalArgumentException.class, + () -> consumer.offsetsForTimes(Collections.singletonMap(new TopicPartition( + "topic1", 1), ListOffsetsRequest.LATEST_TIMESTAMP), + Duration.ofMillis(1))); + + assertThrows(IllegalArgumentException.class, + () -> consumer.offsetsForTimes(Collections.singletonMap(new TopicPartition( + "topic1", 1), ListOffsetsRequest.MAX_TIMESTAMP), + Duration.ofMillis(1))); + } + + @Test + public void testOffsetsForTimes() { + consumer = setup(); + Map expectedResult = mockOffsetAndTimestamp(); + Map timestampToSearch = mockTimestampToSearch(); + + doReturn(expectedResult).when(applicationEventHandler).addAndGet(any(), any()); + Map result = + assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, Duration.ofMillis(1))); + assertEquals(expectedResult, result); + verify(applicationEventHandler).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() { + consumer = setup(); + TopicPartition tp = new TopicPartition("topic1", 0); + Map expectedResult = + Collections.singletonMap(tp, null); + Map timestampToSearch = Collections.singletonMap(tp, 5L); + + Map result = + assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, + Duration.ofMillis(0))); + assertEquals(expectedResult, result); + verify(applicationEventHandler, never()).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + ArgumentMatchers.isA(Timer.class)); + } + + @Test + public void testWakeupCommitted() { + consumer = setup(); + final HashMap offsets = mockTopicPartitionOffset(); + doAnswer(invocation -> { + CompletableApplicationEvent event = invocation.getArgument(0); + Timer timer = invocation.getArgument(1); + assertTrue(event instanceof FetchCommittedOffsetsApplicationEvent); + assertTrue(event.future().isCompletedExceptionally()); + return ConsumerUtils.getResult(event.future(), timer); + }) + .when(applicationEventHandler) + .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + + consumer.wakeup(); + assertThrows(WakeupException.class, () -> consumer.committed(offsets.keySet())); + assertNull(consumer.wakeupTrigger().getPendingTask()); + } + + @Test public void testRefreshCommittedOffsetsSuccess() { + consumer = setup(); TopicPartition partition = new TopicPartition("t1", 1); Set partitions = Collections.singleton(partition); Map committedOffsets = Collections.singletonMap(partition, new OffsetAndMetadata(10L)); @@ -143,6 +639,7 @@ public void testRefreshCommittedOffsetsSuccess() { @Test public void testRefreshCommittedOffsetsSuccessButNoCommittedOffsetsFound() { + consumer = setup(); TopicPartition partition = new TopicPartition("t1", 1); Set partitions = Collections.singleton(partition); Map committedOffsets = Collections.emptyMap(); @@ -151,102 +648,403 @@ public void testRefreshCommittedOffsetsSuccessButNoCommittedOffsetsFound() { @Test public void testRefreshCommittedOffsetsShouldNotResetIfFailedWithTimeout() { + consumer = setup(); testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(true); } @Test public void testRefreshCommittedOffsetsNotCalledIfNoGroupId() { + consumer = setupWithoutGroupId(); // Create consumer without group id so committed offsets are not used for updating positions - resetWithEmptyGroupId(); testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(false); } private void testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(boolean committedOffsetsEnabled) { - // Uncompleted future that will time out if used - CompletableFuture> committedFuture = new CompletableFuture<>(); + // Do not complete future that is passed into addAndGet, to timeout + doReturn(mkMap()) + .when(applicationEventHandler) + .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); consumer.assign(singleton(new TopicPartition("t1", 1))); - try (MockedConstruction ignored = offsetFetchEventMocker(committedFuture)) { - // Poll with 250ms timeout to give the background thread time to process the events without timing out - consumer.poll(Duration.ofMillis(250)); + // Poll with 250ms timeout to give the background thread time to process the events without timing out + consumer.poll(Duration.ofMillis(250)); + + verify(applicationEventHandler, atLeast(1)) + .addAndGet(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + if (committedOffsetsEnabled) { + // Verify there was an FetchCommittedOffsets event and no ResetPositions event verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); - - if (committedOffsetsEnabled) { - // Verify there was an FetchCommittedOffsets event and no ResetPositions event - verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); - verify(applicationEventHandler, never()) - .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); - } else { - // Verify there was not any FetchCommittedOffsets event but there should be a ResetPositions - verify(applicationEventHandler, never()) - .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); - verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); - } + .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + verify(applicationEventHandler, never()) + .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + } else { + // Verify there was not any FetchCommittedOffsets event but there should be a ResetPositions + verify(applicationEventHandler, never()) + .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + verify(applicationEventHandler, atLeast(1)) + .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); } } private void testRefreshCommittedOffsetsSuccess(Set partitions, Map committedOffsets) { - CompletableFuture> committedFuture = new CompletableFuture<>(); - committedFuture.complete(committedOffsets); + doReturn(committedOffsets) + .when(applicationEventHandler) + .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); + doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); + consumer.assign(partitions); - try (MockedConstruction ignored = offsetFetchEventMocker(committedFuture)) { - // Poll with 250ms timeout to give the background thread time to process the events without timing out - consumer.poll(Duration.ofMillis(250)); - verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); - verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); - verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); - } + // Poll with 250ms timeout to give the background thread time to process the events without timing out + consumer.poll(Duration.ofMillis(250)); + + verify(applicationEventHandler, atLeast(1)) + .addAndGet(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + verify(applicationEventHandler, atLeast(1)) + .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + verify(applicationEventHandler, atLeast(1)) + .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); } - private void prepAutocommitOnClose() { - Node node = testBuilder.metadata.fetch().nodes().get(0); - testBuilder.client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "group-id", node)); - if (!testBuilder.subscriptions.allConsumed().isEmpty()) { - List topicPartitions = new ArrayList<>(testBuilder.subscriptions.assignedPartitionsList()); - testBuilder.client.prepareResponse(mockAutocommitResponse( - topicPartitions, - (short) 1, - Errors.NONE).responseBody()); - } + @Test + public void testSubscribeGeneratesEvent() { + consumer = setup(); + String topic = "topic1"; + consumer.subscribe(singletonList(topic)); + assertEquals(singleton(topic), consumer.subscription()); + assertTrue(consumer.assignment().isEmpty()); + verify(applicationEventHandler).add(ArgumentMatchers.isA(SubscriptionChangeApplicationEvent.class)); } - private ClientResponse mockAutocommitResponse(final List topicPartitions, - final short apiKeyVersion, - final Errors error) { - OffsetCommitResponseData responseData = new OffsetCommitResponseData(); - List responseTopics = new ArrayList<>(); - topicPartitions.forEach(tp -> { - responseTopics.add(new OffsetCommitResponseData.OffsetCommitResponseTopic() - .setName(tp.topic()) - .setPartitions(Collections.singletonList( - new OffsetCommitResponseData.OffsetCommitResponsePartition() - .setErrorCode(error.code()) - .setPartitionIndex(tp.partition())))); - }); - responseData.setTopics(responseTopics); - OffsetCommitResponse response = mock(OffsetCommitResponse.class); - when(response.data()).thenReturn(responseData); - return new ClientResponse( - new RequestHeader(ApiKeys.OFFSET_COMMIT, apiKeyVersion, "", 1), - null, - "-1", - testBuilder.time.milliseconds(), - testBuilder.time.milliseconds(), - false, - null, - null, - new OffsetCommitResponse(responseData) + @Test + public void testUnsubscribeGeneratesUnsubscribeEvent() { + consumer = setup(); + Mockito.doAnswer(invocation -> { + CompletableApplicationEvent event = invocation.getArgument(0); + assertTrue(event instanceof UnsubscribeApplicationEvent); + event.future().complete(null); + return null; + }).when(applicationEventHandler).add(any()); + + consumer.unsubscribe(); + + assertTrue(consumer.subscription().isEmpty()); + assertTrue(consumer.assignment().isEmpty()); + } + + @Test + public void testSubscribeToEmptyListActsAsUnsubscribe() { + consumer = setup(); + + Mockito.doAnswer(invocation -> { + CompletableApplicationEvent event = invocation.getArgument(0); + assertTrue(event instanceof UnsubscribeApplicationEvent); + event.future().complete(null); + return null; + }).when(applicationEventHandler).add(any()); + + consumer.subscribe(Collections.emptyList()); + assertTrue(consumer.subscription().isEmpty()); + assertTrue(consumer.assignment().isEmpty()); + } + + @Test + public void testSubscribeToNullTopicCollection() { + consumer = setup(); + assertThrows(IllegalArgumentException.class, () -> consumer.subscribe((List) null)); + } + + @Test + public void testSubscriptionOnNullTopic() { + consumer = setup(); + assertThrows(IllegalArgumentException.class, () -> consumer.subscribe(singletonList(null))); + } + + @Test + public void testSubscriptionOnEmptyTopic() { + consumer = setup(); + String emptyTopic = " "; + assertThrows(IllegalArgumentException.class, () -> consumer.subscribe(singletonList(emptyTopic))); + } + + @Test + public void testGroupMetadataAfterCreationWithGroupIdIsNull() { + final Properties props = requiredConsumerProperties(); + final ConsumerConfig config = new ConsumerConfig(props); + consumer = setup(config); + + assertFalse(config.unused().contains(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG)); + assertFalse(config.unused().contains(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED)); + final Throwable exception = assertThrows(InvalidGroupIdException.class, consumer::groupMetadata); + assertEquals( + "To use the group management or offset commit APIs, you must " + + "provide a valid " + ConsumerConfig.GROUP_ID_CONFIG + " in the consumer configuration.", + exception.getMessage() ); } + + @Test + public void testGroupMetadataAfterCreationWithGroupIdIsNotNull() { + final String groupId = "consumerGroupA"; + final Properties props = requiredConsumerPropertiesAndGroupId("consumerGroupA"); + final ConsumerConfig config = new ConsumerConfig(props); + + consumer = setup(config); + final ConsumerGroupMetadata groupMetadata = consumer.groupMetadata(); + + assertEquals(groupId, groupMetadata.groupId()); + assertEquals(Optional.empty(), groupMetadata.groupInstanceId()); + assertEquals(JoinGroupRequest.UNKNOWN_GENERATION_ID, groupMetadata.generationId()); + assertEquals(JoinGroupRequest.UNKNOWN_MEMBER_ID, groupMetadata.memberId()); + } + + @Test + public void testGroupMetadataAfterCreationWithGroupIdIsNotNullAndGroupInstanceIdSet() { + final String groupId = "consumerGroupA"; + final String groupInstanceId = "groupInstanceId1"; + final Properties props = requiredConsumerPropertiesAndGroupId(groupId); + props.put(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG, groupInstanceId); + final ConsumerConfig config = new ConsumerConfig(props); + consumer = setup(config); + + final ConsumerGroupMetadata groupMetadata = consumer.groupMetadata(); + + assertEquals(groupId, groupMetadata.groupId()); + assertEquals(Optional.of(groupInstanceId), groupMetadata.groupInstanceId()); + assertEquals(JoinGroupRequest.UNKNOWN_GENERATION_ID, groupMetadata.generationId()); + assertEquals(JoinGroupRequest.UNKNOWN_MEMBER_ID, groupMetadata.memberId()); + } + + @Test + public void testGroupMetadataUpdateSingleCall() { + final String groupId = "consumerGroupA"; + final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId(groupId)); + consumer = setup(config); + + doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); + doReturn(mkMap()).when(applicationEventHandler).addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + + final int generation = 1; + final String memberId = "newMemberId"; + final ConsumerGroupMetadata expectedGroupMetadata = new ConsumerGroupMetadata( + groupId, + generation, + memberId, + Optional.empty() + ); + final GroupMetadataUpdateEvent groupMetadataUpdateEvent = new GroupMetadataUpdateEvent( + generation, + memberId + ); + backgroundEventQueue.add(groupMetadataUpdateEvent); + consumer.assign(singletonList(new TopicPartition("topic", 0))); + consumer.poll(Duration.ZERO); + + final ConsumerGroupMetadata actualGroupMetadata = consumer.groupMetadata(); + + assertEquals(expectedGroupMetadata, actualGroupMetadata); + + final ConsumerGroupMetadata secondActualGroupMetadataWithoutUpdate = consumer.groupMetadata(); + + assertEquals(expectedGroupMetadata, secondActualGroupMetadataWithoutUpdate); + } + + @Test + public void testBackgroundError() { + final String groupId = "consumerGroupA"; + final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId(groupId)); + consumer = setup(config); + + final KafkaException expectedException = new KafkaException("Nobody expects the Spanish Inquisition"); + final ErrorBackgroundEvent errorBackgroundEvent = new ErrorBackgroundEvent(expectedException); + backgroundEventQueue.add(errorBackgroundEvent); + consumer.assign(singletonList(new TopicPartition("topic", 0))); + + final KafkaException exception = assertThrows(KafkaException.class, () -> consumer.poll(Duration.ZERO)); + + assertEquals(expectedException.getMessage(), exception.getMessage()); + } + + @Test + public void testMultipleBackgroundErrors() { + final String groupId = "consumerGroupA"; + final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId(groupId)); + consumer = setup(config); + + final KafkaException expectedException1 = new KafkaException("Nobody expects the Spanish Inquisition"); + final ErrorBackgroundEvent errorBackgroundEvent1 = new ErrorBackgroundEvent(expectedException1); + backgroundEventQueue.add(errorBackgroundEvent1); + final KafkaException expectedException2 = new KafkaException("Spam, Spam, Spam"); + final ErrorBackgroundEvent errorBackgroundEvent2 = new ErrorBackgroundEvent(expectedException2); + backgroundEventQueue.add(errorBackgroundEvent2); + consumer.assign(singletonList(new TopicPartition("topic", 0))); + + final KafkaException exception = assertThrows(KafkaException.class, () -> consumer.poll(Duration.ZERO)); + + assertEquals(expectedException1.getMessage(), exception.getMessage()); + assertTrue(backgroundEventQueue.isEmpty()); + } + + @Test + public void testGroupRemoteAssignorUnusedIfGroupIdUndefined() { + final Properties props = requiredConsumerProperties(); + props.put(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, "someAssignor"); + final ConsumerConfig config = new ConsumerConfig(props); + consumer = setup(config); + + assertTrue(config.unused().contains(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG)); + } + + @Test + public void testGroupRemoteAssignorUnusedInGenericProtocol() { + final Properties props = requiredConsumerProperties(); + props.put(ConsumerConfig.GROUP_ID_CONFIG, "consumerGroupA"); + props.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.GENERIC.name().toLowerCase(Locale.ROOT)); + props.put(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, "someAssignor"); + final ConsumerConfig config = new ConsumerConfig(props); + consumer = setup(config); + + assertTrue(config.unused().contains(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG)); + } + + @Test + public void testGroupRemoteAssignorUsedInConsumerProtocol() { + final Properties props = requiredConsumerProperties(); + props.put(ConsumerConfig.GROUP_ID_CONFIG, "consumerGroupA"); + props.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CONSUMER.name().toLowerCase(Locale.ROOT)); + props.put(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, "someAssignor"); + final ConsumerConfig config = new ConsumerConfig(props); + consumer = setup(config); + + assertFalse(config.unused().contains(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG)); + } + + @Test + public void testGroupIdNull() { + final Properties props = requiredConsumerProperties(); + props.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, 10000); + props.put(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED, true); + final ConsumerConfig config = new ConsumerConfig(props); + + consumer = setup(config); + assertFalse(config.unused().contains(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG)); + assertFalse(config.unused().contains(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED)); + } + + @Test + public void testGroupIdNotNullAndValid() { + final Properties props = requiredConsumerProperties(); + props.put(ConsumerConfig.GROUP_ID_CONFIG, "consumerGroupA"); + props.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, 10000); + props.put(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED, true); + final ConsumerConfig config = new ConsumerConfig(props); + + consumer = setup(config); + assertTrue(config.unused().contains(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG)); + assertTrue(config.unused().contains(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED)); + } + + @Test + public void testGroupIdEmpty() { + testInvalidGroupId(""); + } + + @Test + public void testGroupIdOnlyWhitespaces() { + testInvalidGroupId(" "); + } + + private void testInvalidGroupId(final String groupId) { + final Properties props = requiredConsumerProperties(); + props.put(ConsumerConfig.GROUP_ID_CONFIG, groupId); + final ConsumerConfig config = new ConsumerConfig(props); + + final Exception exception = assertThrows( + KafkaException.class, + () -> consumer = setup(config) + ); + + assertEquals("Failed to construct kafka consumer", exception.getMessage()); + } + + private Properties requiredConsumerPropertiesAndGroupId(final String groupId) { + final Properties props = requiredConsumerProperties(); + props.put(ConsumerConfig.GROUP_ID_CONFIG, groupId); + return props; + } + + private Properties requiredConsumerProperties() { + final Properties props = new Properties(); + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9091"); + return props; + } + + @Test + public void testLongPollWaitIsLimited() { + consumer = setup(); + String topicName = "topic1"; + consumer.subscribe(singletonList(topicName)); + + assertEquals(singleton(topicName), consumer.subscription()); + assertTrue(consumer.assignment().isEmpty()); + + final int partition = 3; + final TopicPartition tp = new TopicPartition(topicName, partition); + final List> records = asList( + new ConsumerRecord<>(topicName, partition, 2, "key1", "value1"), + new ConsumerRecord<>(topicName, partition, 3, "key2", "value2") + ); + + // On the first iteration, return no data; on the second, return two records + doAnswer(invocation -> { + // Mock the subscription being assigned as the first fetch is collected + consumer.subscriptions().assignFromSubscribed(Collections.singleton(tp)); + return Fetch.empty(); + }) + .doAnswer(invocation -> Fetch.forPartition(tp, records, true)) + .when(fetchCollector).collectFetch(any(FetchBuffer.class)); + + // And then poll for up to 10000ms, which should return 2 records without timing out + ConsumerRecords returnedRecords = consumer.poll(Duration.ofMillis(10000)); + assertEquals(2, returnedRecords.count()); + + assertEquals(singleton(topicName), consumer.subscription()); + assertEquals(singleton(tp), consumer.assignment()); + } + + + private HashMap mockTopicPartitionOffset() { + final TopicPartition t0 = new TopicPartition("t0", 2); + final TopicPartition t1 = new TopicPartition("t0", 3); + HashMap topicPartitionOffsets = new HashMap<>(); + topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L)); + topicPartitionOffsets.put(t1, new OffsetAndMetadata(20L)); + return topicPartitionOffsets; + } + + private HashMap mockOffsetAndTimestamp() { + final TopicPartition t0 = new TopicPartition("t0", 2); + final TopicPartition t1 = new TopicPartition("t0", 3); + HashMap offsetAndTimestamp = new HashMap<>(); + offsetAndTimestamp.put(t0, new OffsetAndTimestamp(5L, 1L)); + offsetAndTimestamp.put(t1, new OffsetAndTimestamp(6L, 3L)); + return offsetAndTimestamp; + } + + private HashMap mockTimestampToSearch() { + final TopicPartition t0 = new TopicPartition("t0", 2); + final TopicPartition t1 = new TopicPartition("t0", 3); + HashMap timestampToSearch = new HashMap<>(); + timestampToSearch.put(t0, 1L); + timestampToSearch.put(t1, 2L); + return timestampToSearch; + } + } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java deleted file mode 100644 index dd4e4503edc35..0000000000000 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerUnitTest.java +++ /dev/null @@ -1,959 +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.clients.consumer.internals; - -import static java.util.Arrays.asList; -import static java.util.Collections.singleton; -import static java.util.Collections.singletonList; -import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED; -import static org.apache.kafka.common.utils.Utils.mkEntry; -import static org.apache.kafka.common.utils.Utils.mkMap; -import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertNull; -import static org.junit.jupiter.api.Assertions.assertSame; -import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.junit.jupiter.api.Assertions.fail; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -import java.time.Duration; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.Optional; -import java.util.Properties; -import java.util.Set; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import org.apache.kafka.clients.Metadata.LeaderAndEpoch; -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.consumer.GroupProtocol; -import org.apache.kafka.clients.consumer.OffsetAndMetadata; -import org.apache.kafka.clients.consumer.OffsetAndTimestamp; -import org.apache.kafka.clients.consumer.OffsetCommitCallback; -import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; -import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; -import org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; -import org.apache.kafka.clients.consumer.internals.events.GroupMetadataUpdateEvent; -import org.apache.kafka.clients.consumer.internals.events.ListOffsetsApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.NewTopicsMetadataUpdateRequestEvent; -import org.apache.kafka.clients.consumer.internals.events.FetchCommittedOffsetsApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.SubscriptionChangeApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.UnsubscribeApplicationEvent; -import org.apache.kafka.common.KafkaException; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.errors.GroupAuthorizationException; -import org.apache.kafka.common.errors.InvalidGroupIdException; -import org.apache.kafka.common.errors.TimeoutException; -import org.apache.kafka.common.errors.WakeupException; -import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.requests.JoinGroupRequest; -import org.apache.kafka.common.requests.ListOffsetsRequest; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.utils.MockTime; -import org.apache.kafka.common.utils.Time; -import org.apache.kafka.common.utils.Timer; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.MethodSource; -import org.mockito.ArgumentCaptor; -import org.mockito.ArgumentMatchers; -import org.mockito.Mockito; - -@SuppressWarnings("unchecked") -public class AsyncKafkaConsumerUnitTest { - - private AsyncKafkaConsumer consumer = null; - - private final Time time = new MockTime(); - private final FetchCollector fetchCollector = mock(FetchCollector.class); - private final ApplicationEventHandler applicationEventHandler = mock(ApplicationEventHandler.class); - private final ConsumerMetadata metadata = mock(ConsumerMetadata.class); - private final LinkedBlockingQueue backgroundEventQueue = new LinkedBlockingQueue<>(); - - @AfterEach - public void resetAll() { - backgroundEventQueue.clear(); - if (consumer != null) { - consumer.close(); - } - consumer = null; - Mockito.framework().clearInlineMocks(); - } - - private AsyncKafkaConsumer setup() { - final Properties props = requiredConsumerProperties(); - props.put(ConsumerConfig.GROUP_ID_CONFIG, "group-id"); - final ConsumerConfig config = new ConsumerConfig(props); - return setup(config); - } - - private AsyncKafkaConsumer setupWithEmptyGroupId() { - final Properties props = requiredConsumerPropertiesAndGroupId(""); - final ConsumerConfig config = new ConsumerConfig(props); - return setup(config); - } - - private AsyncKafkaConsumer setup(ConsumerConfig config) { - return new AsyncKafkaConsumer<>( - config, - new StringDeserializer(), - new StringDeserializer(), - time, - (a, b, c, d, e, f) -> applicationEventHandler, - (a, b, c, d, e, f, g) -> fetchCollector, - (a, b, c, d) -> metadata, - backgroundEventQueue - ); - } - - @Test - public void testGroupIdNull() { - final Properties props = requiredConsumerProperties(); - props.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, 10000); - props.put(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED, true); - final ConsumerConfig config = new ConsumerConfig(props); - - consumer = setup(config); - assertFalse(config.unused().contains(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG)); - assertFalse(config.unused().contains(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED)); - } - - @Test - public void testGroupIdNotNullAndValid() { - final Properties props = requiredConsumerProperties(); - props.put(ConsumerConfig.GROUP_ID_CONFIG, "consumerGroupA"); - props.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, 10000); - props.put(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED, true); - final ConsumerConfig config = new ConsumerConfig(props); - - consumer = setup(config); - assertTrue(config.unused().contains(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG)); - assertTrue(config.unused().contains(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED)); - } - - @Test - public void testGroupIdEmpty() { - testInvalidGroupId(""); - } - - @Test - public void testGroupIdOnlyWhitespaces() { - testInvalidGroupId(" "); - } - - private void testInvalidGroupId(final String groupId) { - final Properties props = requiredConsumerProperties(); - props.put(ConsumerConfig.GROUP_ID_CONFIG, groupId); - final ConsumerConfig config = new ConsumerConfig(props); - - final Exception exception = assertThrows( - KafkaException.class, - () -> consumer = setup(config) - ); - - assertEquals("Failed to construct kafka consumer", exception.getMessage()); - } - - private Properties requiredConsumerPropertiesAndGroupId(final String groupId) { - final Properties props = requiredConsumerProperties(); - props.put(ConsumerConfig.GROUP_ID_CONFIG, groupId); - return props; - } - - private Properties requiredConsumerProperties() { - final Properties props = new Properties(); - props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9091"); - return props; - } - - @Test - public void testInvalidGroupId() { - KafkaException e = assertThrows(KafkaException.class, this::setupWithEmptyGroupId); - assertTrue(e.getCause() instanceof InvalidGroupIdException); - } - - @Test - public void testSuccessfulStartupShutdown() { - consumer = setup(); - assertDoesNotThrow(() -> consumer.close()); - } - - @Test - public void testFailOnClosedConsumer() { - consumer.close(); - final IllegalStateException res = assertThrows(IllegalStateException.class, consumer::assignment); - assertEquals("This consumer has already been closed.", res.getMessage()); - } - - @Test - public void testAssign() { - consumer = setup(); - final TopicPartition tp = new TopicPartition("foo", 3); - consumer.assign(singleton(tp)); - assertTrue(consumer.subscription().isEmpty()); - assertTrue(consumer.assignment().contains(tp)); - verify(applicationEventHandler).add(any(AssignmentChangeApplicationEvent.class)); - verify(applicationEventHandler).add(any(NewTopicsMetadataUpdateRequestEvent.class)); - } - - @Test - public void testAssignOnNullTopicPartition() { - consumer = setup(); - assertThrows(IllegalArgumentException.class, () -> consumer.assign(null)); - } - - @Test - public void testAssignOnEmptyTopicPartition() { - consumer = setup(); - - Mockito.doAnswer(invocation -> { - CompletableApplicationEvent event = invocation.getArgument(0); - assertTrue(event instanceof UnsubscribeApplicationEvent); - event.future().complete(null); - return null; - }).when(applicationEventHandler).add(any()); - - consumer.assign(Collections.emptyList()); - assertTrue(consumer.subscription().isEmpty()); - assertTrue(consumer.assignment().isEmpty()); - } - - @Test - public void testAssignOnNullTopicInPartition() { - consumer = setup(); - assertThrows(IllegalArgumentException.class, () -> consumer.assign(singleton(new TopicPartition(null, 0)))); - } - - @Test - public void testAssignOnEmptyTopicInPartition() { - consumer = setup(); - assertThrows(IllegalArgumentException.class, () -> consumer.assign(singleton(new TopicPartition(" ", 0)))); - } - - @Test - public void testFencedInstanceException() { - consumer = setup(); - Mockito.doAnswer(invocation -> { - CompletableApplicationEvent event = invocation.getArgument(0); - assertTrue(event instanceof CommitApplicationEvent); - event.future().completeExceptionally(Errors.FENCED_INSTANCE_ID.exception()); - return null; - }).when(applicationEventHandler).add(any()); - - assertDoesNotThrow(() -> consumer.commitAsync()); - } - - @Test - public void testCommitSyncLeaderEpochUpdate() { - consumer = setup(); - final TopicPartition t0 = new TopicPartition("t0", 2); - final TopicPartition t1 = new TopicPartition("t0", 3); - HashMap topicPartitionOffsets = new HashMap<>(); - topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L, Optional.of(2), "")); - topicPartitionOffsets.put(t1, new OffsetAndMetadata(20L, Optional.of(1), "")); - Mockito.doAnswer(invocation -> { - CompletableApplicationEvent event = invocation.getArgument(0); - assertTrue(event instanceof CommitApplicationEvent); - event.future().complete(null); - return null; - }).when(applicationEventHandler).add(any()); - - assertDoesNotThrow(() -> consumer.commitSync(topicPartitionOffsets)); - - verify(metadata).updateLastSeenEpochIfNewer(t0, 2); - verify(metadata).updateLastSeenEpochIfNewer(t1, 1); - } - - @Test - public void testCommitAsyncWithNullCallback() { - consumer = setup(); - final TopicPartition t0 = new TopicPartition("t0", 2); - final TopicPartition t1 = new TopicPartition("t0", 3); - HashMap offsets = new HashMap<>(); - offsets.put(t0, new OffsetAndMetadata(10L)); - offsets.put(t1, new OffsetAndMetadata(20L)); - - consumer.commitAsync(offsets, null); - - final ArgumentCaptor commitEventCaptor = ArgumentCaptor.forClass(CommitApplicationEvent.class); - verify(applicationEventHandler).add(commitEventCaptor.capture()); - final CommitApplicationEvent commitEvent = commitEventCaptor.getValue(); - assertEquals(offsets, commitEvent.offsets()); - assertDoesNotThrow(() -> commitEvent.future().complete(null)); - assertDoesNotThrow(() -> consumer.commitAsync(offsets, null)); - } - - @Test - public void testCommitAsyncLeaderEpochUpdate() { - consumer = setup(); - OffsetCommitCallback callback = mock(OffsetCommitCallback.class); - final TopicPartition t0 = new TopicPartition("t0", 2); - final TopicPartition t1 = new TopicPartition("t0", 3); - HashMap topicPartitionOffsets = new HashMap<>(); - topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L, Optional.of(2), "")); - topicPartitionOffsets.put(t1, new OffsetAndMetadata(20L, Optional.of(1), "")); - consumer.assign(Arrays.asList(t0, t1)); - - consumer.commitAsync(topicPartitionOffsets, callback); - - verify(metadata).updateLastSeenEpochIfNewer(t0, 2); - verify(metadata).updateLastSeenEpochIfNewer(t1, 1); - } - - @ParameterizedTest - @MethodSource("commitExceptionSupplier") - public void testCommitAsyncUserSuppliedCallback(Exception exception) { - consumer = setup(); - - Map offsets = new HashMap<>(); - offsets.put(new TopicPartition("my-topic", 1), new OffsetAndMetadata(200L)); - - Mockito.doAnswer(invocation -> { - CompletableApplicationEvent event = invocation.getArgument(0); - assertTrue(event instanceof CommitApplicationEvent); - if (exception == null) { - event.future().complete(null); - } else { - event.future().completeExceptionally(exception); - } - event.future().complete(null); - return null; - }).when(applicationEventHandler).add(any()); - - MockCommitCallback callback = new MockCommitCallback(); - assertDoesNotThrow(() -> consumer.commitAsync(offsets, callback)); - consumer.maybeInvokeCommitCallbacks(); - - if (exception == null) { - assertNull(callback.exception); - } else { - assertSame(exception.getClass(), callback.exception.getClass()); - } - } - - private static Stream commitExceptionSupplier() { - return Stream.of( - null, // For the successful completion scenario - new KafkaException("Test exception"), - new GroupAuthorizationException("Group authorization exception")); - } - - @Test - public void testEnsureCallbackExecutedByApplicationThread() { - consumer = setup(); - final String currentThread = Thread.currentThread().getName(); - MockCommitCallback callback = new MockCommitCallback(); - - Mockito.doAnswer(invocation -> { - CompletableApplicationEvent event = invocation.getArgument(0); - assertTrue(event instanceof CommitApplicationEvent); - event.future().complete(null); - return null; - }).when(applicationEventHandler).add(any()); - - assertDoesNotThrow(() -> consumer.commitAsync(new HashMap<>(), callback)); - assertEquals(1, consumer.callbacks()); - consumer.maybeInvokeCommitCallbacks(); - assertEquals(currentThread, callback.completionThread); - } - - private static class MockCommitCallback implements OffsetCommitCallback { - public int invoked = 0; - public Exception exception = null; - public String completionThread; - - @Override - public void onComplete(Map offsets, Exception exception) { - invoked++; - this.completionThread = Thread.currentThread().getName(); - this.exception = exception; - } - } - - @Test - public void testCommitted() { - consumer = setup(); - Map topicPartitionOffsets = mockTopicPartitionOffset(); - - when(applicationEventHandler.addAndGet(any(), any())).thenAnswer(invocation -> { - CompletableApplicationEvent event = invocation.getArgument(0); - assertTrue(event instanceof FetchCommittedOffsetsApplicationEvent); - return topicPartitionOffsets; - }); - - assertDoesNotThrow(() -> consumer.committed(topicPartitionOffsets.keySet(), Duration.ofMillis(1000))); - } - - @Test - public void testCommittedExceptionThrown() { - consumer = setup(); - Map topicPartitionOffsets = mockTopicPartitionOffset(); - when(applicationEventHandler.addAndGet(any(), any())).thenAnswer(invocation -> { - CompletableApplicationEvent event = invocation.getArgument(0); - assertTrue(event instanceof FetchCommittedOffsetsApplicationEvent); - throw new KafkaException("Test exception"); - }); - - assertThrows(KafkaException.class, () -> consumer.committed(topicPartitionOffsets.keySet(), Duration.ofMillis(1000))); - } - - @Test - public void testCommittedLeaderEpochUpdate() { - consumer = setup(); - final TopicPartition t0 = new TopicPartition("t0", 2); - final TopicPartition t1 = new TopicPartition("t0", 3); - final TopicPartition t2 = new TopicPartition("t0", 4); - HashMap topicPartitionOffsets = new HashMap<>(); - topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L, Optional.of(2), "")); - topicPartitionOffsets.put(t1, null); - topicPartitionOffsets.put(t2, new OffsetAndMetadata(20L, Optional.of(3), "")); - when(applicationEventHandler.addAndGet(any(), any())).thenAnswer(invocation -> { - CompletableApplicationEvent event = invocation.getArgument(0); - assertTrue(event instanceof FetchCommittedOffsetsApplicationEvent); - return topicPartitionOffsets; - }); - - assertDoesNotThrow(() -> consumer.committed(topicPartitionOffsets.keySet(), Duration.ofMillis(1000))); - - verify(metadata).updateLastSeenEpochIfNewer(t0, 2); - verify(metadata).updateLastSeenEpochIfNewer(t2, 3); - } - - @Test - @SuppressWarnings("deprecation") - public void testPollLongThrowsException() { - consumer = setup(); - Exception e = assertThrows(UnsupportedOperationException.class, () -> consumer.poll(0L)); - assertEquals("Consumer.poll(long) is not supported when \"group.protocol\" is \"consumer\". " + - "This method is deprecated and will be removed in the next major release.", e.getMessage()); - } - - @Test - public void testBeginningOffsetsFailsIfNullPartitions() { - consumer = setup(); - assertThrows(NullPointerException.class, () -> consumer.beginningOffsets(null, - Duration.ofMillis(1))); - } - - @Test - public void testBeginningOffsets() { - consumer = setup(); - Map expectedOffsetsAndTimestamp = - mockOffsetAndTimestamp(); - Set partitions = expectedOffsetsAndTimestamp.keySet(); - doReturn(expectedOffsetsAndTimestamp).when(applicationEventHandler).addAndGet(any(), any()); - Map result = - assertDoesNotThrow(() -> consumer.beginningOffsets(partitions, - Duration.ofMillis(1))); - Map expectedOffsets = expectedOffsetsAndTimestamp.entrySet().stream() - .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().offset())); - assertEquals(expectedOffsets, result); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), - ArgumentMatchers.isA(Timer.class)); - } - - @Test - public void testBeginningOffsetsThrowsKafkaExceptionForUnderlyingExecutionFailure() { - consumer = setup(); - Set partitions = mockTopicPartitionOffset().keySet(); - Throwable eventProcessingFailure = new KafkaException("Unexpected failure " + - "processing List Offsets event"); - doThrow(eventProcessingFailure).when(applicationEventHandler).addAndGet(any(), any()); - Throwable consumerError = assertThrows(KafkaException.class, - () -> consumer.beginningOffsets(partitions, - Duration.ofMillis(1))); - assertEquals(eventProcessingFailure, consumerError); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); - } - - @Test - public void testBeginningOffsetsTimeoutOnEventProcessingTimeout() { - consumer = setup(); - doThrow(new TimeoutException()).when(applicationEventHandler).addAndGet(any(), any()); - assertThrows(TimeoutException.class, - () -> consumer.beginningOffsets( - Collections.singletonList(new TopicPartition("t1", 0)), - Duration.ofMillis(1))); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), - ArgumentMatchers.isA(Timer.class)); - } - - @Test - public void testOffsetsForTimesOnNullPartitions() { - consumer = setup(); - assertThrows(NullPointerException.class, () -> consumer.offsetsForTimes(null, - Duration.ofMillis(1))); - } - - @Test - public void testOffsetsForTimesFailsOnNegativeTargetTimes() { - consumer = setup(); - assertThrows(IllegalArgumentException.class, - () -> consumer.offsetsForTimes(Collections.singletonMap(new TopicPartition( - "topic1", 1), ListOffsetsRequest.EARLIEST_TIMESTAMP), - Duration.ofMillis(1))); - - assertThrows(IllegalArgumentException.class, - () -> consumer.offsetsForTimes(Collections.singletonMap(new TopicPartition( - "topic1", 1), ListOffsetsRequest.LATEST_TIMESTAMP), - Duration.ofMillis(1))); - - assertThrows(IllegalArgumentException.class, - () -> consumer.offsetsForTimes(Collections.singletonMap(new TopicPartition( - "topic1", 1), ListOffsetsRequest.MAX_TIMESTAMP), - Duration.ofMillis(1))); - } - - @Test - public void testOffsetsForTimes() { - consumer = setup(); - Map expectedResult = mockOffsetAndTimestamp(); - Map timestampToSearch = mockTimestampToSearch(); - - doReturn(expectedResult).when(applicationEventHandler).addAndGet(any(), any()); - Map result = - assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, Duration.ofMillis(1))); - assertEquals(expectedResult, result); - verify(applicationEventHandler).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() { - consumer = setup(); - TopicPartition tp = new TopicPartition("topic1", 0); - Map expectedResult = - Collections.singletonMap(tp, null); - Map timestampToSearch = Collections.singletonMap(tp, 5L); - - Map result = - assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, - Duration.ofMillis(0))); - assertEquals(expectedResult, result); - verify(applicationEventHandler, never()).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), - ArgumentMatchers.isA(Timer.class)); - } - - @Test - public void testGroupMetadataAfterCreationWithGroupIdIsNull() { - final Properties props = requiredConsumerProperties(); - final ConsumerConfig config = new ConsumerConfig(props); - consumer = setup(config); - - assertFalse(config.unused().contains(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG)); - assertFalse(config.unused().contains(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED)); - final Throwable exception = assertThrows(InvalidGroupIdException.class, consumer::groupMetadata); - assertEquals( - "To use the group management or offset commit APIs, you must " + - "provide a valid " + ConsumerConfig.GROUP_ID_CONFIG + " in the consumer configuration.", - exception.getMessage() - ); - } - - @Test - public void testGroupMetadataAfterCreationWithGroupIdIsNotNull() { - final String groupId = "consumerGroupA"; - final Properties props = requiredConsumerPropertiesAndGroupId("consumerGroupA"); - final ConsumerConfig config = new ConsumerConfig(props); - - consumer = setup(config); - final ConsumerGroupMetadata groupMetadata = consumer.groupMetadata(); - - assertEquals(groupId, groupMetadata.groupId()); - assertEquals(Optional.empty(), groupMetadata.groupInstanceId()); - assertEquals(JoinGroupRequest.UNKNOWN_GENERATION_ID, groupMetadata.generationId()); - assertEquals(JoinGroupRequest.UNKNOWN_MEMBER_ID, groupMetadata.memberId()); - } - - @Test - public void testGroupMetadataAfterCreationWithGroupIdIsNotNullAndGroupInstanceIdSet() { - final String groupId = "consumerGroupA"; - final String groupInstanceId = "groupInstanceId1"; - final Properties props = requiredConsumerPropertiesAndGroupId(groupId); - props.put(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG, groupInstanceId); - final ConsumerConfig config = new ConsumerConfig(props); - consumer = setup(config); - - final ConsumerGroupMetadata groupMetadata = consumer.groupMetadata(); - - assertEquals(groupId, groupMetadata.groupId()); - assertEquals(Optional.of(groupInstanceId), groupMetadata.groupInstanceId()); - assertEquals(JoinGroupRequest.UNKNOWN_GENERATION_ID, groupMetadata.generationId()); - assertEquals(JoinGroupRequest.UNKNOWN_MEMBER_ID, groupMetadata.memberId()); - } - - @Test - public void testGroupMetadataUpdateSingleCall() { - final String groupId = "consumerGroupA"; - final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId(groupId)); - consumer = setup(config); - - doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); - doReturn(mkMap()).when(applicationEventHandler).addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); - - final int generation = 1; - final String memberId = "newMemberId"; - final ConsumerGroupMetadata expectedGroupMetadata = new ConsumerGroupMetadata( - groupId, - generation, - memberId, - Optional.empty() - ); - final GroupMetadataUpdateEvent groupMetadataUpdateEvent = new GroupMetadataUpdateEvent( - generation, - memberId - ); - backgroundEventQueue.add(groupMetadataUpdateEvent); - consumer.assign(singletonList(new TopicPartition("topic", 0))); - consumer.poll(Duration.ZERO); - - final ConsumerGroupMetadata actualGroupMetadata = consumer.groupMetadata(); - - assertEquals(expectedGroupMetadata, actualGroupMetadata); - - final ConsumerGroupMetadata secondActualGroupMetadataWithoutUpdate = consumer.groupMetadata(); - - assertEquals(expectedGroupMetadata, secondActualGroupMetadataWithoutUpdate); - } - - @Test - public void testBackgroundError() { - final String groupId = "consumerGroupA"; - final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId(groupId)); - consumer = setup(config); - - final KafkaException expectedException = new KafkaException("Nobody expects the Spanish Inquisition"); - final ErrorBackgroundEvent errorBackgroundEvent = new ErrorBackgroundEvent(expectedException); - backgroundEventQueue.add(errorBackgroundEvent); - consumer.assign(singletonList(new TopicPartition("topic", 0))); - - final KafkaException exception = assertThrows(KafkaException.class, () -> consumer.poll(Duration.ZERO)); - - assertEquals(expectedException.getMessage(), exception.getMessage()); - } - - @Test - public void testMultipleBackgroundErrors() { - final String groupId = "consumerGroupA"; - final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId(groupId)); - consumer = setup(config); - - final KafkaException expectedException1 = new KafkaException("Nobody expects the Spanish Inquisition"); - final ErrorBackgroundEvent errorBackgroundEvent1 = new ErrorBackgroundEvent(expectedException1); - backgroundEventQueue.add(errorBackgroundEvent1); - final KafkaException expectedException2 = new KafkaException("Spam, Spam, Spam"); - final ErrorBackgroundEvent errorBackgroundEvent2 = new ErrorBackgroundEvent(expectedException2); - backgroundEventQueue.add(errorBackgroundEvent2); - consumer.assign(singletonList(new TopicPartition("topic", 0))); - - final KafkaException exception = assertThrows(KafkaException.class, () -> consumer.poll(Duration.ZERO)); - - assertEquals(expectedException1.getMessage(), exception.getMessage()); - assertTrue(backgroundEventQueue.isEmpty()); - } - - @Test - public void testGroupRemoteAssignorUnusedIfGroupIdUndefined() { - final Properties props = requiredConsumerProperties(); - props.put(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, "someAssignor"); - final ConsumerConfig config = new ConsumerConfig(props); - consumer = setup(config); - - assertTrue(config.unused().contains(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG)); - } - - @Test - public void testGroupRemoteAssignorUnusedInGenericProtocol() { - final Properties props = requiredConsumerProperties(); - props.put(ConsumerConfig.GROUP_ID_CONFIG, "consumerGroupA"); - props.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.GENERIC.name().toLowerCase(Locale.ROOT)); - props.put(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, "someAssignor"); - final ConsumerConfig config = new ConsumerConfig(props); - consumer = setup(config); - - assertTrue(config.unused().contains(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG)); - } - - @Test - public void testGroupRemoteAssignorUsedInConsumerProtocol() { - final Properties props = requiredConsumerProperties(); - props.put(ConsumerConfig.GROUP_ID_CONFIG, "consumerGroupA"); - props.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CONSUMER.name().toLowerCase(Locale.ROOT)); - props.put(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, "someAssignor"); - final ConsumerConfig config = new ConsumerConfig(props); - consumer = setup(config); - - assertFalse(config.unused().contains(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG)); - } - - @Test - public void testLongPollWaitIsLimited() { - consumer = setup(); - String topicName = "topic1"; - consumer.subscribe(singletonList(topicName)); - - assertEquals(singleton(topicName), consumer.subscription()); - assertTrue(consumer.assignment().isEmpty()); - - final int partition = 3; - final TopicPartition tp = new TopicPartition(topicName, partition); - final List> records = asList( - new ConsumerRecord<>(topicName, partition, 2, "key1", "value1"), - new ConsumerRecord<>(topicName, partition, 3, "key2", "value2") - ); - - // On the first iteration, return no data; on the second, return two records - doAnswer(invocation -> { - // Mock the subscription being assigned as the first fetch is collected - consumer.subscriptions().assignFromSubscribed(Collections.singleton(tp)); - return Fetch.empty(); - }) - .doAnswer(invocation -> Fetch.forPartition(tp, records, true)) - .when(fetchCollector).collectFetch(any(FetchBuffer.class)); - - // And then poll for up to 10000ms, which should return 2 records without timing out - ConsumerRecords returnedRecords = consumer.poll(Duration.ofMillis(10000)); - assertEquals(2, returnedRecords.count()); - - assertEquals(singleton(topicName), consumer.subscription()); - assertEquals(singleton(tp), consumer.assignment()); - } - - @Test - public void testSubscribeGeneratesEvent() { - consumer = setup(); - String topic = "topic1"; - consumer.subscribe(singletonList(topic)); - assertEquals(singleton(topic), consumer.subscription()); - assertTrue(consumer.assignment().isEmpty()); - verify(applicationEventHandler).add(ArgumentMatchers.isA(SubscriptionChangeApplicationEvent.class)); - } - - @Test - public void testUnsubscribeGeneratesUnsubscribeEvent() { - consumer = setup(); - Mockito.doAnswer(invocation -> { - CompletableApplicationEvent event = invocation.getArgument(0); - assertTrue(event instanceof UnsubscribeApplicationEvent); - event.future().complete(null); - return null; - }).when(applicationEventHandler).add(any()); - - consumer.unsubscribe(); - - assertTrue(consumer.subscription().isEmpty()); - assertTrue(consumer.assignment().isEmpty()); - } - - @Test - public void testSubscribeToEmptyListActsAsUnsubscribe() { - consumer = setup(); - - Mockito.doAnswer(invocation -> { - CompletableApplicationEvent event = invocation.getArgument(0); - assertTrue(event instanceof UnsubscribeApplicationEvent); - event.future().complete(null); - return null; - }).when(applicationEventHandler).add(any()); - - consumer.subscribe(Collections.emptyList()); - assertTrue(consumer.subscription().isEmpty()); - assertTrue(consumer.assignment().isEmpty()); - } - - @Test - public void testSubscribeToNullTopicCollection() { - consumer = setup(); - assertThrows(IllegalArgumentException.class, () -> consumer.subscribe((List) null)); - } - - @Test - public void testSubscriptionOnNullTopic() { - consumer = setup(); - assertThrows(IllegalArgumentException.class, () -> consumer.subscribe(singletonList(null))); - } - - @Test - public void testSubscriptionOnEmptyTopic() { - consumer = setup(); - String emptyTopic = " "; - assertThrows(IllegalArgumentException.class, () -> consumer.subscribe(singletonList(emptyTopic))); - } - - - @Test - public void testWakeupBeforeCallingPoll() { - consumer = setup(); - final String topicName = "foo"; - final int partition = 3; - final TopicPartition tp = new TopicPartition(topicName, partition); - doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); - Map offsets = mkMap(mkEntry(tp, new OffsetAndMetadata(1))); - doReturn(offsets).when(applicationEventHandler).addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); - doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); - consumer.assign(singleton(tp)); - - consumer.wakeup(); - - assertThrows(WakeupException.class, () -> consumer.poll(Duration.ZERO)); - assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); - } - - @Test - public void testWakeupAfterEmptyFetch() { - consumer = setup(); - final String topicName = "foo"; - final int partition = 3; - final TopicPartition tp = new TopicPartition(topicName, partition); - doAnswer(invocation -> { - consumer.wakeup(); - return Fetch.empty(); - }).when(fetchCollector).collectFetch(any(FetchBuffer.class)); - Map offsets = mkMap(mkEntry(tp, new OffsetAndMetadata(1))); - doReturn(offsets).when(applicationEventHandler).addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); - doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); - consumer.assign(singleton(tp)); - - assertThrows(WakeupException.class, () -> consumer.poll(Duration.ofMinutes(1))); - assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); - } - - @Test - public void testWakeupAfterNonEmptyFetch() { - consumer = setup(); - final String topicName = "foo"; - final int partition = 3; - final TopicPartition tp = new TopicPartition(topicName, partition); - final List> records = asList( - new ConsumerRecord<>(topicName, partition, 2, "key1", "value1"), - new ConsumerRecord<>(topicName, partition, 3, "key2", "value2") - ); - doAnswer(invocation -> { - consumer.wakeup(); - return Fetch.forPartition(tp, records, true); - }).when(fetchCollector).collectFetch(Mockito.any(FetchBuffer.class)); - Map offsets = mkMap(mkEntry(tp, new OffsetAndMetadata(1))); - doReturn(offsets).when(applicationEventHandler).addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); - doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); - consumer.assign(singleton(tp)); - - // since wakeup() is called when the non-empty fetch is returned the wakeup should be ignored - assertDoesNotThrow(() -> consumer.poll(Duration.ofMinutes(1))); - // the previously ignored wake-up should not be ignored in the next call - assertThrows(WakeupException.class, () -> consumer.poll(Duration.ZERO)); - } - - @Test - public void testClearWakeupTriggerAfterPoll() { - consumer = setup(); - final String topicName = "foo"; - final int partition = 3; - final TopicPartition tp = new TopicPartition(topicName, partition); - final List> records = asList( - new ConsumerRecord<>(topicName, partition, 2, "key1", "value1"), - new ConsumerRecord<>(topicName, partition, 3, "key2", "value2") - ); - doReturn(Fetch.forPartition(tp, records, true)) - .when(fetchCollector).collectFetch(any(FetchBuffer.class)); - Map offsets = mkMap(mkEntry(tp, new OffsetAndMetadata(1))); - doReturn(offsets).when(applicationEventHandler).addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); - doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); - consumer.assign(singleton(tp)); - - consumer.poll(Duration.ZERO); - - assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); - } - - @Test - public void testWakeupCommitted() { - consumer = setup(); - final HashMap offsets = mockTopicPartitionOffset(); - doAnswer(invocation -> { - CompletableApplicationEvent event = invocation.getArgument(0); - Timer timer = invocation.getArgument(1); - assertTrue(event instanceof FetchCommittedOffsetsApplicationEvent); - assertTrue(event.future().isCompletedExceptionally()); - return ConsumerUtils.getResult(event.future(), timer); - }) - .when(applicationEventHandler) - .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); - - consumer.wakeup(); - assertThrows(WakeupException.class, () -> consumer.committed(offsets.keySet())); - assertNull(consumer.wakeupTrigger().getPendingTask()); - } - - private HashMap mockTopicPartitionOffset() { - final TopicPartition t0 = new TopicPartition("t0", 2); - final TopicPartition t1 = new TopicPartition("t0", 3); - HashMap topicPartitionOffsets = new HashMap<>(); - topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L)); - topicPartitionOffsets.put(t1, new OffsetAndMetadata(20L)); - return topicPartitionOffsets; - } - - private HashMap mockOffsetAndTimestamp() { - final TopicPartition t0 = new TopicPartition("t0", 2); - final TopicPartition t1 = new TopicPartition("t0", 3); - HashMap offsetAndTimestamp = new HashMap<>(); - offsetAndTimestamp.put(t0, new OffsetAndTimestamp(5L, 1L)); - offsetAndTimestamp.put(t1, new OffsetAndTimestamp(6L, 3L)); - return offsetAndTimestamp; - } - - private HashMap mockTimestampToSearch() { - final TopicPartition t0 = new TopicPartition("t0", 2); - final TopicPartition t1 = new TopicPartition("t0", 3); - HashMap timestampToSearch = new HashMap<>(); - timestampToSearch.put(t0, 1L); - timestampToSearch.put(t1, 2L); - return timestampToSearch; - } - -} - From 60037dd2c9da41b65926f4ed8ee53b4bb43b9e95 Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Mon, 11 Dec 2023 12:43:20 +0100 Subject: [PATCH 15/21] updates --- .../internals/AsyncKafkaConsumerTest.java | 397 ++++++++++-------- 1 file changed, 220 insertions(+), 177 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index 955fd126f447f..13bde3da17868 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -61,6 +61,7 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.clients.consumer.OffsetCommitCallback; +import org.apache.kafka.clients.consumer.RetriableCommitFailedException; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; @@ -79,6 +80,8 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.GroupAuthorizationException; import org.apache.kafka.common.errors.InvalidGroupIdException; +import org.apache.kafka.common.errors.NetworkException; +import org.apache.kafka.common.errors.RetriableException; import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.protocol.Errors; @@ -90,6 +93,7 @@ import org.apache.kafka.common.utils.Timer; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.function.Executable; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; import org.mockito.ArgumentCaptor; @@ -162,6 +166,14 @@ public void testInvalidGroupId() { assertTrue(e.getCause() instanceof InvalidGroupIdException); } + @Test + public void testFailOnClosedConsumer() { + consumer = setup(); + consumer.close(); + final IllegalStateException res = assertThrows(IllegalStateException.class, consumer::assignment); + assertEquals("This consumer has already been closed.", res.getMessage()); + } + @Test public void testCommitAsyncWithNullCallback() { consumer = setup(); @@ -181,13 +193,6 @@ public void testCommitAsyncWithNullCallback() { assertDoesNotThrow(() -> consumer.commitAsync(offsets, null)); } - private static Stream commitExceptionSupplier() { - return Stream.of( - null, // For the successful completion scenario - new KafkaException("Test exception"), - new GroupAuthorizationException("Group authorization exception")); - } - @ParameterizedTest @MethodSource("commitExceptionSupplier") public void testCommitAsyncUserSuppliedCallback(Exception exception) { @@ -195,18 +200,11 @@ public void testCommitAsyncUserSuppliedCallback(Exception exception) { Map offsets = new HashMap<>(); offsets.put(new TopicPartition("my-topic", 1), new OffsetAndMetadata(200L)); - - Mockito.doAnswer(invocation -> { - CompletableApplicationEvent event = invocation.getArgument(0); - assertTrue(event instanceof CommitApplicationEvent); - if (exception == null) { - event.future().complete(null); - } else { - event.future().completeExceptionally(exception); - } - event.future().complete(null); - return null; - }).when(applicationEventHandler).add(any()); + if (exception == null) { + mockCommitApplicationEventCompleted(); + } else { + mockCommitApplicationEventException(exception); + } MockCommitCallback callback = new MockCommitCallback(); assertDoesNotThrow(() -> consumer.commitAsync(offsets, callback)); @@ -219,15 +217,17 @@ public void testCommitAsyncUserSuppliedCallback(Exception exception) { } } + private static Stream commitExceptionSupplier() { + return Stream.of( + null, // For the successful completion scenario + new KafkaException("Test exception"), + new GroupAuthorizationException("Group authorization exception")); + } + @Test public void testFencedInstanceException() { consumer = setup(); - Mockito.doAnswer(invocation -> { - CompletableApplicationEvent event = invocation.getArgument(0); - assertTrue(event instanceof CommitApplicationEvent); - event.future().completeExceptionally(Errors.FENCED_INSTANCE_ID.exception()); - return null; - }).when(applicationEventHandler).add(any()); + mockCommitApplicationEventException(Errors.FENCED_INSTANCE_ID.exception()); assertDoesNotThrow(() -> consumer.commitAsync()); } @@ -236,27 +236,10 @@ public void testFencedInstanceException() { public void testCommitted() { consumer = setup(); Map topicPartitionOffsets = mockTopicPartitionOffset(); - - when(applicationEventHandler.addAndGet(any(), any())).thenAnswer(invocation -> { - CompletableApplicationEvent event = invocation.getArgument(0); - assertTrue(event instanceof FetchCommittedOffsetsApplicationEvent); - return topicPartitionOffsets; - }); + mockFetchedCommittedOffsetApplicationEventCompleted(topicPartitionOffsets); assertDoesNotThrow(() -> consumer.committed(topicPartitionOffsets.keySet(), Duration.ofMillis(1000))); - } - - @Test - public void testCommittedExceptionThrown() { - consumer = setup(); - Map topicPartitionOffsets = mockTopicPartitionOffset(); - when(applicationEventHandler.addAndGet(any(), any())).thenAnswer(invocation -> { - CompletableApplicationEvent event = invocation.getArgument(0); - assertTrue(event instanceof FetchCommittedOffsetsApplicationEvent); - throw new KafkaException("Test exception"); - }); - - assertThrows(KafkaException.class, () -> consumer.committed(topicPartitionOffsets.keySet(), Duration.ofMillis(1000))); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), any()); } @Test @@ -269,18 +252,27 @@ public void testCommittedLeaderEpochUpdate() { topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L, Optional.of(2), "")); topicPartitionOffsets.put(t1, null); topicPartitionOffsets.put(t2, new OffsetAndMetadata(20L, Optional.of(3), "")); - when(applicationEventHandler.addAndGet(any(), any())).thenAnswer(invocation -> { - CompletableApplicationEvent event = invocation.getArgument(0); - assertTrue(event instanceof FetchCommittedOffsetsApplicationEvent); - return topicPartitionOffsets; - }); + mockFetchedCommittedOffsetApplicationEventCompleted(topicPartitionOffsets); assertDoesNotThrow(() -> consumer.committed(topicPartitionOffsets.keySet(), Duration.ofMillis(1000))); verify(metadata).updateLastSeenEpochIfNewer(t0, 2); verify(metadata).updateLastSeenEpochIfNewer(t2, 3); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), any()); } + @Test + public void testCommittedExceptionThrown() { + consumer = setup(); + Map offsets = mockTopicPartitionOffset(); + when(applicationEventHandler.addAndGet(any(), any())).thenAnswer(invocation -> { + CompletableApplicationEvent event = invocation.getArgument(0); + assertTrue(event instanceof FetchCommittedOffsetsApplicationEvent); + throw new KafkaException("Test exception"); + }); + + assertThrows(KafkaException.class, () -> consumer.committed(offsets.keySet(), Duration.ofMillis(1000))); + } @Test public void testWakeupBeforeCallingPoll() { @@ -290,7 +282,7 @@ public void testWakeupBeforeCallingPoll() { final TopicPartition tp = new TopicPartition(topicName, partition); doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); Map offsets = mkMap(mkEntry(tp, new OffsetAndMetadata(1))); - doReturn(offsets).when(applicationEventHandler).addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + mockFetchedCommittedOffsetApplicationEventCompleted(offsets); doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); consumer.assign(singleton(tp)); @@ -311,7 +303,7 @@ public void testWakeupAfterEmptyFetch() { return Fetch.empty(); }).when(fetchCollector).collectFetch(any(FetchBuffer.class)); Map offsets = mkMap(mkEntry(tp, new OffsetAndMetadata(1))); - doReturn(offsets).when(applicationEventHandler).addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + mockFetchedCommittedOffsetApplicationEventCompleted(offsets); doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); consumer.assign(singleton(tp)); @@ -334,7 +326,7 @@ public void testWakeupAfterNonEmptyFetch() { return Fetch.forPartition(tp, records, true); }).when(fetchCollector).collectFetch(Mockito.any(FetchBuffer.class)); Map offsets = mkMap(mkEntry(tp, new OffsetAndMetadata(1))); - doReturn(offsets).when(applicationEventHandler).addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + mockFetchedCommittedOffsetApplicationEventCompleted(offsets); doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); consumer.assign(singleton(tp)); @@ -357,7 +349,7 @@ public void testClearWakeupTriggerAfterPoll() { doReturn(Fetch.forPartition(tp, records, true)) .when(fetchCollector).collectFetch(any(FetchBuffer.class)); Map offsets = mkMap(mkEntry(tp, new OffsetAndMetadata(1))); - doReturn(offsets).when(applicationEventHandler).addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + mockFetchedCommittedOffsetApplicationEventCompleted(offsets); doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); consumer.assign(singleton(tp)); @@ -371,13 +363,7 @@ public void testEnsureCallbackExecutedByApplicationThread() { consumer = setup(); final String currentThread = Thread.currentThread().getName(); MockCommitCallback callback = new MockCommitCallback(); - - Mockito.doAnswer(invocation -> { - CompletableApplicationEvent event = invocation.getArgument(0); - assertTrue(event instanceof CommitApplicationEvent); - event.future().complete(null); - return null; - }).when(applicationEventHandler).add(any()); + mockCommitApplicationEventCompleted(); assertDoesNotThrow(() -> consumer.commitAsync(new HashMap<>(), callback)); assertEquals(1, consumer.callbacks()); @@ -385,6 +371,22 @@ public void testEnsureCallbackExecutedByApplicationThread() { assertEquals(currentThread, callback.completionThread); } + @Test + public void testEnsureCommitSyncExecutedCommitAsyncCallbacks() { + consumer = setup(); + final HashMap offsets = mockTopicPartitionOffset(); + MockCommitCallback callback = new MockCommitCallback(); + assertDoesNotThrow(() -> consumer.commitAsync(offsets, callback)); + + final ArgumentCaptor commitEventCaptor = ArgumentCaptor.forClass(CommitApplicationEvent.class); + verify(applicationEventHandler).add(commitEventCaptor.capture()); + final CommitApplicationEvent commitEvent = commitEventCaptor.getValue(); + commitEvent.future().completeExceptionally(new NetworkException("Test exception")); + + assertMockCommitCallbackInvoked(() -> consumer.commitSync(), + callback, + Errors.NETWORK_EXCEPTION); + } @Test @SuppressWarnings("deprecation") @@ -392,15 +394,7 @@ public void testPollLongThrowsException() { consumer = setup(); Exception e = assertThrows(UnsupportedOperationException.class, () -> consumer.poll(0L)); assertEquals("Consumer.poll(long) is not supported when \"group.protocol\" is \"consumer\". " + - "This method is feprecated and will be removed in the next major release.", e.getMessage()); - } - - @Test - public void testFailOnClosedConsumer() { - consumer = setup(); - consumer.close(); - final IllegalStateException res = assertThrows(IllegalStateException.class, consumer::assignment); - assertEquals("This consumer has already been closed.", res.getMessage()); + "This method is deprecated and will be removed in the next major release.", e.getMessage()); } @Test @@ -411,34 +405,71 @@ public void testCommitSyncLeaderEpochUpdate() { HashMap topicPartitionOffsets = new HashMap<>(); topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L, Optional.of(2), "")); topicPartitionOffsets.put(t1, new OffsetAndMetadata(20L, Optional.of(1), "")); - Mockito.doAnswer(invocation -> { - CompletableApplicationEvent event = invocation.getArgument(0); - assertTrue(event instanceof CommitApplicationEvent); - event.future().complete(null); - return null; - }).when(applicationEventHandler).add(any()); + mockCommitApplicationEventCompleted(); + + consumer.assign(Arrays.asList(t0, t1)); assertDoesNotThrow(() -> consumer.commitSync(topicPartitionOffsets)); verify(metadata).updateLastSeenEpochIfNewer(t0, 2); verify(metadata).updateLastSeenEpochIfNewer(t1, 1); + verify(applicationEventHandler).add(ArgumentMatchers.isA(CommitApplicationEvent.class)); } @Test public void testCommitAsyncLeaderEpochUpdate() { consumer = setup(); - OffsetCommitCallback callback = mock(OffsetCommitCallback.class); + MockCommitCallback callback = new MockCommitCallback(); final TopicPartition t0 = new TopicPartition("t0", 2); final TopicPartition t1 = new TopicPartition("t0", 3); HashMap topicPartitionOffsets = new HashMap<>(); topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L, Optional.of(2), "")); topicPartitionOffsets.put(t1, new OffsetAndMetadata(20L, Optional.of(1), "")); + consumer.assign(Arrays.asList(t0, t1)); - consumer.commitAsync(topicPartitionOffsets, callback); + assertDoesNotThrow(() -> consumer.commitAsync(topicPartitionOffsets, callback)); verify(metadata).updateLastSeenEpochIfNewer(t0, 2); verify(metadata).updateLastSeenEpochIfNewer(t1, 1); + verify(applicationEventHandler).add(ArgumentMatchers.isA(CommitApplicationEvent.class)); + } + + @Test + public void testEnsurePollExecutedCommitAsyncCallbacks() { + consumer = setup(); + MockCommitCallback callback = new MockCommitCallback(); + mockCommitApplicationEventCompleted(); + doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); + mockFetchedCommittedOffsetApplicationEventCompleted(mkMap()); + + consumer.assign(Collections.singleton(new TopicPartition("foo", 0))); + assertDoesNotThrow(() -> consumer.commitAsync(new HashMap<>(), callback)); + assertMockCommitCallbackInvoked(() -> consumer.poll(Duration.ZERO), + callback, + null); + } + + @Test + public void testEnsureShutdownExecutedCommitAsyncCallbacks() { + consumer = setup(); + MockCommitCallback callback = new MockCommitCallback(); + mockCommitApplicationEventCompleted(); + assertDoesNotThrow(() -> consumer.commitAsync(new HashMap<>(), callback)); + assertMockCommitCallbackInvoked(() -> consumer.close(), + callback, + null); + } + + private void assertMockCommitCallbackInvoked(final Executable task, + final MockCommitCallback callback, + final Errors errors) { + assertDoesNotThrow(task); + assertEquals(1, callback.invoked); + if (errors == null) + assertNull(callback.exception); + else if (errors.exception() instanceof RetriableException) + assertTrue(callback.exception instanceof RetriableCommitFailedException); } private static class MockCommitCallback implements OffsetCommitCallback { @@ -474,13 +505,7 @@ public void testAssignOnNullTopicPartition() { @Test public void testAssignOnEmptyTopicPartition() { consumer = setup(); - - Mockito.doAnswer(invocation -> { - CompletableApplicationEvent event = invocation.getArgument(0); - assertTrue(event instanceof UnsubscribeApplicationEvent); - event.future().complete(null); - return null; - }).when(applicationEventHandler).add(any()); + mockUnsubscribeApplicationEventCompleted(); consumer.assign(Collections.emptyList()); assertTrue(consumer.subscription().isEmpty()); @@ -534,7 +559,8 @@ public void testBeginningOffsetsThrowsKafkaExceptionForUnderlyingExecutionFailur () -> consumer.beginningOffsets(partitions, Duration.ofMillis(1))); assertEquals(eventProcessingFailure, consumerError); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + ArgumentMatchers.isA(Timer.class)); } @Test @@ -560,19 +586,19 @@ public void testOffsetsForTimesOnNullPartitions() { public void testOffsetsForTimesFailsOnNegativeTargetTimes() { consumer = setup(); assertThrows(IllegalArgumentException.class, - () -> consumer.offsetsForTimes(Collections.singletonMap(new TopicPartition( - "topic1", 1), ListOffsetsRequest.EARLIEST_TIMESTAMP), - Duration.ofMillis(1))); + () -> consumer.offsetsForTimes(Collections.singletonMap(new TopicPartition( + "topic1", 1), ListOffsetsRequest.EARLIEST_TIMESTAMP), + Duration.ofMillis(1))); assertThrows(IllegalArgumentException.class, - () -> consumer.offsetsForTimes(Collections.singletonMap(new TopicPartition( - "topic1", 1), ListOffsetsRequest.LATEST_TIMESTAMP), - Duration.ofMillis(1))); + () -> consumer.offsetsForTimes(Collections.singletonMap(new TopicPartition( + "topic1", 1), ListOffsetsRequest.LATEST_TIMESTAMP), + Duration.ofMillis(1))); assertThrows(IllegalArgumentException.class, - () -> consumer.offsetsForTimes(Collections.singletonMap(new TopicPartition( - "topic1", 1), ListOffsetsRequest.MAX_TIMESTAMP), - Duration.ofMillis(1))); + () -> consumer.offsetsForTimes(Collections.singletonMap(new TopicPartition( + "topic1", 1), ListOffsetsRequest.MAX_TIMESTAMP), + Duration.ofMillis(1))); } @Test @@ -583,10 +609,10 @@ public void testOffsetsForTimes() { doReturn(expectedResult).when(applicationEventHandler).addAndGet(any(), any()); Map result = - assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, Duration.ofMillis(1))); + assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, Duration.ofMillis(1))); assertEquals(expectedResult, result); verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), - ArgumentMatchers.isA(Timer.class)); + ArgumentMatchers.isA(Timer.class)); } // This test ensures same behaviour as the current consumer when offsetsForTimes is called @@ -597,12 +623,12 @@ public void testOffsetsForTimesWithZeroTimeout() { consumer = setup(); TopicPartition tp = new TopicPartition("topic1", 0); Map expectedResult = - Collections.singletonMap(tp, null); + Collections.singletonMap(tp, null); Map timestampToSearch = Collections.singletonMap(tp, 5L); Map result = - assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, - Duration.ofMillis(0))); + assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, + Duration.ofMillis(0))); assertEquals(expectedResult, result); verify(applicationEventHandler, never()).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); @@ -654,62 +680,11 @@ public void testRefreshCommittedOffsetsShouldNotResetIfFailedWithTimeout() { @Test public void testRefreshCommittedOffsetsNotCalledIfNoGroupId() { - consumer = setupWithoutGroupId(); // Create consumer without group id so committed offsets are not used for updating positions + consumer = setupWithoutGroupId(); testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(false); } - private void testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(boolean committedOffsetsEnabled) { - // Do not complete future that is passed into addAndGet, to timeout - doReturn(mkMap()) - .when(applicationEventHandler) - .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); - doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); - - consumer.assign(singleton(new TopicPartition("t1", 1))); - - // Poll with 250ms timeout to give the background thread time to process the events without timing out - consumer.poll(Duration.ofMillis(250)); - - verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); - - if (committedOffsetsEnabled) { - // Verify there was an FetchCommittedOffsets event and no ResetPositions event - verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); - verify(applicationEventHandler, never()) - .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); - } else { - // Verify there was not any FetchCommittedOffsets event but there should be a ResetPositions - verify(applicationEventHandler, never()) - .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); - verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); - } - } - - private void testRefreshCommittedOffsetsSuccess(Set partitions, - Map committedOffsets) { - doReturn(committedOffsets) - .when(applicationEventHandler) - .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); - doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); - doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); - - consumer.assign(partitions); - - // Poll with 250ms timeout to give the background thread time to process the events without timing out - consumer.poll(Duration.ofMillis(250)); - - verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); - verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); - verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); - } - @Test public void testSubscribeGeneratesEvent() { consumer = setup(); @@ -723,33 +698,24 @@ public void testSubscribeGeneratesEvent() { @Test public void testUnsubscribeGeneratesUnsubscribeEvent() { consumer = setup(); - Mockito.doAnswer(invocation -> { - CompletableApplicationEvent event = invocation.getArgument(0); - assertTrue(event instanceof UnsubscribeApplicationEvent); - event.future().complete(null); - return null; - }).when(applicationEventHandler).add(any()); + mockUnsubscribeApplicationEventCompleted(); consumer.unsubscribe(); assertTrue(consumer.subscription().isEmpty()); assertTrue(consumer.assignment().isEmpty()); + verify(applicationEventHandler).add(ArgumentMatchers.isA(UnsubscribeApplicationEvent.class)); } @Test public void testSubscribeToEmptyListActsAsUnsubscribe() { consumer = setup(); - - Mockito.doAnswer(invocation -> { - CompletableApplicationEvent event = invocation.getArgument(0); - assertTrue(event instanceof UnsubscribeApplicationEvent); - event.future().complete(null); - return null; - }).when(applicationEventHandler).add(any()); + mockUnsubscribeApplicationEventCompleted(); consumer.subscribe(Collections.emptyList()); assertTrue(consumer.subscription().isEmpty()); assertTrue(consumer.assignment().isEmpty()); + verify(applicationEventHandler).add(ArgumentMatchers.isA(UnsubscribeApplicationEvent.class)); } @Test @@ -790,10 +756,9 @@ public void testGroupMetadataAfterCreationWithGroupIdIsNull() { @Test public void testGroupMetadataAfterCreationWithGroupIdIsNotNull() { final String groupId = "consumerGroupA"; - final Properties props = requiredConsumerPropertiesAndGroupId("consumerGroupA"); - final ConsumerConfig config = new ConsumerConfig(props); - + final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId(groupId)); consumer = setup(config); + final ConsumerGroupMetadata groupMetadata = consumer.groupMetadata(); assertEquals(groupId, groupMetadata.groupId()); @@ -826,7 +791,7 @@ public void testGroupMetadataUpdateSingleCall() { consumer = setup(config); doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); - doReturn(mkMap()).when(applicationEventHandler).addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + mockFetchedCommittedOffsetApplicationEventCompleted(mkMap()); final int generation = 1; final String memberId = "newMemberId"; @@ -929,21 +894,20 @@ public void testGroupIdNull() { props.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, 10000); props.put(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED, true); final ConsumerConfig config = new ConsumerConfig(props); - consumer = setup(config); + assertFalse(config.unused().contains(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG)); assertFalse(config.unused().contains(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED)); } @Test public void testGroupIdNotNullAndValid() { - final Properties props = requiredConsumerProperties(); - props.put(ConsumerConfig.GROUP_ID_CONFIG, "consumerGroupA"); + final Properties props = requiredConsumerPropertiesAndGroupId("consumerGroupA"); props.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, 10000); props.put(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED, true); final ConsumerConfig config = new ConsumerConfig(props); - consumer = setup(config); + assertTrue(config.unused().contains(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG)); assertTrue(config.unused().contains(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED)); } @@ -959,8 +923,7 @@ public void testGroupIdOnlyWhitespaces() { } private void testInvalidGroupId(final String groupId) { - final Properties props = requiredConsumerProperties(); - props.put(ConsumerConfig.GROUP_ID_CONFIG, groupId); + final Properties props = requiredConsumerPropertiesAndGroupId(groupId); final ConsumerConfig config = new ConsumerConfig(props); final Exception exception = assertThrows( @@ -985,6 +948,50 @@ private Properties requiredConsumerProperties() { return props; } + private void testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(boolean committedOffsetsEnabled) { + mockFetchedCommittedOffsetApplicationEventException(new TimeoutException()); + doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); + + consumer.assign(singleton(new TopicPartition("t1", 1))); + + consumer.poll(Duration.ZERO); + + verify(applicationEventHandler, atLeast(1)) + .addAndGet(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + + if (committedOffsetsEnabled) { + // Verify there was an FetchCommittedOffsets event and no ResetPositions event + verify(applicationEventHandler, atLeast(1)) + .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + verify(applicationEventHandler, never()) + .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + } else { + // Verify there was not any FetchCommittedOffsets event but there should be a ResetPositions + verify(applicationEventHandler, never()) + .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + verify(applicationEventHandler, atLeast(1)) + .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + } + } + + private void testRefreshCommittedOffsetsSuccess(Set partitions, + Map committedOffsets) { + mockFetchedCommittedOffsetApplicationEventCompleted(committedOffsets); + doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); + doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); + + consumer.assign(partitions); + + consumer.poll(Duration.ZERO); + + verify(applicationEventHandler, atLeast(1)) + .addAndGet(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + verify(applicationEventHandler, atLeast(1)) + .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + verify(applicationEventHandler, atLeast(1)) + .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + } + @Test public void testLongPollWaitIsLimited() { consumer = setup(); @@ -1006,9 +1013,9 @@ public void testLongPollWaitIsLimited() { // Mock the subscription being assigned as the first fetch is collected consumer.subscriptions().assignFromSubscribed(Collections.singleton(tp)); return Fetch.empty(); - }) - .doAnswer(invocation -> Fetch.forPartition(tp, records, true)) - .when(fetchCollector).collectFetch(any(FetchBuffer.class)); + }).doAnswer(invocation -> { + return Fetch.forPartition(tp, records, true); + }).when(fetchCollector).collectFetch(any(FetchBuffer.class)); // And then poll for up to 10000ms, which should return 2 records without timing out ConsumerRecords returnedRecords = consumer.poll(Duration.ofMillis(10000)); @@ -1046,5 +1053,41 @@ private HashMap mockTimestampToSearch() { return timestampToSearch; } + private void mockCommitApplicationEventException(Exception ex) { + doAnswer(invocation -> { + CommitApplicationEvent event = invocation.getArgument(0); + event.future().completeExceptionally(ex); + return null; + }).when(applicationEventHandler).add(ArgumentMatchers.isA(CommitApplicationEvent.class)); + } + + private void mockCommitApplicationEventCompleted() { + doAnswer(invocation -> { + CommitApplicationEvent event = invocation.getArgument(0); + event.future().complete(null); + return null; + }).when(applicationEventHandler).add(ArgumentMatchers.isA(CommitApplicationEvent.class)); + } + + private void mockFetchedCommittedOffsetApplicationEventCompleted(final Map committedOffsets) { + doReturn(committedOffsets) + .when(applicationEventHandler) + .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + } + + private void mockFetchedCommittedOffsetApplicationEventException(Exception ex) { + doThrow(ex) + .when(applicationEventHandler) + .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + } + + private void mockUnsubscribeApplicationEventCompleted() { + doAnswer(invocation -> { + UnsubscribeApplicationEvent event = invocation.getArgument(0); + event.future().complete(null); + return null; + }).when(applicationEventHandler).add(ArgumentMatchers.isA(UnsubscribeApplicationEvent.class)); + } + } From 6377a9db1a8b6c2572cef0bcd1d969ec6ef75b9c Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Mon, 11 Dec 2023 12:45:30 +0100 Subject: [PATCH 16/21] checkstyle --- .../kafka/clients/consumer/internals/AsyncKafkaConsumer.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index e677f30454579..b16ae22ff8d4c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -1592,7 +1592,9 @@ int callbacks() { } // Visible for testing - SubscriptionState subscriptions() { return subscriptions; } + SubscriptionState subscriptions() { + return subscriptions; + } /** * Utility class that helps the application thread to invoke user registered {@link OffsetCommitCallback}. This is From 66a0c15a0cc018cb79c38ab47a13250139b13e79 Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Mon, 11 Dec 2023 20:23:57 +0100 Subject: [PATCH 17/21] PR comments --- .../internals/AsyncKafkaConsumer.java | 7 +- .../internals/AsyncKafkaConsumerTest.java | 230 ++++++++++-------- 2 files changed, 128 insertions(+), 109 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index b16ae22ff8d4c..2736efa14257a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -268,7 +268,7 @@ private void process(final GroupMetadataUpdateEvent event) { final Deserializer keyDeserializer, final Deserializer valueDeserializer, final Time time, - final ApplicationEventHandlerSupplier applicationEventHandlerFactory, + final ApplicationEventHandlerFactory applicationEventHandlerFactory, final FetchCollectorFactory fetchCollectorFactory, final ConsumerMetadataFactory metadataFactory, final LinkedBlockingQueue backgroundEventQueue) { @@ -498,7 +498,7 @@ private void process(final GroupMetadataUpdateEvent event) { } // auxiliary interface for testing - interface ApplicationEventHandlerSupplier { + interface ApplicationEventHandlerFactory { ApplicationEventHandler build( final LogContext logContext, @@ -1579,8 +1579,7 @@ private void maybeThrowFencedInstanceException() { } } - // Visible for testing - void maybeInvokeCommitCallbacks() { + private void maybeInvokeCommitCallbacks() { if (callbacks() > 0) { invoker.executeCallbacks(); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index 13bde3da17868..06ef989ca3d06 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -16,29 +16,6 @@ */ package org.apache.kafka.clients.consumer.internals; -import static java.util.Arrays.asList; -import static java.util.Collections.singleton; -import static java.util.Collections.singletonList; -import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED; -import static org.apache.kafka.common.utils.Utils.mkEntry; -import static org.apache.kafka.common.utils.Utils.mkMap; -import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertNull; -import static org.junit.jupiter.api.Assertions.assertSame; -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.atLeast; -import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - import java.time.Duration; import java.util.Arrays; import java.util.Collections; @@ -100,6 +77,29 @@ import org.mockito.ArgumentMatchers; import org.mockito.Mockito; +import static java.util.Arrays.asList; +import static java.util.Collections.singleton; +import static java.util.Collections.singletonList; +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED; +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertSame; +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.atLeast; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + @SuppressWarnings("unchecked") public class AsyncKafkaConsumerTest { @@ -121,27 +121,27 @@ public void resetAll() { Mockito.framework().clearInlineMocks(); } - private AsyncKafkaConsumer setup() { + private AsyncKafkaConsumer newConsumer() { final Properties props = requiredConsumerProperties(); props.put(ConsumerConfig.GROUP_ID_CONFIG, "group-id"); final ConsumerConfig config = new ConsumerConfig(props); - return setup(config); + return newConsumer(config); } - private AsyncKafkaConsumer setupWithoutGroupId() { + private AsyncKafkaConsumer newConsumerWithoutGroupId() { final Properties props = requiredConsumerProperties(); final ConsumerConfig config = new ConsumerConfig(props); - return setup(config); + return newConsumer(config); } @SuppressWarnings("UnusedReturnValue") - private AsyncKafkaConsumer setupWithEmptyGroupId() { + private AsyncKafkaConsumer newConsumerWithEmptyGroupId() { final Properties props = requiredConsumerPropertiesAndGroupId(""); final ConsumerConfig config = new ConsumerConfig(props); - return setup(config); + return newConsumer(config); } - private AsyncKafkaConsumer setup(ConsumerConfig config) { + private AsyncKafkaConsumer newConsumer(ConsumerConfig config) { return new AsyncKafkaConsumer<>( config, new StringDeserializer(), @@ -156,19 +156,19 @@ private AsyncKafkaConsumer setup(ConsumerConfig config) { @Test public void testSuccessfulStartupShutdown() { - consumer = setup(); + consumer = newConsumer(); assertDoesNotThrow(() -> consumer.close()); } @Test public void testInvalidGroupId() { - KafkaException e = assertThrows(KafkaException.class, this::setupWithEmptyGroupId); + KafkaException e = assertThrows(KafkaException.class, this::newConsumerWithEmptyGroupId); assertTrue(e.getCause() instanceof InvalidGroupIdException); } @Test public void testFailOnClosedConsumer() { - consumer = setup(); + consumer = newConsumer(); consumer.close(); final IllegalStateException res = assertThrows(IllegalStateException.class, consumer::assignment); assertEquals("This consumer has already been closed.", res.getMessage()); @@ -176,7 +176,7 @@ public void testFailOnClosedConsumer() { @Test public void testCommitAsyncWithNullCallback() { - consumer = setup(); + consumer = newConsumer(); final TopicPartition t0 = new TopicPartition("t0", 2); final TopicPartition t1 = new TopicPartition("t0", 3); HashMap offsets = new HashMap<>(); @@ -193,58 +193,74 @@ public void testCommitAsyncWithNullCallback() { assertDoesNotThrow(() -> consumer.commitAsync(offsets, null)); } + @Test + public void testCommitAsyncUserSuppliedCallbackNoException() { + consumer = newConsumer(); + + Map offsets = new HashMap<>(); + offsets.put(new TopicPartition("my-topic", 1), new OffsetAndMetadata(200L)); + mockCommitApplicationEventCompleted(); + + MockCommitCallback callback = new MockCommitCallback(); + assertDoesNotThrow(() -> consumer.commitAsync(offsets, callback)); + forceCommitCallbackInvocation(); + + assertNull(callback.exception); + } + @ParameterizedTest @MethodSource("commitExceptionSupplier") - public void testCommitAsyncUserSuppliedCallback(Exception exception) { - consumer = setup(); + public void testCommitAsyncUserSuppliedCallbackWithException(Exception exception) { + consumer = newConsumer(); Map offsets = new HashMap<>(); offsets.put(new TopicPartition("my-topic", 1), new OffsetAndMetadata(200L)); - if (exception == null) { - mockCommitApplicationEventCompleted(); - } else { - mockCommitApplicationEventException(exception); - } + mockCommitApplicationEventException(exception); MockCommitCallback callback = new MockCommitCallback(); assertDoesNotThrow(() -> consumer.commitAsync(offsets, callback)); - consumer.maybeInvokeCommitCallbacks(); + forceCommitCallbackInvocation(); - if (exception == null) { - assertNull(callback.exception); - } else { - assertSame(exception.getClass(), callback.exception.getClass()); - } + assertSame(exception.getClass(), callback.exception.getClass()); } private static Stream commitExceptionSupplier() { return Stream.of( - null, // For the successful completion scenario new KafkaException("Test exception"), new GroupAuthorizationException("Group authorization exception")); } @Test - public void testFencedInstanceException() { - consumer = setup(); - mockCommitApplicationEventException(Errors.FENCED_INSTANCE_ID.exception()); + public void testCommitAsyncWithFencedException() { + consumer = newConsumer(); + final HashMap offsets = mockTopicPartitionOffset(); + MockCommitCallback callback = new MockCommitCallback(); - assertDoesNotThrow(() -> consumer.commitAsync()); + consumer.assign(offsets.keySet()); + + assertDoesNotThrow(() -> consumer.commitAsync(offsets, callback)); + + final ArgumentCaptor commitEventCaptor = ArgumentCaptor.forClass(CommitApplicationEvent.class); + verify(applicationEventHandler).add(commitEventCaptor.capture()); + final CommitApplicationEvent commitEvent = commitEventCaptor.getValue(); + commitEvent.future().completeExceptionally(Errors.FENCED_INSTANCE_ID.exception()); + + assertThrows(Errors.FENCED_INSTANCE_ID.exception().getClass(), () -> consumer.commitAsync()); } @Test public void testCommitted() { - consumer = setup(); + consumer = newConsumer(); Map topicPartitionOffsets = mockTopicPartitionOffset(); mockFetchedCommittedOffsetApplicationEventCompleted(topicPartitionOffsets); - assertDoesNotThrow(() -> consumer.committed(topicPartitionOffsets.keySet(), Duration.ofMillis(1000))); + assertEquals(topicPartitionOffsets, consumer.committed(topicPartitionOffsets.keySet(), Duration.ofMillis(1000))); verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), any()); } @Test public void testCommittedLeaderEpochUpdate() { - consumer = setup(); + consumer = newConsumer(); final TopicPartition t0 = new TopicPartition("t0", 2); final TopicPartition t1 = new TopicPartition("t0", 3); final TopicPartition t2 = new TopicPartition("t0", 4); @@ -263,7 +279,7 @@ public void testCommittedLeaderEpochUpdate() { @Test public void testCommittedExceptionThrown() { - consumer = setup(); + consumer = newConsumer(); Map offsets = mockTopicPartitionOffset(); when(applicationEventHandler.addAndGet(any(), any())).thenAnswer(invocation -> { CompletableApplicationEvent event = invocation.getArgument(0); @@ -276,7 +292,7 @@ public void testCommittedExceptionThrown() { @Test public void testWakeupBeforeCallingPoll() { - consumer = setup(); + consumer = newConsumer(); final String topicName = "foo"; final int partition = 3; final TopicPartition tp = new TopicPartition(topicName, partition); @@ -294,7 +310,7 @@ public void testWakeupBeforeCallingPoll() { @Test public void testWakeupAfterEmptyFetch() { - consumer = setup(); + consumer = newConsumer(); final String topicName = "foo"; final int partition = 3; final TopicPartition tp = new TopicPartition(topicName, partition); @@ -313,7 +329,7 @@ public void testWakeupAfterEmptyFetch() { @Test public void testWakeupAfterNonEmptyFetch() { - consumer = setup(); + consumer = newConsumer(); final String topicName = "foo"; final int partition = 3; final TopicPartition tp = new TopicPartition(topicName, partition); @@ -338,7 +354,7 @@ public void testWakeupAfterNonEmptyFetch() { @Test public void testClearWakeupTriggerAfterPoll() { - consumer = setup(); + consumer = newConsumer(); final String topicName = "foo"; final int partition = 3; final TopicPartition tp = new TopicPartition(topicName, partition); @@ -360,20 +376,20 @@ public void testClearWakeupTriggerAfterPoll() { @Test public void testEnsureCallbackExecutedByApplicationThread() { - consumer = setup(); + consumer = newConsumer(); final String currentThread = Thread.currentThread().getName(); MockCommitCallback callback = new MockCommitCallback(); mockCommitApplicationEventCompleted(); assertDoesNotThrow(() -> consumer.commitAsync(new HashMap<>(), callback)); assertEquals(1, consumer.callbacks()); - consumer.maybeInvokeCommitCallbacks(); + forceCommitCallbackInvocation(); assertEquals(currentThread, callback.completionThread); } @Test public void testEnsureCommitSyncExecutedCommitAsyncCallbacks() { - consumer = setup(); + consumer = newConsumer(); final HashMap offsets = mockTopicPartitionOffset(); MockCommitCallback callback = new MockCommitCallback(); assertDoesNotThrow(() -> consumer.commitAsync(offsets, callback)); @@ -391,7 +407,7 @@ public void testEnsureCommitSyncExecutedCommitAsyncCallbacks() { @Test @SuppressWarnings("deprecation") public void testPollLongThrowsException() { - consumer = setup(); + consumer = newConsumer(); Exception e = assertThrows(UnsupportedOperationException.class, () -> consumer.poll(0L)); assertEquals("Consumer.poll(long) is not supported when \"group.protocol\" is \"consumer\". " + "This method is deprecated and will be removed in the next major release.", e.getMessage()); @@ -399,7 +415,7 @@ public void testPollLongThrowsException() { @Test public void testCommitSyncLeaderEpochUpdate() { - consumer = setup(); + consumer = newConsumer(); final TopicPartition t0 = new TopicPartition("t0", 2); final TopicPartition t1 = new TopicPartition("t0", 3); HashMap topicPartitionOffsets = new HashMap<>(); @@ -418,7 +434,7 @@ public void testCommitSyncLeaderEpochUpdate() { @Test public void testCommitAsyncLeaderEpochUpdate() { - consumer = setup(); + consumer = newConsumer(); MockCommitCallback callback = new MockCommitCallback(); final TopicPartition t0 = new TopicPartition("t0", 2); final TopicPartition t1 = new TopicPartition("t0", 3); @@ -437,7 +453,7 @@ public void testCommitAsyncLeaderEpochUpdate() { @Test public void testEnsurePollExecutedCommitAsyncCallbacks() { - consumer = setup(); + consumer = newConsumer(); MockCommitCallback callback = new MockCommitCallback(); mockCommitApplicationEventCompleted(); doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); @@ -452,7 +468,7 @@ public void testEnsurePollExecutedCommitAsyncCallbacks() { @Test public void testEnsureShutdownExecutedCommitAsyncCallbacks() { - consumer = setup(); + consumer = newConsumer(); MockCommitCallback callback = new MockCommitCallback(); mockCommitApplicationEventCompleted(); assertDoesNotThrow(() -> consumer.commitAsync(new HashMap<>(), callback)); @@ -487,7 +503,7 @@ public void onComplete(Map offsets, Exception @Test public void testAssign() { - consumer = setup(); + consumer = newConsumer(); final TopicPartition tp = new TopicPartition("foo", 3); consumer.assign(singleton(tp)); assertTrue(consumer.subscription().isEmpty()); @@ -498,13 +514,13 @@ public void testAssign() { @Test public void testAssignOnNullTopicPartition() { - consumer = setup(); + consumer = newConsumer(); assertThrows(IllegalArgumentException.class, () -> consumer.assign(null)); } @Test public void testAssignOnEmptyTopicPartition() { - consumer = setup(); + consumer = newConsumer(); mockUnsubscribeApplicationEventCompleted(); consumer.assign(Collections.emptyList()); @@ -514,26 +530,26 @@ public void testAssignOnEmptyTopicPartition() { @Test public void testAssignOnNullTopicInPartition() { - consumer = setup(); + consumer = newConsumer(); assertThrows(IllegalArgumentException.class, () -> consumer.assign(singleton(new TopicPartition(null, 0)))); } @Test public void testAssignOnEmptyTopicInPartition() { - consumer = setup(); + consumer = newConsumer(); assertThrows(IllegalArgumentException.class, () -> consumer.assign(singleton(new TopicPartition(" ", 0)))); } @Test public void testBeginningOffsetsFailsIfNullPartitions() { - consumer = setup(); + consumer = newConsumer(); assertThrows(NullPointerException.class, () -> consumer.beginningOffsets(null, Duration.ofMillis(1))); } @Test public void testBeginningOffsets() { - consumer = setup(); + consumer = newConsumer(); Map expectedOffsetsAndTimestamp = mockOffsetAndTimestamp(); Set partitions = expectedOffsetsAndTimestamp.keySet(); @@ -550,7 +566,7 @@ public void testBeginningOffsets() { @Test public void testBeginningOffsetsThrowsKafkaExceptionForUnderlyingExecutionFailure() { - consumer = setup(); + consumer = newConsumer(); Set partitions = mockTopicPartitionOffset().keySet(); Throwable eventProcessingFailure = new KafkaException("Unexpected failure " + "processing List Offsets event"); @@ -565,7 +581,7 @@ public void testBeginningOffsetsThrowsKafkaExceptionForUnderlyingExecutionFailur @Test public void testBeginningOffsetsTimeoutOnEventProcessingTimeout() { - consumer = setup(); + consumer = newConsumer(); doThrow(new TimeoutException()).when(applicationEventHandler).addAndGet(any(), any()); assertThrows(TimeoutException.class, () -> consumer.beginningOffsets( @@ -577,14 +593,14 @@ public void testBeginningOffsetsTimeoutOnEventProcessingTimeout() { @Test public void testOffsetsForTimesOnNullPartitions() { - consumer = setup(); + consumer = newConsumer(); assertThrows(NullPointerException.class, () -> consumer.offsetsForTimes(null, Duration.ofMillis(1))); } @Test public void testOffsetsForTimesFailsOnNegativeTargetTimes() { - consumer = setup(); + consumer = newConsumer(); assertThrows(IllegalArgumentException.class, () -> consumer.offsetsForTimes(Collections.singletonMap(new TopicPartition( "topic1", 1), ListOffsetsRequest.EARLIEST_TIMESTAMP), @@ -603,7 +619,7 @@ public void testOffsetsForTimesFailsOnNegativeTargetTimes() { @Test public void testOffsetsForTimes() { - consumer = setup(); + consumer = newConsumer(); Map expectedResult = mockOffsetAndTimestamp(); Map timestampToSearch = mockTimestampToSearch(); @@ -620,7 +636,7 @@ public void testOffsetsForTimes() { // OffsetAndTimestamp as value. @Test public void testOffsetsForTimesWithZeroTimeout() { - consumer = setup(); + consumer = newConsumer(); TopicPartition tp = new TopicPartition("topic1", 0); Map expectedResult = Collections.singletonMap(tp, null); @@ -636,7 +652,7 @@ public void testOffsetsForTimesWithZeroTimeout() { @Test public void testWakeupCommitted() { - consumer = setup(); + consumer = newConsumer(); final HashMap offsets = mockTopicPartitionOffset(); doAnswer(invocation -> { CompletableApplicationEvent event = invocation.getArgument(0); @@ -656,7 +672,7 @@ public void testWakeupCommitted() { @Test public void testRefreshCommittedOffsetsSuccess() { - consumer = setup(); + consumer = newConsumer(); TopicPartition partition = new TopicPartition("t1", 1); Set partitions = Collections.singleton(partition); Map committedOffsets = Collections.singletonMap(partition, new OffsetAndMetadata(10L)); @@ -665,7 +681,7 @@ public void testRefreshCommittedOffsetsSuccess() { @Test public void testRefreshCommittedOffsetsSuccessButNoCommittedOffsetsFound() { - consumer = setup(); + consumer = newConsumer(); TopicPartition partition = new TopicPartition("t1", 1); Set partitions = Collections.singleton(partition); Map committedOffsets = Collections.emptyMap(); @@ -674,20 +690,20 @@ public void testRefreshCommittedOffsetsSuccessButNoCommittedOffsetsFound() { @Test public void testRefreshCommittedOffsetsShouldNotResetIfFailedWithTimeout() { - consumer = setup(); + consumer = newConsumer(); testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(true); } @Test public void testRefreshCommittedOffsetsNotCalledIfNoGroupId() { // Create consumer without group id so committed offsets are not used for updating positions - consumer = setupWithoutGroupId(); + consumer = newConsumerWithoutGroupId(); testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(false); } @Test public void testSubscribeGeneratesEvent() { - consumer = setup(); + consumer = newConsumer(); String topic = "topic1"; consumer.subscribe(singletonList(topic)); assertEquals(singleton(topic), consumer.subscription()); @@ -697,7 +713,7 @@ public void testSubscribeGeneratesEvent() { @Test public void testUnsubscribeGeneratesUnsubscribeEvent() { - consumer = setup(); + consumer = newConsumer(); mockUnsubscribeApplicationEventCompleted(); consumer.unsubscribe(); @@ -709,7 +725,7 @@ public void testUnsubscribeGeneratesUnsubscribeEvent() { @Test public void testSubscribeToEmptyListActsAsUnsubscribe() { - consumer = setup(); + consumer = newConsumer(); mockUnsubscribeApplicationEventCompleted(); consumer.subscribe(Collections.emptyList()); @@ -720,19 +736,19 @@ public void testSubscribeToEmptyListActsAsUnsubscribe() { @Test public void testSubscribeToNullTopicCollection() { - consumer = setup(); + consumer = newConsumer(); assertThrows(IllegalArgumentException.class, () -> consumer.subscribe((List) null)); } @Test public void testSubscriptionOnNullTopic() { - consumer = setup(); + consumer = newConsumer(); assertThrows(IllegalArgumentException.class, () -> consumer.subscribe(singletonList(null))); } @Test public void testSubscriptionOnEmptyTopic() { - consumer = setup(); + consumer = newConsumer(); String emptyTopic = " "; assertThrows(IllegalArgumentException.class, () -> consumer.subscribe(singletonList(emptyTopic))); } @@ -741,7 +757,7 @@ public void testSubscriptionOnEmptyTopic() { public void testGroupMetadataAfterCreationWithGroupIdIsNull() { final Properties props = requiredConsumerProperties(); final ConsumerConfig config = new ConsumerConfig(props); - consumer = setup(config); + consumer = newConsumer(config); assertFalse(config.unused().contains(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG)); assertFalse(config.unused().contains(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED)); @@ -757,7 +773,7 @@ public void testGroupMetadataAfterCreationWithGroupIdIsNull() { public void testGroupMetadataAfterCreationWithGroupIdIsNotNull() { final String groupId = "consumerGroupA"; final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId(groupId)); - consumer = setup(config); + consumer = newConsumer(config); final ConsumerGroupMetadata groupMetadata = consumer.groupMetadata(); @@ -774,7 +790,7 @@ public void testGroupMetadataAfterCreationWithGroupIdIsNotNullAndGroupInstanceId final Properties props = requiredConsumerPropertiesAndGroupId(groupId); props.put(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG, groupInstanceId); final ConsumerConfig config = new ConsumerConfig(props); - consumer = setup(config); + consumer = newConsumer(config); final ConsumerGroupMetadata groupMetadata = consumer.groupMetadata(); @@ -788,7 +804,7 @@ public void testGroupMetadataAfterCreationWithGroupIdIsNotNullAndGroupInstanceId public void testGroupMetadataUpdateSingleCall() { final String groupId = "consumerGroupA"; final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId(groupId)); - consumer = setup(config); + consumer = newConsumer(config); doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); mockFetchedCommittedOffsetApplicationEventCompleted(mkMap()); @@ -822,7 +838,7 @@ public void testGroupMetadataUpdateSingleCall() { public void testBackgroundError() { final String groupId = "consumerGroupA"; final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId(groupId)); - consumer = setup(config); + consumer = newConsumer(config); final KafkaException expectedException = new KafkaException("Nobody expects the Spanish Inquisition"); final ErrorBackgroundEvent errorBackgroundEvent = new ErrorBackgroundEvent(expectedException); @@ -838,7 +854,7 @@ public void testBackgroundError() { public void testMultipleBackgroundErrors() { final String groupId = "consumerGroupA"; final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId(groupId)); - consumer = setup(config); + consumer = newConsumer(config); final KafkaException expectedException1 = new KafkaException("Nobody expects the Spanish Inquisition"); final ErrorBackgroundEvent errorBackgroundEvent1 = new ErrorBackgroundEvent(expectedException1); @@ -859,7 +875,7 @@ public void testGroupRemoteAssignorUnusedIfGroupIdUndefined() { final Properties props = requiredConsumerProperties(); props.put(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, "someAssignor"); final ConsumerConfig config = new ConsumerConfig(props); - consumer = setup(config); + consumer = newConsumer(config); assertTrue(config.unused().contains(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG)); } @@ -871,7 +887,7 @@ public void testGroupRemoteAssignorUnusedInGenericProtocol() { props.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.GENERIC.name().toLowerCase(Locale.ROOT)); props.put(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, "someAssignor"); final ConsumerConfig config = new ConsumerConfig(props); - consumer = setup(config); + consumer = newConsumer(config); assertTrue(config.unused().contains(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG)); } @@ -883,7 +899,7 @@ public void testGroupRemoteAssignorUsedInConsumerProtocol() { props.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CONSUMER.name().toLowerCase(Locale.ROOT)); props.put(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, "someAssignor"); final ConsumerConfig config = new ConsumerConfig(props); - consumer = setup(config); + consumer = newConsumer(config); assertFalse(config.unused().contains(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG)); } @@ -894,7 +910,7 @@ public void testGroupIdNull() { props.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, 10000); props.put(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED, true); final ConsumerConfig config = new ConsumerConfig(props); - consumer = setup(config); + consumer = newConsumer(config); assertFalse(config.unused().contains(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG)); assertFalse(config.unused().contains(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED)); @@ -906,7 +922,7 @@ public void testGroupIdNotNullAndValid() { props.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, 10000); props.put(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED, true); final ConsumerConfig config = new ConsumerConfig(props); - consumer = setup(config); + consumer = newConsumer(config); assertTrue(config.unused().contains(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG)); assertTrue(config.unused().contains(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED)); @@ -928,7 +944,7 @@ private void testInvalidGroupId(final String groupId) { final Exception exception = assertThrows( KafkaException.class, - () -> consumer = setup(config) + () -> consumer = newConsumer(config) ); assertEquals("Failed to construct kafka consumer", exception.getMessage()); @@ -994,7 +1010,7 @@ private void testRefreshCommittedOffsetsSuccess(Set partitions, @Test public void testLongPollWaitIsLimited() { - consumer = setup(); + consumer = newConsumer(); String topicName = "topic1"; consumer.subscribe(singletonList(topicName)); @@ -1089,5 +1105,9 @@ private void mockUnsubscribeApplicationEventCompleted() { }).when(applicationEventHandler).add(ArgumentMatchers.isA(UnsubscribeApplicationEvent.class)); } + private void forceCommitCallbackInvocation() { + // Invokes callback + consumer.commitAsync(); + } } From e6430cafdaf15b8611f6addc8347473540b797b9 Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Tue, 12 Dec 2023 10:32:41 +0100 Subject: [PATCH 18/21] remove unused code. make fenced exception test more unflaky. assert( ... instanceof ... ) -> assertIsInstanceOf(...) --- .../internals/AsyncKafkaConsumer.java | 39 ---------- .../internals/AsyncKafkaConsumerTest.java | 59 +++++++-------- .../internals/ConsumerTestBuilder.java | 75 ------------------- 3 files changed, 29 insertions(+), 144 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 2736efa14257a..d9ac661c436e8 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -89,7 +89,6 @@ import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Optional; import java.util.OptionalLong; import java.util.Set; @@ -378,44 +377,6 @@ private void process(final GroupMetadataUpdateEvent event) { } } - // Visible for testing - AsyncKafkaConsumer(LogContext logContext, - String clientId, - Deserializers deserializers, - FetchBuffer fetchBuffer, - FetchCollector fetchCollector, - ConsumerInterceptors interceptors, - Time time, - ApplicationEventHandler applicationEventHandler, - BlockingQueue backgroundEventQueue, - Metrics metrics, - SubscriptionState subscriptions, - ConsumerMetadata metadata, - long retryBackoffMs, - int defaultApiTimeoutMs, - List assignors, - String groupId) { - this.log = logContext.logger(getClass()); - this.subscriptions = subscriptions; - this.clientId = clientId; - this.fetchBuffer = fetchBuffer; - this.fetchCollector = fetchCollector; - this.isolationLevel = IsolationLevel.READ_UNCOMMITTED; - this.interceptors = Objects.requireNonNull(interceptors); - this.time = time; - this.backgroundEventProcessor = new BackgroundEventProcessor(logContext, backgroundEventQueue); - this.metrics = metrics; - this.groupMetadata = initializeGroupMetadata(groupId, Optional.empty()); - this.metadata = metadata; - this.retryBackoffMs = retryBackoffMs; - this.defaultApiTimeoutMs = defaultApiTimeoutMs; - this.deserializers = deserializers; - this.applicationEventHandler = applicationEventHandler; - this.assignors = assignors; - this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, "consumer"); - this.clientTelemetryReporter = Optional.empty(); - } - // Visible for testing AsyncKafkaConsumer(LogContext logContext, Time time, diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index 06ef989ca3d06..93a226f76f090 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -86,6 +86,7 @@ import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertSame; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -163,7 +164,7 @@ public void testSuccessfulStartupShutdown() { @Test public void testInvalidGroupId() { KafkaException e = assertThrows(KafkaException.class, this::newConsumerWithEmptyGroupId); - assertTrue(e.getCause() instanceof InvalidGroupIdException); + assertInstanceOf(InvalidGroupIdException.class, e.getCause()); } @Test @@ -199,7 +200,7 @@ public void testCommitAsyncUserSuppliedCallbackNoException() { Map offsets = new HashMap<>(); offsets.put(new TopicPartition("my-topic", 1), new OffsetAndMetadata(200L)); - mockCommitApplicationEventCompleted(); + completeCommitApplicationEventExceptionally(); MockCommitCallback callback = new MockCommitCallback(); assertDoesNotThrow(() -> consumer.commitAsync(offsets, callback)); @@ -215,7 +216,7 @@ public void testCommitAsyncUserSuppliedCallbackWithException(Exception exception Map offsets = new HashMap<>(); offsets.put(new TopicPartition("my-topic", 1), new OffsetAndMetadata(200L)); - mockCommitApplicationEventException(exception); + completeCommitApplicationEventExceptionally(exception); MockCommitCallback callback = new MockCommitCallback(); assertDoesNotThrow(() -> consumer.commitAsync(offsets, callback)); @@ -236,8 +237,6 @@ public void testCommitAsyncWithFencedException() { final HashMap offsets = mockTopicPartitionOffset(); MockCommitCallback callback = new MockCommitCallback(); - consumer.assign(offsets.keySet()); - assertDoesNotThrow(() -> consumer.commitAsync(offsets, callback)); final ArgumentCaptor commitEventCaptor = ArgumentCaptor.forClass(CommitApplicationEvent.class); @@ -252,7 +251,7 @@ public void testCommitAsyncWithFencedException() { public void testCommitted() { consumer = newConsumer(); Map topicPartitionOffsets = mockTopicPartitionOffset(); - mockFetchedCommittedOffsetApplicationEventCompleted(topicPartitionOffsets); + completeFetchedCommittedOffsetApplicationEventSuccessfully(topicPartitionOffsets); assertEquals(topicPartitionOffsets, consumer.committed(topicPartitionOffsets.keySet(), Duration.ofMillis(1000))); verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), any()); @@ -268,7 +267,7 @@ public void testCommittedLeaderEpochUpdate() { topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L, Optional.of(2), "")); topicPartitionOffsets.put(t1, null); topicPartitionOffsets.put(t2, new OffsetAndMetadata(20L, Optional.of(3), "")); - mockFetchedCommittedOffsetApplicationEventCompleted(topicPartitionOffsets); + completeFetchedCommittedOffsetApplicationEventSuccessfully(topicPartitionOffsets); assertDoesNotThrow(() -> consumer.committed(topicPartitionOffsets.keySet(), Duration.ofMillis(1000))); @@ -283,7 +282,7 @@ public void testCommittedExceptionThrown() { Map offsets = mockTopicPartitionOffset(); when(applicationEventHandler.addAndGet(any(), any())).thenAnswer(invocation -> { CompletableApplicationEvent event = invocation.getArgument(0); - assertTrue(event instanceof FetchCommittedOffsetsApplicationEvent); + assertInstanceOf(FetchCommittedOffsetsApplicationEvent.class, event); throw new KafkaException("Test exception"); }); @@ -298,7 +297,7 @@ public void testWakeupBeforeCallingPoll() { final TopicPartition tp = new TopicPartition(topicName, partition); doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); Map offsets = mkMap(mkEntry(tp, new OffsetAndMetadata(1))); - mockFetchedCommittedOffsetApplicationEventCompleted(offsets); + completeFetchedCommittedOffsetApplicationEventSuccessfully(offsets); doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); consumer.assign(singleton(tp)); @@ -319,7 +318,7 @@ public void testWakeupAfterEmptyFetch() { return Fetch.empty(); }).when(fetchCollector).collectFetch(any(FetchBuffer.class)); Map offsets = mkMap(mkEntry(tp, new OffsetAndMetadata(1))); - mockFetchedCommittedOffsetApplicationEventCompleted(offsets); + completeFetchedCommittedOffsetApplicationEventSuccessfully(offsets); doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); consumer.assign(singleton(tp)); @@ -342,7 +341,7 @@ public void testWakeupAfterNonEmptyFetch() { return Fetch.forPartition(tp, records, true); }).when(fetchCollector).collectFetch(Mockito.any(FetchBuffer.class)); Map offsets = mkMap(mkEntry(tp, new OffsetAndMetadata(1))); - mockFetchedCommittedOffsetApplicationEventCompleted(offsets); + completeFetchedCommittedOffsetApplicationEventSuccessfully(offsets); doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); consumer.assign(singleton(tp)); @@ -365,7 +364,7 @@ public void testClearWakeupTriggerAfterPoll() { doReturn(Fetch.forPartition(tp, records, true)) .when(fetchCollector).collectFetch(any(FetchBuffer.class)); Map offsets = mkMap(mkEntry(tp, new OffsetAndMetadata(1))); - mockFetchedCommittedOffsetApplicationEventCompleted(offsets); + completeFetchedCommittedOffsetApplicationEventSuccessfully(offsets); doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); consumer.assign(singleton(tp)); @@ -379,7 +378,7 @@ public void testEnsureCallbackExecutedByApplicationThread() { consumer = newConsumer(); final String currentThread = Thread.currentThread().getName(); MockCommitCallback callback = new MockCommitCallback(); - mockCommitApplicationEventCompleted(); + completeCommitApplicationEventExceptionally(); assertDoesNotThrow(() -> consumer.commitAsync(new HashMap<>(), callback)); assertEquals(1, consumer.callbacks()); @@ -421,7 +420,7 @@ public void testCommitSyncLeaderEpochUpdate() { HashMap topicPartitionOffsets = new HashMap<>(); topicPartitionOffsets.put(t0, new OffsetAndMetadata(10L, Optional.of(2), "")); topicPartitionOffsets.put(t1, new OffsetAndMetadata(20L, Optional.of(1), "")); - mockCommitApplicationEventCompleted(); + completeCommitApplicationEventExceptionally(); consumer.assign(Arrays.asList(t0, t1)); @@ -455,9 +454,9 @@ public void testCommitAsyncLeaderEpochUpdate() { public void testEnsurePollExecutedCommitAsyncCallbacks() { consumer = newConsumer(); MockCommitCallback callback = new MockCommitCallback(); - mockCommitApplicationEventCompleted(); + completeCommitApplicationEventExceptionally(); doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); - mockFetchedCommittedOffsetApplicationEventCompleted(mkMap()); + completeFetchedCommittedOffsetApplicationEventSuccessfully(mkMap()); consumer.assign(Collections.singleton(new TopicPartition("foo", 0))); assertDoesNotThrow(() -> consumer.commitAsync(new HashMap<>(), callback)); @@ -470,7 +469,7 @@ public void testEnsurePollExecutedCommitAsyncCallbacks() { public void testEnsureShutdownExecutedCommitAsyncCallbacks() { consumer = newConsumer(); MockCommitCallback callback = new MockCommitCallback(); - mockCommitApplicationEventCompleted(); + completeCommitApplicationEventExceptionally(); assertDoesNotThrow(() -> consumer.commitAsync(new HashMap<>(), callback)); assertMockCommitCallbackInvoked(() -> consumer.close(), callback, @@ -485,7 +484,7 @@ private void assertMockCommitCallbackInvoked(final Executable task, if (errors == null) assertNull(callback.exception); else if (errors.exception() instanceof RetriableException) - assertTrue(callback.exception instanceof RetriableCommitFailedException); + assertInstanceOf(RetriableCommitFailedException.class, callback.exception); } private static class MockCommitCallback implements OffsetCommitCallback { @@ -521,7 +520,7 @@ public void testAssignOnNullTopicPartition() { @Test public void testAssignOnEmptyTopicPartition() { consumer = newConsumer(); - mockUnsubscribeApplicationEventCompleted(); + completeUnsubscribeApplicationEventSuccessfully(); consumer.assign(Collections.emptyList()); assertTrue(consumer.subscription().isEmpty()); @@ -657,7 +656,7 @@ public void testWakeupCommitted() { doAnswer(invocation -> { CompletableApplicationEvent event = invocation.getArgument(0); Timer timer = invocation.getArgument(1); - assertTrue(event instanceof FetchCommittedOffsetsApplicationEvent); + assertInstanceOf(FetchCommittedOffsetsApplicationEvent.class, event); assertTrue(event.future().isCompletedExceptionally()); return ConsumerUtils.getResult(event.future(), timer); }) @@ -714,7 +713,7 @@ public void testSubscribeGeneratesEvent() { @Test public void testUnsubscribeGeneratesUnsubscribeEvent() { consumer = newConsumer(); - mockUnsubscribeApplicationEventCompleted(); + completeUnsubscribeApplicationEventSuccessfully(); consumer.unsubscribe(); @@ -726,7 +725,7 @@ public void testUnsubscribeGeneratesUnsubscribeEvent() { @Test public void testSubscribeToEmptyListActsAsUnsubscribe() { consumer = newConsumer(); - mockUnsubscribeApplicationEventCompleted(); + completeUnsubscribeApplicationEventSuccessfully(); consumer.subscribe(Collections.emptyList()); assertTrue(consumer.subscription().isEmpty()); @@ -807,7 +806,7 @@ public void testGroupMetadataUpdateSingleCall() { consumer = newConsumer(config); doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); - mockFetchedCommittedOffsetApplicationEventCompleted(mkMap()); + completeFetchedCommittedOffsetApplicationEventSuccessfully(mkMap()); final int generation = 1; final String memberId = "newMemberId"; @@ -965,7 +964,7 @@ private Properties requiredConsumerProperties() { } private void testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(boolean committedOffsetsEnabled) { - mockFetchedCommittedOffsetApplicationEventException(new TimeoutException()); + completeFetchedCommittedOffsetApplicationEventExceptionally(new TimeoutException()); doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); consumer.assign(singleton(new TopicPartition("t1", 1))); @@ -992,7 +991,7 @@ private void testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(boolean co private void testRefreshCommittedOffsetsSuccess(Set partitions, Map committedOffsets) { - mockFetchedCommittedOffsetApplicationEventCompleted(committedOffsets); + completeFetchedCommittedOffsetApplicationEventSuccessfully(committedOffsets); doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); @@ -1069,7 +1068,7 @@ private HashMap mockTimestampToSearch() { return timestampToSearch; } - private void mockCommitApplicationEventException(Exception ex) { + private void completeCommitApplicationEventExceptionally(Exception ex) { doAnswer(invocation -> { CommitApplicationEvent event = invocation.getArgument(0); event.future().completeExceptionally(ex); @@ -1077,7 +1076,7 @@ private void mockCommitApplicationEventException(Exception ex) { }).when(applicationEventHandler).add(ArgumentMatchers.isA(CommitApplicationEvent.class)); } - private void mockCommitApplicationEventCompleted() { + private void completeCommitApplicationEventExceptionally() { doAnswer(invocation -> { CommitApplicationEvent event = invocation.getArgument(0); event.future().complete(null); @@ -1085,19 +1084,19 @@ private void mockCommitApplicationEventCompleted() { }).when(applicationEventHandler).add(ArgumentMatchers.isA(CommitApplicationEvent.class)); } - private void mockFetchedCommittedOffsetApplicationEventCompleted(final Map committedOffsets) { + private void completeFetchedCommittedOffsetApplicationEventSuccessfully(final Map committedOffsets) { doReturn(committedOffsets) .when(applicationEventHandler) .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); } - private void mockFetchedCommittedOffsetApplicationEventException(Exception ex) { + private void completeFetchedCommittedOffsetApplicationEventExceptionally(Exception ex) { doThrow(ex) .when(applicationEventHandler) .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); } - private void mockUnsubscribeApplicationEventCompleted() { + private void completeUnsubscribeApplicationEventSuccessfully() { doAnswer(invocation -> { UnsubscribeApplicationEvent event = invocation.getArgument(0); event.future().complete(null); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java index 28a68e9f751cd..8fe9b527bffb1 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java @@ -97,10 +97,6 @@ public class ConsumerTestBuilder implements Closeable { final MockClient client; final Optional groupInfo; - public ConsumerTestBuilder() { - this(Optional.empty()); - } - public ConsumerTestBuilder(Optional groupInfo) { this(groupInfo, true, true); } @@ -298,77 +294,6 @@ public void close() { } } - public static class ApplicationEventHandlerTestBuilder extends ConsumerTestBuilder { - - public final ApplicationEventHandler applicationEventHandler; - - public ApplicationEventHandlerTestBuilder(Optional groupInfo, boolean enableAutoCommit, boolean enableAutoTick) { - super(groupInfo, enableAutoCommit, enableAutoTick); - this.applicationEventHandler = spy(new ApplicationEventHandler( - logContext, - time, - applicationEventQueue, - () -> applicationEventProcessor, - () -> networkClientDelegate, - () -> requestManagers)); - } - - @Override - public void close() { - closeQuietly(applicationEventHandler, ApplicationEventHandler.class.getSimpleName()); - } - } - - public static class AsyncKafkaConsumerTestBuilder extends ApplicationEventHandlerTestBuilder { - - final AsyncKafkaConsumer consumer; - - final FetchCollector fetchCollector; - - public AsyncKafkaConsumerTestBuilder(Optional groupInfo, boolean enableAutoCommit, boolean enableAutoTick) { - super(groupInfo, enableAutoCommit, enableAutoTick); - String clientId = config.getString(CommonClientConfigs.CLIENT_ID_CONFIG); - List assignors = ConsumerPartitionAssignor.getAssignorInstances( - config.getList(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), - config.originals(Collections.singletonMap(ConsumerConfig.CLIENT_ID_CONFIG, clientId)) - ); - Deserializers deserializers = new Deserializers<>(new StringDeserializer(), new StringDeserializer()); - this.fetchCollector = spy(new FetchCollector<>(logContext, - metadata, - subscriptions, - fetchConfig, - deserializers, - metricsManager, - time)); - this.consumer = spy(new AsyncKafkaConsumer<>( - logContext, - clientId, - deserializers, - new FetchBuffer(logContext), - fetchCollector, - new ConsumerInterceptors<>(Collections.emptyList()), - time, - applicationEventHandler, - backgroundEventQueue, - metrics, - subscriptions, - metadata, - retryBackoffMs, - 60000, - assignors, - groupInfo.map(groupInformation -> groupInformation.groupState.groupId).orElse(null))); - } - - @Override - public void close() { - consumer.close(); - } - - public void close(final Duration timeout) { - consumer.close(timeout); - } - } - public static class GroupInformation { final GroupState groupState; From 59e385b243ab089f9cdd6901dec29eba77dbbbd5 Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Tue, 12 Dec 2023 10:39:28 +0100 Subject: [PATCH 19/21] optimize imports --- .../internals/AsyncKafkaConsumerTest.java | 27 ++++++++++--------- 1 file changed, 14 insertions(+), 13 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index 93a226f76f090..372ee8e50d379 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -16,19 +16,6 @@ */ package org.apache.kafka.clients.consumer.internals; -import java.time.Duration; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.Optional; -import java.util.Properties; -import java.util.Set; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.stream.Collectors; -import java.util.stream.Stream; import org.apache.kafka.clients.Metadata.LeaderAndEpoch; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; @@ -77,6 +64,20 @@ import org.mockito.ArgumentMatchers; import org.mockito.Mockito; +import java.time.Duration; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Optional; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.stream.Collectors; +import java.util.stream.Stream; + import static java.util.Arrays.asList; import static java.util.Collections.singleton; import static java.util.Collections.singletonList; From dde99b34d4fc5425de56462311e0f6a56ee2f0fa Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Tue, 12 Dec 2023 11:39:17 +0100 Subject: [PATCH 20/21] checkstyle --- .../clients/consumer/internals/ConsumerTestBuilder.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java index 08db1a2653b1d..d3e58d56304e5 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java @@ -21,9 +21,7 @@ import org.apache.kafka.clients.GroupRebalanceConfig; import org.apache.kafka.clients.MockClient; import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor; import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; @@ -38,10 +36,7 @@ import org.apache.kafka.common.utils.Timer; import java.io.Closeable; -import java.time.Duration; -import java.util.Collections; import java.util.HashMap; -import java.util.List; import java.util.Optional; import java.util.Properties; import java.util.concurrent.BlockingQueue; From 5e734fe168f27a49cfbddc396ea6a4fdd2ccb3a3 Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Mon, 18 Dec 2023 18:57:24 +0100 Subject: [PATCH 21/21] merge fixes --- .../consumer/internals/AsyncKafkaConsumerTest.java | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index 54da9291511b1..19734698afba9 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -31,9 +31,9 @@ import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableBackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackNeededEvent; -import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.EventProcessor; import org.apache.kafka.clients.consumer.internals.events.FetchCommittedOffsetsApplicationEvent; @@ -80,11 +80,8 @@ import java.util.Properties; import java.util.Set; import java.util.SortedSet; -import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; @@ -869,11 +866,13 @@ public void testListenerCallbacksInvoke(List partitions = Collections.emptySortedSet(); @@ -1135,6 +1134,7 @@ public void testLongPollWaitIsLimited() { */ @Test public void testProcessBackgroundEventsWithInitialDelay() throws Exception { + consumer = newConsumer(); Time time = new MockTime(); Timer timer = time.timer(1000); CompletableFuture future = mock(CompletableFuture.class); @@ -1170,6 +1170,7 @@ public void testProcessBackgroundEventsWithInitialDelay() throws Exception { */ @Test public void testProcessBackgroundEventsWithoutDelay() { + consumer = newConsumer(); Time time = new MockTime(); Timer timer = time.timer(1000); @@ -1191,6 +1192,7 @@ public void testProcessBackgroundEventsWithoutDelay() { */ @Test public void testProcessBackgroundEventsTimesOut() throws Exception { + consumer = newConsumer(); Time time = new MockTime(); Timer timer = time.timer(1000); CompletableFuture future = mock(CompletableFuture.class); @@ -1209,10 +1211,6 @@ public void testProcessBackgroundEventsTimesOut() throws Exception { } } - private void assertNoPendingWakeup(final WakeupTrigger wakeupTrigger) { - assertNull(wakeupTrigger.getPendingTask()); - } - private HashMap mockTopicPartitionOffset() { final TopicPartition t0 = new TopicPartition("t0", 2); final TopicPartition t1 = new TopicPartition("t0", 3);