divijvaidya opened a new pull request, #13111: URL: https://github.com/apache/kafka/pull/13111
## Change Controller should update Zk with locally cached TopicId (when available) instead of assigning a new one when Zk doesn't have a TopicId. ## Motivation for this change This problem was highlighted in KAFKA-14190 and since then, multiple users have complained about the problem [HERE](https://lists.apache.org/thread/jzk4tyd1xs1wwj0bpkdnxpw0m152qw1f) (mailing list), [HERE](https://lists.apache.org/thread/4rqrd6kqd0nrc248fv1tmdn0088947hv) (mailing list) and [HERE](https://the-asf.slack.com/archives/CE7HWJPHA/p1671529649633529) (ASF slack channel). ## Description of the problem In certain situations, it is possible that the TopicId stored locally on a broker for a topic differs from the topicId stored for that topic on Zk. Currently, such situation arises when users use a <2.8 client to alterPartitions for a topic on a >=2.8 (including latest 3.4) brokers AND they use `--zookeeper` flag from the client. Note that `--zookeeper` has been marked deprecated for a long time and has been replaced by `--bootstrap-server` which doesn't face this problem. The result of topic Id discrepancy leads to availability loss for the topic until user performs the mitigation steps listed in KAFKA-14190. The exact sequence of steps are: 1. User uses pre 2.8 client to create a new topic in zookeeper directly 2. No TopicId is generated in Zookeeper 3. KafkaController listens to the ZNode, and a `TopicChange` event is created, During handling on this event, controller notices that there is no TopicId, it generated a new one and updates Zk. 4. At this stage, Zk has a TopicId. 5. User uses pre 2.8 client to increase the number of partitions for this topic 6. The client will replace/overwrite the entire existing Znode with new placement information. **This will delete the existing TopicId in Zk (that was created by controller in step 3).** 7. Next time KafkaController interacts with this ZNode, it will generate a new TopicId. 8. Note that we now have two different TopicIds for this topic name. 9. Broker may have a different topicId (older one) in metadata file and will complain about the mismatch when they encounter a new TopicId. ## Testing I have added a test with this change which asserts that TopicId for a topic is immutable i.e. once assigned, it does not change. This test fails before this change and passes after this change. ## Side effects of this fix There are no additional side effects of this change. No additional calls to Zk. We are only updating the TopicId from a locally cached value instead of assigning a new one. ## Caveats This code change does not fix the problem completely. The code change assumes that controller would have the TopicId locally so that it can update Zk but situations such as controller failover, that may not be true. More specifically, we will still end up having two different topic Ids in cases when controller failover takes place between the time when Zk TopicID was overwritten/removed and time when controller could update the TopicId with local value. However, this code change should fix majority of the scenario that are impacted by this bug and a separate PR would be filed to fix the minority scenarios of controller failover during the exact duration. ## Release Due to the simple nature of the fix and the number of user requests, I would request to consider adding this to 3.4.0 and backporting to as many previous version as we can. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org