You are viewing a plain text version of this content. The canonical link for it is here.
Posted to users@kafka.apache.org by Luciano Afranllie <li...@gmail.com> on 2017/07/18 15:44:16 UTC

Consumer offsets partitions size much bigger than others

Hi

One of our Kafka brokers was running out of disk space and when we checked
the file size in the kafka log dir we observed the following

$ du -h . --max-depth=2 | grep '__consumer_offsets'
4.0K    ./kafka-logs/__consumer_offsets-16
4.0K    ./kafka-logs/__consumer_offsets-40
35G     ./kafka-logs/__consumer_offsets-44
4.0K    ./kafka-logs/__consumer_offsets-8
4.0K    ./kafka-logs/__consumer_offsets-38
4.0K    ./kafka-logs/__consumer_offsets-20
4.0K    ./kafka-logs/__consumer_offsets-34
4.0K    ./kafka-logs/__consumer_offsets-18
4.0K    ./kafka-logs/__consumer_offsets-32
251G    ./kafka-logs/__consumer_offsets-14
4.0K    ./kafka-logs/__consumer_offsets-4
4.0K    ./kafka-logs/__consumer_offsets-26
4.0K    ./kafka-logs/__consumer_offsets-12
4.0K    ./kafka-logs/__consumer_offsets-30
4.0K    ./kafka-logs/__consumer_offsets-6
4.0K    ./kafka-logs/__consumer_offsets-2
4.0K    ./kafka-logs/__consumer_offsets-24
4.0K    ./kafka-logs/__consumer_offsets-36
4.0K    ./kafka-logs/__consumer_offsets-46
4.0K    ./kafka-logs/__consumer_offsets-42
4.0K    ./kafka-logs/__consumer_offsets-22
4.0K    ./kafka-logs/__consumer_offsets-0
4.0K    ./kafka-logs/__consumer_offsets-28
4.0K    ./kafka-logs/__consumer_offsets-10
4.0K    ./kafka-logs/__consumer_offsets-48

As you can see, two of the log files (partition 44 and 14) have a huge
size. Do you have a hint to understand what could be happening here? May be
for some reason this partitions are not being compacted?

By the way, this is the description of the __consumer_offsets topic.

# ./bin/kafka-topics.sh --describe --zookeeper x.x.x.x:2181 --topic
__consumer_offsets
Topic:__consumer_offsets        PartitionCount:50       ReplicationFactor:1

Configs:segment.bytes=104857600,cleanup.policy=compact,compression.type=uncompressed
        Topic: __consumer_offsets       Partition: 0    Leader: 1
Replicas: 1     Isr: 1
        Topic: __consumer_offsets       Partition: 1    Leader: 2
Replicas: 2     Isr: 2
        Topic: __consumer_offsets       Partition: 2    Leader: 1
Replicas: 1     Isr: 1
        Topic: __consumer_offsets       Partition: 3    Leader: 2
Replicas: 2     Isr: 2
        Topic: __consumer_offsets       Partition: 4    Leader: 1
Replicas: 1     Isr: 1
        Topic: __consumer_offsets       Partition: 5    Leader: 2
Replicas: 2     Isr: 2
        Topic: __consumer_offsets       Partition: 6    Leader: 1
Replicas: 1     Isr: 1
        Topic: __consumer_offsets       Partition: 7    Leader: 2
Replicas: 2     Isr: 2
        Topic: __consumer_offsets       Partition: 8    Leader: 1
Replicas: 1     Isr: 1
        Topic: __consumer_offsets       Partition: 9    Leader: 2
Replicas: 2     Isr: 2
        Topic: __consumer_offsets       Partition: 10   Leader: 1
Replicas: 1     Isr: 1
        Topic: __consumer_offsets       Partition: 11   Leader: 2
Replicas: 2     Isr: 2
        Topic: __consumer_offsets       Partition: 12   Leader: 1
Replicas: 1     Isr: 1
        Topic: __consumer_offsets       Partition: 13   Leader: 2
Replicas: 2     Isr: 2
        Topic: __consumer_offsets       Partition: 14   Leader: 1
Replicas: 1     Isr: 1
        Topic: __consumer_offsets       Partition: 15   Leader: 2
Replicas: 2     Isr: 2
        Topic: __consumer_offsets       Partition: 16   Leader: 1
Replicas: 1     Isr: 1
        Topic: __consumer_offsets       Partition: 17   Leader: 2
Replicas: 2     Isr: 2
        Topic: __consumer_offsets       Partition: 18   Leader: 1
Replicas: 1     Isr: 1
        Topic: __consumer_offsets       Partition: 19   Leader: 2
Replicas: 2     Isr: 2
        Topic: __consumer_offsets       Partition: 20   Leader: 1
Replicas: 1     Isr: 1
        Topic: __consumer_offsets       Partition: 21   Leader: 2
Replicas: 2     Isr: 2
        Topic: __consumer_offsets       Partition: 22   Leader: 1
Replicas: 1     Isr: 1
        Topic: __consumer_offsets       Partition: 23   Leader: 2
Replicas: 2     Isr: 2
        Topic: __consumer_offsets       Partition: 24   Leader: 1
Replicas: 1     Isr: 1
        Topic: __consumer_offsets       Partition: 25   Leader: 2
Replicas: 2     Isr: 2
        Topic: __consumer_offsets       Partition: 26   Leader: 1
Replicas: 1     Isr: 1
        Topic: __consumer_offsets       Partition: 27   Leader: 2
Replicas: 2     Isr: 2
        Topic: __consumer_offsets       Partition: 28   Leader: 1
Replicas: 1     Isr: 1
        Topic: __consumer_offsets       Partition: 29   Leader: 2
Replicas: 2     Isr: 2
        Topic: __consumer_offsets       Partition: 30   Leader: 1
Replicas: 1     Isr: 1
        Topic: __consumer_offsets       Partition: 31   Leader: 2
Replicas: 2     Isr: 2
        Topic: __consumer_offsets       Partition: 32   Leader: 1
Replicas: 1     Isr: 1
        Topic: __consumer_offsets       Partition: 33   Leader: 2
Replicas: 2     Isr: 2
        Topic: __consumer_offsets       Partition: 34   Leader: 1
Replicas: 1     Isr: 1
        Topic: __consumer_offsets       Partition: 35   Leader: 2
Replicas: 2     Isr: 2
        Topic: __consumer_offsets       Partition: 36   Leader: 1
Replicas: 1     Isr: 1
        Topic: __consumer_offsets       Partition: 37   Leader: 2
Replicas: 2     Isr: 2
        Topic: __consumer_offsets       Partition: 38   Leader: 1
Replicas: 1     Isr: 1
        Topic: __consumer_offsets       Partition: 39   Leader: 2
Replicas: 2     Isr: 2
        Topic: __consumer_offsets       Partition: 40   Leader: 1
Replicas: 1     Isr: 1
        Topic: __consumer_offsets       Partition: 41   Leader: 2
Replicas: 2     Isr: 2
        Topic: __consumer_offsets       Partition: 42   Leader: 1
Replicas: 1     Isr: 1
        Topic: __consumer_offsets       Partition: 43   Leader: 2
Replicas: 2     Isr: 2
        Topic: __consumer_offsets       Partition: 44   Leader: 1
Replicas: 1     Isr: 1
        Topic: __consumer_offsets       Partition: 45   Leader: 2
Replicas: 2     Isr: 2
        Topic: __consumer_offsets       Partition: 46   Leader: 1
Replicas: 1     Isr: 1
        Topic: __consumer_offsets       Partition: 47   Leader: 2
Replicas: 2     Isr: 2
        Topic: __consumer_offsets       Partition: 48   Leader: 1
Replicas: 1     Isr: 1
        Topic: __consumer_offsets       Partition: 49   Leader: 2
Replicas: 2     Isr: 2

Regards
Luciano

Re: Consumer offsets partitions size much bigger than others

Posted by Luciano Afranllie <li...@gmail.com>.
Thanks James

The issue was that the log cleaner was disabled in that cluster due to an
old configuration that we have since 0.9.0.0.

Regards
Luciano

On Tue, Jul 18, 2017 at 7:04 PM, James Cheng <wu...@gmail.com> wrote:

