Daniel Urban created KAFKA-18871: ------------------------------------ Summary: KRaft migration rollback causes downtime Key: KAFKA-18871 URL: https://issues.apache.org/jira/browse/KAFKA-18871 Project: Kafka Issue Type: Bug Components: kraft, migration Affects Versions: 3.9.0 Reporter: Daniel Urban
When testing the KRaft migration rollback feature, found the following scenario: # Execute KRaft migration on a 3 broker 3 ZK node cluster to the last step, but do not finalize the migration. ## In the test, we put a slow but continuous produce+consume load on the cluster, with a topic (partitions=3, RF=3, min ISR=2) # Start the rollback procedure # First we roll back the brokers from KRaft mode to migration mode (both controller and ZK configs are set, process roles are removed, {{zookeeper.metadata.migration.enable}} is true) # Then we delete the KRaft controllers, delete the /controller znode # Then we immediately start rolling the brokers 1 by 1 to ZK mode by removing the {{zookeeper.metadata.migration.enable}} flag and the controller.* configurations. # At this point, when we restart the 1st broker (let's call it broker 0) into ZK mode, we find an issue which occurs ~1 out of 20 times: If broker 0 is not in the ISR for one of the partitions at this point, it can simply never become part of the ISR. Since we are aiming for zero downtime, we check the ISR states of partitions between broker restarts, and our process gets blocked at this point. We have tried multiple workarounds at this point, but it seems that there is no workaround which still ensures zero downtime. Some more details about the process: * We are using Strimzi to drive this process, but have verified that Strimzi follows the documented steps precisely. * When we reach the error state, it doesn't matter which broker became the controller through the ZK node, the brokers still in migration mode get stuck, and they flood the logs with the following error: {code:java} 2025-02-26 10:55:21,985 WARN [RaftManager id=0] Error connecting to node kraft-rollback-kafka-controller-pool-5.kraft-rollback-kafka-kafka-brokers.csm-op-test-kraft-rollback-e7798bef.svc.cluster.local:9090 (id: 5 rack: null) (org.apache.kafka.clients.NetworkClient) [kafka-raft-outbound-request-thread] java.net.UnknownHostException: kraft-rollback-kafka-controller-pool-5.kraft-rollback-kafka-kafka-brokers.csm-op-test-kraft-rollback-e7798bef.svc.cluster.local at java.base/java.net.InetAddress$CachedAddresses.get(InetAddress.java:801) at java.base/java.net.InetAddress.getAllByName0(InetAddress.java:1533) at java.base/java.net.InetAddress.getAllByName(InetAddress.java:1385) at java.base/java.net.InetAddress.getAllByName(InetAddress.java:1306) at org.apache.kafka.clients.DefaultHostResolver.resolve(DefaultHostResolver.java:27) at org.apache.kafka.clients.ClientUtils.resolve(ClientUtils.java:125) at org.apache.kafka.clients.ClusterConnectionStates$NodeConnectionState.resolveAddresses(ClusterConnectionStates.java:536) at org.apache.kafka.clients.ClusterConnectionStates$NodeConnectionState.currentAddress(ClusterConnectionStates.java:511) at org.apache.kafka.clients.ClusterConnectionStates$NodeConnectionState.access$200(ClusterConnectionStates.java:466) at org.apache.kafka.clients.ClusterConnectionStates.currentAddress(ClusterConnectionStates.java:173) at org.apache.kafka.clients.NetworkClient.initiateConnect(NetworkClient.java:1075) at org.apache.kafka.clients.NetworkClient.ready(NetworkClient.java:321) at org.apache.kafka.server.util.InterBrokerSendThread.sendRequests(InterBrokerSendThread.java:146) at org.apache.kafka.server.util.InterBrokerSendThread.pollOnce(InterBrokerSendThread.java:109) at org.apache.kafka.server.util.InterBrokerSendThread.doWork(InterBrokerSendThread.java:137) at org.apache.kafka.server.util.ShutdownableThread.run(ShutdownableThread.java:136) {code} * Manually verified the last offsets of the replicas, and broker 0 is caught up in the partition. * Even after stopping the produce load, the issue persists. * Even after removing the /controller node manually (to retrigger election), regardless of which broker becomes the controller, the issue persists. Based on the above, it seems that during the rollback, brokers in migration mode cannot handle the KRaft controllers being removed from the system. Since broker 0 is caught up in the partition, we suspect that the other brokers (still in migration mode) do not respect the controller state in ZK, and do not report changes in the ISR of the partitions they are leading. This means that if a replica becomes out of sync in the last restart (e.g. due to a slow broker restart), we cannot restart the brokers while ensuring zero downtime. -- This message was sent by Atlassian Jira (v8.20.10#820010)