[GitHub] flink pull request #2873: [backport] [FLINK-5158] [ckPtCoord] Handle excepti...
Github user tillrohrmann closed the pull request at: https://github.com/apache/flink/pull/2873 --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2873: [backport] [FLINK-5158] [ckPtCoord] Handle excepti...
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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2873: [backport] [FLINK-5158] [ckPtCoord] Handle excepti...
Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/2873#discussion_r90258544 --- 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 { --- End diff -- Missing JavaDocs, maybe add that this needs to be called in checkpoint lock scope --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2873: [backport] [FLINK-5158] [ckPtCoord] Handle excepti...
Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/2873#discussion_r90258796 --- 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()) { + pendingCheckpoint.discard(userClassLoader); + } + + if (completedCheckpoint != null) { + // we failed to store the completed checkpoint. Let's clean up + final CompletedCheckpoint cc = completedCheckpoint; + + executor.execute(new Runnable() { + @Override + public void run() { + try { + cc.discard(userClassLoader); + } catch (Exception nestedException) { + LOG.warn("Could not properly discard completed checkpoint {}.", cc.getCheckpointID(), nestedException); + } + } + }); } + + throw new CheckpointException("Could not complete the pending checkpoint " + checkpointId + '.', exception); + } finally { + pendingCheckpoints.remove(checkpointId); + + triggerQueuedRequests(); + } + + LOG.info("Completed checkpoint {} (in {} ms).", checkpointId, completedCheckpoint.getDuration()); + + if (LOG.isDebugEnabled()) { --- End diff -- While rebasing you have to make sure to copy the updated string builder here --- 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 infrastruct...@apache.org or file a JIRA ticket
[GitHub] flink pull request #2873: [backport] [FLINK-5158] [ckPtCoord] Handle excepti...
Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/2873#discussion_r90257889 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java --- @@ -651,64 +651,33 @@ public boolean receiveDeclineMessage(DeclineCheckpoint message) { * * @throws Exception If the checkpoint cannot be added to the completed checkpoint store. */ - public boolean receiveAcknowledgeMessage(AcknowledgeCheckpoint message) throws Exception { + public boolean receiveAcknowledgeMessage(AcknowledgeCheckpoint message) throws CheckpointException { if (shutdown || message == null) { return false; } if (!job.equals(message.getJob())) { - LOG.error("Received AcknowledgeCheckpoint message for wrong job: {}", message); + LOG.error("Received wrong AcknowledgeCheckpoint message for job {}: {}", job, message); return false; } final long checkpointId = message.getCheckpointId(); - CompletedCheckpoint completed = null; - PendingCheckpoint checkpoint; - - // Flag indicating whether the ack message was for a known pending - // checkpoint. - boolean isPendingCheckpoint; - synchronized (lock) { // we need to check inside the lock for being shutdown as well, otherwise we // get races and invalid error log messages if (shutdown) { return false; } - checkpoint = pendingCheckpoints.get(checkpointId); + final PendingCheckpoint checkpoint = pendingCheckpoints.get(checkpointId); if (checkpoint != null && !checkpoint.isDiscarded()) { - isPendingCheckpoint = true; switch (checkpoint.acknowledgeTask(message.getTaskExecutionId(), message.getState(), message.getStateSize(), null)) { case SUCCESS: // TODO: Give KV-state to the acknowledgeTask method --- End diff -- Unrelated, but could you remove this TODO since this has been addressed for 1.2, but probably won't be addressed in 1.1 --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2873: [backport] [FLINK-5158] [ckPtCoord] Handle excepti...
GitHub user tillrohrmann opened a pull request: https://github.com/apache/flink/pull/2873 [backport] [FLINK-5158] [ckPtCoord] Handle exceptions from CompletedCheckpointStore in CheckpointCoordinator Backport of the #2872 for the release 1.1 branch. Handle exceptions from the CompletedCheckpointStore properly in the CheckpointCoordinator. This means that in case of an exception, the completed checkpoint will be properly cleaned up and also the triggering of subsequent checkpoints will be started. You can merge this pull request into a Git repository by running: $ git pull https://github.com/tillrohrmann/flink backportFixCheckpointCoordinatorExceptionHandling Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/2873.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #2873 commit c68c08f7b478f354a5c432f8640a344dcf553190 Author: Till RohrmannDate: 2016-11-24T17:16:28Z [FLINK-5158] [ckPtCoord] Handle exceptions from CompletedCheckpointStore in CheckpointCoordinator Handle exceptions from the CompletedCheckpointStore properly in the CheckpointCoordinator. This means that in case of an exception, the completed checkpoint will be properly cleaned up and also the triggering of subsequent checkpoints will be started. --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---