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

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

Github user zhijiangW commented on a diff in the pull request:

    https://github.com/apache/flink/pull/6254#discussion_r200247127
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java
 ---
    @@ -479,6 +508,9 @@ void onSenderBacklog(int backlog) throws IOException {
     
                        numRequiredBuffers = backlog + initialCredit;
                        while (bufferQueue.getAvailableBufferSize() < 
numRequiredBuffers && !isWaitingForFloatingBuffers) {
    +                           // TODO: this will take a lock in the 
LocalBufferPool as well and needs to be done
    +                           // outside the synchronized block (which is a 
bit difficult trying to acquire the
    +                           // lock only once!
                                Buffer buffer = 
inputGate.getBufferPool().requestBuffer();
    --- End diff --
    
    In my previous implementation, I added a new method in `BufferProvider` 
interface to request batch of buffers at a time.  To do so, it only needs lock 
only once in `LocalBufferPool` and may be more efficient from lock side. The 
only concern is that the floating buffer distribution may be not fair in all 
the channels. So we want to implement two strategies, one fair strategy is 
requesting one buffer at a time, another greedy strategy is requesting all 
required buffers at a time, then comparing these strategies to check 
performance. Maybe it behaves different in different scenarios. I am planing to 
submit this JIRA soon. What do you think?


> Deadlock during canceling task and recycling exclusive buffer
> -------------------------------------------------------------
>
>                 Key: FLINK-9676
>                 URL: https://issues.apache.org/jira/browse/FLINK-9676
>             Project: Flink
>          Issue Type: Bug
>          Components: Network
>    Affects Versions: 1.5.0
>            Reporter: zhijiang
>            Assignee: Nico Kruber
>            Priority: Critical
>              Labels: pull-request-available
>             Fix For: 1.6.0, 1.5.1
>
>
> It may cause deadlock between task canceler thread and task thread.
> The detail is as follows:
> {{Task canceler thread -> IC1#releaseAllResources -> recycle floating buffers 
> -> {color:#d04437}lock{color}(LocalBufferPool#availableMemorySegments) -> 
> IC2#notifyBufferAvailable}} > {color:#d04437}try to 
> lock{color}(IC2#bufferQueue)
> {{Task thread -> IC2#recycle -> {color:#d04437}lock{color}(IC2#bufferQueue) 
> -> bufferQueue#addExclusiveBuffer}} -> {{floatingBuffer#recycleBuffer}} -> 
> {color:#d04437}try to lock{color}(LocalBufferPool#availableMemorySegments)
> One solution is that {{listener#notifyBufferAvailable}} can be called outside 
> the {{synchronized(availableMemorySegments) in }}{{LocalBufferPool#recycle.}}
> The existing RemoteInputChannelTest#testConcurrentOnSenderBacklogAndRecycle 
> can cover this case but the deadlock probability is very low, so this UT is 
> not stable.



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

Reply via email to