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

ASF GitHub Bot commented on FLINK-10491:
----------------------------------------

NicoK commented on a change in pull request #6809: [FLINK-10491][network] Pass 
BufferPoolOwner in the constructor of LocalBufferPool
URL: https://github.com/apache/flink/pull/6809#discussion_r223748143
 
 

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java
 ##########
 @@ -349,11 +346,13 @@ public void setNumBuffers(int numBuffers) throws 
IOException {
 
                        returnExcessMemorySegments();
 
-                       // If there is a registered owner and we have still 
requested more buffers than our
-                       // size, trigger a recycle via the owner.
-                       if (owner != null && numberOfRequestedMemorySegments > 
currentPoolSize) {
-                               
owner.releaseMemory(numberOfRequestedMemorySegments - currentPoolSize);
-                       }
+                       numReleased = numberOfRequestedMemorySegments - 
currentPoolSize;
+               }
+
+               // If there is a registered owner and we have still requested 
more buffers than our
+               // size, trigger a recycle via the owner.
+               if (owner != null && numReleased > 0) {
+                       owner.releaseMemory(numReleased);
 
 Review comment:
   I'd say, this is safe.
   Since recycled memory segments will eventually become available, a blocking 
call by any other thread on getting a buffer will still wait until it is 
actually recycled. What other interactions were you thinking about?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


> Deadlock during spilling data in SpillableSubpartition 
> -------------------------------------------------------
>
>                 Key: FLINK-10491
>                 URL: https://issues.apache.org/jira/browse/FLINK-10491
>             Project: Flink
>          Issue Type: Bug
>          Components: Network
>    Affects Versions: 1.6.1, 1.5.4
>            Reporter: Piotr Nowojski
>            Assignee: zhijiang
>            Priority: Critical
>              Labels: pull-request-available
>
> Originally reported here: 
> [https://lists.apache.org/thread.html/472c8f4a2711c5e217fadd9a88f8c73670218e7432bb81ba3f5076db@%3Cuser.flink.apache.org%3E]
> Thread dump (from 1.5.3 version) showing two deadlocked threads, because they 
> are taking two locks in different order:
> {noformat}
> Thread-1
> "DataSink (DATA#HadoopFileOutputFormat ) (1/2)@11002" prio=5 tid=0x3e2 nid=NA 
> waiting for monitor entry
> waiting for Map (Key Extractor) (1/10)@9967 to release lock on <0x2dfb> (a 
> java.util.ArrayDeque)
> at 
> org.apache.flink.runtime.io.network.partition.SpillableSubpartition.releaseMemory(SpillableSubpartition.java:223)
> at 
> org.apache.flink.runtime.io.network.partition.ResultPartition.releaseMemory(ResultPartition.java:373)
> at 
> org.apache.flink.runtime.io.network.buffer.LocalBufferPool.setNumBuffers(LocalBufferPool.java:355)
> - locked <0x2dfd> (a java.util.ArrayDeque)
> at 
> org.apache.flink.runtime.io.network.buffer.NetworkBufferPool.redistributeBuffers(NetworkBufferPool.java:402)
> at 
> org.apache.flink.runtime.io.network.buffer.NetworkBufferPool.recycleMemorySegments(NetworkBufferPool.java:203)
> - locked <0x2da5> (a java.lang.Object)
> at 
> org.apache.flink.runtime.io.network.buffer.NetworkBufferPool.recycleMemorySegments(NetworkBufferPool.java:193)
> at 
> org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.returnExclusiveSegments(SingleInputGate.java:318)
> at 
> org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel.releaseAllResources(RemoteInputChannel.java:259)
> at 
> org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.getNextBufferOrEvent(SingleInputGate.java:578)
> at 
> org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.pollNextBufferOrEvent(SingleInputGate.java:507)
> at 
> org.apache.flink.runtime.io.network.partition.consumer.UnionInputGate.waitAndGetNextInputGate(UnionInputGate.java:213)
> at 
> org.apache.flink.runtime.io.network.partition.consumer.UnionInputGate.getNextBufferOrEvent(UnionInputGate.java:163)
> at 
> org.apache.flink.runtime.io.network.api.reader.AbstractRecordReader.getNextRecord(AbstractRecordReader.java:86)
> at 
> org.apache.flink.runtime.io.network.api.reader.MutableRecordReader.next(MutableRecordReader.java:47)
> at 
> org.apache.flink.runtime.operators.util.ReaderIterator.next(ReaderIterator.java:73)
> at 
> org.apache.flink.runtime.operators.DataSinkTask.invoke(DataSinkTask.java:216)
> at org.apache.flink.runtime.taskmanager.Task.run(Task.java:703)
> at java.lang.Thread.run(Thread.java:745)
> Thread-2
> "Map (Key Extractor) (1/10)@9967" prio=5 tid=0xaab nid=NA waiting for monitor 
> entry
> java.lang.Thread.State: BLOCKED
> blocks DataSink (DATA#HadoopFileOutputFormat ) (1/2)@11002
> waiting for DataSink (DATA#HadoopFileOutputFormat ) (1/2)@11002 to release 
> lock on <0x2dfd> (a java.util.ArrayDeque)
> at 
> org.apache.flink.runtime.io.network.buffer.LocalBufferPool.recycle(LocalBufferPool.java:261)
> at 
> org.apache.flink.runtime.io.network.buffer.NetworkBuffer.deallocate(NetworkBuffer.java:171)
> at 
> org.apache.flink.shaded.netty4.io.netty.buffer.AbstractReferenceCountedByteBuf.release(AbstractReferenceCountedByteBuf.java:106)
> at 
> org.apache.flink.runtime.io.network.buffer.NetworkBuffer.recycleBuffer(NetworkBuffer.java:146)
> at 
> org.apache.flink.runtime.io.network.buffer.BufferConsumer.close(BufferConsumer.java:110)
> at 
> org.apache.flink.runtime.io.network.partition.SpillableSubpartition.spillFinishedBufferConsumers(SpillableSubpartition.java:271)
> at 
> org.apache.flink.runtime.io.network.partition.SpillableSubpartition.add(SpillableSubpartition.java:117)
> - locked <0x2dfb> (a java.util.ArrayDeque)
> at 
> org.apache.flink.runtime.io.network.partition.SpillableSubpartition.add(SpillableSubpartition.java:96)
> - locked <0x2dfc> (a 
> org.apache.flink.runtime.io.network.partition.SpillableSubpartition)
> at 
> org.apache.flink.runtime.io.network.partition.ResultPartition.addBufferConsumer(ResultPartition.java:255)
> at 
> org.apache.flink.runtime.io.network.api.writer.RecordWriter.requestNewBufferBuilder(RecordWriter.java:211)
> at 
> org.apache.flink.runtime.io.network.api.writer.RecordWriter.sendToTarget(RecordWriter.java:142)
> at 
> org.apache.flink.runtime.io.network.api.writer.RecordWriter.emit(RecordWriter.java:105)
> at 
> org.apache.flink.runtime.operators.shipping.OutputCollector.collect(OutputCollector.java:65)
> at 
> org.apache.flink.runtime.operators.util.metrics.CountingCollector.collect(CountingCollector.java:35)
> at org.apache.flink.runtime.operators.MapDriver.run(MapDriver.java:103)
> at org.apache.flink.runtime.operators.BatchTask.run(BatchTask.java:503)
> at org.apache.flink.runtime.operators.BatchTask.invoke(BatchTask.java:368)
> at org.apache.flink.runtime.taskmanager.Task.run(Task.java:703)
> at java.lang.Thread.run(Thread.java:745)
> {noformat}
> The same pattern can occur on the master (but with slightly shifted line 
> numbers)
> CC [~NicoK]



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to