Skip to content

Conversation

Wosin
Copy link
Contributor

@Wosin Wosin commented Jul 12, 2018

What is the purpose of the change

Currently flink removes all blobs connected with the job, no matter if the job itself was removed successfully. This is not the desired behavior.

Brief change log

  • Blobs and data will be removed only if the job itself will be removed sucessfully

Verifying this change

This change is a trivial rework / code cleanup without any test coverage.

Does this pull request potentially affect one of the following parts:

  • Dependencies (does it add or upgrade a dependency): no
  • The public API, i.e., is any changed class annotated with @Public(Evolving): no
  • The serializers: no
  • The runtime per-record code paths (performance sensitive): no
  • Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: yes
  • The S3 file system connector: no

Documentation

  • Does this pull request introduce a new feature? no
  • If yes, how is the feature documented? not applicable

}

jobManagerMetricGroup.removeJob(jobID)

Copy link
Contributor

Choose a reason for hiding this comment

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

this line can also be removed

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 opening this PR @Wosin. I think we only need to make the blobServer.cleanupJob call dependent on the success of the SubmittedJobGraphStore#removeJobGraph call.

Furthermore, we should also do the same in the Dispatcher.java:577.

It would be great to add a test for the cleanup behaviour in the Dispatcher.

case Some(future) => future.onComplete{
case scala.util.Success(_) => {
libraryCacheManager.unregisterJob(jobID)
blobServer.cleanupJob(jobID, removeJobFromStateBackend)
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 move these this line in the future where we remove the job from the SubmittedJobGraphStore?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Technically we can, but this changes the return type of the future as cleanupJob does indeed return something.

case scala.util.Success(_) => {
libraryCacheManager.unregisterJob(jobID)
blobServer.cleanupJob(jobID, removeJobFromStateBackend)
jobManagerMetricGroup.removeJob(jobID)
Copy link
Contributor

Choose a reason for hiding this comment

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

I think we could always execute this call independent of whether the removal from the SubmittedJobGraphStore was successful or not.

futureOption match {
case Some(future) => future.onComplete{
case scala.util.Success(_) => {
libraryCacheManager.unregisterJob(jobID)
Copy link
Contributor

Choose a reason for hiding this comment

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

This call should also be called if the removal of the job from the SubmittedJobGraphStore failed because it does not remove any HA files.

@Wosin
Copy link
Contributor Author

Wosin commented Jul 16, 2018

Hey,
I think it should be okay now :) If it is i will squash it to have one commit only.

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 @Wosin. LGTM. Merging this PR.

tillrohrmann pushed a commit to tillrohrmann/flink that referenced this pull request Jul 18, 2018
@Wosin
Copy link
Contributor Author

Wosin commented Jul 18, 2018

Wait, I have found an issue with my code. I will update the PR accordingly.
Sorry for the trouble.

@tillrohrmann
Copy link
Contributor

tillrohrmann commented Jul 18, 2018

No worries @Wosin. I've already fixed the problem. You can see it here: tillrohrmann@8b3a849

@Wosin
Copy link
Contributor Author

Wosin commented Jul 18, 2018

Ok! Thanks.

asfgit pushed a commit that referenced this pull request Jul 18, 2018
@asfgit asfgit closed this in f6b2e8c Jul 18, 2018
asfgit pushed a commit that referenced this pull request Jul 18, 2018
sampathBhat pushed a commit to sampathBhat/flink that referenced this pull request Jul 26, 2018
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants