Afaik, the current (most recent) segment is not touched by the cleaner. Not sure if this might be the problem in your case.

Regards,
Harald.

On 11.08.2016 11:22, Christiane Lemke wrote:
Hello all,

@Tom - thank you for your answer :)

Here's the link to a gist with a minimal example:
https://gist.github.com/anonymous/f78184eaeec3ee82b15182aec24a432a

Here's the commands to create the compacted topic on a local kafka:
kafka-topics.sh -zookeeper 127.0.0.1:2181 --create --topic compaction-test
--partitions 1 --replication-factor 1
kafka-topics.sh -zookeeper 127.0.0.1:2181 --alter --topic compaction-test
--config min.cleanable.dirty.ratio=0.01 --config cleanup.policy=compact
--config segment.ms=100 --config delete.retention.ms=100

Here is the relevant part of the server.properties
log.retention.check.interval.ms=100
log.cleaner.delete.retention.ms=100
log.cleaner.enable=true
log.cleaner.min.cleanable.ratio=0.01

I am not sure what a tombstone would actually look like if consumed. I
still see all the messages, just with null values.
I believe the log.cleaner.delete.retention.ms should be responsible for how
log a tombstone is kept - I set it to a very small time for both topic and
server config, to be sure.
My hope was that the message would be deleted during the test.

Any ideas still appreciated :)

Best, Christiane



On 10 August 2016 at 12:05, Tom Crayford <tcrayf...@heroku.com> wrote:

I'd possibly check the log segments in question, by using the
DumpLogSegments tool.

Note that Kafka keeps a deleted tombstone for 24h by default after a
deletion. Are you checking for the key and the value being present during
this testing?

Sample code for the producer messages would be useful as well.

Thanks

Tom Crayford
Heroku Kafka

On Wednesday, 10 August 2016, Christiane Lemke <christiane.le...@gmail.com

wrote:

Hi all,

I am trying to set up a minimal example understanding log compaction
behaviour using kafka-clients-0.10.0.0.jar. I got the compaction
behaviour
working fine, however, when trying to delete a message explicitly by
writing a null value, the message seems to not be deleted.

These are my settings for my topic for compaction and deletion kicking in
as soon as possible: Configs:min.cleanable.dirty.ratio=0.01,
delete.retention.ms=100,retention.ms=100,segment.ms
=100,cleanup.policy=compact

The offending consumer record looks like this:
[ConsumerRecord(topic = compation-test, partition = 0, offset = 33,
CreateTime = 1470812816735, checksum = 3859648886, serialized key size =
16, serialized value size = -1, key = 9c9bde71-29ec-4687-ab24-
9459f5fc0d34,
value = null)]

I can see the cleaner threads running fine, producing output like this:
[2016-08-10 08:24:51,601] INFO Cleaner 0: Cleaning segment 37 in log
tns_ticket-0 (last modified Wed Aug 10 07:52:26 UTC 2016) into 0,
retaining
deletes. (kafka.log.LogCleaner)
(retaining deletes?)

I am running out of ideas of settings to try - are there any ideas about
what I might have missed or misunderstood?

Any hint greatly appreciated :)

Best, Christiane



Reply via email to