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

Dmitry Bugaychenko commented on KAFKA-1407:
-------------------------------------------

Checked out the logs in more details - there were problems in connection with 
ZK and it looks like they ended up in the deadlock between ZkClient thread and 
delete-topics threads:

{code}
[2014-04-22 02:37:40,597] INFO zookeeper state changed (Disconnected) 
(org.I0Itec.zkclient.ZkClient)
...
[2014-04-22 02:37:42,067] INFO zookeeper state changed (SyncConnected) 
(org.I0Itec.zkclient.ZkClient)
...
[2014-04-22 02:40:18,217] INFO zookeeper state changed (Disconnected) 
(org.I0Itec.zkclient.ZkClient)
...
[2014-04-22 02:40:42,523] INFO zookeeper state changed (Expired) 
(org.I0Itec.zkclient.ZkClient)
[2014-04-22 02:40:42,525] INFO [SessionExpirationListener on 1], ZK expired; 
shut down all controller components and try to re-elect 
(kafka.controller.KafkaController$SessionExpirationListener)
[2014-04-22 02:40:42,527] DEBUG Shutting down task scheduler. 
(kafka.utils.KafkaScheduler)
[2014-04-22 02:40:42,528] ERROR Error handling event ZkEvent[New session event 
sent to kafka.controller.KafkaController$SessionExpirationListener@6fb84e18] 
(org.Itec.zkclient.ZkEventThread)
 ava.lang.IllegalStateException: Kafka scheduler has not been started
      at kafka.utils.KafkaScheduler.ensureStarted(KafkaScheduler.scala:116)
      at kafka.utils.KafkaScheduler.shutdown(KafkaScheduler.scala:86)
      at 
kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply$mcV$sp(KafkaController.scala:339)
      at 
kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply(KafkaController.scala:337)
      at 
kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply(KafkaController.scala:337)
      at kafka.utils.Utils$.inLock(Utils.scala:538)
      at 
kafka.controller.KafkaController.onControllerResignation(KafkaController.scala:337)
      at 
kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply$mcZ$sp(KafkaController.scala:1068)
      at 
kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply(KafkaController.scala:1067)
      at 
kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply(KafkaController.scala:1067)
      at kafka.utils.Utils$.inLock(Utils.scala:538)
      at 
kafka.controller.KafkaController$SessionExpirationListener.handleNewSession(KafkaController.scala:1067)
      at org.I0Itec.zkclient.ZkClient$4.run(ZkClient.java:472)
      at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71)
[2014-04-22 02:40:42,528] INFO Session establishment complete on server 
devlnx2.odnoklassniki.ru/172.18.10.101:2181, sessionid = 0x14545879f6c52a6, 
negotiated timeout = 6000 (org.apache.zookeeper.ClientCnxn)
[2014-04-22 02:40:42,529] INFO zookeeper state changed (SyncConnected) 
(org.I0Itec.zkclient.ZkClient)
[2014-04-22 02:40:42,529] INFO re-registering broker info in ZK for broker 1 
(kafka.server.KafkaHealthcheck)
[2014-04-22 02:40:42,843] INFO Registered broker 1 at path /brokers/ids/1 with 
address devlnx5:9092. (kafka.utils.ZkUtils$)
[2014-04-22 02:40:42,843] INFO done re-registering broker 
(kafka.server.KafkaHealthcheck)
[2014-04-22 02:40:42,844] INFO Subscribing to /brokers/topics path to watch for 
new topics (kafka.server.KafkaHealthcheck)
[2014-04-22 02:40:44,841] INFO [ControllerEpochListener on 1]: Initialized 
controller epoch to 53 and zk version 52 
(kafka.controller.ControllerEpochListener)
[2014-04-22 02:40:46,152] INFO Reloading nodes from 
/cluster/databus-monitoring-service/servers 
(one.zookeeper.client.childmon.NodeChildrenMonitor)
[2014-04-22 02:40:47,323] INFO 1 successfully elected as leader 
(kafka.server.ZookeeperLeaderElector)
[2014-04-22 02:40:47,324] INFO [Controller 1]: Broker 1 starting become 
controller state transition (kafka.controller.KafkaController)
[2014-04-22 02:40:47,392] INFO Reloading nodes from 
/cluster/databus-monitoring-service/servers 
(one.zookeeper.client.childmon.NodeChildrenMonitor)
[2014-04-22 02:40:48,389] INFO [Controller 1]: Controller 1 incremented epoch 
to 54 (kafka.controller.KafkaController)
[2014-04-22 02:41:01,041] INFO [Controller 1]: Broker 1 is ready to serve as 
the new controller with epoch 54 (kafka.controller.KafkaController)
[2014-04-22 02:41:01,042] INFO [Controller 1]: Starting preferred replica 
leader election for partitions  (kafka.controller.KafkaController)
[2014-04-22 02:41:01,043] INFO [Partition state machine on Controller 1]: 
Invoking state change to OnlinePartition for partitions  
(kafka.controller.PartitionStateMachine)
[2014-04-22 02:41:01,048] INFO [Controller 1]: starting the partition rebalance 
scheduler (kafka.controller.KafkaController)
[2014-04-22 02:41:01,048] DEBUG Initializing task scheduler. 
(kafka.utils.KafkaScheduler)
[2014-04-22 02:41:01,049] DEBUG Scheduling task partition-rebalance-thread with 
initial delay 5000 ms and period 300000 ms. (kafka.utils.KafkaScheduler)
[2014-04-22 02:41:01,050] INFO [delete-topics-thread], Starting  
(kafka.controller.TopicDeletionManager$DeleteTopicsThread)
[2014-04-22 02:41:01,053] INFO New leader is 1 
(kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
[2014-04-22 02:41:01,054] INFO Waiting for signal to start or continue topic 
deletion (kafka.controller.TopicDeletionManager)
[2014-04-22 02:41:01,055] INFO [ControllerEpochListener on 1]: Initialized 
controller epoch to 54 and zk version 53 
(kafka.controller.ControllerEpochListener)
[2014-04-22 02:41:01,283] INFO [ReplicaFetcherThread-0-2], Stopped  
(kafka.server.ReplicaFetcherThread)
[2014-04-22 02:41:01,283] INFO [ReplicaFetcherThread-0-2], Shutdown completed 
(kafka.server.ReplicaFetcherThread)
...
[2014-04-22 02:48:33,567] INFO zookeeper state changed (Disconnected) 
(org.I0Itec.zkclient.ZkClient)
[2014-04-22 02:48:34,812] INFO zookeeper state changed (SyncConnected) 
(org.I0Itec.zkclient.ZkClient)
[2014-04-22 02:48:40,380] INFO zookeeper state changed (Disconnected) 
(org.I0Itec.zkclient.ZkClient)
[2014-04-22 02:48:49,008] INFO zookeeper state changed (Expired) 
(org.I0Itec.zkclient.ZkClient)
[2014-04-22 02:48:49,010] INFO [SessionExpirationListener on 1], ZK expired; 
shut down all controller components and try to re-elect 
(kafka.controller.KafkaController$SessionExpirationListener)
[2014-04-22 02:48:49,010] DEBUG Shutting down task scheduler. 
(kafka.utils.KafkaScheduler)
[2014-04-22 02:48:49,011] INFO [delete-topics-thread], Shutting down 
(kafka.controller.TopicDeletionManager$DeleteTopicsThread)
...
[2014-04-22 02:48:51,079] INFO zookeeper state changed (SyncConnected) 
(org.I0Itec.zkclient.ZkClient)
...
[2014-04-22 02:49:19,988] INFO Partition [metrics,6] on broker 1: Shrinking ISR 
for partition [metrics,6] from 1,3 to 1 (kafka.cluster.Partition)
[2014-04-22 02:49:19,990] ERROR Conditional update of path 
/brokers/topics/metrics/partitions/6/state with data 
{"controller_epoch":52,"leader":1,"version":1,"leader_epoch":46,"isr":[1]} and 
expected version 80 failed due to 
org.apache.zookeeper.KeeperException$BadVersionException: KeeperErrorCode = 
BadVersion for /brokers/topics/metrics/partitions/6/state (kafka.utils.ZkUtils$)
[2014-04-22 02:49:19,991] INFO Partition [metrics,6] on broker 1: Cached 
zkVersion [80] not equal to that in zookeeper, skip updating ISR 
(kafka.cluster.Partition)
{code}

{code}
"ZkClient-EventThread-37-devlnx2:2181" daemon prio=10 tid=0x00007f6c78590000 
nid=0x40b2 waiting on condition [

   java.lang.Thread.State: WAITING (parking)
      at sun.misc.Unsafe.park(Native Method)
      - parking to wait for  <0x000000071fbcfae8> (a 
java.util.concurrent.CountDownLatch$Sync)
      at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186)
      at 
java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronize

      at 
java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSync

      at 
java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchr

      at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:236)
      at kafka.utils.ShutdownableThread.shutdown(ShutdownableThread.scala:36)
      at 
