Re: [PR] KAFKA-16000 Migrated MembershipManagerImplTest away from ConsumerTestBuilder [kafka]

2024-06-24 Thread via GitHub


philipnee commented on code in PR #16312:
URL: https://github.com/apache/kafka/pull/16312#discussion_r1651126643


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java:
##
@@ -1161,8 +1155,9 @@ public void 
testNewAssignmentReplacesPreviousOneWaitingOnMetadata() {
 membershipManager.poll(time.milliseconds());
 
 Set expectedAssignment = Collections.singleton(new 
TopicPartition(topicName, 0));
+HashSet expectedSet = new 
HashSet<>(expectedAssignment);

Review Comment:
   hey why exactly do we need this?



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



Re: [PR] KAFKA-16000 Migrated MembershipManagerImplTest away from ConsumerTestBuilder [kafka]

2024-06-24 Thread via GitHub


philipnee commented on code in PR #16312:
URL: https://github.com/apache/kafka/pull/16312#discussion_r1651126643


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java:
##
@@ -1161,8 +1155,9 @@ public void 
testNewAssignmentReplacesPreviousOneWaitingOnMetadata() {
 membershipManager.poll(time.milliseconds());
 
 Set expectedAssignment = Collections.singleton(new 
TopicPartition(topicName, 0));
+HashSet expectedSet = new 
HashSet<>(expectedAssignment);

Review Comment:
   `Set` instead



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



Re: [PR] KAFKA-16000 Migrated MembershipManagerImplTest away from ConsumerTestBuilder [kafka]

2024-06-22 Thread via GitHub


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


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java:
##
@@ -747,7 +741,7 @@ public void 
testDelayedReconciliationResultAppliedWhenTargetChangedWithMetadataU
 membershipManager.poll(time.milliseconds());
 
 assertEquals(Collections.emptySet(), 
membershipManager.topicsAwaitingReconciliation());
-
verify(subscriptionState).assignFromSubscribed(topicPartitions(topic2Assignment,
 topic2Metadata));
+
verify(subscriptionState).assignFromSubscribedAwaitingCallback(eq(topicPartitions(topic2Assignment,
 topic2Metadata)), eq(topicPartitions(topic2Assignment, topic2Metadata)));

Review Comment:
   is there a reason to use the `eq` here, that we're providing the to specific 
arguments? 



##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java:
##
@@ -2489,7 +2490,9 @@ private void 
testRevocationCompleted(MembershipManagerImpl membershipManager,
 verify(subscriptionState).markPendingRevocation(anySet());
 List expectedTopicPartitionAssignment =
 buildTopicPartitions(expectedCurrentAssignment);
-verify(subscriptionState).assignFromSubscribed(new 
HashSet<>(expectedTopicPartitionAssignment));
+HashSet expectedSet = new 
HashSet<>(expectedTopicPartitionAssignment);
+HashSet emptySet = new HashSet<>();

Review Comment:
   we could simplify removing this and just reference `Collections.emptySet()` 
on the 2nd param of the assignFromSubscribedAwaitingCallback



##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java:
##
@@ -747,7 +741,7 @@ public void 
testDelayedReconciliationResultAppliedWhenTargetChangedWithMetadataU
 membershipManager.poll(time.milliseconds());
 
 assertEquals(Collections.emptySet(), 
membershipManager.topicsAwaitingReconciliation());
-
verify(subscriptionState).assignFromSubscribed(topicPartitions(topic2Assignment,
 topic2Metadata));
+
verify(subscriptionState).assignFromSubscribedAwaitingCallback(eq(topicPartitions(topic2Assignment,
 topic2Metadata)), eq(topicPartitions(topic2Assignment, topic2Metadata)));

Review Comment:
   if we don't need it let's remove it in other places when possible



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



Re: [PR] KAFKA-16000 Migrated MembershipManagerImplTest away from ConsumerTestBuilder [kafka]

2024-06-22 Thread via GitHub


brenden20 commented on PR #16312:
URL: https://github.com/apache/kafka/pull/16312#issuecomment-2183474544

   Thank you for the feedback @lianetm, I just made all the suggested 
improvements!


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



Re: [PR] KAFKA-16000 Migrated MembershipManagerImplTest away from ConsumerTestBuilder [kafka]

2024-06-21 Thread via GitHub


brenden20 commented on PR #16312:
URL: https://github.com/apache/kafka/pull/16312#issuecomment-2177253091

   @lianetm thank you for the feedback! I have implemented all suggestions, it 
is looking really good now I think!


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



Re: [PR] KAFKA-16000 Migrated MembershipManagerImplTest away from ConsumerTestBuilder [kafka]

2024-06-21 Thread via GitHub


brenden20 commented on code in PR #16312:
URL: https://github.com/apache/kafka/pull/16312#discussion_r1645190334


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java:
##
@@ -366,12 +384,12 @@ public void testFencingWhenStateIsPrepareLeaving() {
 completeCallback(callbackEvent, membershipManager);
 assertEquals(MemberState.UNSUBSCRIBED, membershipManager.state());
 assertEquals(ConsumerGroupHeartbeatRequest.LEAVE_GROUP_MEMBER_EPOCH, 
membershipManager.memberEpoch());
-verify(membershipManager).notifyEpochChange(Optional.empty(), 
Optional.empty());

Review Comment:
   Makes sense, I have reverted that change



##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java:
##
@@ -386,15 +404,16 @@ public void 
testNewAssignmentIgnoredWhenStateIsPrepareLeaving() {
 receiveAssignment(topicId, Arrays.asList(0, 1), membershipManager);
 assertEquals(MemberState.PREPARE_LEAVING, membershipManager.state());
 assertTrue(membershipManager.topicsAwaitingReconciliation().isEmpty());
-verify(membershipManager, never()).markReconciliationInProgress();
 
 // When callback completes member should transition to LEAVING.
 completeCallback(callbackEvent, membershipManager);
+membershipManager.transitionToSendingLeaveGroup(false);

Review Comment:
   Removed now



##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java:
##
@@ -2457,10 +2538,7 @@ private CompletableFuture 
mockRevocationNoCallbacks(boolean withAutoCommit
 doNothing().when(subscriptionState).markPendingRevocation(anySet());
 
when(subscriptionState.rebalanceListener()).thenReturn(Optional.empty()).thenReturn(Optional.empty());
 if (withAutoCommit) {
-when(commitRequestManager.autoCommitEnabled()).thenReturn(true);
-CompletableFuture commitResult = new CompletableFuture<>();
-
when(commitRequestManager.maybeAutoCommitSyncBeforeRevocation(anyLong())).thenReturn(commitResult);
-return commitResult;

Review Comment:
   I have reverted changes here



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



Re: [PR] KAFKA-16000 Migrated MembershipManagerImplTest away from ConsumerTestBuilder [kafka]

2024-06-18 Thread via GitHub


lianetm commented on PR #16312:
URL: https://github.com/apache/kafka/pull/16312#issuecomment-2176982668

   Hey @brenden20, thanks for the changes! Took a first pass, left a few 
comments. I see some other calls around tests blocked on commits that seem 
confusing but I expect they may all align better if we review how we use the 
commit manager and are able to simplify to a mock maybe. Thanks!


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



Re: [PR] KAFKA-16000 Migrated MembershipManagerImplTest away from ConsumerTestBuilder [kafka]

2024-06-18 Thread via GitHub


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


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java:
##
@@ -2457,10 +2538,7 @@ private CompletableFuture 
mockRevocationNoCallbacks(boolean withAutoCommit
 doNothing().when(subscriptionState).markPendingRevocation(anySet());
 
when(subscriptionState.rebalanceListener()).thenReturn(Optional.empty()).thenReturn(Optional.empty());
 if (withAutoCommit) {
-when(commitRequestManager.autoCommitEnabled()).thenReturn(true);
-CompletableFuture commitResult = new CompletableFuture<>();
-
when(commitRequestManager.maybeAutoCommitSyncBeforeRevocation(anyLong())).thenReturn(commitResult);
-return commitResult;

Review Comment:
   uhm removing this will make that in all tests that want to block on commit 
we will have to call the same 
`when(commitRequestManager.maybeAutoCommitSyncBeforeRevocation...` we had here 
I guess? I do see that you had to add this line in many places in the end. 
Again, seems related to how we're using an insntace of commitRequestManager 
mentioned in comments above. (and if we are able to move to a mocked 
commitManager this should probably stay?)  



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



Re: [PR] KAFKA-16000 Migrated MembershipManagerImplTest away from ConsumerTestBuilder [kafka]

2024-06-18 Thread via GitHub


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


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java:
##
@@ -141,23 +136,41 @@ private MembershipManagerImpl 
createMembershipManagerJoiningGroup(String groupIn
 }
 
 private MembershipManagerImpl createMembershipManager(String 
groupInstanceId) {
-return spy(new MembershipManagerImpl(
+return new MembershipManagerImpl(
 GROUP_ID, Optional.ofNullable(groupInstanceId), REBALANCE_TIMEOUT, 
Optional.empty(),
 subscriptionState, commitRequestManager, metadata, logContext, 
Optional.empty(),
-backgroundEventHandler, time, rebalanceMetricsManager));
+backgroundEventHandler, time, rebalanceMetricsManager);
 }
 
 private MembershipManagerImpl createMembershipManagerJoiningGroup(String 
groupInstanceId,
   String 
serverAssignor) {
-MembershipManagerImpl manager = spy(new MembershipManagerImpl(
+MembershipManagerImpl manager = new MembershipManagerImpl(
 GROUP_ID, Optional.ofNullable(groupInstanceId), 
REBALANCE_TIMEOUT,
 Optional.ofNullable(serverAssignor), subscriptionState, 
commitRequestManager,
 metadata, logContext, Optional.empty(), 
backgroundEventHandler, time,
-rebalanceMetricsManager));
+rebalanceMetricsManager);
 manager.transitionToJoining();
 return manager;
 }
 
+private void createCommitRequestManager(boolean autoCommit) {
+ConsumerConfig config = mock(ConsumerConfig.class);
+if (autoCommit) {
+
when(config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)).thenReturn(true);
+}
+
+commitRequestManager = new CommitRequestManager(

Review Comment:
   I would aim for not having an actual commitRequestManager here, a mock 
should do, but I may be missing something. Check my other comments to get rid 
of the `createCommitRequestManager` (and maybe we can end up removing this?)



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



Re: [PR] KAFKA-16000 Migrated MembershipManagerImplTest away from ConsumerTestBuilder [kafka]

2024-06-18 Thread via GitHub


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


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java:
##
@@ -81,53 +82,47 @@
 import static org.mockito.ArgumentMatchers.anyCollection;
 import static org.mockito.ArgumentMatchers.anyLong;
 import static org.mockito.ArgumentMatchers.anySet;
+import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.Mockito.clearInvocations;
 import static org.mockito.Mockito.doNothing;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+@SuppressWarnings("ClassDataAbstractionCoupling")
 public class MembershipManagerImplTest {
 
 private static final String GROUP_ID = "test-group";
 private static final String MEMBER_ID = "test-member-1";
 private static final int REBALANCE_TIMEOUT = 100;
 private static final int MEMBER_EPOCH = 1;
 
-private final LogContext logContext = new LogContext();
+private LogContext logContext;
 private SubscriptionState subscriptionState;
 private ConsumerMetadata metadata;
-
 private CommitRequestManager commitRequestManager;
-
-private ConsumerTestBuilder testBuilder;
 private BlockingQueue backgroundEventQueue;
 private BackgroundEventHandler backgroundEventHandler;
 private Time time;
 private Metrics metrics;
 private RebalanceMetricsManager rebalanceMetricsManager;
 
+@SuppressWarnings("unchecked")
 @BeforeEach
 public void setup() {
-testBuilder = new 
ConsumerTestBuilder(ConsumerTestBuilder.createDefaultGroupInformation());
-metadata = testBuilder.metadata;
-subscriptionState = testBuilder.subscriptions;
-commitRequestManager = 
testBuilder.commitRequestManager.orElseThrow(IllegalStateException::new);
-backgroundEventQueue = testBuilder.backgroundEventQueue;
-backgroundEventHandler = testBuilder.backgroundEventHandler;
+metadata = mock(ConsumerMetadata.class);
+subscriptionState = mock(SubscriptionState.class);
+commitRequestManager = mock(CommitRequestManager.class);
+backgroundEventHandler = mock(BackgroundEventHandler.class);
+backgroundEventQueue = mock(BlockingQueue.class);

Review Comment:
   heads-up, having these as mocks means we cannot retrieve the events like we 
used to (backgroundEventQueue.poll), that I notice is still used to get the 
event to `performCallback`. I would say having it as a mock is the right thing 
to do, but we need to fix how we get the events from it. Ex. 
[this](https://github.com/apache/kafka/blob/53ec9733c2bf8b122fbd193d8d00b8e25cbd648f/clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java#L2165-L2169)
 section to get the callback event to complete should be updated to retrieve 
the event with an argument captor instead (similar to how it's done 
[here](https://github.com/apache/kafka/blob/f2a552a1ebaa0ce933f90343655b63e2472856d9/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java#L807-L809),
 but retrieving a `ConsumerRebalanceListenerCallbackNeededEvent`)



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



Re: [PR] KAFKA-16000 Migrated MembershipManagerImplTest away from ConsumerTestBuilder [kafka]

2024-06-18 Thread via GitHub


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


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java:
##
@@ -696,21 +721,23 @@ public void 
testDelayedReconciliationResultAppliedWhenTargetChangedWithMetadataU
 // Member receives and reconciles topic1-partition0
 Uuid topicId1 = Uuid.randomUuid();
 String topic1 = "topic1";
+
when(commitRequestManager.maybeAutoCommitSyncBeforeRevocation(anyLong())).thenReturn(CompletableFuture.completedFuture(null));
 MembershipManagerImpl membershipManager =
 mockMemberSuccessfullyReceivesAndAcksAssignment(topicId1, 
topic1, Collections.singletonList(0));
 membershipManager.onHeartbeatRequestSent();
 assertEquals(MemberState.STABLE, membershipManager.state());
-clearInvocations(membershipManager, subscriptionState);
+//clearInvocations(membershipManager, subscriptionState);

Review Comment:
   let's remove the comment if we don't need the line anymore



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



Re: [PR] KAFKA-16000 Migrated MembershipManagerImplTest away from ConsumerTestBuilder [kafka]

2024-06-18 Thread via GitHub


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


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java:
##
@@ -660,19 +688,16 @@ public void 
testSameAssignmentReconciledAgainWithMissingTopic() {
 );
 membershipManager.onHeartbeatRequestSent();
 assertEquals(MemberState.RECONCILING, membershipManager.state());
-clearInvocations(membershipManager);
 
 // Receive extended assignment - assignment received but no 
reconciliation triggered
 
membershipManager.onHeartbeatSuccess(createConsumerGroupHeartbeatResponse(assignment2).data());
 assertEquals(MemberState.RECONCILING, membershipManager.state());
-verifyReconciliationNotTriggered(membershipManager);

Review Comment:
   similar to comment above, I would keep this kind of verification (spying on 
the membershipMgr only). The membership mgr holds the state machine for the 
member as part of a group, it's all intertwined, we've seen how small changes 
in one membership func affects/breaks states and transitions, so all coverage 
we can maintain here is important. 



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



Re: [PR] KAFKA-16000 Migrated MembershipManagerImplTest away from ConsumerTestBuilder [kafka]

2024-06-18 Thread via GitHub


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


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java:
##
@@ -386,15 +404,16 @@ public void 
testNewAssignmentIgnoredWhenStateIsPrepareLeaving() {
 receiveAssignment(topicId, Arrays.asList(0, 1), membershipManager);
 assertEquals(MemberState.PREPARE_LEAVING, membershipManager.state());
 assertTrue(membershipManager.topicsAwaitingReconciliation().isEmpty());
-verify(membershipManager, never()).markReconciliationInProgress();
 
 // When callback completes member should transition to LEAVING.
 completeCallback(callbackEvent, membershipManager);
+membershipManager.transitionToSendingLeaveGroup(false);

Review Comment:
   uhm we shouldn't call this here ourselves. The test wants to ensure that 
when the callbacks complete, the membershipMgr internally calls this 
`transitionToSendingLeaveGroup`. If for some reason the transition to leaving 
was not happening we should review what's missing/failing that does not let the 
flow make it to 
[this](https://github.com/apache/kafka/blob/f2a552a1ebaa0ce933f90343655b63e2472856d9/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java#L695)
 point, where the internal call to transitionToSendingLeaveGroup should happen. 



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



Re: [PR] KAFKA-16000 Migrated MembershipManagerImplTest away from ConsumerTestBuilder [kafka]

2024-06-18 Thread via GitHub


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


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java:
##
@@ -366,12 +384,12 @@ public void testFencingWhenStateIsPrepareLeaving() {
 completeCallback(callbackEvent, membershipManager);
 assertEquals(MemberState.UNSUBSCRIBED, membershipManager.state());
 assertEquals(ConsumerGroupHeartbeatRequest.LEAVE_GROUP_MEMBER_EPOCH, 
membershipManager.memberEpoch());
-verify(membershipManager).notifyEpochChange(Optional.empty(), 
Optional.empty());

Review Comment:
   Even though I totally agree with removing all the spies there were on other 
components, I would lean towards spying only on this membershipManager 
component in cases like this, where it seems valuable to verify on it. It would 
only mean that we spy on it when needed with `membershipManager = 
spy(createMemberInStableState())` on the specific tests that do verify, and not 
spying on membershipmanager for all test that in the end don;t need it. Makes 
sense?



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



Re: [PR] KAFKA-16000 Migrated MembershipManagerImplTest away from ConsumerTestBuilder [kafka]

2024-06-18 Thread via GitHub


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


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java:
##
@@ -220,6 +233,7 @@ public void 
testTransitionToReconcilingIfEmptyAssignmentReceived() {
 
 @Test
 public void testMemberIdAndEpochResetOnFencedMembers() {
+createCommitRequestManager(false);

Review Comment:
   we shouldn't need an actual `commitRequestManager` here, where we don't need 
anything specific related to commits and a mock should do. I guess you could 
remove it and just return a completed future for the autoCommit before 
revocation :
   
   
`when(commitRequestManager.maybeAutoCommitSyncBeforeRevocation(anyLong())).thenReturn(CompletableFuture.completedFuture(null));`
   
   Probably doing it once inside the `createMemberInStableState` will allow you 
to remove most of the createCommitRequestManager calls



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



Re: [PR] KAFKA-16000 Migrated MembershipManagerImplTest away from ConsumerTestBuilder [kafka]

2024-06-12 Thread via GitHub


brenden20 commented on PR #16312:
URL: https://github.com/apache/kafka/pull/16312#issuecomment-2163965706

   > Hey @brenden20 , assuming it's something we cannot avoid in this case, you 
can suppress it with @SuppressWarnings("ClassDataAbstractionCoupling") (it's 
done in other places too)
   
   Great to know! I will take another look at the code to make sure it is 
necessary, if so I will add that annotation. Thank you!


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



Re: [PR] KAFKA-16000 Migrated MembershipManagerImplTest away from ConsumerTestBuilder [kafka]

2024-06-12 Thread via GitHub


lianetm commented on PR #16312:
URL: https://github.com/apache/kafka/pull/16312#issuecomment-2163964029

   Hey @brenden20 , assuming it's something we cannot avoid in this case, you 
can suppress it with @SuppressWarnings("ClassDataAbstractionCoupling") (it's 
done in other places too)


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



Re: [PR] KAFKA-16000 Migrated MembershipManagerImplTest away from ConsumerTestBuilder [kafka]

2024-06-12 Thread via GitHub


brenden20 commented on PR #16312:
URL: https://github.com/apache/kafka/pull/16312#issuecomment-2163905913

   @philipnee @lianetm @kirktrue I am getting a checkstyle failure, I have one 
too many class data abstractions, I am unsure on where I can fix this. Any 
ideas?


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



[PR] KAFKA-16000 Migrated MembershipManagerImplTest away from ConsumerTestBuilder [kafka]

2024-06-12 Thread via GitHub


brenden20 opened a new pull request, #16312:
URL: https://github.com/apache/kafka/pull/16312

   I completely migrated MembershipManagerImplTest away from 
ConsumerTestBuilder and removed all spy objects. All tests passing, none were 
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: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org