[ https://issues.apache.org/jira/browse/KAFKA-405?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Neha Narkhede updated KAFKA-405: -------------------------------- Attachment: kafka-405-v3.patch Jay's comments 4. Changed the write operation for highwatermark file to be atomic. 5. The sleep is in place to allow the follower to send another fetch request to the leader to allow the leader to tell it the latest leader high watermark. It cannot be fixed by flushing more frequently 9. Added the ability to set the name of a thread in KafkaScheduler. Also, saw that the KafkaScheduler took in a isDaemon variable, but didn't really use it. Refactored KafkaScheduler to create daemon/non-daemon threads with different names. 10. There was an assertion that protects against this in the only API that called FileMessageSet.truncateTo. Moved that to FileMessageSet instead and changed it to throw KafkaException. Also, handled all exceptions in the become follower/become leader state change API to log an error stating that the state change failed. This will make debugging easier. 11. Have marked methods not in the public interface for ReplicaManager as private? Agree that there is some room for refactoring. Added your suggestion to KAFKA-351 that we have filed to cover the refactoring of ReplicaManager and KafkaZookeeper. Currently, with the controller patch, KafkaZookeeper is going to look very different. So I'd rather wait until controller patch is in. Jun's comments 20. There will only be one segment that will fit this criteria => segment.start >= hw && segment.endOffset < hw. That code truncates the one and only segment that matches this criteria 21. The setHighwatermark variable and its references are deleted as part of KAFKA-350 22. 1. Changed to recordLeaderLogEndOffset() 2. Changed close() to shutdown() for LogManager, ReplicaManager and KafkaZookeeper 3. Good point. Fixed it to read from the file only on startup 23. Yes, it might be ok to have the file hidden. Other fixes - Log.scala 1. truncateTo() had bugs in that it used the size() API on the FileMessageSet of the segment to get the absolute end offset. Fixed it to use the absoluteEndOffset() API of LogSegment instead. > Improve the high water mark maintenance to store high watermarks for all > partitions in a single file on disk > ------------------------------------------------------------------------------------------------------------ > > Key: KAFKA-405 > URL: https://issues.apache.org/jira/browse/KAFKA-405 > Project: Kafka > Issue Type: Bug > Affects Versions: 0.8 > Reporter: Neha Narkhede > Assignee: Neha Narkhede > Attachments: kafka-405-v1.patch, kafka-405-v2.patch, > kafka-405-v3.patch > > > KAFKA-46 introduced per partition leader high watermarks. But it stores those > in one file per partition. A more performant solution would be to store all > high watermarks in a single file on disk -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira