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

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

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

    https://github.com/apache/flink/pull/1341#discussion_r44529482
  
    --- Diff: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java
 ---
    @@ -173,4 +174,12 @@ public void registerTimer(long time, Triggerable 
target) {
                        }
                }
        }
    +
    +   /**
    +    * Returns true if checkpointing is enabled for the running job.
    +    * @return true if checkpointing is enabled.
    +    */
    +   public boolean isCheckpointingEnabled() {
    +           return 
taskEnvironment.getTaskConfiguration().getBoolean(StreamConfig.CHECKPOINTING_ENABLED,
 false);
    --- End diff --
    
    That breaks the way that config values are accessed. The right way is
    `new 
StreamConfig(taskEnvironment.getTaskConfiguration()).isCheckpointingEnabled()`.
    
    Then you also need not change the visibility of the flag.


> Add periodic offset commit to Kafka Consumer if checkpointing is disabled
> -------------------------------------------------------------------------
>
>                 Key: FLINK-2974
>                 URL: https://issues.apache.org/jira/browse/FLINK-2974
>             Project: Flink
>          Issue Type: Improvement
>          Components: Kafka Connector
>            Reporter: Robert Metzger
>
> Flink only writes the offsets from the consumer into ZK if checkpointing is 
> enabled.
> We should have a similar feature to Kafka's autocommit in our consumer.
> Issue reported by user: 
> http://stackoverflow.com/questions/33501574/flink-kafka-why-am-i-losing-messages



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

Reply via email to