UladzislauBlok commented on code in PR #20511: URL: https://github.com/apache/kafka/pull/20511#discussion_r2337336868
########## 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: minor: full import? ########## 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: out of curiosity: why this is optional, if we anyway we'll throw exception at line 292 in case of null? ########## 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); + try { + listener.get().onTasksAssigned(assignment); + } catch (WakeupException | InterruptException e) { Review Comment: No interruption reset? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org