David Arthur created KAFKA-16667:
------------------------------------

             Summary: KRaftMigrationDriver gets stuck after successive failovers
                 Key: KAFKA-16667
                 URL: https://issues.apache.org/jira/browse/KAFKA-16667
             Project: Kafka
          Issue Type: Bug
          Components: controller, migration
            Reporter: David Arthur


This is a continuation of KAFKA-16171.

It turns out that the active KRaftMigrationDriver can get a stale read from ZK 
after becoming the active controller in ZK (i.e., writing to "/controller").

Because ZooKeeper only offers linearizability on writes to a given ZNode, it is 
possible that we get a stale read on the "/migration" ZNode after writing to 
"/controller" (and "/controller_epoch") when becoming active. 

 

The history looks like this:
 # Node B becomes leader in the Raft layer. KRaftLeaderEvents are enqueued on 
all KRaftMigrationDriver-s
 # Node A writes some state to ZK, updates "/migration", and checks 
"/controller_epoch" in one transaction. This happens before B claims controller 
leadership in ZK. The "/migration" state is updated from X to Y
 # Node B claims leadership by updating "/controller" and "/controller_epoch". 
Leader B reads "/migration" state X
 # Node A tries to write some state, fails on "/controller_epoch" check op.
 # Node A processes new leader and becomes inactive

 

This does not violate consistency guarantees made by ZooKeeper.

 

> Write operations in ZooKeeper are {_}linearizable{_}. In other words, each 
> {{write}} will appear to take effect atomically at some point between when 
> the client issues the request and receives the corresponding response.

and 

> Read operations in ZooKeeper are _not linearizable_ since they can return 
> potentially stale data. This is because a {{read}} in ZooKeeper is not a 
> quorum operation and a server will respond immediately to a client that is 
> performing a {{{}read{}}}.

 

--- 

 

The impact of this stale read is the same as KAFKA-16171. The 
KRaftMigrationDriver never gets past SYNC_KRAFT_TO_ZK because it has a stale 
zkVersion for the "/migration" ZNode. The result is brokers never learn about 
the new controller and cannot update any partition state.

The workaround for this bug is to re-elect the controller by shutting down the 
active KRaft controller. 

This bug was found during a migration where the KRaft controller was rapidly 
failing over due to an excess of metadata. 



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

Reply via email to