Skip to content

Enhance exponential backoff #335

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
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 @@ -18,28 +18,54 @@

package org.apache.flink.statefun.flink.core.backpressure;

import static java.lang.Math.random;
import static java.lang.Math.round;

import java.time.Duration;
import java.util.Objects;
import org.apache.flink.annotation.VisibleForTesting;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

public final class BoundedExponentialBackoff {
private static final Logger LOG = LoggerFactory.getLogger(BoundedExponentialBackoff.class);

private final Timer timer;
private final long requestStartTimeInNanos;
private final long maxRequestDurationInNanos;
private final double backOffIncreaseFactor;
private final double jitter;

private long nextSleepTimeNanos;

public BoundedExponentialBackoff(Duration initialBackoffDuration, Duration maxRequestDuration) {
this(SystemNanoTimer.instance(), initialBackoffDuration, maxRequestDuration);
public BoundedExponentialBackoff(
Duration initialBackoffDuration,
double backOffIncreaseFactor,
double jitter,
Duration maxRequestDuration) {
this(
SystemNanoTimer.instance(),
initialBackoffDuration,
backOffIncreaseFactor,
jitter,
maxRequestDuration);
}

@VisibleForTesting
BoundedExponentialBackoff(
Timer timer, Duration initialBackoffDuration, Duration maxRequestDuration) {
Timer timer,
Duration initialBackoffDuration,
double backOffIncreaseFactor,
double jitter,
Duration maxRequestDuration) {
this.timer = Objects.requireNonNull(timer);
this.requestStartTimeInNanos = timer.now();
this.backOffIncreaseFactor = backOffIncreaseFactor;
this.jitter = jitter;
this.maxRequestDurationInNanos = maxRequestDuration.toNanos();
this.nextSleepTimeNanos = initialBackoffDuration.toNanos();
this.nextSleepTimeNanos =
round(initialBackoffDuration.toNanos() * (1.0 + jitter * (-1.0 + 2.0 * random())));
}

public boolean applyNow() {
Expand All @@ -49,6 +75,12 @@ public boolean applyNow() {
if (actualSleep <= 0) {
return false;
}

LOG.info(
String.format(
"Applying exponential backoff. Sleeping for %f seconds.",
actualSleep * Math.pow(10, -9)));

timer.sleep(actualSleep);
return true;
}
Expand All @@ -60,7 +92,8 @@ private long remainingNanosUntilDeadLine() {

private long nextAmountOfNanosToSleep() {
final long current = nextSleepTimeNanos;
nextSleepTimeNanos *= 2;
nextSleepTimeNanos =
round(nextSleepTimeNanos * backOffIncreaseFactor * (1 + jitter * (-1 + 2 * random())));
return current;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -40,10 +40,11 @@

@SuppressWarnings("NullableProblems")
final class RetryingCallback implements Callback {
private static final Duration INITIAL_BACKOFF_DURATION = Duration.ofMillis(10);
private static final Duration INITIAL_BACKOFF_DURATION = Duration.ofMillis(1000);
private static final double BACKOFF_JITTER = 0.1;
private static final double BACKOFF_INCREASE_FACTOR = 2;

private static final Set<Integer> RETRYABLE_HTTP_CODES =
new HashSet<>(Arrays.asList(409, 420, 408, 429, 499, 500));
private static final Set<Integer> RETRYABLE_HTTP_CODES = new HashSet<>(Arrays.asList(409, 420, 408, 429, 499, 500));

private static final Logger LOG = LoggerFactory.getLogger(RetryingCallback.class);

Expand All @@ -61,7 +62,8 @@ final class RetryingCallback implements Callback {
Timeout timeout,
BooleanSupplier isShutdown) {
this.resultFuture = new CompletableFuture<>();
this.backoff = new BoundedExponentialBackoff(INITIAL_BACKOFF_DURATION, duration(timeout));
this.backoff = new BoundedExponentialBackoff(
INITIAL_BACKOFF_DURATION, BACKOFF_INCREASE_FACTOR, BACKOFF_JITTER, duration(timeout));
this.requestSummary = requestSummary;
this.metrics = metrics;
this.isShutdown = Objects.requireNonNull(isShutdown);
Expand Down Expand Up @@ -90,13 +92,22 @@ private void onFailureUnsafe(Call call, IOException cause) {
if (isShutdown.getAsBoolean()) {
throw new IllegalStateException("An exception caught during shutdown.", cause);
}

final double callDurationSeconds = timeSinceRequestStartedInNanoseconds() / Math.pow(10, 9);
LOG.warn(
"Retriable exception caught while trying to deliver a message: " + requestSummary, cause);
"Retriable exception caught after "
+ callDurationSeconds
+ " seconds while trying to deliver a message: "
+ requestSummary,
cause);
metrics.remoteInvocationFailures();

if (!retryAfterApplyingBackoff(call)) {
throw new IllegalStateException(
"Maximal request time has elapsed. Last cause is attached", cause);
"Maximal request time has elapsed after "
+ callDurationSeconds
+ " seconds. Last cause is attached",
cause);
}
}

Expand All @@ -109,9 +120,12 @@ private void onResponseUnsafe(Call call, Response response) {
throw new IllegalStateException("Non successful HTTP response code " + response.code());
}
if (!retryAfterApplyingBackoff(call)) {
final double callDurationSeconds = timeSinceRequestStartedInNanoseconds() / Math.pow(10, 9);

throw new IllegalStateException(
"Maximal request time has elapsed. Last known error is: invalid HTTP response code "
+ response.code());
String.format(
"Maximal request time has elapsed after %f seconds. Last known error is: invalid HTTP response code %d",
callDurationSeconds, response.code()));
}
}

Expand All @@ -130,7 +144,8 @@ private boolean retryAfterApplyingBackoff(Call call) {
}

/**
* Executes the runnable, and completes {@link #resultFuture} with any exceptions thrown, during
* Executes the runnable, and completes {@link #resultFuture} with any
* exceptions thrown, during
* its execution.
*/
private void tryWithFuture(RunnableWithException runnable) {
Expand All @@ -147,8 +162,12 @@ private static Duration duration(Timeout timeout) {
}

private void endTimingRequest() {
final long nanosecondsElapsed = System.nanoTime() - requestStarted;
final long nanosecondsElapsed = this.timeSinceRequestStartedInNanoseconds();
final long millisecondsElapsed = TimeUnit.NANOSECONDS.toMillis(nanosecondsElapsed);
metrics.remoteInvocationLatency(millisecondsElapsed);
}

private long timeSinceRequestStartedInNanoseconds() {
return System.nanoTime() - requestStarted;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@
public class BoundedExponentialBackoffTest {
private final FakeNanoClock fakeTime = new FakeNanoClock();
private final BoundedExponentialBackoff backoffUnderTest =
new BoundedExponentialBackoff(fakeTime, Duration.ofSeconds(1), Duration.ofMinutes(1));
new BoundedExponentialBackoff(fakeTime, Duration.ofSeconds(1), 2, 0.1, Duration.ofMinutes(1));

@Test
public void simpleUsage() {
Expand Down