Details
-
Bug
-
Status: Closed
-
Critical
-
Resolution: Fixed
-
1.14.0, 1.15.0
Description
This issue was extracted from and initially reported in FLINK-25185. It is most likely caused by FLINK-24035.
Java stack information for the threads listed above: =================================================== "Canceler for Source: Custom Source -> Filter (7/12)#14176 (0fbb8a89616ca7a40e473adad51f236f).": at org.apache.flink.runtime.io.network.buffer.NetworkBufferPool.destroyBufferPool(NetworkBufferPool.java:420) - waiting to lock <0x0000000082937f28> (a java.lang.Object) at org.apache.flink.runtime.io.network.buffer.LocalBufferPool.lazyDestroy(LocalBufferPool.java:567) at org.apache.flink.runtime.io.network.partition.ResultPartition.closeBufferPool(ResultPartition.java:264) at org.apache.flink.runtime.io.network.partition.ResultPartition.fail(ResultPartition.java:276) at org.apache.flink.runtime.taskmanager.Task.failAllResultPartitions(Task.java:999) at org.apache.flink.runtime.taskmanager.Task.access$100(Task.java:138) at org.apache.flink.runtime.taskmanager.Task$TaskCanceler.run(Task.java:1669) at java.lang.Thread.run(Thread.java:748) "Canceler for Map -> Map (6/12)#14176 (6195862d199aa4d52c12f25b39904725).": at org.apache.flink.runtime.io.network.buffer.LocalBufferPool.setNumBuffers(LocalBufferPool.java:585) - waiting to lock <0x0000000097108898> (a java.util.ArrayDeque) at org.apache.flink.runtime.io.network.buffer.NetworkBufferPool.redistributeBuffers(NetworkBufferPool.java:544) at org.apache.flink.runtime.io.network.buffer.NetworkBufferPool.destroyBufferPool(NetworkBufferPool.java:424) - locked <0x0000000082937f28> (a java.lang.Object) at org.apache.flink.runtime.io.network.buffer.LocalBufferPool.lazyDestroy(LocalBufferPool.java:567) at org.apache.flink.runtime.io.network.partition.ResultPartition.closeBufferPool(ResultPartition.java:264) at org.apache.flink.runtime.io.network.partition.ResultPartition.fail(ResultPartition.java:276) at org.apache.flink.runtime.taskmanager.Task.failAllResultPartitions(Task.java:999) at org.apache.flink.runtime.taskmanager.Task.access$100(Task.java:138) at org.apache.flink.runtime.taskmanager.Task$TaskCanceler.run(Task.java:1669) at java.lang.Thread.run(Thread.java:748) "Map -> Sink: Unnamed (7/12)#14176": at org.apache.flink.runtime.io.network.buffer.NetworkBufferPool.recycleMemorySegments(NetworkBufferPool.java:256) - waiting to lock <0x0000000082937f28> (a java.lang.Object) at org.apache.flink.runtime.io.network.buffer.NetworkBufferPool.internalRequestMemorySegments(NetworkBufferPool.ja at org.apache.flink.runtime.io.network.buffer.NetworkBufferPool.requestMemorySegmentsBlocking(NetworkBufferPool.ja at org.apache.flink.runtime.io.network.buffer.LocalBufferPool.reserveSegments(LocalBufferPool.java:247) - locked <0x0000000097108898> (a java.util.ArrayDeque) at org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.setupChannels(SingleInputGate.java:497) at org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.setup(SingleInputGate.java:276) 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) Found 1 deadlock.
https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=28297&view=logs&j=a57e0635-3fad-5b08-57c7-a4142d7d6fa9&t=2ef0effc-1da1-50e5-c2bd-aab434b1c5b7&l=19003
https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=28306&view=logs&j=5c8e7682-d68f-54d1-16a2-a09310218a49&t=86f654fa-ab48-5c1a-25f4-7e7f6afb9bba&l=19832
CC kevin.cyj
Attachments
Issue Links
- is caused by
-
FLINK-24035 Fix the deadlock issue caused by buffer listeners may not be notified
- Closed
- is related to
-
FLINK-25185 StreamFaultToleranceTestBase hangs on AZP
- Closed
- links to