[ 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-v1.patch This patch improves the high watermark persistence strategy to store the high watermark values for all partitions in a single file. The changes include - 1. Log.scala 1.1 Moved the highwatermark file out of the Log. Since highwatermark is related to replication state and not log persistence, it makes sense for it to not be part of kafka.log 1.2 Since the log recovery logic requires to modify log segments, recoverUptoLastCheckpointedHighWatermark() API stays in Log.scala. It is passed in the last checkpointed high watermark from ReplicaManager during make follower state change 2. ReplicaManager 2.1 Added a startup API to ReplicaManager to be consistent with all other components. 2.2 Added a scheduler that will checkpoint high watermarks at defaultFlushIntervalMs rate. I didn't think it was useful to introduce another config to control the rate at which high watermarks are flushed to disk, so I reused the one we have for flushing log segments 2.3 Added a checkpointHighwaterMark() API that will iterate through all the local replicas for each partition and write the high watermark file in the following format number of entries (4 bytes) topic (UTF) partition (4 bytes) highwatermark (8 bytes) 2.4 Added a readCheckpointedHighWatermark() API that reads the high watermark file to get the latest high watermark for a particular topic/partition. This method is called once per partition on startup, and during every make follower state change. 3. HighWatermarkPersistenceTest Added a couple of unit tests to verify that the new high watermark persistence code is working. This will further get tested during system testing, once KAFKA-350 is checked in > 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-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