Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished

2021-07-23 Thread Till Rohrmann
t; From:Till Rohrmann > > Send Time:2021 Jul. 22 (Thu.) 17:57 > > To:Yun Gao > > Cc:Piotr Nowojski ; dev ; > Yun Gao ; Piotr Nowojski < > pi...@ververica.com> > > Subject:Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks > Finished > > > >

Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished

2021-07-23 Thread Dawid Wysakowicz
:2021 Jul. 22 (Thu.) 17:57 > To:Yun Gao > Cc:Piotr Nowojski ; dev ; Yun Gao > ; Piotr Nowojski > Subject:Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished > > Thanks everyone for this discussion. I think this is very helpful. > > I do agree with Piotr's pr

Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished

2021-07-22 Thread Yun Gao
1 Jul. 22 (Thu.) 15:33 To:dev ; Yun Gao Cc:Till Rohrmann ; Yun Gao ; Piotr Nowojski Subject:Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished Hi Guowei, > Thank Dawid and Piotr for sharing the problem. +1 to EndInput/Finish can > becalled repeatedly. Just to cl

Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished

2021-07-22 Thread Till Rohrmann
ry thanks for the careful checks on the whole process. > > Best, > Yun > > > ---------------------- > From:Piotr Nowojski > Send Time:2021 Jul. 22 (Thu.) 15:33 > To:dev ; Yun Gao > Cc:Till Rohrmann ; Yun Gao > ; Piotr No

Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished

2021-07-22 Thread Yun Gao
jski Send Time:2021 Jul. 22 (Thu.) 15:33 To:dev ; Yun Gao Cc:Till Rohrmann ; Yun Gao ; Piotr Nowojski Subject:Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished Hi Guowei, > Thank Dawid and Piotr for sharing the problem. +1 to EndInput/Finish can > becalled repeat

Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished

2021-07-22 Thread Piotr Nowojski
s.apache.org/jira/browse/FLINK-21080. > > Best, > Yun > > > -- > From:Piotr Nowojski > Send Time:2021 Jul. 22 (Thu.) 02:46 > To:dev > Cc:Yun Gao ; Till Rohrmann ; > Yun Gao ; Piotr Nowojski < >

Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished

2021-07-22 Thread Yun Gao
-- From:Piotr Nowojski Send Time:2021 Jul. 22 (Thu.) 02:46 To:dev Cc:Yun Gao ; Till Rohrmann ; Yun Gao ; Piotr Nowojski Subject:Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished Hi Steven, > I probably missed sth h

Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished

2021-07-22 Thread Guowei Ma
mers or by processing > a > > > next > > > > >>>> > > > > >>>> record from upstream. > > > > >>>> > > > > >>>> > > > > >>>> > > > > > > > > >&

Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished

2021-07-21 Thread Piotr Nowojski
> *Checkpointing from a single subtask / UnionListState case* > > > There are operators that checkpoint from a single subtask only. Usually > > > from the subtask index=0. If we let those subtasks finish, subsequent > > > checkpoints will miss this information. &

Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished

2021-07-21 Thread Steven Wu
;>> > > >>> Hi, > > >>> > > >>> Very thanks Dawid for the thoughts! > > >>> > > >>> Currently I also do not have different opinions regarding this part. > > >>> But I have one more issue to confirm: du

Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished

2021-07-21 Thread Piotr Nowojski
ckpoint (but we could not skip the savepoint). To allow users to > >>> express > >>> the logic, we have proposed to add one more method to StreamOperator & > >>> CheckpointListener: > >>> > >>> interface StreamOperator { > >>

Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished

2021-07-21 Thread Dawid Wysakowicz
version.  > > Best > Yun > > -- > From:Dawid Wysakowicz > Send Time:2021 Jul. 19 (Mon.) 17:51 > To:dev ; Till Rohrmann > Cc:Yun Gao ; Yun Gao > > Subject:Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After

Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished

2021-07-19 Thread Yun Gao
Wysakowicz Send Time:2021 Jul. 19 (Mon.) 17:51 To:dev ; Till Rohrmann Cc:Yun Gao ; Yun Gao Subject:Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished Small correction. I meant we need to adjust the EndOfInputEvent of course. Best, Dawid On 19/07/2021 11:48, Dawid Wysakowicz wrote

Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished

2021-07-19 Thread Dawid Wysakowicz
t;> } >>> } >>> >>> interface CheckpointListener { >>> >>> default boolean requiresFinalCheckpoint() { >>> return true; >>> } >>> } >>> >>> class AbstractUdfStreamOperator { >>> &g

Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished

2021-07-19 Thread Dawid Wysakowicz
always keep the >>> window as is, and restore them after restart. >>> >>>> Then for the finish / stop-with-savepoint --drain, I think perhaps it >>> depends on the Triggers. For >>> >>>> event-time triggers / process time triggers, it would be reasonable to >>> flush all

Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished

2021-07-19 Thread Till Rohrmann
ner && > ((CheckpointListener) userFunction).requiresFinalCheckpoint(); > } > } > > > I think we should still keep the change ? > > Best, > Yun > > ------Original Mail ------ > *Sender:*Dawid Wysakowicz > > *Send Dat

Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished

