There are several bugs in 0.9 around consumer offsets and compaction and log cleaning.
The easiest path forward is to upgrade to the latest 0.11.x. We ended up going to somewhat extreme lengths to deal with 100GB+ consumer offsets. When we tested an upgrade we noticed that when it started compacting that down (in the order of 45 minutes) it correlated with consumer group coordinator errors, effectively causing consumers to be unable to consume until the compacting finished. I don't know if that is still a bug. We got around this by adding a couple of new upgraded brokers to the cluster and doing partition reassignments to it (as replicas) for the affected __consumer_offset partitions, waiting for them to successfully compact, and then doing a cut over which made the two replicated ones the only copies which had the "live" brokers delete their large bugged ones, then cutting back to the original assignments. It was also necessary to then do some further partition reassignments to work around a bug where the log cleaner thread died -- the move triggered a segment clean bypassing the issue causing the log cleaner thread to die. That final one was reported as a bug but it hasn't been looked at. The workaround is easy. Once we had all the offsets small again (generally < 100/150MB), proceeding with the normal upgrade was painless. It took about 3 days to upgrade one of our clusters using this method, but it resulted in no consumer down time which was all that mattered :) On Wed, Jan 17, 2018 at 10:07 AM, Shravan R <skr...@gmail.com> wrote: > BTW, I see log segments as old as last year and offsets.retention.minutes > is set to 4 days. Any reason why it may have not been deleted? > > -rw-r--r-- 1 kafka kafka 104857532 Apr 5 2017 00000000000000000000.log > -rw-r--r-- 1 kafka kafka 104857564 Apr 6 2017 00000000000001219197.log > -rw-r--r-- 1 kafka kafka 104856962 Apr 6 2017 00000000000002438471.log > -rw-r--r-- 1 kafka kafka 104857392 Apr 6 2017 00000000000003657738.log > -rw-r--r-- 1 kafka kafka 104857564 Apr 6 2017 00000000000004877010.log > -rw-r--r-- 1 kafka kafka 104857392 Apr 7 2017 00000000000006096284.log > -rw-r--r-- 1 kafka kafka 104857478 Apr 7 2017 00000000000007315556.log > -rw-r--r-- 1 kafka kafka 104857306 Apr 7 2017 00000000000008534829.log > -rw-r--r-- 1 kafka kafka 104857134 Apr 7 2017 00000000000009754100.log > -rw-r--r-- 1 kafka kafka 104857564 Apr 7 2017 00000000000010973369.log > -rw-r--r-- 1 kafka kafka 104857564 Apr 7 2017 00000000000012192643.log > -rw-r--r-- 1 kafka kafka 104857578 Apr 7 2017 00000000000013411917.log > > > On Tue, Jan 16, 2018 at 1:04 PM, Shravan R <skr...@gmail.com> wrote: > > > I looked into it. I played with log.cleaner.dedupe.buffer.size between > > 256MB to 2GB while keeping log.cleaner.threads=1 but that did not help > > me. I helped me to recover from __consumer_offsets-33 but got into a > > similar exception on another partition. There no lags on our system and > > that is not a concern at this time. Is there any work around or tuning > that > > I can do? > > > > Thanks, > > SK > > > > On Tue, Jan 16, 2018 at 10:56 AM, naresh Goud < > nareshgoud.du...@gmail.com> > > wrote: > > > >> Can you check if jira KAFKA-3894 helps? > >> > >> > >> Thank you, > >> Naresh > >> > >> On Tue, Jan 16, 2018 at 10:28 AM Shravan R <skr...@gmail.com> wrote: > >> > >> > We are running Kafka-0.9 and I am seeing large __consumer_offsets on > >> some > >> > of the partitions of the order of 100GB or more. I see some of the log > >> and > >> > index files are more than a year old. I see the following properties > >> that > >> > are of interest. > >> > > >> > offsets.retention.minutes=5769 (4 Days) > >> > log.cleaner.dedupe.buffer.size=256000000 (256MB) > >> > num.recovery.threads.per.data.dir=4 > >> > log.cleaner.enable=true > >> > log.cleaner.threads=1 > >> > > >> > > >> > Upon restarting of the broker, I see the below exception which clearly > >> > indicates a problem with dedupe buffer size. However, I see the dedupe > >> > buffer size is set to 256MB which is far more than what the log > >> complains > >> > about (37MB). What could be the problem here? How can I get the > offsets > >> > topic size under manageable size? > >> > > >> > > >> > 2018-01-15 21:26:51,434 ERROR kafka.log.LogCleaner: > >> > [kafka-log-cleaner-thread-0], Error due to > >> > java.lang.IllegalArgumentException: requirement failed: 990238234 > >> messages > >> > in segment __consumer_offsets-33/00000000000000000000.log but offset > >> map > >> > can > >> > fit only 37499999. 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:591) > >> > at > >> > kafka.log.Cleaner$$anonfun$buildOffsetMap$4.apply( > LogCleaner.scala:587) > >> > at > >> > > >> > scala.collection.immutable.Stream$StreamWithFilter.foreach( > >> Stream.scala:570) > >> > at kafka.log.Cleaner.buildOffsetMap(LogCleaner.scala:587) > >> > at kafka.log.Cleaner.clean(LogCleaner.scala:329) > >> > at > >> > kafka.log.LogCleaner$CleanerThread.cleanOrSleep(LogCleaner.scala:237) > >> > at kafka.log.LogCleaner$CleanerThread.doWork( > LogCleaner.scala: > >> 215) > >> > at kafka.utils.ShutdownableThread.run( > ShutdownableThread.scala: > >> 63) > >> > 2018-01-15 21:26:51,436 INFO kafka.log.LogCleaner: > >> > [kafka-log-cleaner-thread-0], Stopped > >> > > >> > > >> > > >> > Thanks, > >> > -SK > >> > > >> > > > > >