flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "zhijiang (JIRA)" <j...@apache.org>
Subject [jira] [Updated] (FLINK-7282) Credit-based Network Flow Control
Date Thu, 27 Jul 2017 11:06:00 GMT

     [ https://issues.apache.org/jira/browse/FLINK-7282?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]

zhijiang updated FLINK-7282:
----------------------------
    Description: 
This is a part of work for network stack improvements proposed in [~StephanEwen]  's [FLIP|https://docs.google.com/document/d/1chTOuOqe0sBsjldA_r-wXYeSIhU2zRGpUaTaik7QZ84/edit#]

Backpressure currently happens very naturally through the TCP network connections and the
bounded buffering capacity. The downsides are :

* All channels multiplexed into the same TCP connection stall together, as soon as one channel
has backpressure.
* Under backpressure, connections can not transport checkpoint barriers.

This flink-managed flow control is similar to the window-based advertisement mechanism in
TCP. The basic approaches are the following:

* Each RemoteInputChannel has fixed exclusive buffers as initial credits, and SingleInputGate
has a fixed buffer pool for managing floating buffers for all RemoteInputChannels.
* RemoteInputChannel as receiver notifies the current available credits to the sender side.
* Senders must never send buffers without credit, that means all the buffers sent must be
accepted by receivers so no buffers accumulated on the network wire.  
* Senders also send the current size of backlog that indicates how many buffers are available
on the sender side. The receivers use this information to decide how to request floating buffers
from the fixed buffer pool.

To avoid immediate commits affecting master branch, it will be implemented into a separate
feature branch.

  was:
This is a part of work for network stack improvements proposed in [~StephanEwen]  's [FLIP|https://docs.google.com/document/d/1chTOuOqe0sBsjldA_r-wXYeSIhU2zRGpUaTaik7QZ84/edit#]

Backpressure currently happens very naturally through the TCP network connections and the
bounded buffering capacity. The downsides are :

* All channels multiplexed into the same TCP connection stall together, as soon as one channel
has backpressure.
* Under backpressure, connections can not transport checkpoint barriers.

This flink-managed flow control is similar to the window-based advertisement mechanism in
TCP. The basic approaches are the following:

* Each RemoteInputChannel has fixed exclusive buffers as initial credits, and SingleInputGate
has a fixed buffer pool for managing floating buffers for all RemoteInputChannels.
* RemoteInputChannel as receiver notifies the current available credits to the sender side.
* Senders must never send buffers without credit, that means all the buffers sent must be
accepted by receivers so no buffers accumulated on the network wire.  
* Senders also sends the current size of backlog that indicates how many buffers are available
on the sender side. The receivers use this information to decide how to request floating buffers
from the fixed buffer pool.

To avoid immediate commits affecting master branch, it will be implemented into a separate
feature branch.


> Credit-based Network Flow Control
> ---------------------------------
>
>                 Key: FLINK-7282
>                 URL: https://issues.apache.org/jira/browse/FLINK-7282
>             Project: Flink
>          Issue Type: New Feature
>          Components: Network
>            Reporter: zhijiang
>
> This is a part of work for network stack improvements proposed in [~StephanEwen]  's
[FLIP|https://docs.google.com/document/d/1chTOuOqe0sBsjldA_r-wXYeSIhU2zRGpUaTaik7QZ84/edit#]
> Backpressure currently happens very naturally through the TCP network connections and
the bounded buffering capacity. The downsides are :
> * All channels multiplexed into the same TCP connection stall together, as soon as one
channel has backpressure.
> * Under backpressure, connections can not transport checkpoint barriers.
> This flink-managed flow control is similar to the window-based advertisement mechanism
in TCP. The basic approaches are the following:
> * Each RemoteInputChannel has fixed exclusive buffers as initial credits, and SingleInputGate
has a fixed buffer pool for managing floating buffers for all RemoteInputChannels.
> * RemoteInputChannel as receiver notifies the current available credits to the sender
side.
> * Senders must never send buffers without credit, that means all the buffers sent must
be accepted by receivers so no buffers accumulated on the network wire.  
> * Senders also send the current size of backlog that indicates how many buffers are available
on the sender side. The receivers use this information to decide how to request floating buffers
from the fixed buffer pool.
> To avoid immediate commits affecting master branch, it will be implemented into a separate
feature branch.



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

Mime
View raw message