You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/12/23 16:28:02 UTC

[GitHub] [flink] tillrohrmann commented on a change in pull request #14425: [FLINK-19773][runtime] Implement ExponentialDelayStrategy

tillrohrmann commented on a change in pull request #14425:
URL: https://github.com/apache/flink/pull/14425#discussion_r548010140



##########
File path: flink-core/src/main/java/org/apache/flink/api/common/restartstrategy/RestartStrategies.java
##########
@@ -85,6 +85,30 @@ public static FailureRateRestartStrategyConfiguration failureRateRestart(
 		return new FailureRateRestartStrategyConfiguration(failureRate, failureInterval, delayInterval);
 	}
 
+	/**
+	 * Generates a ExponentialDelayRestartStrategyConfiguration.
+	 *
+	 * @param initialBackoff Starting duration between restarts
+	 * @param maxBackoff The highest possible duration between restarts
+	 * @param backoffMultiplier Delay multiplier how many times is the delay longer than before
+	 * @param resetBackoffThreshold How long the job must run smoothly to reset the time interval
+	 * @param jitterFactor How much the delay may differ (in percentage)
+	 */
+	public static ExponentialDelayRestartStrategyConfiguration exponentialDelayRestart(
+		Time initialBackoff,
+		Time maxBackoff,
+		double backoffMultiplier,
+		Time resetBackoffThreshold,
+		double jitterFactor
+	) {

Review comment:
       nit: using the style of this file, the closing parenthesis is usually put in the same line as the last parameter.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExponentialDelayRestartBackoffTimeStrategy.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.runtime.executiongraph.failover.flip1;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.RestartStrategyOptions;
+import org.apache.flink.util.clock.Clock;
+import org.apache.flink.util.clock.SystemClock;
+
+import java.util.concurrent.ThreadLocalRandom;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Restart strategy which tries to restart indefinitely number of times with a exponential backoff time in between.

Review comment:
       ```suggestion
    * Restart strategy which tries to restart indefinitely number of times with an exponential backoff time in between.
   ```

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExponentialDelayRestartBackoffTimeStrategyTest.java
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.runtime.executiongraph.failover.flip1;
+
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.clock.ManualClock;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.lessThanOrEqualTo;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Unit tests for {@link ExponentialDelayRestartBackoffTimeStrategy}.
+ */
+public class ExponentialDelayRestartBackoffTimeStrategyTest extends TestLogger {
+
+	private final Exception failure = new Exception();
+
+	@Test
+	public void testAlwaysRestart() throws Exception {
+		final ExponentialDelayRestartBackoffTimeStrategy restartStrategy =
+			new ExponentialDelayRestartBackoffTimeStrategy(
+				new ManualClock(),
+				1L,
+				3L,
+				2.0,
+				4L,
+				0.25);
+
+		for (int i = 0; i < 13; i++) {
+			assertTrue(restartStrategy.canRestart());
+			restartStrategy.notifyFailure(failure);
+		}
+	}
+
+	@Test
+	public void testInitialBackoff() throws Exception {
+		long initialBackoffMS = 42L;
+
+		final ExponentialDelayRestartBackoffTimeStrategy restartStrategy =
+			new ExponentialDelayRestartBackoffTimeStrategy(
+				new ManualClock(),
+				initialBackoffMS,
+				45L,
+				2.0,
+				8L,
+				0);
+
+		assertThat(restartStrategy.getBackoffTime(), is(initialBackoffMS));
+	}
+
+	@Test
+	public void testMaxBackoff() throws Exception {
+		final long maxBackoffMS = 6L;
+
+		final ExponentialDelayRestartBackoffTimeStrategy restartStrategy =
+			new ExponentialDelayRestartBackoffTimeStrategy(
+				new ManualClock(),
+				1L,
+				maxBackoffMS,
+				2.0,
+				8L,
+				0.25);
+
+		for (int i = 0; i < 10; i++) {
+			restartStrategy.notifyFailure(failure);
+			assertThat(restartStrategy.getBackoffTime(), lessThanOrEqualTo(maxBackoffMS));
+		}
+	}
+
+	@Test
+	public void testResetBackoff() throws Exception {
+		final long initialBackoffMS = 1L;
+		final long resetBackoffThresholdMS = 8L;
+		final ManualClock clock = new ManualClock();
+
+		final ExponentialDelayRestartBackoffTimeStrategy restartStrategy =
+			new ExponentialDelayRestartBackoffTimeStrategy(clock,
+				initialBackoffMS, 5L, 2.0, resetBackoffThresholdMS, 0.25);
+
+		clock.advanceTime(resetBackoffThresholdMS + restartStrategy.getBackoffTime() - 1, TimeUnit.MILLISECONDS);
+		restartStrategy.notifyFailure(failure);
+		assertThat("Backoff should be increased", restartStrategy.getBackoffTime(), is(2L));
+
+		clock.advanceTime(resetBackoffThresholdMS + restartStrategy.getBackoffTime(), TimeUnit.MILLISECONDS);
+		restartStrategy.notifyFailure(failure);
+		assertThat(
+			"Backoff should be reset",
+			restartStrategy.getBackoffTime(),
+			is(initialBackoffMS));
+	}
+
+	@Test
+	public void testBackoffMultiplier() throws Exception {
+		long initialBackoffMS = 4L;
+		double jitterFactor = 0;
+		double backoffMultiplier = 2.3;
+		long maxBackoffMS = 300L;
+
+		final ExponentialDelayRestartBackoffTimeStrategy restartStrategy =
+			new ExponentialDelayRestartBackoffTimeStrategy(
+				new ManualClock(),
+				initialBackoffMS,
+				maxBackoffMS,
+				backoffMultiplier,
+				8L,
+				jitterFactor);
+
+		restartStrategy.notifyFailure(failure);
+		assertThat(restartStrategy.getBackoffTime(), is(9L)); // 4 * 2.3
+
+		restartStrategy.notifyFailure(failure);
+		assertThat(restartStrategy.getBackoffTime(), is(20L)); // 9 * 2.3
+	}
+
+	@Test
+	public void testJitter() throws Exception {
+		final long initialBackoffMS = 2L;
+		final long maxBackoffMS = 7L;
+
+		final ExponentialDelayRestartBackoffTimeStrategy restartStrategy =
+			new ExponentialDelayRestartBackoffTimeStrategy(
+				new ManualClock(),
+				initialBackoffMS,
+				maxBackoffMS,
+				2.0,
+				1L,
+				0.25);
+
+		restartStrategy.notifyFailure(failure);
+		assertCorrectRandomRange(restartStrategy::getBackoffTime, 3L, 4L, 5L);
+
+		restartStrategy.notifyFailure(failure);
+		assertCorrectRandomRange(restartStrategy::getBackoffTime, 6L, 7L);
+
+		restartStrategy.notifyFailure(failure);
+		assertCorrectRandomRange(restartStrategy::getBackoffTime, 6L, 7L);
+	}
+
+	@Test
+	public void testJitterNoHigherThanMax() throws Exception {
+		double jitterFactor = 1;
+
+		final ExponentialDelayRestartBackoffTimeStrategy restartStrategy =
+			new ExponentialDelayRestartBackoffTimeStrategy(
+				new ManualClock(),
+				1L,
+				9L,
+				2.0,
+				8L,
+				jitterFactor);
+
+		assertCorrectRandomRange(restartStrategy::getBackoffTime, 0L, 1L, 2L);
+
+		restartStrategy.notifyFailure(failure);
+		assertCorrectRandomRange(restartStrategy::getBackoffTime, 0L, 1L, 2L, 3L, 4L);
+
+		restartStrategy.notifyFailure(failure);
+		assertCorrectRandomRange(
+			restartStrategy::getBackoffTime,
+			0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L);

Review comment:
       Shouldn't we set the max backoff time to something like `7` in order to test that we will respect it? At the moment it is set to `9` which means that we won't exceed the max backoff time after three failures if I am not mistaken.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org