You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2017/08/12 15:39:01 UTC

[jira] [Commented] (FLINK-7416) Implement Netty receiver outgoing pipeline for credit-based

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

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

GitHub user zhijiangW opened a pull request:

    https://github.com/apache/flink/pull/4533

    [FLINK-7416][network] Implement Netty receiver outgoing pipeline for credit-based

    ## What is the purpose of the change
    
    The receiver notifies the available buffers called unannounced credit here, then the sender controls sending data based on receiver's credit. 
    
    This PR realizes the process of sending credit via receiver's netty handler pipeline.
    
    This PR is based on #4509 whose commits are also included for passing travis, so review the fourth commit for this PR change.
    
    ## Brief change log
    
      - *Add new `AddCredit` message and add credit field in existing `PartitionRequest` message*
      - *Notify the initial credit via `PartitionRequest` message*
      - *Enqueue input channel in the pipeline for sending unannounced credit*
      - *Set method of `releaseAllResources` as public in `RemoteInputChannel for test call*
    
    ## Verifying this change
    
    This change added tests and can be verified as follows:
    
      - *Added test that verifies the netty message serialization for add credit and partition request*
      - *Added test to verify that the input channel is enqueued to trigger send credit by handler *
    
    ## Does this pull request potentially affect one of the following parts:
    
      - Dependencies (does it add or upgrade a dependency): (no)
      - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (no)
      - The serializers: (no)
      - The runtime per-record code paths (performance sensitive): (yes)
      - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: (no)
    
    ## Documentation
    
      - Does this pull request introduce a new feature? (yes)
      - If yes, how is the feature documented? (docs)
    


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/zhijiangW/flink FLINK-7416

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/4533.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #4533
    
----
commit a64016633ef53e7116ce588a4110cff587acadf6
Author: Zhijiang <wa...@aliyun.com>
Date:   2017-08-07T09:31:17Z

    [FLINK-7378][core]Implement the FixedBufferPool for floating buffers of SingleInputGate

commit 9b7ee9ff559a3bcfe7b8f08802cf55349bb72abc
Author: Zhijiang <wa...@aliyun.com>
Date:   2017-08-09T05:43:56Z

    [FLINK-7394][core]Implement basic InputChannel for credit-based logic

commit 0e0cb51b2ef97c73ed4d17629b1928fb911c9c92
Author: Zhijiang <wa...@aliyun.com>
Date:   2017-08-10T05:29:13Z

    [FLINK-7406][network]Implement Netty receiver incoming pipeline for credit-based

commit 855f81b6184730408c5a13d69aebbd627dd53aea
Author: Zhijiang <wa...@aliyun.com>
Date:   2017-08-12T14:13:25Z

    [FLINK-7416][network] Implement Netty receiver outgoing pipeline for credit-based

----


> Implement Netty receiver outgoing pipeline for credit-based
> -----------------------------------------------------------
>
>                 Key: FLINK-7416
>                 URL: https://issues.apache.org/jira/browse/FLINK-7416
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Network
>            Reporter: zhijiang
>            Assignee: zhijiang
>             Fix For: 1.4.0
>
>
> This is a part of work for credit-based network flow control.
> The related works are :
> * The {{InputChannel}} notifies the initial credit which equals to the number of exclusive buffers per channel via {{PartitionRequest}} message.
> *  We define another message called {{AddCredit}} to notify the incremental credit during data shuffle. 
> * Whenever an {{InputChannel}}’s unannounced credit goes up from zero, the channel is enqueued in the pipeline.
> * Whenever the channel becomes writable, it takes the next {{InputChannel}} and sends its unannounced credit. The credit is reset to zero after each sent.
> * That way, messages are sent as often as the network has capacity and contain as much credit as available for the channel at that point in time. Otherwise, it would only add latency to the announcements and not increase throughput.



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