You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by StefanRRichter <gi...@git.apache.org> on 2017/01/27 17:04:00 UTC

[GitHub] flink pull request #3228: [FLINK-5663] Prevent leaking SafetyNetCloseableReg...

GitHub user StefanRRichter opened a pull request:

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

    [FLINK-5663] Prevent leaking SafetyNetCloseableRegistry though InheritableThreadLocal

    This PR prevents the `SafetyNetCloseableRegistry` from leaking into pooled threads through `InheritableThreadLocal`. 
    
    As first step, we use `ThreadLocal` instead of `InheritableThreadLocal` to hold the closeable registries.
    
    Additionally, we also create safety nets for the file system at the scope of the checkpointing thread. We hope that this covers already covers most cases. Other threads could actually also create safety nets for their scope right now.
    
    As a last change, we made the reaper thread a singleton, because we could potentially create more registries now and it is not required to have one reaper thread per registry. 

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

    $ git pull https://github.com/StefanRRichter/flink safetyNet2

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

    https://github.com/apache/flink/pull/3228.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 #3228
    
----
commit 21e2a31ece3c56a9d79cb127f9829f770ebe56cf
Author: Stefan Richter <s....@data-artisans.com>
Date:   2017-01-27T15:32:35Z

    [FLINK-5663] Prevent leaking safetynet closeable registry

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3228: [FLINK-5663] Prevent leaking SafetyNetCloseableRegistry t...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on the issue:

    https://github.com/apache/flink/pull/3228
  
    Concerning the reaper thread:
      - It is not really broken in the release branch, only has a few more threads then necessary.
      - So far we have strived to make sure Flink does not leave any lingering Threads at all (as validated by the MiniCluster thread), because it actually messes up testing setups from many users that repeatedly execute programs with the LocalEnvironment. That would be good to keep.
      - One can probably stop the thread when all registries are closed and re-spawn it when new registries come. That would be 
    
    In summary: I think lingering threads are a type of regression, actually. Introducing that for something that is not broken right now is something I would not do for a release. Especially given that there is probably a cleaner solution that can both implement the improvement and not have the lingering threads regression. Let's do that for the 1.2.1/1.3 release.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3228: [FLINK-5663] Prevent leaking SafetyNetCloseableRegistry t...

Posted by StefanRRichter <gi...@git.apache.org>.
Github user StefanRRichter commented on the issue:

    https://github.com/apache/flink/pull/3228
  
    Closed in favour of #3229 and #3230 .


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3228: [FLINK-5663] Prevent leaking SafetyNetCloseableRegistry t...

Posted by StefanRRichter <gi...@git.apache.org>.
Github user StefanRRichter commented on the issue:

    https://github.com/apache/flink/pull/3228
  
    Ok, maybe what we are talking about is respawning the reaper thread if we go again from 0 to 1. That is possible.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3228: [FLINK-5663] Prevent leaking SafetyNetCloseableRegistry t...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on the issue:

    https://github.com/apache/flink/pull/3228
  
    Making the registry not inherited is a good quick fix.
    
    How about moving the initialization and closing of that registry into the task's trigger checkpoint action? https://github.com/apache/flink/blob/master/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java#L1119
    
    That way it is tied to the dedicated asynchronous thread, which is safer, I think. Having the registry initialization/closing in the `triggerCheckpoint(...)` method means the method can never be called by the main thread. It probably does not happen currently, but it seems quite easy to accidentally violate.
    
    There is also some additional refactoring in this pull request that makes the reaper thread a static variable, shared across all registries. Currently that thread seems to never stopped by anyone. How about factoring out the changes for the single static reaper thread into a separate issue (that we do not merge for 1.2) and addressing the thread stopping as well (via an atomic count how many registries are currently open, stopping it when ging to zero, starting when going from zero).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3228: [FLINK-5663] Prevent leaking SafetyNetCloseableRegistry t...

Posted by StefanRRichter <gi...@git.apache.org>.
Github user StefanRRichter commented on the issue:

    https://github.com/apache/flink/pull/3228
  
    I agree about the first part. 
    
    About the reaper thread, I was also concerned about stopping and the counter is not a fix: there can be 0 registries but at some point a new task could come up, right?
    
    The reaper is now a daemon thread, so it will go down with the JVM, before which all streams get closed anyways. Or am I missing something?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3228: [FLINK-5663] Prevent leaking SafetyNetCloseableRegistry t...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on the issue:

    https://github.com/apache/flink/pull/3228
  
    To clarify: The "static reaper thread" change is a good idea, but the change seems a bit incomplete without well defined shutdown of the thread.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3228: [FLINK-5663] Prevent leaking SafetyNetCloseableReg...

Posted by StefanRRichter <gi...@git.apache.org>.
Github user StefanRRichter closed the pull request at:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3228: [FLINK-5663] Prevent leaking SafetyNetCloseableRegistry t...

Posted by StefanRRichter <gi...@git.apache.org>.
Github user StefanRRichter commented on the issue:

    https://github.com/apache/flink/pull/3228
  
    cc @StephanEwen 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---