Do you roll the controller last?

I suspect this is more to do with the way you are rolling the cluster (which I 
am still not clear on the need for) rather than some kind of bug in Kafka 
(though that could of course be the case).

Tom

On 04/03/2022 01:59, Dhirendra Singh wrote:

> Hi Tom,
> During the rolling restart we check for under replicated partition count to 
> be zero in the readiness probe before restarting the next POD in order.
> This issue never occurred before. It started after we upgraded kafka version 
> from 2.5.0 to 2.7.1.
> So i suspect some bug introduced in the version after 2.5.0.
>
> Thanks,
> Dhirendra.
>
> On Thu, Mar 3, 2022 at 11:09 PM Thomas Cooper <c...@tomcooper.dev> wrote:
>
>> I suspect this nightly rolling will have something to do with your issues. 
>> If you are just rolling the stateful set in order, with no dependence on 
>> maintaining minISR and other Kafka considerations you are going to hit 
>> issues.
>>
>> If you are running on Kubernetes I would suggest using an Operator like 
>> [Strimzi](https://strimzi.io/) which will do a lot of the Kafka admin tasks 
>> like this for you automatically.
>>
>> Tom
>>
>> On 03/03/2022 16:28, Dhirendra Singh wrote:
>>
>>> Hi Tom,
>>> Doing the nightly restart is the decision of the cluster admin. I have no 
>>> control on it.
>>> We have implementation using stateful set. restart is triggered by updating 
>>> a annotation in the pod.
>>> Issue is not triggered by kafka cluster restart but the zookeeper servers 
>>> restart.
>>> Thanks,
>>> Dhirendra.
>>>
>>> On Thu, Mar 3, 2022 at 7:19 PM Thomas Cooper <c...@tomcooper.dev> wrote:
>>>
>>>> Hi Dhirenda,
>>>>
>>>> Firstly, I am interested in why are you restarting the ZK and Kafka 
>>>> cluster every night?
>>>>
>>>> Secondly, how are you doing the restarts. For example, in 
>>>> [Strimzi](https://strimzi.io/), when we roll the Kafka cluster we leave 
>>>> the designated controller broker until last. For each of the other brokers 
>>>> we wait until all the partitions they are leaders for are above their 
>>>> minISR and then we roll the broker. In this way we maintain availability 
>>>> and make sure leadership can move off the rolling broker temporarily.
>>>>
>>>> Cheers,
>>>>
>>>> Tom Cooper
>>>>
>>>> [@tomncooper](https://twitter.com/tomncooper) | https://tomcooper.dev
>>>>
>>>> On 03/03/2022 07:38, Dhirendra Singh wrote:
>>>>
>>>>> Hi All,
>>>>>
>>>>> We have kafka cluster running in kubernetes. kafka version we are using is
>>>>> 2.7.1.
>>>>> Every night zookeeper servers and kafka brokers are restarted.
>>>>> After the nightly restart of the zookeeper servers some partitions remain
>>>>> stuck in under replication. This happens randomly but not at every nightly
>>>>> restart.
>>>>> Partitions remain under replicated until kafka broker with the partition
>>>>> leader is restarted.
>>>>> For example partition 4 of consumer_offsets topic remain under replicated
>>>>> and we see following error in the log...
>>>>>
>>>>> [2022-02-28 04:01:20,217] WARN [Partition __consumer_offsets-4 broker=1]
>>>>> Controller failed to update ISR to PendingExpandIsr(isr=Set(1),
>>>>> newInSyncReplicaId=2) due to unexpected UNKNOWN_SERVER_ERROR. Retrying.
>>>>> (kafka.cluster.Partition)
>>>>> [2022-02-28 04:01:20,217] ERROR [broker-1-to-controller] Uncaught error in
>>>>> request completion: (org.apache.kafka.clients.NetworkClient)
>>>>> java.lang.IllegalStateException: Failed to enqueue `AlterIsr` request with
>>>>> state LeaderAndIsr(leader=1, leaderEpoch=2728, isr=List(1, 2),
>>>>> zkVersion=4719) for partition __consumer_offsets-4
>>>>> at kafka.cluster.Partition.sendAlterIsrRequest(Partition.scala:1403)
>>>>> at
>>>>> kafka.cluster.Partition.$anonfun$handleAlterIsrResponse$1(Partition.scala:1438)
>>>>> at kafka.cluster.Partition.handleAlterIsrResponse(Partition.scala:1417)
>>>>> at
>>>>> kafka.cluster.Partition.$anonfun$sendAlterIsrRequest$1(Partition.scala:1398)
>>>>> at
>>>>> kafka.cluster.Partition.$anonfun$sendAlterIsrRequest$1$adapted(Partition.scala:1398)
>>>>> at
>>>>> kafka.server.AlterIsrManagerImpl.$anonfun$handleAlterIsrResponse$8(AlterIsrManager.scala:166)
>>>>> at
>>>>> kafka.server.AlterIsrManagerImpl.$anonfun$handleAlterIsrResponse$8$adapted(AlterIsrManager.scala:163)
>>>>> at scala.collection.immutable.List.foreach(List.scala:333)
>>>>> at
>>>>> kafka.server.AlterIsrManagerImpl.handleAlterIsrResponse(AlterIsrManager.scala:163)
>>>>> at
>>>>> kafka.server.AlterIsrManagerImpl.responseHandler$1(AlterIsrManager.scala:94)
>>>>> at
>>>>> kafka.server.AlterIsrManagerImpl.$anonfun$sendRequest$2(AlterIsrManager.scala:104)
>>>>> at
>>>>> kafka.server.BrokerToControllerRequestThread.handleResponse(BrokerToControllerChannelManagerImpl.scala:175)
>>>>> at
>>>>> kafka.server.BrokerToControllerRequestThread.$anonfun$generateRequests$1(BrokerToControllerChannelManagerImpl.scala:158)
>>>>> at
>>>>> org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:109)
>>>>> at
>>>>> org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:586)
>>>>> at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:578)
>>>>> at 
>>>>> kafka.common.InterBrokerSendThread.doWork(InterBrokerSendThread.scala:71)
>>>>> at
>>>>> kafka.server.BrokerToControllerRequestThread.doWork(BrokerToControllerChannelManagerImpl.scala:183)
>>>>> at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:96)
>>>>> Looks like some kind of race condition bug...anyone has any idea ?
>>>>>
>>>>> Thanks,
>>>>> Dhirendra
>>
>> --
>>
>> Tom Cooper
>>
>> [@tomncooper](https://twitter.com/tomncooper) | tomcooper.dev

--

Tom Cooper

[@tomncooper](https://twitter.com/tomncooper) | tomcooper.dev

Reply via email to