> It's possible that the log-cleaning thread has crashed. That is the thread
> that implements log compaction.
>
> Look in the log-cleaner.log file in your kafka debuglog directory to see
> if there is any indication that it has crashed (error messages, stack
> traces, etc).
>
> What version of kafka are you using? 0.10 and prior had some bugs in the
> log-cleaner thread that might sometimes cause it to crash. Those were fixed
> in later versions, but it's always possible there might still be more bugs
> there.
>
> I notice that your __consumer_offsets topic only has replication-factor=1.
> How many brokers are in your cluster? You should increase the replication
> factor to 3.
>
> Older versions of kafka would try to auto-create the __consumer_offsets
> topic with replication-factor 3 but if there were fewer than 3 brokers in
> the cluster, then they would simply use the number of brokers in the
> cluster. What that means is that if your cluster only had 1 broker running
> at the time the topic was auto-created, that it would be created with
> replication-factor 1. This has been fixed in later brokers, so that it will
> always create topics with the specified number of replicas or will throw
> loud errors in the event you don't have enough brokers.
>
> -James
>
> > On Jul 18, 2017, at 8:44 AM, Luciano Afranllie <li...@gmail.com>
> wrote:
> >
> > Hi
> >
> > One of our Kafka brokers was running out of disk space and when we
> checked
> > the file size in the kafka log dir we observed the following
> >
> > $ du -h . --max-depth=2 | grep '__consumer_offsets'
> > 4.0K    ./kafka-logs/__consumer_offsets-16
> > 4.0K    ./kafka-logs/__consumer_offsets-40
> > 35G     ./kafka-logs/__consumer_offsets-44
> > 4.0K    ./kafka-logs/__consumer_offsets-8
> > 4.0K    ./kafka-logs/__consumer_offsets-38
> > 4.0K    ./kafka-logs/__consumer_offsets-20
> > 4.0K    ./kafka-logs/__consumer_offsets-34
> > 4.0K    ./kafka-logs/__consumer_offsets-18
> > 4.0K    ./kafka-logs/__consumer_offsets-32
> > 251G    ./kafka-logs/__consumer_offsets-14
> > 4.0K    ./kafka-logs/__consumer_offsets-4
> > 4.0K    ./kafka-logs/__consumer_offsets-26
> > 4.0K    ./kafka-logs/__consumer_offsets-12
> > 4.0K    ./kafka-logs/__consumer_offsets-30
> > 4.0K    ./kafka-logs/__consumer_offsets-6
> > 4.0K    ./kafka-logs/__consumer_offsets-2
> > 4.0K    ./kafka-logs/__consumer_offsets-24
> > 4.0K    ./kafka-logs/__consumer_offsets-36
> > 4.0K    ./kafka-logs/__consumer_offsets-46
> > 4.0K    ./kafka-logs/__consumer_offsets-42
> > 4.0K    ./kafka-logs/__consumer_offsets-22
> > 4.0K    ./kafka-logs/__consumer_offsets-0
> > 4.0K    ./kafka-logs/__consumer_offsets-28
> > 4.0K    ./kafka-logs/__consumer_offsets-10
> > 4.0K    ./kafka-logs/__consumer_offsets-48
> >
> > As you can see, two of the log files (partition 44 and 14) have a huge
> > size. Do you have a hint to understand what could be happening here? May
> be
> > for some reason this partitions are not being compacted?
> >
> > By the way, this is the description of the __consumer_offsets topic.
> >
> > # ./bin/kafka-topics.sh --describe --zookeeper x.x.x.x:2181 --topic
> > __consumer_offsets
> > Topic:__consumer_offsets        PartitionCount:50
>  ReplicationFactor:1
> >
> > Configs:segment.bytes=104857600,cleanup.policy=compact,compression.type=
> uncompressed
> >        Topic: __consumer_offsets       Partition: 0    Leader: 1
> > Replicas: 1     Isr: 1
> >        Topic: __consumer_offsets       Partition: 1    Leader: 2
> > Replicas: 2     Isr: 2
> >        Topic: __consumer_offsets       Partition: 2    Leader: 1
> > Replicas: 1     Isr: 1
> >        Topic: __consumer_offsets       Partition: 3    Leader: 2
> > Replicas: 2     Isr: 2
> >        Topic: __consumer_offsets       Partition: 4    Leader: 1
> > Replicas: 1     Isr: 1
> >        Topic: __consumer_offsets       Partition: 5    Leader: 2
> > Replicas: 2     Isr: 2
> >        Topic: __consumer_offsets       Partition: 6    Leader: 1
> > Replicas: 1     Isr: 1
> >        Topic: __consumer_offsets       Partition: 7    Leader: 2
> > Replicas: 2     Isr: 2
> >        Topic: __consumer_offsets       Partition: 8    Leader: 1
> > Replicas: 1     Isr: 1
> >        Topic: __consumer_offsets       Partition: 9    Leader: 2
> > Replicas: 2     Isr: 2
> >        Topic: __consumer_offsets       Partition: 10   Leader: 1
> > Replicas: 1     Isr: 1
> >        Topic: __consumer_offsets       Partition: 11   Leader: 2
> > Replicas: 2     Isr: 2
> >        Topic: __consumer_offsets       Partition: 12   Leader: 1
> > Replicas: 1     Isr: 1
> >        Topic: __consumer_offsets       Partition: 13   Leader: 2
> > Replicas: 2     Isr: 2
> >        Topic: __consumer_offsets       Partition: 14   Leader: 1
> > Replicas: 1     Isr: 1
> >        Topic: __consumer_offsets       Partition: 15   Leader: 2
> > Replicas: 2     Isr: 2
> >        Topic: __consumer_offsets       Partition: 16   Leader: 1
> > Replicas: 1     Isr: 1
> >        Topic: __consumer_offsets       Partition: 17   Leader: 2
> > Replicas: 2     Isr: 2
> >        Topic: __consumer_offsets       Partition: 18   Leader: 1
> > Replicas: 1     Isr: 1
> >        Topic: __consumer_offsets       Partition: 19   Leader: 2
> > Replicas: 2     Isr: 2
> >        Topic: __consumer_offsets       Partition: 20   Leader: 1
> > Replicas: 1     Isr: 1
> >        Topic: __consumer_offsets       Partition: 21   Leader: 2
> > Replicas: 2     Isr: 2
> >        Topic: __consumer_offsets       Partition: 22   Leader: 1
> > Replicas: 1     Isr: 1
> >        Topic: __consumer_offsets       Partition: 23   Leader: 2
> > Replicas: 2     Isr: 2
> >        Topic: __consumer_offsets       Partition: 24   Leader: 1
> > Replicas: 1     Isr: 1
> >        Topic: __consumer_offsets       Partition: 25   Leader: 2
> > Replicas: 2     Isr: 2
> >        Topic: __consumer_offsets       Partition: 26   Leader: 1
> > Replicas: 1     Isr: 1
> >        Topic: __consumer_offsets       Partition: 27   Leader: 2
> > Replicas: 2     Isr: 2
> >        Topic: __consumer_offsets       Partition: 28   Leader: 1
> > Replicas: 1     Isr: 1
> >        Topic: __consumer_offsets       Partition: 29   Leader: 2
> > Replicas: 2     Isr: 2
> >        Topic: __consumer_offsets       Partition: 30   Leader: 1
> > Replicas: 1     Isr: 1
> >        Topic: __consumer_offsets       Partition: 31   Leader: 2
> > Replicas: 2     Isr: 2
> >        Topic: __consumer_offsets       Partition: 32   Leader: 1
> > Replicas: 1     Isr: 1
> >        Topic: __consumer_offsets       Partition: 33   Leader: 2
> > Replicas: 2     Isr: 2
> >        Topic: __consumer_offsets       Partition: 34   Leader: 1
> > Replicas: 1     Isr: 1
> >        Topic: __consumer_offsets       Partition: 35   Leader: 2
> > Replicas: 2     Isr: 2
> >        Topic: __consumer_offsets       Partition: 36   Leader: 1
> > Replicas: 1     Isr: 1
> >        Topic: __consumer_offsets       Partition: 37   Leader: 2
> > Replicas: 2     Isr: 2
> >        Topic: __consumer_offsets       Partition: 38   Leader: 1
> > Replicas: 1     Isr: 1
> >        Topic: __consumer_offsets       Partition: 39   Leader: 2
> > Replicas: 2     Isr: 2
> >        Topic: __consumer_offsets       Partition: 40   Leader: 1
> > Replicas: 1     Isr: 1
> >        Topic: __consumer_offsets       Partition: 41   Leader: 2
> > Replicas: 2     Isr: 2
> >        Topic: __consumer_offsets       Partition: 42   Leader: 1
> > Replicas: 1     Isr: 1
> >        Topic: __consumer_offsets       Partition: 43   Leader: 2
> > Replicas: 2     Isr: 2
> >        Topic: __consumer_offsets       Partition: 44   Leader: 1
> > Replicas: 1     Isr: 1
> >        Topic: __consumer_offsets       Partition: 45   Leader: 2
> > Replicas: 2     Isr: 2
> >        Topic: __consumer_offsets       Partition: 46   Leader: 1
> > Replicas: 1     Isr: 1
> >        Topic: __consumer_offsets       Partition: 47   Leader: 2
> > Replicas: 2     Isr: 2
> >        Topic: __consumer_offsets       Partition: 48   Leader: 1
> > Replicas: 1     Isr: 1
> >        Topic: __consumer_offsets       Partition: 49   Leader: 2
> > Replicas: 2     Isr: 2
> >
> > Regards
> > Luciano
>
>

