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

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

pnowojski 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_r226288887
 
 

 ##########
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java
 ##########
 @@ -205,6 +216,66 @@ protected void testAddOnPartition(final 
ResultPartitionType pipelined)
                }
        }
 
+       @Test
+       public void testReleaseMemoryOnBlockingPartition() throws Exception {
+               testReleaseMemory(ResultPartitionType.BLOCKING);
+       }
+
+       @Test
+       public void testReleaseMemoryOnPipelinedPartition() throws Exception {
+               testReleaseMemory(ResultPartitionType.PIPELINED);
+       }
+
+       /**
+        * Tests {@link ResultPartition#releaseMemory(int)} on a working 
partition.
+        *
+        * @param resultPartitionType the result partition type to set up
+        */
+       private void testReleaseMemory(final ResultPartitionType 
resultPartitionType) throws Exception {
+               final int numBuffers = 10;
+               final NetworkEnvironment network = new NetworkEnvironment(
 
 Review comment:
   Could you deduplicate this constructor with other use cases? It is used in 
tests in 5 other places, 2 of them are almost exactly the same as this one and 
three others further duplicate this:
   ```
                        new LocalConnectionManager(),
                        new ResultPartitionManager(),
                        new TaskEventDispatcher(),
                        new KvStateRegistry(),
                        null,
                        null,
                        IOManager.IOMode.SYNC,
   ```
   block.
   
   Please either provide some more constructors to deduplicate this or provide 
a `NetworkEnvironmentBuilder` (or `public static class Builder` nested in 
`NetworkEnvironment`) that would handle those default values. I would 
personally prefer the builder, since it's better for the future.

----------------------------------------------------------------
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:
us...@infra.apache.org


> 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.5.4, 1.6.1
>            Reporter: Piotr Nowojski
>            Assignee: zhijiang
>            Priority: Critical
>              Labels: pull-request-available
>             Fix For: 1.5.6, 1.6.3, 1.7.0
>
>
> 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