Re: log compaction across partitions

2021-01-05 Thread Shaohan Yin
Hi, Normally the same keys will lead to the same partition. And log compaction works on a single topic-partition. Cheers On Wed, 6 Jan 2021 at 07:40, Jacob Botuck wrote: > If I somehow have 2 records with the same key in different partitions in a > topic with log compaction enabled, wi

log compaction across partitions

2021-01-05 Thread Jacob Botuck
If I somehow have 2 records with the same key in different partitions in a topic with log compaction enabled, will log compaction remove the old record? Or do the records need to be in the same partition for log compaction to work?

Re: kafka log compaction

2020-06-18 Thread Ricardo Ferreira
Pushkar, "1. Would setting the cleanup policy to compact (and No delete) would always retain the latest value for a key?" -- Yes. This is the purpose of this setting. "2. Does parameters like segment.bytes, retention.ms also play any role in compaction?" -- They don't play any role in compacti

kafka log compaction

2020-06-17 Thread Pushkar Deole
Hi All I want some of my topics to retain data forever without any deletion since those topics hold static data that is always required by application. Also, for these topic I want to retain latest value for key. I believe the cleanup policy of 'compact' would meet my needs. I have following quest

Re: AW: Configuration of log compaction

2019-04-29 Thread Ethan Stein
rt the log cleaner manually without restarting the whole > broker?> > > > > Thanks for your help.> > > > > Best,> > > Claudia> > > > > -Ursprüngliche Nachricht-> > > Von: Liam Clarke > > > Gesendet

AW: Configuration of log compaction

2019-02-28 Thread Claudia Wegmann
etreff: AW: Configuration of log compaction Hi Liam, My brokers are running in version 2.0.0 Restarting was not a problem, it just seemed "a lot" for just restarting the log cleaner. I will think about piping the logs of the log cleaner to elk. And I will definitely keep an eye on the logs to see

AW: Configuration of log compaction

2018-12-18 Thread Claudia Wegmann
anks again! Best, Claudia -Ursprüngliche Nachricht- Von: Liam Clarke Gesendet: Dienstag, 18. Dezember 2018 23:05 An: users@kafka.apache.org Betreff: Re: Configuration of log compaction Kia ora Claudia, What version Kafka are you running? I can't find the specified code in Kafk

Re: Configuration of log compaction

2018-12-18 Thread Liam Clarke
anks for your help. > > Best, > Claudia > > -Ursprüngliche Nachricht- > Von: Liam Clarke > Gesendet: Montag, 17. Dezember 2018 23:06 > An: users@kafka.apache.org > Betreff: Re: Configuration of log compaction > > Hi Claudia, > > Anything useful in the

AW: Configuration of log compaction

2018-12-18 Thread Claudia Wegmann
: Re: Configuration of log compaction Hi Claudia, Anything useful in the log cleaner log files? Cheers, Liam Clarke On Tue, 18 Dec. 2018, 3:18 am Claudia Wegmann Hi, > > thanks for the quick response. > > My problem is not, that no new segments are created, but that segments >

Re: Configuration of log compaction

2018-12-17 Thread Liam Clarke
> Gesendet: Montag, 17. Dezember 2018 14:28 > An: users@kafka.apache.org > Betreff: Re: Configuration of log compaction > > Hello! > Please check whether the segment.ms configuration on topic will help > you to solve your problem. > > https://kafka.apache.org/documentation/ &

RE: Configuration of log compaction

2018-12-17 Thread Komal Babu
: 17 December 2018 14:18 To: users@kafka.apache.org Subject: AW: Configuration of log compaction Hi, thanks for the quick response. My problem is not, that no new segments are created, but that segments with old data do not get compacted. I had to restart one broker because there was no diskspace

AW: Configuration of log compaction

2018-12-17 Thread Claudia Wegmann
something? Best, Claudia -Ursprüngliche Nachricht- Von: Spico Florin Gesendet: Montag, 17. Dezember 2018 14:28 An: users@kafka.apache.org Betreff: Re: Configuration of log compaction Hello! Please check whether the segment.ms configuration on topic will help you to solve your problem

Re: Configuration of log compaction

2018-12-17 Thread Spico Florin
is running wild for some time now and > seems to grow indefinitely. When I check the log file of the first > segment, there is a lot of data in it, that should have been compacted > already. > > So I guess I did not configure everything correctly for log compaction to > work as e

Configuration of log compaction

2018-12-17 Thread Claudia Wegmann
unning wild for some time now and seems to grow indefinitely. When I check the log file of the first segment, there is a lot of data in it, that should have been compacted already. So I guess I did not configure everything correctly for log compaction to work as expected. What config par

Re: manually trigger log compaction

2018-10-03 Thread Brett Rann
ng kafka-configs script. You need 'topics' as entity > type and --alter directive. The changes are made cluster-wide. > > Try the help documentation for kafka-configs. > > Regards, > > On Wed, 26 Sep 2018 at 16:16, Xu, Nan wrote: > > > Hi, > > > >

Re: manually trigger log compaction

2018-09-26 Thread M. Manna
to manually trigger a log compaction for a > certain topic? > > Thanks, > Nan > > -- > This message, and any attachments, is for the intended recipient(s) only, > may contain information that is privil

manually trigger log compaction

2018-09-26 Thread Xu, Nan
Hi, Wondering is there a way to manually trigger a log compaction for a certain topic? Thanks, Nan -- This message, and any attachments, is for the intended recipient(s) only, may contain information that is privileged

Re: How set log compaction policies at cluster level

2018-06-14 Thread Manikumar
apache.org/documentation/#topicconfigs On Thu, Jun 14, 2018 at 3:51 PM David Espinosa wrote: > Hi all, > > I would like to apply log compaction configuration for any topic in my > kafka cluster, as default properties. These configuration properties are: > >- cleanup.polic

How set log compaction policies at cluster level

2018-06-14 Thread David Espinosa
Hi all, I would like to apply log compaction configuration for any topic in my kafka cluster, as default properties. These configuration properties are: - cleanup.policy - delete.retention.ms - segment.ms - min.cleanable.dirty.ratio I have tried to place them in the

Log Compaction configuration over all topics in cluster

2018-05-09 Thread David Espinosa
Hi all, I would like to apply log compaction configuration for any topic in my kafka cluster, as default properties. These configuration properties are: - cleanup.policy - delete.retention.ms - segment.ms - min.cleanable.dirty.ratio I have tried to place them in the server.properties

Kafka Log Compaction (LogCleaner) and retry

2018-04-23 Thread Rabin Banerjee
Hi All, I have a doubt regarding what will happen if there is a retry to a message and log compaction is enabled. retries 10 retry.backoff.ms 1000 For example the data looks like this 1-->State1 (sent) 1-->State2 (failed, waiting for retry) 1-->State3 (sent) 1-->State2 (State2 is

RE: log compaction v log rotation - best of the two worlds

2018-03-21 Thread Kopacki, Tomasz (Nokia - PL/Wroclaw)
karls...@csi.se] Sent: Wednesday, March 21, 2018 11:21 AM To: users@kafka.apache.org Subject: Re: log compaction v log rotation - best of the two worlds alt1) if you can store a generation counter in the value of the "latest value" topic you could do as follows topic latest_value key [id] top

Re: log compaction v log rotation - best of the two worlds

2018-03-21 Thread Svante Karlsson
> > Tomasz Kopacki > > DevOps Engineer @ Nokia > > > > -Original Message- > > From: Manikumar [mailto:manikumar.re...@gmail.com] > > Sent: Wednesday, March 21, 2018 11:03 AM > > To: users@kafka.apache.org > > Subject: Re: log compaction v log rot

Re: log compaction v log rotation - best of the two worlds

2018-03-21 Thread Manikumar
policy is 'delete' and it > still work ? > > Sincerely, > Tomasz Kopacki > DevOps Engineer @ Nokia > > -Original Message- > From: Manikumar [mailto:manikumar.re...@gmail.com] > Sent: Wednesday, March 21, 2018 11:03 AM > To: users@kafka.apache.org > Subj

RE: log compaction v log rotation - best of the two worlds

2018-03-21 Thread Kopacki, Tomasz (Nokia - PL/Wroclaw)
ubject: Re: log compaction v log rotation - best of the two worlds Not sure if understood requirement correctly. one option is to use two compacted topic topics. one is for current state of the resource and one is for history. and use tombstones whenever you want to clear them. On Wed, Mar 21, 20

Re: log compaction v log rotation - best of the two worlds

