GitHub user StefanRRichter opened a pull request: https://github.com/apache/flink/pull/5058
[FLINK-5465] [streaming] Wait for pending timer threads to finish or ⦠â¦to exceed a time limit in exceptional stream task shutdown ## What is the purpose of the change When stream tasks are going through their cleanup in the failover case, pending timer threads can still access native resources of a state backend after the backend's disposal. In some cases, this can crash the JVM. The obvious fix is to wait until all timers are finished before disposing those resources, and the main reason why we did not change this is that (in theory) a the user code of a triggering timer can block forever and suppress or delay restarts in failover cases. However, the situation has changed a bit since this topic was discussed for the last time and there is now also a watchdog that ensures that tasks are terminated after some time of inactivity. I would propose a middle ground that probably catches close to all of those problems, while still ensuring a reasonable fast shutdown. I would suggest to wait for the termination of the threadpool that runs all timer events until a time limit. Typically, there is a very limited number of in-flight timers and they are usually short lived. A wait interval of a few seconds should basically fix this problem, even though there can still be very rare cases of very long running timers that also happen to still access the disposed resource s. But the likelihood of this scenario should be reduced by orders of magnitude and in particular, the cascading effect should be mitigated. ## Brief change log - *SteamTasks wait in their cleanup code for a certain time limit to give the timer service a chance to finish all pending timer threads. * ## Verifying this change This change added tests to `SystemProcessingTimeServiceTest`. ## 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) You can merge this pull request into a Git repository by running: $ git pull https://github.com/StefanRRichter/flink FLINK-5465-wait-timer-shutdown Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/5058.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 #5058 ---- commit 3c6f0f3377abb35ed5b04fdcdb599705bc0af162 Author: Stefan Richter <s.rich...@data-artisans.com> Date: 2017-11-22T16:52:35Z [FLINK-5465] [streaming] Wait for pending timer threads to finish or to exceed a time limit in exceptional stream task shutdown ---- ---