You are viewing a plain text version of this content. The canonical link for it is here.
Posted to users@kafka.apache.org by John Holland <jo...@objectpartners.com> on 2015/09/18 14:31:24 UTC

Log Cleaner Thread Stops

I've been experiencing this issue across several of our environments ever
since we enabled the log cleaner for the __consumer_offsets topic.

We are on version 0.8.2.1 of kafka, using the new producer.  All of our
consumers are set to commit to kafka only.

Below is the stack trace in the log I've encountered across several
different clusters.  A simple restart of kafka will allow compaction to
continue on all of the other partitions but the incorrect one will always
fail.

Here are the values for it from the kafka-topics --describe command:

Topic:__consumer_offsets PartitionCount:50 ReplicationFactor:3
Configs:segment.bytes=104857600,cleanup.policy=compact

Are there any recommendations on how to prevent this and the best way to
recover from this exception?  This is causing disk space to fill up quickly
on the node.

I did see an open issue that seems very similar to this
https://issues.apache.org/jira/browse/KAFKA-1641 but this is the
__consumer_offsets topic which I have not had any part in setting up nor
producing to.

[2015-09-18 02:57:25,520] INFO Cleaner 0: Beginning cleaning of log
__consumer_offsets-17. (kafka.log.LogCleaner)
[2015-09-18 02:57:25,520] INFO Cleaner 0: Building offset map for
__consumer_offsets-17... (kafka.log.LogCleaner)
[2015-09-18 02:57:25,609] INFO Cleaner 0: Building offset map for log
__consumer_offsets-17 for 46 segments in offset range [468079184,
528707475). (kafka.log.LogCleaner)
[2015-09-18 02:57:25,645] ERROR [kafka-log-cleaner-thread-0], Error due to
 (kafka.log.LogCleaner)
java.lang.IllegalArgumentException: requirement failed: Last clean offset
is 468079184 but segment base offset is 0 for log __consumer_offsets-17.
        at scala.Predef$.require(Predef.scala:233)
        at kafka.log.Cleaner.buildOffsetMap(LogCleaner.scala:509)
        at kafka.log.Cleaner.clean(LogCleaner.scala:307)
        at
kafka.log.LogCleaner$CleanerThread.cleanOrSleep(LogCleaner.scala:221)
        at kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:199)
        at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:60)
[2015-09-18 02:57:25,654] INFO [kafka-log-cleaner-thread-0], Stopped
 (kafka.log.LogCleaner)

-John

Re: Log Cleaner Thread Stops

Posted by Todd Palino <tp...@gmail.com>.
This is correct, compression isn't used for the offsets at all. If, for
some reason, you do have either a compressed or a corrupt message somewhere
in the topic, the method I mentioned previously will flush it out. We
haven't seen that as a recurring problem, so fixing it once is sufficient.

-Todd


On Mon, Sep 28, 2015 at 9:53 AM, Jason Rosenberg <jb...@squareup.com> wrote:

> Just to clarify too, if the only use case for log-compaction we use is for
> the __consumer_offsets, we should be ok, correct?  I assume compression is
> not used by default for consumer offsets?
>
> Jason
>
> On Fri, Sep 25, 2015 at 12:15 AM, Todd Palino <tp...@gmail.com> wrote:
>
> > For now, that's the way it is. Historically, we've only monitored the lag
> > for our infrastructure applications. Other users are responsible for
> their
> > own checking, typically using the maxlag mbean or some application
> specific
> > metric. Besides the core, we've probably got a dozen or so consumers
> moved
> > over to Kafka committed offsets at this point.
> >
> > Of course, just those apps do cover well over a hundred consumer groups
> :)
> >
> > -Todd
> >
> > On Thursday, September 24, 2015, James Cheng <jc...@tivo.com> wrote:
> >
> > >
> > > > On Sep 24, 2015, at 8:11 PM, Todd Palino <tpalino@gmail.com
> > > <javascript:;>> wrote:
> > > >
> > > > Well, in general you can't currently use compressed messages in any
> > topic
> > > > that has compaction turned on regardless of whether or not you are
> > using
> > > > Kafka-committed offsets. The log compaction thread will die either
> way.
> > > > There's only one compression thread for the broker that runs on all
> > > topics
> > > > that use compaction.
> > > >
> > > > Jason, to address your question, it's probably wise to wait for now.
> > > > Zookeeper offsets work, so unless it's broke, don't fix it for now.
> > We're
> > > > using Kafka-committed offsets at LinkedIn for our mirror makers and
> our
> > > > auditor application (both of which are considered infrastructure
> > > > applications for Kafka), but we're not encouraging other internal
> users
> > > to
> > > > switch over just yet.
> > > >
> > >
> > > Burrow depends on kafka-commited offsets, doesn’t it? I guess that
> means
> > > Burrow is only being used to monitor your mirror makers and auditor
> > > application, then?
> > >
> > > -James
> > >
> > > > -Todd
> > > >
> > > >
> > > > On Wed, Sep 23, 2015 at 3:21 PM, James Cheng <jcheng@tivo.com
> > > <javascript:;>> wrote:
> > > >
> > > >>
> > > >> On Sep 18, 2015, at 10:25 AM, Todd Palino <tpalino@gmail.com
> > > <javascript:;>> wrote:
> > > >>
> > > >>> I think the last major issue with log compaction (that it couldn't
> > > handle
> > > >>> compressed messages) was committed as part of
> > > >>> https://issues.apache.org/jira/browse/KAFKA-1374 in August, but
> I'm
> > > not
> > > >>> certain what version this will end up in. It may be part of
> 0.8.2.2.
> > > >>>
> > > >>> Regardless, you'll probably be OK now. We've found that once we
> clean
> > > >> this
> > > >>> issue up once it doesn't appear to recur. As long as you're not
> > writing
> > > >> in
> > > >>> compressed messages to a log compacted topic (and that won't happen
> > > with
> > > >>> __consumer_offsets, as it's managed by the brokers themselves - it
> > > would
> > > >>> only be if you were using other log compacted topics), you're
> likely
> > in
> > > >> the
> > > >>> clear now.
> > > >>>
> > > >>
> > > >> Todd,
> > > >>
> > > >> If I understand your description of the problem, you are saying that
> > > >> enabling log compaction on a topic with compressed messages can
> > (will?)
> > > >> cause the log cleaner to crash when it encounters those compressed
> > > >> messages. And the death of the cleaner thread will prevent log
> > > compaction
> > > >> from running on other topics, even ones that don't have compressed
> > > messages.
> > > >>
> > > >> That means if we have a cluster where we want to use log compaction
> on
> > > >> *any* topic, we need to either:
> > > >> 1) apply https://issues.apache.org/jira/browse/KAFKA-1374 (or
> upgrade
> > > to
> > > >> some version it is applied)
> > > >> OR
> > > >> 2) make sure that we don't use compressed messages in *any* topic
> that
> > > has
> > > >> log compaction turned on.
> > > >>
> > > >> And, more specifically, if we want to make use of
> __consumer_offsets,
> > > then
> > > >> we cannot use compressed messages in any topic that has compaction
> > > turned
> > > >> on.
> > > >>
> > > >> Is that right?
> > > >> -James
> > > >>
> > > >>> -Todd
> > > >>>
> > > >>>
> > > >>> On Fri, Sep 18, 2015 at 9:54 AM, John Holland <
> > > >>> john.holland@objectpartners.com <javascript:;>> wrote:
> > > >>>
> > > >>>> Thanks!
> > > >>>>
> > > >>>> I did what you suggested and it worked except it was necessary for
> > me
> > > to
> > > >>>> remove the cleaner-offset-checkpoint file from the data directory
> > and
> > > >>>> restart the servers.  The log indicates all is well.
> > > >>>>
> > > >>>> Do you know what version the fix to this will be in? I'm not
> looking
> > > >>>> forward to dealing with this on a reoccurring basis.
> > > >>>>
> > > >>>> -John
> > > >>>>
> > > >>>> On Fri, Sep 18, 2015 at 8:48 AM Todd Palino <tpalino@gmail.com
> > > <javascript:;>> wrote:
> > > >>>>
> > > >>>>> Yes, this is a known concern, and it should be fixed with recent
> > > >> commits.
> > > >>>>> In the meantime, you'll have to do a little manual cleanup.
> > > >>>>>
> > > >>>>> The problem you're running into is a corrupt message in the
> offsets
> > > >>>> topic.
> > > >>>>> We've seen this a lot. What you need to do is set the topic
> > > >> configuration
> > > >>>>> to remove the cleanup.policy config, and set retention.ms and
> > > >> segment.ms
> > > >>>>> to
> > > >>>>> something reasonably low. I suggest using a value of 3 or 4 times
> > > your
> > > >>>>> commit interval for consumers. Then wait until the log segments
> are
> > > >>>> reaped
> > > >>>>> (wait twice as long as the retention.ms you chose, to be safe).
> > Once
> > > >>>> this
> > > >>>>> is done, you can set the topic configuration back the way it was
> > > >> (remove
> > > >>>>> segment.ms and retention.ms configs, and set
> > > cleanup.policy=compact).
> > > >>>>> Lastly, you'll need to do a rolling bounce of the cluster to
> > restart
> > > >> the
> > > >>>>> brokers (which restarts the log cleaner threads). Technically,
> you
> > > only
> > > >>>>> need to restart brokers where the threads have died, but it's
> > easier
> > > to
> > > >>>>> just restart all of them.
> > > >>>>>
> > > >>>>> Keep in mind that when you do this, you are deleting old offsets.
> > If
> > > >> your
> > > >>>>> consumers are all live and healthy, this shouldn't be a problem
> > > because
> > > >>>>> they will just continue to commit their offsets properly. But if
> > you
> > > >> have
> > > >>>>> an offline consumer, you'll lose the committed offsets by doing
> > this.
> > > >>>>>
> > > >>>>> -Todd
> > > >>>>>
> > > >>>>>
> > > >>>>> On Fri, Sep 18, 2015 at 5:31 AM, John Holland <
> > > >>>>> john.holland@objectpartners.com <javascript:;>> wrote:
> > > >>>>>
> > > >>>>>> I've been experiencing this issue across several of our
> > environments
> > > >>>> ever
> > > >>>>>> since we enabled the log cleaner for the __consumer_offsets
> topic.
> > > >>>>>>
> > > >>>>>> We are on version 0.8.2.1 of kafka, using the new producer.  All
> > of
> > > >> our
> > > >>>>>> consumers are set to commit to kafka only.
> > > >>>>>>
> > > >>>>>> Below is the stack trace in the log I've encountered across
> > several
> > > >>>>>> different clusters.  A simple restart of kafka will allow
> > compaction
> > > >> to
> > > >>>>>> continue on all of the other partitions but the incorrect one
> will
> > > >>>> always
> > > >>>>>> fail.
> > > >>>>>>
> > > >>>>>> Here are the values for it from the kafka-topics --describe
> > command:
> > > >>>>>>
> > > >>>>>> Topic:__consumer_offsets PartitionCount:50 ReplicationFactor:3
> > > >>>>>> Configs:segment.bytes=104857600,cleanup.policy=compact
> > > >>>>>>
> > > >>>>>> Are there any recommendations on how to prevent this and the
> best
> > > way
> > > >>>> to
> > > >>>>>> recover from this exception?  This is causing disk space to fill
> > up
> > > >>>>> quickly
> > > >>>>>> on the node.
> > > >>>>>>
> > > >>>>>> I did see an open issue that seems very similar to this
> > > >>>>>> https://issues.apache.org/jira/browse/KAFKA-1641 but this is
> the
> > > >>>>>> __consumer_offsets topic which I have not had any part in
> setting
> > up
> > > >>>> nor
> > > >>>>>> producing to.
> > > >>>>>>
> > > >>>>>> [2015-09-18 02:57:25,520] INFO Cleaner 0: Beginning cleaning of
> > log
> > > >>>>>> __consumer_offsets-17. (kafka.log.LogCleaner)
> > > >>>>>> [2015-09-18 02:57:25,520] INFO Cleaner 0: Building offset map
> for
> > > >>>>>> __consumer_offsets-17... (kafka.log.LogCleaner)
> > > >>>>>> [2015-09-18 02:57:25,609] INFO Cleaner 0: Building offset map
> for
> > > log
> > > >>>>>> __consumer_offsets-17 for 46 segments in offset range
> [468079184,
> > > >>>>>> 528707475). (kafka.log.LogCleaner)
> > > >>>>>> [2015-09-18 02:57:25,645] ERROR [kafka-log-cleaner-thread-0],
> > Error
> > > >> due
> > > >>>>> to
> > > >>>>>> (kafka.log.LogCleaner)
> > > >>>>>> java.lang.IllegalArgumentException: requirement failed: Last
> clean
> > > >>>> offset
> > > >>>>>> is 468079184 but segment base offset is 0 for log
> > > >>>> __consumer_offsets-17.
> > > >>>>>>       at scala.Predef$.require(Predef.scala:233)
> > > >>>>>>       at kafka.log.Cleaner.buildOffsetMap(LogCleaner.scala:509)
> > > >>>>>>       at kafka.log.Cleaner.clean(LogCleaner.scala:307)
> > > >>>>>>       at
> > > >>>>>>
> > > kafka.log.LogCleaner$CleanerThread.cleanOrSleep(LogCleaner.scala:221)
> > > >>>>>>       at
> > > >>>>> kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:199)
> > > >>>>>>       at
> > > >>>>> kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:60)
> > > >>>>>> [2015-09-18 02:57:25,654] INFO [kafka-log-cleaner-thread-0],
> > Stopped
> > > >>>>>> (kafka.log.LogCleaner)
> > > >>>>>>
> > > >>>>>> -John
> > > >>>>>>
> > > >>>>>
> > > >>>>
> > > >>
> > > >>
> > >
> > >
> > > ________________________________
> > >
> > > This email and any attachments may contain confidential and privileged
> > > material for the sole use of the intended recipient. Any review,
> copying,
> > > or distribution of this email (or any attachments) by others is
> > prohibited.
> > > If you are not the intended recipient, please contact the sender
> > > immediately and permanently delete this email and any attachments. No
> > > employee or agent of TiVo Inc. is authorized to conclude any binding
> > > agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo
> > > Inc. may only be made by a signed written agreement.
> > >
> >
>