kafka.controller.TopicDeletionManager.shutdown(TopicDeletionManager.scala:93)
      at 
kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply$mcV$sp(KafkaController.scal

      at 
kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply(KafkaController.scala:337)
      at 
kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply(KafkaController.scala:337)
      at kafka.utils.Utils$.inLock(Utils.scala:538)
      at 
kafka.controller.KafkaController.onControllerResignation(KafkaController.scala:337)
      at 
kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply$mcZ$sp(K

      at 
kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply(KafkaCon

      at 
kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply(KafkaCon

      at kafka.utils.Utils$.inLock(Utils.scala:538)
      at 
kafka.controller.KafkaController$SessionExpirationListener.handleNewSession(KafkaController.scala:106

      at org.I0Itec.zkclient.ZkClient$4.run(ZkClient.java:472)
      at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71)

"delete-topics-thread" prio=10 tid=0x00007f6c2801d000 nid=0x61f1 waiting on 
condition [0x00007f6cbf2f1000]
   java.lang.Thread.State: WAITING (parking)
      at sun.misc.Unsafe.park(Native Method)
      - parking to wait for  <0x000000071f594c08> (a 
java.util.concurrent.locks.ReentrantLock$NonfairSync)
      at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186)
      at 
java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronize

      at 
java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:8

      at 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronize

      at 
kafka.controller.TopicDeletionManager.kafka$controller$TopicDeletionManager$$awaitTopicDeletionNotifi

      at 
kafka.controller.TopicDeletionManager$DeleteTopicsThread$$anonfun$doWork$1.apply$mcV$sp(TopicDeletion

      at 
kafka.controller.TopicDeletionManager$DeleteTopicsThread$$anonfun$doWork$1.apply(TopicDeletionManager

      at 
kafka.controller.TopicDeletionManager$DeleteTopicsThread$$anonfun$doWork$1.apply(TopicDeletionManager

      at kafka.utils.Utils$.inLock(Utils.scala:538)
      at 
kafka.controller.TopicDeletionManager$DeleteTopicsThread.doWork(TopicDeletionManager.scala:333)
      at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:51)
{code}

We are using 0.8.1 and I tried to find an option to disbale topic deletion, but 
didn't manage...

> Broker can not return to ISR because of BadVersionException
> -----------------------------------------------------------
>
>                 Key: KAFKA-1407
>                 URL: https://issues.apache.org/jira/browse/KAFKA-1407
>             Project: Kafka
>          Issue Type: Bug
>          Components: controller
>    Affects Versions: 0.8.1
>            Reporter: Dmitry Bugaychenko
>            Assignee: Neha Narkhede
>
> Each morning we found a broker out of ISR at stuck with log full of messages:
> {code}
> INFO   | jvm 1    | 2014/04/21 08:36:21 | [2014-04-21 09:36:21,907] ERROR 
> Conditional update of path /brokers/topics/topic2/partitions/1/state with 
> data 
> {"controller_epoch":46,"leader":2,"version":1,"leader_epoch":38,"isr":[2]} 
> and expected version 53 failed due to 
> org.apache.zookeeper.KeeperException$BadVersionException: KeeperErrorCode = 
> BadVersion for /brokers/topics/topic2/partitions/1/state 
> (kafka.utils.ZkUtils$)
> INFO   | jvm 1    | 2014/04/21 08:36:21 | [2014-04-21 09:36:21,907] INFO 
> Partition [topic2,1] on broker 2: Cached zkVersion [53] not equal to that in 
> zookeeper, skip updating ISR (kafka.cluster.Partition)
> {code}
> It seems that it can not recover after short netwrok break down and the only 
> way to return it is restart it using kill -9.



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Reply via email to