From 2fc35c81be0e7d06f003621f71e61aa1ab1a8ff0 Mon Sep 17 00:00:00 2001 From: NICOLAS GUYOMAR Date: Tue, 7 Jan 2025 15:51:02 +0100 Subject: [PATCH] MINOR : Improve Exception log in NotEnoughReplicasException(#12394) Reviewers: Manikumar Reddy --- core/src/main/scala/kafka/cluster/Partition.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 9a9286ff51ae2..e58f5824e767b 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -1398,8 +1398,9 @@ class Partition(val topicPartition: TopicPartition, // Avoid writing to leader if there are not enough insync replicas to make it safe if (inSyncSize < minIsr && requiredAcks == -1) { - throw new NotEnoughReplicasException(s"The size of the current ISR ${partitionState.isr} " + - s"is insufficient to satisfy the min.isr requirement of $minIsr for partition $topicPartition") + throw new NotEnoughReplicasException(s"The size of the current ISR : $inSyncSize " + + s"is insufficient to satisfy the min.isr requirement of $minIsr for partition $topicPartition, " + + s"live replica(s) broker.id are : $inSyncReplicaIds") } val info = leaderLog.appendAsLeader(records, leaderEpoch = this.leaderEpoch, origin,