[ https://issues.apache.org/jira/browse/KAFKA-16162?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Omnia Ibrahim resolved KAFKA-16162. ----------------------------------- Fix Version/s: 3.7.0 Resolution: Fixed > New created topics are unavailable after upgrading to 3.7 > --------------------------------------------------------- > > Key: KAFKA-16162 > URL: https://issues.apache.org/jira/browse/KAFKA-16162 > Project: Kafka > Issue Type: Bug > Affects Versions: 3.7.0 > Reporter: Luke Chen > Assignee: Gaurav Narula > Priority: Blocker > Fix For: 3.7.0 > > > In 3.7, we introduced the KIP-858 JBOD feature, and the brokerRegistration > request will include the `LogDirs` fields with UUID for each log dir in each > broker. This info will be stored in the controller and used to identify if > the log dir is known and online while handling AssignReplicasToDirsRequest > [here|https://github.com/apache/kafka/blob/trunk/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java#L2093]. > > While upgrading from old version, the kafka cluster will run in 3.7 binary > with old metadata version, and then upgrade to newer version using > kafka-features.sh. That means, while brokers startup and send the > brokerRegistration request, it'll be using older metadata version without > `LogDirs` fields included. And it makes the controller has no log dir info > for all brokers. Later, after upgraded, if new topic is created, the flow > will go like this: > 1. Controller assign replicas and adds in metadata log > 2. brokers fetch the metadata and apply it > 3. ReplicaManager#maybeUpdateTopicAssignment will update topic assignment > 4. After sending ASSIGN_REPLICAS_TO_DIRS to controller with replica > assignment, controller will think the log dir in current replica is offline, > so triggering offline handler, and reassign leader to another replica, and > offline, until no more replicas to assign, so assigning leader to -1 (i.e. no > leader) > So, the results will be that new created topics are unavailable (with no > leader) because the controller thinks all log dir are offline. > {code:java} > lukchen@lukchen-mac kafka % bin/kafka-topics.sh --describe --topic > quickstart-events3 --bootstrap-server localhost:9092 > > Topic: quickstart-events3 TopicId: s8s6tEQyRvmjKI6ctNTgPg PartitionCount: > 3 ReplicationFactor: 3 Configs: segment.bytes=1073741824 > Topic: quickstart-events3 Partition: 0 Leader: none > Replicas: 7,2,6 Isr: 6 > Topic: quickstart-events3 Partition: 1 Leader: none > Replicas: 2,6,7 Isr: 6 > Topic: quickstart-events3 Partition: 2 Leader: none > Replicas: 6,7,2 Isr: 6 > {code} > The log snippet in the controller : > {code:java} > # handling 1st assignReplicaToDirs request > [2024-01-18 19:34:47,370] DEBUG [QuorumController id=1] Broker 6 assigned > partition quickstart-events3:0 to OFFLINE dir 7K5JBERyyqFFxIXSXYluJA > (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,370] DEBUG [QuorumController id=1] Broker 6 assigned > partition quickstart-events3:2 to OFFLINE dir 7K5JBERyyqFFxIXSXYluJA > (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,371] DEBUG [QuorumController id=1] Broker 6 assigned > partition quickstart-events3:1 to OFFLINE dir 7K5JBERyyqFFxIXSXYluJA > (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,372] DEBUG [QuorumController id=1] > offline-dir-assignment: changing partition(s): quickstart-events3-0, > quickstart-events3-2, quickstart-events3-1 > (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,372] DEBUG [QuorumController id=1] partition change for > quickstart-events3-0 with topic ID 6ZIeidfiSTWRiOAmGEwn_g: directories: > [AAAAAAAAAAAAAAAAAAAAAA, AAAAAAAAAAAAAAAAAAAAAA, AAAAAAAAAAAAAAAAAAAAAA] -> > [7K5JBERyyqFFxIXSXYluJA, AAAAAAAAAAAAAAAAAAAAAA, AAAAAAAAAAAAAAAAAAAAAA], > partitionEpoch: 0 -> 1 (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,372] DEBUG [QuorumController id=1] Replayed partition > change PartitionChangeRecord(partitionId=0, topicId=6ZIeidfiSTWRiOAmGEwn_g, > isr=null, leader=-2, replicas=null, removingReplicas=null, > addingReplicas=null, leaderRecoveryState=-1, > directories=[7K5JBERyyqFFxIXSXYluJA, AAAAAAAAAAAAAAAAAAAAAA, > AAAAAAAAAAAAAAAAAAAAAA], eligibleLeaderReplicas=null, lastKnownELR=null) for > topic quickstart-events3 > (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,372] DEBUG [QuorumController id=1] partition change for > quickstart-events3-2 with topic ID 6ZIeidfiSTWRiOAmGEwn_g: directories: > [AAAAAAAAAAAAAAAAAAAAAA, AAAAAAAAAAAAAAAAAAAAAA, AAAAAAAAAAAAAAAAAAAAAA] -> > [AAAAAAAAAAAAAAAAAAAAAA, 7K5JBERyyqFFxIXSXYluJA, AAAAAAAAAAAAAAAAAAAAAA], > partitionEpoch: 0 -> 1 (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,372] DEBUG [QuorumController id=1] Replayed partition > change PartitionChangeRecord(partitionId=2, topicId=6ZIeidfiSTWRiOAmGEwn_g, > isr=null, leader=-2, replicas=null, removingReplicas=null, > addingReplicas=null, leaderRecoveryState=-1, > directories=[AAAAAAAAAAAAAAAAAAAAAA, 7K5JBERyyqFFxIXSXYluJA, > AAAAAAAAAAAAAAAAAAAAAA], eligibleLeaderReplicas=null, lastKnownELR=null) for > topic quickstart-events3 > (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,372] DEBUG [QuorumController id=1] partition change for > quickstart-events3-1 with topic ID 6ZIeidfiSTWRiOAmGEwn_g: directories: > [AAAAAAAAAAAAAAAAAAAAAA, AAAAAAAAAAAAAAAAAAAAAA, AAAAAAAAAAAAAAAAAAAAAA] -> > [AAAAAAAAAAAAAAAAAAAAAA, AAAAAAAAAAAAAAAAAAAAAA, 7K5JBERyyqFFxIXSXYluJA], > partitionEpoch: 0 -> 1 (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,372] DEBUG [QuorumController id=1] Replayed partition > change PartitionChangeRecord(partitionId=1, topicId=6ZIeidfiSTWRiOAmGEwn_g, > isr=null, leader=-2, replicas=null, removingReplicas=null, > addingReplicas=null, leaderRecoveryState=-1, > directories=[AAAAAAAAAAAAAAAAAAAAAA, AAAAAAAAAAAAAAAAAAAAAA, > 7K5JBERyyqFFxIXSXYluJA], eligibleLeaderReplicas=null, lastKnownELR=null) for > topic quickstart-events3 > (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,372] DEBUG [QuorumController id=1] partition change for > quickstart-events3-0 with topic ID 6ZIeidfiSTWRiOAmGEwn_g: isr: [6, 7, 2] -> > [7, 2], leader: 6 -> 7, leaderEpoch: 0 -> 1, partitionEpoch: 1 -> 2 > (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,373] DEBUG [QuorumController id=1] Replayed partition > change PartitionChangeRecord(partitionId=0, topicId=6ZIeidfiSTWRiOAmGEwn_g, > isr=[7, 2], leader=7, replicas=null, removingReplicas=null, > addingReplicas=null, leaderRecoveryState=-1, directories=null, > eligibleLeaderReplicas=null, lastKnownELR=null) for topic quickstart-events3 > (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,373] DEBUG [QuorumController id=1] partition change for > quickstart-events3-2 with topic ID 6ZIeidfiSTWRiOAmGEwn_g: isr: [2, 6, 7] -> > [2, 7], partitionEpoch: 1 -> 2 > (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,373] DEBUG [QuorumController id=1] Replayed partition > change PartitionChangeRecord(partitionId=2, topicId=6ZIeidfiSTWRiOAmGEwn_g, > isr=[2, 7], leader=-2, replicas=null, removingReplicas=null, > addingReplicas=null, leaderRecoveryState=-1, directories=null, > eligibleLeaderReplicas=null, lastKnownELR=null) for topic quickstart-events3 > (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,373] DEBUG [QuorumController id=1] partition change for > quickstart-events3-1 with topic ID 6ZIeidfiSTWRiOAmGEwn_g: isr: [7, 2, 6] -> > [7, 2], partitionEpoch: 1 -> 2 > (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,373] DEBUG [QuorumController id=1] Replayed partition > change PartitionChangeRecord(partitionId=1, topicId=6ZIeidfiSTWRiOAmGEwn_g, > isr=[7, 2], leader=-2, replicas=null, removingReplicas=null, > addingReplicas=null, leaderRecoveryState=-1, directories=null, > eligibleLeaderReplicas=null, lastKnownELR=null) for topic quickstart-events3 > (org.apache.kafka.controller.ReplicationControlManager) > # handling 2nd assignReplicaToDirs request > [2024-01-18 19:34:47,373] DEBUG [QuorumController id=1] Broker 7 assigned > partition quickstart-events3:0 to OFFLINE dir 7V9WGfqd-WUS-k61NdrzcQ > (org.apache.kafka.controller.ReplicationControlManager) > !!! dir:AAAAAAAAAAAAAAAAAAAAAA;;;[] > [2024-01-18 19:34:47,374] DEBUG [QuorumController id=1] Broker 7 assigned > partition quickstart-events3:2 to OFFLINE dir 7V9WGfqd-WUS-k61NdrzcQ > (org.apache.kafka.controller.ReplicationControlManager) > !!! dir:AAAAAAAAAAAAAAAAAAAAAA;;;[] > [2024-01-18 19:34:47,374] DEBUG [QuorumController id=1] Broker 7 assigned > partition quickstart-events3:1 to OFFLINE dir 7V9WGfqd-WUS-k61NdrzcQ > (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,375] DEBUG [QuorumController id=1] > offline-dir-assignment: changing partition(s): quickstart-events3-0, > quickstart-events3-2, quickstart-events3-1 > (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,375] DEBUG [QuorumController id=1] partition change for > quickstart-events3-0 with topic ID 6ZIeidfiSTWRiOAmGEwn_g: directories: > [7K5JBERyyqFFxIXSXYluJA, AAAAAAAAAAAAAAAAAAAAAA, AAAAAAAAAAAAAAAAAAAAAA] -> > [7K5JBERyyqFFxIXSXYluJA, 7V9WGfqd-WUS-k61NdrzcQ, AAAAAAAAAAAAAAAAAAAAAA], > partitionEpoch: 2 -> 3 (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,375] DEBUG [QuorumController id=1] Replayed partition > change PartitionChangeRecord(partitionId=0, topicId=6ZIeidfiSTWRiOAmGEwn_g, > isr=null, leader=-2, replicas=null, removingReplicas=null, > addingReplicas=null, leaderRecoveryState=-1, > directories=[7K5JBERyyqFFxIXSXYluJA, 7V9WGfqd-WUS-k61NdrzcQ, > AAAAAAAAAAAAAAAAAAAAAA], eligibleLeaderReplicas=null, lastKnownELR=null) for > topic quickstart-events3 > (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,375] DEBUG [QuorumController id=1] partition change for > quickstart-events3-2 with topic ID 6ZIeidfiSTWRiOAmGEwn_g: directories: > [AAAAAAAAAAAAAAAAAAAAAA, 7K5JBERyyqFFxIXSXYluJA, AAAAAAAAAAAAAAAAAAAAAA] -> > [AAAAAAAAAAAAAAAAAAAAAA, 7K5JBERyyqFFxIXSXYluJA, 7V9WGfqd-WUS-k61NdrzcQ], > partitionEpoch: 2 -> 3 (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,375] DEBUG [QuorumController id=1] Replayed partition > change PartitionChangeRecord(partitionId=2, topicId=6ZIeidfiSTWRiOAmGEwn_g, > isr=null, leader=-2, replicas=null, removingReplicas=null, > addingReplicas=null, leaderRecoveryState=-1, > directories=[AAAAAAAAAAAAAAAAAAAAAA, 7K5JBERyyqFFxIXSXYluJA, > 7V9WGfqd-WUS-k61NdrzcQ], eligibleLeaderReplicas=null, lastKnownELR=null) for > topic quickstart-events3 > (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,375] DEBUG [QuorumController id=1] partition change for > quickstart-events3-1 with topic ID 6ZIeidfiSTWRiOAmGEwn_g: directories: > [AAAAAAAAAAAAAAAAAAAAAA, AAAAAAAAAAAAAAAAAAAAAA, 7K5JBERyyqFFxIXSXYluJA] -> > [7V9WGfqd-WUS-k61NdrzcQ, AAAAAAAAAAAAAAAAAAAAAA, 7K5JBERyyqFFxIXSXYluJA], > partitionEpoch: 2 -> 3 (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,375] DEBUG [QuorumController id=1] Replayed partition > change PartitionChangeRecord(partitionId=1, topicId=6ZIeidfiSTWRiOAmGEwn_g, > isr=null, leader=-2, replicas=null, removingReplicas=null, > addingReplicas=null, leaderRecoveryState=-1, > directories=[7V9WGfqd-WUS-k61NdrzcQ, AAAAAAAAAAAAAAAAAAAAAA, > 7K5JBERyyqFFxIXSXYluJA], eligibleLeaderReplicas=null, lastKnownELR=null) for > topic quickstart-events3 > (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,375] DEBUG [QuorumController id=1] partition change for > quickstart-events3-0 with topic ID 6ZIeidfiSTWRiOAmGEwn_g: isr: [7, 2] -> > [2], leader: 7 -> 2, leaderEpoch: 1 -> 2, partitionEpoch: 3 -> 4 > (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,375] DEBUG [QuorumController id=1] Replayed partition > change PartitionChangeRecord(partitionId=0, topicId=6ZIeidfiSTWRiOAmGEwn_g, > isr=[2], leader=2, replicas=null, removingReplicas=null, addingReplicas=null, > leaderRecoveryState=-1, directories=null, eligibleLeaderReplicas=null, > lastKnownELR=null) for topic quickstart-events3 > (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,375] DEBUG [QuorumController id=1] partition change for > quickstart-events3-2 with topic ID 6ZIeidfiSTWRiOAmGEwn_g: isr: [2, 7] -> > [2], partitionEpoch: 3 -> 4 > (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,375] DEBUG [QuorumController id=1] Replayed partition > change PartitionChangeRecord(partitionId=2, topicId=6ZIeidfiSTWRiOAmGEwn_g, > isr=[2], leader=-2, replicas=null, removingReplicas=null, > addingReplicas=null, leaderRecoveryState=-1, directories=null, > eligibleLeaderReplicas=null, lastKnownELR=null) for topic quickstart-events3 > (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,375] DEBUG [QuorumController id=1] partition change for > quickstart-events3-1 with topic ID 6ZIeidfiSTWRiOAmGEwn_g: isr: [7, 2] -> > [2], leader: 7 -> 2, leaderEpoch: 0 -> 1, partitionEpoch: 3 -> 4 > (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,375] DEBUG [QuorumController id=1] Replayed partition > change PartitionChangeRecord(partitionId=1, topicId=6ZIeidfiSTWRiOAmGEwn_g, > isr=[2], leader=2, replicas=null, removingReplicas=null, addingReplicas=null, > leaderRecoveryState=-1, directories=null, eligibleLeaderReplicas=null, > lastKnownELR=null) for topic quickstart-events3 > (org.apache.kafka.controller.ReplicationControlManager) > # handling 3rd assignReplicaToDirs request > [2024-01-18 19:34:47,378] DEBUG [QuorumController id=1] Broker 2 assigned > partition quickstart-events3:0 to OFFLINE dir 97WGe1Sf0cWqegCPEUExiw > (org.apache.kafka.controller.ReplicationControlManager) > !!! dir:AAAAAAAAAAAAAAAAAAAAAA;;;[] > [2024-01-18 19:34:47,378] DEBUG [QuorumController id=1] Broker 2 assigned > partition quickstart-events3:2 to OFFLINE dir 97WGe1Sf0cWqegCPEUExiw > (org.apache.kafka.controller.ReplicationControlManager) > !!! dir:AAAAAAAAAAAAAAAAAAAAAA;;;[] > [2024-01-18 19:34:47,378] DEBUG [QuorumController id=1] Broker 2 assigned > partition quickstart-events3:1 to OFFLINE dir 97WGe1Sf0cWqegCPEUExiw > (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,379] DEBUG [QuorumController id=1] > offline-dir-assignment: changing partition(s): quickstart-events3-0, > quickstart-events3-2, quickstart-events3-1 > (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,379] DEBUG [QuorumController id=1] partition change for > quickstart-events3-0 with topic ID 6ZIeidfiSTWRiOAmGEwn_g: directories: > [7K5JBERyyqFFxIXSXYluJA, 7V9WGfqd-WUS-k61NdrzcQ, AAAAAAAAAAAAAAAAAAAAAA] -> > [7K5JBERyyqFFxIXSXYluJA, 7V9WGfqd-WUS-k61NdrzcQ, 97WGe1Sf0cWqegCPEUExiw], > partitionEpoch: 4 -> 5 (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,379] DEBUG [QuorumController id=1] Replayed partition > change PartitionChangeRecord(partitionId=0, topicId=6ZIeidfiSTWRiOAmGEwn_g, > isr=null, leader=-2, replicas=null, removingReplicas=null, > addingReplicas=null, leaderRecoveryState=-1, > directories=[7K5JBERyyqFFxIXSXYluJA, 7V9WGfqd-WUS-k61NdrzcQ, > 97WGe1Sf0cWqegCPEUExiw], eligibleLeaderReplicas=null, lastKnownELR=null) for > topic quickstart-events3 > (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,379] DEBUG [QuorumController id=1] partition change for > quickstart-events3-2 with topic ID 6ZIeidfiSTWRiOAmGEwn_g: directories: > [AAAAAAAAAAAAAAAAAAAAAA, 7K5JBERyyqFFxIXSXYluJA, 7V9WGfqd-WUS-k61NdrzcQ] -> > [97WGe1Sf0cWqegCPEUExiw, 7K5JBERyyqFFxIXSXYluJA, 7V9WGfqd-WUS-k61NdrzcQ], > partitionEpoch: 4 -> 5 (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,379] DEBUG [QuorumController id=1] Replayed partition > change PartitionChangeRecord(partitionId=2, topicId=6ZIeidfiSTWRiOAmGEwn_g, > isr=null, leader=-2, replicas=null, removingReplicas=null, > addingReplicas=null, leaderRecoveryState=-1, > directories=[97WGe1Sf0cWqegCPEUExiw, 7K5JBERyyqFFxIXSXYluJA, > 7V9WGfqd-WUS-k61NdrzcQ], eligibleLeaderReplicas=null, lastKnownELR=null) for > topic quickstart-events3 > (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,379] DEBUG [QuorumController id=1] partition change for > quickstart-events3-1 with topic ID 6ZIeidfiSTWRiOAmGEwn_g: directories: > [7V9WGfqd-WUS-k61NdrzcQ, AAAAAAAAAAAAAAAAAAAAAA, 7K5JBERyyqFFxIXSXYluJA] -> > [7V9WGfqd-WUS-k61NdrzcQ, 97WGe1Sf0cWqegCPEUExiw, 7K5JBERyyqFFxIXSXYluJA], > partitionEpoch: 4 -> 5 (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,379] DEBUG [QuorumController id=1] Replayed partition > change PartitionChangeRecord(partitionId=1, topicId=6ZIeidfiSTWRiOAmGEwn_g, > isr=null, leader=-2, replicas=null, removingReplicas=null, > addingReplicas=null, leaderRecoveryState=-1, > directories=[7V9WGfqd-WUS-k61NdrzcQ, 97WGe1Sf0cWqegCPEUExiw, > 7K5JBERyyqFFxIXSXYluJA], eligibleLeaderReplicas=null, lastKnownELR=null) for > topic quickstart-events3 > (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,379] DEBUG [QuorumController id=1] partition change for > quickstart-events3-0 with topic ID 6ZIeidfiSTWRiOAmGEwn_g: leader: 2 -> -1, > leaderEpoch: 2 -> 3, partitionEpoch: 5 -> 6 > (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,379] DEBUG [QuorumController id=1] Replayed partition > change PartitionChangeRecord(partitionId=0, topicId=6ZIeidfiSTWRiOAmGEwn_g, > isr=null, leader=-1, replicas=null, removingReplicas=null, > addingReplicas=null, leaderRecoveryState=-1, directories=null, > eligibleLeaderReplicas=null, lastKnownELR=null) for topic quickstart-events3 > (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,379] DEBUG [QuorumController id=1] partition change for > quickstart-events3-2 with topic ID 6ZIeidfiSTWRiOAmGEwn_g: leader: 2 -> -1, > leaderEpoch: 0 -> 1, partitionEpoch: 5 -> 6 > (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,379] DEBUG [QuorumController id=1] Replayed partition > change PartitionChangeRecord(partitionId=2, topicId=6ZIeidfiSTWRiOAmGEwn_g, > isr=null, leader=-1, replicas=null, removingReplicas=null, > addingReplicas=null, leaderRecoveryState=-1, directories=null, > eligibleLeaderReplicas=null, lastKnownELR=null) for topic quickstart-events3 > (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,379] DEBUG [QuorumController id=1] partition change for > quickstart-events3-1 with topic ID 6ZIeidfiSTWRiOAmGEwn_g: leader: 2 -> -1, > leaderEpoch: 1 -> 2, partitionEpoch: 5 -> 6 > (org.apache.kafka.controller.ReplicationControlManager) > [2024-01-18 19:34:47,379] DEBUG [QuorumController id=1] Replayed partition > change PartitionChangeRecord(partitionId=1, topicId=6ZIeidfiSTWRiOAmGEwn_g, > isr=null, leader=-1, replicas=null, removingReplicas=null, > addingReplicas=null, leaderRecoveryState=-1, directories=null, > eligibleLeaderReplicas=null, lastKnownELR=null) for topic quickstart-events3 > (org.apache.kafka.controller.ReplicationControlManager) > {code} > The issue will be fixed after restarting the broker because after restarting, > the brokerRegistration request will be sent using the latest metadata version > including `logDirs` fields. > Steps to reproduce: > 1. Deploy a Kafka 3.6 cluster > 2. Upgrade to 3.7 > 3. Creating a new topic > 4. Describe the topic -- This message was sent by Atlassian Jira (v8.20.10#820010)