2021-07-19 Thread Dawid Wysakowicz
> |}| > | > | > > I think we should still keep the change ?  > > Best, > Yun > > --Original Mail ------ > *Sender:*Dawid Wysakowicz > *Send Date:*Sun Jul 18 18:44:50 2021 > *Recipients:*Flink Dev , Yun Gao > > *Subj

Re: Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished

2021-07-19 Thread Yun Gao
userFunction).requiresFinalCheckpoint(); } } I think we should still keep the change ? Best, Yun --Original Mail -- Sender:Dawid Wysakowicz Send Date:Sun Jul 18 18:44:50 2021 Recipients:Flink Dev , Yun Gao Subject:Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Task

Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished

2021-07-18 Thread Dawid Wysakowicz
s to > >> be short, perhaps it is enough > >> to do re-computation from the scratch in consideration of the tradeoff > >> between the performance and > >> the complexity ? > >> > >> 3) We are going to emit the EndOfInput event exactly after t

Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished

2021-07-16 Thread Yun Gao
ckpoint or a savepoint. Best, Yun -- From:Till Rohrmann Send Time:2021 Jul. 16 (Fri.) 23:05 To:Yun Gao Cc:dev Subject:Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished Thanks for your responses Dawid and Yun,

Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished

2021-07-16 Thread Till Rohrmann
; > >> > > >> 2) For the computation logic, in fact currently we benefit a lot from > > >> some shortcuts on all-to-all > > >> edges and job vertex with all tasks running, these shortcuts could do > > >> checks on the job vertex level > > &g

Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished

2021-07-16 Thread Yun Gao
s would only be sent once for each task ? Best, Yun -- From:Till Rohrmann Send Time:2021 Jul. 16 (Fri.) 17:26 To:Yun Gao Cc:dev Subject:Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished Ok, so the plan is that finish() will flush all pending events and then send t

Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished

2021-07-16 Thread Dawid Wysakowicz
ot making the whole thing clear in the FLIP again, if >> there are no other concerns I'll update the FLIP with the above conclusions >> to make it precise in this part. >> >> >> Best, >> Yun >> >> ---

Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished

2021-07-16 Thread Till Rohrmann
no other concerns I'll update the FLIP with the above conclusions > to make it precise in this part. > > > Best, > Yun > > ---------- > From:Till Rohrmann > Send Time:2021 Jul. 16 (Fri.) 16:00 > To:dev > Cc:

Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished

2021-07-16 Thread Yun Gao
16:00 To:dev Cc:Yun Gao Subject:Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished I think we should try to sort this out because it might affect how and when finish() will be called (or in general how the operator lifecycle looks like). To give an example let's ta

Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished

2021-07-16 Thread Yun Gao
Best, Yun [1] https://issues.apache.org/jira/browse/FLINK-21132 -- From:Piotr Nowojski Send Time:2021 Jul. 16 (Fri.) 13:48 To:dev Cc:Yun Gao Subject:Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished Hi Til

Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished

2021-07-16 Thread Till Rohrmann
> > >> easily encode this information > > >> into the shortcuts on the job vertex level. And since the time seems > to > > >> be short, perhaps it is enough > > >> to do re-computation from the scratch in consideration of the tradeoff > > >> between the performance and > > >> the

Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished

2021-07-15 Thread Piotr Nowojski
e tradeoff > >> between the performance and > >> the complexity ? > >> > >> 3) We are going to emit the EndOfInput event exactly after the finish() > >> method and before the last > >> snapshotState() method so that we could shut down the whole topology > with > >> a single final checkp

Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished

2021-07-15 Thread Till Rohrmann
> Very sorry for not include enough details for this part and I'll >> complement the FLIP with the details on >> the process of the final checkpoint / savepoint. >> >> Best, >> Yun >> >> >> >> --

Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished

2021-07-15 Thread Till Rohrmann
ough details for this part and I'll > complement the FLIP with the details on > the process of the final checkpoint / savepoint. > > Best, > Yun > > > > ------------------ > From:Till Rohrmann > Send Time:2021 Jul. 14 (Wed

Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished

2021-07-14 Thread Yun Gao
Yun -- From:Till Rohrmann Send Time:2021 Jul. 14 (Wed.) 22:05 To:dev Subject:Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished Hi everyone, I am a bit late to the voting party but let me ask three questions: 1) Why do we execute the trig

Re: [RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished

2021-07-14 Thread Till Rohrmann
Hi everyone, I am a bit late to the voting party but let me ask three questions: 1) Why do we execute the trigger plan computation in the main thread if we cannot guarantee that all tasks are still running when triggering the checkpoint? Couldn't we do the computation in a different thread in

[RESULT][VOTE] FLIP-147: Support Checkpoint After Tasks Finished

2021-07-02 Thread Yun Gao
Hi there, Since the voting time of FLIP-147[1] has passed, I'm closing the vote now. There were seven +1 votes ( 6 / 7 are bindings) and no -1 votes: - Dawid Wysakowicz (binding) - Piotr Nowojski(binding) - Jiangang Liu (binding) - Arvid Heise (binding) - Jing Zhang (binding) - Leonard Xu