[ 
https://issues.apache.org/jira/browse/KAFKA-2729?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16203198#comment-16203198
 ] 

Francesco vigotti commented on KAFKA-2729:
------------------------------------------

I'm having the same issue and definitely losing trust in kafka, every 2 months 
there is something that force me to reset the whole cluster, I'm searching for 
a good alternative for a distributed-persisted-fast-queue for a while.. yet to 
find something that give me a good vibe.. 

anyway I'm facing this same issue with some small differences
- restarting all brokers ( together and rolling-restart ) didn't fix it..

all brokers in the cluster log such errors :
--- broker 5 

{code:java}

[2017-10-13 08:13:57,429] ERROR [ReplicaFetcherThread-0-2], Error for partition 
[__consumer_offsets,17] to broker 
2:org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is 
not the leader for that topic-partition. (kafka.server.ReplicaFetcherThread)
[2017-10-13 08:13:57,429] ERROR [ReplicaFetcherThread-0-2], Error for partition 
[__consumer_offsets,23] to broker 
2:org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is 
not the leader for that topic-partition. (kafka.server.ReplicaFetcherThread)
[2017-10-13 08:13:57,429] ERROR [ReplicaFetcherThread-0-2], Error for partition 
[__consumer_offsets,47] to broker 
2:org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is 
not the leader for that topic-partition. (kafka.server.ReplicaFetcherThread)
[2017-10-13 08:13:57,429] ERROR [ReplicaFetcherThread-0-2], Error for partition 
[__consumer_offsets,29] to broker 
2:org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is 
not the leader for that topic-partition. (kafka.server.ReplicaFetcherThread)

{code}

--- broker3

)
{code:java}

[2017-10-13 08:13:58,547] INFO Partition [__consumer_offsets,20] on broker 3: 
Expanding ISR for partition __consumer_offsets-20 from 3,2 to 3,2,5 
(kafka.cluster.Partition)
[2017-10-13 08:13:58,551] INFO Partition [__consumer_offsets,44] on broker 3: 
Expanding ISR for partition __consumer_offsets-44 from 3,2 to 3,2,5 
(kafka.cluster.Partition)
[2017-10-13 08:13:58,554] INFO Partition [__consumer_offsets,5] on broker 3: 
Expanding ISR for partition __consumer_offsets-5 from 2,3 to 2,3,5 
(kafka.cluster.Partition)
[2017-10-13 08:13:58,557] INFO Partition [__consumer_offsets,26] on broker 3: 
Expanding ISR for partition __consumer_offsets-26 from 3,2 to 3,2,5 
(kafka.cluster.Partition)
[2017-10-13 08:13:58,563] INFO Partition [__consumer_offsets,29] on broker 3: 
Expanding ISR for partition __consumer_offsets-29 from 2,3 to 2,3,5 
(kafka.cluster.Partition)
[2017-10-13 08:13:58,566] INFO Partition [__consumer_offsets,32] on broker 3: 
Expanding ISR for partition __consumer_offsets-32 from 3,2 to 3,2,5 
(kafka.cluster.Partition)
[2017-10-13 08:13:58,570] INFO Partition [legacyJavaVarT,2] on broker 3: 
Expanding ISR for partition legacyJavaVarT-2 from 3 to 3,5 
(kafka.cluster.Partition)
[2017-10-13 08:13:58,573] INFO Partition [test4,3] on broker 3: Expanding ISR 
for partition test4-3 from 2,3 to 2,3,5 (kafka.cluster.Partition)
[2017-10-13 08:13:58,577] INFO Partition [test4,0] on broker 3: Expanding ISR 
for partition test4-0 from 3,2 to 3,2,5 (kafka.cluster.Partition)
[2017-10-13 08:13:58,582] INFO Partition [test3,5] on broker 3: Expanding ISR 
for partition test3-5 from 3 to 3,5 (kafka.cluster.Partition)

{code}


--- broker2 

{code:java}

[2017-10-13 08:13:36,289] INFO Partition [__consumer_offsets,11] on broker 2: 
Expanding ISR for partition __consumer_offsets-11 from 2,5 to 2,5,3 
(kafka.cluster.Partition)
[2017-10-13 08:13:36,293] INFO Partition [__consumer_offsets,41] on broker 2: 
Expanding ISR for partition __consumer_offsets-41 from 2,5 to 2,5,3 
(kafka.cluster.Partition)
[2017-10-13 08:13:36,296] INFO Partition [test3,2] on broker 2: Expanding ISR 
for partition test3-2 from 2 to 2,3 (kafka.cluster.Partition)
[2017-10-13 08:13:36,300] INFO Partition [__consumer_offsets,23] on broker 2: 
Expanding ISR for partition __consumer_offsets-23 from 2,5 to 2,5,3 
(kafka.cluster.Partition)
[2017-10-13 08:13:36,304] INFO Partition [__consumer_offsets,5] on broker 2: 
Expanding ISR for partition __consumer_offsets-5 from 2,5 to 2,5,3 
(kafka.cluster.Partition)
[2017-10-13 08:13:36,337] INFO Partition [__consumer_offsets,35] on broker 2: 
Expanding ISR for partition __consumer_offsets-35 from 2,5 to 2,5,3 
(kafka.cluster.Partition)
[2017-10-13 08:13:36,372] INFO Partition [test_mainlog,24] on broker 2: 
Expanding ISR for partition test_mainlog-24 from 2 to 2,3 
(kafka.cluster.Partition)
[2017-10-13 08:13:36,375] INFO Partition [test_mainlog,6] on broker 2: 
Expanding ISR for partition test_mainlog-6 from 2 to 2,3 
(kafka.cluster.Partition)
[2017-10-13 08:13:36,379] INFO Partition [test_mainlog,18] on broker 2: 
Expanding ISR for partition test_mainlog-18 from 2 to 2,3 
(kafka.cluster.Partition)
[2017-10-13 08:13:36,384] INFO Partition [test_mainlog,0] on broker 2: 
Expanding ISR for partition test_mainlog-0 from 2 to 2,3 
(kafka.cluster.Partition)
[2017-10-13 08:13:36,388] INFO Partition [test_mainlog,12] on broker 2: 
Expanding ISR for partition test_mainlog-12 from 2 to 2,3 
(kafka.cluster.Partition)
[2017-10-13 08:13:40,367] INFO [ReplicaFetcherManager on broker 2] Removed 
fetcher for partitions __consumer_offsets-47 
(kafka.server.ReplicaFetcherManager)
[2017-10-13 08:13:40,367] INFO Truncating log __consumer_offsets-47 to offset 
0. (kafka.log.Log)
[2017-10-13 08:13:40,374] INFO [ReplicaFetcherThread-0-3], Starting  
(kafka.server.ReplicaFetcherThread)
[2017-10-13 08:13:40,374] INFO [ReplicaFetcherManager on broker 2] Added 
fetcher for partitions List([__consumer_offsets-47, initOffset 0 to broker 
BrokerEndPoint(3,--hidden----.73,9092)] ) (kafka.server.ReplicaFetcherManager)
[2017-10-13 08:13:40,376] ERROR [ReplicaFetcherThread-0-3], Error for partition 
[__consumer_offsets,47] to broker 
3:org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is 
not the leader for that topic-partition. (kafka.server.ReplicaFetcherThread)
[2017-10-13 08:13:40,393] INFO [ReplicaFetcherManager on broker 2] Removed 
fetcher for partitions __consumer_offsets-29 
(kafka.server.ReplicaFetcherManager)
[2017-10-13 08:13:40,393] INFO Truncating log __consumer_offsets-29 to offset 
0. (kafka.log.Log)
[2017-10-13 08:13:40,402] INFO [ReplicaFetcherManager on broker 2] Added 
fetcher for partitions List([__consumer_offsets-29, initOffset 0 to broker 
BrokerEndPoint(3,--hidden----.73,9092)] ) (kafka.server.ReplicaFetcherManager)
[2017-10-13 08:13:40,403] ERROR [ReplicaFetcherThread-0-3], Error for partition 
[__consumer_offsets,29] to broker 
3:org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is 
not the leader for that topic-partition. (kafka.server.ReplicaFetcherThread)
[2017-10-13 08:13:40,407] INFO [ReplicaFetcherManager on broker 2] Removed 
fetcher for partitions __consumer_offsets-41 
(kafka.server.ReplicaFetcherManager)
[2017-10-13 08:13:40,407] INFO Truncating log __consumer_offsets-41 to offset 
0. (kafka.log.Log)
[2017-10-13 08:13:40,413] INFO [ReplicaFetcherManager on broker 2] Added 
fetcher for partitions List([__consumer_offsets-41, initOffset 0 to broker 
BrokerEndPoint(3,--hidden----.73,9092)] ) (kafka.server.ReplicaFetcherManager)
[2017-10-13 08:13:40,414] ERROR [ReplicaFetcherThread-0-3], Error for partition 
[__consumer_offsets,41] to broker 
3:org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is 
not the leader for that topic-partition. (kafka.server.ReplicaFetcherThread)
[2017-10-13 08:13:40,419] INFO [ReplicaFetcherManager on broker 2] Removed 
fetcher for partitions test_mainlog-6 (kafka.server.ReplicaFetcherManager)
[2017-10-13 08:13:40,419] INFO Truncating log test_mainlog-6 to offset 
4997933406. (kafka.log.Log)
[2017-10-13 08:13:40,425] INFO [ReplicaFetcherManager on broker 2] Added 
fetcher for partitions List([test_mainlog-6, initOffset 4997933406 to broker 
BrokerEndPoint(3,--hidden----.73,9092)] ) (kafka.server.ReplicaFetcherManager)
[2017-10-13 08:13:40,432] INFO [ReplicaFetcherManager on broker 2] Removed 
fetcher for partitions __consumer_offsets-17 
(kafka.server.ReplicaFetcherManager)
[2017-10-13 08:13:40,432] INFO Truncating log __consumer_offsets-17 to offset 
0. (kafka.log.Log)
[2017-10-13 08:13:40,438] INFO [ReplicaFetcherManager on broker 2] Added 
fetcher for partitions List([__consumer_offsets-17, initOffset 0 to broker 
BrokerEndPoint(3,--hidden----.73,9092)] ) (kafka.server.ReplicaFetcherManager)
[2017-10-13 08:13:40,443] INFO [ReplicaFetcherManager on broker 2] Removed 
fetcher for partitions test_mainlog-0 (kafka.server.ReplicaFetcherManager)
[2017-10-13 08:13:40,443] INFO Truncating log test_mainlog-0 to offset 
5704085814. (kafka.log.Log)
[2017-10-13 08:13:40,449] INFO [ReplicaFetcherManager on broker 2] Added 
fetcher for partitions List([test_mainlog-0, initOffset 5704085814 to broker 
BrokerEndPoint(3,--hidden----.73,9092)] ) (kafka.server.ReplicaFetcherManager)
[2017-10-13 08:13:40,464] INFO [ReplicaFetcherManager on broker 2] Removed 
fetcher for partitions __consumer_offsets-14 
(kafka.server.ReplicaFetcherManager)
[2017-10-13 08:13:40,464] INFO Truncating log __consumer_offsets-14 to offset 
0. (kafka.log.Log)
[2017-10-13 08:13:40,472] INFO [ReplicaFetcherManager on broker 2] Added 
fetcher for partitions List([__consumer_offsets-14, initOffset 0 to broker 
BrokerEndPoint(3,--hidden----.73,9092)] ) (kafka.server.ReplicaFetcherManager)

{code}







those logs goes for hours and the cluster never recover, the only things that 
change something is when I repeatedly from zookeeper
delete /controller # repeatedly untill it get assigned to kafka3 node 

and at this point all errors stop ( no more error logs ) , kafka seems working, 
kafkamanager show offsets for all partitions ( while some offset was missing ) 
, data ingestion /consumption works , the only things that presages something 
wrong is that on one topic with 30 partitions and replication 2 there is 1 
broker skew ( 1 broker have 1 partitions more than normal and one broker have 1 
partition less than normal )
and the situation remain stable with this small anomaly for hours..  nodes 
delete indexes, delete segments , roll new segments.. 

If i now delete the controller again, or restart the kafka3-node evreything 
goes to the previous situation again ( all errors logged ) and at this point I 
don't even know how to recover , the only "fix" I'm left to try is to wipe the 
whole cluster data and restart  :( but what to do then if this happens again in 
future ?

I don't know why two nodes seems to have a ("broken controller" ??) and the 
cluster remain in this in-consistent state forever.. 
If you have any suggestion... on what to inspect / how to try to fix , those 
are very welcomed..

Thank you,
Francesco



> Cached zkVersion not equal to that in zookeeper, broker not recovering.
> -----------------------------------------------------------------------
>
>                 Key: KAFKA-2729
>                 URL: https://issues.apache.org/jira/browse/KAFKA-2729
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 0.8.2.1, 0.9.0.0, 0.10.0.0, 0.10.1.0, 0.11.0.0
>            Reporter: Danil Serdyuchenko
>
> After a small network wobble where zookeeper nodes couldn't reach each other, 
> we started seeing a large number of undereplicated partitions. The zookeeper 
> cluster recovered, however we continued to see a large number of 
> undereplicated partitions. Two brokers in the kafka cluster were showing this 
> in the logs:
> {code}
> [2015-10-27 11:36:00,888] INFO Partition 
> [__samza_checkpoint_event-creation_1,3] on broker 5: Shrinking ISR for 
> partition [__samza_checkpoint_event-creation_1,3] from 6,5 to 5 
> (kafka.cluster.Partition)
> [2015-10-27 11:36:00,891] INFO Partition 
> [__samza_checkpoint_event-creation_1,3] on broker 5: Cached zkVersion [66] 
> not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
> {code}
> For all of the topics on the effected brokers. Both brokers only recovered 
> after a restart. Our own investigation yielded nothing, I was hoping you 
> could shed some light on this issue. Possibly if it's related to: 
> https://issues.apache.org/jira/browse/KAFKA-1382 , however we're using 
> 0.8.2.1.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to