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

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

GitHub user zhijiangW opened a pull request:

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

    [FLINK-8547][network] Implement CheckpointBarrierHandler not to spill data 
for exactly-once

    ## What is the purpose of the change
    
    *Currently in exactly-once mode, the BarrierBuffer would block inputs with 
barriers until all inputs have received the barrier for a given checkpoint. To 
avoid back-pressuring the input streams which may cause distributed deadlocks, 
the BarrierBuffer has to spill the data in disk files to recycle the buffers 
for blocked channels.*
    
    *Based on credit-based flow control, every channel has exclusive buffers, 
so it is no need to spill data for avoiding deadlock. Then we implement a new 
CheckpointBarrierHandler for only buffering the data for blocked channels for 
better performance.*
    
    *And this new CheckpointBarrierHandler can also be configured to use or not 
in order to rollback the original mode for unexpected risks.*
    
    ## Brief change log
    
      - *Implement the new `CreditBasedBarrierBuffer` and 
`CreditBasedBufferBlocker` for buffering data in blocked channels in 
exactly-once mode.*
      - *Define the parameter `taskmanager.exactly-once.blocking.data.enabled` 
for enabling the new handler or not.*
    
    ## Verifying this change
    
    This change added tests and can be verified as follows:
    
      - *Added tests for the logic of `CreditBasedBarrierBuffer`*
      - *Added tests for the logic of `CreditBasedBufferBlocker`*
    
    ## 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)
      - The S3 file system connector: (no)
    
    ## Documentation
    
      - Does this pull request introduce a new feature? (no)
      - If yes, how is the feature documented? (not applicable)


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

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

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

    https://github.com/apache/flink/pull/5400.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 #5400
    
----
commit 4d08e5d58c732e8f835016b48edc4494f8cb26fe
Author: Zhijiang <wangzhijiang999@...>
Date:   2018-02-02T07:45:49Z

    [FLINK-8547][network] Implement CheckpointBarrierHandler not to spill data 
for exactly-once

----


> Implement CheckpointBarrierHandler not to spill data for exactly-once based 
> on credit-based flow control
> --------------------------------------------------------------------------------------------------------
>
>                 Key: FLINK-8547
>                 URL: https://issues.apache.org/jira/browse/FLINK-8547
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Network
>    Affects Versions: 1.5.0
>            Reporter: zhijiang
>            Assignee: zhijiang
>            Priority: Major
>
> Currently in exactly-once mode, the {{BarrierBuffer}} would block inputs with 
> barriers until all inputs have received the barrier for a given checkpoint. 
> To avoid back-pressuring the input streams which may cause distributed 
> deadlocks, the {{BarrierBuffer}} has to spill the data in disk files to 
> recycle the buffers for blocked channels.
>  
> Based on credit-based flow control, every channel has exclusive buffers, so 
> it is no need to spill data for avoiding deadlock. Then we implement a new 
> {{CheckpointBarrierHandler}} for only buffering the data for blocked channels 
> for better performance.
>  
> And this new {{CheckpointBarrierHandler}} can also be configured to use or 
> not in order to rollback the original mode for unexpected risks.



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

Reply via email to