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

Luke Chen commented on KAFKA-18871:
-----------------------------------

[~durban] , thanks for the debug log. It is very clear. Here's the timeline of 
the issue happened:

 

broker 0 is the leader of "kraft-test-topic-2" (partition 2).

controller is still the KRaft controller 3 at 15:31
{code:java}
2025-03-03 15:31:08,193 DEBUG [Controller id=2] Broker 3 has been elected as 
the controller, so stopping the election process. 
(kafka.controller.KafkaController) [controller-event-thread]{code}
 

Controller 3 handled a alterPartition request for "kraft-test-topic-2" and 
update with {*}partitionEpoch 21{*}, ** and *leaderEpoch 9*
{code:java}
2025-03-03 15:31:38,806 DEBUG Controller successfully handled AlterPartition 
request for kraft-test-topic-2: PartitionData(partitionIndex=2, errorCode=0, 
leaderId=0, leaderEpoch=9, isr=[0, 2, 1], leaderRecoveryState=0, 
partitionEpoch=21) (kafka.server.DefaultAlterPartitionManager) 
[zk-broker-0-to-controller-alter-partition-channel-manager]{code}
 

 

At 15:37, KRaft controller is removed, broker 0 is elected as the controller, 
reading metadata from the ZK

 
{code:java}
025-03-03 15:37:38,738 INFO [Controller id=0] 0 successfully elected as the 
controller. Epoch incremented to 4 and epoch zk version is now 4 
(kafka.controller.KafkaController) [controller-event-thread]
...

2025-03-03 15:37:38,908 DEBUG Completed 
request:{"isForwarded":false,"requestHeader":{"requestApiKey":4,"requestApiVersion":7,"correlationId":1,"clientId":"0","requestApiKeyName":"LEADER_AND_ISR"},"request":{"controllerId":0,"isKRaftController":false,"controllerEpoch":4,"brokerEpoch":8589935157,"type":0,"topicStates":[{"topicName":"kraft-test-topic","topicId":"5ssx9gNUTyyjwBsNqSofwA","partitionStates":[{"partitionIndex":0,"controllerEpoch":1,"leader":2,"leaderEpoch":10,"isr":[0,2,1],"partitionEpoch":22,"replicas":[2,1,0],"addingReplicas":[],"removingReplicas":[],"isNew":false,"leaderRecoveryState":0},{"partitionIndex":1,"controllerEpoch":1,"leader":0,"leaderEpoch":9,"isr":[0,2,1],"partitionEpoch":20,"replicas":[1,0,2],"addingReplicas":[],"removingReplicas":[],"isNew":false,"leaderRecoveryState":0},{"partitionIndex":2,"controllerEpoch":1,"leader":0,"leaderEpoch":9,"isr":[0,2,1],"partitionEpoch":19,"replicas":[0,2,1],"addingReplicas":[],"removingReplicas":[],"isNew":false,"leaderRecoveryState":0}]}]...{code}
 

 

Now, the metadata loaded from ZK, the partitionEpoch of "kraft-test-topic-2" is 
becoming *19*

Later, when broker 0 receives the LeaderAndISR request from the controller, it 
rejects it because the partitionEpoch from the controller 19 is less than its 
own epoch 21, but you can see, the leaderEpoch sending from controller is 10, 
but the old leaderEpoch above is 9. We failed to update the leaderEpoch here.

 
{code:java}
2025-03-03 15:37:39,287 INFO [Broker id=0] Skipped the become-leader state 
change for kraft-test-topic-2 with topic id Some(5ssx9gNUTyyjwBsNqSofwA) and 
partition state LeaderAndIsrPartitionState(topicName='kraft-test-topic', 
partitionIndex=2, controllerEpoch=4, leader=0, leaderEpoch=10, isr=[0, 2], 
partitionEpoch=20, replicas=[0, 2, 1], addingReplicas=[], removingReplicas=[], 
isNew=false, leaderRecoveryState=0) since the leader is already at a newer 
partition epoch 21. (state.change.logger) 
[control-plane-kafka-request-handler-0]
{code}
 

Later in the fetch request for "kraft-test-topic-2", we got 
UnknownLeaderEpochException
{code:java}
2025-03-03 15:38:18,538 DEBUG [KafkaApi-0] Fetch request with correlation id 0 
from client broker-1-fetcher-0 on partition 
5ssx9gNUTyyjwBsNqSofwA:kraft-test-topic-2 failed due to 
org.apache.kafka.common.errors.UnknownLeaderEpochException 
(kafka.server.KafkaApis) [data-plane-kafka-request-handler-4]{code}
{code:java}
2025-03-03 15:38:18,560 DEBUG [ReplicaFetcher replicaId=1, leaderId=0, 
fetcherId=0] Remote broker has a smaller leader epoch for partition 
kraft-test-topic-2 than this replica's current leader epoch of 10. 
(kafka.server.ReplicaFetcherThread) [ReplicaFetcherThread-0-0]{code}
 

So, we cannot successfully fetch data from the leader, and causes it becoming 
out of ISR.

 

I don't think the 2nd logs share the same cause (i.e. 
UnknownLeaderEpochException) as the 1st logs. But I think they have the same 
root cause: The metadata in ZK and KRaft controller are not consistent.

It's sad that so far we don't have a reliable way to make sure the data is 
consistent. ([~mumrah] , could you confirm?) We have a `ZkWriteBehindLag` 
metric, but even if we verify it's 0 and run the KRaft deprovision, there's 
still a time gap for new metadata comes in KRaft and without syncing to ZK. 
When in dual-write mode, the metadata is writing to KRaft and then ZK. So it's 
always possible to be inconsistent. But in normal migrating process, it won't 
be a problem since we're moving to KRaft controller and it always writes to 
KRaft first. The KRaft controller metadata is always up-to-date. Unfortunately 
in the rollback process, it's opposite. We want to move back to ZK, and it 
opens the possibility to have this metadata inconsistency.

 

Under the current design, I don't think we have any workaround for it. My 
suggestion is that when rolling back the migration, we should recommend users 
to shutdown all the brokers before rolling back. [~mumrah] , thoughts?

 

> 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
>            Priority: Critical
>         Attachments: cluster-operator.log.zip, controller_logs.zip, 
> kraft-rollback-bug.zip, kraft-rollback-kafka-default-pool-0.zip, 
> kraft-rollback-kafka-default-pool-1.zip, 
> kraft-rollback-kafka-default-pool-2.zip, 
> strimzi-cluster-operator-7bc47d488f-p4ltv.zip
>
>
> 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)

Reply via email to