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

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

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

    https://github.com/apache/flink/pull/951#discussion_r35778711
  
    --- Diff: 
flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java
 ---
    @@ -269,10 +270,19 @@ private void setVertexConfig(Integer vertexID, 
StreamConfig config,
                config.setNumberOfOutputs(nonChainableOutputs.size());
                config.setNonChainedOutputs(nonChainableOutputs);
                config.setChainedOutputs(chainableOutputs);
    -           config.setStateMonitoring(streamGraph.isCheckpointingEnabled());
    -           
config.setStateHandleProvider(streamGraph.getStateHandleProvider());
    +
    +           
config.setCheckpointingEnabled(streamGraph.isCheckpointingEnabled());
    +           if (streamGraph.isCheckpointingEnabled()) {
    +                   
config.setCheckpointMode(streamGraph.getCheckpointingMode());
    +                   
config.setStateHandleProvider(streamGraph.getStateHandleProvider());
    +           } else {
    +                   // the at least once input handler is slightly cheaper 
(in the absence of checkpoints),
    +                   // so we use that one if checkpointing is not enabled
    +                   
config.setCheckpointMode(CheckpointingMode.AT_LEAST_ONCE);
    --- End diff --
    
    So we set an at_least_once handler even though we won't receive any 
barriers? Why not add an OFF mode which will use a "no-op barrier handler" or 
skip the barrier handler  altogether? 


> Add an API switch to select between "exactly once" and "at least once" fault 
> tolerance
> --------------------------------------------------------------------------------------
>
>                 Key: FLINK-2407
>                 URL: https://issues.apache.org/jira/browse/FLINK-2407
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Streaming
>    Affects Versions: 0.10
>            Reporter: Stephan Ewen
>            Assignee: Stephan Ewen
>             Fix For: 0.10
>
>
> Based on the addition of the BarrierTracker, we can add a switch to choose 
> between the two modes "exactly once" and "at least once".



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to