Paolo Patierno created KAFKA-16101:
--------------------------------------

             Summary: Kafka cluster unavailable during KRaft migration rollback 
procedure
                 Key: KAFKA-16101
                 URL: https://issues.apache.org/jira/browse/KAFKA-16101
             Project: Kafka
          Issue Type: Bug
          Components: kraft
    Affects Versions: 3.6.1
            Reporter: Paolo Patierno


Hello,

I was trying the KRaft migration rollback procedure locally and I came across a 
potential bug or anyway a situation where the cluster is not usable/available 
for a certain amount of time.

In order to test the procedure, I start with a one broker and one zookeeper 
node cluster. Then I start the migration with a one KRaft controller node. The 
migration runs fine and it reaches the point of "dual write" state.

>From this point, I try to run the rollback procedure as described in the 
>documentation.

As first step, this involves ...
 * stopping the broker
 * removing the __cluster_metadata folder
 * removing ZooKeeper migration flag and controller(s) related configuration 
from the broker
 * restarting the broker

With the above steps done, the broker starts in ZooKeeper mode (no migration, 
no KRaft controllers knowledge) and it keeps logging the following messages in 
DEBUG:
{code:java}
[2024-01-08 11:51:20,608] DEBUG 
[zk-broker-0-to-controller-forwarding-channel-manager]: Controller isn't 
cached, looking for local metadata changes 
(kafka.server.BrokerToControllerRequestThread)
[2024-01-08 11:51:20,608] DEBUG 
[zk-broker-0-to-controller-forwarding-channel-manager]: No controller provided, 
retrying after backoff (kafka.server.BrokerToControllerRequestThread)
[2024-01-08 11:51:20,629] DEBUG 
[zk-broker-0-to-controller-alter-partition-channel-manager]: Controller isn't 
cached, looking for local metadata changes 
(kafka.server.BrokerToControllerRequestThread)
[2024-01-08 11:51:20,629] DEBUG 
[zk-broker-0-to-controller-alter-partition-channel-manager]: No controller 
provided, retrying after backoff (kafka.server.BrokerToControllerRequestThread) 
{code}
What's happening should be clear.

The /controller znode in ZooKeeper still reports the KRaft controller 
(broker.id = 1) as controller. The broker get it from the znode but doesn't 
know how to reach it.

The issue is that until the procedure is complete with the next steps (shutting 
down KRaft controller, deleting /controller znode), the cluster is unusable. 
Any admin or client operation against the broker doesn't work, just hangs, the 
broker doesn't reply.

Imagining this scenario to a more complex one with 10-20-50 brokers and 
partitions' replicas spread across them, when the brokers are rolled one by one 
(in ZK mode) reporting the above error, the topics will become not available 
one after the other, until all brokers are in such a state and nothing can 
work. This is because from a KRaft controller perspective (still running), the 
brokers are not available anymore and the partitions' replicas are out of sync.

Of course, as soon as you complete the rollback procedure, after deleting the 
/controller znode, the brokers are able to elect a new controller among them 
and everything recovers to work.

My first question ... isn't the cluster supposed to work during rollback and 
being always available during the rollback when the procedure is not completed 
yet? Or having the cluster not available is an assumption during the rollback, 
until it's complete?

This "unavailability" time window could be reduced by deleting the /controller 
znode before shutting down the KRaft controllers to allow the brokers electing 
a new controller among them, but in this case, could be a race condition where 
KRaft controllers still running could steal leadership again?

Or is there anything missing in the documentation maybe which is driving to 
this problem?



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

Reply via email to