Return-Path: X-Original-To: apmail-kafka-dev-archive@www.apache.org Delivered-To: apmail-kafka-dev-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id BA5FBEE2D for ; Mon, 28 Jan 2013 21:43:12 +0000 (UTC) Received: (qmail 77875 invoked by uid 500); 28 Jan 2013 21:43:12 -0000 Delivered-To: apmail-kafka-dev-archive@kafka.apache.org Received: (qmail 77827 invoked by uid 500); 28 Jan 2013 21:43:12 -0000 Mailing-List: contact dev-help@kafka.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@kafka.apache.org Delivered-To: mailing list dev@kafka.apache.org Received: (qmail 77818 invoked by uid 99); 28 Jan 2013 21:43:12 -0000 Received: from arcas.apache.org (HELO arcas.apache.org) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 28 Jan 2013 21:43:12 +0000 Date: Mon, 28 Jan 2013 21:43:12 +0000 (UTC) From: "Jay Kreps (JIRA)" To: dev@kafka.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Updated] (KAFKA-631) Implement log compaction MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ 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-v8.patch Patch v8 includes Jun's comments. Specifically: Cleaner: 70.1 Nice catch. Buffers now grow in offset map building. Also changed both offset map building and cleaning to keep the same buffer size for the duration of the segment to avoid growing and shrinking too frequently. 70.2 The message payload can be null and this is used to indicate a delete (note that null messages do go into the offset map but never survive a cleaning). Currently though there is no way to set the payload to null and a number of bugs around null payloads. I will be opening a ticket to solve those. 70.3 Usually catching Throwable is a mistake, I think. I.e. if we are out of memory, the thread should die. Log: 71.1 Removed the comment about rebuilding indexes. 71.2 Improved formatting for log statement in maybeRoll() 71.3 Nice catch. Incrementing truncates count in truncateFullyAndStartAt() KafkaConfig 72. It would be easily to implement something where the log cleaner starts only if we have a log with dedupe. However it is a little trickier with topics that are dynamically added or for which the config is changed dynamically. I would like to leave it simple/stupid for now and when we have the config change stuff ironed out make the cleaner dynamically start when the first log becomes dedupe-enabled. LogOffsetTest: 73. Changed log.retention.check.interval.ms to 5*60*1000LogOffsetTest.createBrokerConfig() 74. Added apache header to CleanerConfig, LogConfig, TestLogCleaning. 75. Added a comment for TestLogCleaning. 76. The bytes of a cryptographic hash are supposed to be uniformly distributed, so just using the first 4 bytes should be fine (I have previously tested this and it works well empirically too). > 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, KAFKA-631-v8.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