Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-5681

Make ReaperThread for SafetyNetCloseableRegistry a singleton

    Details

    • Type: Improvement
    • Status: Closed
    • Priority: Major
    • Resolution: Fixed
    • Affects Version/s: None
    • Fix Version/s: 1.3.0, 1.2.1
    • Component/s: Core
    • Labels:
      None

      Description

      Currently, each SafetyNetCloseableRegistry spawns an own ReaperThread. However, this duty could also be fulfilled by a single ReaperThread that is shared by all registries to save unnecessary threads.

        Issue Links

          Activity

          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user StefanRRichter opened a pull request:

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

          FLINK-5681 Make ReaperThread for SafetyNetCloseableRegistry a singleton

          Currently, each `SafetyNetCloseableRegistry` spawns an own ReaperThread. However, this duty could also be fulfilled by a single ReaperThread that is shared by all registries to save unnecessary threads.

          This PR makes 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.

          PR sits on top or #3229 .

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

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

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

          https://github.com/apache/flink/pull/3230.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 #3230


          commit 31ae956fdd83a2f65bf22c3ad601d4d65ad61439
          Author: Stefan Richter <s.richter@data-artisans.com>
          Date: 2017-01-27T18:47:12Z

          FLINK-5663 Prevent leaking SafetyNetCloseableRegistry through InheritableThreadLocal

          commit ec968c2e380ceb8403176b7744ed3b88ace3f340
          Author: Stefan Richter <s.richter@data-artisans.com>
          Date: 2017-01-27T18:51:21Z

          FLINK-5681 Make ReaperThread for SafetyNetCloseableRegistry a singleton


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user StefanRRichter opened a pull request: https://github.com/apache/flink/pull/3230 FLINK-5681 Make ReaperThread for SafetyNetCloseableRegistry a singleton Currently, each `SafetyNetCloseableRegistry` spawns an own ReaperThread. However, this duty could also be fulfilled by a single ReaperThread that is shared by all registries to save unnecessary threads. This PR makes 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. PR sits on top or #3229 . You can merge this pull request into a Git repository by running: $ git pull https://github.com/StefanRRichter/flink safetyNetSingletonReaperThread Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/3230.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 #3230 commit 31ae956fdd83a2f65bf22c3ad601d4d65ad61439 Author: Stefan Richter <s.richter@data-artisans.com> Date: 2017-01-27T18:47:12Z FLINK-5663 Prevent leaking SafetyNetCloseableRegistry through InheritableThreadLocal commit ec968c2e380ceb8403176b7744ed3b88ace3f340 Author: Stefan Richter <s.richter@data-artisans.com> Date: 2017-01-27T18:51:21Z FLINK-5681 Make ReaperThread for SafetyNetCloseableRegistry a singleton
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StephanEwen commented on the issue:

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

          Quick question about this change: The reaper thread now does not remove the closeables any more from the `closeableToRef` map. Isn't that a potential memory leak over time?

          Show
          githubbot ASF GitHub Bot added a comment - Github user StephanEwen commented on the issue: https://github.com/apache/flink/pull/3230 Quick question about this change: The reaper thread now does not remove the closeables any more from the `closeableToRef` map. Isn't that a potential memory leak over time?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StefanRRichter commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3230#discussion_r98358866

          — Diff: flink-core/src/main/java/org/apache/flink/core/fs/SafetyNetCloseableRegistry.java —
          @@ -99,42 +120,61 @@ protected void doUnRegister(
          implements Closeable {

          private final Closeable innerCloseable;
          + private final SafetyNetCloseableRegistry closeableRegistry;
          private final String debugString;

          public PhantomDelegatingCloseableRef(
          WrappingProxyCloseable<? extends Closeable> referent,
          + SafetyNetCloseableRegistry closeableRegistry,
          ReferenceQueue<? super WrappingProxyCloseable<? extends Closeable>> q)

          { super(referent, q); this.innerCloseable = Preconditions.checkNotNull(WrappingProxyUtil.stripProxy(referent)); + this.closeableRegistry = Preconditions.checkNotNull(closeableRegistry); this.debugString = referent.toString(); }
          • public Closeable getInnerCloseable() { - return innerCloseable; - }

            -
            public String getDebugString()

            { return debugString; }

          @Override
          public void close() throws IOException {
          + synchronized (closeableRegistry.getSynchronizationLock()) {
          + closeableRegistry.closeableToRef.remove(innerCloseable);
          — End diff –

          Removal is done here now, because the reaper thread should not keep/locate right references for all registries

          Show
          githubbot ASF GitHub Bot added a comment - Github user StefanRRichter commented on a diff in the pull request: https://github.com/apache/flink/pull/3230#discussion_r98358866 — Diff: flink-core/src/main/java/org/apache/flink/core/fs/SafetyNetCloseableRegistry.java — @@ -99,42 +120,61 @@ protected void doUnRegister( implements Closeable { private final Closeable innerCloseable; + private final SafetyNetCloseableRegistry closeableRegistry; private final String debugString; public PhantomDelegatingCloseableRef( WrappingProxyCloseable<? extends Closeable> referent, + SafetyNetCloseableRegistry closeableRegistry, ReferenceQueue<? super WrappingProxyCloseable<? extends Closeable>> q) { super(referent, q); this.innerCloseable = Preconditions.checkNotNull(WrappingProxyUtil.stripProxy(referent)); + this.closeableRegistry = Preconditions.checkNotNull(closeableRegistry); this.debugString = referent.toString(); } public Closeable getInnerCloseable() { - return innerCloseable; - } - public String getDebugString() { return debugString; } @Override public void close() throws IOException { + synchronized (closeableRegistry.getSynchronizationLock()) { + closeableRegistry.closeableToRef.remove(innerCloseable); — End diff – Removal is done here now, because the reaper thread should not keep/locate right references for all registries
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user asfgit closed the pull request at:

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

          Show
          githubbot ASF GitHub Bot added a comment - Github user asfgit closed the pull request at: https://github.com/apache/flink/pull/3230
          Hide
          StephanEwen Stephan Ewen added a comment -

          Fixed in

          • 1.2.1 via c365a34b83e913d9c6c509627fae784435b056a2
          • 1.3.0 via ec3eb593ae93123cf54cd34c452618d8bd0a7876
          Show
          StephanEwen Stephan Ewen added a comment - Fixed in 1.2.1 via c365a34b83e913d9c6c509627fae784435b056a2 1.3.0 via ec3eb593ae93123cf54cd34c452618d8bd0a7876

            People

            • Assignee:
              srichter Stefan Richter
              Reporter:
              srichter Stefan Richter
            • Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development