Re: Log Cleaner Thread Stops

Posted by Jason Rosenberg <jb...@squareup.com>.
Just to clarify too, if the only use case for log-compaction we use is for
the __consumer_offsets, we should be ok, correct?  I assume compression is
not used by default for consumer offsets?

Jason

On Fri, Sep 25, 2015 at 12:15 AM, Todd Palino <tp...@gmail.com> wrote:

> For now, that's the way it is. Historically, we've only monitored the lag
> for our infrastructure applications. Other users are responsible for their
> own checking, typically using the maxlag mbean or some application specific
> metric. Besides the core, we've probably got a dozen or so consumers moved
> over to Kafka committed offsets at this point.
>
> Of course, just those apps do cover well over a hundred consumer groups :)
>
> -Todd
>
> On Thursday, September 24, 2015, James Cheng <jc...@tivo.com> wrote:
>
> >
> > > On Sep 24, 2015, at 8:11 PM, Todd Palino <tpalino@gmail.com
> > <javascript:;>> wrote:
> > >
> > > Well, in general you can't currently use compressed messages in any
> topic
> > > that has compaction turned on regardless of whether or not you are
> using
> > > Kafka-committed offsets. The log compaction thread will die either way.
> > > There's only one compression thread for the broker that runs on all
> > topics
> > > that use compaction.
> > >
> > > Jason, to address your question, it's probably wise to wait for now.
> > > Zookeeper offsets work, so unless it's broke, don't fix it for now.
> We're
> > > using Kafka-committed offsets at LinkedIn for our mirror makers and our
> > > auditor application (both of which are considered infrastructure
> > > applications for Kafka), but we're not encouraging other internal users
> > to
> > > switch over just yet.
> > >
> >
> > Burrow depends on kafka-commited offsets, doesn’t it? I guess that means
> > Burrow is only being used to monitor your mirror makers and auditor
> > application, then?
> >
> > -James
> >
> > > -Todd
> > >
> > >
> > > On Wed, Sep 23, 2015 at 3:21 PM, James Cheng <jcheng@tivo.com
> > <javascript:;>> wrote:
> > >
> > >>
> > >> On Sep 18, 2015, at 10:25 AM, Todd Palino <tpalino@gmail.com
> > <javascript:;>> wrote:
> > >>
> > >>> I think the last major issue with log compaction (that it couldn't
> > handle
> > >>> compressed messages) was committed as part of
> > >>> https://issues.apache.org/jira/browse/KAFKA-1374 in August, but I'm
> > not
> > >>> certain what version this will end up in. It may be part of 0.8.2.2.
> > >>>
> > >>> Regardless, you'll probably be OK now. We've found that once we clean
> > >> this
> > >>> issue up once it doesn't appear to recur. As long as you're not
> writing
> > >> in
> > >>> compressed messages to a log compacted topic (and that won't happen
> > with
> > >>> __consumer_offsets, as it's managed by the brokers themselves - it
> > would
> > >>> only be if you were using other log compacted topics), you're likely
> in
> > >> the
> > >>> clear now.
> > >>>
> > >>
> > >> Todd,
> > >>
> > >> If I understand your description of the problem, you are saying that
> > >> enabling log compaction on a topic with compressed messages can
> (will?)
> > >> cause the log cleaner to crash when it encounters those compressed
> > >> messages. And the death of the cleaner thread will prevent log
> > compaction
> > >> from running on other topics, even ones that don't have compressed
> > messages.
> > >>
> > >> That means if we have a cluster where we want to use log compaction on
> > >> *any* topic, we need to either:
> > >> 1) apply https://issues.apache.org/jira/browse/KAFKA-1374 (or upgrade
> > to
> > >> some version it is applied)
> > >> OR
> > >> 2) make sure that we don't use compressed messages in *any* topic that
> > has
> > >> log compaction turned on.
> > >>
> > >> And, more specifically, if we want to make use of __consumer_offsets,
> > then
> > >> we cannot use compressed messages in any topic that has compaction
> > turned
> > >> on.
> > >>
> > >> Is that right?
> > >> -James
> > >>
> > >>> -Todd
> > >>>
> > >>>
> > >>> On Fri, Sep 18, 2015 at 9:54 AM, John Holland <
> > >>> john.holland@objectpartners.com <javascript:;>> wrote:
> > >>>
> > >>>> Thanks!
> > >>>>
> > >>>> I did what you suggested and it worked except it was necessary for
> me
> > to
> > >>>> remove the cleaner-offset-checkpoint file from the data directory
> and
> > >>>> restart the servers.  The log indicates all is well.
> > >>>>
> > >>>> Do you know what version the fix to this will be in? I'm not looking
> > >>>> forward to dealing with this on a reoccurring basis.
> > >>>>
> > >>>> -John
> > >>>>
> > >>>> On Fri, Sep 18, 2015 at 8:48 AM Todd Palino <tpalino@gmail.com
> > <javascript:;>> wrote:
> > >>>>
> > >>>>> Yes, this is a known concern, and it should be fixed with recent
> > >> commits.
> > >>>>> In the meantime, you'll have to do a little manual cleanup.
> > >>>>>
> > >>>>> The problem you're running into is a corrupt message in the offsets
> > >>>> topic.
> > >>>>> We've seen this a lot. What you need to do is set the topic
> > >> configuration
> > >>>>> to remove the cleanup.policy config, and set retention.ms and
> > >> segment.ms
> > >>>>> to
> > >>>>> something reasonably low. I suggest using a value of 3 or 4 times
> > your
> > >>>>> commit interval for consumers. Then wait until the log segments are
> > >>>> reaped
> > >>>>> (wait twice as long as the retention.ms you chose, to be safe).
> Once
> > >>>> this
> > >>>>> is done, you can set the topic configuration back the way it was
> > >> (remove
> > >>>>> segment.ms and retention.ms configs, and set
> > cleanup.policy=compact).
> > >>>>> Lastly, you'll need to do a rolling bounce of the cluster to
> restart
> > >> the
> > >>>>> brokers (which restarts the log cleaner threads). Technically, you
> > only
> > >>>>> need to restart brokers where the threads have died, but it's
> easier
> > to
> > >>>>> just restart all of them.
> > >>>>>
> > >>>>> Keep in mind that when you do this, you are deleting old offsets.
> If
> > >> your
> > >>>>> consumers are all live and healthy, this shouldn't be a problem
> > because
> > >>>>> they will just continue to commit their offsets properly. But if
> you
> > >> have
> > >>>>> an offline consumer, you'll lose the committed offsets by doing
> this.
> > >>>>>
> > >>>>> -Todd
> > >>>>>
> > >>>>>
> > >>>>> On Fri, Sep 18, 2015 at 5:31 AM, John Holland <
> > >>>>> john.holland@objectpartners.com <javascript:;>> wrote:
> > >>>>>
> > >>>>>> I've been experiencing this issue across several of our
> environments
> > >>>> ever
> > >>>>>> since we enabled the log cleaner for the __consumer_offsets topic.
> > >>>>>>
> > >>>>>> We are on version 0.8.2.1 of kafka, using the new producer.  All
> of
> > >> our
> > >>>>>> consumers are set to commit to kafka only.
> > >>>>>>
> > >>>>>> Below is the stack trace in the log I've encountered across
> several
> > >>>>>> different clusters.  A simple restart of kafka will allow
> compaction
> > >> to
> > >>>>>> continue on all of the other partitions but the incorrect one will
> > >>>> always
> > >>>>>> fail.
> > >>>>>>
> > >>>>>> Here are the values for it from the kafka-topics --describe
> command:
> > >>>>>>
> > >>>>>> Topic:__consumer_offsets PartitionCount:50 ReplicationFactor:3
> > >>>>>> Configs:segment.bytes=104857600,cleanup.policy=compact
> > >>>>>>
> > >>>>>> Are there any recommendations on how to prevent this and the best
> > way
> > >>>> to
> > >>>>>> recover from this exception?  This is causing disk space to fill
> up
> > >>>>> quickly
> > >>>>>> on the node.
> > >>>>>>
> > >>>>>> I did see an open issue that seems very similar to this
> > >>>>>> https://issues.apache.org/jira/browse/KAFKA-1641 but this is the
> > >>>>>> __consumer_offsets topic which I have not had any part in setting
> up
> > >>>> nor
> > >>>>>> producing to.
> > >>>>>>
> > >>>>>> [2015-09-18 02:57:25,520] INFO Cleaner 0: Beginning cleaning of
> log
> > >>>>>> __consumer_offsets-17. (kafka.log.LogCleaner)
> > >>>>>> [2015-09-18 02:57:25,520] INFO Cleaner 0: Building offset map for
> > >>>>>> __consumer_offsets-17... (kafka.log.LogCleaner)
> > >>>>>> [2015-09-18 02:57:25,609] INFO Cleaner 0: Building offset map for
> > log
> > >>>>>> __consumer_offsets-17 for 46 segments in offset range [468079184,
> > >>>>>> 528707475). (kafka.log.LogCleaner)
> > >>>>>> [2015-09-18 02:57:25,645] ERROR [kafka-log-cleaner-thread-0],
> Error
> > >> due
> > >>>>> to
> > >>>>>> (kafka.log.LogCleaner)
> > >>>>>> java.lang.IllegalArgumentException: requirement failed: Last clean
> > >>>> offset
> > >>>>>> is 468079184 but segment base offset is 0 for log
> > >>>> __consumer_offsets-17.
> > >>>>>>       at scala.Predef$.require(Predef.scala:233)
> > >>>>>>       at kafka.log.Cleaner.buildOffsetMap(LogCleaner.scala:509)
> > >>>>>>       at kafka.log.Cleaner.clean(LogCleaner.scala:307)
> > >>>>>>       at
> > >>>>>>
> > kafka.log.LogCleaner$CleanerThread.cleanOrSleep(LogCleaner.scala:221)
> > >>>>>>       at
> > >>>>> kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:199)
> > >>>>>>       at
> > >>>>> kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:60)
> > >>>>>> [2015-09-18 02:57:25,654] INFO [kafka-log-cleaner-thread-0],
> Stopped
> > >>>>>> (kafka.log.LogCleaner)
> > >>>>>>
> > >>>>>> -John
> > >>>>>>
> > >>>>>
> > >>>>
> > >>
> > >>
> >
> >
> > ________________________________
> >
> > This email and any attachments may contain confidential and privileged
> > material for the sole use of the intended recipient. Any review, copying,
> > or distribution of this email (or any attachments) by others is
> prohibited.
> > If you are not the intended recipient, please contact the sender
> > immediately and permanently delete this email and any attachments. No
> > employee or agent of TiVo Inc. is authorized to conclude any binding
> > agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo
> > Inc. may only be made by a signed written agreement.
> >
>

