Skip to content
Closed
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
@@ -0,0 +1,26 @@
<table class="table table-bordered">
Copy link
Contributor

Choose a reason for hiding this comment

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

Copy link
Contributor Author

Choose a reason for hiding this comment

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

It is generated by the method in the doc.

<thead>
<tr>
<th class="text-left" style="width: 20%">Key</th>
<th class="text-left" style="width: 15%">Default</th>
<th class="text-left" style="width: 65%">Description</th>
</tr>
</thead>
<tbody>
<tr>
<td><h5>restart-backoff-time-strategy.failure-rate.backoff-time</h5></td>
<td style="word-wrap: break-word;">0</td>
<td>Backoff time in milliseconds between two consecutive restart attempts.</td>
</tr>
<tr>
<td><h5>restart-backoff-time-strategy.failure-rate.failure-rate-interval</h5></td>
<td style="word-wrap: break-word;">60000</td>
<td>Time interval in milliseconds for measuring failure rate.</td>
</tr>
<tr>
<td><h5>restart-backoff-time-strategy.failure-rate.max-failures-per-interval</h5></td>
<td style="word-wrap: break-word;">1</td>
<td>Maximum number of failures in given time interval before failing a job.</td>
</tr>
</tbody>
</table>
Copy link
Contributor

Choose a reason for hiding this comment

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

This needs to be regenerated.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

OK

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.configuration;

import org.apache.flink.annotation.PublicEvolving;

/**
* Configuration options for the RestartBackoffTimeStrategy.
*/
@PublicEvolving
public class RestartBackoffTimeStrategyOptions {
Copy link
Contributor

Choose a reason for hiding this comment

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

maybe we do not want a config class for each restart strategy (basically a separate high level chapter in generated user docs)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The config of both FixedDelayRestartBackoffTimeStrategy and FailureRateRestartBackoffTimeStrategy will put in this file, which are strategies for the new scheduler.

/**
* Maximum number of failures in given time interval {@link #RESTART_BACKOFF_TIME_STRATEGY_FAILURE_RATE_FAILURE_RATE_INTERVAL}
* before failing a job in FailureRateRestartBackoffTimeStrategy.
*/
@PublicEvolving
public static final ConfigOption<Integer> RESTART_BACKOFF_TIME_STRATEGY_FAILURE_RATE_MAX_FAILURES_PER_INTERVAL = ConfigOptions
.key("restart-backoff-time-strategy.failure-rate.max-failures-per-interval")
.defaultValue(1)
.withDescription("Maximum number of failures in given time interval before failing a job.");

/**
* Time interval in which greater amount of failures than {@link #RESTART_BACKOFF_TIME_STRATEGY_FAILURE_RATE_MAX_FAILURES_PER_INTERVAL}
* causes job fail in FailureRateRestartBackoffTimeStrategy.
*/
@PublicEvolving
public static final ConfigOption<Long> RESTART_BACKOFF_TIME_STRATEGY_FAILURE_RATE_FAILURE_RATE_INTERVAL = ConfigOptions
.key("restart-backoff-time-strategy.failure-rate.failure-rate-interval")
.defaultValue(60_000L)
.withDescription("Time interval in milliseconds for measuring failure rate.");

/**
* Backoff time between two consecutive restart attempts in FailureRateRestartBackoffTimeStrategy.
*/
@PublicEvolving
public static final ConfigOption<Long> RESTART_BACKOFF_TIME_STRATEGY_FAILURE_RATE_FAILURE_RATE_BACKOFF_TIME = ConfigOptions
.key("restart-backoff-time-strategy.failure-rate.backoff-time")
.defaultValue(0L)
.withDescription("Backoff time in milliseconds between two consecutive restart attempts.");
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,149 @@
/*
* 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.runtime.util.clock.Clock;
import org.apache.flink.runtime.util.clock.SystemClock;

import java.util.ArrayDeque;
import java.util.Deque;

import static org.apache.flink.configuration.RestartBackoffTimeStrategyOptions.RESTART_BACKOFF_TIME_STRATEGY_FAILURE_RATE_FAILURE_RATE_BACKOFF_TIME;
import static org.apache.flink.configuration.RestartBackoffTimeStrategyOptions.RESTART_BACKOFF_TIME_STRATEGY_FAILURE_RATE_FAILURE_RATE_INTERVAL;
import static org.apache.flink.configuration.RestartBackoffTimeStrategyOptions.RESTART_BACKOFF_TIME_STRATEGY_FAILURE_RATE_MAX_FAILURES_PER_INTERVAL;
import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* Restart strategy which can restart when failure rate is not exceeded.
*/
public class FailureRateRestartBackoffTimeStrategy implements RestartBackoffTimeStrategy {

private final long failuresIntervalMS;

private final long backoffTimeMS;

private final int maxFailuresPerInterval;

private final Deque<Long> failureTimestamps;

private final String strategyString;

private final Clock clock;

FailureRateRestartBackoffTimeStrategy(Clock clock, int maxFailuresPerInterval, long failuresIntervalMS, long backoffTimeMS) {

Copy link
Contributor

Choose a reason for hiding this comment

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

failuresIntervalMS would be better to show that it is in milliseconds format.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

OK

checkArgument(maxFailuresPerInterval > 0, "Maximum number of restart attempts per time unit must be greater than 0.");
checkArgument(failuresIntervalMS > 0, "Failures interval must be greater than 0 ms.");
checkArgument(backoffTimeMS >= 0, "Backoff time must be at least 0 ms.");

this.failuresIntervalMS = failuresIntervalMS;
this.backoffTimeMS = backoffTimeMS;
this.maxFailuresPerInterval = maxFailuresPerInterval;
this.failureTimestamps = new ArrayDeque<>(maxFailuresPerInterval);
this.strategyString = generateStrategyString();
this.clock = checkNotNull(clock);
}

@Override
public boolean canRestart() {
if (isFailureTimestampsQueueFull()) {
Long now = clock.absoluteTimeMillis();
Long earliestFailure = failureTimestamps.peek();

return (now - earliestFailure) > failuresIntervalMS;
Copy link
Contributor

Choose a reason for hiding this comment

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

How exactly is the interval defined? Assuming my interval is 2L and I have two failures T(0), T(2) occurring at time 0 and 2, then it would not restart. Thus, the boundaries of the interval [0, 2] are inclusive. An alternative could be to make the right boundary exclusive [0, 2).

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, I think both are OK. Let’s keep it as now, since it is also consistent with the definition of legacy FailureRateRestartStrategy.

} else {
return true;
}
}

@Override
public long getBackoffTime() {
return backoffTimeMS;
Copy link
Contributor

Choose a reason for hiding this comment

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

Shall we have exception to get backoff time when restarting is suppressed?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

getBackoffTime is called only when canRestart returns true, with this contract restarting will not be suppressed.

}

@Override
public void notifyFailure(Throwable cause) {
if (isFailureTimestampsQueueFull()) {
failureTimestamps.remove();
}
failureTimestamps.add(clock.absoluteTimeMillis());
}

@Override
public String toString() {
return strategyString;
}

private boolean isFailureTimestampsQueueFull() {
return failureTimestamps.size() >= maxFailuresPerInterval;
}

private String generateStrategyString() {
StringBuilder str = new StringBuilder("FailureRateRestartBackoffTimeStrategy(");
str.append("FailureRateRestartBackoffTimeStrategy(failuresIntervalMS=");
str.append(failuresIntervalMS);
str.append(",backoffTimeMS=");
str.append(backoffTimeMS);
str.append(",maxFailuresPerInterval=");
str.append(maxFailuresPerInterval);
str.append(")");

return str.toString();
}

public static FailureRateRestartBackoffTimeStrategyFactory createFactory(final Configuration configuration) {
return new FailureRateRestartBackoffTimeStrategyFactory(
configuration.getInteger(RESTART_BACKOFF_TIME_STRATEGY_FAILURE_RATE_MAX_FAILURES_PER_INTERVAL),
configuration.getLong(RESTART_BACKOFF_TIME_STRATEGY_FAILURE_RATE_FAILURE_RATE_INTERVAL),
configuration.getLong(RESTART_BACKOFF_TIME_STRATEGY_FAILURE_RATE_FAILURE_RATE_BACKOFF_TIME));
}

/**
* The factory for creating {@link FailureRateRestartBackoffTimeStrategy}.
*/
public static class FailureRateRestartBackoffTimeStrategyFactory implements RestartBackoffTimeStrategy.Factory {

private final int maxFailuresPerInterval;

private final long failuresIntervalMS;

private final long backoffTimeMS;

public FailureRateRestartBackoffTimeStrategyFactory(
int maxFailuresPerInterval,
long failuresIntervalMS,
long backoffTimeMS) {

this.maxFailuresPerInterval = maxFailuresPerInterval;
this.failuresIntervalMS = failuresIntervalMS;
this.backoffTimeMS = backoffTimeMS;
}

@Override
public RestartBackoffTimeStrategy create() {
return new FailureRateRestartBackoffTimeStrategy(
SystemClock.getInstance(),
maxFailuresPerInterval,
failuresIntervalMS,
backoffTimeMS);
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,83 @@
/*
* 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.runtime.util.clock.ManualClock;
import org.apache.flink.util.TestLogger;

import org.junit.Test;

import java.util.concurrent.TimeUnit;

import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;

/**
* Unit tests for {@link FailureRateRestartBackoffTimeStrategy}.
*/
public class FailureRateRestartBackoffTimeStrategyTest extends TestLogger {

private final Exception failure = new Exception();

@Test
public void testManyFailuresWithinRate() {
final int numFailures = 3;
final long intervalMS = 1L;

ManualClock clock = new ManualClock();

final FailureRateRestartBackoffTimeStrategy restartStrategy =
new FailureRateRestartBackoffTimeStrategy(clock, 1, intervalMS, 0);

for (int failuresLeft = numFailures; failuresLeft > 0; failuresLeft--) {
assertTrue(restartStrategy.canRestart());
restartStrategy.notifyFailure(failure);
clock.advanceTime(intervalMS + 1, 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.

I guess the reason why we increment by intervalMS + 1 is that the right boundary of the failure interval is inclusive, right?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, you are right.

}

assertTrue(restartStrategy.canRestart());
}

@Test
public void testFailuresExceedingRate() {
final int numFailures = 3;
final long intervalMS = 10_000L;

final FailureRateRestartBackoffTimeStrategy restartStrategy =
new FailureRateRestartBackoffTimeStrategy(new ManualClock(), numFailures, intervalMS, 0);

for (int failuresLeft = numFailures; failuresLeft > 0; failuresLeft--) {
assertTrue(restartStrategy.canRestart());
restartStrategy.notifyFailure(failure);
}

assertFalse(restartStrategy.canRestart());
}

@Test
public void testBackoffTime() {
final long backoffTimeMS = 10_000L;

final FailureRateRestartBackoffTimeStrategy restartStrategy =
new FailureRateRestartBackoffTimeStrategy(new ManualClock(), 1, 1, backoffTimeMS);

assertEquals(backoffTimeMS, restartStrategy.getBackoffTime());
}
}