[ 
https://issues.apache.org/jira/browse/KAFKA-631?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jay Kreps updated KAFKA-631:
----------------------------

    Attachment: KAFKA-631-v5.patch

Great review. Attached patch v5 that addresses most of these issues:

1.1 Fixed "enableClenaer", dedupe is actually a word and is spelled dedupe, 
though, I think…
2. Changed
3.1 This is hard to explain, but changed it to "the minimum ratio of dirty log 
to total log for a log to eligible for cleaning"
3.2 Changed to ms.
3.3 Done
4.1. Done
4.2 Ah, nice catch. Fixed. Added test for it.
5.1 "Confusing but sophisticated" is my middle name. Basically I didn't like 
the code duplication and it seemed nice to see all the criteria whenever we 
roll. Fixed the ordering.
6.1 Fixed
6.2 I think you are saying we could change this to a require() call, right? 
Made that change.
6.3 Argh, you're right. I didn't think of that problem. It isn't easily 
fixable. Let's continue the review and I will think of a fix for this as a 
follow-up item. It isn't a critical problem because effectively you just 
duplicate a segment of the log needlessly (but with very low probability). The 
old segment will mask that portion of the new segment, but I don't think there 
is any other bad thing that happens.
6.4 CleanerTest.testSegmentGrouping() is a beast
6.5 Done
6.6 It can but it seems reasonable to ask for the last known cleaner point?
7.1 Fixed
7.2 3MB should be enough for anyone. :-) No the real reason is because I 
require you to have a 1MB read buffer, a 1MB write buffer which I cleverly 
subtract from the cleaner total buffer size. I don't think we need to make 
these configurable since 1MB is a good size (bigger won't help, and smaller 
will hurt). So you must have at least 2MB, but if you are trying to set a 
dedupe buffer that is less than 1MB well that is crazy. Maybe this is just me 
being a little too accurate about memory accounting and a better approach would 
just be to not count the I/O buffers at all. In that case the question is what 
is the minimum we should set for the cleaner buffer?
8.1 We can't do topicOverrides since these are full log configs not overrides. 
I suppose topicConfigs is better in case there was a question of what the 
String in the map was. Changed that.
9.1 Fixed.
10.1 Fixed
11. Not sure I follow. If you update the time manually and then call tick() we 
basically do "catch up" executing the tasks in order of next execution and 
cycling their period until we are caught up. The key point is that the user is 
the one who advances the time not the scheduler. That is the user says "it is 
now 12:15" and we execute our backlog of tasks. Perhaps you are saying that it 
should work the other way where the scheduler advances the clock rather than 
vice versa?


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