Re: Log Cleaner Thread Stops

Posted by Todd Palino <tp...@gmail.com>.
For now, that's the way it is. Historically, we've only monitored the lag
for our infrastructure applications. Other users are responsible for their
own checking, typically using the maxlag mbean or some application specific
metric. Besides the core, we've probably got a dozen or so consumers moved
over to Kafka committed offsets at this point.

Of course, just those apps do cover well over a hundred consumer groups :)

-Todd

On Thursday, September 24, 2015, James Cheng <jc...@tivo.com> wrote:

>
> > On Sep 24, 2015, at 8:11 PM, Todd Palino <tpalino@gmail.com
> <javascript:;>> wrote:
> >
> > Well, in general you can't currently use compressed messages in any topic
> > that has compaction turned on regardless of whether or not you are using
> > Kafka-committed offsets. The log compaction thread will die either way.
> > There's only one compression thread for the broker that runs on all
> topics
> > that use compaction.
> >
> > Jason, to address your question, it's probably wise to wait for now.
> > Zookeeper offsets work, so unless it's broke, don't fix it for now. We're
> > using Kafka-committed offsets at LinkedIn for our mirror makers and our
> > auditor application (both of which are considered infrastructure
> > applications for Kafka), but we're not encouraging other internal users
> to
> > switch over just yet.
> >
>
> Burrow depends on kafka-commited offsets, doesn’t it? I guess that means
> Burrow is only being used to monitor your mirror makers and auditor
> application, then?
>
> -James
>
> > -Todd
> >
> >
> > On Wed, Sep 23, 2015 at 3:21 PM, James Cheng <jcheng@tivo.com
> <javascript:;>> wrote:
> >
> >>
> >> On Sep 18, 2015, at 10:25 AM, Todd Palino <tpalino@gmail.com
> <javascript:;>> wrote:
> >>
> >>> I think the last major issue with log compaction (that it couldn't
> handle
> >>> compressed messages) was committed as part of
> >>> https://issues.apache.org/jira/browse/KAFKA-1374 in August, but I'm
> not
> >>> certain what version this will end up in. It may be part of 0.8.2.2.
> >>>
> >>> Regardless, you'll probably be OK now. We've found that once we clean
> >> this
> >>> issue up once it doesn't appear to recur. As long as you're not writing
> >> in
> >>> compressed messages to a log compacted topic (and that won't happen
> with
> >>> __consumer_offsets, as it's managed by the brokers themselves - it
> would
> >>> only be if you were using other log compacted topics), you're likely in
> >> the
> >>> clear now.
> >>>
> >>
> >> Todd,
> >>
> >> If I understand your description of the problem, you are saying that
> >> enabling log compaction on a topic with compressed messages can (will?)
> >> cause the log cleaner to crash when it encounters those compressed
> >> messages. And the death of the cleaner thread will prevent log
> compaction
> >> from running on other topics, even ones that don't have compressed
> messages.
> >>
> >> That means if we have a cluster where we want to use log compaction on
> >> *any* topic, we need to either:
> >> 1) apply https://issues.apache.org/jira/browse/KAFKA-1374 (or upgrade
> to
> >> some version it is applied)
> >> OR
> >> 2) make sure that we don't use compressed messages in *any* topic that
> has
> >> log compaction turned on.
> >>
> >> And, more specifically, if we want to make use of __consumer_offsets,
> then
> >> we cannot use compressed messages in any topic that has compaction
> turned
> >> on.
> >>
> >> Is that right?
> >> -James
> >>
> >>> -Todd
> >>>
> >>>
> >>> On Fri, Sep 18, 2015 at 9:54 AM, John Holland <
> >>> john.holland@objectpartners.com <javascript:;>> wrote:
> >>>
> >>>> Thanks!
> >>>>
> >>>> I did what you suggested and it worked except it was necessary for me
> to
> >>>> remove the cleaner-offset-checkpoint file from the data directory and
> >>>> restart the servers.  The log indicates all is well.
> >>>>
> >>>> Do you know what version the fix to this will be in? I'm not looking
> >>>> forward to dealing with this on a reoccurring basis.
> >>>>
> >>>> -John
> >>>>
> >>>> On Fri, Sep 18, 2015 at 8:48 AM Todd Palino <tpalino@gmail.com
> <javascript:;>> wrote:
> >>>>
> >>>>> Yes, this is a known concern, and it should be fixed with recent
> >> commits.
> >>>>> In the meantime, you'll have to do a little manual cleanup.
> >>>>>
> >>>>> The problem you're running into is a corrupt message in the offsets
> >>>> topic.
> >>>>> We've seen this a lot. What you need to do is set the topic
> >> configuration
> >>>>> to remove the cleanup.policy config, and set retention.ms and
> >> segment.ms
> >>>>> to
> >>>>> something reasonably low. I suggest using a value of 3 or 4 times
> your
> >>>>> commit interval for consumers. Then wait until the log segments are
> >>>> reaped
> >>>>> (wait twice as long as the retention.ms you chose, to be safe). Once
> >>>> this
> >>>>> is done, you can set the topic configuration back the way it was
> >> (remove
> >>>>> segment.ms and retention.ms configs, and set
> cleanup.policy=compact).
> >>>>> Lastly, you'll need to do a rolling bounce of the cluster to restart
> >> the
> >>>>> brokers (which restarts the log cleaner threads). Technically, you
> only
> >>>>> need to restart brokers where the threads have died, but it's easier
> to
> >>>>> just restart all of them.
> >>>>>
> >>>>> Keep in mind that when you do this, you are deleting old offsets. If
> >> your
> >>>>> consumers are all live and healthy, this shouldn't be a problem
> because
> >>>>> they will just continue to commit their offsets properly. But if you
> >> have
> >>>>> an offline consumer, you'll lose the committed offsets by doing this.
> >>>>>
> >>>>> -Todd
> >>>>>
> >>>>>
> >>>>> On Fri, Sep 18, 2015 at 5:31 AM, John Holland <
> >>>>> john.holland@objectpartners.com <javascript:;>> wrote:
> >>>>>
> >>>>>> I've been experiencing this issue across several of our environments
> >>>> ever
> >>>>>> since we enabled the log cleaner for the __consumer_offsets topic.
> >>>>>>
> >>>>>> We are on version 0.8.2.1 of kafka, using the new producer.  All of
> >> our
> >>>>>> consumers are set to commit to kafka only.
> >>>>>>
> >>>>>> Below is the stack trace in the log I've encountered across several
> >>>>>> different clusters.  A simple restart of kafka will allow compaction
> >> to
> >>>>>> continue on all of the other partitions but the incorrect one will
> >>>> always
> >>>>>> fail.
> >>>>>>
> >>>>>> Here are the values for it from the kafka-topics --describe command:
> >>>>>>
> >>>>>> Topic:__consumer_offsets PartitionCount:50 ReplicationFactor:3
> >>>>>> Configs:segment.bytes=104857600,cleanup.policy=compact
> >>>>>>
> >>>>>> Are there any recommendations on how to prevent this and the best
> way
> >>>> to
> >>>>>> recover from this exception?  This is causing disk space to fill up
> >>>>> quickly
> >>>>>> on the node.
> >>>>>>
> >>>>>> I did see an open issue that seems very similar to this
> >>>>>> https://issues.apache.org/jira/browse/KAFKA-1641 but this is the
> >>>>>> __consumer_offsets topic which I have not had any part in setting up
> >>>> nor
> >>>>>> producing to.
> >>>>>>
> >>>>>> [2015-09-18 02:57:25,520] INFO Cleaner 0: Beginning cleaning of log
> >>>>>> __consumer_offsets-17. (kafka.log.LogCleaner)
> >>>>>> [2015-09-18 02:57:25,520] INFO Cleaner 0: Building offset map for
> >>>>>> __consumer_offsets-17... (kafka.log.LogCleaner)
> >>>>>> [2015-09-18 02:57:25,609] INFO Cleaner 0: Building offset map for
> log
> >>>>>> __consumer_offsets-17 for 46 segments in offset range [468079184,
> >>>>>> 528707475). (kafka.log.LogCleaner)
> >>>>>> [2015-09-18 02:57:25,645] ERROR [kafka-log-cleaner-thread-0], Error
> >> due
> >>>>> to
> >>>>>> (kafka.log.LogCleaner)
> >>>>>> java.lang.IllegalArgumentException: requirement failed: Last clean
> >>>> offset
> >>>>>> is 468079184 but segment base offset is 0 for log
> >>>> __consumer_offsets-17.
> >>>>>>       at scala.Predef$.require(Predef.scala:233)
> >>>>>>       at kafka.log.Cleaner.buildOffsetMap(LogCleaner.scala:509)
> >>>>>>       at kafka.log.Cleaner.clean(LogCleaner.scala:307)
> >>>>>>       at
> >>>>>>
> kafka.log.LogCleaner$CleanerThread.cleanOrSleep(LogCleaner.scala:221)
> >>>>>>       at
> >>>>> kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:199)
> >>>>>>       at
> >>>>> kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:60)
> >>>>>> [2015-09-18 02:57:25,654] INFO [kafka-log-cleaner-thread-0], Stopped
> >>>>>> (kafka.log.LogCleaner)
> >>>>>>
> >>>>>> -John
> >>>>>>
> >>>>>
> >>>>
> >>
> >>
>
>
> ________________________________
>
> This email and any attachments may contain confidential and privileged
> material for the sole use of the intended recipient. Any review, copying,
> or distribution of this email (or any attachments) by others is prohibited.
> If you are not the intended recipient, please contact the sender
> immediately and permanently delete this email and any attachments. No
> employee or agent of TiVo Inc. is authorized to conclude any binding
> agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo
> Inc. may only be made by a signed written agreement.
>

