[ 
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

        

Reply via email to