Skip to content

Conversation

@StephanEwen
Copy link
Contributor

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.

@StephanEwen
Copy link
Contributor Author

@uce I think you'd be the best person to review this.

…iewAsyncIO.

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.
Copy link
Contributor

@uce uce left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for looking into this. Changes are good.

+1 to merge.

StephanEwen added a commit to StephanEwen/flink that referenced this pull request Sep 27, 2016
…iewAsyncIO.

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.

This closes apache#2444
@asfgit asfgit closed this in 9090291 Sep 27, 2016
liuyuzhong pushed a commit to liuyuzhong/flink that referenced this pull request Dec 5, 2016
…iewAsyncIO.

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.

This closes apache#2444
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants