Skip to content

Commit

Permalink
[FLINK-22431] Add information when and why the AdaptiveScheduler rest…
Browse files Browse the repository at this point in the history
…arts or fails jobs

This commit adds info log statements to tell the user when and why it restarts or fails a job.

This closes apache#15736.
  • Loading branch information
tillrohrmann committed Apr 26, 2021
1 parent fb31e28 commit 6be9668
Show file tree
Hide file tree
Showing 5 changed files with 17 additions and 14 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -1058,7 +1058,7 @@ public Executing.FailureResult howToHandleFailure(Throwable failure) {
restartBackoffTimeStrategy.notifyFailure(failure);
if (restartBackoffTimeStrategy.canRestart()) {
return Executing.FailureResult.canRestart(
Duration.ofMillis(restartBackoffTimeStrategy.getBackoffTime()));
failure, Duration.ofMillis(restartBackoffTimeStrategy.getBackoffTime()));
} else {
return Executing.FailureResult.canNotRestart(
new JobException(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -85,12 +85,14 @@ private void handleAnyFailure(Throwable cause) {
final FailureResult failureResult = context.howToHandleFailure(cause);

if (failureResult.canRestart()) {
getLogger().info("Restarting job.", failureResult.getFailureCause());
context.goToRestarting(
getExecutionGraph(),
getExecutionGraphHandler(),
getOperatorCoordinatorHandler(),
failureResult.getBackoffTime());
} else {
getLogger().info("Failing job.", failureResult.getFailureCause());
context.goToFailing(
getExecutionGraph(),
getExecutionGraphHandler(),
Expand Down Expand Up @@ -281,9 +283,9 @@ CompletableFuture<String> goToStopWithSavepoint(
static final class FailureResult {
@Nullable private final Duration backoffTime;

@Nullable private final Throwable failureCause;
private final Throwable failureCause;

private FailureResult(@Nullable Duration backoffTime, @Nullable Throwable failureCause) {
private FailureResult(Throwable failureCause, @Nullable Duration backoffTime) {
this.backoffTime = backoffTime;
this.failureCause = failureCause;
}
Expand All @@ -299,20 +301,18 @@ Duration getBackoffTime() {
}

Throwable getFailureCause() {
Preconditions.checkState(
failureCause != null,
"Failure result must not be restartable to return a failure cause.");
return failureCause;
}

/**
* Creates a FailureResult which allows to restart the job.
*
* @param failureCause failureCause for restarting the job
* @param backoffTime backoffTime to wait before restarting the job
* @return FailureResult which allows to restart the job
*/
static FailureResult canRestart(Duration backoffTime) {
return new FailureResult(backoffTime, null);
static FailureResult canRestart(Throwable failureCause, Duration backoffTime) {
return new FailureResult(failureCause, backoffTime);
}

/**
Expand All @@ -322,7 +322,7 @@ static FailureResult canRestart(Duration backoffTime) {
* @return FailureResult which does not allow to restart the job
*/
static FailureResult canNotRestart(Throwable failureCause) {
return new FailureResult(null, failureCause);
return new FailureResult(failureCause, null);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -179,12 +179,14 @@ private void handleAnyFailure(Throwable cause) {
final Executing.FailureResult failureResult = context.howToHandleFailure(cause);

if (failureResult.canRestart()) {
getLogger().info("Restarting job.", failureResult.getFailureCause());
context.goToRestarting(
getExecutionGraph(),
getExecutionGraphHandler(),
getOperatorCoordinatorHandler(),
failureResult.getBackoffTime());
} else {
getLogger().info("Failing job.", failureResult.getFailureCause());
context.goToFailing(
getExecutionGraph(),
getExecutionGraphHandler(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -140,7 +140,7 @@ public void testRecoverableGlobalFailureTransitionsToRestarting() throws Excepti
ctx.setExpectRestarting(
(restartingArguments ->
assertThat(restartingArguments.getBackoffTime(), is(duration))));
ctx.setHowToHandleFailure((t) -> Executing.FailureResult.canRestart(duration));
ctx.setHowToHandleFailure((t) -> Executing.FailureResult.canRestart(t, duration));
exec.handleGlobalFailure(new RuntimeException("Recoverable error"));
}
}
Expand Down Expand Up @@ -234,7 +234,8 @@ public void testFailureReportedViaUpdateTaskExecutionStateCausesRestart() throws
new ExecutingStateBuilder()
.setExecutionGraph(returnsFailedStateExecutionGraph)
.build(ctx);
ctx.setHowToHandleFailure((ign) -> Executing.FailureResult.canRestart(Duration.ZERO));
ctx.setHowToHandleFailure(
(throwable) -> Executing.FailureResult.canRestart(throwable, Duration.ZERO));
ctx.setExpectRestarting(assertNonNull());

exec.updateTaskExecutionState(createFailingStateTransition());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -174,7 +174,7 @@ public void testRestartOnGlobalFailureIfRestartConfigured() throws Exception {
StopWithSavepoint sws = createStopWithSavepoint(ctx);
ctx.setStopWithSavepoint(sws);
ctx.setHowToHandleFailure(
(ignore) -> Executing.FailureResult.canRestart(Duration.ZERO));
(throwable) -> Executing.FailureResult.canRestart(throwable, Duration.ZERO));

ctx.setExpectRestarting(assertNonNull());

Expand Down Expand Up @@ -229,7 +229,7 @@ public void testRestartingOnUpdateTaskExecutionStateWithRestart() throws Excepti
createStopWithSavepoint(ctx, new StateTrackingMockExecutionGraph());
ctx.setStopWithSavepoint(sws);
ctx.setHowToHandleFailure(
(ignore) -> Executing.FailureResult.canRestart(Duration.ZERO));
(throwable) -> Executing.FailureResult.canRestart(throwable, Duration.ZERO));

ctx.setExpectRestarting(assertNonNull());

Expand Down Expand Up @@ -277,7 +277,7 @@ public void testRestartOnTaskFailureAfterSavepointCompletion() throws Exception
ctx.setStopWithSavepoint(sws);

ctx.setHowToHandleFailure(
(ignore) -> Executing.FailureResult.canRestart(Duration.ZERO));
(throwable) -> Executing.FailureResult.canRestart(throwable, Duration.ZERO));

ctx.setExpectRestarting(assertNonNull());

Expand Down

0 comments on commit 6be9668

Please sign in to comment.