[ https://issues.apache.org/jira/browse/KAFKA-5247?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16018186#comment-16018186 ]
ASF GitHub Bot commented on KAFKA-5247: --------------------------------------- GitHub user apurvam opened a pull request: https://github.com/apache/kafka/pull/3108 KAFKA-5247: Materialize committed offsets in offset order With this patch, offset commits are always materialized according to the order of the commit records in the offsets topic. Before this patch, transactional offset commits were materialized in transaction order. However, the log cleaner will always preserve the record with the greatest offset. This meant that if there was a mix of offset commits from a consumer and a transactional producer, then it we would switch from transactional order to offset order after cleaning, resulting in an inconsistent state. You can merge this pull request into a Git repository by running: $ git pull https://github.com/apurvam/kafka KAFKA-5247-materialize-committed-offsets-in-offset-order Alternatively you can review and apply these changes as the patch at: https://github.com/apache/kafka/pull/3108.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #3108 ---- commit f6efd565023d440c6d15091609442ff61ad6f85a Author: Apurva Mehta <apu...@confluent.io> Date: 2017-05-19T18:31:48Z KAFKA-5247 materialize offset commits in offset order Updated the GroupMetadata to keep track of the offset in the __consumer_offsets topic for the commit record for a given offset commit. We only update the offsets cache when a given offset is committed if the offset of the commit record in the offsets topic is greater than the offset of the existing materialized offset. This way, if we have a mix of transactional and non transactional offset commits for the same group, we will always materialize the offset commtis in offset order. commit 20ee45422130f197791600891a9872826d510ca7 Author: Apurva Mehta <apu...@confluent.io> Date: 2017-05-19T22:35:27Z Update the return values of the GroupMetadata.remove* methods commit 2fd79d1680711cdd746233dfbeaea957e65e67d8 Author: Apurva Mehta <apu...@confluent.io> Date: 2017-05-19T23:49:08Z Minor cleanups and added unit tests commit 7e5f2820809d9a085333e1fa97efd13207e5a4e0 Author: Apurva Mehta <apu...@confluent.io> Date: 2017-05-20T00:02:13Z Remove erroneous comment ---- > Consumer GroupCoordinator should continue to materialize committed offsets in > offset order even for transactional offset commits > -------------------------------------------------------------------------------------------------------------------------------- > > Key: KAFKA-5247 > URL: https://issues.apache.org/jira/browse/KAFKA-5247 > Project: Kafka > Issue Type: Sub-task > Reporter: Apurva Mehta > Assignee: Apurva Mehta > Priority: Blocker > Labels: exactly-once > Fix For: 0.11.0.0 > > > In the TxnOffsetCommit patch, we thought it was ok for the group coordinator > to use "transaction order" semantics when updating the cache, but we weren't > thinking about the log cleaner. > The log cleaner uses offset order when cleaning which means that the key with > the largest offset always wins. So if we use transaction order when > dynamically updating the cache, we will get different results from when we're > loading the cache (even if the loading logic also uses transaction order). > The fix should be straightforward: we need to remember the offset in the > offsets topic of the offset that we cache. Then we only update it if the new > entry has a higher offset. -- This message was sent by Atlassian JIRA (v6.3.15#6346)