Re: Behaviour of the BucketingSink when checkpoints fail

2017-04-28 Thread Aljoscha Krettek
Hi, Yes, basically all the exactly-once/at-least-once guarantees are not given if checkpointing does not work correctly. For example, this will also be the case when reading from Kafka and writing to Kafka. Best, Aljoscha > On 28. Apr 2017, at 15:53, Yassine MARZOUGUI > wrote: > > Hi Aljosch

Re: Behaviour of the BucketingSink when checkpoints fail

2017-04-28 Thread Yassine MARZOUGUI
Hi Aljoscha, Thank you for your response. I guess then I will manually rename the pending files. Does this however mean that the BucketingSink is not exactly-once as it is described is the docs, since in this case (failure of the job and failure of checkpoints) there will be duplicates? Or am I mi

Re: Behaviour of the BucketingSink when checkpoints fail

2017-04-28 Thread Aljoscha Krettek
Hi, Yes, your analysis is correct. The pending files are not recognised as such because they were never in any checkpointed state that could be restored. I’m afraid it’s not possible to build the sink state just from the files existing in the output folder. The reason we have state in the first

Behaviour of the BucketingSink when checkpoints fail

2017-04-28 Thread Yassine MARZOUGUI
Hi all, I'm have a failed job containing a BucketingSink. The last successful checkpoint was before the source started emitting data. The following checkpoints all failed due to the long timeout as I mentioned here : http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Checkpoints-v