kirktrue commented on code in PR #16686:
URL: https://github.com/apache/kafka/pull/16686#discussion_r1835191016
##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractMembershipManager.java:
##########
@@ -1129,6 +1168,12 @@ private CompletableFuture<Void> assignPartitions(
if (exception == null) {
// Enable newly added partitions to start fetching and
updating positions for them.
subscriptions.enablePartitionsAwaitingCallback(addedPartitions);
+
+ Set<TopicPartition> allAssignedPartitions =
assignedPartitions.stream()
+ .map(tip -> new TopicPartition(tip.topic(),
tip.partition()))
+ .collect(Collectors.toSet());
+
+ notifyAssignmentChange(allAssignedPartitions);
Review Comment:
Moved `notifyAssignmentChange()` to right after
`subscriptions.assignFromSubscribedAwaitingCallback()` in
`updateSubscriptionAwaitingCallback()`.
##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java:
##########
@@ -653,27 +655,67 @@ public void testUnsubscribeOnClose() {
false));
completeUnsubscribeApplicationEventSuccessfully();
consumer.close(Duration.ZERO);
- verifyUnsubscribeEvent(subscriptions);
+
verify(applicationEventHandler).addAndGet(any(LeaveGroupOnCloseEvent.class));
}
@Test
- public void testFailedPartitionRevocationOnClose() {
+ public void testCloseLeavesGroupDespiteOnPartitionsLostError() {
// If rebalance listener failed to execute during close, we still send
the leave group,
// and proceed with closing the consumer.
+ Throwable rootError = new KafkaException("Intentional error");
+ Set<TopicPartition> partitions = singleton(new
TopicPartition("topic1", 0));
+
SubscriptionState subscriptions = mock(SubscriptionState.class);
+ when(subscriptions.assignedPartitions()).thenReturn(partitions);
+ ConsumerRebalanceListenerInvoker invoker =
mock(ConsumerRebalanceListenerInvoker.class);
+ doAnswer(invocation ->
rootError).when(invoker).invokePartitionsLost(any(SortedSet.class));
+
consumer = spy(newConsumer(
mock(FetchBuffer.class),
new ConsumerInterceptors<>(Collections.emptyList()),
+ invoker,
+ subscriptions,
+ "group-id",
+ "client-id",
+ false));
+ consumer.setGroupAssignmentSnapshot(partitions);
+
+ Throwable t = assertThrows(KafkaException.class, () ->
consumer.close(Duration.ZERO));
+ assertNotNull(t.getCause());
+ assertEquals(rootError, t.getCause());
+
+ verify(applicationEventHandler).add(any(CommitOnCloseEvent.class));
+
verify(applicationEventHandler).addAndGet(any(LeaveGroupOnCloseEvent.class));
+ }
+
+ @ParameterizedTest
+ @ValueSource(longs = {0, ConsumerUtils.DEFAULT_CLOSE_TIMEOUT_MS})
+ public void testCloseLeavesGroupDespiteInterrupt(long timeoutMs) {
+ Set<TopicPartition> partitions = singleton(new
TopicPartition("topic1", 0));
+
+ SubscriptionState subscriptions = mock(SubscriptionState.class);
+ when(subscriptions.assignedPartitions()).thenReturn(partitions);
+
when(applicationEventHandler.addAndGet(any(CompletableApplicationEvent.class))).thenThrow(InterruptException.class);
+ consumer = spy(newConsumer(
+ mock(FetchBuffer.class),
+ mock(ConsumerInterceptors.class),
mock(ConsumerRebalanceListenerInvoker.class),
subscriptions,
"group-id",
"client-id",
false));
- doThrow(new
KafkaException()).when(consumer).processBackgroundEvents(any(), any(), any());
- assertThrows(KafkaException.class, () ->
consumer.close(Duration.ZERO));
- verifyUnsubscribeEvent(subscriptions);
- // Close operation should carry on even if the unsubscribe fails
- verify(applicationEventHandler).close(any(Duration.class));
+
+ Duration timeout = Duration.ofMillis(timeoutMs);
+
+ try {
+ Thread.currentThread().interrupt();
Review Comment:
Good call. Removed.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]