Skip to content

Commit

Permalink
[FLINK-33698][datastream] Fix the backoff calculation of ExponentialB…
Browse files Browse the repository at this point in the history
…ackoffDelayRetryStrategy in AsyncRetryStrategies

This closes apache#23830
  • Loading branch information
xiangyuf authored Dec 2, 2023
1 parent 1403feb commit 5da214c
Show file tree
Hide file tree
Showing 2 changed files with 67 additions and 3 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,12 @@
import java.util.Optional;
import java.util.function.Predicate;

/** Utility class to create concrete {@link AsyncRetryStrategy}. */
/**
* Utility class to create concrete {@link AsyncRetryStrategy}.
*
* <p><b>NOTICE:</b> For performance reasons, this utility's {@link AsyncRetryStrategy}
* implementation assumes the attempt always start from 1 and will only increase by 1 each time.
*/
public class AsyncRetryStrategies {
public static final NoRetryStrategy NO_RETRY_STRATEGY = new NoRetryStrategy();

Expand Down Expand Up @@ -151,10 +156,10 @@ public static class ExponentialBackoffDelayRetryStrategy<OUT>
private static final long serialVersionUID = 1L;
private final int maxAttempts;
private final long maxRetryDelay;
private final long initialDelay;
private final double multiplier;
private final Predicate<Collection<OUT>> resultPredicate;
private final Predicate<Throwable> exceptionPredicate;

private long lastRetryDelay;

public ExponentialBackoffDelayRetryStrategy(
Expand All @@ -169,6 +174,7 @@ public ExponentialBackoffDelayRetryStrategy(
this.multiplier = multiplier;
this.resultPredicate = resultPredicate;
this.exceptionPredicate = exceptionPredicate;
this.initialDelay = initialDelay;
this.lastRetryDelay = initialDelay;
}

Expand All @@ -180,9 +186,11 @@ public boolean canRetry(int currentAttempts) {
@Override
public long getBackoffTimeMillis(int currentAttempts) {
if (currentAttempts <= 1) {
// equivalent to initial delay
// reset to initialDelay
this.lastRetryDelay = initialDelay;
return lastRetryDelay;
}

long backoff = Math.min((long) (lastRetryDelay * multiplier), maxRetryDelay);
this.lastRetryDelay = backoff;
return backoff;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,56 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.flink.streaming.util.retryable;

import org.apache.flink.streaming.api.functions.async.AsyncRetryStrategy;
import org.apache.flink.util.TestLogger;

import org.junit.Assert;
import org.junit.Test;

/** Tests for the {@link AsyncRetryStrategies}. */
public class AsyncRetryStrategiesTest extends TestLogger {

@Test
public void testExponentialBackoffDelayRetryStrategy() {
int maxAttempts = 10;
long initialDelay = 100L;
long maxRetryDelay = 2000L;
double multiplier = 2;

AsyncRetryStrategy<Void> exponentialBackoffDelayRetryStrategy =
new AsyncRetryStrategies.ExponentialBackoffDelayRetryStrategyBuilder<Void>(
maxAttempts, initialDelay, maxRetryDelay, multiplier)
.build();

Assert.assertTrue(exponentialBackoffDelayRetryStrategy.canRetry(maxAttempts));
Assert.assertFalse(exponentialBackoffDelayRetryStrategy.canRetry(maxAttempts + 1));

// test if this strategy can be reused.
for (int j = 1; j <= 5; j++) {
long currentDelay = initialDelay;

for (int i = 1; i <= maxAttempts; i++) {
Assert.assertEquals(
currentDelay, exponentialBackoffDelayRetryStrategy.getBackoffTimeMillis(i));
currentDelay = Math.min((long) (currentDelay * multiplier), maxRetryDelay);
}
}
}
}

0 comments on commit 5da214c

Please sign in to comment.