Github user uce commented on a diff in the pull request:
https://github.com/apache/flink/pull/2873#discussion_r90258638
--- Diff:
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
---
@@ -731,46 +700,100 @@ public boolean
receiveAcknowledgeMessage(AcknowledgeCheckpoint message) throws E
discardState(message.getState());
}
+
+ return true;
}
else if (checkpoint != null) {
// this should not happen
throw new IllegalStateException(
"Received message for discarded
but non-removed checkpoint " + checkpointId);
}
else {
+ boolean wasPendingCheckpoint;
+
// message is for an unknown checkpoint, or
comes too late (checkpoint disposed)
if
(recentPendingCheckpoints.contains(checkpointId)) {
- isPendingCheckpoint = true;
+ wasPendingCheckpoint = true;
LOG.warn("Received late message for now
expired checkpoint attempt {}.", checkpointId);
}
else {
LOG.debug("Received message for an
unknown checkpoint {}.", checkpointId);
- isPendingCheckpoint = false;
+ wasPendingCheckpoint = false;
}
// try to discard the state so that we don't
have lingering state lying around
discardState(message.getState());
+
+ return wasPendingCheckpoint;
+ }
+ }
+ }
+
+ private void completePendingCheckpoint(PendingCheckpoint
pendingCheckpoint) throws CheckpointException {
+ final long checkpointId = pendingCheckpoint.getCheckpointId();
+ CompletedCheckpoint completedCheckpoint = null;
+
+ try {
+ completedCheckpoint =
pendingCheckpoint.finalizeCheckpoint();
+
+
completedCheckpointStore.addCheckpoint(completedCheckpoint);
+
+ rememberRecentCheckpointId(checkpointId);
+
dropSubsumedCheckpoints(completedCheckpoint.getTimestamp());
+
+ onFullyAcknowledgedCheckpoint(completedCheckpoint);
+ } catch (Exception exception) {
+ // abort the current pending checkpoint if it has not
been discarded yet
+ if(!pendingCheckpoint.isDiscarded()) {
--- End diff --
missing whitespace after if
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---