Re: Log Cleaner Thread Stops

Posted by James Cheng <jc...@tivo.com>.
> On Sep 24, 2015, at 8:11 PM, Todd Palino <tp...@gmail.com> wrote:
>
> Well, in general you can't currently use compressed messages in any topic
> that has compaction turned on regardless of whether or not you are using
> Kafka-committed offsets. The log compaction thread will die either way.
> There's only one compression thread for the broker that runs on all topics
> that use compaction.
>
> Jason, to address your question, it's probably wise to wait for now.
> Zookeeper offsets work, so unless it's broke, don't fix it for now. We're
> using Kafka-committed offsets at LinkedIn for our mirror makers and our
> auditor application (both of which are considered infrastructure
> applications for Kafka), but we're not encouraging other internal users to
> switch over just yet.
>

Burrow depends on kafka-commited offsets, doesn’t it? I guess that means Burrow is only being used to monitor your mirror makers and auditor application, then?

-James

> -Todd
>
>
> On Wed, Sep 23, 2015 at 3:21 PM, James Cheng <jc...@tivo.com> wrote:
>
>>
>> On Sep 18, 2015, at 10:25 AM, Todd Palino <tp...@gmail.com> wrote:
>>
>>> I think the last major issue with log compaction (that it couldn't handle
>>> compressed messages) was committed as part of
>>> https://issues.apache.org/jira/browse/KAFKA-1374 in August, but I'm not
>>> certain what version this will end up in. It may be part of 0.8.2.2.
>>>
>>> Regardless, you'll probably be OK now. We've found that once we clean
>> this
>>> issue up once it doesn't appear to recur. As long as you're not writing
>> in
>>> compressed messages to a log compacted topic (and that won't happen with
>>> __consumer_offsets, as it's managed by the brokers themselves - it would
>>> only be if you were using other log compacted topics), you're likely in
>> the
>>> clear now.
>>>
>>
>> Todd,
>>
>> If I understand your description of the problem, you are saying that
>> enabling log compaction on a topic with compressed messages can (will?)
>> cause the log cleaner to crash when it encounters those compressed
>> messages. And the death of the cleaner thread will prevent log compaction
>> from running on other topics, even ones that don't have compressed messages.
>>
>> That means if we have a cluster where we want to use log compaction on
>> *any* topic, we need to either:
>> 1) apply https://issues.apache.org/jira/browse/KAFKA-1374 (or upgrade to
>> some version it is applied)
>> OR
>> 2) make sure that we don't use compressed messages in *any* topic that has
>> log compaction turned on.
>>
>> And, more specifically, if we want to make use of __consumer_offsets, then
>> we cannot use compressed messages in any topic that has compaction turned
>> on.
>>
>> Is that right?
>> -James
>>
>>> -Todd
>>>
>>>
>>> On Fri, Sep 18, 2015 at 9:54 AM, John Holland <
>>> john.holland@objectpartners.com> wrote:
>>>
>>>> Thanks!
>>>>
>>>> I did what you suggested and it worked except it was necessary for me to
>>>> remove the cleaner-offset-checkpoint file from the data directory and
>>>> restart the servers.  The log indicates all is well.
>>>>
>>>> Do you know what version the fix to this will be in? I'm not looking
>>>> forward to dealing with this on a reoccurring basis.
>>>>
>>>> -John
>>>>
>>>> On Fri, Sep 18, 2015 at 8:48 AM Todd Palino <tp...@gmail.com> wrote:
>>>>
>>>>> Yes, this is a known concern, and it should be fixed with recent
>> commits.
>>>>> In the meantime, you'll have to do a little manual cleanup.
>>>>>
>>>>> The problem you're running into is a corrupt message in the offsets
>>>> topic.
>>>>> We've seen this a lot. What you need to do is set the topic
>> configuration
>>>>> to remove the cleanup.policy config, and set retention.ms and
>> segment.ms
>>>>> to
>>>>> something reasonably low. I suggest using a value of 3 or 4 times your
>>>>> commit interval for consumers. Then wait until the log segments are
>>>> reaped
>>>>> (wait twice as long as the retention.ms you chose, to be safe). Once
>>>> this
>>>>> is done, you can set the topic configuration back the way it was
>> (remove
>>>>> segment.ms and retention.ms configs, and set cleanup.policy=compact).
>>>>> Lastly, you'll need to do a rolling bounce of the cluster to restart
>> the
>>>>> brokers (which restarts the log cleaner threads). Technically, you only
>>>>> need to restart brokers where the threads have died, but it's easier to
>>>>> just restart all of them.
>>>>>
>>>>> Keep in mind that when you do this, you are deleting old offsets. If
>> your
>>>>> consumers are all live and healthy, this shouldn't be a problem because
>>>>> they will just continue to commit their offsets properly. But if you
>> have
>>>>> an offline consumer, you'll lose the committed offsets by doing this.
>>>>>
>>>>> -Todd
>>>>>
>>>>>
>>>>> On Fri, Sep 18, 2015 at 5:31 AM, John Holland <
>>>>> john.holland@objectpartners.com> wrote:
>>>>>
>>>>>> I've been experiencing this issue across several of our environments
>>>> ever
>>>>>> since we enabled the log cleaner for the __consumer_offsets topic.
>>>>>>
>>>>>> We are on version 0.8.2.1 of kafka, using the new producer.  All of
>> our
>>>>>> consumers are set to commit to kafka only.
>>>>>>
>>>>>> Below is the stack trace in the log I've encountered across several
>>>>>> different clusters.  A simple restart of kafka will allow compaction
>> to
>>>>>> continue on all of the other partitions but the incorrect one will
>>>> always
>>>>>> fail.
>>>>>>
>>>>>> Here are the values for it from the kafka-topics --describe command:
>>>>>>
>>>>>> Topic:__consumer_offsets PartitionCount:50 ReplicationFactor:3
>>>>>> Configs:segment.bytes=104857600,cleanup.policy=compact
>>>>>>
>>>>>> Are there any recommendations on how to prevent this and the best way
>>>> to
>>>>>> recover from this exception?  This is causing disk space to fill up
>>>>> quickly
>>>>>> on the node.
>>>>>>
>>>>>> I did see an open issue that seems very similar to this
>>>>>> https://issues.apache.org/jira/browse/KAFKA-1641 but this is the
>>>>>> __consumer_offsets topic which I have not had any part in setting up
>>>> nor
>>>>>> producing to.
>>>>>>
>>>>>> [2015-09-18 02:57:25,520] INFO Cleaner 0: Beginning cleaning of log
>>>>>> __consumer_offsets-17. (kafka.log.LogCleaner)
>>>>>> [2015-09-18 02:57:25,520] INFO Cleaner 0: Building offset map for
>>>>>> __consumer_offsets-17... (kafka.log.LogCleaner)
>>>>>> [2015-09-18 02:57:25,609] INFO Cleaner 0: Building offset map for log
>>>>>> __consumer_offsets-17 for 46 segments in offset range [468079184,
>>>>>> 528707475). (kafka.log.LogCleaner)
>>>>>> [2015-09-18 02:57:25,645] ERROR [kafka-log-cleaner-thread-0], Error
>> due
>>>>> to
>>>>>> (kafka.log.LogCleaner)
>>>>>> java.lang.IllegalArgumentException: requirement failed: Last clean
>>>> offset
>>>>>> is 468079184 but segment base offset is 0 for log
>>>> __consumer_offsets-17.
>>>>>>       at scala.Predef$.require(Predef.scala:233)
>>>>>>       at kafka.log.Cleaner.buildOffsetMap(LogCleaner.scala:509)
>>>>>>       at kafka.log.Cleaner.clean(LogCleaner.scala:307)
>>>>>>       at
>>>>>> kafka.log.LogCleaner$CleanerThread.cleanOrSleep(LogCleaner.scala:221)
>>>>>>       at
>>>>> kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:199)
>>>>>>       at
>>>>> kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:60)
>>>>>> [2015-09-18 02:57:25,654] INFO [kafka-log-cleaner-thread-0], Stopped
>>>>>> (kafka.log.LogCleaner)
>>>>>>
>>>>>> -John
>>>>>>
>>>>>
>>>>
>>
>>


