You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Neha Narkhede (JIRA)" <ji...@apache.org> on 2012/07/16 23:10:34 UTC

[jira] [Created] (KAFKA-405) Improve the high water mark maintenance to store high watermarks for all partitions in a single file on disk

Neha Narkhede created KAFKA-405:
-----------------------------------

             Summary: 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


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

        

[jira] [Commented] (KAFKA-405) Improve the high water mark maintenance to store high watermarks for all partitions in a single file on disk

Posted by "Jay Kreps (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/KAFKA-405?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13420740#comment-13420740 ] 

Jay Kreps commented on KAFKA-405:
---------------------------------

Also, ReplicaManager:

This class has a very odd public interface. Instead of managing replicas it has a bunch of passive calls--addLocalReplica(), addRemoteReplica(), etc. Who calls these? KafkaServer seems to have its own wrapper for these. Then it passes these methods on as arguments to KafkaZookeeper. Does this make sense? I think it would be good if ReplicaManager handled replica management, even if that means it depends on zookeeper.


                
> 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-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

        

[jira] [Commented] (KAFKA-405) Improve the high water mark maintenance to store high watermarks for all partitions in a single file on disk

Posted by "Jay Kreps (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/KAFKA-405?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13420731#comment-13420731 ] 

Jay Kreps commented on KAFKA-405:
---------------------------------

4. The concern I have is that fs writes are not atomic unless they are < 1 block. What happens if the broker fails in the middle of a write? Also is there a reason we can't just have a plain text file? That will be a little bulkier, but the good thing is you can cat it and see what is there. I think that will be a lot nicer operationally then another binary format...
5. Can we do that without the Thread.sleeps? For example it would seem this would be accomplished by not using a time based flush interval. Also
9. Can you add that facility then to KafkaScheduler? Use Thread.setName() with a wrapper runnable. I think making lots of single-threaded thread pools is too hacky.

