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

Dawid Wysakowicz commented on FLINK-18832:
------------------------------------------

As far as I can tell currently that can not occur. The Blink batch planner, 
which is the only user of the lazy scheduling with blocking data exchanges 
disables the output flashers by setting: {{execEnv.setBufferTimeout(-1);}}. I 
am working on exposing the "{{BATCH}}" behaviour in the DataStream as well but 
I did not set this parameter, as I was not aware of the consequences. I think 
it is not a pressing issue if we say it is an illegal combination.


> BoundedBlockingSubpartition does not work with StreamTask
> ---------------------------------------------------------
>
>                 Key: FLINK-18832
>                 URL: https://issues.apache.org/jira/browse/FLINK-18832
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Network, Runtime / Task
>    Affects Versions: 1.10.1, 1.12.0, 1.11.1
>            Reporter: Till Rohrmann
>            Assignee: Zhijiang
>            Priority: Major
>
> The {{BoundedBlockingSubpartition}} does not work with a {{StreamTask}} 
> because the {{StreamTask}} instantiates an {{OutputFlusher}} which 
> concurrently accesses the {{BoundedBlockingSubpartition}}. This concurrency 
> can lead to a double closing of the underlying {{BufferConsumer}} which 
> manifests in this stack trace:
> {code}
> [9:15 PM] Caused by: 
> org.apache.flink.shaded.netty4.io.netty.util.IllegalReferenceCountException: 
> refCnt: 0, increment: 1
>       at 
> org.apache.flink.shaded.netty4.io.netty.util.internal.ReferenceCountUpdater.retain0(ReferenceCountUpdater.java:123)
>       at 
> org.apache.flink.shaded.netty4.io.netty.util.internal.ReferenceCountUpdater.retain(ReferenceCountUpdater.java:110)
>       at 
> org.apache.flink.shaded.netty4.io.netty.buffer.AbstractReferenceCountedByteBuf.retain(AbstractReferenceCountedByteBuf.java:80)
>       at 
> org.apache.flink.runtime.io.network.buffer.NetworkBuffer.retainBuffer(NetworkBuffer.java:174)
>       at 
> org.apache.flink.runtime.io.network.buffer.NetworkBuffer.retainBuffer(NetworkBuffer.java:47)
>       at 
> org.apache.flink.runtime.io.network.buffer.ReadOnlySlicedNetworkBuffer.retainBuffer(ReadOnlySlicedNetworkBuffer.java:127)
>       at 
> org.apache.flink.runtime.io.network.buffer.ReadOnlySlicedNetworkBuffer.retainBuffer(ReadOnlySlicedNetworkBuffer.java:41)
>       at 
> org.apache.flink.runtime.io.network.buffer.BufferConsumer.build(BufferConsumer.java:108)
>       at 
> org.apache.flink.runtime.io.network.partition.BoundedBlockingSubpartition.writeAndCloseBufferConsumer(BoundedBlockingSubpartition.java:156)
>       at 
> org.apache.flink.runtime.io.network.partition.BoundedBlockingSubpartition.flushCurrentBuffer(BoundedBlockingSubpartition.java:144)
>       at 
> org.apache.flink.runtime.io.network.partition.BoundedBlockingSubpartition.flush(BoundedBlockingSubpartition.java:135)
>       at 
> org.apache.flink.runtime.io.network.partition.ResultPartition.flushAll(ResultPartition.java:245)
>       at 
> org.apache.flink.runtime.io.network.api.writer.RecordWriter.flushAll(RecordWriter.java:183)
>       at 
> org.apache.flink.streaming.runtime.io.RecordWriterOutput.flush(RecordWriterOutput.java:156)
>       at 
> org.apache.flink.streaming.runtime.tasks.OperatorChain.flushOutputs(OperatorChain.java:344)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.afterInvoke(StreamTask.java:602)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:544)
>       at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:721)
>       at org.apache.flink.runtime.taskmanager.Task.run(Task.java:546)
>       at java.lang.Thread.run(Thread.java:748)
> {code}



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

Reply via email to