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

Matthias Pohl commented on FLINK-26568:
---------------------------------------

Thanks to [~chesnay]: The stacktraces indicate that there is some race 
condition in reserving and freeing buffers (see [Stacktrace in 
logs|https://dev.azure.com/mapohl/flink/_build/results?buildId=845&view=logs&j=0a15d512-44ac-5ba5-97ab-13a5d066c22c&t=9a028d19-6c4b-5a4e-d378-03fca149d0b1&l=7700]
 where lock {{0x00000000afd2df30}} is held with other threads blocking on it):
{code}
"Flat Map -> Sink: Unnamed (9/12)#44786" #9135620 prio=5 os_prio=0 
tid=0x00007fd2f41b5000 nid=0xcfd76 waiting for monitor entry 
[0x00007fd2b9042000]
   java.lang.Thread.State: BLOCKED (on object monitor)
        at 
org.apache.flink.runtime.io.network.buffer.LocalBufferPool.setNumBuffers(LocalBufferPool.java:598)
        - waiting to lock <0x00000000afd2df30> (a java.util.ArrayDeque)
        at 
org.apache.flink.runtime.io.network.buffer.NetworkBufferPool.redistributeBuffers(NetworkBufferPool.java:652)
        at 
org.apache.flink.runtime.io.network.buffer.NetworkBufferPool.internalCreateBufferPool(NetworkBufferPool.java:509)
        - locked <0x0000000082e36d70> (a java.lang.Object)
        at 
org.apache.flink.runtime.io.network.buffer.NetworkBufferPool.createBufferPool(NetworkBufferPool.java:451)
        at 
org.apache.flink.runtime.io.network.partition.consumer.SingleInputGateFactory.lambda$createBufferPoolFactory$3(SingleInputGateFactory.java:306)
        at 
org.apache.flink.runtime.io.network.partition.consumer.SingleInputGateFactory$$Lambda$1153/781071075.get(Unknown
 Source)
        at 
org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.setup(SingleInputGate.java:274)
        at 
org.apache.flink.runtime.taskmanager.InputGateWithMetrics.setup(InputGateWithMetrics.java:105)
        at 
org.apache.flink.runtime.taskmanager.Task.setupPartitionsAndGates(Task.java:965)
        at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:652)
        at org.apache.flink.runtime.taskmanager.Task.run(Task.java:563)
        at java.lang.Thread.run(Thread.java:748)
"Flat Map -> Sink: Unnamed (6/12)#44786" #9135619 prio=5 os_prio=0 
tid=0x00007fd2f4195000 nid=0xcfd75 in Object.wait() [0x00007fd2b9143000]
   java.lang.Thread.State: TIMED_WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        at 
org.apache.flink.runtime.io.network.buffer.NetworkBufferPool.internalRequestMemorySegments(NetworkBufferPool.java:243)
        - locked <0x0000000082c7dda8> (a java.util.ArrayDeque)
        at 
org.apache.flink.runtime.io.network.buffer.NetworkBufferPool.requestPooledMemorySegmentsBlocking(NetworkBufferPool.java:179)
        at 
org.apache.flink.runtime.io.network.buffer.LocalBufferPool.reserveSegments(LocalBufferPool.java:247)
        - locked <0x00000000afd2df30> (a java.util.ArrayDeque)
        at 
org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.setupChannels(SingleInputGate.java:517)
        at 
org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.setup(SingleInputGate.java:277)
        at 
org.apache.flink.runtime.taskmanager.InputGateWithMetrics.setup(InputGateWithMetrics.java:105)
        at 
org.apache.flink.runtime.taskmanager.Task.setupPartitionsAndGates(Task.java:965)
        at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:652)
        at org.apache.flink.runtime.taskmanager.Task.run(Task.java:563)
        at java.lang.Thread.run(Thread.java:748)
{code}

> BlockingShuffleITCase.testDeletePartitionFileOfBoundedBlockingShuffle timing 
> out on Azure
> -----------------------------------------------------------------------------------------
>
>                 Key: FLINK-26568
>                 URL: https://issues.apache.org/jira/browse/FLINK-26568
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Coordination, Runtime / Task, Tests
>    Affects Versions: 1.15.0
>            Reporter: Matthias Pohl
>            Priority: Critical
>              Labels: test-stability
>             Fix For: 1.15.0
>
>
> [This 
> build|https://dev.azure.com/mapohl/flink/_build/results?buildId=845&view=logs&j=0a15d512-44ac-5ba5-97ab-13a5d066c22c&t=9a028d19-6c4b-5a4e-d378-03fca149d0b1&l=12865]
>  timed out due the test 
> {{BlockingShuffleITCase.testDeletePartitionFileOfBoundedBlockingShuffle}} not 
> finishing.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

Reply via email to