Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-5158

Handle ZooKeeperCompletedCheckpointStore exceptions in CheckpointCoordinator

    Details

      Description

      The checkpoint coordinator does not properly handle exceptions when trying to store completed checkpoints. As a result, completed checkpoints are not properly cleaned up and even worse, the CheckpointCoordinator might get stuck stopping triggering checkpoints.

        Issue Links

          Activity

          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tillrohrmann closed the pull request at:

          https://github.com/apache/flink/pull/2873

          Show
          githubbot ASF GitHub Bot added a comment - Github user tillrohrmann closed the pull request at: https://github.com/apache/flink/pull/2873
          Hide
          till.rohrmann Till Rohrmann added a comment -

          Fixed in 1.2.0 via 0c42d258e9d9d30eeeee7d0f1487ef0ac8b90fa4
          Fixed in 1.1.4 via 4b734d7b8726200e5293c32f2cb9e8c77db4d378

          Show
          till.rohrmann Till Rohrmann added a comment - Fixed in 1.2.0 via 0c42d258e9d9d30eeeee7d0f1487ef0ac8b90fa4 Fixed in 1.1.4 via 4b734d7b8726200e5293c32f2cb9e8c77db4d378
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user asfgit closed the pull request at:

          https://github.com/apache/flink/pull/2872

          Show
          githubbot ASF GitHub Bot added a comment - Github user asfgit closed the pull request at: https://github.com/apache/flink/pull/2872
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tillrohrmann commented on the issue:

          https://github.com/apache/flink/pull/2872

          `ScalaShellITCase` hung in the failing travis build. Merging...

          Show
          githubbot ASF GitHub Bot added a comment - Github user tillrohrmann commented on the issue: https://github.com/apache/flink/pull/2872 `ScalaShellITCase` hung in the failing travis build. Merging...
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tillrohrmann commented on the issue:

          https://github.com/apache/flink/pull/2872

          Forwarding @uce's review from the backport to this PR.

          Rebasing on the latest master and if Travis gives green light, then I will merge this PR.

          Show
          githubbot ASF GitHub Bot added a comment - Github user tillrohrmann commented on the issue: https://github.com/apache/flink/pull/2872 Forwarding @uce's review from the backport to this PR. Rebasing on the latest master and if Travis gives green light, then I will merge this PR.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tillrohrmann commented on the issue:

          https://github.com/apache/flink/pull/2873

          Thanks for the review @uce. I have addressed your comments.

          Show
          githubbot ASF GitHub Bot added a comment - Github user tillrohrmann commented on the issue: https://github.com/apache/flink/pull/2873 Thanks for the review @uce. I have addressed your comments.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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

          Show
          githubbot ASF GitHub Bot added a comment - 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
          Hide
          githubbot ASF GitHub Bot added a comment -

          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

          Show
          githubbot ASF GitHub Bot added a comment - 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
          Hide
          githubbot ASF GitHub Bot added a comment -

          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

          Show
          githubbot ASF GitHub Bot added a comment - 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
          Hide
          githubbot ASF GitHub Bot added a comment -

          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

          Show
          githubbot ASF GitHub Bot added a comment - 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
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tillrohrmann commented on the issue:

          https://github.com/apache/flink/pull/2873

          @uce if we want to include this into the release 1.1 branch, then we could merge it. The problem has been reported by Twitter.

          Show
          githubbot ASF GitHub Bot added a comment - Github user tillrohrmann commented on the issue: https://github.com/apache/flink/pull/2873 @uce if we want to include this into the release 1.1 branch, then we could merge it. The problem has been reported by Twitter.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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 Rohrmann <trohrmann@apache.org>
          Date: 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.


          Show
          githubbot ASF GitHub Bot added a comment - 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 Rohrmann <trohrmann@apache.org> Date: 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.
          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user tillrohrmann opened a pull request:

          https://github.com/apache/flink/pull/2872

          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.

          You can merge this pull request into a Git repository by running:

          $ git pull https://github.com/tillrohrmann/flink fixCheckpointCoordinatorExceptionHandling

          Alternatively you can review and apply these changes as the patch at:

          https://github.com/apache/flink/pull/2872.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 #2872


          commit 063a696b4eb5a259c714818c0b0ba5cc883a596d
          Author: Till Rohrmann <trohrmann@apache.org>
          Date: 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.


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user tillrohrmann opened a pull request: https://github.com/apache/flink/pull/2872 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. You can merge this pull request into a Git repository by running: $ git pull https://github.com/tillrohrmann/flink fixCheckpointCoordinatorExceptionHandling Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/2872.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 #2872 commit 063a696b4eb5a259c714818c0b0ba5cc883a596d Author: Till Rohrmann <trohrmann@apache.org> Date: 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.

            People

            • Assignee:
              till.rohrmann Till Rohrmann
              Reporter:
              till.rohrmann Till Rohrmann
            • Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development