Hi Liam,

thanks for the pointer. I found out, that the log cleaner on all kafka brokers 
died with the following error message:

[2018-12-04 15:33:24,886] INFO Cleaner 0: Caught segment overflow error during 
cleaning: Detected offset overflow at offset -1 in segment 
LogSegment(baseOffset=3605669, size=6970326) (kafka.log.LogCleaner)
[2018-12-04 15:33:24,958] ERROR [kafka-log-cleaner-thread-0]: Error due to 
(kafka.log.LogCleaner)
java.lang.IllegalArgumentException: requirement failed: Split operation is only 
permitted for segments with overflow
        at scala.Predef$.require(Predef.scala:277)
        at kafka.log.Log.splitOverflowedSegment(Log.scala:1873)
        at kafka.log.Cleaner.cleanSegments(LogCleaner.scala:517)
        at kafka.log.Cleaner.$anonfun$doClean$6(LogCleaner.scala:465)
        at kafka.log.Cleaner.$anonfun$doClean$6$adapted(LogCleaner.scala:464)
        at scala.collection.immutable.List.foreach(List.scala:389)
        at kafka.log.Cleaner.doClean(LogCleaner.scala:464)
        at kafka.log.Cleaner.clean(LogCleaner.scala:442)
        at kafka.log.LogCleaner$CleanerThread.cleanOrSleep(LogCleaner.scala:303)
        at kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:289)
        at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:82)
[2018-12-04 15:33:24,959] INFO [kafka-log-cleaner-thread-0]: Stopped 
(kafka.log.LogCleaner)

This brought up some new questions:
1.) How do I avoid this error? Did I misconfigure the creation of segments?
2.) Is there a way to monitor the log cleaner?
3.) Can I restart the log cleaner manually without restarting the whole broker?

Thanks for your help.

Best,
Claudia

-----Ursprüngliche Nachricht-----
Von: Liam Clarke <liam.cla...@adscale.co.nz> 
Gesendet: Montag, 17. Dezember 2018 23:06
An: users@kafka.apache.org
Betreff: 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 <c.wegm...@kasasi.de wrote:

> 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 left. After 
> recreating all indexes etc. the broker recognized the old data and 
> compacted it correctly. I had to restart all other brokers of the 
> cluster, too, for them to also recognize the old data and start compacting.
>
> So I guess, before restarting the brokers where to busy to 
> compact/delete old data? Is there a configuration to ensure compaction 
> after a certain amount of time or something?
>
> Best,
> Claudia
>
> -----Ursprüngliche Nachricht-----
> Von: Spico Florin <spicoflo...@gmail.com>
> 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/
>
>
> https://stackoverflow.com/questions/41048041/kafka-deletes-segments-ev
> en-before-segment-size-is-reached
>
> Regards,
>  Florin
>
> segment.ms This configuration controls the period of time after which 
> Kafka will force the log to roll even if the segment file isn't full 
> to ensure that retention can delete or compact old data. long 
> 604800000 [1,...] log.roll.ms medium
>
> On Mon, Dec 17, 2018 at 12:28 PM Claudia Wegmann <c.wegm...@kasasi.de>
> wrote:
>
> > Dear kafka users,
> >
> > I've got a problem on one of my kafka clusters. I use this cluster 
> > with kafka streams applications. Some of this stream apps use a 
> > kafka state store. Therefore a changelog topic is created for those 
> > stores with cleanup policy "compact". One of these topics 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 expected. What config parameters do have 
> > influence on log compaction? And how to set them, when I want data 
> > older than 4 hours to be compacted?
> >
> > Thanks in advance.
> >
> > Best,
> > Claudia
> >
>

Reply via email to