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 <tpal...@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.holl...@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 <tpal...@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.holl...@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 > > > > > > > > > >