[
https://issues.apache.org/jira/browse/FLINK-5823?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15949138#comment-15949138
]
ASF GitHub Bot commented on FLINK-5823:
---------------------------------------
Github user tillrohrmann commented on a diff in the pull request:
https://github.com/apache/flink/pull/3522#discussion_r108451291
--- Diff:
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
---
@@ -758,17 +769,36 @@ else if (checkpoint != null) {
* @throws CheckpointException if the completion failed
*/
private void completePendingCheckpoint(PendingCheckpoint
pendingCheckpoint) throws CheckpointException {
+ assert Thread.holdsLock(lock);
+
final long checkpointId = pendingCheckpoint.getCheckpointId();
- CompletedCheckpoint completedCheckpoint = null;
+ final boolean externalize =
+
completedCheckpointStore.requiresExternalizedCheckpoints() ||
+
pendingCheckpoint.getProps().externalizeCheckpoint() ||
+ pendingCheckpoint.getProps().isSavepoint();
+
+ CompletedCheckpoint completedCheckpoint = null;
try {
// externalize the checkpoint if required
- if
(pendingCheckpoint.getProps().externalizeCheckpoint()) {
+ if (externalize) {
completedCheckpoint =
pendingCheckpoint.finalizeCheckpointExternalized();
} else {
completedCheckpoint =
pendingCheckpoint.finalizeCheckpointNonExternalized();
}
+ try {
+ if (globalStateHooks != null) {
+ completedCheckpoint.registerDisposeHook(
+
globalStateHooks.createCheckpointDisposeHook(completedCheckpoint));
+ }
+ }
+ catch (Exception e) {
+ // catch all exception, to not let errors in
cleanup hooks fail the checkpoint
+ LOG.warn("Failed to create the cleanup hook for
checkpoint " + checkpointId +
+ ". Generic cleanup path will be
executed...");
--- End diff --
`e` might be interesting to log since it could contain information what
went wrong.
> Store Checkpoint Root Metadata in StateBackend (not in HA custom store)
> -----------------------------------------------------------------------
>
> Key: FLINK-5823
> URL: https://issues.apache.org/jira/browse/FLINK-5823
> Project: Flink
> Issue Type: Sub-task
> Components: State Backends, Checkpointing
> Reporter: Stephan Ewen
> Assignee: Stephan Ewen
> Fix For: 1.3.0
>
>
--
This message was sent by Atlassian JIRA
(v6.3.15#6346)