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

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_r162103769
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactory.java
 ---
    @@ -164,11 +165,13 @@ private boolean dispose() {
                private void handInChannel(Channel channel) {
                        synchronized (connectLock) {
                                try {
    -                                   PartitionRequestClientHandler 
requestHandler = channel.pipeline()
    -                                                   
.get(PartitionRequestClientHandler.class);
    +                                   NetworkClientHandler clientHandler = 
channel.pipeline().get(PartitionRequestClientHandler.class);
    +                                   if (clientHandler == null) {
    +                                           clientHandler = 
channel.pipeline().get(CreditBasedPartitionRequestClientHandler.class);
    +                                   }
    --- End diff --
    
    if you let `NetworkClientHandler` extend from `ChannelHandler`, then this 
can be simplified to
    ```
    NetworkClientHandler clientHandler = 
channel.pipeline().get(NetworkClientHandler.class);
    ```


> 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
>            Priority: Major
>             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
(v7.6.3#76005)

Reply via email to