[ 
https://issues.apache.org/jira/browse/KAFKA-20022?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=18051561#comment-18051561
 ] 

David Arthur commented on KAFKA-20022:
--------------------------------------

If restarting the controllers still results in the check op failing, then it 
seems likely some other process is updating your ZK. 

 

After migrating the brokers to KRaft, you can stay in dual write mode as long 
as you'd like. At this point the cluster is running KRaft while keeping ZK up 
to date. I would recommend staying in this mode long enough to validate that 
your client workloads are working as expected and that the cluster is stable.

> Kafka Dual Write Mode Sync Failure
> ----------------------------------
>
>                 Key: KAFKA-20022
>                 URL: https://issues.apache.org/jira/browse/KAFKA-20022
>             Project: Kafka
>          Issue Type: Bug
>          Components: controller
>    Affects Versions: 3.9.0
>            Reporter: Shubham Raj
>            Assignee: David Arthur
>            Priority: Major
>
> Hi,
> We migrated our Kafka cluster (v3.9) to *dual write mode* three weeks ago as 
> part of the planned one-month transition away from ZooKeeper. Recently, the 
> controller sync between the ZooKeeper and KRaft metadata went out of 
> alignment. As a result, the cluster is no longer in dual write mode. As 
> proposed in KAFKA-16171, attempts to restart the controllers did not restore 
> sync, and the ZooKeeper metadata is now lagging behind as per logs.
> *Impact*
>  * Dual write mode is no longer active, increasing risk of metadata 
> divergence.
>  * ZooKeeper metadata is stale compared to KRaft.
>  * Migration timeline is at risk.
> *Repetitive Logs in leader controller:*
> {code:java}
> [2025-12-29 01:44:23,852] ERROR Encountered zk migration fault: Unhandled 
> error in SyncKRaftMetadataEvent 
> (org.apache.kafka.server.fault.LoggingFaultHandler)
> java.lang.RuntimeException: Check op on KRaft Migration ZNode failed. Sent 
> zkVersion = 5349155. This indicates that another KRaft controller is making 
> writes to ZooKeeper.
>         at 
> kafka.zk.KafkaZkClient.handleUnwrappedMigrationResult$1(KafkaZkClient.scala:2050)
>         at 
> kafka.zk.KafkaZkClient.unwrapMigrationResponse$1(KafkaZkClient.scala:2076)
>         at 
> kafka.zk.KafkaZkClient.$anonfun$retryMigrationRequestsUntilConnected$2(KafkaZkClient.scala:2101)
>         at 
> scala.collection.StrictOptimizedIterableOps.map(StrictOptimizedIterableOps.scala:100)
>         at 
> scala.collection.StrictOptimizedIterableOps.map$(StrictOptimizedIterableOps.scala:87)
>         at scala.collection.mutable.ArrayBuffer.map(ArrayBuffer.scala:42)
>         at 
> kafka.zk.KafkaZkClient.retryMigrationRequestsUntilConnected(KafkaZkClient.scala:2101)
>         at 
> kafka.zk.migration.ZkTopicMigrationClient.$anonfun$createTopic$1(ZkTopicMigrationClient.scala:137)
>         at 
> kafka.zk.migration.ZkTopicMigrationClient.createTopic(ZkTopicMigrationClient.scala:111)
>         at 
> org.apache.kafka.metadata.migration.KRaftMigrationZkWriter.lambda$null$3(KRaftMigrationZkWriter.java:233)
>         at 
> org.apache.kafka.metadata.migration.KRaftMigrationDriver.applyMigrationOperation(KRaftMigrationDriver.java:246)
>         at 
> org.apache.kafka.metadata.migration.KRaftMigrationDriver.applyMigrationOperation(KRaftMigrationDriver.java:240)
>         at 
> org.apache.kafka.metadata.migration.KRaftMigrationDriver.access$300(KRaftMigrationDriver.java:63)
>         at 
> org.apache.kafka.metadata.migration.KRaftMigrationDriver$SyncKRaftMetadataEvent.lambda$run$0(KRaftMigrationDriver.java:844)
>         at 
> org.apache.kafka.metadata.migration.KRaftMigrationDriver.lambda$countingOperationConsumer$6(KRaftMigrationDriver.java:970)
>         at 
> org.apache.kafka.metadata.migration.KRaftMigrationZkWriter.lambda$handleTopicsSnapshot$4(KRaftMigrationZkWriter.java:230)
>         at java.base/java.lang.Iterable.forEach(Iterable.java:75)
>         at 
> org.apache.kafka.metadata.migration.KRaftMigrationZkWriter.handleTopicsSnapshot(KRaftMigrationZkWriter.java:228)
>         at 
> org.apache.kafka.metadata.migration.KRaftMigrationZkWriter.handleSnapshot(KRaftMigrationZkWriter.java:96)
>         at 
> org.apache.kafka.metadata.migration.KRaftMigrationDriver$SyncKRaftMetadataEvent.run(KRaftMigrationDriver.java:843)
>         at 
> org.apache.kafka.queue.KafkaEventQueue$EventContext.run(KafkaEventQueue.java:132)
>         at 
> org.apache.kafka.queue.KafkaEventQueue$EventHandler.handleEvents(KafkaEventQueue.java:215)
>         at 
> org.apache.kafka.queue.KafkaEventQueue$EventHandler.run(KafkaEventQueue.java:186)
>         at java.base/java.lang.Thread.run(Thread.java:840)
> [2025-12-29 01:44:23,852] TRACE [KRaftMigrationDriver id=10002] Received 
> metadata delta, but the controller is not in dual-write mode. Ignoring this 
> metadata update. (org.apache.kafka.metadata.migration.KRaftMigrationDriver)
> [2025-12-29 01:44:23,852] TRACE [KRaftMigrationDriver id=10002] Received 
> metadata delta, but the controller is not in dual-write mode. Ignoring this 
> metadata update. (org.apache.kafka.metadata.migration.KRaftMigrationDriver)
> [2025-12-29 01:44:23,852] TRACE [KRaftMigrationDriver id=10002] Received 
> metadata delta, but the controller is not in dual-write mode. Ignoring this 
> metadata update. (org.apache.kafka.metadata.migration.KRaftMigrationDriver)
> [2025-12-29 01:44:23,852] TRACE [KRaftMigrationDriver id=10002] Received 
> metadata delta, but the controller is not in dual-write mode. Ignoring this 
> metadata update. (org.apache.kafka.metadata.migration.KRaftMigrationDriver)
> [2025-12-29 01:44:23,852] TRACE [KRaftMigrationDriver id=10002] Received 
> metadata delta, but the controller is not in dual-write mode. Ignoring this 
> metadata update. (org.apache.kafka.metadata.migration.KRaftMigrationDriver)
>  {code}
>  
>  
> *cluster status*
> {code:java}
> ClusterId:              QBC8K1kNS02Sl9930_QDAA
> LeaderId:               10002
> LeaderEpoch:            253
> HighWatermark:          12515984
> MaxFollowerLag:         0
> MaxFollowerLagTimeMs:   111
> CurrentVoters:          [10001,10002,10003]
> CurrentObservers:       
> [5,4,1104,6,3,1,1103,1112,1108,1107,1109,1101,1102,2,1106,1110,1111,1105]
>  {code}
>  
> *Migration data in zookeeper*
> {code:java}
> In [4]: zk_client.get('/kafka/qa/migration')
> Out[4]: 
> (b'{"version":0,"kraft_metadata_offset":10840503,"kraft_controller_id":10002,"kraft_metadata_epoch":170,"kraft_controller_epoch":253}',
>  ZnodeStat(czxid=7129652820618, mzxid=7176892421062, ctime=1765176701970, 
> mtime=1766986727917, version=5349156, cversion=0, aversion=0, 
> ephemeralOwner=0, dataLength=130, numChildren=0, pzxid=7129652820618))
>  {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to