You are viewing a plain text version of this content. The canonical link for it is here.
Posted to users@kafka.apache.org by Arjun <ar...@socialtwist.com> on 2014/06/19 15:57:15 UTC

Broker not in ISR.

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



Re: Broker not in ISR.

Posted by Guozhang Wang <wa...@gmail.com>.
The number of URP is a good metric to monitor, if it becomes non-zero then
it usually indicates a broker failure (either soft failure or total
crashed).

Guozhang


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

> One small doubt on this. If we keep on monitoring the "number of under
> replicated partitions" and "ISR shrinks and Expansions", could we have
> found this error earlier?
> Can you please suggest me what should i be monitoring so that i can get
> earlier.
>
> Thanks
> Arjun Narasimha K
>
>
> On Friday 20 June 2014 10:13 AM, Guozhang Wang wrote:
>
>> 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

Re: Broker not in ISR.

Posted by Arjun <ar...@socialtwist.com>.
One small doubt on this. If we keep on monitoring the "number of under 
replicated partitions" and "ISR shrinks and Expansions", could we have 
found this error earlier?
Can you please suggest me what should i be monitoring so that i can get 
earlier.

Thanks
Arjun Narasimha K

On Friday 20 June 2014 10:13 AM, Guozhang Wang wrote:
> 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
>>>>
>>>>
>>>>
>>>>
>


Re: Broker not in ISR.

Posted by Guozhang Wang <wa...@gmail.com>.
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

Re: Broker not in ISR.

Posted by Arjun <ar...@socialtwist.com>.
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
>>
>>
>>
>


Re: Broker not in ISR.

Posted by Arjun <ar...@socialtwist.com>.
Hi

The controller log doesn't say much. It just says the following. The 
only thing I got from the logs is at start things were fine. There are 
soem partitions who has the broker 3 as leader. But after that there is 
no log and nothing is there to see.
(sry for the long log, i dont know what may be of important to you while 
dissecting the issue, so added what all I can)

2014-05-23 10:30:55,623] INFO [Controller 2]: Broker 2 starting become 
controller state transition (kafka.controller.KafkaController)
[2014-05-23 10:30:55,665] INFO [Controller 2]: Controller 2 incremented 
epoch to 2 (kafka.controller.KafkaController)
[2014-05-23 10:30:56,584] DEBUG [Channel manager on controller 2]: 
Controller 2 trying to connect to broker 1 
(kafka.controller.ControllerChannelManager)
[2014-05-23 10:30:56,601] DEBUG [Channel manager on controller 2]: 
Controller 2 trying to connect to broker 2 
(kafka.controller.ControllerChannelManager)
[2014-05-23 10:30:56,638] INFO [Controller-2-to-broker-2-send-thread], 
Starting (kafka.controller.RequestSendThread)
[2014-05-23 10:30:56,639] INFO [Controller-2-to-broker-1-send-thread], 
Starting (kafka.controller.RequestSendThread)
[2014-05-23 10:30:56,659] INFO [Controller 2]: Currently active brokers 
in the cluster: Set(1, 2) (kafka.controller.KafkaController)
[2014-05-23 10:30:56,694] INFO [Controller 2]: Currently shutting 
brokers in the cluster: Set() (kafka.controller.KafkaController)
[2014-05-23 10:30:56,726] INFO [Controller 2]: Current list of topics in 
the cluster: Set(topic1) (kafka.controller.KafkaController)
[2014-05-23 10:30:56,785] INFO [Replica state machine on controller 2]: 
Invoking state change to OnlineReplica for replicas 
PartitionAndReplica(topic1,1,2),PartitionAndReplica(topic1,5,1),PartitionAndReplica(topic1,6,2),PartitionAndReplica(topic1,9,2),PartitionAndReplica(topic1,11,2),PartitionAndReplica(topic1,8,1),PartitionAndReplica(topic1,11,1),PartitionAndReplica(topic1,0,2),PartitionAndReplica(topic1,2,1),PartitionAndReplica(topic1,7,2),PartitionAndReplica(topic1,3,2),PartitionAndReplica(topic1,4,1),PartitionAndReplica(topic1,5,2),PartitionAndReplica(topic1,6,1),PartitionAndReplica(topic1,0,1),PartitionAndReplica(topic1,10,1) 
(kafka.controller.ReplicaStateMachine)
[2014-05-23 10:30:56,953] INFO [Replica state machine on controller 2]: 
Started replica state machine with initial state -> Map((topic1,10,1) -> 
OnlineReplica, (topic1,0,1) -> OnlineReplica, (topic1,6,1) -> 
OnlineReplica, (topic1,2,0) -> OfflineReplica, (topic1,3,0) -> 
OfflineReplica, (topic1,5,2) -> OnlineReplica, (topic1,4,1) -> 
OnlineReplica, (topic1,7,2) -> OnlineReplica, (topic1,3,2) -> 
OnlineReplica, (topic1,8,0) -> OfflineReplica, (topic1,0,2) -> 
OnlineReplica, (topic1,2,1) -> OnlineReplica, (topic1,1,0) -> 
OfflineReplica, (topic1,11,1) -> OnlineReplica, (topic1,9,0) -> 
OfflineReplica, (topic1,8,1) -> OnlineReplica, (topic1,11,2) -> 
OnlineReplica, (topic1,4,0) -> OfflineReplica, (topic1,9,2) -> 
OnlineReplica, (topic1,10,0) -> OfflineReplica, (topic1,7,0) -> 
OfflineReplica, (topic1,6,2) -> OnlineReplica, (topic1,5,1) -> 
OnlineReplica, (topic1,1,2) -> OnlineReplica) 
(kafka.controller.ReplicaStateMachine)
[2014-05-23 10:30:57,196] DEBUG [OfflinePartitionLeaderSelector]: Some 
broker in ISR is alive for [topic1,10]. Select 1 from ISR 1 to be the 
leader. (kafka.controller.OfflinePartitionLeaderSelector)
[2014-05-23 10:30:57,197] INFO [OfflinePartitionLeaderSelector]: 
Selected new leader and ISR { "ISR":"1", "leader":"1", "leaderEpoch":"1" 
} for offline partition [topic1,10] 
(kafka.controller.OfflinePartitionLeaderSelector)
[2014-05-23 10:30:57,206] DEBUG [Partition state machine on Controller 
2]: After leader election, leader cache is updated to Map([topic1,0] -> 
(Leader:2,ISR:2,1,LeaderEpoch:0,ControllerEpoch:1), [topic1,3] -> 
(Leader:2,ISR:2,0,LeaderEpoch:0,ControllerEpoch:1), [topic1,10] -> 
(Leader:1,ISR:1,LeaderEpoch:1,ControllerEpoch:2), [topic1,8] -> 
(Leader:1,ISR:1,0,LeaderEpoch:0,ControllerEpoch:1), [topic1,6] -> 
(Leader:2,ISR:2,1,LeaderEpoch:0,ControllerEpoch:1), [topic1,2] -> 
(Leader:1,ISR:1,0,LeaderEpoch:0,ControllerEpoch:1), [topic1,9] -> 
(Leader:2,ISR:2,0,LeaderEpoch:0,ControllerEpoch:1), [topic1,4] -> 
(Leader:0,ISR:0,1,LeaderEpoch:0,ControllerEpoch:1), [topic1,5] -> 
(Leader:1,ISR:1,2,LeaderEpoch:0,ControllerEpoch:1), [topic1,7] -> 
(Leader:0,ISR:0,2,LeaderEpoch:0,ControllerEpoch:1), [topic1,1] -> 
(Leader:0,ISR:0,2,LeaderEpoch:0,ControllerEpoch:1), [topic1,11] -> 
(Leader:1,ISR:1,2,LeaderEpoch:0,ControllerEpoch:1)) 
(kafka.controller.PartitionStateMachine)
[2014-05-23 10:30:57,227] DEBUG [OfflinePartitionLeaderSelector]: Some 
broker in ISR is alive for [topic1,4]. Select 1 from ISR 1 to be the 
leader. (kafka.controller.OfflinePartitionLeaderSelector)
[2014-05-23 10:30:57,227] INFO [OfflinePartitionLeaderSelector]: 
Selected new leader and ISR { "ISR":"1", "leader":"1", "leaderEpoch":"1" 
} for offline partition [topic1,4] 
(kafka.controller.OfflinePartitionLeaderSelector)
[2014-05-23 10:30:57,325] DEBUG [Partition state machine on Controller 
2]: After leader election, leader cache is updated to Map([topic1,0] -> 
(Leader:2,ISR:2,1,LeaderEpoch:0,ControllerEpoch:1), [topic1,3] -> 
(Leader:2,ISR:2,0,LeaderEpoch:0,ControllerEpoch:1), [topic1,10] -> 
(Leader:1,ISR:1,LeaderEpoch:1,ControllerEpoch:2), [topic1,8] -> 
(Leader:1,ISR:1,0,LeaderEpoch:0,ControllerEpoch:1), [topic1,6] -> 
(Leader:2,ISR:2,1,LeaderEpoch:0,ControllerEpoch:1), [topic1,2] -> 
(Leader:1,ISR:1,0,LeaderEpoch:0,ControllerEpoch:1), [topic1,9] -> 
(Leader:2,ISR:2,0,LeaderEpoch:0,ControllerEpoch:1), [topic1,4] -> 
(Leader:1,ISR:1,LeaderEpoch:1,ControllerEpoch:2), [topic1,5] -> 
(Leader:1,ISR:1,2,LeaderEpoch:0,ControllerEpoch:1), [topic1,7] -> 
(Leader:0,ISR:0,2,LeaderEpoch:0,ControllerEpoch:1), [topic1,1] -> 
(Leader:0,ISR:0,2,LeaderEpoch:0,ControllerEpoch:1), [topic1,11] -> 
(Leader:1,ISR:1,2,LeaderEpoch:0,ControllerEpoch:1)) 
(kafka.controller.PartitionStateMachine)
[2014-05-23 10:30:57,329] DEBUG [OfflinePartitionLeaderSelector]: Some 
broker in ISR is alive for [topic1,7]. Select 2 from ISR 2 to be the 
leader. (kafka.controller.OfflinePartitionLeaderSelector)
[2014-05-23 10:30:57,329] INFO [OfflinePartitionLeaderSelector]: 
Selected new leader and ISR { "ISR":"2", "leader":"2", "leaderEpoch":"1" 
} for offline partition [topic1,7] 
(kafka.controller.OfflinePartitionLeaderSelector)
[2014-05-23 10:30:57,368] DEBUG [Partition state machine on Controller 
2]: After leader election, leader cache is updated to Map([topic1,0] -> 
(Leader:2,ISR:2,1,LeaderEpoch:0,ControllerEpoch:1), [topic1,3] -> 
(Leader:2,ISR:2,0,LeaderEpoch:0,ControllerEpoch:1), [topic1,10] -> 
(Leader:1,ISR:1,LeaderEpoch:1,ControllerEpoch:2), [topic1,8] -> 
(Leader:1,ISR:1,0,LeaderEpoch:0,ControllerEpoch:1), [topic1,6] -> 
(Leader:2,ISR:2,1,LeaderEpoch:0,ControllerEpoch:1), [topic1,2] -> 
(Leader:1,ISR:1,0,LeaderEpoch:0,ControllerEpoch:1), [topic1,9] -> 
(Leader:2,ISR:2,0,LeaderEpoch:0,ControllerEpoch:1), [topic1,4] -> 
(Leader:1,ISR:1,LeaderEpoch:1,ControllerEpoch:2), [topic1,5] -> 
(Leader:1,ISR:1,2,LeaderEpoch:0,ControllerEpoch:1), [topic1,7] -> 
(Leader:2,ISR:2,LeaderEpoch:1,ControllerEpoch:2), [topic1,1] -> 
(Leader:0,ISR:0,2,LeaderEpoch:0,ControllerEpoch:1), [topic1,11] -> 
(Leader:1,ISR:1,2,LeaderEpoch:0,ControllerEpoch:1)) 
(kafka.controller.PartitionStateMachine)
[2014-05-23 10:30:57,377] DEBUG [OfflinePartitionLeaderSelector]: Some 
broker in ISR is alive for [topic1,1]. Select 2 from ISR 2 to be the 
leader. (kafka.controller.OfflinePartitionLeaderSelector)
[2014-05-23 10:30:57,377] INFO [OfflinePartitionLeaderSelector]: 
Selected new leader and ISR { "ISR":"2", "leader":"2", "leaderEpoch":"1" 
} for offline partition [topic1,1] 
(kafka.controller.OfflinePartitionLeaderSelector)
[2014-05-23 10:30:57,386] DEBUG [Partition state machine on Controller 
2]: After leader election, leader cache is updated to Map([topic1,0] -> 
(Leader:2,ISR:2,1,LeaderEpoch:0,ControllerEpoch:1), [topic1,3] -> 
(Leader:2,ISR:2,0,LeaderEpoch:0,ControllerEpoch:1), [topic1,10] -> 
(Leader:1,ISR:1,LeaderEpoch:1,ControllerEpoch:2), [topic1,8] -> 
(Leader:1,ISR:1,0,LeaderEpoch:0,ControllerEpoch:1), [topic1,6] -> 
(Leader:2,ISR:2,1,LeaderEpoch:0,ControllerEpoch:1), [topic1,2] -> 
(Leader:1,ISR:1,0,LeaderEpoch:0,ControllerEpoch:1), [topic1,9] -> 
(Leader:2,ISR:2,0,LeaderEpoch:0,ControllerEpoch:1), [topic1,4] -> 
(Leader:1,ISR:1,LeaderEpoch:1,ControllerEpoch:2), [topic1,5] -> 
(Leader:1,ISR:1,2,LeaderEpoch:0,ControllerEpoch:1), [topic1,7] -> 
(Leader:2,ISR:2,LeaderEpoch:1,ControllerEpoch:2), [topic1,1] -> 
(Leader:2,ISR:2,LeaderEpoch:1,ControllerEpoch:2), [topic1,11] -> 
(Leader:1,ISR:1,2,LeaderEpoch:0,ControllerEpoch:1)) 
(kafka.controller.PartitionStateMachine)
[2014-05-23 10:30:57,391] INFO [Partition state machine on Controller 
2]: Started partition state machine with initial state -> Map([topic1,0] 
-> OnlinePartition, [topic1,3] -> OnlinePartition, [topic1,10] -> 
OnlinePartition, [topic1,8] -> OnlinePartition, [topic1,6] -> 
OnlinePartition, [topic1,2] -> OnlinePartition, [topic1,9] -> 
OnlinePartition, [topic1,4] -> OnlinePartition, [topic1,5] -> 
OnlinePartition, [topic1,7] -> OnlinePartition, [topic1,1] -> 
OnlinePartition, [topic1,11] -> OnlinePartition) 
(kafka.controller.PartitionStateMachine)
[2014-05-23 10:30:57,405] INFO [Controller 2]: Broker 2 is ready to 
serve as the new controller with epoch 2 (kafka.controller.KafkaController)
[2014-05-23 10:30:57,408] INFO [Controller 2]: Partitions being 
reassigned: Map() (kafka.controller.KafkaController)
[2014-05-23 10:30:57,412] INFO [Controller 2]: Partitions already 
reassigned: List() (kafka.controller.KafkaController)
[2014-05-23 10:30:57,414] INFO [Controller 2]: Resuming reassignment of 
partitions: Map() (kafka.controller.KafkaController)
[2014-05-23 10:30:57,432] INFO [Controller 2]: Partitions undergoing 
preferred replica election: (kafka.controller.KafkaController)
[2014-05-23 10:30:57,433] INFO [Controller 2]: Partitions that completed 
preferred replica election: (kafka.controller.KafkaController)
[2014-05-23 10:30:57,435] INFO [Partition state machine on Controller 
2]: Invoking state change to OnlinePartition for partitions  
(kafka.controller.PartitionStateMachine)
[2014-05-23 10:30:57,519] DEBUG [ControllerEpochListener on 2]: 
Controller epoch listener fired with new epoch 2 
(kafka.controller.ControllerEpochListener)
[2014-05-23 10:30:57,520] INFO [ControllerEpochListener on 2]: 
Initialized controller epoch to 2 and zk version 1 
(kafka.controller.ControllerEpochListener)
[2014-05-23 12:19:38,719] DEBUG preRegister called. 
Server=com.sun.jmx.mbeanserver.JmxMBeanServer@2dcb25f1, 
name=log4j:logger=kafka.controller (kafka.controller)
[2014-05-23 12:19:48,239] INFO [ControllerEpochListener on 2]: 
Initialized controller epoch to 5 and zk version 4 
(kafka.controller.ControllerEpochListener)
[2014-05-23 12:19:48,552] INFO [Controller 2]: Controller starting up 
(kafka.controller.KafkaController)
[2014-05-23 12:19:48,910] INFO [Controller 2]: Controller startup 
complete (kafka.controller.KafkaController)
[2014-05-23 12:23:33,743] DEBUG preRegister called. 
Server=com.sun.jmx.mbeanserver.JmxMBeanServer@2dcb25f1, 
name=log4j:logger=kafka.controller (kafka.controller)
[2014-05-23 12:26:59,840] INFO [ControllerEpochListener on 2]: 
Initialized controller epoch to 5 and zk version 4 
(kafka.controller.ControllerEpochListener)
[2014-05-23 12:27:00,052] INFO [Controller 2]: Controller starting up 
(kafka.controller.KafkaController)
[2014-05-23 12:27:00,351] INFO [Controller 2]: Controller startup 
complete (kafka.controller.KafkaController)

And there is no log after this.

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
>>
>>
>>
>


Re: Broker not in ISR.

Posted by Guozhang Wang <wa...@gmail.com>.
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