Also
- FileChannel.truncateTo sets the size to whatever is given and calls truncate, it would be good to be a little more defensive. If the offset given is larger than size we should handle that gracefully (throw illegalargumentexception or something). Currently it would call truncate() on the filechannel which would have no effect but it would set the size to the new size which would not match the size of the file, which might cause odd things to happen.
- HighwaterMarkCheckpoint.scala: new RandomAccessFile(path + "/" + HighwaterMarkCheckpoint.highWatermarkFileName. Should use new File(path, filename) for portability.
- Can you mark any method not in the public interface for ReplicaManager as private? It is currently really hard to tell what the capabilities it provides...

                
> 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-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

        

[jira] [Updated] (KAFKA-405) Improve the high water mark maintenance to store high watermarks for all partitions in a single file on disk

Posted by "Neha Narkhede (JIRA)" <ji...@apache.org>.
     [ 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-v2.patch

1. Good point. Changed the name to Log.truncateTo(targetOffset)
2. It is possible to truncate multiple segments. The truncateTo API handles that. It deletes segments that have start offset > targetOffset. Only one segment will ever need to be truncated. Rest will have to be deleted.
3. Changed FileMessageSet.truncateUpto to truncateTo to make it consistent with Log.truncateTo
4. Created wrapper HighwatermarkCheckpoint. Documented the file format here. I had thought about atomic updates but there is only one thread that serializes the checkpoints, so didn't think swapping the old file with the new one would be required, no ?
5. As for unit testing, I've added a new test HighwatermarkPersistenceTest that tests the writing/reading high watermark values for single as well as multiple partitions.
6. I think it is a good idea to version the high watermark file, just in case we didn't cover something that we need to in the future
7. We have a JIRA open for fixing all getters/setters, so I'll defer that change. The logEndOffset logic is a little tricky. It seems correct to not expose a separate API to set the logEndOffsetUpdateTime and just let the logEndOffset setter API do it. But, here is the problem. The leader needs to record its own log end offset update time while appending messages to local log. However, since the Log doesn't know anything about logEndOffsetUpdateTime, its append API cannot set the udpate time. Also, the leader cannot use the logEndOffset setter API since its log end offset is recorded by its local Log object. The logEndOffset setter API is meant only to record the follower's log end offset. But since it makes sense for the update time to be updated while setting the logEndOffset, I've fixed it. Basically, the logEndOffset() setter API updates only the logEndOffset time when a local log exists for the replica. For all other cases, it updates both the logEndOffset as well as the logEndOffsetUpdateTime 
8. Yeah, there are several callers that use the getReplica() API and don't always re-throw an exception. Some are re-throwing an error while others are using the Option to return some default value for some state of the Replica (highwatermark). And case match in Scala is good for that since it always evaluates to a value, a try catch block doesn't. But if all the callers throw an exception, then it makes sense to have getReplica throw it instead. 
9. You have a valid point about KafkaScheduler usage. However, we name the thread appropriately with every instance of the scheduler. Ideally, if there was a way to override the base thread name independently with the same scheduler, it would be possible to use a single scheduler.
10. Good point about abbreviations. Fixed that. Let's standardize on this.

Also, changed the name to updateLEO to updateLeo

                
> 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-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

       

[jira] [Updated] (KAFKA-405) Improve the high water mark maintenance to store high watermarks for all partitions in a single file on disk

Posted by "Neha Narkhede (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/KAFKA-405?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Neha Narkhede updated KAFKA-405:
--------------------------------

    Attachment:     (was: kafka-405-v2.patch)
    
> 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-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

        

[jira] [Work started] (KAFKA-405) Improve the high water mark maintenance to store high watermarks for all partitions in a single file on disk

Posted by "Neha Narkhede (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/KAFKA-405?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Work on KAFKA-405 started by Neha Narkhede.

> 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
>
> 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

        

[jira] [Updated] (KAFKA-405) Improve the high water mark maintenance to store high watermarks for all partitions in a single file on disk

Posted by "Neha Narkhede (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/KAFKA-405?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Neha Narkhede updated KAFKA-405:
--------------------------------

    Status: Patch Available  (was: In Progress)
    
> 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

        

[jira] [Assigned] (KAFKA-405) Improve the high water mark maintenance to store high watermarks for all partitions in a single file on disk

Posted by "Neha Narkhede (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/KAFKA-405?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Neha Narkhede reassigned KAFKA-405:
-----------------------------------

    Assignee: Neha Narkhede
    
> 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
>
> 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

        

[jira] [Updated] (KAFKA-405) Improve the high water mark maintenance to store high watermarks for all partitions in a single file on disk

Posted by "Neha Narkhede (JIRA)" <ji...@apache.org>.
     [ 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

        

[jira] [Commented] (KAFKA-405) Improve the high water mark maintenance to store high watermarks for all partitions in a single file on disk

Posted by "Jay Kreps (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/KAFKA-405?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13417913#comment-13417913 ] 

Jay Kreps commented on KAFKA-405:
---------------------------------

Some of these comments are not directly related to this change but this is just the first time I have looked at this code in detail

- I think Log.recoverUptoLastCheckpointedHighWatermark(lastKnownHW: Long) is misnamed. I don't think the log should know anything about hw marks, and it also isn't doing recovery (e.g. checking the validity of the log), I think it is just blindly truncating the log. Can we change it to Log.truncateTo
- Is it possible that we need to truncate more than one segment? I.e. couldn't the segment to be truncated not be the last segment (unlikely with 1gb segments, but still a problem)
- Can we change the api in MessageSet.truncateUpto should be truncateUpTo or truncateTo
- Can you make a wrapper for the RandomAccessFile called HighwaterMarkCheckpoint and put the logic related to that there. Intuitively the logic for serializing a checkpoint shouldn't be mixed into ReplicaManager. Can you also document the file format? Is there a reason this can't be plain text? Also I think a better approach to the updates would be to create a new file, write to it, and then move it over the old one; this will make the update atomic. Not sure if that is needed...
- It would be good to have a test that covered the HighwaterMarkCheckpoint file read and write. Just basic reading/writing, nothing fancy.
- Do we need to version the hw mark checkpoint file format? I.e. maybe the first line of the file is version=x or something... Not sure if that is needed but I am paranoid about persistent formats after blowing that and being stuck. This would let format changes be handled automatically.
- Can we fix the setters/getters in Replica.scala and make changes to the leo update the leoUpdateTime. Currently I think it is encumbant on the caller to do these two things together which is odd...
- I think the use of KafkaScheduler is not quite right. This is a thread pool meant to execute many tasks. There should really only be one for all of kafka, not one per background thread. You should probably pass in a central instance as an argument rather than making two new ones.
- Also I notice that ReplicaManager.getReplica returns an Option. But then everyone who calls it needs to check the option and return an exception if it is not found. Can we just have getReplica either return the Replica or throw the exception?
- I think abbreviations should be in the form updateLeo not updateLEO and updateIsr not updateISR. Let's standardize that.
                
> 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

        

[jira] [Updated] (KAFKA-405) Improve the high water mark maintenance to store high watermarks for all partitions in a single file on disk

Posted by "Neha Narkhede (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/KAFKA-405?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Neha Narkhede updated KAFKA-405:
--------------------------------

    Resolution: Fixed
        Status: Resolved  (was: Patch Available)

Thanks a lot for the timely reviews ! Checked in v4 
                
> 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-405-v4.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

        

[jira] [Commented] (KAFKA-405) Improve the high water mark maintenance to store high watermarks for all partitions in a single file on disk

Posted by "Jun Rao (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/KAFKA-405?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13426790#comment-13426790 ] 

Jun Rao commented on KAFKA-405:
-------------------------------

Just had a look of v4. A couple of minor comments:

40. HighwaterMarkCheckpoint:
40.1 If tempHwFile already exists, we can just overwrite it since we know hwFile is always safe.
40.2 There is no need to delete hwFile first and then rename tempHwFile to it. Rename should do the deletion. Currently, if we fail at the bad time, we could end up without a hwFile.
                
> 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-405-v4.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

        

[jira] [Updated] (KAFKA-405) Improve the high water mark maintenance to store high watermarks for all partitions in a single file on disk

Posted by "Neha Narkhede (JIRA)" <ji...@apache.org>.
     [ 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

        

[jira] [Updated] (KAFKA-405) Improve the high water mark maintenance to store high watermarks for all partitions in a single file on disk

Posted by "Neha Narkhede (JIRA)" <ji...@apache.org>.
     [ 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-v4.patch

Removed the sleeps in LogRecoveryTest. Will probably checkin this version of the patch
                
> 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-405-v4.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

        

[jira] [Commented] (KAFKA-405) Improve the high water mark maintenance to store high watermarks for all partitions in a single file on disk

Posted by "Jay Kreps (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/KAFKA-405?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13422713#comment-13422713 ] 

Jay Kreps commented on KAFKA-405:
---------------------------------

+1 
                
> 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

        

[jira] [Commented] (KAFKA-405) Improve the high water mark maintenance to store high watermarks for all partitions in a single file on disk

Posted by "Jun Rao (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/KAFKA-405?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13421086#comment-13421086 ] 

Jun Rao commented on KAFKA-405:
-------------------------------

Thanks for patch v2. Some comments:
20. Log.truncateTo(): The following code seems to be used just for getting the first segment. Can we just use segmentToBeTruncated(0)?
      segmentToBeTruncated match {
        case Some(segment) =>
          val truncatedSegmentIndex = segments.view.indexOf(segment)
          segments.truncLast(truncatedSegmentIndex)
        case None =>
      }

21. FileMessageSet: Do we need setHighWaterMark? It seems it's always the same as setSize.

22. ReplicaManager:
22.1 recordLeaderLogUpdate(): Could we rename it to recordLeaderLogEndOffset()?
22.2 close(): Could we rename it to shutdown to map startup()?
22.3 readCheckpointedHighWatermark(): We should just read the HW from memory. The on-disk version is only useful on broker startup when we populate the in-memory HW using the on disk version.

23. HighwaterMarkCheckpoint: Is it better to name the file ".highwaterMark" so that it's hidden?

                
> 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-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

        

[jira] [Updated] (KAFKA-405) Improve the high water mark maintenance to store high watermarks for all partitions in a single file on disk

Posted by "Neha Narkhede (JIRA)" <ji...@apache.org>.
     [ 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-v2.patch
    
> 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-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