KAFKA-12268: Make early poll return opt-in#10096
KAFKA-12268: Make early poll return opt-in#10096vvcephei wants to merge 1 commit intoapache:trunkfrom
Conversation
|
Hey @rajinisivaram and @chia7712 , Here is my proposed fix for https://issues.apache.org/jira/browse/KAFKA-12268 , based on our discussion in #10022 and in the mailing list (https://lists.apache.org/thread.html/ra3a73e74c2bb9768ce33dcf0492e2f0fd2f120044541c6b216a67cca%40%3Cdev.kafka.apache.org%3E) I don't know if anyone has a preference on the config name itself. I also considered just Also, please take a look at the logic in the Consumer. I did my best to make it both efficient and legible, but I'm not sure I achieved either goal. Thanks for spotting the problem and for the discussion, and thanks in advance for the reviews! Also, as the original reviewer of the feature, I'd appreciate a review from @guozhangwang if you have time. |
| if (!records.isEmpty()) { | ||
|
|
||
| // We only need to save off the metadata if we are NOT supposed to return early on metadata-only responses | ||
| if (!longPollExitEarlyOnMetadata) { |
There was a problem hiding this comment.
How about updating nullableSeenMetadata only if we don't return records?
if (longPollShouldReturn(records)) {
...
} else {
if (nullableSeenMetadata == null) {
nullableSeenMetadata = new HashMap<>(records.metadata());
} else {
nullableSeenMetadata.putAll(records.metadata());
}
}The benefit from above code is that we don't need to handle duplicate metadata which exists on both FetchedRecords and nullableSeenMetadata when it succeed to get records and metadata in first loop.
| consumerProps.put(ConsumerConfig.ALLOW_AUTO_CREATE_TOPICS_CONFIG, "false"); | ||
|
|
||
| // enable early return on metadata | ||
| consumerProps.put(ConsumerConfig.LONG_POLL_MODE_CONFIG, ConsumerConfig.LONG_POLL_RETURN_ON_RECORDS); |
There was a problem hiding this comment.
It should be LONG_POLL_RETURN_ON_RESPONSE rather than LONG_POLL_RETURN_ON_RECORDS
There was a problem hiding this comment.
Oy, you are right. Thanks!
chia7712
left a comment
There was a problem hiding this comment.
It seems to me the constructors of ConsumerRecords and ConsumerRecords.Metadata should be package-private rather than public. Otherwise, it could cause troubles when we are going to add more data in the future. WDYT?
| for (Map.Entry<TopicPartition, FetchedRecords.FetchMetadata> entry : nullableMetadataMap.entrySet()) { | ||
| result.put( | ||
| entry.getKey(), | ||
| new ConsumerRecords.Metadata( |
There was a problem hiding this comment.
Not sure why we need to have FetchedRecords.FetchMetadata? It is always converted to ConsumerRecords.Metadata directly. The records in FetchedRecords is ConsumerRecord. Maybe we can make FetchedRecords use ConsumerRecords.Metadata also.
| " included in the response. Callers who are only interested in seeing records should leave the default in" + | ||
| " place, and callers who are interested in maintaining fresh local information about the current lag should" + | ||
| " enable 'return_on_response'." + | ||
| " Note that the Consumer metrics are always updated upon receipt of the fetch responses."; |
There was a problem hiding this comment.
Thanks @ijuma ,
This is KIP-695. The currently approved version of it includes just a behavior change and no config. The behavior change is that, if you call Consumer#poll and there are some completed fetch responses, but none of them have records, we still return a ConsumerRecords object with no records so that we can return the new metadata (via ConsumerRecords#metadata().
That behavior change struck @chia7712 and @rajinisivaram as not ok (for reasons discussed in https://lists.apache.org/thread.html/ra3a73e74c2bb9768ce33dcf0492e2f0fd2f120044541c6b216a67cca%40%3Cdev.kafka.apache.org%3E ).
In that thread, we agreed to instead introduce an opt-in config, which is what this doc is about. Which part doesn't seem great to you?
There was a problem hiding this comment.
Having yet another config for something people should not really care about. I think you should revive the KIP thread with the proposed updates and give people a chance to comment on it. Also the KIP should cover the compatibility impact that is behind this update.
There was a problem hiding this comment.
Thanks @ijuma .
I do agree with you. In the vote thread, others made the case that the behavior change is not ok. If you disagree with the conclusion of the vote thread and want to argue that it actually is ok for Consumer#poll to return when there are only metadata updates, I'm happy to reconsider the decision.
I didn't update the text of the KIP yet because I suspected that this PR might inspire a new round of discussion.
|
Replced by #10137 |
Committer Checklist (excluded from commit message)