Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

2021-01-14 Thread Yun Gao
ksFinished-TriggeringCheckpointsAfterTasksFinished -- From:Yun Gao Send Time:2021 Jan. 13 (Wed.) 16:09 To:dev ; user Subject:Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished Hi all, I updated the FLIP[1] to reflect the major

Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

2021-01-13 Thread Yun Gao
foreFinish -- From:Yun Gao Send Time:2021 Jan. 12 (Tue.) 10:30 To:Khachatryan Roman Cc:dev ; user Subject:Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished Hi Roman, Very thanks for the

Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

2021-01-11 Thread Yun Gao
r waiting for the flush of pipeline result partition. Glad that we have the same viewpoints on this issue. :) Best, Yun -- From:Khachatryan Roman Send Time:2021 Jan. 11 (Mon.) 19:14 To:Yun Gao Cc:dev ; user Subject:

Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

2021-01-11 Thread Khachatryan Roman
imization and > postpone it to future versions ? > > Best, > Yun > > > > ---------- > From:Khachatryan Roman > Send Time:2021 Jan. 11 (Mon.) 05:46 > To:Yun Gao > Cc:Arvid Heise ; dev ; us

Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

2021-01-11 Thread Yun Gao
SCUSS] FLIP-147: Support Checkpoints After Tasks Finished Thanks a lot for your answers Yun, > In detail, support we have a job with the graph A -> B -> C, support in one > checkpoint A has reported FINISHED, CheckpointCoordinator would > choose B as the new "source" t

Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

2021-01-10 Thread Khachatryan Roman
is called > with > EndOfPartition) and then taking snapshot for the input channels, as the > normal unaligned checkpoints does for the InputChannel side. Then > we would be able to ensure the finished tasks always have an empty state. > > I'll also optimize the FLIP to make it mo

Re: Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

2021-01-07 Thread Yun Gao
ear~ Best, Yun --Original Mail -- Sender:Khachatryan Roman Send Date:Thu Jan 7 21:55:52 2021 Recipients:Arvid Heise CC:dev , user Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished Thanks for starting this discussion (and sorry for probably duplicated questions,

Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

2021-01-07 Thread Khachatryan Roman
Thanks for starting this discussion (and sorry for probably duplicated questions, I couldn't find them answered in FLIP or this thread). 1. Option 1 is said to be not preferable because it wastes resources and adds complexity (new event). However, the resources would be wasted for a relatively

Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

2021-01-07 Thread Arvid Heise
> > We could introduce an interface, sth like `RequiresFinalization` or > `FinalizationListener` (all bad names). The operator itself knows when > it is ready to completely shut down, Async I/O would wait for all > requests, sink would potentially wait for a given number of checkpoints. > The

Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

2021-01-07 Thread Aljoscha Krettek
This is somewhat unrelated to the discussion about how to actually do the triggering when sources shut down, I'll write on that separately. I just wanted to get this quick thought out. For letting operators decide whether they actually want to wait for a final checkpoint, which is relevant at

Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

2021-01-05 Thread Yun Gao
, Yun -- From:Arvid Heise Send Time:2021 Jan. 5 (Tue.) 17:16 To:Yun Gao Cc:Aljoscha Krettek ; dev ; user Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished Hi Yun, 1. I'd think that this is an orthogonal issue, which I'd solve separately. M

Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

2021-01-05 Thread Arvid Heise
wiki.apache.org/confluence/display/FLINK/FLIP-147%3A+Support+Checkpoints+After+Tasks+Finished#FLIP147:SupportCheckpointsAfterTasksFinished-Option3.Allowtaskstofinish > > -------------------------- > From:Yun Gao > Send Time:2020 Dec. 16 (Wed.) 11:07 > To:Aljoscha Krettek ; dev ; > user > Subject:Re:

Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

2020-12-24 Thread Yun Gao
stofinish -- From:Yun Gao Send Time:2020 Dec. 16 (Wed.) 11:07 To:Aljoscha Krettek ; dev ; user Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished Hi Aljoscha, Very thanks for the feedbacks! For the remaining issues: > 1. You mean we would insert "art

Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

2020-12-15 Thread Yun Gao
d Time:2020 Dec. 15 (Tue.) 18:11 To:dev Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished Thanks for the thorough update! I'll answer inline. On 14.12.20 16:33, Yun Gao wrote: > 1. To include EndOfPartition into consideration for barrier alignment at > t

Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

2020-12-15 Thread Aljoscha Krettek
Thanks for the thorough update! I'll answer inline. On 14.12.20 16:33, Yun Gao wrote: 1. To include EndOfPartition into consideration for barrier alignment at the TM side, we now tend to decouple the logic for EndOfPartition with the normal alignment behaviors to avoid the complex

Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

2020-12-14 Thread Yun Gao
Hi all, I would like to resume this discussion for supporting checkpoints after tasks Finished :) Based on the previous discussion, we now implement a version of PoC [1] to try the idea. During the PoC we also met with some possible issues: 1. To include EndOfPartition into

Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

2020-10-14 Thread Yun Gao
nd Time:2020 Oct. 13 (Tue.) 17:25 To:Yun Gao Cc:Arvid Heise ; Flink Dev ; User-Flink Subject:Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished Thanks for starting this discussion Yun Gao, I have three comments/questions: 1) When restarting all tasks independent of the status at ch

Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

2020-10-13 Thread Till Rohrmann
; the status of tasks on restoring from previous checkpoints. It would also > requires some modification in the task side to support input channels that > are finished on starting. > > But in the first version, I think we might simplify this issue by still > restart all the tasks, but let

Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

2020-10-13 Thread Yun Gao
e finished sources to exit directly? The new Source API would terminate directly since there is no pending splits and the legacy sources would be dealt specially by skipped execution if the source operator is fully finished before. We would be able to turn to the final solution gradually in the next steps. Best, Yun --

Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

2020-10-12 Thread Arvid Heise
restart all the tasks, but let the finished sources to exit directly? The > new Source API would terminate directly since there is no pending splits > and the legacy sources would be dealt specially by skipped execution if the > source operator is fully finished before. We would be able to turn to the > final

Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

2020-10-12 Thread Yun Gao
splits and the legacy sources would be dealt specially by skipped execution if the source operator is fully finished before. We would be able to turn to the final solution gradually in the next steps. Best, Yun ------ From:Arvid Heise Send Time:2020 Oct. 12 (Mon.) 15:

Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

2020-10-12 Thread Arvid Heise
.flink.apache.org%3E > [5] > https://cwiki.apache.org/confluence/display/FLINK/FLIP-143%3A+Unified+Sink+API > [6] > https://cwiki.apache.org/confluence/display/FLINK/FLIP-147%3A+Support+Checkpoints+After+Tasks+Finished > > --Original Mail -- > *Sender:*Yun Gao

Re: [DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

2020-10-09 Thread Yun Gao
I [6] https://cwiki.apache.org/confluence/display/FLINK/FLIP-147%3A+Support+Checkpoints+After+Tasks+Finished --Original Mail -- Sender:Yun Gao Send Date:Fri Oct 9 14:16:52 2020 Recipients:Flink Dev , User-Flink Subject:[DISCUSS] FLIP-147: Support Checkpoints After T

[DISCUSS] FLIP-147: Support Checkpoints After Tasks Finished

2020-10-09 Thread Yun Gao
Hi, devs & users As discussed in FLIP-131 [1], Flink will make DataStream the unified API for processing bounded and unbounded data in both streaming and blocking modes. However, one long-standing problem for the streaming mode is that currently Flink does not support checkpoints after some