Marcos, Radu, Are either of you seeing messages saying "Cached zkVersion [...] not equal to that in zookeeper"? If so, you may be hitting https://issues.apache.org/jira/browse/KAFKA-3042 <https://issues.apache.org/jira/browse/KAFKA-3042>
Not sure if that helps you, since I haven't been able it isolate a cause. I think it's related to flaky connections to the zookeeper cluster. -James > On Mar 22, 2017, at 11:46 AM, Marcos Juarez <mjua...@gmail.com> wrote: > > We're seeing the same exact pattern of ISR shrinking/resizing, mostly on > partitions with the largest volume, with thousands of messages per second. > It happens at least a hundred times a day in our production cluster. We do > have hundreds of topics in our cluster, most of them with 20 or more > partitions, but most of them see only a few hundred messages per minute. > > We're running Kafka 0.10.0.1, and we thought upgrading to the 0.10.1.1 > version would fix the issue, but we've already deployed that version to our > staging cluster, and we're seeing the same problem. We still haven't tried > out the latest 0.10.2.0 version, but I don't see any evidence pointing to a > fix for that. > > This ticket seems to have some similar details, but it doesn't seem like > there has been follow-up, and there's no target release for fixing: > > https://issues.apache.org/jira/browse/KAFKA-4674 > <https://issues.apache.org/jira/browse/KAFKA-4674> > > Jun Ma, what exactly did you do to failover the controller to a new broker? > If that works for you, I'd like to try it in our staging clusters. > > Thanks, > > Marcos Juarez > > > > > > On Wed, Mar 22, 2017 at 11:55 AM, Jun MA <mj.saber1...@gmail.com > <mailto:mj.saber1...@gmail.com>> wrote: > I have similar issue with our cluster. We don’t know the root cause but we > have some interesting observation. > > 1. We do see correlation between ISR churn and fetcher connection > close/create. > > > 2. We’ve tried to add a broker which doesn’t have any partitions on it > dedicate to the controller (rolling restart existing brokers and failover the > controller to the newly added broker), and that indeed eliminate the random > ISR churn. We have a cluster of 6 brokers (besides the dedicated controller) > and each one has about 300 partitions on it. I suspect that kafka broker > cannot handle running controller + 300 partitions. > > Anyway that’s so far what I got, I’d also like to know how to debug this. > We’re running kafka 0.9.0.1 with heap size 8G. > > Thanks, > Jun > >> On Mar 22, 2017, at 7:06 AM, Manikumar <manikumar.re...@gmail.com >> <mailto:manikumar.re...@gmail.com>> wrote: >> >> Any erros related to zookeeper seesion timeout? We can also check for >> excesssive GC. >> Some times this may due to forming multiple controllers due to soft >> failures. >> You can check ActiveControllerCount on brokers, only one broker in the >> cluster should have 1. >> Also check for network issues/partitions >> >> On Wed, Mar 22, 2017 at 7:21 PM, Radu Radutiu <rradu...@gmail.com >> <mailto:rradu...@gmail.com>> wrote: >> >>> Hello, >>> >>> Does anyone know how I can debug high ISR churn on the kafka leader on a >>> cluster without traffic? I have 2 topics on a 4 node cluster (replica 4 >>> and replica 3) and both show constant changes of the number of insync >>> replicas: >>> >>> [2017-03-22 15:30:10,945] INFO Partition [__consumer_offsets,0] on broker >>> 2: Expanding ISR for partition __consumer_offsets-0 from 2,4 to 2,4,5 >>> (kafka.cluster.Partition) >>> [2017-03-22 15:31:41,193] INFO Partition [__consumer_offsets,0] on broker >>> 2: Shrinking ISR for partition [__consumer_offsets,0] from 2,4,5 to 2,5 >>> (kafka.cluster.Partition) >>> [2017-03-22 15:31:41,195] INFO Partition [__consumer_offsets,0] on broker >>> 2: Expanding ISR for partition __consumer_offsets-0 from 2,5 to 2,5,4 >>> (kafka.cluster.Partition) >>> [2017-03-22 15:35:03,443] INFO Partition [__consumer_offsets,0] on broker >>> 2: Shrinking ISR for partition [__consumer_offsets,0] from 2,5,4 to 2,5 >>> (kafka.cluster.Partition) >>> [2017-03-22 15:35:03,445] INFO Partition [__consumer_offsets,0] on broker >>> 2: Expanding ISR for partition __consumer_offsets-0 from 2,5 to 2,5,4 >>> (kafka.cluster.Partition) >>> [2017-03-22 15:37:01,443] INFO Partition [__consumer_offsets,0] on broker >>> 2: Shrinking ISR for partition [__consumer_offsets,0] from 2,5,4 to 2,4 >>> (kafka.cluster.Partition) >>> [2017-03-22 15:37:01,445] INFO Partition [__consumer_offsets,0] on broker >>> 2: Expanding ISR for partition __consumer_offsets-0 from 2,4 to 2,4,5 >>> (kafka.cluster.Partition) >>> >>> and >>> >>> [2017-03-22 15:09:52,646] INFO Partition [topic1,0] on broker 5: Shrinking >>> ISR for partition [topic1,0] from 5,2,4 to 5,4 (kafka.cluster.Partition) >>> [2017-03-22 15:09:52,648] INFO Partition [topic1,0] on broker 5: Expanding >>> ISR for partition topic1-0 from 5,4 to 5,4,2 (kafka.cluster.Partition) >>> [2017-03-22 15:24:05,646] INFO Partition [topic1,0] on broker 5: Shrinking >>> ISR for partition [topic1,0] from 5,4,2 to 5,4 (kafka.cluster.Partition) >>> [2017-03-22 15:24:05,648] INFO Partition [topic1,0] on broker 5: Expanding >>> ISR for partition topic1-0 from 5,4 to 5,4,2 (kafka.cluster.Partition) >>> [2017-03-22 15:26:49,599] INFO Partition [topic1,0] on broker 5: Expanding >>> ISR for partition topic1-0 from 5,4,2 to 5,4,2,1 (kafka.cluster.Partition) >>> [2017-03-22 15:27:46,396] INFO Partition [topic1,0] on broker 5: Shrinking >>> ISR for partition [topic1,0] from 5,4,2,1 to 5,4,1 >>> (kafka.cluster.Partition) >>> [2017-03-22 15:27:46,398] INFO Partition [topic1,0] on broker 5: Expanding >>> ISR for partition topic1-0 from 5,4,1 to 5,4,1,2 (kafka.cluster.Partition) >>> [2017-03-22 15:45:47,896] INFO Partition [topic1,0] on broker 5: Shrinking >>> ISR for partition [topic1,0] from 5,4,1,2 to 5,1,2 >>> (kafka.cluster.Partition) >>> [2017-03-22 15:45:47,898] INFO Partition [topic1,0] on broker 5: Expanding >>> ISR for partition topic1-0 from 5,1,2 to 5,1,2,4 (kafka.cluster.Partition) >>> (END) >>> >>> I have tried increasing the num.network.threads (now 8) and >>> num.replica.fetchers (now 2) but nothing has changed. >>> >>> The kafka server config is: >>> >>> default.replication.factor=4 >>> log.retention.check.interval.ms >>> <http://log.retention.check.interval.ms/>=300000 >>> log.retention.hours=168 >>> log.roll.hours=24 >>> log.segment.bytes=104857600 >>> min.insync.replicas=2 >>> num.io.threads=8 >>> num.network.threads=15 >>> num.partitions=1 >>> num.recovery.threads.per.data.dir=1 >>> num.replica.fetchers=2 >>> offsets.topic.num.partitions=1 >>> offsets.topic.replication.factor=3 >>> replica.lag.time.max.ms <http://replica.lag.time.max.ms/>=500 >>> socket.receive.buffer.bytes=102400 >>> socket.request.max.bytes=104857600 >>> socket.send.buffer.bytes=102400 >>> unclean.leader.election.enable=false >>> zookeeper.connection.timeout.ms >>> <http://zookeeper.connection.timeout.ms/>=3000 >>> >>> Best regards, >>> Radu >>> > >