This is an automated email from the ASF dual-hosted git repository.
cmccabe pushed a commit to branch 4.0
in repository https://gitbox.apache.org/repos/asf/kafka.git
The following commit(s) were added to refs/heads/4.0 by this push:
new 43a7a3acfd8 MINOR: Prevent broker fencing by adjusting
resendExponentialBackoff in BrokerLifecycleManager (#19061)
43a7a3acfd8 is described below
commit 43a7a3acfd82c9863692fb413f90b7ebd76aa923
Author: Mahsa Seifikar <[email protected]>
AuthorDate: Mon Mar 3 15:03:15 2025 -0500
MINOR: Prevent broker fencing by adjusting resendExponentialBackoff in
BrokerLifecycleManager (#19061)
This PR reduces `maxInterval` for `resendExponentialBackoff` in
`BrokerLifecycleManager` class from `broker.session.timeout.ms` to half
of its value. Setting `maxInterval` to `broker.session.timeout.ms`
caused brokers to be fenced if a resend attempt occurred near the
timeout threshold, leading to unnecessary broker fencing.
Reviewers: Colin P. McCabe <[email protected]>
---
core/src/main/scala/kafka/server/BrokerLifecycleManager.scala | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala
b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala
index de8f82ddb30..36c666b6467 100644
--- a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala
+++ b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala
@@ -97,7 +97,7 @@ class BrokerLifecycleManager(
* The exponential backoff to use for resending communication.
*/
private val resendExponentialBackoff =
- new ExponentialBackoff(100, 2, config.brokerSessionTimeoutMs.toLong, 0.02)
+ new ExponentialBackoff(100, 2, config.brokerSessionTimeoutMs.toLong / 2,
0.02)
/**
* The number of times we've tried and failed to communicate. This variable
can only be