AndrewJSchofield commented on code in PR #18851:
URL: https://github.com/apache/kafka/pull/18851#discussion_r1952295803
##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManagerTest.java:
##########
@@ -862,6 +867,81 @@ public void testCommitAsyncWithSubscriptionChange() {
}
+ @Test
+ public void testCommitSyncWithSubscriptionChange() {
+ buildRequestManager();
+
+ assignFromSubscribed(singleton(tp0));
+
+ assertEquals(1, sendFetches());
+ assertFalse(shareConsumeRequestManager.hasCompletedFetches());
+
+ client.prepareResponse(fullFetchResponse(tip0, records,
acquiredRecords, Errors.NONE));
+ networkClientDelegate.poll(time.timer(0));
+ assertTrue(shareConsumeRequestManager.hasCompletedFetches());
+
+ Acknowledgements acknowledgements = Acknowledgements.empty();
+ acknowledgements.add(1L, AcknowledgeType.ACCEPT);
+ acknowledgements.add(2L, AcknowledgeType.ACCEPT);
+ acknowledgements.add(3L, AcknowledgeType.REJECT);
+
+ subscriptions.subscribeToShareGroup(Collections.singleton(topicName2));
+ subscriptions.assignFromSubscribed(Collections.singleton(t2p0));
+
+ client.updateMetadata(
+ RequestTestUtils.metadataUpdateWithIds(1,
singletonMap(topicName2, 1),
+ tp -> validLeaderEpoch, topicIds, false));
+
+ shareConsumeRequestManager.commitSync(Collections.singletonMap(tip0,
new NodeAcknowledgements(0, acknowledgements)),
+ calculateDeadlineMs(time.timer(100)));
+
+ assertEquals(1, shareConsumeRequestManager.sendAcknowledgements());
+
+ client.prepareResponse(fullAcknowledgeResponse(tip1, Errors.NONE));
Review Comment:
These new tests that use topic 2 are still referring to `tip1` which seems a
mistake.
##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManager.java:
##########
@@ -630,6 +663,26 @@ public CompletableFuture<Void> acknowledgeOnClose(final
Map<TopicIdPartition, No
return closeFuture;
}
+ /**
+ *
+ * @return True if nodeId is the leader or if leader information is not
present for the topicIdPartition.
+ * Returns false if nodeId is not the leader for the given topicIdPartition
+ */
+ private boolean isNodeLeader(int nodeId, TopicIdPartition
topicIdPartition) {
Review Comment:
This is a bit confusing. How about `isLeaderKnownToHaveChanged`?
--
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]