Skip to content

Commit

Permalink
[hotfix] Reduce logging verbosity from the checkpoint-related REST ha…
Browse files Browse the repository at this point in the history
…ndlers

Before the change, everytime the REST UI tried to access the checkpointing statistics, an ERROR message was logged.

When using a BATCH job, this leads to log files looking like this:

2020-11-13 15:27:38,785 ERROR org.apache.flink.runtime.rest.handler.job.checkpoints.CheckpointConfigHandler [] - Exception occurred in REST handler: Checkpointing is not enabled for this job (6df1c639d1904f8b7a54cf6a649ab567).
2020-11-13 15:27:38,788 ERROR org.apache.flink.runtime.rest.handler.job.checkpoints.CheckpointConfigHandler [] - Exception occurred in REST handler: Checkpointing is not enabled for this job (6df1c639d1904f8b7a54cf6a649ab567).
2020-11-13 15:27:38,788 ERROR org.apache.flink.runtime.rest.handler.job.checkpoints.CheckpointingStatisticsHandler [] - Exception occurred in REST handler: Checkpointing has not been enabled.
2020-11-13 15:27:38,793 ERROR org.apache.flink.runtime.rest.handler.job.checkpoints.CheckpointingStatisticsHandler [] - Exception occurred in REST handler: Checkpointing has not been enabled.
2020-11-13 15:27:38,793 ERROR org.apache.flink.runtime.rest.handler.job.checkpoints.CheckpointConfigHandler [] - Exception occurred in REST handler: Checkpointing is not enabled for this job (6df1c639d1904f8b7a54cf6a649ab567).
2020-11-13 15:27:38,797 ERROR org.apache.flink.runtime.rest.handler.job.checkpoints.CheckpointingStatisticsHandler [] - Exception occurred in REST handler: Checkpointing has not been enabled.
2020-11-13 15:27:38,797 ERROR org.apache.flink.runtime.rest.handler.job.checkpoints.CheckpointConfigHandler [] - Exception occurred in REST handler: Checkpointing is not enabled for this job (6df1c639d1904f8b7a54cf6a649ab567).
2020-11-13 15:27:38,802 ERROR org.apache.flink.runtime.rest.handler.job.checkpoints.CheckpointingStatisticsHandler [] - Exception occurred in REST handler: Checkpointing has not been enabled.

With this change, these log messages are surpressed.
  • Loading branch information
rmetzger committed Nov 20, 2020
1 parent c8b7863 commit 8dca9fa
Show file tree
Hide file tree
Showing 4 changed files with 25 additions and 3 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -223,7 +223,7 @@ private CompletableFuture<Void> handleException(Throwable throwable, ChannelHand
String truncatedStackTrace = Ascii.truncate(stackTrace, maxLength, "...");
if (log.isDebugEnabled()) {
log.error("Exception occurred in REST handler.", rhe);
} else {
} else if (rhe.logException()) {
log.error("Exception occurred in REST handler: {}", rhe.getMessage());
}
return HandlerUtils.sendErrorResponse(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,17 +30,39 @@ public class RestHandlerException extends FlinkException {

private final int responseCode;

// This flag indicates whether the AbstractHandler should log about this exception on INFO level or not.
private final LoggingBehavior loggingBehavior;

public RestHandlerException(String errorMessage, HttpResponseStatus httpResponseStatus) {
super(errorMessage);
this.responseCode = httpResponseStatus.code();
this.loggingBehavior = LoggingBehavior.LOG;
}

public RestHandlerException(String errorMessage, HttpResponseStatus httpResponseStatus, LoggingBehavior loggingBehavior) {
super(errorMessage);
this.responseCode = httpResponseStatus.code();
this.loggingBehavior = loggingBehavior;
}

public RestHandlerException(String errorMessage, HttpResponseStatus httpResponseStatus, Throwable cause) {
super(errorMessage, cause);
this.responseCode = httpResponseStatus.code();
this.loggingBehavior = LoggingBehavior.LOG;
}

public HttpResponseStatus getHttpResponseStatus() {
return HttpResponseStatus.valueOf(responseCode);
}

public boolean logException() {
return LoggingBehavior.LOG == loggingBehavior;
}

/**
* Enum to control logging behavior of RestHandlerExceptions.
*/
public enum LoggingBehavior {
LOG, IGNORE
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,7 @@ private static CheckpointConfigInfo createCheckpointConfigInfo(AccessExecutionGr
if (checkpointCoordinatorConfiguration == null) {
throw new RestHandlerException(
"Checkpointing is not enabled for this job (" + executionGraph.getJobID() + ").",
HttpResponseStatus.NOT_FOUND);
HttpResponseStatus.NOT_FOUND, RestHandlerException.LoggingBehavior.IGNORE);
} else {
CheckpointRetentionPolicy retentionPolicy = checkpointCoordinatorConfiguration.getCheckpointRetentionPolicy();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -91,7 +91,7 @@ private static CheckpointingStatistics createCheckpointingStatistics(AccessExecu
final CheckpointStatsSnapshot checkpointStatsSnapshot = executionGraph.getCheckpointStatsSnapshot();

if (checkpointStatsSnapshot == null) {
throw new RestHandlerException("Checkpointing has not been enabled.", HttpResponseStatus.NOT_FOUND);
throw new RestHandlerException("Checkpointing has not been enabled.", HttpResponseStatus.NOT_FOUND, RestHandlerException.LoggingBehavior.IGNORE);
} else {
final CheckpointStatsCounts checkpointStatsCounts = checkpointStatsSnapshot.getCounts();

Expand Down

0 comments on commit 8dca9fa

Please sign in to comment.