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

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

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

    https://github.com/apache/flink/pull/4552#discussion_r157760560
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandler.java
 ---
    @@ -158,32 +164,44 @@ public void exceptionCaught(ChannelHandlerContext 
ctx, Throwable cause) throws E
        @Override
        public void channelRead(ChannelHandlerContext ctx, Object msg) throws 
Exception {
                try {
    -                   if (!bufferListener.hasStagedBufferOrEvent() && 
stagedMessages.isEmpty()) {
    -                           decodeMsg(msg, false);
    -                   }
    -                   else {
    -                           stagedMessages.add(msg);
    -                   }
    -           }
    -           catch (Throwable t) {
    +                   decodeMsg(msg);
    +           } catch (Throwable t) {
                        notifyAllChannelsOfErrorAndClose(t);
                }
        }
     
    +   @Override
    +   public void userEventTriggered(ChannelHandlerContext ctx, Object msg) 
throws Exception {
    +           if (msg instanceof RemoteInputChannel) {
    +                   boolean triggerWrite = 
inputChannelsWithCredit.isEmpty();
    --- End diff --
    
    please re-add the comment here, too


> Implement Netty sender incoming pipeline for credit-based
> ---------------------------------------------------------
>
>                 Key: FLINK-7456
>                 URL: https://issues.apache.org/jira/browse/FLINK-7456
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Network
>            Reporter: zhijiang
>            Assignee: zhijiang
>             Fix For: 1.5.0
>
>
> This is a part of work for credit-based network flow control.
> On sender side, each subpartition view maintains an atomic integer 
> {{currentCredit}} from receiver. Once receiving the messages of 
> {{PartitionRequest}} and {{AddCredit}}, the {{currentCredit}} is added by 
> deltas.
> Each view also maintains an atomic boolean field to mark it as registered 
> available for transfer to make sure it is enqueued in handler only once. If 
> the {{currentCredit}} increases from zero and there are available buffers in 
> the subpartition, the corresponding view will be enqueued for transferring 
> data.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to