Skip to content

KAFKA-12268: Make early poll return opt-in#10096

Closed
vvcephei wants to merge 1 commit intoapache:trunkfrom
vvcephei:kafka-12268-opt-in-early-poll-return
Closed

KAFKA-12268: Make early poll return opt-in#10096
vvcephei wants to merge 1 commit intoapache:trunkfrom
vvcephei:kafka-12268-opt-in-early-poll-return

Conversation

@vvcephei
Copy link
Copy Markdown
Contributor

  • Revert the default Consumer#poll behavior back to early return on records only
  • Add config to enable early return on metadata or records
  • Set the return-on-metadata config in Streams to support KIP-695
  • Revert the undesired addition of Hamcrest to the Client module
  • Revert the now unnecessary poll-until-records logic in PlaintextConsumerTest

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

@vvcephei
Copy link
Copy Markdown
Contributor Author

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 enable.early.poll.return.on.metadata=false|true, but maybe there will be some other behavior we might add as another "mode" in the future. No strong opinion here.

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.

Copy link
Copy Markdown
Member

@chia7712 chia7712 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@vvcephei Thanks for nice patch. Some minor comments are left. Please take a look.

Also, could you update javadocs of KafkaConsumer#poll to mention this new config?

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) {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sounds good!

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);
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It should be LONG_POLL_RETURN_ON_RESPONSE rather than LONG_POLL_RETURN_ON_RECORDS

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Oy, you are right. Thanks!

Copy link
Copy Markdown
Member

@chia7712 chia7712 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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(
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.";
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm, this doesn't seem great.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In what KIP was this approved?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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?

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

@vvcephei
Copy link
Copy Markdown
Contributor Author

vvcephei commented Mar 2, 2021

Replced by #10137

@vvcephei vvcephei closed this Mar 2, 2021
@vvcephei vvcephei deleted the kafka-12268-opt-in-early-poll-return branch March 2, 2021 14:21
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants