Thanks for correcting me, Tom.  I got confused with warn log message.

On Tue, Jul 19, 2016 at 5:45 PM, Tom Crayford <tcrayf...@heroku.com> wrote:

> Manikumar,
>
> How will that help? Increasing the number of log cleaner threads will lead
> to *less* memory for the buffer per thread, as it's divided up among
> available threads.
>
> Lawrence, I'm reasonably sure you're hitting KAFKA-3587 here, and should
> upgrade to 0.10 ASAP. As far as I'm aware Kafka doesn't have any
> backporting or stable versions policy, so the only ways to get that patch
> are a) upgrade b) backport the patch yourself. b) seems extremely risky to
> me
>
> Thanks
>
> Tom
>
> On Tue, Jul 19, 2016 at 5:49 AM, Manikumar Reddy <
> manikumar.re...@gmail.com>
> wrote:
>
> > Try increasing log cleaner threads.
> >
> > On Tue, Jul 19, 2016 at 1:40 AM, Lawrence Weikum <lwei...@pandora.com>
> > wrote:
> >
> > > It seems that the log-cleaner is still failing no matter what settings
> I
> > > give it.
> > >
> > > Here is the full output from one of our brokers:
> > > [2016-07-18 13:00:40,726] ERROR [kafka-log-cleaner-thread-0], Error due
> > > to  (kafka.log.LogCleaner)
> > > java.lang.IllegalArgumentException: requirement failed: 192053210
> > messages
> > > in segment __consumer_offsets-15/00000000000000000000.log but offset
> map
> > > can fit only 74999999. You can increase log.cleaner.dedupe.buffer.size
> or
> > > decrease log.cleaner.threads
> > >         at scala.Predef$.require(Predef.scala:219)
> > >         at
> > > kafka.log.Cleaner$$anonfun$buildOffsetMap$4.apply(LogCleaner.scala:584)
> > >         at
> > > kafka.log.Cleaner$$anonfun$buildOffsetMap$4.apply(LogCleaner.scala:580)
> > >         at
> > >
> >
> scala.collection.immutable.Stream$StreamWithFilter.foreach(Stream.scala:570)
> > >         at kafka.log.Cleaner.buildOffsetMap(LogCleaner.scala:580)
> > >         at kafka.log.Cleaner.clean(LogCleaner.scala:322)
> > >         at
> > > kafka.log.LogCleaner$CleanerThread.cleanOrSleep(LogCleaner.scala:230)
> > >         at
> > kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:208)
> > >         at
> > kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:63)
> > > [2016-07-18 13:00:40,732] INFO [kafka-log-cleaner-thread-0], Stopped
> > > (kafka.log.LogCleaner)
> > >
> > > Currently, I have heap allocation up to 64GB, only one log-cleaning
> > thread
> > > is set to run, and log.cleaner.dedupe.buffer.size is 2GB.  I get this
> > error
> > > if I try to increase it anymore:
> > >
> > > WARN [kafka-log-cleaner-thread-0], Cannot use more than 2G of cleaner
> > > buffer space per cleaner thread, ignoring excess buffer space...
> > > (kafka.log.LogCleaner)
> > >
> > > Is there something else I can do to help the broker compact the
> > > __consumer_offset topics?
> > >
> > > Thank you again for your help!
> > >
> > > Lawrence Weikum
> > >
> > > On 7/13/16, 1:06 PM, "Rakesh Vidyadharan" <rvidyadha...@gracenote.com>
> > > wrote:
> > >
> > > We ran into this as well, and I ended up with the following that works
> > for
> > > us.
> > >
> > > log.cleaner.dedupe.buffer.size=536870912
> > > log.cleaner.io.buffer.size=20000000
> > >
> > >
> > >
> > >
> > >
> > > On 13/07/2016 14:01, "Lawrence Weikum" <lwei...@pandora.com> wrote:
> > >
> > > >Apologies. Here is the full trace from a broker:
> > > >
> > > >[2016-06-24 09:57:39,881] ERROR [kafka-log-cleaner-thread-0], Error
> due
> > > to  (kafka.log.LogCleaner)
> > > >java.lang.IllegalArgumentException: requirement failed: 9730197928
> > > messages in segment __consumer_offsets-36/00000000000000000000.log but
> > > offset map can fit only 5033164. You can increase
> > > log.cleaner.dedupe.buffer.size or decrease log.cleaner.threads
> > > >        at scala.Predef$.require(Predef.scala:219)
> > > >        at
> > > kafka.log.Cleaner$$anonfun$buildOffsetMap$4.apply(LogCleaner.scala:584)
> > > >        at
> > > kafka.log.Cleaner$$anonfun$buildOffsetMap$4.apply(LogCleaner.scala:580)
> > > >        at
> > >
> >
> scala.collection.immutable.Stream$StreamWithFilter.foreach(Stream.scala:570)
> > > >        at kafka.log.Cleaner.buildOffsetMap(LogCleaner.scala:580)
> > > >        at kafka.log.Cleaner.clean(LogCleaner.scala:322)
> > > >        at
> > > kafka.log.LogCleaner$CleanerThread.cleanOrSleep(LogCleaner.scala:230)
> > > >        at
> > kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:208)
> > > >        at
> > kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:63)
> > > >[2016-06-24 09:57:39,881] INFO [kafka-log-cleaner-thread-0], Stopped
> > > (kafka.log.LogCleaner)
> > > >
> > > >
> > > >Is log.cleaner.dedupe.buffer.size a broker setting?  What is a good
> > > number to set it to?
> > > >
> > > >
> > > >
> > > >Lawrence Weikum
> > > >
> > > >
> > > >On 7/13/16, 11:18 AM, "Manikumar Reddy" <manikumar.re...@gmail.com>
> > > wrote:
> > > >
> > > >Can you post the complete error stack trace?   Yes, you need to
> > > >restart the affected
> > > >brokers.
> > > >You can tweak log.cleaner.dedupe.buffer.size,
> log.cleaner.io.buffer.size
> > > >configs.
> > > >
> > > >Some related JIRAs:
> > > >
> > > >https://issues.apache.org/jira/browse/KAFKA-3587
> > > >https://issues.apache.org/jira/browse/KAFKA-3894
> > > >https://issues.apache.org/jira/browse/KAFKA-3915
> > > >
> > > >On Wed, Jul 13, 2016 at 10:36 PM, Lawrence Weikum <
> lwei...@pandora.com>
> > > >wrote:
> > > >
> > > >> Oh interesting. I didn’t know about that log file until now.
> > > >>
> > > >> The only error that has been populated among all brokers showing
> this
> > > >> behavior is:
> > > >>
> > > >> ERROR [kafka-log-cleaner-thread-0], Error due to
> > (kafka.log.LogCleaner)
> > > >>
> > > >> Then we see many messages like this:
> > > >>
> > > >> INFO Compaction for partition [__consumer_offsets,30] is resumed
> > > >> (kafka.log.LogCleaner)
> > > >> INFO The cleaning for partition [__consumer_offsets,30] is aborted
> > > >> (kafka.log.LogCleaner)
> > > >>
> > > >> Using Visual VM, I do not see any log-cleaner threads in those
> > > brokers.  I
> > > >> do see it in the brokers not showing this behavior though.
> > > >>
> > > >> Any idea why the LogCleaner failed?
> > > >>
> > > >> As a temporary fix, should we restart the affected brokers?
> > > >>
> > > >> Thanks again!
> > > >>
> > > >>
> > > >> Lawrence Weikum
> > > >>
> > > >> On 7/13/16, 10:34 AM, "Manikumar Reddy" <manikumar.re...@gmail.com>
> > > wrote:
> > > >>
> > > >> Hi,
> > > >>
> > > >> Are you seeing any errors in log-cleaner.log?  The log-cleaner
> thread
> > > can
> > > >> crash on certain errors.
> > > >>
> > > >> Thanks
> > > >> Manikumar
> > > >>
> > > >> On Wed, Jul 13, 2016 at 9:54 PM, Lawrence Weikum <
> lwei...@pandora.com
> > >
> > > >> wrote:
> > > >>
> > > >> > Hello,
> > > >> >
> > > >> > We’re seeing a strange behavior in Kafka 0.9.0.1 which occurs
> about
> > > every
> > > >> > other week.  I’m curious if others have seen it and know of a
> > > solution.
> > > >> >
> > > >> > Setup and Scenario:
> > > >> >
> > > >> > -          Brokers initially setup with log compaction turned off
> > > >> >
> > > >> > -          After 30 days, log compaction was turned on
> > > >> >
> > > >> > -          At this time, the number of Open FDs was ~ 30K per
> > broker.
> > > >> >
> > > >> > -          After 2 days, the __consumer_offsets topic was
> compacted
> > > >> > fully.  Open FDs reduced to ~5K per broker.
> > > >> >
> > > >> > -          Cluster has been under normal load for roughly 7 days.
> > > >> >
> > > >> > -          At the 7 day mark, __consumer_offsets topic seems to
> have
> > > >> > stopped compacting on two of the brokers, and on those brokers,
> the
> > FD
> > > >> > count is up to ~25K.
> > > >> >
> > > >> >
> > > >> > We have tried rebalancing the partitions before.  The first time,
> > the
> > > >> > destination broker had compacted the data fine and open FDs were
> > low.
> > > The
> > > >> > second time, the destination broker kept the FDs open.
> > > >> >
> > > >> >
> > > >> > In all the broker logs, we’re seeing this messages:
> > > >> > INFO [Group Metadata Manager on Broker 8]: Removed 0 expired
> offsets
> > > in 0
> > > >> > milliseconds. (kafka.coordinator.GroupMetadataManager)
> > > >> >
> > > >> > There are only 4 consumers at the moment on the cluster; one topic
> > > with
> > > >> 92
> > > >> > partitions.
> > > >> >
> > > >> > Is there a reason why log compaction may stop working or why the
> > > >> > __consumer_offsets topic would start holding thousands of FDs?
> > > >> >
> > > >> > Thank you all for your help!
> > > >> >
> > > >> > Lawrence Weikum
> > > >> >
> > > >> >
> > > >>
> > > >>
> > > >>
> > > >
> > > >
> > >
> > >
> > >
> >
>

Reply via email to