Fabian, that sounds good. Should I recap some bullets in an email and start a 
new thread then?

Thanks, Ashish

> On Mar 22, 2018, at 5:16 AM, Fabian Hueske <fhue...@gmail.com> wrote:
> 
> Hi Ashish,
> 
> Agreed! 
> I think the right approach would be to gather the requirements and start a 
> discussion on the dev mailing list. 
> Contributors and committers who are more familiar with the checkpointing and 
> recovery internals should discuss a solution that can be integrated and 
> doesn't break with the current mechanism.
> For instance (not sure whether this is feasible or solves the problem) one 
> could only do local checkpoints and not write to the distributed persistent 
> storage. That would bring down checkpointing costs and the recovery life 
> cycle would not need to be radically changed.
> 
> Best, Fabian
> 
> 2018-03-20 22:56 GMT+01:00 Ashish Pokharel <ashish...@yahoo.com 
> <mailto:ashish...@yahoo.com>>:
> I definitely like the idea of event based checkpointing :) 
> 
> Fabian, I do agree with your point that it is not possible to take a rescue 
> checkpoint consistently. The basis here however is not around the operator 
> that actually failed. It’s to avoid data loss across 100s (probably 1000s of 
> parallel operators) which are being restarted and are “healthy”. We have 100k 
> (nearing million soon) elements pushing data. Losing few seconds worth of 
> data for few is not good but “acceptable” as long as damage can be 
> controlled. Of course, we are going to use rocksdb + 2-phase commit with 
> Kafka where we need exactly once guarantees. The proposal of “fine grain 
> recovery 
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-1+%3A+Fine+Grained+Recovery+from+Task+Failures
>  
> <https://cwiki.apache.org/confluence/display/FLINK/FLIP-1+:+Fine+Grained+Recovery+from+Task+Failures>”
>  seems like a good start at least from damage control perspective but even 
> with that it feels like something like “event based approach” can be done for 
> a sub-set of job graph that are “healthy”. 
> 
> Thanks, Ashish
> 
> 
>> On Mar 20, 2018, at 9:53 AM, Fabian Hueske <fhue...@gmail.com 
>> <mailto:fhue...@gmail.com>> wrote:
>> 
>> Well, that's not that easy to do, because checkpoints must be coordinated 
>> and triggered the JobManager.
>> Also, the checkpointing mechanism with flowing checkpoint barriers (to 
>> ensure checkpoint consistency) won't work once a task failed because it 
>> cannot continue processing and forward barriers. If the task failed with an 
>> OOME, the whole JVM is gone anyway.
>> I don't think it is possible to take something like a consistent rescue 
>> checkpoint in case of a failure. 
>> 
>> I might be possible to checkpoint application state of non-failed tasks, but 
>> this would result in data loss for the failed task and we would need to 
>> weigh the use cases for such a feature are the implementation effort.
>> Maybe there are better ways to address such use cases.
>> 
>> Best, Fabian
>> 
>> 2018-03-20 6:43 GMT+01:00 makeyang <riverbuild...@hotmail.com 
>> <mailto:riverbuild...@hotmail.com>>:
>> currently there is only time based way to trigger a checkpoint. based on this
>> discussion, I think flink need to introduce event based way to trigger
>> checkpoint such as restart a task manager should be count as a event.
>> 
>> 
>> 
>> --
>> Sent from: 
>> http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/ 
>> <http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/>
>> 
> 
> 

Reply via email to