Skip to content
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-16243: Make sure that we do not exceed max poll interval inside poll #15372

Merged
merged 4 commits into from
Feb 20, 2024

Conversation

lucasbru
Copy link
Member

The consumer keeps a poll timer, which is used to ensure liveness of the application thread. The poll timer automatically updates while the Consumer.poll(Duration) method is blocked, while the newer consumer only updates the poll timer when a new call to Consumer.poll(Duration) is issued. This means that the kafka-console-consumer.sh tools, which uses a very long timeout by default, works differently with the new consumer, with the consumer proactively rejoining the group during long poll timeouts.

This change solves the problem by (a) repeatedly sending PollApplicationEvents to the background thread, not just on the first call of poll and (b) making sure that the application thread doesn't block for so long that it runs out of max.poll.interval.

An integration test is added to make sure that we do not rejoin the group when a long poll timeout is used with a low max.poll.interval.

Committer Checklist (excluded from commit message)

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

… poll

The consumer keeps a poll timer which is used to ensure liveness of the application thread. The poll timer automatically updates while the `Consumer.poll(Duration)` method is blocked, while the newer consumer only updates the poll timer when a new call to `Consumer.poll(Duration)` is issued. This means that the kafka-console-consumer.sh tools, which uses a very long timeout by default, works differently with the new consumer, with the consumer proactively rejoining the group during long poll timeouts.

We solve the problem by (a) repeatedly sending `PollApplicationEvents` to the background thread, not just on the first call of `poll` and (b) making sure that the application thread doesn't block for so long that it runs out of `max.poll.interval`.

An integration test is added to make sure that we do not rejoin the group when a long poll timeout is used with a low `max.poll.interval`.
Copy link
Member

@AndrewJSchofield AndrewJSchofield left a comment

Choose a reason for hiding this comment

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

lgtm

@lucasbru
Copy link
Member Author

@cadonna could you please have a look?


val initialAssignedCalls = listener.callsToAssigned

consumer.poll(Duration.ofMillis(2000));
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: semicolon not needed here, same in the next 2 lines

Copy link
Member Author

Choose a reason for hiding this comment

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

Done

@lianetm
Copy link
Contributor

lianetm commented Feb 15, 2024

thanks for the changes! lgtm.

@lianetm
Copy link
Contributor

lianetm commented Feb 16, 2024

I like the last commit msg :)

@lucasbru
Copy link
Member Author

Hey @mjsax . This PR is right now waiting for @cadonna's review. If you have time, you could take a look to get into the 848 work.

Copy link
Contributor

@cadonna cadonna left a 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, @lucasbru !

I have two minor comments about comments 🙂 .

For the rest, LGTM!


consumer.poll(Duration.ofMillis(2000))

// Give enough time to rejoin
Copy link
Contributor

Choose a reason for hiding this comment

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

nit:
This comment is a bit confusing. What is it supposed to clarify?

Copy link
Member Author

Choose a reason for hiding this comment

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

Done

consumer.poll(Duration.ofMillis(500))
consumer.poll(Duration.ofMillis(500))

// Check that we did not rejoin
Copy link
Contributor

Choose a reason for hiding this comment

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

Do we need this comment? I think it would be better to delete it and to rename initialAssignedCalls to something more meaningful like callsToAssignedAfterFirstRebalance or callsToAssignedBeforePolls.

Copy link
Member Author

Choose a reason for hiding this comment

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

Done

@lucasbru lucasbru merged commit 5854139 into apache:trunk Feb 20, 2024
1 check failed
@lucasbru lucasbru deleted the ctr_fix_poll branch February 20, 2024 09:48
) {
return 0L;
}
return Math.min(pollTimer.remainingMs() / 2, heartbeatRequestState.nextHeartbeatMs(currentTimeMs));
Copy link
Member

Choose a reason for hiding this comment

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

For my own education: why / 2 ?

Copy link
Member Author

Choose a reason for hiding this comment

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

It's somewhat arbitrary. We need to make sure that the application thread doesn't block so long that the poll timer expires. We want to let it unblock sometime before the timer expires, and send an event to the background thread that it's still polling, and give the background thread time to process the event and reset the poll timer.

clolov pushed a commit to clolov/kafka that referenced this pull request Apr 5, 2024
… poll (apache#15372)

The consumer keeps a poll timer, which is used to ensure liveness of the application thread. The poll timer automatically updates while the Consumer.poll(Duration) method is blocked, while the newer consumer only updates the poll timer when a new call to Consumer.poll(Duration) is issued. This means that the kafka-console-consumer.sh tools, which uses a very long timeout by default, works differently with the new consumer, with the consumer proactively rejoining the group during long poll timeouts.

This change solves the problem by (a) repeatedly sending PollApplicationEvents to the background thread, not just on the first call of poll and (b) making sure that the application thread doesn't block for so long that it runs out of max.poll.interval.

An integration test is added to make sure that we do not rejoin the group when a long poll timeout is used with a low max.poll.interval.

Reviewers: Lianet Magrans <[email protected]>, Andrew Schofield <[email protected]>, Bruno Cadonna <[email protected]>
Phuc-Hong-Tran pushed a commit to Phuc-Hong-Tran/kafka that referenced this pull request Jun 6, 2024
… poll (apache#15372)

The consumer keeps a poll timer, which is used to ensure liveness of the application thread. The poll timer automatically updates while the Consumer.poll(Duration) method is blocked, while the newer consumer only updates the poll timer when a new call to Consumer.poll(Duration) is issued. This means that the kafka-console-consumer.sh tools, which uses a very long timeout by default, works differently with the new consumer, with the consumer proactively rejoining the group during long poll timeouts.

This change solves the problem by (a) repeatedly sending PollApplicationEvents to the background thread, not just on the first call of poll and (b) making sure that the application thread doesn't block for so long that it runs out of max.poll.interval.

An integration test is added to make sure that we do not rejoin the group when a long poll timeout is used with a low max.poll.interval.

Reviewers: Lianet Magrans <[email protected]>, Andrew Schofield <[email protected]>, Bruno Cadonna <[email protected]>
@cadonna cadonna added the ctr Consumer Threading Refactor (KIP-848) label Dec 28, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
ctr Consumer Threading Refactor (KIP-848)
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants