Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@
import java.util.Collections;

import static org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.PollResult.EMPTY;
import static org.apache.kafka.clients.consumer.internals.RequestState.RETRY_BACKOFF_JITTER;

/**
* <p>Manages the request creation and response handling for the heartbeat. The module creates a
Expand Down Expand Up @@ -113,7 +114,7 @@ public abstract class AbstractHeartbeatRequestManager<R extends AbstractResponse
long retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
long retryBackoffMaxMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MAX_MS_CONFIG);
this.heartbeatRequestState = new HeartbeatRequestState(logContext, time, 0, retryBackoffMs,
retryBackoffMaxMs, maxPollIntervalMs);
retryBackoffMaxMs, RETRY_BACKOFF_JITTER);
this.pollTimer = time.timer(maxPollIntervalMs);
this.metricsManager = metricsManager;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,7 @@
import java.util.stream.IntStream;

import static org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.PollResult.EMPTY;
import static org.apache.kafka.clients.consumer.internals.RequestState.RETRY_BACKOFF_JITTER;

/**
* <p>Manages the request creation and response handling for the streams group heartbeat. The class creates a
Expand Down Expand Up @@ -330,7 +331,7 @@ public StreamsGroupHeartbeatRequestManager(final LogContext logContext,
0,
retryBackoffMs,
retryBackoffMaxMs,
maxPollIntervalMs
RETRY_BACKOFF_JITTER
);
this.pollTimer = time.timer(maxPollIntervalMs);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,9 @@ public ExponentialBackoff(long initialInterval, int multiplier, long maxInterval
this.initialInterval = Math.min(maxInterval, initialInterval);
this.multiplier = multiplier;
this.maxInterval = maxInterval;
if (jitter < 0 || jitter > 1) {
throw new IllegalArgumentException("jitter must be between 0 and 1");
Copy link
Member

Choose a reason for hiding this comment

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

Plesae include the jitter value in the error message. It will be very useful for debugging

Copy link
Member Author

Choose a reason for hiding this comment

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

thanks, updated in 5d15580

}
this.jitter = jitter;
this.expMax = maxInterval > initialInterval ?
Math.log(maxInterval / (double) Math.max(initialInterval, 1)) / Math.log(multiplier) : 0;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import org.junit.jupiter.api.Test;

import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;

public class ExponentialBackoffTest {
Expand Down Expand Up @@ -54,4 +55,14 @@ public void testExponentialBackoffWithoutJitter() {
assertEquals(400, exponentialBackoff.backoff(2));
assertEquals(400, exponentialBackoff.backoff(3));
}

@Test
public void testExponentialBackoffWithInvalidJitter() {
assertEquals("jitter must be between 0 and 1",
assertThrows(IllegalArgumentException.class,
() -> new ExponentialBackoff(100, 2, 400, -1)).getMessage());
assertEquals("jitter must be between 0 and 1",
assertThrows(IllegalArgumentException.class,
() -> new ExponentialBackoff(100, 2, 400, 3000)).getMessage());
}
}