Fyi, this is the Jira issue for tracking the issue: 
https://issues.apache.org/jira/browse/FLINK-2491 
<https://issues.apache.org/jira/browse/FLINK-2491>

Aljoscha

> On 6. Mar 2018, at 02:32, Nico Kruber <n...@data-artisans.com> wrote:
> 
> There are still some upcoming changes for the network stack, but most of
> the heavy stuff it already through - you may track this under
> https://issues.apache.org/jira/browse/FLINK-8581
> 
> FLIP-6 is somewhat similar and currently only undergoes some stability
> improvements/bug fixing. The architectural changes are merged now.
> 
> 
> Nico
> 
> On 06/03/18 11:24, Paris Carbone wrote:
>> Hey,
>> 
>> Indeed checkpointing iterations and dealing with closed sources are 
>> orthogonal issues, that is why the latter is not part of FLIP-15. Though, 
>> you kinda need both to have meaningful checkpoints for jobs with iterations.
>> One has to do with correctness (checkpointing strongly connected components 
>> in the execution graph) and the other about termination (terminating the 
>> checkpointing protocol when certain tasks ‘finish’).
>> 
>> I am willing to help out resolving the first issue, though I prefer to wait 
>> for ongoing changes in the network model and FLIP-6 to be finalised to apply 
>> this change properly (are they?). 
>> 
>> Paris
>> 
>>> On 6 Mar 2018, at 10:51, Nico Kruber <n...@data-artisans.com> wrote:
>>> 
>>> Hi Ken,
>>> sorry, I was mislead by the fact that you are using iterations and those
>>> were only documented for the DataSet API.
>>> 
>>> Running checkpoints with closed sources sounds like a more general thing
>>> than being part of the iterations rework of FLIP-15. I couldn't dig up
>>> anything on jira regarding this improvement either.
>>> 
>>> @Stephan: is this documented somewhere?
>>> 
>>> 
>>> Nico
>>> 
>>> On 02/03/18 23:55, Ken Krugler wrote:
>>>> Hi Stephan,
>>>> 
>>>> Thanks for the update.
>>>> 
>>>> So is support for “running checkpoints with closed sources” part
>>>> of FLIP-15
>>>> <https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66853132>,
>>>> or something separate?
>>>> 
>>>> Regards,
>>>> 
>>>> — Ken
>>>> 
>>>>> On Mar 1, 2018, at 9:07 AM, Stephan Ewen <se...@apache.org
>>>>> <mailto:se...@apache.org>> wrote:
>>>>> 
>>>>> @Ken The issue you are running into is that Checkpointing works
>>>>> currently only until the job reaches the point where the pipeline
>>>>> starts to drain out, meaning when the sources are done. In your case,
>>>>> the source is done immediately, sending out only one tuple.
>>>>> 
>>>>> Running checkpoints with closed sources is something that's on the
>>>>> feature list and will come soon…
>>>> 
>>>> --------------------------------------------
>>>> http://about.me/kkrugler
>>>> +1 530-210-6378
>>>> 
>>> 
>> 
> 

Reply via email to