flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From StephanEwen <...@git.apache.org>
Subject [GitHub] flink pull request #2444: [FLINK-4543] [network] Fix potential deadlock in S...
Date Wed, 31 Aug 2016 14:31:09 GMT
GitHub user StephanEwen opened a pull request:

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

    [FLINK-4543] [network] Fix potential deadlock in SpilledSubpartitionViewAsyncIO

    The deadlock could occur in cases where the SpilledSubpartitionViewAsyncIO would simultaneously
try to
    release a buffer and encounter an error in another thread.
    
    The field of congestion was the listener, which is now replaced by an AtomicReference,
removing the
    necessity to lock in the case of reporting the error.
    
    The deadlock stack traces were:
    
    ```
    Found one Java-level deadlock:
    =============================
    "pool-1-thread-2":
      waiting to lock monitor 0x00007fec2c006168 (object 0x00000000ef661c20, a java.lang.Object),
      which is held by "IOManager reader thread #1"
    "IOManager reader thread #1":
      waiting to lock monitor 0x00007fec2c005ea8 (object 0x00000000ef62c8a8, a java.lang.Object),
      which is held by "pool-1-thread-2"
    
    Java stack information for the threads listed above:
    ===================================================
    "pool-1-thread-2":
            at org.apache.flink.runtime.io.network.partition.SpilledSubpartitionViewAsyncIO.notifyError(SpilledSubpartitionViewAsyncIO.java:309)
            - waiting to lock <0x00000000ef661c20> (a java.lang.Object)
            at org.apache.flink.runtime.io.network.partition.SpilledSubpartitionViewAsyncIO.onAvailableBuffer(SpilledSubpartitionViewAsyncIO.java:261)
            at org.apache.flink.runtime.io.network.partition.SpilledSubpartitionViewAsyncIO.access$300(SpilledSubpartitionViewAsyncIO.java:42)
            at org.apache.flink.runtime.io.network.partition.SpilledSubpartitionViewAsyncIO$BufferProviderCallback.onEvent(SpilledSubpartitionViewAsyncIO.java:380)
            at org.apache.flink.runtime.io.network.partition.SpilledSubpartitionViewAsyncIO$BufferProviderCallback.onEvent(SpilledSubpartitionViewAsyncIO.java:366)
            at org.apache.flink.runtime.io.network.util.TestPooledBufferProvider$PooledBufferProviderRecycler.recycle(TestPooledBufferProvider.java:135)
            - locked <0x00000000ef62c8a8> (a java.lang.Object)
            at org.apache.flink.runtime.io.network.buffer.Buffer.recycle(Buffer.java:118)
            - locked <0x00000000ef9597c0> (a java.lang.Object)
            at org.apache.flink.runtime.io.network.util.TestConsumerCallback$RecyclingCallback.onBuffer(TestConsumerCallback.java:72)
            at org.apache.flink.runtime.io.network.util.TestSubpartitionConsumer.call(TestSubpartitionConsumer.java:87)
            at org.apache.flink.runtime.io.network.util.TestSubpartitionConsumer.call(TestSubpartitionConsumer.java:39)
            at java.util.concurrent.FutureTask.run(FutureTask.java:266)
            at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
            at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
            at java.lang.Thread.run(Thread.java:745)
    "IOManager reader thread #1":
            at org.apache.flink.runtime.io.network.util.TestPooledBufferProvider$PooledBufferProviderRecycler.recycle(TestPooledBufferProvider.java:126)
            - waiting to lock <0x00000000ef62c8a8> (a java.lang.Object)
            at org.apache.flink.runtime.io.network.buffer.Buffer.recycle(Buffer.java:118)
            - locked <0x00000000efa016f0> (a java.lang.Object)
            at org.apache.flink.runtime.io.network.partition.SpilledSubpartitionViewAsyncIO.returnBufferFromIOThread(SpilledSubpartitionViewAsyncIO.java:275)
            - locked <0x00000000ef661c20> (a java.lang.Object)
            at org.apache.flink.runtime.io.network.partition.SpilledSubpartitionViewAsyncIO.access$100(SpilledSubpartitionViewAsyncIO.java:42)
            at org.apache.flink.runtime.io.network.partition.SpilledSubpartitionViewAsyncIO$IOThreadCallback.requestSuccessful(SpilledSubpartitionViewAsyncIO.java:343)
            at org.apache.flink.runtime.io.network.partition.SpilledSubpartitionViewAsyncIO$IOThreadCallback.requestSuccessful(SpilledSubpartitionViewAsyncIO.java:333)
            at org.apache.flink.runtime.io.disk.iomanager.AsynchronousFileIOChannel.handleProcessedBuffer(AsynchronousFileIOChannel.java:199)
            at org.apache.flink.runtime.io.disk.iomanager.BufferReadRequest.requestDone(AsynchronousFileIOChannel.java:435)
            at org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync$ReaderThread.run(IOManagerAsync.java:408)
    
    Found 1 deadlock.
    ```

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

    $ git pull https://github.com/StephanEwen/incubator-flink deadlock

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

    https://github.com/apache/flink/pull/2444.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 #2444
    
----
commit 702afe937d53943e5c64b27bcff0466c7deffbdb
Author: Stephan Ewen <sewen@apache.org>
Date:   2016-08-31T14:22:34Z

    [FLINK-4543] [network] Fix potential deadlock in SpilledSubpartitionViewAsyncIO.
    
    The deadlock could occur in cases where the SpilledSubpartitionViewAsyncIO would simultaneously
try to
    release a buffer and encounter an error in another thread.
    
    The field of congestion was the listener, which is now replaced by an AtomicReference,
removing the
    necessity to lock in the case of reporting the error.

----


---
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.
---

Mime
View raw message