No, broker 5 is alive with log.
[2014-06-11 13:59:45,170] ERROR Conditional update of path
/brokers/topics/topicTRACE/partitions/0/state with data
{"controller_epoch":1,"leader":5,"version":1,"leader_epoch":0,"isr":[5]} and
expected version 2 failed due to
org.apache.zookeeper.KeeperException$BadVersionException: KeeperErrorCode =
BadVersion for /brokers/topics/topicTRACE/partitions/0/state
(kafka.utils.ZkUtils$)
How can I issue metadata request?
Thanks.
On Jun 11, 2014, at 1:26 PM, Jun Rao <[email protected]> wrote:
> The info from kafka-topics is the correct one. Is broker 5 dead? It seems
> that you can issue metadata request to it.
>
> Thanks,
>
> Jun
>
>
> On Tue, Jun 10, 2014 at 8:26 PM, Bongyeon Kim <[email protected]>
> wrote:
>
>> With some WARN log, Yes, it is.
>>
>> and I found interesting things separately before I mentioned.
>> I have another clusters. I run 2 brokers on 1 machine for test. and I see
>> same problem before I mentioned, but I can’t see any error log on
>> controller.log.
>>
>> At this time, when I list topics with kafka-topic tool, I see information
>> like below.
>>
>> == kafka-topics.sh ===================================================
>>
>> Topic:topicTRACE PartitionCount:2 ReplicationFactor:2
>> Configs:retention.ms=3600000
>> Topic: topicTRACE Partition: 0 Leader: 6 Replicas:
>> 5,6 Isr: 6
>> Topic: topicTRACE Partition: 1 Leader: 6 Replicas:
>> 6,5 Isr: 6,5
>>
>> ======================================================================
>>
>> but, producer keeps producing broker 5 which seems to be dead.
>>
>> when I get metadata from broker using my version of java api tool, info of
>> leader and isr is different.
>> Speaking properly, metadata from one broker is same as data from
>> kafka-topics tool, metadata from another broker is different.
>>
>> ========================================================================
>>
>> $ bin/kafka-run-class.sh com.kthcorp.daisy.ccprt.util.KafkaMetadata
>> c-ccp-tk1-a60:9091 topicTRACE
>> ---- topic info ----
>> partition: 0, leader: 5, replica: [id:5,host:c-ccp-tk1-a60,port:9091,
>> id:6,host:c-ccp-tk1-a60,port:9092], isr:
>> [id:5,host:c-ccp-tk1-a60,port:9091, id:6,host:c-ccp-tk1-a60,port:9092]
>> partition: 1, leader: 6, replica: [id:6,host:c-ccp-tk1-a60,port:9092,
>> id:5,host:c-ccp-tk1-a60,port:9091], isr:
>> [id:6,host:c-ccp-tk1-a60,port:9092, id:5,host:c-ccp-tk1-a60,port:9091]
>>
>> ========================================================================
>>
>> $ bin/kafka-run-class.sh com.kthcorp.daisy.ccprt.util.KafkaMetadata
>> c-ccp-tk1-a60:9092 topicTRACE
>> ---- topic info ----
>> partition: 0, leader: 6, replica: [id:5,host:c-ccp-tk1-a60,port:9091,
>> id:6,host:c-ccp-tk1-a60,port:9092], isr: [id:6,host:c-ccp-tk1-a60,port:9092]
>> partition: 1, leader: 6, replica: [id:6,host:c-ccp-tk1-a60,port:9092,
>> id:5,host:c-ccp-tk1-a60,port:9091], isr:
>> [id:6,host:c-ccp-tk1-a60,port:9092, id:5,host:c-ccp-tk1-a60,port:9091]
>>
>> ========================================================================
>>
>> which one is correct? why is it happened?
>>
>>
>> Thanks~
>>
>>
>>
>> On Jun 10, 2014, at 11:28 PM, Jun Rao <[email protected]> wrote:
>>
>>> Ok. Was this host (broker id:1,host:c-ccp-tk1-a58,port:9091) up when the
>>> controller had SocketTimeoutException?
>>>
>>> Thanks,
>>>
>>> Jun
>>>
>>>
>>> On Mon, Jun 9, 2014 at 10:11 PM, Bongyeon Kim <[email protected]>
>>> wrote:
>>>
>>>> No, I can see any ZK session expiration log.
>>>>
>>>> What I have to do to prevent this? Increasing '
>>>> zookeeper.session.timeout.ms'
>>>> can help?
>>>>
>>>>
>>>> On Tue, Jun 10, 2014 at 12:58 PM, Jun Rao <[email protected]> wrote:
>>>>
>>>>> This is probably related to kafka-1382. The root cause is likely ZK
>>>> session
>>>>> expiration in the broker. Did you see any?
>>>>>
>>>>> Thanks,
>>>>>
>>>>> Jun
>>>>>
>>>>>
>>>>> On Mon, Jun 9, 2014 at 8:11 PM, Bongyeon Kim <[email protected]>
>>>>> wrote:
>>>>>
>>>>>> Hi, team.
>>>>>>
>>>>>> I’m using 0.8.1.
>>>>>> I found some strange log repeatedly on server.log in one of my brokers
>>>>> and
>>>>>> it keeps logging until now.
>>>>>>
>>>>>> server.log
>>>>>>
>>>>>
>>>>
>> ======================================================================================
>>>>>> ...
>>>>>> [2014-06-09 10:41:47,402] ERROR Conditional update of path
>>>>>> /brokers/topics/topicTRACE/partitions/1/state with data
>>>>>>
>>>>>
>>>>
>> {"controller_epoch":19,"leader":2,"version":1,"leader_epoch":43,"isr":[4,2]}
>>>>>> and expected version 439 failed due to
>>>>>> org.apache.zookeeper.KeeperException$BadVersionException:
>>>>> KeeperErrorCode =
>>>>>> BadVersion for /brokers/topics/topicTRACE/partitions/1/state
>>>>>> (kafka.utils.ZkUtils$)
>>>>>> [2014-06-09 10:41:47,402] INFO Partition [topicTRACE,1] on broker 2:
>>>>>> Cached zkVersion [439] not equal to that in zookeeper, skip updating
>>>> ISR
>>>>>> (kafka.cluster.Partition)
>>>>>> [2014-06-09 10:41:47,402] INFO Partition [topicDEBUG,0] on broker 2:
>>>>>> Shrinking ISR for partition [topicDEBUG,0] from 1,3,2 to 2
>>>>>> (kafka.cluster.Partition)
>>>>>> [2014-06-09 10:41:47,416] ERROR Conditional update of path
>>>>>> /brokers/topics/topicDEBUG/partitions/0/state with data
>>>>>>
>>>>>
>>>>
>> {"controller_epoch":19,"leader":2,"version":1,"leader_epoch":43,"isr":[2]}
>>>>>> and expected version 1424 failed due to
>>>>>> org.apache.zookeeper.KeeperException$BadVersionException:
>>>>> KeeperErrorCode =
>>>>>> BadVersion for /brokers/topics/topicDEBUG/partitions/0/state
>>>>>> (kafka.utils.ZkUtils$)
>>>>>> [2014-06-09 10:41:47,432] INFO Partition [topicDEBUG,0] on broker 2:
>>>>>> Cached zkVersion [1424] not equal to that in zookeeper, skip updating
>>>> ISR
>>>>>> (kafka.cluster.Partition)
>>>>>> [2014-06-09 10:41:47,432] INFO Partition [topicCDR,3] on broker 2:
>>>>>> Shrinking ISR for partition [topicCDR,3] from 4,1,2 to 2
>>>>>> (kafka.cluster.Partition)
>>>>>> [2014-06-09 10:41:47,435] ERROR Conditional update of path
>>>>>> /brokers/topics/topicCDR/partitions/3/state with data
>>>>>>
>>>>>
>>>>
>> {"controller_epoch":19,"leader":2,"version":1,"leader_epoch":46,"isr":[2]}
>>>>>> and expected version 541 failed due to
>>>>>> org.apache.zookeeper.KeeperException$BadVersionException:
>>>>> KeeperErrorCode =
>>>>>> BadVersion for /brokers/topics/topicCDR/partitions/3/state
>>>>>> (kafka.utils.ZkUtils$)
>>>>>> [2014-06-09 10:41:47,435] INFO Partition [topicCDR,3] on broker 2:
>>>> Cached
>>>>>> zkVersion [541] not equal to that in zookeeper, skip updating ISR
>>>>>> (kafka.cluster.Partition)
>>>>>> [2014-06-09 10:41:48,426] INFO Partition [topicTRACE,1] on broker 2:
>>>>>> Shrinking ISR for partition [topicTRACE,1] from 4,3,2 to 4,2
>>>>>> (kafka.cluster.Partition)
>>>>>> ...
>>>>>>
>>>>>>
>>>>>
>>>>
>> =================================================================================================
>>>>>>
>>>>>> and found some error and warning in controller.log
>>>>>>
>>>>>>
>>>>>> controller.log
>>>>>>
>>>>>
>>>>
>> ======================================================================================
>>>>>> ...
>>>>>> [2014-06-09 10:42:03,962] WARN [Controller-3-to-broker-1-send-thread],
>>>>>> Controller 3 fails to send a request to broker
>>>>>> id:1,host:c-ccp-tk1-a58,port:9091 (kafka.controller.RequestSendThread)
>>>>>> java.net.SocketTimeoutException
>>>>>> at
>>>>>>
>> sun.nio.ch.SocketAdaptor$SocketInputStream.read(SocketAdaptor.java:229)
>>>>>> at
>>>>> sun.nio.ch.ChannelInputStream.read(ChannelInputStream.java:103)
>>>>>> at
>>>>>>
>>>>>
>>>>
>> java.nio.channels.Channels$ReadableByteChannelImpl.read(Channels.java:385)
>>>>>> at kafka.utils.Utils$.read(Utils.scala:375)
>>>>>> at
>>>>>>
>>>>>
>>>>
>> kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:54)
>>>>>> at
>>>>>> kafka.network.Receive$class.readCompletely(Transmission.scala:56)
>>>>>> at
>>>>>>
>>>>>
>>>>
>> kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29)
>>>>>> at
>>>>> kafka.network.BlockingChannel.receive(BlockingChannel.scala:100)
>>>>>> at
>>>>>>
>>>>>
>>>>
>> kafka.controller.RequestSendThread.doWork(ControllerChannelManager.scala:146)
>>>>>> at
>>>>> kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:51)
>>>>>> [2014-06-09 10:42:03,964] ERROR
>> [Controller-3-to-broker-1-send-thread],
>>>>>> Controller 3 epoch 21 failed to send UpdateMetadata request with
>>>>>> correlation id 1 to broker id:1,host:c-ccp-tk1-a58,port:9091.
>>>>> Reconnecting
>>>>>> to broker. (kafka.controller.RequestSendThread)
>>>>>> java.nio.channels.ClosedChannelException
>>>>>> at kafka.network.BlockingChannel.send(BlockingChannel.scala:89)
>>>>>> at
>>>>>>
>>>>>
>>>>
>> kafka.controller.RequestSendThread.liftedTree1$1(ControllerChannelManager.scala:132)
>>>>>> at
>>>>>>
>>>>>
>>>>
>> kafka.controller.RequestSendThread.doWork(ControllerChannelManager.scala:131)
>>>>>> at
>>>>> kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:51)
>>>>>>
>>>>>> ...
>>>>>>
>>>>>> [2014-06-09 10:42:38,064] WARN [OfflinePartitionLeaderSelector]: No
>>>>> broker
>>>>>> in ISR is alive for [topicTRACE,0]. Elect leader 3 from live brokers
>> 3.
>>>>>> There's potential data loss.
>>>>>> (kafka.controller.OfflinePartitionLeaderSelector)
>>>>>> ...
>>>>>>
>>>>>>
>>>>>
>>>>
>> =================================================================================================
>>>>>>
>>>>>> Why is this happen? Is there any possibilities data loss?
>>>>>> To normalize my brokers, What I have to do? Do I have to restart this
>>>>>> broker?
>>>>>>
>>>>>>
>>>>>> Thanks in advance.
>>>>>>
>>>>>>
>>>>>>
>>>>>
>>>>
>>>>
>>>>
>>>> --
>>>> *Sincerely*
>>>> *,**Bongyeon Kim*
>>>>
>>>> Java Developer & Engineer
>>>> Seoul, Korea
>>>> Mobile: +82-10-9369-1314
>>>> Email: [email protected]
>>>> Twitter: http://twitter.com/tigerby
>>>> Facebook: http://facebook.com/tigerby
>>>> Wiki: http://tigerby.com
>>>>
>>
>>