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

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

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

    https://github.com/apache/flink/pull/2020#discussion_r65209426
  
    --- Diff: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
 ---
    @@ -877,15 +880,15 @@ public TimeCharacteristic 
getStreamTimeCharacteristic() {
     
        /**
         * Reads the given file line-by-line and creates a data stream that 
contains a string with the contents of each such
    -    * line. Depending on the provided {@link WatchType}, the source may 
periodically monitor (every {@code interval} ms)
    -    * the path for new data ({@link WatchType#REPROCESS_WITH_APPENDED}), 
or process once the data currently in the path and
    -    * exit ({@link WatchType#PROCESS_ONCE}).
    +    * line. Depending on the provided {@link ProcessingMode}, the source 
may periodically monitor (every {@code interval} ms)
    +    * the path for new data ({@link ProcessingMode#PROCESS_CONTINUOUSLY}), 
or process once the data currently in the path and
    +    * exit ({@link ProcessingMode#PROCESS_ONCE}).
         *
         * <p>
    -    * <b> NOTES ON CHECKPOINTING: </b> If the {@code watchType} is set to 
{@link WatchType#PROCESS_ONCE}, the source
    -    * (which executes the {@link FileSplitMonitoringFunction}) monitors 
the path <b>once</b>, creates the
    +    * <b> NOTES ON CHECKPOINTING: </b> If the {@code watchType} is set to 
{@link ProcessingMode#PROCESS_ONCE}, the source
    +    * (which executes the {@link ContinuousFileMonitoringFunction}) 
monitors the path <b>once</b>, creates the
    --- End diff --
    
    I think we can remove the "(which executes the {@link 
ContinuousFileMonitoringFunction})" part. The source does not execute this, it 
is itself the source. 


> Make Streaming File Sources Persistent
> --------------------------------------
>
>                 Key: FLINK-2314
>                 URL: https://issues.apache.org/jira/browse/FLINK-2314
>             Project: Flink
>          Issue Type: Improvement
>          Components: Streaming
>    Affects Versions: 0.9
>            Reporter: Stephan Ewen
>            Assignee: Kostas Kloudas
>
> Streaming File sources should participate in the checkpointing. They should 
> track the bytes they read from the file and checkpoint it.
> One can look at the sequence generating source function for an example of a 
> checkpointed source.



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

Reply via email to