lianetm commented on code in PR #20521:
URL: https://github.com/apache/kafka/pull/20521#discussion_r2413565526
##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java:
##########
@@ -779,11 +835,22 @@ private void updatePatternSubscription(Cluster cluster) {
// Join the group if not already part of it, or just send the updated
subscription
// to the broker on the next poll. Note that this is done even if no
topics matched
// the regex, to ensure the member joins the group if needed (with
empty subscription).
-
requestManagers.consumerHeartbeatRequestManager.get().membershipManager().onSubscriptionUpdated();
+ membershipManager.onSubscriptionUpdated();
}
// Visible for testing
int metadataVersionSnapshot() {
return metadataVersionSnapshot;
}
+
+ /**
+ * Ideally the {@link AbstractMembershipManager#onSubscriptionUpdated()}
API could be invoked directly for the
+ * three membership managers, but unfortunately {@link
StreamsMembershipManager} doesn't extend from
Review Comment:
let's update this if my comment above makes sense (share mgr unrelated to
pattern subscriptions)
##########
clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java:
##########
@@ -951,7 +950,7 @@ public void
verifyNoCoordinatorLookupForManualAssignmentWithSeek(GroupProtocol g
client.prepareResponse(listOffsetsResponse(Map.of(tp0, 50L)));
client.prepareResponse(fetchResponse(tp0, 50L, 5));
- ConsumerRecords<String, String> records = (ConsumerRecords<String,
String>) consumer.poll(Duration.ofMillis(1));
+ ConsumerRecords<String, String> records =
ConsumerPollTestUtils.waitForRecords(consumer);
Review Comment:
seems this is not needed (after the fixes to how long we block on poll I
expect).
Actually, similar for almost all changes in this file probably. I tried
running the trunk version of of it and all tests pass except for the
`testCurrentLag` (great news btw!, can we revert all the changes if
unnecessary).
##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java:
##########
@@ -217,35 +228,15 @@ public void process(ApplicationEvent event) {
}
}
- private void process(final PollEvent event) {
- // Trigger a reconciliation that can safely commit offsets if needed
to rebalance,
- // as we're processing before any new fetching starts in the app thread
+ private void process(final SharePollEvent event) {
requestManagers.consumerMembershipManager.ifPresent(consumerMembershipManager ->
consumerMembershipManager.maybeReconcile(true));
- if (requestManagers.commitRequestManager.isPresent()) {
- CommitRequestManager commitRequestManager =
requestManagers.commitRequestManager.get();
- commitRequestManager.updateTimerAndMaybeCommit(event.pollTimeMs());
- // all commit request generation points have been passed,
- // so it's safe to notify the app thread could proceed and start
fetching
- event.markReconcileAndAutoCommitComplete();
- requestManagers.consumerHeartbeatRequestManager.ifPresent(hrm -> {
- hrm.membershipManager().onConsumerPoll();
- hrm.resetPollTimer(event.pollTimeMs());
- });
- requestManagers.streamsGroupHeartbeatRequestManager.ifPresent(hrm
-> {
- hrm.membershipManager().onConsumerPoll();
- hrm.resetPollTimer(event.pollTimeMs());
- });
- } else {
- // safe to unblock - no auto-commit risk here:
- // 1. commitRequestManager is not present
- // 2. shareConsumer has no auto-commit mechanism
- event.markReconcileAndAutoCommitComplete();
- requestManagers.shareHeartbeatRequestManager.ifPresent(hrm -> {
- hrm.membershipManager().onConsumerPoll();
- hrm.resetPollTimer(event.pollTimeMs());
- });
- }
+ requestManagers.shareHeartbeatRequestManager.ifPresent(hrm -> {
+ ShareMembershipManager membershipManager = hrm.membershipManager();
+
maybeUpdatePatternSubscription(membershipManager::onSubscriptionUpdated);
Review Comment:
The share consumer only subscribes to explicit topic names so no need to
check for pattern subscription really.
We should just need this for the consumerMembershipMgr and streams.
##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java:
##########
@@ -1771,15 +1827,9 @@ private Fetch<K, V> pollForFetches(Timer timer) {
return fetch;
}
- // send any new fetches (won't resend pending fetches)
- sendFetches(timer);
-
// We do not want to be stuck blocking in poll if we are missing some
positions
// since the offset lookup may be backing off after a failure
-
- // NOTE: the use of cachedSubscriptionHasAllFetchPositions means we
MUST call
- // updateAssignmentMetadataIfNeeded before this method.
- if (!cachedSubscriptionHasAllFetchPositions && pollTimeout >
retryBackoffMs) {
+ if (pollTimeout > retryBackoffMs) {
Review Comment:
my concern with `hasAllFetchPositions` is that even though it's
`synchronized`, it's at the func level, but not on the `assignment`
collection,. So I expect we could still get a race where the hasAll is called
from the app thread here only only, but in the background the assignment
collection changes (in the end it's a simple LinkedhashMap used with an
iterator, so we could get ConcurrentModificationException I expect)
--
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]