Re: Consumer offsets partitions size much bigger than others

Posted by James Cheng <wu...@gmail.com>.
It's possible that the log-cleaning thread has crashed. That is the thread that implements log compaction.

Look in the log-cleaner.log file in your kafka debuglog directory to see if there is any indication that it has crashed (error messages, stack traces, etc).

What version of kafka are you using? 0.10 and prior had some bugs in the log-cleaner thread that might sometimes cause it to crash. Those were fixed in later versions, but it's always possible there might still be more bugs there.

I notice that your __consumer_offsets topic only has replication-factor=1. How many brokers are in your cluster? You should increase the replication factor to 3. 

Older versions of kafka would try to auto-create the __consumer_offsets topic with replication-factor 3 but if there were fewer than 3 brokers in the cluster, then they would simply use the number of brokers in the cluster. What that means is that if your cluster only had 1 broker running at the time the topic was auto-created, that it would be created with replication-factor 1. This has been fixed in later brokers, so that it will always create topics with the specified number of replicas or will throw loud errors in the event you don't have enough brokers.

-James

> On Jul 18, 2017, at 8:44 AM, Luciano Afranllie <li...@gmail.com> wrote:
> 
> Hi
> 
> One of our Kafka brokers was running out of disk space and when we checked
> the file size in the kafka log dir we observed the following
> 
> $ du -h . --max-depth=2 | grep '__consumer_offsets'
> 4.0K    ./kafka-logs/__consumer_offsets-16
> 4.0K    ./kafka-logs/__consumer_offsets-40
> 35G     ./kafka-logs/__consumer_offsets-44
> 4.0K    ./kafka-logs/__consumer_offsets-8
> 4.0K    ./kafka-logs/__consumer_offsets-38
> 4.0K    ./kafka-logs/__consumer_offsets-20
> 4.0K    ./kafka-logs/__consumer_offsets-34
> 4.0K    ./kafka-logs/__consumer_offsets-18
> 4.0K    ./kafka-logs/__consumer_offsets-32
> 251G    ./kafka-logs/__consumer_offsets-14
> 4.0K    ./kafka-logs/__consumer_offsets-4
> 4.0K    ./kafka-logs/__consumer_offsets-26
> 4.0K    ./kafka-logs/__consumer_offsets-12
> 4.0K    ./kafka-logs/__consumer_offsets-30
> 4.0K    ./kafka-logs/__consumer_offsets-6
> 4.0K    ./kafka-logs/__consumer_offsets-2
> 4.0K    ./kafka-logs/__consumer_offsets-24
> 4.0K    ./kafka-logs/__consumer_offsets-36
> 4.0K    ./kafka-logs/__consumer_offsets-46
> 4.0K    ./kafka-logs/__consumer_offsets-42
> 4.0K    ./kafka-logs/__consumer_offsets-22
> 4.0K    ./kafka-logs/__consumer_offsets-0
> 4.0K    ./kafka-logs/__consumer_offsets-28
> 4.0K    ./kafka-logs/__consumer_offsets-10
> 4.0K    ./kafka-logs/__consumer_offsets-48
> 
> As you can see, two of the log files (partition 44 and 14) have a huge
> size. Do you have a hint to understand what could be happening here? May be
> for some reason this partitions are not being compacted?
> 
> By the way, this is the description of the __consumer_offsets topic.
> 
> # ./bin/kafka-topics.sh --describe --zookeeper x.x.x.x:2181 --topic
> __consumer_offsets
> Topic:__consumer_offsets        PartitionCount:50       ReplicationFactor:1
> 
> Configs:segment.bytes=104857600,cleanup.policy=compact,compression.type=uncompressed
>        Topic: __consumer_offsets       Partition: 0    Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 1    Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 2    Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 3    Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 4    Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 5    Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 6    Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 7    Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 8    Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 9    Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 10   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 11   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 12   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 13   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 14   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 15   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 16   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 17   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 18   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 19   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 20   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 21   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 22   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 23   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 24   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 25   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 26   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 27   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 28   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 29   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 30   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 31   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 32   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 33   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 34   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 35   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 36   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 37   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 38   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 39   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 40   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 41   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 42   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 43   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 44   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 45   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 46   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 47   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 48   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 49   Leader: 2
> Replicas: 2     Isr: 2
> 
> Regards
> Luciano


