[
https://issues.apache.org/jira/browse/KAFKA-14196?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17600002#comment-17600002
]
Philip Nee commented on KAFKA-14196:
------------------------------------
I guess it should be related to KAFKA-4689
Looking at the coordinator code, I'm not entirely sure if the commented
statement is still true
{code:java}
// wait for commit offset response before timer expired
if (autoCommitOffsetRequestFuture != null) {
Timer pollTimer = timer.remainingMs() < joinPrepareTimer.remainingMs() ?
timer : joinPrepareTimer; {code}
Because the new API timer.remainingMs will always be 0. as set here
{code:java}
if (!ensureActiveGroup(waitForJoinGroup ? timer : time.timer(0L))) {code}
If that's the case, I don't think we ever actually wait for the async commit to
finish. Instead, as we can see in the test log, onJoinPrepare continue to get
retriggered, until the async commit is completed.
Maybe the commit should always be blocking here?
> Flaky OffsetValidationTest seems to indicate potential duplication issue
> during rebalance
> -----------------------------------------------------------------------------------------
>
> Key: KAFKA-14196
> URL: https://issues.apache.org/jira/browse/KAFKA-14196
> Project: Kafka
> Issue Type: Bug
> Components: clients, consumer
> Affects Versions: 3.2.1
> Reporter: Philip Nee
> Assignee: Philip Nee
> Priority: Major
>
> Several flaky tests under OffsetValidationTest are indicating potential
> consumer duplication issue, when autocommit is enabled. Below shows the
> failure message:
>
> {code:java}
> Total consumed records 3366 did not match consumed position 3331 {code}
>
> After investigating the log, I discovered that the data consumed between the
> start of a rebalance event and the async commit was lost for those failing
> tests. In the example below, the rebalance event kicks in at around
> 1662054846995 (first record), and the async commit of the offset 3739 is
> completed at around 1662054847015 (right before partitions_revoked).
>
> {code:java}
> {"timestamp":1662054846995,"name":"records_consumed","count":3,"partitions":[{"topic":"test_topic","partition":0,"count":3,"minOffset":3739,"maxOffset":3741}]}
> {"timestamp":1662054846998,"name":"records_consumed","count":2,"partitions":[{"topic":"test_topic","partition":0,"count":2,"minOffset":3742,"maxOffset":3743}]}
> {"timestamp":1662054847008,"name":"records_consumed","count":2,"partitions":[{"topic":"test_topic","partition":0,"count":2,"minOffset":3744,"maxOffset":3745}]}
> {"timestamp":1662054847016,"name":"partitions_revoked","partitions":[{"topic":"test_topic","partition":0}]}
> {"timestamp":1662054847031,"name":"partitions_assigned","partitions":[{"topic":"test_topic","partition":0}]}
> {"timestamp":1662054847038,"name":"records_consumed","count":23,"partitions":[{"topic":"test_topic","partition":0,"count":23,"minOffset":3739,"maxOffset":3761}]}
> {code}
> A few things to note here:
> # This is highly flaky, I found 1/4 runs will fail the tests
> # Manually calling commitSync in the onPartitionsRevoke cb seems to
> alleviate the issue
> # Setting includeMetadataInTimeout to false also seems to alleviate the
> issue.
>
--
This message was sent by Atlassian Jira
(v8.20.10#820010)