-
Notifications
You must be signed in to change notification settings - Fork 13.6k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
KAFKA-16194: Do not return records from poll if group metadata unknown #15369
Conversation
a064c7e
to
bc0f1d3
Compare
abbcde2
to
1699673
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for the PR! I left some comments.
(I did not check the tests in detail, because I'm not sure about the current main implementation)
@@ -98,6 +99,8 @@ | |||
import java.util.concurrent.LinkedBlockingQueue; | |||
import java.util.concurrent.TimeUnit; | |||
import java.util.concurrent.atomic.AtomicReference; | |||
import java.util.function.Supplier; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
not used
@@ -727,6 +732,17 @@ public ConsumerRecords<K, V> poll(final Duration timeout) { | |||
} | |||
} | |||
|
|||
private boolean isGenerationKnown() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
How about this one-liner
return groupMetadata.filter(g -> g.generationId() != JoinGroupRequest.UNKNOWN_GENERATION_ID).isPresent();
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I am not sure this is correct, because I can use user assigned partitions but still use a group ID to manage the offsets broker-side, if I remember correctly.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Probably, I can do something like
if (subscriptions.hasAutoAssignedPartitions()) {
return groupMetadata.filter(g -> g.generationId() != JoinGroupRequest.UNKNOWN_GENERATION_ID).isPresent();
}
return true;
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Makes sense. But then I'm a bit confused by the method name, because it just returns true when there is no generation...
@@ -1467,6 +1483,7 @@ public void unsubscribe() { | |||
} catch (TimeoutException e) { | |||
log.error("Failed while waiting for the unsubscribe event to complete"); | |||
} | |||
groupMetadata = initializeGroupMetadata(groupMetadata.get().groupId(), Optional.empty()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This looks like an independent fix? Maybe mention in the PR description
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done!
if (!fetch.isEmpty()) { | ||
if (fetch.records().isEmpty()) { | ||
log.trace("Returning empty records from `poll()` " | ||
if (isGenerationKnown()) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If this check is false... where do we block? It seems like we busy loop with 100% CPU, but maybe I'm wrong
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The busy loop should not be too bad since the group metadata update event is added to the queue when also the new assignment is set in the membership manager.
Moreover, this is kind of a temporary solution until https://issues.apache.org/jira/browse/KAFKA-16285 is done.
fc64170
to
9e9f895
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If you want, you could improve the name of isGenerationKnown
, but LGTM.
b3f5edb
to
bd4ecee
Compare
Hmm after this was merged, I'm seeing failed builds on every master build.
I see this build was fine so not sure if there was some other issue or if the issue is external. |
Looks like it was actually this one: #15396. I will follow up there. |
apache#15369) Due to the asynchronous nature of the async consumer, it might happen that on the application thread the group metadata is not known after the first poll returns records. If the offsets of those records are then send to a transaction with txnProducer.sendOffsetsToTransaction(offsetsToCommit, groupMetadata); and then the transaction is committed, the group coordinator will raise an error saying that the member is not known since the member in groupMetadata is still -1 before the metadata is updated. This commit avoids this error by not returning any records from poll() until the group metadata is updated, i.e., the member ID and the generation ID (a.k.a. member epoch) are known. This check is only done if group management is used. Additionally, this commit resets the group metadata when the consumer unsubscribes. Reviewer: Lucas Brutschy <lbrutschy@confluent.io>
Due to the asynchronous nature of the async consumer, it might happen that on the application thread the group metadata is not known after the first poll returns records. If the offsets of those records are then send to a transaction with
and then the transaction is committed, the group coordinator will raise an error saying that the member is not known since the member in
groupMetadata
is still-1
before the metadata is updated.This commit avoids this error by not returning any records from
poll()
until the group metadata is updated, i.e., the member ID and the generation ID (a.k.a. member epoch) are known. This check is only done if group management is used.Additionally, this commit resets the group metadata when the consumer unsubscribes.
Committer Checklist (excluded from commit message)