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

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

GitHub user kl0u reopened a pull request:

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

    [FLINK-2314] Make Streaming File Sources Persistent

    This PR solves FLINK-2314 and combines a number of sub-tasks. In addition, 
it solves FLINK-3896 which was introduced as part of this task.
    
    The way File Input sources are now processed is the following:
         * One task monitors (parallelism 1) a user-specified path for new 
files/data
         * The above task assigns FileInputSplits to downstream (parallel) 
readers to actually read the data
    
    The monitoring entity scans the path, splits the files to be processed in 
splits, and assigns them downstream. For now, two modes are supported. These 
are the PROCESS_ONCE which just processes the current contents of the path and 
exits, and the REPROCESS_WITH_APPENDED which periodically monitors the path and 
reprocesses new files and (the entire contents of) files with new data.
    
    In addition, these sources are checkpointed, i.e. in the case of a task 
failure the job will resume from where it left off.
    
    Finally, some changes were introduced in the way we are handling 
FileInputFormats after discussions with @aljoscha .

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

    $ git pull https://github.com/kl0u/flink api_ft_files

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

    https://github.com/apache/flink/pull/2020.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 #2020
    
----
commit 457727b4b2c7bfad914ad9876dd4135355de732f
Author: kl0u <kklou...@gmail.com>
Date:   2016-04-10T14:56:42Z

    [FLINK-3717] Make FileInputFormat checkpointable
    
    This adds a new interface called CheckpointableInputFormat
    which describes input formats whose state is queryable,
    i.e. getCurrentChannelState() returns where the reader is
    in the underlying source, and they can resume reading from
    a user-specified position.
    
    This functionality is not yet leveraged by current readers.

commit edac9fea48200e62fc38b96926d9254c93830499
Author: kl0u <kklou...@gmail.com>
Date:   2016-04-18T14:37:54Z

    [FLINK-3889] Refactor File Monitoring Source
    
    This is meant to replace the different file
    reading sources in Flink streaming. Now there is
    one monitoring source with DOP 1 monitoring a
    directory and assigning input split to downstream
    readers.
    
    In addition, it makes the new features added by
    FLINK-3717 work together with the aforementioned entities
    (the monitor and the readers) in order to have
    fault tolerant file sources and exactly once guarantees.
    
    This does not replace the old API calls. This
    will be done in a future commit.

commit d343d1143514e97b2ce9acabcbcc2fdaf2f89814
Author: kl0u <kklou...@gmail.com>
Date:   2016-05-10T16:56:58Z

    [FLINK-3896] Allow a StreamTask to be Externally Cancelled
    
    It adds a method failExternally() to the StreamTask, so that custom 
Operators
    can make their containing task fail when needed.

commit 966244c703012e8674e9786a033f7d779ceb6f73
Author: kl0u <kklou...@gmail.com>
Date:   2016-05-18T14:44:45Z

    [FLINK-2314] Make Streaming File Sources Persistent
    
    This commit takes the changes from the previous
    commits and wires them into the API, both Java and Scala.
    
    While doing so, some changes were introduced to the
    classes actually doing the work, either as bug fixes, or
    as new design choices.

commit f17b5318fb84b3111ac8407ef11e719c1fb9b360
Author: kl0u <kklou...@gmail.com>
Date:   2016-05-27T11:56:44Z

    Integrating the PR Comments.

commit 66e1423ae512f60c955113d1fd564e50663d4ea2
Author: kl0u <kklou...@gmail.com>
Date:   2016-05-31T23:07:20Z

    Final comments

commit 61d2a1a9cc75f0c698e0bca9c1e6adb612336f01
Author: kl0u <kklou...@gmail.com>
Date:   2016-06-01T12:45:46Z

    Final Commnents.

commit e1dac4b506d470299bd9504c344797910394fe59
Author: kl0u <kklou...@gmail.com>
Date:   2016-06-06T09:19:29Z

    Fixing broken test

----


> 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