[
https://issues.apache.org/jira/browse/FLINK-7067?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16212463#comment-16212463
]
ASF GitHub Bot commented on FLINK-7067:
---------------------------------------
Github user uce commented on the issue:
https://github.com/apache/flink/pull/4254
@tillrohrmann Thanks for looking over this. The `TestingCluster` is
definitely preferable. I don't recall how I ended up with the custom setup
instead of the `TestingCluster`.
I changed the test to wait for another checkpoint after the failed
savepoint. I also considered this for the initial PR, but went with mocking in
order to test the case that periodic checkpoints were not activated before the
cancellation [1]. I think the current variant is a good compromise between
completeness and simplicity though.
[1] As seen in the diff of `JobManager.scala`, we only activate the
periodic scheduler after a failed cancellation iff it was activated before
cancellation. This case can't be tested robustly with the current approach. We
could wait for some time and if no checkpoint arrives in that time consider
checkpoints as not accidentally activated, but that's not robust. I would
therefore ignore this case if you don't have another idea.
> Cancel with savepoint does not restart checkpoint scheduler on failure
> ----------------------------------------------------------------------
>
> Key: FLINK-7067
> URL: https://issues.apache.org/jira/browse/FLINK-7067
> Project: Flink
> Issue Type: Bug
> Components: State Backends, Checkpointing
> Affects Versions: 1.3.1
> Reporter: Ufuk Celebi
> Assignee: Ufuk Celebi
> Priority: Blocker
> Fix For: 1.4.0, 1.3.3
>
>
> The `CancelWithSavepoint` action of the JobManager first stops the checkpoint
> scheduler, then triggers a savepoint, and cancels the job after the savepoint
> completes.
> If the savepoint fails, the command should not have any side effects and we
> don't cancel the job. The issue is that the checkpoint scheduler is not
> restarted though.
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)