Stephan Ewen created FLINK-2341:
-----------------------------------

             Summary: Deadlock in SpilledSubpartitionViewAsyncIO
                 Key: FLINK-2341
                 URL: https://issues.apache.org/jira/browse/FLINK-2341
             Project: Flink
          Issue Type: Bug
          Components: Distributed Runtime
    Affects Versions: 0.9, 0.10
            Reporter: Stephan Ewen
            Assignee: Ufuk Celebi
            Priority: Critical
             Fix For: 0.9, 0.10


It may be that the deadlock is because of the way the 
{{SpilledSubpartitionViewTest}} is written

{code}
Found one Java-level deadlock:
=============================
"pool-25-thread-2":
  waiting to lock monitor 0x00007f66f4932468 (object 0x00000000fa1478f0, a 
java.lang.Object),
  which is held by "IOManager reader thread #1"
"IOManager reader thread #1":
  waiting to lock monitor 0x00007f66f4931160 (object 0x00000000fa029768, a 
java.lang.Object),
  which is held by "pool-25-thread-2"

Java stack information for the threads listed above:
===================================================
"pool-25-thread-2":
        at 
org.apache.flink.runtime.io.network.partition.SpilledSubpartitionViewAsyncIO.notifyError(SpilledSubpartitionViewAsyncIO.java:304)
        - waiting to lock <0x00000000fa1478f0> (a java.lang.Object)
        at 
org.apache.flink.runtime.io.network.partition.SpilledSubpartitionViewAsyncIO.onAvailableBuffer(SpilledSubpartitionViewAsyncIO.java:256)
        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:367)
        at 
org.apache.flink.runtime.io.network.partition.SpilledSubpartitionViewAsyncIO$BufferProviderCallback.onEvent(SpilledSubpartitionViewAsyncIO.java:353)
        at 
org.apache.flink.runtime.io.network.util.TestPooledBufferProvider$PooledBufferProviderRecycler.recycle(TestPooledBufferProvider.java:135)
        - locked <0x00000000fa029768> (a java.lang.Object)
        at 
org.apache.flink.runtime.io.network.buffer.Buffer.recycle(Buffer.java:119)
        - locked <0x00000000fa3a1a20> (a java.lang.Object)
        at 
org.apache.flink.runtime.io.network.util.TestSubpartitionConsumer.call(TestSubpartitionConsumer.java:95)
        at 
org.apache.flink.runtime.io.network.util.TestSubpartitionConsumer.call(TestSubpartitionConsumer.java:39)
        at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
        at java.util.concurrent.FutureTask.run(FutureTask.java:166)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1146)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        at java.lang.Thread.run(Thread.java:701)
"IOManager reader thread #1":
        at 
org.apache.flink.runtime.io.network.util.TestPooledBufferProvider$PooledBufferProviderRecycler.recycle(TestPooledBufferProvider.java:127)
        - waiting to lock <0x00000000fa029768> (a java.lang.Object)
        at 
org.apache.flink.runtime.io.network.buffer.Buffer.recycle(Buffer.java:119)
        - locked <0x00000000fa3a1ea0> (a java.lang.Object)
        at 
org.apache.flink.runtime.io.network.partition.SpilledSubpartitionViewAsyncIO.returnBufferFromIOThread(SpilledSubpartitionViewAsyncIO.java:270)
        - locked <0x00000000fa1478f0> (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:338)
        at 
org.apache.flink.runtime.io.network.partition.SpilledSubpartitionViewAsyncIO$IOThreadCallback.requestSuccessful(SpilledSubpartitionViewAsyncIO.java:328)
        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:431)
        at 
org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync$ReaderThread.run(IOManagerAsync.java:377)
{code}

The full log with the deadlock stack traces can be found here:

https://s3.amazonaws.com/archive.travis-ci.org/jobs/70232347/log.txt



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

Reply via email to