2018-03-21 Thread Manikumar
; From: Manikumar [mailto:manikumar.re...@gmail.com] > Sent: Wednesday, March 21, 2018 10:17 AM > To: users@kafka.apache.org > Subject: Re: log compaction v log rotation - best of the two worlds > > We can enable both compaction and retention for a topic by setting > cleanup.policy=&q

RE: log compaction v log rotation - best of the two worlds

2018-03-21 Thread Kopacki, Tomasz (Nokia - PL/Wroclaw)
; messages because I need to do this based of the lifecycle of the resource not just their age. Sincerely, Tomasz Kopacki DevOps Engineer @ Nokia -Original Message- From: Manikumar [mailto:manikumar.re...@gmail.com] Sent: Wednesday, March 21, 2018 10:17 AM To: users@kafka.apache.org Su

Re: log compaction v log rotation - best of the two worlds

2018-03-21 Thread Manikumar
gt; wrote: > Hi, > I've been recently exploring log handling in kafka and I wonder if/how can > I mixed log compaction with log rotation. > A little background first: > I have an application that uses kafka topics as a backend for event > sourcing. Messages represents

log compaction v log rotation - best of the two worlds

2018-03-21 Thread Kopacki, Tomasz (Nokia - PL/Wroclaw)
Hi, I've been recently exploring log handling in kafka and I wonder if/how can I mixed log compaction with log rotation. A little background first: I have an application that uses kafka topics as a backend for event sourcing. Messages represents change of state of my 'resources'

Re: Log Compaction Not Picking up Topic [solved]

2017-10-25 Thread Elmar Weber
Hello Xin, hello Jan, worked perfectly. I did a build of an image based on 0.11.0.1 and applied the missing patch, cleaning went through and resulted in the expected size. Thanks a lot for the quick help, Elmar On 10/25/2017 1:03 PM, Xin Li wrote: Hey Elmar, The only thing you need to do i

Re: Log Compaction Not Picking up Topic

2017-10-25 Thread Xin Li
Hey Elmar, The only thing you need to do is upgrade, Kafka track cleaned offset using cleaner-offset-checkpoint file. Best, Xin Xin Li Data EngineeringXin.Li@ trivago.com www.trivago.com

Re: Log Compaction Not Picking up Topic

2017-10-25 Thread Xin Li
Hey, Because of the overflow the calculation for dirty ratios is minus, and I guess upgrade is the one time for good fix. And we running that for quite a while, so far so good. Best, Xin Xin Li Data EngineeringXin.Li@ trivago.com <

Re: Log Compaction Not Picking up Topic

2017-10-25 Thread Elmar Weber
Hi, thanks, I'll give it a try, we run on Kubernetes so it's not a big issue to replicate the whole env including data. One question I'd have left: - How can I force a re-compaction over the whole topic? Because I guess the Log Cleaner market everything so far as not able to clean, how wil

Re: Log Compaction Not Picking up Topic

2017-10-25 Thread Jan Filipiak
Hi, unfortunatly there is nothing trivial you could do here. Without upgrading your kafkas you can only bounce the partition back and forth between brokers so they compact while its still small. With upgrading you could also just cherrypick this very commit or put a logstatement to verify.

Re: Log Compaction Not Picking up Topic

2017-10-25 Thread Elmar Weber
Hi, On 10/25/2017 12:15 PM, Xin Li wrote: > I think that is a bug, and should be fixed in this task https://issues.apache.org/jira/browse/KAFKA-6030. > We experience that in our kafka cluster, we just check out the 11.0.2 version, build it ourselves. thanks for the hint, as it looks like a c

Re: Log Compaction Not Picking up Topic

2017-10-25 Thread Xin Li
Hey, I think that is a bug, and should be fixed in this task https://issues.apache.org/jira/browse/KAFKA-6030. We experience that in our kafka cluster, we just check out the 11.0.2 version, build it ourselves. Best, Xin Xin Li Data EngineeringXin.Li@

Re: Log Compaction Not Picking up Topic

2017-10-25 Thread Elmar Weber
On 10/25/2017 12:03 PM, Manikumar wrote: any errors in log cleaner logs? Not as far as I can see root@kafka-1:/opt/kafka/logs# cat log-cleaner.log* | grep -i error (empty) However, I've seen that it actually did cleaning of the whole topic (excerpts below), but it didn't seem to find anythin

Re: Log Compaction Not Picking up Topic

2017-10-25 Thread Manikumar
any errors in log cleaner logs? On Wed, Oct 25, 2017 at 3:12 PM, Elmar Weber wrote: > Hello, > > I'm having trouble getting Kafka to compact a topic. It's over 300GB and > has enough segments to warrant cleaning. It should only be about 40 GB > (there is a copy in a db that is unique on the key)

Log Compaction Not Picking up Topic

2017-10-25 Thread Elmar Weber
Hello, I'm having trouble getting Kafka to compact a topic. It's over 300GB and has enough segments to warrant cleaning. It should only be about 40 GB (there is a copy in a db that is unique on the key). Below are the configs we have (default broker) and topic override. Is there something I

Re: The idea of "composite key" to make log compaction more flexible - question / proposal

2017-10-09 Thread Michal Michalski
custom partitioner, no? > > https://kafka.apache.org/0110/javadoc/org/apache/kafka/ > clients/producer/Partitioner.html > > -Jay > > On Mon, Oct 2, 2017 at 6:29 AM Michal Michalski < > michal.michal...@zalando.ie> > wrote: > > > Hi, > > > >

Re: The idea of "composite key" to make log compaction more flexible - question / proposal

2017-10-05 Thread Jay Kreps
I think you can do this now by using a custom partitioner, no? https://kafka.apache.org/0110/javadoc/org/apache/kafka/clients/producer/Partitioner.html -Jay On Mon, Oct 2, 2017 at 6:29 AM Michal Michalski wrote: > Hi, > > TL;DR: I'd love to be able to make log compaction more &

The idea of "composite key" to make log compaction more flexible - question / proposal

2017-10-02 Thread Michal Michalski
Hi, TL;DR: I'd love to be able to make log compaction more "granular" than just per-partition-key, so I was thinking about the concept of a "composite key", where partitioning logic is using one part of the key, while compaction uses the whole key - is this something des

Re: Log compaction failed because offset map doesn't have enough space

2017-05-17 Thread Tom Crayford
Hi, You should upgrade Kafka versions, this was a bug fixed in KAFKA-3894: https://issues.apache.org/jira/browse/KAFKA-3894 Generally it's a very good idea to keep on top of Kafka version upgrades, there are numerous bugs fixed with every release, and it's stability goes up each time. On Tue, Ma

Log compaction failed because offset map doesn't have enough space

2017-05-16 Thread Jun Ma
Hi team, We are having a issue with compacting __consumer_offsets topic in our cluster. We’re seeing logs in log-cleaner.log saying: [2017-05-16 11:56:28,993] INFO Cleaner 0: Building offset map for log __consumer_offsets-15 for 349 segments in offset range [0, 619265471). (kafka.log.LogCleaner)

Re: using kafka log compaction withour key

2017-01-30 Thread Ewen Cheslack-Postava
The log compaction functionality uses the key to determine which records to deduplicate. You can think of it (very roughly) as deleting entries from a hash map as the value for each key is overwritten. This functionality doesn't have much of a point unless you include keys in your records.

using kafka log compaction withour key

2017-01-26 Thread Samy CHBINOU
Hello, Is is possible to use log compaction without key? I think in that case buffer will contain only one line of data value? Is that correct? thanks

Re: log compaction

2016-11-04 Thread Francesco laTorre
iated :) Cheers, Francesco On 4 November 2016 at 01:38, Becket Qin wrote: > Hi Francesco, > > There are a few things to think about before turning on log compaction for > a topic. > > 1. Does the topic have non-keyed message? Log compaction only works if all > the messages have a

Re: log compaction

2016-11-03 Thread Becket Qin
Hi Francesco, There are a few things to think about before turning on log compaction for a topic. 1. Does the topic have non-keyed message? Log compaction only works if all the messages have a key. 2. The log cleaner needs some memory to build the offset map for log compaction, so the memory

log compaction

2016-11-02 Thread Francesco laTorre
Hi, We want to enable log compaction on an existing topic (in production). Is it a safe operation or there are things to take into consideration ? Kafka version 0.8 Cheers, Francesco -- <http://www.openbet.com/> Francesco laTorre Senior Developer T: +44 208 742 1600 +44 203 249 8

Re: Process to enable log compaction on a cluster

2016-10-12 Thread Mario Ricci
__consumer_offset topic and > > also delete certain topics. I would like to know the right process to do > > this. > > > > Now since I am having close 1.8T of data on __consumer_offset topic and > > more in the topics data, any log compaction and log deletion/trunction is

Log Compaction: How Key comparison works?

2016-09-27 Thread Kamal C
Hi all, The log compaction article [1] doesn't explains how key comparison takes place. AFAIK, Kafka don't de-serialize the records and using MD5 algorithm. I'm using Kafka java client-v10. Could someone explain whether the below statements are correct: 1. Key can be of any da

Re: Verify log compaction

2016-08-11 Thread Harald Kirsch
at 4:10 PM David Yu wrote: Hi, We are using Kafka 0.9.0.0. One of our topic is set to use log compaction. We have also set log.cleaner.enable. However, we suspected that the topic is not being compacted. What is the best way for us to verify the compaction is happening? Thanks, David

Re: Verify log compaction

2016-08-10 Thread David Yu
hread > death in the log-cleaner.log. > > > On Fri, Jul 29, 2016 at 4:10 PM David Yu wrote: > > > Hi, > > > > We are using Kafka 0.9.0.0. One of our topic is set to use log > compaction. > > We have also set log.cleaner.enable. However, we suspected that the

Re: Log compaction leaves empty files?

2016-08-09 Thread Dustin Cote
;> I note that your last modified times are all the same. Are you maybe >> using >> Java 7? There's some details here that a JDK bug in Java 7 causes the >> last >> modified time to get updated on broker restart: >> https://issues.apache.org/jira/browse/KAFKA-3802

Re: Log compaction leaves empty files?

2016-08-08 Thread Harald Kirsch
? There's some details here that a JDK bug in Java 7 causes the last modified time to get updated on broker restart: https://issues.apache.org/jira/browse/KAFKA-3802 On Fri, Aug 5, 2016 at 6:12 AM, Harald Kirsch wrote: Hi, experimenting with log compaction, I see Kafka go through all the

Re: Log compaction leaves empty files?

2016-08-05 Thread Dustin Cote
12 AM, Harald Kirsch wrote: > Hi, > > experimenting with log compaction, I see Kafka go through all the steps, > in particular I see positive messages in log-cleaner.log and *.deleted > files. Yet once the *.deleted segment files have disappeared, the segment > and index files wit

Log compaction leaves empty files?

2016-08-05 Thread Harald Kirsch
Hi, experimenting with log compaction, I see Kafka go through all the steps, in particular I see positive messages in log-cleaner.log and *.deleted files. Yet once the *.deleted segment files have disappeared, the segment and index files with size 0 are still kept. I stopped and restarted

Re: Verify log compaction

2016-07-29 Thread John Holland
en the __consumer_offets topic would grow out of control. Kafka logs the thread death in the log-cleaner.log. On Fri, Jul 29, 2016 at 4:10 PM David Yu wrote: > Hi, > > We are using Kafka 0.9.0.0. One of our topic is set to use log compaction. > We have also set log.cleaner.enable.

Verify log compaction

2016-07-29 Thread David Yu
Hi, We are using Kafka 0.9.0.0. One of our topic is set to use log compaction. We have also set log.cleaner.enable. However, we suspected that the topic is not being compacted. What is the best way for us to verify the compaction is happening? Thanks, David

Re: Process to enable log compaction on a cluster

2016-07-03 Thread Sathyakumar Seshachalam
retained topics . > > Now at a place, where I can afford to compact __consumer_offset topic and > also delete certain topics. I would like to know the right process to do > this. > > Now since I am having close 1.8T of data on __consumer_offset topic and > more in the topics data,

Process to enable log compaction on a cluster

2016-07-03 Thread Sathyakumar Seshachalam
know the right process to do this. Now since I am having close 1.8T of data on __consumer_offset topic and more in the topics data, any log compaction and log deletion/trunction is going to take time. Should I do this node by node. Will Kafka's replication come in the way. (I have read

Re: Log Compaction not Deleting on Kafka 0.9.0.1

2016-06-22 Thread Ismael Juma
s not > guaranteed. > > We're going to be upstreaming a ticket shortly based on this and other > issues we've seen with log compaction. > > On Wed, Jun 22, 2016 at 6:03 PM, Lawrence Weikum > wrote: > > > We seem to be having a strange issue with a cluster of

Re: Log Compaction not Deleting on Kafka 0.9.0.1

2016-06-22 Thread Tom Crayford
This smells like an bug to me. On Wed, Jun 22, 2016 at 6:54 PM, Lawrence Weikum wrote: > Fascinating. > > We are seeing no errors or warning in the logs after restart. It appears > on this broker that the compaction thread is working: > > [2016-06-22 10:33:49,179] INFO Rolled new log segment fo

Re: Log Compaction not Deleting on Kafka 0.9.0.1

2016-06-22 Thread Lawrence Weikum
Fascinating. We are seeing no errors or warning in the logs after restart. It appears on this broker that the compaction thread is working: [2016-06-22 10:33:49,179] INFO Rolled new log segment for '__consumer_offsets-28' in 1 ms. (kafka.log.Log) [2016-06-22 10:34:00,968] INFO Deleting segme

Re: Log Compaction not Deleting on Kafka 0.9.0.1

2016-06-22 Thread Tom Crayford
will keep up whilst kafka catches up with replication, but that's not guaranteed. We're going to be upstreaming a ticket shortly based on this and other issues we've seen with log compaction. On Wed, Jun 22, 2016 at 6:03 PM, Lawrence Weikum wrote: > We seem to be having

Log Compaction not Deleting on Kafka 0.9.0.1

2016-06-22 Thread Lawrence Weikum
We seem to be having a strange issue with a cluster of ours; specifically with the __consumer_offsets topic. When we brought the cluster online, log compaction was turned off. Realizing our mistake, we turned it on, but only after the topic had over 31,018,699,972 offsets committed to it

Re: Regarding Kafka Log compaction Features

2016-05-06 Thread Spico Florin
hi! please have a look at this article. it help me touse the log compaction feature mechanism i hope thtat it helps. regards, florin http://www.shayne.me/blog/2015/2015-06-25-everything-about-kafka-part-2/ On Thursday, May 5, 2016, Behera, Himansu (Contractor) < himansu_

Regarding Kafka Log compaction Features

2016-05-05 Thread Behera, Himansu (Contractor)
Hi Team, I am working on implementing the kafka log compaction feature in my project. Please find the server. Properties. I have made all the config changes needed/suggested in the kafka log compaction forum.But was not to able to resolve the issue. My use as follows: Step 1:.We send a

Re: How to work around log compaction error (0.8.2.2)

2016-04-28 Thread Rakesh Vidyadharan
Hi Manikumar, No, we are not using compression on our topics. I will try out Todd Palino’s suggestion regarding the offsets topic. Thanks Rakesh On 27/04/2016 23:34, "Manikumar Reddy" wrote: >Hi, > > Are you enabling log compaction on a topic with compressed messages?

Re: How to work around log compaction error (0.8.2.2)

2016-04-27 Thread Manikumar Reddy
Hi, Are you enabling log compaction on a topic with compressed messages? If yes, then that might be the reason for the exception. 0.8.2.2 Log Compaction does not support compressed messages. This got fixed in 0.9.0.0 (KAFKA-1641, KAFKA-1374) Check below mail thread for some corrective

How to work around log compaction error (0.8.2.2)

2016-04-27 Thread Rakesh Vidyadharan
Hello, We enabled log compaction on a few topics, as we want to preserve permanently the latest versions of messages published to specific topics. After enabling compaction, the log cleaner thread dies with the same error for the topics we tried it on. It looks like kafka has starting offset

Re: Metrics for Log Compaction

2016-04-15 Thread Manikumar Reddy
Hi, log compaction related JMX metric object names are given below. kafka.log:type=LogCleaner,name=cleaner-recopy-percent kafka.log:type=LogCleaner,name=max-buffer-utilization-percent kafka.log:type=LogCleaner,name=max-clean-time-secs kafka.log:type=LogCleanerManager,name=max-dirty-percent

Re: Metrics for Log Compaction

2016-04-15 Thread Manikumar Reddy
PM, Kasim Doctor wrote: > Hi everyone, > > We are starting to use log compaction for one of our topics and I was > wondering if there are any specific metrics exposed to monitor how often > compaction took place and/or how many records (with some metadata related > to parti

Metrics for Log Compaction

2016-04-13 Thread Kasim Doctor
Hi everyone, We are starting to use log compaction for one of our topics and I was wondering if there are any specific metrics exposed to monitor how often compaction took place and/or how many records (with some metadata related to partition) were deleted ? I looked at JMX metrics that Kafka

Re: Log Compaction v. Retention

2015-11-13 Thread Mayuresh Gharat
I think you can mark a tombstone (null value) for the keys explicitly, that will eventually delete those messages from the log. Thanks, Mayuresh On Wed, Nov 11, 2015 at 6:18 AM, Morgan Kenyon wrote: > I came across an interesting question on StackOverflow dealing with the > difference between

Re: log compaction scaling with ~100m messages

2015-10-08 Thread vipul jhawar
Is there >> any problem with this approach? >> I have heard from some people using kafka, that kafka has a problem when >> doing log compaction with those many number of keys. >> Another topic might have around 10 different K/V pairs for each key in >> the primary to

Re: log compaction scaling with ~100m messages

2015-10-08 Thread Jan Filipiak
, Feroze Daud wrote: hi! We have a use case where we want to store ~100m keys in kafka. Is there any problem with this approach? I have heard from some people using kafka, that kafka has a problem when doing log compaction with those many number of keys. Another topic might have around 10

Re: log compaction scaling with ~100m messages

2015-10-08 Thread Feroze Daud
entire dataset? On Wednesday, October 7, 2015 11:16 AM, Joel Koshy wrote: Using log compaction is well-suited for applications that use Kafka directly and need to persist some state associated with its processing. So something like offset management for consumers is a good fit

Re: log compaction scaling with ~100m messages

2015-10-07 Thread Joel Koshy
Using log compaction is well-suited for applications that use Kafka directly and need to persist some state associated with its processing. So something like offset management for consumers <http://www.slideshare.net/jjkoshy/offset-management-in-kafka> is a good fit. Another good use-case

log compaction scaling with ~100m messages

2015-10-06 Thread Feroze Daud
hi! We have a use case where we want to store ~100m keys in kafka. Is there any problem with this approach? I have heard from some people using kafka, that kafka has a problem when doing log compaction with those many number of keys. Another topic might have around 10 different K/V pairs for

Re: configuring log compaction

2015-08-09 Thread David Judd
value of a key. > > log.retention.hours is only used by (1). For log compaction, currently it > is not compacting the logs by its age, but by the dirty ratio in bytes > (uncompacted log size / total log size). The config is > log.cleaner.min.cleanable.ratio. > > So it might b

Re: configuring log compaction

2015-08-09 Thread Jiangjie Qin
Actually Kafka only support two mutually exclusive log cleanup policy: 1) delete logs after retention period passed. 2) compact the log to only keep the last updated value of a key. log.retention.hours is only used by (1). For log compaction, currently it is not compacting the logs by its age

configuring log compaction

2015-08-09 Thread David Judd
Hi, I have a beginner's question. I'd like to confirm that I'm interpreting the docs correctly. My goal is a setup where, by default, logs are compacted after 7 days and the compacted records are retained indefinitely. Does the following combination of settings achieve this? log.retention.hours=

Re: Log compaction not working as expected

2015-06-17 Thread Jan Filipiak
ents are rolled. Best Jan On 16.06.2015 14:05, Shayne S wrote: Some further information, and is this a bug? I'm using 0.8.2.1. Log compaction will only occur on the non active segments. Intentional or not, it seems that the last segment is always the active segment. In other words, an

Re: Log compaction not working as expected

2015-06-17 Thread Shayne S
t; Jan > > > On 16.06.2015 14:05, Shayne S wrote: > >> Some further information, and is this a bug? I'm using 0.8.2.1. >> >> Log compaction will only occur on the non active segments. Intentional or >> not, it seems that the last segment is always the active

Re: Log compaction not working as expected

2015-06-17 Thread Jan Filipiak
? I'm using 0.8.2.1. Log compaction will only occur on the non active segments. Intentional or not, it seems that the last segment is always the active segment. In other words, an expired segment will not be cleaned until a new segment has been created. As a result, a log won't be compa

Re: Log compaction not working as expected

2015-06-16 Thread Manikumar Reddy
> > Some improvements are proposed here, > > https://issues.apache.org/jira/browse/KAFKA-1981 > > > > > > Manikumar > > > > On Tue, Jun 16, 2015 at 5:35 PM, Shayne S wrote: > > > > > Some further information, and is this a bug? I'm usi

Re: Log compaction not working as expected

2015-06-16 Thread Shayne S
> > > Manikumar > > On Tue, Jun 16, 2015 at 5:35 PM, Shayne S wrote: > > > Some further information, and is this a bug? I'm using 0.8.2.1. > > > > Log compaction will only occur on the non active segments. Intentional > or > > not, it seems that the las

Re: Log compaction not working as expected

2015-06-16 Thread Manikumar Reddy
0.8.2.1. > > Log compaction will only occur on the non active segments. Intentional or > not, it seems that the last segment is always the active segment. In other > words, an expired segment will not be cleaned until a new segment has been > created. > > As a result, a log won&

Re: Log compaction not working as expected

2015-06-16 Thread Shayne S
Some further information, and is this a bug? I'm using 0.8.2.1. Log compaction will only occur on the non active segments. Intentional or not, it seems that the last segment is always the active segment. In other words, an expired segment will not be cleaned until a new segment has

Log compaction not working as expected

2015-06-12 Thread Shayne S
Hi, I'm new to Kafka and having trouble with log compaction. I'm attempting to set up topics that will aggressively compact, but so far I'm having trouble getting complete compaction at all. The topic is configured like so: Topic:beer_archive PartitionCount:20 Repl

Re: Pulling Snapshots from Kafka, Log compaction last compact offset

2015-05-13 Thread Jonathan Hodges
s HDFS (or HBase), I suspect Sqoop still has a place. > > > > > > 3) Due to its parallelism and general purpose JDBC connector, I suspect > > > that Sqoop is even a very viable way of getting data into Kafka. > > > > > > Gwen > > > > > >

Re: Pulling Snapshots from Kafka, Log compaction last compact offset

2015-05-10 Thread Hisham Mardam-Bey
> the > > > eventual goal is HDFS (or HBase), I suspect Sqoop still has a place. > > > > > > 3) Due to its parallelism and general purpose JDBC connector, I suspect > > > that Sqoop is even a very viable way of getting data into Kafka. > > > > >

Re: Pulling Snapshots from Kafka, Log compaction last compact offset

2015-05-09 Thread Gwen Shapira
allelism and general purpose JDBC connector, I suspect > > that Sqoop is even a very viable way of getting data into Kafka. > > > > Gwen > > > > > > On Thu, Apr 30, 2015 at 2:27 PM, Jan Filipiak > > wrote: > > > > > Hello Everyone, > > >

Re: Pulling Snapshots from Kafka, Log compaction last compact offset

2015-05-02 Thread Jonathan Hodges
ke people to help me with. When > > pulling a snapshot of a partition into HDFS using a camus-like > application > > I feel the need of keeping a Set of all keys read so far and stop as soon > > as I find a key beeing already in my set. I use this as an indicator of > how &

Re: Pulling Snapshots from Kafka, Log compaction last compact offset

2015-04-30 Thread Gwen Shapira
bout the recent example of replicating PostgresSQL > Changes to Kafka. My view on the log compaction feature always had been a > very sceptical one, but now with its great potential exposed to the wide > public, I think its an awesome feature. Especially when pulling this data > int

Pulling Snapshots from Kafka, Log compaction last compact offset

2015-04-30 Thread Jan Filipiak
Hello Everyone, I am quite exited about the recent example of replicating PostgresSQL Changes to Kafka. My view on the log compaction feature always had been a very sceptical one, but now with its great potential exposed to the wide public, I think its an awesome feature. Especially when

Re: Log compaction recover strategy

2015-03-10 Thread Pierre-Yves Ritschard
On 03/10/2015 05:48 PM, Mayuresh Gharat wrote: > How do you typically handle workers starting, always start at offset 0 > to make sure the view is correctly recreated ? > ---> You will have to reset the offsets to 0 and the offset reset policy to > earliest in consumer. Yup, as expected. > > Ho

Re: Log compaction recover strategy

2015-03-10 Thread Mayuresh Gharat
that was consumed in the past and the offset currently being consumed. Thanks, Mayuresh On Tue, Mar 10, 2015 at 5:51 AM, Pierre-Yves Ritschard wrote: > Hi kafka, > > I've started implementing simple materialized views with the log > compaction feature to test it out, and it w

Log compaction recover strategy

2015-03-10 Thread Pierre-Yves Ritschard
Hi kafka, I've started implementing simple materialized views with the log compaction feature to test it out, and it works great. I'll share the code and an accompanying article shortly but first wanted to discuss some of the production implications my sandbox has. I've separated

  1   2   >