[
https://issues.apache.org/jira/browse/KAFKA-16886?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Igor Soarez reopened KAFKA-16886:
---------------------------------
> KRaft partition reassignment failed after upgrade to 3.7.0
> -----------------------------------------------------------
>
> Key: KAFKA-16886
> URL: https://issues.apache.org/jira/browse/KAFKA-16886
> Project: Kafka
> Issue Type: Bug
> Affects Versions: 3.7.0
> Reporter: Luke Chen
> Assignee: Igor Soarez
> Priority: Blocker
> Fix For: 3.8.0, 3.7.1
>
>
> Before upgrade, the topic image doesn't have dirID for the assignment. After
> upgrade, the assignment has the dirID. So in the
> {{{}ReplicaManager#applyDelta{}}}, we'll have have directoryId changes in
> {{{}localChanges{}}}, which will invoke {{AssignmentEvent}}
> [here|https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/server/ReplicaManager.scala#L2748].
> With that, we'll get the unexpected {{NOT_LEADER_OR_FOLLOWER}} error.
> Reproduce steps:
> # Launch a 3.6.0 controller and a 3.6.0 broker(BrokerA) in Kraft mode;
> # Create a topic with 1 partition;
> # Upgrade Broker A, B, Controllers to 3.7.0
> # Upgrade MV to 3.7: ./bin/kafka-features.sh --bootstrap-server
> localhost:9092 upgrade --metadata 3.7
> # reassign the step 2 partition to Broker B
>
> The logs in broker B:
> {code:java}
> [2024-05-31 15:33:25,763] INFO [ReplicaFetcherManager on broker 2] Removed
> fetcher for partitions Set(t1-0) (kafka.server.ReplicaFetcherManager)
> [2024-05-31 15:33:25,837] INFO [ReplicaFetcherManager on broker 2] Removed
> fetcher for partitions Set(t1-0) (kafka.server.ReplicaFetcherManager)
> [2024-05-31 15:33:25,837] INFO [ReplicaAlterLogDirsManager on broker 2]
> Removed fetcher for partitions Set(t1-0)
> (kafka.server.ReplicaAlterLogDirsManager)
> [2024-05-31 15:33:25,853] INFO Log for partition t1-0 is renamed to
> /tmp/kraft-broker-logs/t1-0.3e6d8bebc1c04f3186ad6cf63145b6fd-delete and is
> scheduled for deletion (kafka.log.LogManager)
> [2024-05-31 15:33:26,279] ERROR Controller returned error
> NOT_LEADER_OR_FOLLOWER for assignment of partition
> PartitionData(partitionIndex=0, errorCode=6) into directory
> oULBCf49aiRXaWJpO3I-GA (org.apache.kafka.server.AssignmentsManager)
> [2024-05-31 15:33:26,280] WARN Re-queueing assignments:
> [Assignment\{timestampNs=26022187148625, partition=t1:0,
> dir=/tmp/kraft-broker-logs, reason='Applying metadata delta'}]
> (org.apache.kafka.server.AssignmentsManager)
> [2024-05-31 15:33:26,786] ERROR Controller returned error
> NOT_LEADER_OR_FOLLOWER for assignment of partition
> PartitionData(partitionIndex=0, errorCode=6) into directory
> oULBCf49aiRXaWJpO3I-GA (org.apache.kafka.server.AssignmentsManager)
> [2024-05-31 15:33:27,296] WARN Re-queueing assignments:
> [Assignment\{timestampNs=26022187148625, partition=t1:0,
> dir=/tmp/kraft-broker-logs, reason='Applying metadata delta'}]
> (org.apache.kafka.server.AssignmentsManager)
> ...{{}}
> {code}
>
> Logs in controller:
> {code:java}
> [2024-05-31 15:33:25,727] INFO [QuorumController id=1] Successfully altered 1
> out of 1 partition reassignment(s).
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-05-31 15:33:25,727] INFO [QuorumController id=1] Replayed partition
> assignment change PartitionChangeRecord(partitionId=0,
> topicId=tMiJOQznTLKtOZ8rLqdgqw, isr=null, leader=-2, replicas=[6, 2],
> removingReplicas=[2], addingReplicas=[6], leaderRecoveryState=-1,
> directories=[RuDIAGGJrTG2NU6tEOkbHw, AAAAAAAAAAAAAAAAAAAAAA],
> eligibleLeaderReplicas=null, lastKnownElr=null) for topic t1
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-05-31 15:33:25,802] INFO [QuorumController id=1] AlterPartition request
> from node 2 for t1-0 completed the ongoing partition reassignment and
> triggered a leadership change. Returning NEW_LEADER_ELECTED.
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-05-31 15:33:25,802] INFO [QuorumController id=1] UNCLEAN partition
> change for t1-0 with topic ID tMiJOQznTLKtOZ8rLqdgqw: replicas: [6, 2] ->
> [6], directories: [RuDIAGGJrTG2NU6tEOkbHw, AAAAAAAAAAAAAAAAAAAAAA] ->
> [RuDIAGGJrTG2NU6tEOkbHw], isr: [2] -> [6], removingReplicas: [2] -> [],
> addingReplicas: [6] -> [], leader: 2 -> 6, leaderEpoch: 3 -> 4,
> partitionEpoch: 5 -> 6 (org.apache.kafka.controller.ReplicationControlManager)
> [2024-05-31 15:33:25,802] INFO [QuorumController id=1] Replayed partition
> assignment change PartitionChangeRecord(partitionId=0,
> topicId=tMiJOQznTLKtOZ8rLqdgqw, isr=[6], leader=6, replicas=[6],
> removingReplicas=[], addingReplicas=[], leaderRecoveryState=-1,
> directories=[RuDIAGGJrTG2NU6tEOkbHw], eligibleLeaderReplicas=null,
> lastKnownElr=null) for topic t1
> (org.apache.kafka.controller.ReplicationControlManager)
> [2024-05-31 15:33:26,277] WARN [QuorumController id=1]
> AssignReplicasToDirsRequest from broker 2 references non assigned partition
> t1-0 (org.apache.kafka.controller.ReplicationControlManager)
> [2024-05-31 15:33:26,785] WARN [QuorumController id=1]
> AssignReplicasToDirsRequest from broker 2 references non assigned partition
> t1-0 (org.apache.kafka.controller.ReplicationControlManager)
> [2024-05-31 15:33:27,293] WARN [QuorumController id=1]
> AssignReplicasToDirsRequest from broker 2 references non assigned partition
> t1-0 (org.apache.kafka.controller.ReplicationControlManager){{}}
> {code}
--
This message was sent by Atlassian Jira
(v8.20.10#820010)