azagrebin commented on a change in pull request #6567: [FLINK-10074] Allowable
number of checkpoint failures
URL: https://github.com/apache/flink/pull/6567#discussion_r214398085
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/CheckpointExceptionHandlerFactory.java
##########
@@ -48,12 +48,41 @@ public CheckpointExceptionHandler
createCheckpointExceptionHandler(
*/
static final class FailingCheckpointExceptionHandler implements
CheckpointExceptionHandler {
+ final Environment environment;
+ final int tolerableNumber;
+ long latestFailedCheckpointID;
+ int cpFailureCounter;
+
+ FailingCheckpointExceptionHandler(Environment environment) {
+ this.environment = environment;
+ this.cpFailureCounter = 0;
+ this.tolerableNumber =
environment.getExecutionConfig().getTaskTolerableCheckpointFailuresNumber();
+ }
+
@Override
public void tryHandleCheckpointException(
CheckpointMetaData checkpointMetaData,
Exception exception) throws Exception {
- throw exception;
+ if (needThrowCheckpointException(checkpointMetaData)) {
+ throw exception;
+ }
+ }
+
+ private boolean needThrowCheckpointException(CheckpointMetaData
checkpointMetaData) {
+ if (tolerableNumber == 0) {
+ return true;
+ }
+
+ if (checkpointMetaData.getCheckpointId() -
latestFailedCheckpointID == 1) {
Review comment:
I think rather than relying on sequential numbering of checkpoints,
it is better we add one more signal:
`CheckpointExceptionHandler.checkpointSucceeded()` where the counter is reset.
This method can be called in `AsyncCheckpointRunnable.run()`, e.g. after
`reportCompletedSnapshotStates` is done:
```
owner.asynchronousCheckpointExceptionHandler.checkpointSucceeded(); //
forward it to synchronousCheckpointExceptionHandler inside
```
The checkpoints finish concurrently, so I think we have to use an
`AtomicInteger` for the `cpFailureCounter` and
`cpFailureCounter.incrementAndGet()`.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
With regards,
Apache Git Services