Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-7067] [jobmanager] Fix side effects after failed cancel-job-with-savepoint #4254

Closed
wants to merge 2 commits into from

Conversation

uce
Copy link
Contributor

@uce uce commented Jul 4, 2017

If a cancel-job-with-savepoint request fails, this has an unintended side effect on the respective job if it has periodic checkpoints enabled. The periodic checkpoint scheduler is stopped before triggering the savepoint, but not restarted if a savepoint fails and the job is not cancelled.

This fix makes sure that the periodic checkpoint scheduler is restarted iff periodic checkpoints were enabled before.

I have the test in a separate commit, because it uses Reflection to update a private field with a spied upon instance of the CheckpointCoordinator in order to test the expected behaviour. This is super fragile and ugly, but the alternatives require a large refactoring (use factories that can be set during tests) or don't test this corner case behaviour. The separate commit makes it easier to remove/revert it at a future point in time.

I would like to merge this to release-1.3 and master.

// again. Therefore, we verify two calls for stop. Since we
// spy (I know...) on the coordinator after the job has
// started, we don't count calls before spying.
verify(spiedCoord, times(1)).startCheckpointScheduler();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could we not re-attempt a cancel-with-savepoint? If the coordinator is shutdown it will fail; if it was restarted it should succeed (provided we adjust the failing source to only fail the first time). Then we wouldn't need the spying but would actually just test observable behavior.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The thing is that the stopping of the scheduler is part of the expected behaviour of cancel-with-job-savepoint, because we don't want any checkpoints between the savepoint and cancel job (https://issues.apache.org/jira/browse/FLINK-4717). I think for that we do need the spying :-( It was simply not fully tested before... Does this make sense or am I missing your point?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can't we check that the submitted tasks sees another checkpoint barrier after a savepoint has been triggered? That way we would get around spying on the CheckpointCoordinator.

@StephanEwen
Copy link
Contributor

I think this is a meaningful fix.

I would suggest to do the tests different, though. The tests of the CheckpointCoordinator overdo the mockito stuff so heavily that it becomes an extremely hard job to change anything in the CheckpointCoordinator. Mocks are super maintenance heavy, compared to actual test implementations of interfaces or classes.

Copy link
Contributor

@tillrohrmann tillrohrmann left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for your contribution @uce. Changes look good. Maybe we could make the test easier to maintain without much effort. Wouldn't it be enough to wait on a checkpoint barrier after receiving savepoint barrier in the FailOnSavepointStatefulTask?

true,
TestingTaskManager.class);

ActorGateway taskManager = new AkkaActorGateway(taskManagerRef, leaderId);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can't we simply use a TestingCluster here for all the setup work?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Definitely +1

// again. Therefore, we verify two calls for stop. Since we
// spy (I know...) on the coordinator after the job has
// started, we don't count calls before spying.
verify(spiedCoord, times(1)).startCheckpointScheduler();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can't we check that the submitted tasks sees another checkpoint barrier after a savepoint has been triggered? That way we would get around spying on the CheckpointCoordinator.

@uce
Copy link
Contributor Author

uce commented Oct 20, 2017

@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.

@tillrohrmann
Copy link
Contributor

tillrohrmann commented Oct 20, 2017

I think it's alright that way. Thanks for addressing this issue so swiftly.

@uce
Copy link
Contributor Author

uce commented Oct 20, 2017

Cool! I'll rebase this and merge after Travis gives the green light.

@uce uce force-pushed the 7067-restart_checkpoint_scheduler branch 2 times, most recently from 96a99a2 to 37fe380 Compare October 20, 2017 12:48
There is no need to make the helper methods public. No other class
should even use this inner test helper invokable.
…ncel-job-with-savepoint

Problem: If a cancel-job-with-savepoint request fails, this has an
unintended side effect on the respective job if it has periodic
checkpoints enabled. The periodic checkpoint scheduler is stopped
before triggering the savepoint, but not restarted if a savepoint
fails and the job is not cancelled.

This commit makes sure that the periodic checkpoint scheduler is
restarted iff periodic checkpoints were enabled before.
@uce uce force-pushed the 7067-restart_checkpoint_scheduler branch from 2bb3bfb to c9f1fa7 Compare October 23, 2017 08:30
@uce
Copy link
Contributor Author

uce commented Oct 23, 2017

Travis gave the green light, merging this now.

@asfgit asfgit closed this in e49bc42 Oct 23, 2017
uce added a commit to uce/flink that referenced this pull request Oct 23, 2017
…ncel-job-with-savepoint

Problem: If a cancel-job-with-savepoint request fails, this has an
unintended side effect on the respective job if it has periodic
checkpoints enabled. The periodic checkpoint scheduler is stopped
before triggering the savepoint, but not restarted if a savepoint
fails and the job is not cancelled.

This commit makes sure that the periodic checkpoint scheduler is
restarted iff periodic checkpoints were enabled before.

This closes apache#4254.
@uce uce deleted the 7067-restart_checkpoint_scheduler branch November 22, 2017 09:14
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
5 participants