[ 
https://issues.apache.org/jira/browse/KAFKA-631?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13564072#comment-13564072
 ] 

Jun Rao commented on KAFKA-631:
-------------------------------

Thanks for patch v7. Looks good overall. Some comments:

70. LogCleaner:
70.1 buildOffsetMap(): need to consider grow readBuffer to accomodate for 
maxMessageSize.
70.2 celanInto(): Can the payload ever be null?
        val retainRecord = lastOffset < 0 || (entry.offset >= lastOffset && 
entry.message.payload != null)
70.3 CleanerThread.run(): Should we catch all Throwables, instead of Exceptions?

71. Log:
71.1 loadSegments(): The following comment is no longer true since it can 
happen to a segment with SwapFileSuffix.
        if(!hasIndex) {
          // this can only happen if someone manually deletes the index file
71.2 maybeRoll(): move .format in debug to a separate line.
71.3 truncateFullyAndStartAt(): This one behaves in the same way as truncateTo 
and is called directly from ReplicaFetcherThread. So need to increment 
truncates here too.

72. KafkaConfig: Why do we have log.cleaner.enable? Shouldn't log cleaner be 
automatically enabled if logCleanupPolicy is dedup?


                
> Implement log compaction
> ------------------------
>
>                 Key: KAFKA-631
>                 URL: https://issues.apache.org/jira/browse/KAFKA-631
>             Project: Kafka
>          Issue Type: New Feature
>          Components: core
>    Affects Versions: 0.8.1
>            Reporter: Jay Kreps
>            Assignee: Jay Kreps
>         Attachments: KAFKA-631-v1.patch, KAFKA-631-v2.patch, 
> KAFKA-631-v3.patch, KAFKA-631-v4.patch, KAFKA-631-v5.patch, 
> KAFKA-631-v6.patch, KAFKA-631-v7.patch
>
>
> Currently Kafka has only one way to bound the space of the log, namely by 
> deleting old segments. The policy that controls which segments are deleted 
> can be configured based either on the number of bytes to retain or the age of 
> the messages. This makes sense for event or log data which has no notion of 
> primary key. However lots of data has a primary key and consists of updates 
> by primary key. For this data it would be nice to be able to ensure that the 
> log contained at least the last version of every key.
> As an example, say that the Kafka topic contains a sequence of User Account 
> messages, each capturing the current state of a given user account. Rather 
> than simply discarding old segments, since the set of user accounts is 
> finite, it might make more sense to delete individual records that have been 
> made obsolete by a more recent update for the same key. This would ensure 
> that the topic contained at least the current state of each record.

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

Reply via email to