splett2 commented on code in PR #14444: URL: https://github.com/apache/kafka/pull/14444#discussion_r1358490055
########## core/src/main/scala/kafka/server/KafkaApis.scala: ########## @@ -559,6 +560,26 @@ class KafkaApis(val requestChannel: RequestChannel, } } + case class LeaderNode(leaderId: Int, leaderEpoch: Int, node: Node) + + private def getCurrentLeader(tp: TopicPartition): LeaderNode = { + val partitionInfoOrError = replicaManager.getPartitionOrError(tp) + val (leaderId, leaderEpoch) = partitionInfoOrError match { + case Right(x) => + (x.leaderReplicaIdOpt.getOrElse(-1), x.getLeaderEpoch) + case Left(x) => + debug(s"Unable to retrieve local leaderId and Epoch with error $x, falling back to metadata cache") + metadataCache.getPartitionInfo(tp.topic, tp.partition) match { + case Some(pinfo) => (pinfo.leader(), pinfo.leaderEpoch()) + case None => (-1, -1) + } + } + val leaderNode: Node = metadataCache.getAliveBrokerNode(leaderId, config.interBrokerListenerName).getOrElse({ Review Comment: we shouldn't be passing through the interbroker listener name, we should be using the listener used by the original request to be consistent with the metadata request. Is it simpler if we just consult the metadata cache? In KRaft mode, the metadata cache is the source of truth for partition leadership and is updated before the partition state gets updated. ########## core/src/main/scala/kafka/server/KafkaApis.scala: ########## @@ -620,6 +642,21 @@ class KafkaApis(val requestChannel: RequestChannel, topicPartition, status.error.exceptionName)) } + + if (request.header.apiVersion >= 10) { + status.currentLeader = { + status.error match { + case Errors.NOT_LEADER_OR_FOLLOWER | Errors.FENCED_LEADER_EPOCH => Review Comment: produce requests should never receive FENCED_LEADER_EPOCH. also, shouldn't this go in the above `if` block? -- 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