lucasbru commented on code in PR #20511:
URL: https://github.com/apache/kafka/pull/20511#discussion_r2339132075


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java:
##########
@@ -187,23 +187,14 @@ public class AsyncKafkaConsumer<K, V> implements 
ConsumerDelegate<K, V> {
      */
     private class BackgroundEventProcessor implements 
EventProcessor<BackgroundEvent> {
 
-        private Optional<StreamsRebalanceListener> streamsRebalanceListener = 
Optional.empty();
-        private final Optional<StreamsRebalanceData> streamsRebalanceData;
+        private final Optional<StreamsRebalanceListenerInvoker> 
streamsRebalanceListenerInvoker;
 
         public BackgroundEventProcessor() {
-            this.streamsRebalanceData = Optional.empty();
+            this.streamsRebalanceListenerInvoker = Optional.empty();
         }
 
-        public BackgroundEventProcessor(final Optional<StreamsRebalanceData> 
streamsRebalanceData) {
-            this.streamsRebalanceData = streamsRebalanceData;
-        }
-
-        private void setStreamsRebalanceListener(final 
StreamsRebalanceListener streamsRebalanceListener) {
-            if (streamsRebalanceData.isEmpty()) {
-                throw new IllegalStateException("Background event processor 
was not created to be used with Streams " +
-                    "rebalance protocol events");
-            }
-            this.streamsRebalanceListener = 
Optional.of(streamsRebalanceListener);
+        public BackgroundEventProcessor(final 
Optional<StreamsRebalanceListenerInvoker> streamsRebalanceListenerInvoker) {

Review Comment:
   Removed the constructor.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java:
##########
@@ -187,23 +187,14 @@ public class AsyncKafkaConsumer<K, V> implements 
ConsumerDelegate<K, V> {
      */
     private class BackgroundEventProcessor implements 
EventProcessor<BackgroundEvent> {
 
-        private Optional<StreamsRebalanceListener> streamsRebalanceListener = 
Optional.empty();
-        private final Optional<StreamsRebalanceData> streamsRebalanceData;
+        private final Optional<StreamsRebalanceListenerInvoker> 
streamsRebalanceListenerInvoker;

Review Comment:
   True. Removed the second field and used the reference in the outer class 
instead.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java:
##########
@@ -278,44 +269,28 @@ private void 
processStreamsOnAllTasksLostCallbackNeededEvent(final StreamsOnAllT
 
         private StreamsOnTasksRevokedCallbackCompletedEvent 
invokeOnTasksRevokedCallback(final Set<StreamsRebalanceData.TaskId> 
activeTasksToRevoke,
                                                                                
          final CompletableFuture<Void> future) {
-            final Optional<Exception> exceptionFromCallback = 
streamsRebalanceListener().onTasksRevoked(activeTasksToRevoke);
+            final Optional<Exception> exceptionFromCallback = 
Optional.ofNullable(streamsRebalanceListenerInvoker().invokeTasksRevoked(activeTasksToRevoke));
             final Optional<KafkaException> error = exceptionFromCallback.map(e 
-> ConsumerUtils.maybeWrapAsKafkaException(e, "Task revocation callback throws 
an error"));

Review Comment:
   This should work as expected. Indeed Optional.ofNullable will give 
Optional.empty for null, and the Optional.map will not be executed for 
Optional.empty.
   
   This is actually a suggested simplification by IntelliJ



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java:
##########
@@ -1964,7 +1957,11 @@ public void subscribe(Collection<String> topics, 
ConsumerRebalanceListener liste
 
     public void subscribe(Collection<String> topics, StreamsRebalanceListener 
streamsRebalanceListener) {
         subscribeInternal(topics, Optional.empty());
-        
backgroundEventProcessor.setStreamsRebalanceListener(streamsRebalanceListener);
+        if (streamsRebalanceListenerInvoker.isPresent()) {
+            
streamsRebalanceListenerInvoker.get().setRebalanceListener(streamsRebalanceListener);
+        } else {
+            throw new IllegalStateException("Consumer was not created to be 
used with Streams rebalance protocol events");
+        }

Review Comment:
   Done



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListenerInvokerTest.java:
##########
@@ -0,0 +1,337 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.consumer.internals;
+
+import org.apache.kafka.common.errors.InterruptException;
+import org.apache.kafka.common.errors.WakeupException;
+import org.apache.kafka.common.utils.LogContext;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+import org.mockito.junit.jupiter.MockitoSettings;
+import org.mockito.quality.Strictness;
+
+import java.util.Optional;
+import java.util.Set;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+@ExtendWith(MockitoExtension.class)
+@MockitoSettings(strictness = Strictness.STRICT_STUBS)
+public class StreamsRebalanceListenerInvokerTest {
+
+    @Mock
+    private StreamsRebalanceListener mockListener;
+
+    @Mock
+    private StreamsRebalanceData streamsRebalanceData;
+
+    private StreamsRebalanceListenerInvoker invoker;
+    private final LogContext logContext = new LogContext();
+
+    @BeforeEach
+    public void setup() {
+        invoker = new StreamsRebalanceListenerInvoker(logContext, 
streamsRebalanceData);
+    }
+
+    @Test
+    public void testConstructorInitializesWithEmptyListener() {
+        // When invoker is constructed, it should have no listener set 
initially
+        // This is verified by testing that invoke methods return null when no 
listener is present
+        assertNull(invoker.invokeAllTasksRevoked());
+        assertNull(invoker.invokeAllTasksLost());
+    }
+
+    @Test
+    public void testSetRebalanceListener() {
+        // Test setting a listener

Review Comment:
   Done



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListenerInvokerTest.java:
##########
@@ -0,0 +1,337 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.consumer.internals;
+
+import org.apache.kafka.common.errors.InterruptException;
+import org.apache.kafka.common.errors.WakeupException;
+import org.apache.kafka.common.utils.LogContext;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+import org.mockito.junit.jupiter.MockitoSettings;
+import org.mockito.quality.Strictness;
+
+import java.util.Optional;
+import java.util.Set;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+@ExtendWith(MockitoExtension.class)
+@MockitoSettings(strictness = Strictness.STRICT_STUBS)
+public class StreamsRebalanceListenerInvokerTest {
+
+    @Mock
+    private StreamsRebalanceListener mockListener;
+
+    @Mock
+    private StreamsRebalanceData streamsRebalanceData;
+
+    private StreamsRebalanceListenerInvoker invoker;
+    private final LogContext logContext = new LogContext();
+
+    @BeforeEach
+    public void setup() {
+        invoker = new StreamsRebalanceListenerInvoker(logContext, 
streamsRebalanceData);
+    }
+
+    @Test
+    public void testConstructorInitializesWithEmptyListener() {
+        // When invoker is constructed, it should have no listener set 
initially
+        // This is verified by testing that invoke methods return null when no 
listener is present
+        assertNull(invoker.invokeAllTasksRevoked());
+        assertNull(invoker.invokeAllTasksLost());
+    }
+
+    @Test
+    public void testSetRebalanceListener() {
+        // Test setting a listener
+        invoker.setRebalanceListener(mockListener);
+        
+        // Verify listener is set by checking that methods no longer return 
null immediately
+        // (we'll mock the dependencies needed for actual invocation)
+        StreamsRebalanceData.Assignment mockAssignment = 
createMockAssignment();
+        
when(streamsRebalanceData.reconciledAssignment()).thenReturn(mockAssignment);
+        when(mockListener.onTasksRevoked(any())).thenReturn(Optional.empty());
+        
+        // Should now invoke the listener instead of returning null immediately
+        Exception result = invoker.invokeAllTasksRevoked();
+        assertNull(result); // No exception thrown by mock listener
+        verify(mockListener).onTasksRevoked(eq(mockAssignment.activeTasks()));
+    }
+
+    @Test
+    public void testSetRebalanceListenerWithNull() {
+        // Test setting listener to null

Review Comment:
   Done



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java:
##########
@@ -278,44 +269,28 @@ private void 
processStreamsOnAllTasksLostCallbackNeededEvent(final StreamsOnAllT
 
         private StreamsOnTasksRevokedCallbackCompletedEvent 
invokeOnTasksRevokedCallback(final Set<StreamsRebalanceData.TaskId> 
activeTasksToRevoke,
                                                                                
          final CompletableFuture<Void> future) {
-            final Optional<Exception> exceptionFromCallback = 
streamsRebalanceListener().onTasksRevoked(activeTasksToRevoke);
+            final Optional<Exception> exceptionFromCallback = 
Optional.ofNullable(streamsRebalanceListenerInvoker().invokeTasksRevoked(activeTasksToRevoke));
             final Optional<KafkaException> error = exceptionFromCallback.map(e 
-> ConsumerUtils.maybeWrapAsKafkaException(e, "Task revocation callback throws 
an error"));

Review Comment:
   I think I disagree here, since when you look at `invokeRabalanceCallbacks`, 
we are doing it outside the constructors in the consumer case as well. I would 
prefer to keep this consistent. So doing it inside the event constructors in 
one case but not the other would seem inconsistent to me. If you insist, I 
would prefer changing it on both codepaths.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java:
##########
@@ -1964,7 +1957,11 @@ public void subscribe(Collection<String> topics, 
ConsumerRebalanceListener liste
 
     public void subscribe(Collection<String> topics, StreamsRebalanceListener 
streamsRebalanceListener) {
         subscribeInternal(topics, Optional.empty());
-        
backgroundEventProcessor.setStreamsRebalanceListener(streamsRebalanceListener);
+        if (streamsRebalanceListenerInvoker.isPresent()) {

Review Comment:
   Done



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListenerInvoker.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.consumer.internals;
+
+import org.apache.kafka.common.errors.InterruptException;
+import org.apache.kafka.common.errors.WakeupException;
+import org.apache.kafka.common.utils.LogContext;
+
+import org.slf4j.Logger;
+
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * This class encapsulates the invocation of the callback methods defined in 
the {@link StreamsRebalanceListener}
+ * interface. When streams group task assignment changes, these methods are 
invoked. This class wraps those
+ * callback calls with some logging and error handling.
+ */
+public class StreamsRebalanceListenerInvoker {
+
+    private final Logger log;
+
+    private final StreamsRebalanceData streamsRebalanceData;
+    private Optional<StreamsRebalanceListener> listener;

Review Comment:
   The thing is that the invoker is constructed with the consumer, but the 
listener is only registered with the `subscribe` call. So I'd rather handle the 
intermediate state - where we have a streams consumer but no listener yet - 
explicitly. But I get your point that we do not really have to allow this case. 
So I'm throwing illegal state exceptions now when the listener is invoked but 
not yet registered.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListenerInvoker.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.consumer.internals;
+
+import org.apache.kafka.common.errors.InterruptException;
+import org.apache.kafka.common.errors.WakeupException;
+import org.apache.kafka.common.utils.LogContext;
+
+import org.slf4j.Logger;
+
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * This class encapsulates the invocation of the callback methods defined in 
the {@link StreamsRebalanceListener}
+ * interface. When streams group task assignment changes, these methods are 
invoked. This class wraps those
+ * callback calls with some logging and error handling.
+ */
+public class StreamsRebalanceListenerInvoker {
+
+    private final Logger log;
+
+    private final StreamsRebalanceData streamsRebalanceData;
+    private Optional<StreamsRebalanceListener> listener;
+
+    StreamsRebalanceListenerInvoker(LogContext logContext, 
StreamsRebalanceData streamsRebalanceData) {
+        this.log = logContext.logger(getClass());
+        this.listener = Optional.empty();
+        this.streamsRebalanceData = streamsRebalanceData;
+    }
+
+    public void setRebalanceListener(StreamsRebalanceListener 
streamsRebalanceListener) {
+        this.listener = Optional.ofNullable(streamsRebalanceListener);
+    }
+
+    public Exception invokeAllTasksRevoked() {
+        if (listener.isPresent()) {
+            return 
invokeTasksRevoked(streamsRebalanceData.reconciledAssignment().activeTasks());
+        }
+
+        return null;
+    }
+
+    public Exception invokeTasksAssigned(final StreamsRebalanceData.Assignment 
assignment) {
+        if (listener.isPresent()) {
+            log.info("Adding newly assigned tasks: {}", assignment);

Review Comment:
   The consumer rebalance listener logs `Adding newly assigned partitions:` in 
this case, I just copied its behavior for consistency. But generally, I agree. 
   
   I updated the log line indicate that the rebalance listener is invoked.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java:
##########
@@ -278,44 +269,28 @@ private void 
processStreamsOnAllTasksLostCallbackNeededEvent(final StreamsOnAllT
 
         private StreamsOnTasksRevokedCallbackCompletedEvent 
invokeOnTasksRevokedCallback(final Set<StreamsRebalanceData.TaskId> 
activeTasksToRevoke,
                                                                                
          final CompletableFuture<Void> future) {
-            final Optional<Exception> exceptionFromCallback = 
streamsRebalanceListener().onTasksRevoked(activeTasksToRevoke);
+            final Optional<Exception> exceptionFromCallback = 
Optional.ofNullable(streamsRebalanceListenerInvoker().invokeTasksRevoked(activeTasksToRevoke));
             final Optional<KafkaException> error = exceptionFromCallback.map(e 
-> ConsumerUtils.maybeWrapAsKafkaException(e, "Task revocation callback throws 
an error"));
             return new StreamsOnTasksRevokedCallbackCompletedEvent(future, 
error);
         }
 
         private StreamsOnTasksAssignedCallbackCompletedEvent 
invokeOnTasksAssignedCallback(final StreamsRebalanceData.Assignment assignment,
                                                                                
            final CompletableFuture<Void> future) {
             final Optional<KafkaException> error;
-            final Optional<Exception> exceptionFromCallback = 
streamsRebalanceListener().onTasksAssigned(assignment);
-            if (exceptionFromCallback.isPresent()) {
-                error = 
Optional.of(ConsumerUtils.maybeWrapAsKafkaException(exceptionFromCallback.get(),
 "Task assignment callback throws an error"));
-            } else {
-                error = Optional.empty();
-                streamsRebalanceData().setReconciledAssignment(assignment);
-            }
+            final Optional<Exception> exceptionFromCallback = 
Optional.ofNullable(streamsRebalanceListenerInvoker().invokeTasksAssigned(assignment));
+            error = exceptionFromCallback.map(e -> 
ConsumerUtils.maybeWrapAsKafkaException(e, "Task assignment callback throws an 
error"));

Review Comment:
   See above.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java:
##########
@@ -1532,21 +1512,34 @@ private void runRebalanceCallbacksOnClose() {
 

Review Comment:
   Urgs. I think `rebalanceListenerInvoker` is never null. It's created even 
when no rebalanceListener is defined. I removed that part of the if condition, 
since it's confusing. 



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListenerInvoker.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.consumer.internals;
+
+import org.apache.kafka.common.errors.InterruptException;
+import org.apache.kafka.common.errors.WakeupException;
+import org.apache.kafka.common.utils.LogContext;
+
+import org.slf4j.Logger;
+
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * This class encapsulates the invocation of the callback methods defined in 
the {@link StreamsRebalanceListener}
+ * interface. When streams group task assignment changes, these methods are 
invoked. This class wraps those
+ * callback calls with some logging and error handling.
+ */
+public class StreamsRebalanceListenerInvoker {
+
+    private final Logger log;
+
+    private final StreamsRebalanceData streamsRebalanceData;
+    private Optional<StreamsRebalanceListener> listener;
+
+    StreamsRebalanceListenerInvoker(LogContext logContext, 
StreamsRebalanceData streamsRebalanceData) {
+        this.log = logContext.logger(getClass());
+        this.listener = Optional.empty();
+        this.streamsRebalanceData = streamsRebalanceData;
+    }
+
+    public void setRebalanceListener(StreamsRebalanceListener 
streamsRebalanceListener) {
+        this.listener = Optional.ofNullable(streamsRebalanceListener);

Review Comment:
   Throwing nullpointerException instead now.



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java:
##########
@@ -2210,6 +2210,75 @@ private void markOffsetsReadyForCommitEvent() {
         
}).when(applicationEventHandler).add(ArgumentMatchers.isA(CommitEvent.class));
     }
 
+    @Test
+    public void 
testCloseInvokesStreamsRebalanceListenerOnTasksRevokedWhenMemberEpochPositive() 
{
+        final String groupId = "streamsGroup";
+        final StreamsRebalanceData streamsRebalanceData = new 
StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of());
+        
+        try (final MockedStatic<RequestManagers> requestManagers = 
mockStatic(RequestManagers.class)) {
+            consumer = 
newConsumerWithStreamRebalanceData(requiredConsumerConfigAndGroupId(groupId), 
streamsRebalanceData);
+            StreamsRebalanceListener mockStreamsListener = 
mock(StreamsRebalanceListener.class);
+            
when(mockStreamsListener.onTasksRevoked(any())).thenReturn(Optional.empty());
+            consumer.subscribe(singletonList("topic"), mockStreamsListener);
+            final MemberStateListener groupMetadataUpdateListener = 
captureGroupMetadataUpdateListener(requestManagers);
+            final int memberEpoch = 42;
+            final String memberId = "memberId";
+            
groupMetadataUpdateListener.onMemberEpochUpdated(Optional.of(memberEpoch), 
memberId);
+            
+            consumer.close(CloseOptions.timeout(Duration.ZERO));
+            
+            verify(mockStreamsListener).onTasksRevoked(any());
+        }
+    }
+    
+    @Test
+    public void 
testCloseInvokesStreamsRebalanceListenerOnAllTasksLostWhenMemberEpochZeroOrNegative()
 {
+        // Test that close() calls 
streamsRebalanceListener.invokeAllTasksLost() when memberEpoch <= 0

Review Comment:
   Done



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListenerInvokerTest.java:
##########
@@ -0,0 +1,337 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.consumer.internals;
+
+import org.apache.kafka.common.errors.InterruptException;
+import org.apache.kafka.common.errors.WakeupException;
+import org.apache.kafka.common.utils.LogContext;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+import org.mockito.junit.jupiter.MockitoSettings;
+import org.mockito.quality.Strictness;
+
+import java.util.Optional;
+import java.util.Set;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+@ExtendWith(MockitoExtension.class)
+@MockitoSettings(strictness = Strictness.STRICT_STUBS)
+public class StreamsRebalanceListenerInvokerTest {
+
+    @Mock
+    private StreamsRebalanceListener mockListener;
+
+    @Mock
+    private StreamsRebalanceData streamsRebalanceData;
+
+    private StreamsRebalanceListenerInvoker invoker;
+    private final LogContext logContext = new LogContext();
+
+    @BeforeEach
+    public void setup() {
+        invoker = new StreamsRebalanceListenerInvoker(logContext, 
streamsRebalanceData);
+    }
+
+    @Test
+    public void testConstructorInitializesWithEmptyListener() {
+        // When invoker is constructed, it should have no listener set 
initially
+        // This is verified by testing that invoke methods return null when no 
listener is present
+        assertNull(invoker.invokeAllTasksRevoked());
+        assertNull(invoker.invokeAllTasksLost());
+    }
+
+    @Test
+    public void testSetRebalanceListener() {
+        // Test setting a listener
+        invoker.setRebalanceListener(mockListener);
+        
+        // Verify listener is set by checking that methods no longer return 
null immediately
+        // (we'll mock the dependencies needed for actual invocation)
+        StreamsRebalanceData.Assignment mockAssignment = 
createMockAssignment();
+        
when(streamsRebalanceData.reconciledAssignment()).thenReturn(mockAssignment);
+        when(mockListener.onTasksRevoked(any())).thenReturn(Optional.empty());
+        
+        // Should now invoke the listener instead of returning null immediately
+        Exception result = invoker.invokeAllTasksRevoked();
+        assertNull(result); // No exception thrown by mock listener
+        verify(mockListener).onTasksRevoked(eq(mockAssignment.activeTasks()));
+    }
+
+    @Test
+    public void testSetRebalanceListenerWithNull() {
+        // Test setting listener to null
+        invoker.setRebalanceListener(null);
+        
+        // Should behave as if no listener is set
+        assertNull(invoker.invokeAllTasksRevoked());
+        assertNull(invoker.invokeAllTasksLost());
+    }
+
+    @Test
+    public void testSetRebalanceListenerOverwritesExisting() {
+        StreamsRebalanceListener firstListener = 
org.mockito.Mockito.mock(StreamsRebalanceListener.class);
+        StreamsRebalanceListener secondListener = 
org.mockito.Mockito.mock(StreamsRebalanceListener.class);
+        
+        StreamsRebalanceData.Assignment mockAssignment = 
createMockAssignment();
+        
when(streamsRebalanceData.reconciledAssignment()).thenReturn(mockAssignment);
+        
when(secondListener.onTasksRevoked(any())).thenReturn(Optional.empty());
+        
+        // Set first listener
+        invoker.setRebalanceListener(firstListener);
+        
+        // Overwrite with second listener
+        invoker.setRebalanceListener(secondListener);
+        
+        // Should use second listener
+        invoker.invokeAllTasksRevoked();
+        verify(firstListener, never()).onTasksRevoked(any());
+        
verify(secondListener).onTasksRevoked(eq(mockAssignment.activeTasks()));
+    }
+
+    @Test
+    public void testInvokeAllTasksRevokedWithNoListener() {
+        // When no listener is set, should return null
+        Exception result = invoker.invokeAllTasksRevoked();
+        assertNull(result);
+    }
+
+    @Test
+    public void testInvokeAllTasksRevokedWithListener() {
+        invoker.setRebalanceListener(mockListener);
+        
+        StreamsRebalanceData.Assignment mockAssignment = 
createMockAssignment();
+        
when(streamsRebalanceData.reconciledAssignment()).thenReturn(mockAssignment);
+        when(mockListener.onTasksRevoked(any())).thenReturn(Optional.empty());
+        
+        Exception result = invoker.invokeAllTasksRevoked();
+        
+        assertNull(result);
+        verify(mockListener).onTasksRevoked(eq(mockAssignment.activeTasks()));
+    }

Review Comment:
   Done



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListenerInvokerTest.java:
##########
@@ -0,0 +1,337 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.consumer.internals;
+
+import org.apache.kafka.common.errors.InterruptException;
+import org.apache.kafka.common.errors.WakeupException;
+import org.apache.kafka.common.utils.LogContext;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+import org.mockito.junit.jupiter.MockitoSettings;
+import org.mockito.quality.Strictness;
+
+import java.util.Optional;
+import java.util.Set;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+@ExtendWith(MockitoExtension.class)
+@MockitoSettings(strictness = Strictness.STRICT_STUBS)
+public class StreamsRebalanceListenerInvokerTest {
+
+    @Mock
+    private StreamsRebalanceListener mockListener;
+
+    @Mock
+    private StreamsRebalanceData streamsRebalanceData;
+
+    private StreamsRebalanceListenerInvoker invoker;
+    private final LogContext logContext = new LogContext();
+
+    @BeforeEach
+    public void setup() {
+        invoker = new StreamsRebalanceListenerInvoker(logContext, 
streamsRebalanceData);
+    }
+
+    @Test
+    public void testConstructorInitializesWithEmptyListener() {
+        // When invoker is constructed, it should have no listener set 
initially
+        // This is verified by testing that invoke methods return null when no 
listener is present
+        assertNull(invoker.invokeAllTasksRevoked());
+        assertNull(invoker.invokeAllTasksLost());
+    }
+
+    @Test
+    public void testSetRebalanceListener() {
+        // Test setting a listener
+        invoker.setRebalanceListener(mockListener);
+        
+        // Verify listener is set by checking that methods no longer return 
null immediately
+        // (we'll mock the dependencies needed for actual invocation)
+        StreamsRebalanceData.Assignment mockAssignment = 
createMockAssignment();
+        
when(streamsRebalanceData.reconciledAssignment()).thenReturn(mockAssignment);
+        when(mockListener.onTasksRevoked(any())).thenReturn(Optional.empty());
+        
+        // Should now invoke the listener instead of returning null immediately
+        Exception result = invoker.invokeAllTasksRevoked();
+        assertNull(result); // No exception thrown by mock listener
+        verify(mockListener).onTasksRevoked(eq(mockAssignment.activeTasks()));
+    }
+
+    @Test
+    public void testSetRebalanceListenerWithNull() {
+        // Test setting listener to null
+        invoker.setRebalanceListener(null);
+        
+        // Should behave as if no listener is set
+        assertNull(invoker.invokeAllTasksRevoked());
+        assertNull(invoker.invokeAllTasksLost());
+    }
+
+    @Test
+    public void testSetRebalanceListenerOverwritesExisting() {
+        StreamsRebalanceListener firstListener = 
org.mockito.Mockito.mock(StreamsRebalanceListener.class);
+        StreamsRebalanceListener secondListener = 
org.mockito.Mockito.mock(StreamsRebalanceListener.class);

Review Comment:
   Done



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java:
##########
@@ -2210,6 +2210,75 @@ private void markOffsetsReadyForCommitEvent() {
         
}).when(applicationEventHandler).add(ArgumentMatchers.isA(CommitEvent.class));
     }
 
+    @Test
+    public void 
testCloseInvokesStreamsRebalanceListenerOnTasksRevokedWhenMemberEpochPositive() 
{
+        final String groupId = "streamsGroup";
+        final StreamsRebalanceData streamsRebalanceData = new 
StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of());
+        
+        try (final MockedStatic<RequestManagers> requestManagers = 
mockStatic(RequestManagers.class)) {
+            consumer = 
newConsumerWithStreamRebalanceData(requiredConsumerConfigAndGroupId(groupId), 
streamsRebalanceData);
+            StreamsRebalanceListener mockStreamsListener = 
mock(StreamsRebalanceListener.class);
+            
when(mockStreamsListener.onTasksRevoked(any())).thenReturn(Optional.empty());
+            consumer.subscribe(singletonList("topic"), mockStreamsListener);
+            final MemberStateListener groupMetadataUpdateListener = 
captureGroupMetadataUpdateListener(requestManagers);
+            final int memberEpoch = 42;
+            final String memberId = "memberId";
+            
groupMetadataUpdateListener.onMemberEpochUpdated(Optional.of(memberEpoch), 
memberId);
+            
+            consumer.close(CloseOptions.timeout(Duration.ZERO));
+            
+            verify(mockStreamsListener).onTasksRevoked(any());
+        }
+    }
+    
+    @Test
+    public void 
testCloseInvokesStreamsRebalanceListenerOnAllTasksLostWhenMemberEpochZeroOrNegative()
 {
+        // Test that close() calls 
streamsRebalanceListener.invokeAllTasksLost() when memberEpoch <= 0
+        final String groupId = "streamsGroup";
+        final StreamsRebalanceData streamsRebalanceData = new 
StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of());
+        
+        try (final MockedStatic<RequestManagers> requestManagers = 
mockStatic(RequestManagers.class)) {
+            consumer = 
newConsumerWithStreamRebalanceData(requiredConsumerConfigAndGroupId(groupId), 
streamsRebalanceData);
+            StreamsRebalanceListener mockStreamsListener = 
mock(StreamsRebalanceListener.class);
+            
when(mockStreamsListener.onAllTasksLost()).thenReturn(Optional.empty());
+            consumer.subscribe(singletonList("topic"), mockStreamsListener);
+            final MemberStateListener groupMetadataUpdateListener = 
captureGroupMetadataUpdateListener(requestManagers);
+            final int memberEpoch = 0;
+            final String memberId = "memberId";
+            
groupMetadataUpdateListener.onMemberEpochUpdated(Optional.of(memberEpoch), 
memberId);
+            
+            consumer.close(CloseOptions.timeout(Duration.ZERO));
+            
+            verify(mockStreamsListener).onAllTasksLost();
+        }
+    }
+    
+    @Test
+    public void testCloseWrapsStreamsRebalanceListenerException() {
+        final String groupId = "streamsGroup";
+        final StreamsRebalanceData streamsRebalanceData = new 
StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of());
+        
+        try (final MockedStatic<RequestManagers> requestManagers = 
mockStatic(RequestManagers.class)) {
+            consumer = 
newConsumerWithStreamRebalanceData(requiredConsumerConfigAndGroupId(groupId), 
streamsRebalanceData);
+            StreamsRebalanceListener mockStreamsListener = 
mock(StreamsRebalanceListener.class);
+            RuntimeException testException = new RuntimeException("Test 
streams listener exception");
+            
doThrow(testException).when(mockStreamsListener).onTasksRevoked(any());
+            consumer.subscribe(singletonList("topic"), mockStreamsListener);
+            final MemberStateListener groupMetadataUpdateListener = 
captureGroupMetadataUpdateListener(requestManagers);
+            final int memberEpoch = 1;
+            final String memberId = "memberId";
+            
groupMetadataUpdateListener.onMemberEpochUpdated(Optional.of(memberEpoch), 
memberId);
+            
+            KafkaException thrownException = 
assertThrows(KafkaException.class, 
+                () -> consumer.close(CloseOptions.timeout(Duration.ZERO)));
+            
+            assertNotNull(thrownException.getCause());
+            assertTrue(thrownException.getCause() instanceof RuntimeException);

Review Comment:
   Done



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListenerInvokerTest.java:
##########
@@ -0,0 +1,337 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.consumer.internals;
+
+import org.apache.kafka.common.errors.InterruptException;
+import org.apache.kafka.common.errors.WakeupException;
+import org.apache.kafka.common.utils.LogContext;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+import org.mockito.junit.jupiter.MockitoSettings;
+import org.mockito.quality.Strictness;
+
+import java.util.Optional;
+import java.util.Set;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+@ExtendWith(MockitoExtension.class)
+@MockitoSettings(strictness = Strictness.STRICT_STUBS)
+public class StreamsRebalanceListenerInvokerTest {
+
+    @Mock
+    private StreamsRebalanceListener mockListener;
+
+    @Mock
+    private StreamsRebalanceData streamsRebalanceData;
+
+    private StreamsRebalanceListenerInvoker invoker;
+    private final LogContext logContext = new LogContext();
+
+    @BeforeEach
+    public void setup() {
+        invoker = new StreamsRebalanceListenerInvoker(logContext, 
streamsRebalanceData);
+    }
+
+    @Test
+    public void testConstructorInitializesWithEmptyListener() {
+        // When invoker is constructed, it should have no listener set 
initially
+        // This is verified by testing that invoke methods return null when no 
listener is present
+        assertNull(invoker.invokeAllTasksRevoked());
+        assertNull(invoker.invokeAllTasksLost());
+    }
+
+    @Test
+    public void testSetRebalanceListener() {
+        // Test setting a listener
+        invoker.setRebalanceListener(mockListener);
+        
+        // Verify listener is set by checking that methods no longer return 
null immediately
+        // (we'll mock the dependencies needed for actual invocation)
+        StreamsRebalanceData.Assignment mockAssignment = 
createMockAssignment();
+        
when(streamsRebalanceData.reconciledAssignment()).thenReturn(mockAssignment);
+        when(mockListener.onTasksRevoked(any())).thenReturn(Optional.empty());
+        
+        // Should now invoke the listener instead of returning null immediately
+        Exception result = invoker.invokeAllTasksRevoked();
+        assertNull(result); // No exception thrown by mock listener
+        verify(mockListener).onTasksRevoked(eq(mockAssignment.activeTasks()));
+    }
+
+    @Test
+    public void testSetRebalanceListenerWithNull() {
+        // Test setting listener to null
+        invoker.setRebalanceListener(null);
+        
+        // Should behave as if no listener is set
+        assertNull(invoker.invokeAllTasksRevoked());
+        assertNull(invoker.invokeAllTasksLost());
+    }
+
+    @Test
+    public void testSetRebalanceListenerOverwritesExisting() {
+        StreamsRebalanceListener firstListener = 
org.mockito.Mockito.mock(StreamsRebalanceListener.class);
+        StreamsRebalanceListener secondListener = 
org.mockito.Mockito.mock(StreamsRebalanceListener.class);
+        
+        StreamsRebalanceData.Assignment mockAssignment = 
createMockAssignment();
+        
when(streamsRebalanceData.reconciledAssignment()).thenReturn(mockAssignment);
+        
when(secondListener.onTasksRevoked(any())).thenReturn(Optional.empty());
+        
+        // Set first listener
+        invoker.setRebalanceListener(firstListener);
+        
+        // Overwrite with second listener
+        invoker.setRebalanceListener(secondListener);
+        
+        // Should use second listener
+        invoker.invokeAllTasksRevoked();
+        verify(firstListener, never()).onTasksRevoked(any());
+        
verify(secondListener).onTasksRevoked(eq(mockAssignment.activeTasks()));
+    }
+
+    @Test
+    public void testInvokeAllTasksRevokedWithNoListener() {
+        // When no listener is set, should return null
+        Exception result = invoker.invokeAllTasksRevoked();
+        assertNull(result);
+    }

Review Comment:
   Done



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListenerInvokerTest.java:
##########
@@ -0,0 +1,337 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.consumer.internals;
+
+import org.apache.kafka.common.errors.InterruptException;
+import org.apache.kafka.common.errors.WakeupException;
+import org.apache.kafka.common.utils.LogContext;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+import org.mockito.junit.jupiter.MockitoSettings;
+import org.mockito.quality.Strictness;
+
+import java.util.Optional;
+import java.util.Set;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+@ExtendWith(MockitoExtension.class)
+@MockitoSettings(strictness = Strictness.STRICT_STUBS)
+public class StreamsRebalanceListenerInvokerTest {
+
+    @Mock
+    private StreamsRebalanceListener mockListener;
+
+    @Mock
+    private StreamsRebalanceData streamsRebalanceData;
+
+    private StreamsRebalanceListenerInvoker invoker;
+    private final LogContext logContext = new LogContext();
+
+    @BeforeEach
+    public void setup() {
+        invoker = new StreamsRebalanceListenerInvoker(logContext, 
streamsRebalanceData);
+    }
+
+    @Test
+    public void testConstructorInitializesWithEmptyListener() {
+        // When invoker is constructed, it should have no listener set 
initially
+        // This is verified by testing that invoke methods return null when no 
listener is present
+        assertNull(invoker.invokeAllTasksRevoked());
+        assertNull(invoker.invokeAllTasksLost());
+    }
+
+    @Test
+    public void testSetRebalanceListener() {
+        // Test setting a listener
+        invoker.setRebalanceListener(mockListener);
+        
+        // Verify listener is set by checking that methods no longer return 
null immediately
+        // (we'll mock the dependencies needed for actual invocation)
+        StreamsRebalanceData.Assignment mockAssignment = 
createMockAssignment();
+        
when(streamsRebalanceData.reconciledAssignment()).thenReturn(mockAssignment);
+        when(mockListener.onTasksRevoked(any())).thenReturn(Optional.empty());
+        
+        // Should now invoke the listener instead of returning null immediately
+        Exception result = invoker.invokeAllTasksRevoked();
+        assertNull(result); // No exception thrown by mock listener
+        verify(mockListener).onTasksRevoked(eq(mockAssignment.activeTasks()));
+    }
+
+    @Test
+    public void testSetRebalanceListenerWithNull() {
+        // Test setting listener to null
+        invoker.setRebalanceListener(null);
+        
+        // Should behave as if no listener is set
+        assertNull(invoker.invokeAllTasksRevoked());
+        assertNull(invoker.invokeAllTasksLost());
+    }
+
+    @Test
+    public void testSetRebalanceListenerOverwritesExisting() {
+        StreamsRebalanceListener firstListener = 
org.mockito.Mockito.mock(StreamsRebalanceListener.class);
+        StreamsRebalanceListener secondListener = 
org.mockito.Mockito.mock(StreamsRebalanceListener.class);
+        
+        StreamsRebalanceData.Assignment mockAssignment = 
createMockAssignment();
+        
when(streamsRebalanceData.reconciledAssignment()).thenReturn(mockAssignment);
+        
when(secondListener.onTasksRevoked(any())).thenReturn(Optional.empty());
+        
+        // Set first listener
+        invoker.setRebalanceListener(firstListener);
+        
+        // Overwrite with second listener
+        invoker.setRebalanceListener(secondListener);
+        
+        // Should use second listener
+        invoker.invokeAllTasksRevoked();
+        verify(firstListener, never()).onTasksRevoked(any());
+        
verify(secondListener).onTasksRevoked(eq(mockAssignment.activeTasks()));
+    }
+
+    @Test
+    public void testInvokeAllTasksRevokedWithNoListener() {
+        // When no listener is set, should return null
+        Exception result = invoker.invokeAllTasksRevoked();
+        assertNull(result);
+    }
+
+    @Test
+    public void testInvokeAllTasksRevokedWithListener() {
+        invoker.setRebalanceListener(mockListener);
+        
+        StreamsRebalanceData.Assignment mockAssignment = 
createMockAssignment();
+        
when(streamsRebalanceData.reconciledAssignment()).thenReturn(mockAssignment);
+        when(mockListener.onTasksRevoked(any())).thenReturn(Optional.empty());
+        
+        Exception result = invoker.invokeAllTasksRevoked();
+        
+        assertNull(result);
+        verify(mockListener).onTasksRevoked(eq(mockAssignment.activeTasks()));
+    }
+
+    @Test
+    public void testInvokeTasksAssignedWithNoListener() {
+        StreamsRebalanceData.Assignment assignment = createMockAssignment();
+        
+        Exception result = invoker.invokeTasksAssigned(assignment);
+        
+        assertNull(result);
+        verify(mockListener, never()).onTasksAssigned(any());
+    }
+
+    @Test
+    public void testInvokeTasksAssignedWithListener() {
+        invoker.setRebalanceListener(mockListener);
+        StreamsRebalanceData.Assignment assignment = createMockAssignment();
+        
when(mockListener.onTasksAssigned(assignment)).thenReturn(Optional.empty());
+        
+        Exception result = invoker.invokeTasksAssigned(assignment);
+        
+        assertNull(result);
+        verify(mockListener).onTasksAssigned(eq(assignment));
+    }
+
+    @Test
+    public void testInvokeTasksAssignedWithWakeupException() {
+        invoker.setRebalanceListener(mockListener);
+        StreamsRebalanceData.Assignment assignment = createMockAssignment();
+        WakeupException wakeupException = new WakeupException();
+        
doThrow(wakeupException).when(mockListener).onTasksAssigned(assignment);
+        
+        WakeupException thrownException = assertThrows(WakeupException.class, 
+            () -> invoker.invokeTasksAssigned(assignment));
+        
+        assertEquals(wakeupException, thrownException);
+        verify(mockListener).onTasksAssigned(eq(assignment));
+    }
+
+    @Test
+    public void testInvokeTasksAssignedWithInterruptException() {
+        invoker.setRebalanceListener(mockListener);
+        StreamsRebalanceData.Assignment assignment = createMockAssignment();
+        InterruptException interruptException = new InterruptException("Test 
interrupt");
+        
doThrow(interruptException).when(mockListener).onTasksAssigned(assignment);
+        
+        InterruptException thrownException = 
assertThrows(InterruptException.class, 
+            () -> invoker.invokeTasksAssigned(assignment));
+        
+        assertEquals(interruptException, thrownException);
+        verify(mockListener).onTasksAssigned(eq(assignment));
+    }
+
+    @Test
+    public void testInvokeTasksAssignedWithOtherException() {
+        invoker.setRebalanceListener(mockListener);
+        StreamsRebalanceData.Assignment assignment = createMockAssignment();
+        RuntimeException runtimeException = new RuntimeException("Test 
exception");
+        
doThrow(runtimeException).when(mockListener).onTasksAssigned(assignment);
+        
+        Exception result = invoker.invokeTasksAssigned(assignment);
+        
+        assertEquals(runtimeException, result);
+        verify(mockListener).onTasksAssigned(eq(assignment));
+    }
+
+    @Test
+    public void testInvokeTasksRevokedWithNoListener() {
+        Set<StreamsRebalanceData.TaskId> tasks = createMockTasks();
+        
+        Exception result = invoker.invokeTasksRevoked(tasks);
+        
+        assertNull(result);
+        verify(mockListener, never()).onTasksRevoked(any());
+    }
+
+    @Test
+    public void testInvokeTasksRevokedWithListener() {
+        invoker.setRebalanceListener(mockListener);
+        Set<StreamsRebalanceData.TaskId> tasks = createMockTasks();
+        when(mockListener.onTasksRevoked(tasks)).thenReturn(Optional.empty());
+        
+        Exception result = invoker.invokeTasksRevoked(tasks);
+        
+        assertNull(result);
+        verify(mockListener).onTasksRevoked(eq(tasks));
+    }
+
+    @Test
+    public void testInvokeTasksRevokedWithWakeupException() {
+        invoker.setRebalanceListener(mockListener);
+        Set<StreamsRebalanceData.TaskId> tasks = createMockTasks();
+        WakeupException wakeupException = new WakeupException();
+        doThrow(wakeupException).when(mockListener).onTasksRevoked(tasks);
+        
+        WakeupException thrownException = assertThrows(WakeupException.class, 
+            () -> invoker.invokeTasksRevoked(tasks));
+        
+        assertEquals(wakeupException, thrownException);
+        verify(mockListener).onTasksRevoked(eq(tasks));
+    }
+
+    @Test
+    public void testInvokeTasksRevokedWithInterruptException() {
+        invoker.setRebalanceListener(mockListener);
+        Set<StreamsRebalanceData.TaskId> tasks = createMockTasks();
+        InterruptException interruptException = new InterruptException("Test 
interrupt");
+        doThrow(interruptException).when(mockListener).onTasksRevoked(tasks);
+        
+        InterruptException thrownException = 
assertThrows(InterruptException.class, 
+            () -> invoker.invokeTasksRevoked(tasks));
+        
+        assertEquals(interruptException, thrownException);
+        verify(mockListener).onTasksRevoked(eq(tasks));
+    }
+
+    @Test
+    public void testInvokeTasksRevokedWithOtherException() {
+        invoker.setRebalanceListener(mockListener);
+        Set<StreamsRebalanceData.TaskId> tasks = createMockTasks();
+        RuntimeException runtimeException = new RuntimeException("Test 
exception");
+        doThrow(runtimeException).when(mockListener).onTasksRevoked(tasks);
+        
+        Exception result = invoker.invokeTasksRevoked(tasks);
+        
+        assertEquals(runtimeException, result);
+        verify(mockListener).onTasksRevoked(eq(tasks));
+    }
+
+    @Test
+    public void testInvokeAllTasksLostWithNoListener() {

Review Comment:
   Done



-- 
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]

Reply via email to