You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Chesnay Schepler (Jira)" <ji...@apache.org> on 2020/07/14 07:54:00 UTC

[jira] [Updated] (FLINK-18595) Deadlock during job shutdown

     [ https://issues.apache.org/jira/browse/FLINK-18595?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Chesnay Schepler updated FLINK-18595:
-------------------------------------
    Priority: Blocker  (was: Major)

> Deadlock during job shutdown
> ----------------------------
>
>                 Key: FLINK-18595
>                 URL: https://issues.apache.org/jira/browse/FLINK-18595
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Network
>    Affects Versions: 1.10.0
>            Reporter: Dian Fu
>            Priority: Blocker
>
> https://travis-ci.org/github/apache/flink/jobs/707843779
> {code}
> Found one Java-level deadlock:
> =============================
> "Canceler for Flat Map -> Sink: Unnamed (9/12) (b87b3f2cae66987d94399f12d7fb4641).":
>   waiting to lock monitor 0x00007f51f655e228 (object 0x00000000812b9180, a org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel$AvailableBufferQueue),
>   which is held by "Flat Map -> Sink: Unnamed (9/12)"
> "Flat Map -> Sink: Unnamed (9/12)":
>   waiting to lock monitor 0x000055fb00bb4b88 (object 0x00000000812b9210, a org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel$AvailableBufferQueue),
>   which is held by "Canceler for Flat Map -> Sink: Unnamed (9/12) (b87b3f2cae66987d94399f12d7fb4641)."
> Java stack information for the threads listed above:
> ===================================================
> "Canceler for Flat Map -> Sink: Unnamed (9/12) (b87b3f2cae66987d94399f12d7fb4641).":
> 	at org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel.notifyBufferAvailable(RemoteInputChannel.java:360)
> 	- waiting to lock <0x00000000812b9180> (a org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel$AvailableBufferQueue)
> 	at org.apache.flink.runtime.io.network.buffer.LocalBufferPool.fireBufferAvailableNotification(LocalBufferPool.java:315)
> 	at org.apache.flink.runtime.io.network.b4511: No such processuffer.LocalBufferPool.recycle(LocalBufferPool.java:305)
> 	at org.apache.flink.runtime.io.network.buffer.NetworkBuffer.deallocate(NetworkBuffer.java:197)
> 	at org.apache.flink.shaded.netty4.io.netty.buffer.AbstractReferenceCountedByteBuf.handleRelease(AbstractReferenceCountedByteBuf.java:110)
> 	at org.apache.flink.shaded.netty4.io.netty.buffer.AbstractReferenceCountedByteBuf.release(AbstractReferenceCountedByteBuf.java:100)
> 	at org.apache.flink.runtime.io.network.buffer.NetworkBuffer.recycleBuffer(NetworkBuffer.java:171)
> 	at org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel$AvailableBufferQueue.releaseAll(RemoteInputChannel.java:665)
> 	at org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel.releaseAllResources(RemoteInputChannel.java:254)
> 	- locked <0x00000000812b9210> (a org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel$AvailableBufferQueue)
> 	at org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.close(SingleInputGate.java:431)
> 	- locked <0x0000000080ba2488> (a java.lang.Object)
> 	at org.apache.flink.runtime.taskmanager.InputGateWithMetrics.close(InputGateWithMetrics.java:85)
> 	at org.apache.flink.runtime.taskmanager.Task.closeNetworkResources(Task.java:901)
> 	at org.apache.flink.runtime.taskmanager.Task$$Lambda$434/985222953.run(Unknown Source)
> 	at org.apache.flink.runtime.taskmanager.Task$TaskCanceler.run(Task.java:1370)
> 	at java.lang.Thread.run(Thread.java:748)
> "Flat Map -> Sink: Unnamed (9/12)":
> 	at org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel.notifyBufferAvailable(RemoteInputChannel.java:360)
> 	- waiting to lock <0x00000000812b9210> (a org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel$AvailableBufferQueue)
> 	at org.apache.flink.runtime.io.network.buffer.LocalBufferPool.fireBufferAvailableNotification(LocalBufferPool.java:315)
> 	at org.apache.flink.runtime.io.network.buffer.LocalBufferPool.recycle(LocalBufferPool.java:305)
> 	at org.apache.flink.runtime.io.network.buffer.NetworkBuffer.deallocate(NetworkBuffer.java:197)
> 	at org.apache.flink.shaded.netty4.io.netty.buffer.AbstractReferenceCountedByteBuf.handleRelease(AbstractReferenceCountedByteBuf.java:110)
> 	at org.apache.flink.shaded.netty4.io.netty.buffer.AbstractReferenceCountedByteBuf.release(AbstractReferenceCountedByteBuf.java:100)
> 	at org.apache.flink.runtime.io.network.buffer.NetworkBuffer.recycleBuffer(NetworkBuffer.java:171)
> 	at org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel$AvailableBufferQueue.addExclusiveBuffer(RemoteInputChannel.java:629)
> 	at org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel.recycle(RemoteInputChannel.java:314)
> 	- locked <0x00000000812b9180> (a org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel$AvailableBufferQueue)
> 	at org.apache.flink.runtime.io.network.buffer.NetworkBuffer.deallocate(NetworkBuffer.java:197)
> 	at org.apache.flink.shaded.netty4.io.netty.buffer.AbstractReferenceCountedByteBuf.handleRelease(AbstractReferenceCountedByteBuf.java:110)
> 	at org.apache.flink.shaded.netty4.io.netty.buffer.AbstractReferenceCountedByteBuf.release(AbstractReferenceCountedByteBuf.java:100)
> 	at org.apache.flink.runtime.io.network.buffer.NetworkBuffer.recycleBuffer(NetworkBuffer.java:171)
> 	at org.apache.flink.streaming.runtime.io.CachedBufferStorage.close(CachedBufferStorage.java:113)
> 	at org.apache.flink.streaming.runtime.io.CheckpointedInputGate.cleanup(CheckpointedInputGate.java:216)
> 	at org.apache.flink.streaming.runtime.io.StreamTaskNetworkInput.close(StreamTaskNetworkInput.java:208)
> 	at org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.close(StreamOneInputProcessor.java:82)
> 	at org.apache.flink.streaming.runtime.tasks.StreamTask.cleanup(StreamTask.java:298)
> 	at org.apache.flink.streaming.runtime.tasks.StreamTask.cleanUpInvoke(StreamTask.java:555)
> 	at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:480)
> 	at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:708)
> 	at org.apache.flink.runtime.taskmanager.T
> ask.run(Task.java:533)
> 	at java.lang.Thread.run(Thread.java:748)
> Found 1 deadlock.
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)