________________________________

This email and any attachments may contain confidential and privileged material for the sole use of the intended recipient. Any review, copying, or distribution of this email (or any attachments) by others is prohibited. If you are not the intended recipient, please contact the sender immediately and permanently delete this email and any attachments. No employee or agent of TiVo Inc. is authorized to conclude any binding agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo Inc. may only be made by a signed written agreement.

Re: Log Cleaner Thread Stops

Posted by Todd Palino <tp...@gmail.com>.
Well, in general you can't currently use compressed messages in any topic
that has compaction turned on regardless of whether or not you are using
Kafka-committed offsets. The log compaction thread will die either way.
There's only one compression thread for the broker that runs on all topics
that use compaction.

Jason, to address your question, it's probably wise to wait for now.
Zookeeper offsets work, so unless it's broke, don't fix it for now. We're
using Kafka-committed offsets at LinkedIn for our mirror makers and our
auditor application (both of which are considered infrastructure
applications for Kafka), but we're not encouraging other internal users to
switch over just yet.

-Todd


On Wed, Sep 23, 2015 at 3:21 PM, James Cheng <jc...@tivo.com> wrote:

>
> On Sep 18, 2015, at 10:25 AM, Todd Palino <tp...@gmail.com> wrote:
>
> > I think the last major issue with log compaction (that it couldn't handle
> > compressed messages) was committed as part of
> > https://issues.apache.org/jira/browse/KAFKA-1374 in August, but I'm not
> > certain what version this will end up in. It may be part of 0.8.2.2.
> >
> > Regardless, you'll probably be OK now. We've found that once we clean
> this
> > issue up once it doesn't appear to recur. As long as you're not writing
> in
> > compressed messages to a log compacted topic (and that won't happen with
> > __consumer_offsets, as it's managed by the brokers themselves - it would
> > only be if you were using other log compacted topics), you're likely in
> the
> > clear now.
> >
>
> Todd,
>
> If I understand your description of the problem, you are saying that
> enabling log compaction on a topic with compressed messages can (will?)
> cause the log cleaner to crash when it encounters those compressed
> messages. And the death of the cleaner thread will prevent log compaction
> from running on other topics, even ones that don't have compressed messages.
>
> That means if we have a cluster where we want to use log compaction on
> *any* topic, we need to either:
> 1) apply https://issues.apache.org/jira/browse/KAFKA-1374 (or upgrade to
> some version it is applied)
> OR
> 2) make sure that we don't use compressed messages in *any* topic that has
> log compaction turned on.
>
> And, more specifically, if we want to make use of __consumer_offsets, then
> we cannot use compressed messages in any topic that has compaction turned
> on.
>
> Is that right?
> -James
>
> > -Todd
> >
> >
> > On Fri, Sep 18, 2015 at 9:54 AM, John Holland <
> > john.holland@objectpartners.com> wrote:
> >
> >> Thanks!
> >>
> >> I did what you suggested and it worked except it was necessary for me to
> >> remove the cleaner-offset-checkpoint file from the data directory and
> >> restart the servers.  The log indicates all is well.
> >>
> >> Do you know what version the fix to this will be in? I'm not looking
> >> forward to dealing with this on a reoccurring basis.
> >>
> >> -John
> >>
> >> On Fri, Sep 18, 2015 at 8:48 AM Todd Palino <tp...@gmail.com> wrote:
> >>
> >>> Yes, this is a known concern, and it should be fixed with recent
> commits.
> >>> In the meantime, you'll have to do a little manual cleanup.
> >>>
> >>> The problem you're running into is a corrupt message in the offsets
> >> topic.
> >>> We've seen this a lot. What you need to do is set the topic
> configuration
> >>> to remove the cleanup.policy config, and set retention.ms and
> segment.ms
> >>> to
> >>> something reasonably low. I suggest using a value of 3 or 4 times your
> >>> commit interval for consumers. Then wait until the log segments are
> >> reaped
> >>> (wait twice as long as the retention.ms you chose, to be safe). Once
> >> this
> >>> is done, you can set the topic configuration back the way it was
> (remove
> >>> segment.ms and retention.ms configs, and set cleanup.policy=compact).
> >>> Lastly, you'll need to do a rolling bounce of the cluster to restart
> the
> >>> brokers (which restarts the log cleaner threads). Technically, you only
> >>> need to restart brokers where the threads have died, but it's easier to
> >>> just restart all of them.
> >>>
> >>> Keep in mind that when you do this, you are deleting old offsets. If
> your
> >>> consumers are all live and healthy, this shouldn't be a problem because
> >>> they will just continue to commit their offsets properly. But if you
> have
> >>> an offline consumer, you'll lose the committed offsets by doing this.
> >>>
> >>> -Todd
> >>>
> >>>
> >>> On Fri, Sep 18, 2015 at 5:31 AM, John Holland <
> >>> john.holland@objectpartners.com> wrote:
> >>>
> >>>> I've been experiencing this issue across several of our environments
> >> ever
> >>>> since we enabled the log cleaner for the __consumer_offsets topic.
> >>>>
> >>>> We are on version 0.8.2.1 of kafka, using the new producer.  All of
> our
> >>>> consumers are set to commit to kafka only.
> >>>>
> >>>> Below is the stack trace in the log I've encountered across several
> >>>> different clusters.  A simple restart of kafka will allow compaction
> to
> >>>> continue on all of the other partitions but the incorrect one will
> >> always
> >>>> fail.
> >>>>
> >>>> Here are the values for it from the kafka-topics --describe command:
> >>>>
> >>>> Topic:__consumer_offsets PartitionCount:50 ReplicationFactor:3
> >>>> Configs:segment.bytes=104857600,cleanup.policy=compact
> >>>>
> >>>> Are there any recommendations on how to prevent this and the best way
> >> to
> >>>> recover from this exception?  This is causing disk space to fill up
> >>> quickly
> >>>> on the node.
> >>>>
> >>>> I did see an open issue that seems very similar to this
> >>>> https://issues.apache.org/jira/browse/KAFKA-1641 but this is the
> >>>> __consumer_offsets topic which I have not had any part in setting up
> >> nor
> >>>> producing to.
> >>>>
> >>>> [2015-09-18 02:57:25,520] INFO Cleaner 0: Beginning cleaning of log
> >>>> __consumer_offsets-17. (kafka.log.LogCleaner)
> >>>> [2015-09-18 02:57:25,520] INFO Cleaner 0: Building offset map for
> >>>> __consumer_offsets-17... (kafka.log.LogCleaner)
> >>>> [2015-09-18 02:57:25,609] INFO Cleaner 0: Building offset map for log
> >>>> __consumer_offsets-17 for 46 segments in offset range [468079184,
> >>>> 528707475). (kafka.log.LogCleaner)
> >>>> [2015-09-18 02:57:25,645] ERROR [kafka-log-cleaner-thread-0], Error
> due
> >>> to
> >>>> (kafka.log.LogCleaner)
> >>>> java.lang.IllegalArgumentException: requirement failed: Last clean
> >> offset
> >>>> is 468079184 but segment base offset is 0 for log
> >> __consumer_offsets-17.
> >>>>        at scala.Predef$.require(Predef.scala:233)
> >>>>        at kafka.log.Cleaner.buildOffsetMap(LogCleaner.scala:509)
> >>>>        at kafka.log.Cleaner.clean(LogCleaner.scala:307)
> >>>>        at
> >>>> kafka.log.LogCleaner$CleanerThread.cleanOrSleep(LogCleaner.scala:221)
> >>>>        at
> >>> kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:199)
> >>>>        at
> >>> kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:60)
> >>>> [2015-09-18 02:57:25,654] INFO [kafka-log-cleaner-thread-0], Stopped
> >>>> (kafka.log.LogCleaner)
> >>>>
> >>>> -John
> >>>>
> >>>
> >>
>
>

Re: Log Cleaner Thread Stops

Posted by James Cheng <jc...@tivo.com>.
On Sep 18, 2015, at 10:25 AM, Todd Palino <tp...@gmail.com> wrote:

> I think the last major issue with log compaction (that it couldn't handle
> compressed messages) was committed as part of
> https://issues.apache.org/jira/browse/KAFKA-1374 in August, but I'm not
> certain what version this will end up in. It may be part of 0.8.2.2.
> 
> Regardless, you'll probably be OK now. We've found that once we clean this
> issue up once it doesn't appear to recur. As long as you're not writing in
> compressed messages to a log compacted topic (and that won't happen with
> __consumer_offsets, as it's managed by the brokers themselves - it would
> only be if you were using other log compacted topics), you're likely in the
> clear now.
> 

Todd,

If I understand your description of the problem, you are saying that enabling log compaction on a topic with compressed messages can (will?) cause the log cleaner to crash when it encounters those compressed messages. And the death of the cleaner thread will prevent log compaction from running on other topics, even ones that don't have compressed messages.

That means if we have a cluster where we want to use log compaction on *any* topic, we need to either:
1) apply https://issues.apache.org/jira/browse/KAFKA-1374 (or upgrade to some version it is applied)
OR
2) make sure that we don't use compressed messages in *any* topic that has log compaction turned on.

And, more specifically, if we want to make use of __consumer_offsets, then we cannot use compressed messages in any topic that has compaction turned on.

Is that right?
-James

> -Todd
> 
> 
> On Fri, Sep 18, 2015 at 9:54 AM, John Holland <
> john.holland@objectpartners.com> wrote:
> 
>> Thanks!
>> 
>> I did what you suggested and it worked except it was necessary for me to
>> remove the cleaner-offset-checkpoint file from the data directory and
>> restart the servers.  The log indicates all is well.
>> 
>> Do you know what version the fix to this will be in? I'm not looking
>> forward to dealing with this on a reoccurring basis.
>> 
>> -John
>> 
>> On Fri, Sep 18, 2015 at 8:48 AM Todd Palino <tp...@gmail.com> wrote:
>> 
>>> Yes, this is a known concern, and it should be fixed with recent commits.
>>> In the meantime, you'll have to do a little manual cleanup.
>>> 
>>> The problem you're running into is a corrupt message in the offsets
>> topic.
>>> We've seen this a lot. What you need to do is set the topic configuration
>>> to remove the cleanup.policy config, and set retention.ms and segment.ms
>>> to
>>> something reasonably low. I suggest using a value of 3 or 4 times your
>>> commit interval for consumers. Then wait until the log segments are
>> reaped
>>> (wait twice as long as the retention.ms you chose, to be safe). Once
>> this
>>> is done, you can set the topic configuration back the way it was (remove
>>> segment.ms and retention.ms configs, and set cleanup.policy=compact).
>>> Lastly, you'll need to do a rolling bounce of the cluster to restart the
>>> brokers (which restarts the log cleaner threads). Technically, you only
>>> need to restart brokers where the threads have died, but it's easier to
>>> just restart all of them.
>>> 
>>> Keep in mind that when you do this, you are deleting old offsets. If your
>>> consumers are all live and healthy, this shouldn't be a problem because
>>> they will just continue to commit their offsets properly. But if you have
>>> an offline consumer, you'll lose the committed offsets by doing this.
>>> 
>>> -Todd
>>> 
>>> 
>>> On Fri, Sep 18, 2015 at 5:31 AM, John Holland <
>>> john.holland@objectpartners.com> wrote:
>>> 
>>>> I've been experiencing this issue across several of our environments
>> ever
>>>> since we enabled the log cleaner for the __consumer_offsets topic.
>>>> 
>>>> We are on version 0.8.2.1 of kafka, using the new producer.  All of our
>>>> consumers are set to commit to kafka only.
>>>> 
>>>> Below is the stack trace in the log I've encountered across several
>>>> different clusters.  A simple restart of kafka will allow compaction to
>>>> continue on all of the other partitions but the incorrect one will
>> always
>>>> fail.
>>>> 
>>>> Here are the values for it from the kafka-topics --describe command:
>>>> 
>>>> Topic:__consumer_offsets PartitionCount:50 ReplicationFactor:3
>>>> Configs:segment.bytes=104857600,cleanup.policy=compact
>>>> 
>>>> Are there any recommendations on how to prevent this and the best way
>> to
>>>> recover from this exception?  This is causing disk space to fill up
>>> quickly
>>>> on the node.
>>>> 
>>>> I did see an open issue that seems very similar to this
>>>> https://issues.apache.org/jira/browse/KAFKA-1641 but this is the
>>>> __consumer_offsets topic which I have not had any part in setting up
>> nor
>>>> producing to.
>>>> 
>>>> [2015-09-18 02:57:25,520] INFO Cleaner 0: Beginning cleaning of log
>>>> __consumer_offsets-17. (kafka.log.LogCleaner)
>>>> [2015-09-18 02:57:25,520] INFO Cleaner 0: Building offset map for
>>>> __consumer_offsets-17... (kafka.log.LogCleaner)
>>>> [2015-09-18 02:57:25,609] INFO Cleaner 0: Building offset map for log
>>>> __consumer_offsets-17 for 46 segments in offset range [468079184,
>>>> 528707475). (kafka.log.LogCleaner)
>>>> [2015-09-18 02:57:25,645] ERROR [kafka-log-cleaner-thread-0], Error due
>>> to
>>>> (kafka.log.LogCleaner)
>>>> java.lang.IllegalArgumentException: requirement failed: Last clean
>> offset
>>>> is 468079184 but segment base offset is 0 for log
>> __consumer_offsets-17.
>>>>        at scala.Predef$.require(Predef.scala:233)
>>>>        at kafka.log.Cleaner.buildOffsetMap(LogCleaner.scala:509)
>>>>        at kafka.log.Cleaner.clean(LogCleaner.scala:307)
>>>>        at
>>>> kafka.log.LogCleaner$CleanerThread.cleanOrSleep(LogCleaner.scala:221)
>>>>        at
>>> kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:199)
>>>>        at
>>> kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:60)
>>>> [2015-09-18 02:57:25,654] INFO [kafka-log-cleaner-thread-0], Stopped
>>>> (kafka.log.LogCleaner)
>>>> 
>>>> -John
>>>> 
>>> 
>> 


