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

Jeff Widman commented on KAFKA-3042:
------------------------------------

[~lindong] thanks for the offer to help and sorry for the slow response. 

I'm not exactly sure how to repro, but below I copied a sanitized version of 
our internal wiki page documenting our findings as we tried to figure out what 
was happening and how we got into the state of mis-matched controller epoch for 
controller vs random partition. It's not the most polished, more of a train of 
thought put to paper as we debugged.

Reading through it, it appeared that broker 3 lost connection to zookeeper, 
then when it came back, it elected itself controller, but somehow ended up in a 
state where the broker 3 controller had a list of brokers that was completely 
empty. This doesn't make logical sense because if a broker is controller, then 
it should list itself in active brokers. But somehow it happened. Then 
following that, the active epoch for the controller is 134, but the active 
epoch listed by a random partition in zookeeper is 133. So that created the 
version mismatch. 

More details below, and I also have access to the detailed Kafka logs (but not 
ZK logs) beyond just the snippets if you need anything else. They will get 
rotated out of elasticsearch within a few months and disappear, so hopefully we 
can get to the bottom of this before that.


{code}
3 node cluster. 
Broker 1 is controller.
Zookeeper GC pause meant that broker 3 lost connection. 
When it came back, broker 3 thought it was controller, but thought there were 
no alive brokers--see the empty set referenced in the logs below. This alone 
seems incorrect because if a broker is a controller, you'd think it would 
include itself in the set.


See the following in the logs:


[2017-03-17 21:32:15,812] ERROR Controller 3 epoch 134 initiated state change 
for partition [topic_name,626] from OfflinePartition to OnlinePartition failed 
(s
tate.change.logger)
kafka.common.NoReplicaOnlineException: No replica for partition 
[topic_name,626] is alive. Live brokers are: [Set()], Assigned replicas are: 
[List(1, 3)]
at 
kafka.controller.OfflinePartitionLeaderSelector.selectLeader(PartitionLeaderSelector.scala:75)
at 
kafka.controller.PartitionStateMachine.electLeaderForPartition(PartitionStateMachine.scala:345)
at 
kafka.controller.PartitionStateMachine.kafka$controller$PartitionStateMachine$$handleStateChange(PartitionStateMachine.scala:205)
at 
kafka.controller.PartitionStateMachine$$anonfun$triggerOnlinePartitionStateChange$3.apply(PartitionStateMachine.scala:120)
at 
kafka.controller.PartitionStateMachine$$anonfun$triggerOnlinePartitionStateChange$3.apply(PartitionStateMachine.scala:117)
at 
scala.collection.TraversableLike$WithFilter$$anonfun$foreach$1.apply(TraversableLike.scala:772)
at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98)
at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98)
at scala.collection.mutable.HashTable$class.foreachEntry(HashTable.scala:226)
at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:39)
at scala.collection.mutable.HashMap.foreach(HashMap.scala:98)
at 
scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:771)
at 
kafka.controller.PartitionStateMachine.triggerOnlinePartitionStateChange(PartitionStateMachine.scala:117)
at 
kafka.controller.PartitionStateMachine.startup(PartitionStateMachine.scala:70)
at 
kafka.controller.KafkaController.onControllerFailover(KafkaController.scala:335)
at 
kafka.controller.KafkaController$$anonfun$1.apply$mcV$sp(KafkaController.scala:166)
at kafka.server.ZookeeperLeaderElector.elect(ZookeeperLeaderElector.scala:84)
at 
kafka.server.ZookeeperLeaderElector$LeaderChangeListener$$anonfun$handleDataDeleted$1.apply$mcZ$sp(ZookeeperLeaderElector.scala:146)
at 
kafka.server.ZookeeperLeaderElector$LeaderChangeListener$$anonfun$handleDataDeleted$1.apply(ZookeeperLeaderElector.scala:141)
at 
kafka.server.ZookeeperLeaderElector$LeaderChangeListener$$anonfun$handleDataDeleted$1.apply(ZookeeperLeaderElector.scala:141)
at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:231)
at 
kafka.server.ZookeeperLeaderElector$LeaderChangeListener.handleDataDeleted(ZookeeperLeaderElector.scala:141)
at org.I0Itec.zkclient.ZkClient$9.run(ZkClient.java:824)
at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71)

Looking at the code + error message, the controller is unaware of active 
brokers. However, there are assigned replicas. We checked the log files under 
/data/kafka and they had m_times greater than the exception timestamp, plus our 
producers and consumers seemed to be working, so the cluster is successfully 
passing data around. The controller situation is just screwed up.


    [2017-03-17 21:32:43,976] ERROR Controller 3 epoch 134 initiated state 
change for partition [app_summary_86400,660] from OfflinePartition to 
OnlinePartition failed (state.change.logger)

    kafka.common.NoReplicaOnlineException: No replica for partition 
[app_summary,660] is alive. Live brokers are: [Set()], Assigned replicas are: 
[List(2, 3)]

            at 
kafka.controller.OfflinePartitionLeaderSelector.selectLeader(PartitionLeaderSelector.scala:75)
    Next log line:

    [2017-03-17 21:32:44,033] INFO [Partition state machine on Controller 3]: 
Started partition state machine with initial state -> Map([topic,34] -> 
OfflinePartition, [topic1,0] -> OfflinePartition, [topic2,790] -> 
OfflinePartition, [topic3,20] -> OfflinePartition, etc.

    [2017-03-17 21:32:44,172] INFO [Controller 3]: Broker 3 is ready to serve 
as the new controller with epoch 134 (kafka.controller.KafkaController)

    [2017-03-17 21:32:44,175] INFO [Controller 3]: Starting preferred replica 
leader election for partitions  (kafka.controller.KafkaController)

    [2017-03-17 21:32:44,176] INFO [Partition state machine on Controller 3]: 
Invoking state change to OnlinePartition for partitions  
(kafka.controller.PartitionStateMachine)

    [2017-03-17 21:32:44,241] INFO [Controller 3]: starting the partition 
rebalance scheduler (kafka.controller.KafkaController)

    [2017-03-17 21:32:44,248] INFO [delete-topics-thread-3], Starting  
(kafka.controller.TopicDeletionManager$DeleteTopicsThread)

    [2017-03-17 21:32:44,255] INFO [SessionExpirationListener on 3], ZK 
expired; shut down all controller components and try to re-elect 
(kafka.controller.KafkaController$SessionExpirationListener)

    [2017-03-17 21:32:44,264] INFO [delete-topics-thread-3], Shutting down 
(kafka.controller.TopicDeletionManager$DeleteTopicsThread)

    [2017-03-17 21:32:44,269] INFO [delete-topics-thread-3], Stopped  
(kafka.controller.TopicDeletionManager$DeleteTopicsThread)

    [2017-03-17 21:32:44,269] INFO [delete-topics-thread-3], Shutdown completed 
(kafka.controller.TopicDeletionManager$DeleteTopicsThread)

    [2017-03-17 21:32:44,276] INFO [Partition state machine on Controller 3]: 
Stopped partition state machine (kafka.controller.PartitionStateMachine)

    [2017-03-17 21:32:44,276] INFO [Replica state machine on controller 3]: 
Stopped replica state machine (kafka.controller.ReplicaStateMachine)

    [2017-03-17 21:32:44,277] INFO [Controller 3]: Broker 3 resigned as the 
controller (kafka.controller.KafkaController)

    [2017-03-17 21:32:44,302] INFO re-registering broker info in ZK for broker 
3 (kafka.server.KafkaHealthcheck$SessionExpireListener)

    [2017-03-17 21:32:44,302] INFO Creating /brokers/ids/3 (is it secure? 
false) (kafka.utils.ZKCheckedEphemeral)

    [2017-03-17 21:32:44,306] INFO Result of znode creation is: OK 
(kafka.utils.ZKCheckedEphemeral)

    [2017-03-17 21:32:44,306] INFO Registered broker 3 at path /brokers/ids/3 
with addresses: PLAINTEXT -> EndPoint(kafka03.domain.net,9092,PLAINTEXT) 
(kafka.utils.ZkUtils)

    [2017-03-17 21:32:44,310] INFO done re-registering broker 
(kafka.server.KafkaHealthcheck$SessionExpireListener)

    [2017-03-17 21:32:44,311] INFO Subscribing to /brokers/topics path to watch 
for new topics (kafka.server.KafkaHealthcheck$SessionExpireListener)

    [2017-03-17 21:32:44,374] INFO New leader is 3 
(kafka.server.ZookeeperLeaderElector$LeaderChangeListener)

    [2017-03-17 21:32:44,380] INFO New leader is 3 
(kafka.server.ZookeeperLeaderElector$LeaderChangeListener)

    [2017-03-17 21:32:44,389] INFO [BrokerChangeListener on Controller 3]: 
Broker change listener fired for path /brokers/ids with children 2,3 
(kafka.controller.ReplicaStateMachine$BrokerChangeListener)


New thought: Since the controller (broker 3) created a new "partition state 
machine" with a load of partitions set to OffLine, I may be able to check if 
these partitions are being replicated.
The purpose of this persistence to validate that partitions are being 
replicated is to answer the question whether we have simply a broken controller 
or a situation where there are indeed zero ISR's for all partitions.


    Choosing partition topic_name-102 from this data: OfflinePartition, 
[topic_name,102] 

    /opt/ns/kafka/bin/kafka-topics.sh --describe --zookeeper zookeeper02:2181 | 
grep topic_name


    Topic: topic_name Partition: 102 Leader: -1 Replicas: 1,2 Isr: 

    Replicas or leaders on brokers 1 and 2.
    Broker 1 

    root@kafka01:/data/kafka/topic_name-102# ls -l

    total 100824

    -rw-r--r-- 1 user group  10485760 Mar 22 18:53 00000000000008485179.index

    -rw-r--r-- 1 user group 103046463 Mar 22 18:53 00000000000008485179.log

    Broker 2

    root@kafka02:/data/kafka/topic_name-102# ls -l

    total 100632

    -rw-r--r-- 1 user group  10485760 Mar 22 18:53 00000000000008485690.index

    -rw-r--r-- 1 user group 102849910 Mar 22 18:54 00000000000008485690.log


Conslusion: Replicas are working correctly. Hence, the following log entry 
suggests broker 3 has an inaccurate set of brokers as Set() is empty: 
kafka.common.NoReplicaOnlineException: No replica for partition 
[topic_name,660] is alive. Live brokers are: [Set()], Assigned replicas are: 
[List(2, 3)]

So what to do?

    We have a cluster that is working and data is being replicated as expected. 
    The controller appears to be operating independently and it may not even be 
aware of itself if the set should include itself. 

New data:

Broker 2:

[2017-03-17 21:31:11,385] INFO Unable to read additional data from server 
sessionid 0x3583df6c8b3e4c2, likely server has closed socket, closing socket 
connection and attempting reconnect (org.apache.zookeeper.ClientCnxn)

[2017-03-17 21:31:11,485] INFO Waiting for keeper state SyncConnected 
(org.I0Itec.zkclient.ZkClient)

[2017-03-17 21:31:11,485] INFO Waiting for keeper state SyncConnected 
(org.I0Itec.zkclient.ZkClient)

[2017-03-17 21:31:11,485] INFO Waiting for keeper state SyncConnected 
(org.I0Itec.zkclient.ZkClient)

[2017-03-17 21:31:11,485] INFO zookeeper state changed (Disconnected) 
(org.I0Itec.zkclient.ZkClient)

[2017-03-17 21:31:11,485] INFO Waiting for keeper state SyncConnected 
(org.I0Itec.zkclient.ZkClient)

[2017-03-17 21:31:11,485] INFO Waiting for keeper state SyncConnected 
(org.I0Itec.zkclient.ZkClient)

[2017-03-17 21:31:11,485] INFO Waiting for keeper state SyncConnected 
(org.I0Itec.zkclient.ZkClient)

[2017-03-17 21:31:11,485] INFO Waiting for keeper state SyncConnected 
(org.I0Itec.zkclient.ZkClient)

[2017-03-17 21:31:11,485] INFO Waiting for keeper state SyncConnected 
(org.I0Itec.zkclient.ZkClient)

[2017-03-17 21:31:11,485] INFO Waiting for keeper state SyncConnected 
(org.I0Itec.zkclient.ZkClient)

[2017-03-17 21:31:12,179] INFO Opening socket connection to server 
zookeeper01.domain.net/172.16.192.38:2181. Will not attempt to authenticate 
using SASL (unknown error) (org.apache.zookeeper.ClientCnxn)

[2017-03-17 21:31:12,180] INFO Socket connection established to 
zookeeper01.domain.net/172.16.192.38:2181, initiating session 
(org.apache.zookeeper.ClientCnxn)

[2017-03-17 21:31:12,182] INFO zookeeper state changed (Expired) 
(org.I0Itec.zkclient.ZkClient)

[2017-03-17 21:31:12,182] INFO Unable to reconnect to ZooKeeper service, 
session 0x3583df6c8b3e4c2 has expired, closing socket connection 
(org.apache.zookeeper.ClientCnxn)

[2017-03-17 21:31:12,192] INFO Initiating client connection, 
connectString=zookeeper01:2181,zookeeper02:2181,zookeeper03:2181 
sessionTimeout=6000 watcher=org.I0Itec.zkclient.ZkClient@1e6061e2 
(org.apache.zookeeper.ZooKeeper)

[2017-03-17 21:31:12,212] INFO EventThread shut down 
(org.apache.zookeeper.ClientCnxn)

[2017-03-17 21:31:12,212] INFO Opening socket connection to server 
zookeeper02.domain.net/172.16.192.39:2181. Will not attempt to authenticate 
using SASL (unknown error) (org.apache.zookeeper.ClientCnxn)

[2017-03-17 21:31:12,216] INFO Socket connection established to 
zookeeper02.domain.net/172.16.192.39:2181, initiating session 
(org.apache.zookeeper.ClientCnxn)

[2017-03-17 21:31:12,223] INFO Session establishment complete on server 
zookeeper02.domain.net/172.16.192.39:2181, sessionid = 0x25ac3ba5a354b72, 
negotiated timeout = 60000 (org.apache.zookeeper.ClientCnxn)

[2017-03-17 21:31:12,223] INFO zookeeper state changed (SyncConnected) 
(org.I0Itec.zkclient.ZkClient)

[2017-03-17 21:31:12,283] INFO Waiting for keeper state SyncConnected 
(org.I0Itec.zkclient.ZkClient)

[2017-03-17 21:31:12,283] INFO Waiting for keeper state SyncConnected 
(org.I0Itec.zkclient.ZkClient)

[2017-03-17 21:31:12,346] INFO New leader is 3 
(kafka.server.ZookeeperLeaderElector$LeaderChangeListener)

[2017-03-17 21:31:12,379] INFO [SessionExpirationListener on 2], ZK expired; 
shut down all controller components and try to re-elect 
(kafka.controller.KafkaController$SessionExpirationListener)

[2017-03-17 21:31:12,517] INFO [Partition state machine on Controller 2]: 
Stopped partition state machine (kafka.controller.PartitionStateMachine)

[2017-03-17 21:31:12,523] INFO [Replica state machine on controller 2]: Stopped 
replica state machine (kafka.controller.ReplicaStateMachine)

[2017-03-17 21:31:12,527] INFO [Controller 2]: Broker 2 resigned as the 
controller (kafka.controller.KafkaController)

[2017-03-17 21:31:12,593] INFO re-registering broker info in ZK for broker 2 
(kafka.server.KafkaHealthcheck$SessionExpireListener)

[2017-03-17 21:31:12,675] INFO Creating /brokers/ids/2 (is it secure? false) 
(kafka.utils.ZKCheckedEphemeral)

[2017-03-17 21:31:12,687] INFO Result of znode creation is: OK 
(kafka.utils.ZKCheckedEphemeral)

[2017-03-17 21:31:12,687] INFO Registered broker 2 at path /brokers/ids/2 with 
addresses: PLAINTEXT -> EndPoint(kafka02.domain.net,9092,PLAINTEXT) 
(kafka.utils.ZkUtils)

[2017-03-17 21:31:12,692] INFO done re-registering broker 
(kafka.server.KafkaHealthcheck$SessionExpireListener)

[2017-03-17 21:31:12,693] INFO Subscribing to /brokers/topics path to watch for 
new topics (kafka.server.KafkaHealthcheck$SessionExpireListener)

[2017-03-17 21:31:12,720] INFO New leader is 3 
(kafka.server.ZookeeperLeaderElector$LeaderChangeListener)


Broker 1 around the same timeframe: As fate would have it, the server.log file 
starts after the transition period, but there appears to be controller activity:


[2017-03-17 21:32:40,128] INFO [Controller 1]: New leader and ISR for partition 
[topic_names,236] is {"leader":2,"leader_epoch":1,"isr":[2]} 
(kafka.controller.KafkaController)

[2017-03-17 21:32:40,131] INFO [Controller 1]: New leader and ISR for partition 
[topic_names,62] is {"leader":-1,"leader_epoch":5,"isr":[2]} 
(kafka.controller.KafkaController)

[2017-03-17 21:32:40,134] INFO [Controller 1]: New leader and ISR for partition 
[topic_names,226] is {"leader":3,"leader_epoch":2,"isr":[3]} 
(kafka.controller.KafkaController)


Alas, notice "Controller 1"


Flurry of log entries:

[2017-03-17 21:34:02,233] WARN [Channel manager on controller 1]: Not sending 
request 
{controller_id=1,controller_epoch=133,delete_partitions=0,partitions=[{topic=topic_name,
 partition=127}]} to broker 1, since it is offline. 
(kafka.controller.ControllerChannelManager)


Of interest is the timestamp and controller_epoch=133. During this timeframe, 
the controller_epoch was changed to 134.

Now this:

[2017-03-17 21:34:02,460] INFO New leader is 3 
(kafka.server.ZookeeperLeaderElector$LeaderChangeListener)

So, broker 1 finally figured it out.

Possibly no concern.


Timeframe analysis (assumes hosts are time synced)


2: [2017-03-17 21:31:12,346] INFO New leader is 3 
(kafka.server.ZookeeperLeaderElector$LeaderChangeListener)

zk_controller node /controller mtime = Fri Mar 17 21:32:19 UTC 2017 (may have 
been modified subsequently)

3: [2017-03-17 21:32:44,033] INFO [Partition state machine on Controller 3]: 
Started partition state machine with initial state

1: [2017-03-17 21:34:02,460] INFO New leader is 3 
(kafka.server.ZookeeperLeaderElector$LeaderChangeListener)


Need to implicitly determine time broker 3 became controller through zookeeper:

[zk: localhost:2181(CONNECTED) 12] get /controller

{"version":1,"brokerid":3,"timestamp":"1489786272094"}

cZxid = 0x6b6081705b

ctime = Fri Mar 17 21:32:19 UTC 2017

mZxid = 0x6b6081705b

mtime = Fri Mar 17 21:32:19 UTC 2017

pZxid = 0x6b6081705b

cversion = 0

dataVersion = 0

aclVersion = 0

ephemeralOwner = 0x35ac3ba55a94cfa

dataLength = 54

numChildren = 0

[zk: localhost:2181(CONNECTED) 13] get /controller_epoch

134

cZxid = 0x100000012

ctime = Tue Oct 28 10:52:48 UTC 2014

mZxid = 0x6b60817117

mtime = Fri Mar 17 21:32:19 UTC 2017

pZxid = 0x100000012

cversion = 0

dataVersion = 133

aclVersion = 0

ephemeralOwner = 0x0

dataLength = 3

numChildren = 0


_________________________________________________________________

I believe the zookeeper leader had a GC Pause event. Notice the time gap in the 
log. Not logging during this time is not itself a problem, but an indicator 
that would halp explain why all of a sudden most or all of the sessions were 
expired because of a timeout.

Furthermore, we noted earlier that zookeeper was maxed out for its heap (and 
still is):

PID USER      PR  NI  VIRT  RES  SHR S %CPU %MEM    TIME+  COMMAND

12319 nsadmin   20   0 7186m 4.1g 2520 S   20 52.8 155799:14 java  


4.1g Resident. 


2017-03-17 21:31:05,623 [myid:3] - INFO  
[NIOServerCxn.Factory:0.0.0.0/0.0.0.0:2181:NIOServerCnxn@1007] - Closed socket 
connection for client /172.16.128.96:42435 (no session established for client)


2017-03-17 21:32:16,362 [myid:3] - INFO  [SessionTracker:ZooKeeperServer@347] - 
Expiring session 0x25ac3ba5a354a29, timeout of 60000ms exceeded

2017-03-17 21:32:16,841 [myid:3] - INFO  [SessionTracker:ZooKeeperServer@347] - 
Expiring session 0x359e383d1a0601e, timeout of 60000ms exceeded

2017-03-17 21:32:16,850 [myid:3] - INFO  [SessionTracker:ZooKeeperServer@347] - 
Expiring session 0x15a2dead2a01687, timeout of 60000ms exceeded

2017-03-17 21:32:16,858 [myid:3] - INFO  [SessionTracker:ZooKeeperServer@347] - 
Expiring session 0x25ac3ba5a351b9e, timeout of 60000ms exceeded

2017-03-17 21:32:16,858 [myid:3] - INFO  [SessionTracker:ZooKeeperServer@347] - 
Expiring session 0x15a79dd534fa28c, timeout of 60000ms exceeded

2017-03-17 21:32:16,862 [myid:3] - INFO  [SessionTracker:ZooKeeperServer@347] - 
Expiring session 0x359e383d1a06034, timeout of 60000ms exceeded

2017-03-17 21:32:16,871 [myid:3] - INFO  [SessionTracker:ZooKeeperServer@347] - 
Expiring session 0x25ac3ba5a3506ee, timeout of 60000ms exceeded

2017-03-17 21:32:16,871 [myid:3] - INFO  [SessionTracker:ZooKeeperServer@347] - 
Expiring session 0x25a79dd53c5b252, timeout of 60000ms exceeded

2017-03-17 21:32:16,886 [myid:3] - INFO  [SessionTracker:ZooKeeperServer@347] - 
Expiring session 0x35a8fa582a16ce9, timeout of 60000ms exceeded

2017-03-17 21:32:16,895 [myid:3] - INFO  [SessionTracker:ZooKeeperServer@347] - 
Expiring session 0x15a79dd534fa297, timeout of 60000ms exceeded

2017-03-17 21:32:16,901 [myid:3] - INFO  [SessionTracker:ZooKeeperServer@347] - 
Expiring session 0x35a8fa582a157d5, timeout of 60000ms exceeded

2017-03-17 21:32:16,905 [myid:3] - INFO  [SessionTracker:ZooKeeperServer@347] - 
Expiring session 0x35a8fa582a17c40, timeout of 60000ms exceeded

2017-03-17 21:32:16,911 [myid:3] - INFO  [SessionTracker:ZooKeeperServer@347] - 
Expiring session 0x359e383d1a05fb2, timeout of 60000ms exceeded

2017-03-17 21:32:16,911 [myid:3] - INFO  [SessionTracker:ZooKeeperServer@347] - 
Expiring session 0x35a2dead1961653, timeout of 60000ms exceeded

2017-03-17 21:32:16,913 [myid:3] - INFO  [SessionTracker:ZooKeeperServer@347] - 
Expiring session 0x259e383d4e15ccc, timeout of 60000ms exceeded

2017-03-17 21:32:16,914 [myid:3] - INFO  [SessionTracker:ZooKeeperServer@347] - 
Expiring session 0x25a79dd53c58eda, timeout of 60000ms exceeded

2017-03-17 21:32:16,921 [myid:3] - INFO  [SessionTracker:ZooKeeperServer@347] - 
Expiring session 0x15a79dd534fa27d, timeout of 60000ms exceeded

2017-03-17 21:32:16,922 [myid:3] - INFO  [SessionTracker:ZooKeeperServer@347] - 
Expiring session 0x15a79dd534fa27c, timeout of 60000ms exceeded

2017-03-17 21:32:16,931 [myid:3] - INFO  [SessionTracker:ZooKeeperServer@347] - 
Expiring session 0x259e383d4e15cef, timeout of 60000ms exceeded

2017-03-17 21:32:16,931 [myid:3] - INFO  [SessionTracker:ZooKeeperServer@347] - 
Expiring session 0x25a79dd53c58f14, timeout of 60000ms exceeded


    Let's look to see if the election for the brokers happened during this 
period.

Time period under analysis: 

2017-03-17 21:31:05,62 to

2017-03-17 21:32:16,362

69.742 seconds


Notes:

Check ZK status:
zookeeper                        RUNNING    pid 12319, uptime 213 days, 16:58:06

    Zookeeper03 has been running for 213 days.


Broker 2 log during GC pause on zookeeper03

[2017-03-17 21:31:11,385] INFO Unable to read additional data from server 
sessionid 0x3583df6c8b3e4c2, likely server has closed socket, closing socket 
connection and attempting reconnect (org.apache.zookeeper.ClientCnxn)

[2017-03-17 21:31:11,485] INFO Waiting for keeper state SyncConnected 
(org.I0Itec.zkclient.ZkClient)

[2017-03-17 21:31:11,485] INFO Waiting for keeper state SyncConnected 
(org.I0Itec.zkclient.ZkClient)

[2017-03-17 21:31:11,485] INFO Waiting for keeper state SyncConnected 
(org.I0Itec.zkclient.ZkClient)

[2017-03-17 21:31:11,485] INFO zookeeper state changed (Disconnected) 
(org.I0Itec.zkclient.ZkClient)

[2017-03-17 21:31:11,485] INFO Waiting for keeper state SyncConnected 
(org.I0Itec.zkclient.ZkClient)

[2017-03-17 21:31:11,485] INFO Waiting for keeper state SyncConnected 
(org.I0Itec.zkclient.ZkClient)

[2017-03-17 21:31:11,485] INFO Waiting for keeper state SyncConnected 
(org.I0Itec.zkclient.ZkClient)

[2017-03-17 21:31:11,485] INFO Waiting for keeper state SyncConnected 
(org.I0Itec.zkclient.ZkClient)

[2017-03-17 21:31:11,485] INFO Waiting for keeper state SyncConnected 
(org.I0Itec.zkclient.ZkClient)

[2017-03-17 21:31:11,485] INFO Waiting for keeper state SyncConnected 
(org.I0Itec.zkclient.ZkClient)

[2017-03-17 21:31:12,179] INFO Opening socket connection to server 
zookeeper01.domain.net/172.16.192.38:2181. Will not attempt to authenticate 
using SASL (unknown error) (org.apache.zookeeper.ClientCnxn)

[2017-03-17 21:31:12,180] INFO Socket connection established to 
zookeeper01.domain.net/172.16.192.38:2181, initiating session 
(org.apache.zookeeper.ClientCnxn)

[2017-03-17 21:31:12,182] INFO zookeeper state changed (Expired) 
(org.I0Itec.zkclient.ZkClient)

[2017-03-17 21:31:12,182] INFO Unable to reconnect to ZooKeeper service, 
session 0x3583df6c8b3e4c2 has expired, closing socket connection 
(org.apache.zookeeper.ClientCnxn)

[2017-03-17 21:31:12,192] INFO Initiating client connection, 
connectString=zookeeper01:2181,zookeeper02:2181,zookeeper03:2181 
sessionTimeout=6000 watcher=org.I0Itec.zkclient.ZkClient@1e6061e2 
(org.apache.zookeeper.ZooKeeper)

[2017-03-17 21:31:12,212] INFO EventThread shut down 
(org.apache.zookeeper.ClientCnxn)

[2017-03-17 21:31:12,212] INFO Opening socket connection to server 
zookeeper02.domain.net/172.16.192.39:2181. Will not attempt to authenticate 
using SASL (unknown error) (org.apache.zookeeper.ClientCnxn)

[2017-03-17 21:31:12,216] INFO Socket connection established to 
zookeeper02.domain.net/172.16.192.39:2181, initiating session 
(org.apache.zookeeper.ClientCnxn)

[2017-03-17 21:31:12,223] INFO Session establishment complete on server 
zookeeper02.domain.net/172.16.192.39:2181, sessionid = 0x25ac3ba5a354b72, 
negotiated timeout = 60000 (org.apache.zookeeper.ClientCnxn)

[2017-03-17 21:31:12,223] INFO zookeeper state changed (SyncConnected) 
(org.I0Itec.zkclient.ZkClient)

[2017-03-17 21:31:12,283] INFO Waiting for keeper state SyncConnected 
(org.I0Itec.zkclient.ZkClient)

[2017-03-17 21:31:12,283] INFO Waiting for keeper state SyncConnected 
(org.I0Itec.zkclient.ZkClient)

[2017-03-17 21:31:12,346] INFO New leader is 3 
(kafka.server.ZookeeperLeaderElector$LeaderChangeListener)

[2017-03-17 21:31:12,379] INFO [SessionExpirationListener on 2], ZK expired; 
shut down all controller components and try to re-elect 
(kafka.controller.KafkaController$SessionExpirationListener)

[2017-03-17 21:31:12,517] INFO [Partition state machine on Controller 2]: 
Stopped partition state machine (kafka.controller.PartitionStateMachine)

[2017-03-17 21:31:12,523] INFO [Replica state machine on controller 2]: Stopped 
replica state machine (kafka.controller.ReplicaStateMachine)

[2017-03-17 21:31:12,527] INFO [Controller 2]: Broker 2 resigned as the 
controller (kafka.controller.KafkaController)

[2017-03-17 21:31:12,593] INFO re-registering broker info in ZK for broker 2 
(kafka.server.KafkaHealthcheck$SessionExpireListener)

[2017-03-17 21:31:12,675] INFO Creating /brokers/ids/2 (is it secure? false) 
(kafka.utils.ZKCheckedEphemeral)

[2017-03-17 21:31:12,687] INFO Result of znode creation is: OK 
(kafka.utils.ZKCheckedEphemeral)

[2017-03-17 21:31:12,687] INFO Registered broker 2 at path /brokers/ids/2 with 
addresses: PLAINTEXT -> EndPoint(kafka02.domain.net,9092,PLAINTEXT) 
(kafka.utils.ZkUtils)

[2017-03-17 21:31:12,692] INFO done re-registering broker 
(kafka.server.KafkaHealthcheck$SessionExpireListener)

[2017-03-17 21:31:12,693] INFO Subscribing to /brokers/topics path to watch for 
new topics (kafka.server.KafkaHealthcheck$SessionExpireListener)

[2017-03-17 21:31:12,720] INFO New leader is 3 
(kafka.server.ZookeeperLeaderElector$LeaderChangeListener)

[2017-03-17 21:31:46,284] INFO Partition [topic,326] on broker 2: Shrinking ISR 
for partition [topic,326] from 2,3 to 2 (kafka.cluster.Partition)

[2017-03-17 21:31:53,562] INFO Partition [topic,326] on broker 2: Expanding ISR 
for partition [topic,326] from 2 to 2,3 (kafka.cluster.Partition)


Broker 3: After the event window: 

[2017-03-17 21:32:44,172] INFO [Controller 3]: Broker 3 is ready to serve as 
the new controller with epoch 134 (kafka.controller.KafkaController)

[2017-03-17 21:32:44,175] INFO [Controller 3]: Starting preferred replica 
leader election for partitions  (kafka.controller.KafkaController)

[2017-03-17 21:32:44,176] INFO [Partition state machine on Controller 3]: 
Invoking state change to OnlinePartition for partitions  
(kafka.controller.PartitionStateMachine)

[2017-03-17 21:32:44,241] INFO [Controller 3]: starting the partition rebalance 
scheduler (kafka.controller.KafkaController)

[2017-03-17 21:32:44,248] INFO [delete-topics-thread-3], Starting  
(kafka.controller.TopicDeletionManager$DeleteTopicsThread)

[2017-03-17 21:32:44,255] INFO [SessionExpirationListener on 3], ZK expired; 
shut down all controller components and try to re-elect 
(kafka.controller.KafkaController$SessionExpirationListener)

[2017-03-17 21:32:44,264] INFO [delete-topics-thread-3], Shutting down 
(kafka.controller.TopicDeletionManager$DeleteTopicsThread)

[2017-03-17 21:32:44,269] INFO [delete-topics-thread-3], Stopped  
(kafka.controller.TopicDeletionManager$DeleteTopicsThread)

[2017-03-17 21:32:44,269] INFO [delete-topics-thread-3], Shutdown completed 
(kafka.controller.TopicDeletionManager$DeleteTopicsThread)

[2017-03-17 21:32:44,276] INFO [Partition state machine on Controller 3]: 
Stopped partition state machine (kafka.controller.PartitionStateMachine)

[2017-03-17 21:32:44,276] INFO [Replica state machine on controller 3]: Stopped 
replica state machine (kafka.controller.ReplicaStateMachine)

[2017-03-17 21:32:44,277] INFO [Controller 3]: Broker 3 resigned as the 
controller (kafka.controller.KafkaController)

[2017-03-17 21:32:44,302] INFO re-registering broker info in ZK for broker 3 
(kafka.server.KafkaHealthcheck$SessionExpireListener)

[2017-03-17 21:32:44,302] INFO Creating /brokers/ids/3 (is it secure? false) 
(kafka.utils.ZKCheckedEphemeral)

[2017-03-17 21:32:44,306] INFO Result of znode creation is: OK 
(kafka.utils.ZKCheckedEphemeral)

[2017-03-17 21:32:44,306] INFO Registered broker 3 at path /brokers/ids/3 with 
addresses: PLAINTEXT -> EndPoint(kafka03.domain.net,9092,PLAINTEXT) 
(kafka.utils.ZkUtils)

[2017-03-17 21:32:44,310] INFO done re-registering broker 
(kafka.server.KafkaHealthcheck$SessionExpireListener)

[2017-03-17 21:32:44,311] INFO Subscribing to /brokers/topics path to watch for 
new topics (kafka.server.KafkaHealthcheck$SessionExpireListener)

[2017-03-17 21:32:44,374] INFO New leader is 3 
(kafka.server.ZookeeperLeaderElector$LeaderChangeListener)

[2017-03-17 21:32:44,380] INFO New leader is 3 
(kafka.server.ZookeeperLeaderElector$LeaderChangeListener)

[2017-03-17 21:32:44,389] INFO [BrokerChangeListener on Controller 3]: Broker 
change listener fired for path /brokers/ids with children 2,3 
(kafka.controller.ReplicaStateMachine$BrokerChangeListener)



Observation:

For a random partition in zookeeper, it lists the controller_epoch as 133.

However, the active epoch for the controller is 134.


[zk: localhost:2181(CONNECTED) 8] get 
/brokers/topics/topic_name/partitions/35/state

{"controller_epoch":133,"leader":-1,"version":1,"leader_epoch":7,"isr":[]}

cZxid = 0x5e001706a4

ctime = Sun Aug 21 09:56:29 UTC 2016

mZxid = 0x6b6085d8cd

mtime = Fri Mar 17 21:34:00 UTC 2017

pZxid = 0x5e001706a4

cversion = 0

dataVersion = 71

aclVersion = 0

ephemeralOwner = 0x0

dataLength = 74

numChildren = 0


[zk: localhost:2181(CONNECTED) 9] get /controller_epoch

134

cZxid = 0x100000012

ctime = Tue Oct 28 10:52:48 UTC 2014

mZxid = 0x6b60817117

mtime = Fri Mar 17 21:32:19 UTC 2017

pZxid = 0x100000012

cversion = 0

dataVersion = 133

aclVersion = 0

ephemeralOwner = 0x0

dataLength = 3

numChildren = 0


At this point, we triggered a controller re-election by removing the 
/controller znode and once it came back everything worked correctly:

Zookeeper03:

[zk: localhost:2181(CONNECTED) 16] rmr /controller

[zk: localhost:2181(CONNECTED) 17] get 
/brokers/topics/topic_name/partitions/35/state

{"controller_epoch":135,"leader":3,"version":1,"leader_epoch":8,"isr":[3,2]}

cZxid = 0x5e001706a4

ctime = Sun Aug 21 09:56:29 UTC 2016

mZxid = 0x6bb8cb85e2

mtime = Thu Mar 23 00:22:25 UTC 2017

pZxid = 0x5e001706a4

cversion = 0

dataVersion = 73

aclVersion = 0

ephemeralOwner = 0x0

dataLength = 76

numChildren = 0


Broker 3:

[2017-03-23 00:20:16,875] INFO [Partition state machine on Controller 3]: 
Stopped partition state machine (kafka.controller.PartitionStateMachine)

[2017-03-23 00:20:16,875] INFO [Replica state machine on controller 3]: Stopped 
replica state machine (kafka.controller.ReplicaStateMachine)

[2017-03-23 00:20:16,875] INFO [Controller 3]: Broker 3 resigned as the 
controller (kafka.controller.KafkaController)

[2017-03-23 00:20:16,879] INFO Creating /controller (is it secure? false) 
(kafka.utils.ZKCheckedEphemeral)

[2017-03-23 00:20:16,882] INFO Result of znode creation is: OK 
(kafka.utils.ZKCheckedEphemeral)

[2017-03-23 00:20:16,882] INFO 3 successfully elected as leader 
(kafka.server.ZookeeperLeaderElector)

[2017-03-23 00:20:16,882] INFO [Controller 3]: Broker 3 starting become 
controller state transition (kafka.controller.KafkaController)

[2017-03-23 00:20:16,888] INFO [Controller 3]: Initialized controller epoch to 
134 and zk version 133 (kafka.controller.KafkaController)

[2017-03-23 00:20:16,890] INFO [Controller 3]: Controller 3 incremented epoch 
to 135 (kafka.controller.KafkaController)

[2017-03-23 00:20:50,092] INFO [Controller-3-to-broker-2-send-thread], Starting 
 (kafka.controller.RequestSendThread)

[2017-03-23 00:20:50,094] INFO [Controller-3-to-broker-1-send-thread], Starting 
 (kafka.controller.RequestSendThread)

[2017-03-23 00:20:50,097] INFO [Controller-3-to-broker-3-send-thread], Starting 
 (kafka.controller.RequestSendThread)

[2017-03-23 00:20:50,100] INFO [Controller 3]: Partitions undergoing preferred 
replica election:  (kafka.controller.KafkaController)

[2017-03-23 00:20:50,100] INFO [Controller 3]: Partitions that completed 
preferred replica election:  (kafka.controller.KafkaController)

[2017-03-23 00:20:50,100] INFO [Controller 3]: Resuming preferred replica 
election for partitions:  (kafka.controller.KafkaController)

[2017-03-23 00:20:50,102] INFO [Controller 3]: Partitions being reassigned: 
Map() (kafka.controller.KafkaController)

[2017-03-23 00:20:50,102] INFO [Controller 3]: Partitions already reassigned: 
List() (kafka.controller.KafkaController)

[2017-03-23 00:20:50,102] INFO [Controller 3]: Resuming reassignment of 
partitions: Map() (kafka.controller.KafkaController)

[2017-03-23 00:20:50,161] INFO [Controller 3]: List of topics to be deleted:  
(kafka.controller.KafkaController)

[2017-03-23 00:20:50,161] INFO [Controller 3]: List of topics ineligible for 
deletion:  (kafka.controller.KafkaController)

[2017-03-23 00:20:50,162] INFO [Controller 3]: Currently active brokers in the 
cluster: Set(1, 2, 3) (kafka.controller.KafkaController)

[2017-03-23 00:20:50,162] INFO [Controller 3]: Currently shutting brokers in 
the cluster: Set() (kafka.controller.KafkaController)

[2017-03-23 00:20:50,162] INFO [Controller 3]: Current list of topics in the 
cluster: Set(big list of correct topics) (kafka.controller.KafkaController)
{code}

> updateIsr should stop after failed several times due to zkVersion issue
> -----------------------------------------------------------------------
>
>                 Key: KAFKA-3042
>                 URL: https://issues.apache.org/jira/browse/KAFKA-3042
>             Project: Kafka
>          Issue Type: Bug
>          Components: controller
>    Affects Versions: 0.8.2.1
>         Environment: jdk 1.7
> centos 6.4
>            Reporter: Jiahongchao
>            Assignee: Dong Lin
>              Labels: reliability
>             Fix For: 0.11.0.0
>
>         Attachments: controller.log, server.log.2016-03-23-01, 
> state-change.log
>
>
> sometimes one broker may repeatly log
> "Cached zkVersion 54 not equal to that in zookeeper, skip updating ISR"
> I think this is because the broker consider itself as the leader in fact it's 
> a follower.
> So after several failed tries, it need to find out who is the leader



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to