Skip to content

RATIS-2408. Add configurable exponential backoff reconnection for Netty DataStream client.#1349

Open
slfan1989 wants to merge 3 commits intoapache:masterfrom
slfan1989:RATIS-2408
Open

RATIS-2408. Add configurable exponential backoff reconnection for Netty DataStream client.#1349
slfan1989 wants to merge 3 commits intoapache:masterfrom
slfan1989:RATIS-2408

Conversation

@slfan1989
Copy link
Contributor

@slfan1989 slfan1989 commented Feb 15, 2026

What changes were proposed in this pull request?

Implement configurable exponential backoff with jitter for Netty DataStream client reconnections to improve reliability and reduce resource usage during connection failures.

Problem

Currently, the Netty DataStream client uses a fixed 100ms delay for reconnection attempts when the connection fails. This approach has several limitations:

  1. Resource waste: During network issues or server unavailability, constant 100ms retry intervals create unnecessary load
  2. Thundering herd: Multiple clients reconnecting simultaneously can overwhelm the server
  3. Lack of configurability: Users cannot tune reconnection behavior for their specific use cases

Solution

Implement configurable exponential backoff with jitter for DataStream client reconnections:

  1. Configuration Support:

    • raft.client.datastream.reconnect.delay - Initial reconnection delay (default: 100ms)
    • raft.client.datastream.reconnect.max-delay - Maximum backoff delay (default: 5s)
  2. Exponential Backoff:

    • Delay doubles on each failed attempt: 100ms → 200ms → 400ms → 800ms → 1600ms → 5000ms
    • Resets to initial delay upon successful connection
  3. Jitter (0.5x-1.5x):

    • Randomizes actual delay to avoid synchronized reconnection storms
    • Example: 1000ms base → actual delay between 500ms-1500ms
  4. Concurrent Safety:

    • Prevents duplicate reconnection scheduling using atomic flags
    • Ensures cleanup even if reconnection is short-circuited
  5. Adaptive Logging:

    • INFO level for short delays (≤500ms) - normal reconnection
    • WARN level for long delays (>500ms) - persistent failures

What is the link to the Apache JIRA

RATIS-2408. Add configurable exponential backoff reconnection for Netty DataStream client.

How was this patch tested?

Unit Test.

@slfan1989 slfan1989 marked this pull request as ready for review February 15, 2026 10:19
@szetszwo
Copy link
Contributor

@slfan1989 , thanks a lot for working on this!

it is a great idea to use exponential backoff. We use a similar idea for raft.server.log.appender.retry.policy. We have a RetryPolicy interface and a MultipleLinearRandomRetry implementation, which is very similar to Hadoop's implementation

How about adding a new conf raft.netty.dataStream.client.reconnect.policy which uses MultipleLinearRandomRetry ?

@slfan1989
Copy link
Contributor Author

@slfan1989 , thanks a lot for working on this!

it is a great idea to use exponential backoff. We use a similar idea for raft.server.log.appender.retry.policy. We have a RetryPolicy interface and a MultipleLinearRandomRetry implementation, which is very similar to Hadoop's implementation

How about adding a new conf raft.netty.dataStream.client.reconnect.policy which uses MultipleLinearRandomRetry ?

@szetszwo Thanks for the suggestion and for pointing out the existing retry policy!

Yeah, I agree — since we already have MultipleLinearRandomRetry working well for log appender,
reusing the same policy for raft.netty.dataStream.client.reconnect.policy makes a lot of sense.

@slfan1989
Copy link
Contributor Author

@szetszwo Thank you for the great suggestion! I took a closer look at MultipleLinearRandomRetry - it's indeed a well-designed implementation similar to Hadoop's approach.

I noticed that Ratis already has an ExponentialBackoffRetry strategy. For this PR, would it be acceptable to implement ExponentialBackoffRetry first? We can use reconnect.delay / reconnect.max-delay (+ reconnect.max-attempts) to build the exponential backoff as the default policy.

Then in a follow-up PR, I can add a configurable raft.netty.dataStream.client.reconnect.policy property to support multiple retry strategies (including MultipleLinearRandomRetry), similar to how raft.server.log.appender.retry.policy works. This way users will have the flexibility to choose different retry policies based on their needs.

Does this approach sound reasonable?

Copy link
Contributor

@szetszwo szetszwo left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@slfan1989 , thanks for the update!

Please see the comments inlined and also https://issues.apache.org/jira/secure/attachment/13080856/1349_review.patch (need to update the test)

Comment on lines +166 to +170
Preconditions.assertTrue(minReconnectMillis > 0, () -> "minReconnectMillis = " + minReconnectMillis + " <= 0");
Preconditions.assertTrue(maxReconnectMillis >= minReconnectMillis,
() -> "maxReconnectMillis = " + maxReconnectMillis + " < minReconnectMillis = " + minReconnectMillis);
Preconditions.assertTrue(maxReconnectAttempts >= 0,
() -> "maxReconnectAttempts = " + maxReconnectAttempts + " < 0");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's move the preconditions to the ExponentialBackoffRetry constructor:

  private ExponentialBackoffRetry(TimeDuration baseSleepTime, TimeDuration maxSleepTime, int maxAttempts) {
    Objects.requireNonNull(baseSleepTime, "baseSleepTime == null");
    Preconditions.assertTrue(baseSleepTime.isPositive(), () -> "baseSleepTime = " + baseSleepTime + " <= 0");
    Objects.requireNonNull(maxSleepTime, "maxSleepTime == null");
    Preconditions.assertTrue(maxSleepTime.compareTo(baseSleepTime) >= 0,
        () -> "maxSleepTime = " + maxSleepTime + " < baseSleepTime = " + baseSleepTime);
    Preconditions.assertTrue(maxAttempts >= 0, () -> "maxAttempts = " + maxAttempts + " < 0");

    this.baseSleepTime = baseSleepTime;
    this.maxSleepTime = maxSleepTime;
    this.maxAttempts = maxAttempts;
  }

Comment on lines +179 to +219

/**
* Initial delay for reconnect attempts.
* The delay doubles on each failure with 0.5x-1.5x jitter.
*/
String RECONNECT_DELAY_KEY = PREFIX + ".reconnect.delay";
TimeDuration RECONNECT_DELAY_DEFAULT = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS);
static TimeDuration reconnectDelay(RaftProperties properties) {
return getTimeDuration(properties.getTimeDuration(RECONNECT_DELAY_DEFAULT.getUnit()),
RECONNECT_DELAY_KEY, RECONNECT_DELAY_DEFAULT, getDefaultLog());
}
static void setReconnectDelay(RaftProperties properties, TimeDuration delay) {
setTimeDuration(properties::setTimeDuration, RECONNECT_DELAY_KEY, delay);
}

/**
* Maximum delay for reconnect attempts.
* The backoff increases until this upper bound.
*/
String RECONNECT_MAX_DELAY_KEY = PREFIX + ".reconnect.max-delay";
TimeDuration RECONNECT_MAX_DELAY_DEFAULT = TimeDuration.valueOf(5, TimeUnit.SECONDS);
static TimeDuration reconnectMaxDelay(RaftProperties properties) {
return getTimeDuration(properties.getTimeDuration(RECONNECT_MAX_DELAY_DEFAULT.getUnit()),
RECONNECT_MAX_DELAY_KEY, RECONNECT_MAX_DELAY_DEFAULT, getDefaultLog());
}
static void setReconnectMaxDelay(RaftProperties properties, TimeDuration delay) {
setTimeDuration(properties::setTimeDuration, RECONNECT_MAX_DELAY_KEY, delay);
}

/**
* Maximum number of reconnect attempts.
* Use {@link Integer#MAX_VALUE} for unlimited attempts.
*/
String RECONNECT_MAX_ATTEMPTS_KEY = PREFIX + ".reconnect.max-attempts";
int RECONNECT_MAX_ATTEMPTS_DEFAULT = Integer.MAX_VALUE;
static int reconnectMaxAttempts(RaftProperties properties) {
return getInt(properties::getInt, RECONNECT_MAX_ATTEMPTS_KEY, RECONNECT_MAX_ATTEMPTS_DEFAULT, getDefaultLog());
}
static void setReconnectMaxAttempts(RaftProperties properties, int maxAttempts) {
setInt(properties::setInt, RECONNECT_MAX_ATTEMPTS_KEY, maxAttempts);
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's use a single conf so that we may support other RetryPolicy later.

      /** A retry policy specified in comma separated format. */
      String RECONNECT_POLICY_KEY = PREFIX + ".reconnect.policy";
      /** ExponentialBackoffRetry with base sleep 100ms, max sleep 5ms and max attempt 100. */
      String RECONNECT_POLICY_DEFAULT = "ExponentialBackoffRetry,100ms,5s,100";
      static String reconnectPolicy(RaftProperties properties) {
        return properties.get(RECONNECT_POLICY_KEY, RECONNECT_POLICY_DEFAULT);
      }
      static void setReconnectPolicy(RaftProperties properties, String retryPolicy) {
        properties.set(RECONNECT_POLICY_KEY, retryPolicy);
      }

Comment on lines +171 to +175
this.reconnectPolicy = ExponentialBackoffRetry.newBuilder()
.setBaseSleepTime(reconnectDelay)
.setMaxSleepTime(reconnectMaxDelay)
.setMaxAttempts(maxReconnectAttempts)
.build();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's pass reconnectPolicy. Then, we may support other RetryPolicy in the future.

    Connection(InetSocketAddress address, WorkerGroupGetter workerGroup,
        Supplier<ChannelInitializer<SocketChannel>> channelInitializerSupplier, RetryPolicy reconnectPolicy) {
      this.address = address;
      this.workerGroup = workerGroup;
      this.channelInitializerSupplier = channelInitializerSupplier;
      this.reconnectPolicy = reconnectPolicy;
      this.ref = new AtomicReference<>(MemoizedSupplier.valueOf(this::connect));
    }

if (!future.isSuccess()) {
scheduleReconnect(Connection.this + " failed", future.cause());
} else {
resetReconnectAttempts();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's just set it here and remove the method.

                reconnectAttempts.set(0);

Comment on lines +245 to +246
final long delayMillis = Math.max(1L, action.getSleepTime().toLong(TimeUnit.MILLISECONDS));
final TimeDuration delay = TimeDuration.valueOf(delayMillis, TimeUnit.MILLISECONDS);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's get the delay directly from the action:

      final TimeDuration delay = action.getSleepTime();

}
if (cause != null) {
LOG.warn("", cause);
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Print also the attempt:

      if (cause != null) {
        LOG.warn("{}: {}; reconnect to {} in {} for attempt {}",
            this, message, address, delay, attempt, cause);
      } else if (delay.compareTo(FIVE_HUNDRED_MS) < 0) {
        LOG.info("{}: {}; reconnect to {} in {} for attempt {}", this, message, address, delay, attempt);
      } else {
        LOG.warn("{}: {}; reconnect to {} in {} for attempt {}", this, message, address, delay, attempt);
      }

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants