Some more information split brain issue is happening with the controller. when brokers (including the active controller) lose connection with zookeeper, for few seconds 2 brokers are the active controller. following is the log of broker 1 and broker 0. At the time when connection to zookeeper was lost broker 1 was the active controller
Broker 0 log: [2022-03-14 04:02:29,813] WARN Session 0x3003837bae70005 for server zookeeper.svc.cluster.local/172.30.252.43:2181, unexpected error, closing socket connection and attempting reconnect (org.apache.zookeeper.ClientCnxn) java.io.IOException: Connection reset by peer at java.base/sun.nio.ch.FileDispatcherImpl.read0(Native Method) at java.base/sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:39) at java.base/sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:276) at java.base/sun.nio.ch.IOUtil.read(IOUtil.java:233) at java.base/sun.nio.ch.IOUtil.read(IOUtil.java:223) at java.base/sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:358) at org.apache.zookeeper.ClientCnxnSocketNIO.doIO(ClientCnxnSocketNIO.java:77) at org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:365) at org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1223) [2022-03-14 04:02:29,816] INFO Unable to read additional data from server sessionid 0x3003a92cdbb0000, likely server has closed socket, closing socket connection and attempting reconnect (org.apache.zookeeper.ClientCnxn) [2022-03-14 04:02:31,333] INFO Opening socket connection to server zookeeper.svc.cluster.local/172.30.252.43:2181. Will not attempt to authenticate using SASL (unknown error) (org.apache.zookeeper.ClientCnxn) [2022-03-14 04:02:31,335] INFO Socket connection established, initiating session, client: /10.130.96.38:34308, server: zookeeper.svc.cluster.local/ 172.30.252.43:2181 (org.apache.zookeeper.ClientCnxn) [2022-03-14 04:02:31,349] INFO Session establishment complete on server zookeeper.svc.cluster.local/172.30.252.43:2181, sessionid = 0x3003a92cdbb0000, negotiated timeout = 4000 (org.apache.zookeeper.ClientCnxn) [2022-03-14 04:02:31,710] INFO Opening socket connection to server zookeeper.svc.cluster.local/172.30.252.43:2181. Will not attempt to authenticate using SASL (unknown error) (org.apache.zookeeper.ClientCnxn) [2022-03-14 04:02:34,315] INFO [Controller id=0] 0 successfully elected as the controller. Epoch incremented to 913 and epoch zk version is now 913 (kafka.controller.KafkaController) [2022-03-14 04:02:36,466] WARN [Partition __consumer_offsets-0 broker=0] Failed to update ISR to PendingExpandIsr(isr=Set(0), newInSyncReplicaId=2) due to unexpected UNKNOWN_SERVER_ERROR. Retrying. (kafka.cluster.Partition) [2022-03-14 04:02:36,467] ERROR [broker-0-to-controller] Uncaught error in request completion: (org.apache.kafka.clients.NetworkClient) java.lang.IllegalStateException: Failed to enqueue ISR change state LeaderAndIsr(leader=0, leaderEpoch=2995, isr=List(0, 2), zkVersion=5163) for partition __consumer_offsets-0 at kafka.cluster.Partition.sendAlterIsrRequest(Partition.scala:1379) at kafka.cluster.Partition.$anonfun$handleAlterIsrResponse$1(Partition.scala:1413) at kafka.cluster.Partition.handleAlterIsrResponse(Partition.scala:1392) at kafka.cluster.Partition.$anonfun$sendAlterIsrRequest$1(Partition.scala:1370) at kafka.cluster.Partition.$anonfun$sendAlterIsrRequest$1$adapted(Partition.scala:1370) at kafka.server.DefaultAlterIsrManager.$anonfun$handleAlterIsrResponse$8(AlterIsrManager.scala:262) at kafka.server.DefaultAlterIsrManager.$anonfun$handleAlterIsrResponse$8$adapted(AlterIsrManager.scala:259) at scala.collection.immutable.List.foreach(List.scala:333) at kafka.server.DefaultAlterIsrManager.handleAlterIsrResponse(AlterIsrManager.scala:259) at kafka.server.DefaultAlterIsrManager$$anon$1.onComplete(AlterIsrManager.scala:179) at kafka.server.BrokerToControllerRequestThread.handleResponse(BrokerToControllerChannelManager.scala:362) at kafka.server.BrokerToControllerRequestThread.$anonfun$generateRequests$1(BrokerToControllerChannelManager.scala:333) at org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:109) at org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:584) at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:576) at kafka.common.InterBrokerSendThread.pollOnce(InterBrokerSendThread.scala:74) at kafka.server.BrokerToControllerRequestThread.doWork(BrokerToControllerChannelManager.scala:368) at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:96) [2022-03-14 04:02:36,467] WARN [Broker id=0] Received update metadata request with correlation id 20 from an old controller 1 with epoch 912. Latest known controller epoch is 913 (state.change.logger) Broker 1 log: [2022-03-14 04:02:29,815] INFO Unable to read additional data from server sessionid 0x3003a92cdbb0002, likely server has closed socket, closing socket connection and attempting reconnect (org.apache.zookeeper.ClientCnxn) [2022-03-14 04:02:29,815] INFO Unable to read additional data from server sessionid 0x10037c8c66b0001, likely server has closed socket, closing socket connection and attempting reconnect (org.apache.zookeeper.ClientCnxn) [2022-03-14 04:02:31,715] INFO Opening socket connection to server zookeeper.svc.cluster.local/172.30.252.43:2181. Will not attempt to authenticate using SASL (unknown error) (org.apache.zookeeper.ClientCnxn) [2022-03-14 04:02:31,910] INFO Opening socket connection to server zookeeper.svc.cluster.local/172.30.252.43:2181. Will not attempt to authenticate using SASL (unknown error) (org.apache.zookeeper.ClientCnxn) [2022-03-14 04:02:34,722] INFO Socket error occurred: zookeeper.svc.cluster.local/172.30.252.43:2181: No route to host (org.apache.zookeeper.ClientCnxn) [2022-03-14 04:02:34,724] INFO Socket error occurred: zookeeper.svc.cluster.local/172.30.252.43:2181: No route to host (org.apache.zookeeper.ClientCnxn) [2022-03-14 04:02:35,177] DEBUG [Controller id=1] Updating ISRs for partitions: Set(__consumer_offsets-0). (kafka.controller.KafkaController) [2022-03-14 04:02:36,172] INFO Opening socket connection to server zookeeper.svc.cluster.local/172.30.252.43:2181. Will not attempt to authenticate using SASL (unknown error) (org.apache.zookeeper.ClientCnxn) [2022-03-14 04:02:36,174] INFO Socket connection established, initiating session, client: /10.130.72.182:43696, server: zookeeper.svc.cluster.local/ 172.30.252.43:2181 (org.apache.zookeeper.ClientCnxn) [2022-03-14 04:02:36,181] WARN Unable to reconnect to ZooKeeper service, session 0x3003a92cdbb0002 has expired (org.apache.zookeeper.ClientCnxn) [2022-03-14 04:02:36,181] INFO Unable to reconnect to ZooKeeper service, session 0x3003a92cdbb0002 has expired, closing socket connection (org.apache.zookeeper.ClientCnxn) [2022-03-14 04:02:36,181] INFO EventThread shut down for session: 0x3003a92cdbb0002 (org.apache.zookeeper.ClientCnxn) [2022-03-14 04:02:36,182] INFO [ZooKeeperClient Kafka server] Session expired. (kafka.zookeeper.ZooKeeperClient) [2022-03-14 04:02:36,454] ERROR [Controller id=1] Failed to update ISR for partition __consumer_offsets-0 (kafka.controller.KafkaController) org.apache.zookeeper.KeeperException$SessionExpiredException: KeeperErrorCode = Session expired at org.apache.zookeeper.KeeperException.create(KeeperException.java:134) at kafka.zk.KafkaZkClient$.kafka$zk$KafkaZkClient$$unwrapResponseWithControllerEpochCheck(KafkaZkClient.scala:2008) at kafka.zk.KafkaZkClient.$anonfun$retryRequestsUntilConnected$2(KafkaZkClient.scala:1770) at scala.collection.StrictOptimizedIterableOps.map(StrictOptimizedIterableOps.scala:99) at scala.collection.StrictOptimizedIterableOps.map$(StrictOptimizedIterableOps.scala:86) at scala.collection.mutable.ArrayBuffer.map(ArrayBuffer.scala:42) at kafka.zk.KafkaZkClient.retryRequestsUntilConnected(KafkaZkClient.scala:1770) at kafka.zk.KafkaZkClient.setTopicPartitionStatesRaw(KafkaZkClient.scala:204) at kafka.zk.KafkaZkClient.updateLeaderAndIsr(KafkaZkClient.scala:262) at kafka.controller.KafkaController.processAlterIsr(KafkaController.scala:2338) at kafka.controller.KafkaController.process(KafkaController.scala:2468) at kafka.controller.QueuedEvent.process(ControllerEventManager.scala:52) at kafka.controller.ControllerEventManager$ControllerEventThread.process$1(ControllerEventManager.scala:130) at kafka.controller.ControllerEventManager$ControllerEventThread.$anonfun$doWork$1(ControllerEventManager.scala:133) at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.scala:18) at kafka.metrics.KafkaTimer.time(KafkaTimer.scala:31) at kafka.controller.ControllerEventManager$ControllerEventThread.doWork(ControllerEventManager.scala:133) at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:96) [2022-03-14 04:02:36,455] INFO [Controller id=1 epoch=912] Sending UpdateMetadata request to brokers HashSet(0, 1, 2) for 1 partitions (state.change.logger) [2022-03-14 04:02:36,457] DEBUG [Controller id=1] Resigning (kafka.controller.KafkaController) [2022-03-14 04:02:36,457] DEBUG [Controller id=1] Unregister BrokerModifications handler for Set(0, 1, 2) (kafka.controller.KafkaController) [2022-03-14 04:02:36,555] DEBUG [Controller id=1] Broker 0 has been elected as the controller, so stopping the election process. (kafka.controller.KafkaController) Timelines from the log. Broker 0 was elected new controller at 04:02:34 At 04:02:35 Broker 1 still think it is the active controller. At 04:02:36 Broker 1 realized Broker 0 is the new active controller. How to resolve this split brain issue ? Thanks, Dhirendra On Wed, Mar 9, 2022 at 10:11 AM Dhirendra Singh <dhirendr...@gmail.com> wrote: > Some more information. > > It only happens when brokers lose connectivity to zookeeper and it results > in change in active controller. Issue does not occur always but randomly. > Issue never occurs when there is no change in active controller when > brokers lose connectivity to zookeeper. > As Fares pointed earlier, issue get resolved when the broker which is the > active controller is restarted. But this requires manual intervention. > So I assume this is some kind of bug (probably race condition) in > controller. > > Thanks, > Dhirendra. > > On Tue, Mar 8, 2022 at 10:40 AM Liam Clarke-Hutchinson < > lclar...@redhat.com> wrote: > >> Hi Dhirendra, so after looking into your stack trace further, it shows >> that >> the AlterISRRequest is failing when trying to interact with ZooKeeper. It >> doesn't give us more information as to why currently. >> >> Can you please set some loggers to DEBUG level to help analyse the issue >> further? >> >> These ones: >> >> - kafka.zk.KafkaZkClient >> - kafka.utils.ReplicationUtils >> - kafka.server.ZkIsrManager >> >> >> If you can do that and then share any log output that would be great :) >> >> Cheers, >> >> Liam >> >> On Tue, 8 Mar 2022 at 17:50, Dhirendra Singh <dhirendr...@gmail.com> >> wrote: >> >> > Hi Thomas, >> > I see the IllegalStateException but as i pasted earlier it is >> > 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 >> > >> > I upgraded to version 2.8.1 but issue is not resolved. >> > >> > Thanks, >> > Dhirendra. >> > >> > On Mon, Mar 7, 2022 at 5:45 AM Liam Clarke-Hutchinson < >> lclar...@redhat.com >> > > >> > wrote: >> > >> > > Ah, I may have seen this error before. Dhirendra Singh, If you grep >> your >> > > logs, you may find an IllegalStateException or two. >> > > >> > > https://issues.apache.org/jira/browse/KAFKA-12948 >> > > >> > > You need to upgrade to 2.7.2 if this is the issue you're hitting. >> > > >> > > Kind regards, >> > > >> > > Liam Clarke-Hutchinson >> > > >> > > On Sun, 6 Mar 2022 at 04:30, Mailbox - Dhirendra Kumar Singh < >> > > dhirendr...@gmail.com> wrote: >> > > >> > > > Let me rephrase my issue. >> > > > Issue occur when broker loose connectivity to zookeeper server. >> > > > Connectivity loss can happen due to many reasons…zookeeper servers >> > > getting >> > > > bounced, due to some network glitch etc… >> > > > >> > > > After the brokers reconnect to zookeeper server I expect the kafka >> > > cluster >> > > > to come back in stable state by itself without any manual >> intervention. >> > > > but instead few partitions remain under replicated due to the error >> I >> > > > pasted earlier. >> > > > >> > > > I feel this is some kind of bug. I am going to file a bug. >> > > > >> > > > >> > > > >> > > > Thanks, >> > > > >> > > > Dhirendra. >> > > > >> > > > >> > > > >> > > > From: Thomas Cooper <c...@tomcooper.dev> >> > > > Sent: Friday, March 4, 2022 7:01 PM >> > > > To: Dhirendra Singh <dhirendr...@gmail.com> >> > > > Cc: users@kafka.apache.org >> > > > Subject: Re: Few partitions stuck in under replication >> > > > >> > > > >> > > > >> > > > 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 >> > > <mailto: >> > > > 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 >> > > <mailto: >> > > > 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 < >> > > > https://tomcooper.dev> >> > > > >> > > > >> > > > >> > > > -- >> > > > >> > > > Tom Cooper >> > > > >> > > > @tomncooper <https://twitter.com/tomncooper> | tomcooper.dev < >> > > > https://tomcooper.dev> >> > > > >> > > > >> > > >> > >> >