[ 
https://issues.apache.org/jira/browse/FLINK-4910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15605445#comment-15605445
 ] 

ASF GitHub Bot commented on FLINK-4910:
---------------------------------------

GitHub user StefanRRichter opened a pull request:

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

    [FLINK-4910] Introduce safety net for closing file system streams

    This PR introduces a safety net against leaking/unclosed streams obtained 
through ``FileSystem``.
    
    The main addition is ``SafetyNetCloseableRegistry``, which tracks all 
opened streams and uses PhantomReferences to wrapper classes around the streams 
in order to detect leaking resources after GC of the wrapper class.
    
    Wrapper classes are ``ClosingFSDataInputStream``, 
``ClosingFSDataOutputStream``, and ``SafetyNetWrapperFileSystem``, where the 
latter wraps all created streams as closing and registers them with the 
``SafetyNetCloseableRegistry``.
    
    ``InheritableThreadLocal<SafetyNetCloseableRegistry>`` is used to scope the 
registry per Task, where the creation happens at the beginning of a Task's run 
method and cleanup happens at it's end (and on cancelation).


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

    $ git pull https://github.com/StefanRRichter/flink closeable-io

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

    https://github.com/apache/flink/pull/2691.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 #2691
    
----
commit 384e6d3e3c9640afcbe79d732e34819fcdd44d8e
Author: Stefan Richter <s.rich...@data-artisans.com>
Date:   2016-10-24T15:49:54Z

    phantom closeable registry

commit 13e719bb7f19d3a4109aa39bdd75a310bdf80bfe
Author: Stefan Richter <s.rich...@data-artisans.com>
Date:   2016-10-24T22:14:59Z

    Abstractions to avoid code duplication

commit 2e5409c64eff58d28198acf2cab4ea8aa47e8fd1
Author: Stefan Richter <s.rich...@data-artisans.com>
Date:   2016-10-24T22:21:48Z

    Renames

commit c66049397ebfb1b1297277c518d821e08a1cf36a
Author: Stefan Richter <s.rich...@data-artisans.com>
Date:   2016-10-24T23:27:51Z

    Logging and removed sync

commit b024717646e2af3abe4f01f00921abd815d5781f
Author: Stefan Richter <s.rich...@data-artisans.com>
Date:   2016-10-25T08:44:27Z

    Debug messages

commit 020e21e91a193a95f642361463a746a1fd2be67b
Author: Stefan Richter <s.rich...@data-artisans.com>
Date:   2016-10-25T11:33:38Z

    Integration via Fs Wrapper

commit e86b41575d2b11ca3d492007c746b03ac5907941
Author: Stefan Richter <s.rich...@data-artisans.com>
Date:   2016-10-25T12:43:46Z

    Proper positioning of calls to register/unregister

commit ca79e84680dcbedeaa1b2583f6bc6a77a97abb43
Author: Stefan Richter <s.rich...@data-artisans.com>
Date:   2016-10-25T13:07:40Z

    Documentation and minor fixes

----


> Introduce safety net for closing file system streams
> ----------------------------------------------------
>
>                 Key: FLINK-4910
>                 URL: https://issues.apache.org/jira/browse/FLINK-4910
>             Project: Flink
>          Issue Type: Improvement
>            Reporter: Stefan Richter
>            Assignee: Stefan Richter
>
> Streams that are opened through {{FileSystem}} must be closed at the end of 
> their life cycle. However, we found hints that some code forgets to close 
> such streams.
> We should introduce i) a mechanism that closes leaking unclosed streams after 
> usage and ii) provides logging that helps us to track down and fi the sources 
> of such leaks.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to