Re: Broker deregisters from ZK, but stays alive and does not rejoin the cluster

2019-03-23 Thread Joe Ammann
I've filed https://issues.apache.org/jira/browse/KAFKA-8151, and tried to keep 
the descriptions of the different symptoms apart.

I have yet to collect detailed information about a case of symptom 2 to happen.

And I will try 2.2RC later today

CU, Joe

On 3/23/19 1:17 AM, Ismael Juma wrote:
> I'd suggest filing a single JIRA as a first step. Please test the 2.2 RC 
> before filing if possible. Please include enough details for someone else to 
> reproduce.
> 
> Thanks!
> 
> Ismael
> 
> On Fri, Mar 22, 2019, 3:14 PM Joe Ammann mailto:j...@pyx.ch>> 
> wrote:
> 
> Hi Ismael
> 
> I've done a few more tests, and it seems that I'm able to "reproduce" 
> various kinds of problems in Kafka 2.1.1 in out DEV. I can force these by 
> faking an outage of Zookeeper. What I do for my tests is freeze (kill -STOP) 
> 2 out of 3 ZK instances, let the Kafka brokers continue, then thaw the ZK 
> instances (kill -CONT) and see what happens.
> 
> The ZK nodes always very quickly reunite and build a Quorum after thawing.
> 
> But the Kafka brokers (running on the same 3 Linux VMs) quite often show 
> problems after this procedure (most of the time they successfully re-register 
> and continue to work). I've seen 3 different kinds of problems (this is why I 
> put "reproduce" in quotes, I can never predict what will happen)
> 
> - the brokers get their ZK sessions expired (obviously) and sometimes 
> only 2 of 3 re-register under /brokers/ids. The 3rd broker doesn't 
> re-register for some reason (that's the problem I originally described)
> - the brokers all re-register and re-elect a new controller. But that new 
> controller does not fully work. For example it doesn't process partition 
> reassignment requests and or does not transfer partition leadership after I 
> kill a broker
> - the previous controller gets "dead-locked" (it has 3-4 of the important 
> controller threads in a lock) and hence does not perform any of it's 
> controller duties. But it regards itsself still as the valid controller and 
> is accepted by the other brokers
> 
> We have seen variants of these behaviours in TEST and PROD during the 
> last days. Of course there not provoked by kill -STOP, but rather by the 
> stalled underlying Linux VMs (we're heavily working on getting those replaced 
> by bare metal, but it may take some time).
> 
> Before I start filing JIRA's
> 
> - I feel this behaviour is so totally wierd, that I hardly can believe 
> it's Kafka bugs. They should have hit the community really hard and have been 
> uncovered quickly. So I'm rather guessing I'm doing something terribly wrong. 
> Any clue what that might be?
> - if I really start filing JIRA's should it rather be one single, or one 
> per error scenario
> 
> On 3/21/19 4:05 PM, Ismael Juma wrote:
> > Hi Joe,
> >
> > This is not expected behaviour, please file a JIRA.
> >
> > Ismael
> >
> > On Mon, Mar 18, 2019 at 7:29 AM Joe Ammann   >> wrote:
> >
> >     Hi all
> >
> >     We're running several clusters (mostly with 3 brokers) with 2.1.1
> >
> >     We quite regularly see the pattern that one of the 3 brokers 
> "detaches" from ZK (the broker id is not registered anymore under 
> /brokers/ids). We assume that the root cause for this is that the brokers are 
> running on VMs (due to company policy, no alternative) and that the VM gets 
> "stalled" for several minutes due to missing resources on the VMware ESX host.
> >
> >     This is not new behaviour with 2.1.1, we already saw it with 
> 0.10.2.1 before.
> 
> 
> -- 
> CU, Joe
> 


-- 
CU, Joe


Re: Broker deregisters from ZK, but stays alive and does not rejoin the cluster

2019-03-23 Thread Peter Levart
Hi, Joe

I think I observed a similar lockup as you describe in 3rd variant. The
controller broker was partialy stuck but other brokers still regarded it as
the controller. Unfortunately the broker was restarted by an unpatient
admin before I had a chance to investigate. The simpthoms were as follows:
- producer clients blocked in obtaining metadata about partitions from
brokers
- other brokers experienced socket connect timeouts while trying to contact
the troubling broker
- other brokers did not isolate the troubling broker.

So It may be that you're not alone in experiencing these problems.

Regards, Peter

On Fri, 22 Mar 2019, 23:15 Joe Ammann,  wrote:

> Hi Ismael
>
> I've done a few more tests, and it seems that I'm able to "reproduce"
> various kinds of problems in Kafka 2.1.1 in out DEV. I can force these by
> faking an outage of Zookeeper. What I do for my tests is freeze (kill
> -STOP) 2 out of 3 ZK instances, let the Kafka brokers continue, then thaw
> the ZK instances (kill -CONT) and see what happens.
>
> The ZK nodes always very quickly reunite and build a Quorum after thawing.
>
> But the Kafka brokers (running on the same 3 Linux VMs) quite often show
> problems after this procedure (most of the time they successfully
> re-register and continue to work). I've seen 3 different kinds of problems
> (this is why I put "reproduce" in quotes, I can never predict what will
> happen)
>
> - the brokers get their ZK sessions expired (obviously) and sometimes only
> 2 of 3 re-register under /brokers/ids. The 3rd broker doesn't re-register
> for some reason (that's the problem I originally described)
> - the brokers all re-register and re-elect a new controller. But that new
> controller does not fully work. For example it doesn't process partition
> reassignment requests and or does not transfer partition leadership after I
> kill a broker
> - the previous controller gets "dead-locked" (it has 3-4 of the important
> controller threads in a lock) and hence does not perform any of it's
> controller duties. But it regards itsself still as the valid controller and
> is accepted by the other brokers
>
> We have seen variants of these behaviours in TEST and PROD during the last
> days. Of course there not provoked by kill -STOP, but rather by the stalled
> underlying Linux VMs (we're heavily working on getting those replaced by
> bare metal, but it may take some time).
>
> Before I start filing JIRA's
>
> - I feel this behaviour is so totally wierd, that I hardly can believe
> it's Kafka bugs. They should have hit the community really hard and have
> been uncovered quickly. So I'm rather guessing I'm doing something terribly
> wrong. Any clue what that might be?
> - if I really start filing JIRA's should it rather be one single, or one
> per error scenario
>
> On 3/21/19 4:05 PM, Ismael Juma wrote:
> > Hi Joe,
> >
> > This is not expected behaviour, please file a JIRA.
> >
> > Ismael
> >
> > On Mon, Mar 18, 2019 at 7:29 AM Joe Ammann  j...@pyx.ch>> wrote:
> >
> > Hi all
> >
> > We're running several clusters (mostly with 3 brokers) with 2.1.1
> >
> > We quite regularly see the pattern that one of the 3 brokers
> "detaches" from ZK (the broker id is not registered anymore under
> /brokers/ids). We assume that the root cause for this is that the brokers
> are running on VMs (due to company policy, no alternative) and that the VM
> gets "stalled" for several minutes due to missing resources on the VMware
> ESX host.
> >
> > This is not new behaviour with 2.1.1, we already saw it with
> 0.10.2.1 before.
>
>
> --
> CU, Joe
>


Re: Broker deregisters from ZK, but stays alive and does not rejoin the cluster

2019-03-22 Thread Ismael Juma
I'd suggest filing a single JIRA as a first step. Please test the 2.2 RC
before filing if possible. Please include enough details for someone else
to reproduce.

Thanks!

Ismael

On Fri, Mar 22, 2019, 3:14 PM Joe Ammann  wrote:

> Hi Ismael
>
> I've done a few more tests, and it seems that I'm able to "reproduce"
> various kinds of problems in Kafka 2.1.1 in out DEV. I can force these by
> faking an outage of Zookeeper. What I do for my tests is freeze (kill
> -STOP) 2 out of 3 ZK instances, let the Kafka brokers continue, then thaw
> the ZK instances (kill -CONT) and see what happens.
>
> The ZK nodes always very quickly reunite and build a Quorum after thawing.
>
> But the Kafka brokers (running on the same 3 Linux VMs) quite often show
> problems after this procedure (most of the time they successfully
> re-register and continue to work). I've seen 3 different kinds of problems
> (this is why I put "reproduce" in quotes, I can never predict what will
> happen)
>
> - the brokers get their ZK sessions expired (obviously) and sometimes only
> 2 of 3 re-register under /brokers/ids. The 3rd broker doesn't re-register
> for some reason (that's the problem I originally described)
> - the brokers all re-register and re-elect a new controller. But that new
> controller does not fully work. For example it doesn't process partition
> reassignment requests and or does not transfer partition leadership after I
> kill a broker
> - the previous controller gets "dead-locked" (it has 3-4 of the important
> controller threads in a lock) and hence does not perform any of it's
> controller duties. But it regards itsself still as the valid controller and
> is accepted by the other brokers
>
> We have seen variants of these behaviours in TEST and PROD during the last
> days. Of course there not provoked by kill -STOP, but rather by the stalled
> underlying Linux VMs (we're heavily working on getting those replaced by
> bare metal, but it may take some time).
>
> Before I start filing JIRA's
>
> - I feel this behaviour is so totally wierd, that I hardly can believe
> it's Kafka bugs. They should have hit the community really hard and have
> been uncovered quickly. So I'm rather guessing I'm doing something terribly
> wrong. Any clue what that might be?
> - if I really start filing JIRA's should it rather be one single, or one
> per error scenario
>
> On 3/21/19 4:05 PM, Ismael Juma wrote:
> > Hi Joe,
> >
> > This is not expected behaviour, please file a JIRA.
> >
> > Ismael
> >
> > On Mon, Mar 18, 2019 at 7:29 AM Joe Ammann  j...@pyx.ch>> wrote:
> >
> > Hi all
> >
> > We're running several clusters (mostly with 3 brokers) with 2.1.1
> >
> > We quite regularly see the pattern that one of the 3 brokers
> "detaches" from ZK (the broker id is not registered anymore under
> /brokers/ids). We assume that the root cause for this is that the brokers
> are running on VMs (due to company policy, no alternative) and that the VM
> gets "stalled" for several minutes due to missing resources on the VMware
> ESX host.
> >
> > This is not new behaviour with 2.1.1, we already saw it with
> 0.10.2.1 before.
>
>
> --
> CU, Joe
>


Re: Broker deregisters from ZK, but stays alive and does not rejoin the cluster

2019-03-22 Thread Joe Ammann
Hi Ismael

I've done a few more tests, and it seems that I'm able to "reproduce" various 
kinds of problems in Kafka 2.1.1 in out DEV. I can force these by faking an 
outage of Zookeeper. What I do for my tests is freeze (kill -STOP) 2 out of 3 
ZK instances, let the Kafka brokers continue, then thaw the ZK instances (kill 
-CONT) and see what happens.

The ZK nodes always very quickly reunite and build a Quorum after thawing.

But the Kafka brokers (running on the same 3 Linux VMs) quite often show 
problems after this procedure (most of the time they successfully re-register 
and continue to work). I've seen 3 different kinds of problems (this is why I 
put "reproduce" in quotes, I can never predict what will happen)

- the brokers get their ZK sessions expired (obviously) and sometimes only 2 of 
3 re-register under /brokers/ids. The 3rd broker doesn't re-register for some 
reason (that's the problem I originally described)
- the brokers all re-register and re-elect a new controller. But that new 
controller does not fully work. For example it doesn't process partition 
reassignment requests and or does not transfer partition leadership after I 
kill a broker
- the previous controller gets "dead-locked" (it has 3-4 of the important 
controller threads in a lock) and hence does not perform any of it's controller 
duties. But it regards itsself still as the valid controller and is accepted by 
the other brokers

We have seen variants of these behaviours in TEST and PROD during the last 
days. Of course there not provoked by kill -STOP, but rather by the stalled 
underlying Linux VMs (we're heavily working on getting those replaced by bare 
metal, but it may take some time).

Before I start filing JIRA's

- I feel this behaviour is so totally wierd, that I hardly can believe it's 
Kafka bugs. They should have hit the community really hard and have been 
uncovered quickly. So I'm rather guessing I'm doing something terribly wrong. 
Any clue what that might be?
- if I really start filing JIRA's should it rather be one single, or one per 
error scenario

On 3/21/19 4:05 PM, Ismael Juma wrote:
> Hi Joe,
> 
> This is not expected behaviour, please file a JIRA.
> 
> Ismael
> 
> On Mon, Mar 18, 2019 at 7:29 AM Joe Ammann mailto:j...@pyx.ch>> 
> wrote:
> 
> Hi all
> 
> We're running several clusters (mostly with 3 brokers) with 2.1.1
> 
> We quite regularly see the pattern that one of the 3 brokers "detaches" 
> from ZK (the broker id is not registered anymore under /brokers/ids). We 
> assume that the root cause for this is that the brokers are running on VMs 
> (due to company policy, no alternative) and that the VM gets "stalled" for 
> several minutes due to missing resources on the VMware ESX host.
> 
> This is not new behaviour with 2.1.1, we already saw it with 0.10.2.1 
> before.


-- 
CU, Joe


Re: Broker deregisters from ZK, but stays alive and does not rejoin the cluster

2019-03-21 Thread Ismael Juma
Hi Joe,

This is not expected behaviour, please file a JIRA.

Ismael

On Mon, Mar 18, 2019 at 7:29 AM Joe Ammann  wrote:

> Hi all
>
> We're running several clusters (mostly with 3 brokers) with 2.1.1
>
> We quite regularly see the pattern that one of the 3 brokers "detaches"
> from ZK (the broker id is not registered anymore under /brokers/ids). We
> assume that the root cause for this is that the brokers are running on VMs
> (due to company policy, no alternative) and that the VM gets "stalled" for
> several minutes due to missing resources on the VMware ESX host.
>
> This is not new behaviour with 2.1.1, we already saw it with 0.10.2.1
> before.
>
> The sequence of events is normally something like the following
> - cluster is running ok
> - one broker "gets stalled", not pingable anymore
> - partitions go to underreplicated
> - failed broker comes back and reports that ZK session was expired [1]
> - some of the brokers that were ok report leader election problems [2]
> - the failed/revived broker logs errors continuosly about expired session
> [3]
>
> This goes on, until we restart the broker on the VM that had failed. Until
> we do this restart, the failed broker seems to think it is working
> perfectly ok. We're monitoring all brokers via JMX, and that one does not
> report any problems. It claims in the JMX values to be the leader of a
> number of partitions, and have 0 underreplicated partitions. Whilst the
> other brokers rightfully report via JMX that they in turn do have
> underreplicate paritions. This then causes alerts to go off about the
> brokers that still work in degraded mode, while the one that is really
> broken appears green/ok.
>
> Is this in any way expected behaviour? That a Kafka broker gets its ZK
> session expired but continues to run (just issues the errors in [3]). I
> would have expected that the broker would shutdown itsself in a similar
> manner it does when it's unable to register with ZK on startup.
>
> Any idea how I could best detect this situation in monitoring? I'm
> thinking about after polling the broker via JMX, I also poll ZK to check if
> /brokers/ids/ node exists. If not, restart that broker.
>
> BTW: I do know that probably the best answer is: "just run your ZK/Kafka
> on hardware, not VMs". We're working on that, but company policies seem to
> prefer outages over spending a little money).
>
> --
> CU, Joe
>
> [1]
>
> [2019-03-18 02:27:13,043] INFO [ZooKeeperClient] Session expired.
> (kafka.zookeeper.ZooKeeperClient)
>
> [2]
>
> [2019-03-18 02:27:20,283] ERROR [Controller id=3 epoch=94562] Controller 3
> epoch 94562 failed to change state for partition __consumer_offsets-4 from
> OnlinePartition to OnlinePartition (state.change.logger)
> kafka.common.StateChangeFailedException: Failed to elect leader for
> partition __consumer_offsets-4 under strategy
> PreferredReplicaPartitionLeaderElectionStrategy
> at
> kafka.controller.PartitionStateMachine$$anonfun$doElectLeaderForPartitions$3.apply(PartitionStateMachine.scala:366)
> at
> kafka.controller.PartitionStateMachine$$anonfun$doElectLeaderForPartitions$3.apply(PartitionStateMachine.scala:364)
> at
> scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
> at
> scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
> at
> kafka.controller.PartitionStateMachine.doElectLeaderForPartitions(PartitionStateMachine.scala:364)
> at
> kafka.controller.PartitionStateMachine.electLeaderForPartitions(PartitionStateMachine.scala:292)
> at
> kafka.controller.PartitionStateMachine.doHandleStateChanges(PartitionStateMachine.scala:210)
> at
> kafka.controller.PartitionStateMachine.handleStateChanges(PartitionStateMachine.scala:133)
> at
> kafka.controller.KafkaController.kafka$controller$KafkaController$$onPreferredReplicaElection(KafkaController.scala:624)
> at
> kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerAutoLeaderRebalance$3.apply(KafkaController.scala:974)
> at
> kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerAutoLeaderRebalance$3.apply(KafkaController.scala:955)
> at scala.collection.immutable.Map$Map4.foreach(Map.scala:188)
> at
> kafka.controller.KafkaController.kafka$controller$KafkaController$$checkAndTriggerAutoLeaderRebalance(KafkaController.scala:955)
> at
> kafka.controller.KafkaController$AutoPreferredReplicaLeaderElection$.process(KafkaController.scala:986)
> at
> kafka.controller.ControllerEventManager$ControllerEventThread$$anonfun$doWork$1.apply$mcV$sp(ControllerEventManager.scala:89)
> at
> kafka.controller.ControllerEventManager$ControllerEventThread$$anonfun$doWork$1.apply(ControllerEventManager.scala:89)
> at
> kafka.controller.ControllerEventManager$ControllerEventThread$$anonfun$doWork$1.apply(ControllerEventManager.scala:89)
> at 

Re: Broker deregisters from ZK, but stays alive and does not rejoin the cluster

2019-03-21 Thread Joe Ammann
Hi all

I investigated a bit deeper and I came to the conclusion that it's probably 
expected behaviour, that a broker keeps running after loosing the ZK session 
and does not necessarily restart or reconnect automatically. 
https://cwiki-test.apache.org/confluence/display/KAFKA/KIP-217%3A+Expose+a+timeout+to+allow+an+expired+ZK+session+to+be+re-created
 and tickets related to that KIP reflect that it is probably better to keep it 
running even without ZK connectivity.

In the meantime, I also found a JMX property (kafka.server -> 
SessionExpireListener -> SessionState) which reflects exactly this state. We'll 
add that to our monitoring and restart a broker if the state is CLOSED, but 
Zookeeper is available again.

CU, Joe

On 3/18/19 3:29 PM, Joe Ammann wrote:
> Hi all
> 
> We're running several clusters (mostly with 3 brokers) with 2.1.1
> 
> We quite regularly see the pattern that one of the 3 brokers "detaches" from 
> ZK (the broker id is not registered anymore under /brokers/ids). We assume 
> that the root cause for this is that the brokers are running on VMs (due to 
> company policy, no alternative) and that the VM gets "stalled" for several 
> minutes due to missing resources on the VMware ESX host.
> 
> This is not new behaviour with 2.1.1, we already saw it with 0.10.2.1 before.
> 
> The sequence of events is normally something like the following
> - cluster is running ok
> - one broker "gets stalled", not pingable anymore
> - partitions go to underreplicated
> - failed broker comes back and reports that ZK session was expired [1]
> - some of the brokers that were ok report leader election problems [2]
> - the failed/revived broker logs errors continuosly about expired session [3]
> 
> This goes on, until we restart the broker on the VM that had failed. Until we 
> do this restart, the failed broker seems to think it is working perfectly ok. 
> We're monitoring all brokers via JMX, and that one does not report any 
> problems. It claims in the JMX values to be the leader of a number of 
> partitions, and have 0 underreplicated partitions. Whilst the other brokers 
> rightfully report via JMX that they in turn do have underreplicate paritions. 
> This then causes alerts to go off about the brokers that still work in 
> degraded mode, while the one that is really broken appears green/ok.
> 
> Is this in any way expected behaviour? That a Kafka broker gets its ZK 
> session expired but continues to run (just issues the errors in [3]). I would 
> have expected that the broker would shutdown itsself in a similar manner it 
> does when it's unable to register with ZK on startup.
> 
> Any idea how I could best detect this situation in monitoring? I'm thinking 
> about after polling the broker via JMX, I also poll ZK to check if 
> /brokers/ids/ node exists. If not, restart that broker.
> 
> BTW: I do know that probably the best answer is: "just run your ZK/Kafka on 
> hardware, not VMs". We're working on that, but company policies seem to 
> prefer outages over spending a little money).


Broker deregisters from ZK, but stays alive and does not rejoin the cluster

2019-03-18 Thread Joe Ammann
Hi all

We're running several clusters (mostly with 3 brokers) with 2.1.1

We quite regularly see the pattern that one of the 3 brokers "detaches" from ZK 
(the broker id is not registered anymore under /brokers/ids). We assume that 
the root cause for this is that the brokers are running on VMs (due to company 
policy, no alternative) and that the VM gets "stalled" for several minutes due 
to missing resources on the VMware ESX host.

This is not new behaviour with 2.1.1, we already saw it with 0.10.2.1 before.

The sequence of events is normally something like the following
- cluster is running ok
- one broker "gets stalled", not pingable anymore
- partitions go to underreplicated
- failed broker comes back and reports that ZK session was expired [1]
- some of the brokers that were ok report leader election problems [2]
- the failed/revived broker logs errors continuosly about expired session [3]

This goes on, until we restart the broker on the VM that had failed. Until we 
do this restart, the failed broker seems to think it is working perfectly ok. 
We're monitoring all brokers via JMX, and that one does not report any 
problems. It claims in the JMX values to be the leader of a number of 
partitions, and have 0 underreplicated partitions. Whilst the other brokers 
rightfully report via JMX that they in turn do have underreplicate paritions. 
This then causes alerts to go off about the brokers that still work in degraded 
mode, while the one that is really broken appears green/ok.

Is this in any way expected behaviour? That a Kafka broker gets its ZK session 
expired but continues to run (just issues the errors in [3]). I would have 
expected that the broker would shutdown itsself in a similar manner it does 
when it's unable to register with ZK on startup.

Any idea how I could best detect this situation in monitoring? I'm thinking 
about after polling the broker via JMX, I also poll ZK to check if 
/brokers/ids/ node exists. If not, restart that broker.

BTW: I do know that probably the best answer is: "just run your ZK/Kafka on 
hardware, not VMs". We're working on that, but company policies seem to prefer 
outages over spending a little money).

-- 
CU, Joe

[1]

[2019-03-18 02:27:13,043] INFO [ZooKeeperClient] Session expired. 
(kafka.zookeeper.ZooKeeperClient)

[2]

[2019-03-18 02:27:20,283] ERROR [Controller id=3 epoch=94562] Controller 3 
epoch 94562 failed to change state for partition __consumer_offsets-4 from 
OnlinePartition to OnlinePartition (state.change.logger) 
kafka.common.StateChangeFailedException: Failed to elect leader for partition 
__consumer_offsets-4 under strategy 
PreferredReplicaPartitionLeaderElectionStrategy
at 
kafka.controller.PartitionStateMachine$$anonfun$doElectLeaderForPartitions$3.apply(PartitionStateMachine.scala:366)
at 
kafka.controller.PartitionStateMachine$$anonfun$doElectLeaderForPartitions$3.apply(PartitionStateMachine.scala:364)
at 
scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
at 
kafka.controller.PartitionStateMachine.doElectLeaderForPartitions(PartitionStateMachine.scala:364)
at 
kafka.controller.PartitionStateMachine.electLeaderForPartitions(PartitionStateMachine.scala:292)
at 
kafka.controller.PartitionStateMachine.doHandleStateChanges(PartitionStateMachine.scala:210)
at 
kafka.controller.PartitionStateMachine.handleStateChanges(PartitionStateMachine.scala:133)
at 
kafka.controller.KafkaController.kafka$controller$KafkaController$$onPreferredReplicaElection(KafkaController.scala:624)
at 
kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerAutoLeaderRebalance$3.apply(KafkaController.scala:974)
at 
kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerAutoLeaderRebalance$3.apply(KafkaController.scala:955)
at scala.collection.immutable.Map$Map4.foreach(Map.scala:188)
at 
kafka.controller.KafkaController.kafka$controller$KafkaController$$checkAndTriggerAutoLeaderRebalance(KafkaController.scala:955)
at 
kafka.controller.KafkaController$AutoPreferredReplicaLeaderElection$.process(KafkaController.scala:986)
at 
kafka.controller.ControllerEventManager$ControllerEventThread$$anonfun$doWork$1.apply$mcV$sp(ControllerEventManager.scala:89)
at 
kafka.controller.ControllerEventManager$ControllerEventThread$$anonfun$doWork$1.apply(ControllerEventManager.scala:89)
at 
kafka.controller.ControllerEventManager$ControllerEventThread$$anonfun$doWork$1.apply(ControllerEventManager.scala:89)
at kafka.metrics.KafkaTimer.time(KafkaTimer.scala:31)
at 
kafka.controller.ControllerEventManager$ControllerEventThread.doWork(ControllerEventManager.scala:88)
at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:82)
 
[3]

[2019-03-18