Re: Consumer offsets partitions size much bigger than others

Posted by James Cheng <wu...@gmail.com>.
It's possible that the log-cleaning thread has crashed. That is the thread that implements log compaction.

Look in the log-cleaner.log file in your kafka debuglog directory to see if there is any indication that it has crashed (error messages, stack traces, etc).

What version of kafka are you using? 0.10 and prior had some bugs in the log-cleaner thread that might sometimes cause it to crash. Those were fixed in later versions, but it's always possible there might still be more bugs there.

I notice that your __consumer_offsets topic only has replication-factor=1. How many brokers are in your cluster? You should increase the replication factor to 3. 

Older versions of kafka would try to auto-create the __consumer_offsets topic with replication-factor 3 but if there were fewer than 3 brokers in the cluster, then they would simply use the number of brokers in the cluster. What that means is that if your cluster only had 1 broker running at the time the topic was auto-created, that it would be created with replication-factor 1. This has been fixed in later brokers, so that it will always create topics with the specified number of replicas or will throw loud errors in the event you don't have enough brokers.

-James

> On Jul 18, 2017, at 8:44 AM, Luciano Afranllie <li...@gmail.com> wrote:
> 
> Hi
> 
> One of our Kafka brokers was running out of disk space and when we checked
> the file size in the kafka log dir we observed the following
> 
> $ du -h . --max-depth=2 | grep '__consumer_offsets'
> 4.0K    ./kafka-logs/__consumer_offsets-16
> 4.0K    ./kafka-logs/__consumer_offsets-40
> 35G     ./kafka-logs/__consumer_offsets-44
> 4.0K    ./kafka-logs/__consumer_offsets-8
> 4.0K    ./kafka-logs/__consumer_offsets-38
> 4.0K    ./kafka-logs/__consumer_offsets-20
> 4.0K    ./kafka-logs/__consumer_offsets-34
> 4.0K    ./kafka-logs/__consumer_offsets-18
> 4.0K    ./kafka-logs/__consumer_offsets-32
> 251G    ./kafka-logs/__consumer_offsets-14
> 4.0K    ./kafka-logs/__consumer_offsets-4
> 4.0K    ./kafka-logs/__consumer_offsets-26
> 4.0K    ./kafka-logs/__consumer_offsets-12
> 4.0K    ./kafka-logs/__consumer_offsets-30
> 4.0K    ./kafka-logs/__consumer_offsets-6
> 4.0K    ./kafka-logs/__consumer_offsets-2
> 4.0K    ./kafka-logs/__consumer_offsets-24
> 4.0K    ./kafka-logs/__consumer_offsets-36
> 4.0K    ./kafka-logs/__consumer_offsets-46
> 4.0K    ./kafka-logs/__consumer_offsets-42
> 4.0K    ./kafka-logs/__consumer_offsets-22
> 4.0K    ./kafka-logs/__consumer_offsets-0
> 4.0K    ./kafka-logs/__consumer_offsets-28
> 4.0K    ./kafka-logs/__consumer_offsets-10
> 4.0K    ./kafka-logs/__consumer_offsets-48
> 
> As you can see, two of the log files (partition 44 and 14) have a huge
> size. Do you have a hint to understand what could be happening here? May be
> for some reason this partitions are not being compacted?
> 
> By the way, this is the description of the __consumer_offsets topic.
> 
> # ./bin/kafka-topics.sh --describe --zookeeper x.x.x.x:2181 --topic
> __consumer_offsets
> Topic:__consumer_offsets        PartitionCount:50       ReplicationFactor:1
> 
> Configs:segment.bytes=104857600,cleanup.policy=compact,compression.type=uncompressed
>        Topic: __consumer_offsets       Partition: 0    Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 1    Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 2    Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 3    Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 4    Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 5    Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 6    Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 7    Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 8    Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 9    Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 10   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 11   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 12   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 13   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 14   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 15   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 16   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 17   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 18   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 19   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 20   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 21   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 22   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 23   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 24   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 25   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 26   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 27   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 28   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 29   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 30   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 31   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 32   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 33   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 34   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 35   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 36   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 37   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 38   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 39   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 40   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 41   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 42   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 43   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 44   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 45   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 46   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 47   Leader: 2
> Replicas: 2     Isr: 2
>        Topic: __consumer_offsets       Partition: 48   Leader: 1
> Replicas: 1     Isr: 1
>        Topic: __consumer_offsets       Partition: 49   Leader: 2
> Replicas: 2     Isr: 2
> 
> Regards
> Luciano