Re: Log Cleaner Thread Stops

Posted by Jason Rosenberg <jb...@squareup.com>.
It looks like that fix will not be included in a release until 0.9.0.0.

I'm thinking maybe it makes sense not to switch to kafka storage for
offsets until then?

Jason

On Fri, Sep 18, 2015 at 1:25 PM, Todd Palino <tp...@gmail.com> wrote:

> I think the last major issue with log compaction (that it couldn't handle
> compressed messages) was committed as part of
> https://issues.apache.org/jira/browse/KAFKA-1374 in August, but I'm not
> certain what version this will end up in. It may be part of 0.8.2.2.
>
> Regardless, you'll probably be OK now. We've found that once we clean this
> issue up once it doesn't appear to recur. As long as you're not writing in
> compressed messages to a log compacted topic (and that won't happen with
> __consumer_offsets, as it's managed by the brokers themselves - it would
> only be if you were using other log compacted topics), you're likely in the
> clear now.
>
> -Todd
>
>
> On Fri, Sep 18, 2015 at 9:54 AM, John Holland <
> john.holland@objectpartners.com> wrote:
>
> > Thanks!
> >
> > I did what you suggested and it worked except it was necessary for me to
> > remove the cleaner-offset-checkpoint file from the data directory and
> > restart the servers.  The log indicates all is well.
> >
> > Do you know what version the fix to this will be in? I'm not looking
> > forward to dealing with this on a reoccurring basis.
> >
> > -John
> >
> > On Fri, Sep 18, 2015 at 8:48 AM Todd Palino <tp...@gmail.com> wrote:
> >
> > > Yes, this is a known concern, and it should be fixed with recent
> commits.
> > > In the meantime, you'll have to do a little manual cleanup.
> > >
> > > The problem you're running into is a corrupt message in the offsets
> > topic.
> > > We've seen this a lot. What you need to do is set the topic
> configuration
> > > to remove the cleanup.policy config, and set retention.ms and
> segment.ms
> > > to
> > > something reasonably low. I suggest using a value of 3 or 4 times your
> > > commit interval for consumers. Then wait until the log segments are
> > reaped
> > > (wait twice as long as the retention.ms you chose, to be safe). Once
> > this
> > > is done, you can set the topic configuration back the way it was
> (remove
> > > segment.ms and retention.ms configs, and set cleanup.policy=compact).
> > > Lastly, you'll need to do a rolling bounce of the cluster to restart
> the
> > > brokers (which restarts the log cleaner threads). Technically, you only
> > > need to restart brokers where the threads have died, but it's easier to
> > > just restart all of them.
> > >
> > > Keep in mind that when you do this, you are deleting old offsets. If
> your
> > > consumers are all live and healthy, this shouldn't be a problem because
> > > they will just continue to commit their offsets properly. But if you
> have
> > > an offline consumer, you'll lose the committed offsets by doing this.
> > >
> > > -Todd
> > >
> > >
> > > On Fri, Sep 18, 2015 at 5:31 AM, John Holland <
> > > john.holland@objectpartners.com> wrote:
> > >
> > > > I've been experiencing this issue across several of our environments
> > ever
> > > > since we enabled the log cleaner for the __consumer_offsets topic.
> > > >
> > > > We are on version 0.8.2.1 of kafka, using the new producer.  All of
> our
> > > > consumers are set to commit to kafka only.
> > > >
> > > > Below is the stack trace in the log I've encountered across several
> > > > different clusters.  A simple restart of kafka will allow compaction
> to
> > > > continue on all of the other partitions but the incorrect one will
> > always
> > > > fail.
> > > >
> > > > Here are the values for it from the kafka-topics --describe command:
> > > >
> > > > Topic:__consumer_offsets PartitionCount:50 ReplicationFactor:3
> > > > Configs:segment.bytes=104857600,cleanup.policy=compact
> > > >
> > > > Are there any recommendations on how to prevent this and the best way
> > to
> > > > recover from this exception?  This is causing disk space to fill up
> > > quickly
> > > > on the node.
> > > >
> > > > I did see an open issue that seems very similar to this
> > > > https://issues.apache.org/jira/browse/KAFKA-1641 but this is the
> > > > __consumer_offsets topic which I have not had any part in setting up
> > nor
> > > > producing to.
> > > >
> > > > [2015-09-18 02:57:25,520] INFO Cleaner 0: Beginning cleaning of log
> > > > __consumer_offsets-17. (kafka.log.LogCleaner)
> > > > [2015-09-18 02:57:25,520] INFO Cleaner 0: Building offset map for
> > > > __consumer_offsets-17... (kafka.log.LogCleaner)
> > > > [2015-09-18 02:57:25,609] INFO Cleaner 0: Building offset map for log
> > > > __consumer_offsets-17 for 46 segments in offset range [468079184,
> > > > 528707475). (kafka.log.LogCleaner)
> > > > [2015-09-18 02:57:25,645] ERROR [kafka-log-cleaner-thread-0], Error
> due
> > > to
> > > >  (kafka.log.LogCleaner)
> > > > java.lang.IllegalArgumentException: requirement failed: Last clean
> > offset
> > > > is 468079184 but segment base offset is 0 for log
> > __consumer_offsets-17.
> > > >         at scala.Predef$.require(Predef.scala:233)
> > > >         at kafka.log.Cleaner.buildOffsetMap(LogCleaner.scala:509)
> > > >         at kafka.log.Cleaner.clean(LogCleaner.scala:307)
> > > >         at
> > > > kafka.log.LogCleaner$CleanerThread.cleanOrSleep(LogCleaner.scala:221)
> > > >         at
> > > kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:199)
> > > >         at
> > > kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:60)
> > > > [2015-09-18 02:57:25,654] INFO [kafka-log-cleaner-thread-0], Stopped
> > > >  (kafka.log.LogCleaner)
> > > >
> > > > -John
> > > >
> > >
> >
>

Re: Log Cleaner Thread Stops

Posted by Todd Palino <tp...@gmail.com>.
I think the last major issue with log compaction (that it couldn't handle
compressed messages) was committed as part of
https://issues.apache.org/jira/browse/KAFKA-1374 in August, but I'm not
certain what version this will end up in. It may be part of 0.8.2.2.

Regardless, you'll probably be OK now. We've found that once we clean this
issue up once it doesn't appear to recur. As long as you're not writing in
compressed messages to a log compacted topic (and that won't happen with
__consumer_offsets, as it's managed by the brokers themselves - it would
only be if you were using other log compacted topics), you're likely in the
clear now.

-Todd


On Fri, Sep 18, 2015 at 9:54 AM, John Holland <
john.holland@objectpartners.com> wrote:

> Thanks!
>
> I did what you suggested and it worked except it was necessary for me to
> remove the cleaner-offset-checkpoint file from the data directory and
> restart the servers.  The log indicates all is well.
>
> Do you know what version the fix to this will be in? I'm not looking
> forward to dealing with this on a reoccurring basis.
>
> -John
>
> On Fri, Sep 18, 2015 at 8:48 AM Todd Palino <tp...@gmail.com> wrote:
>
> > Yes, this is a known concern, and it should be fixed with recent commits.
> > In the meantime, you'll have to do a little manual cleanup.
> >
> > The problem you're running into is a corrupt message in the offsets
> topic.
> > We've seen this a lot. What you need to do is set the topic configuration
> > to remove the cleanup.policy config, and set retention.ms and segment.ms
> > to
> > something reasonably low. I suggest using a value of 3 or 4 times your
> > commit interval for consumers. Then wait until the log segments are
> reaped
> > (wait twice as long as the retention.ms you chose, to be safe). Once
> this
> > is done, you can set the topic configuration back the way it was (remove
> > segment.ms and retention.ms configs, and set cleanup.policy=compact).
> > Lastly, you'll need to do a rolling bounce of the cluster to restart the
> > brokers (which restarts the log cleaner threads). Technically, you only
> > need to restart brokers where the threads have died, but it's easier to
> > just restart all of them.
> >
> > Keep in mind that when you do this, you are deleting old offsets. If your
> > consumers are all live and healthy, this shouldn't be a problem because
> > they will just continue to commit their offsets properly. But if you have
> > an offline consumer, you'll lose the committed offsets by doing this.
> >
> > -Todd
> >
> >
> > On Fri, Sep 18, 2015 at 5:31 AM, John Holland <
> > john.holland@objectpartners.com> wrote:
> >
> > > I've been experiencing this issue across several of our environments
> ever
> > > since we enabled the log cleaner for the __consumer_offsets topic.
> > >
> > > We are on version 0.8.2.1 of kafka, using the new producer.  All of our
> > > consumers are set to commit to kafka only.
> > >
> > > Below is the stack trace in the log I've encountered across several
> > > different clusters.  A simple restart of kafka will allow compaction to
> > > continue on all of the other partitions but the incorrect one will
> always
> > > fail.
> > >
> > > Here are the values for it from the kafka-topics --describe command:
> > >
> > > Topic:__consumer_offsets PartitionCount:50 ReplicationFactor:3
> > > Configs:segment.bytes=104857600,cleanup.policy=compact
> > >
> > > Are there any recommendations on how to prevent this and the best way
> to
> > > recover from this exception?  This is causing disk space to fill up
> > quickly
> > > on the node.
> > >
> > > I did see an open issue that seems very similar to this
> > > https://issues.apache.org/jira/browse/KAFKA-1641 but this is the
> > > __consumer_offsets topic which I have not had any part in setting up
> nor
> > > producing to.
> > >
> > > [2015-09-18 02:57:25,520] INFO Cleaner 0: Beginning cleaning of log
> > > __consumer_offsets-17. (kafka.log.LogCleaner)
> > > [2015-09-18 02:57:25,520] INFO Cleaner 0: Building offset map for
> > > __consumer_offsets-17... (kafka.log.LogCleaner)
> > > [2015-09-18 02:57:25,609] INFO Cleaner 0: Building offset map for log
> > > __consumer_offsets-17 for 46 segments in offset range [468079184,
> > > 528707475). (kafka.log.LogCleaner)
> > > [2015-09-18 02:57:25,645] ERROR [kafka-log-cleaner-thread-0], Error due
> > to
> > >  (kafka.log.LogCleaner)
> > > java.lang.IllegalArgumentException: requirement failed: Last clean
> offset
> > > is 468079184 but segment base offset is 0 for log
> __consumer_offsets-17.
> > >         at scala.Predef$.require(Predef.scala:233)
> > >         at kafka.log.Cleaner.buildOffsetMap(LogCleaner.scala:509)
> > >         at kafka.log.Cleaner.clean(LogCleaner.scala:307)
> > >         at
> > > kafka.log.LogCleaner$CleanerThread.cleanOrSleep(LogCleaner.scala:221)
> > >         at
> > kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:199)
> > >         at
> > kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:60)
> > > [2015-09-18 02:57:25,654] INFO [kafka-log-cleaner-thread-0], Stopped
> > >  (kafka.log.LogCleaner)
> > >
> > > -John
> > >
> >
>

Re: Log Cleaner Thread Stops

Posted by John Holland <jo...@objectpartners.com>.
Thanks!

I did what you suggested and it worked except it was necessary for me to
remove the cleaner-offset-checkpoint file from the data directory and
restart the servers.  The log indicates all is well.

Do you know what version the fix to this will be in? I'm not looking
forward to dealing with this on a reoccurring basis.

-John

On Fri, Sep 18, 2015 at 8:48 AM Todd Palino <tp...@gmail.com> wrote:

> Yes, this is a known concern, and it should be fixed with recent commits.
> In the meantime, you'll have to do a little manual cleanup.
>
> The problem you're running into is a corrupt message in the offsets topic.
> We've seen this a lot. What you need to do is set the topic configuration
> to remove the cleanup.policy config, and set retention.ms and segment.ms
> to
> something reasonably low. I suggest using a value of 3 or 4 times your
> commit interval for consumers. Then wait until the log segments are reaped
> (wait twice as long as the retention.ms you chose, to be safe). Once this
> is done, you can set the topic configuration back the way it was (remove
> segment.ms and retention.ms configs, and set cleanup.policy=compact).
> Lastly, you'll need to do a rolling bounce of the cluster to restart the
> brokers (which restarts the log cleaner threads). Technically, you only
> need to restart brokers where the threads have died, but it's easier to
> just restart all of them.
>
> Keep in mind that when you do this, you are deleting old offsets. If your
> consumers are all live and healthy, this shouldn't be a problem because
> they will just continue to commit their offsets properly. But if you have
> an offline consumer, you'll lose the committed offsets by doing this.
>
> -Todd
>
>
> On Fri, Sep 18, 2015 at 5:31 AM, John Holland <
> john.holland@objectpartners.com> wrote:
>
> > I've been experiencing this issue across several of our environments ever
> > since we enabled the log cleaner for the __consumer_offsets topic.
> >
> > We are on version 0.8.2.1 of kafka, using the new producer.  All of our
> > consumers are set to commit to kafka only.
> >
> > Below is the stack trace in the log I've encountered across several
> > different clusters.  A simple restart of kafka will allow compaction to
> > continue on all of the other partitions but the incorrect one will always
> > fail.
> >
> > Here are the values for it from the kafka-topics --describe command:
> >
> > Topic:__consumer_offsets PartitionCount:50 ReplicationFactor:3
> > Configs:segment.bytes=104857600,cleanup.policy=compact
> >
> > Are there any recommendations on how to prevent this and the best way to
> > recover from this exception?  This is causing disk space to fill up
> quickly
> > on the node.
> >
> > I did see an open issue that seems very similar to this
> > https://issues.apache.org/jira/browse/KAFKA-1641 but this is the
> > __consumer_offsets topic which I have not had any part in setting up nor
> > producing to.
> >
> > [2015-09-18 02:57:25,520] INFO Cleaner 0: Beginning cleaning of log
> > __consumer_offsets-17. (kafka.log.LogCleaner)
> > [2015-09-18 02:57:25,520] INFO Cleaner 0: Building offset map for
> > __consumer_offsets-17... (kafka.log.LogCleaner)
> > [2015-09-18 02:57:25,609] INFO Cleaner 0: Building offset map for log
> > __consumer_offsets-17 for 46 segments in offset range [468079184,
> > 528707475). (kafka.log.LogCleaner)
> > [2015-09-18 02:57:25,645] ERROR [kafka-log-cleaner-thread-0], Error due
> to
> >  (kafka.log.LogCleaner)
> > java.lang.IllegalArgumentException: requirement failed: Last clean offset
> > is 468079184 but segment base offset is 0 for log __consumer_offsets-17.
> >         at scala.Predef$.require(Predef.scala:233)
> >         at kafka.log.Cleaner.buildOffsetMap(LogCleaner.scala:509)
> >         at kafka.log.Cleaner.clean(LogCleaner.scala:307)
> >         at
> > kafka.log.LogCleaner$CleanerThread.cleanOrSleep(LogCleaner.scala:221)
> >         at
> kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:199)
> >         at
> kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:60)
> > [2015-09-18 02:57:25,654] INFO [kafka-log-cleaner-thread-0], Stopped
> >  (kafka.log.LogCleaner)
> >
> > -John
> >
>

Re: Log Cleaner Thread Stops

Posted by Todd Palino <tp...@gmail.com>.
Yes, this is a known concern, and it should be fixed with recent commits.
In the meantime, you'll have to do a little manual cleanup.

The problem you're running into is a corrupt message in the offsets topic.
We've seen this a lot. What you need to do is set the topic configuration
to remove the cleanup.policy config, and set retention.ms and segment.ms to
something reasonably low. I suggest using a value of 3 or 4 times your
commit interval for consumers. Then wait until the log segments are reaped
(wait twice as long as the retention.ms you chose, to be safe). Once this
is done, you can set the topic configuration back the way it was (remove
segment.ms and retention.ms configs, and set cleanup.policy=compact).
Lastly, you'll need to do a rolling bounce of the cluster to restart the
brokers (which restarts the log cleaner threads). Technically, you only
need to restart brokers where the threads have died, but it's easier to
just restart all of them.

Keep in mind that when you do this, you are deleting old offsets. If your
consumers are all live and healthy, this shouldn't be a problem because
they will just continue to commit their offsets properly. But if you have
an offline consumer, you'll lose the committed offsets by doing this.

-Todd


On Fri, Sep 18, 2015 at 5:31 AM, John Holland <
john.holland@objectpartners.com> wrote:

> I've been experiencing this issue across several of our environments ever
> since we enabled the log cleaner for the __consumer_offsets topic.
>
> We are on version 0.8.2.1 of kafka, using the new producer.  All of our
> consumers are set to commit to kafka only.
>
> Below is the stack trace in the log I've encountered across several
> different clusters.  A simple restart of kafka will allow compaction to
> continue on all of the other partitions but the incorrect one will always
> fail.
>
> Here are the values for it from the kafka-topics --describe command:
>
> Topic:__consumer_offsets PartitionCount:50 ReplicationFactor:3
> Configs:segment.bytes=104857600,cleanup.policy=compact
>
> Are there any recommendations on how to prevent this and the best way to
> recover from this exception?  This is causing disk space to fill up quickly
> on the node.
>
> I did see an open issue that seems very similar to this
> https://issues.apache.org/jira/browse/KAFKA-1641 but this is the
> __consumer_offsets topic which I have not had any part in setting up nor
> producing to.
>
> [2015-09-18 02:57:25,520] INFO Cleaner 0: Beginning cleaning of log
> __consumer_offsets-17. (kafka.log.LogCleaner)
> [2015-09-18 02:57:25,520] INFO Cleaner 0: Building offset map for
> __consumer_offsets-17... (kafka.log.LogCleaner)
> [2015-09-18 02:57:25,609] INFO Cleaner 0: Building offset map for log
> __consumer_offsets-17 for 46 segments in offset range [468079184,
> 528707475). (kafka.log.LogCleaner)
> [2015-09-18 02:57:25,645] ERROR [kafka-log-cleaner-thread-0], Error due to
>  (kafka.log.LogCleaner)
> java.lang.IllegalArgumentException: requirement failed: Last clean offset
> is 468079184 but segment base offset is 0 for log __consumer_offsets-17.
>         at scala.Predef$.require(Predef.scala:233)
>         at kafka.log.Cleaner.buildOffsetMap(LogCleaner.scala:509)
>         at kafka.log.Cleaner.clean(LogCleaner.scala:307)
>         at
> kafka.log.LogCleaner$CleanerThread.cleanOrSleep(LogCleaner.scala:221)
>         at kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:199)
>         at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:60)
> [2015-09-18 02:57:25,654] INFO [kafka-log-cleaner-thread-0], Stopped
>  (kafka.log.LogCleaner)
>
> -John
>