lianetm commented on code in PR #16449:
URL: https://github.com/apache/kafka/pull/16449#discussion_r1670733263


##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java:
##########
@@ -25,62 +25,98 @@
 import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate;
 import org.apache.kafka.clients.consumer.internals.OffsetsRequestManager;
 import org.apache.kafka.clients.consumer.internals.RequestManagers;
+import org.apache.kafka.clients.consumer.internals.SubscriptionState;
 import org.apache.kafka.clients.consumer.internals.TopicMetadataRequestManager;
 import org.apache.kafka.common.utils.LogContext;
 
-import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 
 import java.util.Collections;
 import java.util.List;
 import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
 
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 public class ApplicationEventProcessorTest {
     private final ConsumerMetadata metadata = mock(ConsumerMetadata.class);
     private ApplicationEventProcessor processor;
     private CommitRequestManager commitRequestManager;
     private HeartbeatRequestManager heartbeatRequestManager;
     private MembershipManager membershipManager;
+    private SubscriptionState subscriptionState;
 
-    @BeforeEach
-    public void setup() {
+    private void setupProcessor(boolean withGroupId) {
         LogContext logContext = new LogContext();
         OffsetsRequestManager offsetsRequestManager = 
mock(OffsetsRequestManager.class);
         TopicMetadataRequestManager topicMetadataRequestManager = 
mock(TopicMetadataRequestManager.class);
         FetchRequestManager fetchRequestManager = 
mock(FetchRequestManager.class);
-        CoordinatorRequestManager coordinatorRequestManager = 
mock(CoordinatorRequestManager.class);
-        commitRequestManager = mock(CommitRequestManager.class);
-        heartbeatRequestManager = mock(HeartbeatRequestManager.class);
-        membershipManager = mock(MembershipManager.class);
-        RequestManagers requestManagers = new RequestManagers(
-            logContext,
-            offsetsRequestManager,
-            topicMetadataRequestManager,
-            fetchRequestManager,
-            Optional.of(coordinatorRequestManager),
-            Optional.of(commitRequestManager),
-            Optional.of(heartbeatRequestManager),
-            Optional.of(membershipManager)
-        );
+        RequestManagers requestManagers;
+        if (withGroupId) {
+            CoordinatorRequestManager coordinatorRequestManager = 
mock(CoordinatorRequestManager.class);
+            commitRequestManager = mock(CommitRequestManager.class);
+            membershipManager = mock(MembershipManager.class);
+            heartbeatRequestManager = mock(HeartbeatRequestManager.class);
+            subscriptionState = mock(SubscriptionState.class);

Review Comment:
   we don't need these vars, so what about we remove them all and pass the mock 
directly to the requestManagers constructor? 
   
   Actually, we could consider simplifying more and reduce the whole block from 
ln 57 to ln 85 to a much shorter single path for creating the requestManagers:
   ```suggestion
           RequestManagers requestManagers = new RequestManagers(
               logContext,
               offsetsRequestManager,
               topicMetadataRequestManager,
               fetchRequestManager,
               withGroupId ? Optional.of(mock(CoordinatorRequestManager.class)) 
: Optional.empty(),
               withGroupId ? Optional.of(mock(CommitRequestManager.class)) : 
Optional.empty(),
               withGroupId ? Optional.of(mock(HeartbeatRequestManager.class)) : 
Optional.empty(),
               withGroupId ? Optional.of(mock(MembershipManager.class)) : 
Optional.empty());
   ```
   



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

Reply via email to