[ https://issues.apache.org/jira/browse/KAFKA-1012?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13750956#comment-13750956 ]
Tejas Patil commented on KAFKA-1012: ------------------------------------ Thanks a lot [~junrao] for all the awesome comments !!! 20. ZookeeperConsumerConnector: 20.1 Good suggestion from design point of view. Would include in next patch. 20.2 Would include in next patch 20.3 With 20.1, this would move to OffsetClientManager. Would wait for KAFKA-989 to get merged in trunk so that I could use it in my patch. 20.4 "loading offsets" fails rebalance and then rebalance would be re-tried. It can be argued that subsequent retries can fail too as those happen w/o any delay. We could have an infinite loop with some exponential delay till offset fetch does not get an offset loading code. It is expected that the offsets topic logs for a single partition would be small in size and the broker should not take much time to load .... so we might not need to have a loop. Would keep this open for discussion. 21. KafkaApis: 21.1 This would just save the #connections on the controller node and not the rest brokers. With the thick client change, this would not be needed. 21.2 Clients, written in different langauges, might choose any ack level. To be on safe side, all cases are handled. Would keep as it is. 21.3 This could have been possible if there was no "else" clause. Would keep as it is. 21.4 +1 for the suggestion. Would include in next patch. 22. OffsetManager: 22.1 timestamp would be used in coming patch. 22.2 SUPERB CATCH :) How about "currOffset = m.nextOffset" ? Would include in next patch. 22.3 This was done to unblock the offset fetch requests right after loading is done. In the coming patch with timestamps being stored in offset table, this would change. 23. +1. Would include in next patch. 24. The coming patch would include a stnadalone utility to do offset cleanup. Here is the section in wiki page: https://cwiki.apache.org/confluence/display/KAFKA/Inbuilt+Consumer+Offset+Management#InbuiltConsumerOffsetManagement-\5\Offsetscleanupfromoffsettable%2CZkandlogs 25. Migrating existing consumers: The patch changes the structure of the offset records in Zk (it stores offset + delimiter + timestamp + delimiter + metadata). This could be reverted for time being to use the old format (just the offset). The new binary can then be deployed bouncing brokers one by one. Consumers would still directly write to Zk. Those need to be upgraded too after done with brokers. Brokers would continue to use Zk to save offsets but along with that, the logs would get populated from offset commits. Also, we could explicitly write the offsets from Zk to logs for those entries which did not have any commits after this new binary is deployed. Now that we are sure that all the offset info is in the logs, we can switch the config to use the inbuilt offset manager. As the data is in logs, loading of offsets would bring up the offset table entries. There would be no need to have any migration tool. > Implement an Offset Manager and hook offset requests to it > ---------------------------------------------------------- > > Key: KAFKA-1012 > URL: https://issues.apache.org/jira/browse/KAFKA-1012 > Project: Kafka > Issue Type: Sub-task > Components: consumer > Reporter: Tejas Patil > Assignee: Tejas Patil > Priority: Minor > Attachments: KAFKA-1012.patch, KAFKA-1012-v2.patch > > > After KAFKA-657, we have a protocol for consumers to commit and fetch offsets > from brokers. Currently, consumers are not using this API and directly > talking with Zookeeper. > This Jira will involve following: > 1. Add a special topic in kafka for storing offsets > 2. Add an OffsetManager interface which would handle storing, accessing, > loading and maintaining consumer offsets > 3. Implement offset managers for both of these 2 choices : existing ZK based > storage or inbuilt storage for offsets. > 4. Leader brokers would now maintain an additional hash table of offsets for > the group-topic-partitions that they lead > 5. Consumers should now use the OffsetCommit and OffsetFetch API -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira