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, <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 <j...@pyx.ch <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
>

Reply via email to