It seems the third broker went down at around 10:30:57, then back up at
12:27:00,351, but the new controller trying to update its status and
failed. I suspect it is hitting this issue.

https://issues.apache.org/jira/browse/KAFKA-1096

Guozhang


On Thu, Jun 19, 2014 at 9:23 PM, Arjun <ar...@socialtwist.com> wrote:

> I think I found something related to this. This i found in some other
> nodes controller log. Am i correct in suspecting this as the issue. what
> might have gone wrong. From log it seems, the third node just got added and
> some error occurred while handling the broker change.
> There are no errors in the controller log after this.
> I think I will try to restart the broker and see if the changes will take
> place or not.
> Can some one suggest how we can over come this sort of thing, I mean what
> monitoring strategy should we put in place to detect this early.
>
> [2014-05-23 12:26:59,590] INFO [BrokerChangeListener on Controller 0]:
> Broker change listener fired for path /brokers/ids with children 2,1,0
> (kafka.controller.ReplicaStateMachine$BrokerChangeListener)
> [2014-05-23 12:26:59,625] INFO [BrokerChangeListener on Controller 0]:
> Newly added brokers: 2, deleted brokers: , all live brokers: 2,1,0
> (kafka.controller.ReplicaStateMachine$BrokerChangeListener)
> [2014-05-23 12:26:59,626] DEBUG [Channel manager on controller 0]:
> Controller 0 trying to connect to broker 2 (kafka.controller.
> ControllerChannelManager)
> [2014-05-23 12:28:02,763] ERROR [BrokerChangeListener on Controller 0]:
> Error while handling broker changes (kafka.controller.ReplicaStateMachine$
> BrokerChangeListener)
> java.net.ConnectException: Connection timed out
>         at sun.nio.ch.Net.connect(Native Method)
>         at sun.nio.ch.SocketChannelImpl.connect(SocketChannelImpl.
> java:500)
>         at kafka.network.BlockingChannel.connect(BlockingChannel.scala:57)
>         at kafka.controller.ControllerChannelManager.kafka$controller$
> ControllerChannelManager$$addNewBroker(ControllerChannelManager.scala:84)
>         at kafka.controller.ControllerChannelManager.addBroker(
> ControllerChannelManager.scala:65)
>         at kafka.controller.ReplicaStateMachine$
> BrokerChangeListener$$anonfun$handleChildChange$1$$anonfun$
> apply$mcV$sp$6.apply(ReplicaStateMachine.scala:269)
>         at kafka.controller.ReplicaStateMachine$
> BrokerChangeListener$$anonfun$handleChildChange$1$$anonfun$
> apply$mcV$sp$6.apply(ReplicaStateMachine.scala:269)
>         at scala.collection.immutable.Set$Set1.foreach(Set.scala:81)
>         at kafka.controller.ReplicaStateMachine$
> BrokerChangeListener$$anonfun$handleChildChange$1.apply$mcV$
> sp(ReplicaStateMachine.scala:269)
>         at kafka.controller.ReplicaStateMachine$
> BrokerChangeListener$$anonfun$handleChildChange$1.apply(
> ReplicaStateMachine.scala:260)
>         at kafka.controller.ReplicaStateMachine$
> BrokerChangeListener$$anonfun$handleChildChange$1.apply(
> ReplicaStateMachine.scala:260)
>         at kafka.metrics.KafkaTimer.time(KafkaTimer.scala:33)
>         at kafka.controller.ReplicaStateMachine$BrokerChangeListener.
> handleChildChange(ReplicaStateMachine.scala:259)
>         at org.I0Itec.zkclient.ZkClient$7.run(ZkClient.java:568)
>         at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71)
>
>
> On Thursday 19 June 2014 08:41 PM, Guozhang Wang wrote:
>
>> Could you check the controller log to see if broker 2 once has a soft
>> failure and hence its leadership been migrated to other brokers?
>>
>>
>> On Thu, Jun 19, 2014 at 6:57 AM, Arjun <ar...@socialtwist.com> wrote:
>>
>>  Hi,
>>>
>>> I have a set up of 3 kafka servers, with a replication factor of 2.
>>> I have only one topic in this setup as of now.
>>>
>>> bin/kafka-list-topic.sh --zookeeper server1:2181,server2:2181,
>>> server3:2181
>>> --topic topic1
>>> topic: topic1    partition: 0    leader: 1    replicas: 2,1    isr: 1
>>> topic: topic1    partition: 1    leader: 0    replicas: 0,2    isr: 0
>>> topic: topic1    partition: 2    leader: 1    replicas: 1,0    isr: 0,1
>>> topic: topic1    partition: 3    leader: 0    replicas: 2,0    isr: 0
>>> topic: topic1    partition: 4    leader: 0    replicas: 0,1    isr: 0,1
>>> topic: topic1    partition: 5    leader: 1    replicas: 1,2    isr: 1
>>> topic: topic1    partition: 6    leader: 1    replicas: 2,1    isr: 1
>>> topic: topic1    partition: 7    leader: 0    replicas: 0,2    isr: 0
>>> topic: topic1    partition: 8    leader: 1    replicas: 1,0    isr: 0,1
>>> topic: topic1    partition: 9    leader: 0    replicas: 2,0    isr: 0
>>> topic: topic1    partition: 10    leader: 0    replicas: 0,1    isr: 0,1
>>> topic: topic1    partition: 11    leader: 1    replicas: 1,2    isr: 1
>>>
>>> The Third broker is not in the ISR list. There are no errors in the logs.
>>> The Thread dump doesn't have any thread with "RepliacaFetcherManager"
>>> *Thread Dump
>>> ------------------------------------------------------------
>>> ------------------------------------------------------------
>>> ------------------------------
>>> *2014-06-19 13:27:39
>>> Full thread dump Java HotSpot(TM) 64-Bit Server VM (20.4-b02 mixed mode):
>>>
>>> "RMI TCP Connection(idle)" daemon prio=10 tid=0x00007fccec004800
>>> nid=0x201f waiting on condition [0x00007fcce540f000]
>>>     java.lang.Thread.State: TIMED_WAITING (parking)
>>>          at sun.misc.Unsafe.park(Native Method)
>>>          - parking to wait for  <0x00000000bc30e6c8> (a
>>> java.util.concurrent.SynchronousQueue$TransferStack)
>>>          at java.util.concurrent.locks.LockSupport.parkNanos(
>>> LockSupport.java:196)
>>>          at java.util.concurrent.SynchronousQueue$
>>> TransferStack.awaitFulfill(SynchronousQueue.java:424)
>>>          at java.util.concurrent.SynchronousQueue$
>>> TransferStack.transfer(
>>> SynchronousQueue.java:323)
>>>          at java.util.concurrent.SynchronousQueue.poll(
>>> SynchronousQueue.java:874)
>>>          at java.util.concurrent.ThreadPoolExecutor.getTask(
>>> ThreadPoolExecutor.java:945)
>>>          at java.util.concurrent.ThreadPoolExecutor$Worker.run(
>>> ThreadPoolExecutor.java:907)
>>>          at java.lang.Thread.run(Thread.java:662)
>>>
>>> "JMX server connection timeout 30" daemon prio=10 tid=0x00007fccf800a800
>>> nid=0x555 in Object.wait() [0x00007fcce530e000]
>>>     java.lang.Thread.State: TIMED_WAITING (on object monitor)
>>>          at java.lang.Object.wait(Native Method)
>>>          at com.sun.jmx.remote.internal.ServerCommunicatorAdmin$
>>> Timeout.run(ServerCommunicatorAdmin.java:150)
>>>          - locked <0x00000000bc39a640> (a [I)
>>>          at java.lang.Thread.run(Thread.java:662)
>>>
>>> "RMI Scheduler(0)" daemon prio=10 tid=0x00007fccf0040000 nid=0x550
>>> waiting
>>> on condition [0x00007fcce5510000]
>>>     java.lang.Thread.State: TIMED_WAITING (parking)
>>>          at sun.misc.Unsafe.park(Native Method)
>>>          - parking to wait for  <0x00000000bc2e1fe8> (a
>>> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>>>          at java.util.concurrent.locks.LockSupport.parkNanos(
>>> LockSupport.java:196)
>>>          at java.util.concurrent.locks.AbstractQueuedSynchronizer$
>>> ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2025)
>>>          at java.util.concurrent.DelayQueue.take(DelayQueue.java:164)
>>>          at java.util.concurrent.ScheduledThreadPoolExecutor$
>>> DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:609)
>>>          at java.util.concurrent.ScheduledThreadPoolExecutor$
>>> DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:602)
>>>          at java.util.concurrent.ThreadPoolExecutor.getTask(
>>> ThreadPoolExecutor.java:947)
>>>          at java.util.concurrent.ThreadPoolExecutor$Worker.run(
>>> ThreadPoolExecutor.java:907)
>>>          at java.lang.Thread.run(Thread.java:662)
>>>
>>> "kafka-logflusher-1" daemon prio=10 tid=0x00007fcd102b9800 nid=0x54d
>>> waiting on condition [0x00007fcce5813000]
>>>     java.lang.Thread.State: TIMED_WAITING (parking)
>>>          at sun.misc.Unsafe.park(Native Method)
>>> ------------------------------------------------------------
>>> ------------------------------------------------------------
>>> -----------------------------------------------
>>>
>>> I haven't seen any GC pauses in the system. JMX max lag (
>>> "kafka.server":name="([-.\w]+)-MaxLag",type="ReplicaFetcherManager") for
>>> this node is 0.
>>>
>>> We have restarted the nodes one after the other and we cant make this
>>> node
>>> to push to ISR.
>>> Can some one please  let me know, how to push this node to ISR.
>>>
>>>
>>> Thanks
>>> Arjun Narasimha Kota
>>>
>>>
>>>
>>>
>>
>


-- 
-- Guozhang

Reply via email to